1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.regionserver;
19
20 import static org.junit.Assert.*;
21 import static org.mockito.Mockito.*;
22 import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.STRIPE_START_KEY;
23 import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.STRIPE_END_KEY;
24 import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.OPEN_KEY;
25
26 import java.io.IOException;
27 import java.util.ArrayList;
28 import java.util.Arrays;
29 import java.util.List;
30 import java.util.TreeMap;
31
32 import org.apache.hadoop.conf.Configuration;
33 import org.apache.hadoop.fs.FileSystem;
34 import org.apache.hadoop.fs.Path;
35 import org.apache.hadoop.hbase.Cell;
36 import org.apache.hadoop.hbase.HBaseConfiguration;
37 import org.apache.hadoop.hbase.HColumnDescriptor;
38 import org.apache.hadoop.hbase.HRegionInfo;
39 import org.apache.hadoop.hbase.KeyValue;
40 import org.apache.hadoop.hbase.KeyValue.KVComparator;
41 import org.apache.hadoop.hbase.TableName;
42 import org.apache.hadoop.hbase.io.compress.Compression;
43 import org.apache.hadoop.hbase.io.hfile.HFile;
44 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
45 import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactor;
46 import org.apache.hadoop.hbase.testclassification.SmallTests;
47 import org.apache.hadoop.hbase.util.Bytes;
48 import org.junit.Test;
49 import org.junit.experimental.categories.Category;
50 import org.mockito.invocation.InvocationOnMock;
51 import org.mockito.stubbing.Answer;
52
53
54 @Category(SmallTests.class)
55 public class TestStripeCompactor {
56 private static final byte[] NAME_OF_THINGS = Bytes.toBytes("foo");
57 private static final TableName TABLE_NAME = TableName.valueOf(NAME_OF_THINGS, NAME_OF_THINGS);
58
59 private static final byte[] KEY_B = Bytes.toBytes("bbb");
60 private static final byte[] KEY_C = Bytes.toBytes("ccc");
61 private static final byte[] KEY_D = Bytes.toBytes("ddd");
62
63 private static final KeyValue KV_A = kvAfter(Bytes.toBytes("aaa"));
64 private static final KeyValue KV_B = kvAfter(KEY_B);
65 private static final KeyValue KV_C = kvAfter(KEY_C);
66 private static final KeyValue KV_D = kvAfter(KEY_D);
67
68 private static KeyValue kvAfter(byte[] key) {
69 return new KeyValue(Arrays.copyOf(key, key.length + 1), 0L);
70 }
71
72 private static <T> T[] a(T... a) {
73 return a;
74 }
75
76 private static KeyValue[] e() {
77 return TestStripeCompactor.<KeyValue>a();
78 }
79
80 @Test
81 public void testBoundaryCompactions() throws Exception {
82
83 verifyBoundaryCompaction(a(KV_A, KV_A, KV_B, KV_B, KV_C, KV_D),
84 a(OPEN_KEY, KEY_B, KEY_D, OPEN_KEY), a(a(KV_A, KV_A), a(KV_B, KV_B, KV_C), a(KV_D)));
85 verifyBoundaryCompaction(a(KV_B, KV_C), a(KEY_B, KEY_C, KEY_D), a(a(KV_B), a(KV_C)));
86 verifyBoundaryCompaction(a(KV_B, KV_C), a(KEY_B, KEY_D), new KeyValue[][] { a(KV_B, KV_C) });
87 }
88
89 @Test
90 public void testBoundaryCompactionEmptyFiles() throws Exception {
91
92 verifyBoundaryCompaction(
93 a(KV_B), a(KEY_B, KEY_C, KEY_D, OPEN_KEY), a(a(KV_B), null, null), null, null, false);
94 verifyBoundaryCompaction(a(KV_A, KV_C),
95 a(OPEN_KEY, KEY_B, KEY_C, KEY_D), a(a(KV_A), null, a(KV_C)), null, null, false);
96
97 verifyBoundaryCompaction(
98 e(), a(OPEN_KEY, KEY_B, KEY_C, OPEN_KEY), a(null, null, e()), null, null, false);
99
100 verifyBoundaryCompaction(
101 e(), a(OPEN_KEY, KEY_B, KEY_C, OPEN_KEY), a(null, e(), null), KEY_B, KEY_C, false);
102 verifyBoundaryCompaction(
103 e(), a(OPEN_KEY, KEY_B, KEY_C, OPEN_KEY), a(e(), e(), null), OPEN_KEY, KEY_C, false);
104
105 verifyBoundaryCompaction(a(KV_A), a(OPEN_KEY, KEY_B, KEY_C, KEY_D, OPEN_KEY),
106 a(a(KV_A), e(), e(), null), KEY_B, KEY_D, false);
107 verifyBoundaryCompaction(a(KV_C), a(OPEN_KEY, KEY_B, KEY_C, KEY_D, OPEN_KEY),
108 a(null, null, a(KV_C), e()), KEY_C, OPEN_KEY, false);
109
110 }
111
112 public static void verifyBoundaryCompaction(
113 KeyValue[] input, byte[][] boundaries, KeyValue[][] output) throws Exception {
114 verifyBoundaryCompaction(input, boundaries, output, null, null, true);
115 }
116
117 public static void verifyBoundaryCompaction(KeyValue[] input, byte[][] boundaries,
118 KeyValue[][] output, byte[] majorFrom, byte[] majorTo, boolean allFiles)
119 throws Exception {
120 StoreFileWritersCapture writers = new StoreFileWritersCapture();
121 StripeCompactor sc = createCompactor(writers, input);
122 List<Path> paths =
123 sc.compact(createDummyRequest(), Arrays.asList(boundaries), majorFrom, majorTo);
124 writers.verifyKvs(output, allFiles, true);
125 if (allFiles) {
126 assertEquals(output.length, paths.size());
127 writers.verifyBoundaries(boundaries);
128 }
129 }
130
131 @Test
132 public void testSizeCompactions() throws Exception {
133
134 verifySizeCompaction(a(KV_A, KV_A, KV_B, KV_C, KV_D), 3, 2, OPEN_KEY, OPEN_KEY,
135 a(a(KV_A, KV_A), a(KV_B, KV_C), a(KV_D)));
136 verifySizeCompaction(a(KV_A, KV_B, KV_C, KV_D), 4, 1, OPEN_KEY, OPEN_KEY,
137 a(a(KV_A), a(KV_B), a(KV_C), a(KV_D)));
138 verifySizeCompaction(a(KV_B, KV_C), 2, 1, KEY_B, KEY_D, a(a(KV_B), a(KV_C)));
139
140 verifySizeCompaction(a(KV_A, KV_A, KV_A, KV_C, KV_D), 3, 2, OPEN_KEY, OPEN_KEY,
141 a(a(KV_A, KV_A, KV_A), a(KV_C, KV_D)));
142 verifySizeCompaction(a(KV_A, KV_B, KV_B, KV_C), 3, 1, OPEN_KEY, OPEN_KEY,
143 a(a(KV_A), a(KV_B, KV_B), a(KV_C)));
144
145 verifySizeCompaction(a(KV_A, KV_B, KV_C, KV_D), 2, 1, OPEN_KEY, OPEN_KEY,
146 a(a(KV_A), a(KV_B, KV_C, KV_D)));
147 verifySizeCompaction(a(KV_A, KV_B, KV_C), 1, Long.MAX_VALUE, OPEN_KEY, KEY_D,
148 new KeyValue[][] { a(KV_A, KV_B, KV_C) });
149
150 verifySizeCompaction(a(KV_A, KV_B, KV_C, KV_D), Integer.MAX_VALUE, 2, OPEN_KEY, OPEN_KEY,
151 a(a(KV_A, KV_B), a(KV_C, KV_D)));
152 }
153
154 public static void verifySizeCompaction(KeyValue[] input, int targetCount, long targetSize,
155 byte[] left, byte[] right, KeyValue[][] output) throws Exception {
156 StoreFileWritersCapture writers = new StoreFileWritersCapture();
157 StripeCompactor sc = createCompactor(writers, input);
158 List<Path> paths = sc.compact(
159 createDummyRequest(), targetCount, targetSize, left, right, null, null);
160 assertEquals(output.length, paths.size());
161 writers.verifyKvs(output, true, true);
162 List<byte[]> boundaries = new ArrayList<byte[]>();
163 boundaries.add(left);
164 for (int i = 1; i < output.length; ++i) {
165 boundaries.add(output[i][0].getRow());
166 }
167 boundaries.add(right);
168 writers.verifyBoundaries(boundaries.toArray(new byte[][] {}));
169 }
170
171 private static StripeCompactor createCompactor(
172 StoreFileWritersCapture writers, KeyValue[] input) throws Exception {
173 Configuration conf = HBaseConfiguration.create();
174 final Scanner scanner = new Scanner(input);
175
176
177 HColumnDescriptor col = new HColumnDescriptor(NAME_OF_THINGS);
178 ScanInfo si = new ScanInfo(col, Long.MAX_VALUE, 0, new KVComparator());
179 Store store = mock(Store.class);
180 when(store.getFamily()).thenReturn(col);
181 when(store.getScanInfo()).thenReturn(si);
182 when(store.areWritesEnabled()).thenReturn(true);
183 when(store.getFileSystem()).thenReturn(mock(FileSystem.class));
184 when(store.getRegionInfo()).thenReturn(new HRegionInfo(TABLE_NAME));
185 when(store.createWriterInTmp(anyLong(), any(Compression.Algorithm.class),
186 anyBoolean(), anyBoolean(), anyBoolean())).thenAnswer(writers);
187 when(store.getComparator()).thenReturn(new KVComparator());
188
189 return new StripeCompactor(conf, store) {
190 @Override
191 protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
192 long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,
193 byte[] dropDeletesToRow) throws IOException {
194 return scanner;
195 }
196
197 @Override
198 protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
199 ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
200 return scanner;
201 }
202 };
203 }
204
205 private static CompactionRequest createDummyRequest() throws Exception {
206
207
208 StoreFile sf = mock(StoreFile.class);
209 StoreFile.Reader r = mock(StoreFile.Reader.class);
210 when(r.length()).thenReturn(1L);
211 when(r.getBloomFilterType()).thenReturn(BloomType.NONE);
212 when(r.getHFileReader()).thenReturn(mock(HFile.Reader.class));
213 when(r.getStoreFileScanner(anyBoolean(), anyBoolean(), anyBoolean(), anyLong()))
214 .thenReturn(mock(StoreFileScanner.class));
215 when(sf.getReader()).thenReturn(r);
216 when(sf.createReader()).thenReturn(r);
217 return new CompactionRequest(Arrays.asList(sf));
218 }
219
220 private static class Scanner implements InternalScanner {
221 private final ArrayList<KeyValue> kvs;
222 public Scanner(KeyValue... kvs) {
223 this.kvs = new ArrayList<KeyValue>(Arrays.asList(kvs));
224 }
225
226 @Override
227 public boolean next(List<Cell> results) throws IOException {
228 if (kvs.isEmpty()) return false;
229 results.add(kvs.remove(0));
230 return !kvs.isEmpty();
231 }
232 @Override
233 public boolean next(List<Cell> result, int limit) throws IOException {
234 return next(result);
235 }
236 @Override
237 public void close() throws IOException {}
238 }
239
240
241 public static class StoreFileWritersCapture implements
242 Answer<StoreFile.Writer>, StripeMultiFileWriter.WriterFactory {
243 public static class Writer {
244 public ArrayList<KeyValue> kvs = new ArrayList<KeyValue>();
245 public TreeMap<byte[], byte[]> data = new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
246 }
247
248 private List<Writer> writers = new ArrayList<Writer>();
249
250 @Override
251 public StoreFile.Writer createWriter() throws IOException {
252 final Writer realWriter = new Writer();
253 writers.add(realWriter);
254 StoreFile.Writer writer = mock(StoreFile.Writer.class);
255 doAnswer(new Answer<Object>() {
256 public Object answer(InvocationOnMock invocation) {
257 return realWriter.kvs.add((KeyValue)invocation.getArguments()[0]);
258 }}).when(writer).append(any(KeyValue.class));
259 doAnswer(new Answer<Object>() {
260 public Object answer(InvocationOnMock invocation) {
261 Object[] args = invocation.getArguments();
262 return realWriter.data.put((byte[])args[0], (byte[])args[1]);
263 }}).when(writer).appendFileInfo(any(byte[].class), any(byte[].class));
264 return writer;
265 }
266
267 @Override
268 public StoreFile.Writer answer(InvocationOnMock invocation) throws Throwable {
269 return createWriter();
270 }
271
272 public void verifyKvs(KeyValue[][] kvss, boolean allFiles, boolean requireMetadata) {
273 if (allFiles) {
274 assertEquals(kvss.length, writers.size());
275 }
276 int skippedWriters = 0;
277 for (int i = 0; i < kvss.length; ++i) {
278 KeyValue[] kvs = kvss[i];
279 if (kvs != null) {
280 Writer w = writers.get(i - skippedWriters);
281 if (requireMetadata) {
282 assertNotNull(w.data.get(STRIPE_START_KEY));
283 assertNotNull(w.data.get(STRIPE_END_KEY));
284 } else {
285 assertNull(w.data.get(STRIPE_START_KEY));
286 assertNull(w.data.get(STRIPE_END_KEY));
287 }
288 assertEquals(kvs.length, w.kvs.size());
289 for (int j = 0; j < kvs.length; ++j) {
290 assertEquals(kvs[j], w.kvs.get(j));
291 }
292 } else {
293 assertFalse(allFiles);
294 ++skippedWriters;
295 }
296 }
297 }
298
299 public void verifyBoundaries(byte[][] boundaries) {
300 assertEquals(boundaries.length - 1, writers.size());
301 for (int i = 0; i < writers.size(); ++i) {
302 assertArrayEquals("i = " + i, boundaries[i], writers.get(i).data.get(STRIPE_START_KEY));
303 assertArrayEquals("i = " + i, boundaries[i + 1], writers.get(i).data.get(STRIPE_END_KEY));
304 }
305 }
306 }
307 }