View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.regionserver;
20  
21  import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY;
22  import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY_BYTES;
23  import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
24  import static org.junit.Assert.assertEquals;
25  import static org.junit.Assert.assertNotNull;
26  import static org.junit.Assert.assertTrue;
27  import static org.junit.Assert.fail;
28  import static org.mockito.Matchers.any;
29  import static org.mockito.Mockito.doAnswer;
30  import static org.mockito.Mockito.mock;
31  import static org.mockito.Mockito.spy;
32  import static org.mockito.Mockito.when;
33  
34  import java.io.IOException;
35  import java.util.ArrayList;
36  import java.util.Collection;
37  import java.util.Collections;
38  import java.util.List;
39  import java.util.concurrent.CountDownLatch;
40  import java.util.concurrent.TimeUnit;
41  
42  import org.apache.hadoop.conf.Configuration;
43  import org.apache.hadoop.fs.FSDataOutputStream;
44  import org.apache.hadoop.fs.FileStatus;
45  import org.apache.hadoop.fs.FileSystem;
46  import org.apache.hadoop.fs.Path;
47  import org.apache.hadoop.hbase.ChoreService;
48  import org.apache.hadoop.hbase.HBaseConfiguration;
49  import org.apache.hadoop.hbase.HBaseTestCase;
50  import org.apache.hadoop.hbase.HBaseTestCase.HRegionIncommon;
51  import org.apache.hadoop.hbase.HBaseTestingUtility;
52  import org.apache.hadoop.hbase.HColumnDescriptor;
53  import org.apache.hadoop.hbase.HConstants;
54  import org.apache.hadoop.hbase.HTableDescriptor;
55  import org.apache.hadoop.hbase.Waiter;
56  import org.apache.hadoop.hbase.client.Delete;
57  import org.apache.hadoop.hbase.client.Durability;
58  import org.apache.hadoop.hbase.client.Put;
59  import org.apache.hadoop.hbase.io.hfile.HFileScanner;
60  import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
61  import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
62  import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
63  import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
64  import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
65  import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
66  import org.apache.hadoop.hbase.security.User;
67  import org.apache.hadoop.hbase.testclassification.MediumTests;
68  import org.apache.hadoop.hbase.util.Bytes;
69  import org.apache.hadoop.hbase.util.Pair;
70  import org.apache.hadoop.hbase.util.Threads;
71  import org.apache.hadoop.hbase.wal.WAL;
72  import org.junit.After;
73  import org.junit.Assume;
74  import org.junit.Before;
75  import org.junit.Rule;
76  import org.junit.Test;
77  import org.junit.experimental.categories.Category;
78  import org.junit.rules.TestName;
79  import org.mockito.Mockito;
80  import org.mockito.invocation.InvocationOnMock;
81  import org.mockito.stubbing.Answer;
82  
83  
84  /**
85   * Test compaction framework and common functions
86   */
87  @Category(MediumTests.class)
88  public class TestCompaction {
89    @Rule public TestName name = new TestName();
90    private static final HBaseTestingUtility UTIL = HBaseTestingUtility.createLocalHTU();
91    protected Configuration conf = UTIL.getConfiguration();
92  
93    private HRegion r = null;
94    private HTableDescriptor htd = null;
95    private static final byte [] COLUMN_FAMILY = fam1;
96    private final byte [] STARTROW = Bytes.toBytes(START_KEY);
97    private static final byte [] COLUMN_FAMILY_TEXT = COLUMN_FAMILY;
98    private int compactionThreshold;
99    private byte[] secondRowBytes, thirdRowBytes;
100   private static final long MAX_FILES_TO_COMPACT = 10;
101   private final byte[] FAMILY = Bytes.toBytes("cf");
102 
103   /** constructor */
104   public TestCompaction() {
105     super();
106 
107     // Set cache flush size to 1MB
108     conf.setInt(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 1024 * 1024);
109     conf.setInt(HConstants.HREGION_MEMSTORE_BLOCK_MULTIPLIER, 100);
110     conf.set(CompactionThroughputControllerFactory.HBASE_THROUGHPUT_CONTROLLER_KEY,
111       NoLimitThroughputController.class.getName());
112     compactionThreshold = conf.getInt("hbase.hstore.compactionThreshold", 3);
113 
114     secondRowBytes = START_KEY_BYTES.clone();
115     // Increment the least significant character so we get to next row.
116     secondRowBytes[START_KEY_BYTES.length - 1]++;
117     thirdRowBytes = START_KEY_BYTES.clone();
118     thirdRowBytes[START_KEY_BYTES.length - 1] += 2;
119   }
120 
121   @Before
122   public void setUp() throws Exception {
123     this.htd = UTIL.createTableDescriptor(name.getMethodName());
124     if (name.getMethodName().equals("testCompactionSeqId")) {
125       UTIL.getConfiguration().set("hbase.hstore.compaction.kv.max", "10");
126       UTIL.getConfiguration().set(
127           DefaultStoreEngine.DEFAULT_COMPACTOR_CLASS_KEY,
128           DummyCompactor.class.getName());
129       HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
130       hcd.setMaxVersions(65536);
131       this.htd.addFamily(hcd);
132     }
133     this.r = UTIL.createLocalHRegion(htd, null, null);
134   }
135 
136   @After
137   public void tearDown() throws Exception {
138     WAL wal = r.getWAL();
139     this.r.close();
140     wal.close();
141   }
142 
143   /**
144    * Verify that you can stop a long-running compaction
145    * (used during RS shutdown)
146    * @throws Exception
147    */
148   @Test
149   public void testInterruptCompaction() throws Exception {
150     assertEquals(0, count());
151 
152     // lower the polling interval for this test
153     int origWI = HStore.closeCheckInterval;
154     HStore.closeCheckInterval = 10*1000; // 10 KB
155 
156     try {
157       // Create a couple store files w/ 15KB (over 10KB interval)
158       int jmax = (int) Math.ceil(15.0/compactionThreshold);
159       byte [] pad = new byte[1000]; // 1 KB chunk
160       for (int i = 0; i < compactionThreshold; i++) {
161         HRegionIncommon loader = new HRegionIncommon(r);
162         Put p = new Put(Bytes.add(STARTROW, Bytes.toBytes(i)));
163         p.setDurability(Durability.SKIP_WAL);
164         for (int j = 0; j < jmax; j++) {
165           p.add(COLUMN_FAMILY, Bytes.toBytes(j), pad);
166         }
167         HBaseTestCase.addContent(loader, Bytes.toString(COLUMN_FAMILY));
168         loader.put(p);
169         loader.flushcache();
170       }
171 
172       HRegion spyR = spy(r);
173       doAnswer(new Answer() {
174         @Override
175         public Object answer(InvocationOnMock invocation) throws Throwable {
176           r.writestate.writesEnabled = false;
177           return invocation.callRealMethod();
178         }
179       }).when(spyR).doRegionCompactionPrep();
180 
181       // force a minor compaction, but not before requesting a stop
182       spyR.compactStores();
183 
184       // ensure that the compaction stopped, all old files are intact,
185       Store s = r.stores.get(COLUMN_FAMILY);
186       assertEquals(compactionThreshold, s.getStorefilesCount());
187       assertTrue(s.getStorefilesSize() > 15*1000);
188       // only one empty dir exists in temp dir
189       FileStatus[] ls = r.getFilesystem().listStatus(r.getRegionFileSystem().getTempDir());
190       assertEquals(1, ls.length);
191       Path storeTempDir = new Path(r.getRegionFileSystem().getTempDir(), Bytes.toString(COLUMN_FAMILY));
192       assertTrue(r.getFilesystem().exists(storeTempDir));
193       ls = r.getFilesystem().listStatus(storeTempDir);
194       assertEquals(0, ls.length);
195     } finally {
196       // don't mess up future tests
197       r.writestate.writesEnabled = true;
198       HStore.closeCheckInterval = origWI;
199 
200       // Delete all Store information once done using
201       for (int i = 0; i < compactionThreshold; i++) {
202         Delete delete = new Delete(Bytes.add(STARTROW, Bytes.toBytes(i)));
203         byte [][] famAndQf = {COLUMN_FAMILY, null};
204         delete.deleteFamily(famAndQf[0]);
205         r.delete(delete);
206       }
207       r.flush(true);
208 
209       // Multiple versions allowed for an entry, so the delete isn't enough
210       // Lower TTL and expire to ensure that all our entries have been wiped
211       final int ttl = 1000;
212       for (Store hstore: this.r.stores.values()) {
213         HStore store = (HStore)hstore;
214         ScanInfo old = store.getScanInfo();
215         ScanInfo si = new ScanInfo(old.getConfiguration(), old.getFamily(),
216             old.getMinVersions(), old.getMaxVersions(), ttl,
217             old.getKeepDeletedCells(), 0, old.getComparator());
218         store.setScanInfo(si);
219       }
220       Thread.sleep(ttl);
221 
222       r.compact(true);
223       assertEquals(0, count());
224     }
225   }
226 
227   private int count() throws IOException {
228     int count = 0;
229     for (StoreFile f: this.r.stores.
230         get(COLUMN_FAMILY_TEXT).getStorefiles()) {
231       HFileScanner scanner = f.getReader().getScanner(false, false);
232       if (!scanner.seekTo()) {
233         continue;
234       }
235       do {
236         count++;
237       } while(scanner.next());
238     }
239     return count;
240   }
241 
242   private void createStoreFile(final HRegion region) throws IOException {
243     createStoreFile(region, Bytes.toString(COLUMN_FAMILY));
244   }
245 
246   private void createStoreFile(final HRegion region, String family) throws IOException {
247     HRegionIncommon loader = new HRegionIncommon(region);
248     HBaseTestCase.addContent(loader, family);
249     loader.flushcache();
250   }
251 
252   @Test
253   public void testCompactionWithCorruptResult() throws Exception {
254     int nfiles = 10;
255     for (int i = 0; i < nfiles; i++) {
256       createStoreFile(r);
257     }
258     HStore store = (HStore) r.getStore(COLUMN_FAMILY);
259 
260     Collection<StoreFile> storeFiles = store.getStorefiles();
261     DefaultCompactor tool = (DefaultCompactor)store.storeEngine.getCompactor();
262     tool.compactForTesting(storeFiles, false);
263 
264     // Now lets corrupt the compacted file.
265     FileSystem fs = store.getFileSystem();
266     // default compaction policy created one and only one new compacted file
267     Path dstPath = store.getRegionFileSystem().createTempName();
268     FSDataOutputStream stream = fs.create(dstPath, null, true, 512, (short)3, (long)1024, null);
269     stream.writeChars("CORRUPT FILE!!!!");
270     stream.close();
271     Path origPath = store.getRegionFileSystem().commitStoreFile(
272       Bytes.toString(COLUMN_FAMILY), dstPath);
273 
274     try {
275       ((HStore)store).moveFileIntoPlace(origPath);
276     } catch (Exception e) {
277       // The complete compaction should fail and the corrupt file should remain
278       // in the 'tmp' directory;
279       assert (fs.exists(origPath));
280       assert (!fs.exists(dstPath));
281       System.out.println("testCompactionWithCorruptResult Passed");
282       return;
283     }
284     fail("testCompactionWithCorruptResult failed since no exception was" +
285         "thrown while completing a corrupt file");
286   }
287 
288   /**
289    * Create a custom compaction request and be sure that we can track it through the queue, knowing
290    * when the compaction is completed.
291    */
292   @Test
293   public void testTrackingCompactionRequest() throws Exception {
294     // setup a compact/split thread on a mock server
295     HRegionServer mockServer = Mockito.mock(HRegionServer.class);
296     Mockito.when(mockServer.getConfiguration()).thenReturn(r.getBaseConf());
297     CompactSplitThread thread = new CompactSplitThread(mockServer);
298     Mockito.when(mockServer.getCompactSplitThread()).thenReturn(thread);
299 
300     // setup a region/store with some files
301     Store store = r.getStore(COLUMN_FAMILY);
302     createStoreFile(r);
303     for (int i = 0; i < MAX_FILES_TO_COMPACT + 1; i++) {
304       createStoreFile(r);
305     }
306 
307     CountDownLatch latch = new CountDownLatch(1);
308     TrackableCompactionRequest request = new TrackableCompactionRequest(latch);
309     thread.requestCompaction(r, store, "test custom comapction", Store.PRIORITY_USER, request,null);
310     // wait for the latch to complete.
311     latch.await();
312 
313     thread.interruptIfNecessary();
314   }
315 
316   @Test
317   public void testCompactionFailure() throws Exception {
318     // setup a compact/split thread on a mock server
319     HRegionServer mockServer = Mockito.mock(HRegionServer.class);
320     Mockito.when(mockServer.getConfiguration()).thenReturn(r.getBaseConf());
321     CompactSplitThread thread = new CompactSplitThread(mockServer);
322     Mockito.when(mockServer.getCompactSplitThread()).thenReturn(thread);
323 
324     // setup a region/store with some files
325     Store store = r.getStore(COLUMN_FAMILY);
326     createStoreFile(r);
327     for (int i = 0; i < HStore.DEFAULT_BLOCKING_STOREFILE_COUNT - 1; i++) {
328       createStoreFile(r);
329     }
330 
331     HRegion mockRegion = Mockito.spy(r);
332     Mockito.when(mockRegion.checkSplit()).thenThrow(new IndexOutOfBoundsException());
333 
334     MetricsRegionWrapper metricsWrapper = new MetricsRegionWrapperImpl(r);
335 
336     long preCompletedCount = metricsWrapper.getNumCompactionsCompleted();
337     long preFailedCount = metricsWrapper.getNumCompactionsFailed();
338 
339     CountDownLatch latch = new CountDownLatch(1);
340     TrackableCompactionRequest request = new TrackableCompactionRequest(latch);
341     thread.requestCompaction(mockRegion, store, "test custom comapction", Store.PRIORITY_USER,
342         request, null);
343     // wait for the latch to complete.
344     latch.await(120, TimeUnit.SECONDS);
345 
346     // compaction should have completed and been marked as failed due to error in split request
347     long postCompletedCount = metricsWrapper.getNumCompactionsCompleted();
348     long postFailedCount = metricsWrapper.getNumCompactionsFailed();
349 
350     assertTrue("Completed count should have increased (pre=" + preCompletedCount +
351         ", post="+postCompletedCount+")",
352         postCompletedCount > preCompletedCount);
353     assertTrue("Failed count should have increased (pre=" + preFailedCount +
354         ", post=" + postFailedCount + ")",
355         postFailedCount > preFailedCount);
356   }
357 
358   /**
359    * Test no new Compaction requests are generated after calling stop compactions
360    */
361   @Test
362   public void testStopStartCompaction() throws IOException {
363     // setup a compact/split thread on a mock server
364     HRegionServer mockServer = Mockito.mock(HRegionServer.class);
365     Mockito.when(mockServer.getConfiguration()).thenReturn(r.getBaseConf());
366     final CompactSplitThread thread = new CompactSplitThread(mockServer);
367     Mockito.when(mockServer.getCompactSplitThread()).thenReturn(thread);
368     // setup a region/store with some files
369     Store store = r.getStore(COLUMN_FAMILY);
370     createStoreFile(r);
371     for (int i = 0; i < HStore.DEFAULT_BLOCKING_STOREFILE_COUNT - 1; i++) {
372       createStoreFile(r);
373     }
374     thread.switchCompaction(false);
375     thread.requestCompaction(r, store, "test", Store.PRIORITY_USER, new CompactionRequest(), null);
376     assertEquals(false, thread.isCompactionsEnabled());
377     assertEquals(0, thread.getLongCompactions().getActiveCount() + thread.getShortCompactions()
378       .getActiveCount());
379     thread.switchCompaction(true);
380     assertEquals(true, thread.isCompactionsEnabled());
381     // Make sure no compactions have run.
382     assertEquals(0, thread.getLongCompactions().getCompletedTaskCount() +
383         thread.getShortCompactions().getCompletedTaskCount());
384     // Request a compaction and make sure it is submitted successfully.
385     assertNotNull(thread.requestCompaction(r, store, "test", Store.PRIORITY_USER,
386         new CompactionRequest(), null));
387     // Wait until the compaction finishes.
388     Waiter.waitFor(UTIL.getConfiguration(), 5000, new Waiter.Predicate<Exception>() {
389       @Override
390       public boolean evaluate() throws Exception {
391         return thread.getLongCompactions().getCompletedTaskCount() +
392             thread.getShortCompactions().getCompletedTaskCount() == 1;
393       }
394     });
395     // Make sure there are no compactions running.
396     assertEquals(0, thread.getLongCompactions().getActiveCount()
397         + thread.getShortCompactions().getActiveCount());
398   }
399 
400   @Test
401   public void testInterruptingRunningCompactions() throws Exception {
402     // setup a compact/split thread on a mock server
403     conf.set(CompactionThroughputControllerFactory.HBASE_THROUGHPUT_CONTROLLER_KEY,
404       WaitThroughPutController.class.getName());
405     HRegionServer mockServer = Mockito.mock(HRegionServer.class);
406     Mockito.when(mockServer.getConfiguration()).thenReturn(r.getBaseConf());
407     CompactSplitThread thread = new CompactSplitThread(mockServer);
408 
409     Mockito.when(mockServer.getCompactSplitThread()).thenReturn(thread);
410 
411     // setup a region/store with some files
412     Store store = r.getStore(COLUMN_FAMILY);
413     int jmax = (int) Math.ceil(15.0 / compactionThreshold);
414     byte[] pad = new byte[1000]; // 1 KB chunk
415     for (int i = 0; i < compactionThreshold; i++) {
416       HRegionIncommon loader = new HRegionIncommon(r);
417       Put p = new Put(Bytes.add(STARTROW, Bytes.toBytes(i)));
418       p.setDurability(Durability.SKIP_WAL);
419       for (int j = 0; j < jmax; j++) {
420         p.addColumn(COLUMN_FAMILY, Bytes.toBytes(j), pad);
421       }
422       HBaseTestCase.addContent(loader, Bytes.toString(COLUMN_FAMILY));
423       loader.put(p);
424       r.flush(true);
425     }
426     Store s = r.getStore(COLUMN_FAMILY);
427     int initialFiles = s.getStorefilesCount();
428 
429     thread.requestCompaction(r, store, "test custom comapction", Store.PRIORITY_USER,
430       new CompactionRequest(), null);
431 
432     Thread.sleep(3000);
433     thread.switchCompaction(false);
434     assertEquals(initialFiles, s.getStorefilesCount());
435     //don't mess up future tests
436     thread.switchCompaction(true);
437   }
438 
439   /**
440    * HBASE-7947: Regression test to ensure adding to the correct list in the
441    * {@link CompactSplitThread}
442    * @throws Exception on failure
443    */
444   @Test
445   public void testMultipleCustomCompactionRequests() throws Exception {
446     // setup a compact/split thread on a mock server
447     HRegionServer mockServer = Mockito.mock(HRegionServer.class);
448     Mockito.when(mockServer.getConfiguration()).thenReturn(r.getBaseConf());
449     CompactSplitThread thread = new CompactSplitThread(mockServer);
450     Mockito.when(mockServer.getCompactSplitThread()).thenReturn(thread);
451 
452     // setup a region/store with some files
453     int numStores = r.getStores().size();
454     List<Pair<CompactionRequest, Store>> requests =
455         new ArrayList<Pair<CompactionRequest, Store>>(numStores);
456     CountDownLatch latch = new CountDownLatch(numStores);
457     // create some store files and setup requests for each store on which we want to do a
458     // compaction
459     for (Store store : r.getStores()) {
460       createStoreFile(r, store.getColumnFamilyName());
461       createStoreFile(r, store.getColumnFamilyName());
462       createStoreFile(r, store.getColumnFamilyName());
463       requests
464           .add(new Pair<CompactionRequest, Store>(new TrackableCompactionRequest(latch), store));
465     }
466 
467     thread.requestCompaction(r, "test mulitple custom comapctions", Store.PRIORITY_USER,
468       Collections.unmodifiableList(requests), null);
469 
470     // wait for the latch to complete.
471     latch.await();
472 
473     thread.interruptIfNecessary();
474   }
475 
476   private class StoreMockMaker extends StatefulStoreMockMaker {
477     public ArrayList<StoreFile> compacting = new ArrayList<StoreFile>();
478     public ArrayList<StoreFile> notCompacting = new ArrayList<StoreFile>();
479     private ArrayList<Integer> results;
480 
481     public StoreMockMaker(ArrayList<Integer> results) {
482       this.results = results;
483     }
484 
485     public class TestCompactionContext extends CompactionContext {
486       private List<StoreFile> selectedFiles;
487       public TestCompactionContext(List<StoreFile> selectedFiles) {
488         super();
489         this.selectedFiles = selectedFiles;
490       }
491 
492       @Override
493       public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
494         return new ArrayList<StoreFile>();
495       }
496 
497       @Override
498       public boolean select(List<StoreFile> filesCompacting, boolean isUserCompaction,
499           boolean mayUseOffPeak, boolean forceMajor) throws IOException {
500         this.request = new CompactionRequest(selectedFiles);
501         this.request.setPriority(getPriority());
502         return true;
503       }
504 
505       @Override
506       public List<Path> compact(ThroughputController throughputController, User user)
507           throws IOException {
508         finishCompaction(this.selectedFiles);
509         return new ArrayList<Path>();
510       }
511     }
512 
513     @Override
514     public synchronized CompactionContext selectCompaction() {
515       CompactionContext ctx = new TestCompactionContext(new ArrayList<StoreFile>(notCompacting));
516       compacting.addAll(notCompacting);
517       notCompacting.clear();
518       try {
519         ctx.select(null, false, false, false);
520       } catch (IOException ex) {
521         fail("Shouldn't happen");
522       }
523       return ctx;
524     }
525 
526     @Override
527     public synchronized void cancelCompaction(Object object) {
528       TestCompactionContext ctx = (TestCompactionContext)object;
529       compacting.removeAll(ctx.selectedFiles);
530       notCompacting.addAll(ctx.selectedFiles);
531     }
532 
533     public synchronized void finishCompaction(List<StoreFile> sfs) {
534       if (sfs.isEmpty()) return;
535       synchronized (results) {
536         results.add(sfs.size());
537       }
538       compacting.removeAll(sfs);
539     }
540 
541     @Override
542     public int getPriority() {
543       return 7 - compacting.size() - notCompacting.size();
544     }
545   }
546 
547   public class BlockingStoreMockMaker extends StatefulStoreMockMaker {
548     BlockingCompactionContext blocked = null;
549 
550     public class BlockingCompactionContext extends CompactionContext {
551       public volatile boolean isInCompact = false;
552 
553       public void unblock() {
554         synchronized (this) { this.notifyAll(); }
555       }
556 
557       @Override
558       public List<Path> compact(ThroughputController throughputController, User user)
559           throws IOException {
560         try {
561           isInCompact = true;
562           synchronized (this) {
563             // FIXME: This wait may spuriously wake up, so this test is likely to be flaky
564             this.wait();
565           }
566         } catch (InterruptedException e) {
567           Assume.assumeNoException(e);
568         }
569         return new ArrayList<Path>();
570       }
571 
572       @Override
573       public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
574         return new ArrayList<StoreFile>();
575       }
576 
577       @Override
578       public boolean select(List<StoreFile> f, boolean i, boolean m, boolean e)
579           throws IOException {
580         this.request = new CompactionRequest(new ArrayList<StoreFile>());
581         return true;
582       }
583     }
584 
585     @Override
586     public CompactionContext selectCompaction() {
587       this.blocked = new BlockingCompactionContext();
588       try {
589         this.blocked.select(null, false, false, false);
590       } catch (IOException ex) {
591         fail("Shouldn't happen");
592       }
593       return this.blocked;
594     }
595 
596     @Override
597     public void cancelCompaction(Object object) {}
598 
599     @Override
600     public int getPriority() {
601       return Integer.MIN_VALUE; // some invalid value, see createStoreMock
602     }
603 
604     public BlockingCompactionContext waitForBlocking() {
605       while (this.blocked == null || !this.blocked.isInCompact) {
606         Threads.sleepWithoutInterrupt(50);
607       }
608       BlockingCompactionContext ctx = this.blocked;
609       this.blocked = null;
610       return ctx;
611     }
612 
613     @Override
614     public Store createStoreMock(String name) throws Exception {
615       return createStoreMock(Integer.MIN_VALUE, name);
616     }
617 
618     public Store createStoreMock(int priority, String name) throws Exception {
619       // Override the mock to always return the specified priority.
620       Store s = super.createStoreMock(name);
621       when(s.getCompactPriority()).thenReturn(priority);
622       return s;
623     }
624   }
625 
626   /** Test compaction priority management and multiple compactions per store (HBASE-8665). */
627   @Test
628   public void testCompactionQueuePriorities() throws Exception {
629     // Setup a compact/split thread on a mock server.
630     final Configuration conf = HBaseConfiguration.create();
631     HRegionServer mockServer = mock(HRegionServer.class);
632     when(mockServer.isStopped()).thenReturn(false);
633     when(mockServer.getConfiguration()).thenReturn(conf);
634     when(mockServer.getChoreService()).thenReturn(new ChoreService("test"));
635     CompactSplitThread cst = new CompactSplitThread(mockServer);
636     when(mockServer.getCompactSplitThread()).thenReturn(cst);
637     //prevent large compaction thread pool stealing job from small compaction queue.
638     cst.shutdownLongCompactions();
639     // Set up the region mock that redirects compactions.
640     HRegion r = mock(HRegion.class);
641     when(
642       r.compact(any(CompactionContext.class), any(Store.class),
643         any(ThroughputController.class), any(User.class))).then(new Answer<Boolean>() {
644       @Override
645       public Boolean answer(InvocationOnMock invocation) throws Throwable {
646         invocation.getArgumentAt(0, CompactionContext.class).compact(
647           invocation.getArgumentAt(2, ThroughputController.class), null);
648         return true;
649       }
650     });
651 
652     // Set up store mocks for 2 "real" stores and the one we use for blocking CST.
653     ArrayList<Integer> results = new ArrayList<Integer>();
654     StoreMockMaker sm = new StoreMockMaker(results), sm2 = new StoreMockMaker(results);
655     Store store = sm.createStoreMock("store1"), store2 = sm2.createStoreMock("store2");
656     BlockingStoreMockMaker blocker = new BlockingStoreMockMaker();
657 
658     // First, block the compaction thread so that we could muck with queue.
659     cst.requestSystemCompaction(r, blocker.createStoreMock(1, "b-pri1"), "b-pri1");
660     BlockingStoreMockMaker.BlockingCompactionContext currentBlock = blocker.waitForBlocking();
661 
662     // Add 4 files to store1, 3 to store2, and queue compactions; pri 3 and 4 respectively.
663     for (int i = 0; i < 4; ++i) {
664       sm.notCompacting.add(createFile());
665     }
666     cst.requestSystemCompaction(r, store, "s1-pri3");
667     for (int i = 0; i < 3; ++i) {
668       sm2.notCompacting.add(createFile());
669     }
670     cst.requestSystemCompaction(r, store2, "s2-pri4");
671     // Now add 2 more files to store1 and queue compaction - pri 1.
672     for (int i = 0; i < 2; ++i) {
673       sm.notCompacting.add(createFile());
674     }
675     cst.requestSystemCompaction(r, store, "s1-pri1");
676     // Finally add blocking compaction with priority 2.
677     cst.requestSystemCompaction(r, blocker.createStoreMock(2, "b-pri2"), "b-pri2");
678 
679     // Unblock the blocking compaction; we should run pri1 and become block again in pri2.
680     currentBlock.unblock();
681     currentBlock = blocker.waitForBlocking();
682     // Pri1 should have "compacted" all 6 files.
683     assertEquals(1, results.size());
684     assertEquals(6, results.get(0).intValue());
685     // Add 2 files to store 1 (it has 2 files now).
686     for (int i = 0; i < 2; ++i) {
687       sm.notCompacting.add(createFile());
688     }
689     // Now we have pri4 for store 2 in queue, and pri3 for store1; store1's current priority
690     // is 5, however, so it must not preempt store 2. Add blocking compaction at the end.
691     cst.requestSystemCompaction(r, blocker.createStoreMock(7, "b-pri7"), "b-pri7");
692     currentBlock.unblock();
693     currentBlock = blocker.waitForBlocking();
694     assertEquals(3, results.size());
695     assertEquals(3, results.get(1).intValue()); // 3 files should go before 2 files.
696     assertEquals(2, results.get(2).intValue());
697 
698     currentBlock.unblock();
699     cst.interruptIfNecessary();
700   }
701 
702   /**
703    * Firstly write 10 cells (with different time stamp) to a qualifier and flush
704    * to hfile1, then write 10 cells (with different time stamp) to the same
705    * qualifier and flush to hfile2. The latest cell (cell-A) in hfile1 and the
706    * oldest cell (cell-B) in hfile2 are with the same time stamp but different
707    * sequence id, and will get scanned successively during compaction.
708    * <p/>
709    * We set compaction.kv.max to 10 so compaction will scan 10 versions each
710    * round, meanwhile we set keepSeqIdPeriod=0 in {@link DummyCompactor} so all
711    * 10 versions of hfile2 will be written out with seqId cleaned (set to 0)
712    * including cell-B, then when scanner goes to cell-A it will cause a scan
713    * out-of-order assertion error before HBASE-16931
714    *
715    * @throws Exception
716    *           if error occurs during the test
717    */
718   @Test
719   public void testCompactionSeqId() throws Exception {
720     final byte[] ROW = Bytes.toBytes("row");
721     final byte[] QUALIFIER = Bytes.toBytes("qualifier");
722 
723     long timestamp = 10000;
724 
725     // row1/cf:a/10009/Put/vlen=2/seqid=11 V: v9
726     // row1/cf:a/10008/Put/vlen=2/seqid=10 V: v8
727     // row1/cf:a/10007/Put/vlen=2/seqid=9 V: v7
728     // row1/cf:a/10006/Put/vlen=2/seqid=8 V: v6
729     // row1/cf:a/10005/Put/vlen=2/seqid=7 V: v5
730     // row1/cf:a/10004/Put/vlen=2/seqid=6 V: v4
731     // row1/cf:a/10003/Put/vlen=2/seqid=5 V: v3
732     // row1/cf:a/10002/Put/vlen=2/seqid=4 V: v2
733     // row1/cf:a/10001/Put/vlen=2/seqid=3 V: v1
734     // row1/cf:a/10000/Put/vlen=2/seqid=2 V: v0
735     for (int i = 0; i < 10; i++) {
736       Put put = new Put(ROW);
737       put.addColumn(FAMILY, QUALIFIER, timestamp + i, Bytes.toBytes("v" + i));
738       r.put(put);
739     }
740     r.flush(true);
741 
742     // row1/cf:a/10018/Put/vlen=3/seqid=16 V: v18
743     // row1/cf:a/10017/Put/vlen=3/seqid=17 V: v17
744     // row1/cf:a/10016/Put/vlen=3/seqid=18 V: v16
745     // row1/cf:a/10015/Put/vlen=3/seqid=19 V: v15
746     // row1/cf:a/10014/Put/vlen=3/seqid=20 V: v14
747     // row1/cf:a/10013/Put/vlen=3/seqid=21 V: v13
748     // row1/cf:a/10012/Put/vlen=3/seqid=22 V: v12
749     // row1/cf:a/10011/Put/vlen=3/seqid=23 V: v11
750     // row1/cf:a/10010/Put/vlen=3/seqid=24 V: v10
751     // row1/cf:a/10009/Put/vlen=2/seqid=25 V: v9
752     for (int i = 18; i > 8; i--) {
753       Put put = new Put(ROW);
754       put.addColumn(FAMILY, QUALIFIER, timestamp + i, Bytes.toBytes("v" + i));
755       r.put(put);
756     }
757     r.flush(true);
758     r.compact(true);
759   }
760 
761   public static class DummyCompactor extends DefaultCompactor {
762     public DummyCompactor(Configuration conf, Store store) {
763       super(conf, store);
764       this.keepSeqIdPeriod = 0;
765     }
766   }
767 
768   private static StoreFile createFile() throws Exception {
769     StoreFile sf = mock(StoreFile.class);
770     when(sf.getPath()).thenReturn(new Path("file"));
771     StoreFile.Reader r = mock(StoreFile.Reader.class);
772     when(r.length()).thenReturn(10L);
773     when(sf.getReader()).thenReturn(r);
774     return sf;
775   }
776 
777   /**
778    * Simple {@link CompactionRequest} on which you can wait until the requested compaction finishes.
779    */
780   public static class TrackableCompactionRequest extends CompactionRequest {
781     private CountDownLatch done;
782 
783     /**
784      * Constructor for a custom compaction. Uses the setXXX methods to update the state of the
785      * compaction before being used.
786      */
787     public TrackableCompactionRequest(CountDownLatch finished) {
788       super();
789       this.done = finished;
790     }
791 
792     @Override
793     public void afterExecute() {
794       super.afterExecute();
795       this.done.countDown();
796     }
797   }
798 
799   /**
800    * Simple {@link CompactionLifeCycleTracker} on which you can wait until the requested compaction
801    * finishes.
802    */
803   public static class WaitThroughPutController extends NoLimitThroughputController{
804 
805     public WaitThroughPutController() {
806     }
807 
808     @Override
809     public long control(String compactionName, long size) throws InterruptedException {
810       Thread.sleep(6000000);
811       return 6000000;
812     }
813   }
814 }