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 private 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 public Object answer(InvocationOnMock invocation) throws Throwable {
164 r.writestate.writesEnabled = false;
165 return invocation.callRealMethod();
166 }
167 }).when(spyR).doRegionCompactionPrep();
168
169
170 spyR.compactStores();
171
172
173 Store s = r.stores.get(COLUMN_FAMILY);
174 assertEquals(compactionThreshold, s.getStorefilesCount());
175 assertTrue(s.getStorefilesSize() > 15*1000);
176
177 FileStatus[] ls = r.getFilesystem().listStatus(r.getRegionFileSystem().getTempDir());
178 assertEquals(0, ls.length);
179
180 } finally {
181
182 r.writestate.writesEnabled = true;
183 HStore.closeCheckInterval = origWI;
184
185
186 for (int i = 0; i < compactionThreshold; i++) {
187 Delete delete = new Delete(Bytes.add(STARTROW, Bytes.toBytes(i)));
188 byte [][] famAndQf = {COLUMN_FAMILY, null};
189 delete.deleteFamily(famAndQf[0]);
190 r.delete(delete);
191 }
192 r.flush(true);
193
194
195
196 final int ttl = 1000;
197 for (Store hstore: this.r.stores.values()) {
198 HStore store = (HStore)hstore;
199 ScanInfo old = store.getScanInfo();
200 ScanInfo si = new ScanInfo(old.getConfiguration(), old.getFamily(),
201 old.getMinVersions(), old.getMaxVersions(), ttl,
202 old.getKeepDeletedCells(), 0, old.getComparator());
203 store.setScanInfo(si);
204 }
205 Thread.sleep(ttl);
206
207 r.compact(true);
208 assertEquals(0, count());
209 }
210 }
211
212 private int count() throws IOException {
213 int count = 0;
214 for (StoreFile f: this.r.stores.
215 get(COLUMN_FAMILY_TEXT).getStorefiles()) {
216 HFileScanner scanner = f.getReader().getScanner(false, false);
217 if (!scanner.seekTo()) {
218 continue;
219 }
220 do {
221 count++;
222 } while(scanner.next());
223 }
224 return count;
225 }
226
227 private void createStoreFile(final HRegion region) throws IOException {
228 createStoreFile(region, Bytes.toString(COLUMN_FAMILY));
229 }
230
231 private void createStoreFile(final HRegion region, String family) throws IOException {
232 HRegionIncommon loader = new HRegionIncommon(region);
233 HBaseTestCase.addContent(loader, family);
234 loader.flushcache();
235 }
236
237 @Test
238 public void testCompactionWithCorruptResult() throws Exception {
239 int nfiles = 10;
240 for (int i = 0; i < nfiles; i++) {
241 createStoreFile(r);
242 }
243 HStore store = (HStore) r.getStore(COLUMN_FAMILY);
244
245 Collection<StoreFile> storeFiles = store.getStorefiles();
246 DefaultCompactor tool = (DefaultCompactor)store.storeEngine.getCompactor();
247 tool.compactForTesting(storeFiles, false);
248
249
250 FileSystem fs = store.getFileSystem();
251
252 Path dstPath = store.getRegionFileSystem().createTempName();
253 FSDataOutputStream stream = fs.create(dstPath, null, true, 512, (short)3, (long)1024, null);
254 stream.writeChars("CORRUPT FILE!!!!");
255 stream.close();
256 Path origPath = store.getRegionFileSystem().commitStoreFile(
257 Bytes.toString(COLUMN_FAMILY), dstPath);
258
259 try {
260 ((HStore)store).moveFileIntoPlace(origPath);
261 } catch (Exception e) {
262
263
264 assert (fs.exists(origPath));
265 assert (!fs.exists(dstPath));
266 System.out.println("testCompactionWithCorruptResult Passed");
267 return;
268 }
269 fail("testCompactionWithCorruptResult failed since no exception was" +
270 "thrown while completing a corrupt file");
271 }
272
273
274
275
276
277 @Test
278 public void testTrackingCompactionRequest() throws Exception {
279
280 HRegionServer mockServer = Mockito.mock(HRegionServer.class);
281 Mockito.when(mockServer.getConfiguration()).thenReturn(r.getBaseConf());
282 CompactSplitThread thread = new CompactSplitThread(mockServer);
283 Mockito.when(mockServer.getCompactSplitThread()).thenReturn(thread);
284
285
286 Store store = r.getStore(COLUMN_FAMILY);
287 createStoreFile(r);
288 for (int i = 0; i < MAX_FILES_TO_COMPACT + 1; i++) {
289 createStoreFile(r);
290 }
291
292 CountDownLatch latch = new CountDownLatch(1);
293 TrackableCompactionRequest request = new TrackableCompactionRequest(latch);
294 thread.requestCompaction(r, store, "test custom comapction", Store.PRIORITY_USER, request,null);
295
296 latch.await();
297
298 thread.interruptIfNecessary();
299 }
300
301
302
303
304
305
306 @Test
307 public void testMultipleCustomCompactionRequests() throws Exception {
308
309 HRegionServer mockServer = Mockito.mock(HRegionServer.class);
310 Mockito.when(mockServer.getConfiguration()).thenReturn(r.getBaseConf());
311 CompactSplitThread thread = new CompactSplitThread(mockServer);
312 Mockito.when(mockServer.getCompactSplitThread()).thenReturn(thread);
313
314
315 int numStores = r.getStores().size();
316 List<Pair<CompactionRequest, Store>> requests =
317 new ArrayList<Pair<CompactionRequest, Store>>(numStores);
318 CountDownLatch latch = new CountDownLatch(numStores);
319
320
321 for (Store store : r.getStores()) {
322 createStoreFile(r, store.getColumnFamilyName());
323 createStoreFile(r, store.getColumnFamilyName());
324 createStoreFile(r, store.getColumnFamilyName());
325 requests
326 .add(new Pair<CompactionRequest, Store>(new TrackableCompactionRequest(latch), store));
327 }
328
329 thread.requestCompaction(r, "test mulitple custom comapctions", Store.PRIORITY_USER,
330 Collections.unmodifiableList(requests), null);
331
332
333 latch.await();
334
335 thread.interruptIfNecessary();
336 }
337
338 private class StoreMockMaker extends StatefulStoreMockMaker {
339 public ArrayList<StoreFile> compacting = new ArrayList<StoreFile>();
340 public ArrayList<StoreFile> notCompacting = new ArrayList<StoreFile>();
341 private ArrayList<Integer> results;
342
343 public StoreMockMaker(ArrayList<Integer> results) {
344 this.results = results;
345 }
346
347 public class TestCompactionContext extends CompactionContext {
348 private List<StoreFile> selectedFiles;
349 public TestCompactionContext(List<StoreFile> selectedFiles) {
350 super();
351 this.selectedFiles = selectedFiles;
352 }
353
354 @Override
355 public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
356 return new ArrayList<StoreFile>();
357 }
358
359 @Override
360 public boolean select(List<StoreFile> filesCompacting, boolean isUserCompaction,
361 boolean mayUseOffPeak, boolean forceMajor) throws IOException {
362 this.request = new CompactionRequest(selectedFiles);
363 this.request.setPriority(getPriority());
364 return true;
365 }
366
367 @Override
368 public List<Path> compact(CompactionThroughputController throughputController)
369 throws IOException {
370 return compact(throughputController, null);
371 }
372
373 @Override
374 public List<Path> compact(CompactionThroughputController throughputController, User user)
375 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 when(mockServer.getChoreService()).thenReturn(new ChoreService("test"));
507 CompactSplitThread cst = new CompactSplitThread(mockServer);
508 when(mockServer.getCompactSplitThread()).thenReturn(cst);
509
510 cst.shutdownLongCompactions();
511
512 HRegion r = mock(HRegion.class);
513 when(
514 r.compact(any(CompactionContext.class), any(Store.class),
515 any(CompactionThroughputController.class), any(User.class))).then(new Answer<Boolean>() {
516 public Boolean answer(InvocationOnMock invocation) throws Throwable {
517 invocation.getArgumentAt(0, CompactionContext.class).compact(
518 invocation.getArgumentAt(2, CompactionThroughputController.class));
519 return true;
520 }
521 });
522
523
524 ArrayList<Integer> results = new ArrayList<Integer>();
525 StoreMockMaker sm = new StoreMockMaker(results), sm2 = new StoreMockMaker(results);
526 Store store = sm.createStoreMock("store1"), store2 = sm2.createStoreMock("store2");
527 BlockingStoreMockMaker blocker = new BlockingStoreMockMaker();
528
529
530 cst.requestSystemCompaction(r, blocker.createStoreMock(1, "b-pri1"), "b-pri1");
531 BlockingStoreMockMaker.BlockingCompactionContext currentBlock = blocker.waitForBlocking();
532
533
534 for (int i = 0; i < 4; ++i) {
535 sm.notCompacting.add(createFile());
536 }
537 cst.requestSystemCompaction(r, store, "s1-pri3");
538 for (int i = 0; i < 3; ++i) {
539 sm2.notCompacting.add(createFile());
540 }
541 cst.requestSystemCompaction(r, store2, "s2-pri4");
542
543 for (int i = 0; i < 2; ++i) {
544 sm.notCompacting.add(createFile());
545 }
546 cst.requestSystemCompaction(r, store, "s1-pri1");
547
548 cst.requestSystemCompaction(r, blocker.createStoreMock(2, "b-pri2"), "b-pri2");
549
550
551 currentBlock.unblock();
552 currentBlock = blocker.waitForBlocking();
553
554 assertEquals(1, results.size());
555 assertEquals(6, results.get(0).intValue());
556
557 for (int i = 0; i < 2; ++i) {
558 sm.notCompacting.add(createFile());
559 }
560
561
562 cst.requestSystemCompaction(r, blocker.createStoreMock(7, "b-pri7"), "b-pri7");
563 currentBlock.unblock();
564 currentBlock = blocker.waitForBlocking();
565 assertEquals(3, results.size());
566 assertEquals(3, results.get(1).intValue());
567 assertEquals(2, results.get(2).intValue());
568
569 currentBlock.unblock();
570 cst.interruptIfNecessary();
571 }
572
573 private static StoreFile createFile() throws Exception {
574 StoreFile sf = mock(StoreFile.class);
575 when(sf.getPath()).thenReturn(new Path("file"));
576 StoreFile.Reader r = mock(StoreFile.Reader.class);
577 when(r.length()).thenReturn(10L);
578 when(sf.getReader()).thenReturn(r);
579 return sf;
580 }
581
582
583
584
585 public static class TrackableCompactionRequest extends CompactionRequest {
586 private CountDownLatch done;
587
588
589
590
591
592 public TrackableCompactionRequest(CountDownLatch finished) {
593 super();
594 this.done = finished;
595 }
596
597 @Override
598 public void afterExecute() {
599 super.afterExecute();
600 this.done.countDown();
601 }
602 }
603 }