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