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