mirror of https://github.com/apache/druid.git
Merge pull request #452 from metamx/endpoints
New endpoints for new (scalable) coordinator console
This commit is contained in:
commit
42b6482eea
|
@ -26,12 +26,16 @@ import com.google.common.collect.Lists;
|
|||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.MapUtils;
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.guava.Comparators;
|
||||
import io.druid.client.DruidDataSource;
|
||||
import io.druid.client.DruidServer;
|
||||
import io.druid.client.InventoryView;
|
||||
import io.druid.client.indexing.IndexingServiceClient;
|
||||
import io.druid.db.DatabaseSegmentManager;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -86,7 +90,7 @@ public class DatasourcesResource
|
|||
@QueryParam("simple") String simple
|
||||
)
|
||||
{
|
||||
Response.ResponseBuilder builder = Response.status(Response.Status.OK);
|
||||
Response.ResponseBuilder builder = Response.ok();
|
||||
if (full != null) {
|
||||
return builder.entity(getDataSources()).build();
|
||||
} else if (simple != null) {
|
||||
|
@ -128,17 +132,69 @@ public class DatasourcesResource
|
|||
@Path("/{dataSourceName}")
|
||||
@Produces("application/json")
|
||||
public Response getTheDataSource(
|
||||
@PathParam("dataSourceName") final String dataSourceName
|
||||
@PathParam("dataSourceName") final String dataSourceName,
|
||||
@QueryParam("full") final String full
|
||||
)
|
||||
{
|
||||
DruidDataSource dataSource = getDataSource(dataSourceName.toLowerCase());
|
||||
if (dataSource == null) {
|
||||
return Response.status(Response.Status.NOT_FOUND).build();
|
||||
return Response.noContent().build();
|
||||
}
|
||||
|
||||
if (full != null) {
|
||||
return Response.ok(dataSource).build();
|
||||
}
|
||||
|
||||
Map<String, Object> tiers = Maps.newHashMap();
|
||||
Map<String, Object> segments = Maps.newHashMap();
|
||||
Map<String, Map<String, Object>> retVal = ImmutableMap.of(
|
||||
"tiers", tiers,
|
||||
"segments", segments
|
||||
);
|
||||
|
||||
int totalSegmentCount = 0;
|
||||
long totalSegmentSize = 0;
|
||||
long minTime = Long.MAX_VALUE;
|
||||
long maxTime = Long.MIN_VALUE;
|
||||
for (DruidServer druidServer : serverInventoryView.getInventory()) {
|
||||
DruidDataSource druidDataSource = druidServer.getDataSource(dataSourceName);
|
||||
|
||||
long dataSourceSegmentSize = 0;
|
||||
for (DataSegment dataSegment : druidDataSource.getSegments()) {
|
||||
dataSourceSegmentSize += dataSegment.getSize();
|
||||
if (dataSegment.getInterval().getStartMillis() < minTime) {
|
||||
minTime = dataSegment.getInterval().getStartMillis();
|
||||
}
|
||||
if (dataSegment.getInterval().getEndMillis() > maxTime) {
|
||||
maxTime = dataSegment.getInterval().getEndMillis();
|
||||
}
|
||||
}
|
||||
|
||||
// segment stats
|
||||
totalSegmentCount += druidDataSource.getSegments().size();
|
||||
totalSegmentSize += dataSourceSegmentSize;
|
||||
|
||||
// tier stats
|
||||
Map<String, Object> tierStats = (Map) tiers.get(druidServer.getTier());
|
||||
if (tierStats == null) {
|
||||
tierStats = Maps.newHashMap();
|
||||
tiers.put(druidServer.getTier(), tierStats);
|
||||
}
|
||||
int segmentCount = MapUtils.getInt(tierStats, "segmentCount", 0);
|
||||
tierStats.put("segmentCount", segmentCount + druidDataSource.getSegments().size());
|
||||
|
||||
long segmentSize = MapUtils.getLong(tierStats, "size", 0L);
|
||||
tierStats.put("size", segmentSize + dataSourceSegmentSize);
|
||||
}
|
||||
|
||||
segments.put("count", totalSegmentCount);
|
||||
segments.put("size", totalSegmentSize);
|
||||
segments.put("minTime", new DateTime(minTime));
|
||||
segments.put("maxTime", new DateTime(maxTime));
|
||||
|
||||
return Response.ok(retVal).build();
|
||||
}
|
||||
|
||||
@POST
|
||||
@Path("/{dataSourceName}")
|
||||
@Consumes("application/json")
|
||||
|
@ -147,10 +203,10 @@ public class DatasourcesResource
|
|||
)
|
||||
{
|
||||
if (!databaseSegmentManager.enableDatasource(dataSourceName)) {
|
||||
return Response.status(Response.Status.NOT_FOUND).build();
|
||||
return Response.noContent().build();
|
||||
}
|
||||
|
||||
return Response.status(Response.Status.OK).build();
|
||||
return Response.ok().build();
|
||||
}
|
||||
|
||||
@DELETE
|
||||
|
@ -163,15 +219,14 @@ public class DatasourcesResource
|
|||
)
|
||||
{
|
||||
if (indexingServiceClient == null) {
|
||||
return Response.ok().entity(ImmutableMap.of("error", "no indexing service found")).build();
|
||||
return Response.ok(ImmutableMap.of("error", "no indexing service found")).build();
|
||||
}
|
||||
if (kill != null && Boolean.valueOf(kill)) {
|
||||
try {
|
||||
indexingServiceClient.killSegments(dataSourceName, new Interval(interval));
|
||||
}
|
||||
catch (Exception e) {
|
||||
return Response.status(Response.Status.NOT_FOUND)
|
||||
.entity(
|
||||
return Response.serverError().entity(
|
||||
ImmutableMap.of(
|
||||
"error",
|
||||
"Exception occurred. Are you sure you have an indexing service?"
|
||||
|
@ -181,11 +236,144 @@ public class DatasourcesResource
|
|||
}
|
||||
} else {
|
||||
if (!databaseSegmentManager.removeDatasource(dataSourceName)) {
|
||||
return Response.status(Response.Status.NOT_FOUND).build();
|
||||
return Response.noContent().build();
|
||||
}
|
||||
}
|
||||
|
||||
return Response.status(Response.Status.OK).build();
|
||||
return Response.ok().build();
|
||||
}
|
||||
|
||||
@GET
|
||||
@Path("/{dataSourceName}/intervals")
|
||||
@Produces("application/json")
|
||||
public Response getSegmentDataSourceIntervals(
|
||||
@PathParam("dataSourceName") String dataSourceName,
|
||||
@QueryParam("simple") String simple,
|
||||
@QueryParam("full") String full
|
||||
)
|
||||
{
|
||||
final DruidDataSource dataSource = getDataSource(dataSourceName.toLowerCase());
|
||||
|
||||
if (dataSource == null) {
|
||||
return Response.noContent().build();
|
||||
}
|
||||
|
||||
final Comparator<Interval> comparator = Comparators.inverse(Comparators.intervalsByStartThenEnd());
|
||||
|
||||
if (full != null) {
|
||||
final Map<Interval, Map<String, Object>> retVal = Maps.newTreeMap(comparator);
|
||||
for (DataSegment dataSegment : dataSource.getSegments()) {
|
||||
Map<String, Object> segments = retVal.get(dataSegment.getInterval());
|
||||
if (segments == null) {
|
||||
segments = Maps.newHashMap();
|
||||
retVal.put(dataSegment.getInterval(), segments);
|
||||
}
|
||||
|
||||
Pair<DataSegment, Set<String>> val = getSegment(dataSegment.getIdentifier());
|
||||
|
||||
segments.put("id", dataSegment.getIdentifier());
|
||||
segments.put("metadata", val.lhs);
|
||||
segments.put("servers", val.rhs);
|
||||
}
|
||||
|
||||
return Response.ok(retVal).build();
|
||||
}
|
||||
|
||||
if (simple != null) {
|
||||
final Map<Interval, Map<String, Object>> retVal = Maps.newHashMap();
|
||||
for (DataSegment dataSegment : dataSource.getSegments()) {
|
||||
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 Set<Interval> intervals = Sets.newTreeSet(comparator);
|
||||
for (DataSegment dataSegment : dataSource.getSegments()) {
|
||||
intervals.add(dataSegment.getInterval());
|
||||
}
|
||||
|
||||
return Response.ok(intervals).build();
|
||||
}
|
||||
|
||||
@GET
|
||||
@Path("/{dataSourceName}/intervals/{interval}")
|
||||
@Produces("application/json")
|
||||
public Response getSegmentDataSourceSpecificInterval(
|
||||
@PathParam("dataSourceName") String dataSourceName,
|
||||
@PathParam("interval") String interval,
|
||||
@QueryParam("simple") String simple,
|
||||
@QueryParam("full") String full
|
||||
)
|
||||
{
|
||||
final DruidDataSource dataSource = getDataSource(dataSourceName.toLowerCase());
|
||||
final Interval theInterval = new Interval(interval.replace("_", "/"));
|
||||
|
||||
if (dataSource == null || interval == null) {
|
||||
return Response.noContent().build();
|
||||
}
|
||||
|
||||
final Comparator<Interval> comparator = Comparators.inverse(Comparators.intervalsByStartThenEnd());
|
||||
if (full != null) {
|
||||
final Map<Interval, Map<String, Object>> retVal = Maps.newTreeMap(comparator);
|
||||
for (DataSegment dataSegment : dataSource.getSegments()) {
|
||||
if (theInterval.contains(dataSegment.getInterval())) {
|
||||
Map<String, Object> segments = retVal.get(dataSegment.getInterval());
|
||||
if (segments == null) {
|
||||
segments = Maps.newHashMap();
|
||||
retVal.put(dataSegment.getInterval(), segments);
|
||||
}
|
||||
|
||||
Pair<DataSegment, Set<String>> val = getSegment(dataSegment.getIdentifier());
|
||||
|
||||
segments.put("id", dataSegment.getIdentifier());
|
||||
segments.put("metadata", val.lhs);
|
||||
segments.put("servers", val.rhs);
|
||||
}
|
||||
}
|
||||
|
||||
return Response.ok(retVal).build();
|
||||
}
|
||||
|
||||
if (simple != null) {
|
||||
final Map<Interval, Map<String, Object>> retVal = Maps.newHashMap();
|
||||
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 Set<Interval> intervals = Sets.newTreeSet(comparator);
|
||||
for (DataSegment dataSegment : dataSource.getSegments()) {
|
||||
if (theInterval.contains(dataSegment.getInterval())) {
|
||||
intervals.add(dataSegment.getInterval());
|
||||
}
|
||||
}
|
||||
|
||||
return Response.ok(intervals).build();
|
||||
}
|
||||
|
||||
@GET
|
||||
|
@ -198,10 +386,10 @@ public class DatasourcesResource
|
|||
{
|
||||
DruidDataSource dataSource = getDataSource(dataSourceName.toLowerCase());
|
||||
if (dataSource == null) {
|
||||
return Response.status(Response.Status.NOT_FOUND).build();
|
||||
return Response.noContent().build();
|
||||
}
|
||||
|
||||
Response.ResponseBuilder builder = Response.status(Response.Status.OK);
|
||||
Response.ResponseBuilder builder = Response.ok();
|
||||
if (full != null) {
|
||||
return builder.entity(dataSource.getSegments()).build();
|
||||
}
|
||||
|
@ -212,7 +400,7 @@ public class DatasourcesResource
|
|||
new Function<DataSegment, Object>()
|
||||
{
|
||||
@Override
|
||||
public Object apply(@Nullable DataSegment segment)
|
||||
public Object apply(DataSegment segment)
|
||||
{
|
||||
return segment.getIdentifier();
|
||||
}
|
||||
|
@ -231,15 +419,18 @@ public class DatasourcesResource
|
|||
{
|
||||
DruidDataSource dataSource = getDataSource(dataSourceName.toLowerCase());
|
||||
if (dataSource == null) {
|
||||
return Response.status(Response.Status.NOT_FOUND).build();
|
||||
return Response.noContent().build();
|
||||
}
|
||||
|
||||
for (DataSegment segment : dataSource.getSegments()) {
|
||||
if (segment.getIdentifier().equalsIgnoreCase(segmentId)) {
|
||||
return Response.status(Response.Status.OK).entity(segment).build();
|
||||
Pair<DataSegment, Set<String>> retVal = getSegment(segmentId);
|
||||
|
||||
if (retVal != null) {
|
||||
return Response.ok(
|
||||
ImmutableMap.of("metadata", retVal.lhs, "servers", retVal.rhs)
|
||||
).build();
|
||||
}
|
||||
}
|
||||
return Response.status(Response.Status.NOT_FOUND).build();
|
||||
|
||||
return Response.noContent().build();
|
||||
}
|
||||
|
||||
@DELETE
|
||||
|
@ -250,10 +441,10 @@ public class DatasourcesResource
|
|||
)
|
||||
{
|
||||
if (!databaseSegmentManager.removeSegment(dataSourceName, segmentId)) {
|
||||
return Response.status(Response.Status.NOT_FOUND).build();
|
||||
return Response.noContent().build();
|
||||
}
|
||||
|
||||
return Response.status(Response.Status.OK).build();
|
||||
return Response.ok().build();
|
||||
}
|
||||
|
||||
@POST
|
||||
|
@ -265,10 +456,27 @@ public class DatasourcesResource
|
|||
)
|
||||
{
|
||||
if (!databaseSegmentManager.enableSegment(segmentId)) {
|
||||
return Response.status(Response.Status.NOT_FOUND).build();
|
||||
return Response.noContent().build();
|
||||
}
|
||||
|
||||
return Response.status(Response.Status.OK).build();
|
||||
return Response.ok().build();
|
||||
}
|
||||
|
||||
@GET
|
||||
@Path("/{dataSourceName}/tiers")
|
||||
@Produces("application/json")
|
||||
public Response getSegmentDataSourceTiers(
|
||||
@PathParam("dataSourceName") String dataSourceName
|
||||
)
|
||||
{
|
||||
Set<String> retVal = Sets.newHashSet();
|
||||
for (DruidServer druidServer : serverInventoryView.getInventory()) {
|
||||
if (druidServer.getDataSource(dataSourceName) != null) {
|
||||
retVal.add(druidServer.getTier());
|
||||
}
|
||||
}
|
||||
|
||||
return Response.ok(retVal).build();
|
||||
}
|
||||
|
||||
private DruidDataSource getDataSource(final String dataSourceName)
|
||||
|
@ -345,4 +553,23 @@ public class DatasourcesResource
|
|||
);
|
||||
return dataSources;
|
||||
}
|
||||
|
||||
private Pair<DataSegment, Set<String>> getSegment(String segmentId)
|
||||
{
|
||||
DataSegment theSegment = null;
|
||||
Set<String> servers = Sets.newHashSet();
|
||||
for (DruidServer druidServer : serverInventoryView.getInventory()) {
|
||||
DataSegment currSegment = druidServer.getSegments().get(segmentId);
|
||||
if (currSegment != null) {
|
||||
theSegment = currSegment;
|
||||
servers.add(druidServer.getHost());
|
||||
}
|
||||
}
|
||||
|
||||
if (theSegment == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
return new Pair<>(theSegment, servers);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -48,6 +48,8 @@ public class ServersResource
|
|||
return new ImmutableMap.Builder<String, Object>()
|
||||
.put("host", input.getHost())
|
||||
.put("tier", input.getTier())
|
||||
.put("type", input.getType())
|
||||
.put("priority", input.getPriority())
|
||||
.put("currSize", input.getCurrSize())
|
||||
.put("maxSize", input.getMaxSize())
|
||||
.build();
|
||||
|
|
|
@ -19,18 +19,19 @@
|
|||
|
||||
package io.druid.server.http;
|
||||
|
||||
import com.google.api.client.util.Lists;
|
||||
import com.google.api.client.util.Maps;
|
||||
import com.google.common.collect.HashBasedTable;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.collect.Table;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.MapUtils;
|
||||
import io.druid.client.DruidDataSource;
|
||||
import io.druid.client.DruidServer;
|
||||
import io.druid.client.InventoryView;
|
||||
|
||||
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.Response;
|
||||
|
@ -86,4 +87,35 @@ public class TiersResource
|
|||
|
||||
return builder.entity(tiers).build();
|
||||
}
|
||||
|
||||
@GET
|
||||
@Path("/{tierName}")
|
||||
@Produces("application/json")
|
||||
public Response getTierDatasources(
|
||||
@PathParam("tierName") String tierName
|
||||
)
|
||||
{
|
||||
Set<String> retVal = Sets.newHashSet();
|
||||
for (DruidServer druidServer : serverInventoryView.getInventory()) {
|
||||
if (druidServer.getTier().equalsIgnoreCase(tierName)) {
|
||||
retVal.addAll(
|
||||
Lists.newArrayList(
|
||||
Iterables.transform(
|
||||
druidServer.getDataSources(),
|
||||
new Function<DruidDataSource, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(DruidDataSource input)
|
||||
{
|
||||
return input.getName();
|
||||
}
|
||||
}
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
return Response.ok(retVal).build();
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue