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