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