improved coordinator and datasourceresource tests

This commit is contained in:
Parag Jain 2015-06-23 00:12:33 -05:00
parent 6763e3780a
commit 58cc3954ff
2 changed files with 235 additions and 70 deletions

View File

@ -81,13 +81,16 @@ public class DruidCoordinatorTest extends CuratorTestBase
private ConcurrentMap<String, LoadQueuePeon> loadManagementPeons; private ConcurrentMap<String, LoadQueuePeon> loadManagementPeons;
private LoadQueuePeon loadQueuePeon; private LoadQueuePeon loadQueuePeon;
private MetadataRuleManager metadataRuleManager; private MetadataRuleManager metadataRuleManager;
private SegmentReplicantLookup segmentReplicantLookup;
private CountDownLatch leaderAnnouncerLatch; private CountDownLatch leaderAnnouncerLatch;
private CountDownLatch leaderUnannouncerLatch;
private PathChildrenCache pathChildrenCache; private PathChildrenCache pathChildrenCache;
private DruidCoordinatorConfig druidCoordinatorConfig; private DruidCoordinatorConfig druidCoordinatorConfig;
private ObjectMapper objectMapper; private ObjectMapper objectMapper;
private JacksonConfigManager configManager; private JacksonConfigManager configManager;
private DruidNode druidNode;
private static final String LOADPATH = "/druid/loadqueue/localhost:1234"; private static final String LOADPATH = "/druid/loadqueue/localhost:1234";
private static final long COORDINATOR_START_DELAY = 1;
private static final long COORDINATOR_PERIOD = 100;
@Before @Before
public void setUp() throws Exception public void setUp() throws Exception
@ -108,9 +111,8 @@ public class DruidCoordinatorTest extends CuratorTestBase
setupServerAndCurator(); setupServerAndCurator();
curator.start(); curator.start();
curator.create().creatingParentsIfNeeded().forPath(LOADPATH); curator.create().creatingParentsIfNeeded().forPath(LOADPATH);
segmentReplicantLookup = null;
objectMapper = new DefaultObjectMapper(); objectMapper = new DefaultObjectMapper();
druidCoordinatorConfig = new TestDruidCoordinatorConfig(new Duration(1), new Duration(6000), null, null, null, false, false); druidCoordinatorConfig = new TestDruidCoordinatorConfig(new Duration(COORDINATOR_START_DELAY), new Duration(COORDINATOR_PERIOD), null, null, null, false, false);
pathChildrenCache = new PathChildrenCache(curator, LOADPATH, true, true, Execs.singleThreaded("coordinator_test_path_children_cache-%d")); pathChildrenCache = new PathChildrenCache(curator, LOADPATH, true, true, Execs.singleThreaded("coordinator_test_path_children_cache-%d"));
loadQueuePeon = new LoadQueuePeon( loadQueuePeon = new LoadQueuePeon(
curator, curator,
@ -120,6 +122,7 @@ public class DruidCoordinatorTest extends CuratorTestBase
Execs.singleThreaded("coordinator_test_load_queue_peon-%d"), Execs.singleThreaded("coordinator_test_load_queue_peon-%d"),
druidCoordinatorConfig druidCoordinatorConfig
); );
druidNode = new DruidNode("hey", "what", 1234);
loadManagementPeons = new MapMaker().makeMap(); loadManagementPeons = new MapMaker().makeMap();
scheduledExecutorFactory = new ScheduledExecutorFactory() scheduledExecutorFactory = new ScheduledExecutorFactory()
{ {
@ -130,6 +133,7 @@ public class DruidCoordinatorTest extends CuratorTestBase
} }
}; };
leaderAnnouncerLatch = new CountDownLatch(1); leaderAnnouncerLatch = new CountDownLatch(1);
leaderUnannouncerLatch = new CountDownLatch(1);
coordinator = new DruidCoordinator( coordinator = new DruidCoordinator(
druidCoordinatorConfig, druidCoordinatorConfig,
new ZkPathsConfig() new ZkPathsConfig()
@ -157,8 +161,14 @@ public class DruidCoordinatorTest extends CuratorTestBase
// count down when this coordinator becomes the leader // count down when this coordinator becomes the leader
leaderAnnouncerLatch.countDown(); leaderAnnouncerLatch.countDown();
} }
@Override
public void unannounce(DruidNode node)
{
leaderUnannouncerLatch.countDown();
}
}, },
new DruidNode("hey", "what", 1234), druidNode,
loadManagementPeons loadManagementPeons
); );
} }
@ -235,19 +245,21 @@ public class DruidCoordinatorTest extends CuratorTestBase
EasyMock.verify(metadataRuleManager); EasyMock.verify(metadataRuleManager);
} }
@Test @Test(timeout = 1500L)
public void testCoordinatorRun() throws Exception{ public void testCoordinatorRun() throws Exception{
String dataSource = "dataSource1"; String dataSource = "dataSource1";
String tier= "hot"; String tier= "hot";
Rule foreverLoadRule = new ForeverLoadRule(ImmutableMap.of(tier, 2)); Rule foreverLoadRule = new ForeverLoadRule(ImmutableMap.of(tier, 2));
EasyMock.expect(metadataRuleManager.getRulesWithDefault(EasyMock.anyString())) EasyMock.expect(metadataRuleManager.getRulesWithDefault(EasyMock.anyString()))
.andReturn(ImmutableList.of(foreverLoadRule)).atLeastOnce(); .andReturn(ImmutableList.of(foreverLoadRule)).atLeastOnce();
metadataRuleManager.stop();
EasyMock.expectLastCall().once();
EasyMock.replay(metadataRuleManager); EasyMock.replay(metadataRuleManager);
DruidDataSource[] druidDataSources = { DruidDataSource[] druidDataSources = {
new DruidDataSource(dataSource, new HashMap()) new DruidDataSource(dataSource, new HashMap())
}; };
DataSegment dataSegment = new DataSegment(dataSource, new Interval("2010-01-01/P1D"), "v1", null, null, null, null, 0x9, 0); final DataSegment dataSegment = new DataSegment(dataSource, new Interval("2010-01-01/P1D"), "v1", null, null, null, null, 0x9, 0);
druidDataSources[0].addSegment("0", dataSegment); druidDataSources[0].addSegment("0", dataSegment);
EasyMock.expect(databaseSegmentManager.isStarted()).andReturn(true).anyTimes(); EasyMock.expect(databaseSegmentManager.isStarted()).andReturn(true).anyTimes();
@ -259,17 +271,7 @@ public class DruidCoordinatorTest extends CuratorTestBase
EasyMock.expect(immutableDruidDataSource.getSegments()) EasyMock.expect(immutableDruidDataSource.getSegments())
.andReturn(ImmutableSet.of(dataSegment)).atLeastOnce(); .andReturn(ImmutableSet.of(dataSegment)).atLeastOnce();
EasyMock.replay(immutableDruidDataSource); EasyMock.replay(immutableDruidDataSource);
EasyMock.expect(druidServer.toImmutableDruidServer()).andReturn( druidServer = new DruidServer("server1", "localhost", 5L, "historical", tier, 0);
new ImmutableDruidServer(
new DruidServerMetadata("server1", "localhost", 5L, "historical", tier, 0),
1L,
ImmutableMap.<String, ImmutableDruidDataSource>of(),
ImmutableMap.<String, DataSegment>of()
)
).atLeastOnce();
EasyMock.expect(druidServer.isAssignable()).andReturn(true).anyTimes();
EasyMock.expect(druidServer.getName()).andReturn("server1").anyTimes();
EasyMock.replay(druidServer);
loadManagementPeons.put("server1", loadQueuePeon); loadManagementPeons.put("server1", loadQueuePeon);
EasyMock.expect(serverInventoryView.getInventory()).andReturn( EasyMock.expect(serverInventoryView.getInventory()).andReturn(
@ -282,18 +284,21 @@ public class DruidCoordinatorTest extends CuratorTestBase
throw ex; throw ex;
} }
EasyMock.expect(serverInventoryView.isStarted()).andReturn(true).anyTimes(); EasyMock.expect(serverInventoryView.isStarted()).andReturn(true).anyTimes();
serverInventoryView.stop();
EasyMock.expectLastCall().once();
EasyMock.replay(serverInventoryView); EasyMock.replay(serverInventoryView);
coordinator.start(); coordinator.start();
// Wait for this coordinator to become leader // Wait for this coordinator to become leader
try { try {
Assert.assertTrue(leaderAnnouncerLatch.await(4, TimeUnit.SECONDS)); leaderAnnouncerLatch.await();
} }
catch (InterruptedException ex) { catch (InterruptedException ex) {
throw ex; throw ex;
} }
// This coordinator should be leader by now // This coordinator should be leader by now
Assert.assertTrue(coordinator.isLeader()); Assert.assertTrue(coordinator.isLeader());
Assert.assertEquals(druidNode.getHostAndPort(), coordinator.getCurrentLeader());
final CountDownLatch assignSegmentLatch = new CountDownLatch(1); final CountDownLatch assignSegmentLatch = new CountDownLatch(1);
pathChildrenCache.getListenable().addListener( pathChildrenCache.getListenable().addListener(
@ -306,6 +311,8 @@ public class DruidCoordinatorTest extends CuratorTestBase
{ {
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 //Coordinator should try to assign segment to druidServer historical
//Simulate historical loading segment
druidServer.addDataSegment(dataSegment.getIdentifier(), dataSegment);
assignSegmentLatch.countDown(); assignSegmentLatch.countDown();
} }
} }
@ -317,20 +324,33 @@ public class DruidCoordinatorTest extends CuratorTestBase
catch (Exception ex) { catch (Exception ex) {
throw ex; throw ex;
} }
Assert.assertTrue(assignSegmentLatch.await(4, TimeUnit.SECONDS));
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())); curator.delete().guaranteed().forPath(ZKPaths.makePath(LOADPATH, dataSegment.getIdentifier()));
// Wait for coordinator thread to run so that replication status is updated
while (coordinator.getSegmentAvailability().snapshot().get(dataSource) != 0) {
Thread.sleep(50);
}
Map segmentAvailability = coordinator.getSegmentAvailability().snapshot();
Assert.assertEquals(1, segmentAvailability.size());
Assert.assertEquals(0l, segmentAvailability.get(dataSource));
Map<String, CountingMap<String>> replicationStatus = coordinator.getReplicationStatus(); Map<String, CountingMap<String>> replicationStatus = coordinator.getReplicationStatus();
Assert.assertNotNull(replicationStatus); Assert.assertNotNull(replicationStatus);
Assert.assertEquals(1, replicationStatus.entrySet().size()); Assert.assertEquals(1, replicationStatus.entrySet().size());
CountingMap<String> dataSourceMap = replicationStatus.get(tier); CountingMap<String> dataSourceMap = replicationStatus.get(tier);
Assert.assertNotNull(dataSourceMap); Assert.assertNotNull(dataSourceMap);
Assert.assertEquals(1, dataSourceMap.size()); Assert.assertEquals(1, dataSourceMap.size());
Assert.assertNotNull(dataSourceMap.get(dataSource)); Assert.assertNotNull(dataSourceMap.get(dataSource));
// ServerInventoryView and historical DruidServer are mocked // Simulated the adding of segment to druidServer during SegmentChangeRequestLoad event
// so the historical will never announce the segment and thus server inventory view will not be updated // The load rules asks for 2 replicas, therefore 1 replica should still be pending
// The load rules asks for 2 replicas, therefore 2 replicas will still be pending Assert.assertEquals(1l, dataSourceMap.get(dataSource).get());
Assert.assertEquals(2l, dataSourceMap.get(dataSource).get()); coordinator.stop();
EasyMock.verify(druidServer); leaderUnannouncerLatch.await();
Assert.assertFalse(coordinator.isLeader());
Assert.assertNull(coordinator.getCurrentLeader());
EasyMock.verify(serverInventoryView); EasyMock.verify(serverInventoryView);
EasyMock.verify(metadataRuleManager); EasyMock.verify(metadataRuleManager);
} }

