View Javadoc

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