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.hadoop.conf.Configuration;
46  import org.apache.hadoop.fs.Path;
47  import org.apache.hadoop.hbase.Cell;
48  import org.apache.hadoop.hbase.HBaseConfiguration;
49  import org.apache.hadoop.hbase.HColumnDescriptor;
50  import org.apache.hadoop.hbase.HRegionInfo;
51  import org.apache.hadoop.hbase.KeyValue;
52  import org.apache.hadoop.hbase.io.compress.Compression;
53  import org.apache.hadoop.hbase.io.hfile.HFile;
54  import org.apache.hadoop.hbase.regionserver.BloomType;
55  import org.apache.hadoop.hbase.regionserver.InternalScanner;
56  import org.apache.hadoop.hbase.regionserver.ScanType;
57  import org.apache.hadoop.hbase.regionserver.Store;
58  import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
59  import org.apache.hadoop.hbase.regionserver.StoreFile;
60  import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
61  import org.apache.hadoop.hbase.regionserver.StripeMultiFileWriter;
62  import org.apache.hadoop.hbase.regionserver.StripeStoreConfig;
63  import org.apache.hadoop.hbase.regionserver.StripeStoreFileManager;
64  import org.apache.hadoop.hbase.regionserver.StripeStoreFlusher;
65  import org.apache.hadoop.hbase.regionserver.TestStripeCompactor.StoreFileWritersCapture;
66  import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy.StripeInformationProvider;
67  import org.apache.hadoop.hbase.testclassification.SmallTests;
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, NoLimitCompactionThroughputController.INSTANCE);
213     verify(sc, only()).compact(eq(scr.getRequest()), anyInt(), anyLong(), aryEq(OPEN_KEY),
214       aryEq(OPEN_KEY), aryEq(OPEN_KEY), aryEq(OPEN_KEY),
215       any(NoLimitCompactionThroughputController.class));
216   }
217 
218   @Test
219   public void testInitialCountFromL0() throws Exception {
220     Configuration conf = HBaseConfiguration.create();
221     conf.setInt(StripeStoreConfig.MIN_FILES_L0_KEY, 2);
222     StripeCompactionPolicy policy = createPolicy(
223         conf, defaultSplitSize, defaultSplitCount, 2, false);
224     StripeCompactionPolicy.StripeInformationProvider si = createStripesL0Only(3, 8);
225     verifyCompaction(policy, si, si.getStorefiles(), true, 2, 12L, OPEN_KEY, OPEN_KEY, true);
226     si = createStripesL0Only(3, 10); // If result would be too large, split into smaller parts.
227     verifyCompaction(policy, si, si.getStorefiles(), true, 3, 10L, OPEN_KEY, OPEN_KEY, true);
228     policy = createPolicy(conf, defaultSplitSize, defaultSplitCount, 6, false);
229     verifyCompaction(policy, si, si.getStorefiles(), true, 6, 5L, OPEN_KEY, OPEN_KEY, true);
230   }
231 
232   @Test
233   public void testExistingStripesFromL0() throws Exception {
234     Configuration conf = HBaseConfiguration.create();
235     conf.setInt(StripeStoreConfig.MIN_FILES_L0_KEY, 3);
236     StripeCompactionPolicy.StripeInformationProvider si = createStripes(3, KEY_A);
237     verifyCompaction(
238         createPolicy(conf), si, si.getLevel0Files(), null, null, si.getStripeBoundaries());
239   }
240 
241   @Test
242   public void testNothingToCompactFromL0() throws Exception {
243     Configuration conf = HBaseConfiguration.create();
244     conf.setInt(StripeStoreConfig.MIN_FILES_L0_KEY, 4);
245     StripeCompactionPolicy.StripeInformationProvider si = createStripesL0Only(3, 10);
246     StripeCompactionPolicy policy = createPolicy(conf);
247     verifyNoCompaction(policy, si);
248 
249     si = createStripes(3, KEY_A);
250     verifyNoCompaction(policy, si);
251   }
252 
253   @Test
254   public void testSplitOffStripe() throws Exception {
255     Configuration conf = HBaseConfiguration.create();
256     // First test everything with default split count of 2, then split into more.
257     conf.setInt(StripeStoreConfig.MIN_FILES_KEY, 2);
258     Long[] toSplit = new Long[] { defaultSplitSize - 2, 1L, 1L };
259     Long[] noSplit = new Long[] { defaultSplitSize - 2, 1L };
260     long splitTargetSize = (long)(defaultSplitSize / defaultSplitCount);
261     // Don't split if not eligible for compaction.
262     StripeCompactionPolicy.StripeInformationProvider si =
263         createStripesWithSizes(0, 0, new Long[] { defaultSplitSize - 2, 2L });
264     assertNull(createPolicy(conf).selectCompaction(si, al(), false));
265     // Make sure everything is eligible.
266     conf.setFloat(CompactionConfiguration.RATIO_KEY, 500f);
267     StripeCompactionPolicy policy = createPolicy(conf);
268     verifyWholeStripesCompaction(policy, si, 0, 0, null, 2, splitTargetSize);
269     // Add some extra stripes...
270     si = createStripesWithSizes(0, 0, noSplit, noSplit, toSplit);
271     verifyWholeStripesCompaction(policy, si, 2, 2, null, 2, splitTargetSize);
272     // In the middle.
273     si = createStripesWithSizes(0, 0, noSplit, toSplit, noSplit);
274     verifyWholeStripesCompaction(policy, si, 1, 1, null, 2, splitTargetSize);
275     // No split-off with different config (larger split size).
276     // However, in this case some eligible stripe will just be compacted alone.
277     StripeCompactionPolicy specPolicy = createPolicy(
278         conf, defaultSplitSize + 1, defaultSplitCount, defaultInitialCount, false);
279     verifySingleStripeCompaction(specPolicy, si, 1, null);
280   }
281 
282   @Test
283   public void testSplitOffStripeOffPeak() throws Exception {
284     // for HBASE-11439
285     Configuration conf = HBaseConfiguration.create();
286     conf.setInt(StripeStoreConfig.MIN_FILES_KEY, 2);
287     // Select the last 2 files.
288     StripeCompactionPolicy.StripeInformationProvider si =
289         createStripesWithSizes(0, 0, new Long[] { defaultSplitSize - 2, 1L, 1L });
290     assertEquals(2, createPolicy(conf).selectCompaction(si, al(), false).getRequest().getFiles()
291         .size());
292     // Make sure everything is eligible in offpeak.
293     conf.setFloat("hbase.hstore.compaction.ratio.offpeak", 500f);
294     assertEquals(3, createPolicy(conf).selectCompaction(si, al(), true).getRequest().getFiles()
295         .size());
296   }
297 
298   @Test
299   public void testSplitOffStripeDropDeletes() throws Exception {
300     Configuration conf = HBaseConfiguration.create();
301     conf.setInt(StripeStoreConfig.MIN_FILES_KEY, 2);
302     StripeCompactionPolicy policy = createPolicy(conf);
303     Long[] toSplit = new Long[] { defaultSplitSize / 2, defaultSplitSize / 2 };
304     Long[] noSplit = new Long[] { 1L };
305     long splitTargetSize = (long)(defaultSplitSize / defaultSplitCount);
306 
307     // Verify the deletes can be dropped if there are no L0 files.
308     StripeCompactionPolicy.StripeInformationProvider si =
309         createStripesWithSizes(0, 0, noSplit, toSplit);
310     verifyWholeStripesCompaction(policy, si, 1, 1,    true, null, splitTargetSize);
311     // But cannot be dropped if there are.
312     si = createStripesWithSizes(2, 2, noSplit, toSplit);
313     verifyWholeStripesCompaction(policy, si, 1, 1,    false, null, splitTargetSize);
314   }
315 
316   @SuppressWarnings("unchecked")
317   @Test
318   public void testMergeExpiredFiles() throws Exception {
319     ManualEnvironmentEdge edge = new ManualEnvironmentEdge();
320     long now = defaultTtl + 2;
321     edge.setValue(now);
322     EnvironmentEdgeManager.injectEdge(edge);
323     try {
324       StoreFile expiredFile = createFile(), notExpiredFile = createFile();
325       when(expiredFile.getReader().getMaxTimestamp()).thenReturn(now - defaultTtl - 1);
326       when(notExpiredFile.getReader().getMaxTimestamp()).thenReturn(now - defaultTtl + 1);
327       List<StoreFile> expired = Lists.newArrayList(expiredFile, expiredFile);
328       List<StoreFile> notExpired = Lists.newArrayList(notExpiredFile, notExpiredFile);
329       List<StoreFile> mixed = Lists.newArrayList(expiredFile, notExpiredFile);
330 
331       StripeCompactionPolicy policy = createPolicy(HBaseConfiguration.create(),
332           defaultSplitSize, defaultSplitCount, defaultInitialCount, true);
333       // Merge expired if there are eligible stripes.
334       StripeCompactionPolicy.StripeInformationProvider si =
335           createStripesWithFiles(expired, expired, expired);
336       verifyWholeStripesCompaction(policy, si, 0, 2, null, 1, Long.MAX_VALUE, false);
337       // Don't merge if nothing expired.
338       si = createStripesWithFiles(notExpired, notExpired, notExpired);
339       assertNull(policy.selectCompaction(si, al(), false));
340       // Merge one expired stripe with next.
341       si = createStripesWithFiles(notExpired, expired, notExpired);
342       verifyWholeStripesCompaction(policy, si, 1, 2, null, 1, Long.MAX_VALUE, false);
343       // Merge the biggest run out of multiple options.
344       // Merge one expired stripe with next.
345       si = createStripesWithFiles(notExpired, expired, notExpired, expired, expired, notExpired);
346       verifyWholeStripesCompaction(policy, si, 3, 4, null, 1, Long.MAX_VALUE, false);
347       // Stripe with a subset of expired files is not merged.
348       si = createStripesWithFiles(expired, expired, notExpired, expired, mixed);
349       verifyWholeStripesCompaction(policy, si, 0, 1, null, 1, Long.MAX_VALUE, false);
350     } finally {
351       EnvironmentEdgeManager.reset();
352     }
353   }
354 
355   @SuppressWarnings("unchecked")
356   @Test
357   public void testMergeExpiredStripes() throws Exception {
358     // HBASE-11397
359     ManualEnvironmentEdge edge = new ManualEnvironmentEdge();
360     long now = defaultTtl + 2;
361     edge.setValue(now);
362     EnvironmentEdgeManager.injectEdge(edge);
363     try {
364       StoreFile expiredFile = createFile(), notExpiredFile = createFile();
365       when(expiredFile.getReader().getMaxTimestamp()).thenReturn(now - defaultTtl - 1);
366       when(notExpiredFile.getReader().getMaxTimestamp()).thenReturn(now - defaultTtl + 1);
367       List<StoreFile> expired = Lists.newArrayList(expiredFile, expiredFile);
368       List<StoreFile> notExpired = Lists.newArrayList(notExpiredFile, notExpiredFile);
369 
370       StripeCompactionPolicy policy =
371           createPolicy(HBaseConfiguration.create(), defaultSplitSize, defaultSplitCount,
372             defaultInitialCount, true);
373 
374       // Merge all three expired stripes into one.
375       StripeCompactionPolicy.StripeInformationProvider si =
376           createStripesWithFiles(expired, expired, expired);
377       verifyMergeCompatcion(policy, si, 0, 2);
378 
379       // Merge two adjacent expired stripes into one.
380       si = createStripesWithFiles(notExpired, expired, notExpired, expired, expired, notExpired);
381       verifyMergeCompatcion(policy, si, 3, 4);
382     } finally {
383       EnvironmentEdgeManager.reset();
384     }
385   }
386 
387   private static StripeCompactionPolicy.StripeInformationProvider createStripesWithFiles(
388       List<StoreFile>... stripeFiles) throws Exception {
389     return createStripesWithFiles(createBoundaries(stripeFiles.length),
390         Lists.newArrayList(stripeFiles), new ArrayList<StoreFile>());
391   }
392 
393   @Test
394   public void testSingleStripeDropDeletes() throws Exception {
395     Configuration conf = HBaseConfiguration.create();
396     StripeCompactionPolicy policy = createPolicy(conf);
397     // Verify the deletes can be dropped if there are no L0 files.
398     Long[][] stripes = new Long[][] { new Long[] { 3L, 2L, 2L, 2L }, new Long[] { 6L } };
399     StripeInformationProvider si = createStripesWithSizes(0, 0, stripes);
400     verifySingleStripeCompaction(policy, si, 0, true);
401     // But cannot be dropped if there are.
402     si = createStripesWithSizes(2, 2, stripes);
403     verifySingleStripeCompaction(policy, si, 0, false);
404     // Unless there are enough to cause L0 compaction.
405     si = createStripesWithSizes(6, 2, stripes);
406     ConcatenatedLists<StoreFile> sfs = new ConcatenatedLists<StoreFile>();
407     sfs.addSublist(si.getLevel0Files());
408     sfs.addSublist(si.getStripes().get(0));
409     verifyCompaction(
410         policy, si, sfs, si.getStartRow(0), si.getEndRow(0), si.getStripeBoundaries());
411     // If we cannot actually compact all files in some stripe, L0 is chosen.
412     si = createStripesWithSizes(6, 2,
413         new Long[][] { new Long[] { 10L, 1L, 1L, 1L, 1L }, new Long[] { 12L } });
414     verifyCompaction(policy, si, si.getLevel0Files(), null, null, si.getStripeBoundaries());
415   }
416 
417   /********* HELPER METHODS ************/
418   private static StripeCompactionPolicy createPolicy(
419       Configuration conf) throws Exception {
420     return createPolicy(conf, defaultSplitSize, defaultSplitCount, defaultInitialCount, false);
421   }
422 
423   private static StripeCompactionPolicy createPolicy(Configuration conf,
424       long splitSize, float splitCount, int initialCount, boolean hasTtl) throws Exception {
425     conf.setLong(StripeStoreConfig.SIZE_TO_SPLIT_KEY, splitSize);
426     conf.setFloat(StripeStoreConfig.SPLIT_PARTS_KEY, splitCount);
427     conf.setInt(StripeStoreConfig.INITIAL_STRIPE_COUNT_KEY, initialCount);
428     StoreConfigInformation sci = mock(StoreConfigInformation.class);
429     when(sci.getStoreFileTtl()).thenReturn(hasTtl ? defaultTtl : Long.MAX_VALUE);
430     StripeStoreConfig ssc = new StripeStoreConfig(conf, sci);
431     return new StripeCompactionPolicy(conf, sci, ssc);
432   }
433 
434   private static ArrayList<StoreFile> al(StoreFile... sfs) {
435     return new ArrayList<StoreFile>(Arrays.asList(sfs));
436   }
437 
438   private void verifyMergeCompatcion(StripeCompactionPolicy policy, StripeInformationProvider si,
439       int from, int to) throws Exception {
440     StripeCompactionPolicy.StripeCompactionRequest scr = policy.selectCompaction(si, al(), false);
441     Collection<StoreFile> sfs = getAllFiles(si, from, to);
442     verifyCollectionsEqual(sfs, scr.getRequest().getFiles());
443 
444     // All the Stripes are expired, so the Compactor will not create any Writers. We need to create
445     // an empty file to preserve metadata
446     StripeCompactor sc = createCompactor();
447     List<Path> paths = scr.execute(sc, NoLimitCompactionThroughputController.INSTANCE);
448     assertEquals(1, paths.size());
449   }
450 
451   /**
452    * Verify the compaction that includes several entire stripes.
453    * @param policy Policy to test.
454    * @param si Stripe information pre-set with stripes to test.
455    * @param from Starting stripe.
456    * @param to Ending stripe (inclusive).
457    * @param dropDeletes Whether to drop deletes from compaction range.
458    * @param count Expected # of resulting stripes, null if not checked.
459    * @param size Expected target stripe size, null if not checked.
460    */
461   private void verifyWholeStripesCompaction(StripeCompactionPolicy policy,
462       StripeInformationProvider si, int from, int to, Boolean dropDeletes,
463       Integer count, Long size, boolean needsCompaction) throws IOException {
464     verifyCompaction(policy, si, getAllFiles(si, from, to), dropDeletes,
465         count, size, si.getStartRow(from), si.getEndRow(to), needsCompaction);
466   }
467 
468   private void verifyWholeStripesCompaction(StripeCompactionPolicy policy,
469       StripeInformationProvider si, int from, int to, Boolean dropDeletes,
470       Integer count, Long size) throws IOException {
471     verifyWholeStripesCompaction(policy, si, from, to, dropDeletes, count, size, true);
472   }
473 
474   private void verifySingleStripeCompaction(StripeCompactionPolicy policy,
475       StripeInformationProvider si, int index, Boolean dropDeletes) throws IOException {
476     verifyWholeStripesCompaction(policy, si, index, index, dropDeletes, 1, null, true);
477   }
478 
479   /**
480    * Verify no compaction is needed or selected.
481    * @param policy Policy to test.
482    * @param si Stripe information pre-set with stripes to test.
483    */
484   private void verifyNoCompaction(
485       StripeCompactionPolicy policy, StripeInformationProvider si) throws IOException {
486     assertNull(policy.selectCompaction(si, al(), false));
487     assertFalse(policy.needsCompactions(si, al()));
488   }
489 
490   /**
491    * Verify arbitrary compaction.
492    * @param policy Policy to test.
493    * @param si Stripe information pre-set with stripes to test.
494    * @param sfs Files that should be compacted.
495    * @param dropDeletesFrom Row from which to drop deletes.
496    * @param dropDeletesTo Row to which to drop deletes.
497    * @param boundaries Expected target stripe boundaries.
498    */
499   private void verifyCompaction(StripeCompactionPolicy policy, StripeInformationProvider si,
500       Collection<StoreFile> sfs, byte[] dropDeletesFrom, byte[] dropDeletesTo,
501       final List<byte[]> boundaries) throws Exception {
502     StripeCompactor sc = mock(StripeCompactor.class);
503     assertTrue(policy.needsCompactions(si, al()));
504     StripeCompactionPolicy.StripeCompactionRequest scr = policy.selectCompaction(si, al(), false);
505     verifyCollectionsEqual(sfs, scr.getRequest().getFiles());
506     scr.execute(sc, NoLimitCompactionThroughputController.INSTANCE);
507     verify(sc, times(1)).compact(eq(scr.getRequest()), argThat(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     }), dropDeletesFrom == null ? isNull(byte[].class) : aryEq(dropDeletesFrom),
519       dropDeletesTo == null ? isNull(byte[].class) : aryEq(dropDeletesTo),
520       any(NoLimitCompactionThroughputController.class));
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, NoLimitCompactionThroughputController.INSTANCE);
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       any(NoLimitCompactionThroughputController.class));
547   }
548 
549   /** Verify arbitrary flush. */
550   protected void verifyFlush(StripeCompactionPolicy policy, StripeInformationProvider si,
551       KeyValue[] input, KeyValue[][] expected, byte[][] boundaries) throws IOException {
552     StoreFileWritersCapture writers = new StoreFileWritersCapture();
553     StripeStoreFlusher.StripeFlushRequest req = policy.selectFlush(si, input.length);
554     StripeMultiFileWriter mw = req.createWriter();
555     mw.init(null, writers, new KeyValue.KVComparator());
556     for (KeyValue kv : input) {
557       mw.append(kv);
558     }
559     boolean hasMetadata = boundaries != null;
560     mw.commitWriters(0, false);
561     writers.verifyKvs(expected, true, hasMetadata);
562     if (hasMetadata) {
563       writers.verifyBoundaries(boundaries);
564     }
565   }
566 
567 
568   private byte[] dropDeletesMatcher(Boolean dropDeletes, byte[] value) {
569     return dropDeletes == null ? any(byte[].class)
570             : (dropDeletes.booleanValue() ? aryEq(value) : isNull(byte[].class));
571   }
572 
573   private void verifyCollectionsEqual(Collection<StoreFile> sfs, Collection<StoreFile> scr) {
574     // Dumb.
575     assertEquals(sfs.size(), scr.size());
576     assertTrue(scr.containsAll(sfs));
577   }
578 
579   private static List<StoreFile> getAllFiles(
580       StripeInformationProvider si, int fromStripe, int toStripe) {
581     ArrayList<StoreFile> expected = new ArrayList<StoreFile>();
582     for (int i = fromStripe; i <= toStripe; ++i) {
583       expected.addAll(si.getStripes().get(i));
584     }
585     return expected;
586   }
587 
588   /**
589    * @param l0Count Number of L0 files.
590    * @param boundaries Target boundaries.
591    * @return Mock stripes.
592    */
593   private static StripeInformationProvider createStripes(
594       int l0Count, byte[]... boundaries) throws Exception {
595     List<Long> l0Sizes = new ArrayList<Long>();
596     for (int i = 0; i < l0Count; ++i) {
597       l0Sizes.add(5L);
598     }
599     List<List<Long>> sizes = new ArrayList<List<Long>>();
600     for (int i = 0; i <= boundaries.length; ++i) {
601       sizes.add(Arrays.asList(Long.valueOf(5)));
602     }
603     return createStripes(Arrays.asList(boundaries), sizes, l0Sizes);
604   }
605 
606   /**
607    * @param l0Count Number of L0 files.
608    * @param l0Size Size of each file.
609    * @return Mock stripes.
610    */
611   private static StripeInformationProvider createStripesL0Only(
612       int l0Count, long l0Size) throws Exception {
613     List<Long> l0Sizes = new ArrayList<Long>();
614     for (int i = 0; i < l0Count; ++i) {
615       l0Sizes.add(l0Size);
616     }
617     return createStripes(null, new ArrayList<List<Long>>(), l0Sizes);
618   }
619 
620   /**
621    * @param l0Count Number of L0 files.
622    * @param l0Size Size of each file.
623    * @param sizes Sizes of the files; each sub-array representing a stripe.
624    * @return Mock stripes.
625    */
626   private static StripeInformationProvider createStripesWithSizes(
627       int l0Count, long l0Size, Long[]... sizes) throws Exception {
628     ArrayList<List<Long>> sizeList = new ArrayList<List<Long>>();
629     for (Long[] size : sizes) {
630       sizeList.add(Arrays.asList(size));
631     }
632     return createStripesWithSizes(l0Count, l0Size, sizeList);
633   }
634 
635   private static StripeInformationProvider createStripesWithSizes(
636       int l0Count, long l0Size, List<List<Long>> sizes) throws Exception {
637     List<byte[]> boundaries = createBoundaries(sizes.size());
638     List<Long> l0Sizes = new ArrayList<Long>();
639     for (int i = 0; i < l0Count; ++i) {
640       l0Sizes.add(l0Size);
641     }
642     return createStripes(boundaries, sizes, l0Sizes);
643   }
644 
645   private static List<byte[]> createBoundaries(int stripeCount) {
646     byte[][] keys = new byte[][] { KEY_A, KEY_B, KEY_C, KEY_D, KEY_E };
647     assert stripeCount <= keys.length + 1;
648     List<byte[]> boundaries = new ArrayList<byte[]>();
649     boundaries.addAll(Arrays.asList(keys).subList(0, stripeCount - 1));
650     return boundaries;
651   }
652 
653   private static StripeInformationProvider createStripes(List<byte[]> boundaries,
654       List<List<Long>> stripeSizes, List<Long> l0Sizes) throws Exception {
655     List<List<StoreFile>> stripeFiles = new ArrayList<List<StoreFile>>(stripeSizes.size());
656     for (List<Long> sizes : stripeSizes) {
657       List<StoreFile> sfs = new ArrayList<StoreFile>();
658       for (Long size : sizes) {
659         sfs.add(createFile(size));
660       }
661       stripeFiles.add(sfs);
662     }
663     List<StoreFile> l0Files = new ArrayList<StoreFile>();
664     for (Long size : l0Sizes) {
665       l0Files.add(createFile(size));
666     }
667     return createStripesWithFiles(boundaries, stripeFiles, l0Files);
668   }
669 
670   /**
671    * This method actually does all the work.
672    */
673   private static StripeInformationProvider createStripesWithFiles(List<byte[]> boundaries,
674       List<List<StoreFile>> stripeFiles, List<StoreFile> l0Files) throws Exception {
675     ArrayList<ImmutableList<StoreFile>> stripes = new ArrayList<ImmutableList<StoreFile>>();
676     ArrayList<byte[]> boundariesList = new ArrayList<byte[]>();
677     StripeInformationProvider si = mock(StripeInformationProvider.class);
678     if (!stripeFiles.isEmpty()) {
679       assert stripeFiles.size() == (boundaries.size() + 1);
680       boundariesList.add(OPEN_KEY);
681       for (int i = 0; i <= boundaries.size(); ++i) {
682         byte[] startKey = ((i == 0) ? OPEN_KEY : boundaries.get(i - 1));
683         byte[] endKey = ((i == boundaries.size()) ? OPEN_KEY : boundaries.get(i));
684         boundariesList.add(endKey);
685         for (StoreFile sf : stripeFiles.get(i)) {
686           setFileStripe(sf, startKey, endKey);
687         }
688         stripes.add(ImmutableList.copyOf(stripeFiles.get(i)));
689         when(si.getStartRow(eq(i))).thenReturn(startKey);
690         when(si.getEndRow(eq(i))).thenReturn(endKey);
691       }
692     }
693     ConcatenatedLists<StoreFile> sfs = new ConcatenatedLists<StoreFile>();
694     sfs.addAllSublists(stripes);
695     sfs.addSublist(l0Files);
696     when(si.getStorefiles()).thenReturn(sfs);
697     when(si.getStripes()).thenReturn(stripes);
698     when(si.getStripeBoundaries()).thenReturn(boundariesList);
699     when(si.getStripeCount()).thenReturn(stripes.size());
700     when(si.getLevel0Files()).thenReturn(l0Files);
701     return si;
702   }
703 
704   private static StoreFile createFile(long size) throws Exception {
705     StoreFile sf = mock(StoreFile.class);
706     when(sf.getPath()).thenReturn(new Path("moo"));
707     StoreFile.Reader r = mock(StoreFile.Reader.class);
708     when(r.getEntries()).thenReturn(size);
709     when(r.length()).thenReturn(size);
710     when(r.getBloomFilterType()).thenReturn(BloomType.NONE);
711     when(r.getHFileReader()).thenReturn(mock(HFile.Reader.class));
712     when(r.getStoreFileScanner(anyBoolean(), anyBoolean(), anyBoolean(), anyLong())).thenReturn(
713       mock(StoreFileScanner.class));
714     when(sf.getReader()).thenReturn(r);
715     when(sf.createReader()).thenReturn(r);
716     return sf;
717   }
718 
719   private static StoreFile createFile() throws Exception {
720     return createFile(0);
721   }
722 
723   private static void setFileStripe(StoreFile sf, byte[] startKey, byte[] endKey) {
724     when(sf.getMetadataValue(StripeStoreFileManager.STRIPE_START_KEY)).thenReturn(startKey);
725     when(sf.getMetadataValue(StripeStoreFileManager.STRIPE_END_KEY)).thenReturn(endKey);
726   }
727 
728   private static StripeCompactor createCompactor() throws Exception {
729     HColumnDescriptor col = new HColumnDescriptor(Bytes.toBytes("foo"));
730     StoreFileWritersCapture writers = new StoreFileWritersCapture();
731     Store store = mock(Store.class);
732     HRegionInfo info = mock(HRegionInfo.class);
733     when(info.getRegionNameAsString()).thenReturn("testRegion");
734     when(store.getFamily()).thenReturn(col);
735     when(store.getRegionInfo()).thenReturn(info);
736     when(
737       store.createWriterInTmp(anyLong(), any(Compression.Algorithm.class), anyBoolean(),
738         anyBoolean(), anyBoolean())).thenAnswer(writers);
739 
740     Configuration conf = HBaseConfiguration.create();
741     final Scanner scanner = new Scanner();
742     return new StripeCompactor(conf, store) {
743       @Override
744       protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
745           long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,
746           byte[] dropDeletesToRow) throws IOException {
747         return scanner;
748       }
749 
750       @Override
751       protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
752           ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
753         return scanner;
754       }
755     };
756   }
757 
758   private static class Scanner implements InternalScanner {
759     private final ArrayList<KeyValue> kvs;
760 
761     public Scanner(KeyValue... kvs) {
762       this.kvs = new ArrayList<KeyValue>(Arrays.asList(kvs));
763     }
764 
765     @Override
766     public boolean next(List<Cell> results) throws IOException {
767       if (kvs.isEmpty()) return false;
768       results.add(kvs.remove(0));
769       return !kvs.isEmpty();
770     }
771 
772     @Override
773     public boolean next(List<Cell> result, int limit) throws IOException {
774       return next(result);
775     }
776 
777     @Override
778     public void close() throws IOException {
779     }
780   }
781 }