HBASE-9836 Intermittent TestRegionObserverScannerOpenHook#testRegionObserverCompactionTimeStacking failure
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1536583 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
322eec45f8
commit
3bebeca24d
|
@ -26,8 +26,10 @@ import java.io.IOException;
|
|||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.NavigableSet;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.Coprocessor;
|
||||
|
@ -53,9 +55,12 @@ import org.apache.hadoop.hbase.regionserver.HStore;
|
|||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
import org.apache.hadoop.hbase.regionserver.ScanType;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
@ -203,6 +208,31 @@ public class TestRegionObserverScannerOpenHook {
|
|||
+ r, r.listCells());
|
||||
}
|
||||
|
||||
/*
|
||||
* Custom HRegion which uses CountDownLatch to signal the completion of compaction
|
||||
*/
|
||||
public static class CompactionCompletionNotifyingRegion extends HRegion {
|
||||
private static volatile CountDownLatch compactionStateChangeLatch = null;
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
public CompactionCompletionNotifyingRegion(Path tableDir, HLog log,
|
||||
FileSystem fs, Configuration confParam, HRegionInfo info,
|
||||
HTableDescriptor htd, RegionServerServices rsServices) {
|
||||
super(tableDir, log, fs, confParam, info, htd, rsServices);
|
||||
}
|
||||
|
||||
public CountDownLatch getCompactionStateChangeLatch() {
|
||||
if (compactionStateChangeLatch == null) compactionStateChangeLatch = new CountDownLatch(1);
|
||||
return compactionStateChangeLatch;
|
||||
}
|
||||
@Override
|
||||
public boolean compact(CompactionContext compaction, Store store) throws IOException {
|
||||
boolean ret = super.compact(compaction, store);
|
||||
if (ret) compactionStateChangeLatch.countDown();
|
||||
return ret;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Unfortunately, the easiest way to test this is to spin up a mini-cluster since we want to do
|
||||
* the usual compaction mechanism on the region, rather than going through the backdoor to the
|
||||
|
@ -212,6 +242,7 @@ public class TestRegionObserverScannerOpenHook {
|
|||
public void testRegionObserverCompactionTimeStacking() throws Exception {
|
||||
// setup a mini cluster so we can do a real compaction on a region
|
||||
Configuration conf = UTIL.getConfiguration();
|
||||
conf.setClass(HConstants.REGION_IMPL, CompactionCompletionNotifyingRegion.class, HRegion.class);
|
||||
conf.setInt("hbase.hstore.compaction.min", 2);
|
||||
UTIL.startMiniCluster();
|
||||
String tableName = "testRegionObserverCompactionTimeStacking";
|
||||
|
@ -239,7 +270,9 @@ public class TestRegionObserverScannerOpenHook {
|
|||
assertEquals("More than 1 region serving test table with 1 row", 1, regions.size());
|
||||
HRegion region = regions.get(0);
|
||||
admin.flush(region.getRegionName());
|
||||
|
||||
CountDownLatch latch = ((CompactionCompletionNotifyingRegion)region)
|
||||
.getCompactionStateChangeLatch();
|
||||
|
||||
// put another row and flush that too
|
||||
put = new Put(Bytes.toBytes("anotherrow"));
|
||||
put.add(A, A, A);
|
||||
|
@ -249,7 +282,7 @@ public class TestRegionObserverScannerOpenHook {
|
|||
|
||||
// run a compaction, which normally would should get rid of the data
|
||||
// wait for the compaction checker to complete
|
||||
Thread.sleep(1000);
|
||||
latch.await();
|
||||
// check both rows to ensure that they aren't there
|
||||
Get get = new Get(ROW);
|
||||
Result r = table.get(get);
|
||||
|
|
Loading…
Reference in New Issue