From 85eacb255981445d56a90d9641ff1308eb587200 Mon Sep 17 00:00:00 2001 From: binlijin Date: Fri, 1 Nov 2019 16:23:02 +0800 Subject: [PATCH 1/2] HBASE-23241 TestExecutorService sometimes fail --- .../hadoop/hbase/executor/TestExecutorService.java | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/executor/TestExecutorService.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/executor/TestExecutorService.java index 205c6c6f0900..37cc0d22207b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/executor/TestExecutorService.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/executor/TestExecutorService.java @@ -231,9 +231,11 @@ public void testSnapshotHandlers() throws Exception { executorService.startExecutorService(ExecutorType.MASTER_SNAPSHOT_OPERATIONS, 1); CountDownLatch latch = new CountDownLatch(1); + final AtomicInteger counter = new AtomicInteger(0); executorService.submit(new EventHandler(server, EventType.C_M_SNAPSHOT_TABLE) { @Override public void process() throws IOException { + counter.incrementAndGet(); try { latch.await(); } catch (InterruptedException e) { @@ -242,9 +244,19 @@ public void process() throws IOException { } }); + // The EventHandler will increment counter when it starts. + int maxTries = 10; + int sleepInterval = 10; + int tries = 0; + while (counter.get() < 1 && tries < maxTries) { + LOG.info("Waiting for event handlers to start..."); + Thread.sleep(sleepInterval); + tries++; + } + int activeCount = executorService.getExecutor(ExecutorType.MASTER_SNAPSHOT_OPERATIONS) .getThreadPoolExecutor().getActiveCount(); - Assert.assertEquals(activeCount, 1); + Assert.assertEquals(1, activeCount); latch.countDown(); Waiter.waitFor(conf, 3000, () -> { int count = executorService.getExecutor(ExecutorType.MASTER_SNAPSHOT_OPERATIONS) From 31b0a5f82b92be46ae08433053aac801ababaf10 Mon Sep 17 00:00:00 2001 From: binlijin Date: Mon, 4 Nov 2019 12:08:16 +0800 Subject: [PATCH 2/2] fix comment --- .../hbase/executor/TestExecutorService.java | 17 +++++------------ 1 file changed, 5 insertions(+), 12 deletions(-) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/executor/TestExecutorService.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/executor/TestExecutorService.java index 37cc0d22207b..c93e951a4b25 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/executor/TestExecutorService.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/executor/TestExecutorService.java @@ -30,6 +30,7 @@ import java.io.StringWriter; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.conf.Configuration; @@ -231,11 +232,11 @@ public void testSnapshotHandlers() throws Exception { executorService.startExecutorService(ExecutorType.MASTER_SNAPSHOT_OPERATIONS, 1); CountDownLatch latch = new CountDownLatch(1); - final AtomicInteger counter = new AtomicInteger(0); + CountDownLatch waitForEventToStart = new CountDownLatch(1); executorService.submit(new EventHandler(server, EventType.C_M_SNAPSHOT_TABLE) { @Override public void process() throws IOException { - counter.incrementAndGet(); + waitForEventToStart.countDown(); try { latch.await(); } catch (InterruptedException e) { @@ -244,16 +245,8 @@ public void process() throws IOException { } }); - // The EventHandler will increment counter when it starts. - int maxTries = 10; - int sleepInterval = 10; - int tries = 0; - while (counter.get() < 1 && tries < maxTries) { - LOG.info("Waiting for event handlers to start..."); - Thread.sleep(sleepInterval); - tries++; - } - + //Wait EventHandler to start + waitForEventToStart.await(10, TimeUnit.SECONDS); int activeCount = executorService.getExecutor(ExecutorType.MASTER_SNAPSHOT_OPERATIONS) .getThreadPoolExecutor().getActiveCount(); Assert.assertEquals(1, activeCount);