View File

@ -20,6 +20,7 @@
package io.druid.server.http; package io.druid.server.http;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import io.druid.client.DruidDataSource; import io.druid.client.DruidDataSource;
import io.druid.client.DruidServer; import io.druid.client.DruidServer;
import io.druid.client.InventoryView; import io.druid.client.InventoryView;
@ -27,6 +28,7 @@ import io.druid.timeline.DataSegment;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import javax.ws.rs.core.Response; import javax.ws.rs.core.Response;
@ -35,21 +37,71 @@ import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.TreeMap;
import java.util.TreeSet;
public class DatasourcesResourceTest public class DatasourcesResourceTest
{ {
private InventoryView inventoryView;
private DruidServer server;
private List<DruidDataSource> listDataSources;
private List<DataSegment> dataSegmentList;
@Before
public void setUp()
{
inventoryView = EasyMock.createStrictMock(InventoryView.class);
server = EasyMock.createStrictMock(DruidServer.class);
dataSegmentList = new ArrayList<>();
dataSegmentList.add(
new DataSegment(
"datasource1",
new Interval("2010-01-01/P1D"),
null,
null,
null,
null,
null,
0x9,
0
)
);
dataSegmentList.add(
new DataSegment(
"datasource1",
new Interval("2010-01-22/P1D"),
null,
null,
null,
null,
null,
0x9,
0
)
);
dataSegmentList.add(
new DataSegment(
"datasource2",
new Interval("2010-01-01/P1D"),
null,
null,
null,
null,
null,
0x9,
0
)
);
listDataSources = new ArrayList<>();
listDataSources.add(new DruidDataSource("datasource1", new HashMap()).addSegment("part1", dataSegmentList.get(0)));
listDataSources.add(new DruidDataSource("datasource2", new HashMap()).addSegment("part1", dataSegmentList.get(1)));
}
@Test @Test
public void testGetFullQueryableDataSources() throws Exception public void testGetFullQueryableDataSources() throws Exception
{ {
InventoryView inventoryView = EasyMock.createStrictMock(InventoryView.class);
DruidServer server = EasyMock.createStrictMock(DruidServer.class);
DruidDataSource[] druidDataSources = {
new DruidDataSource("datasource1", new HashMap()),
new DruidDataSource("datasource2", new HashMap())
};
EasyMock.expect(server.getDataSources()).andReturn( EasyMock.expect(server.getDataSources()).andReturn(
ImmutableList.of(druidDataSources[0], druidDataSources[1]) ImmutableList.of(listDataSources.get(0), listDataSources.get(1))
).atLeastOnce(); ).atLeastOnce();
EasyMock.expect(inventoryView.getInventory()).andReturn( EasyMock.expect(inventoryView.getInventory()).andReturn(
ImmutableList.of(server) ImmutableList.of(server)
@ -62,7 +114,7 @@ public class DatasourcesResourceTest
result.toArray(resultantDruidDataSources); result.toArray(resultantDruidDataSources);
Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(200, response.getStatus());
Assert.assertEquals(2, resultantDruidDataSources.length); Assert.assertEquals(2, resultantDruidDataSources.length);
Assert.assertArrayEquals(druidDataSources, resultantDruidDataSources); Assert.assertArrayEquals(listDataSources.toArray(), resultantDruidDataSources);
response = datasourcesResource.getQueryableDataSources(null, null); response = datasourcesResource.getQueryableDataSources(null, null);
List<String> result1 = (List<String>) response.getEntity(); List<String> result1 = (List<String>) response.getEntity();
@ -76,27 +128,6 @@ public class DatasourcesResourceTest
@Test @Test
public void testGetSimpleQueryableDataSources() throws Exception public void testGetSimpleQueryableDataSources() throws Exception
{ {
InventoryView inventoryView = EasyMock.createStrictMock(InventoryView.class);
DruidServer server = EasyMock.createStrictMock(DruidServer.class);
List<Map<String, Object>> input = new ArrayList(2);
HashMap<String, Object> dataSourceProp1 = new HashMap<>();
dataSourceProp1.put("name", "datasource1");
dataSourceProp1.put("partitionName", "partition");
dataSourceProp1.put("datasegment",
new DataSegment("datasource1", new Interval("2010-01-01/P1D"), null, null, null, null, null, 0x9, 0));
HashMap<String, Object> dataSourceProp2 = new HashMap<>();
dataSourceProp2.put("name", "datasource2");
dataSourceProp2.put("partitionName", "partition");
dataSourceProp2.put("datasegment",
new DataSegment("datasource2", new Interval("2010-01-01/P1D"), null, null, null, null, null, 0x9, 0));
input.add(dataSourceProp1);
input.add(dataSourceProp2);
List<DruidDataSource> listDataSources = new ArrayList<>();
for(Map<String, Object> entry : input){
listDataSources.add(new DruidDataSource(entry.get("name").toString(), new HashMap())
.addSegment(entry.get("partitionName").toString(), (DataSegment)entry.get("datasegment")));
}
EasyMock.expect(server.getDataSources()).andReturn( EasyMock.expect(server.getDataSources()).andReturn(
listDataSources listDataSources
).atLeastOnce(); ).atLeastOnce();
@ -119,7 +150,7 @@ public class DatasourcesResourceTest
List<Map<String, Object>> results = (List<Map<String, Object>>) response.getEntity(); List<Map<String, Object>> results = (List<Map<String, Object>>) response.getEntity();
int index = 0; int index = 0;
for (Map<String, Object> entry : results) { for (Map<String, Object> entry : results) {
Assert.assertEquals(input.get(index).get("name"), entry.get("name").toString()); Assert.assertEquals(listDataSources.get(index).getName(), entry.get("name").toString());
Assert.assertTrue(((Map) ((Map) entry.get("properties")).get("tiers")).containsKey(null)); Assert.assertTrue(((Map) ((Map) entry.get("properties")).get("tiers")).containsKey(null));
Assert.assertNotNull((((Map) entry.get("properties")).get("segments"))); Assert.assertNotNull((((Map) entry.get("properties")).get("segments")));
Assert.assertEquals(1, ((Map) ((Map) entry.get("properties")).get("segments")).get("count")); Assert.assertEquals(1, ((Map) ((Map) entry.get("properties")).get("segments")).get("count"));
@ -131,8 +162,6 @@ public class DatasourcesResourceTest
@Test @Test
public void testFullGetTheDataSource() throws Exception public void testFullGetTheDataSource() throws Exception
{ {
InventoryView inventoryView = EasyMock.createStrictMock(InventoryView.class);
DruidServer server = EasyMock.createStrictMock(DruidServer.class);
DruidDataSource dataSource1 = new DruidDataSource("datasource1", new HashMap()); DruidDataSource dataSource1 = new DruidDataSource("datasource1", new HashMap());
EasyMock.expect(server.getDataSource("datasource1")).andReturn( EasyMock.expect(server.getDataSource("datasource1")).andReturn(
dataSource1 dataSource1
@ -153,8 +182,6 @@ public class DatasourcesResourceTest
@Test @Test
public void testNullGetTheDataSource() throws Exception public void testNullGetTheDataSource() throws Exception
{ {
InventoryView inventoryView = EasyMock.createStrictMock(InventoryView.class);
DruidServer server = EasyMock.createStrictMock(DruidServer.class);
EasyMock.expect(server.getDataSource("none")).andReturn(null).atLeastOnce(); EasyMock.expect(server.getDataSource("none")).andReturn(null).atLeastOnce();
EasyMock.expect(inventoryView.getInventory()).andReturn( EasyMock.expect(inventoryView.getInventory()).andReturn(
ImmutableList.of(server) ImmutableList.of(server)
@ -169,11 +196,11 @@ public class DatasourcesResourceTest
@Test @Test
public void testSimpleGetTheDataSource() throws Exception public void testSimpleGetTheDataSource() throws Exception
{ {
InventoryView inventoryView = EasyMock.createStrictMock(InventoryView.class);
DruidServer server = EasyMock.createStrictMock(DruidServer.class);
DruidDataSource dataSource1 = new DruidDataSource("datasource1", new HashMap()); DruidDataSource dataSource1 = new DruidDataSource("datasource1", new HashMap());
dataSource1.addSegment("partition", dataSource1.addSegment(
new DataSegment("datasegment1", new Interval("2010-01-01/P1D"), null, null, null, null, null, 0x9, 0)); "partition",
new DataSegment("datasegment1", new Interval("2010-01-01/P1D"), null, null, null, null, null, 0x9, 0)
);
EasyMock.expect(server.getDataSource("datasource1")).andReturn( EasyMock.expect(server.getDataSource("datasource1")).andReturn(
dataSource1 dataSource1
).atLeastOnce(); ).atLeastOnce();
@ -193,4 +220,122 @@ public class DatasourcesResourceTest
Assert.assertNotNull(result.get("segments").get("maxTime").toString(), "2010-01-02T00:00:00.000Z"); Assert.assertNotNull(result.get("segments").get("maxTime").toString(), "2010-01-02T00:00:00.000Z");
EasyMock.verify(inventoryView, server); EasyMock.verify(inventoryView, server);
} }
@Test
public void testGetSegmentDataSourceIntervals()
{
server = new DruidServer("who", "host", 1234, "historical", "tier1", 0);
server.addDataSegment(dataSegmentList.get(0).getIdentifier(), dataSegmentList.get(0));
server.addDataSegment(dataSegmentList.get(1).getIdentifier(), dataSegmentList.get(1));
server.addDataSegment(dataSegmentList.get(2).getIdentifier(), dataSegmentList.get(2));
EasyMock.expect(inventoryView.getInventory()).andReturn(
ImmutableList.of(server)
).atLeastOnce();
EasyMock.replay(inventoryView);
List<Interval> expectedIntervals = new ArrayList<>();
expectedIntervals.add(new Interval("2010-01-22T00:00:00.000Z/2010-01-23T00:00:00.000Z"));
expectedIntervals.add(new Interval("2010-01-01T00:00:00.000Z/2010-01-02T00:00:00.000Z"));
DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null);
Response response = datasourcesResource.getSegmentDataSourceIntervals("invalidDataSource", null, null);
Assert.assertEquals(response.getEntity(), null);
response = datasourcesResource.getSegmentDataSourceIntervals("datasource1", null, null);
TreeSet<Interval> actualIntervals = (TreeSet) response.getEntity();
Assert.assertEquals(2, actualIntervals.size());
Assert.assertEquals(expectedIntervals.get(0), actualIntervals.first());
Assert.assertEquals(expectedIntervals.get(1), actualIntervals.last());
response = datasourcesResource.getSegmentDataSourceIntervals("datasource1", "simple", null);
TreeMap<Interval, Map<String, Object>> results = (TreeMap) response.getEntity();
Assert.assertEquals(2, results.size());
Assert.assertEquals(expectedIntervals.get(0), results.firstKey());
Assert.assertEquals(expectedIntervals.get(1), results.lastKey());
Assert.assertEquals(1, results.firstEntry().getValue().get("count"));
Assert.assertEquals(1, results.lastEntry().getValue().get("count"));
response = datasourcesResource.getSegmentDataSourceIntervals("datasource1", null, "full");
results = ((TreeMap<Interval, Map<String, Object>>) response.getEntity());
int i = 1;
for (Map.Entry<Interval, Map<String, Object>> entry : results.entrySet()) {
Assert.assertEquals(dataSegmentList.get(i).getInterval(), entry.getKey());
Assert.assertEquals(
dataSegmentList.get(i),
((Map<String, Object>) entry.getValue().get(dataSegmentList.get(i).getIdentifier())).get(
"metadata"
)
);
i--;
}
EasyMock.verify(inventoryView);
}
@Test
public void testGetSegmentDataSourceSpecificInterval()
{
server = new DruidServer("who", "host", 1234, "historical", "tier1", 0);
server.addDataSegment(dataSegmentList.get(0).getIdentifier(), dataSegmentList.get(0));
server.addDataSegment(dataSegmentList.get(1).getIdentifier(), dataSegmentList.get(1));
server.addDataSegment(dataSegmentList.get(2).getIdentifier(), dataSegmentList.get(2));
EasyMock.expect(inventoryView.getInventory()).andReturn(
ImmutableList.of(server)
).atLeastOnce();
EasyMock.replay(inventoryView);
DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null);
Response response = datasourcesResource.getSegmentDataSourceSpecificInterval(
"invalidDataSource",
"2010-01-01/P1D",
null,
null
);
Assert.assertEquals(null, response.getEntity());
response = datasourcesResource.getSegmentDataSourceSpecificInterval(
"datasource1",
"2010-03-01/P1D",
null,
null
); // interval not present in the datasource
Assert.assertEquals(ImmutableSet.of(), response.getEntity());
response = datasourcesResource.getSegmentDataSourceSpecificInterval("datasource1", "2010-01-01/P1D", null, null);
Assert.assertEquals(ImmutableSet.of(dataSegmentList.get(0).getIdentifier()), response.getEntity());
response = datasourcesResource.getSegmentDataSourceSpecificInterval("datasource1", "2010-01-01/P1M", null, null);
Assert.assertEquals(
ImmutableSet.of(dataSegmentList.get(1).getIdentifier(), dataSegmentList.get(0).getIdentifier()),
response.getEntity()
);
response = datasourcesResource.getSegmentDataSourceSpecificInterval(
"datasource1",
"2010-01-01/P1M",
"simple",
null
);
HashMap<Interval, Map<String, Object>> results = ((HashMap<Interval, Map<String, Object>>) response.getEntity());
Assert.assertEquals(2, results.size());
int i;
for (i = 0; i < 2; i++) {
Assert.assertTrue(results.containsKey(dataSegmentList.get(i).getInterval()));
Assert.assertEquals(1, (results.get(dataSegmentList.get(i).getInterval())).get("count"));
}
response = datasourcesResource.getSegmentDataSourceSpecificInterval("datasource1", "2010-01-01/P1M", null, "full");
TreeMap<Interval, Map<String, Object>> results1 = ((TreeMap<Interval, Map<String, Object>>) response.getEntity());
i = 1;
for (Map.Entry<Interval, Map<String, Object>> entry : results1.entrySet()) {
Assert.assertEquals(dataSegmentList.get(i).getInterval(), entry.getKey());
Assert.assertEquals(
dataSegmentList.get(i),
((Map<String, Object>) entry.getValue().get(dataSegmentList.get(i).getIdentifier())).get(
"metadata"
)
);
i--;
}
EasyMock.verify(inventoryView);
}
} }