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