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 a526868c3639..228e4f933f61 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 @@ -25,7 +25,9 @@ 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.QueryContexts; import io.druid.query.TableDataSource; import io.druid.query.filter.DimFilter; import io.druid.query.filter.InDimFilter; @@ -35,6 +37,7 @@ import org.joda.time.Interval; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -137,7 +140,8 @@ public Query withDataSource(DataSource dataSource) @Override public Query withOverriddenContext(Map contextOverrides) { - return ScanQueryBuilder.copy(this).context(computeOverriddenContext(getContext(), contextOverrides)).build(); + return ScanQueryBuilder.copy(this).context(QueryContexts.computeOverriddenContext(getContext(), contextOverrides)) + .build(); } public ScanQuery withDimFilter(DimFilter dimFilter) @@ -279,6 +283,21 @@ public ScanQueryBuilder dataSource(DataSource ds) return this; } + public ScanQueryBuilder updateDistributionTarget() + { + if (context == null) { + context = new HashMap<>(); + } + context.put( + QueryContexts.DISTRIBUTION_TARGET_SOURCE, + new DataSourceWithSegmentSpec( + BaseQuery.getLeafDataSource(dataSource), + querySegmentSpec + ) + ); + return this; + } + public ScanQueryBuilder intervals(QuerySegmentSpec q) { querySegmentSpec = q; @@ -299,7 +318,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..7dc780f2a438 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; 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/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/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/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 9539a85b377f..b3d8d57634af 100644 --- a/processing/src/main/java/io/druid/query/BaseQuery.java +++ b/processing/src/main/java/io/druid/query/BaseQuery.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import io.druid.java.util.common.guava.Sequence; @@ -43,7 +43,6 @@ public static void checkInterrupted() } } - public static final String QUERYID = "queryId"; private final DataSource dataSource; private final boolean descending; private final Map context; @@ -61,18 +60,23 @@ public BaseQuery( Preconditions.checkNotNull(querySegmentSpec, "querySegmentSpec can't be null"); this.dataSource = dataSource; - this.context = context; + this.context = context == null ? Maps.newTreeMap() : context; this.querySegmentSpec = querySegmentSpec; this.descending = descending; } @JsonProperty - @Override public DataSource getDataSource() { return dataSource; } + @Override + public List getDataSources() + { + return ImmutableList.of(new DataSourceWithSegmentSpec(dataSource, querySegmentSpec)); + } + @JsonProperty @Override public boolean isDescending() @@ -92,18 +96,18 @@ public Sequence run(QuerySegmentWalker walker, Map context) return run(querySegmentSpec.lookup(this, walker), context); } - public Sequence run(QueryRunner runner, Map context) + public List getIntervals() { - return runner.run(this, context); + return querySegmentSpec.getIntervals(); } @Override - public List getIntervals() + public Duration getDuration(DataSource dataSource) { - return querySegmentSpec.getIntervals(); + Preconditions.checkArgument(this.dataSource.equals(dataSource)); + return getDuration(); } - @Override public Duration getDuration() { if (duration == null) { @@ -126,49 +130,6 @@ public Map getContext() return context; } - @Override - public ContextType getContextValue(String key) - { - return context == null ? null : (ContextType) context.get(key); - } - - @Override - public ContextType getContextValue(String key, ContextType defaultValue) - { - ContextType retVal = getContextValue(key); - return retVal == null ? defaultValue : retVal; - } - - @Override - public boolean getContextBoolean(String key, boolean defaultValue) - { - return QueryContexts.parseBoolean(this, key, defaultValue); - } - - /** - * @deprecated use {@link #computeOverriddenContext(Map, Map) computeOverriddenContext(getContext(), overrides))} - * instead. This method may be removed in the next minor or major version of Druid. - */ - @Deprecated - protected Map computeOverridenContext(final Map overrides) - { - return computeOverriddenContext(getContext(), overrides); - } - - protected static Map computeOverriddenContext( - final Map context, - final Map overrides - ) - { - Map overridden = Maps.newTreeMap(); - if (context != null) { - overridden.putAll(context); - } - overridden.putAll(overrides); - - return overridden; - } - @Override public Ordering getResultOrdering() { @@ -176,18 +137,6 @@ public Ordering getResultOrdering() return descending ? retVal.reverse() : retVal; } - @Override - public String getId() - { - return (String) getContextValue(QUERYID); - } - - @Override - public Query withId(String id) - { - return withOverriddenContext(ImmutableMap.of(QUERYID, id)); - } - @Override public boolean equals(Object o) { @@ -203,18 +152,16 @@ 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; } - if (dataSource != null ? !dataSource.equals(baseQuery.dataSource) : baseQuery.dataSource != null) { + if (!dataSource.equals(baseQuery.dataSource)) { return false; } if (duration != null ? !duration.equals(baseQuery.duration) : baseQuery.duration != null) { return false; } - if (querySegmentSpec != null - ? !querySegmentSpec.equals(baseQuery.querySegmentSpec) - : baseQuery.querySegmentSpec != null) { + if (!querySegmentSpec.equals(baseQuery.querySegmentSpec)) { return false; } @@ -224,11 +171,62 @@ public boolean equals(Object o) @Override public int hashCode() { - int result = dataSource != null ? dataSource.hashCode() : 0; + int result = dataSource.hashCode(); result = 31 * result + (descending ? 1 : 0); - result = 31 * result + (context != null ? context.hashCode() : 0); - result = 31 * result + (querySegmentSpec != null ? querySegmentSpec.hashCode() : 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 distributeBy(new DataSourceWithSegmentSpec(BaseQuery.getLeafDataSource(dataSource), querySegmentSpec)); + } + + @Override + public Query withQuerySegmentSpec(DataSource dataSource, QuerySegmentSpec spec) + { + Preconditions.checkArgument(this.dataSource.equals(dataSource)); + final BaseQuery result = (BaseQuery) withQuerySegmentSpec(spec); + if (getDistributionTarget() != null && getDistributionTarget().getDataSource().equals(dataSource)) { + return result.updateDistributionTarget(); + } else { + return result; + } + } + + @Override + public Query withQuerySegmentSpec(String concatenatedDataSourceName, QuerySegmentSpec spec) + { + Preconditions.checkArgument(this.dataSource.getConcatenatedName().equals(concatenatedDataSourceName)); + return withQuerySegmentSpec(this.dataSource, spec); + } + + @Override + public Query replaceDataSource(DataSource oldDataSource, DataSource newDataSource) + { + Preconditions.checkArgument(this.dataSource.equals(oldDataSource)); + return withDataSource(newDataSource); + } + + public abstract Query withQuerySegmentSpec(QuerySegmentSpec spec); + public abstract Query withDataSource(DataSource dataSource); + + public static > DataSource getLeafDataSource( + BaseQuery query + ) + { + return getLeafDataSource(query.getDataSource()); + } + + 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/BySegmentQueryRunner.java b/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java index e57b9471ce3b..382cedd4c070 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..c890c594c488 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.base.Preconditions; import java.util.List; @@ -35,5 +36,19 @@ }) public interface DataSource { - public List getNames(); + default String getConcatenatedName() + { + final List names = getNames(); + Preconditions.checkState(!names.isEmpty()); + return names.size() > 1 ? names.toString() : names.get(0); + } + + default String getFirstName() + { + final List names = getNames(); + Preconditions.checkState(!names.isEmpty()); + return names.get(0); + } + + List getNames(); } 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..32d0995170f6 --- /dev/null +++ b/processing/src/main/java/io/druid/query/DataSourceWithSegmentSpec.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; + +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 org.joda.time.Interval; + +import java.util.List; +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; + private final QuerySegmentSpec querySegmentSpec; + + @JsonCreator + public DataSourceWithSegmentSpec( + @JsonProperty("dataSource") DataSource dataSource, + @JsonProperty("intervals") QuerySegmentSpec querySegmentSpec) + { + this.dataSource = Preconditions.checkNotNull(dataSource); + this.querySegmentSpec = Preconditions.checkNotNull(querySegmentSpec); + } + + @JsonProperty + public DataSource getDataSource() + { + return dataSource; + } + + @JsonProperty + public QuerySegmentSpec getQuerySegmentSpec() + { + return querySegmentSpec; + } + + public List getInterval() + { + return querySegmentSpec.getIntervals(); + } + + @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); + } + + @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 a19053be8789..9ee9dc9fe571 100644 --- a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java @@ -22,7 +22,7 @@ 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.google.common.collect.Iterables; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import org.joda.time.Interval; @@ -45,45 +45,43 @@ public DefaultQueryMetrics(ObjectMapper jsonMapper) @Override public void query(QueryType query) { - dataSource(query); + dataSourcesAndDurations(query); queryType(query); - interval(query); + intervals(query); hasFilters(query); - duration(query); queryId(query); } @Override - public void dataSource(QueryType query) + public void dataSourcesAndDurations(QueryType query) { - builder.setDimension(DruidMetrics.DATASOURCE, DataSourceUtil.getMetricName(query.getDataSource())); - } - - @Override - public void queryType(QueryType query) - { - builder.setDimension(DruidMetrics.TYPE, query.getType()); + builder.setDimension( + DruidMetrics.DATASOURCE, + DataSourceUtil.getMetricName(getOnlyDataSourceWithSegmentSpec(query).getDataSource()) + ); + builder.setDimension("duration", query.getTotalDuration().toString()); } @Override - public void interval(QueryType query) + 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 hasFilters(QueryType query) + public void queryType(QueryType query) { - builder.setDimension("hasFilters", String.valueOf(query.hasFilters())); + builder.setDimension(DruidMetrics.TYPE, query.getType()); } @Override - public void duration(QueryType query) + public void hasFilters(QueryType query) { - builder.setDimension("duration", query.getDuration().toString()); + builder.setDimension("hasFilters", String.valueOf(query.hasFilters())); } @Override @@ -98,11 +96,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) { @@ -223,4 +217,11 @@ public void emit(ServiceEmitter emitter) } metrics.clear(); } + + private static > DataSourceWithSegmentSpec getOnlyDataSourceWithSegmentSpec( + QueryType query + ) + { + return Iterables.getOnlyElement(query.getDataSources()); + } } diff --git a/processing/src/main/java/io/druid/query/Druids.java b/processing/src/main/java/io/druid/query/Druids.java index da81fa9a80ac..c1adf3b30f83 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; @@ -433,6 +434,21 @@ public TimeseriesQueryBuilder dataSource(DataSource ds) return this; } + public TimeseriesQueryBuilder updateDistributionTarget() + { + if (context == null) { + context = new HashMap<>(); + } + context.put( + QueryContexts.DISTRIBUTION_TARGET_SOURCE, + new DataSourceWithSegmentSpec( + BaseQuery.getLeafDataSource(dataSource), + querySegmentSpec + ) + ); + return this; + } + public TimeseriesQueryBuilder intervals(QuerySegmentSpec q) { querySegmentSpec = q; @@ -517,7 +533,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; } } @@ -612,6 +632,21 @@ public SearchQueryBuilder dataSource(DataSource d) return this; } + public SearchQueryBuilder updateDistributionTarget() + { + if (context == null) { + context = new HashMap<>(); + } + context.put( + QueryContexts.DISTRIBUTION_TARGET_SOURCE, + new DataSourceWithSegmentSpec( + BaseQuery.getLeafDataSource(dataSource), + querySegmentSpec + ) + ); + return this; + } + public SearchQueryBuilder filters(String dimensionName, String value) { dimFilter = new SelectorDimFilter(dimensionName, value, null); @@ -744,7 +779,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; } } @@ -818,6 +857,21 @@ public TimeBoundaryQueryBuilder dataSource(DataSource ds) return this; } + public TimeBoundaryQueryBuilder updateDistributionTarget() + { + if (context == null) { + context = new HashMap<>(); + } + context.put( + QueryContexts.DISTRIBUTION_TARGET_SOURCE, + new DataSourceWithSegmentSpec( + BaseQuery.getLeafDataSource(dataSource), + querySegmentSpec + ) + ); + return this; + } + public TimeBoundaryQueryBuilder intervals(QuerySegmentSpec q) { querySegmentSpec = q; @@ -862,7 +916,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; } } @@ -1163,6 +1221,21 @@ public SelectQueryBuilder dataSource(DataSource ds) return this; } + public SelectQueryBuilder updateDistributionTarget() + { + if (context == null) { + context = new HashMap<>(); + } + context.put( + QueryContexts.DISTRIBUTION_TARGET_SOURCE, + new DataSourceWithSegmentSpec( + BaseQuery.getLeafDataSource(dataSource), + querySegmentSpec + ) + ); + return this; + } + public SelectQueryBuilder intervals(QuerySegmentSpec q) { querySegmentSpec = q; @@ -1189,7 +1262,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 5e8b529b1535..1dcb0a49f45a 100644 --- a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java @@ -72,9 +72,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 +114,10 @@ public Sequence apply(Interval singleInterval) ), executor, queryWatcher ).run( - query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Arrays.asList(singleInterval))), + query.withQuerySegmentSpec( + spec.getDataSource(), + new MultipleIntervalSegmentSpec(Arrays.asList(singleInterval)) + ), responseContext ); } 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/Queries.java b/processing/src/main/java/io/druid/query/Queries.java index d62050965b42..bbc025d4ea59 100644 --- a/processing/src/main/java/io/druid/query/Queries.java +++ b/processing/src/main/java/io/druid/query/Queries.java @@ -23,9 +23,14 @@ 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 io.druid.query.spec.QuerySegmentSpec; +import org.joda.time.Duration; +import org.joda.time.Interval; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Set; @@ -80,4 +85,36 @@ 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<>(); + final List intervals = new ArrayList<>(); + + query.getDataSources().forEach(spec -> { + datasourceNames.addAll(spec.getDataSource().getNames()); + spec.getQuerySegmentSpec().getIntervals().stream().map(Interval::toString).forEach(intervals::add); + }); + + 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 9ad178161ead..f07204a6322a 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,7 +36,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; @@ -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(); + List getDataSources(); boolean hasFilters(); @@ -72,31 +75,74 @@ 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); + } - List getIntervals(); + Duration getDuration(DataSource dataSource); - Duration getDuration(); + default Duration getDuration(DataSourceWithSegmentSpec dataSourceWithSegmentSpec) + { + return getDuration(dataSourceWithSegmentSpec.getDataSource()); + } + + default Duration getTotalDuration() + { + Duration totalDuration = new Duration(0); + for (DataSourceWithSegmentSpec spec : getDataSources()) { + totalDuration = totalDuration.plus(Queries.getTotalDuration(spec.getQuerySegmentSpec())); + } + return totalDuration; + } Map getContext(); - ContextType getContextValue(String key); + default ContextType getContextValue(String key) + { + return (ContextType) getContext().get(key); + } - ContextType getContextValue(String key, ContextType defaultValue); + default ContextType getContextValue(String key, ContextType defaultValue) + { + ContextType retVal = getContextValue(key); + return retVal == null ? defaultValue : retVal; + } - boolean getContextBoolean(String key, boolean defaultValue); + default boolean getContextBoolean(String key, boolean defaultValue) + { + return QueryContexts.parseBoolean(this, key, defaultValue); + } boolean isDescending(); Ordering getResultOrdering(); - Query withOverriddenContext(Map contextOverride); + default String getId() + { + return (String) getContextValue(QueryContexts.QUERYID); + } + + default Query withId(String id) + { + return withOverriddenContext(ImmutableMap.of(QueryContexts.QUERYID, id)); + } - Query withQuerySegmentSpec(QuerySegmentSpec spec); + default DataSourceWithSegmentSpec getDistributionTarget() + { + return getContextValue(QueryContexts.DISTRIBUTION_TARGET_SOURCE); + } + + default Query distributeBy(DataSourceWithSegmentSpec spec) + { + return withOverriddenContext(ImmutableMap.of(QueryContexts.DISTRIBUTION_TARGET_SOURCE, spec)); + } + + Query withOverriddenContext(Map contextOverride); - Query withId(String id); + Query withQuerySegmentSpec(DataSource dataSource, QuerySegmentSpec spec); - String getId(); + Query withQuerySegmentSpec(String concatenatedDataSourceName, QuerySegmentSpec spec); - Query withDataSource(DataSource dataSource); + Query replaceDataSource(DataSource oldDataSource, DataSource newDataSource); } diff --git a/processing/src/main/java/io/druid/query/QueryContexts.java b/processing/src/main/java/io/druid/query/QueryContexts.java index 5a0409e430d4..bfa8f6ce605a 100644 --- a/processing/src/main/java/io/druid/query/QueryContexts.java +++ b/processing/src/main/java/io/druid/query/QueryContexts.java @@ -21,14 +21,19 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; import io.druid.java.util.common.ISE; +import java.util.Map; + public class QueryContexts { + public static final String QUERYID = "queryId"; public static final String PRIORITY_KEY = "priority"; public static final String TIMEOUT_KEY = "timeout"; public static final String DEFAULT_TIMEOUT_KEY = "defaultTimeout"; public static final String CHUNK_PERIOD_KEY = "chunkPeriod"; + public static final String DISTRIBUTION_TARGET_SOURCE = "distributionTargetSource"; public static final boolean DEFAULT_BY_SEGMENT = false; public static final boolean DEFAULT_POPULATE_CACHE = true; @@ -171,4 +176,18 @@ static boolean parseBoolean(Query query, String key, boolean defaultValue throw new ISE("Unknown type [%s]. Cannot parse!", val.getClass()); } } + + public static Map computeOverriddenContext( + final Map context, + final Map overrides + ) + { + Preconditions.checkNotNull(context, "context"); + Preconditions.checkNotNull(context, "overrides"); + Map overridden = Maps.newTreeMap(); + overridden.putAll(context); + overridden.putAll(overrides); + + return overridden; + } } diff --git a/processing/src/main/java/io/druid/query/QueryDataSource.java b/processing/src/main/java/io/druid/query/QueryDataSource.java index 0797f2f5f807..35702eb41ba1 100644 --- a/processing/src/main/java/io/druid/query/QueryDataSource.java +++ b/processing/src/main/java/io/druid/query/QueryDataSource.java @@ -24,12 +24,13 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import java.util.List; +import java.util.stream.Collectors; @JsonTypeName("query") public class QueryDataSource implements DataSource { @JsonProperty - private final Query query; + private final Query query; @JsonCreator public QueryDataSource(@JsonProperty("query") Query query) @@ -40,11 +41,13 @@ public QueryDataSource(@JsonProperty("query") Query query) @Override public List getNames() { - return query.getDataSource().getNames(); + return query.getDataSources().stream() + .flatMap(spec -> spec.getDataSource().getNames().stream()) + .collect(Collectors.toList()); } @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 3afc3a122cb8..c571f2058016 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -145,31 +145,20 @@ public interface QueryMetrics> */ void query(QueryType query); - /** - * Sets {@link Query#getDataSource()} of the given query as dimension. - */ - void dataSource(QueryType query); + void dataSourcesAndDurations(QueryType query); + + void intervals(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. - */ - void duration(QueryType query); - /** * Sets {@link Query#getId()} of the given query as dimension. */ 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..4c0773aa136f 100644 --- a/processing/src/main/java/io/druid/query/QueryToolChest.java +++ b/processing/src/main/java/io/druid/query/QueryToolChest.java @@ -179,4 +179,26 @@ public List filterSegments(QueryType query, List annotateDistributionTarget(QueryRunner runner) + { + return (query, responseContext) -> { + final BaseQuery groupByQuery = (BaseQuery) query; + return runner.run( + (Query) groupByQuery.updateDistributionTarget(), + responseContext + ); + }; + } } 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..2d04d306bab1 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.withQuerySegmentSpec( + 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/SubqueryQueryRunner.java b/processing/src/main/java/io/druid/query/SubqueryQueryRunner.java index e3dc7356c243..fc5c1b1c6752 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; @@ -26,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; @@ -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 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/TimewarpOperator.java b/processing/src/main/java/io/druid/query/TimewarpOperator.java index 0444d95d518c..0c956995fcb0 100644 --- a/processing/src/main/java/io/druid/query/TimewarpOperator.java +++ b/processing/src/main/java/io/druid/query/TimewarpOperator.java @@ -32,7 +32,7 @@ import org.joda.time.Interval; import org.joda.time.Period; -import java.util.Arrays; +import java.util.Collections; import java.util.Map; @@ -85,14 +85,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.withQuerySegmentSpec( + spec.getDataSource(), + new MultipleIntervalSegmentSpec(Collections.singletonList(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..9b94f51fdc14 100644 --- a/processing/src/main/java/io/druid/query/UnionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/UnionQueryRunner.java @@ -41,31 +41,37 @@ public UnionQueryRunner( @Override public Sequence run(final Query query, final Map responseContext) { - DataSource dataSource = query.getDataSource(); - if (dataSource instanceof UnionDataSource) { + if (query instanceof BaseQuery) { + final BaseQuery singleSourceBaseQuery = (BaseQuery) 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); + // Multi-source query types are not supported yet + throw new UnsupportedOperationException(); } } - } 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 e987fe2c4fb6..a68600ac8d2c 100644 --- a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceMetadataQuery.java +++ b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceMetadataQuery.java @@ -27,6 +27,7 @@ import io.druid.query.DataSource; import io.druid.query.Druids; import io.druid.query.Query; +import io.druid.query.QueryContexts; import io.druid.query.Result; import io.druid.query.filter.DimFilter; import io.druid.query.spec.MultipleIntervalSegmentSpec; @@ -84,7 +85,7 @@ public String getType() @Override public DataSourceMetadataQuery withOverriddenContext(Map contextOverrides) { - Map newContext = computeOverriddenContext(getContext(), contextOverrides); + Map newContext = QueryContexts.computeOverriddenContext(getContext(), contextOverrides); return Druids.DataSourceMetadataQueryBuilder.copy(this).context(newContext).build(); } 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..077ec0b42b0b 100644 --- a/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java @@ -24,12 +24,11 @@ import com.google.common.base.Function; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import io.druid.java.util.common.StringUtils; +import io.druid.query.cache.CacheKeyBuilder; import io.druid.query.extraction.ExtractionFn; import io.druid.segment.DimensionSelector; import io.druid.segment.column.ValueType; -import java.nio.ByteBuffer; import java.util.Arrays; import java.util.List; @@ -64,17 +63,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 +85,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 @@ -132,12 +159,10 @@ public boolean mustDecorate() @Override public byte[] getCacheKey() { - byte[] dimensionBytes = StringUtils.toUtf8(dimension); - - return ByteBuffer.allocate(1 + dimensionBytes.length) - .put(CACHE_TYPE_ID) - .put(dimensionBytes) - .array(); + return new CacheKeyBuilder(CACHE_TYPE_ID) + .appendString(dataSourceName) + .appendString(dimension) + .build(); } @Override @@ -150,6 +175,7 @@ public boolean preservesOrdering() public String toString() { return "DefaultDimensionSpec{" + + "dataSource='" + dataSourceName + '\'' + "dimension='" + dimension + '\'' + ", outputName='" + outputName + '\'' + ", outputType='" + outputType + '\'' + @@ -169,6 +195,9 @@ public boolean equals(Object o) DefaultDimensionSpec that = (DefaultDimensionSpec) o; + if (dataSourceName != null ? !dataSourceName.equals(that.dataSourceName) : that.dataSourceName != null) { + return false; + } if (dimension != null ? !dimension.equals(that.dimension) : that.dimension != null) { return false; } @@ -186,6 +215,7 @@ public boolean equals(Object o) public int hashCode() { int result = dimension != null ? dimension.hashCode() : 0; + result = 31 * result + (dataSourceName != null ? dataSourceName.hashCode() : 0); result = 31 * result + (outputName != null ? outputName.hashCode() : 0); result = 31 * result + (outputType != null ? outputType.hashCode() : 0); return result; 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..27fc8ad2c51f 100644 --- a/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java @@ -22,19 +22,18 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import io.druid.java.util.common.StringUtils; +import io.druid.query.cache.CacheKeyBuilder; import io.druid.query.extraction.ExtractionFn; import io.druid.segment.DimensionSelector; import io.druid.segment.column.ValueType; -import java.nio.ByteBuffer; - /** */ 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 +41,7 @@ public class ExtractionDimensionSpec implements DimensionSpec @JsonCreator public ExtractionDimensionSpec( + @JsonProperty("dataSource") String dataSourceName, @JsonProperty("dimension") String dimension, @JsonProperty("outputName") String outputName, @JsonProperty("outputType") ValueType outputType, @@ -53,6 +53,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 +62,26 @@ 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 + @JsonProperty("dataSource") + public String getDataSourceName() + { + return dataSourceName; } @Override @@ -114,14 +127,11 @@ public boolean mustDecorate() @Override public byte[] getCacheKey() { - byte[] dimensionBytes = StringUtils.toUtf8(dimension); - byte[] dimExtractionFnBytes = extractionFn.getCacheKey(); - - return ByteBuffer.allocate(1 + dimensionBytes.length + dimExtractionFnBytes.length) - .put(CACHE_TYPE_ID) - .put(dimensionBytes) - .put(dimExtractionFnBytes) - .array(); + return new CacheKeyBuilder(CACHE_TYPE_ID) + .appendString(dataSourceName) + .appendString(dimension) + .appendCacheable(extractionFn) + .build(); } @Override @@ -134,6 +144,7 @@ public boolean preservesOrdering() public String toString() { return "ExtractionDimensionSpec{" + + "dataSource='" + dataSourceName + '\'' + "dimension='" + dimension + '\'' + ", extractionFn=" + extractionFn + ", outputName='" + outputName + '\'' + @@ -153,6 +164,9 @@ public boolean equals(Object o) ExtractionDimensionSpec that = (ExtractionDimensionSpec) o; + if (dataSourceName != null ? !dataSourceName.equals(that.dataSourceName) : that.dataSourceName != null) { + return false; + } if (extractionFn != null ? !extractionFn.equals(that.extractionFn) : that.extractionFn != null) { return false; } @@ -173,6 +187,7 @@ public boolean equals(Object o) public int hashCode() { int result = dimension != null ? dimension.hashCode() : 0; + result = 31 * result + (dataSourceName != null ? dataSourceName.hashCode() : 0); result = 31 * result + (extractionFn != null ? extractionFn.hashCode() : 0); result = 31 * result + (outputName != null ? outputName.hashCode() : 0); result = 31 * result + (outputType != null ? outputType.hashCode() : 0); 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 97448a55cb10..50857b3bceb5 100644 --- a/processing/src/main/java/io/druid/query/dimension/LookupDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/LookupDimensionSpec.java @@ -24,9 +24,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.base.Strings; -import io.druid.java.util.common.StringUtils; +import io.druid.query.cache.CacheKeyBuilder; import io.druid.query.extraction.ExtractionFn; -import io.druid.query.filter.DimFilterUtils; import io.druid.query.lookup.LookupExtractionFn; import io.druid.query.lookup.LookupExtractor; import io.druid.query.lookup.LookupReferencesManager; @@ -34,12 +33,14 @@ import io.druid.segment.column.ValueType; import javax.annotation.Nullable; -import java.nio.ByteBuffer; 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 +66,7 @@ public class LookupDimensionSpec implements DimensionSpec @JsonCreator public LookupDimensionSpec( + @JsonProperty("dataSource") String dataSourceName, @JsonProperty("dimension") String dimension, @JsonProperty("outputName") String outputName, @JsonProperty("lookup") LookupExtractor lookup, @@ -75,6 +77,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 +99,37 @@ 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 + @JsonProperty("dataSource") + public String getDataSourceName() + { + return dataSourceName; + } + @Override @JsonProperty public String getDimension() @@ -166,30 +200,18 @@ public boolean mustDecorate() @Override public byte[] getCacheKey() { - byte[] dimensionBytes = StringUtils.toUtf8(dimension); - byte[] dimExtractionFnBytes = Strings.isNullOrEmpty(name) - ? getLookup().getCacheKey() - : StringUtils.toUtf8(name); - byte[] outputNameBytes = StringUtils.toUtf8(outputName); - byte[] replaceWithBytes = StringUtils.toUtf8(Strings.nullToEmpty(replaceMissingValueWith)); - - - return ByteBuffer.allocate(6 - + dimensionBytes.length - + outputNameBytes.length - + dimExtractionFnBytes.length - + replaceWithBytes.length) - .put(CACHE_TYPE_ID) - .put(dimensionBytes) - .put(DimFilterUtils.STRING_SEPARATOR) - .put(outputNameBytes) - .put(DimFilterUtils.STRING_SEPARATOR) - .put(dimExtractionFnBytes) - .put(DimFilterUtils.STRING_SEPARATOR) - .put(replaceWithBytes) - .put(DimFilterUtils.STRING_SEPARATOR) - .put(retainMissingValue ? (byte) 1 : (byte) 0) - .array(); + final CacheKeyBuilder builder = new CacheKeyBuilder(CACHE_TYPE_ID) + .appendString(dataSourceName) + .appendString(dimension); + if (Strings.isNullOrEmpty(name)) { + builder.appendCacheable(getLookup()); + } else { + builder.appendString(name); + } + return builder.appendString(outputName) + .appendString(replaceMissingValueWith) + .appendBoolean(retainMissingValue) + .build(); } @Override @@ -216,6 +238,9 @@ public boolean equals(Object o) if (optimize != that.optimize) { return false; } + if (dataSourceName != null ? !dataSourceName.equals(that.dataSourceName) : that.dataSourceName != null) { + return false; + } if (!getDimension().equals(that.getDimension())) { return false; } @@ -238,6 +263,7 @@ public boolean equals(Object o) public int hashCode() { int result = getDimension().hashCode(); + result = 31 * result + (dataSourceName != null ? dataSourceName.hashCode() : 0); result = 31 * result + getOutputName().hashCode(); result = 31 * result + (getLookup() != null ? getLookup().hashCode() : 0); result = 31 * result + (retainMissingValue ? 1 : 0); diff --git a/processing/src/main/java/io/druid/query/extraction/ExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/ExtractionFn.java index 4518a4cf8ef4..7105ba135383 100644 --- a/processing/src/main/java/io/druid/query/extraction/ExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/ExtractionFn.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import io.druid.java.util.common.Cacheable; import io.druid.query.lookup.LookupExtractionFn; import io.druid.query.lookup.RegisteredLookupExtractionFn; @@ -53,15 +54,8 @@ * regular expression with a capture group. When the regular expression matches the value of a dimension, * the value captured by the group is used for grouping operations instead of the dimension value. */ -public interface ExtractionFn +public interface ExtractionFn extends Cacheable { - /** - * Returns a byte[] unique to all concrete implementations of DimExtractionFn. This byte[] is used to - * generate a cache key for the specific query. - * - * @return a byte[] unit to all concrete implements of DimExtractionFn - */ - public byte[] getCacheKey(); /** * The "extraction" function. This should map an Object into some String value. 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 553e8a2a95c0..c7b2d6f3f49e 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java @@ -39,8 +39,10 @@ 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.QueryContexts; import io.druid.query.QueryDataSource; import io.druid.query.TableDataSource; import io.druid.query.aggregation.AggregatorFactory; @@ -64,10 +66,12 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.TreeMap; /** */ @@ -469,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) @@ -508,6 +512,18 @@ public Builder setDataSource(Query query) return this; } + public Builder updateDistributionTarget() + { + if (context == null) { + context = new HashMap<>(); + } + context.put( + QueryContexts.DISTRIBUTION_TARGET_SOURCE, + new DataSourceWithSegmentSpec(getLeafDataSource(dataSource), querySegmentSpec) + ); + return this; + } + public Builder setInterval(QuerySegmentSpec interval) { return setQuerySegmentSpec(interval); @@ -679,13 +695,17 @@ 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; } public Builder overrideContext(Map contextOverride) { - this.context = computeOverriddenContext(context, contextOverride); + this.context = QueryContexts.computeOverriddenContext(context, contextOverride); 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 ddf9c4277c3f..d7284ebfc4e6 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/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index 211d287e7061..f8443a667c6f 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 @@ -198,7 +198,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 e13b4e7c6fb5..c3dd0921b74d 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 @@ -42,6 +42,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; @@ -158,7 +159,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 a query + * @param foundNum number of merge buffers found so far + * + * @return required number of merge buffers + */ + 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. @@ -167,12 +176,34 @@ 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; - } else { - return countRequiredMergeBufferNum(((QueryDataSource) dataSource).getQuery(), 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; // keep the maximum number of consecutive groupBys found so far + 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 foundFromChild = countGroupByLayers(queryDataSource.getQuery(), foundNum + 1); + maxFoundFromChildren = Math.max(foundFromChild, maxFoundFromChildren); + } else { + // Reset foundNum because a non-groupBy is found + final int foundFromChild = countGroupByLayers(queryDataSource.getQuery(), 0); + maxFoundFromChildren = Math.max(foundFromChild, maxFoundFromChildren); + } + } + } + + return Math.max(maxFoundFromChildren, foundNum); } @Override 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..993803bc5b1d --- /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 JoinPredicate accept(JoinPredicateVisitor visitor) + { + return 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..3d083748fa04 --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/AndPredicate.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.join; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; + +import java.util.List; +import java.util.Objects; + +public class AndPredicate implements JoinPredicate +{ + private final List predicates; + + @JsonCreator + public AndPredicate( + @JsonProperty("predicates") List predicates + ) + { + Preconditions.checkArgument(predicates != null && !predicates.isEmpty(), "predicates"); + this.predicates = predicates; + } + + @JsonProperty + public List getPredicates() + { + return predicates; + } + + @Override + public JoinPredicate accept(JoinPredicateVisitor visitor) + { + return 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..b1322ca1ca7d --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/BinaryPredicate.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.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; + if (!getType().equals(that.getType())) { + return false; + } + + if (!left.equals(that.left)) { + return false; + } + + return right.equals(that.right); + } + + @Override + public int hashCode() + { + return Objects.hash(getType(), left, right); + } +} diff --git a/processing/src/main/java/io/druid/query/join/DataSourceJoinInputSpec.java b/processing/src/main/java/io/druid/query/join/DataSourceJoinInputSpec.java new file mode 100644 index 000000000000..d71ed5a23c3d --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/DataSourceJoinInputSpec.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.DataSource; +import io.druid.query.Queries; +import io.druid.query.spec.QuerySegmentSpec; +import org.joda.time.Duration; + +import java.util.Objects; + +public class DataSourceJoinInputSpec implements JoinInputSpec +{ + private final DataSource dataSource; + private final QuerySegmentSpec querySegmentSpec; + private volatile Duration duration; + + @JsonCreator + public DataSourceJoinInputSpec( + @JsonProperty("dataSource") DataSource dataSource, + @JsonProperty("intervals") QuerySegmentSpec querySegmentSpec + ) + { + this.dataSource = Objects.requireNonNull(dataSource); + this.querySegmentSpec = Objects.requireNonNull(querySegmentSpec); + } + + public DataSourceJoinInputSpec( + 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 = Queries.getTotalDuration(querySegmentSpec); + } + return duration; + } + + @Override + public boolean equals(Object o) + { + if (o == this) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + DataSourceJoinInputSpec that = (DataSourceJoinInputSpec) o; + if (!dataSource.equals(that.dataSource)) { + return false; + } + + return querySegmentSpec.equals(that.querySegmentSpec); + } + + @Override + public int hashCode() + { + return Objects.hash(dataSource, querySegmentSpec); + } + + @Override + public DataSourceJoinInputSpec accept(JoinSpecVisitor visitor) + { + return visitor.visit(this); + } +} 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..972b7f95b6da --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/DefaultJoinQueryMetrics.java @@ -0,0 +1,99 @@ +/* + * 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.base.Preconditions; +import io.druid.query.DataSourceUtil; +import io.druid.query.DataSourceWithSegmentSpec; +import io.druid.query.DefaultQueryMetrics; +import io.druid.query.DruidMetrics; + +public class DefaultJoinQueryMetrics extends DefaultQueryMetrics 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 dataSourcesAndDurations(JoinQuery query) + { + builder.setDimension( + "dataSourcesAndDurations", + query.getDataSources().stream() + .map( + spec -> "{dataSource=" + DataSourceUtil.getMetricName(spec.getDataSource()) + + ",duration=" + query.getDuration(spec.getDataSource()) + "}" + ) + .toArray(String[]::new) + ); + } + + @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) + { + builder.setDimension("numDataSources", String.valueOf(query.getDataSources().size())); + } + + @Override + public void distributionTarget(JoinQuery query) + { + final DataSourceWithSegmentSpec distributionTarget = query.getDistributionTarget(); + Preconditions.checkNotNull(distributionTarget, "distributionTarget"); + builder.setDimension( + "distributionTarget", + DataSourceUtil.getMetricName(distributionTarget.getDataSource()) + ); + } + + @Override + public void distributionTargetDuration(JoinQuery query) + { + final DataSourceWithSegmentSpec distributionTarget = query.getDistributionTarget(); + Preconditions.checkNotNull(distributionTarget, "distributionTarget"); + builder.setDimension( + "distributionTargetDuration", + query.getDuration(distributionTarget.getDataSource()).toString() + ); + } +} diff --git a/processing/src/main/java/io/druid/query/join/DimensionPredicate.java b/processing/src/main/java/io/druid/query/join/DimensionPredicate.java new file mode 100644 index 000000000000..3d94673725bc --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/DimensionPredicate.java @@ -0,0 +1,79 @@ +/* + * 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 io.druid.query.dimension.DimensionSpec; + +import java.util.Objects; + +public class DimensionPredicate implements JoinPredicate +{ + private final DimensionSpec dimension; + + @JsonCreator + public DimensionPredicate( + @JsonProperty("dimension") DimensionSpec dimension + ) + { + this.dimension = Preconditions.checkNotNull(dimension); + } + + @JsonProperty + public DimensionSpec getDimension() + { + return dimension; + } + + @Override + public JoinPredicate accept(JoinPredicateVisitor visitor) + { + return 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; + } + + DimensionPredicate that = (DimensionPredicate) 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..6fbca095451a --- /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 JoinPredicate accept(JoinPredicateVisitor visitor) + { + return 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..b0b56db28b77 --- /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 JoinPredicate accept(JoinPredicateVisitor visitor) + { + return 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..6a85d02cb277 --- /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 = DataSourceJoinInputSpec.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..13898bc0182f --- /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 = DimensionPredicate.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(); + + JoinPredicate 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..c09dfbec05f4 --- /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(DimensionPredicate 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..52a0712bbd0f --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/JoinQuery.java @@ -0,0 +1,410 @@ +/* + * 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.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.QueryContexts; +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 MultiSourceBaseQuery +{ + 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 builder() + { + 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 List getDataSources() + { + final List found = new ArrayList<>(); + final JoinSpecVisitor visitor = new JoinSpecVisitor() + { + @Override + public DataSourceJoinInputSpec visit(DataSourceJoinInputSpec 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 Duration getDuration(DataSource dataSource) + { + for (DataSourceWithSegmentSpec sourceWithSegmentSpec : getDataSources()) { + if (sourceWithSegmentSpec.getDataSource().equals(dataSource)) { + return Queries.getTotalDuration(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, + QueryContexts.computeOverriddenContext(getContext(), contextOverride) + ); + } + + @Override + public Query withQuerySegmentSpec(DataSource dataSource, QuerySegmentSpec spec) + { + return withQuerySegmentSpec(dataSource.getFirstName(), spec); + } + + @Override + public Query withQuerySegmentSpec(String concatenatedDataSourceName, 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 DataSourceJoinInputSpec visit(DataSourceJoinInputSpec dataInput) + { + if (dataInput.getDataSource().getConcatenatedName().equals(concatenatedDataSourceName)) { + return new DataSourceJoinInputSpec(dataInput.getDataSource(), spec); + } else { + return dataInput; + } + } + }; + + return new JoinQuery( + joinSpec.accept(visitor), + granularity, + dimensions, + metrics, + virtualColumns, + filter, + getContext() + ); + } + + @Override + public Query replaceDataSource(DataSource oldDataSource, DataSource newDataSource) + { + 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 DataSourceJoinInputSpec visit(DataSourceJoinInputSpec dataInput) + { + if (dataInput.getDataSource().equals(oldDataSource)) { + return new DataSourceJoinInputSpec(newDataSource, 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/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/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..1f2834522ca2 --- /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 DataSourceJoinInputSpec visit(DataSourceJoinInputSpec dataSourceJoinInputSpec) + { + return dataSourceJoinInputSpec; + } +} 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..3dde1dee6901 --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/LiteralPredicate.java @@ -0,0 +1,79 @@ +/* + * 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 JoinPredicate accept(JoinPredicateVisitor visitor) + { + return 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; + if (literal == null ? that.literal != null : !literal.equals(that.literal)) { + return false; + } + + return true; + } + + @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..d87c29ba0dd7 --- /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 JoinPredicate accept(JoinPredicateVisitor visitor) + { + return 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..36bee747fd85 --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/NotPredicate.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 com.google.common.base.Preconditions; + +import java.util.Objects; + +public class NotPredicate implements JoinPredicate +{ + private final JoinPredicate predicate; + + @JsonCreator + public NotPredicate( + @JsonProperty("base") JoinPredicate predicate + ) + { + this.predicate = Preconditions.checkNotNull(predicate); + } + + @JsonProperty("base") + public JoinPredicate getBase() + { + return predicate; + } + + @Override + public JoinPredicate accept(JoinPredicateVisitor visitor) + { + return 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..4eec01c7f6db --- /dev/null +++ b/processing/src/main/java/io/druid/query/join/OrPredicate.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.join; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; + +import java.util.List; +import java.util.Objects; + +public class OrPredicate implements JoinPredicate +{ + private final List predicates; + + @JsonCreator + public OrPredicate( + @JsonProperty("predicates") List predicates + ) + { + Preconditions.checkArgument(predicates != null && !predicates.isEmpty(), "predicates"); + this.predicates = predicates; + } + + @JsonProperty + public List getPredicates() + { + return predicates; + } + + @Override + public JoinPredicate accept(JoinPredicateVisitor visitor) + { + return 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..b2d024273feb --- /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 JoinPredicate accept(JoinPredicateVisitor visitor) + { + return visitor.visit(this); + } +} diff --git a/processing/src/main/java/io/druid/query/lookup/LookupExtractor.java b/processing/src/main/java/io/druid/query/lookup/LookupExtractor.java index 0400c21ae1e6..41a8c3a7d423 100644 --- a/processing/src/main/java/io/druid/query/lookup/LookupExtractor.java +++ b/processing/src/main/java/io/druid/query/lookup/LookupExtractor.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import io.druid.java.util.common.Cacheable; import io.druid.query.extraction.MapLookupExtractor; import javax.annotation.Nullable; @@ -35,7 +36,7 @@ @JsonSubTypes(value = { @JsonSubTypes.Type(name = "map", value = MapLookupExtractor.class) }) -public abstract class LookupExtractor +public abstract class LookupExtractor implements Cacheable { /** * Apply a particular lookup methodology to the input string @@ -101,14 +102,6 @@ public Map> unapplyAll(Iterable values) return map; } - /** - * Create a cache key for use in results caching - * - * @return A byte array that can be used to uniquely identify if results of a prior lookup can use the cached values - */ - - public abstract byte[] getCacheKey(); - // make this abstract again once @drcrallen fix the metmax lookup implementation. public boolean isOneToOne() { 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 130025c60770..6550d3df592d 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 @@ -29,6 +29,7 @@ import io.druid.query.DataSource; import io.druid.query.Druids; import io.druid.query.Query; +import io.druid.query.QueryContexts; import io.druid.query.TableDataSource; import io.druid.query.UnionDataSource; import io.druid.query.filter.DimFilter; @@ -233,7 +234,7 @@ public byte[] getAnalysisTypesCacheKey() @Override public Query withOverriddenContext(Map contextOverride) { - Map newContext = computeOverriddenContext(getContext(), contextOverride); + Map newContext = QueryContexts.computeOverriddenContext(getContext(), contextOverride); return Druids.SegmentMetadataQueryBuilder.copy(this).context(newContext).build(); } 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 a9971907dd5d..d1fb0b2c4837 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 @@ -28,6 +28,7 @@ import io.druid.query.DataSource; import io.druid.query.Druids; import io.druid.query.Query; +import io.druid.query.QueryContexts; import io.druid.query.Result; import io.druid.query.dimension.DimensionSpec; import io.druid.query.filter.DimFilter; @@ -108,7 +109,7 @@ public Query> withDataSource(DataSource dataSource) @Override public SearchQuery withOverriddenContext(Map contextOverrides) { - Map newContext = computeOverriddenContext(getContext(), contextOverrides); + Map newContext = QueryContexts.computeOverriddenContext(getContext(), contextOverrides); return Druids.SearchQueryBuilder.copy(this).context(newContext).build(); } 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 72059580c4f2..ce676edc134c 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQuery.java +++ b/processing/src/main/java/io/druid/query/select/SelectQuery.java @@ -28,6 +28,7 @@ import io.druid.query.DataSource; import io.druid.query.Druids; import io.druid.query.Query; +import io.druid.query.QueryContexts; import io.druid.query.Result; import io.druid.query.dimension.DimensionSpec; import io.druid.query.filter.DimFilter; @@ -158,7 +159,7 @@ public Query> withDataSource(DataSource dataSource) public SelectQuery withOverriddenContext(Map contextOverrides) { - Map newContext = computeOverriddenContext(getContext(), contextOverrides); + Map newContext = QueryContexts.computeOverriddenContext(getContext(), contextOverrides); return Druids.SelectQueryBuilder.copy(this).context(newContext).build(); } 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..34ffe7ddd404 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,50 @@ import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; /** */ 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); + // 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(); - final String newName = String.format("%s_%s_%s", query.getType(), query.getDataSource(), query.getIntervals()); + // If any other data sources contain dataSourceName, then error + 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( + "%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 d466cd211406..ea8b6aba21fe 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java @@ -29,6 +29,7 @@ import io.druid.query.DataSource; import io.druid.query.Druids; import io.druid.query.Query; +import io.druid.query.QueryContexts; import io.druid.query.Result; import io.druid.query.filter.DimFilter; import io.druid.query.spec.MultipleIntervalSegmentSpec; @@ -105,7 +106,7 @@ public String getBound() @Override public TimeBoundaryQuery withOverriddenContext(Map contextOverrides) { - Map newContext = computeOverriddenContext(getContext(), contextOverrides); + Map newContext = QueryContexts.computeOverriddenContext(getContext(), contextOverrides); return Druids.TimeBoundaryQueryBuilder.copy(this).context(newContext).build(); } 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..b2d016cb96a0 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; 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 328400291348..829a320fe6d9 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java @@ -29,6 +29,7 @@ import io.druid.query.Druids; import io.druid.query.Queries; import io.druid.query.Query; +import io.druid.query.QueryContexts; import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; @@ -142,7 +143,7 @@ public Query> withDataSource(DataSource dataSource public TimeseriesQuery withOverriddenContext(Map contextOverrides) { - Map newContext = computeOverriddenContext(getContext(), contextOverrides); + Map newContext = QueryContexts.computeOverriddenContext(getContext(), contextOverrides); return Druids.TimeseriesQueryBuilder.copy(this).context(newContext).build(); } 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 df7583ebd760..c36e63f884a2 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQuery.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQuery.java @@ -28,6 +28,7 @@ import io.druid.query.DataSource; import io.druid.query.Queries; import io.druid.query.Query; +import io.druid.query.QueryContexts; import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; @@ -201,7 +202,8 @@ public TopNQuery withThreshold(int threshold) public TopNQuery withOverriddenContext(Map contextOverrides) { - return new TopNQueryBuilder(this).context(computeOverriddenContext(getContext(), contextOverrides)).build(); + return new TopNQueryBuilder(this).context(QueryContexts.computeOverriddenContext(getContext(), contextOverrides)) + .build(); } public TopNQuery withDimFilter(DimFilter dimFilter) 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 4a06a486f390..a012cedea924 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java @@ -22,8 +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.QueryMetrics; +import io.druid.query.DataSourceWithSegmentSpec; +import io.druid.query.QueryContexts; import io.druid.query.TableDataSource; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; @@ -39,8 +41,10 @@ import org.joda.time.Interval; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.TreeMap; /** * A Builder for TopNQuery. @@ -76,7 +80,6 @@ public class TopNQueryBuilder private List aggregatorSpecs; private List postAggregatorSpecs; private Map context; - private QueryMetrics queryMetrics; public TopNQueryBuilder() { @@ -105,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() @@ -209,6 +212,21 @@ public TopNQueryBuilder dataSource(String d) return this; } + public TopNQueryBuilder updateDistributionTarget() + { + if (context == null) { + context = new HashMap<>(); + } + context.put( + QueryContexts.DISTRIBUTION_TARGET_SOURCE, + new DataSourceWithSegmentSpec( + BaseQuery.getLeafDataSource(dataSource), + querySegmentSpec + ) + ); + return this; + } + public TopNQueryBuilder virtualColumns(VirtualColumns virtualColumns) { this.virtualColumns = virtualColumns; @@ -327,13 +345,11 @@ public TopNQueryBuilder postAggregators(List p) public TopNQueryBuilder context(Map c) { - context = c; - return this; - } - - public TopNQueryBuilder queryMetrics(QueryMetrics m) - { - queryMetrics = m; + if (context == null) { + context = new HashMap<>(c); + } else { + context.putAll(c); + } return this; } } 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..0fd0bb491f7b --- /dev/null +++ b/processing/src/test/java/io/druid/query/DataSourceUtilTest.java @@ -0,0 +1,45 @@ +/* + * 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 org.junit.Assert; +import org.junit.Test; + +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)); + } +} diff --git a/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java b/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java index 63740e9e730c..62f7cbb487cf 100644 --- a/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java @@ -50,7 +50,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) @@ -81,7 +81,7 @@ public void testDefaultQueryMetricsQuery() 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")); @@ -92,7 +92,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/processing/src/test/java/io/druid/query/IntervalChunkingQueryRunnerTest.java b/processing/src/test/java/io/druid/query/IntervalChunkingQueryRunnerTest.java index 194b12e6bdc5..f72502bb8848 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 = ((TimeseriesQuery)query).updateDistributionTarget(); 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 = ((TimeseriesQuery)query).updateDistributionTarget(); executors.execute(EasyMock.anyObject(Runnable.class)); EasyMock.expectLastCall().times(2); diff --git a/processing/src/test/java/io/druid/query/QueryContextsTest.java b/processing/src/test/java/io/druid/query/QueryContextsTest.java index 2a6945b9d3cf..f90789bcad2f 100644 --- a/processing/src/test/java/io/druid/query/QueryContextsTest.java +++ b/processing/src/test/java/io/druid/query/QueryContextsTest.java @@ -78,7 +78,7 @@ public Query withOverriddenContext(Map contextOverride) getDataSource(), getQuerySegmentSpec(), isDescending(), - BaseQuery.computeOverriddenContext(getContext(), contextOverride) + QueryContexts.computeOverriddenContext(getContext(), contextOverride) ); } } diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index c1206adae34d..ed70e7413cae 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -75,6 +75,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; /** */ @@ -332,7 +333,7 @@ public void remove() }; } - public static > List> makeQueryRunners( + public static > List> makeQueryRunners( QueryRunnerFactory factory ) throws IOException @@ -434,7 +435,7 @@ public void remove() }; } - public static > QueryRunner makeQueryRunner( + public static > QueryRunner makeQueryRunner( QueryRunnerFactory factory, String resourceFileName, final String runnerName @@ -448,7 +449,7 @@ public static > QueryRunner makeQueryRunner( ); } - public static > QueryRunner makeQueryRunner( + public static > QueryRunner makeQueryRunner( QueryRunnerFactory factory, Segment adapter, final String runnerName @@ -457,7 +458,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 +529,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 = 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.withQuerySegmentSpec( - new SpecificSegmentSpec( - new SegmentDescriptor( - holder.getInterval(), - holder.getVersion(), - 0 - ) - ) - ); + Query running = query; + for (DataSourceWithSegmentSpec spec : query.getDataSources()) { + running = query.withQuerySegmentSpec( + 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..65a40c49fe90 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)((BaseQuery)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)((BaseQuery)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..89895cca72d9 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(), + ((BaseQuery>)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(), + ((BaseQuery>)query).getIntervals().get(0).getStart(), new TimeseriesResultValue(ImmutableMap.of("metric", 2)) ), new Result<>( - query.getIntervals().get(0).getEnd(), + ((BaseQuery>)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,18 @@ public Sequence> run( Sequences.toList(queryRunner.run(query, Maps.newHashMap()), Lists.>newArrayList()) ); } + + private static Query setDistributionTarget(Query query) + { + final DataSourceWithSegmentSpec sourceWithSegmentSpec = Iterables.getOnlyElement(query.getDataSources()); + return query.withOverriddenContext( + ImmutableMap.of( + QueryContexts.DISTRIBUTION_TARGET_SOURCE, + new DataSourceWithSegmentSpec( + 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 b7b914fe5ab4..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(query.getDataSource() instanceof TableDataSource); - String dsName = Iterables.getOnlyElement(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)); @@ -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/dimension/DefaultDimensionSpecTest.java b/processing/src/test/java/io/druid/query/dimension/DefaultDimensionSpecTest.java index 7d612be2d720..76bafc3e79ea 100644 --- a/processing/src/test/java/io/druid/query/dimension/DefaultDimensionSpecTest.java +++ b/processing/src/test/java/io/druid/query/dimension/DefaultDimensionSpecTest.java @@ -44,6 +44,19 @@ public void testEqualsSerde() throws IOException Assert.assertEquals(spec.hashCode(), other.hashCode()); } + @Test + public void testEqualsSerdeWithDataSource() throws IOException + { + final String name = "foo"; + final DimensionSpec spec = new DefaultDimensionSpec("source1", name, name); + final String json = mapper.writeValueAsString(spec); + final DimensionSpec other = mapper.readValue(json, DimensionSpec.class); + Assert.assertEquals("source1", other.getDataSourceName()); + Assert.assertEquals(spec.toString(), other.toString()); + Assert.assertEquals(spec, other); + Assert.assertEquals(spec.hashCode(), other.hashCode()); + } + @Test public void testEqualsSerdeWithType() throws IOException { diff --git a/processing/src/test/java/io/druid/query/dimension/ExtractionDimensionSpecTest.java b/processing/src/test/java/io/druid/query/dimension/ExtractionDimensionSpecTest.java index 9e703df4c754..5aa94b6617c9 100644 --- a/processing/src/test/java/io/druid/query/dimension/ExtractionDimensionSpecTest.java +++ b/processing/src/test/java/io/druid/query/dimension/ExtractionDimensionSpecTest.java @@ -27,6 +27,8 @@ import org.junit.Assert; import org.junit.Test; +import java.io.IOException; + public class ExtractionDimensionSpecTest { @Test @@ -61,6 +63,46 @@ public void testSerde() throws Exception ); } + @Test + public void testSerdeWithDataSourceName() throws IOException + { + final ObjectMapper objectMapper = new DefaultObjectMapper(); + final String json = "{\n" + + " \"type\" : \"extraction\",\n" + + " \"dataSource\" : \"source\",\n" + + " \"dimension\" : \"myDim\",\n" + + " \"outputName\" : \"first3Letters\",\n" + + " \"outputType\" : \"STRING\",\n" + + " \"extractionFn\" : {\n" + + " \"type\" : \"regex\",\n" + + " \"expr\" : \"(...).*\",\n" + + " \"index\" : 1,\n" + + " \"replaceMissingValue\" : false,\n" + + " \"replaceMissingValueWith\" : null\n" + + " }\n" + + "}"; + + final ExtractionDimensionSpec extractionDimensionSpec = (ExtractionDimensionSpec) objectMapper.readValue( + json, + DimensionSpec.class + ); + + Assert.assertEquals("source", extractionDimensionSpec.getDataSourceName()); + Assert.assertEquals("first3Letters", extractionDimensionSpec.getOutputName()); + Assert.assertEquals("myDim", extractionDimensionSpec.getDimension()); + Assert.assertNotNull(extractionDimensionSpec.getExtractionFn()); + Assert.assertEquals(ValueType.STRING, extractionDimensionSpec.getOutputType()); + Assert.assertTrue(extractionDimensionSpec.getExtractionFn() instanceof RegexDimExtractionFn); + + Assert.assertEquals( + extractionDimensionSpec, + objectMapper.readValue( + objectMapper.writeValueAsBytes(extractionDimensionSpec), + DimensionSpec.class + ) + ); + } + @Test public void testSerdeWithType() throws Exception { 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 147c8e1bf64f..60547cab811b 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -2357,10 +2357,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( @@ -2653,10 +2654,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( @@ -3441,10 +3443,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( @@ -3774,10 +3777,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( @@ -3883,10 +3887,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/DefaultJoinQueryMetricsTest.java b/processing/src/test/java/io/druid/query/join/DefaultJoinQueryMetricsTest.java new file mode 100644 index 000000000000..a9521620912b --- /dev/null +++ b/processing/src/test/java/io/druid/query/join/DefaultJoinQueryMetricsTest.java @@ -0,0 +1,219 @@ +/* + * 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 DimensionPredicate(new DefaultDimensionSpec("src1", "dim1", "dim1")), + new DimensionPredicate(new DefaultDimensionSpec("src2", "dim1", "dim1")) + ), + new EqualPredicate( + new AddPredicate( + new DimensionPredicate(new DefaultDimensionSpec("src2", "dim2", "dim2")), + new LiteralPredicate("10") + ), + new AddPredicate( + new DimensionPredicate(new DefaultDimensionSpec("src1", "dim2", "dim2")), + new DimensionPredicate(new DefaultDimensionSpec("src1", "dim3", "dim3")) + ) + ) + ) + ), + new DataSourceJoinInputSpec(new TableDataSource("src1"), QueryRunnerTestHelper.firstToThird), + new DataSourceJoinInputSpec(new TableDataSource("src2"), QueryRunnerTestHelper.firstToThird) + ); + + final JoinSpec joinSpec = new JoinSpec( + JoinType.INNER, + new EqualPredicate( + new DimensionPredicate(new DefaultDimensionSpec("j1", "dim4", "dim4")), + new DimensionPredicate(new DefaultDimensionSpec("src3", "dim4", "dim4")) + ), + leftChildSpec, + new DataSourceJoinInputSpec(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); + } + + @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 DimensionPredicate(new DefaultDimensionSpec("src1", "dim1", "dim1")), + new DimensionPredicate(new DefaultDimensionSpec("src2", "dim1", "dim1")) + ), + new EqualPredicate( + new AddPredicate( + new DimensionPredicate(new DefaultDimensionSpec("src2", "dim2", "dim2")), + new LiteralPredicate("10") + ), + new AddPredicate( + new DimensionPredicate(new DefaultDimensionSpec("src1", "dim2", "dim2")), + new DimensionPredicate(new DefaultDimensionSpec("src1", "dim3", "dim3")) + ) + ) + ) + ), + new DataSourceJoinInputSpec(new TableDataSource("src1"), QueryRunnerTestHelper.firstToThird), + new DataSourceJoinInputSpec(new TableDataSource("src2"), QueryRunnerTestHelper.firstToThird) + ); + + final JoinSpec joinSpec = new JoinSpec( + JoinType.INNER, + new EqualPredicate( + new DimensionPredicate(new DefaultDimensionSpec("j1", "dim4", "dim4")), + new DimensionPredicate(new DefaultDimensionSpec("src3", "dim4", "dim4")) + ), + leftChildSpec, + new DataSourceJoinInputSpec(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) + ); + } +} 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..7f31482fd37a --- /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 DimensionPredicate(new DefaultDimensionSpec("src1", "dim1", "dim1")), + new DimensionPredicate(new DefaultDimensionSpec("src2", "dim1", "dim1")) + ), + new EqualPredicate( + new AddPredicate( + new DimensionPredicate(new DefaultDimensionSpec("src2", "dim2", "dim2")), + new LiteralPredicate("10") + ), + new AddPredicate( + new DimensionPredicate(new DefaultDimensionSpec("src1", "dim2", "dim2")), + new DimensionPredicate(new DefaultDimensionSpec("src1", "dim3", "dim3")) + ) + ) + ) + ), + new DataSourceJoinInputSpec(new TableDataSource("src1"), QueryRunnerTestHelper.firstToThird), + new DataSourceJoinInputSpec(new TableDataSource("src2"), QueryRunnerTestHelper.firstToThird) + ); + + final JoinSpec joinSpec = new JoinSpec( + JoinType.INNER, + new EqualPredicate( + new DimensionPredicate(new DefaultDimensionSpec("j1", "dim4", "dim4")), + new DimensionPredicate(new DefaultDimensionSpec("src3", "dim4", "dim4")) + ), + leftChildSpec, + new DataSourceJoinInputSpec(new TableDataSource("src3"), QueryRunnerTestHelper.firstToThird) + ); + + final Query 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 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..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\"}," @@ -68,12 +68,12 @@ 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}," - + "\"context\":null}"; + + "\"context\":{}}"; SelectQuery query = new SelectQuery( new TableDataSource(QueryRunnerTestHelper.dataSource), @@ -104,25 +104,25 @@ 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\":[],"; 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/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java index 04bf6d66dc82..9e567b5684bc 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 ) @@ -105,6 +106,7 @@ public Yielder toYielder( new CountAggregatorFactory("rows") ) ) + .updateDistributionTarget() .build(); Sequence results = queryRunner.run(query, responseContext); Sequences.toList(results, Lists.newArrayList()); @@ -167,6 +169,7 @@ public void run() ); } }, + "foo", new SpecificSegmentSpec( descriptor ) @@ -182,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/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/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 c86123b0ec85..173701d156cf 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -57,6 +57,7 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.query.BySegmentResultValueClass; import io.druid.query.CacheStrategy; +import io.druid.query.DataSourceWithSegmentSpec; import io.druid.query.Query; import io.druid.query.QueryContexts; import io.druid.query.QueryRunner; @@ -164,7 +165,9 @@ public Sequence run(final Query query, final Map responseC contextBuilder.put("bySegment", true); } - TimelineLookup timeline = serverView.getTimeline(query.getDataSource()); + final DataSourceWithSegmentSpec spec = query.getDistributionTarget(); + + TimelineLookup timeline = serverView.getTimeline(spec.getDataSource()); if (timeline == null) { return Sequences.empty(); @@ -183,7 +186,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 +222,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 +339,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 +432,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.withQuerySegmentSpec(spec.getDataSource(), segmentSpec), + responseContext + ); } else { // bySegment queries need to be de-serialized, see DirectDruidClient.run() @@ -439,7 +445,7 @@ private void addSequencesFromServer(ArrayList> listOfSequences) @SuppressWarnings("unchecked") final Sequence>> resultSequence = clientQueryable.run( - bySegmentQuery.withQuerySegmentSpec(segmentSpec), + bySegmentQuery.withQuerySegmentSpec(spec.getDataSource(), segmentSpec), responseContext ); @@ -472,7 +478,7 @@ public Result> apply(Result>> runningSequence = clientQueryable.run( - rewrittenQuery.withQuerySegmentSpec(segmentSpec), + rewrittenQuery.withQuerySegmentSpec(spec.getDataSource(), 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..1aa703c5e6ac 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 sourceWithSegmentSpec = query.getDistributionTarget(); + final String dataSourceName = Iterables.getOnlyElement(sourceWithSegmentSpec.getDataSource().getNames()); + final Map partitionChiefs = chiefs.get(dataSourceName); return partitionChiefs == null ? new NoopQueryRunner() @@ -195,7 +198,12 @@ 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.withQuerySegmentSpec( + sourceWithSegmentSpec.getDataSource(), + 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..cda9a3d0bfce 100644 --- a/server/src/main/java/io/druid/server/BrokerQueryResource.java +++ b/server/src/main/java/io/druid/server/BrokerQueryResource.java @@ -28,8 +28,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.QueryContexts; import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChestWarehouse; import io.druid.server.http.security.StateResourceFilter; @@ -102,11 +104,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( + QueryContexts.DISTRIBUTION_TARGET_SOURCE, + key -> query.getDataSources().get(0) + ); 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..a1fc0bad16c1 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 List 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 f161e9a064bf..1e1585b33767 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; @@ -196,7 +197,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); } @@ -205,13 +206,15 @@ 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(); + for (DataSourceWithSegmentSpec eachSource : query.getDataSources()) { + 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 { @@ -227,7 +230,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..628240162b69 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,60 @@ 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 = query.getDataSources().stream() + .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 -> { + // 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() + .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 +353,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 +388,7 @@ public QueryRunner apply(PartitionChunk input) ); } - private String getDataSourceName(DataSource dataSource) + private static String getDataSourceName(DataSource dataSource) { return Iterables.getOnlyElement(dataSource.getNames()); } @@ -342,22 +399,62 @@ 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 = query.getDataSources().stream() + .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 -> { + // 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() + .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 +467,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 +510,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 +557,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 +568,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..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,9 +20,8 @@ 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; import io.druid.server.RequestLogLine; import org.slf4j.MDC; @@ -57,18 +56,16 @@ 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", query.getDataSource().toString()); + MDC.put("dataSource", 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 - ? 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/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..59cc6dfa351a 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++) { @@ -1966,6 +2003,7 @@ public void run() expected, runner.run( query.withQuerySegmentSpec( + spec.getDataSource(), new MultipleIntervalSegmentSpec( ImmutableList.of( actualQueryInterval @@ -2155,6 +2193,8 @@ public void testQueryCaching( expectedResultsRangeEnd = i + 1; } + final DataSourceWithSegmentSpec spec = query.getDistributionTarget(); + runWithMocks( new Runnable() { @@ -2190,6 +2230,7 @@ public Iterable>> apply(@Nullable Integer input) ), runner.run( query.withQuerySegmentSpec( + spec.getDataSource(), 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/query/dimension/LookupDimensionSpecTest.java b/server/src/test/java/io/druid/query/dimension/LookupDimensionSpecTest.java index 6c94da7f8967..b752d1f7949f 100644 --- a/server/src/test/java/io/druid/query/dimension/LookupDimensionSpecTest.java +++ b/server/src/test/java/io/druid/query/dimension/LookupDimensionSpecTest.java @@ -63,6 +63,17 @@ public class LookupDimensionSpecTest private final DimensionSpec lookupDimSpec = new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, null, null, null, true ); + private final DimensionSpec lookupDimSpecWithDataSource = new LookupDimensionSpec( + "dataSourceName", + "dimName", + "outputName", + MAP_LOOKUP_EXTRACTOR, + true, + null, + null, + null, + true + ); @Parameters @@ -84,7 +95,8 @@ private Object[] parametersForTestSerDesr() new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, true, null, null, null, true), new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, "Missing_value", null, null, true), new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, null, null, null, true), - new LookupDimensionSpec("dimName", "outputName", null, false, null, "name", LOOKUP_REF_MANAGER, true) + new LookupDimensionSpec("dimName", "outputName", null, false, null, "name", LOOKUP_REF_MANAGER, true), + new LookupDimensionSpec("dataSource", "dimName", "outputName", MAP_LOOKUP_EXTRACTOR, true, null, null, null, true) }; } @@ -193,6 +205,20 @@ public Object[] parametersForTestGetCacheKey() new Object[]{ new LookupDimensionSpec("dimName", "outputName", null, false, null, "name", LOOKUP_REF_MANAGER, true), false + }, + new Object[]{ + new LookupDimensionSpec( + "dataSourceName", + "dimName", + "outputName", + MAP_LOOKUP_EXTRACTOR, + true, + null, + null, + null, + true + ), + true } }; } @@ -201,7 +227,14 @@ public Object[] parametersForTestGetCacheKey() @Parameters public void testGetCacheKey(DimensionSpec dimensionSpec, boolean expectedResult) { - Assert.assertEquals(expectedResult, Arrays.equals(lookupDimSpec.getCacheKey(), dimensionSpec.getCacheKey())); + if (dimensionSpec.getDataSourceName() != null) { + Assert.assertEquals( + expectedResult, + Arrays.equals(lookupDimSpecWithDataSource.getCacheKey(), dimensionSpec.getCacheKey()) + ); + } else { + Assert.assertEquals(expectedResult, Arrays.equals(lookupDimSpec.getCacheKey(), dimensionSpec.getCacheKey())); + } } @Test 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..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,9 +424,10 @@ private Future assertQueryable( .granularity(granularity) .limit(10000) .query("wow") + .updateDistributionTarget() .build(); final QueryRunner> runner = serverManager.getQueryRunnerForIntervals( - query, + query.updateDistributionTarget(), intervals ); return serverManagerExec.submit( @@ -574,9 +575,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 @@ -592,6 +593,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..437bc60db78d 100644 --- a/server/src/test/java/io/druid/server/log/LoggingRequestLoggerTest.java +++ b/server/src/test/java/io/druid/server/log/LoggingRequestLoggerTest.java @@ -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")); 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..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,7 +30,9 @@ 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; @@ -150,6 +152,9 @@ private Sequence executeSelect( final SelectQuery baseQuery ) { + final SelectQuery running = (SelectQuery) baseQuery.distributeBy( + new DataSourceWithSegmentSpec(BaseQuery.getLeafDataSource(baseQuery), baseQuery.getQuerySegmentSpec()) + ); Preconditions.checkState(queryBuilder.getGrouping() == null, "grouping must be null"); final List fieldList = queryBuilder.getRowType().getFieldList(); @@ -178,7 +183,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 +262,17 @@ private Sequence executeTimeseries( final TimeseriesQuery query ) { + final TimeseriesQuery running = (TimeseriesQuery) query.distributeBy( + new DataSourceWithSegmentSpec(BaseQuery.getLeafDataSource(query), query.getQuerySegmentSpec()) + ); 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 +301,16 @@ private Sequence executeTopN( final TopNQuery query ) { + final TopNQuery running = (TopNQuery) query.distributeBy( + new DataSourceWithSegmentSpec(BaseQuery.getLeafDataSource(query), query.getQuerySegmentSpec()) + ); 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 +342,14 @@ private Sequence executeGroupBy( ) { final List fieldList = queryBuilder.getRowType().getFieldList(); + final GroupByQuery running = (GroupByQuery) query.distributeBy( + new DataSourceWithSegmentSpec(BaseQuery.getLeafDataSource(query), query.getQuerySegmentSpec()) + ); - 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..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 @@ -305,7 +305,9 @@ private DruidTable computeTable(final String dataSource) true ); - final Sequence sequence = segmentMetadataQuery.run(walker, Maps.newHashMap()); + final Sequence sequence = segmentMetadataQuery + .updateDistributionTarget() + .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 1ef1fe6d2097..c2d6836d9aa9 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() ), @@ -1487,6 +1524,7 @@ public void testFilteredAggregations() throws Exception ) )) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1518,6 +1556,7 @@ public void testExpressionAggregations() throws Exception )) )) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1544,6 +1583,7 @@ public void testInFilter() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1566,6 +1606,7 @@ public void testCountStarWithDegenerateFilter() throws Exception .filters(SELECTOR("dim2", "a", null)) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1587,6 +1628,7 @@ public void testCountStarWithNotOfDegenerateFilter() throws Exception .filters(null) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of() @@ -1606,6 +1648,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( @@ -1627,6 +1670,7 @@ public void testCountStarWithBoundFilterSimplifyAnd() throws Exception .filters(SELECTOR("dim1", "abc", null)) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1648,6 +1692,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( @@ -1669,6 +1714,7 @@ public void testCountStarWithTimeFilter() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1689,6 +1735,7 @@ public void testCountStarWithSinglePointInTime() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1715,6 +1762,7 @@ public void testCountStarWithTwoPointsInTime() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1755,6 +1803,7 @@ public void testCountStarWithComplexDisjointTimeFilter() throws Exception ) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1796,6 +1845,7 @@ public void testCountStarWithNotOfComplexDisjointTimeFilter() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1826,6 +1876,7 @@ public void testCountStarWithNotTimeFilter() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1849,6 +1900,7 @@ public void testCountStarWithTimeAndDimFilter() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1885,6 +1937,7 @@ public void testCountStarWithTimeOrDimFilter() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1918,6 +1971,7 @@ public void testCountStarWithTimeFilterOnLongColumn() throws Exception ) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1951,6 +2005,7 @@ public void testSelectDistinctWithCascadeExtractionFilter() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -1980,6 +2035,7 @@ public void testSelectDistinctWithStrlenFilter() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2005,6 +2061,7 @@ public void testSelectDistinctWithLimit() throws Exception .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC)) .threshold(10) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2029,6 +2086,7 @@ public void testSelectDistinctWithSortAsOuterQuery() throws Exception .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC)) .threshold(10) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2053,6 +2111,7 @@ public void testSelectDistinctWithSortAsOuterQuery2() throws Exception .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC)) .threshold(5) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2089,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( @@ -2122,6 +2182,7 @@ public void testCountDistinct() throws Exception ) ) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2157,6 +2218,7 @@ public void testExactCountDistinct() throws Exception new CountAggregatorFactory("a0") )) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2189,6 +2251,7 @@ public void testApproxCountDistinctWhenHllDisabled() throws Exception ) ) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2230,6 +2293,7 @@ public void testExactCountDistinctWithGroupingAndOtherAggregators() throws Excep new CountAggregatorFactory("a1") )) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2277,6 +2341,7 @@ public void testApproxCountDistinct() throws Exception ) ) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2331,6 +2396,7 @@ public void testDoubleNestedGroupBy() throws Exception new CountAggregatorFactory("a1") )) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2368,6 +2434,7 @@ public void testExactCountDistinctUsingSubquery() throws Exception new LongSumAggregatorFactory("a0", "a0"), new CountAggregatorFactory("a1") )) + .updateDistributionTarget() .setContext(QUERY_CONTEXT_DEFAULT) .build() ), @@ -2404,6 +2471,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() @@ -2426,6 +2494,7 @@ public void testTopNFilterJoin() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2476,6 +2545,7 @@ public void testRemovableLeftJoin() throws Exception Integer.MAX_VALUE ) ) + .updateDistributionTarget() .setContext(QUERY_CONTEXT_DEFAULT) .build() ), @@ -2514,6 +2584,7 @@ public void testExactCountDistinctOfSemiJoinResult() throws Exception new SubstringDimExtractionFn(0, 1) ))) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build(), GroupByQuery.builder() .setDataSource( @@ -2538,6 +2609,7 @@ public void testExactCountDistinctOfSemiJoinResult() throws Exception new CountAggregatorFactory("a0") )) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), @@ -2578,6 +2650,7 @@ public void testExactCountDistinctUsingSubqueryWithWherePushDown() throws Except new CountAggregatorFactory("a1") )) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2618,6 +2691,7 @@ public void testExactCountDistinctUsingSubqueryWithWhereToOuterFilter() throws E new CountAggregatorFactory("a1") )) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2694,6 +2768,7 @@ public void testCompareExactAndApproximateCountDistinctUsingSubquery() throws Ex ) ) ) + .updateDistributionTarget() .setContext(QUERY_CONTEXT_DEFAULT) .build() ), @@ -2736,6 +2811,7 @@ public void testHistogramUsingSubquery() throws Exception new CountAggregatorFactory("a0") )) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2786,6 +2862,7 @@ public void testHistogramUsingSubqueryWithSort() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2845,6 +2922,7 @@ public void testCountDistinctArithmetic() throws Exception )) )) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2880,6 +2958,7 @@ public void testCountDistinctOfSubstring() throws Exception ) ) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2913,6 +2992,7 @@ public void testRegexpExtract() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -2955,6 +3035,7 @@ public void testGroupBySortPushDown() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3002,6 +3083,7 @@ public void testGroupByLimitPushDownWithHavingOnLong() throws Exception ) .setHavingSpec(new DimFilterHavingSpec(NUMERIC_SELECTOR("a0", "1", null))) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3028,6 +3110,7 @@ public void testFilterOnTimeFloor() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3049,6 +3132,7 @@ public void testFilterOnCurrentTimestamp() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3084,6 +3168,7 @@ public void testFilterOnCurrentTimestampLosAngeles() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_LOS_ANGELES) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3104,6 +3189,7 @@ public void testFilterOnCurrentTimestampOnView() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3128,6 +3214,7 @@ public void testFilterOnCurrentTimestampLosAngelesOnView() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_LOS_ANGELES) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3153,6 +3240,7 @@ public void testFilterOnNotTimeFloor() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3175,6 +3263,7 @@ public void testFilterOnTimeFloorComparison() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3197,6 +3286,7 @@ public void testFilterOnTimeFloorComparisonMisaligned() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3220,6 +3310,7 @@ public void testFilterOnTimeExtract() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3264,6 +3355,7 @@ public void testFilterOnTimeFloorMisaligned() throws Exception .granularity(Granularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of() @@ -3286,6 +3378,7 @@ public void testGroupByFloor() throws Exception )) .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3331,6 +3424,7 @@ public void testGroupByFloorWithOrderBy() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3394,6 +3488,7 @@ public void testGroupByFloorTimeAndOneOtherDimensionWithOrderBy() throws Excepti ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3432,6 +3527,7 @@ public void testGroupByStringLength() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3487,6 +3583,7 @@ public void testFilterAndGroupByLookup() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3523,6 +3620,7 @@ public void testCountDistinctOfLookup() throws Exception ) )) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3548,6 +3646,7 @@ public void testTimeseries() throws Exception .granularity(Granularities.MONTH) .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt"))) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3575,6 +3674,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( @@ -3606,6 +3706,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() @@ -3654,6 +3755,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( @@ -3685,6 +3787,7 @@ public void testTimeseriesDescending() throws Exception .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt"))) .descending(true) .context(TIMESERIES_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3738,6 +3841,7 @@ public void testGroupByExtractYear() throws Exception Integer.MAX_VALUE ) ) + .updateDistributionTarget() .setContext(QUERY_CONTEXT_DEFAULT) .build() ), @@ -3779,6 +3883,7 @@ public void testGroupByExtractFloorTime() throws Exception ) .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3820,6 +3925,7 @@ public void testGroupByExtractFloorTimeLosAngeles() throws Exception ) .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_LOS_ANGELES) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3872,6 +3978,7 @@ public void testTimeseriesWithLimitNoTopN() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3909,6 +4016,7 @@ public void testTimeseriesWithLimit() throws Exception .metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC)) .threshold(1) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3956,6 +4064,7 @@ public void testGroupByTimeAndOtherDimension() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -3985,6 +4094,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) @@ -4010,6 +4120,7 @@ public void testUsingSubqueryAsFilter() throws Exception ) ) .setContext(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -4057,6 +4168,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) @@ -4070,6 +4182,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) @@ -4089,6 +4202,7 @@ public void testUsingSubqueryAsFilterOnTwoColumns() throws Exception ) ) .context(QUERY_CONTEXT_DEFAULT) + .updateDistributionTarget() .build() ), ImmutableList.of( @@ -4114,6 +4228,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) @@ -4129,6 +4244,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( @@ -4166,6 +4282,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) ); }