1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
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.apache.hadoop.hbase.HBaseTestingUtility.fam2;
25 import static org.junit.Assert.assertEquals;
26 import static org.junit.Assert.assertNotNull;
27 import static org.junit.Assert.assertNull;
28 import static org.junit.Assert.assertTrue;
29 import static org.junit.Assert.fail;
30 import static org.mockito.Matchers.any;
31 import static org.mockito.Mockito.doAnswer;
32 import static org.mockito.Mockito.mock;
33 import static org.mockito.Mockito.spy;
34 import static org.mockito.Mockito.when;
35
36 import java.io.IOException;
37 import java.util.ArrayList;
38 import java.util.Collection;
39 import java.util.Collections;
40 import java.util.HashMap;
41 import java.util.List;
42 import java.util.Map;
43 import java.util.Map.Entry;
44 import java.util.concurrent.CountDownLatch;
45
46 import org.apache.commons.logging.Log;
47 import org.apache.commons.logging.LogFactory;
48 import org.apache.hadoop.conf.Configuration;
49 import org.apache.hadoop.fs.FSDataOutputStream;
50 import org.apache.hadoop.fs.FileStatus;
51 import org.apache.hadoop.fs.FileSystem;
52 import org.apache.hadoop.fs.Path;
53 import org.apache.hadoop.hbase.Cell;
54 import org.apache.hadoop.hbase.CellUtil;
55 import org.apache.hadoop.hbase.HBaseConfiguration;
56 import org.apache.hadoop.hbase.HBaseTestCase;
57 import org.apache.hadoop.hbase.HBaseTestCase.HRegionIncommon;
58 import org.apache.hadoop.hbase.HBaseTestingUtility;
59 import org.apache.hadoop.hbase.HConstants;
60 import org.apache.hadoop.hbase.HTableDescriptor;
61 import org.apache.hadoop.hbase.MediumTests;
62 import org.apache.hadoop.hbase.client.Delete;
63 import org.apache.hadoop.hbase.client.Durability;
64 import org.apache.hadoop.hbase.client.Get;
65 import org.apache.hadoop.hbase.client.Put;
66 import org.apache.hadoop.hbase.client.Result;
67 import org.apache.hadoop.hbase.client.Scan;
68 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
69 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
70 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl;
71 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
72 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
73 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
74 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
75 import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
76 import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
77 import org.apache.hadoop.hbase.regionserver.wal.HLog;
78 import org.apache.hadoop.hbase.util.Bytes;
79 import org.apache.hadoop.hbase.util.Pair;
80 import org.apache.hadoop.hbase.util.Threads;
81 import org.junit.After;
82 import org.junit.Assume;
83 import org.junit.Before;
84 import org.junit.Rule;
85 import org.junit.Test;
86 import org.junit.experimental.categories.Category;
87 import org.junit.rules.TestName;
88 import org.mockito.Mockito;
89 import org.mockito.invocation.InvocationOnMock;
90 import org.mockito.stubbing.Answer;
91
92
93
94
95
96 @Category(MediumTests.class)
97 public class TestCompaction {
98 @Rule public TestName name = new TestName();
99 static final Log LOG = LogFactory.getLog(TestCompaction.class.getName());
100 private static final HBaseTestingUtility UTIL = new HBaseTestingUtility().createLocalHTU();
101 protected Configuration conf = UTIL.getConfiguration();
102
103 private HRegion r = null;
104 private HTableDescriptor htd = null;
105 private Path compactionDir = null;
106 private Path regionCompactionDir = null;
107 private static final byte [] COLUMN_FAMILY = fam1;
108 private final byte [] STARTROW = Bytes.toBytes(START_KEY);
109 private static final byte [] COLUMN_FAMILY_TEXT = COLUMN_FAMILY;
110 private int compactionThreshold;
111 private byte[] firstRowBytes, secondRowBytes, thirdRowBytes;
112 final private byte[] col1, col2;
113 private static final long MAX_FILES_TO_COMPACT = 10;
114
115
116 public TestCompaction() {
117 super();
118
119
120 conf.setInt(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 1024*1024);
121 conf.setInt("hbase.hregion.memstore.block.multiplier", 100);
122 compactionThreshold = conf.getInt("hbase.hstore.compactionThreshold", 3);
123
124 firstRowBytes = START_KEY_BYTES;
125 secondRowBytes = START_KEY_BYTES.clone();
126
127 secondRowBytes[START_KEY_BYTES.length - 1]++;
128 thirdRowBytes = START_KEY_BYTES.clone();
129 thirdRowBytes[START_KEY_BYTES.length - 1] += 2;
130 col1 = Bytes.toBytes("column1");
131 col2 = Bytes.toBytes("column2");
132 }
133
134 @Before
135 public void setUp() throws Exception {
136 this.htd = UTIL.createTableDescriptor(name.getMethodName());
137 this.r = UTIL.createLocalHRegion(htd, null, null);
138 }
139
140 @After
141 public void tearDown() throws Exception {
142 HLog hlog = r.getLog();
143 this.r.close();
144 hlog.closeAndDelete();
145 }
146
147
148
149
150
151
152
153 @Test
154 public void testMajorCompactingToNoOutput() throws IOException {
155 createStoreFile(r);
156 for (int i = 0; i < compactionThreshold; i++) {
157 createStoreFile(r);
158 }
159
160 InternalScanner s = r.getScanner(new Scan());
161 do {
162 List<Cell> results = new ArrayList<Cell>();
163 boolean result = s.next(results);
164 r.delete(new Delete(CellUtil.cloneRow(results.get(0))));
165 if (!result) break;
166 } while(true);
167 s.close();
168
169 r.flushcache();
170
171 r.compactStores(true);
172 s = r.getScanner(new Scan());
173 int counter = 0;
174 do {
175 List<Cell> results = new ArrayList<Cell>();
176 boolean result = s.next(results);
177 if (!result) break;
178 counter++;
179 } while(true);
180 assertEquals(0, counter);
181 }
182
183
184
185
186
187
188 @Test
189 public void testMajorCompaction() throws Exception {
190 majorCompaction();
191 }
192
193 @Test
194 public void testDataBlockEncodingInCacheOnly() throws Exception {
195 majorCompactionWithDataBlockEncoding(true);
196 }
197
198 @Test
199 public void testDataBlockEncodingEverywhere() throws Exception {
200 majorCompactionWithDataBlockEncoding(false);
201 }
202
203 public void majorCompactionWithDataBlockEncoding(boolean inCacheOnly)
204 throws Exception {
205 Map<HStore, HFileDataBlockEncoder> replaceBlockCache =
206 new HashMap<HStore, HFileDataBlockEncoder>();
207 for (Entry<byte[], Store> pair : r.getStores().entrySet()) {
208 HStore store = (HStore) pair.getValue();
209 HFileDataBlockEncoder blockEncoder = store.getDataBlockEncoder();
210 replaceBlockCache.put(store, blockEncoder);
211 final DataBlockEncoding inCache = DataBlockEncoding.PREFIX;
212 final DataBlockEncoding onDisk = inCacheOnly ? DataBlockEncoding.NONE :
213 inCache;
214 store.setDataBlockEncoderInTest(new HFileDataBlockEncoderImpl(onDisk));
215 }
216
217 majorCompaction();
218
219
220 for (Entry<HStore, HFileDataBlockEncoder> entry :
221 replaceBlockCache.entrySet()) {
222 entry.getKey().setDataBlockEncoderInTest(entry.getValue());
223 }
224 }
225
226 private void majorCompaction() throws Exception {
227 createStoreFile(r);
228 for (int i = 0; i < compactionThreshold; i++) {
229 createStoreFile(r);
230 }
231
232 HBaseTestCase.addContent(new HRegionIncommon(r), Bytes.toString(COLUMN_FAMILY));
233
234
235
236
237
238 Result result = r.get(new Get(STARTROW).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100));
239 assertEquals(compactionThreshold, result.size());
240
241
242 for (Store store : this.r.stores.values()) {
243 assertNull(store.getCompactionProgress());
244 }
245
246 r.flushcache();
247 r.compactStores(true);
248
249
250 int storeCount = 0;
251 for (Store store : this.r.stores.values()) {
252 CompactionProgress progress = store.getCompactionProgress();
253 if( progress != null ) {
254 ++storeCount;
255 assertTrue(progress.currentCompactedKVs > 0);
256 assertTrue(progress.totalCompactingKVs > 0);
257 }
258 assertTrue(storeCount > 0);
259 }
260
261
262
263 byte [] secondRowBytes = START_KEY_BYTES.clone();
264 secondRowBytes[START_KEY_BYTES.length - 1]++;
265
266
267 result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).
268 setMaxVersions(100));
269 LOG.debug("Row " + Bytes.toStringBinary(secondRowBytes) + " after " +
270 "initial compaction: " + result);
271 assertEquals("Invalid number of versions of row "
272 + Bytes.toStringBinary(secondRowBytes) + ".", compactionThreshold,
273 result.size());
274
275
276
277
278
279
280 LOG.debug("Adding deletes to memstore and flushing");
281 Delete delete = new Delete(secondRowBytes, System.currentTimeMillis());
282 byte [][] famAndQf = {COLUMN_FAMILY, null};
283 delete.deleteFamily(famAndQf[0]);
284 r.delete(delete);
285
286
287 result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100));
288 assertTrue("Second row should have been deleted", result.isEmpty());
289
290 r.flushcache();
291
292 result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100));
293 assertTrue("Second row should have been deleted", result.isEmpty());
294
295
296 createSmallerStoreFile(this.r);
297 r.flushcache();
298
299 result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100));
300 assertTrue("Second row should still be deleted", result.isEmpty());
301
302
303 r.compactStores(true);
304 assertEquals(r.getStore(COLUMN_FAMILY_TEXT).getStorefiles().size(), 1);
305
306 result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100));
307 assertTrue("Second row should still be deleted", result.isEmpty());
308
309
310
311
312 verifyCounts(3,0);
313
314
315
316 final int ttl = 1000;
317 for (Store hstore : this.r.stores.values()) {
318 HStore store = ((HStore) hstore);
319 ScanInfo old = store.getScanInfo();
320 ScanInfo si = new ScanInfo(old.getFamily(),
321 old.getMinVersions(), old.getMaxVersions(), ttl,
322 old.getKeepDeletedCells(), 0, old.getComparator());
323 store.setScanInfo(si);
324 }
325 Thread.sleep(1000);
326
327 r.compactStores(true);
328 int count = count();
329 assertEquals("Should not see anything after TTL has expired", 0, count);
330 }
331
332 @Test
333 public void testTimeBasedMajorCompaction() throws Exception {
334
335 int delay = 10 * 1000;
336 float jitterPct = 0.20f;
337 conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, delay);
338 conf.setFloat("hbase.hregion.majorcompaction.jitter", jitterPct);
339
340 HStore s = ((HStore) r.getStore(COLUMN_FAMILY));
341 s.storeEngine.getCompactionPolicy().setConf(conf);
342 try {
343 createStoreFile(r);
344 createStoreFile(r);
345 r.compactStores(true);
346
347
348 createStoreFile(r);
349 r.compactStores(false);
350 assertEquals(2, s.getStorefilesCount());
351
352
353 RatioBasedCompactionPolicy
354 c = (RatioBasedCompactionPolicy)s.storeEngine.getCompactionPolicy();
355 Collection<StoreFile> storeFiles = s.getStorefiles();
356 long mcTime = c.getNextMajorCompactTime(storeFiles);
357 for (int i = 0; i < 10; ++i) {
358 assertEquals(mcTime, c.getNextMajorCompactTime(storeFiles));
359 }
360
361
362 long jitter = Math.round(delay * jitterPct);
363 assertTrue(delay - jitter <= mcTime && mcTime <= delay + jitter);
364
365
366 Thread.sleep(mcTime);
367
368
369 r.compactStores(false);
370 assertEquals(1, s.getStorefilesCount());
371 } finally {
372
373 conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 1000*60*60*24);
374 conf.setFloat("hbase.hregion.majorcompaction.jitter", 0.20F);
375
376 createStoreFile(r);
377 r.compactStores(true);
378 assertEquals(1, s.getStorefilesCount());
379 }
380 }
381
382 @Test
383 public void testMinorCompactionWithDeleteRow() throws Exception {
384 Delete deleteRow = new Delete(secondRowBytes);
385 testMinorCompactionWithDelete(deleteRow);
386 }
387
388 @Test
389 public void testMinorCompactionWithDeleteColumn1() throws Exception {
390 Delete dc = new Delete(secondRowBytes);
391
392 dc.deleteColumns(fam2, col2);
393 testMinorCompactionWithDelete(dc);
394 }
395
396 @Test
397 public void testMinorCompactionWithDeleteColumn2() throws Exception {
398 Delete dc = new Delete(secondRowBytes);
399 dc.deleteColumn(fam2, col2);
400
401
402
403
404
405
406
407 testMinorCompactionWithDelete(dc, 3);
408 }
409
410 @Test
411 public void testMinorCompactionWithDeleteColumnFamily() throws Exception {
412 Delete deleteCF = new Delete(secondRowBytes);
413 deleteCF.deleteFamily(fam2);
414 testMinorCompactionWithDelete(deleteCF);
415 }
416
417 @Test
418 public void testMinorCompactionWithDeleteVersion1() throws Exception {
419 Delete deleteVersion = new Delete(secondRowBytes);
420 deleteVersion.deleteColumns(fam2, col2, 2);
421
422
423
424 testMinorCompactionWithDelete(deleteVersion, 1);
425 }
426
427 @Test
428 public void testMinorCompactionWithDeleteVersion2() throws Exception {
429 Delete deleteVersion = new Delete(secondRowBytes);
430 deleteVersion.deleteColumn(fam2, col2, 1);
431
432
433
434
435
436 testMinorCompactionWithDelete(deleteVersion, 3);
437 }
438
439
440
441
442
443
444
445 private void testMinorCompactionWithDelete(Delete delete) throws Exception {
446 testMinorCompactionWithDelete(delete, 0);
447 }
448 private void testMinorCompactionWithDelete(Delete delete, int expectedResultsAfterDelete) throws Exception {
449 HRegionIncommon loader = new HRegionIncommon(r);
450 for (int i = 0; i < compactionThreshold + 1; i++) {
451 HBaseTestCase.addContent(loader, Bytes.toString(fam1), Bytes.toString(col1), firstRowBytes, thirdRowBytes, i);
452 HBaseTestCase.addContent(loader, Bytes.toString(fam1), Bytes.toString(col2), firstRowBytes, thirdRowBytes, i);
453 HBaseTestCase.addContent(loader, Bytes.toString(fam2), Bytes.toString(col1), firstRowBytes, thirdRowBytes, i);
454 HBaseTestCase.addContent(loader, Bytes.toString(fam2), Bytes.toString(col2), firstRowBytes, thirdRowBytes, i);
455 r.flushcache();
456 }
457
458 Result result = r.get(new Get(firstRowBytes).addColumn(fam1, col1).setMaxVersions(100));
459 assertEquals(compactionThreshold, result.size());
460 result = r.get(new Get(secondRowBytes).addColumn(fam2, col2).setMaxVersions(100));
461 assertEquals(compactionThreshold, result.size());
462
463
464
465
466
467 r.delete(delete);
468
469
470 result = r.get(new Get(secondRowBytes).addColumn(fam2, col2).setMaxVersions(100));
471 assertEquals(expectedResultsAfterDelete, result.size());
472
473 result = r.get(new Get(firstRowBytes).addColumn(fam1, col1).setMaxVersions(100));
474 assertEquals(compactionThreshold, result.size());
475
476 r.flushcache();
477
478
479
480
481 result = r.get(new Get(secondRowBytes).addColumn(fam2, col2).setMaxVersions(100));
482 assertEquals(expectedResultsAfterDelete, result.size());
483
484 result = r.get(new Get(firstRowBytes).addColumn(fam1, col1).setMaxVersions(100));
485 assertEquals(compactionThreshold, result.size());
486
487
488 Store store2 = this.r.stores.get(fam2);
489 int numFiles1 = store2.getStorefiles().size();
490 assertTrue("Was expecting to see 4 store files", numFiles1 > compactionThreshold);
491 ((HStore)store2).compactRecentForTestingAssumingDefaultPolicy(compactionThreshold);
492 int numFiles2 = store2.getStorefiles().size();
493
494 assertTrue("Number of store files should go down", numFiles1 > numFiles2);
495
496 assertTrue("Was not supposed to be a major compaction", numFiles2 > 1);
497
498
499 result = r.get(new Get(secondRowBytes).addColumn(fam2, col2).setMaxVersions(100));
500 assertEquals(expectedResultsAfterDelete, result.size());
501
502 result = r.get(new Get(firstRowBytes).addColumn(fam1, col1).setMaxVersions(100));
503 assertEquals(compactionThreshold, result.size());
504 }
505
506 private void verifyCounts(int countRow1, int countRow2) throws Exception {
507 int count1 = 0;
508 int count2 = 0;
509 for (StoreFile f: this.r.stores.get(COLUMN_FAMILY_TEXT).getStorefiles()) {
510 HFileScanner scanner = f.getReader().getScanner(false, false);
511 scanner.seekTo();
512 do {
513 byte [] row = scanner.getKeyValue().getRow();
514 if (Bytes.equals(row, STARTROW)) {
515 count1++;
516 } else if(Bytes.equals(row, secondRowBytes)) {
517 count2++;
518 }
519 } while(scanner.next());
520 }
521 assertEquals(countRow1,count1);
522 assertEquals(countRow2,count2);
523 }
524
525
526
527
528
529
530 @Test
531 public void testInterruptCompaction() throws Exception {
532 assertEquals(0, count());
533
534
535 int origWI = HStore.closeCheckInterval;
536 HStore.closeCheckInterval = 10*1000;
537
538 try {
539
540 int jmax = (int) Math.ceil(15.0/compactionThreshold);
541 byte [] pad = new byte[1000];
542 for (int i = 0; i < compactionThreshold; i++) {
543 HRegionIncommon loader = new HRegionIncommon(r);
544 Put p = new Put(Bytes.add(STARTROW, Bytes.toBytes(i)));
545 p.setDurability(Durability.SKIP_WAL);
546 for (int j = 0; j < jmax; j++) {
547 p.add(COLUMN_FAMILY, Bytes.toBytes(j), pad);
548 }
549 HBaseTestCase.addContent(loader, Bytes.toString(COLUMN_FAMILY));
550 loader.put(p);
551 loader.flushcache();
552 }
553
554 HRegion spyR = spy(r);
555 doAnswer(new Answer() {
556 public Object answer(InvocationOnMock invocation) throws Throwable {
557 r.writestate.writesEnabled = false;
558 return invocation.callRealMethod();
559 }
560 }).when(spyR).doRegionCompactionPrep();
561
562
563 spyR.compactStores();
564
565
566 Store s = r.stores.get(COLUMN_FAMILY);
567 assertEquals(compactionThreshold, s.getStorefilesCount());
568 assertTrue(s.getStorefilesSize() > 15*1000);
569
570 FileStatus[] ls = r.getFilesystem().listStatus(r.getRegionFileSystem().getTempDir());
571 assertEquals(0, ls.length);
572
573 } finally {
574
575 r.writestate.writesEnabled = true;
576 HStore.closeCheckInterval = origWI;
577
578
579 for (int i = 0; i < compactionThreshold; i++) {
580 Delete delete = new Delete(Bytes.add(STARTROW, Bytes.toBytes(i)));
581 byte [][] famAndQf = {COLUMN_FAMILY, null};
582 delete.deleteFamily(famAndQf[0]);
583 r.delete(delete);
584 }
585 r.flushcache();
586
587
588
589 final int ttl = 1000;
590 for (Store hstore: this.r.stores.values()) {
591 HStore store = (HStore)hstore;
592 ScanInfo old = store.getScanInfo();
593 ScanInfo si = new ScanInfo(old.getFamily(),
594 old.getMinVersions(), old.getMaxVersions(), ttl,
595 old.getKeepDeletedCells(), 0, old.getComparator());
596 store.setScanInfo(si);
597 }
598 Thread.sleep(ttl);
599
600 r.compactStores(true);
601 assertEquals(0, count());
602 }
603 }
604
605 private int count() throws IOException {
606 int count = 0;
607 for (StoreFile f: this.r.stores.
608 get(COLUMN_FAMILY_TEXT).getStorefiles()) {
609 HFileScanner scanner = f.getReader().getScanner(false, false);
610 if (!scanner.seekTo()) {
611 continue;
612 }
613 do {
614 count++;
615 } while(scanner.next());
616 }
617 return count;
618 }
619
620 private void createStoreFile(final HRegion region) throws IOException {
621 createStoreFile(region, Bytes.toString(COLUMN_FAMILY));
622 }
623
624 private void createStoreFile(final HRegion region, String family) throws IOException {
625 HRegionIncommon loader = new HRegionIncommon(region);
626 HBaseTestCase.addContent(loader, family);
627 loader.flushcache();
628 }
629
630 private void createSmallerStoreFile(final HRegion region) throws IOException {
631 HRegionIncommon loader = new HRegionIncommon(region);
632 HBaseTestCase.addContent(loader, Bytes.toString(COLUMN_FAMILY), ("" +
633 "bbb").getBytes(), null);
634 loader.flushcache();
635 }
636
637 @Test
638 public void testCompactionWithCorruptResult() throws Exception {
639 int nfiles = 10;
640 for (int i = 0; i < nfiles; i++) {
641 createStoreFile(r);
642 }
643 HStore store = (HStore) r.getStore(COLUMN_FAMILY);
644
645 Collection<StoreFile> storeFiles = store.getStorefiles();
646 Compactor tool = store.storeEngine.getCompactor();
647
648 List<Path> newFiles = tool.compactForTesting(storeFiles, false);
649
650
651 FileSystem fs = store.getFileSystem();
652
653 Path dstPath = store.getRegionFileSystem().createTempName();
654 FSDataOutputStream stream = fs.create(dstPath, null, true, 512, (short)3, (long)1024, null);
655 stream.writeChars("CORRUPT FILE!!!!");
656 stream.close();
657 Path origPath = store.getRegionFileSystem().commitStoreFile(
658 Bytes.toString(COLUMN_FAMILY), dstPath);
659
660 try {
661 ((HStore)store).moveFileIntoPlace(origPath);
662 } catch (Exception e) {
663
664
665 assert (fs.exists(origPath));
666 assert (!fs.exists(dstPath));
667 System.out.println("testCompactionWithCorruptResult Passed");
668 return;
669 }
670 fail("testCompactionWithCorruptResult failed since no exception was" +
671 "thrown while completing a corrupt file");
672 }
673
674
675
676
677 @Test
678 public void testNonUserMajorCompactionRequest() throws Exception {
679 Store store = r.getStore(COLUMN_FAMILY);
680 createStoreFile(r);
681 for (int i = 0; i < MAX_FILES_TO_COMPACT + 1; i++) {
682 createStoreFile(r);
683 }
684 store.triggerMajorCompaction();
685
686 CompactionRequest request = store.requestCompaction(Store.NO_PRIORITY, null).getRequest();
687 assertNotNull("Expected to receive a compaction request", request);
688 assertEquals(
689 "System-requested major compaction should not occur if there are too many store files",
690 false,
691 request.isMajor());
692 }
693
694
695
696
697 @Test
698 public void testUserMajorCompactionRequest() throws IOException{
699 Store store = r.getStore(COLUMN_FAMILY);
700 createStoreFile(r);
701 for (int i = 0; i < MAX_FILES_TO_COMPACT + 1; i++) {
702 createStoreFile(r);
703 }
704 store.triggerMajorCompaction();
705 CompactionRequest request = store.requestCompaction(Store.PRIORITY_USER, null).getRequest();
706 assertNotNull("Expected to receive a compaction request", request);
707 assertEquals(
708 "User-requested major compaction should always occur, even if there are too many store files",
709 true,
710 request.isMajor());
711 }
712
713
714
715
716
717 @Test
718 public void testTrackingCompactionRequest() throws Exception {
719
720 HRegionServer mockServer = Mockito.mock(HRegionServer.class);
721 Mockito.when(mockServer.getConfiguration()).thenReturn(r.getBaseConf());
722 CompactSplitThread thread = new CompactSplitThread(mockServer);
723 Mockito.when(mockServer.getCompactSplitThread()).thenReturn(thread);
724
725
726 Store store = r.getStore(COLUMN_FAMILY);
727 createStoreFile(r);
728 for (int i = 0; i < MAX_FILES_TO_COMPACT + 1; i++) {
729 createStoreFile(r);
730 }
731
732 CountDownLatch latch = new CountDownLatch(1);
733 TrackableCompactionRequest request = new TrackableCompactionRequest(latch);
734 thread.requestCompaction(r, store, "test custom comapction", Store.PRIORITY_USER, request);
735
736 latch.await();
737
738 thread.interruptIfNecessary();
739 }
740
741
742
743
744
745
746 @Test
747 public void testMultipleCustomCompactionRequests() throws Exception {
748
749 HRegionServer mockServer = Mockito.mock(HRegionServer.class);
750 Mockito.when(mockServer.getConfiguration()).thenReturn(r.getBaseConf());
751 CompactSplitThread thread = new CompactSplitThread(mockServer);
752 Mockito.when(mockServer.getCompactSplitThread()).thenReturn(thread);
753
754
755 int numStores = r.getStores().size();
756 List<Pair<CompactionRequest, Store>> requests =
757 new ArrayList<Pair<CompactionRequest, Store>>(numStores);
758 CountDownLatch latch = new CountDownLatch(numStores);
759
760
761 for (Store store : r.getStores().values()) {
762 createStoreFile(r, store.getColumnFamilyName());
763 createStoreFile(r, store.getColumnFamilyName());
764 createStoreFile(r, store.getColumnFamilyName());
765 requests
766 .add(new Pair<CompactionRequest, Store>(new TrackableCompactionRequest(latch), store));
767 }
768
769 thread.requestCompaction(r, "test mulitple custom comapctions", Store.PRIORITY_USER,
770 Collections.unmodifiableList(requests));
771
772
773 latch.await();
774
775 thread.interruptIfNecessary();
776 }
777
778 private class StoreMockMaker extends StatefulStoreMockMaker {
779 public ArrayList<StoreFile> compacting = new ArrayList<StoreFile>();
780 public ArrayList<StoreFile> notCompacting = new ArrayList<StoreFile>();
781 private ArrayList<Integer> results;
782
783 public StoreMockMaker(ArrayList<Integer> results) {
784 this.results = results;
785 }
786
787 public class TestCompactionContext extends CompactionContext {
788 private List<StoreFile> selectedFiles;
789 public TestCompactionContext(List<StoreFile> selectedFiles) {
790 super();
791 this.selectedFiles = selectedFiles;
792 }
793
794 @Override
795 public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
796 return new ArrayList<StoreFile>();
797 }
798
799 @Override
800 public boolean select(List<StoreFile> filesCompacting, boolean isUserCompaction,
801 boolean mayUseOffPeak, boolean forceMajor) throws IOException {
802 this.request = new CompactionRequest(selectedFiles);
803 this.request.setPriority(getPriority());
804 return true;
805 }
806
807 @Override
808 public List<Path> compact() throws IOException {
809 finishCompaction(this.selectedFiles);
810 return new ArrayList<Path>();
811 }
812 }
813
814 @Override
815 public synchronized CompactionContext selectCompaction() {
816 CompactionContext ctx = new TestCompactionContext(new ArrayList<StoreFile>(notCompacting));
817 compacting.addAll(notCompacting);
818 notCompacting.clear();
819 try {
820 ctx.select(null, false, false, false);
821 } catch (IOException ex) {
822 fail("Shouldn't happen");
823 }
824 return ctx;
825 }
826
827 @Override
828 public synchronized void cancelCompaction(Object object) {
829 TestCompactionContext ctx = (TestCompactionContext)object;
830 compacting.removeAll(ctx.selectedFiles);
831 notCompacting.addAll(ctx.selectedFiles);
832 }
833
834 public synchronized void finishCompaction(List<StoreFile> sfs) {
835 if (sfs.isEmpty()) return;
836 synchronized (results) {
837 results.add(sfs.size());
838 }
839 compacting.removeAll(sfs);
840 }
841
842 @Override
843 public int getPriority() {
844 return 7 - compacting.size() - notCompacting.size();
845 }
846 }
847
848 public class BlockingStoreMockMaker extends StatefulStoreMockMaker {
849 BlockingCompactionContext blocked = null;
850
851 public class BlockingCompactionContext extends CompactionContext {
852 public volatile boolean isInCompact = false;
853
854 public void unblock() {
855 synchronized (this) { this.notifyAll(); }
856 }
857
858 @Override
859 public List<Path> compact() throws IOException {
860 try {
861 isInCompact = true;
862 synchronized (this) { this.wait(); }
863 } catch (InterruptedException e) {
864 Assume.assumeNoException(e);
865 }
866 return new ArrayList<Path>();
867 }
868
869 @Override
870 public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
871 return new ArrayList<StoreFile>();
872 }
873
874 @Override
875 public boolean select(List<StoreFile> f, boolean i, boolean m, boolean e)
876 throws IOException {
877 this.request = new CompactionRequest(new ArrayList<StoreFile>());
878 return true;
879 }
880 }
881
882 @Override
883 public CompactionContext selectCompaction() {
884 this.blocked = new BlockingCompactionContext();
885 try {
886 this.blocked.select(null, false, false, false);
887 } catch (IOException ex) {
888 fail("Shouldn't happen");
889 }
890 return this.blocked;
891 }
892
893 @Override
894 public void cancelCompaction(Object object) {}
895
896 public int getPriority() {
897 return Integer.MIN_VALUE;
898 }
899
900 public BlockingCompactionContext waitForBlocking() {
901 while (this.blocked == null || !this.blocked.isInCompact) {
902 Threads.sleepWithoutInterrupt(50);
903 }
904 BlockingCompactionContext ctx = this.blocked;
905 this.blocked = null;
906 return ctx;
907 }
908
909 @Override
910 public Store createStoreMock(String name) throws Exception {
911 return createStoreMock(Integer.MIN_VALUE, name);
912 }
913
914 public Store createStoreMock(int priority, String name) throws Exception {
915
916 Store s = super.createStoreMock(name);
917 when(s.getCompactPriority()).thenReturn(priority);
918 return s;
919 }
920 }
921
922
923 @Test
924 public void testCompactionQueuePriorities() throws Exception {
925
926 final Configuration conf = HBaseConfiguration.create();
927 HRegionServer mockServer = mock(HRegionServer.class);
928 when(mockServer.isStopped()).thenReturn(false);
929 when(mockServer.getConfiguration()).thenReturn(conf);
930 CompactSplitThread cst = new CompactSplitThread(mockServer);
931 when(mockServer.getCompactSplitThread()).thenReturn(cst);
932
933
934 HRegion r = mock(HRegion.class);
935 when(r.compact(any(CompactionContext.class), any(Store.class))).then(new Answer<Boolean>() {
936 public Boolean answer(InvocationOnMock invocation) throws Throwable {
937 ((CompactionContext)invocation.getArguments()[0]).compact();
938 return true;
939 }
940 });
941
942
943 ArrayList<Integer> results = new ArrayList<Integer>();
944 StoreMockMaker sm = new StoreMockMaker(results), sm2 = new StoreMockMaker(results);
945 Store store = sm.createStoreMock("store1"), store2 = sm2.createStoreMock("store2");
946 BlockingStoreMockMaker blocker = new BlockingStoreMockMaker();
947
948
949 cst.requestSystemCompaction(r, blocker.createStoreMock(1, "b-pri1"), "b-pri1");
950 BlockingStoreMockMaker.BlockingCompactionContext currentBlock = blocker.waitForBlocking();
951
952
953 for (int i = 0; i < 4; ++i) {
954 sm.notCompacting.add(createFile());
955 }
956 cst.requestSystemCompaction(r, store, "s1-pri3");
957 for (int i = 0; i < 3; ++i) {
958 sm2.notCompacting.add(createFile());
959 }
960 cst.requestSystemCompaction(r, store2, "s2-pri4");
961
962 for (int i = 0; i < 2; ++i) {
963 sm.notCompacting.add(createFile());
964 }
965 cst.requestSystemCompaction(r, store, "s1-pri1");
966
967 cst.requestSystemCompaction(r, blocker.createStoreMock(2, "b-pri2"), "b-pri2");
968
969
970 currentBlock.unblock();
971 currentBlock = blocker.waitForBlocking();
972
973 assertEquals(1, results.size());
974 assertEquals(6, results.get(0).intValue());
975
976 for (int i = 0; i < 2; ++i) {
977 sm.notCompacting.add(createFile());
978 }
979
980
981 cst.requestSystemCompaction(r, blocker.createStoreMock(7, "b-pri7"), "b-pri7");
982 currentBlock.unblock();
983 currentBlock = blocker.waitForBlocking();
984 assertEquals(3, results.size());
985 assertEquals(3, results.get(1).intValue());
986 assertEquals(2, results.get(2).intValue());
987
988 currentBlock.unblock();
989 cst.interruptIfNecessary();
990 }
991
992 private static StoreFile createFile() throws Exception {
993 StoreFile sf = mock(StoreFile.class);
994 when(sf.getPath()).thenReturn(new Path("file"));
995 StoreFile.Reader r = mock(StoreFile.Reader.class);
996 when(r.length()).thenReturn(10L);
997 when(sf.getReader()).thenReturn(r);
998 return sf;
999 }
1000
1001
1002
1003
1004 public static class TrackableCompactionRequest extends CompactionRequest {
1005 private CountDownLatch done;
1006
1007
1008
1009
1010
1011 public TrackableCompactionRequest(CountDownLatch finished) {
1012 super();
1013 this.done = finished;
1014 }
1015
1016 @Override
1017 public void afterExecute() {
1018 super.afterExecute();
1019 this.done.countDown();
1020 }
1021 }
1022 }