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