mirror of https://github.com/apache/druid.git
Make DruidCoordinatorTest fail fast (#4142)
This commit is contained in:
parent
81ba0224c5
commit
b166e13d2b
|
@ -26,7 +26,6 @@ import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.MapMaker;
|
import com.google.common.collect.MapMaker;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
|
|
||||||
import io.druid.client.DruidDataSource;
|
import io.druid.client.DruidDataSource;
|
||||||
import io.druid.client.DruidServer;
|
import io.druid.client.DruidServer;
|
||||||
import io.druid.client.ImmutableDruidDataSource;
|
import io.druid.client.ImmutableDruidDataSource;
|
||||||
|
@ -77,7 +76,6 @@ import java.util.concurrent.atomic.AtomicReference;
|
||||||
public class DruidCoordinatorTest extends CuratorTestBase
|
public class DruidCoordinatorTest extends CuratorTestBase
|
||||||
{
|
{
|
||||||
private DruidCoordinator coordinator;
|
private DruidCoordinator coordinator;
|
||||||
private LoadQueueTaskMaster taskMaster;
|
|
||||||
private MetadataSegmentManager databaseSegmentManager;
|
private MetadataSegmentManager databaseSegmentManager;
|
||||||
private SingleServerInventoryView serverInventoryView;
|
private SingleServerInventoryView serverInventoryView;
|
||||||
private ScheduledExecutorFactory scheduledExecutorFactory;
|
private ScheduledExecutorFactory scheduledExecutorFactory;
|
||||||
|
@ -101,7 +99,6 @@ public class DruidCoordinatorTest extends CuratorTestBase
|
||||||
@Before
|
@Before
|
||||||
public void setUp() throws Exception
|
public void setUp() throws Exception
|
||||||
{
|
{
|
||||||
taskMaster = EasyMock.createMock(LoadQueueTaskMaster.class);
|
|
||||||
druidServer = EasyMock.createMock(DruidServer.class);
|
druidServer = EasyMock.createMock(DruidServer.class);
|
||||||
serverInventoryView = EasyMock.createMock(SingleServerInventoryView.class);
|
serverInventoryView = EasyMock.createMock(SingleServerInventoryView.class);
|
||||||
databaseSegmentManager = EasyMock.createNiceMock(MetadataSegmentManager.class);
|
databaseSegmentManager = EasyMock.createNiceMock(MetadataSegmentManager.class);
|
||||||
|
@ -174,7 +171,7 @@ public class DruidCoordinatorTest extends CuratorTestBase
|
||||||
new NoopServiceEmitter(),
|
new NoopServiceEmitter(),
|
||||||
scheduledExecutorFactory,
|
scheduledExecutorFactory,
|
||||||
null,
|
null,
|
||||||
taskMaster,
|
null,
|
||||||
new NoopServiceAnnouncer(){
|
new NoopServiceAnnouncer(){
|
||||||
@Override
|
@Override
|
||||||
public void announce(DruidNode node)
|
public void announce(DruidNode node)
|
||||||
|
@ -328,10 +325,14 @@ public class DruidCoordinatorTest extends CuratorTestBase
|
||||||
) throws Exception
|
) throws Exception
|
||||||
{
|
{
|
||||||
if(pathChildrenCacheEvent.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)){
|
if(pathChildrenCacheEvent.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)){
|
||||||
//Coordinator should try to assign segment to druidServer historical
|
if (assignSegmentLatch.getCount() > 0) {
|
||||||
//Simulate historical loading segment
|
//Coordinator should try to assign segment to druidServer historical
|
||||||
druidServer.addDataSegment(dataSegment.getIdentifier(), dataSegment);
|
//Simulate historical loading segment
|
||||||
assignSegmentLatch.countDown();
|
druidServer.addDataSegment(dataSegment.getIdentifier(), dataSegment);
|
||||||
|
assignSegmentLatch.countDown();
|
||||||
|
} else {
|
||||||
|
Assert.fail("The same segment is assigned to the same server multiple times");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -371,9 +372,7 @@ public class DruidCoordinatorTest extends CuratorTestBase
|
||||||
Assert.assertNotNull(dataSourceMap.get(dataSource));
|
Assert.assertNotNull(dataSourceMap.get(dataSource));
|
||||||
// Simulated the adding of segment to druidServer during SegmentChangeRequestLoad event
|
// Simulated the adding of segment to druidServer during SegmentChangeRequestLoad event
|
||||||
// The load rules asks for 2 replicas, therefore 1 replica should still be pending
|
// The load rules asks for 2 replicas, therefore 1 replica should still be pending
|
||||||
while(dataSourceMap.get(dataSource).get() != 1L) {
|
Assert.assertEquals(1L, dataSourceMap.get(dataSource).get());
|
||||||
Thread.sleep(50);
|
|
||||||
}
|
|
||||||
|
|
||||||
coordinator.stop();
|
coordinator.stop();
|
||||||
leaderUnannouncerLatch.await();
|
leaderUnannouncerLatch.await();
|
||||||
|
|
Loading…
Reference in New Issue