HBASE-24479: Deflake TestCompaction#testStopStartCompaction (#1820)
HBASE-24479: Deflake TestCompaction#testStopStartCompaction Polling of active compaction count is racy. Tightened the asserts to be more reliable. Reid Chan <reidchan@apache.org>
This commit is contained in:
parent
3c138845d9
commit
6bca1dd1c1
|
@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY;
|
|||
import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY_BYTES;
|
||||
import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
import static org.mockito.Matchers.any;
|
||||
|
@ -51,6 +52,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
|
|||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.Waiter;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
|
@ -361,7 +363,7 @@ public class TestCompaction {
|
|||
// setup a compact/split thread on a mock server
|
||||
HRegionServer mockServer = Mockito.mock(HRegionServer.class);
|
||||
Mockito.when(mockServer.getConfiguration()).thenReturn(r.getBaseConf());
|
||||
CompactSplitThread thread = new CompactSplitThread(mockServer);
|
||||
final CompactSplitThread thread = new CompactSplitThread(mockServer);
|
||||
Mockito.when(mockServer.getCompactSplitThread()).thenReturn(thread);
|
||||
// setup a region/store with some files
|
||||
Store store = r.getStore(COLUMN_FAMILY);
|
||||
|
@ -376,9 +378,23 @@ public class TestCompaction {
|
|||
.getActiveCount());
|
||||
thread.switchCompaction(true);
|
||||
assertEquals(true, thread.isCompactionsEnabled());
|
||||
thread.requestCompaction(r, store, "test", Store.PRIORITY_USER, new CompactionRequest(), null);
|
||||
assertEquals(1, thread.getLongCompactions().getActiveCount() + thread.getShortCompactions()
|
||||
.getActiveCount());
|
||||
// Make sure no compactions have run.
|
||||
assertEquals(0, thread.getLongCompactions().getCompletedTaskCount() +
|
||||
thread.getShortCompactions().getCompletedTaskCount());
|
||||
// Request a compaction and make sure it is submitted successfully.
|
||||
assertNotNull(thread.requestCompaction(r, store, "test", Store.PRIORITY_USER,
|
||||
new CompactionRequest(), null));
|
||||
// Wait until the compaction finishes.
|
||||
Waiter.waitFor(UTIL.getConfiguration(), 5000, new Waiter.Predicate<Exception>() {
|
||||
@Override
|
||||
public boolean evaluate() throws Exception {
|
||||
return thread.getLongCompactions().getCompletedTaskCount() +
|
||||
thread.getShortCompactions().getCompletedTaskCount() == 1;
|
||||
}
|
||||
});
|
||||
// Make sure there are no compactions running.
|
||||
assertEquals(0, thread.getLongCompactions().getActiveCount()
|
||||
+ thread.getShortCompactions().getActiveCount());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
Loading…
Reference in New Issue