diff --git a/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java b/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java index a1c1195..ff2c6d3 100644 --- a/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java +++ b/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java @@ -31,6 +31,7 @@ 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.Coprocessor; import org.apache.hadoop.hbase.HBaseConfiguration; @@ -54,10 +55,12 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer; 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.CompactionRequest; +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; @@ -146,6 +149,7 @@ public class TestRegionObserverScannerOpenHook { } } + HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf, byte[]... families) throws IOException { HTableDescriptor htd = new HTableDescriptor(tableName); @@ -213,6 +217,31 @@ public class TestRegionObserverScannerOpenHook { + r, r.list()); } + /* + * Custom HRegion which uses CountDownLatch to signal the completion of compaction + */ + public static class CompactionCompletionNotifyingRegion extends HRegion { + private static volatile CountDownLatch compactionStateChangeLatch = null; + + 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(CompactionRequest cr) throws IOException { + boolean ret = super.compact(cr); + 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 @@ -223,6 +252,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"; @@ -250,6 +280,8 @@ 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")); @@ -260,8 +292,7 @@ public class TestRegionObserverScannerOpenHook { // run a compaction, which normally would should get rid of the data Store s = region.getStores().get(A); - CountDownLatch latch = new CountDownLatch(1); - WaitableCompactionRequest request = new WaitableCompactionRequest(region, s, latch); + CompactionRequest request = new CompactionRequest(region, s, Store.PRIORITY_USER); rs.compactSplitThread.requestCompaction(region, s, "compact for testRegionObserverCompactionTimeStacking", Store.PRIORITY_USER, request); // wait for the compaction to complete @@ -283,27 +314,4 @@ public class TestRegionObserverScannerOpenHook { table.close(); UTIL.shutdownMiniCluster(); } - - /** - * A simple compaction on which you can wait for the passed in latch until the compaction finishes - * (either successfully or if it failed). - */ - public static class WaitableCompactionRequest extends CompactionRequest { - private CountDownLatch done; - - /** - * Constructor for a custom compaction. Uses the setXXX methods to update the state of the - * compaction before being used. - */ - public WaitableCompactionRequest(HRegion region, Store store, CountDownLatch finished) { - super(region, store, Store.PRIORITY_USER); - this.done = finished; - } - - @Override - public void finishRequest() { - super.finishRequest(); - this.done.countDown(); - } - } } \ No newline at end of file