mirror of https://github.com/apache/druid.git
Merge pull request #1732 from jon-wei/segmentmeta
Add support for a configurable default segment history period for segmentMetadata queries and GET /datasources/<datasourceName> lookups
This commit is contained in:
commit
9705c5139b
|
@ -75,6 +75,12 @@ The broker uses processing configs for nested groupBy queries. And, optionally,
|
|||
|--------|-----------|-------|
|
||||
|`druid.query.search.maxSearchLimit`|Maximum number of search results to return.|1000|
|
||||
|
||||
##### Segment Metadata Query Config
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.query.segmentMetadata.defaultHistory`|When no interval is specified in the query, use a default interval of defaultHistory before the end time of the most recent segment, specified in ISO8601 format. This property also controls the duration of the default interval used by GET /druid/v2/datasources/{dataSourceName} interactions for retrieving datasource dimensions/metrics.|P1W|
|
||||
|
||||
### Caching
|
||||
|
||||
You can optionally only configure caching to be enabled on the broker by setting caching configs here.
|
||||
|
|
|
@ -47,6 +47,10 @@ Returns a list of queryable datasources.
|
|||
|
||||
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.
|
||||
|
||||
If no interval is specified, a default interval spanning a configurable period before the current time will be used. The duration of this interval is specified in ISO8601 format via:
|
||||
|
||||
druid.query.segmentMetadata.defaultHistory
|
||||
|
||||
* `/druid/v2/datasources/{dataSourceName}/dimensions`
|
||||
|
||||
Returns the dimensions of the datasource.
|
||||
|
|
|
@ -25,7 +25,7 @@ There are several main parts to a segment metadata query:
|
|||
|--------|-----------|---------|
|
||||
|queryType|This String should always be "segmentMetadata"; this is the first thing Druid looks at to figure out how to interpret the query|yes|
|
||||
|dataSource|A String or Object defining the data source to query, very similar to a table in a relational database. See [DataSource](../querying/datasource.html) for more information.|yes|
|
||||
|intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes|
|
||||
|intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|no|
|
||||
|toInclude|A JSON Object representing what columns should be included in the result. Defaults to "all".|no|
|
||||
|merge|Merge all individual segment metadata results into a single result|no|
|
||||
|context|See [Context](../querying/query-context.html)|no|
|
||||
|
@ -52,6 +52,13 @@ Timestamp column will have type `LONG`.
|
|||
|
||||
Only columns which are dimensions (ie, have type `STRING`) will have any cardinality. Rest of the columns (timestamp and metric columns) will show cardinality as `null`.
|
||||
|
||||
### intervals
|
||||
|
||||
If an interval is not specified, the query will use a default interval that spans a configurable period before the end time of the most recent segment.
|
||||
|
||||
The length of this default time period is set in the broker configuration via:
|
||||
druid.query.segmentMetadata.defaultHistory
|
||||
|
||||
### toInclude
|
||||
|
||||
There are 3 types of toInclude objects.
|
||||
|
|
|
@ -904,7 +904,8 @@ public class Druids
|
|||
querySegmentSpec,
|
||||
toInclude,
|
||||
merge,
|
||||
context
|
||||
context,
|
||||
false
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,49 @@
|
|||
/*
|
||||
* 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.query.metadata;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.joda.time.Period;
|
||||
import org.joda.time.format.ISOPeriodFormat;
|
||||
import org.joda.time.format.PeriodFormatter;
|
||||
|
||||
|
||||
public class SegmentMetadataQueryConfig
|
||||
{
|
||||
private static final String DEFAULT_PERIOD_STRING = "P1W";
|
||||
private static final PeriodFormatter ISO_FORMATTER = ISOPeriodFormat.standard();
|
||||
|
||||
@JsonProperty
|
||||
private Period defaultHistory = ISO_FORMATTER.parsePeriod(DEFAULT_PERIOD_STRING);
|
||||
|
||||
public SegmentMetadataQueryConfig(String period)
|
||||
{
|
||||
defaultHistory = ISO_FORMATTER.parsePeriod(period);
|
||||
}
|
||||
|
||||
public SegmentMetadataQueryConfig()
|
||||
{
|
||||
}
|
||||
|
||||
public Period getDefaultHistory()
|
||||
{
|
||||
return defaultHistory;
|
||||
}
|
||||
}
|
|
@ -20,10 +20,13 @@ package io.druid.query.metadata;
|
|||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Functions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.guava.MergeSequence;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
|
@ -41,6 +44,8 @@ import io.druid.query.aggregation.MetricManipulationFn;
|
|||
import io.druid.query.metadata.metadata.ColumnAnalysis;
|
||||
import io.druid.query.metadata.metadata.SegmentAnalysis;
|
||||
import io.druid.query.metadata.metadata.SegmentMetadataQuery;
|
||||
import io.druid.timeline.LogicalSegment;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -56,6 +61,16 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAn
|
|||
};
|
||||
private static final byte[] SEGMENT_METADATA_CACHE_PREFIX = new byte[]{0x4};
|
||||
|
||||
private final SegmentMetadataQueryConfig config;
|
||||
|
||||
@Inject
|
||||
public SegmentMetadataQueryQueryToolChest(
|
||||
SegmentMetadataQueryConfig config
|
||||
)
|
||||
{
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueryRunner<SegmentAnalysis> mergeResults(final QueryRunner<SegmentAnalysis> runner)
|
||||
{
|
||||
|
@ -216,6 +231,37 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAn
|
|||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T extends LogicalSegment> List<T> filterSegments(SegmentMetadataQuery query, List<T> segments)
|
||||
{
|
||||
if (!query.isUsingDefaultInterval()) {
|
||||
return segments;
|
||||
}
|
||||
|
||||
if (segments.size() <= 1) {
|
||||
return segments;
|
||||
}
|
||||
|
||||
final T max = segments.get(segments.size() - 1);
|
||||
|
||||
DateTime targetEnd = max.getInterval().getEnd();
|
||||
final Interval targetInterval = new Interval(config.getDefaultHistory(), targetEnd);
|
||||
|
||||
return Lists.newArrayList(
|
||||
Iterables.filter(
|
||||
segments,
|
||||
new Predicate<T>()
|
||||
{
|
||||
@Override
|
||||
public boolean apply(T input)
|
||||
{
|
||||
return (input.getInterval().overlaps(targetInterval));
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
private Ordering<SegmentAnalysis> getOrdering()
|
||||
{
|
||||
return new Ordering<SegmentAnalysis>()
|
||||
|
|
|
@ -20,18 +20,27 @@ package io.druid.query.metadata.metadata;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import io.druid.common.utils.JodaUtils;
|
||||
import io.druid.query.BaseQuery;
|
||||
import io.druid.query.DataSource;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.TableDataSource;
|
||||
import io.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||
import io.druid.query.spec.QuerySegmentSpec;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Map;
|
||||
|
||||
public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
|
||||
{
|
||||
public static final Interval DEFAULT_INTERVAL = new Interval(
|
||||
JodaUtils.MIN_INSTANT, JodaUtils.MAX_INSTANT
|
||||
);
|
||||
|
||||
private final ColumnIncluderator toInclude;
|
||||
private final boolean merge;
|
||||
private final boolean usingDefaultInterval;
|
||||
|
||||
@JsonCreator
|
||||
public SegmentMetadataQuery(
|
||||
|
@ -39,14 +48,29 @@ public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
|
|||
@JsonProperty("intervals") QuerySegmentSpec querySegmentSpec,
|
||||
@JsonProperty("toInclude") ColumnIncluderator toInclude,
|
||||
@JsonProperty("merge") Boolean merge,
|
||||
@JsonProperty("context") Map<String, Object> context
|
||||
@JsonProperty("context") Map<String, Object> context,
|
||||
@JsonProperty("usingDefaultInterval") Boolean useDefaultInterval
|
||||
)
|
||||
{
|
||||
super(dataSource, querySegmentSpec, context);
|
||||
super(
|
||||
dataSource,
|
||||
(querySegmentSpec == null) ? new MultipleIntervalSegmentSpec(Arrays.asList(DEFAULT_INTERVAL))
|
||||
: querySegmentSpec,
|
||||
context
|
||||
);
|
||||
|
||||
if (querySegmentSpec == null) {
|
||||
this.usingDefaultInterval = true;
|
||||
} else {
|
||||
this.usingDefaultInterval = useDefaultInterval == null ? false : useDefaultInterval;
|
||||
}
|
||||
|
||||
this.toInclude = toInclude == null ? new AllColumnIncluderator() : toInclude;
|
||||
this.merge = merge == null ? false : merge;
|
||||
Preconditions.checkArgument(dataSource instanceof TableDataSource, "SegmentMetadataQuery only supports table datasource");
|
||||
Preconditions.checkArgument(
|
||||
dataSource instanceof TableDataSource,
|
||||
"SegmentMetadataQuery only supports table datasource"
|
||||
);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -61,6 +85,12 @@ public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
|
|||
return merge;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public boolean isUsingDefaultInterval()
|
||||
{
|
||||
return usingDefaultInterval;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasFilters()
|
||||
{
|
||||
|
@ -78,7 +108,11 @@ public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
|
|||
{
|
||||
return new SegmentMetadataQuery(
|
||||
getDataSource(),
|
||||
getQuerySegmentSpec(), toInclude, merge, computeOverridenContext(contextOverride)
|
||||
getQuerySegmentSpec(),
|
||||
toInclude,
|
||||
merge,
|
||||
computeOverridenContext(contextOverride),
|
||||
usingDefaultInterval
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -87,7 +121,12 @@ public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
|
|||
{
|
||||
return new SegmentMetadataQuery(
|
||||
getDataSource(),
|
||||
spec, toInclude, merge, getContext());
|
||||
spec,
|
||||
toInclude,
|
||||
merge,
|
||||
getContext(),
|
||||
usingDefaultInterval
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -98,22 +137,34 @@ public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
|
|||
getQuerySegmentSpec(),
|
||||
toInclude,
|
||||
merge,
|
||||
getContext());
|
||||
getContext(),
|
||||
usingDefaultInterval
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
if (!super.equals(o)) return false;
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
if (!super.equals(o)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
SegmentMetadataQuery that = (SegmentMetadataQuery) o;
|
||||
|
||||
if (merge != that.merge) return false;
|
||||
if (toInclude != null ? !toInclude.equals(that.toInclude) : that.toInclude != null) return false;
|
||||
if (merge != that.merge) {
|
||||
return false;
|
||||
}
|
||||
if (usingDefaultInterval != that.usingDefaultInterval) {
|
||||
return false;
|
||||
}
|
||||
return !(toInclude != null ? !toInclude.equals(that.toInclude) : that.toInclude != null);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -122,6 +173,7 @@ public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
|
|||
int result = super.hashCode();
|
||||
result = 31 * result + (toInclude != null ? toInclude.hashCode() : 0);
|
||||
result = 31 * result + (merge ? 1 : 0);
|
||||
result = 31 * result + (usingDefaultInterval ? 1 : 0);
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -99,13 +99,13 @@ public class SegmentAnalyzerTest
|
|||
{
|
||||
final QueryRunner runner = QueryRunnerTestHelper.makeQueryRunner(
|
||||
(QueryRunnerFactory) new SegmentMetadataQueryRunnerFactory(
|
||||
new SegmentMetadataQueryQueryToolChest(),
|
||||
new SegmentMetadataQueryQueryToolChest(new SegmentMetadataQueryConfig()),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||
), index
|
||||
);
|
||||
|
||||
final SegmentMetadataQuery query = new SegmentMetadataQuery(
|
||||
new LegacyDataSource("test"), QuerySegmentSpecs.create("2011/2012"), null, null, null
|
||||
new LegacyDataSource("test"), QuerySegmentSpecs.create("2011/2012"), null, null, null, false
|
||||
);
|
||||
HashMap<String,Object> context = new HashMap<String, Object>();
|
||||
return Sequences.toList(query.run(runner, context), Lists.<SegmentAnalysis>newArrayList());
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.google.common.collect.Iterables;
|
|||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import io.druid.common.utils.JodaUtils;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.query.BySegmentResultValue;
|
||||
import io.druid.query.BySegmentResultValueClass;
|
||||
|
@ -43,6 +44,7 @@ import io.druid.segment.QueryableIndexSegment;
|
|||
import io.druid.segment.TestHelper;
|
||||
import io.druid.segment.TestIndex;
|
||||
import io.druid.segment.column.ValueType;
|
||||
import io.druid.timeline.LogicalSegment;
|
||||
import org.joda.time.Interval;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -55,7 +57,7 @@ import java.util.concurrent.Executors;
|
|||
public class SegmentMetadataQueryTest
|
||||
{
|
||||
private final SegmentMetadataQueryRunnerFactory factory = new SegmentMetadataQueryRunnerFactory(
|
||||
new SegmentMetadataQueryQueryToolChest(),
|
||||
new SegmentMetadataQueryQueryToolChest(new SegmentMetadataQueryConfig()),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||
);
|
||||
|
||||
|
@ -103,6 +105,7 @@ public class SegmentMetadataQueryTest
|
|||
), 71982
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testSegmentMetadataQuery()
|
||||
|
@ -138,7 +141,9 @@ public class SegmentMetadataQueryTest
|
|||
//Note: It is essential to have atleast 2 query runners merged to reproduce the regression bug described in
|
||||
//https://github.com/druid-io/druid/pull/1172
|
||||
//the bug surfaces only when ordering is used which happens only when you have 2 things to compare
|
||||
Lists.<QueryRunner<SegmentAnalysis>>newArrayList(singleSegmentQueryRunner, singleSegmentQueryRunner))),
|
||||
Lists.<QueryRunner<SegmentAnalysis>>newArrayList(singleSegmentQueryRunner, singleSegmentQueryRunner)
|
||||
)
|
||||
),
|
||||
toolChest
|
||||
);
|
||||
|
||||
|
@ -148,7 +153,8 @@ public class SegmentMetadataQueryTest
|
|||
testQuery.withOverriddenContext(ImmutableMap.<String, Object>of("bySegment", true)),
|
||||
Maps.newHashMap()
|
||||
),
|
||||
"failed SegmentMetadata bySegment query");
|
||||
"failed SegmentMetadata bySegment query"
|
||||
);
|
||||
exec.shutdownNow();
|
||||
}
|
||||
|
||||
|
@ -168,4 +174,196 @@ public class SegmentMetadataQueryTest
|
|||
// test serialize and deserialize
|
||||
Assert.assertEquals(query, mapper.readValue(mapper.writeValueAsString(query), Query.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSerdeWithDefaultInterval() throws Exception
|
||||
{
|
||||
String queryStr = "{\n"
|
||||
+ " \"queryType\":\"segmentMetadata\",\n"
|
||||
+ " \"dataSource\":\"test_ds\"\n"
|
||||
+ "}";
|
||||
Query query = mapper.readValue(queryStr, Query.class);
|
||||
Assert.assertTrue(query instanceof SegmentMetadataQuery);
|
||||
Assert.assertEquals("test_ds", Iterables.getOnlyElement(query.getDataSource().getNames()));
|
||||
Assert.assertEquals(new Interval(JodaUtils.MIN_INSTANT, JodaUtils.MAX_INSTANT), query.getIntervals().get(0));
|
||||
Assert.assertTrue(((SegmentMetadataQuery) query).isUsingDefaultInterval());
|
||||
|
||||
// test serialize and deserialize
|
||||
Assert.assertEquals(query, mapper.readValue(mapper.writeValueAsString(query), Query.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDefaultIntervalAndFiltering() throws Exception
|
||||
{
|
||||
SegmentMetadataQuery testQuery = Druids.newSegmentMetadataQueryBuilder()
|
||||
.dataSource("testing")
|
||||
.toInclude(new ListColumnIncluderator(Arrays.asList("placement")))
|
||||
.merge(true)
|
||||
.build();
|
||||
|
||||
Interval expectedInterval = new Interval(
|
||||
JodaUtils.MIN_INSTANT, JodaUtils.MAX_INSTANT
|
||||
);
|
||||
|
||||
/* No interval specified, should use default interval */
|
||||
Assert.assertTrue(testQuery.isUsingDefaultInterval());
|
||||
Assert.assertEquals(testQuery.getIntervals().get(0), expectedInterval);
|
||||
Assert.assertEquals(testQuery.getIntervals().size(), 1);
|
||||
|
||||
List<LogicalSegment> testSegments = Arrays.asList(
|
||||
new LogicalSegment()
|
||||
{
|
||||
@Override
|
||||
public Interval getInterval()
|
||||
{
|
||||
return new Interval("2012-01-01/P1D");
|
||||
}
|
||||
},
|
||||
new LogicalSegment()
|
||||
{
|
||||
@Override
|
||||
public Interval getInterval()
|
||||
{
|
||||
return new Interval("2012-01-01T01/PT1H");
|
||||
}
|
||||
},
|
||||
new LogicalSegment()
|
||||
{
|
||||
@Override
|
||||
public Interval getInterval()
|
||||
{
|
||||
return new Interval("2013-01-05/P1D");
|
||||
}
|
||||
},
|
||||
new LogicalSegment()
|
||||
{
|
||||
@Override
|
||||
public Interval getInterval()
|
||||
{
|
||||
return new Interval("2013-05-20/P1D");
|
||||
}
|
||||
},
|
||||
new LogicalSegment()
|
||||
{
|
||||
@Override
|
||||
public Interval getInterval()
|
||||
{
|
||||
return new Interval("2014-01-05/P1D");
|
||||
}
|
||||
},
|
||||
new LogicalSegment()
|
||||
{
|
||||
@Override
|
||||
public Interval getInterval()
|
||||
{
|
||||
return new Interval("2014-02-05/P1D");
|
||||
}
|
||||
},
|
||||
new LogicalSegment()
|
||||
{
|
||||
@Override
|
||||
public Interval getInterval()
|
||||
{
|
||||
return new Interval("2015-01-19T01/PT1H");
|
||||
}
|
||||
},
|
||||
new LogicalSegment()
|
||||
{
|
||||
@Override
|
||||
public Interval getInterval()
|
||||
{
|
||||
return new Interval("2015-01-20T02/PT1H");
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
/* Test default period filter */
|
||||
List<LogicalSegment> filteredSegments = new SegmentMetadataQueryQueryToolChest(
|
||||
new SegmentMetadataQueryConfig()
|
||||
).filterSegments(
|
||||
testQuery,
|
||||
testSegments
|
||||
);
|
||||
|
||||
List<LogicalSegment> expectedSegments = Arrays.asList(
|
||||
new LogicalSegment()
|
||||
{
|
||||
@Override
|
||||
public Interval getInterval()
|
||||
{
|
||||
return new Interval("2015-01-19T01/PT1H");
|
||||
}
|
||||
},
|
||||
new LogicalSegment()
|
||||
{
|
||||
@Override
|
||||
public Interval getInterval()
|
||||
{
|
||||
return new Interval("2015-01-20T02/PT1H");
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
Assert.assertEquals(filteredSegments.size(), 2);
|
||||
for (int i = 0; i < filteredSegments.size(); i++) {
|
||||
Assert.assertEquals(expectedSegments.get(i).getInterval(), filteredSegments.get(i).getInterval());
|
||||
}
|
||||
|
||||
/* Test 2 year period filtering */
|
||||
SegmentMetadataQueryConfig twoYearPeriodCfg = new SegmentMetadataQueryConfig("P2Y");
|
||||
List<LogicalSegment> filteredSegments2 = new SegmentMetadataQueryQueryToolChest(
|
||||
twoYearPeriodCfg
|
||||
).filterSegments(
|
||||
testQuery,
|
||||
testSegments
|
||||
);
|
||||
|
||||
List<LogicalSegment> expectedSegments2 = Arrays.asList(
|
||||
new LogicalSegment()
|
||||
{
|
||||
@Override
|
||||
public Interval getInterval()
|
||||
{
|
||||
return new Interval("2013-05-20/P1D");
|
||||
}
|
||||
},
|
||||
new LogicalSegment()
|
||||
{
|
||||
@Override
|
||||
public Interval getInterval()
|
||||
{
|
||||
return new Interval("2014-01-05/P1D");
|
||||
}
|
||||
},
|
||||
new LogicalSegment()
|
||||
{
|
||||
@Override
|
||||
public Interval getInterval()
|
||||
{
|
||||
return new Interval("2014-02-05/P1D");
|
||||
}
|
||||
},
|
||||
new LogicalSegment()
|
||||
{
|
||||
@Override
|
||||
public Interval getInterval()
|
||||
{
|
||||
return new Interval("2015-01-19T01/PT1H");
|
||||
}
|
||||
},
|
||||
new LogicalSegment()
|
||||
{
|
||||
@Override
|
||||
public Interval getInterval()
|
||||
{
|
||||
return new Interval("2015-01-20T02/PT1H");
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
Assert.assertEquals(filteredSegments2.size(), 5);
|
||||
for (int i = 0; i < filteredSegments2.size(); i++) {
|
||||
Assert.assertEquals(expectedSegments2.get(i).getInterval(), filteredSegments2.get(i).getInterval());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,6 +30,7 @@ import io.druid.query.datasourcemetadata.DataSourceMetadataQuery;
|
|||
import io.druid.query.datasourcemetadata.DataSourceQueryQueryToolChest;
|
||||
import io.druid.query.metadata.SegmentMetadataQueryQueryToolChest;
|
||||
import io.druid.query.metadata.metadata.SegmentMetadataQuery;
|
||||
import io.druid.query.metadata.SegmentMetadataQueryConfig;
|
||||
import io.druid.query.search.SearchQueryQueryToolChest;
|
||||
import io.druid.query.search.search.SearchQuery;
|
||||
import io.druid.query.search.search.SearchQueryConfig;
|
||||
|
@ -74,5 +75,6 @@ public class QueryToolChestModule implements Module
|
|||
JsonConfigProvider.bind(binder, "druid.query.groupBy", GroupByQueryConfig.class);
|
||||
JsonConfigProvider.bind(binder, "druid.query.search", SearchQueryConfig.class);
|
||||
JsonConfigProvider.bind(binder, "druid.query.topN", TopNQueryConfig.class);
|
||||
JsonConfigProvider.bind(binder, "druid.query.segmentMetadata", SegmentMetadataQueryConfig.class);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import io.druid.client.InventoryView;
|
|||
import io.druid.client.TimelineServerView;
|
||||
import io.druid.client.selector.ServerSelector;
|
||||
import io.druid.query.TableDataSource;
|
||||
import io.druid.query.metadata.SegmentMetadataQueryConfig;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import io.druid.timeline.TimelineLookup;
|
||||
import io.druid.timeline.TimelineObjectHolder;
|
||||
|
@ -62,19 +63,21 @@ public class ClientInfoResource
|
|||
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;
|
||||
private SegmentMetadataQueryConfig segmentMetadataQueryConfig;
|
||||
|
||||
@Inject
|
||||
public ClientInfoResource(
|
||||
InventoryView serverInventoryView,
|
||||
TimelineServerView timelineServerView
|
||||
TimelineServerView timelineServerView,
|
||||
SegmentMetadataQueryConfig segmentMetadataQueryConfig
|
||||
)
|
||||
{
|
||||
this.serverInventoryView = serverInventoryView;
|
||||
this.timelineServerView = timelineServerView;
|
||||
this.segmentMetadataQueryConfig = (segmentMetadataQueryConfig == null) ?
|
||||
new SegmentMetadataQueryConfig() : segmentMetadataQueryConfig;
|
||||
}
|
||||
|
||||
private Map<String, List<DataSegment>> getSegmentsForDatasources()
|
||||
|
@ -117,18 +120,21 @@ public class ClientInfoResource
|
|||
|
||||
Interval theInterval;
|
||||
if (interval == null || interval.isEmpty()) {
|
||||
DateTime now = new DateTime();
|
||||
theInterval = new Interval(now.minusMillis(SEGMENT_HISTORY_MILLIS), now);
|
||||
DateTime now = getCurrentTime();
|
||||
theInterval = new Interval(segmentMetadataQueryConfig.getDefaultHistory(), now);
|
||||
} else {
|
||||
theInterval = new Interval(interval);
|
||||
}
|
||||
|
||||
TimelineLookup<String, ServerSelector> timeline = timelineServerView.getTimeline(new TableDataSource(dataSourceName));
|
||||
Iterable<TimelineObjectHolder<String, ServerSelector>> serversLookup = timeline != null ? timeline.lookup(theInterval) : null;
|
||||
Iterable<TimelineObjectHolder<String, ServerSelector>> serversLookup = timeline != null ? timeline.lookup(
|
||||
theInterval
|
||||
) : null;
|
||||
if (serversLookup == null || Iterables.isEmpty(serversLookup)) {
|
||||
return Collections.EMPTY_MAP;
|
||||
}
|
||||
Map<Interval,Object> servedIntervals = new TreeMap<>(new Comparator<Interval>()
|
||||
Map<Interval, Object> servedIntervals = new TreeMap<>(
|
||||
new Comparator<Interval>()
|
||||
{
|
||||
@Override
|
||||
public int compare(Interval o1, Interval o2)
|
||||
|
@ -139,17 +145,23 @@ public class ClientInfoResource
|
|||
return o1.isBefore(o2) ? -1 : 1;
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
);
|
||||
|
||||
for (TimelineObjectHolder<String, ServerSelector> holder : serversLookup) {
|
||||
servedIntervals.put(holder.getInterval(), ImmutableMap.of(KEY_DIMENSIONS, Sets.newHashSet(), KEY_METRICS, Sets.newHashSet()));
|
||||
servedIntervals.put(
|
||||
holder.getInterval(),
|
||||
ImmutableMap.of(KEY_DIMENSIONS, Sets.newHashSet(), KEY_METRICS, Sets.newHashSet())
|
||||
);
|
||||
}
|
||||
|
||||
List<DataSegment> segments = getSegmentsForDatasources().get(dataSourceName);
|
||||
if (segments == null || segments.isEmpty()) {
|
||||
log.error("Found no DataSegments but TimelineServerView has served intervals. Datasource = %s , Interval = %s",
|
||||
log.error(
|
||||
"Found no DataSegments but TimelineServerView has served intervals. Datasource = %s , Interval = %s",
|
||||
dataSourceName,
|
||||
theInterval);
|
||||
theInterval
|
||||
);
|
||||
throw new RuntimeException("Internal Error");
|
||||
}
|
||||
|
||||
|
@ -201,8 +213,8 @@ public class ClientInfoResource
|
|||
|
||||
Interval theInterval;
|
||||
if (interval == null || interval.isEmpty()) {
|
||||
DateTime now = new DateTime();
|
||||
theInterval = new Interval(now.minusMillis(SEGMENT_HISTORY_MILLIS), now);
|
||||
DateTime now = getCurrentTime();
|
||||
theInterval = new Interval(segmentMetadataQueryConfig.getDefaultHistory(), now);
|
||||
} else {
|
||||
theInterval = new Interval(interval);
|
||||
}
|
||||
|
@ -233,8 +245,8 @@ public class ClientInfoResource
|
|||
|
||||
Interval theInterval;
|
||||
if (interval == null || interval.isEmpty()) {
|
||||
DateTime now = new DateTime();
|
||||
theInterval = new Interval(now.minusMillis(SEGMENT_HISTORY_MILLIS), now);
|
||||
DateTime now = getCurrentTime();
|
||||
theInterval = new Interval(segmentMetadataQueryConfig.getDefaultHistory(), now);
|
||||
} else {
|
||||
theInterval = new Interval(interval);
|
||||
}
|
||||
|
@ -247,4 +259,11 @@ public class ClientInfoResource
|
|||
|
||||
return metrics;
|
||||
}
|
||||
|
||||
protected DateTime getCurrentTime()
|
||||
{
|
||||
return new DateTime();
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -25,11 +25,13 @@ import io.druid.client.InventoryView;
|
|||
import io.druid.client.TimelineServerView;
|
||||
import io.druid.client.selector.ServerSelector;
|
||||
import io.druid.query.TableDataSource;
|
||||
import io.druid.query.metadata.SegmentMetadataQueryConfig;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import io.druid.timeline.VersionedIntervalTimeline;
|
||||
import io.druid.timeline.partition.SingleElementPartitionChunk;
|
||||
|
||||
import org.easymock.EasyMock;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
|
@ -44,6 +46,8 @@ public class ClientInfoResourceTest
|
|||
{
|
||||
private static final String KEY_DIMENSIONS = "dimensions";
|
||||
private static final String KEY_METRICS = "metrics";
|
||||
private static final DateTime FIXED_TEST_TIME = new DateTime(2015, 9, 14, 0, 0); /* always use the same current time for unit tests */
|
||||
|
||||
|
||||
private final String dataSource = "test-data-source";
|
||||
private final String version = "v0";
|
||||
|
@ -53,10 +57,12 @@ public class ClientInfoResourceTest
|
|||
private ClientInfoResource resource;
|
||||
|
||||
@Before
|
||||
public void setup() {
|
||||
public void setup()
|
||||
{
|
||||
VersionedIntervalTimeline<String, ServerSelector> timeline = new VersionedIntervalTimeline<>(Ordering.<String>natural());
|
||||
DruidServer server = new DruidServer("name", "host", 1234, "type", "tier", 0);
|
||||
|
||||
addSegment(timeline, server, "1960-02-13/1961-02-14", ImmutableList.of("d1"), ImmutableList.of("m1"));
|
||||
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"));
|
||||
|
@ -70,11 +76,15 @@ public class ClientInfoResourceTest
|
|||
|
||||
EasyMock.replay(serverInventoryView, timelineServerView);
|
||||
|
||||
resource = new ClientInfoResource(serverInventoryView, timelineServerView);
|
||||
resource = getResourceTestHelper(
|
||||
serverInventoryView, timelineServerView,
|
||||
new SegmentMetadataQueryConfig()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetDatasourceNonFullWithLargeInterval() {
|
||||
public void testGetDatasourceNonFullWithLargeInterval()
|
||||
{
|
||||
Map<String, Object> actual = resource.getDatasource(dataSource, "1975/2050", null);
|
||||
Map<String, ?> expected = ImmutableMap.of(
|
||||
KEY_DIMENSIONS, ImmutableSet.of("d1", "d2"),
|
||||
|
@ -85,13 +95,17 @@ public class ClientInfoResourceTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testGetDatasourceFullWithLargeInterval() {
|
||||
public void testGetDatasourceFullWithLargeInterval()
|
||||
{
|
||||
|
||||
Map<String, Object> actual = resource.getDatasource(dataSource, "1975/2050", "true");
|
||||
Map<String, ?> 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"))
|
||||
"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);
|
||||
|
@ -99,23 +113,65 @@ public class ClientInfoResourceTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testGetDatasourceFullWithSmallInterval() {
|
||||
Map<String,Object> actual = resource.getDatasource(dataSource, "2014-02-13T09:00:00.000Z/2014-02-17T23:00:00.000Z", "true");
|
||||
public void testGetDatasourceFullWithSmallInterval()
|
||||
{
|
||||
Map<String, Object> actual = resource.getDatasource(
|
||||
dataSource,
|
||||
"2014-02-13T09:00:00.000Z/2014-02-17T23:00:00.000Z",
|
||||
"true"
|
||||
);
|
||||
Map<String, ?> 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"))
|
||||
"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<String, ServerSelector> timeline,
|
||||
@Test
|
||||
public void testGetDatasourceWithDefaultInterval()
|
||||
{
|
||||
Map<String, Object> actual = resource.getDatasource(dataSource, null, "false");
|
||||
Assert.assertEquals(actual.size(), 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetDatasourceWithConfiguredDefaultInterval()
|
||||
{
|
||||
ClientInfoResource defaultResource = getResourceTestHelper(
|
||||
serverInventoryView, timelineServerView,
|
||||
new SegmentMetadataQueryConfig("P100Y")
|
||||
);
|
||||
|
||||
Map<String, ?> expected = ImmutableMap.of(
|
||||
"1960-02-13T00:00:00.000Z/1961-02-14T00:00:00.000Z",
|
||||
ImmutableMap.of(KEY_DIMENSIONS, ImmutableSet.of("d1"), KEY_METRICS, ImmutableSet.of("m1")),
|
||||
"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"))
|
||||
);
|
||||
|
||||
Map<String, Object> actual = defaultResource.getDatasource(dataSource, null, "false");
|
||||
Assert.assertEquals(expected, actual);
|
||||
}
|
||||
|
||||
|
||||
private void addSegment(
|
||||
VersionedIntervalTimeline<String, ServerSelector> timeline,
|
||||
DruidServer server,
|
||||
String interval,
|
||||
List<String> dims,
|
||||
List<String> metrics) {
|
||||
List<String> metrics
|
||||
)
|
||||
{
|
||||
DataSegment segment = DataSegment.builder()
|
||||
.dataSource(dataSource)
|
||||
.interval(new Interval(interval))
|
||||
|
@ -128,4 +184,20 @@ public class ClientInfoResourceTest
|
|||
ServerSelector ss = new ServerSelector(segment, null);
|
||||
timeline.add(new Interval(interval), version, new SingleElementPartitionChunk<ServerSelector>(ss));
|
||||
}
|
||||
|
||||
private ClientInfoResource getResourceTestHelper(
|
||||
InventoryView serverInventoryView,
|
||||
TimelineServerView timelineServerView,
|
||||
SegmentMetadataQueryConfig segmentMetadataQueryConfig
|
||||
)
|
||||
{
|
||||
return new ClientInfoResource(serverInventoryView, timelineServerView, segmentMetadataQueryConfig)
|
||||
{
|
||||
@Override
|
||||
protected DateTime getCurrentTime()
|
||||
{
|
||||
return FIXED_TEST_TIME;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue