HBASE-24479: Deflake TestCompaction#testStopStartCompaction (#1820)

Polling of active compaction count is racy. Tightened the asserts
to be more reliable.

Signed-off-by: Reid Chan <reidchan@apache.org>
This commit is contained in:
Bharath Vissapragada 2020-05-31 22:45:31 -07:00
parent 4f49a96258
commit f5b90fcc44
No known key found for this signature in database
GPG Key ID: 18AE42A0B5A93FA7
1 changed files with 16 additions and 8 deletions

View File

@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestCase; import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
@ -448,7 +449,7 @@ public class TestCompaction {
// setup a compact/split thread on a mock server // setup a compact/split thread on a mock server
HRegionServer mockServer = Mockito.mock(HRegionServer.class); HRegionServer mockServer = Mockito.mock(HRegionServer.class);
Mockito.when(mockServer.getConfiguration()).thenReturn(r.getBaseConf()); Mockito.when(mockServer.getConfiguration()).thenReturn(r.getBaseConf());
CompactSplit thread = new CompactSplit(mockServer); final CompactSplit thread = new CompactSplit(mockServer);
Mockito.when(mockServer.getCompactSplitThread()).thenReturn(thread); Mockito.when(mockServer.getCompactSplitThread()).thenReturn(thread);
// setup a region/store with some files // setup a region/store with some files
HStore store = r.getStore(COLUMN_FAMILY); HStore store = r.getStore(COLUMN_FAMILY);
@ -459,19 +460,26 @@ public class TestCompaction {
thread.switchCompaction(false); thread.switchCompaction(false);
thread.requestCompaction(r, store, "test", Store.PRIORITY_USER, thread.requestCompaction(r, store, "test", Store.PRIORITY_USER,
CompactionLifeCycleTracker.DUMMY, null); CompactionLifeCycleTracker.DUMMY, null);
assertEquals(false, thread.isCompactionsEnabled()); assertFalse(thread.isCompactionsEnabled());
int longCompactions = thread.getLongCompactions().getActiveCount(); int longCompactions = thread.getLongCompactions().getActiveCount();
int shortCompactions = thread.getShortCompactions().getActiveCount(); int shortCompactions = thread.getShortCompactions().getActiveCount();
assertEquals("longCompactions=" + longCompactions + "," + assertEquals("longCompactions=" + longCompactions + "," +
"shortCompactions=" + shortCompactions, 0, longCompactions + shortCompactions); "shortCompactions=" + shortCompactions, 0, longCompactions + shortCompactions);
thread.switchCompaction(true); thread.switchCompaction(true);
assertEquals(true, thread.isCompactionsEnabled()); assertTrue(thread.isCompactionsEnabled());
// Make sure no compactions have run.
assertEquals(0, thread.getLongCompactions().getCompletedTaskCount() +
thread.getShortCompactions().getCompletedTaskCount());
// Request a compaction and make sure it is submitted successfully.
thread.requestCompaction(r, store, "test", Store.PRIORITY_USER, thread.requestCompaction(r, store, "test", Store.PRIORITY_USER,
CompactionLifeCycleTracker.DUMMY, null); CompactionLifeCycleTracker.DUMMY, null);
longCompactions = thread.getLongCompactions().getActiveCount(); // Wait until the compaction finishes.
shortCompactions = thread.getShortCompactions().getActiveCount(); Waiter.waitFor(UTIL.getConfiguration(), 5000,
assertEquals("longCompactions=" + longCompactions + "," + (Waiter.Predicate<Exception>) () -> thread.getLongCompactions().getCompletedTaskCount() +
"shortCompactions=" + shortCompactions, 1, longCompactions + shortCompactions); thread.getShortCompactions().getCompletedTaskCount() == 1);
// Make sure there are no compactions running.
assertEquals(0, thread.getLongCompactions().getActiveCount()
+ thread.getShortCompactions().getActiveCount());
} }
@Test public void testInterruptingRunningCompactions() throws Exception { @Test public void testInterruptingRunningCompactions() throws Exception {