1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.regionserver.compactions;
19
20 import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.OPEN_KEY;
21 import static org.junit.Assert.assertEquals;
22 import static org.junit.Assert.assertFalse;
23 import static org.junit.Assert.assertNull;
24 import static org.junit.Assert.assertTrue;
25 import static org.mockito.AdditionalMatchers.aryEq;
26 import static org.mockito.Matchers.any;
27 import static org.mockito.Matchers.anyBoolean;
28 import static org.mockito.Matchers.anyInt;
29 import static org.mockito.Matchers.anyLong;
30 import static org.mockito.Matchers.argThat;
31 import static org.mockito.Matchers.eq;
32 import static org.mockito.Matchers.isNull;
33 import static org.mockito.Mockito.mock;
34 import static org.mockito.Mockito.only;
35 import static org.mockito.Mockito.times;
36 import static org.mockito.Mockito.verify;
37 import static org.mockito.Mockito.when;
38
39 import java.io.IOException;
40 import java.util.ArrayList;
41 import java.util.Arrays;
42 import java.util.Collection;
43 import java.util.Iterator;
44 import java.util.List;
45
46 import org.apache.hadoop.conf.Configuration;
47 import org.apache.hadoop.fs.Path;
48 import org.apache.hadoop.hbase.Cell;
49 import org.apache.hadoop.hbase.HBaseConfiguration;
50 import org.apache.hadoop.hbase.HColumnDescriptor;
51 import org.apache.hadoop.hbase.HRegionInfo;
52 import org.apache.hadoop.hbase.KeyValue;
53 import org.apache.hadoop.hbase.io.compress.Compression;
54 import org.apache.hadoop.hbase.io.hfile.HFile;
55 import org.apache.hadoop.hbase.regionserver.BloomType;
56 import org.apache.hadoop.hbase.regionserver.InternalScanner;
57 import org.apache.hadoop.hbase.regionserver.ScanType;
58 import org.apache.hadoop.hbase.regionserver.Store;
59 import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
60 import org.apache.hadoop.hbase.regionserver.StoreFile;
61 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
62 import org.apache.hadoop.hbase.regionserver.StripeMultiFileWriter;
63 import org.apache.hadoop.hbase.regionserver.StripeStoreConfig;
64 import org.apache.hadoop.hbase.regionserver.StripeStoreFileManager;
65 import org.apache.hadoop.hbase.regionserver.StripeStoreFlusher;
66 import org.apache.hadoop.hbase.regionserver.TestStripeCompactor.StoreFileWritersCapture;
67 import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy.StripeInformationProvider;
68 import org.apache.hadoop.hbase.testclassification.SmallTests;
69 import org.apache.hadoop.hbase.util.Bytes;
70 import org.apache.hadoop.hbase.util.ConcatenatedLists;
71 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
72 import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
73 import org.junit.Test;
74 import org.junit.experimental.categories.Category;
75 import org.mockito.ArgumentMatcher;
76
77 import com.google.common.collect.ImmutableList;
78 import com.google.common.collect.Lists;
79
80 @Category(SmallTests.class)
81 public class TestStripeCompactionPolicy {
82 private static final byte[] KEY_A = Bytes.toBytes("aaa");
83 private static final byte[] KEY_B = Bytes.toBytes("bbb");
84 private static final byte[] KEY_C = Bytes.toBytes("ccc");
85 private static final byte[] KEY_D = Bytes.toBytes("ddd");
86 private static final byte[] KEY_E = Bytes.toBytes("eee");
87 private static final KeyValue KV_A = new KeyValue(KEY_A, 0L);
88 private static final KeyValue KV_B = new KeyValue(KEY_B, 0L);
89 private static final KeyValue KV_C = new KeyValue(KEY_C, 0L);
90 private static final KeyValue KV_D = new KeyValue(KEY_D, 0L);
91 private static final KeyValue KV_E = new KeyValue(KEY_E, 0L);
92
93
94 private static long defaultSplitSize = 18;
95 private static float defaultSplitCount = 1.8F;
96 private final static int defaultInitialCount = 1;
97 private static long defaultTtl = 1000 * 1000;
98
99 @Test
100 public void testNoStripesFromFlush() throws Exception {
101 Configuration conf = HBaseConfiguration.create();
102 conf.setBoolean(StripeStoreConfig.FLUSH_TO_L0_KEY, true);
103 StripeCompactionPolicy policy = createPolicy(conf);
104 StripeInformationProvider si = createStripesL0Only(0, 0);
105
106 KeyValue[] input = new KeyValue[] { KV_A, KV_B, KV_C, KV_D, KV_E };
107 KeyValue[][] expected = new KeyValue[][] { input };
108 verifyFlush(policy, si, input, expected, null);
109 }
110
111 @Test
112 public void testOldStripesFromFlush() throws Exception {
113 StripeCompactionPolicy policy = createPolicy(HBaseConfiguration.create());
114 StripeInformationProvider si = createStripes(0, KEY_C, KEY_D);
115
116 KeyValue[] input = new KeyValue[] { KV_B, KV_C, KV_C, KV_D, KV_E };
117 KeyValue[][] expected = new KeyValue[][] { new KeyValue[] { KV_B },
118 new KeyValue[] { KV_C, KV_C }, new KeyValue[] { KV_D, KV_E } };
119 verifyFlush(policy, si, input, expected, new byte[][] { OPEN_KEY, KEY_C, KEY_D, OPEN_KEY });
120 }
121
122 @Test
123 public void testNewStripesFromFlush() throws Exception {
124 StripeCompactionPolicy policy = createPolicy(HBaseConfiguration.create());
125 StripeInformationProvider si = createStripesL0Only(0, 0);
126 KeyValue[] input = new KeyValue[] { KV_B, KV_C, KV_C, KV_D, KV_E };
127
128 KeyValue[][] expected = new KeyValue[][] { input };
129 verifyFlush(policy, si, input, expected, new byte[][] { OPEN_KEY, OPEN_KEY });
130 }
131
132 @Test
133 public void testSingleStripeCompaction() throws Exception {
134
135 Configuration conf = HBaseConfiguration.create();
136 conf.setFloat(CompactionConfiguration.RATIO_KEY, 1.0F);
137 conf.setInt(StripeStoreConfig.MIN_FILES_KEY, 3);
138 conf.setInt(StripeStoreConfig.MAX_FILES_KEY, 4);
139 conf.setLong(StripeStoreConfig.SIZE_TO_SPLIT_KEY, 1000);
140 StoreConfigInformation sci = mock(StoreConfigInformation.class);
141 StripeStoreConfig ssc = new StripeStoreConfig(conf, sci);
142 StripeCompactionPolicy policy = new StripeCompactionPolicy(conf, sci, ssc) {
143 @Override
144 public StripeCompactionRequest selectCompaction(StripeInformationProvider si,
145 List<StoreFile> filesCompacting, boolean isOffpeak) throws IOException {
146 if (!filesCompacting.isEmpty()) return null;
147 return selectSingleStripeCompaction(si, false, false, isOffpeak);
148 }
149
150 @Override
151 public boolean needsCompactions(
152 StripeInformationProvider si, List<StoreFile> filesCompacting) {
153 if (!filesCompacting.isEmpty()) return false;
154 return needsSingleStripeCompaction(si);
155 }
156 };
157
158
159 StripeInformationProvider si = createStripesWithSizes(0, 0,
160 new Long[] { 2L }, new Long[] { 3L, 3L }, new Long[] { 5L, 1L });
161 verifyNoCompaction(policy, si);
162
163 si = createStripesWithSizes(0, 0,
164 new Long[] { 2L }, new Long[] { 3L, 3L }, new Long[] { 5L, 1L, 1L });
165 assertNull(policy.selectCompaction(si, al(), false));
166 assertTrue(policy.needsCompactions(si, al()));
167
168 si = createStripesWithSizes(0, 0,
169 new Long[] { 2L }, new Long[] { 3L, 3L }, new Long[] { 5L, 4L, 3L });
170 verifySingleStripeCompaction(policy, si, 2, null);
171
172 si = createStripesWithSizes(0, 0,
173 new Long[] { 3L, 2L, 2L }, new Long[] { 2L, 2L, 1L }, new Long[] { 3L, 2L, 2L, 1L });
174 verifySingleStripeCompaction(policy, si, 2, null);
175 si = createStripesWithSizes(0, 0,
176 new Long[] { 5L }, new Long[] { 3L, 2L, 2L, 1L }, new Long[] { 3L, 2L, 2L });
177 verifySingleStripeCompaction(policy, si, 1, null);
178
179 si = createStripesWithSizes(0, 0,
180 new Long[] { 3L, 3L, 3L }, new Long[] { 3L, 1L, 2L }, new Long[] { 3L, 2L, 2L });
181 verifySingleStripeCompaction(policy, si, 1, null);
182
183 si = createStripesWithSizes(0, 0, new Long[] { 5L }, new Long[] { 5L, 4L, 4L, 4L, 4L });
184 List<StoreFile> sfs = si.getStripes().get(1).subList(1, 5);
185 verifyCompaction(policy, si, sfs, null, 1, null, si.getStartRow(1), si.getEndRow(1), true);
186
187 si = createStripesWithSizes(0, 0, new Long[] { 5L }, new Long[] { 50L, 4L, 4L, 4L, 4L });
188 sfs = si.getStripes().get(1).subList(1, 5);
189 verifyCompaction(policy, si, sfs, null, 1, null, si.getStartRow(1), si.getEndRow(1), true);
190 }
191
192 @Test
193 public void testWithParallelCompaction() throws Exception {
194
195
196 assertNull(createPolicy(HBaseConfiguration.create()).selectCompaction(
197 mock(StripeInformationProvider.class), al(createFile()), false));
198 }
199
200 @Test
201 public void testWithReferences() throws Exception {
202 StripeCompactionPolicy policy = createPolicy(HBaseConfiguration.create());
203 StripeCompactor sc = mock(StripeCompactor.class);
204 StoreFile ref = createFile();
205 when(ref.isReference()).thenReturn(true);
206 StripeInformationProvider si = mock(StripeInformationProvider.class);
207 Collection<StoreFile> sfs = al(ref, createFile());
208 when(si.getStorefiles()).thenReturn(sfs);
209
210 assertTrue(policy.needsCompactions(si, al()));
211 StripeCompactionPolicy.StripeCompactionRequest scr = policy.selectCompaction(si, al(), false);
212 assertEquals(si.getStorefiles(), scr.getRequest().getFiles());
213 scr.execute(sc, NoLimitCompactionThroughputController.INSTANCE);
214 verify(sc, only()).compact(eq(scr.getRequest()), anyInt(), anyLong(), aryEq(OPEN_KEY),
215 aryEq(OPEN_KEY), aryEq(OPEN_KEY), aryEq(OPEN_KEY),
216 any(NoLimitCompactionThroughputController.class));
217 }
218
219 @Test
220 public void testInitialCountFromL0() throws Exception {
221 Configuration conf = HBaseConfiguration.create();
222 conf.setInt(StripeStoreConfig.MIN_FILES_L0_KEY, 2);
223 StripeCompactionPolicy policy = createPolicy(
224 conf, defaultSplitSize, defaultSplitCount, 2, false);
225 StripeCompactionPolicy.StripeInformationProvider si = createStripesL0Only(3, 8);
226 verifyCompaction(policy, si, si.getStorefiles(), true, 2, 12L, OPEN_KEY, OPEN_KEY, true);
227 si = createStripesL0Only(3, 10);
228 verifyCompaction(policy, si, si.getStorefiles(), true, 3, 10L, OPEN_KEY, OPEN_KEY, true);
229 policy = createPolicy(conf, defaultSplitSize, defaultSplitCount, 6, false);
230 verifyCompaction(policy, si, si.getStorefiles(), true, 6, 5L, OPEN_KEY, OPEN_KEY, true);
231 }
232
233 @Test
234 public void testExistingStripesFromL0() throws Exception {
235 Configuration conf = HBaseConfiguration.create();
236 conf.setInt(StripeStoreConfig.MIN_FILES_L0_KEY, 3);
237 StripeCompactionPolicy.StripeInformationProvider si = createStripes(3, KEY_A);
238 verifyCompaction(
239 createPolicy(conf), si, si.getLevel0Files(), null, null, si.getStripeBoundaries());
240 }
241
242 @Test
243 public void testNothingToCompactFromL0() throws Exception {
244 Configuration conf = HBaseConfiguration.create();
245 conf.setInt(StripeStoreConfig.MIN_FILES_L0_KEY, 4);
246 StripeCompactionPolicy.StripeInformationProvider si = createStripesL0Only(3, 10);
247 StripeCompactionPolicy policy = createPolicy(conf);
248 verifyNoCompaction(policy, si);
249
250 si = createStripes(3, KEY_A);
251 verifyNoCompaction(policy, si);
252 }
253
254 @Test
255 public void testSplitOffStripe() throws Exception {
256 Configuration conf = HBaseConfiguration.create();
257
258 conf.setInt(StripeStoreConfig.MIN_FILES_KEY, 2);
259 Long[] toSplit = new Long[] { defaultSplitSize - 2, 1L, 1L };
260 Long[] noSplit = new Long[] { defaultSplitSize - 2, 1L };
261 long splitTargetSize = (long)(defaultSplitSize / defaultSplitCount);
262
263 StripeCompactionPolicy.StripeInformationProvider si =
264 createStripesWithSizes(0, 0, new Long[] { defaultSplitSize - 2, 2L });
265 assertNull(createPolicy(conf).selectCompaction(si, al(), false));
266
267 conf.setFloat(CompactionConfiguration.RATIO_KEY, 500f);
268 StripeCompactionPolicy policy = createPolicy(conf);
269 verifyWholeStripesCompaction(policy, si, 0, 0, null, 2, splitTargetSize);
270
271 si = createStripesWithSizes(0, 0, noSplit, noSplit, toSplit);
272 verifyWholeStripesCompaction(policy, si, 2, 2, null, 2, splitTargetSize);
273
274 si = createStripesWithSizes(0, 0, noSplit, toSplit, noSplit);
275 verifyWholeStripesCompaction(policy, si, 1, 1, null, 2, splitTargetSize);
276
277
278 StripeCompactionPolicy specPolicy = createPolicy(
279 conf, defaultSplitSize + 1, defaultSplitCount, defaultInitialCount, false);
280 verifySingleStripeCompaction(specPolicy, si, 1, null);
281 }
282
283 @Test
284 public void testSplitOffStripeOffPeak() throws Exception {
285
286 Configuration conf = HBaseConfiguration.create();
287 conf.setInt(StripeStoreConfig.MIN_FILES_KEY, 2);
288
289 StripeCompactionPolicy.StripeInformationProvider si =
290 createStripesWithSizes(0, 0, new Long[] { defaultSplitSize - 2, 1L, 1L });
291 assertEquals(2, createPolicy(conf).selectCompaction(si, al(), false).getRequest().getFiles()
292 .size());
293
294 conf.setFloat("hbase.hstore.compaction.ratio.offpeak", 500f);
295 assertEquals(3, createPolicy(conf).selectCompaction(si, al(), true).getRequest().getFiles()
296 .size());
297 }
298
299 @Test
300 public void testSplitOffStripeDropDeletes() throws Exception {
301 Configuration conf = HBaseConfiguration.create();
302 conf.setInt(StripeStoreConfig.MIN_FILES_KEY, 2);
303 StripeCompactionPolicy policy = createPolicy(conf);
304 Long[] toSplit = new Long[] { defaultSplitSize / 2, defaultSplitSize / 2 };
305 Long[] noSplit = new Long[] { 1L };
306 long splitTargetSize = (long)(defaultSplitSize / defaultSplitCount);
307
308
309 StripeCompactionPolicy.StripeInformationProvider si =
310 createStripesWithSizes(0, 0, noSplit, toSplit);
311 verifyWholeStripesCompaction(policy, si, 1, 1, true, null, splitTargetSize);
312
313 si = createStripesWithSizes(2, 2, noSplit, toSplit);
314 verifyWholeStripesCompaction(policy, si, 1, 1, false, null, splitTargetSize);
315 }
316
317 @SuppressWarnings("unchecked")
318 @Test
319 public void testMergeExpiredFiles() throws Exception {
320 ManualEnvironmentEdge edge = new ManualEnvironmentEdge();
321 long now = defaultTtl + 2;
322 edge.setValue(now);
323 EnvironmentEdgeManager.injectEdge(edge);
324 try {
325 StoreFile expiredFile = createFile(), notExpiredFile = createFile();
326 when(expiredFile.getReader().getMaxTimestamp()).thenReturn(now - defaultTtl - 1);
327 when(notExpiredFile.getReader().getMaxTimestamp()).thenReturn(now - defaultTtl + 1);
328 List<StoreFile> expired = Lists.newArrayList(expiredFile, expiredFile);
329 List<StoreFile> notExpired = Lists.newArrayList(notExpiredFile, notExpiredFile);
330 List<StoreFile> mixed = Lists.newArrayList(expiredFile, notExpiredFile);
331
332 StripeCompactionPolicy policy = createPolicy(HBaseConfiguration.create(),
333 defaultSplitSize, defaultSplitCount, defaultInitialCount, true);
334
335 StripeCompactionPolicy.StripeInformationProvider si =
336 createStripesWithFiles(expired, expired, expired);
337 verifyWholeStripesCompaction(policy, si, 0, 2, null, 1, Long.MAX_VALUE, false);
338
339 si = createStripesWithFiles(notExpired, notExpired, notExpired);
340 assertNull(policy.selectCompaction(si, al(), false));
341
342 si = createStripesWithFiles(notExpired, expired, notExpired);
343 verifyWholeStripesCompaction(policy, si, 1, 2, null, 1, Long.MAX_VALUE, false);
344
345
346 si = createStripesWithFiles(notExpired, expired, notExpired, expired, expired, notExpired);
347 verifyWholeStripesCompaction(policy, si, 3, 4, null, 1, Long.MAX_VALUE, false);
348
349 si = createStripesWithFiles(expired, expired, notExpired, expired, mixed);
350 verifyWholeStripesCompaction(policy, si, 0, 1, null, 1, Long.MAX_VALUE, false);
351 } finally {
352 EnvironmentEdgeManager.reset();
353 }
354 }
355
356 @SuppressWarnings("unchecked")
357 @Test
358 public void testMergeExpiredStripes() throws Exception {
359
360 ManualEnvironmentEdge edge = new ManualEnvironmentEdge();
361 long now = defaultTtl + 2;
362 edge.setValue(now);
363 EnvironmentEdgeManager.injectEdge(edge);
364 try {
365 StoreFile expiredFile = createFile(), notExpiredFile = createFile();
366 when(expiredFile.getReader().getMaxTimestamp()).thenReturn(now - defaultTtl - 1);
367 when(notExpiredFile.getReader().getMaxTimestamp()).thenReturn(now - defaultTtl + 1);
368 List<StoreFile> expired = Lists.newArrayList(expiredFile, expiredFile);
369 List<StoreFile> notExpired = Lists.newArrayList(notExpiredFile, notExpiredFile);
370
371 StripeCompactionPolicy policy =
372 createPolicy(HBaseConfiguration.create(), defaultSplitSize, defaultSplitCount,
373 defaultInitialCount, true);
374
375
376 StripeCompactionPolicy.StripeInformationProvider si =
377 createStripesWithFiles(expired, expired, expired);
378 verifyMergeCompatcion(policy, si, 0, 2);
379
380
381 si = createStripesWithFiles(notExpired, expired, notExpired, expired, expired, notExpired);
382 verifyMergeCompatcion(policy, si, 3, 4);
383 } finally {
384 EnvironmentEdgeManager.reset();
385 }
386 }
387
388 private static StripeCompactionPolicy.StripeInformationProvider createStripesWithFiles(
389 List<StoreFile>... stripeFiles) throws Exception {
390 return createStripesWithFiles(createBoundaries(stripeFiles.length),
391 Lists.newArrayList(stripeFiles), new ArrayList<StoreFile>());
392 }
393
394 @Test
395 public void testSingleStripeDropDeletes() throws Exception {
396 Configuration conf = HBaseConfiguration.create();
397 StripeCompactionPolicy policy = createPolicy(conf);
398
399 Long[][] stripes = new Long[][] { new Long[] { 3L, 2L, 2L, 2L }, new Long[] { 6L } };
400 StripeInformationProvider si = createStripesWithSizes(0, 0, stripes);
401 verifySingleStripeCompaction(policy, si, 0, true);
402
403 si = createStripesWithSizes(2, 2, stripes);
404 verifySingleStripeCompaction(policy, si, 0, false);
405
406 si = createStripesWithSizes(6, 2, stripes);
407 ConcatenatedLists<StoreFile> sfs = new ConcatenatedLists<StoreFile>();
408 sfs.addSublist(si.getLevel0Files());
409 sfs.addSublist(si.getStripes().get(0));
410 verifyCompaction(
411 policy, si, sfs, si.getStartRow(0), si.getEndRow(0), si.getStripeBoundaries());
412
413 si = createStripesWithSizes(6, 2,
414 new Long[][] { new Long[] { 10L, 1L, 1L, 1L, 1L }, new Long[] { 12L } });
415 verifyCompaction(policy, si, si.getLevel0Files(), null, null, si.getStripeBoundaries());
416
417
418 stripes = new Long[][] { new Long[] { 100L, 3L, 2L, 2L, 2L }, new Long[] { 6L } };
419 si = createStripesWithSizes(0, 0, stripes);
420 List<StoreFile> compact_file = new ArrayList<StoreFile>();
421 Iterator<StoreFile> iter = si.getStripes().get(0).listIterator(1);
422 while (iter.hasNext()) {
423 compact_file.add(iter.next());
424 }
425 verifyCompaction(policy, si, compact_file, false, 1, null, si.getStartRow(0), si.getEndRow(0), true);
426 }
427
428
429 private static StripeCompactionPolicy createPolicy(
430 Configuration conf) throws Exception {
431 return createPolicy(conf, defaultSplitSize, defaultSplitCount, defaultInitialCount, false);
432 }
433
434 private static StripeCompactionPolicy createPolicy(Configuration conf,
435 long splitSize, float splitCount, int initialCount, boolean hasTtl) throws Exception {
436 conf.setLong(StripeStoreConfig.SIZE_TO_SPLIT_KEY, splitSize);
437 conf.setFloat(StripeStoreConfig.SPLIT_PARTS_KEY, splitCount);
438 conf.setInt(StripeStoreConfig.INITIAL_STRIPE_COUNT_KEY, initialCount);
439 StoreConfigInformation sci = mock(StoreConfigInformation.class);
440 when(sci.getStoreFileTtl()).thenReturn(hasTtl ? defaultTtl : Long.MAX_VALUE);
441 StripeStoreConfig ssc = new StripeStoreConfig(conf, sci);
442 return new StripeCompactionPolicy(conf, sci, ssc);
443 }
444
445 private static ArrayList<StoreFile> al(StoreFile... sfs) {
446 return new ArrayList<StoreFile>(Arrays.asList(sfs));
447 }
448
449 private void verifyMergeCompatcion(StripeCompactionPolicy policy, StripeInformationProvider si,
450 int from, int to) throws Exception {
451 StripeCompactionPolicy.StripeCompactionRequest scr = policy.selectCompaction(si, al(), false);
452 Collection<StoreFile> sfs = getAllFiles(si, from, to);
453 verifyCollectionsEqual(sfs, scr.getRequest().getFiles());
454
455
456
457 StripeCompactor sc = createCompactor();
458 List<Path> paths = scr.execute(sc, NoLimitCompactionThroughputController.INSTANCE);
459 assertEquals(1, paths.size());
460 }
461
462
463
464
465
466
467
468
469
470
471
472 private void verifyWholeStripesCompaction(StripeCompactionPolicy policy,
473 StripeInformationProvider si, int from, int to, Boolean dropDeletes,
474 Integer count, Long size, boolean needsCompaction) throws IOException {
475 verifyCompaction(policy, si, getAllFiles(si, from, to), dropDeletes,
476 count, size, si.getStartRow(from), si.getEndRow(to), needsCompaction);
477 }
478
479 private void verifyWholeStripesCompaction(StripeCompactionPolicy policy,
480 StripeInformationProvider si, int from, int to, Boolean dropDeletes,
481 Integer count, Long size) throws IOException {
482 verifyWholeStripesCompaction(policy, si, from, to, dropDeletes, count, size, true);
483 }
484
485 private void verifySingleStripeCompaction(StripeCompactionPolicy policy,
486 StripeInformationProvider si, int index, Boolean dropDeletes) throws IOException {
487 verifyWholeStripesCompaction(policy, si, index, index, dropDeletes, 1, null, true);
488 }
489
490
491
492
493
494
495 private void verifyNoCompaction(
496 StripeCompactionPolicy policy, StripeInformationProvider si) throws IOException {
497 assertNull(policy.selectCompaction(si, al(), false));
498 assertFalse(policy.needsCompactions(si, al()));
499 }
500
501
502
503
504
505
506
507
508
509
510 private void verifyCompaction(StripeCompactionPolicy policy, StripeInformationProvider si,
511 Collection<StoreFile> sfs, byte[] dropDeletesFrom, byte[] dropDeletesTo,
512 final List<byte[]> boundaries) throws Exception {
513 StripeCompactor sc = mock(StripeCompactor.class);
514 assertTrue(policy.needsCompactions(si, al()));
515 StripeCompactionPolicy.StripeCompactionRequest scr = policy.selectCompaction(si, al(), false);
516 verifyCollectionsEqual(sfs, scr.getRequest().getFiles());
517 scr.execute(sc, NoLimitCompactionThroughputController.INSTANCE);
518 verify(sc, times(1)).compact(eq(scr.getRequest()), argThat(new ArgumentMatcher<List<byte[]>>() {
519 @Override
520 public boolean matches(Object argument) {
521 @SuppressWarnings("unchecked")
522 List<byte[]> other = (List<byte[]>) argument;
523 if (other.size() != boundaries.size()) return false;
524 for (int i = 0; i < other.size(); ++i) {
525 if (!Bytes.equals(other.get(i), boundaries.get(i))) return false;
526 }
527 return true;
528 }
529 }), dropDeletesFrom == null ? isNull(byte[].class) : aryEq(dropDeletesFrom),
530 dropDeletesTo == null ? isNull(byte[].class) : aryEq(dropDeletesTo),
531 any(NoLimitCompactionThroughputController.class));
532 }
533
534
535
536
537
538
539
540
541
542
543
544
545 private void verifyCompaction(StripeCompactionPolicy policy, StripeInformationProvider si,
546 Collection<StoreFile> sfs, Boolean dropDeletes, Integer count, Long size,
547 byte[] start, byte[] end, boolean needsCompaction) throws IOException {
548 StripeCompactor sc = mock(StripeCompactor.class);
549 assertTrue(!needsCompaction || policy.needsCompactions(si, al()));
550 StripeCompactionPolicy.StripeCompactionRequest scr = policy.selectCompaction(si, al(), false);
551 verifyCollectionsEqual(sfs, scr.getRequest().getFiles());
552 scr.execute(sc, NoLimitCompactionThroughputController.INSTANCE);
553 verify(sc, times(1)).compact(eq(scr.getRequest()),
554 count == null ? anyInt() : eq(count.intValue()),
555 size == null ? anyLong() : eq(size.longValue()), aryEq(start), aryEq(end),
556 dropDeletesMatcher(dropDeletes, start), dropDeletesMatcher(dropDeletes, end),
557 any(NoLimitCompactionThroughputController.class));
558 }
559
560
561 protected void verifyFlush(StripeCompactionPolicy policy, StripeInformationProvider si,
562 KeyValue[] input, KeyValue[][] expected, byte[][] boundaries) throws IOException {
563 StoreFileWritersCapture writers = new StoreFileWritersCapture();
564 StripeStoreFlusher.StripeFlushRequest req = policy.selectFlush(si, input.length);
565 StripeMultiFileWriter mw = req.createWriter();
566 mw.init(null, writers, new KeyValue.KVComparator());
567 for (KeyValue kv : input) {
568 mw.append(kv);
569 }
570 boolean hasMetadata = boundaries != null;
571 mw.commitWriters(0, false);
572 writers.verifyKvs(expected, true, hasMetadata);
573 if (hasMetadata) {
574 writers.verifyBoundaries(boundaries);
575 }
576 }
577
578
579 private byte[] dropDeletesMatcher(Boolean dropDeletes, byte[] value) {
580 return dropDeletes == null ? any(byte[].class)
581 : (dropDeletes.booleanValue() ? aryEq(value) : isNull(byte[].class));
582 }
583
584 private void verifyCollectionsEqual(Collection<StoreFile> sfs, Collection<StoreFile> scr) {
585
586 assertEquals(sfs.size(), scr.size());
587 assertTrue(scr.containsAll(sfs));
588 }
589
590 private static List<StoreFile> getAllFiles(
591 StripeInformationProvider si, int fromStripe, int toStripe) {
592 ArrayList<StoreFile> expected = new ArrayList<StoreFile>();
593 for (int i = fromStripe; i <= toStripe; ++i) {
594 expected.addAll(si.getStripes().get(i));
595 }
596 return expected;
597 }
598
599
600
601
602
603
604 private static StripeInformationProvider createStripes(
605 int l0Count, byte[]... boundaries) throws Exception {
606 List<Long> l0Sizes = new ArrayList<Long>();
607 for (int i = 0; i < l0Count; ++i) {
608 l0Sizes.add(5L);
609 }
610 List<List<Long>> sizes = new ArrayList<List<Long>>();
611 for (int i = 0; i <= boundaries.length; ++i) {
612 sizes.add(Arrays.asList(Long.valueOf(5)));
613 }
614 return createStripes(Arrays.asList(boundaries), sizes, l0Sizes);
615 }
616
617
618
619
620
621
622 private static StripeInformationProvider createStripesL0Only(
623 int l0Count, long l0Size) throws Exception {
624 List<Long> l0Sizes = new ArrayList<Long>();
625 for (int i = 0; i < l0Count; ++i) {
626 l0Sizes.add(l0Size);
627 }
628 return createStripes(null, new ArrayList<List<Long>>(), l0Sizes);
629 }
630
631
632
633
634
635
636
637 private static StripeInformationProvider createStripesWithSizes(
638 int l0Count, long l0Size, Long[]... sizes) throws Exception {
639 ArrayList<List<Long>> sizeList = new ArrayList<List<Long>>();
640 for (Long[] size : sizes) {
641 sizeList.add(Arrays.asList(size));
642 }
643 return createStripesWithSizes(l0Count, l0Size, sizeList);
644 }
645
646 private static StripeInformationProvider createStripesWithSizes(
647 int l0Count, long l0Size, List<List<Long>> sizes) throws Exception {
648 List<byte[]> boundaries = createBoundaries(sizes.size());
649 List<Long> l0Sizes = new ArrayList<Long>();
650 for (int i = 0; i < l0Count; ++i) {
651 l0Sizes.add(l0Size);
652 }
653 return createStripes(boundaries, sizes, l0Sizes);
654 }
655
656 private static List<byte[]> createBoundaries(int stripeCount) {
657 byte[][] keys = new byte[][] { KEY_A, KEY_B, KEY_C, KEY_D, KEY_E };
658 assert stripeCount <= keys.length + 1;
659 List<byte[]> boundaries = new ArrayList<byte[]>();
660 boundaries.addAll(Arrays.asList(keys).subList(0, stripeCount - 1));
661 return boundaries;
662 }
663
664 private static StripeInformationProvider createStripes(List<byte[]> boundaries,
665 List<List<Long>> stripeSizes, List<Long> l0Sizes) throws Exception {
666 List<List<StoreFile>> stripeFiles = new ArrayList<List<StoreFile>>(stripeSizes.size());
667 for (List<Long> sizes : stripeSizes) {
668 List<StoreFile> sfs = new ArrayList<StoreFile>();
669 for (Long size : sizes) {
670 sfs.add(createFile(size));
671 }
672 stripeFiles.add(sfs);
673 }
674 List<StoreFile> l0Files = new ArrayList<StoreFile>();
675 for (Long size : l0Sizes) {
676 l0Files.add(createFile(size));
677 }
678 return createStripesWithFiles(boundaries, stripeFiles, l0Files);
679 }
680
681
682
683
684 private static StripeInformationProvider createStripesWithFiles(List<byte[]> boundaries,
685 List<List<StoreFile>> stripeFiles, List<StoreFile> l0Files) throws Exception {
686 ArrayList<ImmutableList<StoreFile>> stripes = new ArrayList<ImmutableList<StoreFile>>();
687 ArrayList<byte[]> boundariesList = new ArrayList<byte[]>();
688 StripeInformationProvider si = mock(StripeInformationProvider.class);
689 if (!stripeFiles.isEmpty()) {
690 assert stripeFiles.size() == (boundaries.size() + 1);
691 boundariesList.add(OPEN_KEY);
692 for (int i = 0; i <= boundaries.size(); ++i) {
693 byte[] startKey = ((i == 0) ? OPEN_KEY : boundaries.get(i - 1));
694 byte[] endKey = ((i == boundaries.size()) ? OPEN_KEY : boundaries.get(i));
695 boundariesList.add(endKey);
696 for (StoreFile sf : stripeFiles.get(i)) {
697 setFileStripe(sf, startKey, endKey);
698 }
699 stripes.add(ImmutableList.copyOf(stripeFiles.get(i)));
700 when(si.getStartRow(eq(i))).thenReturn(startKey);
701 when(si.getEndRow(eq(i))).thenReturn(endKey);
702 }
703 }
704 ConcatenatedLists<StoreFile> sfs = new ConcatenatedLists<StoreFile>();
705 sfs.addAllSublists(stripes);
706 sfs.addSublist(l0Files);
707 when(si.getStorefiles()).thenReturn(sfs);
708 when(si.getStripes()).thenReturn(stripes);
709 when(si.getStripeBoundaries()).thenReturn(boundariesList);
710 when(si.getStripeCount()).thenReturn(stripes.size());
711 when(si.getLevel0Files()).thenReturn(l0Files);
712 return si;
713 }
714
715 private static StoreFile createFile(long size) throws Exception {
716 StoreFile sf = mock(StoreFile.class);
717 when(sf.getPath()).thenReturn(new Path("moo"));
718 StoreFile.Reader r = mock(StoreFile.Reader.class);
719 when(r.getEntries()).thenReturn(size);
720 when(r.length()).thenReturn(size);
721 when(r.getBloomFilterType()).thenReturn(BloomType.NONE);
722 when(r.getHFileReader()).thenReturn(mock(HFile.Reader.class));
723 when(r.getStoreFileScanner(anyBoolean(), anyBoolean(), anyBoolean(), anyLong())).thenReturn(
724 mock(StoreFileScanner.class));
725 when(sf.getReader()).thenReturn(r);
726 when(sf.createReader()).thenReturn(r);
727 return sf;
728 }
729
730 private static StoreFile createFile() throws Exception {
731 return createFile(0);
732 }
733
734 private static void setFileStripe(StoreFile sf, byte[] startKey, byte[] endKey) {
735 when(sf.getMetadataValue(StripeStoreFileManager.STRIPE_START_KEY)).thenReturn(startKey);
736 when(sf.getMetadataValue(StripeStoreFileManager.STRIPE_END_KEY)).thenReturn(endKey);
737 }
738
739 private static StripeCompactor createCompactor() throws Exception {
740 HColumnDescriptor col = new HColumnDescriptor(Bytes.toBytes("foo"));
741 StoreFileWritersCapture writers = new StoreFileWritersCapture();
742 Store store = mock(Store.class);
743 HRegionInfo info = mock(HRegionInfo.class);
744 when(info.getRegionNameAsString()).thenReturn("testRegion");
745 when(store.getFamily()).thenReturn(col);
746 when(store.getRegionInfo()).thenReturn(info);
747 when(
748 store.createWriterInTmp(anyLong(), any(Compression.Algorithm.class), anyBoolean(),
749 anyBoolean(), anyBoolean())).thenAnswer(writers);
750
751 Configuration conf = HBaseConfiguration.create();
752 final Scanner scanner = new Scanner();
753 return new StripeCompactor(conf, store) {
754 @Override
755 protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
756 long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,
757 byte[] dropDeletesToRow) throws IOException {
758 return scanner;
759 }
760
761 @Override
762 protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
763 ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
764 return scanner;
765 }
766 };
767 }
768
769 private static class Scanner implements InternalScanner {
770 private final ArrayList<KeyValue> kvs;
771
772 public Scanner(KeyValue... kvs) {
773 this.kvs = new ArrayList<KeyValue>(Arrays.asList(kvs));
774 }
775
776 @Override
777 public boolean next(List<Cell> results) throws IOException {
778 if (kvs.isEmpty()) return false;
779 results.add(kvs.remove(0));
780 return !kvs.isEmpty();
781 }
782
783 @Override
784 public boolean next(List<Cell> result, int limit) throws IOException {
785 return next(result);
786 }
787
788 @Override
789 public void close() throws IOException {
790 }
791 }
792 }