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.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
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
102 public TestCompaction() {
103 super();
104
105
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
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
134
135
136
137 @Test
138 public void testInterruptCompaction() throws Exception {
139 assertEquals(0, count());
140
141
142 int origWI = HStore.closeCheckInterval;
143 HStore.closeCheckInterval = 10*1000;
144
145 try {
146
147 int jmax = (int) Math.ceil(15.0/compactionThreshold);
148 byte [] pad = new byte[1000];
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
171 spyR.compactStores();
172
173
174 Store s = r.stores.get(COLUMN_FAMILY);
175 assertEquals(compactionThreshold, s.getStorefilesCount());
176 assertTrue(s.getStorefilesSize() > 15*1000);
177
178 FileStatus[] ls = r.getFilesystem().listStatus(r.getRegionFileSystem().getTempDir());
179 assertEquals(0, ls.length);
180
181 } finally {
182
183 r.writestate.writesEnabled = true;
184 HStore.closeCheckInterval = origWI;
185
186
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
196
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
251 FileSystem fs = store.getFileSystem();
252
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
264
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
276
277
278 @Test
279 public void testTrackingCompactionRequest() throws Exception {
280
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
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
297 latch.await();
298
299 thread.interruptIfNecessary();
300 }
301
302
303
304
305
306
307 @Test
308 public void testMultipleCustomCompactionRequests() throws Exception {
309
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
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
321
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
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;
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
494 Store s = super.createStoreMock(name);
495 when(s.getCompactPriority()).thenReturn(priority);
496 return s;
497 }
498 }
499
500
501 @Test
502 public void testCompactionQueuePriorities() throws Exception {
503
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
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
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
531 cst.requestSystemCompaction(r, blocker.createStoreMock(1, "b-pri1"), "b-pri1");
532 BlockingStoreMockMaker.BlockingCompactionContext currentBlock = blocker.waitForBlocking();
533
534
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
544 for (int i = 0; i < 2; ++i) {
545 sm.notCompacting.add(createFile());
546 }
547 cst.requestSystemCompaction(r, store, "s1-pri1");
548
549 cst.requestSystemCompaction(r, blocker.createStoreMock(2, "b-pri2"), "b-pri2");
550
551
552 currentBlock.unblock();
553 currentBlock = blocker.waitForBlocking();
554
555 assertEquals(1, results.size());
556 assertEquals(6, results.get(0).intValue());
557
558 for (int i = 0; i < 2; ++i) {
559 sm.notCompacting.add(createFile());
560 }
561
562
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());
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
585
586 public static class TrackableCompactionRequest extends CompactionRequest {
587 private CountDownLatch done;
588
589
590
591
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 }