increase test time out and general clean up

This commit is contained in:
Parag Jain 2016-02-03 13:08:10 -06:00
parent a737877680
commit 9002548eeb
1 changed files with 27 additions and 24 deletions

View File

@ -61,6 +61,7 @@ import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
@ -68,7 +69,6 @@ import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
/**
@ -262,19 +262,24 @@ public class DruidCoordinatorTest extends CuratorTestBase
EasyMock.verify(metadataRuleManager);
}
@Test(timeout = 1500L)
@Test(timeout = 60_000L)
public void testCoordinatorRun() throws Exception{
String dataSource = "dataSource1";
String tier= "hot";
// Setup MetadataRuleManager
Rule foreverLoadRule = new ForeverLoadRule(ImmutableMap.of(tier, 2));
EasyMock.expect(metadataRuleManager.getRulesWithDefault(EasyMock.anyString()))
.andReturn(ImmutableList.of(foreverLoadRule)).atLeastOnce();
metadataRuleManager.stop();
EasyMock.expectLastCall().once();
EasyMock.replay(metadataRuleManager);
// Setup MetadataSegmentManager
DruidDataSource[] druidDataSources = {
new DruidDataSource(dataSource, new HashMap())
new DruidDataSource(dataSource, Collections.<String, String>emptyMap())
};
final DataSegment dataSegment = new DataSegment(dataSource, new Interval("2010-01-01/P1D"), "v1", null, null, null, null, 0x9, 0);
druidDataSources[0].addSegment("0", dataSegment);
@ -288,31 +293,26 @@ public class DruidCoordinatorTest extends CuratorTestBase
EasyMock.expect(immutableDruidDataSource.getSegments())
.andReturn(ImmutableSet.of(dataSegment)).atLeastOnce();
EasyMock.replay(immutableDruidDataSource);
// Setup ServerInventoryView
druidServer = new DruidServer("server1", "localhost", 5L, "historical", tier, 0);
loadManagementPeons.put("server1", loadQueuePeon);
EasyMock.expect(serverInventoryView.getInventory()).andReturn(
ImmutableList.of(druidServer)
).atLeastOnce();
try {
serverInventoryView.start();
EasyMock.expectLastCall().atLeastOnce();
}catch (Exception ex){
throw ex;
}
serverInventoryView.start();
EasyMock.expectLastCall().atLeastOnce();
EasyMock.expect(serverInventoryView.isStarted()).andReturn(true).anyTimes();
serverInventoryView.stop();
EasyMock.expectLastCall().once();
EasyMock.replay(serverInventoryView);
coordinator.start();
// Wait for this coordinator to become leader
try {
leaderAnnouncerLatch.await();
}
catch (InterruptedException ex) {
throw ex;
}
leaderAnnouncerLatch.await();
// This coordinator should be leader by now
Assert.assertTrue(coordinator.isLeader());
Assert.assertEquals(druidNode.getHostAndPort(), coordinator.getCurrentLeader());
@ -335,14 +335,10 @@ public class DruidCoordinatorTest extends CuratorTestBase
}
}
);
try {
pathChildrenCache.start();
}
catch (Exception ex) {
throw ex;
}
pathChildrenCache.start();
assignSegmentLatch.await();
Assert.assertTrue(assignSegmentLatch.await(2, TimeUnit.SECONDS));
Assert.assertEquals(ImmutableMap.of(dataSource, 100.0), coordinator.getLoadStatus());
curator.delete().guaranteed().forPath(ZKPaths.makePath(LOADPATH, dataSegment.getIdentifier()));
// Wait for coordinator thread to run so that replication status is updated
@ -358,7 +354,11 @@ public class DruidCoordinatorTest extends CuratorTestBase
}
// wait historical data to be updated
Thread.sleep(druidCoordinatorConfig.getCoordinatorPeriod().getMillis());
long startMillis = System.currentTimeMillis();
long coordinatorRunPeriodMillis = druidCoordinatorConfig.getCoordinatorPeriod().getMillis();
while (System.currentTimeMillis() - startMillis < coordinatorRunPeriodMillis) {
Thread.sleep(100);
}
Map<String, CountingMap<String>> replicationStatus = coordinator.getReplicationStatus();
Assert.assertNotNull(replicationStatus);
@ -371,10 +371,13 @@ public class DruidCoordinatorTest extends CuratorTestBase
// Simulated the adding of segment to druidServer during SegmentChangeRequestLoad event
// The load rules asks for 2 replicas, therefore 1 replica should still be pending
Assert.assertEquals(1L, dataSourceMap.get(dataSource).get());
coordinator.stop();
leaderUnannouncerLatch.await();
Assert.assertFalse(coordinator.isLeader());
Assert.assertNull(coordinator.getCurrentLeader());
EasyMock.verify(serverInventoryView);
EasyMock.verify(metadataRuleManager);
}