more endpoint cleanup

This commit is contained in:
fjy 2014-03-28 14:31:30 -07:00
parent 676671e575
commit c5fbb67d94
1 changed files with 56 additions and 3 deletions

View File

@ -35,6 +35,7 @@ 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;
@ -131,7 +132,8 @@ 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());
@ -139,7 +141,58 @@ public class DatasourcesResource
return Response.noContent().build();
}
return Response.ok(dataSource).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
@ -418,7 +471,7 @@ public class DatasourcesResource
{
Set<String> retVal = Sets.newHashSet();
for (DruidServer druidServer : serverInventoryView.getInventory()) {
if(druidServer.getDataSource(dataSourceName) != null) {
if (druidServer.getDataSource(dataSourceName) != null) {
retVal.add(druidServer.getTier());
}
}