From a39f4f8bf4a0304ea241dbec6164d506ef6783c9 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Sun, 22 Feb 2015 17:27:56 -0600 Subject: [PATCH] updating ClientInfoResource.getDatasource() with a full flag to return all the served intervals for given data source along with the dimension and metric information --- docs/content/Broker.md | 2 +- .../io/druid/server/ClientInfoResource.java | 124 ++++++++++++++--- .../druid/server/ClientInfoResourceTest.java | 131 ++++++++++++++++++ 3 files changed, 240 insertions(+), 17 deletions(-) create mode 100644 server/src/test/java/io/druid/server/ClientInfoResourceTest.java diff --git a/docs/content/Broker.md b/docs/content/Broker.md index 45fcae12434..713e1ea4a4e 100644 --- a/docs/content/Broker.md +++ b/docs/content/Broker.md @@ -45,7 +45,7 @@ Returns a list of queryable datasources. * `/druid/v2/datasources/{dataSourceName}` -Returns the dimensions and metrics of the datasource. +Returns the dimensions and metrics of the datasource. Optionally, you can provide request parameter "full" to get list of served intervals with dimensions and metrics being served for those intervals. You can also provide request param "interval" explicitly to refer to a particular interval. * `/druid/v2/datasources/{dataSourceName}/dimensions` diff --git a/server/src/main/java/io/druid/server/ClientInfoResource.java b/server/src/main/java/io/druid/server/ClientInfoResource.java index 8d61d30b0f8..552b3e3e3bd 100644 --- a/server/src/main/java/io/druid/server/ClientInfoResource.java +++ b/server/src/main/java/io/druid/server/ClientInfoResource.java @@ -17,17 +17,22 @@ package io.druid.server; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; -import com.google.inject.Inject; import io.druid.client.DruidDataSource; import io.druid.client.DruidServer; import io.druid.client.InventoryView; +import io.druid.client.TimelineServerView; +import io.druid.client.selector.ServerSelector; +import io.druid.query.TableDataSource; import io.druid.timeline.DataSegment; -import org.joda.time.DateTime; -import org.joda.time.Interval; +import io.druid.timeline.TimelineLookup; +import io.druid.timeline.TimelineObjectHolder; + +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; import javax.ws.rs.GET; import javax.ws.rs.Path; @@ -35,25 +40,41 @@ import javax.ws.rs.PathParam; import javax.ws.rs.Produces; import javax.ws.rs.QueryParam; import javax.ws.rs.core.MediaType; -import java.util.List; -import java.util.Map; -import java.util.Set; + +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +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.logger.Logger; /** */ @Path("/druid/v2/datasources") public class ClientInfoResource { + private static final Logger log = new Logger(ClientInfoResource.class); + + private static final String KEY_DIMENSIONS = "dimensions"; + private static final String KEY_METRICS = "metrics"; + private static final int SEGMENT_HISTORY_MILLIS = 7 * 24 * 60 * 60 * 1000; // ONE WEEK private InventoryView serverInventoryView; + private TimelineServerView timelineServerView; @Inject public ClientInfoResource( - InventoryView serverInventoryView + InventoryView serverInventoryView, + TimelineServerView timelineServerView ) { this.serverInventoryView = serverInventoryView; + this.timelineServerView = timelineServerView; } private Map> getSegmentsForDatasources() @@ -83,13 +104,84 @@ public class ClientInfoResource @Produces(MediaType.APPLICATION_JSON) public Map getDatasource( @PathParam("dataSourceName") String dataSourceName, - @QueryParam("interval") String interval + @QueryParam("interval") String interval, + @QueryParam("full") String full ) { - return ImmutableMap.of( - "dimensions", getDatasourceDimensions(dataSourceName, interval), - "metrics", getDatasourceMetrics(dataSourceName, interval) - ); + if(full == null) { + return ImmutableMap. of( + KEY_DIMENSIONS, getDatasourceDimensions(dataSourceName, interval), + KEY_METRICS, getDatasourceMetrics(dataSourceName, interval) + ); + } + + Interval theInterval; + if (interval == null || interval.isEmpty()) { + DateTime now = new DateTime(); + theInterval = new Interval(now.minusMillis(SEGMENT_HISTORY_MILLIS), now); + } else { + theInterval = new Interval(interval); + } + + TimelineLookup timeline = timelineServerView.getTimeline(new TableDataSource(dataSourceName)); + Iterable> serversLookup = timeline != null ? timeline.lookup(theInterval) : null; + if(serversLookup == null || Iterables.isEmpty(serversLookup)) { + return Collections.EMPTY_MAP; + } + Map servedIntervals = new TreeMap<>(new Comparator() + { + @Override + public int compare(Interval o1, Interval o2) + { + if(o1.equals(o2) || o1.overlaps(o2)) { + return 0; + } else { + return o1.isBefore(o2) ? -1 : 1; + } + } + }); + + for (TimelineObjectHolder holder : serversLookup) { + servedIntervals.put(holder.getInterval(), ImmutableMap.of(KEY_DIMENSIONS, Sets.newHashSet(), KEY_METRICS, Sets.newHashSet())); + } + + List segments = getSegmentsForDatasources().get(dataSourceName); + if (segments == null || segments.isEmpty()) { + log.error("Found no DataSegments but TimelineServerView has served intervals. Datasource = %s , Interval = %s", + dataSourceName, + theInterval); + throw new RuntimeException("Internal Error"); + } + + for (DataSegment segment : segments) { + if(servedIntervals.containsKey(segment.getInterval())) { + Map> columns = (Map>)servedIntervals.get(segment.getInterval()); + columns.get(KEY_DIMENSIONS).addAll(segment.getDimensions()); + columns.get(KEY_METRICS).addAll(segment.getMetrics()); + } + } + + //collapse intervals if they abut and have same set of columns + Map result = Maps.newLinkedHashMap(); + Interval curr = null; + Map> cols = null; + for(Map.Entry e : servedIntervals.entrySet()) { + Interval ival = e.getKey(); + if(curr != null && curr.abuts(ival) && cols.equals(e.getValue())) { + curr = curr.withEnd(ival.getEnd()); + } else { + if(curr != null) { + result.put(curr.toString(), cols); + } + curr = ival; + cols = (Map>)e.getValue(); + } + } + //add the last one in + if(curr != null) { + result.put(curr.toString(), cols); + } + return result; } @GET diff --git a/server/src/test/java/io/druid/server/ClientInfoResourceTest.java b/server/src/test/java/io/druid/server/ClientInfoResourceTest.java new file mode 100644 index 00000000000..42a7998aed4 --- /dev/null +++ b/server/src/test/java/io/druid/server/ClientInfoResourceTest.java @@ -0,0 +1,131 @@ +/* + * Druid - a distributed column store. + * Copyright 2015 - Yahoo! Inc. + * + * Licensed 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; + +import java.util.List; +import java.util.Map; + +import io.druid.client.DruidServer; +import io.druid.client.InventoryView; +import io.druid.client.TimelineServerView; +import io.druid.client.selector.ServerSelector; +import io.druid.query.TableDataSource; +import io.druid.timeline.DataSegment; +import io.druid.timeline.VersionedIntervalTimeline; +import io.druid.timeline.partition.SingleElementPartitionChunk; + +import org.easymock.EasyMock; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Ordering; + +public class ClientInfoResourceTest +{ + private static final String KEY_DIMENSIONS = "dimensions"; + private static final String KEY_METRICS = "metrics"; + + private final String dataSource = "test-data-source"; + private final String version = "v0"; + + private InventoryView serverInventoryView; + private TimelineServerView timelineServerView; + private ClientInfoResource resource; + + @Before + public void setup() { + VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>(Ordering.natural()); + DruidServer server = new DruidServer("name", "host", 1234, "type", "tier", 0); + + addSegment(timeline, server, "2014-02-13/2014-02-14", ImmutableList.of("d1"), ImmutableList.of("m1")); + addSegment(timeline, server, "2014-02-14/2014-02-15", ImmutableList.of("d1"), ImmutableList.of("m1")); + addSegment(timeline, server, "2014-02-16/2014-02-17", ImmutableList.of("d1"), ImmutableList.of("m1")); + addSegment(timeline, server, "2014-02-17/2014-02-18", ImmutableList.of("d2"), ImmutableList.of("m2")); + + serverInventoryView = EasyMock.createMock(InventoryView.class); + EasyMock.expect(serverInventoryView.getInventory()).andReturn(ImmutableList.of(server)).anyTimes(); + + timelineServerView = EasyMock.createMock(TimelineServerView.class); + EasyMock.expect(timelineServerView.getTimeline(EasyMock.anyObject(TableDataSource.class))).andReturn(timeline); + + EasyMock.replay(serverInventoryView, timelineServerView); + + resource = new ClientInfoResource(serverInventoryView, timelineServerView); + } + + @Test + public void testGetDatasourceNonFullWithLargeInterval() { + Map actual = resource.getDatasource(dataSource, "1975/2050", null); + Map expected = ImmutableMap.of( + KEY_DIMENSIONS, ImmutableSet.of("d1", "d2"), + KEY_METRICS, ImmutableSet.of("m1", "m2") + ); + EasyMock.verify(serverInventoryView); + Assert.assertEquals(expected, actual); + } + + @Test + public void testGetDatasourceFullWithLargeInterval() { + + Map actual = resource.getDatasource(dataSource, "1975/2050", "true"); + Map expected = ImmutableMap.of( + "2014-02-13T00:00:00.000Z/2014-02-15T00:00:00.000Z", ImmutableMap.of(KEY_DIMENSIONS, ImmutableSet.of("d1"), KEY_METRICS, ImmutableSet.of("m1")), + "2014-02-16T00:00:00.000Z/2014-02-17T00:00:00.000Z", ImmutableMap.of(KEY_DIMENSIONS, ImmutableSet.of("d1"), KEY_METRICS, ImmutableSet.of("m1")), + "2014-02-17T00:00:00.000Z/2014-02-18T00:00:00.000Z", ImmutableMap.of(KEY_DIMENSIONS, ImmutableSet.of("d2"), KEY_METRICS, ImmutableSet.of("m2")) + ); + + EasyMock.verify(serverInventoryView, timelineServerView); + Assert.assertEquals(expected, actual); + } + + @Test + public void testGetDatasourceFullWithSmallInterval() { + Map actual = resource.getDatasource(dataSource, "2014-02-13T09:00:00.000Z/2014-02-17T23:00:00.000Z", "true"); + Map expected = ImmutableMap.of( + "2014-02-13T09:00:00.000Z/2014-02-15T00:00:00.000Z", ImmutableMap.of(KEY_DIMENSIONS, ImmutableSet.of("d1"), KEY_METRICS, ImmutableSet.of("m1")), + "2014-02-16T00:00:00.000Z/2014-02-17T00:00:00.000Z", ImmutableMap.of(KEY_DIMENSIONS, ImmutableSet.of("d1"), KEY_METRICS, ImmutableSet.of("m1")), + "2014-02-17T00:00:00.000Z/2014-02-17T23:00:00.000Z", ImmutableMap.of(KEY_DIMENSIONS, ImmutableSet.of("d2"), KEY_METRICS, ImmutableSet.of("m2")) + ); + + EasyMock.verify(serverInventoryView, timelineServerView); + Assert.assertEquals(expected, actual); + } + + private void addSegment(VersionedIntervalTimeline timeline, + DruidServer server, + String interval, + List dims, + List metrics) { + DataSegment segment = DataSegment.builder() + .dataSource(dataSource) + .interval(new Interval(interval)) + .version(version) + .dimensions(dims) + .metrics(metrics) + .size(1) + .build(); + server.addDataSegment(segment.getIdentifier(), segment); + ServerSelector ss = new ServerSelector(segment, null); + timeline.add(new Interval(interval), version, new SingleElementPartitionChunk(ss)); + } +}