mirror of https://github.com/apache/druid.git
Coordinator - add intervals route.
This commit is contained in:
parent
25a0eb7ed5
commit
6de51e022d
|
@ -193,6 +193,24 @@ Returns all rules for a specified datasource and includes default datasource.
|
||||||
|
|
||||||
Returns audit history of rules for a specified datasource. default value of interval can be specified by setting `druid.audit.manager.auditHistoryMillis` (1 week if not configured) in coordinator runtime.properties
|
Returns audit history of rules for a specified datasource. default value of interval can be specified by setting `druid.audit.manager.auditHistoryMillis` (1 week if not configured) in coordinator runtime.properties
|
||||||
|
|
||||||
|
#### Intervals
|
||||||
|
|
||||||
|
* `/druid/coordinator/v1/intervals`
|
||||||
|
|
||||||
|
Returns all intervals for all datasources with total size and count.
|
||||||
|
|
||||||
|
* `/druid/coordinator/v1/intervals/{interval}`
|
||||||
|
|
||||||
|
Returns aggregated total size and count for all intervals that intersect given isointerval.
|
||||||
|
|
||||||
|
* `/druid/coordinator/v1/intervals/{interval}?simple`
|
||||||
|
|
||||||
|
Returns total size and count for each interval within given isointerval.
|
||||||
|
|
||||||
|
* `/druid/coordinator/v1/intervals/{interval}?full`
|
||||||
|
|
||||||
|
Returns total size and count for each datasource for each interval within given isointerval.
|
||||||
|
|
||||||
|
|
||||||
### POST
|
### POST
|
||||||
|
|
||||||
|
|
|
@ -27,12 +27,14 @@ import com.google.inject.Inject;
|
||||||
import com.metamx.common.MapUtils;
|
import com.metamx.common.MapUtils;
|
||||||
import com.metamx.common.Pair;
|
import com.metamx.common.Pair;
|
||||||
import com.metamx.common.guava.Comparators;
|
import com.metamx.common.guava.Comparators;
|
||||||
|
|
||||||
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;
|
||||||
import io.druid.client.indexing.IndexingServiceClient;
|
import io.druid.client.indexing.IndexingServiceClient;
|
||||||
import io.druid.metadata.MetadataSegmentManager;
|
import io.druid.metadata.MetadataSegmentManager;
|
||||||
import io.druid.timeline.DataSegment;
|
import io.druid.timeline.DataSegment;
|
||||||
|
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
@ -47,11 +49,11 @@ import javax.ws.rs.Produces;
|
||||||
import javax.ws.rs.QueryParam;
|
import javax.ws.rs.QueryParam;
|
||||||
import javax.ws.rs.core.MediaType;
|
import javax.ws.rs.core.MediaType;
|
||||||
import javax.ws.rs.core.Response;
|
import javax.ws.rs.core.Response;
|
||||||
|
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
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.TreeSet;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -82,13 +84,14 @@ public class DatasourcesResource
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
Response.ResponseBuilder builder = Response.ok();
|
Response.ResponseBuilder builder = Response.ok();
|
||||||
|
final Set<DruidDataSource> datasources = InventoryViewUtils.getDataSources(serverInventoryView);
|
||||||
if (full != null) {
|
if (full != null) {
|
||||||
return builder.entity(getDataSources()).build();
|
return builder.entity(datasources).build();
|
||||||
} else if (simple != null) {
|
} else if (simple != null) {
|
||||||
return builder.entity(
|
return builder.entity(
|
||||||
Lists.newArrayList(
|
Lists.newArrayList(
|
||||||
Iterables.transform(
|
Iterables.transform(
|
||||||
getDataSources(),
|
datasources,
|
||||||
new Function<DruidDataSource, Map<String, Object>>()
|
new Function<DruidDataSource, Map<String, Object>>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
@ -105,7 +108,7 @@ public class DatasourcesResource
|
||||||
return builder.entity(
|
return builder.entity(
|
||||||
Lists.newArrayList(
|
Lists.newArrayList(
|
||||||
Iterables.transform(
|
Iterables.transform(
|
||||||
getDataSources(),
|
datasources,
|
||||||
new Function<DruidDataSource, String>()
|
new Function<DruidDataSource, String>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
@ -460,38 +463,6 @@ public class DatasourcesResource
|
||||||
).addSegments(segmentMap);
|
).addSegments(segmentMap);
|
||||||
}
|
}
|
||||||
|
|
||||||
private Set<DruidDataSource> getDataSources()
|
|
||||||
{
|
|
||||||
TreeSet<DruidDataSource> dataSources = Sets.newTreeSet(
|
|
||||||
new Comparator<DruidDataSource>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public int compare(DruidDataSource druidDataSource, DruidDataSource druidDataSource1)
|
|
||||||
{
|
|
||||||
return druidDataSource.getName().compareTo(druidDataSource1.getName());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
dataSources.addAll(
|
|
||||||
Lists.newArrayList(
|
|
||||||
Iterables.concat(
|
|
||||||
Iterables.transform(
|
|
||||||
serverInventoryView.getInventory(),
|
|
||||||
new Function<DruidServer, Iterable<DruidDataSource>>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Iterable<DruidDataSource> apply(DruidServer input)
|
|
||||||
{
|
|
||||||
return input.getDataSources();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
)
|
|
||||||
)
|
|
||||||
)
|
|
||||||
);
|
|
||||||
return dataSources;
|
|
||||||
}
|
|
||||||
|
|
||||||
private Pair<DataSegment, Set<String>> getSegment(String segmentId)
|
private Pair<DataSegment, Set<String>> getSegment(String segmentId)
|
||||||
{
|
{
|
||||||
DataSegment theSegment = null;
|
DataSegment theSegment = null;
|
||||||
|
|
|
@ -0,0 +1,164 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.server.http;
|
||||||
|
|
||||||
|
import com.google.common.collect.Maps;
|
||||||
|
import com.google.inject.Inject;
|
||||||
|
import com.metamx.common.MapUtils;
|
||||||
|
import com.metamx.common.guava.Comparators;
|
||||||
|
|
||||||
|
import io.druid.client.DruidDataSource;
|
||||||
|
import io.druid.client.InventoryView;
|
||||||
|
import io.druid.timeline.DataSegment;
|
||||||
|
|
||||||
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import javax.ws.rs.GET;
|
||||||
|
import javax.ws.rs.Path;
|
||||||
|
import javax.ws.rs.PathParam;
|
||||||
|
import javax.ws.rs.Produces;
|
||||||
|
import javax.ws.rs.QueryParam;
|
||||||
|
import javax.ws.rs.core.MediaType;
|
||||||
|
import javax.ws.rs.core.Response;
|
||||||
|
|
||||||
|
import java.util.Comparator;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
@Path("/druid/coordinator/v1/intervals")
|
||||||
|
public class IntervalsResource
|
||||||
|
{
|
||||||
|
private final InventoryView serverInventoryView;
|
||||||
|
|
||||||
|
@Inject
|
||||||
|
public IntervalsResource(
|
||||||
|
InventoryView serverInventoryView
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.serverInventoryView = serverInventoryView;
|
||||||
|
}
|
||||||
|
|
||||||
|
@GET
|
||||||
|
@Produces(MediaType.APPLICATION_JSON)
|
||||||
|
public Response getIntervals()
|
||||||
|
{
|
||||||
|
final Comparator<Interval> comparator = Comparators.inverse(Comparators.intervalsByStartThenEnd());
|
||||||
|
final Set<DruidDataSource> datasources = InventoryViewUtils.getDataSources(serverInventoryView);
|
||||||
|
|
||||||
|
final Map<Interval, Map<String, Map<String, Object>>> retVal = Maps.newTreeMap(comparator);
|
||||||
|
for (DruidDataSource dataSource : datasources) {
|
||||||
|
for (DataSegment dataSegment : dataSource.getSegments()) {
|
||||||
|
Map<String, Map<String, Object>> interval = retVal.get(dataSegment.getInterval());
|
||||||
|
if (interval == null) {
|
||||||
|
Map<String, Map<String, Object>> tmp = Maps.newHashMap();
|
||||||
|
retVal.put(dataSegment.getInterval(), tmp);
|
||||||
|
}
|
||||||
|
setProperties(retVal, dataSource, dataSegment);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return Response.ok(retVal).build();
|
||||||
|
}
|
||||||
|
|
||||||
|
@GET
|
||||||
|
@Path("/{interval}")
|
||||||
|
@Produces(MediaType.APPLICATION_JSON)
|
||||||
|
public Response getSpecificIntervals(
|
||||||
|
@PathParam("interval") String interval,
|
||||||
|
@QueryParam("simple") String simple,
|
||||||
|
@QueryParam("full") String full
|
||||||
|
)
|
||||||
|
{
|
||||||
|
final Interval theInterval = new Interval(interval.replace("_", "/"));
|
||||||
|
final Set<DruidDataSource> datasources = InventoryViewUtils.getDataSources(serverInventoryView);
|
||||||
|
|
||||||
|
final Comparator<Interval> comparator = Comparators.inverse(Comparators.intervalsByStartThenEnd());
|
||||||
|
if (full != null) {
|
||||||
|
final Map<Interval, Map<String, Map<String, Object>>> retVal = Maps.newTreeMap(comparator);
|
||||||
|
for (DruidDataSource dataSource : datasources) {
|
||||||
|
for (DataSegment dataSegment : dataSource.getSegments()) {
|
||||||
|
if (theInterval.contains(dataSegment.getInterval())) {
|
||||||
|
Map<String, Map<String, Object>> dataSourceInterval = retVal.get(dataSegment.getInterval());
|
||||||
|
if (dataSourceInterval == null) {
|
||||||
|
Map<String, Map<String, Object>> tmp = Maps.newHashMap();
|
||||||
|
retVal.put(dataSegment.getInterval(), tmp);
|
||||||
|
}
|
||||||
|
setProperties(retVal, dataSource, dataSegment);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return Response.ok(retVal).build();
|
||||||
|
}
|
||||||
|
|
||||||
|
if (simple != null) {
|
||||||
|
final Map<Interval, Map<String, Object>> retVal = Maps.newHashMap();
|
||||||
|
for (DruidDataSource dataSource : datasources) {
|
||||||
|
for (DataSegment dataSegment : dataSource.getSegments()) {
|
||||||
|
if (theInterval.contains(dataSegment.getInterval())) {
|
||||||
|
Map<String, Object> properties = retVal.get(dataSegment.getInterval());
|
||||||
|
if (properties == null) {
|
||||||
|
properties = Maps.newHashMap();
|
||||||
|
properties.put("size", dataSegment.getSize());
|
||||||
|
properties.put("count", 1);
|
||||||
|
|
||||||
|
retVal.put(dataSegment.getInterval(), properties);
|
||||||
|
} else {
|
||||||
|
properties.put("size", MapUtils.getLong(properties, "size", 0L) + dataSegment.getSize());
|
||||||
|
properties.put("count", MapUtils.getInt(properties, "count", 0) + 1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return Response.ok(retVal).build();
|
||||||
|
}
|
||||||
|
|
||||||
|
final Map<String, Object> retVal = Maps.newHashMap();
|
||||||
|
for (DruidDataSource dataSource : datasources) {
|
||||||
|
for (DataSegment dataSegment : dataSource.getSegments()) {
|
||||||
|
if (theInterval.contains(dataSegment.getInterval())) {
|
||||||
|
retVal.put("size", MapUtils.getLong(retVal, "size", 0L) + dataSegment.getSize());
|
||||||
|
retVal.put("count", MapUtils.getInt(retVal, "count", 0) + 1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return Response.ok(retVal).build();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void setProperties(
|
||||||
|
final Map<Interval, Map<String, Map<String, Object>>> retVal,
|
||||||
|
DruidDataSource dataSource, DataSegment dataSegment) {
|
||||||
|
Map<String, Object> properties = retVal.get(dataSegment.getInterval()).get(dataSource.getName());
|
||||||
|
if (properties == null) {
|
||||||
|
properties = Maps.newHashMap();
|
||||||
|
properties.put("size", dataSegment.getSize());
|
||||||
|
properties.put("count", 1);
|
||||||
|
|
||||||
|
retVal.get(dataSegment.getInterval()).put(dataSource.getName(), properties);
|
||||||
|
} else {
|
||||||
|
properties.put("size", MapUtils.getLong(properties, "size", 0L) + dataSegment.getSize());
|
||||||
|
properties.put("count", MapUtils.getInt(properties, "count", 0) + 1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,68 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.server.http;
|
||||||
|
|
||||||
|
import io.druid.client.DruidDataSource;
|
||||||
|
import io.druid.client.DruidServer;
|
||||||
|
import io.druid.client.InventoryView;
|
||||||
|
|
||||||
|
import java.util.Comparator;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.TreeSet;
|
||||||
|
|
||||||
|
import com.google.common.base.Function;
|
||||||
|
import com.google.common.collect.Iterables;
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
import com.google.common.collect.Sets;
|
||||||
|
|
||||||
|
public class InventoryViewUtils {
|
||||||
|
|
||||||
|
public static Set<DruidDataSource> getDataSources(InventoryView serverInventoryView)
|
||||||
|
{
|
||||||
|
TreeSet<DruidDataSource> dataSources = Sets.newTreeSet(
|
||||||
|
new Comparator<DruidDataSource>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public int compare(DruidDataSource druidDataSource, DruidDataSource druidDataSource1)
|
||||||
|
{
|
||||||
|
return druidDataSource.getName().compareTo(druidDataSource1.getName());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
dataSources.addAll(
|
||||||
|
Lists.newArrayList(
|
||||||
|
Iterables.concat(
|
||||||
|
Iterables.transform(
|
||||||
|
serverInventoryView.getInventory(),
|
||||||
|
new Function<DruidServer, Iterable<DruidDataSource>>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Iterable<DruidDataSource> apply(DruidServer input)
|
||||||
|
{
|
||||||
|
return input.getDataSources();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
return dataSources;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,190 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.server.http;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
|
||||||
|
import io.druid.client.DruidServer;
|
||||||
|
import io.druid.client.InventoryView;
|
||||||
|
import io.druid.timeline.DataSegment;
|
||||||
|
|
||||||
|
import org.easymock.EasyMock;
|
||||||
|
import org.joda.time.Interval;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import javax.ws.rs.core.Response;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.TreeMap;
|
||||||
|
|
||||||
|
public class IntervalsResourceTest
|
||||||
|
{
|
||||||
|
private InventoryView inventoryView;
|
||||||
|
private DruidServer server;
|
||||||
|
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-01T00:00:00.000Z/P1D"),
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
0x9,
|
||||||
|
20
|
||||||
|
)
|
||||||
|
);
|
||||||
|
dataSegmentList.add(
|
||||||
|
new DataSegment(
|
||||||
|
"datasource1",
|
||||||
|
new Interval("2010-01-22T00:00:00.000Z/P1D"),
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
0x9,
|
||||||
|
10
|
||||||
|
)
|
||||||
|
);
|
||||||
|
dataSegmentList.add(
|
||||||
|
new DataSegment(
|
||||||
|
"datasource2",
|
||||||
|
new Interval("2010-01-01T00:00:00.000Z/P1D"),
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
0x9,
|
||||||
|
5
|
||||||
|
)
|
||||||
|
);
|
||||||
|
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));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGetIntervals()
|
||||||
|
{
|
||||||
|
EasyMock.expect(inventoryView.getInventory()).andReturn(
|
||||||
|
ImmutableList.of(server)
|
||||||
|
).atLeastOnce();
|
||||||
|
EasyMock.replay(inventoryView);
|
||||||
|
|
||||||
|
List<Interval> expectedIntervals = new ArrayList<>();
|
||||||
|
expectedIntervals.add(new Interval("2010-01-01T00:00:00.000Z/2010-01-02T00:00:00.000Z"));
|
||||||
|
expectedIntervals.add(new Interval("2010-01-22T00:00:00.000Z/2010-01-23T00:00:00.000Z"));
|
||||||
|
IntervalsResource intervalsResource = new IntervalsResource(inventoryView);
|
||||||
|
|
||||||
|
Response response = intervalsResource.getIntervals();
|
||||||
|
TreeMap<Interval, Map<String, Map<String, Object>>> actualIntervals = (TreeMap) response.getEntity();
|
||||||
|
Assert.assertEquals(2, actualIntervals.size());
|
||||||
|
Assert.assertEquals(expectedIntervals.get(1), actualIntervals.firstKey());
|
||||||
|
Assert.assertEquals(10L, actualIntervals.get(expectedIntervals.get(1)).get("datasource1").get("size"));
|
||||||
|
Assert.assertEquals(1, actualIntervals.get(expectedIntervals.get(1)).get("datasource1").get("count"));
|
||||||
|
Assert.assertEquals(expectedIntervals.get(0), actualIntervals.lastKey());
|
||||||
|
Assert.assertEquals(20L, actualIntervals.get(expectedIntervals.get(0)).get("datasource1").get("size"));
|
||||||
|
Assert.assertEquals(1, actualIntervals.get(expectedIntervals.get(0)).get("datasource1").get("count"));
|
||||||
|
Assert.assertEquals(5L, actualIntervals.get(expectedIntervals.get(0)).get("datasource2").get("size"));
|
||||||
|
Assert.assertEquals(1, actualIntervals.get(expectedIntervals.get(0)).get("datasource2").get("count"));
|
||||||
|
|
||||||
|
EasyMock.verify(inventoryView);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSimpleGetSpecificIntervals()
|
||||||
|
{
|
||||||
|
EasyMock.expect(inventoryView.getInventory()).andReturn(
|
||||||
|
ImmutableList.of(server)
|
||||||
|
).atLeastOnce();
|
||||||
|
EasyMock.replay(inventoryView);
|
||||||
|
|
||||||
|
List<Interval> expectedIntervals = new ArrayList<>();
|
||||||
|
expectedIntervals.add(new Interval("2010-01-01T00:00:00.000Z/2010-01-02T00:00:00.000Z"));
|
||||||
|
IntervalsResource intervalsResource = new IntervalsResource(inventoryView);
|
||||||
|
|
||||||
|
Response response = intervalsResource.getSpecificIntervals("2010-01-01T00:00:00.000Z/P1D", "simple", null);
|
||||||
|
Map<Interval, Map<String, Object>> actualIntervals = (Map) response.getEntity();
|
||||||
|
Assert.assertEquals(1, actualIntervals.size());
|
||||||
|
Assert.assertTrue(actualIntervals.containsKey(expectedIntervals.get(0)));
|
||||||
|
Assert.assertEquals(25L, actualIntervals.get(expectedIntervals.get(0)).get("size"));
|
||||||
|
Assert.assertEquals(2, actualIntervals.get(expectedIntervals.get(0)).get("count"));
|
||||||
|
|
||||||
|
EasyMock.verify(inventoryView);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFullGetSpecificIntervals()
|
||||||
|
{
|
||||||
|
EasyMock.expect(inventoryView.getInventory()).andReturn(
|
||||||
|
ImmutableList.of(server)
|
||||||
|
).atLeastOnce();
|
||||||
|
EasyMock.replay(inventoryView);
|
||||||
|
|
||||||
|
List<Interval> expectedIntervals = new ArrayList<>();
|
||||||
|
expectedIntervals.add(new Interval("2010-01-01T00:00:00.000Z/2010-01-02T00:00:00.000Z"));
|
||||||
|
IntervalsResource intervalsResource = new IntervalsResource(inventoryView);
|
||||||
|
|
||||||
|
Response response = intervalsResource.getSpecificIntervals("2010-01-01T00:00:00.000Z/P1D", null, "full");
|
||||||
|
TreeMap<Interval, Map<String, Map<String, Object>>> actualIntervals = (TreeMap) response.getEntity();
|
||||||
|
Assert.assertEquals(1, actualIntervals.size());
|
||||||
|
Assert.assertEquals(expectedIntervals.get(0), actualIntervals.firstKey());
|
||||||
|
Assert.assertEquals(20L, actualIntervals.get(expectedIntervals.get(0)).get("datasource1").get("size"));
|
||||||
|
Assert.assertEquals(1, actualIntervals.get(expectedIntervals.get(0)).get("datasource1").get("count"));
|
||||||
|
Assert.assertEquals(5L, actualIntervals.get(expectedIntervals.get(0)).get("datasource2").get("size"));
|
||||||
|
Assert.assertEquals(1, actualIntervals.get(expectedIntervals.get(0)).get("datasource2").get("count"));
|
||||||
|
|
||||||
|
EasyMock.verify(inventoryView);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGetSpecificIntervals()
|
||||||
|
{
|
||||||
|
EasyMock.expect(inventoryView.getInventory()).andReturn(
|
||||||
|
ImmutableList.of(server)
|
||||||
|
).atLeastOnce();
|
||||||
|
EasyMock.replay(inventoryView);
|
||||||
|
|
||||||
|
IntervalsResource intervalsResource = new IntervalsResource(inventoryView);
|
||||||
|
|
||||||
|
Response response = intervalsResource.getSpecificIntervals("2010-01-01T00:00:00.000Z/P1D", null, null);
|
||||||
|
Map<String, Object> actualIntervals = (Map) response.getEntity();
|
||||||
|
Assert.assertEquals(2, actualIntervals.size());
|
||||||
|
Assert.assertEquals(25L, actualIntervals.get("size"));
|
||||||
|
Assert.assertEquals(2, actualIntervals.get("count"));
|
||||||
|
|
||||||
|
EasyMock.verify(inventoryView);
|
||||||
|
}
|
||||||
|
}
|
|
@ -25,6 +25,7 @@ import com.google.inject.Provides;
|
||||||
import com.google.inject.name.Names;
|
import com.google.inject.name.Names;
|
||||||
import com.metamx.common.concurrent.ScheduledExecutorFactory;
|
import com.metamx.common.concurrent.ScheduledExecutorFactory;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
|
|
||||||
import io.airlift.airline.Command;
|
import io.airlift.airline.Command;
|
||||||
import io.druid.client.indexing.IndexingServiceClient;
|
import io.druid.client.indexing.IndexingServiceClient;
|
||||||
import io.druid.guice.ConfigProvider;
|
import io.druid.guice.ConfigProvider;
|
||||||
|
@ -50,6 +51,7 @@ import io.druid.server.http.CoordinatorDynamicConfigsResource;
|
||||||
import io.druid.server.http.CoordinatorRedirectInfo;
|
import io.druid.server.http.CoordinatorRedirectInfo;
|
||||||
import io.druid.server.http.CoordinatorResource;
|
import io.druid.server.http.CoordinatorResource;
|
||||||
import io.druid.server.http.DatasourcesResource;
|
import io.druid.server.http.DatasourcesResource;
|
||||||
|
import io.druid.server.http.IntervalsResource;
|
||||||
import io.druid.server.http.MetadataResource;
|
import io.druid.server.http.MetadataResource;
|
||||||
import io.druid.server.http.RedirectFilter;
|
import io.druid.server.http.RedirectFilter;
|
||||||
import io.druid.server.http.RedirectInfo;
|
import io.druid.server.http.RedirectInfo;
|
||||||
|
@ -58,6 +60,7 @@ import io.druid.server.http.ServersResource;
|
||||||
import io.druid.server.http.TiersResource;
|
import io.druid.server.http.TiersResource;
|
||||||
import io.druid.server.initialization.jetty.JettyServerInitializer;
|
import io.druid.server.initialization.jetty.JettyServerInitializer;
|
||||||
import io.druid.server.router.TieredBrokerConfig;
|
import io.druid.server.router.TieredBrokerConfig;
|
||||||
|
|
||||||
import org.apache.curator.framework.CuratorFramework;
|
import org.apache.curator.framework.CuratorFramework;
|
||||||
import org.eclipse.jetty.server.Server;
|
import org.eclipse.jetty.server.Server;
|
||||||
|
|
||||||
|
@ -132,6 +135,7 @@ public class CliCoordinator extends ServerRunnable
|
||||||
Jerseys.addResource(binder, ServersResource.class);
|
Jerseys.addResource(binder, ServersResource.class);
|
||||||
Jerseys.addResource(binder, DatasourcesResource.class);
|
Jerseys.addResource(binder, DatasourcesResource.class);
|
||||||
Jerseys.addResource(binder, MetadataResource.class);
|
Jerseys.addResource(binder, MetadataResource.class);
|
||||||
|
Jerseys.addResource(binder, IntervalsResource.class);
|
||||||
|
|
||||||
LifecycleModule.register(binder, Server.class);
|
LifecycleModule.register(binder, Server.class);
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue