Fix flaky tests in DruidCoordinatorTest (#10157)

* Fix flaky tests in DruidCoordinatorTest

* Imporve fail msg

* Fix flaky tests in DruidCoordinatorTest
This commit is contained in:
Maytas Monsereenusorn 2020-07-08 20:03:52 -07:00 committed by GitHub
parent 88d20a61a6
commit 54a8fb827d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
1 changed files with 6 additions and 12 deletions

View File

@ -103,12 +103,10 @@ public class DruidCoordinatorTest extends CuratorTestBase
private ObjectMapper objectMapper;
private DruidNode druidNode;
private LatchableServiceEmitter serviceEmitter = new LatchableServiceEmitter();
private boolean serverAddedCountExpected = true;
@Before
public void setUp() throws Exception
{
serverAddedCountExpected = true;
druidServer = EasyMock.createMock(DruidServer.class);
serverInventoryView = EasyMock.createMock(SingleServerInventoryView.class);
segmentsMetadataManager = EasyMock.createNiceMock(SegmentsMetadataManager.class);
@ -385,7 +383,6 @@ public class DruidCoordinatorTest extends CuratorTestBase
druidServer
);
assignSegmentLatch.await();
Assert.assertTrue(serverAddedCountExpected);
final CountDownLatch coordinatorRunLatch = new CountDownLatch(2);
serviceEmitter.latch = coordinatorRunLatch;
@ -486,7 +483,6 @@ public class DruidCoordinatorTest extends CuratorTestBase
final CountDownLatch assignSegmentLatchCold = createCountDownLatchAndSetPathChildrenCacheListenerWithLatch(1, pathChildrenCacheCold, dataSegments, coldServer);
assignSegmentLatchHot.await();
assignSegmentLatchCold.await();
Assert.assertTrue(serverAddedCountExpected);
final CountDownLatch coordinatorRunLatch = new CountDownLatch(2);
serviceEmitter.latch = coordinatorRunLatch;
@ -646,7 +642,6 @@ public class DruidCoordinatorTest extends CuratorTestBase
assignSegmentLatchBroker1.await();
assignSegmentLatchBroker2.await();
assignSegmentLatchPeon.await();
Assert.assertTrue(serverAddedCountExpected);
final CountDownLatch coordinatorRunLatch = new CountDownLatch(2);
serviceEmitter.latch = coordinatorRunLatch;
@ -679,16 +674,15 @@ public class DruidCoordinatorTest extends CuratorTestBase
pathChildrenCache.getListenable().addListener(
(CuratorFramework client, PathChildrenCacheEvent event) -> {
if (CuratorUtils.isChildAdded(event)) {
if (countDownLatch.getCount() > 0) {
DataSegment segment = findSegmentRelatedToCuratorEvent(segments, event);
if (segment != null) {
DataSegment segment = findSegmentRelatedToCuratorEvent(segments, event);
if (segment != null && server.getSegment(segment.getId()) == null) {
if (countDownLatch.getCount() > 0) {
server.addDataSegment(segment);
curator.delete().guaranteed().forPath(event.getData().getPath());
countDownLatch.countDown();
} else {
Assert.fail("The segment path " + event.getData().getPath() + " is not expected");
}
countDownLatch.countDown();
} else {
// The segment is assigned to the server more times than expected
serverAddedCountExpected = false;
}
}
}