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.assertTrue;
26  import static org.junit.Assert.fail;
27  import static org.mockito.Matchers.any;
28  import static org.mockito.Mockito.doAnswer;
29  import static org.mockito.Mockito.mock;
30  import static org.mockito.Mockito.spy;
31  import static org.mockito.Mockito.when;
32  
33  import java.io.IOException;
34  import java.util.ArrayList;
35  import java.util.Collection;
36  import java.util.Collections;
37  import java.util.List;
38  import java.util.concurrent.CountDownLatch;
39  
40  import org.apache.commons.logging.Log;
41  import org.apache.commons.logging.LogFactory;
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.HConstants;
53  import org.apache.hadoop.hbase.HTableDescriptor;
54  import org.apache.hadoop.hbase.security.User;
55  import org.apache.hadoop.hbase.testclassification.MediumTests;
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.compactions.NoLimitCompactionThroughputController;
64  import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController;
65  import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputControllerFactory;
66  import org.apache.hadoop.hbase.wal.WAL;
67  import org.apache.hadoop.hbase.util.Bytes;
68  import org.apache.hadoop.hbase.util.Pair;
69  import org.apache.hadoop.hbase.util.Threads;
70  import org.junit.After;
71  import org.junit.Assume;
72  import org.junit.Before;
73  import org.junit.Rule;
74  import org.junit.Test;
75  import org.junit.experimental.categories.Category;
76  import org.junit.rules.TestName;
77  import org.mockito.Mockito;
78  import org.mockito.invocation.InvocationOnMock;
79  import org.mockito.stubbing.Answer;
80  
81  
82  /**
83   * Test compaction framework and common functions
84   */
85  @Category(MediumTests.class)
86  public class TestCompaction {
87    @Rule public TestName name = new TestName();
88    static final Log LOG = LogFactory.getLog(TestCompaction.class.getName());
89    private static final HBaseTestingUtility UTIL = HBaseTestingUtility.createLocalHTU();
90    protected Configuration conf = UTIL.getConfiguration();
91  
92    private HRegion r = null;
93    private HTableDescriptor htd = null;
94    private static final byte [] COLUMN_FAMILY = fam1;
95    private final byte [] STARTROW = Bytes.toBytes(START_KEY);
96    private static final byte [] COLUMN_FAMILY_TEXT = COLUMN_FAMILY;
97    private int compactionThreshold;
98    private byte[] secondRowBytes, thirdRowBytes;
99    private static final long MAX_FILES_TO_COMPACT = 10;
100 
101   /** constructor */
102   public TestCompaction() {
103     super();
104 
105     // Set cache flush size to 1MB
106     conf.setInt(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 1024 * 1024);
107     conf.setInt(HConstants.HREGION_MEMSTORE_BLOCK_MULTIPLIER, 100);
108     conf.set(CompactionThroughputControllerFactory.HBASE_THROUGHPUT_CONTROLLER_KEY,
109       NoLimitCompactionThroughputController.class.getName());
110     compactionThreshold = conf.getInt("hbase.hstore.compactionThreshold", 3);
111 
112     secondRowBytes = START_KEY_BYTES.clone();
113     // Increment the least significant character so we get to next row.
114     secondRowBytes[START_KEY_BYTES.length - 1]++;
115     thirdRowBytes = START_KEY_BYTES.clone();
116     thirdRowBytes[START_KEY_BYTES.length - 1] += 2;
117   }
118 
119   @Before
120   public void setUp() throws Exception {
121     this.htd = UTIL.createTableDescriptor(name.getMethodName());
122     this.r = UTIL.createLocalHRegion(htd, null, null);
123   }
124 
125   @After
126   public void tearDown() throws Exception {
127     WAL wal = r.getWAL();
128     this.r.close();
129     wal.close();
130   }
131 
132   /**
133    * Verify that you can stop a long-running compaction
134    * (used during RS shutdown)
135    * @throws Exception
136    */
137   @Test
138   public void testInterruptCompaction() throws Exception {
139     assertEquals(0, count());
140 
141     // lower the polling interval for this test
142     int origWI = HStore.closeCheckInterval;
143     HStore.closeCheckInterval = 10*1000; // 10 KB
144 
145     try {
146       // Create a couple store files w/ 15KB (over 10KB interval)
147       int jmax = (int) Math.ceil(15.0/compactionThreshold);
148       byte [] pad = new byte[1000]; // 1 KB chunk
149       for (int i = 0; i < compactionThreshold; i++) {
150         HRegionIncommon loader = new HRegionIncommon(r);
151         Put p = new Put(Bytes.add(STARTROW, Bytes.toBytes(i)));
152         p.setDurability(Durability.SKIP_WAL);
153         for (int j = 0; j < jmax; j++) {
154           p.add(COLUMN_FAMILY, Bytes.toBytes(j), pad);
155         }
156         HBaseTestCase.addContent(loader, Bytes.toString(COLUMN_FAMILY));
157         loader.put(p);
158         loader.flushcache();
159       }
160 
161       HRegion spyR = spy(r);
162       doAnswer(new Answer() {
163         @Override
164         public Object answer(InvocationOnMock invocation) throws Throwable {
165           r.writestate.writesEnabled = false;
166           return invocation.callRealMethod();
167         }
168       }).when(spyR).doRegionCompactionPrep();
169 
170       // force a minor compaction, but not before requesting a stop
171       spyR.compactStores();
172 
173       // ensure that the compaction stopped, all old files are intact,
174       Store s = r.stores.get(COLUMN_FAMILY);
175       assertEquals(compactionThreshold, s.getStorefilesCount());
176       assertTrue(s.getStorefilesSize() > 15*1000);
177       // and no new store files persisted past compactStores()
178       FileStatus[] ls = r.getFilesystem().listStatus(r.getRegionFileSystem().getTempDir());
179       assertEquals(0, ls.length);
180 
181     } finally {
182       // don't mess up future tests
183       r.writestate.writesEnabled = true;
184       HStore.closeCheckInterval = origWI;
185 
186       // Delete all Store information once done using
187       for (int i = 0; i < compactionThreshold; i++) {
188         Delete delete = new Delete(Bytes.add(STARTROW, Bytes.toBytes(i)));
189         byte [][] famAndQf = {COLUMN_FAMILY, null};
190         delete.deleteFamily(famAndQf[0]);
191         r.delete(delete);
192       }
193       r.flush(true);
194 
195       // Multiple versions allowed for an entry, so the delete isn't enough
196       // Lower TTL and expire to ensure that all our entries have been wiped
197       final int ttl = 1000;
198       for (Store hstore: this.r.stores.values()) {
199         HStore store = (HStore)hstore;
200         ScanInfo old = store.getScanInfo();
201         ScanInfo si = new ScanInfo(old.getFamily(),
202             old.getMinVersions(), old.getMaxVersions(), ttl,
203             old.getKeepDeletedCells(), 0, old.getComparator());
204         store.setScanInfo(si);
205       }
206       Thread.sleep(ttl);
207 
208       r.compact(true);
209       assertEquals(0, count());
210     }
211   }
212 
213   private int count() throws IOException {
214     int count = 0;
215     for (StoreFile f: this.r.stores.
216         get(COLUMN_FAMILY_TEXT).getStorefiles()) {
217       HFileScanner scanner = f.getReader().getScanner(false, false);
218       if (!scanner.seekTo()) {
219         continue;
220       }
221       do {
222         count++;
223       } while(scanner.next());
224     }
225     return count;
226   }
227 
228   private void createStoreFile(final HRegion region) throws IOException {
229     createStoreFile(region, Bytes.toString(COLUMN_FAMILY));
230   }
231 
232   private void createStoreFile(final HRegion region, String family) throws IOException {
233     HRegionIncommon loader = new HRegionIncommon(region);
234     HBaseTestCase.addContent(loader, family);
235     loader.flushcache();
236   }
237 
238   @Test
239   public void testCompactionWithCorruptResult() throws Exception {
240     int nfiles = 10;
241     for (int i = 0; i < nfiles; i++) {
242       createStoreFile(r);
243     }
244     HStore store = (HStore) r.getStore(COLUMN_FAMILY);
245 
246     Collection<StoreFile> storeFiles = store.getStorefiles();
247     DefaultCompactor tool = (DefaultCompactor)store.storeEngine.getCompactor();
248     tool.compactForTesting(storeFiles, false);
249 
250     // Now lets corrupt the compacted file.
251     FileSystem fs = store.getFileSystem();
252     // default compaction policy created one and only one new compacted file
253     Path dstPath = store.getRegionFileSystem().createTempName();
254     FSDataOutputStream stream = fs.create(dstPath, null, true, 512, (short)3, (long)1024, null);
255     stream.writeChars("CORRUPT FILE!!!!");
256     stream.close();
257     Path origPath = store.getRegionFileSystem().commitStoreFile(
258       Bytes.toString(COLUMN_FAMILY), dstPath);
259 
260     try {
261       ((HStore)store).moveFileIntoPlace(origPath);
262     } catch (Exception e) {
263       // The complete compaction should fail and the corrupt file should remain
264       // in the 'tmp' directory;
265       assert (fs.exists(origPath));
266       assert (!fs.exists(dstPath));
267       System.out.println("testCompactionWithCorruptResult Passed");
268       return;
269     }
270     fail("testCompactionWithCorruptResult failed since no exception was" +
271         "thrown while completing a corrupt file");
272   }
273 
274   /**
275    * Create a custom compaction request and be sure that we can track it through the queue, knowing
276    * when the compaction is completed.
277    */
278   @Test
279   public void testTrackingCompactionRequest() throws Exception {
280     // setup a compact/split thread on a mock server
281     HRegionServer mockServer = Mockito.mock(HRegionServer.class);
282     Mockito.when(mockServer.getConfiguration()).thenReturn(r.getBaseConf());
283     CompactSplitThread thread = new CompactSplitThread(mockServer);
284     Mockito.when(mockServer.getCompactSplitThread()).thenReturn(thread);
285 
286     // setup a region/store with some files
287     Store store = r.getStore(COLUMN_FAMILY);
288     createStoreFile(r);
289     for (int i = 0; i < MAX_FILES_TO_COMPACT + 1; i++) {
290       createStoreFile(r);
291     }
292 
293     CountDownLatch latch = new CountDownLatch(1);
294     TrackableCompactionRequest request = new TrackableCompactionRequest(latch);
295     thread.requestCompaction(r, store, "test custom comapction", Store.PRIORITY_USER, request,null);
296     // wait for the latch to complete.
297     latch.await();
298 
299     thread.interruptIfNecessary();
300   }
301 
302   /**
303    * HBASE-7947: Regression test to ensure adding to the correct list in the
304    * {@link CompactSplitThread}
305    * @throws Exception on failure
306    */
307   @Test
308   public void testMultipleCustomCompactionRequests() throws Exception {
309     // setup a compact/split thread on a mock server
310     HRegionServer mockServer = Mockito.mock(HRegionServer.class);
311     Mockito.when(mockServer.getConfiguration()).thenReturn(r.getBaseConf());
312     CompactSplitThread thread = new CompactSplitThread(mockServer);
313     Mockito.when(mockServer.getCompactSplitThread()).thenReturn(thread);
314 
315     // setup a region/store with some files
316     int numStores = r.getStores().size();
317     List<Pair<CompactionRequest, Store>> requests =
318         new ArrayList<Pair<CompactionRequest, Store>>(numStores);
319     CountDownLatch latch = new CountDownLatch(numStores);
320     // create some store files and setup requests for each store on which we want to do a
321     // compaction
322     for (Store store : r.getStores()) {
323       createStoreFile(r, store.getColumnFamilyName());
324       createStoreFile(r, store.getColumnFamilyName());
325       createStoreFile(r, store.getColumnFamilyName());
326       requests
327           .add(new Pair<CompactionRequest, Store>(new TrackableCompactionRequest(latch), store));
328     }
329 
330     thread.requestCompaction(r, "test mulitple custom comapctions", Store.PRIORITY_USER,
331       Collections.unmodifiableList(requests), null);
332 
333     // wait for the latch to complete.
334     latch.await();
335 
336     thread.interruptIfNecessary();
337   }
338 
339   private class StoreMockMaker extends StatefulStoreMockMaker {
340     public ArrayList<StoreFile> compacting = new ArrayList<StoreFile>();
341     public ArrayList<StoreFile> notCompacting = new ArrayList<StoreFile>();
342     private ArrayList<Integer> results;
343 
344     public StoreMockMaker(ArrayList<Integer> results) {
345       this.results = results;
346     }
347 
348     public class TestCompactionContext extends CompactionContext {
349       private List<StoreFile> selectedFiles;
350       public TestCompactionContext(List<StoreFile> selectedFiles) {
351         super();
352         this.selectedFiles = selectedFiles;
353       }
354 
355       @Override
356       public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
357         return new ArrayList<StoreFile>();
358       }
359 
360       @Override
361       public boolean select(List<StoreFile> filesCompacting, boolean isUserCompaction,
362           boolean mayUseOffPeak, boolean forceMajor) throws IOException {
363         this.request = new CompactionRequest(selectedFiles);
364         this.request.setPriority(getPriority());
365         return true;
366       }
367 
368       @Override
369       public List<Path> compact(CompactionThroughputController throughputController)
370           throws IOException {
371         return compact(throughputController, null);
372       }
373 
374       @Override
375       public List<Path> compact(CompactionThroughputController throughputController, User user)
376           throws IOException {
377         finishCompaction(this.selectedFiles);
378         return new ArrayList<Path>();
379       }
380     }
381 
382     @Override
383     public synchronized CompactionContext selectCompaction() {
384       CompactionContext ctx = new TestCompactionContext(new ArrayList<StoreFile>(notCompacting));
385       compacting.addAll(notCompacting);
386       notCompacting.clear();
387       try {
388         ctx.select(null, false, false, false);
389       } catch (IOException ex) {
390         fail("Shouldn't happen");
391       }
392       return ctx;
393     }
394 
395     @Override
396     public synchronized void cancelCompaction(Object object) {
397       TestCompactionContext ctx = (TestCompactionContext)object;
398       compacting.removeAll(ctx.selectedFiles);
399       notCompacting.addAll(ctx.selectedFiles);
400     }
401 
402     public synchronized void finishCompaction(List<StoreFile> sfs) {
403       if (sfs.isEmpty()) return;
404       synchronized (results) {
405         results.add(sfs.size());
406       }
407       compacting.removeAll(sfs);
408     }
409 
410     @Override
411     public int getPriority() {
412       return 7 - compacting.size() - notCompacting.size();
413     }
414   }
415 
416   public class BlockingStoreMockMaker extends StatefulStoreMockMaker {
417     BlockingCompactionContext blocked = null;
418 
419     public class BlockingCompactionContext extends CompactionContext {
420       public volatile boolean isInCompact = false;
421 
422       public void unblock() {
423         synchronized (this) { this.notifyAll(); }
424       }
425 
426       @Override
427       public List<Path> compact(CompactionThroughputController throughputController)
428           throws IOException {
429         return compact(throughputController, null);
430       }
431 
432       @Override
433       public List<Path> compact(CompactionThroughputController throughputController, User user)
434           throws IOException {
435         try {
436           isInCompact = true;
437           synchronized (this) {
438             this.wait();
439           }
440         } catch (InterruptedException e) {
441           Assume.assumeNoException(e);
442         }
443         return new ArrayList<Path>();
444       }
445 
446       @Override
447       public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
448         return new ArrayList<StoreFile>();
449       }
450 
451       @Override
452       public boolean select(List<StoreFile> f, boolean i, boolean m, boolean e)
453           throws IOException {
454         this.request = new CompactionRequest(new ArrayList<StoreFile>());
455         return true;
456       }
457     }
458 
459     @Override
460     public CompactionContext selectCompaction() {
461       this.blocked = new BlockingCompactionContext();
462       try {
463         this.blocked.select(null, false, false, false);
464       } catch (IOException ex) {
465         fail("Shouldn't happen");
466       }
467       return this.blocked;
468     }
469 
470     @Override
471     public void cancelCompaction(Object object) {}
472 
473     @Override
474     public int getPriority() {
475       return Integer.MIN_VALUE; // some invalid value, see createStoreMock
476     }
477 
478     public BlockingCompactionContext waitForBlocking() {
479       while (this.blocked == null || !this.blocked.isInCompact) {
480         Threads.sleepWithoutInterrupt(50);
481       }
482       BlockingCompactionContext ctx = this.blocked;
483       this.blocked = null;
484       return ctx;
485     }
486 
487     @Override
488     public Store createStoreMock(String name) throws Exception {
489       return createStoreMock(Integer.MIN_VALUE, name);
490     }
491 
492     public Store createStoreMock(int priority, String name) throws Exception {
493       // Override the mock to always return the specified priority.
494       Store s = super.createStoreMock(name);
495       when(s.getCompactPriority()).thenReturn(priority);
496       return s;
497     }
498   }
499 
500   /** Test compaction priority management and multiple compactions per store (HBASE-8665). */
501   @Test
502   public void testCompactionQueuePriorities() throws Exception {
503     // Setup a compact/split thread on a mock server.
504     final Configuration conf = HBaseConfiguration.create();
505     HRegionServer mockServer = mock(HRegionServer.class);
506     when(mockServer.isStopped()).thenReturn(false);
507     when(mockServer.getConfiguration()).thenReturn(conf);
508     when(mockServer.getChoreService()).thenReturn(new ChoreService("test"));
509     CompactSplitThread cst = new CompactSplitThread(mockServer);
510     when(mockServer.getCompactSplitThread()).thenReturn(cst);
511 
512     // Set up the region mock that redirects compactions.
513     HRegion r = mock(HRegion.class);
514     when(
515       r.compact(any(CompactionContext.class), any(Store.class),
516         any(CompactionThroughputController.class), any(User.class))).then(new Answer<Boolean>() {
517       public Boolean answer(InvocationOnMock invocation) throws Throwable {
518         invocation.getArgumentAt(0, CompactionContext.class).compact(
519           invocation.getArgumentAt(2, CompactionThroughputController.class));
520         return true;
521       }
522     });
523 
524     // Set up store mocks for 2 "real" stores and the one we use for blocking CST.
525     ArrayList<Integer> results = new ArrayList<Integer>();
526     StoreMockMaker sm = new StoreMockMaker(results), sm2 = new StoreMockMaker(results);
527     Store store = sm.createStoreMock("store1"), store2 = sm2.createStoreMock("store2");
528     BlockingStoreMockMaker blocker = new BlockingStoreMockMaker();
529 
530     // First, block the compaction thread so that we could muck with queue.
531     cst.requestSystemCompaction(r, blocker.createStoreMock(1, "b-pri1"), "b-pri1");
532     BlockingStoreMockMaker.BlockingCompactionContext currentBlock = blocker.waitForBlocking();
533 
534     // Add 4 files to store1, 3 to store2, and queue compactions; pri 3 and 4 respectively.
535     for (int i = 0; i < 4; ++i) {
536       sm.notCompacting.add(createFile());
537     }
538     cst.requestSystemCompaction(r, store, "s1-pri3");
539     for (int i = 0; i < 3; ++i) {
540       sm2.notCompacting.add(createFile());
541     }
542     cst.requestSystemCompaction(r, store2, "s2-pri4");
543     // Now add 2 more files to store1 and queue compaction - pri 1.
544     for (int i = 0; i < 2; ++i) {
545       sm.notCompacting.add(createFile());
546     }
547     cst.requestSystemCompaction(r, store, "s1-pri1");
548     // Finally add blocking compaction with priority 2.
549     cst.requestSystemCompaction(r, blocker.createStoreMock(2, "b-pri2"), "b-pri2");
550 
551     // Unblock the blocking compaction; we should run pri1 and become block again in pri2.
552     currentBlock.unblock();
553     currentBlock = blocker.waitForBlocking();
554     // Pri1 should have "compacted" all 6 files.
555     assertEquals(1, results.size());
556     assertEquals(6, results.get(0).intValue());
557     // Add 2 files to store 1 (it has 2 files now).
558     for (int i = 0; i < 2; ++i) {
559       sm.notCompacting.add(createFile());
560     }
561     // Now we have pri4 for store 2 in queue, and pri3 for store1; store1's current priority
562     // is 5, however, so it must not preempt store 2. Add blocking compaction at the end.
563     cst.requestSystemCompaction(r, blocker.createStoreMock(7, "b-pri7"), "b-pri7");
564     currentBlock.unblock();
565     currentBlock = blocker.waitForBlocking();
566     assertEquals(3, results.size());
567     assertEquals(3, results.get(1).intValue()); // 3 files should go before 2 files.
568     assertEquals(2, results.get(2).intValue());
569 
570     currentBlock.unblock();
571     cst.interruptIfNecessary();
572   }
573 
574   private static StoreFile createFile() throws Exception {
575     StoreFile sf = mock(StoreFile.class);
576     when(sf.getPath()).thenReturn(new Path("file"));
577     StoreFile.Reader r = mock(StoreFile.Reader.class);
578     when(r.length()).thenReturn(10L);
579     when(sf.getReader()).thenReturn(r);
580     return sf;
581   }
582 
583   /**
584    * Simple {@link CompactionRequest} on which you can wait until the requested compaction finishes.
585    */
586   public static class TrackableCompactionRequest extends CompactionRequest {
587     private CountDownLatch done;
588 
589     /**
590      * Constructor for a custom compaction. Uses the setXXX methods to update the state of the
591      * compaction before being used.
592      */
593     public TrackableCompactionRequest(CountDownLatch finished) {
594       super();
595       this.done = finished;
596     }
597 
598     @Override
599     public void afterExecute() {
600       super.afterExecute();
601       this.done.countDown();
602     }
603   }
604 }