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