You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bh...@apache.org on 2020/06/01 06:46:00 UTC
[hbase] branch branch-2.3 updated: HBASE-24479: Deflake
TestCompaction#testStopStartCompaction (#1820)
This is an automated email from the ASF dual-hosted git repository.
bharathv pushed a commit to branch branch-2.3
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/branch-2.3 by this push:
new c857ba1 HBASE-24479: Deflake TestCompaction#testStopStartCompaction (#1820)
c857ba1 is described below
commit c857ba116e3b86966eee69dfad7308b42e649adf
Author: Bharath Vissapragada <bh...@apache.org>
AuthorDate: Sun May 31 22:45:31 2020 -0700
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 <re...@apache.org>
---
.../hadoop/hbase/regionserver/TestCompaction.java | 24 ++++++++++++++--------
1 file changed, 16 insertions(+), 8 deletions(-)
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
index 06bab0b..cbc43f0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
@@ -51,6 +51,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;
@@ -369,7 +370,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());
- CompactSplit thread = new CompactSplit(mockServer);
+ final CompactSplit thread = new CompactSplit(mockServer);
Mockito.when(mockServer.getCompactSplitThread()).thenReturn(thread);
// setup a region/store with some files
HStore store = r.getStore(COLUMN_FAMILY);
@@ -380,19 +381,26 @@ public class TestCompaction {
thread.switchCompaction(false);
thread.requestCompaction(r, store, "test", Store.PRIORITY_USER,
CompactionLifeCycleTracker.DUMMY, null);
- assertEquals(false, thread.isCompactionsEnabled());
+ assertFalse(thread.isCompactionsEnabled());
int longCompactions = thread.getLongCompactions().getActiveCount();
int shortCompactions = thread.getShortCompactions().getActiveCount();
assertEquals("longCompactions=" + longCompactions + "," +
"shortCompactions=" + shortCompactions, 0, longCompactions + shortCompactions);
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,
- CompactionLifeCycleTracker.DUMMY, null);
- longCompactions = thread.getLongCompactions().getActiveCount();
- shortCompactions = thread.getShortCompactions().getActiveCount();
- assertEquals("longCompactions=" + longCompactions + "," +
- "shortCompactions=" + shortCompactions, 1, longCompactions + shortCompactions);
+ CompactionLifeCycleTracker.DUMMY, null);
+ // Wait until the compaction finishes.
+ Waiter.waitFor(UTIL.getConfiguration(), 5000,
+ (Waiter.Predicate<Exception>) () -> thread.getLongCompactions().getCompletedTaskCount() +
+ 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 {