Index: hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java (revision 1503002) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java (working copy) @@ -21,14 +21,11 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; import java.io.IOException; -import java.util.Collection; 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.Path; @@ -56,9 +53,7 @@ import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; import org.apache.hadoop.hbase.regionserver.ScanType; import org.apache.hadoop.hbase.regionserver.Store; -import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreScanner; -import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -251,14 +246,8 @@ admin.flush(region.getRegionName()); // 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(s.getStorefiles(), latch); - rs.compactSplitThread.requestCompaction(region, s, - "compact for testRegionObserverCompactionTimeStacking", Store.PRIORITY_USER, request); - // wait for the compaction to complete - latch.await(); - + // wait for the compaction checker to complete + Thread.sleep(1000); // check both rows to ensure that they aren't there Get get = new Get(ROW); Result r = table.get(get); @@ -275,26 +264,4 @@ 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(Collection files, CountDownLatch finished) { - super(files); - this.done = finished; - } - - @Override - public void afterExecute() { - this.done.countDown(); - } - } } \ No newline at end of file Index: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java (revision 1503002) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java (working copy) @@ -392,6 +392,6 @@ public boolean needsCompaction(final Collection storeFiles, final List filesCompacting) { int numCandidates = storeFiles.size() - filesCompacting.size(); - return numCandidates > comConf.getMinFilesToCompact(); + return numCandidates >= comConf.getMinFilesToCompact(); } }