HBASE-23241 TestExecutorService sometimes fail (#782)

Signed-off-by: Wellington Chevreuil <wchevreuil@apache.org>
This commit is contained in:
binlijin 2019-11-04 17:15:50 +08:00 committed by GitHub
parent 3ccfd50bd9
commit 90007b7e80
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
1 changed files with 6 additions and 1 deletions

View File

@ -30,6 +30,7 @@ import java.io.IOException;
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,9 +232,11 @@ public class TestExecutorService {
executorService.startExecutorService(ExecutorType.MASTER_SNAPSHOT_OPERATIONS, 1);
CountDownLatch latch = new CountDownLatch(1);
CountDownLatch waitForEventToStart = new CountDownLatch(1);
executorService.submit(new EventHandler(server, EventType.C_M_SNAPSHOT_TABLE) {
@Override
public void process() throws IOException {
waitForEventToStart.countDown();
try {
latch.await();
} catch (InterruptedException e) {
@ -242,9 +245,11 @@ public class TestExecutorService {
}
});
//Wait EventHandler to start
waitForEventToStart.await(10, TimeUnit.SECONDS);
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)