From 4d470ffcd916577a64d59f6e07e2b0479fe3f15c Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 27 Mar 2017 01:38:03 +0900 Subject: [PATCH 01/15] Extend Query to be able to have multiple data sources and add JoinQuery --- .../java/io/druid/query/scan/ScanQuery.java | 28 +- .../query/scan/ScanQueryQueryToolChest.java | 12 + .../sql/QuantileSqlAggregatorTest.java | 2 + .../overlord/ThreadPoolTaskRunner.java | 8 +- .../common/task/RealtimeIndexTaskTest.java | 1 + .../main/java/io/druid/query/BaseQuery.java | 71 +-- .../io/druid/query/BySegmentQueryRunner.java | 7 +- .../main/java/io/druid/query/DataSource.java | 14 +- .../java/io/druid/query/DataSourceUtil.java | 23 + .../query/DataSourceWithSegmentSpec.java | 80 ++++ .../io/druid/query/DefaultQueryMetrics.java | 18 +- .../src/main/java/io/druid/query/Druids.java | 85 +++- .../druid/query/FluentQueryRunnerBuilder.java | 7 + .../query/IntervalChunkingQueryRunner.java | 9 +- .../src/main/java/io/druid/query/Queries.java | 22 + .../src/main/java/io/druid/query/Query.java | 40 +- .../java/io/druid/query/QueryContextKeys.java | 1 + .../java/io/druid/query/QueryDataSource.java | 9 +- .../java/io/druid/query/QueryMetrics.java | 11 +- .../io/druid/query/QuerySegmentWalker.java | 4 +- .../java/io/druid/query/QueryToolChest.java | 13 + .../ReferenceCountingSegmentQueryRunner.java | 9 +- ...portTimelineMissingSegmentQueryRunner.java | 20 +- .../java/io/druid/query/RetryQueryRunner.java | 29 +- .../io/druid/query/SingleSourceBaseQuery.java | 200 +++++++++ .../io/druid/query/SubqueryQueryRunner.java | 4 +- .../java/io/druid/query/TimewarpOperator.java | 9 +- .../java/io/druid/query/UnionQueryRunner.java | 45 +- .../DataSourceMetadataQuery.java | 4 +- .../DataSourceQueryQueryToolChest.java | 14 + .../dimension/BaseFilteredDimensionSpec.java | 6 + .../query/dimension/DefaultDimensionSpec.java | 30 +- .../druid/query/dimension/DimensionSpec.java | 2 + .../dimension/ExtractionDimensionSpec.java | 18 +- .../query/dimension/LookupDimensionSpec.java | 35 ++ .../io/druid/query/groupby/GroupByQuery.java | 25 +- .../groupby/GroupByQueryQueryToolChest.java | 12 + .../epinephelinae/RowBasedGrouperHelper.java | 2 +- .../groupby/strategy/GroupByStrategyV2.java | 44 +- .../io/druid/query/join/AddPredicate.java | 46 ++ .../io/druid/query/join/AndPredicate.java | 78 ++++ .../io/druid/query/join/BinaryPredicate.java | 69 +++ .../java/io/druid/query/join/DataInput.java | 114 +++++ .../druid/query/join/DimExtractPredicate.java | 78 ++++ .../io/druid/query/join/DividePredicate.java | 46 ++ .../io/druid/query/join/EqualPredicate.java | 47 ++ .../io/druid/query/join/JoinInputSpec.java | 34 ++ .../io/druid/query/join/JoinPredicate.java | 58 +++ .../query/join/JoinPredicateVisitor.java | 62 +++ .../java/io/druid/query/join/JoinQuery.java | 418 ++++++++++++++++++ .../java/io/druid/query/join/JoinSpec.java | 115 +++++ .../io/druid/query/join/JoinSpecVisitor.java | 35 ++ .../java/io/druid/query/join/JoinType.java | 31 ++ .../io/druid/query/join/LiteralPredicate.java | 75 ++++ .../druid/query/join/MultiplyPredicate.java | 46 ++ .../io/druid/query/join/NotPredicate.java | 77 ++++ .../java/io/druid/query/join/OrPredicate.java | 78 ++++ .../druid/query/join/SubtractPredicate.java | 46 ++ .../SegmentMetadataQueryQueryToolChest.java | 12 + .../metadata/SegmentMetadataQuery.java | 4 +- .../search/SearchQueryQueryToolChest.java | 12 + .../query/search/search/SearchQuery.java | 4 +- .../io/druid/query/select/SelectQuery.java | 4 +- .../select/SelectQueryQueryToolChest.java | 12 + .../io/druid/query/spec/QuerySegmentSpec.java | 4 +- .../spec/SpecificSegmentQueryRunner.java | 22 +- .../druid/query/spec/SpecificSegmentSpec.java | 3 +- .../query/timeboundary/TimeBoundaryQuery.java | 4 +- .../TimeBoundaryQueryQueryToolChest.java | 12 + .../query/timeseries/TimeseriesQuery.java | 4 +- .../TimeseriesQueryQueryToolChest.java | 12 + .../java/io/druid/query/topn/TopNQuery.java | 4 +- .../io/druid/query/topn/TopNQueryBuilder.java | 25 +- .../query/topn/TopNQueryQueryToolChest.java | 12 + .../druid/query/DefaultQueryMetricsTest.java | 13 +- .../IntervalChunkingQueryRunnerTest.java | 3 + .../io/druid/query/QueryRunnerTestHelper.java | 42 +- .../io/druid/query/RetryQueryRunnerTest.java | 121 +++-- .../io/druid/query/SchemaEvolutionTest.java | 2 +- .../io/druid/query/TimewarpOperatorTest.java | 29 +- .../io/druid/query/UnionQueryRunnerTest.java | 6 +- .../DefaultGroupByQueryMetricsTest.java | 11 +- .../query/groupby/GroupByQueryRunnerTest.java | 25 +- .../groupby/GroupByQueryRunnerTestHelper.java | 12 +- .../io/druid/query/join/JoinQueryTest.java | 100 +++++ .../metadata/SegmentMetadataQueryTest.java | 14 +- .../query/select/SelectQuerySpecTest.java | 8 +- .../spec/SpecificSegmentQueryRunnerTest.java | 2 + .../DefaultTimeseriesQueryMetricsTest.java | 14 +- .../TimeSeriesUnionQueryRunnerTest.java | 2 +- .../topn/DefaultTopNQueryMetricsTest.java | 14 +- .../topn/TopNQueryQueryToolChestTest.java | 2 +- .../druid/client/CachingClusteredClient.java | 24 +- .../druid/client/selector/HostSelector.java | 6 +- .../segment/realtime/RealtimeManager.java | 15 +- .../appenderator/SinkQuerySegmentWalker.java | 43 +- .../realtime/plumber/RealtimePlumber.java | 6 +- .../io/druid/server/BrokerQueryResource.java | 14 +- .../server/ClientQuerySegmentWalker.java | 3 +- .../java/io/druid/server/QueryManager.java | 12 +- .../java/io/druid/server/QueryResource.java | 23 +- .../server/coordination/ServerManager.java | 167 ++++++- .../server/log/LoggingRequestLogger.java | 6 +- .../router/TieredBrokerHostSelector.java | 64 +-- ...chingClusteredClientFunctionalityTest.java | 20 +- .../client/CachingClusteredClientTest.java | 80 +++- .../segment/realtime/RealtimeManagerTest.java | 38 +- .../appenderator/AppenderatorTest.java | 7 + .../coordination/ServerManagerTest.java | 8 +- .../server/log/LoggingRequestLoggerTest.java | 8 +- .../io/druid/sql/calcite/rel/QueryMaker.java | 27 +- .../druid/sql/calcite/schema/DruidSchema.java | 4 +- .../druid/sql/calcite/CalciteQueryTest.java | 116 +++++ .../SpecificSegmentsQuerySegmentWalker.java | 10 +- 114 files changed, 3319 insertions(+), 478 deletions(-) create mode 100644 processing/src/main/java/io/druid/query/DataSourceWithSegmentSpec.java create mode 100644 processing/src/main/java/io/druid/query/SingleSourceBaseQuery.java create mode 100644 processing/src/main/java/io/druid/query/join/AddPredicate.java create mode 100644 processing/src/main/java/io/druid/query/join/AndPredicate.java create mode 100644 processing/src/main/java/io/druid/query/join/BinaryPredicate.java create mode 100644 processing/src/main/java/io/druid/query/join/DataInput.java create mode 100644 processing/src/main/java/io/druid/query/join/DimExtractPredicate.java create mode 100644 processing/src/main/java/io/druid/query/join/DividePredicate.java create mode 100644 processing/src/main/java/io/druid/query/join/EqualPredicate.java create mode 100644 processing/src/main/java/io/druid/query/join/JoinInputSpec.java create mode 100644 processing/src/main/java/io/druid/query/join/JoinPredicate.java create mode 100644 processing/src/main/java/io/druid/query/join/JoinPredicateVisitor.java create mode 100644 processing/src/main/java/io/druid/query/join/JoinQuery.java create mode 100644 processing/src/main/java/io/druid/query/join/JoinSpec.java create mode 100644 processing/src/main/java/io/druid/query/join/JoinSpecVisitor.java create mode 100644 processing/src/main/java/io/druid/query/join/JoinType.java create mode 100644 processing/src/main/java/io/druid/query/join/LiteralPredicate.java create mode 100644 processing/src/main/java/io/druid/query/join/MultiplyPredicate.java create mode 100644 processing/src/main/java/io/druid/query/join/NotPredicate.java create mode 100644 processing/src/main/java/io/druid/query/join/OrPredicate.java create mode 100644 processing/src/main/java/io/druid/query/join/SubtractPredicate.java create mode 100644 processing/src/test/java/io/druid/query/join/JoinQueryTest.java diff --git a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQuery.java b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQuery.java index df6a4079d5cb..fdbd01e4a0c1 100644 --- a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQuery.java +++ b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQuery.java @@ -23,9 +23,11 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -import io.druid.query.BaseQuery; import io.druid.query.DataSource; +import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.Query; +import io.druid.query.QueryContextKeys; +import io.druid.query.SingleSourceBaseQuery; import io.druid.query.TableDataSource; import io.druid.query.filter.DimFilter; import io.druid.query.filter.InDimFilter; @@ -35,11 +37,12 @@ import org.joda.time.Interval; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; @JsonTypeName("scan") -public class ScanQuery extends BaseQuery +public class ScanQuery extends SingleSourceBaseQuery { public static final String SCAN = "scan"; public static final String RESULT_FORMAT_LIST = "list"; @@ -310,6 +313,21 @@ public ScanQueryBuilder dataSource(DataSource ds) return this; } + public ScanQueryBuilder updateDistributionTarget() + { + if (context == null) { + context = new HashMap<>(); + } + context.put( + QueryContextKeys.DIST_TARGET_SOURCE, + new DataSourceWithSegmentSpec( + SingleSourceBaseQuery.getLeafDataSource(dataSource), + querySegmentSpec + ) + ); + return this; + } + public ScanQueryBuilder intervals(QuerySegmentSpec q) { querySegmentSpec = q; @@ -330,7 +348,11 @@ public ScanQueryBuilder intervals(List l) public ScanQueryBuilder context(Map c) { - context = c; + if (context == null) { + context = new HashMap<>(c); + } else { + context.putAll(c); + } return this; } diff --git a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java index 8cc1c7e3ffce..e40cc7100784 100644 --- a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java +++ b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java @@ -120,4 +120,16 @@ public Sequence run( } }; } + + @Override + public QueryRunner annotateDistributionTarget(QueryRunner runner) + { + return (query, responseContext) -> { + ScanQuery scanQuery = (ScanQuery) query; + return runner.run( + scanQuery.distributeBy(scanQuery.getDataSourceWithSegmentSpec()), + responseContext + ); + }; + } } diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java index e4eb972d320a..61b2a7af81e2 100644 --- a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java @@ -200,6 +200,7 @@ public void testQuantileOnFloatAndLongs() throws Exception new QuantilePostAggregator("a6", "a4:agg", 0.999f), new QuantilePostAggregator("a7", "a7:agg", 0.50f) )) + .updateDistributionTarget() .context(ImmutableMap.of("skipEmptyBuckets", true)) .build(), Iterables.getOnlyElement(queryLogHook.getRecordedQueries()) @@ -261,6 +262,7 @@ public void testQuantileOnComplexColumn() throws Exception new QuantilePostAggregator("a6", "a4:agg", 0.999f) )) .context(ImmutableMap.of("skipEmptyBuckets", true)) + .updateDistributionTarget() .build(), Iterables.getOnlyElement(queryLogHook.getRecordedQueries()) ); diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java index faba5b129f82..e040aa665191 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java @@ -45,6 +45,7 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; import io.druid.java.util.common.lifecycle.LifecycleStop; +import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.NoopQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; @@ -343,11 +344,12 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable QueryRunner getQueryRunnerImpl(Query query) { QueryRunner queryRunner = null; - final String queryDataSource = Iterables.getOnlyElement(query.getDataSource().getNames()); + final DataSourceWithSegmentSpec spec = query.getDistributionTarget(); + final String dataSourceName = Iterables.getOnlyElement(spec.getDataSource().getNames()); for (final ThreadPoolTaskRunnerWorkItem taskRunnerWorkItem : ImmutableList.copyOf(runningItems)) { final Task task = taskRunnerWorkItem.getTask(); - if (task.getDataSource().equals(queryDataSource)) { + if (task.getDataSource().equals(dataSourceName)) { final QueryRunner taskQueryRunner = task.getQueryRunner(query); if (taskQueryRunner != null) { @@ -355,7 +357,7 @@ private QueryRunner getQueryRunnerImpl(Query query) queryRunner = taskQueryRunner; } else { log.makeAlert("Found too many query runners for datasource") - .addData("dataSource", queryDataSource) + .addData("dataSource", dataSourceName) .emit(); } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index 7f4464f7d540..c23c0b941863 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -1081,6 +1081,7 @@ public long sumMetric(final Task task, final String metric) throws Exception ) ).granularity(Granularities.ALL) .intervals("2000/3000") + .updateDistributionTarget() .build(); ArrayList> results = Sequences.toList( diff --git a/processing/src/main/java/io/druid/query/BaseQuery.java b/processing/src/main/java/io/druid/query/BaseQuery.java index 22d0fb1a4ba5..348f4090465b 100644 --- a/processing/src/main/java/io/druid/query/BaseQuery.java +++ b/processing/src/main/java/io/druid/query/BaseQuery.java @@ -20,7 +20,6 @@ package io.druid.query; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; @@ -30,7 +29,6 @@ import org.joda.time.Duration; import org.joda.time.Interval; -import java.util.List; import java.util.Map; /** @@ -105,35 +103,19 @@ public static void checkInterrupted() } public static final String QUERYID = "queryId"; - private final DataSource dataSource; + private final boolean descending; private final Map context; - private final QuerySegmentSpec querySegmentSpec; - private volatile Duration duration; public BaseQuery( - DataSource dataSource, - QuerySegmentSpec querySegmentSpec, boolean descending, Map context ) { - Preconditions.checkNotNull(dataSource, "dataSource can't be null"); - Preconditions.checkNotNull(querySegmentSpec, "querySegmentSpec can't be null"); - - this.dataSource = dataSource; this.context = context; - this.querySegmentSpec = querySegmentSpec; this.descending = descending; } - @JsonProperty - @Override - public DataSource getDataSource() - { - return dataSource; - } - @JsonProperty @Override public boolean isDescending() @@ -141,43 +123,20 @@ public boolean isDescending() return descending; } - @JsonProperty("intervals") - public QuerySegmentSpec getQuerySegmentSpec() - { - return querySegmentSpec; - } - - @Override - public Sequence run(QuerySegmentWalker walker, Map context) - { - return run(querySegmentSpec.lookup(this, walker), context); - } - public Sequence run(QueryRunner runner, Map context) { return runner.run(this, context); } - @Override - public List getIntervals() + public static Duration initDuration(QuerySegmentSpec querySegmentSpec) { - return querySegmentSpec.getIntervals(); - } - - @Override - public Duration getDuration() - { - if (duration == null) { - Duration totalDuration = new Duration(0); - for (Interval interval : querySegmentSpec.getIntervals()) { - if (interval != null) { - totalDuration = totalDuration.plus(interval.toDuration()); - } + Duration totalDuration = new Duration(0); + for (Interval interval : querySegmentSpec.getIntervals()) { + if (interval != null) { + totalDuration = totalDuration.plus(interval.toDuration()); } - duration = totalDuration; } - - return duration; + return totalDuration; } @Override @@ -255,17 +214,6 @@ public boolean equals(Object o) if (context != null ? !context.equals(baseQuery.context) : baseQuery.context != null) { return false; } - if (dataSource != null ? !dataSource.equals(baseQuery.dataSource) : baseQuery.dataSource != null) { - return false; - } - if (duration != null ? !duration.equals(baseQuery.duration) : baseQuery.duration != null) { - return false; - } - if (querySegmentSpec != null - ? !querySegmentSpec.equals(baseQuery.querySegmentSpec) - : baseQuery.querySegmentSpec != null) { - return false; - } return true; } @@ -273,11 +221,8 @@ public boolean equals(Object o) @Override public int hashCode() { - int result = dataSource != null ? dataSource.hashCode() : 0; - result = 31 * result + (descending ? 1 : 0); + int result = (descending ? 1 : 0); result = 31 * result + (context != null ? context.hashCode() : 0); - result = 31 * result + (querySegmentSpec != null ? querySegmentSpec.hashCode() : 0); - result = 31 * result + (duration != null ? duration.hashCode() : 0); return result; } } diff --git a/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java b/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java index a2a8a960b4f8..96b43bf32ffd 100644 --- a/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java @@ -19,6 +19,7 @@ package io.druid.query; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; @@ -61,7 +62,11 @@ public Sequence run(final Query query, Map responseContext new BySegmentResultValueClass( results, segmentIdentifier, - query.getIntervals().get(0) + Iterables.getOnlyElement( + Iterables.getOnlyElement(query.getDataSources()) + .getQuerySegmentSpec() + .getIntervals() + ) ) ) ) diff --git a/processing/src/main/java/io/druid/query/DataSource.java b/processing/src/main/java/io/druid/query/DataSource.java index 6493079ba87b..70e5d77083e2 100644 --- a/processing/src/main/java/io/druid/query/DataSource.java +++ b/processing/src/main/java/io/druid/query/DataSource.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.common.collect.Iterables; import java.util.List; @@ -35,5 +36,16 @@ }) public interface DataSource { - public List getNames(); + default String getConcatenatedName() + { + final List names = getNames(); + return names.size() > 1 ? names.toString() : names.get(0); + } + + default String getFirstName() + { + return Iterables.getFirst(getNames(), null); + } + + List getNames(); } diff --git a/processing/src/main/java/io/druid/query/DataSourceUtil.java b/processing/src/main/java/io/druid/query/DataSourceUtil.java index c624bf87ce48..43b4310257a5 100644 --- a/processing/src/main/java/io/druid/query/DataSourceUtil.java +++ b/processing/src/main/java/io/druid/query/DataSourceUtil.java @@ -19,7 +19,12 @@ package io.druid.query; +import org.joda.time.Interval; + import java.util.List; +import java.util.StringJoiner; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; public class DataSourceUtil { @@ -28,4 +33,22 @@ public static String getMetricName(DataSource dataSource) final List names = dataSource.getNames(); return names.size() == 1 ? names.get(0) : names.toString(); } + + public static String getMetricName(Iterable dataSources) + { + return StreamSupport.stream(dataSources.spliterator(), false) + .map(DataSourceUtil::getMetricName) + .collect(Collectors.joining(",", "[", "]")); + } + + private static final StringJoiner JOINER = new StringJoiner(",", "[", "]"); + private static String getMetricName(DataSourceWithSegmentSpec spec) + { + JOINER.add(getMetricName(spec.getDataSource())).add("="); + JOINER.add(spec.getQuerySegmentSpec().getIntervals().stream() + .map(Interval::toString) + .collect(Collectors.joining(",", "[", "]")) + ); + return JOINER.toString(); + } } diff --git a/processing/src/main/java/io/druid/query/DataSourceWithSegmentSpec.java b/processing/src/main/java/io/druid/query/DataSourceWithSegmentSpec.java new file mode 100644 index 000000000000..104e3d775a2c --- /dev/null +++ b/processing/src/main/java/io/druid/query/DataSourceWithSegmentSpec.java @@ -0,0 +1,80 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.query.spec.QuerySegmentSpec; + +import java.util.Objects; + +public class DataSourceWithSegmentSpec +{ + private final DataSource dataSource; + private final QuerySegmentSpec querySegmentSpec; + + @JsonCreator + public DataSourceWithSegmentSpec( + @JsonProperty("dataSource") DataSource dataSource, + @JsonProperty("intervals") QuerySegmentSpec querySegmentSpec) + { + this.dataSource = dataSource; + this.querySegmentSpec = querySegmentSpec; + } + + @JsonProperty + public DataSource getDataSource() + { + return dataSource; + } + + @JsonProperty + public QuerySegmentSpec getQuerySegmentSpec() + { + return querySegmentSpec; + } + + @Override + public int hashCode() + { + return Objects.hash(dataSource, querySegmentSpec); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + + if (o == null || o.getClass() != getClass()) { + return false; + } + + DataSourceWithSegmentSpec that = (DataSourceWithSegmentSpec) o; + + if (!dataSource.equals(that.dataSource)) { + return false; + } + + return querySegmentSpec.equals(that.querySegmentSpec); + } + +} diff --git a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java index a19053be8789..12d426107f4c 100644 --- a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java @@ -45,18 +45,17 @@ public DefaultQueryMetrics(ObjectMapper jsonMapper) @Override public void query(QueryType query) { - dataSource(query); + dataSources(query); queryType(query); - interval(query); hasFilters(query); duration(query); queryId(query); } @Override - public void dataSource(QueryType query) + public void dataSources(QueryType query) { - builder.setDimension(DruidMetrics.DATASOURCE, DataSourceUtil.getMetricName(query.getDataSource())); + builder.setDimension(DruidMetrics.DATASOURCE, DataSourceUtil.getMetricName(query.getDataSources())); } @Override @@ -65,15 +64,6 @@ public void queryType(QueryType query) builder.setDimension(DruidMetrics.TYPE, query.getType()); } - @Override - public void interval(QueryType query) - { - builder.setDimension( - DruidMetrics.INTERVAL, - query.getIntervals().stream().map(Interval::toString).toArray(String[]::new) - ); - } - @Override public void hasFilters(QueryType query) { @@ -83,7 +73,7 @@ public void hasFilters(QueryType query) @Override public void duration(QueryType query) { - builder.setDimension("duration", query.getDuration().toString()); + builder.setDimension("duration", query.getTotalDuration().toString()); } @Override diff --git a/processing/src/main/java/io/druid/query/Druids.java b/processing/src/main/java/io/druid/query/Druids.java index a835394052f3..4aa14f30864b 100644 --- a/processing/src/main/java/io/druid/query/Druids.java +++ b/processing/src/main/java/io/druid/query/Druids.java @@ -62,6 +62,7 @@ import javax.annotation.Nullable; import java.util.Arrays; import java.util.EnumSet; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -445,6 +446,21 @@ public TimeseriesQueryBuilder dataSource(DataSource ds) return this; } + public TimeseriesQueryBuilder updateDistributionTarget() + { + if (context == null) { + context = new HashMap<>(); + } + context.put( + QueryContextKeys.DIST_TARGET_SOURCE, + new DataSourceWithSegmentSpec( + SingleSourceBaseQuery.getLeafDataSource(dataSource), + querySegmentSpec + ) + ); + return this; + } + public TimeseriesQueryBuilder intervals(QuerySegmentSpec q) { querySegmentSpec = q; @@ -529,7 +545,11 @@ public TimeseriesQueryBuilder postAggregators(List p) public TimeseriesQueryBuilder context(Map c) { - context = c; + if (context == null) { + context = new HashMap<>(c); + } else { + context.putAll(c); + } return this; } } @@ -635,6 +655,21 @@ public SearchQueryBuilder dataSource(DataSource d) return this; } + public SearchQueryBuilder updateDistributionTarget() + { + if (context == null) { + context = new HashMap<>(); + } + context.put( + QueryContextKeys.DIST_TARGET_SOURCE, + new DataSourceWithSegmentSpec( + SingleSourceBaseQuery.getLeafDataSource(dataSource), + querySegmentSpec + ) + ); + return this; + } + public SearchQueryBuilder filters(String dimensionName, String value) { dimFilter = new SelectorDimFilter(dimensionName, value, null); @@ -767,7 +802,11 @@ public SearchQueryBuilder fragments(List q, boolean caseSensitive) public SearchQueryBuilder context(Map c) { - context = c; + if (context == null) { + context = new HashMap<>(c); + } else { + context.putAll(c); + } return this; } } @@ -841,6 +880,21 @@ public TimeBoundaryQueryBuilder dataSource(DataSource ds) return this; } + public TimeBoundaryQueryBuilder updateDistributionTarget() + { + if (context == null) { + context = new HashMap<>(); + } + context.put( + QueryContextKeys.DIST_TARGET_SOURCE, + new DataSourceWithSegmentSpec( + SingleSourceBaseQuery.getLeafDataSource(dataSource), + querySegmentSpec + ) + ); + return this; + } + public TimeBoundaryQueryBuilder intervals(QuerySegmentSpec q) { querySegmentSpec = q; @@ -885,7 +939,11 @@ public TimeBoundaryQueryBuilder filters(DimFilter f) public TimeBoundaryQueryBuilder context(Map c) { - context = c; + if (context == null) { + context = new HashMap<>(c); + } else { + context.putAll(c); + } return this; } } @@ -1175,6 +1233,21 @@ public SelectQueryBuilder dataSource(DataSource ds) return this; } + public SelectQueryBuilder updateDistributionTarget() + { + if (context == null) { + context = new HashMap<>(); + } + context.put( + QueryContextKeys.DIST_TARGET_SOURCE, + new DataSourceWithSegmentSpec( + SingleSourceBaseQuery.getLeafDataSource(dataSource), + querySegmentSpec + ) + ); + return this; + } + public SelectQueryBuilder intervals(QuerySegmentSpec q) { querySegmentSpec = q; @@ -1201,7 +1274,11 @@ public SelectQueryBuilder descending(boolean descending) public SelectQueryBuilder context(Map c) { - context = c; + if (context == null) { + context = new HashMap<>(c); + } else { + context.putAll(c); + } return this; } diff --git a/processing/src/main/java/io/druid/query/FluentQueryRunnerBuilder.java b/processing/src/main/java/io/druid/query/FluentQueryRunnerBuilder.java index 71bf44451cda..a61401cc61b4 100644 --- a/processing/src/main/java/io/druid/query/FluentQueryRunnerBuilder.java +++ b/processing/src/main/java/io/druid/query/FluentQueryRunnerBuilder.java @@ -109,5 +109,12 @@ public FluentQueryRunner mergeResults() toolChest.mergeResults(baseRunner) ); } + + public FluentQueryRunner annotateDistributionTarget() + { + return from( + toolChest.annotateDistributionTarget(baseRunner) + ); + } } } diff --git a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java index a42ce69fde0e..abfc0ebd7e5e 100644 --- a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java @@ -20,6 +20,7 @@ package io.druid.query; import com.google.common.base.Function; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.metamx.emitter.service.ServiceEmitter; import io.druid.java.util.common.granularity.PeriodGranularity; @@ -72,9 +73,10 @@ public Sequence run(final Query query, final Map responseC return baseRunner.run(query, responseContext); } + final DataSourceWithSegmentSpec spec = query.getDistributionTarget(); List chunkIntervals = Lists.newArrayList( FunctionalIterable - .create(query.getIntervals()) + .create(spec.getQuerySegmentSpec().getIntervals()) .transformCat( new Function>() { @@ -113,7 +115,10 @@ public Sequence apply(Interval singleInterval) ), executor, queryWatcher ).run( - query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Arrays.asList(singleInterval))), + query.replaceQuerySegmentSpecWith( + Iterables.getOnlyElement(spec.getDataSource().getNames()), + new MultipleIntervalSegmentSpec(Arrays.asList(singleInterval)) + ), responseContext ); } diff --git a/processing/src/main/java/io/druid/query/Queries.java b/processing/src/main/java/io/druid/query/Queries.java index d62050965b42..1b85ee79e4ea 100644 --- a/processing/src/main/java/io/druid/query/Queries.java +++ b/processing/src/main/java/io/druid/query/Queries.java @@ -23,12 +23,16 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import io.druid.java.util.common.Pair; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; +import org.joda.time.Interval; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; /** */ @@ -80,4 +84,22 @@ public static List prepareAggregations( return postAggs; } + + public static Pair getDataSourceAndIntervalStrings(final Query query) + { + final List datasourceNames = new ArrayList<>(); + final List intervals = new ArrayList<>(); + + query.getDataSources().forEach(spec -> { + datasourceNames.addAll(spec.getDataSource().getNames()); + intervals.addAll( + spec.getQuerySegmentSpec().getIntervals().stream().map(Interval::toString).collect(Collectors.toList()) + ); + }); + + return new Pair<>( + datasourceNames.stream().collect(Collectors.joining(",", "[", "]")), + intervals.stream().collect(Collectors.joining(",", "[", "]")) + ); + } } diff --git a/processing/src/main/java/io/druid/query/Query.java b/processing/src/main/java/io/druid/query/Query.java index 9ad178161ead..eed89597f2a0 100644 --- a/processing/src/main/java/io/druid/query/Query.java +++ b/processing/src/main/java/io/druid/query/Query.java @@ -21,11 +21,13 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Ordering; import io.druid.java.util.common.guava.Sequence; import io.druid.query.datasourcemetadata.DataSourceMetadataQuery; import io.druid.query.filter.DimFilter; import io.druid.query.groupby.GroupByQuery; +import io.druid.query.join.JoinQuery; import io.druid.query.metadata.metadata.SegmentMetadataQuery; import io.druid.query.search.search.SearchQuery; import io.druid.query.select.SelectQuery; @@ -34,10 +36,9 @@ import io.druid.query.timeseries.TimeseriesQuery; import io.druid.query.topn.TopNQuery; import org.joda.time.Duration; -import org.joda.time.Interval; -import java.util.List; import java.util.Map; +import java.util.stream.StreamSupport; @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "queryType") @JsonSubTypes(value = { @@ -48,6 +49,7 @@ @JsonSubTypes.Type(name = Query.SEGMENT_METADATA, value = SegmentMetadataQuery.class), @JsonSubTypes.Type(name = Query.SELECT, value = SelectQuery.class), @JsonSubTypes.Type(name = Query.TOPN, value = TopNQuery.class), + @JsonSubTypes.Type(name = Query.JOIN, value = JoinQuery.class), @JsonSubTypes.Type(name = Query.DATASOURCE_METADATA, value = DataSourceMetadataQuery.class) }) @@ -61,8 +63,9 @@ public interface Query String SELECT = "select"; String TOPN = "topN"; String DATASOURCE_METADATA = "dataSourceMetadata"; + String JOIN = "join"; - DataSource getDataSource(); + Iterable getDataSources(); boolean hasFilters(); @@ -74,9 +77,14 @@ public interface Query Sequence run(QueryRunner runner, Map context); - List getIntervals(); + Duration getDuration(DataSource dataSource); - Duration getDuration(); + default Duration getTotalDuration() + { + return StreamSupport.stream(getDataSources().spliterator(), false) + .map(spec -> BaseQuery.initDuration(spec.getQuerySegmentSpec())) + .reduce(new Duration(0), Duration::plus); + } Map getContext(); @@ -90,13 +98,25 @@ public interface Query Ordering getResultOrdering(); - Query withOverriddenContext(Map contextOverride); - - Query withQuerySegmentSpec(QuerySegmentSpec spec); + String getId(); Query withId(String id); - String getId(); + default DataSourceWithSegmentSpec getDistributionTarget() + { + return getContextValue(QueryContextKeys.DIST_TARGET_SOURCE); + } + + default Query distributeBy(DataSourceWithSegmentSpec spec) + { + return withOverriddenContext(ImmutableMap.of(QueryContextKeys.DIST_TARGET_SOURCE, spec)); + } + + Query withOverriddenContext(Map contextOverride); + + Query replaceQuerySegmentSpecWith(DataSource dataSource, QuerySegmentSpec spec); + + Query replaceQuerySegmentSpecWith(String dataSource, QuerySegmentSpec spec); - Query withDataSource(DataSource dataSource); + Query replaceDataSourceWith(DataSource src, DataSource dst); } diff --git a/processing/src/main/java/io/druid/query/QueryContextKeys.java b/processing/src/main/java/io/druid/query/QueryContextKeys.java index 480dcd551f4f..abc5a7cbb0f4 100644 --- a/processing/src/main/java/io/druid/query/QueryContextKeys.java +++ b/processing/src/main/java/io/druid/query/QueryContextKeys.java @@ -24,4 +24,5 @@ public class QueryContextKeys public static final String PRIORITY = "priority"; public static final String TIMEOUT = "timeout"; public static final String CHUNK_PERIOD = "chunkPeriod"; + public static final String DIST_TARGET_SOURCE = "distTargetSource"; } diff --git a/processing/src/main/java/io/druid/query/QueryDataSource.java b/processing/src/main/java/io/druid/query/QueryDataSource.java index 0797f2f5f807..74c355863386 100644 --- a/processing/src/main/java/io/druid/query/QueryDataSource.java +++ b/processing/src/main/java/io/druid/query/QueryDataSource.java @@ -24,12 +24,14 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; @JsonTypeName("query") public class QueryDataSource implements DataSource { @JsonProperty - private final Query query; + private final Query query; @JsonCreator public QueryDataSource(@JsonProperty("query") Query query) @@ -40,7 +42,10 @@ public QueryDataSource(@JsonProperty("query") Query query) @Override public List getNames() { - return query.getDataSource().getNames(); + final Iterable sourceSpecs = query.getDataSources(); + return StreamSupport.stream(sourceSpecs.spliterator(), false) + .flatMap(spec -> spec.getDataSource().getNames().stream()) + .collect(Collectors.toList()); } @JsonProperty diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index 3afc3a122cb8..45e838d38f72 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -146,27 +146,22 @@ public interface QueryMetrics> void query(QueryType query); /** - * Sets {@link Query#getDataSource()} of the given query as dimension. + * Sets {@link Query#getDataSources()} of the given query as dimension. */ - void dataSource(QueryType query); + void dataSources(QueryType query); /** * Sets {@link Query#getType()} of the given query as dimension. */ void queryType(QueryType query); - /** - * Sets {@link Query#getIntervals()} of the given query as dimension. - */ - void interval(QueryType query); - /** * Sets {@link Query#hasFilters()} of the given query as dimension. */ void hasFilters(QueryType query); /** - * Sets {@link Query#getDuration()} of the given query as dimension. + * Sets {@link Query#getTotalDuration()} of the given query as dimension. */ void duration(QueryType query); diff --git a/processing/src/main/java/io/druid/query/QuerySegmentWalker.java b/processing/src/main/java/io/druid/query/QuerySegmentWalker.java index 5f865df4d8d6..de0b4f6a5e06 100644 --- a/processing/src/main/java/io/druid/query/QuerySegmentWalker.java +++ b/processing/src/main/java/io/druid/query/QuerySegmentWalker.java @@ -34,7 +34,7 @@ public interface QuerySegmentWalker * @param intervals the intervals to find a Queryable for * @return a Queryable object that represents the interval */ - public QueryRunner getQueryRunnerForIntervals(Query query, Iterable intervals); + QueryRunner getQueryRunnerForIntervals(Query query, Iterable intervals); /** * Gets the Queryable for a given list of SegmentSpecs. @@ -44,5 +44,5 @@ public interface QuerySegmentWalker * @param specs the list of SegmentSpecs to find a Queryable for * @return the Queryable object with the given SegmentSpecs */ - public QueryRunner getQueryRunnerForSegments(Query query, Iterable specs); + QueryRunner getQueryRunnerForSegments(Query query, Iterable specs); } diff --git a/processing/src/main/java/io/druid/query/QueryToolChest.java b/processing/src/main/java/io/druid/query/QueryToolChest.java index daabdfcf18e3..3da0fbbf40aa 100644 --- a/processing/src/main/java/io/druid/query/QueryToolChest.java +++ b/processing/src/main/java/io/druid/query/QueryToolChest.java @@ -179,4 +179,17 @@ public List filterSegments(QueryType query, List annotateDistributionTarget(QueryRunner runner); } diff --git a/processing/src/main/java/io/druid/query/ReferenceCountingSegmentQueryRunner.java b/processing/src/main/java/io/druid/query/ReferenceCountingSegmentQueryRunner.java index a85a6f911286..460e14d5ddbf 100644 --- a/processing/src/main/java/io/druid/query/ReferenceCountingSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ReferenceCountingSegmentQueryRunner.java @@ -25,6 +25,7 @@ import io.druid.segment.ReferenceCountingSegment; import java.io.Closeable; +import java.util.List; import java.util.Map; /** @@ -33,17 +34,17 @@ public class ReferenceCountingSegmentQueryRunner implements QueryRunner { private final QueryRunnerFactory> factory; private final ReferenceCountingSegment adapter; - private final SegmentDescriptor descriptor; + private final Map> segmentDescMap; public ReferenceCountingSegmentQueryRunner( QueryRunnerFactory> factory, ReferenceCountingSegment adapter, - SegmentDescriptor descriptor + Map> segmentDescMap ) { this.factory = factory; this.adapter = adapter; - this.descriptor = descriptor; + this.segmentDescMap = segmentDescMap; } @Override @@ -62,7 +63,7 @@ public Sequence run(final Query query, Map responseContext } } else { // Segment was closed before we had a chance to increment the reference count - return new ReportTimelineMissingSegmentQueryRunner(descriptor).run(query, responseContext); + return new ReportTimelineMissingSegmentQueryRunner(segmentDescMap).run(query, responseContext); } } } diff --git a/processing/src/main/java/io/druid/query/ReportTimelineMissingSegmentQueryRunner.java b/processing/src/main/java/io/druid/query/ReportTimelineMissingSegmentQueryRunner.java index a8b6c5e2ffd6..32b40471a4c4 100644 --- a/processing/src/main/java/io/druid/query/ReportTimelineMissingSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ReportTimelineMissingSegmentQueryRunner.java @@ -19,10 +19,10 @@ package io.druid.query; -import com.google.common.collect.Lists; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -30,11 +30,11 @@ */ public class ReportTimelineMissingSegmentQueryRunner implements QueryRunner { - private final SegmentDescriptor descriptor; + private final Map> segmentDescMap; - public ReportTimelineMissingSegmentQueryRunner(SegmentDescriptor descriptor) + public ReportTimelineMissingSegmentQueryRunner(Map> segmentDescMap) { - this.descriptor = descriptor; + this.segmentDescMap = segmentDescMap; } @Override @@ -42,12 +42,12 @@ public Sequence run( Query query, Map responseContext ) { - List missingSegments = (List) responseContext.get(Result.MISSING_SEGMENTS_KEY); - if (missingSegments == null) { - missingSegments = Lists.newArrayList(); - responseContext.put(Result.MISSING_SEGMENTS_KEY, missingSegments); - } - missingSegments.add(descriptor); + Map> missingSegments = + (Map>) responseContext.computeIfAbsent( + Result.MISSING_SEGMENTS_KEY, k -> new HashMap<>() + ); + missingSegments.putAll(segmentDescMap); + return Sequences.empty(); } } diff --git a/processing/src/main/java/io/druid/query/RetryQueryRunner.java b/processing/src/main/java/io/druid/query/RetryQueryRunner.java index 57a5cde5124b..086fb18410c9 100644 --- a/processing/src/main/java/io/druid/query/RetryQueryRunner.java +++ b/processing/src/main/java/io/druid/query/RetryQueryRunner.java @@ -23,9 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; - import com.metamx.emitter.EmittingLogger; - import io.druid.java.util.common.guava.MergeSequence; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; @@ -35,8 +33,10 @@ import io.druid.query.spec.MultipleSpecificSegmentSpec; import io.druid.segment.SegmentMissingException; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Map.Entry; public class RetryQueryRunner implements QueryRunner { @@ -73,18 +73,21 @@ public Yielder toYielder( OutType initValue, YieldingAccumulator accumulator ) { - List missingSegments = getMissingSegments(context); + Map> missingSegments = getMissingSegments(context); if (!missingSegments.isEmpty()) { for (int i = 0; i < config.getNumTries(); i++) { log.info("[%,d] missing segments found. Retry attempt [%,d]", missingSegments.size(), i); - context.put(Result.MISSING_SEGMENTS_KEY, Lists.newArrayList()); - final Query retryQuery = query.withQuerySegmentSpec( - new MultipleSpecificSegmentSpec( - missingSegments - ) - ); + context.put(Result.MISSING_SEGMENTS_KEY, new HashMap<>()); + Query retryQuery = query; + for (Entry> entry : missingSegments.entrySet()) { + retryQuery = query.replaceQuerySegmentSpecWith( + entry.getKey(), + new MultipleSpecificSegmentSpec(entry.getValue()) + ); + } + Sequence retrySequence = baseRunner.run(retryQuery, context); listOfSequences.add(retrySequence); missingSegments = getMissingSegments(context); @@ -93,7 +96,7 @@ public Yielder toYielder( } } - final List finalMissingSegs = getMissingSegments(context); + final Map> finalMissingSegs = getMissingSegments(context); if (!config.isReturnPartialResults() && !finalMissingSegs.isEmpty()) { throw new SegmentMissingException("No results found for segments[%s]", finalMissingSegs); } @@ -111,16 +114,16 @@ public Yielder toYielder( }; } - private List getMissingSegments(final Map context) + private Map> getMissingSegments(final Map context) { final Object maybeMissingSegments = context.get(Result.MISSING_SEGMENTS_KEY); if (maybeMissingSegments == null) { - return Lists.newArrayList(); + return new HashMap<>(); } return jsonMapper.convertValue( maybeMissingSegments, - new TypeReference>() + new TypeReference>>() { } ); diff --git a/processing/src/main/java/io/druid/query/SingleSourceBaseQuery.java b/processing/src/main/java/io/druid/query/SingleSourceBaseQuery.java new file mode 100644 index 000000000000..390d229353a7 --- /dev/null +++ b/processing/src/main/java/io/druid/query/SingleSourceBaseQuery.java @@ -0,0 +1,200 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import io.druid.java.util.common.guava.Sequence; +import io.druid.query.spec.QuerySegmentSpec; +import org.joda.time.Duration; +import org.joda.time.Interval; + +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public abstract class SingleSourceBaseQuery> extends BaseQuery +{ + private final DataSourceWithSegmentSpec dataSourceWithSegment; + private volatile Duration duration; + + public SingleSourceBaseQuery( + DataSource dataSource, + QuerySegmentSpec querySegmentSpec, + boolean descending, + Map context + ) + { + super(descending, context); + Objects.requireNonNull(dataSource, "dataSource can't be null"); + Objects.requireNonNull(querySegmentSpec, "querySegmentSpec can't be null"); + + this.dataSourceWithSegment = new DataSourceWithSegmentSpec(dataSource, querySegmentSpec); + } + + public abstract Query withQuerySegmentSpec(QuerySegmentSpec spec); + public abstract Query withDataSource(DataSource dataSource); + + @Override + public Iterable getDataSources() + { + return ImmutableList.of(dataSourceWithSegment); + } + + public DataSourceWithSegmentSpec getDataSourceWithSegmentSpec() + { + return dataSourceWithSegment; + } + + @JsonProperty + public DataSource getDataSource() + { + return dataSourceWithSegment.getDataSource(); + } + + @JsonProperty("intervals") + public QuerySegmentSpec getQuerySegmentSpec() + { + return dataSourceWithSegment.getQuerySegmentSpec(); + } + + @Override + public Sequence run(QuerySegmentWalker walker, Map context) + { + return run(dataSourceWithSegment.getQuerySegmentSpec().lookup(this, walker), context); + } + + public List getIntervals() + { + return dataSourceWithSegment.getQuerySegmentSpec().getIntervals(); + } + + @Override + public Duration getDuration(DataSource dataSource) + { + Preconditions.checkArgument(this.dataSourceWithSegment.getDataSource().equals(dataSource)); + return getDuration(); + } + + public Duration getDuration() + { + if (duration == null) { + duration = initDuration(dataSourceWithSegment.getQuerySegmentSpec()); + } + + return duration; + } + + @Override + public Query replaceQuerySegmentSpecWith(DataSource dataSource, QuerySegmentSpec spec) + { + Preconditions.checkArgument(this.dataSourceWithSegment.getDataSource().equals(dataSource)); + final Query query = withQuerySegmentSpec(spec); + if (getDistributionTarget() != null) { + if (dataSource.equals(getDistributionTarget().getDataSource())) { + return query.distributeBy(((SingleSourceBaseQuery) query).getDataSourceWithSegmentSpec()); + } + } + return query; + } + + @Override + public Query replaceQuerySegmentSpecWith(String dataSource, QuerySegmentSpec spec) + { + Preconditions.checkArgument(Iterables.getOnlyElement(this.dataSourceWithSegment.getDataSource().getNames()).equals(dataSource)); + final Query query = withQuerySegmentSpec(spec); + if (getDistributionTarget() != null) { + if (dataSource.equals(getDistributionTarget().getDataSource().getConcatenatedName())) { + return query.distributeBy(((SingleSourceBaseQuery) query).getDataSourceWithSegmentSpec()); + } + } + return query; + } + + @Override + public Query replaceDataSourceWith(DataSource src, DataSource dst) + { + Preconditions.checkArgument(this.dataSourceWithSegment.getDataSource().equals(src)); + return withDataSource(dst); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + SingleSourceBaseQuery baseQuery = (SingleSourceBaseQuery) o; + + if (isDescending() != baseQuery.isDescending()) { + return false; + } + if (getContext() != null ? !getContext().equals(baseQuery.getContext()) : baseQuery.getContext() != null) { + return false; + } + if (!dataSourceWithSegment.equals(baseQuery.dataSourceWithSegment)) { + return false; + } + if (duration != null ? !duration.equals(baseQuery.duration) : baseQuery.duration != null) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + int result = dataSourceWithSegment.hashCode(); + result = 31 * result + (isDescending() ? 1 : 0); + result = 31 * result + (getContext() != null ? getContext().hashCode() : 0); + result = 31 * result + (duration != null ? duration.hashCode() : 0); + return result; + } + + public static > DataSourceWithSegmentSpec getLeafDataSourceWithSegmentSpec( + SingleSourceBaseQuery query + ) + { + final DataSourceWithSegmentSpec sourceWithSegmentSpec = query.getDataSourceWithSegmentSpec(); + if (sourceWithSegmentSpec.getDataSource() instanceof QueryDataSource) { + final QueryDataSource queryDataSource = (QueryDataSource) sourceWithSegmentSpec.getDataSource(); + return getLeafDataSourceWithSegmentSpec((SingleSourceBaseQuery) queryDataSource.getQuery()); + } else { + return sourceWithSegmentSpec; + } + } + + public static > DataSource getLeafDataSource(DataSource dataSource) + { + if (dataSource instanceof QueryDataSource) { + final QueryDataSource queryDataSource = (QueryDataSource) dataSource; + return getLeafDataSourceWithSegmentSpec((SingleSourceBaseQuery) queryDataSource.getQuery()).getDataSource(); + } else { + return dataSource; + } + } +} diff --git a/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java b/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java index e3dc7356c243..94c4bc817bfb 100644 --- a/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java +++ b/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java @@ -19,6 +19,7 @@ package io.druid.query; +import com.google.common.base.Preconditions; import io.druid.java.util.common.guava.Sequence; import java.util.Map; @@ -38,7 +39,8 @@ public SubqueryQueryRunner(QueryRunner baseRunner) @Override public Sequence run(final Query query, Map responseContext) { - DataSource dataSource = query.getDataSource(); + Preconditions.checkArgument(query instanceof SingleSourceBaseQuery); + final DataSource dataSource = ((SingleSourceBaseQuery)query).getDataSource(); if (dataSource instanceof QueryDataSource) { return run((Query) ((QueryDataSource) dataSource).getQuery(), responseContext); } else { diff --git a/processing/src/main/java/io/druid/query/TimewarpOperator.java b/processing/src/main/java/io/druid/query/TimewarpOperator.java index 0444d95d518c..77986a4f3678 100644 --- a/processing/src/main/java/io/druid/query/TimewarpOperator.java +++ b/processing/src/main/java/io/druid/query/TimewarpOperator.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; +import com.google.common.collect.Iterables; import io.druid.data.input.MapBasedRow; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; @@ -85,14 +86,18 @@ public Sequence run(final Query query, final Map responseC { final long offset = computeOffset(now); - final Interval interval = query.getIntervals().get(0); + final DataSourceWithSegmentSpec spec = query.getDistributionTarget(); + final Interval interval = spec.getQuerySegmentSpec().getIntervals().get(0); final Interval modifiedInterval = new Interval( Math.min(interval.getStartMillis() + offset, now + offset), Math.min(interval.getEndMillis() + offset, now + offset) ); return Sequences.map( baseRunner.run( - query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Arrays.asList(modifiedInterval))), + query.replaceQuerySegmentSpecWith( + Iterables.getOnlyElement(spec.getDataSource().getNames()), + new MultipleIntervalSegmentSpec(Arrays.asList(modifiedInterval)) + ), responseContext ), new Function() diff --git a/processing/src/main/java/io/druid/query/UnionQueryRunner.java b/processing/src/main/java/io/druid/query/UnionQueryRunner.java index cfb337d9c94d..79f1cee99b48 100644 --- a/processing/src/main/java/io/druid/query/UnionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/UnionQueryRunner.java @@ -41,31 +41,36 @@ public UnionQueryRunner( @Override public Sequence run(final Query query, final Map responseContext) { - DataSource dataSource = query.getDataSource(); - if (dataSource instanceof UnionDataSource) { + if (query instanceof SingleSourceBaseQuery) { + final SingleSourceBaseQuery singleSourceBaseQuery = (SingleSourceBaseQuery) query; - return new MergeSequence<>( - query.getResultOrdering(), - Sequences.simple( - Lists.transform( - ((UnionDataSource) dataSource).getDataSources(), - new Function>() - { - @Override - public Sequence apply(DataSource singleSource) + final DataSource dataSource = singleSourceBaseQuery.getDataSource(); + if (dataSource instanceof UnionDataSource) { + + return new MergeSequence<>( + query.getResultOrdering(), + Sequences.simple( + Lists.transform( + ((UnionDataSource) dataSource).getDataSources(), + new Function>() { - return baseRunner.run( - query.withDataSource(singleSource), - responseContext - ); + @Override + public Sequence apply(DataSource singleSource) + { + return baseRunner.run( + singleSourceBaseQuery.withDataSource(singleSource), + responseContext + ); + } } - } - ) - ) - ); + ) + ) + ); + } else { + return baseRunner.run(query, responseContext); + } } else { return baseRunner.run(query, responseContext); } } - } diff --git a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceMetadataQuery.java b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceMetadataQuery.java index 186fcaf2f6ef..356be1148a4e 100644 --- a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceMetadataQuery.java +++ b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceMetadataQuery.java @@ -23,10 +23,10 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Lists; 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.Result; +import io.druid.query.SingleSourceBaseQuery; import io.druid.query.filter.DimFilter; import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.query.spec.QuerySegmentSpec; @@ -39,7 +39,7 @@ /** */ -public class DataSourceMetadataQuery extends BaseQuery> +public class DataSourceMetadataQuery extends SingleSourceBaseQuery> { public static final Interval MY_Y2K_INTERVAL = new Interval( JodaUtils.MIN_INSTANT, JodaUtils.MAX_INSTANT diff --git a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java index 1c5b2eb00ea7..25aa428b2e75 100644 --- a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java @@ -83,6 +83,20 @@ public boolean apply(T input) ); } + @Override + public QueryRunner> annotateDistributionTarget( + QueryRunner> runner + ) + { + return (query, responseContext) -> { + final DataSourceMetadataQuery dataSourceMetadataQuery = (DataSourceMetadataQuery) query; + return runner.run( + dataSourceMetadataQuery.distributeBy(dataSourceMetadataQuery.getDataSourceWithSegmentSpec()), + responseContext + ); + }; + } + @Override public QueryRunner> mergeResults( final QueryRunner> runner diff --git a/processing/src/main/java/io/druid/query/dimension/BaseFilteredDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/BaseFilteredDimensionSpec.java index 683e2966685a..c2f4277c6a3e 100644 --- a/processing/src/main/java/io/druid/query/dimension/BaseFilteredDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/BaseFilteredDimensionSpec.java @@ -43,6 +43,12 @@ public DimensionSpec getDelegate() return delegate; } + @Override + public String getDataSourceName() + { + return delegate.getDataSourceName(); + } + @Override public String getDimension() { diff --git a/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java index eb5da594ef55..8e61644f70fa 100644 --- a/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java @@ -64,17 +64,20 @@ public DimensionSpec apply(String input) } private static final byte CACHE_TYPE_ID = 0x0; + private final String dataSourceName; private final String dimension; private final String outputName; private final ValueType outputType; @JsonCreator public DefaultDimensionSpec( + @JsonProperty("dataSource") String dataSourceName, @JsonProperty("dimension") String dimension, @JsonProperty("outputName") String outputName, @JsonProperty("outputType") ValueType outputType ) { + this.dataSourceName = dataSourceName; this.dimension = dimension; this.outputType = outputType == null ? ValueType.STRING : outputType; @@ -83,11 +86,36 @@ public DefaultDimensionSpec( } public DefaultDimensionSpec( + String dataSourceName, String dimension, String outputName ) { - this(dimension, outputName, ValueType.STRING); + this(dataSourceName, dimension, outputName, ValueType.STRING); + } + + public DefaultDimensionSpec( + String dimension, + String outputName, + ValueType outputType + ) + { + this(null, dimension, outputName, outputType); + } + + public DefaultDimensionSpec( + String dimension, + String outputName + ) + { + this(null, dimension, outputName); + } + + @Override + @JsonProperty("dataSource") + public String getDataSourceName() + { + return dataSourceName; } @Override diff --git a/processing/src/main/java/io/druid/query/dimension/DimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/DimensionSpec.java index 7749be3120ef..3bafaff75918 100644 --- a/processing/src/main/java/io/druid/query/dimension/DimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/DimensionSpec.java @@ -38,6 +38,8 @@ }) public interface DimensionSpec extends Cacheable { + String getDataSourceName(); + String getDimension(); String getOutputName(); diff --git a/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java index 6f992018c816..55f66a5dfe53 100644 --- a/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java @@ -35,6 +35,7 @@ public class ExtractionDimensionSpec implements DimensionSpec { private static final byte CACHE_TYPE_ID = 0x1; + private final String dataSourceName; private final String dimension; private final ExtractionFn extractionFn; private final String outputName; @@ -42,6 +43,7 @@ public class ExtractionDimensionSpec implements DimensionSpec @JsonCreator public ExtractionDimensionSpec( + @JsonProperty("dataSourceName") String dataSourceName, @JsonProperty("dimension") String dimension, @JsonProperty("outputName") String outputName, @JsonProperty("outputType") ValueType outputType, @@ -53,6 +55,7 @@ public ExtractionDimensionSpec( Preconditions.checkNotNull(dimension, "dimension must not be null"); Preconditions.checkArgument(extractionFn != null || dimExtractionFn != null, "extractionFn must not be null"); + this.dataSourceName = dataSourceName; this.dimension = dimension; this.extractionFn = extractionFn != null ? extractionFn : dimExtractionFn; this.outputType = outputType == null ? ValueType.STRING : outputType; @@ -61,14 +64,25 @@ public ExtractionDimensionSpec( this.outputName = outputName == null ? dimension : outputName; } + public ExtractionDimensionSpec(String dataSourceName, String dimension, String outputName, ExtractionFn extractionFn) + { + this(dataSourceName, dimension, outputName, null, extractionFn, null); + } + public ExtractionDimensionSpec(String dimension, String outputName, ExtractionFn extractionFn) { - this(dimension, outputName, null, extractionFn, null); + this(null, dimension, outputName, null, extractionFn, null); } public ExtractionDimensionSpec(String dimension, String outputName, ValueType outputType, ExtractionFn extractionFn) { - this(dimension, outputName, outputType, extractionFn, null); + this(null, dimension, outputName, outputType, extractionFn, null); + } + + @Override + public String getDataSourceName() + { + return dataSourceName; } @Override diff --git a/processing/src/main/java/io/druid/query/dimension/LookupDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/LookupDimensionSpec.java index 498fb47a8b96..06da33a4b68a 100644 --- a/processing/src/main/java/io/druid/query/dimension/LookupDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/LookupDimensionSpec.java @@ -40,6 +40,9 @@ public class LookupDimensionSpec implements DimensionSpec { private static final byte CACHE_TYPE_ID = 0x4; + @JsonProperty + private final String dataSourceName; + @JsonProperty private final String dimension; @@ -65,6 +68,7 @@ public class LookupDimensionSpec implements DimensionSpec @JsonCreator public LookupDimensionSpec( + @JsonProperty("dataSourceName") String dataSourceName, @JsonProperty("dimension") String dimension, @JsonProperty("outputName") String outputName, @JsonProperty("lookup") LookupExtractor lookup, @@ -75,6 +79,7 @@ public LookupDimensionSpec( @JsonProperty("optimize") Boolean optimize ) { + this.dataSourceName = dataSourceName; this.retainMissingValue = retainMissingValue; this.optimize = optimize == null ? true : optimize; this.replaceMissingValueWith = Strings.emptyToNull(replaceMissingValueWith); @@ -96,6 +101,36 @@ public LookupDimensionSpec( } } + public LookupDimensionSpec( + String dimension, + String outputName, + LookupExtractor lookup, + boolean retainMissingValue, + String replaceMissingValueWith, + String name, + LookupReferencesManager lookupReferencesManager, + Boolean optimize + ) + { + this( + null, + dimension, + outputName, + lookup, + retainMissingValue, + replaceMissingValueWith, + name, + lookupReferencesManager, + optimize + ); + } + + @Override + public String getDataSourceName() + { + return dataSourceName; + } + @Override @JsonProperty public String getDimension() diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java index 9184ad386893..716410d968d1 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java @@ -38,11 +38,13 @@ import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; -import io.druid.query.BaseQuery; import io.druid.query.DataSource; +import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.Queries; import io.druid.query.Query; +import io.druid.query.QueryContextKeys; import io.druid.query.QueryDataSource; +import io.druid.query.SingleSourceBaseQuery; import io.druid.query.TableDataSource; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; @@ -63,6 +65,7 @@ import java.util.Arrays; import java.util.Comparator; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -70,7 +73,7 @@ /** */ -public class GroupByQuery extends BaseQuery +public class GroupByQuery extends SingleSourceBaseQuery { public final static String CTX_KEY_SORT_BY_DIMS_FIRST = "sortByDimsFirst"; @@ -654,6 +657,18 @@ public Builder setDataSource(Query query) return this; } + public Builder updateDistributionTarget() + { + if (context == null) { + context = new HashMap<>(); + } + context.put( + QueryContextKeys.DIST_TARGET_SOURCE, + new DataSourceWithSegmentSpec(getLeafDataSource(dataSource), querySegmentSpec) + ); + return this; + } + public Builder setInterval(QuerySegmentSpec interval) { return setQuerySegmentSpec(interval); @@ -815,7 +830,11 @@ public Builder setPostAggregatorSpecs(List postAggregatorSpecs) public Builder setContext(Map context) { - this.context = context; + if (this.context == null) { + this.context = new HashMap<>(context); + } else { + this.context.putAll(context); + } return this; } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index 85f4cf559adf..95835db53d83 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -373,6 +373,18 @@ public String apply(DimensionSpec input) ); } + @Override + public QueryRunner annotateDistributionTarget(QueryRunner runner) + { + return (query, responseContext) -> { + final GroupByQuery groupByQuery = (GroupByQuery) query; + return runner.run( + groupByQuery.distributeBy(groupByQuery.getDataSourceWithSegmentSpec()), + responseContext + ); + }; + } + @Override public CacheStrategy getCacheStrategy(final GroupByQuery query) { diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index 623513ed8aa7..ecf31e438faf 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -199,7 +199,7 @@ private static TimestampExtractFunction makeTimestampExtractFunction( @Override public long apply(Row row) { - return query.getIntervals().get(0).getStartMillis(); + return query.getQuerySegmentSpec().getIntervals().get(0).getStartMillis(); } }; } else { diff --git a/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV2.java b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV2.java index 45dbd954d1c7..b938f3f0e5ef 100644 --- a/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV2.java +++ b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV2.java @@ -43,6 +43,7 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.nary.BinaryFn; import io.druid.query.DataSource; +import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.DruidProcessingConfig; import io.druid.query.InsufficientResourcesException; import io.druid.query.IntervalChunkingQueryRunnerDecorator; @@ -156,7 +157,15 @@ public GroupByQueryResource prepareResource(GroupByQuery query, boolean willMerg } } - private static int countRequiredMergeBufferNum(Query query, int foundNum) + /** + * Count the maximum length of consecutive groupBy subqueries. + * + * @param query + * @param foundNum + * @param + * @return + */ + private static int countRequiredMergeBufferNum(Query query, int foundNum) { // Note: A broker requires merge buffers for processing the groupBy layers beyond the inner-most one. // For example, the number of required merge buffers for a nested groupBy (groupBy -> groupBy -> table) is 1. @@ -165,11 +174,36 @@ private static int countRequiredMergeBufferNum(Query query, int foundNum) // until the outer groupBy processing completes. // This is same for subsequent groupBy layers, and thus the maximum number of required merge buffers becomes 2. - final DataSource dataSource = query.getDataSource(); - if (foundNum == MAX_MERGE_BUFFER_NUM + 1 || !(dataSource instanceof QueryDataSource)) { - return foundNum - 1; + return countGroupByLayers(query, foundNum) - 1; + } + + @SuppressWarnings("unchecked") + private static int countGroupByLayers(final Query query, final int foundNum) + { + if (foundNum == MAX_MERGE_BUFFER_NUM + 1) { + return foundNum; + } + + int maxFoundFromChildren = -1; + for (DataSourceWithSegmentSpec eachSpec : query.getDataSources()) { + final DataSource dataSource = eachSpec.getDataSource(); + + if (dataSource instanceof QueryDataSource) { + QueryDataSource queryDataSource = (QueryDataSource) dataSource; + if (queryDataSource.getQuery() instanceof GroupByQuery) { + final int tmp = countGroupByLayers(queryDataSource.getQuery(), foundNum + 1); + maxFoundFromChildren = tmp > maxFoundFromChildren ? tmp : maxFoundFromChildren; + } else { + final int tmp = countGroupByLayers(queryDataSource.getQuery(), 0); + maxFoundFromChildren = tmp > maxFoundFromChildren ? tmp : maxFoundFromChildren; + } + } + } + + if (maxFoundFromChildren > -1) { + return maxFoundFromChildren > foundNum ? maxFoundFromChildren : foundNum; } else { - return countRequiredMergeBufferNum(((QueryDataSource) dataSource).getQuery(), foundNum + 1); + return foundNum; } } diff --git a/processing/src/main/java/io/druid/query/join/AddPredicate.java b/processing/src/main/java/io/druid/query/join/AddPredicate.java new file mode 100644 index 000000000000..f40f789dbd6d --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/AddPredicate.java @@ -0,0 +1,46 @@ +/* + * 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.join; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class AddPredicate extends BinaryPredicate +{ + @JsonCreator + public AddPredicate( + @JsonProperty("left") JoinPredicate left, + @JsonProperty("right") JoinPredicate right) + { + super(left, right); + } + + @Override + public void accept(JoinPredicateVisitor visitor) + { + visitor.visit(this); + } + + @Override + public PredicateType getType() + { + return PredicateType.ADD; + } +} diff --git a/processing/src/main/java/io/druid/query/join/AndPredicate.java b/processing/src/main/java/io/druid/query/join/AndPredicate.java new file mode 100644 index 000000000000..7adf739fc86c --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/AndPredicate.java @@ -0,0 +1,78 @@ +/* + * 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.join; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; +import java.util.Objects; + +public class AndPredicate implements JoinPredicate +{ + private final List predicates; + + @JsonCreator + public AndPredicate( + @JsonProperty("predicates") List predicates + ) + { + this.predicates = predicates; + } + + @JsonProperty + public List getPredicates() + { + return predicates; + } + + @Override + public void accept(JoinPredicateVisitor visitor) + { + visitor.visit(this); + } + + @Override + public PredicateType getType() + { + return PredicateType.AND; + } + + @Override + public boolean equals(Object o) + { + if (o == this) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + AndPredicate that = (AndPredicate) o; + return predicates.equals(that.predicates); + } + + @Override + public int hashCode() + { + return Objects.hash(getType(), predicates); + } +} diff --git a/processing/src/main/java/io/druid/query/join/BinaryPredicate.java b/processing/src/main/java/io/druid/query/join/BinaryPredicate.java new file mode 100644 index 000000000000..51f6c9ec72f8 --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/BinaryPredicate.java @@ -0,0 +1,69 @@ +/* + * 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.join; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +public abstract class BinaryPredicate implements JoinPredicate +{ + protected final JoinPredicate left; + protected final JoinPredicate right; + + BinaryPredicate(JoinPredicate left, JoinPredicate right) + { + this.left = Objects.requireNonNull(left); + this.right = Objects.requireNonNull(right); + } + + @JsonProperty + public JoinPredicate getLeft() + { + return left; + } + + @JsonProperty + public JoinPredicate getRight() + { + return right; + } + + @Override + public boolean equals(Object o) + { + if (o == this) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + BinaryPredicate that = (BinaryPredicate) o; + return getType().equals(that.getType()); + } + + @Override + public int hashCode() + { + return Objects.hash(getType(), left, right); + } +} diff --git a/processing/src/main/java/io/druid/query/join/DataInput.java b/processing/src/main/java/io/druid/query/join/DataInput.java new file mode 100644 index 000000000000..ad89a752c12a --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/DataInput.java @@ -0,0 +1,114 @@ +/* + * 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.join; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Iterables; +import io.druid.query.BaseQuery; +import io.druid.query.DataSource; +import io.druid.query.spec.QuerySegmentSpec; +import org.joda.time.Duration; + +import java.util.Objects; + +public class DataInput implements JoinInputSpec +{ + private final DataSource dataSource; + private final QuerySegmentSpec querySegmentSpec; + private volatile Duration duration; + + @JsonCreator + public DataInput( + @JsonProperty("dataSource") DataSource dataSource, + @JsonProperty("intervals") QuerySegmentSpec querySegmentSpec + ) + { + this.dataSource = Objects.requireNonNull(dataSource); + this.querySegmentSpec = Objects.requireNonNull(querySegmentSpec); + } + + public DataInput( + DataSource dataSource, + QuerySegmentSpec querySegmentSpec, + Duration duration + ) + { + this.dataSource = Objects.requireNonNull(dataSource); + this.querySegmentSpec = Objects.requireNonNull(querySegmentSpec); + this.duration = Objects.requireNonNull(duration); + } + + @JsonProperty("dataSource") + public DataSource getDataSource() + { + return dataSource; + } + + public String getName() + { + return Iterables.getOnlyElement(dataSource.getNames()); + } + + @JsonProperty("intervals") + public QuerySegmentSpec getQuerySegmentSpec() + { + return querySegmentSpec; + } + + public Duration getDuration() + { + if (duration == null) { + this.duration = BaseQuery.initDuration(querySegmentSpec); + } + return duration; + } + + @Override + public boolean equals(Object o) + { + if (o == this) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + DataInput that = (DataInput) o; + if (!dataSource.equals(that.dataSource)) { + return false; + } + + return querySegmentSpec.equals(that.querySegmentSpec); + } + + @Override + public int hashCode() + { + return Objects.hash(dataSource, querySegmentSpec); + } + + @Override + public DataInput accept(JoinSpecVisitor visitor) + { + return visitor.visit(this); + } +} diff --git a/processing/src/main/java/io/druid/query/join/DimExtractPredicate.java b/processing/src/main/java/io/druid/query/join/DimExtractPredicate.java new file mode 100644 index 000000000000..31858b7a851b --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/DimExtractPredicate.java @@ -0,0 +1,78 @@ +/* + * 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.join; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.query.dimension.DimensionSpec; + +import java.util.Objects; + +public class DimExtractPredicate implements JoinPredicate +{ + private final DimensionSpec dimension; + + @JsonCreator + public DimExtractPredicate( + @JsonProperty("dimension") DimensionSpec dimension + ) + { + this.dimension = dimension; + } + + @JsonProperty + public DimensionSpec getDimension() + { + return dimension; + } + + @Override + public void accept(JoinPredicateVisitor visitor) + { + visitor.visit(this); + } + + @Override + public PredicateType getType() + { + return PredicateType.DIMENSION; + } + + @Override + public boolean equals(Object o) + { + if (o == this) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + DimExtractPredicate that = (DimExtractPredicate) o; + return dimension.equals(that.dimension); + } + + @Override + public int hashCode() + { + return Objects.hash(getType(), dimension); + } +} diff --git a/processing/src/main/java/io/druid/query/join/DividePredicate.java b/processing/src/main/java/io/druid/query/join/DividePredicate.java new file mode 100644 index 000000000000..63aa759f6a64 --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/DividePredicate.java @@ -0,0 +1,46 @@ +/* + * 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.join; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class DividePredicate extends BinaryPredicate +{ + @JsonCreator + public DividePredicate( + @JsonProperty("left") JoinPredicate left, + @JsonProperty("right") JoinPredicate right) + { + super(left, right); + } + + @Override + public PredicateType getType() + { + return PredicateType.DIVIDE; + } + + @Override + public void accept(JoinPredicateVisitor visitor) + { + visitor.visit(this); + } +} diff --git a/processing/src/main/java/io/druid/query/join/EqualPredicate.java b/processing/src/main/java/io/druid/query/join/EqualPredicate.java new file mode 100644 index 000000000000..9072b660f475 --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/EqualPredicate.java @@ -0,0 +1,47 @@ +/* + * 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.join; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class EqualPredicate extends BinaryPredicate +{ + @JsonCreator + public EqualPredicate( + @JsonProperty("left") JoinPredicate left, + @JsonProperty("right") JoinPredicate right + ) + { + super(left, right); + } + + @Override + public void accept(JoinPredicateVisitor visitor) + { + visitor.visit(this); + } + + @Override + public PredicateType getType() + { + return PredicateType.EQUAL; + } +} diff --git a/processing/src/main/java/io/druid/query/join/JoinInputSpec.java b/processing/src/main/java/io/druid/query/join/JoinInputSpec.java new file mode 100644 index 000000000000..2e665c36f19e --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/JoinInputSpec.java @@ -0,0 +1,34 @@ +/* + * 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.join; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonSubTypes.Type; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @Type(name = "dataSource", value = DataInput.class), + @Type(name = "joinSource", value = JoinSpec.class) +}) +public interface JoinInputSpec +{ + JoinInputSpec accept(JoinSpecVisitor visitor); +} diff --git a/processing/src/main/java/io/druid/query/join/JoinPredicate.java b/processing/src/main/java/io/druid/query/join/JoinPredicate.java new file mode 100644 index 000000000000..b5c15e608878 --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/JoinPredicate.java @@ -0,0 +1,58 @@ +/* + * 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.join; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonSubTypes.Type; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeInfo.Id; + +@JsonTypeInfo(use = Id.NAME, property = "type") +@JsonSubTypes(value = { + @Type(name = "and", value = AndPredicate.class), + @Type(name = "or", value = OrPredicate.class), + @Type(name = "not", value = NotPredicate.class), + @Type(name = "equal", value = EqualPredicate.class), + @Type(name = "dimension", value = DimExtractPredicate.class), + @Type(name = "add", value = AddPredicate.class), + @Type(name = "subtract", value = SubtractPredicate.class), + @Type(name = "multiply", value = MultiplyPredicate.class), + @Type(name = "divide", value = DividePredicate.class), + @Type(name = "literal", value = LiteralPredicate.class) +}) +public interface JoinPredicate +{ + enum PredicateType { + AND, + OR, + NOT, + EQUAL, + DIMENSION, + ADD, + SUBTRACT, + MULTIPLY, + DIVIDE, + LITERAL + } + + PredicateType getType(); + + void accept(JoinPredicateVisitor visitor); +} diff --git a/processing/src/main/java/io/druid/query/join/JoinPredicateVisitor.java b/processing/src/main/java/io/druid/query/join/JoinPredicateVisitor.java new file mode 100644 index 000000000000..c7380c3b01e5 --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/JoinPredicateVisitor.java @@ -0,0 +1,62 @@ +/* + * 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.join; + +public interface JoinPredicateVisitor +{ + default JoinPredicate visit(AndPredicate predicate) + { + for (JoinPredicate eachPredicate: predicate.getPredicates()) { + eachPredicate.accept(this); + } + return predicate; + } + + default JoinPredicate visit(OrPredicate predicate) + { + for (JoinPredicate eachPredicate: predicate.getPredicates()) { + eachPredicate.accept(this); + } + return predicate; + } + + default JoinPredicate visit(NotPredicate predicate) + { + predicate.getBase().accept(this); + return predicate; + } + + default JoinPredicate visit(DimExtractPredicate predicate) + { + return predicate; + } + + default JoinPredicate visit(BinaryPredicate predicate) + { + predicate.getLeft().accept(this); + predicate.getRight().accept(this); + return predicate; + } + + default JoinPredicate visit(LiteralPredicate predicate) + { + return predicate; + } +} diff --git a/processing/src/main/java/io/druid/query/join/JoinQuery.java b/processing/src/main/java/io/druid/query/join/JoinQuery.java new file mode 100644 index 000000000000..66112eb8d295 --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/JoinQuery.java @@ -0,0 +1,418 @@ +/* + * 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.join; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import io.druid.data.input.Row; +import io.druid.java.util.common.granularity.Granularity; +import io.druid.java.util.common.guava.Sequence; +import io.druid.query.BaseQuery; +import io.druid.query.DataSource; +import io.druid.query.DataSourceWithSegmentSpec; +import io.druid.query.Query; +import io.druid.query.QuerySegmentWalker; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.filter.DimFilter; +import io.druid.query.spec.QuerySegmentSpec; +import io.druid.segment.VirtualColumns; +import org.joda.time.Duration; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class JoinQuery extends BaseQuery +{ + private final JoinSpec joinSpec; + private final Granularity granularity; + private final List dimensions; // output dimensions + private final List metrics; // output metrics + private final VirtualColumns virtualColumns; + private final DimFilter filter; + + public static Builder newBuilder() + { + return new Builder(); + } + + @JsonCreator + public JoinQuery( + @JsonProperty("joinSpec") JoinSpec joinSpec, + @JsonProperty("granularity") Granularity granularity, + @JsonProperty("dimensions") List dimensions, + @JsonProperty("metrics") List metrics, + @JsonProperty("virtualColumns") VirtualColumns virtualColumns, + @JsonProperty("filter") DimFilter filter, + @JsonProperty("context") Map context + ) + { + super(false, context); + this.joinSpec = Objects.requireNonNull(joinSpec); + this.granularity = Objects.requireNonNull(granularity); + this.dimensions = dimensions == null ? ImmutableList.of() : dimensions; + this.metrics = metrics == null ? ImmutableList.of() : metrics; + this.virtualColumns = virtualColumns == null ? VirtualColumns.EMPTY : virtualColumns; + this.filter = filter; + Preconditions.checkState(dimensions != null || metrics != null, "At least one dimension or metric must be specified."); + } + + @Override + public Iterable getDataSources() + { + final List found = new ArrayList<>(); + final JoinSpecVisitor visitor = new JoinSpecVisitor() + { + @Override + public DataInput visit(DataInput dataInput) + { + found.add(new DataSourceWithSegmentSpec(dataInput.getDataSource(), dataInput.getQuerySegmentSpec())); + return dataInput; + } + }; + + joinSpec.accept(visitor); + + return found; + } + + @Override + public boolean hasFilters() + { + return filter != null; + } + + @Override + public DimFilter getFilter() + { + return filter; + } + + @Override + public String getType() + { + return Query.JOIN; + } + + @Override + public Sequence run( + QuerySegmentWalker walker, Map context + ) + { + return run(getDistributionTarget().getQuerySegmentSpec().lookup(this, walker), context); + } + + @Override + public Duration getDuration(DataSource dataSource) + { + for (DataSourceWithSegmentSpec sourceWithSegmentSpec : getDataSources()) { + if (sourceWithSegmentSpec.getDataSource().equals(dataSource)) { + return initDuration(sourceWithSegmentSpec.getQuerySegmentSpec()); + } + } + return null; + } + + @JsonProperty + public JoinSpec getJoinSpec() + { + return joinSpec; + } + + @JsonProperty + public Granularity getGranularity() + { + return granularity; + } + + @JsonProperty + public List getDimensions() + { + return dimensions; + } + + @JsonProperty + public List getMetrics() + { + return metrics; + } + + @JsonProperty + public VirtualColumns getVirtualColumns() + { + return virtualColumns; + } + + @Override + public Query withOverriddenContext(Map contextOverride) + { + return new JoinQuery( + joinSpec, + granularity, + dimensions, + metrics, + virtualColumns, + filter, + computeOverridenContext(contextOverride) + ); + } + + @Override + public Query replaceQuerySegmentSpecWith(DataSource dataSource, QuerySegmentSpec spec) + { + return replaceQuerySegmentSpecWith(dataSource.getFirstName(), spec); + } + + @Override + public Query replaceQuerySegmentSpecWith(String dataSource, QuerySegmentSpec spec) + { + final JoinSpecVisitor visitor = new JoinSpecVisitor() + { + @Override + public JoinSpec visit(JoinSpec joinSpec) + { + final JoinInputSpec newLeft = joinSpec.getLeft().accept(this); + final JoinInputSpec newRight = joinSpec.getRight().accept(this); + return new JoinSpec(joinSpec.getJoinType(), joinSpec.getPredicate(), newLeft, newRight); + } + + @Override + public DataInput visit(DataInput dataInput) + { + if (dataInput.getDataSource().getFirstName().equals(dataSource)) { + return new DataInput(dataInput.getDataSource(), spec); + } else { + return dataInput; + } + } + }; + + return new JoinQuery( + joinSpec.accept(visitor), + granularity, + dimensions, + metrics, + virtualColumns, + filter, + getContext() + ); + } + + @Override + public Query replaceDataSourceWith(DataSource src, DataSource dst) + { + final JoinSpecVisitor visitor = new JoinSpecVisitor() + { + @Override + public JoinSpec visit(JoinSpec joinSpec) + { + final JoinInputSpec newLeft = joinSpec.getLeft().accept(this); + final JoinInputSpec newRight = joinSpec.getRight().accept(this); + return new JoinSpec(joinSpec.getJoinType(), joinSpec.getPredicate(), newLeft, newRight); + } + + @Override + public DataInput visit(DataInput dataInput) + { + if (dataInput.getDataSource().equals(src)) { + return new DataInput(dst, dataInput.getQuerySegmentSpec()); + } else { + return dataInput; + } + } + }; + + return new JoinQuery( + joinSpec.accept(visitor), + granularity, + dimensions, + metrics, + virtualColumns, + filter, + getContext() + ); + } + + @Override + public boolean equals(Object o) + { + if (o == this) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + final JoinQuery that = (JoinQuery) o; + if (!joinSpec.equals(that.joinSpec)) { + return false; + } + + if (!granularity.equals(that.granularity)) { + return false; + } + + if (!dimensions.equals(that.dimensions)) { + return false; + } + + if (!metrics.equals(that.metrics)) { + return false; + } + + if (!virtualColumns.equals(that.virtualColumns)) { + return false; + } + + if (!Objects.equals(filter, that.filter)) { + return false; + } + + return Objects.equals(getContext(), that.getContext()); + } + + @Override + public int hashCode() + { + return Objects.hash(getType(), joinSpec, granularity, dimensions, metrics, virtualColumns, filter, getContext()); + } + + public static class Builder + { + private JoinSpec joinSpec; + private Granularity granularity; + private List dimensions; + private List metrics; + private VirtualColumns virtualColumns; + private DimFilter filter; + private Map context; + + public Builder() {} + + public Builder(JoinQuery query) + { + this( + query.joinSpec, + query.granularity, + query.dimensions, + query.metrics, + query.virtualColumns, + query.filter, + query.getContext() + ); + } + + public Builder(Builder builder) + { + this( + builder.joinSpec, + builder.granularity, + builder.dimensions, + builder.metrics, + builder.virtualColumns, + builder.filter, + builder.context + ); + } + + public Builder( + JoinSpec joinSpec, + Granularity granularity, + List dimensions, + List metrics, + VirtualColumns virtualColumns, + DimFilter filter, + Map context + ) + { + this.joinSpec = joinSpec; + this.granularity = granularity; + this.dimensions = dimensions; + this.metrics = metrics; + this.virtualColumns = virtualColumns; + this.filter = filter; + this.context = context; + } + + public Builder setJoinSpec(JoinSpec joinSpec) + { + this.joinSpec = joinSpec; + return this; + } + + public Builder setGranularity(Granularity granularity) + { + this.granularity = granularity; + return this; + } + + public Builder setDimensions(List dimensions) + { + this.dimensions = dimensions; + return this; + } + + public Builder setMetrics(List metrics) + { + this.metrics = metrics; + return this; + } + + public Builder setVirtualColumns(VirtualColumns virtualColumns) + { + this.virtualColumns = virtualColumns; + return this; + } + + public Builder setFilter(DimFilter filter) + { + this.filter = filter; + return this; + } + + public Builder setContext(Map context) + { + if (this.context == null) { + this.context = new HashMap<>(context); + } else { + this.context.putAll(context); + } + return this; + } + + public JoinQuery build() + { + return new JoinQuery( + joinSpec, + granularity, + dimensions, + metrics, + virtualColumns, + filter, + context + ); + } + } +} diff --git a/processing/src/main/java/io/druid/query/join/JoinSpec.java b/processing/src/main/java/io/druid/query/join/JoinSpec.java new file mode 100644 index 000000000000..d1555435a948 --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/JoinSpec.java @@ -0,0 +1,115 @@ +/* + * 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.join; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; + +import java.util.Objects; + +public class JoinSpec implements JoinInputSpec +{ + private final JoinType joinType; + private final JoinPredicate predicate; + private final JoinInputSpec left; + private final JoinInputSpec right; + + @JsonCreator + public JoinSpec( + @JsonProperty("joinType") JoinType joinType, + @JsonProperty("predicate") JoinPredicate predicate, + @JsonProperty("left") JoinInputSpec left, + @JsonProperty("right") JoinInputSpec right + ) + { + this.joinType = Preconditions.checkNotNull(joinType); + this.predicate = Preconditions.checkNotNull(predicate, "%s join requires any predicate", joinType); + this.left = Preconditions.checkNotNull(left); + this.right = Preconditions.checkNotNull(right); + + Preconditions.checkArgument(JoinType.INNER == joinType, "%s join type is not supported yet", joinType); + } + + public JoinSpec accept(JoinSpecVisitor visitor) + { + return visitor.visit(this); + } + + public boolean hasPredicate() + { + return predicate != null; + } + + @JsonProperty + public JoinPredicate getPredicate() + { + return predicate; + } + + @JsonProperty + public JoinType getJoinType() + { + return joinType; + } + + @JsonProperty + public JoinInputSpec getLeft() { + return left; + } + + @JsonProperty + public JoinInputSpec getRight() { + return right; + } + + @Override + public boolean equals(Object o) + { + if (o == this) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + JoinSpec that = (JoinSpec) o; + if (!joinType.equals(that.joinType)) { + return false; + } + + if (!predicate.equals(that.predicate)) { + return false; + } + + if (!left.equals(that.left)) { + return false; + } + + return right.equals(that.right); + } + + @Override + public int hashCode() + { + return Objects.hash(joinType, predicate, left, right); + } +} diff --git a/processing/src/main/java/io/druid/query/join/JoinSpecVisitor.java b/processing/src/main/java/io/druid/query/join/JoinSpecVisitor.java new file mode 100644 index 000000000000..8cf3eb56e7e1 --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/JoinSpecVisitor.java @@ -0,0 +1,35 @@ +/* + * 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.join; + +public interface JoinSpecVisitor +{ + default JoinSpec visit(JoinSpec joinSpec) + { + joinSpec.getLeft().accept(this); + joinSpec.getRight().accept(this); + return joinSpec; + } + + default DataInput visit(DataInput dataInput) + { + return dataInput; + } +} diff --git a/processing/src/main/java/io/druid/query/join/JoinType.java b/processing/src/main/java/io/druid/query/join/JoinType.java new file mode 100644 index 000000000000..773d7afdd5ec --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/JoinType.java @@ -0,0 +1,31 @@ +/* + * 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.join; + +public enum JoinType +{ + CROSS, + INNER, + LEFT_OUTER, + RIGHT_OUTER, + FULL_OUTER, + SEMI, + ANTI, +} diff --git a/processing/src/main/java/io/druid/query/join/LiteralPredicate.java b/processing/src/main/java/io/druid/query/join/LiteralPredicate.java new file mode 100644 index 000000000000..48b3e92546df --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/LiteralPredicate.java @@ -0,0 +1,75 @@ +/* + * 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.join; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +public class LiteralPredicate implements JoinPredicate +{ + private final String literal; + + @JsonCreator + public LiteralPredicate(@JsonProperty("literal") String literal) + { + this.literal = literal; + } + + @JsonProperty + public String getLiteral() + { + return literal; + } + + @Override + public void accept(JoinPredicateVisitor visitor) + { + visitor.visit(this); + } + + @Override + public PredicateType getType() + { + return PredicateType.LITERAL; + } + + @Override + public boolean equals(Object o) + { + if (o == this) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + LiteralPredicate that = (LiteralPredicate) o; + return literal.equals(that.literal); + } + + @Override + public int hashCode() + { + return Objects.hash(getType(), literal); + } +} diff --git a/processing/src/main/java/io/druid/query/join/MultiplyPredicate.java b/processing/src/main/java/io/druid/query/join/MultiplyPredicate.java new file mode 100644 index 000000000000..880fb9230e28 --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/MultiplyPredicate.java @@ -0,0 +1,46 @@ +/* + * 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.join; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class MultiplyPredicate extends BinaryPredicate +{ + @JsonCreator + public MultiplyPredicate( + @JsonProperty("left") JoinPredicate left, + @JsonProperty("right") JoinPredicate right) + { + super(left, right); + } + + @Override + public PredicateType getType() + { + return PredicateType.MULTIPLY; + } + + @Override + public void accept(JoinPredicateVisitor visitor) + { + visitor.visit(this); + } +} diff --git a/processing/src/main/java/io/druid/query/join/NotPredicate.java b/processing/src/main/java/io/druid/query/join/NotPredicate.java new file mode 100644 index 000000000000..5a096a86b64b --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/NotPredicate.java @@ -0,0 +1,77 @@ +/* + * 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.join; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +public class NotPredicate implements JoinPredicate +{ + private final JoinPredicate predicate; + + @JsonCreator + public NotPredicate( + @JsonProperty("base") JoinPredicate predicate + ) + { + this.predicate = predicate; + } + + @JsonProperty("base") + public JoinPredicate getBase() + { + return predicate; + } + + @Override + public void accept(JoinPredicateVisitor visitor) + { + visitor.visit(this); + } + + @Override + public PredicateType getType() + { + return PredicateType.NOT; + } + + @Override + public boolean equals(Object o) + { + if (o == this) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + NotPredicate that = (NotPredicate) o; + return predicate.equals(that.predicate); + } + + @Override + public int hashCode() + { + return Objects.hash(getType(), predicate); + } +} diff --git a/processing/src/main/java/io/druid/query/join/OrPredicate.java b/processing/src/main/java/io/druid/query/join/OrPredicate.java new file mode 100644 index 000000000000..63d670b2747f --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/OrPredicate.java @@ -0,0 +1,78 @@ +/* + * 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.join; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; +import java.util.Objects; + +public class OrPredicate implements JoinPredicate +{ + private final List predicates; + + @JsonCreator + public OrPredicate( + @JsonProperty("predicates") List predicates + ) + { + this.predicates = predicates; + } + + @JsonProperty + public List getPredicates() + { + return predicates; + } + + @Override + public void accept(JoinPredicateVisitor visitor) + { + visitor.visit(this); + } + + @Override + public PredicateType getType() + { + return PredicateType.OR; + } + + @Override + public boolean equals(Object o) + { + if (o == this) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + OrPredicate that = (OrPredicate) o; + return predicates.equals(that.predicates); + } + + @Override + public int hashCode() + { + return Objects.hash(getType(), predicates); + } +} diff --git a/processing/src/main/java/io/druid/query/join/SubtractPredicate.java b/processing/src/main/java/io/druid/query/join/SubtractPredicate.java new file mode 100644 index 000000000000..81913bb1ab29 --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/SubtractPredicate.java @@ -0,0 +1,46 @@ +/* + * 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.join; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class SubtractPredicate extends BinaryPredicate +{ + @JsonCreator + public SubtractPredicate( + @JsonProperty("left") JoinPredicate left, + @JsonProperty("right") JoinPredicate right) + { + super(left, right); + } + + @Override + public PredicateType getType() + { + return PredicateType.SUBTRACT; + } + + @Override + public void accept(JoinPredicateVisitor visitor) + { + visitor.visit(this); + } +} diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index a13f87ccd390..f975505f0dc3 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -261,6 +261,18 @@ public boolean apply(T input) ); } + @Override + public QueryRunner annotateDistributionTarget(QueryRunner runner) + { + return (query, responseContext) -> { + final SegmentMetadataQuery segmentMetadataQuery = (SegmentMetadataQuery) query; + return runner.run( + segmentMetadataQuery.distributeBy(segmentMetadataQuery.getDataSourceWithSegmentSpec()), + responseContext + ); + }; + } + @VisibleForTesting public static SegmentAnalysis mergeAnalyses( final SegmentAnalysis arg1, diff --git a/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java b/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java index 3d9ab5b117ed..920ba7c605c1 100644 --- a/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java +++ b/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java @@ -25,9 +25,9 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; 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.SingleSourceBaseQuery; import io.druid.query.TableDataSource; import io.druid.query.UnionDataSource; import io.druid.query.filter.DimFilter; @@ -42,7 +42,7 @@ import java.util.Map; import java.util.Objects; -public class SegmentMetadataQuery extends BaseQuery +public class SegmentMetadataQuery extends SingleSourceBaseQuery { /* The SegmentMetadataQuery cache key may contain UTF-8 column name strings. * Prepend 0xFF before the analysisTypes as a separator to avoid diff --git a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java index f5ab114f6873..437b193f2637 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -362,6 +362,18 @@ public Sequence> run( ); } + @Override + public QueryRunner> annotateDistributionTarget(QueryRunner> runner) + { + return (query, responseContext) -> { + final SearchQuery searchQuery = (SearchQuery) query; + return runner.run( + searchQuery.distributeBy(searchQuery.getDataSourceWithSegmentSpec()), + responseContext + ); + }; + } + private static class SearchThresholdAdjustingQueryRunner implements QueryRunner> { private final QueryRunner> runner; diff --git a/processing/src/main/java/io/druid/query/search/search/SearchQuery.java b/processing/src/main/java/io/druid/query/search/search/SearchQuery.java index c45a21cddb0d..95d4f53d0b24 100644 --- a/processing/src/main/java/io/druid/query/search/search/SearchQuery.java +++ b/processing/src/main/java/io/druid/query/search/search/SearchQuery.java @@ -24,10 +24,10 @@ import com.google.common.base.Preconditions; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularity; -import io.druid.query.BaseQuery; import io.druid.query.DataSource; import io.druid.query.Query; import io.druid.query.Result; +import io.druid.query.SingleSourceBaseQuery; import io.druid.query.dimension.DimensionSpec; import io.druid.query.filter.DimFilter; import io.druid.query.ordering.StringComparators; @@ -39,7 +39,7 @@ /** */ -public class SearchQuery extends BaseQuery> +public class SearchQuery extends SingleSourceBaseQuery> { private static final SearchSortSpec DEFAULT_SORT_SPEC = new SearchSortSpec(StringComparators.LEXICOGRAPHIC); diff --git a/processing/src/main/java/io/druid/query/select/SelectQuery.java b/processing/src/main/java/io/druid/query/select/SelectQuery.java index 4c0154857c6a..4f66b5d06968 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQuery.java +++ b/processing/src/main/java/io/druid/query/select/SelectQuery.java @@ -24,10 +24,10 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; import io.druid.java.util.common.granularity.Granularity; -import io.druid.query.BaseQuery; import io.druid.query.DataSource; import io.druid.query.Query; import io.druid.query.Result; +import io.druid.query.SingleSourceBaseQuery; import io.druid.query.dimension.DimensionSpec; import io.druid.query.filter.DimFilter; import io.druid.query.spec.QuerySegmentSpec; @@ -40,7 +40,7 @@ /** */ @JsonTypeName("select") -public class SelectQuery extends BaseQuery> +public class SelectQuery extends SingleSourceBaseQuery> { private final DimFilter dimFilter; private final Granularity granularity; diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java index 45948efbe71e..2fce7e37b0d5 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java @@ -435,4 +435,16 @@ public boolean apply(String input) } return queryIntervals; } + + @Override + public QueryRunner> annotateDistributionTarget(QueryRunner> runner) + { + return (query, responseContext) -> { + final SelectQuery selectQuery = (SelectQuery) query; + return runner.run( + selectQuery.distributeBy(selectQuery.getDataSourceWithSegmentSpec()), + responseContext + ); + }; + } } diff --git a/processing/src/main/java/io/druid/query/spec/QuerySegmentSpec.java b/processing/src/main/java/io/druid/query/spec/QuerySegmentSpec.java index 6bc26dc41120..d491397122c2 100644 --- a/processing/src/main/java/io/druid/query/spec/QuerySegmentSpec.java +++ b/processing/src/main/java/io/druid/query/spec/QuerySegmentSpec.java @@ -37,7 +37,7 @@ }) public interface QuerySegmentSpec { - public List getIntervals(); + List getIntervals(); - public QueryRunner lookup(Query query, QuerySegmentWalker walker); + QueryRunner lookup(Query query, QuerySegmentWalker walker); } diff --git a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java index c331ef7556d2..2456b27895c3 100644 --- a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java @@ -19,6 +19,7 @@ package io.druid.query.spec; +import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.collect.Lists; import io.druid.java.util.common.guava.Accumulator; @@ -28,6 +29,7 @@ import io.druid.java.util.common.guava.Yielder; import io.druid.java.util.common.guava.Yielders; import io.druid.java.util.common.guava.YieldingAccumulator; +import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.Result; @@ -37,31 +39,47 @@ import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; /** */ public class SpecificSegmentQueryRunner implements QueryRunner { private final QueryRunner base; + private final String dataSourceName; private final SpecificSegmentSpec specificSpec; public SpecificSegmentQueryRunner( QueryRunner base, + String datasourceName, SpecificSegmentSpec specificSpec ) { this.base = base; + this.dataSourceName = datasourceName; this.specificSpec = specificSpec; } @Override public Sequence run(final Query input, final Map responseContext) { - final Query query = input.withQuerySegmentSpec(specificSpec); + final Query query = input.replaceQuerySegmentSpecWith(dataSourceName, specificSpec); final Thread currThread = Thread.currentThread(); final String currThreadName = currThread.getName(); - final String newName = String.format("%s_%s_%s", query.getType(), query.getDataSource(), query.getIntervals()); + final List specs = StreamSupport + .stream(input.getDataSources().spliterator(), false) + .filter(spec -> spec.getDataSource().getNames().contains(dataSourceName)) + .collect(Collectors.toList()); + Preconditions.checkState(specs.size() == 1); + + final String newName = String.format( + "%s_%s_%s", + query.getType(), + specs.get(0).getDataSource(), + specs.get(0).getQuerySegmentSpec().getIntervals() + ); final Sequence baseSequence = doNamed( currThread, currThreadName, newName, new Supplier>() diff --git a/processing/src/main/java/io/druid/query/spec/SpecificSegmentSpec.java b/processing/src/main/java/io/druid/query/spec/SpecificSegmentSpec.java index fe69a0485180..bdf1d5e86a5b 100644 --- a/processing/src/main/java/io/druid/query/spec/SpecificSegmentSpec.java +++ b/processing/src/main/java/io/druid/query/spec/SpecificSegmentSpec.java @@ -19,6 +19,7 @@ package io.druid.query.spec; +import com.google.common.collect.ImmutableList; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QuerySegmentWalker; @@ -49,7 +50,7 @@ public List getIntervals() @Override public QueryRunner lookup(Query query, QuerySegmentWalker walker) { - return walker.getQueryRunnerForSegments(query, Arrays.asList(descriptor)); + return walker.getQueryRunnerForSegments(query, ImmutableList.of(descriptor)); } public SegmentDescriptor getDescriptor() { return descriptor; } diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java index 4501e40c5602..df96922da248 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java @@ -25,10 +25,10 @@ import com.google.common.collect.Maps; import io.druid.common.utils.JodaUtils; import io.druid.java.util.common.StringUtils; -import io.druid.query.BaseQuery; import io.druid.query.DataSource; import io.druid.query.Query; import io.druid.query.Result; +import io.druid.query.SingleSourceBaseQuery; import io.druid.query.filter.DimFilter; import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.query.spec.QuerySegmentSpec; @@ -42,7 +42,7 @@ /** */ -public class TimeBoundaryQuery extends BaseQuery> +public class TimeBoundaryQuery extends SingleSourceBaseQuery> { public static final Interval MY_Y2K_INTERVAL = new Interval( new DateTime("0000-01-01"), diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index a6569baed713..9f7c66c8259e 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -100,6 +100,18 @@ public boolean apply(T input) ); } + @Override + public QueryRunner> annotateDistributionTarget(QueryRunner> runner) + { + return (query, responseContext) -> { + final TimeBoundaryQuery timeBoundaryQuery = (TimeBoundaryQuery) query; + return runner.run( + timeBoundaryQuery.distributeBy(timeBoundaryQuery.getDataSourceWithSegmentSpec()), + responseContext + ); + }; + } + @Override public QueryRunner> mergeResults( final QueryRunner> runner diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java index ee3b375001a1..7cad4e941f7b 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java @@ -24,11 +24,11 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.collect.ImmutableList; import io.druid.java.util.common.granularity.Granularity; -import io.druid.query.BaseQuery; import io.druid.query.DataSource; import io.druid.query.Queries; import io.druid.query.Query; import io.druid.query.Result; +import io.druid.query.SingleSourceBaseQuery; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.filter.DimFilter; @@ -42,7 +42,7 @@ /** */ @JsonTypeName("timeseries") -public class TimeseriesQuery extends BaseQuery> +public class TimeseriesQuery extends SingleSourceBaseQuery> { private final VirtualColumns virtualColumns; private final DimFilter dimFilter; diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 17fb626ad826..ea89ff0bb1b7 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -230,6 +230,18 @@ public Sequence> run( }, this); } + @Override + public QueryRunner> annotateDistributionTarget(QueryRunner> runner) + { + return (query, responseContext) -> { + final TimeseriesQuery timeseriesQuery = (TimeseriesQuery) query; + return runner.run( + timeseriesQuery.distributeBy(timeseriesQuery.getDataSourceWithSegmentSpec()), + responseContext + ); + }; + } + @Override public Function, Result> makePreComputeManipulatorFn( final TimeseriesQuery query, final MetricManipulationFn fn diff --git a/processing/src/main/java/io/druid/query/topn/TopNQuery.java b/processing/src/main/java/io/druid/query/topn/TopNQuery.java index 29ca0cddcba5..230231650562 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQuery.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQuery.java @@ -24,11 +24,11 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import io.druid.java.util.common.granularity.Granularity; -import io.druid.query.BaseQuery; import io.druid.query.DataSource; import io.druid.query.Queries; import io.druid.query.Query; import io.druid.query.Result; +import io.druid.query.SingleSourceBaseQuery; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.dimension.DimensionSpec; @@ -42,7 +42,7 @@ /** */ -public class TopNQuery extends BaseQuery> +public class TopNQuery extends SingleSourceBaseQuery> { public static final String TOPN = "topN"; diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java b/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java index bdf8dc6153a9..ae5723018069 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java @@ -23,6 +23,9 @@ import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularity; import io.druid.query.DataSource; +import io.druid.query.DataSourceWithSegmentSpec; +import io.druid.query.QueryContextKeys; +import io.druid.query.SingleSourceBaseQuery; import io.druid.query.TableDataSource; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; @@ -38,6 +41,7 @@ import org.joda.time.Interval; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -207,6 +211,21 @@ public TopNQueryBuilder dataSource(String d) return this; } + public TopNQueryBuilder updateDistributionTarget() + { + if (context == null) { + context = new HashMap<>(); + } + context.put( + QueryContextKeys.DIST_TARGET_SOURCE, + new DataSourceWithSegmentSpec( + SingleSourceBaseQuery.getLeafDataSource(dataSource), + querySegmentSpec + ) + ); + return this; + } + public TopNQueryBuilder virtualColumns(VirtualColumns virtualColumns) { this.virtualColumns = virtualColumns; @@ -325,7 +344,11 @@ public TopNQueryBuilder postAggregators(List p) public TopNQueryBuilder context(Map c) { - context = c; + if (context == null) { + context = new HashMap<>(c); + } else { + context.putAll(c); + } return this; } } diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java index a7826dbdb9d9..626888dc3d84 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java @@ -505,6 +505,18 @@ public DimensionAndMetricValueExtractor apply( }; } + @Override + public QueryRunner> annotateDistributionTarget(QueryRunner> runner) + { + return (query, responseContext) -> { + final TopNQuery topNQuery = (TopNQuery) query; + return runner.run( + topNQuery.distributeBy(topNQuery.getDataSourceWithSegmentSpec()), + responseContext + ); + }; + } + static class ThresholdAdjustingQueryRunner implements QueryRunner> { private final QueryRunner> runner; diff --git a/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java b/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java index 63740e9e730c..7a2ae1167889 100644 --- a/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java @@ -29,14 +29,11 @@ import io.druid.query.filter.SelectorDimFilter; import io.druid.query.topn.TopNQuery; import io.druid.query.topn.TopNQueryBuilder; -import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; import java.util.Collections; -import java.util.List; import java.util.Map; -import java.util.stream.Collectors; public class DefaultQueryMetricsTest { @@ -69,19 +66,15 @@ public void testDefaultQueryMetricsQuery() queryMetrics.reportQueryTime(0).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); - Assert.assertEquals(12, actualEvent.size()); + Assert.assertEquals(11, actualEvent.size()); Assert.assertTrue(actualEvent.containsKey("feed")); Assert.assertTrue(actualEvent.containsKey("timestamp")); Assert.assertEquals("", actualEvent.get("host")); Assert.assertEquals("", actualEvent.get("service")); - Assert.assertEquals("xx", actualEvent.get(DruidMetrics.DATASOURCE)); + Assert.assertEquals(DataSourceUtil.getMetricName(query.getDataSources()), actualEvent.get(DruidMetrics.DATASOURCE)); Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE)); - List expectedIntervals = QueryRunnerTestHelper.fullOnInterval.getIntervals(); - List expectedStringIntervals = - expectedIntervals.stream().map(Interval::toString).collect(Collectors.toList()); - Assert.assertEquals(expectedStringIntervals, actualEvent.get(DruidMetrics.INTERVAL)); Assert.assertEquals("true", actualEvent.get("hasFilters")); - Assert.assertEquals(expectedIntervals.get(0).toDuration().toString(), actualEvent.get("duration")); + Assert.assertEquals(query.getTotalDuration().toString(), actualEvent.get("duration")); Assert.assertEquals("", actualEvent.get(DruidMetrics.ID)); Assert.assertEquals("query/time", actualEvent.get("metric")); Assert.assertEquals(0L, actualEvent.get("value")); diff --git a/processing/src/test/java/io/druid/query/IntervalChunkingQueryRunnerTest.java b/processing/src/test/java/io/druid/query/IntervalChunkingQueryRunnerTest.java index 194b12e6bdc5..acb8a762200e 100644 --- a/processing/src/test/java/io/druid/query/IntervalChunkingQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/IntervalChunkingQueryRunnerTest.java @@ -26,6 +26,7 @@ import io.druid.query.Druids.TimeseriesQueryBuilder; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.timeseries.TimeseriesQuery; import org.easymock.EasyMock; import org.junit.Before; import org.junit.Test; @@ -74,6 +75,7 @@ public void testDefaultNoChunking() { @Test public void testChunking() { Query query = queryBuilder.intervals("2015-01-01T00:00:00.000/2015-01-11T00:00:00.000").context(ImmutableMap.of("chunkPeriod", "P1D")).build(); + query = query.distributeBy(((TimeseriesQuery) query).getDataSourceWithSegmentSpec()); executors.execute(EasyMock.anyObject(Runnable.class)); EasyMock.expectLastCall().times(10); @@ -90,6 +92,7 @@ public void testChunking() { @Test public void testChunkingOnMonths() { Query query = queryBuilder.intervals("2015-01-01T00:00:00.000/2015-02-11T00:00:00.000").context(ImmutableMap.of("chunkPeriod", "P1M")).build(); + query = query.distributeBy(((TimeseriesQuery) query).getDataSourceWithSegmentSpec()); executors.execute(EasyMock.anyObject(Runnable.class)); EasyMock.expectLastCall().times(2); diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index c1206adae34d..8559a76c2fa0 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -75,6 +75,8 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; /** */ @@ -332,7 +334,7 @@ public void remove() }; } - public static > List> makeQueryRunners( + public static > List> makeQueryRunners( QueryRunnerFactory factory ) throws IOException @@ -434,7 +436,7 @@ public void remove() }; } - public static > QueryRunner makeQueryRunner( + public static > QueryRunner makeQueryRunner( QueryRunnerFactory factory, String resourceFileName, final String runnerName @@ -448,7 +450,7 @@ public static > QueryRunner makeQueryRunner( ); } - public static > QueryRunner makeQueryRunner( + public static > QueryRunner makeQueryRunner( QueryRunnerFactory factory, Segment adapter, final String runnerName @@ -457,7 +459,7 @@ public static > QueryRunner makeQueryRunner( return makeQueryRunner(factory, segmentId, adapter, runnerName); } - public static > QueryRunner makeQueryRunner( + public static > QueryRunner makeQueryRunner( QueryRunnerFactory factory, String segmentId, Segment adapter, @@ -528,22 +530,28 @@ public static QueryRunner makeFilteringQueryRunner( @Override public Sequence run(Query query, Map responseContext) { - List segments = Lists.newArrayList(); - for (Interval interval : query.getIntervals()) { - segments.addAll(timeline.lookup(interval)); - } + final List segments = StreamSupport + .stream(query.getDataSources().spliterator(), false) + .flatMap(spec -> spec.getQuerySegmentSpec().getIntervals().stream()) + .flatMap(interval -> timeline.lookup(interval).stream()) + .collect(Collectors.toList()); List> sequences = Lists.newArrayList(); for (TimelineObjectHolder holder : toolChest.filterSegments(query, segments)) { Segment segment = holder.getObject().getChunk(0).getObject(); - Query running = query.withQuerySegmentSpec( - new SpecificSegmentSpec( - new SegmentDescriptor( - holder.getInterval(), - holder.getVersion(), - 0 - ) - ) - ); + Query running = query; + for (DataSourceWithSegmentSpec spec : query.getDataSources()) { + running = query.replaceQuerySegmentSpecWith( + spec.getDataSource(), + new SpecificSegmentSpec( + new SegmentDescriptor( + holder.getInterval(), + holder.getVersion(), + 0 + ) + ) + ); + } + sequences.add(factory.createRunner(segment).run(running, responseContext)); } return new MergeSequence<>(query.getResultOrdering(), Sequences.simple(sequences)); diff --git a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java index d7ae4b8b7c1c..ab7a6e129c47 100644 --- a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java @@ -20,6 +20,7 @@ package io.druid.query; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.MapMaker; import com.google.common.collect.Maps; @@ -38,6 +39,7 @@ import org.junit.Test; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -66,19 +68,23 @@ public class RetryQueryRunnerTest public void testRunWithMissingSegments() throws Exception { Map context = new MapMaker().makeMap(); - context.put(Result.MISSING_SEGMENTS_KEY, Lists.newArrayList()); + context.put(Result.MISSING_SEGMENTS_KEY, new HashMap<>()); RetryQueryRunner> runner = new RetryQueryRunner<>( new QueryRunner>() { @Override public Sequence> run(Query query, Map context) { - ((List) context.get(Result.MISSING_SEGMENTS_KEY)).add( - new SegmentDescriptor( - new Interval( - 178888, - 1999999 - ), "test", 1 + final TimeseriesQuery timeseriesQuery = (TimeseriesQuery) query; + ((Map) context.get(Result.MISSING_SEGMENTS_KEY)).put( + Iterables.getOnlyElement(timeseriesQuery.getDataSource().getNames()), + Lists.newArrayList( + new SegmentDescriptor( + new Interval( + 178888, + 1999999 + ), "test", 1 + ) ) ); return Sequences.empty(); @@ -111,7 +117,7 @@ public boolean isReturnPartialResults() Assert.assertTrue( "Should have one entry in the list of missing segments", - ((List) context.get(Result.MISSING_SEGMENTS_KEY)).size() == 1 + ((Map) context.get(Result.MISSING_SEGMENTS_KEY)).size() == 1 ); Assert.assertTrue("Should return an empty sequence as a result", ((List) actualResults).size() == 0); } @@ -122,7 +128,7 @@ public void testRetry() throws Exception { Map context = new MapMaker().makeMap(); context.put("count", 0); - context.put(Result.MISSING_SEGMENTS_KEY, Lists.newArrayList()); + context.put(Result.MISSING_SEGMENTS_KEY, new HashMap<>()); RetryQueryRunner> runner = new RetryQueryRunner<>( new QueryRunner>() { @@ -132,13 +138,17 @@ public Sequence> run( Map context ) { + final TimeseriesQuery timeseriesQuery = (TimeseriesQuery) query; if ((int) context.get("count") == 0) { - ((List) context.get(Result.MISSING_SEGMENTS_KEY)).add( - new SegmentDescriptor( - new Interval( - 178888, - 1999999 - ), "test", 1 + ((Map>) context.get(Result.MISSING_SEGMENTS_KEY)).put( + Iterables.getOnlyElement(timeseriesQuery.getDataSource().getNames()), + Lists.newArrayList( + new SegmentDescriptor( + new Interval( + 178888, + 1999999 + ), "test", 1 + ) ) ); context.put("count", 1); @@ -180,7 +190,7 @@ public Sequence> run( Assert.assertTrue("Should return a list with one element", ((List) actualResults).size() == 1); Assert.assertTrue( "Should have nothing in missingSegment list", - ((List) context.get(Result.MISSING_SEGMENTS_KEY)).size() == 0 + ((Map) context.get(Result.MISSING_SEGMENTS_KEY)).size() == 0 ); } @@ -189,7 +199,7 @@ public void testRetryMultiple() throws Exception { Map context = new MapMaker().makeMap(); context.put("count", 0); - context.put(Result.MISSING_SEGMENTS_KEY, Lists.newArrayList()); + context.put(Result.MISSING_SEGMENTS_KEY, new HashMap<>()); RetryQueryRunner> runner = new RetryQueryRunner<>( new QueryRunner>() { @@ -199,13 +209,17 @@ public Sequence> run( Map context ) { + final TimeseriesQuery timeseriesQuery = (TimeseriesQuery) query; if ((int) context.get("count") < 3) { - ((List) context.get(Result.MISSING_SEGMENTS_KEY)).add( - new SegmentDescriptor( - new Interval( - 178888, - 1999999 - ), "test", 1 + ((Map) context.get(Result.MISSING_SEGMENTS_KEY)).put( + Iterables.getOnlyElement(timeseriesQuery.getDataSource().getNames()), + Lists.newArrayList( + new SegmentDescriptor( + new Interval( + 178888, + 1999999 + ), "test", 1 + ) ) ); context.put("count", (int) context.get("count") + 1); @@ -247,7 +261,7 @@ public Sequence> run( Assert.assertTrue("Should return a list with one element", ((List) actualResults).size() == 1); Assert.assertTrue( "Should have nothing in missingSegment list", - ((List) context.get(Result.MISSING_SEGMENTS_KEY)).size() == 0 + ((Map) context.get(Result.MISSING_SEGMENTS_KEY)).size() == 0 ); } @@ -255,7 +269,7 @@ public Sequence> run( public void testException() throws Exception { Map context = new MapMaker().makeMap(); - context.put(Result.MISSING_SEGMENTS_KEY, Lists.newArrayList()); + context.put(Result.MISSING_SEGMENTS_KEY, new HashMap<>()); RetryQueryRunner> runner = new RetryQueryRunner<>( new QueryRunner>() { @@ -265,12 +279,16 @@ public Sequence> run( Map context ) { - ((List) context.get(Result.MISSING_SEGMENTS_KEY)).add( - new SegmentDescriptor( - new Interval( - 178888, - 1999999 - ), "test", 1 + final TimeseriesQuery timeseriesQuery = (TimeseriesQuery) query; + ((Map>) context.get(Result.MISSING_SEGMENTS_KEY)).put( + Iterables.getOnlyElement(timeseriesQuery.getDataSource().getNames()), + Lists.newArrayList( + new SegmentDescriptor( + new Interval( + 178888, + 1999999 + ), "test", 1 + ) ) ); return Sequences.empty(); @@ -307,7 +325,7 @@ public void testNoDuplicateRetry() throws Exception { Map context = new MapMaker().makeMap(); context.put("count", 0); - context.put(Result.MISSING_SEGMENTS_KEY, Lists.newArrayList()); + context.put(Result.MISSING_SEGMENTS_KEY, new HashMap<>()); RetryQueryRunner> runner = new RetryQueryRunner<>( new QueryRunner>() { @@ -317,9 +335,16 @@ public Sequence> run( Map context ) { + final TimeseriesQuery timeseriesQuery = (TimeseriesQuery) query; if ((int) context.get("count") == 0) { // assume 2 missing segments at first run - ((List) context.get(Result.MISSING_SEGMENTS_KEY)).add( + final Map> missingSegments = + (Map>) context.get(Result.MISSING_SEGMENTS_KEY); + final List list = missingSegments.computeIfAbsent( + Iterables.getOnlyElement(timeseriesQuery.getDataSource().getNames()), + key -> Lists.newArrayList() + ); + list.add( new SegmentDescriptor( new Interval( 178888, @@ -327,7 +352,7 @@ public Sequence> run( ), "test", 1 ) ); - ((List) context.get(Result.MISSING_SEGMENTS_KEY)).add( + list.add( new SegmentDescriptor( new Interval( 178888, @@ -335,6 +360,7 @@ public Sequence> run( ), "test", 2 ) ); + context.put("count", 1); return Sequences.simple( Arrays.asList( @@ -348,14 +374,20 @@ public Sequence> run( ); } else if ((int) context.get("count") == 1) { // this is first retry - Assert.assertTrue("Should retry with 2 missing segments", ((MultipleSpecificSegmentSpec)((BaseQuery)query).getQuerySegmentSpec()).getDescriptors().size() == 2); + Assert.assertTrue( + "Should retry with 2 missing segments", + ((MultipleSpecificSegmentSpec)((SingleSourceBaseQuery)query).getQuerySegmentSpec()).getDescriptors().size() == 2 + ); // assume only left 1 missing at first retry - ((List) context.get(Result.MISSING_SEGMENTS_KEY)).add( - new SegmentDescriptor( - new Interval( - 178888, - 1999999 - ), "test", 2 + ((Map) context.get(Result.MISSING_SEGMENTS_KEY)).put( + Iterables.getOnlyElement(timeseriesQuery.getDataSource().getNames()), + Lists.newArrayList( + new SegmentDescriptor( + new Interval( + 178888, + 1999999 + ), "test", 2 + ) ) ); context.put("count", 2); @@ -371,7 +403,10 @@ public Sequence> run( ); } else { // this is second retry - Assert.assertTrue("Should retry with 1 missing segments", ((MultipleSpecificSegmentSpec)((BaseQuery)query).getQuerySegmentSpec()).getDescriptors().size() == 1); + Assert.assertTrue( + "Should retry with 1 missing segments", + ((MultipleSpecificSegmentSpec)((SingleSourceBaseQuery)query).getQuerySegmentSpec()).getDescriptors().size() == 1 + ); // assume no more missing at second retry context.put("count", 3); return Sequences.simple( @@ -410,7 +445,7 @@ public Sequence> run( Assert.assertTrue("Should return a list with 3 elements", ((List) actualResults).size() == 3); Assert.assertTrue( "Should have nothing in missingSegment list", - ((List) context.get(Result.MISSING_SEGMENTS_KEY)).size() == 0 + ((Map) context.get(Result.MISSING_SEGMENTS_KEY)).size() == 0 ); } } diff --git a/processing/src/test/java/io/druid/query/SchemaEvolutionTest.java b/processing/src/test/java/io/druid/query/SchemaEvolutionTest.java index 69a06f6ce37f..65b140489569 100644 --- a/processing/src/test/java/io/druid/query/SchemaEvolutionTest.java +++ b/processing/src/test/java/io/druid/query/SchemaEvolutionTest.java @@ -97,7 +97,7 @@ public static List inputRowsWithDimensions(final List dimensio ); } - public static > List runQuery( + public static > List runQuery( final QueryType query, final QueryRunnerFactory factory, final List indexes diff --git a/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java b/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java index 2f8ae2181088..0523afaae82d 100644 --- a/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java +++ b/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableList; 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 io.druid.java.util.common.guava.Sequence; @@ -96,7 +97,7 @@ public Sequence> run( new TimeseriesResultValue(ImmutableMap.of("metric", 3)) ), new Result<>( - query.getIntervals().get(0).getEnd(), + ((SingleSourceBaseQuery>)query).getIntervals().get(0).getEnd(), new TimeseriesResultValue(ImmutableMap.of("metric", 5)) ) ) @@ -106,12 +107,13 @@ public Sequence> run( new DateTime("2014-08-02").getMillis() ); - final Query> query = + Query> query = Druids.newTimeseriesQueryBuilder() .dataSource("dummy") .intervals("2014-07-31/2014-08-05") .aggregators(Arrays.asList(new CountAggregatorFactory("count"))) .build(); + query = setDistributionTarget(query); Assert.assertEquals( Lists.newArrayList( @@ -165,10 +167,11 @@ public Sequence> run( new DateTime("2014-08-02").getMillis() ); - final Query> timeBoundaryQuery = + Query> timeBoundaryQuery = Druids.newTimeBoundaryQueryBuilder() .dataSource("dummy") .build(); + timeBoundaryQuery = setDistributionTarget(timeBoundaryQuery); Assert.assertEquals( Lists.newArrayList( @@ -200,11 +203,11 @@ public Sequence> run( return Sequences.simple( ImmutableList.of( new Result<>( - query.getIntervals().get(0).getStart(), + ((SingleSourceBaseQuery>)query).getIntervals().get(0).getStart(), new TimeseriesResultValue(ImmutableMap.of("metric", 2)) ), new Result<>( - query.getIntervals().get(0).getEnd(), + ((SingleSourceBaseQuery>)query).getIntervals().get(0).getEnd(), new TimeseriesResultValue(ImmutableMap.of("metric", 3)) ) ) @@ -214,12 +217,13 @@ public Sequence> run( new DateTime("2014-08-02").getMillis() ); - final Query> query = + Query> query = Druids.newTimeseriesQueryBuilder() .dataSource("dummy") .intervals("2014-08-06/2014-08-08") .aggregators(Arrays.asList(new CountAggregatorFactory("count"))) .build(); + query = setDistributionTarget(query); Assert.assertEquals( Lists.newArrayList( @@ -235,4 +239,17 @@ public Sequence> run( Sequences.toList(queryRunner.run(query, Maps.newHashMap()), Lists.>newArrayList()) ); } + + private static Query setDistributionTarget(Query query) + { + return query.withOverriddenContext( + ImmutableMap.of( + QueryContextKeys.DIST_TARGET_SOURCE, + new DataSourceWithSegmentSpec( + Iterables.getOnlyElement(query.getDataSources()).getDataSource(), + Iterables.getOnlyElement(query.getDataSources()).getQuerySegmentSpec() + ) + ) + ); + } } diff --git a/processing/src/test/java/io/druid/query/UnionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/UnionQueryRunnerTest.java index b7b914fe5ab4..0f396af83dde 100644 --- a/processing/src/test/java/io/druid/query/UnionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/UnionQueryRunnerTest.java @@ -42,8 +42,8 @@ public void testUnionQueryRunner() public Sequence run(Query query, Map responseContext) { // verify that table datasource is passed to baseQueryRunner - Assert.assertTrue(query.getDataSource() instanceof TableDataSource); - String dsName = Iterables.getOnlyElement(query.getDataSource().getNames()); + Assert.assertTrue(((SingleSourceBaseQuery)query).getDataSource() instanceof TableDataSource); + String dsName = Iterables.getOnlyElement(((SingleSourceBaseQuery)query).getDataSource().getNames()); if (dsName.equals("ds1")) { responseContext.put("ds1", "ds1"); return Sequences.simple(Arrays.asList(1, 2, 3)); @@ -71,7 +71,7 @@ public Sequence run(Query query, Map responseContext) .build(); Map responseContext = Maps.newHashMap(); Sequence result = runner.run(q, responseContext); - List res = Sequences.toList(result, Lists.newArrayList()); + List res = Sequences.toList(result, Lists.newArrayList()); Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5, 6), res); // verify response context diff --git a/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java b/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java index f04600c28181..5352d0199877 100644 --- a/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java @@ -25,6 +25,7 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.granularity.PeriodGranularity; import io.druid.query.CachingEmitter; +import io.druid.query.DataSourceUtil; import io.druid.query.DefaultQueryMetricsTest; import io.druid.query.DruidMetrics; import io.druid.query.QueryRunnerTestHelper; @@ -34,13 +35,11 @@ import io.druid.query.extraction.MapLookupExtractor; import io.druid.query.filter.SelectorDimFilter; import io.druid.query.lookup.LookupExtractionFn; -import org.joda.time.Interval; import org.joda.time.Period; import org.junit.Assert; import org.junit.Test; import java.util.Arrays; -import java.util.Collections; import java.util.Map; public class DefaultGroupByQueryMetricsTest @@ -92,17 +91,15 @@ public void testDefaultGroupByQueryMetricsQuery() queryMetrics.reportQueryTime(0).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); - Assert.assertEquals(15, actualEvent.size()); + Assert.assertEquals(14, actualEvent.size()); Assert.assertTrue(actualEvent.containsKey("feed")); Assert.assertTrue(actualEvent.containsKey("timestamp")); Assert.assertEquals("", actualEvent.get("host")); Assert.assertEquals("", actualEvent.get("service")); - Assert.assertEquals(QueryRunnerTestHelper.dataSource, actualEvent.get(DruidMetrics.DATASOURCE)); + Assert.assertEquals(DataSourceUtil.getMetricName(query.getDataSources()), actualEvent.get(DruidMetrics.DATASOURCE)); Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE)); - Interval expectedInterval = new Interval("2011-04-02/2011-04-04"); - Assert.assertEquals(Collections.singletonList(expectedInterval.toString()), actualEvent.get(DruidMetrics.INTERVAL)); Assert.assertEquals("true", actualEvent.get("hasFilters")); - Assert.assertEquals(expectedInterval.toDuration().toString(), actualEvent.get("duration")); + Assert.assertEquals(query.getTotalDuration().toString(), actualEvent.get("duration")); Assert.assertEquals("", actualEvent.get(DruidMetrics.ID)); // GroupBy-specific dimensions diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 07a4b876c67b..1476502b4790 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -2251,10 +2251,11 @@ public Sequence run( ) { // simulate two daily segments - final Query query1 = query.withQuerySegmentSpec( + final GroupByQuery groupByQuery = (GroupByQuery) query; + final Query query1 = groupByQuery.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03"))) ); - final Query query2 = query.withQuerySegmentSpec( + final Query query2 = groupByQuery.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) ); return new MergeSequence( @@ -2547,10 +2548,11 @@ public Sequence run( ) { // simulate two daily segments - final Query query1 = query.withQuerySegmentSpec( + final GroupByQuery groupByQuery = (GroupByQuery) query; + final Query query1 = groupByQuery.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03"))) ); - final Query query2 = query.withQuerySegmentSpec( + final Query query2 = groupByQuery.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) ); return new MergeSequence( @@ -3335,10 +3337,11 @@ public Sequence run( ) { // simulate two daily segments - final Query query1 = query.withQuerySegmentSpec( + final GroupByQuery groupByQuery = (GroupByQuery) query; + final Query query1 = groupByQuery.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03"))) ); - final Query query2 = query.withQuerySegmentSpec( + final Query query2 = groupByQuery.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) ); return new MergeSequence( @@ -3668,10 +3671,11 @@ public Sequence run( ) { // simulate two daily segments - final Query query1 = query.withQuerySegmentSpec( + final GroupByQuery groupByQuery = (GroupByQuery) query; + final Query query1 = groupByQuery.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03"))) ); - final Query query2 = query.withQuerySegmentSpec( + final Query query2 = groupByQuery.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) ); return new MergeSequence( @@ -3777,10 +3781,11 @@ public Sequence run( ) { // simulate two daily segments - final Query query1 = query.withQuerySegmentSpec( + final GroupByQuery groupByQuery = (GroupByQuery) query; + final Query query1 = groupByQuery.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-02/2011-04-03"))) ); - final Query query2 = query.withQuerySegmentSpec( + final Query query2 = groupByQuery.withQuerySegmentSpec( new MultipleIntervalSegmentSpec(Lists.newArrayList(new Interval("2011-04-03/2011-04-04"))) ); return new MergeSequence( diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTestHelper.java index e6a7dc39d7b0..21c03ddf326a 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTestHelper.java @@ -42,12 +42,20 @@ */ public class GroupByQueryRunnerTestHelper { - public static Iterable runQuery(QueryRunnerFactory factory, QueryRunner runner, Query query) + public static Iterable runQuery( + QueryRunnerFactory factory, + QueryRunner runner, + Query query + ) { QueryToolChest toolChest = factory.getToolchest(); QueryRunner theRunner = new FinalizeResultsQueryRunner<>( - toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner)), + toolChest.mergeResults( + toolChest.annotateDistributionTarget( + toolChest.preMergeQueryDecoration(runner) + ) + ), toolChest ); diff --git a/processing/src/test/java/io/druid/query/join/JoinQueryTest.java b/processing/src/test/java/io/druid/query/join/JoinQueryTest.java new file mode 100644 index 000000000000..c731b47887a1 --- /dev/null +++ b/processing/src/test/java/io/druid/query/join/JoinQueryTest.java @@ -0,0 +1,100 @@ +/* + * 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.join; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.Query; +import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.TableDataSource; +import io.druid.query.dimension.DefaultDimensionSpec; +import io.druid.segment.VirtualColumns; +import org.junit.Test; + +import java.io.IOException; + +import static org.junit.Assert.assertEquals; + +public class JoinQueryTest +{ + private static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper(); + + @Test + public void testSerde() throws IOException + { + final JoinSpec leftChildSpec = new JoinSpec( + JoinType.INNER, + new AndPredicate( + ImmutableList.of( + new EqualPredicate( + new DimExtractPredicate(new DefaultDimensionSpec("src1", "dim1", "dim1")), + new DimExtractPredicate(new DefaultDimensionSpec("src2", "dim1", "dim1")) + ), + new EqualPredicate( + new AddPredicate( + new DimExtractPredicate(new DefaultDimensionSpec("src2", "dim2", "dim2")), + new LiteralPredicate("10") + ), + new AddPredicate( + new DimExtractPredicate(new DefaultDimensionSpec("src1", "dim2", "dim2")), + new DimExtractPredicate(new DefaultDimensionSpec("src1", "dim3", "dim3")) + ) + ) + ) + ), + new DataInput(new TableDataSource("src1"), QueryRunnerTestHelper.firstToThird), + new DataInput(new TableDataSource("src2"), QueryRunnerTestHelper.firstToThird) + ); + + final JoinSpec joinSpec = new JoinSpec( + JoinType.INNER, + new EqualPredicate( + new DimExtractPredicate(new DefaultDimensionSpec("j1", "dim4", "dim4")), + new DimExtractPredicate(new DefaultDimensionSpec("src3", "dim4", "dim4")) + ), + leftChildSpec, + new DataInput(new TableDataSource("src3"), QueryRunnerTestHelper.firstToThird) + ); + + final Query query = JoinQuery.newBuilder() + .setJoinSpec(joinSpec) + .setGranularity(QueryRunnerTestHelper.dayGran) + .setDimensions( + ImmutableList.of( + new DefaultDimensionSpec("src1", "dim5", "dim5"), + new DefaultDimensionSpec("src2", "dim5", "dim5"), + new DefaultDimensionSpec("src3", "dim5", "dim5") + ) + ) + .setMetrics( + ImmutableList.of( + "met1", "met2", "met3" + ) + ) + .setVirtualColumns(VirtualColumns.EMPTY) + .build(); + + final String json = JSON_MAPPER.writeValueAsString(query); + final Query fromJson = JSON_MAPPER.readValue(json, Query.class); + + assertEquals(query, fromJson); + } +} diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java index fbb8e25cf57d..bb02eb2db00a 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java @@ -879,8 +879,11 @@ public void testSerde() throws Exception 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("2013-12-04T00:00:00.000Z/2013-12-05T00:00:00.000Z"), query.getIntervals().get(0)); + Assert.assertEquals("test_ds", Iterables.getOnlyElement(((SegmentMetadataQuery)query).getDataSource().getNames())); + Assert.assertEquals( + new Interval("2013-12-04T00:00:00.000Z/2013-12-05T00:00:00.000Z"), + ((SegmentMetadataQuery)query).getIntervals().get(0) + ); Assert.assertEquals(expectedAnalysisTypes, ((SegmentMetadataQuery) query).getAnalysisTypes()); // test serialize and deserialize @@ -896,8 +899,11 @@ public void testSerdeWithDefaultInterval() throws Exception + "}"; 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.assertEquals("test_ds", Iterables.getOnlyElement(((SegmentMetadataQuery)query).getDataSource().getNames())); + Assert.assertEquals( + new Interval(JodaUtils.MIN_INSTANT, JodaUtils.MAX_INSTANT), + ((SegmentMetadataQuery)query).getIntervals().get(0) + ); Assert.assertTrue(((SegmentMetadataQuery) query).isUsingDefaultInterval()); // test serialize and deserialize diff --git a/processing/src/test/java/io/druid/query/select/SelectQuerySpecTest.java b/processing/src/test/java/io/druid/query/select/SelectQuerySpecTest.java index 96bb12a38174..63672f383f69 100644 --- a/processing/src/test/java/io/druid/query/select/SelectQuerySpecTest.java +++ b/processing/src/test/java/io/druid/query/select/SelectQuerySpecTest.java @@ -68,8 +68,8 @@ public void testSerializationLegacyString() throws Exception + "\"filter\":null," + "\"granularity\":{\"type\":\"all\"}," + "\"dimensions\":" - + "[{\"type\":\"default\",\"dimension\":\"market\",\"outputName\":\"market\",\"outputType\":\"STRING\"}," - + "{\"type\":\"default\",\"dimension\":\"quality\",\"outputName\":\"quality\",\"outputType\":\"STRING\"}]," + + "[{\"type\":\"default\",\"dataSource\":null,\"dimension\":\"market\",\"outputName\":\"market\",\"outputType\":\"STRING\"}," + + "{\"type\":\"default\",\"dataSource\":null,\"dimension\":\"quality\",\"outputName\":\"quality\",\"outputType\":\"STRING\"}]," + "\"metrics\":[\"index\"]," + "\"virtualColumns\":[]," + "\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":3,\"fromNext\":true}," @@ -104,8 +104,8 @@ public void testPagingSpecFromNext() throws Exception + "\"filter\":null," + "\"granularity\":{\"type\":\"all\"}," + "\"dimensions\":" - + "[{\"type\":\"default\",\"dimension\":\"market\",\"outputName\":\"market\",\"outputType\":\"STRING\"}," - + "{\"type\":\"default\",\"dimension\":\"quality\",\"outputName\":\"quality\",\"outputType\":\"STRING\"}]," + + "[{\"type\":\"default\",\"dataSource\":null,\"dimension\":\"market\",\"outputName\":\"market\",\"outputType\":\"STRING\"}," + + "{\"type\":\"default\",\"dataSource\":null,\"dimension\":\"quality\",\"outputName\":\"quality\",\"outputType\":\"STRING\"}]," + "\"metrics\":[\"index\"]," + "\"virtualColumns\":[],"; diff --git a/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java index 04bf6d66dc82..3fbe342037d8 100644 --- a/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java @@ -89,6 +89,7 @@ public Yielder toYielder( } }, + "foo", new SpecificSegmentSpec( descriptor ) @@ -167,6 +168,7 @@ public void run() ); } }, + "foo", new SpecificSegmentSpec( descriptor ) diff --git a/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java b/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java index fb03645c0675..75228dfa5863 100644 --- a/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java @@ -22,19 +22,17 @@ import com.metamx.emitter.service.ServiceEmitter; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.CachingEmitter; +import io.druid.query.DataSourceUtil; import io.druid.query.DefaultQueryMetricsTest; import io.druid.query.DruidMetrics; import io.druid.query.Druids; import io.druid.query.QueryRunnerTestHelper; -import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; import java.util.Arrays; import java.util.Collections; -import java.util.List; import java.util.Map; -import java.util.stream.Collectors; public class DefaultTimeseriesQueryMetricsTest { @@ -67,19 +65,15 @@ public void testDefaultTimeseriesQueryMetricsQuery() queryMetrics.reportQueryTime(0).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); - Assert.assertEquals(14, actualEvent.size()); + Assert.assertEquals(13, actualEvent.size()); Assert.assertTrue(actualEvent.containsKey("feed")); Assert.assertTrue(actualEvent.containsKey("timestamp")); Assert.assertEquals("", actualEvent.get("host")); Assert.assertEquals("", actualEvent.get("service")); - Assert.assertEquals(QueryRunnerTestHelper.dataSource, actualEvent.get(DruidMetrics.DATASOURCE)); + Assert.assertEquals(DataSourceUtil.getMetricName(query.getDataSources()), actualEvent.get(DruidMetrics.DATASOURCE)); Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE)); - List expectedIntervals = QueryRunnerTestHelper.fullOnInterval.getIntervals(); - List expectedStringIntervals = - expectedIntervals.stream().map(Interval::toString).collect(Collectors.toList()); - Assert.assertEquals(expectedStringIntervals, actualEvent.get(DruidMetrics.INTERVAL)); Assert.assertEquals("false", actualEvent.get("hasFilters")); - Assert.assertEquals(expectedIntervals.get(0).toDuration().toString(), actualEvent.get("duration")); + Assert.assertEquals(query.getTotalDuration().toString(), actualEvent.get("duration")); Assert.assertEquals("", actualEvent.get(DruidMetrics.ID)); // Timeseries-specific dimensions diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java index 69a1ea435154..da50c97b9547 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java @@ -190,7 +190,7 @@ public Sequence> run( Map responseContext ) { - if (query.getDataSource().equals(new TableDataSource("ds1"))) { + if (((TimeseriesQuery)query).getDataSource().equals(new TableDataSource("ds1"))) { return Sequences.simple(descending ? Lists.reverse(ds1) : ds1); } else { return Sequences.simple(descending ? Lists.reverse(ds2) : ds2); diff --git a/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java b/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java index 185795fc1db0..4d20e3b1b9a6 100644 --- a/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java @@ -24,6 +24,7 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.granularity.Granularities; import io.druid.query.CachingEmitter; +import io.druid.query.DataSourceUtil; import io.druid.query.DefaultQueryMetricsTest; import io.druid.query.DruidMetrics; import io.druid.query.QueryRunnerTestHelper; @@ -31,14 +32,11 @@ import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.ListFilteredDimensionSpec; import io.druid.query.filter.SelectorDimFilter; -import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; import java.util.Collections; -import java.util.List; import java.util.Map; -import java.util.stream.Collectors; public class DefaultTopNQueryMetricsTest { @@ -71,19 +69,15 @@ public void testDefaultTopNQueryMetricsQuery() queryMetrics.reportQueryTime(0).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); - Assert.assertEquals(16, actualEvent.size()); + Assert.assertEquals(15, actualEvent.size()); Assert.assertTrue(actualEvent.containsKey("feed")); Assert.assertTrue(actualEvent.containsKey("timestamp")); Assert.assertEquals("", actualEvent.get("host")); Assert.assertEquals("", actualEvent.get("service")); - Assert.assertEquals("xx", actualEvent.get(DruidMetrics.DATASOURCE)); + Assert.assertEquals(DataSourceUtil.getMetricName(query.getDataSources()), actualEvent.get(DruidMetrics.DATASOURCE)); Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE)); - List expectedIntervals = QueryRunnerTestHelper.fullOnInterval.getIntervals(); - List expectedStringIntervals = - expectedIntervals.stream().map(Interval::toString).collect(Collectors.toList()); - Assert.assertEquals(expectedStringIntervals, actualEvent.get(DruidMetrics.INTERVAL)); Assert.assertEquals("true", actualEvent.get("hasFilters")); - Assert.assertEquals(expectedIntervals.get(0).toDuration().toString(), actualEvent.get("duration")); + Assert.assertEquals(query.getTotalDuration().toString(), actualEvent.get("duration")); Assert.assertEquals("", actualEvent.get(DruidMetrics.ID)); // TopN-specific dimensions diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java index 9e59eb11cde3..c4e8a2768fc9 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java @@ -213,7 +213,7 @@ public void testMinTopNThreshold() throws Exception .intervals(QueryRunnerTestHelper.fullOnInterval) .aggregators(QueryRunnerTestHelper.commonAggregators); - TopNQuery query1 = builder.threshold(10).context(null).build(); + TopNQuery query1 = builder.threshold(10).context(ImmutableMap.of()).build(); MockQueryRunner mockRunner = new MockQueryRunner(runner); new TopNQueryQueryToolChest.ThresholdAdjustingQueryRunner(mockRunner, config) .run(query1, ImmutableMap.of()); diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index b15e912fd3f3..9ae1c1f07f76 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -24,6 +24,7 @@ import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Optional; +import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -58,6 +59,7 @@ import io.druid.query.BaseQuery; import io.druid.query.BySegmentResultValueClass; import io.druid.query.CacheStrategy; +import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; @@ -164,7 +166,12 @@ public Sequence run(final Query query, final Map responseC contextBuilder.put("bySegment", true); } - TimelineLookup timeline = serverView.getTimeline(query.getDataSource()); + final DataSourceWithSegmentSpec spec = query.getDistributionTarget(); + final List dataSourceNames = spec.getDataSource().getNames(); + Preconditions.checkArgument(dataSourceNames.size() > 0); + final String dataSourceName = dataSourceNames.get(0); + + TimelineLookup timeline = serverView.getTimeline(spec.getDataSource()); if (timeline == null) { return Sequences.empty(); @@ -183,7 +190,7 @@ public Sequence run(final Query query, final Map responseC List uncoveredIntervals = Lists.newArrayListWithCapacity(uncoveredIntervalsLimit); boolean uncoveredIntervalsOverflowed = false; - for (Interval interval : query.getIntervals()) { + for (Interval interval : spec.getQuerySegmentSpec().getIntervals()) { Iterable> lookup = timeline.lookup(interval); long startMillis = interval.getStartMillis(); long endMillis = interval.getEndMillis(); @@ -219,7 +226,7 @@ public Sequence run(final Query query, final Map responseC responseContext.put("uncoveredIntervalsOverflowed", uncoveredIntervalsOverflowed); } } else { - for (Interval interval : query.getIntervals()) { + for (Interval interval : spec.getQuerySegmentSpec().getIntervals()) { Iterables.addAll(serversLookup, timeline.lookup(interval)); } } @@ -336,7 +343,7 @@ public ShardSpec apply(PartitionChunk input) log.makeAlert( "No servers found for SegmentDescriptor[%s] for DataSource[%s]?! How can this be?!", segment.rhs, - query.getDataSource() + spec.getDataSource() ).emit(); } else { final DruidServer server = queryableDruidServer.getServer(); @@ -429,7 +436,10 @@ private void addSequencesFromServer(ArrayList> listOfSequences) final Sequence resultSeqToAdd; if (!server.isAssignable() || !populateCache || isBySegment) { // Direct server queryable if (!isBySegment) { - resultSeqToAdd = clientQueryable.run(query.withQuerySegmentSpec(segmentSpec), responseContext); + resultSeqToAdd = clientQueryable.run( + query.replaceQuerySegmentSpecWith(dataSourceName, segmentSpec), + responseContext + ); } else { // bySegment queries need to be de-serialized, see DirectDruidClient.run() @@ -439,7 +449,7 @@ private void addSequencesFromServer(ArrayList> listOfSequences) @SuppressWarnings("unchecked") final Sequence>> resultSequence = clientQueryable.run( - bySegmentQuery.withQuerySegmentSpec(segmentSpec), + bySegmentQuery.replaceQuerySegmentSpecWith(dataSourceName, segmentSpec), responseContext ); @@ -472,7 +482,7 @@ public Result> apply(Result>> runningSequence = clientQueryable.run( - rewrittenQuery.withQuerySegmentSpec(segmentSpec), + rewrittenQuery.replaceQuerySegmentSpecWith(dataSourceName, segmentSpec), responseContext ); resultSeqToAdd = new MergeSequence( diff --git a/server/src/main/java/io/druid/client/selector/HostSelector.java b/server/src/main/java/io/druid/client/selector/HostSelector.java index 3afafac4aa8e..bcc4fa86a1bb 100644 --- a/server/src/main/java/io/druid/client/selector/HostSelector.java +++ b/server/src/main/java/io/druid/client/selector/HostSelector.java @@ -25,9 +25,9 @@ /** */ -public interface HostSelector +public interface HostSelector { - public String getDefaultServiceName(); + String getDefaultServiceName(); - public Pair select(Query query); + Pair select(Query query); } diff --git a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java index f13752b6006c..60181cae1641 100644 --- a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java +++ b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java @@ -35,6 +35,7 @@ import io.druid.java.util.common.guava.CloseQuietly; import io.druid.java.util.common.lifecycle.LifecycleStart; import io.druid.java.util.common.lifecycle.LifecycleStop; +import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.NoopQueryRunner; import io.druid.query.Query; @@ -152,8 +153,9 @@ public FireDepartmentMetrics getMetrics(String datasource) public QueryRunner getQueryRunnerForIntervals(final Query query, Iterable intervals) { final QueryRunnerFactory> factory = conglomerate.findFactory(query); - final Map partitionChiefs = chiefs.get(Iterables.getOnlyElement(query.getDataSource() - .getNames())); + final DataSourceWithSegmentSpec spec = query.getDistributionTarget(); + final Map partitionChiefs = chiefs.get(Iterables.getOnlyElement(spec.getDataSource() + .getNames())); return partitionChiefs == null ? new NoopQueryRunner() : factory.getToolchest().mergeResults( factory.mergeRunners( @@ -177,8 +179,9 @@ public QueryRunner apply(FireChief fireChief) public QueryRunner getQueryRunnerForSegments(final Query query, final Iterable specs) { final QueryRunnerFactory> factory = conglomerate.findFactory(query); - final Map partitionChiefs = chiefs.get(Iterables.getOnlyElement(query.getDataSource() - .getNames())); + final DataSourceWithSegmentSpec spec = query.getDistributionTarget(); + final String dataSourceName = Iterables.getOnlyElement(spec.getDataSource().getNames()); + final Map partitionChiefs = chiefs.get(dataSourceName); return partitionChiefs == null ? new NoopQueryRunner() @@ -195,7 +198,9 @@ public QueryRunner apply(SegmentDescriptor spec) final FireChief retVal = partitionChiefs.get(spec.getPartitionNumber()); return retVal == null ? new NoopQueryRunner() - : retVal.getQueryRunner(query.withQuerySegmentSpec(new SpecificSegmentSpec(spec))); + : retVal.getQueryRunner( + query.replaceQuerySegmentSpecWith(dataSourceName, new SpecificSegmentSpec(spec)) + ); } } ) diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index 37ef7489831f..c9050b1ad860 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -22,6 +22,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.util.concurrent.MoreExecutors; import com.metamx.emitter.EmittingLogger; @@ -35,6 +37,8 @@ import io.druid.java.util.common.guava.FunctionalIterable; import io.druid.query.BySegmentQueryRunner; import io.druid.query.CPUTimeMetricQueryRunner; +import io.druid.query.DataSource; +import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.MetricsEmittingQueryRunner; import io.druid.query.NoopQueryRunner; import io.druid.query.Query; @@ -103,7 +107,10 @@ public SinkQuerySegmentWalker( } @Override - public QueryRunner getQueryRunnerForIntervals(final Query query, final Iterable intervals) + public QueryRunner getQueryRunnerForIntervals( + final Query query, + final Iterable intervals + ) { final Iterable specs = FunctionalIterable .create(intervals) @@ -147,14 +154,25 @@ public SegmentDescriptor apply(final PartitionChunk chunk) } @Override - public QueryRunner getQueryRunnerForSegments(final Query query, final Iterable specs) + public QueryRunner getQueryRunnerForSegments( + final Query query, + final Iterable specs + ) { // We only handle one particular dataSource. Make sure that's what we have, then ignore from here on out. - if (!(query.getDataSource() instanceof TableDataSource) - || !dataSource.equals(((TableDataSource) query.getDataSource()).getName())) { - log.makeAlert("Received query for unknown dataSource") - .addData("dataSource", query.getDataSource()) - .emit(); + int dataSourceNum = 0; + for (DataSourceWithSegmentSpec eachSource : query.getDataSources()) { + if (!(eachSource.getDataSource() instanceof TableDataSource) + || !dataSource.equals(((TableDataSource) eachSource.getDataSource()).getName())) { + log.makeAlert("Received query for unknown dataSource") + .addData("dataSource", eachSource.getDataSource()) + .emit(); + return new NoopQueryRunner<>(); + } + dataSourceNum++; + } + + if (dataSourceNum == 0) { return new NoopQueryRunner<>(); } @@ -166,6 +184,8 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final final QueryToolChest> toolChest = factory.getToolchest(); final boolean skipIncrementalSegment = query.getContextValue(CONTEXT_SKIP_INCREMENTAL_SEGMENT, false); final AtomicLong cpuTimeAccumulator = new AtomicLong(0L); + final DataSource dataSource = Iterables.getOnlyElement(query.getDataSources()).getDataSource(); + final String dataSourceName = Iterables.getOnlyElement(dataSource.getNames()); return CPUTimeMetricQueryRunner.safeBuild( toolChest.mergeResults( @@ -184,12 +204,16 @@ public QueryRunner apply(final SegmentDescriptor descriptor) descriptor.getVersion() ); if (holder == null) { - return new ReportTimelineMissingSegmentQueryRunner<>(descriptor); + return new ReportTimelineMissingSegmentQueryRunner<>( + ImmutableMap.of(dataSourceName, ImmutableList.of(descriptor)) + ); } final PartitionChunk chunk = holder.getChunk(descriptor.getPartitionNumber()); if (chunk == null) { - return new ReportTimelineMissingSegmentQueryRunner<>(descriptor); + return new ReportTimelineMissingSegmentQueryRunner<>( + ImmutableMap.of(dataSourceName, ImmutableList.of(descriptor)) + ); } final Sink theSink = chunk.getObject(); @@ -255,6 +279,7 @@ public QueryRunner apply(final FireHydrant hydrant) sinkSegmentIdentifier, cpuTimeAccumulator ), + Iterables.getOnlyElement(dataSource.getNames()), new SpecificSegmentSpec(descriptor) ); } diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index 161cc6a3905a..2debb7831ed3 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -42,10 +42,11 @@ import io.druid.concurrent.TaskThreadPriority; import io.druid.data.input.Committer; import io.druid.data.input.InputRow; -import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; import io.druid.java.util.common.concurrent.ScheduledExecutors; +import io.druid.java.util.common.granularity.Granularity; +import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactoryConglomerate; @@ -258,7 +259,8 @@ private Sink getSink(long timestamp) public QueryRunner getQueryRunner(final Query query) { // Calling getQueryRunnerForIntervals here works because there's only one segment per interval for RealtimePlumber. - return texasRanger.getQueryRunnerForIntervals(query, query.getIntervals()); + final DataSourceWithSegmentSpec spec= query.getDistributionTarget(); + return texasRanger.getQueryRunnerForIntervals(query, spec.getQuerySegmentSpec().getIntervals()); } @Override diff --git a/server/src/main/java/io/druid/server/BrokerQueryResource.java b/server/src/main/java/io/druid/server/BrokerQueryResource.java index d94966a902aa..072c6c427e89 100644 --- a/server/src/main/java/io/druid/server/BrokerQueryResource.java +++ b/server/src/main/java/io/druid/server/BrokerQueryResource.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; +import com.google.common.collect.Iterables; import com.google.inject.Inject; import com.metamx.emitter.service.ServiceEmitter; import com.sun.jersey.spi.container.ResourceFilters; @@ -28,8 +29,10 @@ import io.druid.client.TimelineServerView; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; -import io.druid.query.Query; +import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.GenericQueryMetricsFactory; +import io.druid.query.Query; +import io.druid.query.QueryContextKeys; import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChestWarehouse; import io.druid.server.http.security.StateResourceFilter; @@ -102,11 +105,16 @@ public Response getQueryTargets( final ResponseContext context = createContext(req.getContentType(), pretty != null); try { Query query = context.getObjectMapper().readValue(in, Query.class); + + final DataSourceWithSegmentSpec spec = (DataSourceWithSegmentSpec) query.getContext().computeIfAbsent( + QueryContextKeys.DIST_TARGET_SOURCE, + key -> Iterables.getFirst(query.getDataSources(), null) + ); return context.ok( ServerViewUtil.getTargetLocations( brokerServerView, - query.getDataSource(), - query.getIntervals(), + spec.getDataSource(), + spec.getQuerySegmentSpec().getIntervals(), numCandidates ) ); diff --git a/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java b/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java index 3af0014439b6..31f4b1277d4f 100644 --- a/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java +++ b/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java @@ -94,11 +94,10 @@ private QueryRunner makeRunner(Query query) ) ) .applyPreMergeDecoration() + .annotateDistributionTarget() .mergeResults() .applyPostMergeDecoration() .emitCPUTimeMetric(emitter) .postProcess(postProcessing); } - - } diff --git a/server/src/main/java/io/druid/server/QueryManager.java b/server/src/main/java/io/druid/server/QueryManager.java index 49252c8c0ad6..46bb41aceae5 100644 --- a/server/src/main/java/io/druid/server/QueryManager.java +++ b/server/src/main/java/io/druid/server/QueryManager.java @@ -24,9 +24,11 @@ import com.google.common.collect.SetMultimap; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.MoreExecutors; +import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.Query; import io.druid.query.QueryWatcher; +import java.util.ArrayList; import java.util.List; import java.util.Set; @@ -60,9 +62,13 @@ public boolean cancelQuery(String id) public void registerQuery(Query query, final ListenableFuture future) { final String id = query.getId(); - final List datasources = query.getDataSource().getNames(); + final Iterable sources = query.getDataSources(); + final List dataSourceNames = new ArrayList<>(); + for (DataSourceWithSegmentSpec eachSource : sources) { + dataSourceNames.addAll(eachSource.getDataSource().getNames()); + } queries.put(id, future); - queryDatasources.putAll(id, datasources); + queryDatasources.putAll(id, dataSourceNames); future.addListener( new Runnable() { @@ -70,7 +76,7 @@ public void registerQuery(Query query, final ListenableFuture future) public void run() { queries.remove(id, future); - for (String datasource : datasources) { + for (String datasource : dataSourceNames) { queryDatasources.remove(id, datasource); } } diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 2bd7d192cbb3..5ca85e2dbb46 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -37,6 +37,7 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.Yielder; import io.druid.java.util.common.guava.Yielders; +import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.DruidMetrics; import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.Query; @@ -202,7 +203,7 @@ public Response doPost( toolChest = warehouse.getToolChest(query); Thread.currentThread() - .setName(String.format("%s[%s_%s_%s]", currThreadName, query.getType(), query.getDataSource().getNames(), queryId)); + .setName(String.format("%s[%s_%s]", currThreadName, query.getType(), queryId)); if (log.isDebugEnabled()) { log.debug("Got query [%s]", query); } @@ -211,13 +212,16 @@ public Response doPost( // This is an experimental feature, see - https://github.com/druid-io/druid/pull/2424 AuthorizationInfo authorizationInfo = (AuthorizationInfo) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); if (authorizationInfo != null) { - for (String dataSource : query.getDataSource().getNames()) { - Access authResult = authorizationInfo.isAuthorized( - new Resource(dataSource, ResourceType.DATASOURCE), - Action.READ - ); - if (!authResult.isAllowed()) { - return Response.status(Response.Status.FORBIDDEN).header("Access-Check-Result", authResult).build(); + final Iterable sources = query.getDataSources(); + for (DataSourceWithSegmentSpec eachSource : sources) { + for (String dataSource : eachSource.getDataSource().getNames()) { + Access authResult = authorizationInfo.isAuthorized( + new Resource(dataSource, ResourceType.DATASOURCE), + Action.READ + ); + if (!authResult.isAllowed()) { + return Response.status(Response.Status.FORBIDDEN).header("Access-Check-Result", authResult).build(); + } } } } else { @@ -233,7 +237,8 @@ public Response doPost( } final Map responseContext = new MapMaker().makeMap(); - final Sequence res = query.run(texasRanger, responseContext); + final Sequence res; + res = query.run(texasRanger, responseContext); if (prevEtag != null && prevEtag.equals(responseContext.get(HDR_ETAG))) { return Response.notModified().build(); diff --git a/server/src/main/java/io/druid/server/coordination/ServerManager.java b/server/src/main/java/io/druid/server/coordination/ServerManager.java index 65d94aab0d16..0fb334d21e02 100644 --- a/server/src/main/java/io/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/io/druid/server/coordination/ServerManager.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Ordering; import com.google.inject.Inject; @@ -67,11 +68,15 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; /** */ @@ -256,13 +261,59 @@ public QueryRunner getQueryRunnerForIntervals(Query query, Iterable> toolChest = factory.getToolchest(); final AtomicLong cpuTimeAccumulator = new AtomicLong(0L); - DataSource dataSource = query.getDataSource(); + final DataSource dataSource = query.getDistributionTarget().getDataSource(); if (!(dataSource instanceof TableDataSource)) { throw new UnsupportedOperationException("data source type '" + dataSource.getClass().getName() + "' unsupported"); } - String dataSourceName = getDataSourceName(dataSource); + final DataSource nonBroadcastDataSource = query.getDistributionTarget().getDataSource(); + final String nonBroadcastDataSourceName = getDataSourceName(nonBroadcastDataSource); + final List dataSourceNames = StreamSupport.stream(query.getDataSources().spliterator(), false) + .map(spec -> getDataSourceName(spec.getDataSource())) + .collect(Collectors.toList()); + + if (dataSourceNames.stream().anyMatch(name -> !dataSources.containsKey(name))) { + return new NoopQueryRunner<>(); + } + + final Map> broadcastSegmentMap = new HashMap<>(); + final Map> broadcastSegmentDescMap = new HashMap<>(); + query.getDataSources().forEach( + spec -> { + final String sourceName = getDataSourceName(spec.getDataSource()); + final VersionedIntervalTimeline timeline = dataSources.get(sourceName); + if (!sourceName.equals(nonBroadcastDataSourceName)) { + spec.getQuerySegmentSpec().getIntervals().stream() + .flatMap(interval -> timeline.lookup(interval).stream()) + .forEach(holder -> + StreamSupport + .stream(holder.getObject().spliterator(), false) + .forEach(chunk -> { + final List descriptors = broadcastSegmentDescMap.computeIfAbsent( + sourceName, + key -> new ArrayList<>() + ); + descriptors.add( + new SegmentDescriptor( + holder.getInterval(), + holder.getVersion(), + chunk.getChunkNumber() + ) + ); + final List segments = broadcastSegmentMap.computeIfAbsent( + sourceName, + key -> new ArrayList<>() + ); + segments.add(chunk.getObject()); + } + ) + ); + } + } + ); - final VersionedIntervalTimeline timeline = dataSources.get(dataSourceName); + final VersionedIntervalTimeline timeline = dataSources.get( + nonBroadcastDataSourceName + ); if (timeline == null) { return new NoopQueryRunner(); @@ -301,15 +352,20 @@ public Iterable> apply( @Override public QueryRunner apply(PartitionChunk input) { + final SegmentDescriptor descriptor = new SegmentDescriptor( + holder.getInterval(), + holder.getVersion(), + input.getChunkNumber() + ); + final Map> segmentDescMap = new HashMap<>(broadcastSegmentDescMap); + segmentDescMap.put(nonBroadcastDataSourceName, ImmutableList.of(descriptor)); return buildAndDecorateQueryRunner( factory, toolChest, + nonBroadcastDataSourceName, input.getObject(), - new SegmentDescriptor( - holder.getInterval(), - holder.getVersion(), - input.getChunkNumber() - ), + descriptor, + segmentDescMap, cpuTimeAccumulator ); } @@ -331,7 +387,7 @@ public QueryRunner apply(PartitionChunk input) ); } - private String getDataSourceName(DataSource dataSource) + private static String getDataSourceName(DataSource dataSource) { return Iterables.getOnlyElement(dataSource.getNames()); } @@ -342,22 +398,61 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable> factory = conglomerate.findFactory(query); if (factory == null) { log.makeAlert("Unknown query type, [%s]", query.getClass()) - .addData("dataSource", query.getDataSource()) .emit(); return new NoopQueryRunner(); } final QueryToolChest> toolChest = factory.getToolchest(); - String dataSourceName = getDataSourceName(query.getDataSource()); + final DataSource nonBroadcastDataSource = query.getDistributionTarget().getDataSource(); + final String nonBroadcastDataSourceName = getDataSourceName(nonBroadcastDataSource); + final List dataSourceNames = StreamSupport.stream(query.getDataSources().spliterator(), false) + .map(spec -> getDataSourceName(spec.getDataSource())) + .collect(Collectors.toList()); - final VersionedIntervalTimeline timeline = dataSources.get( - dataSourceName + if (dataSourceNames.stream().anyMatch(name -> !dataSources.containsKey(name))) { + return new NoopQueryRunner<>(); + } + + final Map> broadcastSegmentMap = new HashMap<>(); + final Map> broadcastSegmentDescMap = new HashMap<>(); + query.getDataSources().forEach( + spec -> { + final String sourceName = getDataSourceName(spec.getDataSource()); + final VersionedIntervalTimeline timeline = dataSources.get(sourceName); + if (!sourceName.equals(nonBroadcastDataSourceName)) { + spec.getQuerySegmentSpec().getIntervals().stream() + .flatMap(interval -> timeline.lookup(interval).stream()) + .forEach(holder -> + StreamSupport + .stream(holder.getObject().spliterator(), false) + .forEach(chunk -> { + final List descriptors = broadcastSegmentDescMap.computeIfAbsent( + sourceName, + key -> new ArrayList<>() + ); + descriptors.add( + new SegmentDescriptor( + holder.getInterval(), + holder.getVersion(), + chunk.getChunkNumber() + ) + ); + final List segments = broadcastSegmentMap.computeIfAbsent( + sourceName, + key -> new ArrayList<>() + ); + segments.add(chunk.getObject()); + } + ) + ); + } + } ); - if (timeline == null) { - return new NoopQueryRunner(); - } + final VersionedIntervalTimeline timeline = dataSources.get( + nonBroadcastDataSourceName + ); final AtomicLong cpuTimeAccumulator = new AtomicLong(0L); @@ -370,23 +465,32 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable> apply(SegmentDescriptor input) { - final PartitionHolder entry = timeline.findEntry( input.getInterval(), input.getVersion() ); + final Map> segmentDescMap = new HashMap<>(broadcastSegmentDescMap); + segmentDescMap.put(nonBroadcastDataSourceName, ImmutableList.of(input)); if (entry == null) { - return Arrays.>asList(new ReportTimelineMissingSegmentQueryRunner(input)); + return Arrays.asList(new ReportTimelineMissingSegmentQueryRunner(segmentDescMap)); } final PartitionChunk chunk = entry.getChunk(input.getPartitionNumber()); if (chunk == null) { - return Arrays.>asList(new ReportTimelineMissingSegmentQueryRunner(input)); + return Arrays.asList(new ReportTimelineMissingSegmentQueryRunner(segmentDescMap)); } final ReferenceCountingSegment adapter = chunk.getObject(); return Arrays.asList( - buildAndDecorateQueryRunner(factory, toolChest, adapter, input, cpuTimeAccumulator) + buildAndDecorateQueryRunner( + factory, + toolChest, + nonBroadcastDataSourceName, + adapter, + input, + segmentDescMap, + cpuTimeAccumulator + ) ); } } @@ -404,11 +508,31 @@ public Iterable> apply(SegmentDescriptor input) ); } + private ReferenceCountingSegment getSegment(String dataSourceName, SegmentDescriptor descriptor) + { + final PartitionHolder entry = dataSources.get(dataSourceName).findEntry( + descriptor.getInterval(), descriptor.getVersion() + ); + + if (entry == null) { + return null; + } + + final PartitionChunk chunk = entry.getChunk(descriptor.getPartitionNumber()); + if (chunk == null) { + return null; + } + + return chunk.getObject(); + } + private QueryRunner buildAndDecorateQueryRunner( final QueryRunnerFactory> factory, final QueryToolChest> toolChest, + final String datasourceName, final ReferenceCountingSegment adapter, final SegmentDescriptor segmentDescriptor, + final Map> segmentDescMap, final AtomicLong cpuTimeAccumulator ) { @@ -431,7 +555,7 @@ private QueryRunner buildAndDecorateQueryRunner( new MetricsEmittingQueryRunner( emitter, toolChest, - new ReferenceCountingSegmentQueryRunner(factory, adapter, segmentDescriptor), + new ReferenceCountingSegmentQueryRunner(factory, adapter, segmentDescMap), QueryMetrics::reportSegmentTime, queryMetrics -> queryMetrics.segment(segmentId) ), @@ -442,6 +566,7 @@ private QueryRunner buildAndDecorateQueryRunner( QueryMetrics::reportSegmentAndCacheTime, queryMetrics -> queryMetrics.segment(segmentId) ).withWaitMeasuredFromNow(), + datasourceName, segmentSpec ), toolChest, diff --git a/server/src/main/java/io/druid/server/log/LoggingRequestLogger.java b/server/src/main/java/io/druid/server/log/LoggingRequestLogger.java index 08b7e97c92e2..45a1f52a41c3 100644 --- a/server/src/main/java/io/druid/server/log/LoggingRequestLogger.java +++ b/server/src/main/java/io/druid/server/log/LoggingRequestLogger.java @@ -21,8 +21,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; - import io.druid.java.util.common.logger.Logger; +import io.druid.query.Queries; import io.druid.query.Query; import io.druid.server.RequestLogLine; import org.slf4j.MDC; @@ -59,11 +59,11 @@ public void log(RequestLogLine requestLogLine) throws IOException try { final Query query = requestLogLine.getQuery(); MDC.put("queryId", query.getId()); - MDC.put("dataSource", query.getDataSource().toString()); + MDC.put("dataSource", (String) Queries.getDataSourceAndIntervalStrings(query).lhs); MDC.put("queryType", query.getType()); MDC.put("hasFilters", Boolean.toString(query.hasFilters())); MDC.put("remoteAddr", requestLogLine.getRemoteAddr()); - MDC.put("duration", query.getDuration().toString()); + MDC.put("duration", query.getTotalDuration().toString()); MDC.put("descending", Boolean.toString(query.isDescending())); if (setContextMDC) { final Iterable> entries = query.getContext() == null diff --git a/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java b/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java index 85af3764ab9b..867764bfea48 100644 --- a/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java +++ b/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java @@ -30,6 +30,9 @@ import io.druid.java.util.common.Pair; import io.druid.java.util.common.lifecycle.LifecycleStart; import io.druid.java.util.common.lifecycle.LifecycleStop; +import io.druid.query.DataSource; +import io.druid.query.DataSourceWithSegmentSpec; +import io.druid.query.Queries; import io.druid.query.Query; import io.druid.server.coordinator.rules.LoadRule; import io.druid.server.coordinator.rules.Rule; @@ -43,7 +46,7 @@ /** */ -public class TieredBrokerHostSelector implements HostSelector +public class TieredBrokerHostSelector implements HostSelector { private static EmittingLogger log = new EmittingLogger(TieredBrokerHostSelector.class); @@ -122,7 +125,8 @@ public String getDefaultServiceName() return tierConfig.getDefaultBrokerServiceName(); } - public Pair select(final Query query) + @Override + public Pair select(final Query query) { synchronized (lock) { if (!ruleManager.isStarted() || !started) { @@ -142,43 +146,47 @@ public Pair select(final Query query) if (brokerServiceName == null) { // For Union Queries tier will be selected on the rules for first dataSource. - List rules = ruleManager.getRulesWithDefault(Iterables.getFirst(query.getDataSource().getNames(), null)); - - // find the rule that can apply to the entire set of intervals - DateTime now = new DateTime(); - int lastRulePosition = -1; - LoadRule baseRule = null; - - for (Interval interval : query.getIntervals()) { - int currRulePosition = 0; - for (Rule rule : rules) { - if (rule instanceof LoadRule && currRulePosition > lastRulePosition && rule.appliesTo(interval, now)) { - lastRulePosition = currRulePosition; - baseRule = (LoadRule) rule; - break; + for (DataSourceWithSegmentSpec spec : query.getDataSources()) { + final DataSource dataSource = spec.getDataSource(); + List rules = ruleManager.getRulesWithDefault(Iterables.getFirst(dataSource.getNames(), null)); + + // find the rule that can apply to the entire set of intervals + DateTime now = new DateTime(); + int lastRulePosition = -1; + LoadRule baseRule = null; + + for (Interval interval : spec.getQuerySegmentSpec().getIntervals()) { + int currRulePosition = 0; + for (Rule rule : rules) { + if (rule instanceof LoadRule && currRulePosition > lastRulePosition && rule.appliesTo(interval, now)) { + lastRulePosition = currRulePosition; + baseRule = (LoadRule) rule; + break; + } + currRulePosition++; } - currRulePosition++; } - } - if (baseRule == null) { - return getDefaultLookup(); - } + if (baseRule == null) { + return getDefaultLookup(); + } - // in the baseRule, find the broker of highest priority - for (Map.Entry entry : tierConfig.getTierToBrokerMap().entrySet()) { - if (baseRule.getTieredReplicants().containsKey(entry.getKey())) { - brokerServiceName = entry.getValue(); - break; + // in the baseRule, find the broker of highest priority + for (Map.Entry entry : tierConfig.getTierToBrokerMap().entrySet()) { + if (baseRule.getTieredReplicants().containsKey(entry.getKey())) { + brokerServiceName = entry.getValue(); + break; + } } } } if (brokerServiceName == null) { + final Pair dataSourceAndInterval = Queries.getDataSourceAndIntervalStrings(query); log.error( "WTF?! No brokerServiceName found for datasource[%s], intervals[%s]. Using default[%s].", - query.getDataSource(), - query.getIntervals(), + dataSourceAndInterval.lhs, + dataSourceAndInterval.rhs, tierConfig.getDefaultBrokerServiceName() ); brokerServiceName = tierConfig.getDefaultBrokerServiceName(); diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java index 88948d718f63..4ad052b63f91 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java @@ -79,53 +79,55 @@ public void testUncoveredInterval() throws Exception { addToTimeline(new Interval("2015-01-04/2015-01-05"), "1"); addToTimeline(new Interval("2015-02-04/2015-02-05"), "1"); - final Druids.TimeseriesQueryBuilder builder = Druids.newTimeseriesQueryBuilder() + final Druids.TimeseriesQueryBuilder builder = Druids + .newTimeseriesQueryBuilder() .dataSource("test") .intervals("2015-01-02/2015-01-03") .granularity("day") .aggregators(Arrays.asList(new CountAggregatorFactory("rows"))) + .updateDistributionTarget() .context(ImmutableMap.of("uncoveredIntervalsLimit", 3)); Map responseContext = new HashMap<>(); client.run(builder.build(), responseContext); Assert.assertNull(responseContext.get("uncoveredIntervals")); - builder.intervals("2015-01-01/2015-01-03"); + builder.intervals("2015-01-01/2015-01-03").updateDistributionTarget(); responseContext = new HashMap<>(); client.run(builder.build(), responseContext); assertUncovered(responseContext, false, "2015-01-01/2015-01-02"); - builder.intervals("2015-01-01/2015-01-04"); + builder.intervals("2015-01-01/2015-01-04").updateDistributionTarget(); responseContext = new HashMap<>(); client.run(builder.build(), responseContext); assertUncovered(responseContext, false, "2015-01-01/2015-01-02", "2015-01-03/2015-01-04"); - builder.intervals("2015-01-02/2015-01-04"); + builder.intervals("2015-01-02/2015-01-04").updateDistributionTarget(); responseContext = new HashMap<>(); client.run(builder.build(), responseContext); assertUncovered(responseContext, false, "2015-01-03/2015-01-04"); - builder.intervals("2015-01-01/2015-01-30"); + builder.intervals("2015-01-01/2015-01-30").updateDistributionTarget(); responseContext = new HashMap<>(); client.run(builder.build(), responseContext); assertUncovered(responseContext, false, "2015-01-01/2015-01-02", "2015-01-03/2015-01-04", "2015-01-05/2015-01-30"); - builder.intervals("2015-01-02/2015-01-30"); + builder.intervals("2015-01-02/2015-01-30").updateDistributionTarget(); responseContext = new HashMap<>(); client.run(builder.build(), responseContext); assertUncovered(responseContext, false, "2015-01-03/2015-01-04", "2015-01-05/2015-01-30"); - builder.intervals("2015-01-04/2015-01-30"); + builder.intervals("2015-01-04/2015-01-30").updateDistributionTarget(); responseContext = new HashMap<>(); client.run(builder.build(), responseContext); assertUncovered(responseContext, false, "2015-01-05/2015-01-30"); - builder.intervals("2015-01-10/2015-01-30"); + builder.intervals("2015-01-10/2015-01-30").updateDistributionTarget(); responseContext = new HashMap<>(); client.run(builder.build(), responseContext); assertUncovered(responseContext, false, "2015-01-10/2015-01-30"); - builder.intervals("2015-01-01/2015-02-25"); + builder.intervals("2015-01-01/2015-02-25").updateDistributionTarget(); responseContext = new HashMap<>(); client.run(builder.build(), responseContext); assertUncovered(responseContext, true, "2015-01-01/2015-01-02", "2015-01-03/2015-01-04", "2015-01-05/2015-02-04"); diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index b81f1ee1b9a8..3819c29044c3 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -67,6 +67,7 @@ import io.druid.java.util.common.guava.nary.TrinaryFn; import io.druid.query.BySegmentResultValueClass; import io.druid.query.DataSource; +import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.MapQueryToolChestWarehouse; @@ -449,6 +450,7 @@ public void run() .granularity(GRANULARITY) .aggregators(AGGS) .postAggregators(POST_AGGS) + .updateDistributionTarget() .context(CONTEXT); QueryRunner runner = new FinalizeResultsQueryRunner( @@ -488,6 +490,7 @@ public void testTimeseriesCaching() throws Exception .granularity(GRANULARITY) .aggregators(AGGS) .postAggregators(POST_AGGS) + .updateDistributionTarget() .context(CONTEXT); QueryRunner runner = new FinalizeResultsQueryRunner( @@ -544,6 +547,7 @@ client, new TimeseriesQueryQueryToolChest( builder.intervals("2011-01-01/2011-01-10") .aggregators(RENAMED_AGGS) .postAggregators(RENAMED_POST_AGGS) + .updateDistributionTarget() .build(), context ) @@ -565,6 +569,7 @@ public void testCachingOverBulkLimitEnforcesLimit() throws Exception .aggregators(AGGS) .postAggregators(POST_AGGS) .context(CONTEXT) + .updateDistributionTarget() .build(); final Map context = new HashMap<>(); @@ -617,6 +622,7 @@ public void testTimeseriesMergingOutOfOrderPartitions() throws Exception .granularity(GRANULARITY) .aggregators(AGGS) .postAggregators(POST_AGGS) + .updateDistributionTarget() .context(CONTEXT); QueryRunner runner = new FinalizeResultsQueryRunner( @@ -663,6 +669,7 @@ client, new TimeseriesQueryQueryToolChest( builder.intervals("2011-01-05/2011-01-10") .aggregators(RENAMED_AGGS) .postAggregators(RENAMED_POST_AGGS) + .updateDistributionTarget() .build(), Maps.newHashMap() ) @@ -680,6 +687,7 @@ public void testTimeseriesCachingTimeZone() throws Exception .granularity(PT1H_TZ_GRANULARITY) .aggregators(AGGS) .postAggregators(POST_AGGS) + .updateDistributionTarget() .context(CONTEXT); QueryRunner runner = new FinalizeResultsQueryRunner( @@ -711,6 +719,7 @@ client, new TimeseriesQueryQueryToolChest( builder.intervals("2011-11-04/2011-11-08") .aggregators(RENAMED_AGGS) .postAggregators(RENAMED_POST_AGGS) + .updateDistributionTarget() .build(), context ) @@ -727,6 +736,7 @@ public void testDisableUseCache() throws Exception .granularity(GRANULARITY) .aggregators(AGGS) .postAggregators(POST_AGGS) + .updateDistributionTarget() .context(CONTEXT); QueryRunner runner = new FinalizeResultsQueryRunner( client, new TimeseriesQueryQueryToolChest( @@ -801,6 +811,7 @@ public void testTopNCaching() throws Exception .granularity(GRANULARITY) .aggregators(AGGS) .postAggregators(POST_AGGS) + .updateDistributionTarget() .context(CONTEXT); QueryRunner runner = new FinalizeResultsQueryRunner( @@ -859,6 +870,7 @@ public void testTopNCaching() throws Exception .metric("imps") .aggregators(RENAMED_AGGS) .postAggregators(DIFF_ORDER_POST_AGGS) + .updateDistributionTarget() .build(), context ) @@ -879,6 +891,7 @@ public void testTopNCachingTimeZone() throws Exception .granularity(PT1H_TZ_GRANULARITY) .aggregators(AGGS) .postAggregators(POST_AGGS) + .updateDistributionTarget() .context(CONTEXT); QueryRunner runner = new FinalizeResultsQueryRunner( @@ -913,6 +926,7 @@ client, new TopNQueryQueryToolChest( .metric("imps") .aggregators(RENAMED_AGGS) .postAggregators(DIFF_ORDER_POST_AGGS) + .updateDistributionTarget() .build(), context ) @@ -980,6 +994,7 @@ public void testTopNCachingEmptyResults() throws Exception .granularity(GRANULARITY) .aggregators(AGGS) .postAggregators(POST_AGGS) + .updateDistributionTarget() .context(CONTEXT); QueryRunner runner = new FinalizeResultsQueryRunner( @@ -1035,6 +1050,7 @@ client, new TopNQueryQueryToolChest( .metric("imps") .aggregators(RENAMED_AGGS) .postAggregators(DIFF_ORDER_POST_AGGS) + .updateDistributionTarget() .build(), context ) @@ -1054,6 +1070,7 @@ public void testTopNOnPostAggMetricCaching() .granularity(GRANULARITY) .aggregators(AGGS) .postAggregators(POST_AGGS) + .updateDistributionTarget() .context(CONTEXT); QueryRunner runner = new FinalizeResultsQueryRunner( @@ -1109,6 +1126,7 @@ client, new TopNQueryQueryToolChest( .metric("avg_imps_per_row_double") .aggregators(AGGS) .postAggregators(DIFF_ORDER_POST_AGGS) + .updateDistributionTarget() .build(), context ) @@ -1126,6 +1144,7 @@ public void testSearchCaching() throws Exception .intervals(SEG_SPEC) .dimensions(Arrays.asList(TOP_DIM)) .query("how") + .updateDistributionTarget() .context(CONTEXT); testQueryCaching( @@ -1183,6 +1202,7 @@ client, new SearchQueryQueryToolChest( ), runner.run( builder.intervals("2011-01-01/2011-01-10") + .updateDistributionTarget() .build(), context ) @@ -1192,7 +1212,8 @@ client, new SearchQueryQueryToolChest( @Test public void testSearchCachingRenamedOutput() throws Exception { - final Druids.SearchQueryBuilder builder = Druids.newSearchQueryBuilder() + final Druids.SearchQueryBuilder builder = Druids + .newSearchQueryBuilder() .dataSource(DATA_SOURCE) .filters(DIM_FILTER) .granularity(GRANULARITY) @@ -1200,6 +1221,7 @@ public void testSearchCachingRenamedOutput() throws Exception .intervals(SEG_SPEC) .dimensions(Arrays.asList(TOP_DIM)) .query("how") + .updateDistributionTarget() .context(CONTEXT); testQueryCaching( @@ -1257,7 +1279,8 @@ client, new SearchQueryQueryToolChest( ), runner.run( builder.intervals("2011-01-01/2011-01-10") - .build(), + .updateDistributionTarget() + .build(), context ) ); @@ -1303,6 +1326,7 @@ public void testSelectCaching() throws Exception .dimensions(Arrays.asList("a")) .metrics(Arrays.asList("rows")) .pagingSpec(new PagingSpec(null, 3)) + .updateDistributionTarget() .context(CONTEXT); testQueryCaching( @@ -1356,6 +1380,7 @@ public void testSelectCaching() throws Exception ), runner.run( builder.intervals("2011-01-01/2011-01-10") + .updateDistributionTarget() .build(), context ) @@ -1368,7 +1393,8 @@ public void testSelectCachingRenamedOutputName() throws Exception final Set dimensions = Sets.newHashSet("a"); final Set metrics = Sets.newHashSet("rows"); - Druids.SelectQueryBuilder builder = Druids.newSelectQueryBuilder() + Druids.SelectQueryBuilder builder = Druids + .newSelectQueryBuilder() .dataSource(DATA_SOURCE) .intervals(SEG_SPEC) .filters(DIM_FILTER) @@ -1376,6 +1402,7 @@ public void testSelectCachingRenamedOutputName() throws Exception .dimensions(Arrays.asList("a")) .metrics(Arrays.asList("rows")) .pagingSpec(new PagingSpec(null, 3)) + .updateDistributionTarget() .context(CONTEXT); testQueryCaching( @@ -1435,7 +1462,8 @@ public void testSelectCachingRenamedOutputName() throws Exception ), runner.run( builder.intervals("2011-01-01/2011-01-10") - .build(), + .updateDistributionTarget() + .build(), context ) ); @@ -1458,8 +1486,9 @@ public void testSelectCachingRenamedOutputName() throws Exception ), runner.run( builder.intervals("2011-01-01/2011-01-10") - .dimensionSpecs(Lists.newArrayList(new DefaultDimensionSpec("a", "a2"))) - .build(), + .dimensionSpecs(Lists.newArrayList(new DefaultDimensionSpec("a", "a2"))) + .updateDistributionTarget() + .build(), context ) ); @@ -1483,6 +1512,7 @@ public void testGroupByCaching() throws Exception .setDimensions(Arrays.asList(new DefaultDimensionSpec("a", "a"))) .setAggregatorSpecs(aggsWithUniques) .setPostAggregatorSpecs(POST_AGGS) + .updateDistributionTarget() .setContext(CONTEXT); final HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector(); @@ -1563,6 +1593,7 @@ public void testGroupByCaching() throws Exception ), runner.run( builder.setInterval("2011-01-05/2011-01-10") + .updateDistributionTarget() .build(), context ), @@ -1579,6 +1610,7 @@ public void testTimeBoundaryCaching() throws Exception .dataSource(CachingClusteredClientTest.DATA_SOURCE) .intervals(CachingClusteredClientTest.SEG_SPEC) .context(CachingClusteredClientTest.CONTEXT) + .updateDistributionTarget() .build(), new Interval("2011-01-01/2011-01-02"), makeTimeBoundaryResult(new DateTime("2011-01-01"), new DateTime("2011-01-01"), new DateTime("2011-01-02")), @@ -1600,6 +1632,7 @@ public void testTimeBoundaryCaching() throws Exception .intervals(CachingClusteredClientTest.SEG_SPEC) .context(CachingClusteredClientTest.CONTEXT) .bound(TimeBoundaryQuery.MAX_TIME) + .updateDistributionTarget() .build(), new Interval("2011-01-01/2011-01-02"), makeTimeBoundaryResult(new DateTime("2011-01-01"), null, new DateTime("2011-01-02")), @@ -1621,6 +1654,7 @@ public void testTimeBoundaryCaching() throws Exception .intervals(CachingClusteredClientTest.SEG_SPEC) .context(CachingClusteredClientTest.CONTEXT) .bound(TimeBoundaryQuery.MIN_TIME) + .updateDistributionTarget() .build(), new Interval("2011-01-01/2011-01-02"), makeTimeBoundaryResult(new DateTime("2011-01-01"), new DateTime("2011-01-01"), null), @@ -1672,6 +1706,7 @@ public void testTimeSeriesWithFilter() throws Exception .granularity(GRANULARITY) .aggregators(AGGS) .postAggregators(POST_AGGS) + .updateDistributionTarget() .context(CONTEXT); QueryRunner runner = new FinalizeResultsQueryRunner( @@ -1744,14 +1779,15 @@ public void testSingleDimensionPruning() throws Exception .build(); final Druids.TimeseriesQueryBuilder builder = Druids.newTimeseriesQueryBuilder() - .dataSource(DATA_SOURCE) - .filters(filter) - .granularity(GRANULARITY) - .intervals(SEG_SPEC) - .context(CONTEXT) - .intervals("2011-01-05/2011-01-10") - .aggregators(RENAMED_AGGS) - .postAggregators(RENAMED_POST_AGGS); + .dataSource(DATA_SOURCE) + .filters(filter) + .granularity(GRANULARITY) + .intervals(SEG_SPEC) + .context(CONTEXT) + .intervals("2011-01-05/2011-01-10") + .aggregators(RENAMED_AGGS) + .updateDistributionTarget() + .postAggregators(RENAMED_POST_AGGS); TimeseriesQuery query = builder.build(); Map context = new HashMap<>(); @@ -1948,6 +1984,7 @@ public Sequence answer() throws Throwable throw new ISE("Unknown query type[%s]", query.getClass()); } } + final DataSourceWithSegmentSpec spec = query.getDistributionTarget(); final Iterable> expected = new ArrayList<>(); for (int intervalNo = 0; intervalNo < i + 1; intervalNo++) { @@ -1965,7 +2002,8 @@ public void run() TestHelper.assertExpectedResults( expected, runner.run( - query.withQuerySegmentSpec( + query.replaceQuerySegmentSpecWith( + Iterables.getOnlyElement(spec.getDataSource().getNames()), new MultipleIntervalSegmentSpec( ImmutableList.of( actualQueryInterval @@ -2155,6 +2193,8 @@ public void testQueryCaching( expectedResultsRangeEnd = i + 1; } + final DataSourceWithSegmentSpec spec = query.getDistributionTarget(); + runWithMocks( new Runnable() { @@ -2189,7 +2229,8 @@ public Iterable>> apply(@Nullable Integer input) ) ), runner.run( - query.withQuerySegmentSpec( + query.replaceQuerySegmentSpecWith( + Iterables.getOnlyElement(spec.getDataSource().getNames()), new MultipleIntervalSegmentSpec( ImmutableList.of( actualQueryInterval @@ -3000,6 +3041,7 @@ public void testTimeBoundaryCachingWhenTimeIsInteger() throws Exception .dataSource(CachingClusteredClientTest.DATA_SOURCE) .intervals(CachingClusteredClientTest.SEG_SPEC) .context(CachingClusteredClientTest.CONTEXT) + .updateDistributionTarget() .build(), new Interval("1970-01-01/1970-01-02"), makeTimeBoundaryResult(new DateTime("1970-01-01"), new DateTime("1970-01-01"), new DateTime("1970-01-02")), @@ -3021,6 +3063,7 @@ public void testTimeBoundaryCachingWhenTimeIsInteger() throws Exception .intervals(CachingClusteredClientTest.SEG_SPEC) .context(CachingClusteredClientTest.CONTEXT) .bound(TimeBoundaryQuery.MAX_TIME) + .updateDistributionTarget() .build(), new Interval("1970-01-01/2011-01-02"), makeTimeBoundaryResult(new DateTime("1970-01-01"), null, new DateTime("1970-01-02")), @@ -3042,6 +3085,7 @@ public void testTimeBoundaryCachingWhenTimeIsInteger() throws Exception .intervals(CachingClusteredClientTest.SEG_SPEC) .context(CachingClusteredClientTest.CONTEXT) .bound(TimeBoundaryQuery.MIN_TIME) + .updateDistributionTarget() .build(), new Interval("1970-01-01/2011-01-02"), makeTimeBoundaryResult(new DateTime("1970-01-01"), new DateTime("1970-01-01"), null), @@ -3067,6 +3111,7 @@ public void testGroupByCachingRenamedAggs() throws Exception .setGranularity(GRANULARITY) .setDimensions(Arrays.asList(new DefaultDimensionSpec("a", "output"))) .setAggregatorSpecs(AGGS) + .updateDistributionTarget() .setContext(CONTEXT); testQueryCaching( @@ -3123,6 +3168,7 @@ public void testGroupByCachingRenamedAggs() throws Exception ), runner.run( builder.setInterval("2011-01-05/2011-01-10") + .updateDistributionTarget() .build(), context ), @@ -3146,6 +3192,7 @@ public void testGroupByCachingRenamedAggs() throws Exception builder.setInterval("2011-01-05/2011-01-10") .setDimensions(Arrays.asList(new DefaultDimensionSpec("a", "output2"))) .setAggregatorSpecs(RENAMED_AGGS) + .updateDistributionTarget() .build(), context ), @@ -3182,6 +3229,7 @@ public void testIfNoneMatch() throws Exception .dataSource(DATA_SOURCE) .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(interval))) .context(ImmutableMap.of("If-None-Match", "aVJV29CJY93rszVW/QBy0arWZo0=")) + .updateDistributionTarget() .build(); diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index 2484030ae135..09fecf1b37b5 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -40,7 +40,7 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.parsers.ParseException; -import io.druid.query.BaseQuery; +import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; @@ -428,6 +428,7 @@ public void testQueryWithInterval() throws IOException, InterruptedException ) ) .setGranularity(QueryRunnerTestHelper.dayGran) + .updateDistributionTarget() .build(); plumber.setRunners(ImmutableMap.of(query.getIntervals().get(0), runner)); plumber2.setRunners(ImmutableMap.of(query.getIntervals().get(0), runner)); @@ -486,6 +487,7 @@ public void testQueryWithSegmentSpec() throws IOException, InterruptedException ) ) .setGranularity(QueryRunnerTestHelper.dayGran) + .updateDistributionTarget() .build(); plumber.setRunners(ImmutableMap.of(query.getIntervals().get(0), runner)); plumber2.setRunners(ImmutableMap.of(query.getIntervals().get(0), runner)); @@ -494,12 +496,13 @@ public void testQueryWithSegmentSpec() throws IOException, InterruptedException factory, realtimeManager3.getQueryRunnerForSegments( query, - ImmutableList.of( + ImmutableList.of( new SegmentDescriptor( new Interval("2011-04-01T00:00:00.000Z/2011-04-03T00:00:00.000Z"), "ver", 0 - )) + ) + ) ), query ); @@ -509,12 +512,13 @@ public void testQueryWithSegmentSpec() throws IOException, InterruptedException factory, realtimeManager3.getQueryRunnerForSegments( query, - ImmutableList.of( + ImmutableList.of( new SegmentDescriptor( new Interval("2011-04-01T00:00:00.000Z/2011-04-03T00:00:00.000Z"), "ver", 1 - )) + ) + ) ), query ); @@ -596,6 +600,7 @@ public void testQueryWithMultipleSegmentSpec() throws IOException, InterruptedEx ) ) .setGranularity(QueryRunnerTestHelper.dayGran) + .updateDistributionTarget() .build(); final Map runnerMap = ImmutableMap.of( @@ -627,8 +632,7 @@ public void testQueryWithMultipleSegmentSpec() throws IOException, InterruptedEx factory, realtimeManager3.getQueryRunnerForSegments( query, - ImmutableList.of( - descriptor_26_28_0) + ImmutableList.of(descriptor_26_28_0) ), query ); @@ -638,8 +642,7 @@ public void testQueryWithMultipleSegmentSpec() throws IOException, InterruptedEx factory, realtimeManager3.getQueryRunnerForSegments( query, - ImmutableList.of( - descriptor_28_29_0) + ImmutableList.of(descriptor_28_29_0) ), query ); @@ -649,8 +652,7 @@ public void testQueryWithMultipleSegmentSpec() throws IOException, InterruptedEx factory, realtimeManager3.getQueryRunnerForSegments( query, - ImmutableList.of( - descriptor_26_28_1) + ImmutableList.of(descriptor_26_28_1) ), query ); @@ -660,8 +662,7 @@ public void testQueryWithMultipleSegmentSpec() throws IOException, InterruptedEx factory, realtimeManager3.getQueryRunnerForSegments( query, - ImmutableList.of( - descriptor_28_29_1) + ImmutableList.of(descriptor_28_29_1) ), query ); @@ -969,15 +970,15 @@ public QueryRunner getQueryRunner(final Query query) throw new UnsupportedOperationException(); } - final BaseQuery baseQuery = (BaseQuery) query; + final DataSourceWithSegmentSpec spec = query.getDistributionTarget(); - if (baseQuery.getQuerySegmentSpec() instanceof MultipleIntervalSegmentSpec) { + if (spec.getQuerySegmentSpec() instanceof MultipleIntervalSegmentSpec) { return factory.getToolchest() .mergeResults( factory.mergeRunners( MoreExecutors.sameThreadExecutor(), Iterables.transform( - baseQuery.getIntervals(), + spec.getQuerySegmentSpec().getIntervals(), new Function>() { @Override @@ -991,13 +992,14 @@ public QueryRunner apply(Interval input) ); } - Assert.assertEquals(1, query.getIntervals().size()); + Assert.assertEquals(1, spec.getQuerySegmentSpec().getIntervals().size()); final SegmentDescriptor descriptor = - ((SpecificSegmentSpec) ((BaseQuery) query).getQuerySegmentSpec()).getDescriptor(); + ((SpecificSegmentSpec) spec.getQuerySegmentSpec()).getDescriptor(); return new SpecificSegmentQueryRunner( runners.get(descriptor.getInterval()), + Iterables.getOnlyElement(spec.getDataSource().getNames()), new SpecificSegmentSpec(descriptor) ); } diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java index 8fb6619dd831..d8347aec4752 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java @@ -266,6 +266,7 @@ public void testQueryByIntervals() throws Exception ) ) .granularity(Granularities.DAY) + .updateDistributionTarget() .build(); final List> results1 = Lists.newArrayList(); @@ -292,6 +293,7 @@ public void testQueryByIntervals() throws Exception ) ) .granularity(Granularities.DAY) + .updateDistributionTarget() .build(); final List> results2 = Lists.newArrayList(); @@ -322,6 +324,7 @@ public void testQueryByIntervals() throws Exception ) ) .granularity(Granularities.DAY) + .updateDistributionTarget() .build(); final List> results3 = Lists.newArrayList(); @@ -356,6 +359,7 @@ public void testQueryByIntervals() throws Exception ) ) .granularity(Granularities.DAY) + .updateDistributionTarget() .build(); final List> results4 = Lists.newArrayList(); @@ -412,6 +416,7 @@ public void testQueryBySegments() throws Exception ) ) ) + .updateDistributionTarget() .build(); final List> results1 = Lists.newArrayList(); @@ -448,6 +453,7 @@ public void testQueryBySegments() throws Exception ) ) ) + .updateDistributionTarget() .build(); final List> results2 = Lists.newArrayList(); @@ -489,6 +495,7 @@ public void testQueryBySegments() throws Exception ) ) ) + .updateDistributionTarget() .build(); final List> results3 = Lists.newArrayList(); diff --git a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java index 7656e7583af9..b960363655f0 100644 --- a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java +++ b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java @@ -426,7 +426,7 @@ private Future assertQueryable( .query("wow") .build(); final QueryRunner> runner = serverManager.getQueryRunnerForIntervals( - query, + query.distributeBy(query.getDataSourceWithSegmentSpec()), intervals ); return serverManagerExec.submit( @@ -592,6 +592,12 @@ public TypeReference getResultTypeReference() { }; } + + @Override + public QueryRunner annotateDistributionTarget(QueryRunner runner) + { + return runner; + } } private static class SegmentForTesting extends AbstractSegment diff --git a/server/src/test/java/io/druid/server/log/LoggingRequestLoggerTest.java b/server/src/test/java/io/druid/server/log/LoggingRequestLoggerTest.java index 18e330fcb02c..410ead41c0c8 100644 --- a/server/src/test/java/io/druid/server/log/LoggingRequestLoggerTest.java +++ b/server/src/test/java/io/druid/server/log/LoggingRequestLoggerTest.java @@ -25,12 +25,12 @@ import com.google.common.base.Charsets; import com.google.common.collect.ImmutableMap; import io.druid.jackson.DefaultObjectMapper; -import io.druid.query.BaseQuery; import io.druid.query.DataSource; import io.druid.query.LegacyDataSource; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QuerySegmentWalker; +import io.druid.query.SingleSourceBaseQuery; import io.druid.query.filter.DimFilter; import io.druid.query.spec.QuerySegmentSpec; import io.druid.server.QueryStats; @@ -132,7 +132,7 @@ public void testLoggingMDC() throws Exception final LoggingRequestLogger requestLogger = new LoggingRequestLogger(new DefaultObjectMapper(), true, false); requestLogger.log(logLine); final Map map = readContextMap(baos.toByteArray()); - Assert.assertEquals("datasource", map.get("dataSource")); + Assert.assertEquals("[datasource]", map.get("dataSource")); Assert.assertEquals("PT86400S", map.get("duration")); Assert.assertEquals("false", map.get("hasFilters")); Assert.assertEquals("fake", map.get("queryType")); @@ -147,7 +147,7 @@ public void testLoggingMDCContext() throws Exception final LoggingRequestLogger requestLogger = new LoggingRequestLogger(new DefaultObjectMapper(), true, true); requestLogger.log(logLine); final Map map = readContextMap(baos.toByteArray()); - Assert.assertEquals("datasource", map.get("dataSource")); + Assert.assertEquals("[datasource]", map.get("dataSource")); Assert.assertEquals("PT86400S", map.get("duration")); Assert.assertEquals("false", map.get("hasFilters")); Assert.assertEquals("fake", map.get("queryType")); @@ -179,7 +179,7 @@ private static Map readContextMap(byte[] bytes) throws Exception } @JsonTypeName("fake") -class FakeQuery extends BaseQuery +class FakeQuery extends SingleSourceBaseQuery { public FakeQuery(DataSource dataSource, QuerySegmentSpec querySegmentSpec, boolean descending, Map context) { diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java b/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java index 5cbf91d4b182..20fdddb813dd 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java @@ -34,6 +34,7 @@ import io.druid.query.QueryDataSource; import io.druid.query.QuerySegmentWalker; import io.druid.query.Result; +import io.druid.query.SingleSourceBaseQuery; import io.druid.query.dimension.DimensionSpec; import io.druid.query.groupby.GroupByQuery; import io.druid.query.select.EventHolder; @@ -150,6 +151,9 @@ private Sequence executeSelect( final SelectQuery baseQuery ) { + final SelectQuery running = (SelectQuery) baseQuery.distributeBy( + SingleSourceBaseQuery.getLeafDataSourceWithSegmentSpec(baseQuery) + ); Preconditions.checkState(queryBuilder.getGrouping() == null, "grouping must be null"); final List fieldList = queryBuilder.getRowType().getFieldList(); @@ -178,7 +182,7 @@ public boolean hasNext() @Override public Sequence next() { - final SelectQuery queryWithPagination = baseQuery.withPagingSpec( + final SelectQuery queryWithPagination = running.withPagingSpec( new PagingSpec( pagingIdentifiers.get(), plannerContext.getPlannerConfig().getSelectThreshold(), @@ -257,14 +261,17 @@ private Sequence executeTimeseries( final TimeseriesQuery query ) { + final TimeseriesQuery running = (TimeseriesQuery) query.distributeBy( + SingleSourceBaseQuery.getLeafDataSourceWithSegmentSpec(query) + ); final List fieldList = queryBuilder.getRowType().getFieldList(); final List dimensions = queryBuilder.getGrouping().getDimensions(); final String timeOutputName = dimensions.isEmpty() ? null : Iterables.getOnlyElement(dimensions).getOutputName(); - Hook.QUERY_PLAN.run(query); + Hook.QUERY_PLAN.run(running); return Sequences.map( - query.run(walker, Maps.newHashMap()), + running.run(walker, Maps.newHashMap()), new Function, Object[]>() { @Override @@ -293,13 +300,16 @@ private Sequence executeTopN( final TopNQuery query ) { + final TopNQuery running = (TopNQuery) query.distributeBy( + SingleSourceBaseQuery.getLeafDataSourceWithSegmentSpec(query) + ); final List fieldList = queryBuilder.getRowType().getFieldList(); - Hook.QUERY_PLAN.run(query); + Hook.QUERY_PLAN.run(running); return Sequences.concat( Sequences.map( - query.run(walker, Maps.newHashMap()), + running.run(walker, Maps.newHashMap()), new Function, Sequence>() { @Override @@ -331,11 +341,14 @@ private Sequence executeGroupBy( ) { final List fieldList = queryBuilder.getRowType().getFieldList(); + final GroupByQuery running = (GroupByQuery) query.distributeBy( + SingleSourceBaseQuery.getLeafDataSourceWithSegmentSpec(query) + ); - Hook.QUERY_PLAN.run(query); + Hook.QUERY_PLAN.run(running); return Sequences.map( - query.run(walker, Maps.newHashMap()), + running.run(walker, Maps.newHashMap()), new Function() { @Override diff --git a/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java b/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java index 38f82c8213c7..9bf86e5bfb46 100644 --- a/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java +++ b/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java @@ -305,7 +305,9 @@ private DruidTable computeTable(final String dataSource) true ); - final Sequence sequence = segmentMetadataQuery.run(walker, Maps.newHashMap()); + final Sequence sequence = segmentMetadataQuery + .distributeBy(segmentMetadataQuery.getDataSourceWithSegmentSpec()) + .run(walker, Maps.newHashMap()); final List results = Sequences.toList(sequence, Lists.newArrayList()); if (results.isEmpty()) { return null; diff --git a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java index 12b7ee2097ca..3996a5887600 100644 --- a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java @@ -343,6 +343,7 @@ public void testSelectStar() throws Exception .intervals(QSS(Filtration.eternity())) .granularity(Granularities.ALL) .pagingSpec(FIRST_PAGING_SPEC) + .updateDistributionTarget() .context(QUERY_CONTEXT_DEFAULT) .build(), Druids.newSelectQueryBuilder() @@ -357,6 +358,7 @@ public void testSelectStar() throws Exception ) ) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -382,6 +384,7 @@ public void testUnqualifiedTableName() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -416,6 +419,7 @@ public void testSelectStarWithLimit() throws Exception .granularity(Granularities.ALL) .pagingSpec(FIRST_PAGING_SPEC) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -438,6 +442,7 @@ public void testSelectStarWithLimitDescending() throws Exception .descending(true) .pagingSpec(FIRST_PAGING_SPEC) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -464,6 +469,7 @@ public void testSelectSingleColumnTwice() throws Exception .descending(false) .pagingSpec(FIRST_PAGING_SPEC) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -487,6 +493,7 @@ public void testSelectSingleColumnWithLimitDescending() throws Exception .descending(true) .pagingSpec(FIRST_PAGING_SPEC) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -521,6 +528,7 @@ public void testGroupBySingleColumnDescendingNoTopN() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -551,6 +559,7 @@ public void testSelfJoinWithFallback() throws Exception .granularity(Granularities.ALL) .pagingSpec(FIRST_PAGING_SPEC) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build(), Druids.newSelectQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) @@ -563,6 +572,7 @@ public void testSelfJoinWithFallback() throws Exception true ) ) + .updateDistributionTarget() .context(QUERY_CONTEXT_DEFAULT) .build(), Druids.newSelectQueryBuilder() @@ -571,6 +581,7 @@ public void testSelfJoinWithFallback() throws Exception .granularity(Granularities.ALL) .filters(NOT(SELECTOR("dim1", "", null))) .pagingSpec(FIRST_PAGING_SPEC) + .updateDistributionTarget() .context(QUERY_CONTEXT_DEFAULT) .build(), Druids.newSelectQueryBuilder() @@ -586,6 +597,7 @@ public void testSelfJoinWithFallback() throws Exception ) ) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -630,6 +642,7 @@ public void testGroupByLong() throws Exception .setDimensions(DIMS(new DefaultDimensionSpec("cnt", "d0", ValueType.LONG))) .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -651,6 +664,7 @@ public void testGroupByFloat() throws Exception .setDimensions(DIMS(new DefaultDimensionSpec("m1", "d0", ValueType.FLOAT))) .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -677,6 +691,7 @@ public void testFilterOnFloat() throws Exception .aggregators(AGGS(new CountAggregatorFactory("a0"))) .filters(SELECTOR("m1", "1.0", null)) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -712,6 +727,7 @@ public void testHavingOnFloat() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -804,6 +820,7 @@ public void testSelectStarWithDimFilter() throws Exception ) ) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build(), Druids.newSelectQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) @@ -823,6 +840,7 @@ public void testSelectStarWithDimFilter() throws Exception ) ) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -851,6 +869,7 @@ public void testSelectStarWithDimFilterAndPaging() throws Exception SELECTOR("dim2", "a", null) ) ) + .updateDistributionTarget() .context(QUERY_CONTEXT_DEFAULT) .build(), Druids.newSelectQueryBuilder() @@ -871,6 +890,7 @@ public void testSelectStarWithDimFilterAndPaging() throws Exception ) ) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build(), Druids.newSelectQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) @@ -890,6 +910,7 @@ public void testSelectStarWithDimFilterAndPaging() throws Exception ) ) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -941,6 +962,7 @@ public void testGroupByWithFilterMatchingNothing() throws Exception new LongMaxAggregatorFactory("a1", "cnt") )) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of() @@ -963,6 +985,7 @@ public void testGroupByWithFilterMatchingNothingWithGroupByLiteral() throws Exce new LongMaxAggregatorFactory("a1", "cnt") )) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of() @@ -981,6 +1004,7 @@ public void testCountStar() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1005,6 +1029,7 @@ public void testCountStarOnView() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1050,6 +1075,7 @@ public void testCountStarWithLikeFilter() throws Exception ) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1076,6 +1102,7 @@ public void testCountStarWithLongColumnFilters() throws Exception ) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1099,6 +1126,7 @@ public void testCountStarWithLongColumnFiltersOnFloatLiterals() throws Exception ) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of() @@ -1116,6 +1144,7 @@ public void testCountStarWithLongColumnFiltersOnFloatLiterals() throws Exception ) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1135,6 +1164,7 @@ public void testCountStarWithLongColumnFiltersOnFloatLiterals() throws Exception ) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of() @@ -1152,6 +1182,7 @@ public void testCountStarWithLongColumnFiltersOnFloatLiterals() throws Exception ) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1173,6 +1204,7 @@ public void testCountStarWithLongColumnFiltersOnTwoPoints() throws Exception .filters(IN("cnt", ImmutableList.of("1", "2"), null)) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1203,6 +1235,7 @@ public void testFilterOnStringAsNumber() throws Exception ) ) ) + .updateDistributionTarget() .setContext(QUERY_CONTEXT_DEFAULT) .build() ), @@ -1260,6 +1293,7 @@ public void testSimpleAggregations() throws Exception ) ) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1300,6 +1334,7 @@ public void testGroupByWithSortOnPostAggregationDefault() throws Exception ) .threshold(3) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1355,6 +1390,7 @@ public void testGroupByWithSortOnPostAggregationNoTopNConfig() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1409,6 +1445,7 @@ public void testGroupByWithSortOnPostAggregationNoTopNContext() throws Exception 3 ) ) + .updateDistributionTarget() .setContext(QUERY_CONTEXT_NO_TOPN) .build() ), @@ -1477,6 +1514,7 @@ public void testFilteredAggregations() throws Exception ) )) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1508,6 +1546,7 @@ public void testExpressionAggregations() throws Exception )) )) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1534,6 +1573,7 @@ public void testInFilter() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1556,6 +1596,7 @@ public void testCountStarWithDegenerateFilter() throws Exception .filters(SELECTOR("dim2", "a", null)) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1577,6 +1618,7 @@ public void testCountStarWithNotOfDegenerateFilter() throws Exception .filters(null) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of() @@ -1596,6 +1638,7 @@ public void testCountStarWithBoundFilterSimplifyOr() throws Exception .filters(BOUND("dim1", "a", "b", false, true, null, StringComparators.LEXICOGRAPHIC)) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1617,6 +1660,7 @@ public void testCountStarWithBoundFilterSimplifyAnd() throws Exception .filters(SELECTOR("dim1", "abc", null)) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1638,6 +1682,7 @@ public void testCountStarWithFilterOnCastedString() throws Exception .filters(NUMERIC_SELECTOR("dim1", "2", null)) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1659,6 +1704,7 @@ public void testCountStarWithTimeFilter() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1679,6 +1725,7 @@ public void testCountStarWithSinglePointInTime() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1705,6 +1752,7 @@ public void testCountStarWithTwoPointsInTime() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1745,6 +1793,7 @@ public void testCountStarWithComplexDisjointTimeFilter() throws Exception ) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1786,6 +1835,7 @@ public void testCountStarWithNotOfComplexDisjointTimeFilter() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1816,6 +1866,7 @@ public void testCountStarWithNotTimeFilter() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1839,6 +1890,7 @@ public void testCountStarWithTimeAndDimFilter() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1875,6 +1927,7 @@ public void testCountStarWithTimeOrDimFilter() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1908,6 +1961,7 @@ public void testCountStarWithTimeFilterOnLongColumn() throws Exception ) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1941,6 +1995,7 @@ public void testSelectDistinctWithCascadeExtractionFilter() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1970,6 +2025,7 @@ public void testSelectDistinctWithStrlenFilter() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1995,6 +2051,7 @@ public void testSelectDistinctWithLimit() throws Exception .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC)) .threshold(10) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2019,6 +2076,7 @@ public void testSelectDistinctWithSortAsOuterQuery() throws Exception .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC)) .threshold(10) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2043,6 +2101,7 @@ public void testSelectDistinctWithSortAsOuterQuery2() throws Exception .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC)) .threshold(5) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2088,6 +2147,7 @@ public void testCountDistinct() throws Exception ) ) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2123,6 +2183,7 @@ public void testExactCountDistinct() throws Exception new CountAggregatorFactory("a0") )) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2155,6 +2216,7 @@ public void testApproxCountDistinctWhenHllDisabled() throws Exception ) ) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2196,6 +2258,7 @@ public void testExactCountDistinctWithGroupingAndOtherAggregators() throws Excep new CountAggregatorFactory("a1") )) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2243,6 +2306,7 @@ public void testApproxCountDistinct() throws Exception ) ) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2297,6 +2361,7 @@ public void testDoubleNestedGroupBy() throws Exception new CountAggregatorFactory("a1") )) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2334,6 +2399,7 @@ public void testExactCountDistinctUsingSubquery() throws Exception new LongSumAggregatorFactory("a0", "a0"), new CountAggregatorFactory("a1") )) + .updateDistributionTarget() .setContext(QUERY_CONTEXT_DEFAULT) .build() ), @@ -2370,6 +2436,7 @@ public void testTopNFilterJoin() throws Exception .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt"))) .metric(new NumericTopNMetricSpec("a0")) .threshold(2) + .updateDistributionTarget() .context(QUERY_CONTEXT_DEFAULT) .build(), GroupByQuery.builder() @@ -2392,6 +2459,7 @@ public void testTopNFilterJoin() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2442,6 +2510,7 @@ public void testRemovableLeftJoin() throws Exception Integer.MAX_VALUE ) ) + .updateDistributionTarget() .setContext(QUERY_CONTEXT_DEFAULT) .build() ), @@ -2480,6 +2549,7 @@ public void testExactCountDistinctOfSemiJoinResult() throws Exception new SubstringDimExtractionFn(0, 1) ))) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build(), GroupByQuery.builder() .setDataSource( @@ -2504,6 +2574,7 @@ public void testExactCountDistinctOfSemiJoinResult() throws Exception new CountAggregatorFactory("a0") )) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), @@ -2544,6 +2615,7 @@ public void testExactCountDistinctUsingSubqueryWithWherePushDown() throws Except new CountAggregatorFactory("a1") )) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2584,6 +2656,7 @@ public void testExactCountDistinctUsingSubqueryWithWhereToOuterFilter() throws E new CountAggregatorFactory("a1") )) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2660,6 +2733,7 @@ public void testCompareExactAndApproximateCountDistinctUsingSubquery() throws Ex ) ) ) + .updateDistributionTarget() .setContext(QUERY_CONTEXT_DEFAULT) .build() ), @@ -2702,6 +2776,7 @@ public void testHistogramUsingSubquery() throws Exception new CountAggregatorFactory("a0") )) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2752,6 +2827,7 @@ public void testHistogramUsingSubqueryWithSort() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2811,6 +2887,7 @@ public void testCountDistinctArithmetic() throws Exception )) )) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2846,6 +2923,7 @@ public void testCountDistinctOfSubstring() throws Exception ) ) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2879,6 +2957,7 @@ public void testRegexpExtract() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2921,6 +3000,7 @@ public void testGroupBySortPushDown() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2968,6 +3048,7 @@ public void testGroupByLimitPushDownWithHavingOnLong() throws Exception ) .setHavingSpec(new DimFilterHavingSpec(NUMERIC_SELECTOR("a0", "1", null))) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2994,6 +3075,7 @@ public void testFilterOnTimeFloor() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3015,6 +3097,7 @@ public void testFilterOnCurrentTimestamp() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3050,6 +3133,7 @@ public void testFilterOnCurrentTimestampLosAngeles() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_LOS_ANGELES) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3070,6 +3154,7 @@ public void testFilterOnCurrentTimestampOnView() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3094,6 +3179,7 @@ public void testFilterOnCurrentTimestampLosAngelesOnView() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_LOS_ANGELES) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3119,6 +3205,7 @@ public void testFilterOnNotTimeFloor() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3141,6 +3228,7 @@ public void testFilterOnTimeFloorComparison() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3163,6 +3251,7 @@ public void testFilterOnTimeFloorComparisonMisaligned() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3186,6 +3275,7 @@ public void testFilterOnTimeExtract() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3230,6 +3320,7 @@ public void testFilterOnTimeFloorMisaligned() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of() @@ -3252,6 +3343,7 @@ public void testGroupByFloor() throws Exception )) .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3297,6 +3389,7 @@ public void testGroupByFloorWithOrderBy() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3360,6 +3453,7 @@ public void testGroupByFloorTimeAndOneOtherDimensionWithOrderBy() throws Excepti ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3398,6 +3492,7 @@ public void testGroupByStringLength() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3453,6 +3548,7 @@ public void testFilterAndGroupByLookup() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3489,6 +3585,7 @@ public void testCountDistinctOfLookup() throws Exception ) )) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3514,6 +3611,7 @@ public void testTimeseries() throws Exception .granularity(Granularities.MONTH) .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3541,6 +3639,7 @@ public void testTimeseriesLosAngeles() throws Exception .granularity(new PeriodGranularity(Period.months(1), null, DateTimeZone.forID(LOS_ANGELES))) .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt"))) .context(TIMESERIES_CONTEXT_LOS_ANGELES) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3572,6 +3671,7 @@ public void testTimeseriesDontSkipEmptyBuckets() throws Exception .granularity(new PeriodGranularity(Period.hours(1), null, DateTimeZone.UTC)) .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt"))) .context(QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS) + .updateDistributionTarget() .build() ), ImmutableList.builder() @@ -3620,6 +3720,7 @@ public void testTimeseriesUsingCastAsDate() throws Exception .granularity(new PeriodGranularity(Period.days(1), null, DateTimeZone.UTC)) .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3651,6 +3752,7 @@ public void testTimeseriesDescending() throws Exception .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt"))) .descending(true) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3704,6 +3806,7 @@ public void testGroupByExtractYear() throws Exception Integer.MAX_VALUE ) ) + .updateDistributionTarget() .setContext(QUERY_CONTEXT_DEFAULT) .build() ), @@ -3745,6 +3848,7 @@ public void testGroupByExtractFloorTime() throws Exception ) .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3786,6 +3890,7 @@ public void testGroupByExtractFloorTimeLosAngeles() throws Exception ) .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_LOS_ANGELES) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3838,6 +3943,7 @@ public void testTimeseriesWithLimitNoTopN() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3875,6 +3981,7 @@ public void testTimeseriesWithLimit() throws Exception .metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC)) .threshold(1) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3922,6 +4029,7 @@ public void testGroupByTimeAndOtherDimension() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3951,6 +4059,7 @@ public void testUsingSubqueryAsFilter() throws Exception .setDimFilter(NOT(SELECTOR("dim1", "", null))) .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0"))) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build(), GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE1) @@ -3976,6 +4085,7 @@ public void testUsingSubqueryAsFilter() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -4023,6 +4133,7 @@ public void testUsingSubqueryAsFilterOnTwoColumns() throws Exception .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0"))) .setHavingSpec(new DimFilterHavingSpec(NUMERIC_SELECTOR("a0", "1", null))) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build(), Druids.newSelectQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) @@ -4036,6 +4147,7 @@ public void testUsingSubqueryAsFilterOnTwoColumns() throws Exception .filters(AND(SELECTOR("dim1", "def", null), SELECTOR("dim2", "abc", null))) .pagingSpec(FIRST_PAGING_SPEC) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build(), Druids.newSelectQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) @@ -4055,6 +4167,7 @@ public void testUsingSubqueryAsFilterOnTwoColumns() throws Exception ) ) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -4080,6 +4193,7 @@ public void testUsingSubqueryWithExtractionFns() throws Exception DIMS(new ExtractionDimensionSpec("dim1", "d0", new SubstringDimExtractionFn(0, 1))) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build(), GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE1) @@ -4095,6 +4209,7 @@ public void testUsingSubqueryWithExtractionFns() throws Exception .setDimensions(DIMS(new DefaultDimensionSpec("dim2", "d0"))) .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -4132,6 +4247,7 @@ public void testUnicodeFilterAndGroupBy() throws Exception )) .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( diff --git a/sql/src/test/java/io/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java b/sql/src/test/java/io/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java index c9b88af530a2..5b35ad02a6e1 100644 --- a/sql/src/test/java/io/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java +++ b/sql/src/test/java/io/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java @@ -29,6 +29,7 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.FunctionalIterable; import io.druid.java.util.common.guava.Sequence; +import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.NoopQueryRunner; import io.druid.query.Query; @@ -199,11 +200,12 @@ public void close() throws IOException private VersionedIntervalTimeline getTimelineForTableDataSource(Query query) { - if (query.getDataSource() instanceof TableDataSource) { - return timelines.get(((TableDataSource) query.getDataSource()).getName()); + final DataSourceWithSegmentSpec spec = query.getDistributionTarget(); + if (spec.getDataSource() instanceof TableDataSource) { + return timelines.get(((TableDataSource) spec.getDataSource()).getName()); } else { throw new UnsupportedOperationException( - String.format("DataSource type[%s] unsupported", query.getDataSource().getClass().getName()) + String.format("DataSource type[%s] unsupported", spec.getDataSource().getClass().getName()) ); } } @@ -219,6 +221,7 @@ private QueryRunner makeBaseRunner( if (timeline == null) { return new NoopQueryRunner<>(); } + final DataSourceWithSegmentSpec spec = query.getDistributionTarget(); return new FinalizeResultsQueryRunner<>( toolChest.mergeResults( @@ -246,6 +249,7 @@ public QueryRunner apply(PartitionChunk chunk) { return new SpecificSegmentQueryRunner( factory.createRunner(chunk.getObject()), + Iterables.getOnlyElement(spec.getDataSource().getNames()), new SpecificSegmentSpec(descriptor) ); } From 293fc0954706fa67080c9191ae22512a7ba4ccec Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 27 Mar 2017 11:50:58 +0900 Subject: [PATCH 02/15] Fix test failure --- .../src/main/java/io/druid/query/DataSourceUtil.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/processing/src/main/java/io/druid/query/DataSourceUtil.java b/processing/src/main/java/io/druid/query/DataSourceUtil.java index 43b4310257a5..7609f858cd7c 100644 --- a/processing/src/main/java/io/druid/query/DataSourceUtil.java +++ b/processing/src/main/java/io/druid/query/DataSourceUtil.java @@ -41,14 +41,14 @@ public static String getMetricName(Iterable dataSourc .collect(Collectors.joining(",", "[", "]")); } - private static final StringJoiner JOINER = new StringJoiner(",", "[", "]"); private static String getMetricName(DataSourceWithSegmentSpec spec) { - JOINER.add(getMetricName(spec.getDataSource())).add("="); - JOINER.add(spec.getQuerySegmentSpec().getIntervals().stream() + final StringJoiner joiner = new StringJoiner("=", "[", "]"); + joiner.add(getMetricName(spec.getDataSource())); + joiner.add(spec.getQuerySegmentSpec().getIntervals().stream() .map(Interval::toString) .collect(Collectors.joining(",", "[", "]")) ); - return JOINER.toString(); + return joiner.toString(); } } From 6ef46fe06535d345eccbc2f48673e0e3a8ac5db2 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 13 Apr 2017 18:48:52 +0900 Subject: [PATCH 03/15] Address comments --- .../java/io/druid/query/scan/ScanQuery.java | 2 +- .../main/java/io/druid/query/BaseQuery.java | 2 +- .../main/java/io/druid/query/DataSource.java | 7 +- .../java/io/druid/query/DataSourceUtil.java | 22 +---- .../query/DataSourceWithSegmentSpec.java | 17 +++- .../io/druid/query/DefaultQueryMetrics.java | 7 +- .../src/main/java/io/druid/query/Druids.java | 8 +- .../query/IntervalChunkingQueryRunner.java | 2 +- .../src/main/java/io/druid/query/Queries.java | 18 ++-- .../src/main/java/io/druid/query/Query.java | 20 +++-- .../java/io/druid/query/QueryContextKeys.java | 2 +- .../java/io/druid/query/QueryDataSource.java | 8 +- .../java/io/druid/query/RetryQueryRunner.java | 2 +- .../io/druid/query/SingleSourceBaseQuery.java | 8 +- .../java/io/druid/query/TimewarpOperator.java | 2 +- .../io/druid/query/groupby/GroupByQuery.java | 2 +- .../java/io/druid/query/join/DataInput.java | 2 +- .../java/io/druid/query/join/JoinQuery.java | 10 +-- .../spec/SpecificSegmentQueryRunner.java | 11 ++- .../io/druid/query/topn/TopNQueryBuilder.java | 2 +- .../io/druid/query/DataSourceUtilTest.java | 85 +++++++++++++++++++ .../io/druid/query/QueryRunnerTestHelper.java | 13 ++- .../io/druid/query/TimewarpOperatorTest.java | 2 +- .../druid/client/CachingClusteredClient.java | 6 +- .../segment/realtime/RealtimeManager.java | 2 +- .../io/druid/server/BrokerQueryResource.java | 5 +- .../java/io/druid/server/QueryManager.java | 2 +- .../java/io/druid/server/QueryResource.java | 5 +- .../server/coordination/ServerManager.java | 12 +-- .../server/log/LoggingRequestLogger.java | 4 +- .../client/CachingClusteredClientTest.java | 4 +- 31 files changed, 191 insertions(+), 103 deletions(-) create mode 100644 processing/src/test/java/io/druid/query/DataSourceUtilTest.java diff --git a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQuery.java b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQuery.java index fdbd01e4a0c1..3686e505f4f1 100644 --- a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQuery.java +++ b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQuery.java @@ -319,7 +319,7 @@ public ScanQueryBuilder updateDistributionTarget() context = new HashMap<>(); } context.put( - QueryContextKeys.DIST_TARGET_SOURCE, + QueryContextKeys.DISTRIBUTION_TARGET_SOURCE, new DataSourceWithSegmentSpec( SingleSourceBaseQuery.getLeafDataSource(dataSource), querySegmentSpec diff --git a/processing/src/main/java/io/druid/query/BaseQuery.java b/processing/src/main/java/io/druid/query/BaseQuery.java index 348f4090465b..fda814096fd4 100644 --- a/processing/src/main/java/io/druid/query/BaseQuery.java +++ b/processing/src/main/java/io/druid/query/BaseQuery.java @@ -128,7 +128,7 @@ public Sequence run(QueryRunner runner, Map context) return runner.run(this, context); } - public static Duration initDuration(QuerySegmentSpec querySegmentSpec) + public static Duration getTotalDuration(QuerySegmentSpec querySegmentSpec) { Duration totalDuration = new Duration(0); for (Interval interval : querySegmentSpec.getIntervals()) { diff --git a/processing/src/main/java/io/druid/query/DataSource.java b/processing/src/main/java/io/druid/query/DataSource.java index 70e5d77083e2..c890c594c488 100644 --- a/processing/src/main/java/io/druid/query/DataSource.java +++ b/processing/src/main/java/io/druid/query/DataSource.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.google.common.collect.Iterables; +import com.google.common.base.Preconditions; import java.util.List; @@ -39,12 +39,15 @@ public interface DataSource default String getConcatenatedName() { final List names = getNames(); + Preconditions.checkState(!names.isEmpty()); return names.size() > 1 ? names.toString() : names.get(0); } default String getFirstName() { - return Iterables.getFirst(getNames(), null); + final List names = getNames(); + Preconditions.checkState(!names.isEmpty()); + return names.get(0); } List getNames(); diff --git a/processing/src/main/java/io/druid/query/DataSourceUtil.java b/processing/src/main/java/io/druid/query/DataSourceUtil.java index 7609f858cd7c..c591e03884dc 100644 --- a/processing/src/main/java/io/druid/query/DataSourceUtil.java +++ b/processing/src/main/java/io/druid/query/DataSourceUtil.java @@ -19,12 +19,9 @@ package io.druid.query; -import org.joda.time.Interval; +import com.google.common.collect.Iterables; import java.util.List; -import java.util.StringJoiner; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; public class DataSourceUtil { @@ -34,21 +31,8 @@ public static String getMetricName(DataSource dataSource) return names.size() == 1 ? names.get(0) : names.toString(); } - public static String getMetricName(Iterable dataSources) + public static String getMetricName(List dataSources) { - return StreamSupport.stream(dataSources.spliterator(), false) - .map(DataSourceUtil::getMetricName) - .collect(Collectors.joining(",", "[", "]")); - } - - private static String getMetricName(DataSourceWithSegmentSpec spec) - { - final StringJoiner joiner = new StringJoiner("=", "[", "]"); - joiner.add(getMetricName(spec.getDataSource())); - joiner.add(spec.getQuerySegmentSpec().getIntervals().stream() - .map(Interval::toString) - .collect(Collectors.joining(",", "[", "]")) - ); - return joiner.toString(); + return Iterables.toString(dataSources); } } diff --git a/processing/src/main/java/io/druid/query/DataSourceWithSegmentSpec.java b/processing/src/main/java/io/druid/query/DataSourceWithSegmentSpec.java index 104e3d775a2c..0177e5a77532 100644 --- a/processing/src/main/java/io/druid/query/DataSourceWithSegmentSpec.java +++ b/processing/src/main/java/io/druid/query/DataSourceWithSegmentSpec.java @@ -21,10 +21,18 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; import io.druid.query.spec.QuerySegmentSpec; import java.util.Objects; +/** + * This class contains the information of queries' input data which consists of a {@link DataSource} and + * a {@link QuerySegmentSpec}. A dataSource can be one or more data sources, or the result of another query. + * A querySegmentSpec indicates time ranges of input data sources. + * As a result, a query will read data corresponding to the specified querySegmentSpec from the dataSource. + */ public class DataSourceWithSegmentSpec { private final DataSource dataSource; @@ -35,8 +43,8 @@ public DataSourceWithSegmentSpec( @JsonProperty("dataSource") DataSource dataSource, @JsonProperty("intervals") QuerySegmentSpec querySegmentSpec) { - this.dataSource = dataSource; - this.querySegmentSpec = querySegmentSpec; + this.dataSource = Preconditions.checkNotNull(dataSource); + this.querySegmentSpec = Preconditions.checkNotNull(querySegmentSpec); } @JsonProperty @@ -77,4 +85,9 @@ public boolean equals(Object o) return querySegmentSpec.equals(that.querySegmentSpec); } + @Override + public String toString() + { + return DataSourceUtil.getMetricName(dataSource) + "=" + Iterables.toString(querySegmentSpec.getIntervals()); + } } diff --git a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java index 12d426107f4c..361c9577e32d 100644 --- a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java @@ -28,8 +28,10 @@ import org.joda.time.Interval; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; public class DefaultQueryMetrics> implements QueryMetrics { @@ -55,7 +57,10 @@ public void query(QueryType query) @Override public void dataSources(QueryType query) { - builder.setDimension(DruidMetrics.DATASOURCE, DataSourceUtil.getMetricName(query.getDataSources())); + final List specs = query.getDataSources().stream() + .map(DataSourceWithSegmentSpec::toString) + .collect(Collectors.toList()); + builder.setDimension(DruidMetrics.DATASOURCE, specs.toArray(new String[specs.size()])); } @Override diff --git a/processing/src/main/java/io/druid/query/Druids.java b/processing/src/main/java/io/druid/query/Druids.java index 4aa14f30864b..cf4a1fed4281 100644 --- a/processing/src/main/java/io/druid/query/Druids.java +++ b/processing/src/main/java/io/druid/query/Druids.java @@ -452,7 +452,7 @@ public TimeseriesQueryBuilder updateDistributionTarget() context = new HashMap<>(); } context.put( - QueryContextKeys.DIST_TARGET_SOURCE, + QueryContextKeys.DISTRIBUTION_TARGET_SOURCE, new DataSourceWithSegmentSpec( SingleSourceBaseQuery.getLeafDataSource(dataSource), querySegmentSpec @@ -661,7 +661,7 @@ public SearchQueryBuilder updateDistributionTarget() context = new HashMap<>(); } context.put( - QueryContextKeys.DIST_TARGET_SOURCE, + QueryContextKeys.DISTRIBUTION_TARGET_SOURCE, new DataSourceWithSegmentSpec( SingleSourceBaseQuery.getLeafDataSource(dataSource), querySegmentSpec @@ -886,7 +886,7 @@ public TimeBoundaryQueryBuilder updateDistributionTarget() context = new HashMap<>(); } context.put( - QueryContextKeys.DIST_TARGET_SOURCE, + QueryContextKeys.DISTRIBUTION_TARGET_SOURCE, new DataSourceWithSegmentSpec( SingleSourceBaseQuery.getLeafDataSource(dataSource), querySegmentSpec @@ -1239,7 +1239,7 @@ public SelectQueryBuilder updateDistributionTarget() context = new HashMap<>(); } context.put( - QueryContextKeys.DIST_TARGET_SOURCE, + QueryContextKeys.DISTRIBUTION_TARGET_SOURCE, new DataSourceWithSegmentSpec( SingleSourceBaseQuery.getLeafDataSource(dataSource), querySegmentSpec diff --git a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java index abfc0ebd7e5e..fc12eb656a10 100644 --- a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java @@ -115,7 +115,7 @@ public Sequence apply(Interval singleInterval) ), executor, queryWatcher ).run( - query.replaceQuerySegmentSpecWith( + query.withQuerySegmentSpec( Iterables.getOnlyElement(spec.getDataSource().getNames()), new MultipleIntervalSegmentSpec(Arrays.asList(singleInterval)) ), diff --git a/processing/src/main/java/io/druid/query/Queries.java b/processing/src/main/java/io/druid/query/Queries.java index 1b85ee79e4ea..76a7947ea49d 100644 --- a/processing/src/main/java/io/druid/query/Queries.java +++ b/processing/src/main/java/io/druid/query/Queries.java @@ -32,7 +32,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.stream.Collectors; /** */ @@ -85,6 +84,14 @@ public static List prepareAggregations( return postAggs; } + /** + * Make string representations of data sources and intervals of a given query. + * The lhs of the result pair is the list of data source names. + * The rhs of the result pair is the list of interval strings in ISO8601 interval format ({@link Interval#toString()}). + * + * @param query a query + * @return a pair of strings of data sources and intervals + */ public static Pair getDataSourceAndIntervalStrings(final Query query) { final List datasourceNames = new ArrayList<>(); @@ -92,14 +99,9 @@ public static Pair getDataSourceAndIntervalStrings(final Que query.getDataSources().forEach(spec -> { datasourceNames.addAll(spec.getDataSource().getNames()); - intervals.addAll( - spec.getQuerySegmentSpec().getIntervals().stream().map(Interval::toString).collect(Collectors.toList()) - ); + spec.getQuerySegmentSpec().getIntervals().stream().map(Interval::toString).forEach(intervals::add); }); - return new Pair<>( - datasourceNames.stream().collect(Collectors.joining(",", "[", "]")), - intervals.stream().collect(Collectors.joining(",", "[", "]")) - ); + return new Pair<>(datasourceNames.toString(), intervals.toString()); } } diff --git a/processing/src/main/java/io/druid/query/Query.java b/processing/src/main/java/io/druid/query/Query.java index eed89597f2a0..1d3ee74aa99b 100644 --- a/processing/src/main/java/io/druid/query/Query.java +++ b/processing/src/main/java/io/druid/query/Query.java @@ -37,8 +37,8 @@ import io.druid.query.topn.TopNQuery; import org.joda.time.Duration; +import java.util.List; import java.util.Map; -import java.util.stream.StreamSupport; @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "queryType") @JsonSubTypes(value = { @@ -65,7 +65,7 @@ public interface Query String DATASOURCE_METADATA = "dataSourceMetadata"; String JOIN = "join"; - Iterable getDataSources(); + List getDataSources(); boolean hasFilters(); @@ -81,9 +81,11 @@ public interface Query default Duration getTotalDuration() { - return StreamSupport.stream(getDataSources().spliterator(), false) - .map(spec -> BaseQuery.initDuration(spec.getQuerySegmentSpec())) - .reduce(new Duration(0), Duration::plus); + Duration totalDuration = new Duration(0); + for (DataSourceWithSegmentSpec spec : getDataSources()) { + totalDuration = totalDuration.plus(BaseQuery.getTotalDuration(spec.getQuerySegmentSpec())); + } + return totalDuration; } Map getContext(); @@ -104,19 +106,19 @@ default Duration getTotalDuration() default DataSourceWithSegmentSpec getDistributionTarget() { - return getContextValue(QueryContextKeys.DIST_TARGET_SOURCE); + return getContextValue(QueryContextKeys.DISTRIBUTION_TARGET_SOURCE); } default Query distributeBy(DataSourceWithSegmentSpec spec) { - return withOverriddenContext(ImmutableMap.of(QueryContextKeys.DIST_TARGET_SOURCE, spec)); + return withOverriddenContext(ImmutableMap.of(QueryContextKeys.DISTRIBUTION_TARGET_SOURCE, spec)); } Query withOverriddenContext(Map contextOverride); - Query replaceQuerySegmentSpecWith(DataSource dataSource, QuerySegmentSpec spec); + Query withQuerySegmentSpec(DataSource dataSource, QuerySegmentSpec spec); - Query replaceQuerySegmentSpecWith(String dataSource, QuerySegmentSpec spec); + Query withQuerySegmentSpec(String dataSource, QuerySegmentSpec spec); Query replaceDataSourceWith(DataSource src, DataSource dst); } diff --git a/processing/src/main/java/io/druid/query/QueryContextKeys.java b/processing/src/main/java/io/druid/query/QueryContextKeys.java index abc5a7cbb0f4..2774395f90f4 100644 --- a/processing/src/main/java/io/druid/query/QueryContextKeys.java +++ b/processing/src/main/java/io/druid/query/QueryContextKeys.java @@ -24,5 +24,5 @@ public class QueryContextKeys public static final String PRIORITY = "priority"; public static final String TIMEOUT = "timeout"; public static final String CHUNK_PERIOD = "chunkPeriod"; - public static final String DIST_TARGET_SOURCE = "distTargetSource"; + public static final String DISTRIBUTION_TARGET_SOURCE = "distributionTargetSource"; } diff --git a/processing/src/main/java/io/druid/query/QueryDataSource.java b/processing/src/main/java/io/druid/query/QueryDataSource.java index 74c355863386..70cbc220d5cd 100644 --- a/processing/src/main/java/io/druid/query/QueryDataSource.java +++ b/processing/src/main/java/io/druid/query/QueryDataSource.java @@ -25,7 +25,6 @@ import java.util.List; import java.util.stream.Collectors; -import java.util.stream.StreamSupport; @JsonTypeName("query") public class QueryDataSource implements DataSource @@ -42,10 +41,9 @@ public QueryDataSource(@JsonProperty("query") Query query) @Override public List getNames() { - final Iterable sourceSpecs = query.getDataSources(); - return StreamSupport.stream(sourceSpecs.spliterator(), false) - .flatMap(spec -> spec.getDataSource().getNames().stream()) - .collect(Collectors.toList()); + return query.getDataSources().stream() + .flatMap(spec -> spec.getDataSource().getNames().stream()) + .collect(Collectors.toList()); } @JsonProperty diff --git a/processing/src/main/java/io/druid/query/RetryQueryRunner.java b/processing/src/main/java/io/druid/query/RetryQueryRunner.java index 086fb18410c9..2d04d306bab1 100644 --- a/processing/src/main/java/io/druid/query/RetryQueryRunner.java +++ b/processing/src/main/java/io/druid/query/RetryQueryRunner.java @@ -82,7 +82,7 @@ public Yielder toYielder( context.put(Result.MISSING_SEGMENTS_KEY, new HashMap<>()); Query retryQuery = query; for (Entry> entry : missingSegments.entrySet()) { - retryQuery = query.replaceQuerySegmentSpecWith( + retryQuery = query.withQuerySegmentSpec( entry.getKey(), new MultipleSpecificSegmentSpec(entry.getValue()) ); diff --git a/processing/src/main/java/io/druid/query/SingleSourceBaseQuery.java b/processing/src/main/java/io/druid/query/SingleSourceBaseQuery.java index 390d229353a7..d90b72246f91 100644 --- a/processing/src/main/java/io/druid/query/SingleSourceBaseQuery.java +++ b/processing/src/main/java/io/druid/query/SingleSourceBaseQuery.java @@ -55,7 +55,7 @@ public SingleSourceBaseQuery( public abstract Query withDataSource(DataSource dataSource); @Override - public Iterable getDataSources() + public List getDataSources() { return ImmutableList.of(dataSourceWithSegment); } @@ -98,14 +98,14 @@ public Duration getDuration(DataSource dataSource) public Duration getDuration() { if (duration == null) { - duration = initDuration(dataSourceWithSegment.getQuerySegmentSpec()); + duration = getTotalDuration(dataSourceWithSegment.getQuerySegmentSpec()); } return duration; } @Override - public Query replaceQuerySegmentSpecWith(DataSource dataSource, QuerySegmentSpec spec) + public Query withQuerySegmentSpec(DataSource dataSource, QuerySegmentSpec spec) { Preconditions.checkArgument(this.dataSourceWithSegment.getDataSource().equals(dataSource)); final Query query = withQuerySegmentSpec(spec); @@ -118,7 +118,7 @@ public Query replaceQuerySegmentSpecWith(DataSource dataSource, QuerySegmentS } @Override - public Query replaceQuerySegmentSpecWith(String dataSource, QuerySegmentSpec spec) + public Query withQuerySegmentSpec(String dataSource, QuerySegmentSpec spec) { Preconditions.checkArgument(Iterables.getOnlyElement(this.dataSourceWithSegment.getDataSource().getNames()).equals(dataSource)); final Query query = withQuerySegmentSpec(spec); diff --git a/processing/src/main/java/io/druid/query/TimewarpOperator.java b/processing/src/main/java/io/druid/query/TimewarpOperator.java index 77986a4f3678..54e7a0b69039 100644 --- a/processing/src/main/java/io/druid/query/TimewarpOperator.java +++ b/processing/src/main/java/io/druid/query/TimewarpOperator.java @@ -94,7 +94,7 @@ public Sequence run(final Query query, final Map responseC ); return Sequences.map( baseRunner.run( - query.replaceQuerySegmentSpecWith( + query.withQuerySegmentSpec( Iterables.getOnlyElement(spec.getDataSource().getNames()), new MultipleIntervalSegmentSpec(Arrays.asList(modifiedInterval)) ), diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java index 716410d968d1..428442fed446 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java @@ -663,7 +663,7 @@ public Builder updateDistributionTarget() context = new HashMap<>(); } context.put( - QueryContextKeys.DIST_TARGET_SOURCE, + QueryContextKeys.DISTRIBUTION_TARGET_SOURCE, new DataSourceWithSegmentSpec(getLeafDataSource(dataSource), querySegmentSpec) ); return this; diff --git a/processing/src/main/java/io/druid/query/join/DataInput.java b/processing/src/main/java/io/druid/query/join/DataInput.java index ad89a752c12a..6f747dba4d34 100644 --- a/processing/src/main/java/io/druid/query/join/DataInput.java +++ b/processing/src/main/java/io/druid/query/join/DataInput.java @@ -76,7 +76,7 @@ public QuerySegmentSpec getQuerySegmentSpec() public Duration getDuration() { if (duration == null) { - this.duration = BaseQuery.initDuration(querySegmentSpec); + this.duration = BaseQuery.getTotalDuration(querySegmentSpec); } return duration; } diff --git a/processing/src/main/java/io/druid/query/join/JoinQuery.java b/processing/src/main/java/io/druid/query/join/JoinQuery.java index 66112eb8d295..cdb356d69450 100644 --- a/processing/src/main/java/io/druid/query/join/JoinQuery.java +++ b/processing/src/main/java/io/druid/query/join/JoinQuery.java @@ -79,7 +79,7 @@ public JoinQuery( } @Override - public Iterable getDataSources() + public List getDataSources() { final List found = new ArrayList<>(); final JoinSpecVisitor visitor = new JoinSpecVisitor() @@ -128,7 +128,7 @@ public Duration getDuration(DataSource dataSource) { for (DataSourceWithSegmentSpec sourceWithSegmentSpec : getDataSources()) { if (sourceWithSegmentSpec.getDataSource().equals(dataSource)) { - return initDuration(sourceWithSegmentSpec.getQuerySegmentSpec()); + return getTotalDuration(sourceWithSegmentSpec.getQuerySegmentSpec()); } } return null; @@ -179,13 +179,13 @@ public Query withOverriddenContext(Map contextOverride) } @Override - public Query replaceQuerySegmentSpecWith(DataSource dataSource, QuerySegmentSpec spec) + public Query withQuerySegmentSpec(DataSource dataSource, QuerySegmentSpec spec) { - return replaceQuerySegmentSpecWith(dataSource.getFirstName(), spec); + return withQuerySegmentSpec(dataSource.getFirstName(), spec); } @Override - public Query replaceQuerySegmentSpecWith(String dataSource, QuerySegmentSpec spec) + public Query withQuerySegmentSpec(String dataSource, QuerySegmentSpec spec) { final JoinSpecVisitor visitor = new JoinSpecVisitor() { diff --git a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java index 2456b27895c3..08c887469bfe 100644 --- a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java @@ -40,7 +40,6 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import java.util.stream.StreamSupport; /** */ @@ -64,14 +63,14 @@ public SpecificSegmentQueryRunner( @Override public Sequence run(final Query input, final Map responseContext) { - final Query query = input.replaceQuerySegmentSpecWith(dataSourceName, specificSpec); + final Query query = input.withQuerySegmentSpec(dataSourceName, specificSpec); final Thread currThread = Thread.currentThread(); final String currThreadName = currThread.getName(); - final List specs = StreamSupport - .stream(input.getDataSources().spliterator(), false) - .filter(spec -> spec.getDataSource().getNames().contains(dataSourceName)) - .collect(Collectors.toList()); + final List specs = input.getDataSources().stream() + .filter(spec -> spec.getDataSource().getNames() + .contains(dataSourceName)) + .collect(Collectors.toList()); Preconditions.checkState(specs.size() == 1); final String newName = String.format( diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java b/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java index ae5723018069..953cc33f390a 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java @@ -217,7 +217,7 @@ public TopNQueryBuilder updateDistributionTarget() context = new HashMap<>(); } context.put( - QueryContextKeys.DIST_TARGET_SOURCE, + QueryContextKeys.DISTRIBUTION_TARGET_SOURCE, new DataSourceWithSegmentSpec( SingleSourceBaseQuery.getLeafDataSource(dataSource), querySegmentSpec diff --git a/processing/src/test/java/io/druid/query/DataSourceUtilTest.java b/processing/src/test/java/io/druid/query/DataSourceUtilTest.java new file mode 100644 index 000000000000..56de6cb8e392 --- /dev/null +++ b/processing/src/test/java/io/druid/query/DataSourceUtilTest.java @@ -0,0 +1,85 @@ +/* + * 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; + +import com.google.common.collect.Lists; +import io.druid.query.spec.MultipleIntervalSegmentSpec; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class DataSourceUtilTest +{ + + @Test + public void testGetMetricNameFromDataSource() + { + final DataSource source = new TableDataSource("table"); + Assert.assertEquals("table", DataSourceUtil.getMetricName(source)); + + final DataSource unionSource = new UnionDataSource( + Lists.newArrayList( + new TableDataSource("src1"), + new TableDataSource("src2"), + new TableDataSource("src3"), + new TableDataSource("src4") + ) + ); + Assert.assertEquals("[src1, src2, src3, src4]", DataSourceUtil.getMetricName(unionSource)); + } + + @Test + public void testGetMetricNameFromDataSourceWithSegmentSpec() + { + final List specs = Lists.newArrayList( + new DataSourceWithSegmentSpec( + new TableDataSource("src1"), + new MultipleIntervalSegmentSpec( + Lists.newArrayList( + new Interval(0, 100), + new Interval(200, 300), + new Interval(400, 500) + ) + ) + ), + new DataSourceWithSegmentSpec( + new TableDataSource("src2"), + new MultipleIntervalSegmentSpec( + Lists.newArrayList( + new Interval(0, 10), + new Interval(20, 30), + new Interval(40, 50) + ) + ) + ) + ); + Assert.assertEquals( + "[src1=[1970-01-01T00:00:00.000Z/1970-01-01T00:00:00.100Z, " + + "1970-01-01T00:00:00.200Z/1970-01-01T00:00:00.300Z, " + + "1970-01-01T00:00:00.400Z/1970-01-01T00:00:00.500Z], " + + "src2=[1970-01-01T00:00:00.000Z/1970-01-01T00:00:00.010Z, " + + "1970-01-01T00:00:00.020Z/1970-01-01T00:00:00.030Z, " + + "1970-01-01T00:00:00.040Z/1970-01-01T00:00:00.050Z]]", + DataSourceUtil.getMetricName(specs) + ); + } +} diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index 8559a76c2fa0..ed70e7413cae 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -76,7 +76,6 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import java.util.stream.StreamSupport; /** */ @@ -530,17 +529,17 @@ public static QueryRunner makeFilteringQueryRunner( @Override public Sequence run(Query query, Map responseContext) { - final List segments = StreamSupport - .stream(query.getDataSources().spliterator(), false) - .flatMap(spec -> spec.getQuerySegmentSpec().getIntervals().stream()) - .flatMap(interval -> timeline.lookup(interval).stream()) - .collect(Collectors.toList()); + final List segments = query.getDataSources().stream() + .flatMap(spec -> spec.getQuerySegmentSpec() + .getIntervals().stream()) + .flatMap(interval -> timeline.lookup(interval).stream()) + .collect(Collectors.toList()); List> sequences = Lists.newArrayList(); for (TimelineObjectHolder holder : toolChest.filterSegments(query, segments)) { Segment segment = holder.getObject().getChunk(0).getObject(); Query running = query; for (DataSourceWithSegmentSpec spec : query.getDataSources()) { - running = query.replaceQuerySegmentSpecWith( + running = query.withQuerySegmentSpec( spec.getDataSource(), new SpecificSegmentSpec( new SegmentDescriptor( diff --git a/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java b/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java index 0523afaae82d..c209cbd6ca79 100644 --- a/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java +++ b/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java @@ -244,7 +244,7 @@ private static Query setDistributionTarget(Query query) { return query.withOverriddenContext( ImmutableMap.of( - QueryContextKeys.DIST_TARGET_SOURCE, + QueryContextKeys.DISTRIBUTION_TARGET_SOURCE, new DataSourceWithSegmentSpec( Iterables.getOnlyElement(query.getDataSources()).getDataSource(), Iterables.getOnlyElement(query.getDataSources()).getQuerySegmentSpec() diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 9ae1c1f07f76..8466263a25d0 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -437,7 +437,7 @@ private void addSequencesFromServer(ArrayList> listOfSequences) if (!server.isAssignable() || !populateCache || isBySegment) { // Direct server queryable if (!isBySegment) { resultSeqToAdd = clientQueryable.run( - query.replaceQuerySegmentSpecWith(dataSourceName, segmentSpec), + query.withQuerySegmentSpec(dataSourceName, segmentSpec), responseContext ); } else { @@ -449,7 +449,7 @@ private void addSequencesFromServer(ArrayList> listOfSequences) @SuppressWarnings("unchecked") final Sequence>> resultSequence = clientQueryable.run( - bySegmentQuery.replaceQuerySegmentSpecWith(dataSourceName, segmentSpec), + bySegmentQuery.withQuerySegmentSpec(dataSourceName, segmentSpec), responseContext ); @@ -482,7 +482,7 @@ public Result> apply(Result>> runningSequence = clientQueryable.run( - rewrittenQuery.replaceQuerySegmentSpecWith(dataSourceName, segmentSpec), + rewrittenQuery.withQuerySegmentSpec(dataSourceName, segmentSpec), responseContext ); resultSeqToAdd = new MergeSequence( diff --git a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java index 60181cae1641..8fb61a8d1cf8 100644 --- a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java +++ b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java @@ -199,7 +199,7 @@ public QueryRunner apply(SegmentDescriptor spec) return retVal == null ? new NoopQueryRunner() : retVal.getQueryRunner( - query.replaceQuerySegmentSpecWith(dataSourceName, new SpecificSegmentSpec(spec)) + query.withQuerySegmentSpec(dataSourceName, new SpecificSegmentSpec(spec)) ); } } diff --git a/server/src/main/java/io/druid/server/BrokerQueryResource.java b/server/src/main/java/io/druid/server/BrokerQueryResource.java index 072c6c427e89..145a63f9a927 100644 --- a/server/src/main/java/io/druid/server/BrokerQueryResource.java +++ b/server/src/main/java/io/druid/server/BrokerQueryResource.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; -import com.google.common.collect.Iterables; import com.google.inject.Inject; import com.metamx.emitter.service.ServiceEmitter; import com.sun.jersey.spi.container.ResourceFilters; @@ -107,8 +106,8 @@ public Response getQueryTargets( Query query = context.getObjectMapper().readValue(in, Query.class); final DataSourceWithSegmentSpec spec = (DataSourceWithSegmentSpec) query.getContext().computeIfAbsent( - QueryContextKeys.DIST_TARGET_SOURCE, - key -> Iterables.getFirst(query.getDataSources(), null) + QueryContextKeys.DISTRIBUTION_TARGET_SOURCE, + key -> query.getDataSources().get(0) ); return context.ok( ServerViewUtil.getTargetLocations( diff --git a/server/src/main/java/io/druid/server/QueryManager.java b/server/src/main/java/io/druid/server/QueryManager.java index 46bb41aceae5..a1fc0bad16c1 100644 --- a/server/src/main/java/io/druid/server/QueryManager.java +++ b/server/src/main/java/io/druid/server/QueryManager.java @@ -62,7 +62,7 @@ public boolean cancelQuery(String id) public void registerQuery(Query query, final ListenableFuture future) { final String id = query.getId(); - final Iterable sources = query.getDataSources(); + final List sources = query.getDataSources(); final List dataSourceNames = new ArrayList<>(); for (DataSourceWithSegmentSpec eachSource : sources) { dataSourceNames.addAll(eachSource.getDataSource().getNames()); diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 5ca85e2dbb46..64ecefb9e3bb 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -178,7 +178,7 @@ public Response doPost( ) throws IOException { final long startNs = System.nanoTime(); - Query query = null; + Query query = null; QueryToolChest toolChest = null; String queryId = null; @@ -212,8 +212,7 @@ public Response doPost( // This is an experimental feature, see - https://github.com/druid-io/druid/pull/2424 AuthorizationInfo authorizationInfo = (AuthorizationInfo) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); if (authorizationInfo != null) { - final Iterable sources = query.getDataSources(); - for (DataSourceWithSegmentSpec eachSource : sources) { + for (DataSourceWithSegmentSpec eachSource : query.getDataSources()) { for (String dataSource : eachSource.getDataSource().getNames()) { Access authResult = authorizationInfo.isAuthorized( new Resource(dataSource, ResourceType.DATASOURCE), diff --git a/server/src/main/java/io/druid/server/coordination/ServerManager.java b/server/src/main/java/io/druid/server/coordination/ServerManager.java index 0fb334d21e02..2addae2a5fef 100644 --- a/server/src/main/java/io/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/io/druid/server/coordination/ServerManager.java @@ -267,9 +267,9 @@ public QueryRunner getQueryRunnerForIntervals(Query query, Iterable dataSourceNames = StreamSupport.stream(query.getDataSources().spliterator(), false) - .map(spec -> getDataSourceName(spec.getDataSource())) - .collect(Collectors.toList()); + final List dataSourceNames = query.getDataSources().stream() + .map(spec -> getDataSourceName(spec.getDataSource())) + .collect(Collectors.toList()); if (dataSourceNames.stream().anyMatch(name -> !dataSources.containsKey(name))) { return new NoopQueryRunner<>(); @@ -406,9 +406,9 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable dataSourceNames = StreamSupport.stream(query.getDataSources().spliterator(), false) - .map(spec -> getDataSourceName(spec.getDataSource())) - .collect(Collectors.toList()); + final List dataSourceNames = query.getDataSources().stream() + .map(spec -> getDataSourceName(spec.getDataSource())) + .collect(Collectors.toList()); if (dataSourceNames.stream().anyMatch(name -> !dataSources.containsKey(name))) { return new NoopQueryRunner<>(); diff --git a/server/src/main/java/io/druid/server/log/LoggingRequestLogger.java b/server/src/main/java/io/druid/server/log/LoggingRequestLogger.java index 45a1f52a41c3..26391f87ad50 100644 --- a/server/src/main/java/io/druid/server/log/LoggingRequestLogger.java +++ b/server/src/main/java/io/druid/server/log/LoggingRequestLogger.java @@ -57,9 +57,9 @@ public void log(RequestLogLine requestLogLine) throws IOException try { if (setMDC) { try { - final Query query = requestLogLine.getQuery(); + final Query query = requestLogLine.getQuery(); MDC.put("queryId", query.getId()); - MDC.put("dataSource", (String) Queries.getDataSourceAndIntervalStrings(query).lhs); + MDC.put("dataSource", Queries.getDataSourceAndIntervalStrings(query).lhs); MDC.put("queryType", query.getType()); MDC.put("hasFilters", Boolean.toString(query.hasFilters())); MDC.put("remoteAddr", requestLogLine.getRemoteAddr()); diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index 3819c29044c3..0fce2bf125b3 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -2002,7 +2002,7 @@ public void run() TestHelper.assertExpectedResults( expected, runner.run( - query.replaceQuerySegmentSpecWith( + query.withQuerySegmentSpec( Iterables.getOnlyElement(spec.getDataSource().getNames()), new MultipleIntervalSegmentSpec( ImmutableList.of( @@ -2229,7 +2229,7 @@ public Iterable>> apply(@Nullable Integer input) ) ), runner.run( - query.replaceQuerySegmentSpecWith( + query.withQuerySegmentSpec( Iterables.getOnlyElement(spec.getDataSource().getNames()), new MultipleIntervalSegmentSpec( ImmutableList.of( From 0a1cc22ca3691d9fee052ac0a729f1f246a0a2fc Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 13 Apr 2017 22:17:45 +0900 Subject: [PATCH 04/15] fix test --- .../java/io/druid/query/DefaultQueryMetricsTest.java | 8 +++++++- .../query/groupby/DefaultGroupByQueryMetricsTest.java | 10 ++++++++-- .../timeseries/DefaultTimeseriesQueryMetricsTest.java | 10 ++++++++-- .../druid/query/topn/DefaultTopNQueryMetricsTest.java | 10 ++++++++-- 4 files changed, 31 insertions(+), 7 deletions(-) diff --git a/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java b/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java index 7a2ae1167889..199ebda7e19c 100644 --- a/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java @@ -34,6 +34,7 @@ import java.util.Collections; import java.util.Map; +import java.util.stream.Collectors; public class DefaultQueryMetricsTest { @@ -71,7 +72,12 @@ public void testDefaultQueryMetricsQuery() Assert.assertTrue(actualEvent.containsKey("timestamp")); Assert.assertEquals("", actualEvent.get("host")); Assert.assertEquals("", actualEvent.get("service")); - Assert.assertEquals(DataSourceUtil.getMetricName(query.getDataSources()), actualEvent.get(DruidMetrics.DATASOURCE)); + Assert.assertEquals( + query.getDataSources().stream() + .map(DataSourceWithSegmentSpec::toString) + .collect(Collectors.toList()), + actualEvent.get(DruidMetrics.DATASOURCE) + ); Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE)); Assert.assertEquals("true", actualEvent.get("hasFilters")); Assert.assertEquals(query.getTotalDuration().toString(), actualEvent.get("duration")); diff --git a/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java b/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java index 5352d0199877..bfd934c6dfab 100644 --- a/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java @@ -25,7 +25,7 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.granularity.PeriodGranularity; import io.druid.query.CachingEmitter; -import io.druid.query.DataSourceUtil; +import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.DefaultQueryMetricsTest; import io.druid.query.DruidMetrics; import io.druid.query.QueryRunnerTestHelper; @@ -41,6 +41,7 @@ import java.util.Arrays; import java.util.Map; +import java.util.stream.Collectors; public class DefaultGroupByQueryMetricsTest { @@ -96,7 +97,12 @@ public void testDefaultGroupByQueryMetricsQuery() Assert.assertTrue(actualEvent.containsKey("timestamp")); Assert.assertEquals("", actualEvent.get("host")); Assert.assertEquals("", actualEvent.get("service")); - Assert.assertEquals(DataSourceUtil.getMetricName(query.getDataSources()), actualEvent.get(DruidMetrics.DATASOURCE)); + Assert.assertEquals( + query.getDataSources().stream() + .map(DataSourceWithSegmentSpec::toString) + .collect(Collectors.toList()), + actualEvent.get(DruidMetrics.DATASOURCE) + ); Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE)); Assert.assertEquals("true", actualEvent.get("hasFilters")); Assert.assertEquals(query.getTotalDuration().toString(), actualEvent.get("duration")); diff --git a/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java b/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java index 75228dfa5863..fb42eb201ee8 100644 --- a/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java @@ -22,7 +22,7 @@ import com.metamx.emitter.service.ServiceEmitter; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.CachingEmitter; -import io.druid.query.DataSourceUtil; +import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.DefaultQueryMetricsTest; import io.druid.query.DruidMetrics; import io.druid.query.Druids; @@ -33,6 +33,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.Map; +import java.util.stream.Collectors; public class DefaultTimeseriesQueryMetricsTest { @@ -70,7 +71,12 @@ public void testDefaultTimeseriesQueryMetricsQuery() Assert.assertTrue(actualEvent.containsKey("timestamp")); Assert.assertEquals("", actualEvent.get("host")); Assert.assertEquals("", actualEvent.get("service")); - Assert.assertEquals(DataSourceUtil.getMetricName(query.getDataSources()), actualEvent.get(DruidMetrics.DATASOURCE)); + Assert.assertEquals( + query.getDataSources().stream() + .map(DataSourceWithSegmentSpec::toString) + .collect(Collectors.toList()), + actualEvent.get(DruidMetrics.DATASOURCE) + ); Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE)); Assert.assertEquals("false", actualEvent.get("hasFilters")); Assert.assertEquals(query.getTotalDuration().toString(), actualEvent.get("duration")); diff --git a/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java b/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java index 4d20e3b1b9a6..36250a134f51 100644 --- a/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java @@ -24,7 +24,7 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.granularity.Granularities; import io.druid.query.CachingEmitter; -import io.druid.query.DataSourceUtil; +import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.DefaultQueryMetricsTest; import io.druid.query.DruidMetrics; import io.druid.query.QueryRunnerTestHelper; @@ -37,6 +37,7 @@ import java.util.Collections; import java.util.Map; +import java.util.stream.Collectors; public class DefaultTopNQueryMetricsTest { @@ -74,7 +75,12 @@ public void testDefaultTopNQueryMetricsQuery() Assert.assertTrue(actualEvent.containsKey("timestamp")); Assert.assertEquals("", actualEvent.get("host")); Assert.assertEquals("", actualEvent.get("service")); - Assert.assertEquals(DataSourceUtil.getMetricName(query.getDataSources()), actualEvent.get(DruidMetrics.DATASOURCE)); + Assert.assertEquals( + query.getDataSources().stream() + .map(DataSourceWithSegmentSpec::toString) + .collect(Collectors.toList()), + actualEvent.get(DruidMetrics.DATASOURCE) + ); Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE)); Assert.assertEquals("true", actualEvent.get("hasFilters")); Assert.assertEquals(query.getTotalDuration().toString(), actualEvent.get("duration")); From e7d6e8a45e06f2bbae2b2da29d8c76dcf9ce7960 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 14 Apr 2017 11:20:43 +0900 Subject: [PATCH 05/15] fix test --- .../java/io/druid/query/scan/ScanQuerySpecTest.java | 4 ++-- .../src/main/java/io/druid/query/BaseQuery.java | 12 +++++------- .../java/io/druid/query/DefaultQueryMetrics.java | 7 +------ .../java/io/druid/query/SingleSourceBaseQuery.java | 4 ++-- .../query/groupby/GroupByQueryQueryToolChest.java | 12 ++++-------- .../io/druid/query/select/SelectQuerySpecTest.java | 10 +++++----- .../io/druid/server/log/LoggingRequestLogger.java | 5 +---- .../java/io/druid/sql/calcite/CalciteQueryTest.java | 1 + 8 files changed, 21 insertions(+), 34 deletions(-) diff --git a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQuerySpecTest.java b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQuerySpecTest.java index 64d92b71547f..29c9d6c50967 100644 --- a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQuerySpecTest.java +++ b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQuerySpecTest.java @@ -43,7 +43,7 @@ public void testSerializationLegacyString() throws Exception + "\"filter\":null," + "\"columns\":[\"market\",\"quality\",\"index\"]," + "\"limit\":3," - + "\"context\":null}"; + + "\"context\":{}}"; String current = "{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"}," @@ -53,7 +53,7 @@ public void testSerializationLegacyString() throws Exception + "\"limit\":3," + "\"filter\":null," + "\"columns\":[\"market\",\"quality\",\"index\"]," - + "\"context\":null," + + "\"context\":{}," + "\"descending\":false}"; ScanQuery query = new ScanQuery( diff --git a/processing/src/main/java/io/druid/query/BaseQuery.java b/processing/src/main/java/io/druid/query/BaseQuery.java index fda814096fd4..b5f798c2773d 100644 --- a/processing/src/main/java/io/druid/query/BaseQuery.java +++ b/processing/src/main/java/io/druid/query/BaseQuery.java @@ -112,7 +112,7 @@ public BaseQuery( Map context ) { - this.context = context; + this.context = context == null ? Maps.newTreeMap() : context; this.descending = descending; } @@ -149,7 +149,7 @@ public Map getContext() @Override public ContextType getContextValue(String key) { - return context == null ? null : (ContextType) context.get(key); + return (ContextType) context.get(key); } @Override @@ -169,9 +169,7 @@ protected Map computeOverridenContext(Map overri { Map overridden = Maps.newTreeMap(); final Map context = getContext(); - if (context != null) { - overridden.putAll(context); - } + overridden.putAll(context); overridden.putAll(overrides); return overridden; @@ -211,7 +209,7 @@ public boolean equals(Object o) if (descending != baseQuery.descending) { return false; } - if (context != null ? !context.equals(baseQuery.context) : baseQuery.context != null) { + if (!context.equals(baseQuery.context)) { return false; } @@ -222,7 +220,7 @@ public boolean equals(Object o) public int hashCode() { int result = (descending ? 1 : 0); - result = 31 * result + (context != null ? context.hashCode() : 0); + result = 31 * result + context.hashCode(); return result; } } diff --git a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java index 361c9577e32d..9e729c3f56a1 100644 --- a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Strings; -import com.google.common.collect.ImmutableMap; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import org.joda.time.Interval; @@ -93,11 +92,7 @@ public void context(QueryType query) try { builder.setDimension( "context", - jsonMapper.writeValueAsString( - query.getContext() == null - ? ImmutableMap.of() - : query.getContext() - ) + jsonMapper.writeValueAsString(query.getContext()) ); } catch (JsonProcessingException e) { diff --git a/processing/src/main/java/io/druid/query/SingleSourceBaseQuery.java b/processing/src/main/java/io/druid/query/SingleSourceBaseQuery.java index d90b72246f91..3e8ba4f68a14 100644 --- a/processing/src/main/java/io/druid/query/SingleSourceBaseQuery.java +++ b/processing/src/main/java/io/druid/query/SingleSourceBaseQuery.java @@ -152,7 +152,7 @@ public boolean equals(Object o) if (isDescending() != baseQuery.isDescending()) { return false; } - if (getContext() != null ? !getContext().equals(baseQuery.getContext()) : baseQuery.getContext() != null) { + if (!getContext().equals(baseQuery.getContext())) { return false; } if (!dataSourceWithSegment.equals(baseQuery.dataSourceWithSegment)) { @@ -170,7 +170,7 @@ public int hashCode() { int result = dataSourceWithSegment.hashCode(); result = 31 * result + (isDescending() ? 1 : 0); - result = 31 * result + (getContext() != null ? getContext().hashCode() : 0); + result = 31 * result + getContext().hashCode(); result = 31 * result + (duration != null ? duration.hashCode() : 0); return result; } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index 95835db53d83..7f20b975313b 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -171,16 +171,12 @@ private Sequence mergeGroupByResults( // Inject outer query context keys into subquery if they don't already exist in the subquery context. // Unlike withOverriddenContext's normal behavior, we want keys present in the subquery to win. final Map subqueryContext = Maps.newTreeMap(); - if (query.getContext() != null) { - for (Map.Entry entry : query.getContext().entrySet()) { - if (entry.getValue() != null) { - subqueryContext.put(entry.getKey(), entry.getValue()); - } + for (Map.Entry entry : query.getContext().entrySet()) { + if (entry.getValue() != null) { + subqueryContext.put(entry.getKey(), entry.getValue()); } } - if (((QueryDataSource) dataSource).getQuery().getContext() != null) { - subqueryContext.putAll(((QueryDataSource) dataSource).getQuery().getContext()); - } + subqueryContext.putAll(((QueryDataSource) dataSource).getQuery().getContext()); subqueryContext.put(GroupByQuery.CTX_KEY_SORT_BY_DIMS_FIRST, false); subquery = (GroupByQuery) ((QueryDataSource) dataSource).getQuery().withOverriddenContext(subqueryContext); } diff --git a/processing/src/test/java/io/druid/query/select/SelectQuerySpecTest.java b/processing/src/test/java/io/druid/query/select/SelectQuerySpecTest.java index 63672f383f69..331f21bad9ae 100644 --- a/processing/src/test/java/io/druid/query/select/SelectQuerySpecTest.java +++ b/processing/src/test/java/io/druid/query/select/SelectQuerySpecTest.java @@ -59,7 +59,7 @@ public void testSerializationLegacyString() throws Exception + "\"metrics\":[\"index\"]," + "\"virtualColumns\":null," + "\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":3}," - + "\"context\":null}"; + + "\"context\":{}}"; String current = "{\"queryType\":\"select\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"}," @@ -73,7 +73,7 @@ public void testSerializationLegacyString() throws Exception + "\"metrics\":[\"index\"]," + "\"virtualColumns\":[]," + "\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":3,\"fromNext\":true}," - + "\"context\":null}"; + + "\"context\":{}}"; SelectQuery query = new SelectQuery( new TableDataSource(QueryRunnerTestHelper.dataSource), @@ -112,17 +112,17 @@ public void testPagingSpecFromNext() throws Exception String withNull = baseQueryJson + "\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":3,\"fromNext\":null}," - + "\"context\":null}"; + + "\"context\":[}}"; String withFalse = baseQueryJson + "\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":3,\"fromNext\":false}," - + "\"context\":null}"; + + "\"context\":{}}"; String withTrue = baseQueryJson + "\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":3,\"fromNext\":true}," - + "\"context\":null}"; + + "\"context\":{}}"; SelectQuery queryWithNull = new SelectQuery( new TableDataSource(QueryRunnerTestHelper.dataSource), diff --git a/server/src/main/java/io/druid/server/log/LoggingRequestLogger.java b/server/src/main/java/io/druid/server/log/LoggingRequestLogger.java index 26391f87ad50..d63bada38b0a 100644 --- a/server/src/main/java/io/druid/server/log/LoggingRequestLogger.java +++ b/server/src/main/java/io/druid/server/log/LoggingRequestLogger.java @@ -20,7 +20,6 @@ package io.druid.server.log; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; import io.druid.java.util.common.logger.Logger; import io.druid.query.Queries; import io.druid.query.Query; @@ -66,9 +65,7 @@ public void log(RequestLogLine requestLogLine) throws IOException MDC.put("duration", query.getTotalDuration().toString()); MDC.put("descending", Boolean.toString(query.isDescending())); if (setContextMDC) { - final Iterable> entries = query.getContext() == null - ? ImmutableList.>of() - : query.getContext().entrySet(); + final Iterable> entries = query.getContext().entrySet(); for (Map.Entry entry : entries) { MDC.put(entry.getKey(), entry.getValue() == null ? "NULL" : entry.getValue().toString()); } diff --git a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java index d56902da7c5f..c2d6836d9aa9 100644 --- a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java @@ -2148,6 +2148,7 @@ public void testSelectDistinctWithSortAsOuterQuery4() throws Exception .metric(new InvertedTopNMetricSpec(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC))) .threshold(5) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( From 317060dbd6a1b6334a0c53bad4ec23ff4ddeec21 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 18 Apr 2017 11:20:23 +0900 Subject: [PATCH 06/15] Add MultiSourceBaseQuery --- .../java/io/druid/query/scan/ScanQuery.java | 10 +- .../query/scan/ScanQueryQueryToolChest.java | 4 +- .../query/scan/ScanQueryRunnerFactory.java | 4 +- .../java/io/druid/query/AsyncQueryRunner.java | 2 +- .../main/java/io/druid/query/BaseQuery.java | 207 +++++++++++------- .../query/ChainedExecutionQueryRunner.java | 2 +- .../io/druid/query/DefaultQueryMetrics.java | 4 +- .../src/main/java/io/druid/query/Druids.java | 16 +- .../druid/query/GroupByMergedQueryRunner.java | 2 +- .../query/IntervalChunkingQueryRunner.java | 2 +- .../io/druid/query/MultiSourceBaseQuery.java | 98 +++++++++ .../src/main/java/io/druid/query/Query.java | 60 ++++- ...eryContextKeys.java => QueryContexts.java} | 35 ++- .../java/io/druid/query/QueryDataSource.java | 2 +- .../java/io/druid/query/QueryMetrics.java | 2 +- .../io/druid/query/SingleSourceBaseQuery.java | 200 ----------------- .../io/druid/query/SubqueryQueryRunner.java | 6 +- .../java/io/druid/query/UnionQueryRunner.java | 4 +- .../DataSourceMetadataQuery.java | 4 +- .../DataSourceQueryQueryToolChest.java | 2 +- .../io/druid/query/groupby/GroupByQuery.java | 8 +- .../groupby/GroupByQueryQueryToolChest.java | 2 +- .../GroupByMergingQueryRunnerV2.java | 4 +- .../groupby/strategy/GroupByStrategyV2.java | 4 +- .../java/io/druid/query/join/DataInput.java | 4 +- .../java/io/druid/query/join/JoinQuery.java | 16 +- .../SegmentMetadataQueryQueryToolChest.java | 2 +- .../SegmentMetadataQueryRunnerFactory.java | 4 +- .../metadata/SegmentMetadataQuery.java | 4 +- .../search/SearchQueryQueryToolChest.java | 2 +- .../query/search/search/SearchQuery.java | 4 +- .../io/druid/query/select/SelectQuery.java | 4 +- .../select/SelectQueryQueryToolChest.java | 2 +- .../query/timeboundary/TimeBoundaryQuery.java | 4 +- .../TimeBoundaryQueryQueryToolChest.java | 4 +- .../query/timeseries/TimeseriesQuery.java | 4 +- .../TimeseriesQueryQueryToolChest.java | 2 +- .../java/io/druid/query/topn/TopNQuery.java | 4 +- .../io/druid/query/topn/TopNQueryBuilder.java | 8 +- .../query/topn/TopNQueryQueryToolChest.java | 2 +- .../io/druid/query/AsyncQueryRunnerTest.java | 2 +- .../ChainedExecutionQueryRunnerTest.java | 2 +- .../IntervalChunkingQueryRunnerTest.java | 4 +- .../io/druid/query/RetryQueryRunnerTest.java | 4 +- .../io/druid/query/TimewarpOperatorTest.java | 13 +- .../io/druid/query/UnionQueryRunnerTest.java | 4 +- .../groupby/GroupByQueryMergeBufferTest.java | 10 +- .../GroupByQueryRunnerFailureTest.java | 8 +- .../io/druid/server/BrokerQueryResource.java | 4 +- .../java/io/druid/server/QueryResource.java | 6 +- .../coordination/ServerManagerTest.java | 2 +- .../server/log/LoggingRequestLoggerTest.java | 4 +- .../io/druid/sql/calcite/rel/QueryMaker.java | 11 +- .../druid/sql/calcite/schema/DruidSchema.java | 2 +- 54 files changed, 419 insertions(+), 411 deletions(-) create mode 100644 processing/src/main/java/io/druid/query/MultiSourceBaseQuery.java rename processing/src/main/java/io/druid/query/{QueryContextKeys.java => QueryContexts.java} (53%) delete mode 100644 processing/src/main/java/io/druid/query/SingleSourceBaseQuery.java diff --git a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQuery.java b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQuery.java index 3686e505f4f1..8e9e09d2f9e4 100644 --- a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQuery.java +++ b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQuery.java @@ -23,11 +23,11 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; +import io.druid.query.BaseQuery; import io.druid.query.DataSource; import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.Query; -import io.druid.query.QueryContextKeys; -import io.druid.query.SingleSourceBaseQuery; +import io.druid.query.QueryContexts; import io.druid.query.TableDataSource; import io.druid.query.filter.DimFilter; import io.druid.query.filter.InDimFilter; @@ -42,7 +42,7 @@ import java.util.Map; @JsonTypeName("scan") -public class ScanQuery extends SingleSourceBaseQuery +public class ScanQuery extends BaseQuery { public static final String SCAN = "scan"; public static final String RESULT_FORMAT_LIST = "list"; @@ -319,9 +319,9 @@ public ScanQueryBuilder updateDistributionTarget() context = new HashMap<>(); } context.put( - QueryContextKeys.DISTRIBUTION_TARGET_SOURCE, + QueryContexts.DISTRIBUTION_TARGET_SOURCE, new DataSourceWithSegmentSpec( - SingleSourceBaseQuery.getLeafDataSource(dataSource), + BaseQuery.getLeafDataSource(dataSource), querySegmentSpec ) ); diff --git a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java index e40cc7100784..2e917fd80573 100644 --- a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java +++ b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java @@ -25,9 +25,9 @@ import io.druid.java.util.common.guava.BaseSequence; import io.druid.java.util.common.guava.CloseQuietly; import io.druid.java.util.common.guava.Sequence; +import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.Query; import io.druid.query.QueryMetrics; -import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.aggregation.MetricManipulationFn; @@ -127,7 +127,7 @@ public QueryRunner annotateDistributionTarget(QueryRunner { ScanQuery scanQuery = (ScanQuery) query; return runner.run( - scanQuery.distributeBy(scanQuery.getDataSourceWithSegmentSpec()), + scanQuery.updateDistributionTarget(), responseContext ); }; diff --git a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryRunnerFactory.java b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryRunnerFactory.java index 712249eac5b7..f597d0094e5b 100644 --- a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryRunnerFactory.java +++ b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryRunnerFactory.java @@ -25,7 +25,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.query.Query; -import io.druid.query.QueryContextKeys; +import io.druid.query.QueryContexts; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; @@ -71,7 +71,7 @@ public Sequence run( final Query query, final Map responseContext ) { - final Number queryTimeout = query.getContextValue(QueryContextKeys.TIMEOUT, null); + final Number queryTimeout = query.getContextValue(QueryContexts.TIMEOUT, null); final long timeoutAt = (queryTimeout == null || queryTimeout.longValue() == 0L) ? JodaUtils.MAX_INSTANT : System.currentTimeMillis() + queryTimeout.longValue(); responseContext.put(CTX_TIMEOUT_AT, timeoutAt); diff --git a/processing/src/main/java/io/druid/query/AsyncQueryRunner.java b/processing/src/main/java/io/druid/query/AsyncQueryRunner.java index 2f6e84c8bc69..73d44aa80e7e 100644 --- a/processing/src/main/java/io/druid/query/AsyncQueryRunner.java +++ b/processing/src/main/java/io/druid/query/AsyncQueryRunner.java @@ -68,7 +68,7 @@ public Sequence call() throws Exception public Sequence get() { try { - Number timeout = query.getContextValue(QueryContextKeys.TIMEOUT); + Number timeout = query.getContextValue(QueryContexts.TIMEOUT); if (timeout == null) { return future.get(); } else { diff --git a/processing/src/main/java/io/druid/query/BaseQuery.java b/processing/src/main/java/io/druid/query/BaseQuery.java index b5f798c2773d..83b1d1243dd5 100644 --- a/processing/src/main/java/io/druid/query/BaseQuery.java +++ b/processing/src/main/java/io/druid/query/BaseQuery.java @@ -20,15 +20,17 @@ package io.druid.query; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; -import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.Sequence; import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.Duration; import org.joda.time.Interval; +import java.util.List; import java.util.Map; /** @@ -37,62 +39,32 @@ public abstract class BaseQuery> implements Query { public static int getContextPriority(Query query, int defaultValue) { - return parseInt(query, "priority", defaultValue); + return QueryContexts.parseInt(query, "priority", defaultValue); } public static boolean getContextBySegment(Query query, boolean defaultValue) { - return parseBoolean(query, "bySegment", defaultValue); + return QueryContexts.parseBoolean(query, "bySegment", defaultValue); } public static boolean getContextPopulateCache(Query query, boolean defaultValue) { - return parseBoolean(query, "populateCache", defaultValue); + return QueryContexts.parseBoolean(query, "populateCache", defaultValue); } public static boolean getContextUseCache(Query query, boolean defaultValue) { - return parseBoolean(query, "useCache", defaultValue); + return QueryContexts.parseBoolean(query, "useCache", defaultValue); } public static boolean getContextFinalize(Query query, boolean defaultValue) { - return parseBoolean(query, "finalize", defaultValue); + return QueryContexts.parseBoolean(query, "finalize", defaultValue); } public static int getContextUncoveredIntervalsLimit(Query query, int defaultValue) { - return parseInt(query, "uncoveredIntervalsLimit", defaultValue); - } - - private static int parseInt(Query query, String key, int defaultValue) - { - Object val = query.getContextValue(key); - if (val == null) { - return defaultValue; - } - if (val instanceof String) { - return Integer.parseInt((String) val); - } else if (val instanceof Integer) { - return (int) val; - } else { - throw new ISE("Unknown type [%s]", val.getClass()); - } - } - - private static boolean parseBoolean(Query query, String key, boolean defaultValue) - { - Object val = query.getContextValue(key); - if (val == null) { - return defaultValue; - } - if (val instanceof String) { - return Boolean.parseBoolean((String) val); - } else if (val instanceof Boolean) { - return (boolean) val; - } else { - throw new ISE("Unknown type [%s]. Cannot parse!", val.getClass()); - } + return QueryContexts.parseInt(query, "uncoveredIntervalsLimit", defaultValue); } public static void checkInterrupted() @@ -102,96 +74,98 @@ public static void checkInterrupted() } } - public static final String QUERYID = "queryId"; - + private final DataSource dataSource; private final boolean descending; private final Map context; + private final QuerySegmentSpec querySegmentSpec; + private volatile Duration duration; public BaseQuery( + DataSource dataSource, + QuerySegmentSpec querySegmentSpec, boolean descending, Map context ) { + Preconditions.checkNotNull(dataSource, "dataSource can't be null"); + Preconditions.checkNotNull(querySegmentSpec, "querySegmentSpec can't be null"); + + this.dataSource = dataSource; this.context = context == null ? Maps.newTreeMap() : context; + this.querySegmentSpec = querySegmentSpec; this.descending = descending; } @JsonProperty - @Override - public boolean isDescending() + public DataSource getDataSource() { - return descending; + return dataSource; } - public Sequence run(QueryRunner runner, Map context) + @Override + public List getDataSources() { - return runner.run(this, context); + return ImmutableList.of(new DataSourceWithSegmentSpec(dataSource, querySegmentSpec)); } - public static Duration getTotalDuration(QuerySegmentSpec querySegmentSpec) + @JsonProperty + @Override + public boolean isDescending() { - Duration totalDuration = new Duration(0); - for (Interval interval : querySegmentSpec.getIntervals()) { - if (interval != null) { - totalDuration = totalDuration.plus(interval.toDuration()); - } - } - return totalDuration; + return descending; } - @Override - @JsonProperty - public Map getContext() + @JsonProperty("intervals") + public QuerySegmentSpec getQuerySegmentSpec() { - return context; + return querySegmentSpec; } @Override - public ContextType getContextValue(String key) + public Sequence run(QuerySegmentWalker walker, Map context) { - return (ContextType) context.get(key); + return run(querySegmentSpec.lookup(this, walker), context); } - @Override - public ContextType getContextValue(String key, ContextType defaultValue) + public List getIntervals() { - ContextType retVal = getContextValue(key); - return retVal == null ? defaultValue : retVal; + return querySegmentSpec.getIntervals(); } @Override - public boolean getContextBoolean(String key, boolean defaultValue) + public Duration getDuration(DataSource dataSource) { - return parseBoolean(this, key, defaultValue); + Preconditions.checkArgument(this.dataSource.equals(dataSource)); + return getDuration(); } - protected Map computeOverridenContext(Map overrides) + public Duration getDuration() { - Map overridden = Maps.newTreeMap(); - final Map context = getContext(); - overridden.putAll(context); - overridden.putAll(overrides); - - return overridden; - } + if (duration == null) { + Duration totalDuration = new Duration(0); + for (Interval interval : querySegmentSpec.getIntervals()) { + if (interval != null) { + totalDuration = totalDuration.plus(interval.toDuration()); + } + } + duration = totalDuration; + } - @Override - public Ordering getResultOrdering() - { - Ordering retVal = Ordering.natural(); - return descending ? retVal.reverse() : retVal; + return duration; } @Override - public String getId() + @JsonProperty + public Map getContext() { - return (String) getContextValue(QUERYID); + return context; } @Override - public Query withId(String id) + public Ordering getResultOrdering() { - return withOverriddenContext(ImmutableMap.of(QUERYID, id)); + Ordering retVal = Ordering.natural(); + return descending ? retVal.reverse() : retVal; } @Override @@ -212,6 +186,15 @@ public boolean equals(Object o) if (!context.equals(baseQuery.context)) { return false; } + if (!dataSource.equals(baseQuery.dataSource)) { + return false; + } + if (duration != null ? !duration.equals(baseQuery.duration) : baseQuery.duration != null) { + return false; + } + if (!querySegmentSpec.equals(baseQuery.querySegmentSpec)) { + return false; + } return true; } @@ -219,8 +202,68 @@ public boolean equals(Object o) @Override public int hashCode() { - int result = (descending ? 1 : 0); + int result = dataSource.hashCode(); + result = 31 * result + (descending ? 1 : 0); result = 31 * result + context.hashCode(); + result = 31 * result + querySegmentSpec.hashCode(); + result = 31 * result + (duration != null ? duration.hashCode() : 0); return result; } + + public Query updateDistributionTarget() + { + return withOverriddenContext( + ImmutableMap.of( + QueryContexts.DISTRIBUTION_TARGET_SOURCE, + new DataSourceWithSegmentSpec(dataSource, querySegmentSpec) + ) + ); + } + + @Override + public Query withQuerySegmentSpec(DataSource dataSource, QuerySegmentSpec spec) + { + Preconditions.checkArgument(this.dataSource.equals(dataSource)); + return withQuerySegmentSpec(spec); + } + + @Override + public Query withQuerySegmentSpec(String dataSource, QuerySegmentSpec spec) + { + Preconditions.checkArgument(this.dataSource.getFirstName().equals(dataSource)); + return withQuerySegmentSpec(spec); + } + + @Override + public Query replaceDataSourceWith(DataSource src, DataSource dst) + { + Preconditions.checkArgument(this.dataSource.equals(src)); + return withDataSource(dst); + } + + public abstract Query withQuerySegmentSpec(QuerySegmentSpec spec); + public abstract Query withDataSource(DataSource dataSource); + + public static > DataSource getLeafDataSource( + BaseQuery query + ) + { + final DataSource dataSource = query.getDataSource(); + if (dataSource instanceof QueryDataSource) { + final QueryDataSource queryDataSource = (QueryDataSource) dataSource; + return getLeafDataSource((BaseQuery) queryDataSource.getQuery()); + } else { + return dataSource; + } + } + + public static DataSource getLeafDataSource(DataSource dataSource) + { + if (dataSource instanceof QueryDataSource) { + final QueryDataSource queryDataSource = (QueryDataSource) dataSource; + return getLeafDataSource((BaseQuery) queryDataSource.getQuery()); + } else { + return dataSource; + } + } } diff --git a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java index 45363cd3a5c7..074cb8a22feb 100644 --- a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java @@ -152,7 +152,7 @@ public Iterable call() throws Exception queryWatcher.registerQuery(query, futures); try { - final Number timeout = query.getContextValue(QueryContextKeys.TIMEOUT, (Number) null); + final Number timeout = query.getContextValue(QueryContexts.TIMEOUT, (Number) null); return new MergeIterable<>( ordering.nullsFirst(), timeout == null ? diff --git a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java index 9e729c3f56a1..49a368d9f378 100644 --- a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java @@ -46,7 +46,7 @@ public DefaultQueryMetrics(ObjectMapper jsonMapper) @Override public void query(QueryType query) { - dataSources(query); + dataSourcesAndIntervals(query); queryType(query); hasFilters(query); duration(query); @@ -54,7 +54,7 @@ public void query(QueryType query) } @Override - public void dataSources(QueryType query) + public void dataSourcesAndIntervals(QueryType query) { final List specs = query.getDataSources().stream() .map(DataSourceWithSegmentSpec::toString) diff --git a/processing/src/main/java/io/druid/query/Druids.java b/processing/src/main/java/io/druid/query/Druids.java index cf4a1fed4281..c2345314bf5e 100644 --- a/processing/src/main/java/io/druid/query/Druids.java +++ b/processing/src/main/java/io/druid/query/Druids.java @@ -452,9 +452,9 @@ public TimeseriesQueryBuilder updateDistributionTarget() context = new HashMap<>(); } context.put( - QueryContextKeys.DISTRIBUTION_TARGET_SOURCE, + QueryContexts.DISTRIBUTION_TARGET_SOURCE, new DataSourceWithSegmentSpec( - SingleSourceBaseQuery.getLeafDataSource(dataSource), + BaseQuery.getLeafDataSource(dataSource), querySegmentSpec ) ); @@ -661,9 +661,9 @@ public SearchQueryBuilder updateDistributionTarget() context = new HashMap<>(); } context.put( - QueryContextKeys.DISTRIBUTION_TARGET_SOURCE, + QueryContexts.DISTRIBUTION_TARGET_SOURCE, new DataSourceWithSegmentSpec( - SingleSourceBaseQuery.getLeafDataSource(dataSource), + BaseQuery.getLeafDataSource(dataSource), querySegmentSpec ) ); @@ -886,9 +886,9 @@ public TimeBoundaryQueryBuilder updateDistributionTarget() context = new HashMap<>(); } context.put( - QueryContextKeys.DISTRIBUTION_TARGET_SOURCE, + QueryContexts.DISTRIBUTION_TARGET_SOURCE, new DataSourceWithSegmentSpec( - SingleSourceBaseQuery.getLeafDataSource(dataSource), + BaseQuery.getLeafDataSource(dataSource), querySegmentSpec ) ); @@ -1239,9 +1239,9 @@ public SelectQueryBuilder updateDistributionTarget() context = new HashMap<>(); } context.put( - QueryContextKeys.DISTRIBUTION_TARGET_SOURCE, + QueryContexts.DISTRIBUTION_TARGET_SOURCE, new DataSourceWithSegmentSpec( - SingleSourceBaseQuery.getLeafDataSource(dataSource), + BaseQuery.getLeafDataSource(dataSource), querySegmentSpec ) ); diff --git a/processing/src/main/java/io/druid/query/GroupByMergedQueryRunner.java b/processing/src/main/java/io/druid/query/GroupByMergedQueryRunner.java index d416589f199a..27493e7e8f5b 100644 --- a/processing/src/main/java/io/druid/query/GroupByMergedQueryRunner.java +++ b/processing/src/main/java/io/druid/query/GroupByMergedQueryRunner.java @@ -178,7 +178,7 @@ private void waitForFutureCompletion( { try { queryWatcher.registerQuery(query, future); - final Number timeout = query.getContextValue(QueryContextKeys.TIMEOUT, (Number) null); + final Number timeout = query.getContextValue(QueryContexts.TIMEOUT, (Number) null); if (timeout == null) { future.get(); } else { diff --git a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java index fc12eb656a10..04c5eb2f1bd4 100644 --- a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java @@ -153,7 +153,7 @@ private Iterable splitInterval(Interval interval, Period period) private Period getChunkPeriod(Query query) { - String p = query.getContextValue(QueryContextKeys.CHUNK_PERIOD, "P0D"); + String p = query.getContextValue(QueryContexts.CHUNK_PERIOD, "P0D"); return Period.parse(p); } } diff --git a/processing/src/main/java/io/druid/query/MultiSourceBaseQuery.java b/processing/src/main/java/io/druid/query/MultiSourceBaseQuery.java new file mode 100644 index 000000000000..057bb0a3852c --- /dev/null +++ b/processing/src/main/java/io/druid/query/MultiSourceBaseQuery.java @@ -0,0 +1,98 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Maps; +import com.google.common.collect.Ordering; +import io.druid.java.util.common.guava.Sequence; + +import java.util.Map; + +public abstract class MultiSourceBaseQuery> implements Query +{ + private final boolean descending; + private final Map context; + + public MultiSourceBaseQuery( + boolean descending, + Map context + ) + { + this.context = context == null ? Maps.newTreeMap() : context; + this.descending = descending; + } + + @Override + public Sequence run(QuerySegmentWalker walker, Map context) + { + return run(getDistributionTarget().getQuerySegmentSpec().lookup(this, walker), context); + } + + @JsonProperty + @Override + public boolean isDescending() + { + return descending; + } + + @Override + @JsonProperty + public Map getContext() + { + return context; + } + + @Override + public Ordering getResultOrdering() + { + Ordering retVal = Ordering.natural(); + return descending ? retVal.reverse() : retVal; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + MultiSourceBaseQuery that = (MultiSourceBaseQuery) o; + if (descending != that.descending) { + return false; + } + if (!context.equals(that.context)) { + return false; + } + return true; + } + + @Override + public int hashCode() + { + int result = descending ? 1 : 0; + result = 31 * result + context.hashCode(); + return result; + } +} diff --git a/processing/src/main/java/io/druid/query/Query.java b/processing/src/main/java/io/druid/query/Query.java index 1d3ee74aa99b..f8c17870798e 100644 --- a/processing/src/main/java/io/druid/query/Query.java +++ b/processing/src/main/java/io/druid/query/Query.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import io.druid.java.util.common.guava.Sequence; import io.druid.query.datasourcemetadata.DataSourceMetadataQuery; @@ -36,6 +37,7 @@ import io.druid.query.timeseries.TimeseriesQuery; import io.druid.query.topn.TopNQuery; import org.joda.time.Duration; +import org.joda.time.Interval; import java.util.List; import java.util.Map; @@ -75,7 +77,10 @@ public interface Query Sequence run(QuerySegmentWalker walker, Map context); - Sequence run(QueryRunner runner, Map context); + default Sequence run(QueryRunner runner, Map context) + { + return runner.run(this, context); + } Duration getDuration(DataSource dataSource); @@ -83,35 +88,61 @@ default Duration getTotalDuration() { Duration totalDuration = new Duration(0); for (DataSourceWithSegmentSpec spec : getDataSources()) { - totalDuration = totalDuration.plus(BaseQuery.getTotalDuration(spec.getQuerySegmentSpec())); + totalDuration = totalDuration.plus(Query.getTotalDuration(spec.getQuerySegmentSpec())); } return totalDuration; } Map getContext(); - ContextType getContextValue(String key); + default Map computeOverridenContext(Map overrides) + { + Map overridden = Maps.newTreeMap(); + final Map context = getContext(); + overridden.putAll(context); + overridden.putAll(overrides); - ContextType getContextValue(String key, ContextType defaultValue); + return overridden; + } - boolean getContextBoolean(String key, boolean defaultValue); + default ContextType getContextValue(String key) + { + return (ContextType) getContext().get(key); + } + + default ContextType getContextValue(String key, ContextType defaultValue) + { + ContextType retVal = getContextValue(key); + return retVal == null ? defaultValue : retVal; + } + + default boolean getContextBoolean(String key, boolean defaultValue) + { + return QueryContexts.parseBoolean(this, key, defaultValue); + } boolean isDescending(); Ordering getResultOrdering(); - String getId(); + default String getId() + { + return (String) getContextValue(QueryContexts.QUERYID); + } - Query withId(String id); + default Query withId(String id) + { + return withOverriddenContext(ImmutableMap.of(QueryContexts.QUERYID, id)); + } default DataSourceWithSegmentSpec getDistributionTarget() { - return getContextValue(QueryContextKeys.DISTRIBUTION_TARGET_SOURCE); + return getContextValue(QueryContexts.DISTRIBUTION_TARGET_SOURCE); } default Query distributeBy(DataSourceWithSegmentSpec spec) { - return withOverriddenContext(ImmutableMap.of(QueryContextKeys.DISTRIBUTION_TARGET_SOURCE, spec)); + return withOverriddenContext(ImmutableMap.of(QueryContexts.DISTRIBUTION_TARGET_SOURCE, spec)); } Query withOverriddenContext(Map contextOverride); @@ -121,4 +152,15 @@ default Query distributeBy(DataSourceWithSegmentSpec spec) Query withQuerySegmentSpec(String dataSource, QuerySegmentSpec spec); Query replaceDataSourceWith(DataSource src, DataSource dst); + + static Duration getTotalDuration(QuerySegmentSpec spec) + { + Duration totalDuration = new Duration(0); + for (Interval interval : spec.getIntervals()) { + if (interval != null) { + totalDuration = totalDuration.plus(interval.toDuration()); + } + } + return totalDuration; + } } diff --git a/processing/src/main/java/io/druid/query/QueryContextKeys.java b/processing/src/main/java/io/druid/query/QueryContexts.java similarity index 53% rename from processing/src/main/java/io/druid/query/QueryContextKeys.java rename to processing/src/main/java/io/druid/query/QueryContexts.java index 2774395f90f4..cfb8f19d16c2 100644 --- a/processing/src/main/java/io/druid/query/QueryContextKeys.java +++ b/processing/src/main/java/io/druid/query/QueryContexts.java @@ -19,10 +19,43 @@ package io.druid.query; -public class QueryContextKeys +import io.druid.java.util.common.ISE; + +public class QueryContexts { + public static final String QUERYID = "queryId"; public static final String PRIORITY = "priority"; public static final String TIMEOUT = "timeout"; public static final String CHUNK_PERIOD = "chunkPeriod"; public static final String DISTRIBUTION_TARGET_SOURCE = "distributionTargetSource"; + + static boolean parseBoolean(Query query, String key, boolean defaultValue) + { + Object val = query.getContextValue(key); + if (val == null) { + return defaultValue; + } + if (val instanceof String) { + return Boolean.parseBoolean((String) val); + } else if (val instanceof Boolean) { + return (boolean) val; + } else { + throw new ISE("Unknown type [%s]. Cannot parse!", val.getClass()); + } + } + + static int parseInt(Query query, String key, int defaultValue) + { + Object val = query.getContextValue(key); + if (val == null) { + return defaultValue; + } + if (val instanceof String) { + return Integer.parseInt((String) val); + } else if (val instanceof Integer) { + return (int) val; + } else { + throw new ISE("Unknown type [%s]", val.getClass()); + } + } } diff --git a/processing/src/main/java/io/druid/query/QueryDataSource.java b/processing/src/main/java/io/druid/query/QueryDataSource.java index 70cbc220d5cd..35702eb41ba1 100644 --- a/processing/src/main/java/io/druid/query/QueryDataSource.java +++ b/processing/src/main/java/io/druid/query/QueryDataSource.java @@ -47,7 +47,7 @@ public List getNames() } @JsonProperty - public Query getQuery() + public Query getQuery() { return query; } diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index 45e838d38f72..7a4d523a4979 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -148,7 +148,7 @@ public interface QueryMetrics> /** * Sets {@link Query#getDataSources()} of the given query as dimension. */ - void dataSources(QueryType query); + void dataSourcesAndIntervals(QueryType query); /** * Sets {@link Query#getType()} of the given query as dimension. diff --git a/processing/src/main/java/io/druid/query/SingleSourceBaseQuery.java b/processing/src/main/java/io/druid/query/SingleSourceBaseQuery.java deleted file mode 100644 index 3e8ba4f68a14..000000000000 --- a/processing/src/main/java/io/druid/query/SingleSourceBaseQuery.java +++ /dev/null @@ -1,200 +0,0 @@ -/* - * 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; - -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; -import io.druid.java.util.common.guava.Sequence; -import io.druid.query.spec.QuerySegmentSpec; -import org.joda.time.Duration; -import org.joda.time.Interval; - -import java.util.List; -import java.util.Map; -import java.util.Objects; - -public abstract class SingleSourceBaseQuery> extends BaseQuery -{ - private final DataSourceWithSegmentSpec dataSourceWithSegment; - private volatile Duration duration; - - public SingleSourceBaseQuery( - DataSource dataSource, - QuerySegmentSpec querySegmentSpec, - boolean descending, - Map context - ) - { - super(descending, context); - Objects.requireNonNull(dataSource, "dataSource can't be null"); - Objects.requireNonNull(querySegmentSpec, "querySegmentSpec can't be null"); - - this.dataSourceWithSegment = new DataSourceWithSegmentSpec(dataSource, querySegmentSpec); - } - - public abstract Query withQuerySegmentSpec(QuerySegmentSpec spec); - public abstract Query withDataSource(DataSource dataSource); - - @Override - public List getDataSources() - { - return ImmutableList.of(dataSourceWithSegment); - } - - public DataSourceWithSegmentSpec getDataSourceWithSegmentSpec() - { - return dataSourceWithSegment; - } - - @JsonProperty - public DataSource getDataSource() - { - return dataSourceWithSegment.getDataSource(); - } - - @JsonProperty("intervals") - public QuerySegmentSpec getQuerySegmentSpec() - { - return dataSourceWithSegment.getQuerySegmentSpec(); - } - - @Override - public Sequence run(QuerySegmentWalker walker, Map context) - { - return run(dataSourceWithSegment.getQuerySegmentSpec().lookup(this, walker), context); - } - - public List getIntervals() - { - return dataSourceWithSegment.getQuerySegmentSpec().getIntervals(); - } - - @Override - public Duration getDuration(DataSource dataSource) - { - Preconditions.checkArgument(this.dataSourceWithSegment.getDataSource().equals(dataSource)); - return getDuration(); - } - - public Duration getDuration() - { - if (duration == null) { - duration = getTotalDuration(dataSourceWithSegment.getQuerySegmentSpec()); - } - - return duration; - } - - @Override - public Query withQuerySegmentSpec(DataSource dataSource, QuerySegmentSpec spec) - { - Preconditions.checkArgument(this.dataSourceWithSegment.getDataSource().equals(dataSource)); - final Query query = withQuerySegmentSpec(spec); - if (getDistributionTarget() != null) { - if (dataSource.equals(getDistributionTarget().getDataSource())) { - return query.distributeBy(((SingleSourceBaseQuery) query).getDataSourceWithSegmentSpec()); - } - } - return query; - } - - @Override - public Query withQuerySegmentSpec(String dataSource, QuerySegmentSpec spec) - { - Preconditions.checkArgument(Iterables.getOnlyElement(this.dataSourceWithSegment.getDataSource().getNames()).equals(dataSource)); - final Query query = withQuerySegmentSpec(spec); - if (getDistributionTarget() != null) { - if (dataSource.equals(getDistributionTarget().getDataSource().getConcatenatedName())) { - return query.distributeBy(((SingleSourceBaseQuery) query).getDataSourceWithSegmentSpec()); - } - } - return query; - } - - @Override - public Query replaceDataSourceWith(DataSource src, DataSource dst) - { - Preconditions.checkArgument(this.dataSourceWithSegment.getDataSource().equals(src)); - return withDataSource(dst); - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - SingleSourceBaseQuery baseQuery = (SingleSourceBaseQuery) o; - - if (isDescending() != baseQuery.isDescending()) { - return false; - } - if (!getContext().equals(baseQuery.getContext())) { - return false; - } - if (!dataSourceWithSegment.equals(baseQuery.dataSourceWithSegment)) { - return false; - } - if (duration != null ? !duration.equals(baseQuery.duration) : baseQuery.duration != null) { - return false; - } - - return true; - } - - @Override - public int hashCode() - { - int result = dataSourceWithSegment.hashCode(); - result = 31 * result + (isDescending() ? 1 : 0); - result = 31 * result + getContext().hashCode(); - result = 31 * result + (duration != null ? duration.hashCode() : 0); - return result; - } - - public static > DataSourceWithSegmentSpec getLeafDataSourceWithSegmentSpec( - SingleSourceBaseQuery query - ) - { - final DataSourceWithSegmentSpec sourceWithSegmentSpec = query.getDataSourceWithSegmentSpec(); - if (sourceWithSegmentSpec.getDataSource() instanceof QueryDataSource) { - final QueryDataSource queryDataSource = (QueryDataSource) sourceWithSegmentSpec.getDataSource(); - return getLeafDataSourceWithSegmentSpec((SingleSourceBaseQuery) queryDataSource.getQuery()); - } else { - return sourceWithSegmentSpec; - } - } - - public static > DataSource getLeafDataSource(DataSource dataSource) - { - if (dataSource instanceof QueryDataSource) { - final QueryDataSource queryDataSource = (QueryDataSource) dataSource; - return getLeafDataSourceWithSegmentSpec((SingleSourceBaseQuery) queryDataSource.getQuery()).getDataSource(); - } else { - return dataSource; - } - } -} diff --git a/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java b/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java index 94c4bc817bfb..fc5c1b1c6752 100644 --- a/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java +++ b/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java @@ -27,7 +27,7 @@ /** * If there's a subquery, run it instead of the outer query */ -public class SubqueryQueryRunner implements QueryRunner +public class SubqueryQueryRunner> implements QueryRunner { private final QueryRunner baseRunner; @@ -39,8 +39,8 @@ public SubqueryQueryRunner(QueryRunner baseRunner) @Override public Sequence run(final Query query, Map responseContext) { - Preconditions.checkArgument(query instanceof SingleSourceBaseQuery); - final DataSource dataSource = ((SingleSourceBaseQuery)query).getDataSource(); + Preconditions.checkArgument(query instanceof BaseQuery); + final DataSource dataSource = ((BaseQuery)query).getDataSource(); if (dataSource instanceof QueryDataSource) { return run((Query) ((QueryDataSource) dataSource).getQuery(), responseContext); } else { diff --git a/processing/src/main/java/io/druid/query/UnionQueryRunner.java b/processing/src/main/java/io/druid/query/UnionQueryRunner.java index 79f1cee99b48..5eaa9c9ff7d9 100644 --- a/processing/src/main/java/io/druid/query/UnionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/UnionQueryRunner.java @@ -41,8 +41,8 @@ public UnionQueryRunner( @Override public Sequence run(final Query query, final Map responseContext) { - if (query instanceof SingleSourceBaseQuery) { - final SingleSourceBaseQuery singleSourceBaseQuery = (SingleSourceBaseQuery) query; + if (query instanceof BaseQuery) { + final BaseQuery singleSourceBaseQuery = (BaseQuery) query; final DataSource dataSource = singleSourceBaseQuery.getDataSource(); if (dataSource instanceof UnionDataSource) { diff --git a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceMetadataQuery.java b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceMetadataQuery.java index 356be1148a4e..186fcaf2f6ef 100644 --- a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceMetadataQuery.java +++ b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceMetadataQuery.java @@ -23,10 +23,10 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Lists; 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.Result; -import io.druid.query.SingleSourceBaseQuery; import io.druid.query.filter.DimFilter; import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.query.spec.QuerySegmentSpec; @@ -39,7 +39,7 @@ /** */ -public class DataSourceMetadataQuery extends SingleSourceBaseQuery> +public class DataSourceMetadataQuery extends BaseQuery> { public static final Interval MY_Y2K_INTERVAL = new Interval( JodaUtils.MIN_INSTANT, JodaUtils.MAX_INSTANT diff --git a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java index 25aa428b2e75..7ace9dc4f1f6 100644 --- a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java @@ -91,7 +91,7 @@ public QueryRunner> annotateDistributionTa return (query, responseContext) -> { final DataSourceMetadataQuery dataSourceMetadataQuery = (DataSourceMetadataQuery) query; return runner.run( - dataSourceMetadataQuery.distributeBy(dataSourceMetadataQuery.getDataSourceWithSegmentSpec()), + dataSourceMetadataQuery.updateDistributionTarget(), responseContext ); }; diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java index 428442fed446..c1ea94087e91 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java @@ -38,13 +38,13 @@ import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; +import io.druid.query.BaseQuery; import io.druid.query.DataSource; import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.Queries; import io.druid.query.Query; -import io.druid.query.QueryContextKeys; +import io.druid.query.QueryContexts; import io.druid.query.QueryDataSource; -import io.druid.query.SingleSourceBaseQuery; import io.druid.query.TableDataSource; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; @@ -73,7 +73,7 @@ /** */ -public class GroupByQuery extends SingleSourceBaseQuery +public class GroupByQuery extends BaseQuery { public final static String CTX_KEY_SORT_BY_DIMS_FIRST = "sortByDimsFirst"; @@ -663,7 +663,7 @@ public Builder updateDistributionTarget() context = new HashMap<>(); } context.put( - QueryContextKeys.DISTRIBUTION_TARGET_SOURCE, + QueryContexts.DISTRIBUTION_TARGET_SOURCE, new DataSourceWithSegmentSpec(getLeafDataSource(dataSource), querySegmentSpec) ); return this; diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index 7f20b975313b..4ab764c2b439 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -375,7 +375,7 @@ public QueryRunner annotateDistributionTarget(QueryRunner runner) return (query, responseContext) -> { final GroupByQuery groupByQuery = (GroupByQuery) query; return runner.run( - groupByQuery.distributeBy(groupByQuery.getDataSourceWithSegmentSpec()), + groupByQuery.updateDistributionTarget(), responseContext ); }; diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java index fb0fe1327b99..5544f6eb3fe5 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java @@ -48,7 +48,7 @@ import io.druid.query.BaseQuery; import io.druid.query.ChainedExecutionQueryRunner; import io.druid.query.Query; -import io.druid.query.QueryContextKeys; +import io.druid.query.QueryContexts; import io.druid.query.QueryInterruptedException; import io.druid.query.QueryRunner; import io.druid.query.QueryWatcher; @@ -145,7 +145,7 @@ public Sequence run(final Query queryParam, final Map // Figure out timeoutAt time now, so we can apply the timeout to both the mergeBufferPool.take and the actual // query processing together. - final Number queryTimeout = query.getContextValue(QueryContextKeys.TIMEOUT, null); + final Number queryTimeout = query.getContextValue(QueryContexts.TIMEOUT, null); final long timeoutAt = queryTimeout == null ? JodaUtils.MAX_INSTANT : System.currentTimeMillis() + queryTimeout.longValue(); diff --git a/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV2.java b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV2.java index b938f3f0e5ef..bc812ad8e64d 100644 --- a/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV2.java +++ b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV2.java @@ -48,7 +48,7 @@ import io.druid.query.InsufficientResourcesException; import io.druid.query.IntervalChunkingQueryRunnerDecorator; import io.druid.query.Query; -import io.druid.query.QueryContextKeys; +import io.druid.query.QueryContexts; import io.druid.query.QueryDataSource; import io.druid.query.QueryRunner; import io.druid.query.QueryWatcher; @@ -142,7 +142,7 @@ public GroupByQueryResource prepareResource(GroupByQuery query, boolean willMerg } else if (requiredMergeBufferNum == 0) { return new GroupByQueryResource(); } else { - final Number timeout = query.getContextValue(QueryContextKeys.TIMEOUT, JodaUtils.MAX_INSTANT); + final Number timeout = query.getContextValue(QueryContexts.TIMEOUT, JodaUtils.MAX_INSTANT); final ResourceHolder> mergeBufferHolders = mergeBufferPool.takeBatch( requiredMergeBufferNum, timeout.longValue() ); diff --git a/processing/src/main/java/io/druid/query/join/DataInput.java b/processing/src/main/java/io/druid/query/join/DataInput.java index 6f747dba4d34..0762c8ee5a1d 100644 --- a/processing/src/main/java/io/druid/query/join/DataInput.java +++ b/processing/src/main/java/io/druid/query/join/DataInput.java @@ -22,8 +22,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Iterables; -import io.druid.query.BaseQuery; import io.druid.query.DataSource; +import io.druid.query.Query; import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.Duration; @@ -76,7 +76,7 @@ public QuerySegmentSpec getQuerySegmentSpec() public Duration getDuration() { if (duration == null) { - this.duration = BaseQuery.getTotalDuration(querySegmentSpec); + this.duration = Query.getTotalDuration(querySegmentSpec); } return duration; } diff --git a/processing/src/main/java/io/druid/query/join/JoinQuery.java b/processing/src/main/java/io/druid/query/join/JoinQuery.java index cdb356d69450..a9642b51b5bb 100644 --- a/processing/src/main/java/io/druid/query/join/JoinQuery.java +++ b/processing/src/main/java/io/druid/query/join/JoinQuery.java @@ -25,12 +25,10 @@ import com.google.common.collect.ImmutableList; import io.druid.data.input.Row; import io.druid.java.util.common.granularity.Granularity; -import io.druid.java.util.common.guava.Sequence; -import io.druid.query.BaseQuery; import io.druid.query.DataSource; import io.druid.query.DataSourceWithSegmentSpec; +import io.druid.query.MultiSourceBaseQuery; import io.druid.query.Query; -import io.druid.query.QuerySegmentWalker; import io.druid.query.dimension.DimensionSpec; import io.druid.query.filter.DimFilter; import io.druid.query.spec.QuerySegmentSpec; @@ -43,7 +41,7 @@ import java.util.Map; import java.util.Objects; -public class JoinQuery extends BaseQuery +public class JoinQuery extends MultiSourceBaseQuery { private final JoinSpec joinSpec; private final Granularity granularity; @@ -115,20 +113,12 @@ public String getType() return Query.JOIN; } - @Override - public Sequence run( - QuerySegmentWalker walker, Map context - ) - { - return run(getDistributionTarget().getQuerySegmentSpec().lookup(this, walker), context); - } - @Override public Duration getDuration(DataSource dataSource) { for (DataSourceWithSegmentSpec sourceWithSegmentSpec : getDataSources()) { if (sourceWithSegmentSpec.getDataSource().equals(dataSource)) { - return getTotalDuration(sourceWithSegmentSpec.getQuerySegmentSpec()); + return Query.getTotalDuration(sourceWithSegmentSpec.getQuerySegmentSpec()); } } return null; diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index f975505f0dc3..8d7a36173826 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -267,7 +267,7 @@ public QueryRunner annotateDistributionTarget(QueryRunner { final SegmentMetadataQuery segmentMetadataQuery = (SegmentMetadataQuery) query; return runner.run( - segmentMetadataQuery.distributeBy(segmentMetadataQuery.getDataSourceWithSegmentSpec()), + segmentMetadataQuery.updateDistributionTarget(), responseContext ); }; diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java index 5620219f8b0a..6ce181495874 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java @@ -35,7 +35,7 @@ import io.druid.query.BaseQuery; import io.druid.query.ConcatQueryRunner; import io.druid.query.Query; -import io.druid.query.QueryContextKeys; +import io.druid.query.QueryContexts; import io.druid.query.QueryInterruptedException; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; @@ -217,7 +217,7 @@ public Sequence call() throws Exception ); try { queryWatcher.registerQuery(query, future); - final Number timeout = query.getContextValue(QueryContextKeys.TIMEOUT, (Number) null); + final Number timeout = query.getContextValue(QueryContexts.TIMEOUT, (Number) null); return timeout == null ? future.get() : future.get(timeout.longValue(), TimeUnit.MILLISECONDS); } catch (InterruptedException e) { diff --git a/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java b/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java index 920ba7c605c1..3d9ab5b117ed 100644 --- a/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java +++ b/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java @@ -25,9 +25,9 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; 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.SingleSourceBaseQuery; import io.druid.query.TableDataSource; import io.druid.query.UnionDataSource; import io.druid.query.filter.DimFilter; @@ -42,7 +42,7 @@ import java.util.Map; import java.util.Objects; -public class SegmentMetadataQuery extends SingleSourceBaseQuery +public class SegmentMetadataQuery extends BaseQuery { /* The SegmentMetadataQuery cache key may contain UTF-8 column name strings. * Prepend 0xFF before the analysisTypes as a separator to avoid diff --git a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java index 437b193f2637..eb3655ad0c81 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -368,7 +368,7 @@ public QueryRunner> annotateDistributionTarget(QueryRu return (query, responseContext) -> { final SearchQuery searchQuery = (SearchQuery) query; return runner.run( - searchQuery.distributeBy(searchQuery.getDataSourceWithSegmentSpec()), + searchQuery.updateDistributionTarget(), responseContext ); }; diff --git a/processing/src/main/java/io/druid/query/search/search/SearchQuery.java b/processing/src/main/java/io/druid/query/search/search/SearchQuery.java index 95d4f53d0b24..c45a21cddb0d 100644 --- a/processing/src/main/java/io/druid/query/search/search/SearchQuery.java +++ b/processing/src/main/java/io/druid/query/search/search/SearchQuery.java @@ -24,10 +24,10 @@ import com.google.common.base.Preconditions; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularity; +import io.druid.query.BaseQuery; import io.druid.query.DataSource; import io.druid.query.Query; import io.druid.query.Result; -import io.druid.query.SingleSourceBaseQuery; import io.druid.query.dimension.DimensionSpec; import io.druid.query.filter.DimFilter; import io.druid.query.ordering.StringComparators; @@ -39,7 +39,7 @@ /** */ -public class SearchQuery extends SingleSourceBaseQuery> +public class SearchQuery extends BaseQuery> { private static final SearchSortSpec DEFAULT_SORT_SPEC = new SearchSortSpec(StringComparators.LEXICOGRAPHIC); diff --git a/processing/src/main/java/io/druid/query/select/SelectQuery.java b/processing/src/main/java/io/druid/query/select/SelectQuery.java index 4f66b5d06968..4c0154857c6a 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQuery.java +++ b/processing/src/main/java/io/druid/query/select/SelectQuery.java @@ -24,10 +24,10 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; import io.druid.java.util.common.granularity.Granularity; +import io.druid.query.BaseQuery; import io.druid.query.DataSource; import io.druid.query.Query; import io.druid.query.Result; -import io.druid.query.SingleSourceBaseQuery; import io.druid.query.dimension.DimensionSpec; import io.druid.query.filter.DimFilter; import io.druid.query.spec.QuerySegmentSpec; @@ -40,7 +40,7 @@ /** */ @JsonTypeName("select") -public class SelectQuery extends SingleSourceBaseQuery> +public class SelectQuery extends BaseQuery> { private final DimFilter dimFilter; private final Granularity granularity; diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java index 2fce7e37b0d5..0b498ed3890c 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java @@ -442,7 +442,7 @@ public QueryRunner> annotateDistributionTarget(QueryRu return (query, responseContext) -> { final SelectQuery selectQuery = (SelectQuery) query; return runner.run( - selectQuery.distributeBy(selectQuery.getDataSourceWithSegmentSpec()), + selectQuery.updateDistributionTarget(), responseContext ); }; diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java index df96922da248..4501e40c5602 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java @@ -25,10 +25,10 @@ import com.google.common.collect.Maps; import io.druid.common.utils.JodaUtils; import io.druid.java.util.common.StringUtils; +import io.druid.query.BaseQuery; import io.druid.query.DataSource; import io.druid.query.Query; import io.druid.query.Result; -import io.druid.query.SingleSourceBaseQuery; import io.druid.query.filter.DimFilter; import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.query.spec.QuerySegmentSpec; @@ -42,7 +42,7 @@ /** */ -public class TimeBoundaryQuery extends SingleSourceBaseQuery> +public class TimeBoundaryQuery extends BaseQuery> { public static final Interval MY_Y2K_INTERVAL = new Interval( new DateTime("0000-01-01"), diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 9f7c66c8259e..4a7fb5c9a6c2 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -32,9 +32,9 @@ import io.druid.query.BySegmentSkippingQueryRunner; import io.druid.query.CacheStrategy; import io.druid.query.DefaultGenericQueryMetricsFactory; +import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.Query; import io.druid.query.QueryMetrics; -import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.Result; @@ -106,7 +106,7 @@ public QueryRunner> annotateDistributionTarget(Q return (query, responseContext) -> { final TimeBoundaryQuery timeBoundaryQuery = (TimeBoundaryQuery) query; return runner.run( - timeBoundaryQuery.distributeBy(timeBoundaryQuery.getDataSourceWithSegmentSpec()), + timeBoundaryQuery.updateDistributionTarget(), responseContext ); }; diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java index a3c82d6470a5..e8165273d3a5 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java @@ -24,11 +24,11 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.collect.ImmutableList; import io.druid.java.util.common.granularity.Granularity; +import io.druid.query.BaseQuery; import io.druid.query.DataSource; import io.druid.query.Queries; import io.druid.query.Query; import io.druid.query.Result; -import io.druid.query.SingleSourceBaseQuery; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.filter.DimFilter; @@ -42,7 +42,7 @@ /** */ @JsonTypeName("timeseries") -public class TimeseriesQuery extends SingleSourceBaseQuery> +public class TimeseriesQuery extends BaseQuery> { private final VirtualColumns virtualColumns; private final DimFilter dimFilter; diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index bd188a1cad61..66b990fb671e 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -252,7 +252,7 @@ public QueryRunner> annotateDistributionTarget(Que return (query, responseContext) -> { final TimeseriesQuery timeseriesQuery = (TimeseriesQuery) query; return runner.run( - timeseriesQuery.distributeBy(timeseriesQuery.getDataSourceWithSegmentSpec()), + timeseriesQuery.updateDistributionTarget(), responseContext ); }; diff --git a/processing/src/main/java/io/druid/query/topn/TopNQuery.java b/processing/src/main/java/io/druid/query/topn/TopNQuery.java index 230231650562..29ca0cddcba5 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQuery.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQuery.java @@ -24,11 +24,11 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import io.druid.java.util.common.granularity.Granularity; +import io.druid.query.BaseQuery; import io.druid.query.DataSource; import io.druid.query.Queries; import io.druid.query.Query; import io.druid.query.Result; -import io.druid.query.SingleSourceBaseQuery; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.dimension.DimensionSpec; @@ -42,7 +42,7 @@ /** */ -public class TopNQuery extends SingleSourceBaseQuery> +public class TopNQuery extends BaseQuery> { public static final String TOPN = "topN"; diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java b/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java index 953cc33f390a..0016bd62064e 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java @@ -22,10 +22,10 @@ import com.google.common.collect.Lists; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularity; +import io.druid.query.BaseQuery; import io.druid.query.DataSource; import io.druid.query.DataSourceWithSegmentSpec; -import io.druid.query.QueryContextKeys; -import io.druid.query.SingleSourceBaseQuery; +import io.druid.query.QueryContexts; import io.druid.query.TableDataSource; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; @@ -217,9 +217,9 @@ public TopNQueryBuilder updateDistributionTarget() context = new HashMap<>(); } context.put( - QueryContextKeys.DISTRIBUTION_TARGET_SOURCE, + QueryContexts.DISTRIBUTION_TARGET_SOURCE, new DataSourceWithSegmentSpec( - SingleSourceBaseQuery.getLeafDataSource(dataSource), + BaseQuery.getLeafDataSource(dataSource), querySegmentSpec ) ); diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java index 626888dc3d84..00816c5e9268 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java @@ -511,7 +511,7 @@ public QueryRunner> annotateDistributionTarget(QueryRunn return (query, responseContext) -> { final TopNQuery topNQuery = (TopNQuery) query; return runner.run( - topNQuery.distributeBy(topNQuery.getDataSourceWithSegmentSpec()), + topNQuery.updateDistributionTarget(), responseContext ); }; diff --git a/processing/src/test/java/io/druid/query/AsyncQueryRunnerTest.java b/processing/src/test/java/io/druid/query/AsyncQueryRunnerTest.java index 1e3d931b5caf..cbd64f5fb4d8 100644 --- a/processing/src/test/java/io/druid/query/AsyncQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/AsyncQueryRunnerTest.java @@ -100,7 +100,7 @@ public Sequence run(Query query, Map responseContext) QueryRunnerTestHelper.NOOP_QUERYWATCHER); Sequence lazy = asyncRunner.run( - query.withOverriddenContext(ImmutableMap.of(QueryContextKeys.TIMEOUT, 1)), + query.withOverriddenContext(ImmutableMap.of(QueryContexts.TIMEOUT, 1)), Collections.EMPTY_MAP); try { diff --git a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java index 67918aef4819..8935745f351d 100644 --- a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java @@ -251,7 +251,7 @@ public Void answer() throws Throwable .dataSource("test") .intervals("2014/2015") .aggregators(Lists.newArrayList(new CountAggregatorFactory("count"))) - .context(ImmutableMap.of(QueryContextKeys.TIMEOUT, 100, "queryId", "test")) + .context(ImmutableMap.of(QueryContexts.TIMEOUT, 100, "queryId", "test")) .build(), context ); diff --git a/processing/src/test/java/io/druid/query/IntervalChunkingQueryRunnerTest.java b/processing/src/test/java/io/druid/query/IntervalChunkingQueryRunnerTest.java index acb8a762200e..f72502bb8848 100644 --- a/processing/src/test/java/io/druid/query/IntervalChunkingQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/IntervalChunkingQueryRunnerTest.java @@ -75,7 +75,7 @@ public void testDefaultNoChunking() { @Test public void testChunking() { Query query = queryBuilder.intervals("2015-01-01T00:00:00.000/2015-01-11T00:00:00.000").context(ImmutableMap.of("chunkPeriod", "P1D")).build(); - query = query.distributeBy(((TimeseriesQuery) query).getDataSourceWithSegmentSpec()); + query = ((TimeseriesQuery)query).updateDistributionTarget(); executors.execute(EasyMock.anyObject(Runnable.class)); EasyMock.expectLastCall().times(10); @@ -92,7 +92,7 @@ public void testChunking() { @Test public void testChunkingOnMonths() { Query query = queryBuilder.intervals("2015-01-01T00:00:00.000/2015-02-11T00:00:00.000").context(ImmutableMap.of("chunkPeriod", "P1M")).build(); - query = query.distributeBy(((TimeseriesQuery) query).getDataSourceWithSegmentSpec()); + query = ((TimeseriesQuery)query).updateDistributionTarget(); executors.execute(EasyMock.anyObject(Runnable.class)); EasyMock.expectLastCall().times(2); diff --git a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java index ab7a6e129c47..65a40c49fe90 100644 --- a/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/RetryQueryRunnerTest.java @@ -376,7 +376,7 @@ public Sequence> run( // this is first retry Assert.assertTrue( "Should retry with 2 missing segments", - ((MultipleSpecificSegmentSpec)((SingleSourceBaseQuery)query).getQuerySegmentSpec()).getDescriptors().size() == 2 + ((MultipleSpecificSegmentSpec)((BaseQuery)query).getQuerySegmentSpec()).getDescriptors().size() == 2 ); // assume only left 1 missing at first retry ((Map) context.get(Result.MISSING_SEGMENTS_KEY)).put( @@ -405,7 +405,7 @@ public Sequence> run( // this is second retry Assert.assertTrue( "Should retry with 1 missing segments", - ((MultipleSpecificSegmentSpec)((SingleSourceBaseQuery)query).getQuerySegmentSpec()).getDescriptors().size() == 1 + ((MultipleSpecificSegmentSpec)((BaseQuery)query).getQuerySegmentSpec()).getDescriptors().size() == 1 ); // assume no more missing at second retry context.put("count", 3); diff --git a/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java b/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java index c209cbd6ca79..89895cca72d9 100644 --- a/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java +++ b/processing/src/test/java/io/druid/query/TimewarpOperatorTest.java @@ -97,7 +97,7 @@ public Sequence> run( new TimeseriesResultValue(ImmutableMap.of("metric", 3)) ), new Result<>( - ((SingleSourceBaseQuery>)query).getIntervals().get(0).getEnd(), + ((BaseQuery>)query).getIntervals().get(0).getEnd(), new TimeseriesResultValue(ImmutableMap.of("metric", 5)) ) ) @@ -203,11 +203,11 @@ public Sequence> run( return Sequences.simple( ImmutableList.of( new Result<>( - ((SingleSourceBaseQuery>)query).getIntervals().get(0).getStart(), + ((BaseQuery>)query).getIntervals().get(0).getStart(), new TimeseriesResultValue(ImmutableMap.of("metric", 2)) ), new Result<>( - ((SingleSourceBaseQuery>)query).getIntervals().get(0).getEnd(), + ((BaseQuery>)query).getIntervals().get(0).getEnd(), new TimeseriesResultValue(ImmutableMap.of("metric", 3)) ) ) @@ -242,12 +242,13 @@ public Sequence> run( private static Query setDistributionTarget(Query query) { + final DataSourceWithSegmentSpec sourceWithSegmentSpec = Iterables.getOnlyElement(query.getDataSources()); return query.withOverriddenContext( ImmutableMap.of( - QueryContextKeys.DISTRIBUTION_TARGET_SOURCE, + QueryContexts.DISTRIBUTION_TARGET_SOURCE, new DataSourceWithSegmentSpec( - Iterables.getOnlyElement(query.getDataSources()).getDataSource(), - Iterables.getOnlyElement(query.getDataSources()).getQuerySegmentSpec() + sourceWithSegmentSpec.getDataSource(), + sourceWithSegmentSpec.getQuerySegmentSpec() ) ) ); diff --git a/processing/src/test/java/io/druid/query/UnionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/UnionQueryRunnerTest.java index 0f396af83dde..f64682a10028 100644 --- a/processing/src/test/java/io/druid/query/UnionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/UnionQueryRunnerTest.java @@ -42,8 +42,8 @@ public void testUnionQueryRunner() public Sequence run(Query query, Map responseContext) { // verify that table datasource is passed to baseQueryRunner - Assert.assertTrue(((SingleSourceBaseQuery)query).getDataSource() instanceof TableDataSource); - String dsName = Iterables.getOnlyElement(((SingleSourceBaseQuery)query).getDataSource().getNames()); + Assert.assertTrue(((BaseQuery)query).getDataSource() instanceof TableDataSource); + String dsName = Iterables.getOnlyElement(((BaseQuery)query).getDataSource().getNames()); if (dsName.equals("ds1")) { responseContext.put("ds1", "ds1"); return Sequences.simple(Arrays.asList(1, 2, 3)); diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java index 460c9d8c8519..083813df5029 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java @@ -32,7 +32,7 @@ import io.druid.data.input.Row; import io.druid.java.util.common.granularity.Granularities; import io.druid.query.DruidProcessingConfig; -import io.druid.query.QueryContextKeys; +import io.druid.query.QueryContexts; import io.druid.query.QueryDataSource; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; @@ -230,7 +230,7 @@ public void testSimpleGroupBy() .setGranularity(Granularities.ALL) .setInterval(QueryRunnerTestHelper.firstToThird) .setAggregatorSpecs(Lists.newArrayList(new LongSumAggregatorFactory("rows", "rows"))) - .setContext(ImmutableMap.of(QueryContextKeys.TIMEOUT, Integers.valueOf(500))) + .setContext(ImmutableMap.of(QueryContexts.TIMEOUT, Integers.valueOf(500))) .build(); GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); @@ -258,7 +258,7 @@ public void testNestedGroupBy() .setGranularity(Granularities.ALL) .setInterval(QueryRunnerTestHelper.firstToThird) .setAggregatorSpecs(Lists.newArrayList(new LongSumAggregatorFactory("rows", "rows"))) - .setContext(ImmutableMap.of(QueryContextKeys.TIMEOUT, Integers.valueOf(500))) + .setContext(ImmutableMap.of(QueryContexts.TIMEOUT, Integers.valueOf(500))) .build(); GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); @@ -297,7 +297,7 @@ public void testDoubleNestedGroupBy() .setGranularity(Granularities.ALL) .setInterval(QueryRunnerTestHelper.firstToThird) .setAggregatorSpecs(Lists.newArrayList(new LongSumAggregatorFactory("rows", "rows"))) - .setContext(ImmutableMap.of(QueryContextKeys.TIMEOUT, Integers.valueOf(500))) + .setContext(ImmutableMap.of(QueryContexts.TIMEOUT, Integers.valueOf(500))) .build(); GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); @@ -349,7 +349,7 @@ public void testTripleNestedGroupBy() .setGranularity(Granularities.ALL) .setInterval(QueryRunnerTestHelper.firstToThird) .setAggregatorSpecs(Lists.newArrayList(new LongSumAggregatorFactory("rows", "rows"))) - .setContext(ImmutableMap.of(QueryContextKeys.TIMEOUT, Integers.valueOf(500))) + .setContext(ImmutableMap.of(QueryContexts.TIMEOUT, Integers.valueOf(500))) .build(); GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java index a2f6f6b8d770..7e1f69b27ed9 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java @@ -33,7 +33,7 @@ import io.druid.java.util.common.granularity.Granularities; import io.druid.query.DruidProcessingConfig; import io.druid.query.InsufficientResourcesException; -import io.druid.query.QueryContextKeys; +import io.druid.query.QueryContexts; import io.druid.query.QueryDataSource; import io.druid.query.QueryInterruptedException; import io.druid.query.QueryRunner; @@ -201,7 +201,7 @@ public void testNotEnoughMergeBuffersOnQueryable() throws IOException .setGranularity(Granularities.ALL) .setInterval(QueryRunnerTestHelper.firstToThird) .setAggregatorSpecs(Lists.newArrayList(new LongSumAggregatorFactory("rows", "rows"))) - .setContext(ImmutableMap.of(QueryContextKeys.TIMEOUT, Integers.valueOf(500))) + .setContext(ImmutableMap.of(QueryContexts.TIMEOUT, Integers.valueOf(500))) .build(); GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); @@ -239,7 +239,7 @@ public void testResourceLimitExceededOnBroker() .setGranularity(Granularities.ALL) .setInterval(QueryRunnerTestHelper.firstToThird) .setAggregatorSpecs(Lists.newArrayList(new LongSumAggregatorFactory("rows", "rows"))) - .setContext(ImmutableMap.of(QueryContextKeys.TIMEOUT, Integers.valueOf(500))) + .setContext(ImmutableMap.of(QueryContexts.TIMEOUT, Integers.valueOf(500))) .build(); GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); @@ -265,7 +265,7 @@ public void testInsufficientResourcesOnBroker() throws IOException .setGranularity(Granularities.ALL) .setInterval(QueryRunnerTestHelper.firstToThird) .setAggregatorSpecs(Lists.newArrayList(new LongSumAggregatorFactory("rows", "rows"))) - .setContext(ImmutableMap.of(QueryContextKeys.TIMEOUT, Integers.valueOf(500))) + .setContext(ImmutableMap.of(QueryContexts.TIMEOUT, Integers.valueOf(500))) .build(); try { diff --git a/server/src/main/java/io/druid/server/BrokerQueryResource.java b/server/src/main/java/io/druid/server/BrokerQueryResource.java index 145a63f9a927..cda9a3d0bfce 100644 --- a/server/src/main/java/io/druid/server/BrokerQueryResource.java +++ b/server/src/main/java/io/druid/server/BrokerQueryResource.java @@ -31,7 +31,7 @@ import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.Query; -import io.druid.query.QueryContextKeys; +import io.druid.query.QueryContexts; import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChestWarehouse; import io.druid.server.http.security.StateResourceFilter; @@ -106,7 +106,7 @@ public Response getQueryTargets( Query query = context.getObjectMapper().readValue(in, Query.class); final DataSourceWithSegmentSpec spec = (DataSourceWithSegmentSpec) query.getContext().computeIfAbsent( - QueryContextKeys.DISTRIBUTION_TARGET_SOURCE, + QueryContexts.DISTRIBUTION_TARGET_SOURCE, key -> query.getDataSources().get(0) ); return context.ok( diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 64ecefb9e3bb..fc8ecf9e339c 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -41,7 +41,7 @@ import io.druid.query.DruidMetrics; import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.Query; -import io.druid.query.QueryContextKeys; +import io.druid.query.QueryContexts; import io.druid.query.QueryInterruptedException; import io.druid.query.QueryMetrics; import io.druid.query.QuerySegmentWalker; @@ -192,10 +192,10 @@ public Response doPost( queryId = UUID.randomUUID().toString(); query = query.withId(queryId); } - if (query.getContextValue(QueryContextKeys.TIMEOUT) == null) { + if (query.getContextValue(QueryContexts.TIMEOUT) == null) { query = query.withOverriddenContext( ImmutableMap.of( - QueryContextKeys.TIMEOUT, + QueryContexts.TIMEOUT, config.getMaxIdleTime().toStandardDuration().getMillis() ) ); diff --git a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java index b960363655f0..7b6eef2471a5 100644 --- a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java +++ b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java @@ -426,7 +426,7 @@ private Future assertQueryable( .query("wow") .build(); final QueryRunner> runner = serverManager.getQueryRunnerForIntervals( - query.distributeBy(query.getDataSourceWithSegmentSpec()), + query.updateDistributionTarget(), intervals ); return serverManagerExec.submit( diff --git a/server/src/test/java/io/druid/server/log/LoggingRequestLoggerTest.java b/server/src/test/java/io/druid/server/log/LoggingRequestLoggerTest.java index 410ead41c0c8..437bc60db78d 100644 --- a/server/src/test/java/io/druid/server/log/LoggingRequestLoggerTest.java +++ b/server/src/test/java/io/druid/server/log/LoggingRequestLoggerTest.java @@ -25,12 +25,12 @@ import com.google.common.base.Charsets; import com.google.common.collect.ImmutableMap; import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.BaseQuery; import io.druid.query.DataSource; import io.druid.query.LegacyDataSource; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QuerySegmentWalker; -import io.druid.query.SingleSourceBaseQuery; import io.druid.query.filter.DimFilter; import io.druid.query.spec.QuerySegmentSpec; import io.druid.server.QueryStats; @@ -179,7 +179,7 @@ private static Map readContextMap(byte[] bytes) throws Exception } @JsonTypeName("fake") -class FakeQuery extends SingleSourceBaseQuery +class FakeQuery extends BaseQuery { public FakeQuery(DataSource dataSource, QuerySegmentSpec querySegmentSpec, boolean descending, Map context) { diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java b/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java index 20fdddb813dd..f4e038189644 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java @@ -30,11 +30,12 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; +import io.druid.query.BaseQuery; import io.druid.query.DataSource; +import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.QueryDataSource; import io.druid.query.QuerySegmentWalker; import io.druid.query.Result; -import io.druid.query.SingleSourceBaseQuery; import io.druid.query.dimension.DimensionSpec; import io.druid.query.groupby.GroupByQuery; import io.druid.query.select.EventHolder; @@ -152,7 +153,7 @@ private Sequence executeSelect( ) { final SelectQuery running = (SelectQuery) baseQuery.distributeBy( - SingleSourceBaseQuery.getLeafDataSourceWithSegmentSpec(baseQuery) + new DataSourceWithSegmentSpec(BaseQuery.getLeafDataSource(baseQuery), baseQuery.getQuerySegmentSpec()) ); Preconditions.checkState(queryBuilder.getGrouping() == null, "grouping must be null"); @@ -262,7 +263,7 @@ private Sequence executeTimeseries( ) { final TimeseriesQuery running = (TimeseriesQuery) query.distributeBy( - SingleSourceBaseQuery.getLeafDataSourceWithSegmentSpec(query) + new DataSourceWithSegmentSpec(BaseQuery.getLeafDataSource(query), query.getQuerySegmentSpec()) ); final List fieldList = queryBuilder.getRowType().getFieldList(); final List dimensions = queryBuilder.getGrouping().getDimensions(); @@ -301,7 +302,7 @@ private Sequence executeTopN( ) { final TopNQuery running = (TopNQuery) query.distributeBy( - SingleSourceBaseQuery.getLeafDataSourceWithSegmentSpec(query) + new DataSourceWithSegmentSpec(BaseQuery.getLeafDataSource(query), query.getQuerySegmentSpec()) ); final List fieldList = queryBuilder.getRowType().getFieldList(); @@ -342,7 +343,7 @@ private Sequence executeGroupBy( { final List fieldList = queryBuilder.getRowType().getFieldList(); final GroupByQuery running = (GroupByQuery) query.distributeBy( - SingleSourceBaseQuery.getLeafDataSourceWithSegmentSpec(query) + new DataSourceWithSegmentSpec(BaseQuery.getLeafDataSource(query), query.getQuerySegmentSpec()) ); Hook.QUERY_PLAN.run(running); diff --git a/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java b/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java index 9bf86e5bfb46..8055a2aaa004 100644 --- a/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java +++ b/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java @@ -306,7 +306,7 @@ private DruidTable computeTable(final String dataSource) ); final Sequence sequence = segmentMetadataQuery - .distributeBy(segmentMetadataQuery.getDataSourceWithSegmentSpec()) + .updateDistributionTarget() .run(walker, Maps.newHashMap()); final List results = Sequences.toList(sequence, Lists.newArrayList()); if (results.isEmpty()) { From 2e8970f09af077a0a1fc6ec968cc1771114a9350 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 18 Apr 2017 14:51:55 +0900 Subject: [PATCH 07/15] Fix test fail --- .../main/java/io/druid/query/BaseQuery.java | 29 +++++++------------ .../src/main/java/io/druid/query/Query.java | 2 +- .../java/io/druid/query/join/JoinQuery.java | 4 +-- .../segment/realtime/RealtimeManager.java | 4 +-- 4 files changed, 16 insertions(+), 23 deletions(-) diff --git a/processing/src/main/java/io/druid/query/BaseQuery.java b/processing/src/main/java/io/druid/query/BaseQuery.java index 83b1d1243dd5..006e91619740 100644 --- a/processing/src/main/java/io/druid/query/BaseQuery.java +++ b/processing/src/main/java/io/druid/query/BaseQuery.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import io.druid.java.util.common.guava.Sequence; @@ -212,26 +211,26 @@ public int hashCode() public Query updateDistributionTarget() { - return withOverriddenContext( - ImmutableMap.of( - QueryContexts.DISTRIBUTION_TARGET_SOURCE, - new DataSourceWithSegmentSpec(dataSource, querySegmentSpec) - ) - ); + return distributeBy(new DataSourceWithSegmentSpec(BaseQuery.getLeafDataSource(dataSource), querySegmentSpec)); } @Override public Query withQuerySegmentSpec(DataSource dataSource, QuerySegmentSpec spec) { Preconditions.checkArgument(this.dataSource.equals(dataSource)); - return withQuerySegmentSpec(spec); + final BaseQuery result = (BaseQuery) withQuerySegmentSpec(spec); + if (getDistributionTarget() != null && getDistributionTarget().getDataSource().equals(dataSource)) { + return result.updateDistributionTarget(); + } else { + return result; + } } @Override - public Query withQuerySegmentSpec(String dataSource, QuerySegmentSpec spec) + public Query withQuerySegmentSpec(String firstDataSourceName, QuerySegmentSpec spec) { - Preconditions.checkArgument(this.dataSource.getFirstName().equals(dataSource)); - return withQuerySegmentSpec(spec); + Preconditions.checkArgument(this.dataSource.getFirstName().equals(firstDataSourceName)); + return withQuerySegmentSpec(this.dataSource, spec); } @Override @@ -248,13 +247,7 @@ public static > DataSource getLeafDataSource( BaseQuery query ) { - final DataSource dataSource = query.getDataSource(); - if (dataSource instanceof QueryDataSource) { - final QueryDataSource queryDataSource = (QueryDataSource) dataSource; - return getLeafDataSource((BaseQuery) queryDataSource.getQuery()); - } else { - return dataSource; - } + return getLeafDataSource(query.getDataSource()); } public static DataSource getLeafDataSource(DataSource dataSource) diff --git a/processing/src/main/java/io/druid/query/Query.java b/processing/src/main/java/io/druid/query/Query.java index f8c17870798e..fe2f3404b6a2 100644 --- a/processing/src/main/java/io/druid/query/Query.java +++ b/processing/src/main/java/io/druid/query/Query.java @@ -149,7 +149,7 @@ default Query distributeBy(DataSourceWithSegmentSpec spec) Query withQuerySegmentSpec(DataSource dataSource, QuerySegmentSpec spec); - Query withQuerySegmentSpec(String dataSource, QuerySegmentSpec spec); + Query withQuerySegmentSpec(String firstDataSourceName, QuerySegmentSpec spec); Query replaceDataSourceWith(DataSource src, DataSource dst); diff --git a/processing/src/main/java/io/druid/query/join/JoinQuery.java b/processing/src/main/java/io/druid/query/join/JoinQuery.java index a9642b51b5bb..2fab9acbb48a 100644 --- a/processing/src/main/java/io/druid/query/join/JoinQuery.java +++ b/processing/src/main/java/io/druid/query/join/JoinQuery.java @@ -175,7 +175,7 @@ public Query withQuerySegmentSpec(DataSource dataSource, QuerySegmentSpec s } @Override - public Query withQuerySegmentSpec(String dataSource, QuerySegmentSpec spec) + public Query withQuerySegmentSpec(String firstDataSourceName, QuerySegmentSpec spec) { final JoinSpecVisitor visitor = new JoinSpecVisitor() { @@ -190,7 +190,7 @@ public JoinSpec visit(JoinSpec joinSpec) @Override public DataInput visit(DataInput dataInput) { - if (dataInput.getDataSource().getFirstName().equals(dataSource)) { + if (dataInput.getDataSource().getFirstName().equals(firstDataSourceName)) { return new DataInput(dataInput.getDataSource(), spec); } else { return dataInput; diff --git a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java index 8fb61a8d1cf8..2e906e2f63de 100644 --- a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java +++ b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java @@ -179,8 +179,8 @@ public QueryRunner apply(FireChief fireChief) public QueryRunner getQueryRunnerForSegments(final Query query, final Iterable specs) { final QueryRunnerFactory> factory = conglomerate.findFactory(query); - final DataSourceWithSegmentSpec spec = query.getDistributionTarget(); - final String dataSourceName = Iterables.getOnlyElement(spec.getDataSource().getNames()); + final DataSourceWithSegmentSpec sourceWithSegmentSpec = query.getDistributionTarget(); + final String dataSourceName = Iterables.getOnlyElement(sourceWithSegmentSpec.getDataSource().getNames()); final Map partitionChiefs = chiefs.get(dataSourceName); return partitionChiefs == null From f9502213fb20ab7848b49f4ccc58f87f52e3c881 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 18 Apr 2017 20:09:06 +0900 Subject: [PATCH 08/15] fix compilation error --- .../src/main/java/io/druid/query/Queries.java | 13 +++++++++++++ processing/src/main/java/io/druid/query/Query.java | 14 +------------- .../main/java/io/druid/query/join/DataInput.java | 4 ++-- .../main/java/io/druid/query/join/JoinQuery.java | 3 ++- 4 files changed, 18 insertions(+), 16 deletions(-) diff --git a/processing/src/main/java/io/druid/query/Queries.java b/processing/src/main/java/io/druid/query/Queries.java index 76a7947ea49d..bbc025d4ea59 100644 --- a/processing/src/main/java/io/druid/query/Queries.java +++ b/processing/src/main/java/io/druid/query/Queries.java @@ -26,6 +26,8 @@ import io.druid.java.util.common.Pair; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; +import io.druid.query.spec.QuerySegmentSpec; +import org.joda.time.Duration; import org.joda.time.Interval; import java.util.ArrayList; @@ -104,4 +106,15 @@ public static Pair getDataSourceAndIntervalStrings(final Que return new Pair<>(datasourceNames.toString(), intervals.toString()); } + + public static Duration getTotalDuration(QuerySegmentSpec spec) + { + Duration totalDuration = new Duration(0); + for (Interval interval : spec.getIntervals()) { + if (interval != null) { + totalDuration = totalDuration.plus(interval.toDuration()); + } + } + return totalDuration; + } } diff --git a/processing/src/main/java/io/druid/query/Query.java b/processing/src/main/java/io/druid/query/Query.java index fe2f3404b6a2..9f8cc13602ea 100644 --- a/processing/src/main/java/io/druid/query/Query.java +++ b/processing/src/main/java/io/druid/query/Query.java @@ -37,7 +37,6 @@ import io.druid.query.timeseries.TimeseriesQuery; import io.druid.query.topn.TopNQuery; import org.joda.time.Duration; -import org.joda.time.Interval; import java.util.List; import java.util.Map; @@ -88,7 +87,7 @@ default Duration getTotalDuration() { Duration totalDuration = new Duration(0); for (DataSourceWithSegmentSpec spec : getDataSources()) { - totalDuration = totalDuration.plus(Query.getTotalDuration(spec.getQuerySegmentSpec())); + totalDuration = totalDuration.plus(Queries.getTotalDuration(spec.getQuerySegmentSpec())); } return totalDuration; } @@ -152,15 +151,4 @@ default Query distributeBy(DataSourceWithSegmentSpec spec) Query withQuerySegmentSpec(String firstDataSourceName, QuerySegmentSpec spec); Query replaceDataSourceWith(DataSource src, DataSource dst); - - static Duration getTotalDuration(QuerySegmentSpec spec) - { - Duration totalDuration = new Duration(0); - for (Interval interval : spec.getIntervals()) { - if (interval != null) { - totalDuration = totalDuration.plus(interval.toDuration()); - } - } - return totalDuration; - } } diff --git a/processing/src/main/java/io/druid/query/join/DataInput.java b/processing/src/main/java/io/druid/query/join/DataInput.java index 0762c8ee5a1d..0f3e1782f2d7 100644 --- a/processing/src/main/java/io/druid/query/join/DataInput.java +++ b/processing/src/main/java/io/druid/query/join/DataInput.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Iterables; import io.druid.query.DataSource; -import io.druid.query.Query; +import io.druid.query.Queries; import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.Duration; @@ -76,7 +76,7 @@ public QuerySegmentSpec getQuerySegmentSpec() public Duration getDuration() { if (duration == null) { - this.duration = Query.getTotalDuration(querySegmentSpec); + this.duration = Queries.getTotalDuration(querySegmentSpec); } return duration; } diff --git a/processing/src/main/java/io/druid/query/join/JoinQuery.java b/processing/src/main/java/io/druid/query/join/JoinQuery.java index 2fab9acbb48a..f9695218ef8b 100644 --- a/processing/src/main/java/io/druid/query/join/JoinQuery.java +++ b/processing/src/main/java/io/druid/query/join/JoinQuery.java @@ -28,6 +28,7 @@ import io.druid.query.DataSource; import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.MultiSourceBaseQuery; +import io.druid.query.Queries; import io.druid.query.Query; import io.druid.query.dimension.DimensionSpec; import io.druid.query.filter.DimFilter; @@ -118,7 +119,7 @@ public Duration getDuration(DataSource dataSource) { for (DataSourceWithSegmentSpec sourceWithSegmentSpec : getDataSources()) { if (sourceWithSegmentSpec.getDataSource().equals(dataSource)) { - return Query.getTotalDuration(sourceWithSegmentSpec.getQuerySegmentSpec()); + return Queries.getTotalDuration(sourceWithSegmentSpec.getQuerySegmentSpec()); } } return null; From eca3c1e0d26a7521c4555300d7fef656de77fc63 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 26 Apr 2017 18:21:10 +0900 Subject: [PATCH 09/15] Add JoinQueryMetrics --- .../io/druid/query/AbstractQueryMetrics.java | 203 ++++++++++++++++++ .../main/java/io/druid/query/BaseQuery.java | 4 +- .../DefaultGenericQueryMetricsFactory.java | 6 +- .../io/druid/query/DefaultQueryMetrics.java | 193 ++--------------- .../query/GenericQueryMetricsFactory.java | 4 +- .../src/main/java/io/druid/query/Query.java | 2 +- .../java/io/druid/query/QueryMetrics.java | 5 - .../query/join/DefaultJoinQueryMetrics.java | 70 ++++++ .../java/io/druid/query/join/JoinQuery.java | 6 +- .../io/druid/query/join/JoinQueryMetrics.java | 42 ++++ .../druid/query/DefaultQueryMetricsTest.java | 4 +- .../coordination/ServerManagerTest.java | 4 +- 12 files changed, 351 insertions(+), 192 deletions(-) create mode 100644 processing/src/main/java/io/druid/query/AbstractQueryMetrics.java create mode 100644 processing/src/main/java/io/druid/query/join/DefaultJoinQueryMetrics.java create mode 100644 processing/src/main/java/io/druid/query/join/JoinQueryMetrics.java diff --git a/processing/src/main/java/io/druid/query/AbstractQueryMetrics.java b/processing/src/main/java/io/druid/query/AbstractQueryMetrics.java new file mode 100644 index 000000000000..3ee02e52b586 --- /dev/null +++ b/processing/src/main/java/io/druid/query/AbstractQueryMetrics.java @@ -0,0 +1,203 @@ +/* + * 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; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Strings; +import com.google.common.collect.Iterables; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.emitter.service.ServiceMetricEvent; +import org.joda.time.Interval; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +public abstract class AbstractQueryMetrics> implements QueryMetrics +{ + protected final ObjectMapper jsonMapper; + protected final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); + protected final Map metrics = new HashMap<>(); + + public AbstractQueryMetrics(ObjectMapper jsonMapper) + { + this.jsonMapper = jsonMapper; + } + + @Override + public void queryType(QueryType query) + { + builder.setDimension(DruidMetrics.TYPE, query.getType()); + } + + @Override + public void hasFilters(QueryType query) + { + builder.setDimension("hasFilters", String.valueOf(query.hasFilters())); + } + + @Override + public void duration(QueryType query) + { + builder.setDimension("duration", query.getTotalDuration().toString()); + } + + @Override + public void queryId(QueryType query) + { + builder.setDimension(DruidMetrics.ID, Strings.nullToEmpty(query.getId())); + } + + @Override + public void context(QueryType query) + { + try { + builder.setDimension( + "context", + jsonMapper.writeValueAsString(query.getContext()) + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } + + @Override + public void server(String host) + { + builder.setDimension("server", host); + } + + @Override + public void remoteAddress(String remoteAddress) + { + builder.setDimension("remoteAddress", remoteAddress); + } + + @Override + public void status(String status) + { + builder.setDimension(DruidMetrics.STATUS, status); + } + + @Override + public void success(boolean success) + { + builder.setDimension("success", String.valueOf(success)); + } + + @Override + public void segment(String segmentIdentifier) + { + builder.setDimension("segment", segmentIdentifier); + } + + @Override + public void chunkInterval(Interval interval) + { + builder.setDimension("chunkInterval", interval.toString()); + } + + @Override + public QueryMetrics reportQueryTime(long timeNs) + { + return defaultTimeMetric("query/time", timeNs); + } + + @Override + public QueryMetrics reportQueryBytes(long byteCount) + { + metrics.put("query/bytes", byteCount); + return this; + } + + @Override + public QueryMetrics reportWaitTime(long timeNs) + { + return defaultTimeMetric("query/wait/time", timeNs); + } + + @Override + public QueryMetrics reportSegmentTime(long timeNs) + { + return defaultTimeMetric("query/segment/time", timeNs); + } + + @Override + public QueryMetrics reportSegmentAndCacheTime(long timeNs) + { + return defaultTimeMetric("query/segmentAndCache/time", timeNs); + } + + @Override + public QueryMetrics reportIntervalChunkTime(long timeNs) + { + return defaultTimeMetric("query/intervalChunk/time", timeNs); + } + + @Override + public QueryMetrics reportCpuTime(long timeNs) + { + metrics.put("query/cpu/time", TimeUnit.NANOSECONDS.toMicros(timeNs)); + return this; + } + + @Override + public QueryMetrics reportNodeTimeToFirstByte(long timeNs) + { + return defaultTimeMetric("query/node/ttfb", timeNs); + } + + @Override + public QueryMetrics reportNodeTime(long timeNs) + { + return defaultTimeMetric("query/node/time", timeNs); + } + + private QueryMetrics defaultTimeMetric(String metricName, long timeNs) + { + metrics.put(metricName, TimeUnit.NANOSECONDS.toMillis(timeNs)); + return this; + } + + @Override + public QueryMetrics reportNodeBytes(long byteCount) + { + metrics.put("query/node/bytes", byteCount); + return this; + } + + @Override + public void emit(ServiceEmitter emitter) + { + for (Map.Entry metric : metrics.entrySet()) { + emitter.emit(builder.build(metric.getKey(), metric.getValue())); + } + metrics.clear(); + } + + private static > DataSourceWithSegmentSpec getOnlyDataSourceWithSegmentSpec( + QueryType query + ) + { + return Iterables.getOnlyElement(query.getDataSources()); + } +} diff --git a/processing/src/main/java/io/druid/query/BaseQuery.java b/processing/src/main/java/io/druid/query/BaseQuery.java index 001b55816d67..313414e31e74 100644 --- a/processing/src/main/java/io/druid/query/BaseQuery.java +++ b/processing/src/main/java/io/druid/query/BaseQuery.java @@ -197,9 +197,9 @@ public Query withQuerySegmentSpec(DataSource dataSource, QuerySegmentSpec spe } @Override - public Query withQuerySegmentSpec(String firstDataSourceName, QuerySegmentSpec spec) + public Query withQuerySegmentSpec(String concatenatedDataSourceName, QuerySegmentSpec spec) { - Preconditions.checkArgument(this.dataSource.getFirstName().equals(firstDataSourceName)); + Preconditions.checkArgument(this.dataSource.getConcatenatedName().equals(concatenatedDataSourceName)); return withQuerySegmentSpec(this.dataSource, spec); } diff --git a/processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java index b71da6e4dbb0..40a72417d4db 100644 --- a/processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java @@ -24,7 +24,7 @@ import com.google.inject.Inject; import io.druid.jackson.DefaultObjectMapper; -public class DefaultGenericQueryMetricsFactory implements GenericQueryMetricsFactory +public class DefaultGenericQueryMetricsFactory implements GenericQueryMetricsFactory> { private static final GenericQueryMetricsFactory INSTANCE = new DefaultGenericQueryMetricsFactory(new DefaultObjectMapper()); @@ -49,9 +49,9 @@ public DefaultGenericQueryMetricsFactory(ObjectMapper jsonMapper) } @Override - public QueryMetrics> makeMetrics(Query query) + public QueryMetrics> makeMetrics(BaseQuery query) { - DefaultQueryMetrics> queryMetrics = new DefaultQueryMetrics<>(jsonMapper); + DefaultQueryMetrics> queryMetrics = new DefaultQueryMetrics<>(jsonMapper); queryMetrics.query(query); return queryMetrics; } diff --git a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java index 49a368d9f378..d3c860f8e776 100644 --- a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java @@ -19,198 +19,47 @@ package io.druid.query; -import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Strings; -import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.emitter.service.ServiceMetricEvent; import org.joda.time.Interval; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; - -public class DefaultQueryMetrics> implements QueryMetrics +public class DefaultQueryMetrics> extends AbstractQueryMetrics { - protected final ObjectMapper jsonMapper; - protected final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); - protected final Map metrics = new HashMap<>(); - public DefaultQueryMetrics(ObjectMapper jsonMapper) { - this.jsonMapper = jsonMapper; + super(jsonMapper); } @Override public void query(QueryType query) { - dataSourcesAndIntervals(query); + dataSource(query); queryType(query); + interval(query); hasFilters(query); duration(query); queryId(query); } - @Override - public void dataSourcesAndIntervals(QueryType query) - { - final List specs = query.getDataSources().stream() - .map(DataSourceWithSegmentSpec::toString) - .collect(Collectors.toList()); - builder.setDimension(DruidMetrics.DATASOURCE, specs.toArray(new String[specs.size()])); - } - - @Override - public void queryType(QueryType query) - { - builder.setDimension(DruidMetrics.TYPE, query.getType()); - } - - @Override - public void hasFilters(QueryType query) - { - builder.setDimension("hasFilters", String.valueOf(query.hasFilters())); - } - - @Override - public void duration(QueryType query) - { - builder.setDimension("duration", query.getTotalDuration().toString()); - } - - @Override - public void queryId(QueryType query) - { - builder.setDimension(DruidMetrics.ID, Strings.nullToEmpty(query.getId())); - } - - @Override - public void context(QueryType query) - { - try { - builder.setDimension( - "context", - jsonMapper.writeValueAsString(query.getContext()) - ); - } - catch (JsonProcessingException e) { - throw new RuntimeException(e); - } - } - - @Override - public void server(String host) - { - builder.setDimension("server", host); - } - - @Override - public void remoteAddress(String remoteAddress) - { - builder.setDimension("remoteAddress", remoteAddress); - } - - @Override - public void status(String status) + /** + * Sets {@link BaseQuery#getDataSource()} of the given query as dimension. + */ + public void dataSource(QueryType query) { - builder.setDimension(DruidMetrics.STATUS, status); + builder.setDimension( + DruidMetrics.DATASOURCE, + DataSourceUtil.getMetricName(query.getDataSource()) + ); } - @Override - public void success(boolean success) - { - builder.setDimension("success", String.valueOf(success)); - } - - @Override - public void segment(String segmentIdentifier) - { - builder.setDimension("segment", segmentIdentifier); - } - - @Override - public void chunkInterval(Interval interval) - { - builder.setDimension("chunkInterval", interval.toString()); - } - - @Override - public QueryMetrics reportQueryTime(long timeNs) - { - return defaultTimeMetric("query/time", timeNs); - } - - @Override - public QueryMetrics reportQueryBytes(long byteCount) - { - metrics.put("query/bytes", byteCount); - return this; - } - - @Override - public QueryMetrics reportWaitTime(long timeNs) - { - return defaultTimeMetric("query/wait/time", timeNs); - } - - @Override - public QueryMetrics reportSegmentTime(long timeNs) - { - return defaultTimeMetric("query/segment/time", timeNs); - } - - @Override - public QueryMetrics reportSegmentAndCacheTime(long timeNs) - { - return defaultTimeMetric("query/segmentAndCache/time", timeNs); - } - - @Override - public QueryMetrics reportIntervalChunkTime(long timeNs) - { - return defaultTimeMetric("query/intervalChunk/time", timeNs); - } - - @Override - public QueryMetrics reportCpuTime(long timeNs) - { - metrics.put("query/cpu/time", TimeUnit.NANOSECONDS.toMicros(timeNs)); - return this; - } - - @Override - public QueryMetrics reportNodeTimeToFirstByte(long timeNs) - { - return defaultTimeMetric("query/node/ttfb", timeNs); - } - - @Override - public QueryMetrics reportNodeTime(long timeNs) - { - return defaultTimeMetric("query/node/time", timeNs); - } - - private QueryMetrics defaultTimeMetric(String metricName, long timeNs) - { - metrics.put(metricName, TimeUnit.NANOSECONDS.toMillis(timeNs)); - return this; - } - - @Override - public QueryMetrics reportNodeBytes(long byteCount) - { - metrics.put("query/node/bytes", byteCount); - return this; - } - - @Override - public void emit(ServiceEmitter emitter) + /** + * Sets {@link BaseQuery#getIntervals()} of the given query as dimension. + */ + public void interval(QueryType query) { - for (Map.Entry metric : metrics.entrySet()) { - emitter.emit(builder.build(metric.getKey(), metric.getValue())); - } - metrics.clear(); + builder.setDimension( + DruidMetrics.INTERVAL, + query.getIntervals().stream() + .map(Interval::toString).toArray(String[]::new) + ); } } diff --git a/processing/src/main/java/io/druid/query/GenericQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/GenericQueryMetricsFactory.java index dd4297606da0..4c8cd44e4c9b 100644 --- a/processing/src/main/java/io/druid/query/GenericQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/GenericQueryMetricsFactory.java @@ -23,11 +23,11 @@ * This factory is used for DI of custom {@link QueryMetrics} implementations for all query types, which don't (yet) * need to emit custom dimensions and/or metrics, i. e. they are good with the generic {@link QueryMetrics} interface. */ -public interface GenericQueryMetricsFactory +public interface GenericQueryMetricsFactory> { /** * Creates a {@link QueryMetrics} for query, which doesn't have predefined QueryMetrics subclass. This method must * call {@link QueryMetrics#query(Query)} with the given query on the created QueryMetrics object before returning. */ - QueryMetrics> makeMetrics(Query query); + QueryMetrics makeMetrics(QueryType query); } diff --git a/processing/src/main/java/io/druid/query/Query.java b/processing/src/main/java/io/druid/query/Query.java index 9f8cc13602ea..106fcb1af904 100644 --- a/processing/src/main/java/io/druid/query/Query.java +++ b/processing/src/main/java/io/druid/query/Query.java @@ -148,7 +148,7 @@ default Query distributeBy(DataSourceWithSegmentSpec spec) Query withQuerySegmentSpec(DataSource dataSource, QuerySegmentSpec spec); - Query withQuerySegmentSpec(String firstDataSourceName, QuerySegmentSpec spec); + Query withQuerySegmentSpec(String concatenatedDataSourceName, QuerySegmentSpec spec); Query replaceDataSourceWith(DataSource src, DataSource dst); } diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index 7a4d523a4979..c25031e2cc99 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -145,11 +145,6 @@ public interface QueryMetrics> */ void query(QueryType query); - /** - * Sets {@link Query#getDataSources()} of the given query as dimension. - */ - void dataSourcesAndIntervals(QueryType query); - /** * Sets {@link Query#getType()} of the given query as dimension. */ diff --git a/processing/src/main/java/io/druid/query/join/DefaultJoinQueryMetrics.java b/processing/src/main/java/io/druid/query/join/DefaultJoinQueryMetrics.java new file mode 100644 index 000000000000..41f5c65e83de --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/DefaultJoinQueryMetrics.java @@ -0,0 +1,70 @@ +/* + * 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.join; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.query.AbstractQueryMetrics; +import io.druid.query.DataSourceUtil; +import io.druid.query.DataSourceWithSegmentSpec; + +public class DefaultJoinQueryMetrics extends AbstractQueryMetrics implements JoinQueryMetrics +{ + public DefaultJoinQueryMetrics(ObjectMapper jsonMapper) + { + super(jsonMapper); + } + + @Override + public void query(JoinQuery query) + { + queryType(query); + distributionTarget(query); + distributionTargetDuration(query); + numDataSources(query); + hasFilters(query); + queryId(query); + } + + @Override + public void numDataSources(JoinQuery query) + { + builder.setDimension("numDataSources", String.valueOf(query.getDataSources().size())); + } + + @Override + public void distributionTarget(JoinQuery query) + { + final DataSourceWithSegmentSpec distributionTarget = query.getDistributionTarget(); + builder.setDimension( + "distributionTarget", + distributionTarget == null ? "" : DataSourceUtil.getMetricName(distributionTarget.getDataSource()) + ); + } + + @Override + public void distributionTargetDuration(JoinQuery query) + { + final DataSourceWithSegmentSpec distributionTarget = query.getDistributionTarget(); + builder.setDimension( + "distributionTargetDuration", + distributionTarget == null ? "" : query.getDuration(distributionTarget.getDataSource()).toString() + ); + } +} diff --git a/processing/src/main/java/io/druid/query/join/JoinQuery.java b/processing/src/main/java/io/druid/query/join/JoinQuery.java index f9695218ef8b..555d8f092250 100644 --- a/processing/src/main/java/io/druid/query/join/JoinQuery.java +++ b/processing/src/main/java/io/druid/query/join/JoinQuery.java @@ -172,11 +172,11 @@ public Query withOverriddenContext(Map contextOverride) @Override public Query withQuerySegmentSpec(DataSource dataSource, QuerySegmentSpec spec) { - return withQuerySegmentSpec(dataSource.getFirstName(), spec); + return withQuerySegmentSpec(dataSource.getConcatenatedName(), spec); } @Override - public Query withQuerySegmentSpec(String firstDataSourceName, QuerySegmentSpec spec) + public Query withQuerySegmentSpec(String concatenatedDataSourceName, QuerySegmentSpec spec) { final JoinSpecVisitor visitor = new JoinSpecVisitor() { @@ -191,7 +191,7 @@ public JoinSpec visit(JoinSpec joinSpec) @Override public DataInput visit(DataInput dataInput) { - if (dataInput.getDataSource().getFirstName().equals(firstDataSourceName)) { + if (dataInput.getDataSource().getConcatenatedName().equals(concatenatedDataSourceName)) { return new DataInput(dataInput.getDataSource(), spec); } else { return dataInput; diff --git a/processing/src/main/java/io/druid/query/join/JoinQueryMetrics.java b/processing/src/main/java/io/druid/query/join/JoinQueryMetrics.java new file mode 100644 index 000000000000..2ddcad8be8ec --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/JoinQueryMetrics.java @@ -0,0 +1,42 @@ +/* + * 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.join; + +import io.druid.query.Query; +import io.druid.query.QueryMetrics; + +public interface JoinQueryMetrics extends QueryMetrics +{ + /** + * Sets the size of {@link Query#getDataSources()} of the given query as dimension. + */ + void numDataSources(JoinQuery query); + + /** + * Sets the dataSource of {@link Query#getDistributionTarget()} of the given query as dimension. + */ + void distributionTarget(JoinQuery query); + + /** + * Sets the duration of {@link Query#getDistributionTarget()} of the given query as dimension. + * @param query + */ + void distributionTargetDuration(JoinQuery query); +} diff --git a/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java b/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java index 199ebda7e19c..18cdd20d0456 100644 --- a/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java @@ -48,7 +48,7 @@ public void testDefaultQueryMetricsQuery() { CachingEmitter cachingEmitter = new CachingEmitter(); ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); - DefaultQueryMetrics> queryMetrics = new DefaultQueryMetrics<>(new DefaultObjectMapper()); + DefaultQueryMetrics> queryMetrics = new DefaultQueryMetrics<>(new DefaultObjectMapper()); TopNQuery query = new TopNQueryBuilder() .dataSource("xx") .granularity(Granularities.ALL) @@ -91,7 +91,7 @@ public void testDefaultQueryMetricsMetricNamesAndUnits() { CachingEmitter cachingEmitter = new CachingEmitter(); ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); - DefaultQueryMetrics> queryMetrics = new DefaultQueryMetrics<>(new DefaultObjectMapper()); + DefaultQueryMetrics> queryMetrics = new DefaultQueryMetrics<>(new DefaultObjectMapper()); testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics); } diff --git a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java index 7b6eef2471a5..991d7ebed7fb 100644 --- a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java +++ b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java @@ -574,9 +574,9 @@ public QueryRunner mergeResults(QueryRunner runner) } @Override - public QueryMetrics> makeMetrics(QueryType query) + public QueryMetrics makeMetrics(QueryType query) { - return new DefaultQueryMetrics<>(new DefaultObjectMapper()); + return (QueryMetrics) new DefaultQueryMetrics<>(new DefaultObjectMapper()); } @Override From 541f0ded80efe0045431e660b795897c914d8a7d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 26 Apr 2017 18:48:44 +0900 Subject: [PATCH 10/15] Fix withQuerySegmentSpec --- .../io/druid/query/IntervalChunkingQueryRunner.java | 3 +-- .../src/main/java/io/druid/query/TimewarpOperator.java | 3 +-- .../src/main/java/io/druid/query/join/JoinQuery.java | 2 +- .../druid/query/spec/SpecificSegmentQueryRunner.java | 6 +++++- .../java/io/druid/client/CachingClusteredClient.java | 10 +++------- .../io/druid/segment/realtime/RealtimeManager.java | 5 ++++- .../io/druid/server/coordination/ServerManager.java | 6 ++++-- .../io/druid/client/CachingClusteredClientTest.java | 4 ++-- 8 files changed, 21 insertions(+), 18 deletions(-) diff --git a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java index 834c18a31bcf..1dcb0a49f45a 100644 --- a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java @@ -20,7 +20,6 @@ package io.druid.query; import com.google.common.base.Function; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.metamx.emitter.service.ServiceEmitter; import io.druid.java.util.common.granularity.PeriodGranularity; @@ -116,7 +115,7 @@ public Sequence apply(Interval singleInterval) executor, queryWatcher ).run( query.withQuerySegmentSpec( - Iterables.getOnlyElement(spec.getDataSource().getNames()), + spec.getDataSource(), new MultipleIntervalSegmentSpec(Arrays.asList(singleInterval)) ), responseContext diff --git a/processing/src/main/java/io/druid/query/TimewarpOperator.java b/processing/src/main/java/io/druid/query/TimewarpOperator.java index 54e7a0b69039..0a8e9059ea39 100644 --- a/processing/src/main/java/io/druid/query/TimewarpOperator.java +++ b/processing/src/main/java/io/druid/query/TimewarpOperator.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; -import com.google.common.collect.Iterables; import io.druid.data.input.MapBasedRow; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; @@ -95,7 +94,7 @@ public Sequence run(final Query query, final Map responseC return Sequences.map( baseRunner.run( query.withQuerySegmentSpec( - Iterables.getOnlyElement(spec.getDataSource().getNames()), + spec.getDataSource(), new MultipleIntervalSegmentSpec(Arrays.asList(modifiedInterval)) ), responseContext diff --git a/processing/src/main/java/io/druid/query/join/JoinQuery.java b/processing/src/main/java/io/druid/query/join/JoinQuery.java index 555d8f092250..74471b4cebf6 100644 --- a/processing/src/main/java/io/druid/query/join/JoinQuery.java +++ b/processing/src/main/java/io/druid/query/join/JoinQuery.java @@ -172,7 +172,7 @@ public Query withOverriddenContext(Map contextOverride) @Override public Query withQuerySegmentSpec(DataSource dataSource, QuerySegmentSpec spec) { - return withQuerySegmentSpec(dataSource.getConcatenatedName(), spec); + return withQuerySegmentSpec(dataSource.getFirstName(), spec); } @Override diff --git a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java index 08c887469bfe..34ffe7ddd404 100644 --- a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java @@ -63,10 +63,14 @@ public SpecificSegmentQueryRunner( @Override public Sequence run(final Query input, final Map responseContext) { - final Query query = input.withQuerySegmentSpec(dataSourceName, specificSpec); + // Here, dataSourceName must be the name of the distributionTarget + final DataSourceWithSegmentSpec distributionTarget = input.getDistributionTarget(); + Preconditions.checkArgument(distributionTarget.getDataSource().getNames().contains(dataSourceName)); + final Query query = input.withQuerySegmentSpec(distributionTarget.getDataSource(), specificSpec); final Thread currThread = Thread.currentThread(); final String currThreadName = currThread.getName(); + // If any other data sources contain dataSourceName, then error final List specs = input.getDataSources().stream() .filter(spec -> spec.getDataSource().getNames() .contains(dataSourceName)) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index f186f7c96feb..173701d156cf 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -24,7 +24,6 @@ import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Optional; -import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -167,9 +166,6 @@ public Sequence run(final Query query, final Map responseC } final DataSourceWithSegmentSpec spec = query.getDistributionTarget(); - final List dataSourceNames = spec.getDataSource().getNames(); - Preconditions.checkArgument(dataSourceNames.size() > 0); - final String dataSourceName = dataSourceNames.get(0); TimelineLookup timeline = serverView.getTimeline(spec.getDataSource()); @@ -437,7 +433,7 @@ private void addSequencesFromServer(ArrayList> listOfSequences) if (!server.isAssignable() || !populateCache || isBySegment) { // Direct server queryable if (!isBySegment) { resultSeqToAdd = clientQueryable.run( - query.withQuerySegmentSpec(dataSourceName, segmentSpec), + query.withQuerySegmentSpec(spec.getDataSource(), segmentSpec), responseContext ); } else { @@ -449,7 +445,7 @@ private void addSequencesFromServer(ArrayList> listOfSequences) @SuppressWarnings("unchecked") final Sequence>> resultSequence = clientQueryable.run( - bySegmentQuery.withQuerySegmentSpec(dataSourceName, segmentSpec), + bySegmentQuery.withQuerySegmentSpec(spec.getDataSource(), segmentSpec), responseContext ); @@ -482,7 +478,7 @@ public Result> apply(Result>> runningSequence = clientQueryable.run( - rewrittenQuery.withQuerySegmentSpec(dataSourceName, segmentSpec), + rewrittenQuery.withQuerySegmentSpec(spec.getDataSource(), segmentSpec), responseContext ); resultSeqToAdd = new MergeSequence( diff --git a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java index 2e906e2f63de..1aa703c5e6ac 100644 --- a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java +++ b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java @@ -199,7 +199,10 @@ public QueryRunner apply(SegmentDescriptor spec) return retVal == null ? new NoopQueryRunner() : retVal.getQueryRunner( - query.withQuerySegmentSpec(dataSourceName, new SpecificSegmentSpec(spec)) + query.withQuerySegmentSpec( + sourceWithSegmentSpec.getDataSource(), + new SpecificSegmentSpec(spec) + ) ); } } diff --git a/server/src/main/java/io/druid/server/coordination/ServerManager.java b/server/src/main/java/io/druid/server/coordination/ServerManager.java index 2addae2a5fef..628240162b69 100644 --- a/server/src/main/java/io/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/io/druid/server/coordination/ServerManager.java @@ -279,7 +279,8 @@ public QueryRunner getQueryRunnerForIntervals(Query query, Iterable> broadcastSegmentDescMap = new HashMap<>(); query.getDataSources().forEach( spec -> { - final String sourceName = getDataSourceName(spec.getDataSource()); + // Get concatenated name to handle broadcasted union dataSource + final String sourceName = spec.getDataSource().getConcatenatedName(); final VersionedIntervalTimeline timeline = dataSources.get(sourceName); if (!sourceName.equals(nonBroadcastDataSourceName)) { spec.getQuerySegmentSpec().getIntervals().stream() @@ -418,7 +419,8 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable> broadcastSegmentDescMap = new HashMap<>(); query.getDataSources().forEach( spec -> { - final String sourceName = getDataSourceName(spec.getDataSource()); + // Get concatenated name to handle broadcasted union dataSource + final String sourceName = spec.getDataSource().getConcatenatedName(); final VersionedIntervalTimeline timeline = dataSources.get(sourceName); if (!sourceName.equals(nonBroadcastDataSourceName)) { spec.getQuerySegmentSpec().getIntervals().stream() diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index 0fce2bf125b3..59cc6dfa351a 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -2003,7 +2003,7 @@ public void run() expected, runner.run( query.withQuerySegmentSpec( - Iterables.getOnlyElement(spec.getDataSource().getNames()), + spec.getDataSource(), new MultipleIntervalSegmentSpec( ImmutableList.of( actualQueryInterval @@ -2230,7 +2230,7 @@ public Iterable>> apply(@Nullable Integer input) ), runner.run( query.withQuerySegmentSpec( - Iterables.getOnlyElement(spec.getDataSource().getNames()), + spec.getDataSource(), new MultipleIntervalSegmentSpec( ImmutableList.of( actualQueryInterval From 557db9d24f458d2b71381f45945bd9743d1d7128 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 26 Apr 2017 19:45:20 +0900 Subject: [PATCH 11/15] Fix test failure --- .../indexing/kafka/KafkaIndexTaskTest.java | 1 + .../druid/query/DefaultQueryMetricsTest.java | 15 ++++++++------- .../DefaultGroupByQueryMetricsTest.java | 17 +++++++---------- .../spec/SpecificSegmentQueryRunnerTest.java | 2 ++ .../DefaultTimeseriesQueryMetricsTest.java | 18 +++++++++--------- .../topn/DefaultTopNQueryMetricsTest.java | 18 +++++++++--------- .../server/coordination/ServerManagerTest.java | 1 + 7 files changed, 37 insertions(+), 35 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index 2132d0623bd4..6162e7ea46d4 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -1611,6 +1611,7 @@ public long countEvents(final Task task) throws Exception ) ).granularity(Granularities.ALL) .intervals("0000/3000") + .updateDistributionTarget() .build(); ArrayList> results = Sequences.toList( diff --git a/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java b/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java index 18cdd20d0456..62f7cbb487cf 100644 --- a/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java @@ -29,10 +29,12 @@ import io.druid.query.filter.SelectorDimFilter; import io.druid.query.topn.TopNQuery; import io.druid.query.topn.TopNQueryBuilder; +import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -67,18 +69,17 @@ public void testDefaultQueryMetricsQuery() queryMetrics.reportQueryTime(0).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); - Assert.assertEquals(11, actualEvent.size()); + Assert.assertEquals(12, actualEvent.size()); Assert.assertTrue(actualEvent.containsKey("feed")); Assert.assertTrue(actualEvent.containsKey("timestamp")); Assert.assertEquals("", actualEvent.get("host")); Assert.assertEquals("", actualEvent.get("service")); - Assert.assertEquals( - query.getDataSources().stream() - .map(DataSourceWithSegmentSpec::toString) - .collect(Collectors.toList()), - actualEvent.get(DruidMetrics.DATASOURCE) - ); + Assert.assertEquals("xx", actualEvent.get(DruidMetrics.DATASOURCE)); Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE)); + List expectedIntervals = QueryRunnerTestHelper.fullOnInterval.getIntervals(); + List expectedStringIntervals = + expectedIntervals.stream().map(Interval::toString).collect(Collectors.toList()); + Assert.assertEquals(expectedStringIntervals, actualEvent.get(DruidMetrics.INTERVAL)); Assert.assertEquals("true", actualEvent.get("hasFilters")); Assert.assertEquals(query.getTotalDuration().toString(), actualEvent.get("duration")); Assert.assertEquals("", actualEvent.get(DruidMetrics.ID)); diff --git a/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java b/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java index bfd934c6dfab..f04600c28181 100644 --- a/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java @@ -25,7 +25,6 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.granularity.PeriodGranularity; import io.druid.query.CachingEmitter; -import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.DefaultQueryMetricsTest; import io.druid.query.DruidMetrics; import io.druid.query.QueryRunnerTestHelper; @@ -35,13 +34,14 @@ import io.druid.query.extraction.MapLookupExtractor; import io.druid.query.filter.SelectorDimFilter; import io.druid.query.lookup.LookupExtractionFn; +import org.joda.time.Interval; import org.joda.time.Period; import org.junit.Assert; import org.junit.Test; import java.util.Arrays; +import java.util.Collections; import java.util.Map; -import java.util.stream.Collectors; public class DefaultGroupByQueryMetricsTest { @@ -92,20 +92,17 @@ public void testDefaultGroupByQueryMetricsQuery() queryMetrics.reportQueryTime(0).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); - Assert.assertEquals(14, actualEvent.size()); + Assert.assertEquals(15, actualEvent.size()); Assert.assertTrue(actualEvent.containsKey("feed")); Assert.assertTrue(actualEvent.containsKey("timestamp")); Assert.assertEquals("", actualEvent.get("host")); Assert.assertEquals("", actualEvent.get("service")); - Assert.assertEquals( - query.getDataSources().stream() - .map(DataSourceWithSegmentSpec::toString) - .collect(Collectors.toList()), - actualEvent.get(DruidMetrics.DATASOURCE) - ); + Assert.assertEquals(QueryRunnerTestHelper.dataSource, actualEvent.get(DruidMetrics.DATASOURCE)); Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE)); + Interval expectedInterval = new Interval("2011-04-02/2011-04-04"); + Assert.assertEquals(Collections.singletonList(expectedInterval.toString()), actualEvent.get(DruidMetrics.INTERVAL)); Assert.assertEquals("true", actualEvent.get("hasFilters")); - Assert.assertEquals(query.getTotalDuration().toString(), actualEvent.get("duration")); + Assert.assertEquals(expectedInterval.toDuration().toString(), actualEvent.get("duration")); Assert.assertEquals("", actualEvent.get(DruidMetrics.ID)); // GroupBy-specific dimensions diff --git a/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java index 3fbe342037d8..9e567b5684bc 100644 --- a/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java @@ -106,6 +106,7 @@ public Yielder toYielder( new CountAggregatorFactory("rows") ) ) + .updateDistributionTarget() .build(); Sequence results = queryRunner.run(query, responseContext); Sequences.toList(results, Lists.newArrayList()); @@ -184,6 +185,7 @@ public void run() new CountAggregatorFactory("rows") ) ) + .updateDistributionTarget() .build(); Sequence results = queryRunner.run( query, diff --git a/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java b/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java index fb42eb201ee8..fb03645c0675 100644 --- a/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java @@ -22,16 +22,17 @@ import com.metamx.emitter.service.ServiceEmitter; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.CachingEmitter; -import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.DefaultQueryMetricsTest; import io.druid.query.DruidMetrics; import io.druid.query.Druids; import io.druid.query.QueryRunnerTestHelper; +import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; import java.util.Arrays; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -66,20 +67,19 @@ public void testDefaultTimeseriesQueryMetricsQuery() queryMetrics.reportQueryTime(0).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); - Assert.assertEquals(13, actualEvent.size()); + Assert.assertEquals(14, actualEvent.size()); Assert.assertTrue(actualEvent.containsKey("feed")); Assert.assertTrue(actualEvent.containsKey("timestamp")); Assert.assertEquals("", actualEvent.get("host")); Assert.assertEquals("", actualEvent.get("service")); - Assert.assertEquals( - query.getDataSources().stream() - .map(DataSourceWithSegmentSpec::toString) - .collect(Collectors.toList()), - actualEvent.get(DruidMetrics.DATASOURCE) - ); + Assert.assertEquals(QueryRunnerTestHelper.dataSource, actualEvent.get(DruidMetrics.DATASOURCE)); Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE)); + List expectedIntervals = QueryRunnerTestHelper.fullOnInterval.getIntervals(); + List expectedStringIntervals = + expectedIntervals.stream().map(Interval::toString).collect(Collectors.toList()); + Assert.assertEquals(expectedStringIntervals, actualEvent.get(DruidMetrics.INTERVAL)); Assert.assertEquals("false", actualEvent.get("hasFilters")); - Assert.assertEquals(query.getTotalDuration().toString(), actualEvent.get("duration")); + Assert.assertEquals(expectedIntervals.get(0).toDuration().toString(), actualEvent.get("duration")); Assert.assertEquals("", actualEvent.get(DruidMetrics.ID)); // Timeseries-specific dimensions diff --git a/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java b/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java index 36250a134f51..185795fc1db0 100644 --- a/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java @@ -24,7 +24,6 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.granularity.Granularities; import io.druid.query.CachingEmitter; -import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.DefaultQueryMetricsTest; import io.druid.query.DruidMetrics; import io.druid.query.QueryRunnerTestHelper; @@ -32,10 +31,12 @@ import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.ListFilteredDimensionSpec; import io.druid.query.filter.SelectorDimFilter; +import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -70,20 +71,19 @@ public void testDefaultTopNQueryMetricsQuery() queryMetrics.reportQueryTime(0).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); - Assert.assertEquals(15, actualEvent.size()); + Assert.assertEquals(16, actualEvent.size()); Assert.assertTrue(actualEvent.containsKey("feed")); Assert.assertTrue(actualEvent.containsKey("timestamp")); Assert.assertEquals("", actualEvent.get("host")); Assert.assertEquals("", actualEvent.get("service")); - Assert.assertEquals( - query.getDataSources().stream() - .map(DataSourceWithSegmentSpec::toString) - .collect(Collectors.toList()), - actualEvent.get(DruidMetrics.DATASOURCE) - ); + Assert.assertEquals("xx", actualEvent.get(DruidMetrics.DATASOURCE)); Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE)); + List expectedIntervals = QueryRunnerTestHelper.fullOnInterval.getIntervals(); + List expectedStringIntervals = + expectedIntervals.stream().map(Interval::toString).collect(Collectors.toList()); + Assert.assertEquals(expectedStringIntervals, actualEvent.get(DruidMetrics.INTERVAL)); Assert.assertEquals("true", actualEvent.get("hasFilters")); - Assert.assertEquals(query.getTotalDuration().toString(), actualEvent.get("duration")); + Assert.assertEquals(expectedIntervals.get(0).toDuration().toString(), actualEvent.get("duration")); Assert.assertEquals("", actualEvent.get(DruidMetrics.ID)); // TopN-specific dimensions diff --git a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java index 991d7ebed7fb..d6b7711a2e3e 100644 --- a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java +++ b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java @@ -424,6 +424,7 @@ private Future assertQueryable( .granularity(granularity) .limit(10000) .query("wow") + .updateDistributionTarget() .build(); final QueryRunner> runner = serverManager.getQueryRunnerForIntervals( query.updateDistributionTarget(), From 59b65cb5211f3b1563ed5cfde87f2659121a349b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 27 Apr 2017 13:47:56 +0900 Subject: [PATCH 12/15] Address comments --- .../io/druid/query/AbstractQueryMetrics.java | 203 ----------------- .../DefaultGenericQueryMetricsFactory.java | 6 +- .../io/druid/query/DefaultQueryMetrics.java | 213 ++++++++++++++++-- .../query/GenericQueryMetricsFactory.java | 4 +- .../src/main/java/io/druid/query/Query.java | 5 + .../java/io/druid/query/QueryMetrics.java | 12 +- .../query/join/DefaultJoinQueryMetrics.java | 25 +- .../java/io/druid/query/join/JoinQuery.java | 2 +- .../join/DefaultJoinQueryMetricsTest.java | 137 +++++++++++ .../io/druid/query/join/JoinQueryTest.java | 2 +- 10 files changed, 377 insertions(+), 232 deletions(-) delete mode 100644 processing/src/main/java/io/druid/query/AbstractQueryMetrics.java create mode 100644 processing/src/test/java/io/druid/query/join/DefaultJoinQueryMetricsTest.java diff --git a/processing/src/main/java/io/druid/query/AbstractQueryMetrics.java b/processing/src/main/java/io/druid/query/AbstractQueryMetrics.java deleted file mode 100644 index 3ee02e52b586..000000000000 --- a/processing/src/main/java/io/druid/query/AbstractQueryMetrics.java +++ /dev/null @@ -1,203 +0,0 @@ -/* - * 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; - -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Strings; -import com.google.common.collect.Iterables; -import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.emitter.service.ServiceMetricEvent; -import org.joda.time.Interval; - -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.TimeUnit; - -public abstract class AbstractQueryMetrics> implements QueryMetrics -{ - protected final ObjectMapper jsonMapper; - protected final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); - protected final Map metrics = new HashMap<>(); - - public AbstractQueryMetrics(ObjectMapper jsonMapper) - { - this.jsonMapper = jsonMapper; - } - - @Override - public void queryType(QueryType query) - { - builder.setDimension(DruidMetrics.TYPE, query.getType()); - } - - @Override - public void hasFilters(QueryType query) - { - builder.setDimension("hasFilters", String.valueOf(query.hasFilters())); - } - - @Override - public void duration(QueryType query) - { - builder.setDimension("duration", query.getTotalDuration().toString()); - } - - @Override - public void queryId(QueryType query) - { - builder.setDimension(DruidMetrics.ID, Strings.nullToEmpty(query.getId())); - } - - @Override - public void context(QueryType query) - { - try { - builder.setDimension( - "context", - jsonMapper.writeValueAsString(query.getContext()) - ); - } - catch (JsonProcessingException e) { - throw new RuntimeException(e); - } - } - - @Override - public void server(String host) - { - builder.setDimension("server", host); - } - - @Override - public void remoteAddress(String remoteAddress) - { - builder.setDimension("remoteAddress", remoteAddress); - } - - @Override - public void status(String status) - { - builder.setDimension(DruidMetrics.STATUS, status); - } - - @Override - public void success(boolean success) - { - builder.setDimension("success", String.valueOf(success)); - } - - @Override - public void segment(String segmentIdentifier) - { - builder.setDimension("segment", segmentIdentifier); - } - - @Override - public void chunkInterval(Interval interval) - { - builder.setDimension("chunkInterval", interval.toString()); - } - - @Override - public QueryMetrics reportQueryTime(long timeNs) - { - return defaultTimeMetric("query/time", timeNs); - } - - @Override - public QueryMetrics reportQueryBytes(long byteCount) - { - metrics.put("query/bytes", byteCount); - return this; - } - - @Override - public QueryMetrics reportWaitTime(long timeNs) - { - return defaultTimeMetric("query/wait/time", timeNs); - } - - @Override - public QueryMetrics reportSegmentTime(long timeNs) - { - return defaultTimeMetric("query/segment/time", timeNs); - } - - @Override - public QueryMetrics reportSegmentAndCacheTime(long timeNs) - { - return defaultTimeMetric("query/segmentAndCache/time", timeNs); - } - - @Override - public QueryMetrics reportIntervalChunkTime(long timeNs) - { - return defaultTimeMetric("query/intervalChunk/time", timeNs); - } - - @Override - public QueryMetrics reportCpuTime(long timeNs) - { - metrics.put("query/cpu/time", TimeUnit.NANOSECONDS.toMicros(timeNs)); - return this; - } - - @Override - public QueryMetrics reportNodeTimeToFirstByte(long timeNs) - { - return defaultTimeMetric("query/node/ttfb", timeNs); - } - - @Override - public QueryMetrics reportNodeTime(long timeNs) - { - return defaultTimeMetric("query/node/time", timeNs); - } - - private QueryMetrics defaultTimeMetric(String metricName, long timeNs) - { - metrics.put(metricName, TimeUnit.NANOSECONDS.toMillis(timeNs)); - return this; - } - - @Override - public QueryMetrics reportNodeBytes(long byteCount) - { - metrics.put("query/node/bytes", byteCount); - return this; - } - - @Override - public void emit(ServiceEmitter emitter) - { - for (Map.Entry metric : metrics.entrySet()) { - emitter.emit(builder.build(metric.getKey(), metric.getValue())); - } - metrics.clear(); - } - - private static > DataSourceWithSegmentSpec getOnlyDataSourceWithSegmentSpec( - QueryType query - ) - { - return Iterables.getOnlyElement(query.getDataSources()); - } -} diff --git a/processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java index 40a72417d4db..b71da6e4dbb0 100644 --- a/processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java @@ -24,7 +24,7 @@ import com.google.inject.Inject; import io.druid.jackson.DefaultObjectMapper; -public class DefaultGenericQueryMetricsFactory implements GenericQueryMetricsFactory> +public class DefaultGenericQueryMetricsFactory implements GenericQueryMetricsFactory { private static final GenericQueryMetricsFactory INSTANCE = new DefaultGenericQueryMetricsFactory(new DefaultObjectMapper()); @@ -49,9 +49,9 @@ public DefaultGenericQueryMetricsFactory(ObjectMapper jsonMapper) } @Override - public QueryMetrics> makeMetrics(BaseQuery query) + public QueryMetrics> makeMetrics(Query query) { - DefaultQueryMetrics> queryMetrics = new DefaultQueryMetrics<>(jsonMapper); + DefaultQueryMetrics> queryMetrics = new DefaultQueryMetrics<>(jsonMapper); queryMetrics.query(query); return queryMetrics; } diff --git a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java index d3c860f8e776..9a31ae038eb6 100644 --- a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java @@ -19,47 +19,228 @@ package io.druid.query; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Strings; +import com.google.common.collect.Iterables; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.emitter.service.ServiceMetricEvent; import org.joda.time.Interval; -public class DefaultQueryMetrics> extends AbstractQueryMetrics +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +public class DefaultQueryMetrics> implements QueryMetrics { + protected final ObjectMapper jsonMapper; + protected final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); + protected final Map metrics = new HashMap<>(); + public DefaultQueryMetrics(ObjectMapper jsonMapper) { - super(jsonMapper); + this.jsonMapper = jsonMapper; } @Override public void query(QueryType query) { - dataSource(query); +// dataSource(query); + dataSourcesAndDurations(query); queryType(query); - interval(query); + intervals(query); hasFilters(query); - duration(query); +// duration(query); queryId(query); } - /** - * Sets {@link BaseQuery#getDataSource()} of the given query as dimension. - */ - public void dataSource(QueryType query) +// /** +// * Sets {@link BaseQuery#getDataSource()} of the given query as dimension. +// */ +// public void dataSource(QueryType query) +// { +// builder.setDimension( +// DruidMetrics.DATASOURCE, +// DataSourceUtil.getMetricName(query.getDataSource()) +// ); +// } + + @Override + public void dataSourcesAndDurations(QueryType query) { builder.setDimension( DruidMetrics.DATASOURCE, - DataSourceUtil.getMetricName(query.getDataSource()) + DataSourceUtil.getMetricName(getOnlyDataSourceWithSegmentSpec(query).getDataSource()) ); + builder.setDimension("duration", query.getTotalDuration().toString()); } - /** - * Sets {@link BaseQuery#getIntervals()} of the given query as dimension. - */ - public void interval(QueryType query) + @Override + public void intervals(QueryType query) { builder.setDimension( DruidMetrics.INTERVAL, - query.getIntervals().stream() - .map(Interval::toString).toArray(String[]::new) + getOnlyDataSourceWithSegmentSpec(query).getQuerySegmentSpec().getIntervals().stream() + .map(Interval::toString).toArray(String[]::new) ); } + + @Override + public void queryType(QueryType query) + { + builder.setDimension(DruidMetrics.TYPE, query.getType()); + } + + @Override + public void hasFilters(QueryType query) + { + builder.setDimension("hasFilters", String.valueOf(query.hasFilters())); + } + +// @Override +// public void duration(QueryType query) +// { +// builder.setDimension("duration", query.getTotalDuration().toString()); +// } + + @Override + public void queryId(QueryType query) + { + builder.setDimension(DruidMetrics.ID, Strings.nullToEmpty(query.getId())); + } + + @Override + public void context(QueryType query) + { + try { + builder.setDimension( + "context", + jsonMapper.writeValueAsString(query.getContext()) + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } + + @Override + public void server(String host) + { + builder.setDimension("server", host); + } + + @Override + public void remoteAddress(String remoteAddress) + { + builder.setDimension("remoteAddress", remoteAddress); + } + + @Override + public void status(String status) + { + builder.setDimension(DruidMetrics.STATUS, status); + } + + @Override + public void success(boolean success) + { + builder.setDimension("success", String.valueOf(success)); + } + + @Override + public void segment(String segmentIdentifier) + { + builder.setDimension("segment", segmentIdentifier); + } + + @Override + public void chunkInterval(Interval interval) + { + builder.setDimension("chunkInterval", interval.toString()); + } + + @Override + public QueryMetrics reportQueryTime(long timeNs) + { + return defaultTimeMetric("query/time", timeNs); + } + + @Override + public QueryMetrics reportQueryBytes(long byteCount) + { + metrics.put("query/bytes", byteCount); + return this; + } + + @Override + public QueryMetrics reportWaitTime(long timeNs) + { + return defaultTimeMetric("query/wait/time", timeNs); + } + + @Override + public QueryMetrics reportSegmentTime(long timeNs) + { + return defaultTimeMetric("query/segment/time", timeNs); + } + + @Override + public QueryMetrics reportSegmentAndCacheTime(long timeNs) + { + return defaultTimeMetric("query/segmentAndCache/time", timeNs); + } + + @Override + public QueryMetrics reportIntervalChunkTime(long timeNs) + { + return defaultTimeMetric("query/intervalChunk/time", timeNs); + } + + @Override + public QueryMetrics reportCpuTime(long timeNs) + { + metrics.put("query/cpu/time", TimeUnit.NANOSECONDS.toMicros(timeNs)); + return this; + } + + @Override + public QueryMetrics reportNodeTimeToFirstByte(long timeNs) + { + return defaultTimeMetric("query/node/ttfb", timeNs); + } + + @Override + public QueryMetrics reportNodeTime(long timeNs) + { + return defaultTimeMetric("query/node/time", timeNs); + } + + private QueryMetrics defaultTimeMetric(String metricName, long timeNs) + { + metrics.put(metricName, TimeUnit.NANOSECONDS.toMillis(timeNs)); + return this; + } + + @Override + public QueryMetrics reportNodeBytes(long byteCount) + { + metrics.put("query/node/bytes", byteCount); + return this; + } + + @Override + public void emit(ServiceEmitter emitter) + { + for (Map.Entry metric : metrics.entrySet()) { + emitter.emit(builder.build(metric.getKey(), metric.getValue())); + } + metrics.clear(); + } + + private static > DataSourceWithSegmentSpec getOnlyDataSourceWithSegmentSpec( + QueryType query + ) + { + return Iterables.getOnlyElement(query.getDataSources()); + } } diff --git a/processing/src/main/java/io/druid/query/GenericQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/GenericQueryMetricsFactory.java index 4c8cd44e4c9b..dd4297606da0 100644 --- a/processing/src/main/java/io/druid/query/GenericQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/GenericQueryMetricsFactory.java @@ -23,11 +23,11 @@ * This factory is used for DI of custom {@link QueryMetrics} implementations for all query types, which don't (yet) * need to emit custom dimensions and/or metrics, i. e. they are good with the generic {@link QueryMetrics} interface. */ -public interface GenericQueryMetricsFactory> +public interface GenericQueryMetricsFactory { /** * Creates a {@link QueryMetrics} for query, which doesn't have predefined QueryMetrics subclass. This method must * call {@link QueryMetrics#query(Query)} with the given query on the created QueryMetrics object before returning. */ - QueryMetrics makeMetrics(QueryType query); + QueryMetrics> makeMetrics(Query query); } diff --git a/processing/src/main/java/io/druid/query/Query.java b/processing/src/main/java/io/druid/query/Query.java index 390e5b8a7bc6..4bba656b1859 100644 --- a/processing/src/main/java/io/druid/query/Query.java +++ b/processing/src/main/java/io/druid/query/Query.java @@ -82,6 +82,11 @@ default Sequence run(QueryRunner runner, Map context) Duration getDuration(DataSource dataSource); + default Duration getDuration(DataSourceWithSegmentSpec dataSourceWithSegmentSpec) + { + return getDuration(dataSourceWithSegmentSpec.getDataSource()); + } + default Duration getTotalDuration() { Duration totalDuration = new Duration(0); diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index c25031e2cc99..e724919de8e6 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -145,6 +145,10 @@ public interface QueryMetrics> */ void query(QueryType query); + void dataSourcesAndDurations(QueryType query); + + void intervals(QueryType query); + /** * Sets {@link Query#getType()} of the given query as dimension. */ @@ -155,10 +159,10 @@ public interface QueryMetrics> */ void hasFilters(QueryType query); - /** - * Sets {@link Query#getTotalDuration()} of the given query as dimension. - */ - void duration(QueryType query); +// /** +// * Sets {@link Query#getTotalDuration()} of the given query as dimension. +// */ +// void duration(QueryType query); /** * Sets {@link Query#getId()} of the given query as dimension. diff --git a/processing/src/main/java/io/druid/query/join/DefaultJoinQueryMetrics.java b/processing/src/main/java/io/druid/query/join/DefaultJoinQueryMetrics.java index 41f5c65e83de..96ea5a02c78a 100644 --- a/processing/src/main/java/io/druid/query/join/DefaultJoinQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/join/DefaultJoinQueryMetrics.java @@ -20,11 +20,12 @@ package io.druid.query.join; import com.fasterxml.jackson.databind.ObjectMapper; -import io.druid.query.AbstractQueryMetrics; import io.druid.query.DataSourceUtil; import io.druid.query.DataSourceWithSegmentSpec; +import io.druid.query.DefaultQueryMetrics; +import io.druid.query.DruidMetrics; -public class DefaultJoinQueryMetrics extends AbstractQueryMetrics implements JoinQueryMetrics +public class DefaultJoinQueryMetrics extends DefaultQueryMetrics implements JoinQueryMetrics { public DefaultJoinQueryMetrics(ObjectMapper jsonMapper) { @@ -42,6 +43,26 @@ public void query(JoinQuery query) queryId(query); } + @Override + public void dataSourcesAndDurations(JoinQuery query) + { + builder.setDimension( + "dataSourcesAndDurations", + DataSourceUtil.getMetricName(query.getDataSources()) + ); + } + + @Override + public void intervals(JoinQuery query) + { + builder.setDimension( + DruidMetrics.INTERVAL, + query.getDataSources().stream() + .map(spec -> spec.getQuerySegmentSpec().getIntervals().toString()) + .toArray(String[]::new) + ); + } + @Override public void numDataSources(JoinQuery query) { diff --git a/processing/src/main/java/io/druid/query/join/JoinQuery.java b/processing/src/main/java/io/druid/query/join/JoinQuery.java index 7a915ded33ce..fa6dbecea4ac 100644 --- a/processing/src/main/java/io/druid/query/join/JoinQuery.java +++ b/processing/src/main/java/io/druid/query/join/JoinQuery.java @@ -52,7 +52,7 @@ public class JoinQuery extends MultiSourceBaseQuery private final VirtualColumns virtualColumns; private final DimFilter filter; - public static Builder newBuilder() + public static Builder builder() { return new Builder(); } diff --git a/processing/src/test/java/io/druid/query/join/DefaultJoinQueryMetricsTest.java b/processing/src/test/java/io/druid/query/join/DefaultJoinQueryMetricsTest.java new file mode 100644 index 000000000000..0eefd51ec5b9 --- /dev/null +++ b/processing/src/test/java/io/druid/query/join/DefaultJoinQueryMetricsTest.java @@ -0,0 +1,137 @@ +/* + * 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.join; + +import com.google.common.collect.ImmutableList; +import com.metamx.emitter.service.ServiceEmitter; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.CachingEmitter; +import io.druid.query.DataSourceWithSegmentSpec; +import io.druid.query.DefaultQueryMetricsTest; +import io.druid.query.DruidMetrics; +import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.TableDataSource; +import io.druid.query.dimension.DefaultDimensionSpec; +import io.druid.segment.VirtualColumns; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Map; + +public class DefaultJoinQueryMetricsTest +{ + + @Test + public void testDefaultJoinQueryMetricsQuery() + { + CachingEmitter cachingEmitter = new CachingEmitter(); + ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); + DefaultJoinQueryMetrics queryMetrics = new DefaultJoinQueryMetrics(new DefaultObjectMapper()); + final JoinSpec leftChildSpec = new JoinSpec( + JoinType.INNER, + new AndPredicate( + ImmutableList.of( + new EqualPredicate( + new DimExtractPredicate(new DefaultDimensionSpec("src1", "dim1", "dim1")), + new DimExtractPredicate(new DefaultDimensionSpec("src2", "dim1", "dim1")) + ), + new EqualPredicate( + new AddPredicate( + new DimExtractPredicate(new DefaultDimensionSpec("src2", "dim2", "dim2")), + new LiteralPredicate("10") + ), + new AddPredicate( + new DimExtractPredicate(new DefaultDimensionSpec("src1", "dim2", "dim2")), + new DimExtractPredicate(new DefaultDimensionSpec("src1", "dim3", "dim3")) + ) + ) + ) + ), + new DataInput(new TableDataSource("src1"), QueryRunnerTestHelper.firstToThird), + new DataInput(new TableDataSource("src2"), QueryRunnerTestHelper.firstToThird) + ); + + final JoinSpec joinSpec = new JoinSpec( + JoinType.INNER, + new EqualPredicate( + new DimExtractPredicate(new DefaultDimensionSpec("j1", "dim4", "dim4")), + new DimExtractPredicate(new DefaultDimensionSpec("src3", "dim4", "dim4")) + ), + leftChildSpec, + new DataInput(new TableDataSource("src3"), QueryRunnerTestHelper.firstToThird) + ); + + JoinQuery query = JoinQuery.builder() + .setJoinSpec(joinSpec) + .setGranularity(QueryRunnerTestHelper.dayGran) + .setDimensions( + ImmutableList.of( + new DefaultDimensionSpec("src1", "dim5", "dim5"), + new DefaultDimensionSpec("src2", "dim5", "dim5"), + new DefaultDimensionSpec("src3", "dim5", "dim5") + ) + ) + .setMetrics( + ImmutableList.of( + "met1", "met2", "met3" + ) + ) + .setVirtualColumns(VirtualColumns.EMPTY) + .build(); + final DataSourceWithSegmentSpec distributionTarget = query.getDataSources().get(0); + query = (JoinQuery) query.distributeBy(distributionTarget); + queryMetrics.query(query); + + queryMetrics.reportQueryTime(0).emit(serviceEmitter); + Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); + Assert.assertEquals(12, actualEvent.size()); + Assert.assertTrue(actualEvent.containsKey("feed")); + Assert.assertTrue(actualEvent.containsKey("timestamp")); + Assert.assertEquals("", actualEvent.get("host")); + Assert.assertEquals("", actualEvent.get("service")); + Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE)); + Assert.assertEquals("false", actualEvent.get("hasFilters")); + Assert.assertEquals("", actualEvent.get(DruidMetrics.ID)); + + // Join-specific dimensions + Assert.assertEquals("3", actualEvent.get("numDataSources")); + Assert.assertEquals( + distributionTarget.getDataSource().toString(), + actualEvent.get("distributionTarget") + ); + Assert.assertEquals( + query.getDuration(distributionTarget).toString(), + actualEvent.get("distributionTargetDuration") + ); + + // Metric + Assert.assertEquals("query/time", actualEvent.get("metric")); + Assert.assertEquals(0L, actualEvent.get("value")); + } + + @Test + public void testDefaultJoinQueryMetricsMetricNamesAndUnits() + { + CachingEmitter cachingEmitter = new CachingEmitter(); + ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); + DefaultJoinQueryMetrics queryMetrics = new DefaultJoinQueryMetrics(new DefaultObjectMapper()); + DefaultQueryMetricsTest.testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics); + } +} \ No newline at end of file diff --git a/processing/src/test/java/io/druid/query/join/JoinQueryTest.java b/processing/src/test/java/io/druid/query/join/JoinQueryTest.java index c731b47887a1..f9a871a79aa1 100644 --- a/processing/src/test/java/io/druid/query/join/JoinQueryTest.java +++ b/processing/src/test/java/io/druid/query/join/JoinQueryTest.java @@ -74,7 +74,7 @@ public void testSerde() throws IOException new DataInput(new TableDataSource("src3"), QueryRunnerTestHelper.firstToThird) ); - final Query query = JoinQuery.newBuilder() + final Query query = JoinQuery.builder() .setJoinSpec(joinSpec) .setGranularity(QueryRunnerTestHelper.dayGran) .setDimensions( From d8d1e3ec12b27e0426b3139dedcac1b7ab0a1702 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 27 Apr 2017 14:25:46 +0900 Subject: [PATCH 13/15] Fix test failure --- .../src/main/java/io/druid/query/groupby/GroupByQuery.java | 3 ++- .../src/main/java/io/druid/query/topn/TopNQueryBuilder.java | 3 ++- .../java/io/druid/query/join/DefaultJoinQueryMetricsTest.java | 2 +- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java index 3f8cf3c60a74..c7b2d6f3f49e 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java @@ -71,6 +71,7 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.TreeMap; /** */ @@ -472,7 +473,7 @@ public Builder(GroupByQuery query) havingSpec = query.getHavingSpec(); limitSpec = query.getLimitSpec(); postProcessingFn = query.postProcessingFn; - context = query.getContext(); + context = new TreeMap<>(query.getContext()); } public Builder(Builder builder) diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java b/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java index 0016bd62064e..a012cedea924 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java @@ -44,6 +44,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.TreeMap; /** * A Builder for TopNQuery. @@ -107,7 +108,7 @@ public TopNQueryBuilder(final TopNQuery query) this.granularity = query.getGranularity(); this.aggregatorSpecs = query.getAggregatorSpecs(); this.postAggregatorSpecs = query.getPostAggregatorSpecs(); - this.context = query.getContext(); + this.context = new TreeMap<>(query.getContext()); } public DataSource getDataSource() diff --git a/processing/src/test/java/io/druid/query/join/DefaultJoinQueryMetricsTest.java b/processing/src/test/java/io/druid/query/join/DefaultJoinQueryMetricsTest.java index 0eefd51ec5b9..d1908666fe06 100644 --- a/processing/src/test/java/io/druid/query/join/DefaultJoinQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/join/DefaultJoinQueryMetricsTest.java @@ -134,4 +134,4 @@ public void testDefaultJoinQueryMetricsMetricNamesAndUnits() DefaultJoinQueryMetrics queryMetrics = new DefaultJoinQueryMetrics(new DefaultObjectMapper()); DefaultQueryMetricsTest.testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics); } -} \ No newline at end of file +} From cb02d239940cf99b968dade35710cc5a9df15041 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 3 May 2017 09:49:03 +0900 Subject: [PATCH 14/15] fix wrong DruidJoinQueryMetrics and remove commented codes --- .../java/io/druid/query/DataSourceUtil.java | 7 -- .../query/DataSourceWithSegmentSpec.java | 7 ++ .../io/druid/query/DefaultQueryMetrics.java | 19 --- .../java/io/druid/query/QueryMetrics.java | 5 - .../query/join/DefaultJoinQueryMetrics.java | 7 +- .../io/druid/query/DataSourceUtilTest.java | 40 ------ .../join/DefaultJoinQueryMetricsTest.java | 114 +++++++++++++++--- 7 files changed, 111 insertions(+), 88 deletions(-) diff --git a/processing/src/main/java/io/druid/query/DataSourceUtil.java b/processing/src/main/java/io/druid/query/DataSourceUtil.java index c591e03884dc..c624bf87ce48 100644 --- a/processing/src/main/java/io/druid/query/DataSourceUtil.java +++ b/processing/src/main/java/io/druid/query/DataSourceUtil.java @@ -19,8 +19,6 @@ package io.druid.query; -import com.google.common.collect.Iterables; - import java.util.List; public class DataSourceUtil @@ -30,9 +28,4 @@ public static String getMetricName(DataSource dataSource) final List names = dataSource.getNames(); return names.size() == 1 ? names.get(0) : names.toString(); } - - public static String getMetricName(List dataSources) - { - return Iterables.toString(dataSources); - } } diff --git a/processing/src/main/java/io/druid/query/DataSourceWithSegmentSpec.java b/processing/src/main/java/io/druid/query/DataSourceWithSegmentSpec.java index 0177e5a77532..32d0995170f6 100644 --- a/processing/src/main/java/io/druid/query/DataSourceWithSegmentSpec.java +++ b/processing/src/main/java/io/druid/query/DataSourceWithSegmentSpec.java @@ -24,7 +24,9 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; import io.druid.query.spec.QuerySegmentSpec; +import org.joda.time.Interval; +import java.util.List; import java.util.Objects; /** @@ -59,6 +61,11 @@ public QuerySegmentSpec getQuerySegmentSpec() return querySegmentSpec; } + public List getInterval() + { + return querySegmentSpec.getIntervals(); + } + @Override public int hashCode() { diff --git a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java index 9a31ae038eb6..9ee9dc9fe571 100644 --- a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java @@ -45,26 +45,13 @@ public DefaultQueryMetrics(ObjectMapper jsonMapper) @Override public void query(QueryType query) { -// dataSource(query); dataSourcesAndDurations(query); queryType(query); intervals(query); hasFilters(query); -// duration(query); queryId(query); } -// /** -// * Sets {@link BaseQuery#getDataSource()} of the given query as dimension. -// */ -// public void dataSource(QueryType query) -// { -// builder.setDimension( -// DruidMetrics.DATASOURCE, -// DataSourceUtil.getMetricName(query.getDataSource()) -// ); -// } - @Override public void dataSourcesAndDurations(QueryType query) { @@ -97,12 +84,6 @@ public void hasFilters(QueryType query) builder.setDimension("hasFilters", String.valueOf(query.hasFilters())); } -// @Override -// public void duration(QueryType query) -// { -// builder.setDimension("duration", query.getTotalDuration().toString()); -// } - @Override public void queryId(QueryType query) { diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index e724919de8e6..c571f2058016 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -159,11 +159,6 @@ public interface QueryMetrics> */ void hasFilters(QueryType query); -// /** -// * Sets {@link Query#getTotalDuration()} of the given query as dimension. -// */ -// void duration(QueryType query); - /** * Sets {@link Query#getId()} of the given query as dimension. */ diff --git a/processing/src/main/java/io/druid/query/join/DefaultJoinQueryMetrics.java b/processing/src/main/java/io/druid/query/join/DefaultJoinQueryMetrics.java index 96ea5a02c78a..9ebafd80280c 100644 --- a/processing/src/main/java/io/druid/query/join/DefaultJoinQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/join/DefaultJoinQueryMetrics.java @@ -48,7 +48,12 @@ public void dataSourcesAndDurations(JoinQuery query) { builder.setDimension( "dataSourcesAndDurations", - DataSourceUtil.getMetricName(query.getDataSources()) + query.getDataSources().stream() + .map( + spec -> "{dataSource=" + DataSourceUtil.getMetricName(spec.getDataSource()) + + ",duration=" + query.getDuration(spec.getDataSource()) + "}" + ) + .toArray(String[]::new) ); } diff --git a/processing/src/test/java/io/druid/query/DataSourceUtilTest.java b/processing/src/test/java/io/druid/query/DataSourceUtilTest.java index 56de6cb8e392..0fd0bb491f7b 100644 --- a/processing/src/test/java/io/druid/query/DataSourceUtilTest.java +++ b/processing/src/test/java/io/druid/query/DataSourceUtilTest.java @@ -20,13 +20,9 @@ package io.druid.query; import com.google.common.collect.Lists; -import io.druid.query.spec.MultipleIntervalSegmentSpec; -import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; -import java.util.List; - public class DataSourceUtilTest { @@ -46,40 +42,4 @@ public void testGetMetricNameFromDataSource() ); Assert.assertEquals("[src1, src2, src3, src4]", DataSourceUtil.getMetricName(unionSource)); } - - @Test - public void testGetMetricNameFromDataSourceWithSegmentSpec() - { - final List specs = Lists.newArrayList( - new DataSourceWithSegmentSpec( - new TableDataSource("src1"), - new MultipleIntervalSegmentSpec( - Lists.newArrayList( - new Interval(0, 100), - new Interval(200, 300), - new Interval(400, 500) - ) - ) - ), - new DataSourceWithSegmentSpec( - new TableDataSource("src2"), - new MultipleIntervalSegmentSpec( - Lists.newArrayList( - new Interval(0, 10), - new Interval(20, 30), - new Interval(40, 50) - ) - ) - ) - ); - Assert.assertEquals( - "[src1=[1970-01-01T00:00:00.000Z/1970-01-01T00:00:00.100Z, " - + "1970-01-01T00:00:00.200Z/1970-01-01T00:00:00.300Z, " - + "1970-01-01T00:00:00.400Z/1970-01-01T00:00:00.500Z], " - + "src2=[1970-01-01T00:00:00.000Z/1970-01-01T00:00:00.010Z, " - + "1970-01-01T00:00:00.020Z/1970-01-01T00:00:00.030Z, " - + "1970-01-01T00:00:00.040Z/1970-01-01T00:00:00.050Z]]", - DataSourceUtil.getMetricName(specs) - ); - } } diff --git a/processing/src/test/java/io/druid/query/join/DefaultJoinQueryMetricsTest.java b/processing/src/test/java/io/druid/query/join/DefaultJoinQueryMetricsTest.java index d1908666fe06..8b204ccd629f 100644 --- a/processing/src/test/java/io/druid/query/join/DefaultJoinQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/join/DefaultJoinQueryMetricsTest.java @@ -79,22 +79,22 @@ public void testDefaultJoinQueryMetricsQuery() ); JoinQuery query = JoinQuery.builder() - .setJoinSpec(joinSpec) - .setGranularity(QueryRunnerTestHelper.dayGran) - .setDimensions( - ImmutableList.of( - new DefaultDimensionSpec("src1", "dim5", "dim5"), - new DefaultDimensionSpec("src2", "dim5", "dim5"), - new DefaultDimensionSpec("src3", "dim5", "dim5") - ) - ) - .setMetrics( - ImmutableList.of( - "met1", "met2", "met3" - ) - ) - .setVirtualColumns(VirtualColumns.EMPTY) - .build(); + .setJoinSpec(joinSpec) + .setGranularity(QueryRunnerTestHelper.dayGran) + .setDimensions( + ImmutableList.of( + new DefaultDimensionSpec("src1", "dim5", "dim5"), + new DefaultDimensionSpec("src2", "dim5", "dim5"), + new DefaultDimensionSpec("src3", "dim5", "dim5") + ) + ) + .setMetrics( + ImmutableList.of( + "met1", "met2", "met3" + ) + ) + .setVirtualColumns(VirtualColumns.EMPTY) + .build(); final DataSourceWithSegmentSpec distributionTarget = query.getDataSources().get(0); query = (JoinQuery) query.distributeBy(distributionTarget); queryMetrics.query(query); @@ -134,4 +134,86 @@ public void testDefaultJoinQueryMetricsMetricNamesAndUnits() DefaultJoinQueryMetrics queryMetrics = new DefaultJoinQueryMetrics(new DefaultObjectMapper()); DefaultQueryMetricsTest.testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics); } + + @Test + public void testDataSourcesAndDurationsAndIntervals() + { + CachingEmitter cachingEmitter = new CachingEmitter(); + ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); + DefaultJoinQueryMetrics queryMetrics = new DefaultJoinQueryMetrics(new DefaultObjectMapper()); + final JoinSpec leftChildSpec = new JoinSpec( + JoinType.INNER, + new AndPredicate( + ImmutableList.of( + new EqualPredicate( + new DimExtractPredicate(new DefaultDimensionSpec("src1", "dim1", "dim1")), + new DimExtractPredicate(new DefaultDimensionSpec("src2", "dim1", "dim1")) + ), + new EqualPredicate( + new AddPredicate( + new DimExtractPredicate(new DefaultDimensionSpec("src2", "dim2", "dim2")), + new LiteralPredicate("10") + ), + new AddPredicate( + new DimExtractPredicate(new DefaultDimensionSpec("src1", "dim2", "dim2")), + new DimExtractPredicate(new DefaultDimensionSpec("src1", "dim3", "dim3")) + ) + ) + ) + ), + new DataInput(new TableDataSource("src1"), QueryRunnerTestHelper.firstToThird), + new DataInput(new TableDataSource("src2"), QueryRunnerTestHelper.firstToThird) + ); + + final JoinSpec joinSpec = new JoinSpec( + JoinType.INNER, + new EqualPredicate( + new DimExtractPredicate(new DefaultDimensionSpec("j1", "dim4", "dim4")), + new DimExtractPredicate(new DefaultDimensionSpec("src3", "dim4", "dim4")) + ), + leftChildSpec, + new DataInput(new TableDataSource("src3"), QueryRunnerTestHelper.firstToThird) + ); + + JoinQuery query = JoinQuery.builder().setJoinSpec(joinSpec) + .setGranularity(QueryRunnerTestHelper.dayGran) + .setDimensions( + ImmutableList.of( + new DefaultDimensionSpec("src1", "dim5", "dim5"), + new DefaultDimensionSpec("src2", "dim5", "dim5"), + new DefaultDimensionSpec("src3", "dim5", "dim5") + ) + ) + .setMetrics( + ImmutableList.of( + "met1", "met2", "met3" + ) + ) + .setVirtualColumns(VirtualColumns.EMPTY) + .build(); + final DataSourceWithSegmentSpec distributionTarget = query.getDataSources().get(0); + query = (JoinQuery) query.distributeBy(distributionTarget); + queryMetrics.dataSourcesAndDurations(query); + queryMetrics.intervals(query); + + queryMetrics.reportQueryTime(0).emit(serviceEmitter); + Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); + Assert.assertEquals(8, actualEvent.size()); + Assert.assertEquals( + ImmutableList.of( + "{dataSource=src1,duration=PT172800S}", + "{dataSource=src2,duration=PT172800S}", + "{dataSource=src3,duration=PT172800S}" + ), + actualEvent.get("dataSourcesAndDurations") + ); + Assert.assertEquals( + ImmutableList.of( + "[2011-04-01T00:00:00.000Z/2011-04-03T00:00:00.000Z]", + "[2011-04-01T00:00:00.000Z/2011-04-03T00:00:00.000Z]", + "[2011-04-01T00:00:00.000Z/2011-04-03T00:00:00.000Z]" + ), + actualEvent.get(DruidMetrics.INTERVAL) + ); + } } From 838cea21717bb4a1b71378190d4423252f648996 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 3 May 2017 14:30:52 +0900 Subject: [PATCH 15/15] Rename DimExtractPredicate to DimensionPredicate --- ...Predicate.java => DimensionPredicate.java} | 6 ++-- .../io/druid/query/join/JoinPredicate.java | 2 +- .../query/join/JoinPredicateVisitor.java | 2 +- .../join/DefaultJoinQueryMetricsTest.java | 28 +++++++++---------- .../io/druid/query/join/JoinQueryTest.java | 14 +++++----- 5 files changed, 26 insertions(+), 26 deletions(-) rename processing/src/main/java/io/druid/query/join/{DimExtractPredicate.java => DimensionPredicate.java} (92%) diff --git a/processing/src/main/java/io/druid/query/join/DimExtractPredicate.java b/processing/src/main/java/io/druid/query/join/DimensionPredicate.java similarity index 92% rename from processing/src/main/java/io/druid/query/join/DimExtractPredicate.java rename to processing/src/main/java/io/druid/query/join/DimensionPredicate.java index 4c65a04d17fc..3d94673725bc 100644 --- a/processing/src/main/java/io/druid/query/join/DimExtractPredicate.java +++ b/processing/src/main/java/io/druid/query/join/DimensionPredicate.java @@ -26,12 +26,12 @@ import java.util.Objects; -public class DimExtractPredicate implements JoinPredicate +public class DimensionPredicate implements JoinPredicate { private final DimensionSpec dimension; @JsonCreator - public DimExtractPredicate( + public DimensionPredicate( @JsonProperty("dimension") DimensionSpec dimension ) { @@ -67,7 +67,7 @@ public boolean equals(Object o) return false; } - DimExtractPredicate that = (DimExtractPredicate) o; + DimensionPredicate that = (DimensionPredicate) o; return dimension.equals(that.dimension); } diff --git a/processing/src/main/java/io/druid/query/join/JoinPredicate.java b/processing/src/main/java/io/druid/query/join/JoinPredicate.java index 71aa844ddc48..13898bc0182f 100644 --- a/processing/src/main/java/io/druid/query/join/JoinPredicate.java +++ b/processing/src/main/java/io/druid/query/join/JoinPredicate.java @@ -30,7 +30,7 @@ @Type(name = "or", value = OrPredicate.class), @Type(name = "not", value = NotPredicate.class), @Type(name = "equal", value = EqualPredicate.class), - @Type(name = "dimension", value = DimExtractPredicate.class), + @Type(name = "dimension", value = DimensionPredicate.class), @Type(name = "add", value = AddPredicate.class), @Type(name = "subtract", value = SubtractPredicate.class), @Type(name = "multiply", value = MultiplyPredicate.class), diff --git a/processing/src/main/java/io/druid/query/join/JoinPredicateVisitor.java b/processing/src/main/java/io/druid/query/join/JoinPredicateVisitor.java index c7380c3b01e5..c09dfbec05f4 100644 --- a/processing/src/main/java/io/druid/query/join/JoinPredicateVisitor.java +++ b/processing/src/main/java/io/druid/query/join/JoinPredicateVisitor.java @@ -43,7 +43,7 @@ default JoinPredicate visit(NotPredicate predicate) return predicate; } - default JoinPredicate visit(DimExtractPredicate predicate) + default JoinPredicate visit(DimensionPredicate predicate) { return predicate; } diff --git a/processing/src/test/java/io/druid/query/join/DefaultJoinQueryMetricsTest.java b/processing/src/test/java/io/druid/query/join/DefaultJoinQueryMetricsTest.java index b321552f2359..a9521620912b 100644 --- a/processing/src/test/java/io/druid/query/join/DefaultJoinQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/join/DefaultJoinQueryMetricsTest.java @@ -49,17 +49,17 @@ public void testDefaultJoinQueryMetricsQuery() new AndPredicate( ImmutableList.of( new EqualPredicate( - new DimExtractPredicate(new DefaultDimensionSpec("src1", "dim1", "dim1")), - new DimExtractPredicate(new DefaultDimensionSpec("src2", "dim1", "dim1")) + new DimensionPredicate(new DefaultDimensionSpec("src1", "dim1", "dim1")), + new DimensionPredicate(new DefaultDimensionSpec("src2", "dim1", "dim1")) ), new EqualPredicate( new AddPredicate( - new DimExtractPredicate(new DefaultDimensionSpec("src2", "dim2", "dim2")), + new DimensionPredicate(new DefaultDimensionSpec("src2", "dim2", "dim2")), new LiteralPredicate("10") ), new AddPredicate( - new DimExtractPredicate(new DefaultDimensionSpec("src1", "dim2", "dim2")), - new DimExtractPredicate(new DefaultDimensionSpec("src1", "dim3", "dim3")) + new DimensionPredicate(new DefaultDimensionSpec("src1", "dim2", "dim2")), + new DimensionPredicate(new DefaultDimensionSpec("src1", "dim3", "dim3")) ) ) ) @@ -71,8 +71,8 @@ public void testDefaultJoinQueryMetricsQuery() final JoinSpec joinSpec = new JoinSpec( JoinType.INNER, new EqualPredicate( - new DimExtractPredicate(new DefaultDimensionSpec("j1", "dim4", "dim4")), - new DimExtractPredicate(new DefaultDimensionSpec("src3", "dim4", "dim4")) + new DimensionPredicate(new DefaultDimensionSpec("j1", "dim4", "dim4")), + new DimensionPredicate(new DefaultDimensionSpec("src3", "dim4", "dim4")) ), leftChildSpec, new DataSourceJoinInputSpec(new TableDataSource("src3"), QueryRunnerTestHelper.firstToThird) @@ -146,17 +146,17 @@ public void testDataSourcesAndDurationsAndIntervals() new AndPredicate( ImmutableList.of( new EqualPredicate( - new DimExtractPredicate(new DefaultDimensionSpec("src1", "dim1", "dim1")), - new DimExtractPredicate(new DefaultDimensionSpec("src2", "dim1", "dim1")) + new DimensionPredicate(new DefaultDimensionSpec("src1", "dim1", "dim1")), + new DimensionPredicate(new DefaultDimensionSpec("src2", "dim1", "dim1")) ), new EqualPredicate( new AddPredicate( - new DimExtractPredicate(new DefaultDimensionSpec("src2", "dim2", "dim2")), + new DimensionPredicate(new DefaultDimensionSpec("src2", "dim2", "dim2")), new LiteralPredicate("10") ), new AddPredicate( - new DimExtractPredicate(new DefaultDimensionSpec("src1", "dim2", "dim2")), - new DimExtractPredicate(new DefaultDimensionSpec("src1", "dim3", "dim3")) + new DimensionPredicate(new DefaultDimensionSpec("src1", "dim2", "dim2")), + new DimensionPredicate(new DefaultDimensionSpec("src1", "dim3", "dim3")) ) ) ) @@ -168,8 +168,8 @@ public void testDataSourcesAndDurationsAndIntervals() final JoinSpec joinSpec = new JoinSpec( JoinType.INNER, new EqualPredicate( - new DimExtractPredicate(new DefaultDimensionSpec("j1", "dim4", "dim4")), - new DimExtractPredicate(new DefaultDimensionSpec("src3", "dim4", "dim4")) + new DimensionPredicate(new DefaultDimensionSpec("j1", "dim4", "dim4")), + new DimensionPredicate(new DefaultDimensionSpec("src3", "dim4", "dim4")) ), leftChildSpec, new DataSourceJoinInputSpec(new TableDataSource("src3"), QueryRunnerTestHelper.firstToThird) diff --git a/processing/src/test/java/io/druid/query/join/JoinQueryTest.java b/processing/src/test/java/io/druid/query/join/JoinQueryTest.java index 90eba09e9e24..7f31482fd37a 100644 --- a/processing/src/test/java/io/druid/query/join/JoinQueryTest.java +++ b/processing/src/test/java/io/druid/query/join/JoinQueryTest.java @@ -45,17 +45,17 @@ public void testSerde() throws IOException new AndPredicate( ImmutableList.of( new EqualPredicate( - new DimExtractPredicate(new DefaultDimensionSpec("src1", "dim1", "dim1")), - new DimExtractPredicate(new DefaultDimensionSpec("src2", "dim1", "dim1")) + new DimensionPredicate(new DefaultDimensionSpec("src1", "dim1", "dim1")), + new DimensionPredicate(new DefaultDimensionSpec("src2", "dim1", "dim1")) ), new EqualPredicate( new AddPredicate( - new DimExtractPredicate(new DefaultDimensionSpec("src2", "dim2", "dim2")), + new DimensionPredicate(new DefaultDimensionSpec("src2", "dim2", "dim2")), new LiteralPredicate("10") ), new AddPredicate( - new DimExtractPredicate(new DefaultDimensionSpec("src1", "dim2", "dim2")), - new DimExtractPredicate(new DefaultDimensionSpec("src1", "dim3", "dim3")) + new DimensionPredicate(new DefaultDimensionSpec("src1", "dim2", "dim2")), + new DimensionPredicate(new DefaultDimensionSpec("src1", "dim3", "dim3")) ) ) ) @@ -67,8 +67,8 @@ public void testSerde() throws IOException final JoinSpec joinSpec = new JoinSpec( JoinType.INNER, new EqualPredicate( - new DimExtractPredicate(new DefaultDimensionSpec("j1", "dim4", "dim4")), - new DimExtractPredicate(new DefaultDimensionSpec("src3", "dim4", "dim4")) + new DimensionPredicate(new DefaultDimensionSpec("j1", "dim4", "dim4")), + new DimensionPredicate(new DefaultDimensionSpec("src3", "dim4", "dim4")) ), leftChildSpec, new DataSourceJoinInputSpec(new TableDataSource("src3"), QueryRunnerTestHelper.firstToThird)