diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java index 2d4d0f5a6501..72fefa93946a 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -25,7 +25,6 @@ import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; import com.google.common.collect.Ordering; import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; import org.apache.druid.benchmark.datagen.BenchmarkSchemas; @@ -58,7 +57,6 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.BySegmentQueryRunner; -import org.apache.druid.query.DataSource; import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.Druids; @@ -89,6 +87,7 @@ import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.query.groupby.strategy.GroupByStrategyV1; import org.apache.druid.query.groupby.strategy.GroupByStrategyV2; +import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.query.timeseries.TimeseriesQuery; @@ -126,7 +125,6 @@ import org.openjdk.jmh.annotations.Warmup; import org.openjdk.jmh.infra.Blackhole; -import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Collections; @@ -134,6 +132,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Optional; import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; import java.util.concurrent.ForkJoinPool; @@ -217,8 +216,17 @@ public void setup() .size(0) .build(); final SegmentGenerator segmentGenerator = closer.register(new SegmentGenerator()); - LOG.info("Starting benchmark setup using cacheDir[%s], rows[%,d].", segmentGenerator.getCacheDir(), rowsPerSegment); - final QueryableIndex index = segmentGenerator.generate(dataSegment, schemaInfo, Granularities.NONE, rowsPerSegment); + LOG.info( + "Starting benchmark setup using cacheDir[%s], rows[%,d].", + segmentGenerator.getCacheDir(), + rowsPerSegment + ); + final QueryableIndex index = segmentGenerator.generate( + dataSegment, + schemaInfo, + Granularities.NONE, + rowsPerSegment + ); queryableIndexes.put(dataSegment, index); } @@ -518,12 +526,10 @@ void addSegmentToServer(DruidServer server, DataSegment segment) .add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(selector)); } - @Nullable @Override - public TimelineLookup getTimeline(DataSource dataSource) + public Optional> getTimeline(DataSourceAnalysis analysis) { - final String table = Iterables.getOnlyElement(dataSource.getNames()); - return timelines.get(table); + return Optional.ofNullable(timelines.get(analysis.getBaseTableDataSource().get().getName())); } @Override @@ -563,7 +569,11 @@ private class SimpleQueryRunner implements QueryRunner private final QueryRunnerFactoryConglomerate conglomerate; private final QueryableIndexSegment segment; - public SimpleQueryRunner(QueryRunnerFactoryConglomerate conglomerate, SegmentId segmentId, QueryableIndex queryableIndex) + public SimpleQueryRunner( + QueryRunnerFactoryConglomerate conglomerate, + SegmentId segmentId, + QueryableIndex queryableIndex + ) { this.conglomerate = conglomerate; this.segment = new QueryableIndexSegment(queryableIndex, segmentId); diff --git a/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/DataSourceOptimizer.java b/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/DataSourceOptimizer.java index bbd6aa8a4f2b..1653539f3e36 100644 --- a/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/DataSourceOptimizer.java +++ b/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/DataSourceOptimizer.java @@ -23,9 +23,11 @@ import com.google.common.collect.ImmutableSortedSet; import com.google.inject.Inject; import org.apache.druid.client.TimelineServerView; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.query.Query; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.topn.TopNQuery; @@ -54,24 +56,24 @@ public class DataSourceOptimizer private ConcurrentHashMap hitCount = new ConcurrentHashMap<>(); private ConcurrentHashMap costTime = new ConcurrentHashMap<>(); private ConcurrentHashMap, AtomicLong>> missFields = new ConcurrentHashMap<>(); - + @Inject - public DataSourceOptimizer(TimelineServerView serverView) + public DataSourceOptimizer(TimelineServerView serverView) { this.serverView = serverView; } /** * Do main work about materialized view selection: transform user query to one or more sub-queries. - * - * In the sub-query, the dataSource is the derivative of dataSource in user query, and sum of all sub-queries' + * + * In the sub-query, the dataSource is the derivative of dataSource in user query, and sum of all sub-queries' * intervals equals the interval in user query - * + * * Derived dataSource with smallest average data size per segment granularity have highest priority to replace the * datasource in user query - * + * * @param query only TopNQuery/TimeseriesQuery/GroupByQuery can be optimized - * @return a list of queries with specified derived dataSources and intervals + * @return a list of queries with specified derived dataSources and intervals */ public List optimize(Query query) { @@ -86,7 +88,7 @@ public List optimize(Query query) // get all derivatives for datasource in query. The derivatives set is sorted by average size of // per segment granularity. Set derivatives = DerivativeDataSourceManager.getDerivatives(datasourceName); - + if (derivatives.isEmpty()) { return Collections.singletonList(query); } @@ -96,10 +98,10 @@ public List optimize(Query query) hitCount.putIfAbsent(datasourceName, new AtomicLong(0)); costTime.putIfAbsent(datasourceName, new AtomicLong(0)); totalCount.get(datasourceName).incrementAndGet(); - + // get all fields which the query required Set requiredFields = MaterializedViewUtils.getRequiredFields(query); - + Set derivativesWithRequiredFields = new HashSet<>(); for (DerivativeDataSource derivativeDataSource : derivatives) { derivativesHitCount.putIfAbsent(derivativeDataSource.getName(), new AtomicLong(0)); @@ -115,14 +117,15 @@ public List optimize(Query query) costTime.get(datasourceName).addAndGet(System.currentTimeMillis() - start); return Collections.singletonList(query); } - + List queries = new ArrayList<>(); List remainingQueryIntervals = (List) query.getIntervals(); - + for (DerivativeDataSource derivativeDataSource : ImmutableSortedSet.copyOf(derivativesWithRequiredFields)) { final List derivativeIntervals = remainingQueryIntervals.stream() .flatMap(interval -> serverView - .getTimeline((new TableDataSource(derivativeDataSource.getName()))) + .getTimeline(DataSourceAnalysis.forDataSource(new TableDataSource(derivativeDataSource.getName()))) + .orElseThrow(() -> new ISE("No timeline for dataSource: %s", derivativeDataSource.getName())) .lookup(interval) .stream() .map(TimelineObjectHolder::getInterval) @@ -133,7 +136,7 @@ public List optimize(Query query) if (derivativeIntervals.isEmpty()) { continue; } - + remainingQueryIntervals = MaterializedViewUtils.minus(remainingQueryIntervals, derivativeIntervals); queries.add( query.withDataSource(new TableDataSource(derivativeDataSource.getName())) @@ -158,13 +161,13 @@ public List optimize(Query query) hitCount.get(datasourceName).incrementAndGet(); costTime.get(datasourceName).addAndGet(System.currentTimeMillis() - start); return queries; - } + } finally { lock.readLock().unlock(); } } - public List getAndResetStats() + public List getAndResetStats() { ImmutableMap derivativesHitCountSnapshot; ImmutableMap totalCountSnapshot; @@ -183,7 +186,7 @@ public List getAndResetStats() hitCount.clear(); costTime.clear(); missFields.clear(); - } + } finally { lock.writeLock().unlock(); } diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index 83881c79cf97..9090bfe168d8 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -49,7 +49,6 @@ import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.emitter.core.Event; import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.query.DataSource; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; @@ -62,6 +61,7 @@ import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.movingaverage.test.TestConfig; +import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.apache.druid.server.ClientQuerySegmentWalker; @@ -84,6 +84,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.Executor; import java.util.concurrent.ForkJoinPool; @@ -305,9 +306,9 @@ public void testQuery() throws IOException new TimelineServerView() { @Override - public TimelineLookup getTimeline(DataSource dataSource) + public Optional> getTimeline(DataSourceAnalysis analysis) { - return null; + return Optional.empty(); } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunner.java index fab5a4f13919..7e7fafac3e8b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunner.java @@ -22,7 +22,6 @@ import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; @@ -51,6 +50,7 @@ import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.server.DruidNode; import org.apache.druid.server.SetAndVerifyContextQueryRunner; import org.apache.druid.server.initialization.ServerConfig; @@ -328,11 +328,13 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable QueryRunner getQueryRunnerImpl(Query query) { QueryRunner queryRunner = null; - final String queryDataSource = Iterables.getOnlyElement(query.getDataSource().getNames()); if (runningItem != null) { + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(query.getDataSource()); final Task task = runningItem.getTask(); - if (task.getDataSource().equals(queryDataSource)) { + + if (analysis.getBaseTableDataSource().isPresent() + && task.getDataSource().equals(analysis.getBaseTableDataSource().get().getName())) { final QueryRunner taskQueryRunner = task.getQueryRunner(query); if (taskQueryRunner != null) { @@ -379,7 +381,7 @@ public String getTaskType() { return task.getType(); } - + @Override public String getDataSource() { diff --git a/processing/src/main/java/org/apache/druid/query/BaseQuery.java b/processing/src/main/java/org/apache/druid/query/BaseQuery.java index e967d54c6017..caaa5e2d67f9 100644 --- a/processing/src/main/java/org/apache/druid/query/BaseQuery.java +++ b/processing/src/main/java/org/apache/druid/query/BaseQuery.java @@ -28,6 +28,7 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.PeriodGranularity; +import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.query.spec.QuerySegmentSpec; import org.joda.time.DateTimeZone; import org.joda.time.Duration; @@ -117,17 +118,11 @@ public QueryRunner getRunner(QuerySegmentWalker walker) } @VisibleForTesting - public static QuerySegmentSpec getQuerySegmentSpecForLookUp(BaseQuery query) + public static QuerySegmentSpec getQuerySegmentSpecForLookUp(BaseQuery query) { - if (query.getDataSource() instanceof QueryDataSource) { - QueryDataSource ds = (QueryDataSource) query.getDataSource(); - Query subquery = ds.getQuery(); - if (subquery instanceof BaseQuery) { - return getQuerySegmentSpecForLookUp((BaseQuery) subquery); - } - throw new IllegalStateException("Invalid subquery type " + subquery.getClass()); - } - return query.getQuerySegmentSpec(); + return DataSourceAnalysis.forDataSource(query.getDataSource()) + .getBaseQuerySegmentSpec() + .orElse(query.getQuerySegmentSpec()); } @Override @@ -270,14 +265,13 @@ public boolean equals(Object o) Objects.equals(dataSource, baseQuery.dataSource) && Objects.equals(context, baseQuery.context) && Objects.equals(querySegmentSpec, baseQuery.querySegmentSpec) && - Objects.equals(duration, baseQuery.duration) && + Objects.equals(getDuration(), baseQuery.getDuration()) && Objects.equals(granularity, baseQuery.granularity); } @Override public int hashCode() { - - return Objects.hash(dataSource, descending, context, querySegmentSpec, duration, granularity); + return Objects.hash(dataSource, descending, context, querySegmentSpec, getDuration(), granularity); } } diff --git a/processing/src/main/java/org/apache/druid/query/DataSource.java b/processing/src/main/java/org/apache/druid/query/DataSource.java index 13a3cf5cc4bf..06fcf21ba41b 100644 --- a/processing/src/main/java/org/apache/druid/query/DataSource.java +++ b/processing/src/main/java/org/apache/druid/query/DataSource.java @@ -23,17 +23,61 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import java.util.List; +import java.util.Set; -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, - include = JsonTypeInfo.As.PROPERTY, - property = "type", - defaultImpl = LegacyDataSource.class) +/** + * Represents a source... of data... for a query. Analogous to the "FROM" clause in SQL. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = LegacyDataSource.class) @JsonSubTypes({ - @JsonSubTypes.Type(value = TableDataSource.class, name = "table"), - @JsonSubTypes.Type(value = QueryDataSource.class, name = "query"), - @JsonSubTypes.Type(value = UnionDataSource.class, name = "union") - }) + @JsonSubTypes.Type(value = TableDataSource.class, name = "table"), + @JsonSubTypes.Type(value = QueryDataSource.class, name = "query"), + @JsonSubTypes.Type(value = UnionDataSource.class, name = "union"), + @JsonSubTypes.Type(value = JoinDataSource.class, name = "join"), + @JsonSubTypes.Type(value = LookupDataSource.class, name = "lookup"), + @JsonSubTypes.Type(value = InlineDataSource.class, name = "inline") +}) public interface DataSource { - List getNames(); + /** + * Returns the names of all table datasources involved in this query. Does not include names for non-tables, like + * lookups or inline datasources. + */ + Set getTableNames(); + + /** + * Returns datasources that this datasource depends on. Will be empty for leaf datasources like 'table'. + */ + List getChildren(); + + /** + * Return a new DataSource, identical to this one, with different children. + */ + DataSource withChildren(List children); + + /** + * Returns true if queries on this dataSource are cacheable at both the result level and per-segment level. + * Currently, dataSources that modify the behavior of per-segment processing are not cacheable (like 'join'). + * Nor are dataSources that do not actually reference segments (like 'inline'), since cache keys are always based + * on segment identifiers. + * + * Note: Ideally, queries on 'join' datasources _would_ be cacheable, but we cannot currently do this due to lacking + * the code necessary to compute cache keys properly. + */ + boolean isCacheable(); + + /** + * Returns true if all servers have a full copy of this datasource. True for things like inline, lookup, etc, or + * for queries of those. + */ + boolean isGlobal(); + + /** + * Returns true if this datasource represents concrete data that can be scanned via a + * {@link org.apache.druid.segment.Segment} adapter of some kind. True for e.g. 'table' but not for 'query' or 'join'. + * + * @see org.apache.druid.query.planning.DataSourceAnalysis#isConcreteBased() which uses this + * @see org.apache.druid.query.planning.DataSourceAnalysis#isConcreteTableBased() which uses this + */ + boolean isConcrete(); } diff --git a/processing/src/main/java/org/apache/druid/query/DataSourceUtil.java b/processing/src/main/java/org/apache/druid/query/DataSourceUtil.java deleted file mode 100644 index fdc7747f2d93..000000000000 --- a/processing/src/main/java/org/apache/druid/query/DataSourceUtil.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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 org.apache.druid.query; - -import java.util.List; - -public class DataSourceUtil -{ - public static String getMetricName(DataSource dataSource) - { - final List names = dataSource.getNames(); - return names.size() == 1 ? names.get(0) : names.toString(); - } -} diff --git a/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java index 145f8dac3527..fcdebd22bc8c 100644 --- a/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java +++ b/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java @@ -20,6 +20,7 @@ package org.apache.druid.query; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.service.ServiceEmitter; @@ -30,7 +31,9 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; /** * DefaultQueryMetrics is unsafe for use from multiple threads. It fails with RuntimeException on access not from the @@ -42,9 +45,22 @@ public class DefaultQueryMetrics> implements QueryMet protected final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); protected final Map metrics = new HashMap<>(); - /** Non final to give subclasses ability to reassign it. */ + /** + * Non final to give subclasses ability to reassign it. + */ protected Thread ownerThread = Thread.currentThread(); + private static String getTableNamesAsString(DataSource dataSource) + { + final Set names = dataSource.getTableNames(); + + if (names.size() == 1) { + return Iterables.getOnlyElement(names); + } else { + return names.stream().sorted().collect(Collectors.toList()).toString(); + } + } + protected void checkModifiedFromOwnerThread() { if (Thread.currentThread() != ownerThread) { @@ -77,7 +93,7 @@ public void query(QueryType query) @Override public void dataSource(QueryType query) { - setDimension(DruidMetrics.DATASOURCE, DataSourceUtil.getMetricName(query.getDataSource())); + setDimension(DruidMetrics.DATASOURCE, getTableNamesAsString(query.getDataSource())); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java new file mode 100644 index 000000000000..9c266ab8ea2e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java @@ -0,0 +1,241 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.query; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.segment.RowAdapter; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ValueType; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.function.Function; +import java.util.function.ToLongFunction; + +/** + * Represents an inline datasource, where the rows are embedded within the DataSource object itself. + * + * The rows are backed by an Iterable, which can be lazy or not. Lazy datasources will only be iterated if someone calls + * {@link #getRows()} and iterates the result, or until someone calls {@link #getRowsAsList()}. + */ +public class InlineDataSource implements DataSource +{ + private final List columnNames; + private final List columnTypes; + private final Iterable rows; + + private InlineDataSource( + final List columnNames, + final List columnTypes, + final Iterable rows + ) + { + this.columnNames = Preconditions.checkNotNull(columnNames, "'columnNames' must be nonnull"); + this.columnTypes = Preconditions.checkNotNull(columnTypes, "'columnTypes' must be nonnull"); + this.rows = Preconditions.checkNotNull(rows, "'rows' must be nonnull"); + + if (columnNames.size() != columnTypes.size()) { + throw new IAE("columnNames and columnTypes must be the same length"); + } + } + + /** + * Factory method for Jackson. Used for inline datasources that were originally encoded as JSON. Private because + * non-Jackson callers should use {@link #fromIterable}. + */ + @JsonCreator + public static InlineDataSource fromJson( + @JsonProperty("columnNames") List columnNames, + @JsonProperty("columnTypes") List columnTypes, + @JsonProperty("rows") List rows + ) + { + return new InlineDataSource(columnNames, columnTypes, rows); + } + + /** + * Creates an inline datasource from an Iterable. The Iterable will not be iterated until someone calls + * {@link #getRows()} and iterates the result, or until someone calls {@link #getRowsAsList()}. + * + * @param columnNames names of each column in the rows + * @param columnTypes types of each column in the rows + * @param rows rows, each of the same length as columnNames and columnTypes + */ + public static InlineDataSource fromIterable( + final List columnNames, + final List columnTypes, + final Iterable rows + ) + { + return new InlineDataSource(columnNames, columnTypes, rows); + } + + @Override + public Set getTableNames() + { + return Collections.emptySet(); + } + + @JsonProperty + public List getColumnNames() + { + return columnNames; + } + + @JsonProperty + public List getColumnTypes() + { + return columnTypes; + } + + /** + * Returns rows as a list. If the original Iterable behind this datasource was a List, this method will return it + * as-is, without copying it. Otherwise, this method will walk the iterable and copy it into a List before returning. + */ + @JsonProperty("rows") + public List getRowsAsList() + { + return rows instanceof List ? ((List) rows) : Lists.newArrayList(rows); + } + + /** + * Returns rows as an Iterable. + */ + @JsonIgnore + public Iterable getRows() + { + return rows; + } + + @Override + public List getChildren() + { + return Collections.emptyList(); + } + + @Override + public DataSource withChildren(List children) + { + if (!children.isEmpty()) { + throw new IAE("Cannot accept children"); + } + + return this; + } + + @Override + public boolean isCacheable() + { + return false; + } + + @Override + public boolean isGlobal() + { + return true; + } + + @Override + public boolean isConcrete() + { + return false; + } + + public Map getRowSignature() + { + final ImmutableMap.Builder retVal = ImmutableMap.builder(); + + for (int i = 0; i < columnNames.size(); i++) { + retVal.put(columnNames.get(i), columnTypes.get(i)); + } + + return retVal.build(); + } + + public RowAdapter rowAdapter() + { + return new RowAdapter() + { + @Override + public ToLongFunction timestampFunction() + { + final int columnNumber = columnNames.indexOf(ColumnHolder.TIME_COLUMN_NAME); + + if (columnNumber >= 0) { + return row -> (long) row[columnNumber]; + } else { + return row -> 0L; + } + } + + @Override + public Function columnFunction(String columnName) + { + final int columnNumber = columnNames.indexOf(columnName); + + if (columnNumber >= 0) { + return row -> row[columnNumber]; + } else { + return row -> null; + } + } + }; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + InlineDataSource that = (InlineDataSource) o; + return Objects.equals(columnNames, that.columnNames) && + Objects.equals(columnTypes, that.columnTypes) && + Objects.equals(rows, that.rows); + } + + @Override + public int hashCode() + { + return Objects.hash(columnNames, columnTypes, rows); + } + + @Override + public String toString() + { + // Don't include 'rows' in stringificatione, because it might be long and/or lazy. + return "InlineDataSource{" + + "columnNames=" + columnNames + + ", columnTypes=" + columnTypes + + '}'; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java new file mode 100644 index 000000000000..087a666c871e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java @@ -0,0 +1,212 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.query; + +import com.fasterxml.jackson.annotation.JacksonInject; +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 org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.segment.join.JoinConditionAnalysis; +import org.apache.druid.segment.join.JoinType; +import org.apache.druid.segment.join.Joinables; + +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Set; + +/** + * Represents a join of two datasources. + * + * Logically, this datasource contains the result of: + * + * (1) prefixing all right-side columns with "rightPrefix" + * (2) then, joining the left and (prefixed) right sides using the provided type and condition + * + * Any columns from the left-hand side that start with "rightPrefix", and are at least one character longer than + * the prefix, will be shadowed. It is up to the caller to ensure that no important columns are shadowed by the + * chosen prefix. + * + * When analyzed by {@link org.apache.druid.query.planning.DataSourceAnalysis}, the right-hand side of this datasource + * will become a {@link org.apache.druid.query.planning.PreJoinableClause} object. + */ +public class JoinDataSource implements DataSource +{ + private final DataSource left; + private final DataSource right; + private final String rightPrefix; + private final JoinConditionAnalysis conditionAnalysis; + private final JoinType joinType; + + private JoinDataSource( + DataSource left, + DataSource right, + String rightPrefix, + JoinConditionAnalysis conditionAnalysis, + JoinType joinType + ) + { + this.left = Preconditions.checkNotNull(left, "left"); + this.right = Preconditions.checkNotNull(right, "right"); + this.rightPrefix = Joinables.validatePrefix(rightPrefix); + this.conditionAnalysis = Preconditions.checkNotNull(conditionAnalysis, "conditionAnalysis"); + this.joinType = Preconditions.checkNotNull(joinType, "joinType"); + } + + @JsonCreator + public static JoinDataSource create( + @JsonProperty("left") DataSource left, + @JsonProperty("right") DataSource right, + @JsonProperty("rightPrefix") String rightPrefix, + @JsonProperty("condition") String condition, + @JsonProperty("joinType") JoinType joinType, + @JacksonInject ExprMacroTable macroTable + ) + { + return new JoinDataSource( + left, + right, + StringUtils.nullToEmptyNonDruidDataString(rightPrefix), + JoinConditionAnalysis.forExpression( + Preconditions.checkNotNull(condition, "condition"), + StringUtils.nullToEmptyNonDruidDataString(rightPrefix), + macroTable + ), + joinType + ); + } + + @Override + public Set getTableNames() + { + final Set names = new HashSet<>(); + names.addAll(left.getTableNames()); + names.addAll(right.getTableNames()); + return names; + } + + @JsonProperty + public DataSource getLeft() + { + return left; + } + + @JsonProperty + public DataSource getRight() + { + return right; + } + + @JsonProperty + public String getRightPrefix() + { + return rightPrefix; + } + + @JsonProperty + public String getCondition() + { + return conditionAnalysis.getOriginalExpression(); + } + + public JoinConditionAnalysis getConditionAnalysis() + { + return conditionAnalysis; + } + + @JsonProperty + public JoinType getJoinType() + { + return joinType; + } + + @Override + public List getChildren() + { + return ImmutableList.of(left, right); + } + + @Override + public DataSource withChildren(List children) + { + if (children.size() != 2) { + throw new IAE("Expected [2] children, got [%d]", children.size()); + } + + return new JoinDataSource(children.get(0), children.get(1), rightPrefix, conditionAnalysis, joinType); + } + + @Override + public boolean isCacheable() + { + return false; + } + + @Override + public boolean isGlobal() + { + return left.isGlobal() && right.isGlobal(); + } + + @Override + public boolean isConcrete() + { + return false; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + JoinDataSource that = (JoinDataSource) o; + return Objects.equals(left, that.left) && + Objects.equals(right, that.right) && + Objects.equals(rightPrefix, that.rightPrefix) && + Objects.equals(conditionAnalysis, that.conditionAnalysis) && + joinType == that.joinType; + } + + @Override + public int hashCode() + { + return Objects.hash(left, right, rightPrefix, conditionAnalysis, joinType); + } + + @Override + public String toString() + { + return "JoinDataSource{" + + "left=" + left + + ", right=" + right + + ", rightPrefix='" + rightPrefix + '\'' + + ", condition=" + conditionAnalysis + + ", joinType=" + joinType + + '}'; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/LookupDataSource.java b/processing/src/main/java/org/apache/druid/query/LookupDataSource.java new file mode 100644 index 000000000000..a2c99f7d1fd3 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/LookupDataSource.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.query; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.IAE; + +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Set; + +/** + * Represents a lookup. + * + * Currently, this datasource is not actually queryable, and attempts to do so will lead to errors. It is here as a + * placeholder for a future time in which it will become queryable. + * + * The "lookupName" referred to here should be provided by a + * {@link org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider}. + */ +public class LookupDataSource implements DataSource +{ + private final String lookupName; + + @JsonCreator + public LookupDataSource( + @JsonProperty("lookup") String lookupName + ) + { + this.lookupName = Preconditions.checkNotNull(lookupName, "lookup"); + } + + @Override + public Set getTableNames() + { + return Collections.emptySet(); + } + + @JsonProperty("lookup") + public String getLookupName() + { + return lookupName; + } + + @Override + public List getChildren() + { + return Collections.emptyList(); + } + + @Override + public DataSource withChildren(List children) + { + if (!children.isEmpty()) { + throw new IAE("Cannot accept children"); + } + + return this; + } + + @Override + public boolean isCacheable() + { + return false; + } + + @Override + public boolean isGlobal() + { + return true; + } + + @Override + public boolean isConcrete() + { + return false; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LookupDataSource that = (LookupDataSource) o; + return Objects.equals(lookupName, that.lookupName); + } + + @Override + public int hashCode() + { + return Objects.hash(lookupName); + } + + @Override + public String toString() + { + return "LookupDataSource{" + + "lookupName='" + lookupName + '\'' + + '}'; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/Queries.java b/processing/src/main/java/org/apache/druid/query/Queries.java index 1fbe33587d00..a2ff0051bf94 100644 --- a/processing/src/main/java/org/apache/druid/query/Queries.java +++ b/processing/src/main/java/org/apache/druid/query/Queries.java @@ -23,8 +23,11 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; import org.apache.druid.guice.annotations.PublicApi; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; +import org.apache.druid.query.planning.DataSourceAnalysis; +import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; import java.util.Collections; import java.util.HashMap; @@ -33,8 +36,6 @@ import java.util.Map; import java.util.Set; -/** - */ @PublicApi public class Queries { @@ -130,4 +131,46 @@ public static List prepareAggregations( return postAggs; } + + /** + * Rewrite "query" to refer to some specific segment descriptors. + * + * The dataSource for "query" must be based on a single table for this operation to be valid. Otherwise, this + * function will throw an exception. + * + * Unlike the seemingly-similar {@code query.withQuerySegmentSpec(new MultipleSpecificSegmentSpec(descriptors))}, + * this this method will walk down subqueries found within the query datasource, if any, and modify the lowest-level + * subquery. The effect is that + * {@code DataSourceAnalysis.forDataSource(query.getDataSource()).getBaseQuerySegmentSpec()} is guaranteed to return + * either {@code new MultipleSpecificSegmentSpec(descriptors)} or empty. + * + * Because {@link BaseQuery#getRunner} is implemented using {@link DataSourceAnalysis#getBaseQuerySegmentSpec}, this + * method will cause the runner to be a specific-segments runner. + */ + public static Query withSpecificSegments(final Query query, final List descriptors) + { + final Query retVal; + + if (query.getDataSource() instanceof QueryDataSource) { + final Query subQuery = ((QueryDataSource) query.getDataSource()).getQuery(); + retVal = query.withDataSource(new QueryDataSource(withSpecificSegments(subQuery, descriptors))); + } else { + retVal = query.withQuerySegmentSpec(new MultipleSpecificSegmentSpec(descriptors)); + } + + // Verify preconditions and invariants, just in case. + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(retVal.getDataSource()); + + if (!analysis.getBaseTableDataSource().isPresent()) { + throw new ISE("Unable to apply specific segments to non-table-based dataSource[%s]", query.getDataSource()); + } + + if (analysis.getBaseQuerySegmentSpec().isPresent() + && !analysis.getBaseQuerySegmentSpec().get().equals(new MultipleSpecificSegmentSpec(descriptors))) { + // If you see the error message below, it's a bug in either this function or in DataSourceAnalysis. + throw new ISE("Unable to apply specific segments to query with dataSource[%s]", query.getDataSource()); + } + + return retVal; + } } diff --git a/processing/src/main/java/org/apache/druid/query/Query.java b/processing/src/main/java/org/apache/druid/query/Query.java index 387509839bb5..37868b4a25d4 100644 --- a/processing/src/main/java/org/apache/druid/query/Query.java +++ b/processing/src/main/java/org/apache/druid/query/Query.java @@ -116,6 +116,12 @@ public interface Query Query withOverriddenContext(Map contextOverride); + /** + * Returns a new query, identical to this one, but with a different associated {@link QuerySegmentSpec}. + * + * This often changes the behavior of {@link #getRunner(QuerySegmentWalker)}, since most queries inherit that method + * from {@link BaseQuery}, which implements it by calling {@link QuerySegmentSpec#lookup}. + */ Query withQuerySegmentSpec(QuerySegmentSpec spec); Query withId(String id); @@ -140,14 +146,4 @@ default Query optimizeForSegment(PerSegmentQueryOptimizationContext optimizat { return this; } - - default List getIntervalsOfInnerMostQuery() - { - if (getDataSource() instanceof QueryDataSource) { - //noinspection unchecked - return ((QueryDataSource) getDataSource()).getQuery().getIntervalsOfInnerMostQuery(); - } else { - return getIntervals(); - } - } } diff --git a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java index 5e5201711d84..94d47d511f48 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java @@ -22,8 +22,13 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import org.apache.druid.java.util.common.IAE; +import java.util.Collections; import java.util.List; +import java.util.Set; @JsonTypeName("query") public class QueryDataSource implements DataSource @@ -34,13 +39,13 @@ public class QueryDataSource implements DataSource @JsonCreator public QueryDataSource(@JsonProperty("query") Query query) { - this.query = query; + this.query = Preconditions.checkNotNull(query, "'query' must be nonnull"); } @Override - public List getNames() + public Set getTableNames() { - return query.getDataSource().getNames(); + return query.getDataSource().getTableNames(); } @JsonProperty @@ -49,6 +54,40 @@ public Query getQuery() return query; } + @Override + public List getChildren() + { + return Collections.singletonList(query.getDataSource()); + } + + @Override + public DataSource withChildren(List children) + { + if (children.size() != 1) { + throw new IAE("Must have exactly one child"); + } + + return new QueryDataSource(query.withDataSource(Iterables.getOnlyElement(children))); + } + + @Override + public boolean isCacheable() + { + return false; + } + + @Override + public boolean isGlobal() + { + return query.getDataSource().isGlobal(); + } + + @Override + public boolean isConcrete() + { + return false; + } + @Override public String toString() { diff --git a/processing/src/main/java/org/apache/druid/query/QueryPlus.java b/processing/src/main/java/org/apache/druid/query/QueryPlus.java index f1884d356242..1b18e9439099 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryPlus.java +++ b/processing/src/main/java/org/apache/druid/query/QueryPlus.java @@ -24,7 +24,6 @@ import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.context.ResponseContext; -import org.apache.druid.query.spec.QuerySegmentSpec; import javax.annotation.Nullable; @@ -125,14 +124,6 @@ private QueryPlus withoutQueryMetrics() } } - /** - * Equivalent of withQuery(getQuery().withQuerySegmentSpec(spec)). - */ - public QueryPlus withQuerySegmentSpec(QuerySegmentSpec spec) - { - return new QueryPlus<>(query.withQuerySegmentSpec(spec), queryMetrics, identity); - } - /** * Equivalent of withQuery(getQuery().withOverriddenContext(ImmutableMap.of(MAX_QUEUED_BYTES_KEY, maxQueuedBytes))). */ diff --git a/processing/src/main/java/org/apache/druid/query/QuerySegmentWalker.java b/processing/src/main/java/org/apache/druid/query/QuerySegmentWalker.java index 970fb6e28c08..7084a80935d1 100644 --- a/processing/src/main/java/org/apache/druid/query/QuerySegmentWalker.java +++ b/processing/src/main/java/org/apache/druid/query/QuerySegmentWalker.java @@ -22,6 +22,7 @@ import org.joda.time.Interval; /** + * An interface for query-handling entry points. */ public interface QuerySegmentWalker { @@ -29,19 +30,27 @@ public interface QuerySegmentWalker * Gets the Queryable for a given interval, the Queryable returned can be any version(s) or partitionNumber(s) * such that it represents the interval. * - * @param query result type - * @param query the query to find a Queryable for + * @param query result type + * @param query the query to find a Queryable for * @param intervals the intervals to find a Queryable for + * * @return a Queryable object that represents the interval */ QueryRunner getQueryRunnerForIntervals(Query query, Iterable intervals); /** - * Gets the Queryable for a given list of SegmentSpecs. + * Gets the Queryable for a given list of SegmentDescriptors. + * + * The descriptors are expected to apply to the base datasource involved in the query, i.e. the one returned by: * - * @param the query result type + *
+   *   DataSourceAnalysis.forDataSource(query.getDataSource()).getBaseDataSource()
+   * 
+ * + * @param the query result type * @param query the query to return a Queryable for * @param specs the list of SegmentSpecs to find a Queryable for + * * @return the Queryable object with the given SegmentSpecs */ QueryRunner getQueryRunnerForSegments(Query query, Iterable specs); diff --git a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java index c271af8e4d5c..b72d9d76eb4e 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java @@ -271,6 +271,20 @@ public List filterSegments(QueryType query, List subquery) + { + return false; + } + /** * Returns a list of field names in the order that {@link #resultsAsArrays} would return them. The returned list will * be the same length as each array returned by {@link #resultsAsArrays}. diff --git a/processing/src/main/java/org/apache/druid/query/RetryQueryRunner.java b/processing/src/main/java/org/apache/druid/query/RetryQueryRunner.java index 6b991b870575..fa337d047899 100644 --- a/processing/src/main/java/org/apache/druid/query/RetryQueryRunner.java +++ b/processing/src/main/java/org/apache/druid/query/RetryQueryRunner.java @@ -30,7 +30,6 @@ import org.apache.druid.java.util.common.guava.YieldingSequenceBase; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.context.ResponseContext; -import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; import org.apache.druid.segment.SegmentMissingException; import java.util.ArrayList; @@ -73,10 +72,8 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat log.info("[%,d] missing segments found. Retry attempt [%,d]", missingSegments.size(), i); context.put(ResponseContext.Key.MISSING_SEGMENTS, new ArrayList<>()); - final QueryPlus retryQueryPlus = queryPlus.withQuerySegmentSpec( - new MultipleSpecificSegmentSpec( - missingSegments - ) + final QueryPlus retryQueryPlus = queryPlus.withQuery( + Queries.withSpecificSegments(queryPlus.getQuery(), missingSegments) ); Sequence retrySequence = baseRunner.run(retryQueryPlus, context); listOfSequences.add(retrySequence); diff --git a/processing/src/main/java/org/apache/druid/query/TableDataSource.java b/processing/src/main/java/org/apache/druid/query/TableDataSource.java index f26aa53b93a5..4c371cf84510 100644 --- a/processing/src/main/java/org/apache/druid/query/TableDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/TableDataSource.java @@ -22,20 +22,22 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.IAE; import java.util.Collections; import java.util.List; +import java.util.Set; @JsonTypeName("table") public class TableDataSource implements DataSource { - @JsonProperty private final String name; @JsonCreator public TableDataSource(@JsonProperty("name") String name) { - this.name = (name == null ? null : name); + this.name = Preconditions.checkNotNull(name, "'name' must be nonnull"); } @JsonProperty @@ -45,9 +47,43 @@ public String getName() } @Override - public List getNames() + public Set getTableNames() { - return Collections.singletonList(name); + return Collections.singleton(name); + } + + @Override + public List getChildren() + { + return Collections.emptyList(); + } + + @Override + public DataSource withChildren(List children) + { + if (!children.isEmpty()) { + throw new IAE("Cannot accept children"); + } + + return this; + } + + @Override + public boolean isCacheable() + { + return true; + } + + @Override + public boolean isGlobal() + { + return false; + } + + @Override + public boolean isConcrete() + { + return true; } @Override @@ -57,7 +93,7 @@ public String toString() } @Override - public boolean equals(Object o) + public final boolean equals(Object o) { if (this == o) { return true; @@ -76,7 +112,7 @@ public boolean equals(Object o) } @Override - public int hashCode() + public final int hashCode() { return name.hashCode(); } diff --git a/processing/src/main/java/org/apache/druid/query/TimewarpOperator.java b/processing/src/main/java/org/apache/druid/query/TimewarpOperator.java index 0beca6849a4f..88c88b324890 100644 --- a/processing/src/main/java/org/apache/druid/query/TimewarpOperator.java +++ b/processing/src/main/java/org/apache/druid/query/TimewarpOperator.java @@ -92,8 +92,11 @@ public Sequence run(final QueryPlus queryPlus, final ResponseContext respo ); return Sequences.map( baseRunner.run( - queryPlus.withQuerySegmentSpec(new MultipleIntervalSegmentSpec( - Collections.singletonList(modifiedInterval))), + queryPlus.withQuery( + queryPlus.getQuery().withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Collections.singletonList(modifiedInterval)) + ) + ), responseContext ), new Function() diff --git a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java index 290ac538b0af..3bd25b017f1d 100644 --- a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java @@ -23,9 +23,12 @@ 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 com.google.common.collect.Iterables; +import org.apache.druid.java.util.common.IAE; import java.util.List; +import java.util.Set; import java.util.stream.Collectors; public class UnionDataSource implements DataSource @@ -41,9 +44,11 @@ public UnionDataSource(@JsonProperty("dataSources") List dataSo } @Override - public List getNames() + public Set getTableNames() { - return dataSources.stream().map(input -> Iterables.getOnlyElement(input.getNames())).collect(Collectors.toList()); + return dataSources.stream() + .map(input -> Iterables.getOnlyElement(input.getTableNames())) + .collect(Collectors.toSet()); } @JsonProperty @@ -52,6 +57,51 @@ public List getDataSources() return dataSources; } + @Override + public List getChildren() + { + return ImmutableList.copyOf(dataSources); + } + + @Override + public DataSource withChildren(List children) + { + if (children.size() != dataSources.size()) { + throw new IAE("Expected [%d] children, got [%d]", dataSources.size(), children.size()); + } + + if (!children.stream().allMatch(dataSource -> dataSource instanceof TableDataSource)) { + throw new IAE("All children must be tables"); + } + + return new UnionDataSource( + children.stream().map(dataSource -> (TableDataSource) dataSource).collect(Collectors.toList()) + ); + } + + @Override + public boolean isCacheable() + { + // Disables result-level caching for 'union' datasources, which doesn't work currently. + // See https://github.com/apache/druid/issues/8713 for reference. + // + // Note that per-segment caching is still effective, since at the time the per-segment cache evaluates a query + // for cacheability, it would have already been rewritten to a query on a single table. + return false; + } + + @Override + public boolean isGlobal() + { + return dataSources.stream().allMatch(DataSource::isGlobal); + } + + @Override + public boolean isConcrete() + { + return dataSources.stream().allMatch(DataSource::isConcrete); + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/query/filter/DimFilterUtils.java b/processing/src/main/java/org/apache/druid/query/filter/DimFilterUtils.java index 03966e944445..00a84fc92d6d 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/DimFilterUtils.java +++ b/processing/src/main/java/org/apache/druid/query/filter/DimFilterUtils.java @@ -20,7 +20,6 @@ package org.apache.druid.query.filter; import com.google.common.base.Function; -import com.google.common.base.Optional; import com.google.common.collect.RangeSet; import org.apache.druid.timeline.partition.ShardSpec; @@ -29,9 +28,11 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; /** + * */ public class DimFilterUtils { @@ -87,14 +88,15 @@ static byte[] computeCacheKey(byte cacheIdKey, List filters) * {@link #filterShards(DimFilter, Iterable, Function, Map)} instead with a cached map * * @param dimFilter The filter to use - * @param input The iterable of objects to be filtered + * @param input The iterable of objects to be filtered * @param converter The function to convert T to ShardSpec that can be filtered by - * @param This can be any type, as long as transform function is provided to convert this to ShardSpec + * @param This can be any type, as long as transform function is provided to convert this to ShardSpec + * * @return The set of filtered object, in the same order as input */ public static Set filterShards(DimFilter dimFilter, Iterable input, Function converter) { - return filterShards(dimFilter, input, converter, new HashMap>>()); + return filterShards(dimFilter, input, converter, new HashMap<>()); } /** @@ -106,15 +108,20 @@ public static Set filterShards(DimFilter dimFilter, Iterable input, Fu * between calls with the same dimFilter to save redundant calls of {@link DimFilter#getDimensionRangeSet(String)} * on same dimensions. * - * @param dimFilter The filter to use - * @param input The iterable of objects to be filtered - * @param converter The function to convert T to ShardSpec that can be filtered by + * @param dimFilter The filter to use + * @param input The iterable of objects to be filtered + * @param converter The function to convert T to ShardSpec that can be filtered by * @param dimensionRangeCache The cache of RangeSets of different dimensions for the dimFilter - * @param This can be any type, as long as transform function is provided to convert this to ShardSpec + * @param This can be any type, as long as transform function is provided to convert this to ShardSpec + * * @return The set of filtered object, in the same order as input */ - public static Set filterShards(DimFilter dimFilter, Iterable input, Function converter, - Map>> dimensionRangeCache) + public static Set filterShards( + final DimFilter dimFilter, + final Iterable input, + final Function converter, + final Map>> dimensionRangeCache + ) { Set retSet = new LinkedHashSet<>(); @@ -127,7 +134,7 @@ public static Set filterShards(DimFilter dimFilter, Iterable input, Fu List dimensions = shard.getDomainDimensions(); for (String dimension : dimensions) { Optional> optFilterRangeSet = dimensionRangeCache - .computeIfAbsent(dimension, d -> Optional.fromNullable(dimFilter.getDimensionRangeSet(d))); + .computeIfAbsent(dimension, d -> Optional.ofNullable(dimFilter.getDimensionRangeSet(d))); if (optFilterRangeSet.isPresent()) { filterDomain.put(dimension, optFilterRangeSet.get()); diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java index acfe77688e6a..ebae09f184f2 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -663,6 +663,26 @@ public ResultRow apply(Object input) }; } + @Override + public boolean canPerformSubquery(Query subquery) + { + Query current = subquery; + + while (current != null) { + if (!(current instanceof GroupByQuery)) { + return false; + } + + if (current.getDataSource() instanceof QueryDataSource) { + current = ((QueryDataSource) current.getDataSource()).getQuery(); + } else { + current = null; + } + } + + return true; + } + @Override public List resultArrayFields(final GroupByQuery query) { diff --git a/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java b/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java new file mode 100644 index 000000000000..4237e50dc473 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java @@ -0,0 +1,282 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.query.planning; + +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.query.BaseQuery; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.JoinDataSource; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryDataSource; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.UnionDataSource; +import org.apache.druid.query.spec.QuerySegmentSpec; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** + * Analysis of a datasource for purposes of deciding how to execute a particular query. + * + * The analysis breaks a datasource down in the following way: + * + *
+ *
+ *                             Q  <-- Possible outer query datasource(s) [may be multiple stacked]
+ *                             |
+ *                             J  <-- Possible join tree, expected to be left-leaning
+ *                            / \
+ *                           J  Dj <--  Other leaf datasources
+ *   Base datasource        / \         which will be joined
+ *  (bottom-leftmost) -->  Db Dj  <---- into the base datasource
+ *
+ * 
+ * + * The base datasource (Db) is returned by {@link #getBaseDataSource()}. The other leaf datasources are returned by + * {@link #getPreJoinableClauses()}. The outer query datasources are available as part of {@link #getDataSource()}, + * which just returns the original datasource that was provided for analysis. + * + * The base datasource (Db) will never be a join, but it can be any other type of datasource (table, query, etc). + * Note that join trees are only flattened if they occur at the top of the overall tree (or underneath an outer query), + * and that join trees are only flattened to the degree that they are left-leaning. Due to these facts, it is possible + * for the base or leaf datasources to include additional joins. + * + * The base datasource is the one that will be considered by the core Druid query stack for scanning via + * {@link org.apache.druid.segment.Segment} and {@link org.apache.druid.segment.StorageAdapter}. The other leaf + * datasources must be joinable onto the base data. + * + * The idea here is to keep things simple and dumb. So we focus only on identifying left-leaning join trees, which map + * neatly onto a series of hash table lookups at query time. The user/system generating the queries, e.g. the druid-sql + * layer (or the end user in the case of native queries), is responsible for containing the smarts to structure the + * tree in a way that will lead to optimal execution. + */ +public class DataSourceAnalysis +{ + private final DataSource dataSource; + private final DataSource baseDataSource; + @Nullable + private final QuerySegmentSpec baseQuerySegmentSpec; + private final List preJoinableClauses; + + private DataSourceAnalysis( + DataSource dataSource, + DataSource baseDataSource, + @Nullable QuerySegmentSpec baseQuerySegmentSpec, + List preJoinableClauses + ) + { + if (baseDataSource instanceof JoinDataSource) { + // The base cannot be a join (this is a class invariant). + // If it happens, it's a bug in the datasource analyzer. + throw new IAE("Base dataSource cannot be a join! Original dataSource was: %s", dataSource); + } + + this.dataSource = dataSource; + this.baseDataSource = baseDataSource; + this.baseQuerySegmentSpec = baseQuerySegmentSpec; + this.preJoinableClauses = preJoinableClauses; + } + + public static DataSourceAnalysis forDataSource(final DataSource dataSource) + { + // Strip outer queries, retaining querySegmentSpecs as we go down (lowest will become the 'baseQuerySegmentSpec'). + QuerySegmentSpec baseQuerySegmentSpec = null; + DataSource current = dataSource; + + while (current instanceof QueryDataSource) { + final Query subQuery = ((QueryDataSource) current).getQuery(); + + if (!(subQuery instanceof BaseQuery)) { + // All builtin query types are BaseQuery, so we only expect this with funky extension queries. + throw new IAE("Cannot analyze subquery of class[%s]", subQuery.getClass().getName()); + } + + baseQuerySegmentSpec = ((BaseQuery) subQuery).getQuerySegmentSpec(); + current = subQuery.getDataSource(); + } + + if (current instanceof JoinDataSource) { + final Pair> flattened = flattenJoin((JoinDataSource) current); + return new DataSourceAnalysis(dataSource, flattened.lhs, baseQuerySegmentSpec, flattened.rhs); + } else { + return new DataSourceAnalysis(dataSource, current, baseQuerySegmentSpec, Collections.emptyList()); + } + } + + /** + * Flatten a datasource into two parts: the left-hand side datasource (the 'base' datasource), and a list of join + * clauses, if any. + * + * @throws IllegalArgumentException if dataSource cannot be fully flattened. + */ + private static Pair> flattenJoin(final JoinDataSource dataSource) + { + DataSource current = dataSource; + final List preJoinableClauses = new ArrayList<>(); + + while (current instanceof JoinDataSource) { + final JoinDataSource joinDataSource = (JoinDataSource) current; + current = joinDataSource.getLeft(); + preJoinableClauses.add( + new PreJoinableClause( + joinDataSource.getRightPrefix(), + joinDataSource.getRight(), + joinDataSource.getJoinType(), + joinDataSource.getConditionAnalysis() + ) + ); + } + + // Join clauses were added in the order we saw them while traversing down, but we need to apply them in the + // going-up order. So reverse them. + Collections.reverse(preJoinableClauses); + + return Pair.of(current, preJoinableClauses); + } + + /** + * Returns the topmost datasource: the original one passed to {@link #forDataSource(DataSource)}. + */ + public DataSource getDataSource() + { + return dataSource; + } + + /** + * Returns the base (bottom-leftmost) datasource. + */ + public DataSource getBaseDataSource() + { + return baseDataSource; + } + + /** + * Returns the same datasource as {@link #getBaseDataSource()}, but only if it is a table. Useful on data servers, + * since they generally can only handle queries where the base datasource is a table. + */ + public Optional getBaseTableDataSource() + { + if (baseDataSource instanceof TableDataSource) { + return Optional.of((TableDataSource) baseDataSource); + } else { + return Optional.empty(); + } + } + + /** + * Returns the {@link QuerySegmentSpec} that is associated with the base datasource, if any. This only happens + * when there is an outer query datasource. In this case, the base querySegmentSpec is the one associated with the + * innermost subquery. + */ + public Optional getBaseQuerySegmentSpec() + { + return Optional.ofNullable(baseQuerySegmentSpec); + } + + /** + * Returns join clauses corresponding to joinable leaf datasources (every leaf except the bottom-leftmost). + */ + public List getPreJoinableClauses() + { + return preJoinableClauses; + } + + /** + * Returns true if all servers have the ability to compute this datasource. These datasources depend only on + * globally broadcast data, like lookups or inline data. + */ + public boolean isGlobal() + { + return dataSource.isGlobal(); + } + + /** + * Returns true if this datasource can be computed by the core Druid query stack via a scan of a concrete base + * datasource. All other datasources involved, if any, must be global. + */ + public boolean isConcreteBased() + { + return baseDataSource.isConcrete() && preJoinableClauses.stream() + .allMatch(clause -> clause.getDataSource().isGlobal()); + } + + /** + * Returns true if this datasource is concrete-based (see {@link #isConcreteBased()}, and the base datasource is a + * 'table' or union of them. This is an important property because it corresponds to datasources that can be handled + * by Druid data servers, like Historicals. + */ + public boolean isConcreteTableBased() + { + // At the time of writing this comment, UnionDataSource children are required to be tables, so the instanceof + // check is redundant. But in the future, we will likely want to support unions of things other than tables, + // so check anyway for future-proofing. + return isConcreteBased() && (baseDataSource instanceof TableDataSource + || (baseDataSource instanceof UnionDataSource && + baseDataSource.getChildren() + .stream() + .allMatch(ds -> ds instanceof TableDataSource))); + } + + /** + * Returns true if this datasource represents a subquery. + */ + public boolean isQuery() + { + return dataSource instanceof QueryDataSource; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DataSourceAnalysis that = (DataSourceAnalysis) o; + return Objects.equals(dataSource, that.dataSource) && + Objects.equals(baseDataSource, that.baseDataSource) && + Objects.equals(baseQuerySegmentSpec, that.baseQuerySegmentSpec) && + Objects.equals(preJoinableClauses, that.preJoinableClauses); + } + + @Override + public int hashCode() + { + return Objects.hash(dataSource, baseDataSource, baseQuerySegmentSpec, preJoinableClauses); + } + + @Override + public String toString() + { + return "DataSourceAnalysis{" + + "dataSource=" + dataSource + + ", baseDataSource=" + baseDataSource + + ", baseQuerySegmentSpec=" + baseQuerySegmentSpec + + ", joinClauses=" + preJoinableClauses + + '}'; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/planning/PreJoinableClause.java b/processing/src/main/java/org/apache/druid/query/planning/PreJoinableClause.java new file mode 100644 index 000000000000..a7bcab301e84 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/planning/PreJoinableClause.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.query.planning; + +import com.google.common.base.Preconditions; +import org.apache.druid.query.DataSource; +import org.apache.druid.segment.join.JoinConditionAnalysis; +import org.apache.druid.segment.join.JoinType; +import org.apache.druid.segment.join.Joinables; + +import java.util.Objects; + +/** + * Like {@link org.apache.druid.segment.join.JoinableClause}, but contains a {@link DataSource} instead of a + * {@link org.apache.druid.segment.join.Joinable}. This is useful because when analyzing joins, we don't want to + * actually create Joinables, since that can be an expensive operation. + * + * Converted into {@link org.apache.druid.segment.join.JoinableClause} by {@link Joinables#createSegmentMapFn}. + */ +public class PreJoinableClause +{ + private final String prefix; + private final DataSource dataSource; + private final JoinType joinType; + private final JoinConditionAnalysis condition; + + PreJoinableClause( + final String prefix, + final DataSource dataSource, + final JoinType joinType, + final JoinConditionAnalysis condition + ) + { + this.prefix = Joinables.validatePrefix(prefix); + this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); + this.joinType = Preconditions.checkNotNull(joinType, "joinType"); + this.condition = Preconditions.checkNotNull(condition, "condition"); + } + + public String getPrefix() + { + return prefix; + } + + public DataSource getDataSource() + { + return dataSource; + } + + public JoinType getJoinType() + { + return joinType; + } + + public JoinConditionAnalysis getCondition() + { + return condition; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PreJoinableClause that = (PreJoinableClause) o; + return Objects.equals(prefix, that.prefix) && + Objects.equals(dataSource, that.dataSource) && + joinType == that.joinType && + Objects.equals(condition, that.condition); + } + + @Override + public int hashCode() + { + return Objects.hash(prefix, dataSource, joinType, condition); + } + + @Override + public String toString() + { + return "JoinClause{" + + "prefix='" + prefix + '\'' + + ", dataSource=" + dataSource + + ", joinType=" + joinType + + ", condition=" + condition + + '}'; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/spec/MultipleSpecificSegmentSpec.java b/processing/src/main/java/org/apache/druid/query/spec/MultipleSpecificSegmentSpec.java index 34a458d2aaf3..5d0c853823ad 100644 --- a/processing/src/main/java/org/apache/druid/query/spec/MultipleSpecificSegmentSpec.java +++ b/processing/src/main/java/org/apache/druid/query/spec/MultipleSpecificSegmentSpec.java @@ -30,6 +30,7 @@ import org.joda.time.Interval; import java.util.List; +import java.util.Objects; /** */ @@ -93,24 +94,13 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - MultipleSpecificSegmentSpec that = (MultipleSpecificSegmentSpec) o; - - if (descriptors != null ? !descriptors.equals(that.descriptors) : that.descriptors != null) { - return false; - } - if (intervals != null ? !intervals.equals(that.intervals) : that.intervals != null) { - return false; - } - - return true; + return Objects.equals(descriptors, that.descriptors); } @Override public int hashCode() { - int result = descriptors != null ? descriptors.hashCode() : 0; - result = 31 * result + (intervals != null ? intervals.hashCode() : 0); - return result; + return Objects.hash(descriptors); } } diff --git a/processing/src/main/java/org/apache/druid/query/spec/SpecificSegmentQueryRunner.java b/processing/src/main/java/org/apache/druid/query/spec/SpecificSegmentQueryRunner.java index 625f0325229e..0cea0dbf325a 100644 --- a/processing/src/main/java/org/apache/druid/query/spec/SpecificSegmentQueryRunner.java +++ b/processing/src/main/java/org/apache/druid/query/spec/SpecificSegmentQueryRunner.java @@ -28,6 +28,7 @@ import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.guava.YieldingAccumulator; +import org.apache.druid.query.Queries; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; @@ -38,6 +39,7 @@ import java.util.Collections; /** + * */ public class SpecificSegmentQueryRunner implements QueryRunner { @@ -56,7 +58,13 @@ public SpecificSegmentQueryRunner( @Override public Sequence run(final QueryPlus input, final ResponseContext responseContext) { - final QueryPlus queryPlus = input.withQuerySegmentSpec(specificSpec); + final QueryPlus queryPlus = input.withQuery( + Queries.withSpecificSegments( + input.getQuery(), + Collections.singletonList(specificSpec.getDescriptor()) + ) + ); + final Query query = queryPlus.getQuery(); final Thread currThread = Thread.currentThread(); diff --git a/processing/src/main/java/org/apache/druid/segment/join/JoinableClause.java b/processing/src/main/java/org/apache/druid/segment/join/JoinableClause.java index 4f985dbf3f50..5e9bfc39bdd6 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/JoinableClause.java +++ b/processing/src/main/java/org/apache/druid/segment/join/JoinableClause.java @@ -22,7 +22,6 @@ import com.google.common.base.Preconditions; import org.apache.druid.java.util.common.IAE; -import javax.annotation.Nullable; import java.util.List; import java.util.Objects; import java.util.stream.Collectors; @@ -39,9 +38,9 @@ public class JoinableClause private final JoinType joinType; private final JoinConditionAnalysis condition; - public JoinableClause(@Nullable String prefix, Joinable joinable, JoinType joinType, JoinConditionAnalysis condition) + public JoinableClause(String prefix, Joinable joinable, JoinType joinType, JoinConditionAnalysis condition) { - this.prefix = prefix != null ? prefix : ""; + this.prefix = Joinables.validatePrefix(prefix); this.joinable = Preconditions.checkNotNull(joinable, "joinable"); this.joinType = Preconditions.checkNotNull(joinType, "joinType"); this.condition = Preconditions.checkNotNull(condition, "condition"); diff --git a/processing/src/main/java/org/apache/druid/segment/join/Joinables.java b/processing/src/main/java/org/apache/druid/segment/join/Joinables.java new file mode 100644 index 000000000000..a14c3dcefd93 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/Joinables.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.segment.join; + +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.segment.column.ColumnHolder; + +import javax.annotation.Nullable; + +/** + * Utility methods for working with {@link Joinable} related classes. + */ +public class Joinables +{ + /** + * Checks that "prefix" is a valid prefix for a join clause (see {@link JoinableClause#getPrefix()}) and, if so, + * returns it. Otherwise, throws an exception. + */ + public static String validatePrefix(@Nullable final String prefix) + { + if (prefix == null) { + throw new IAE("Join clause cannot have null prefix"); + } else if (isPrefixedBy(ColumnHolder.TIME_COLUMN_NAME, prefix) || ColumnHolder.TIME_COLUMN_NAME.equals(prefix)) { + throw new IAE( + "Join clause cannot have prefix[%s], since it would shadow %s", + prefix, + ColumnHolder.TIME_COLUMN_NAME + ); + } else { + return prefix; + } + } + + public static boolean isPrefixedBy(final String columnName, final String prefix) + { + return columnName.startsWith(prefix) && columnName.length() > prefix.length(); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/DataSourceTest.java b/processing/src/test/java/org/apache/druid/query/DataSourceTest.java index 537650881b1c..090570db7acb 100644 --- a/processing/src/test/java/org/apache/druid/query/DataSourceTest.java +++ b/processing/src/test/java/org/apache/druid/query/DataSourceTest.java @@ -20,6 +20,7 @@ package org.apache.druid.query; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; @@ -53,7 +54,10 @@ public void testLegacyDataSource() throws IOException @Test public void testTableDataSource() throws IOException { - DataSource dataSource = JSON_MAPPER.readValue("{\"type\":\"table\", \"name\":\"somedatasource\"}", DataSource.class); + DataSource dataSource = JSON_MAPPER.readValue( + "{\"type\":\"table\", \"name\":\"somedatasource\"}", + DataSource.class + ); Assert.assertEquals(new TableDataSource("somedatasource"), dataSource); } @@ -88,8 +92,8 @@ public void testUnionDataSource() throws Exception Lists.newArrayList(((UnionDataSource) dataSource).getDataSources()) ); Assert.assertEquals( - Lists.newArrayList("ds1", "ds2"), - Lists.newArrayList(dataSource.getNames()) + ImmutableSet.of("ds1", "ds2"), + dataSource.getTableNames() ); final DataSource serde = JSON_MAPPER.readValue(JSON_MAPPER.writeValueAsString(dataSource), DataSource.class); diff --git a/processing/src/test/java/org/apache/druid/query/InlineDataSourceTest.java b/processing/src/test/java/org/apache/druid/query/InlineDataSourceTest.java new file mode 100644 index 000000000000..26e816779270 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/InlineDataSourceTest.java @@ -0,0 +1,287 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.query; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.segment.RowAdapter; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ValueType; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; + +public class InlineDataSourceTest +{ + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + private final AtomicLong iterationCounter = new AtomicLong(); + + private final List rows = ImmutableList.of( + new Object[]{DateTimes.of("2000").getMillis(), "foo", 0d, ImmutableMap.of("n", "0")}, + new Object[]{DateTimes.of("2000").getMillis(), "bar", 1d, ImmutableMap.of("n", "1")}, + new Object[]{DateTimes.of("2000").getMillis(), "baz", 2d, ImmutableMap.of("n", "2")} + ); + + private final Iterable rowsIterable = () -> { + iterationCounter.incrementAndGet(); + return rows.iterator(); + }; + + private final List expectedColumnNames = ImmutableList.of( + ColumnHolder.TIME_COLUMN_NAME, + "str", + "double", + "complex" + ); + + private final List expectedColumnTypes = ImmutableList.of( + ValueType.LONG, + ValueType.STRING, + ValueType.DOUBLE, + ValueType.COMPLEX + ); + + private final InlineDataSource listDataSource = InlineDataSource.fromIterable( + expectedColumnNames, + expectedColumnTypes, + rows + ); + + private final InlineDataSource iterableDataSource = InlineDataSource.fromIterable( + expectedColumnNames, + expectedColumnTypes, + rowsIterable + ); + + @Test + public void test_getTableNames() + { + Assert.assertEquals(Collections.emptySet(), listDataSource.getTableNames()); + Assert.assertEquals(Collections.emptySet(), iterableDataSource.getTableNames()); + } + + @Test + public void test_getColumnNames() + { + Assert.assertEquals(expectedColumnNames, listDataSource.getColumnNames()); + Assert.assertEquals(expectedColumnNames, iterableDataSource.getColumnNames()); + } + + @Test + public void test_getColumnTypes() + { + Assert.assertEquals(expectedColumnTypes, listDataSource.getColumnTypes()); + Assert.assertEquals(expectedColumnTypes, iterableDataSource.getColumnTypes()); + } + + @Test + public void test_getChildren() + { + Assert.assertEquals(Collections.emptyList(), listDataSource.getChildren()); + Assert.assertEquals(Collections.emptyList(), iterableDataSource.getChildren()); + } + + @Test + public void test_getRowSignature() + { + Assert.assertEquals( + ImmutableMap.of( + ColumnHolder.TIME_COLUMN_NAME, ValueType.LONG, + "str", ValueType.STRING, + "double", ValueType.DOUBLE, + "complex", ValueType.COMPLEX + ), + listDataSource.getRowSignature() + ); + } + + @Test + public void test_isCacheable() + { + Assert.assertFalse(listDataSource.isCacheable()); + } + + @Test + public void test_isGlobal() + { + Assert.assertTrue(listDataSource.isGlobal()); + } + + @Test + public void test_isConcrete() + { + Assert.assertFalse(listDataSource.isConcrete()); + } + + @Test + public void test_rowAdapter() + { + final RowAdapter adapter = listDataSource.rowAdapter(); + final Object[] row = rows.get(1); + + Assert.assertEquals(DateTimes.of("2000").getMillis(), adapter.timestampFunction().applyAsLong(row)); + Assert.assertEquals("bar", adapter.columnFunction("str").apply(row)); + Assert.assertEquals(1d, adapter.columnFunction("double").apply(row)); + Assert.assertEquals(ImmutableMap.of("n", "1"), adapter.columnFunction("complex").apply(row)); + } + + @Test + public void test_getRows_list() + { + Assert.assertSame(this.rows, listDataSource.getRowsAsList()); + } + + @Test + public void test_getRows_iterable() + { + final Iterable iterable = iterableDataSource.getRows(); + Assert.assertNotSame(this.rows, iterable); + + // No iteration yet. + Assert.assertEquals(0, iterationCounter.get()); + + assertRowsEqual(this.rows, ImmutableList.copyOf(iterable)); + + // OK, now we've iterated. + Assert.assertEquals(1, iterationCounter.get()); + + // Read again, we should iterate again. + //noinspection MismatchedQueryAndUpdateOfCollection + final List ignored = Lists.newArrayList(iterable); + Assert.assertEquals(2, iterationCounter.get()); + } + + @Test + public void test_getRowsAsList_list() + { + Assert.assertSame(this.rows, listDataSource.getRowsAsList()); + } + + @Test + public void test_getRowsAsList_iterable() + { + final List list = iterableDataSource.getRowsAsList(); + + Assert.assertEquals(1, iterationCounter.get()); + assertRowsEqual(this.rows, list); + + // Read again, we should *not* iterate again (in contrast to "test_getRows_iterable"). + //noinspection MismatchedQueryAndUpdateOfCollection + final List ignored = Lists.newArrayList(list); + Assert.assertEquals(1, iterationCounter.get()); + } + + @Test + public void test_withChildren_empty() + { + Assert.assertSame(listDataSource, listDataSource.withChildren(Collections.emptyList())); + } + + @Test + public void test_withChildren_nonEmpty() + { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Cannot accept children"); + + // Workaround so "withChildren" isn't flagged as unused in the DataSource interface. + ((DataSource) listDataSource).withChildren(ImmutableList.of(new TableDataSource("foo"))); + } + + @Test + public void test_equals() + { + EqualsVerifier.forClass(InlineDataSource.class) + .usingGetClass() + .withNonnullFields("columnNames", "columnTypes", "rows") + .verify(); + } + + @Test + public void test_serde_list() throws Exception + { + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + final InlineDataSource deserialized = (InlineDataSource) jsonMapper.readValue( + jsonMapper.writeValueAsString(listDataSource), + DataSource.class + ); + + Assert.assertEquals(listDataSource.getColumnNames(), deserialized.getColumnNames()); + Assert.assertEquals(listDataSource.getColumnTypes(), deserialized.getColumnTypes()); + assertRowsEqual(listDataSource.getRows(), deserialized.getRows()); + } + + @Test + public void test_serde_iterable() throws Exception + { + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + final InlineDataSource deserialized = (InlineDataSource) jsonMapper.readValue( + jsonMapper.writeValueAsString(iterableDataSource), + DataSource.class + ); + + // Lazy iterables turn into Lists upon serialization. + Assert.assertEquals(listDataSource.getColumnNames(), deserialized.getColumnNames()); + Assert.assertEquals(listDataSource.getColumnTypes(), deserialized.getColumnTypes()); + assertRowsEqual(listDataSource.getRows(), deserialized.getRows()); + + // Should have iterated once. + Assert.assertEquals(1, iterationCounter.get()); + } + + /** + * This method exists because "equals" on two equivalent Object[] won't return true, so we need to check + * for equality deeply. + */ + private static void assertRowsEqual(final Iterable expectedRows, final Iterable actualRows) + { + if (expectedRows instanceof List && actualRows instanceof List) { + // Only check equality deeply when both rows1 and rows2 are Lists, i.e., non-lazy. + final List expectedRowsList = (List) expectedRows; + final List actualRowsList = (List) actualRows; + + final int sz = expectedRowsList.size(); + Assert.assertEquals("number of rows", sz, actualRowsList.size()); + + // Super slow for LinkedLists, but we don't expect those to be used here. + // (They're generally forbidden in Druid except for special cases.) + for (int i = 0; i < sz; i++) { + Assert.assertArrayEquals("row #" + i, expectedRowsList.get(i), actualRowsList.get(i)); + } + } else { + // If they're not both Lists, we don't want to iterate them during equality checks, so do a non-deep check. + // This might still return true if whatever class they are has another way of checking equality. But, usually we + // expect this to return false. + Assert.assertEquals("rows", expectedRows, actualRows); + } + } + +} diff --git a/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java b/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java new file mode 100644 index 000000000000..e0a990e753ae --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.query; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.join.JoinType; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.Collections; + +public class JoinDataSourceTest +{ + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + private final TableDataSource fooTable = new TableDataSource("foo"); + private final TableDataSource barTable = new TableDataSource("bar"); + private final LookupDataSource lookylooLookup = new LookupDataSource("lookyloo"); + + private final JoinDataSource joinTableToLookup = JoinDataSource.create( + fooTable, + lookylooLookup, + "j.", + "x == \"j.x\"", + JoinType.LEFT, + ExprMacroTable.nil() + ); + + private final JoinDataSource joinTableToTable = JoinDataSource.create( + fooTable, + barTable, + "j.", + "x == \"j.x\"", + JoinType.LEFT, + ExprMacroTable.nil() + ); + + @Test + public void test_getTableNames_tableToTable() + { + Assert.assertEquals(ImmutableSet.of("foo", "bar"), joinTableToTable.getTableNames()); + } + + @Test + public void test_getTableNames_tableToLookup() + { + Assert.assertEquals(Collections.singleton("foo"), joinTableToLookup.getTableNames()); + } + + @Test + public void test_getChildren_tableToTable() + { + Assert.assertEquals(ImmutableList.of(fooTable, barTable), joinTableToTable.getChildren()); + } + + @Test + public void test_getChildren_tableToLookup() + { + Assert.assertEquals(ImmutableList.of(fooTable, lookylooLookup), joinTableToLookup.getChildren()); + } + + @Test + public void test_isCacheable_tableToTable() + { + Assert.assertFalse(joinTableToTable.isCacheable()); + } + + @Test + public void test_isCacheable_lookup() + { + Assert.assertFalse(joinTableToLookup.isCacheable()); + } + + @Test + public void test_isConcrete_tableToTable() + { + Assert.assertFalse(joinTableToTable.isConcrete()); + } + + @Test + public void test_isConcrete_tableToLookup() + { + Assert.assertFalse(joinTableToLookup.isConcrete()); + } + + @Test + public void test_isGlobal_tableToTable() + { + Assert.assertFalse(joinTableToTable.isGlobal()); + } + + @Test + public void test_isGlobal_tableToLookup() + { + Assert.assertFalse(joinTableToLookup.isGlobal()); + } + + @Test + public void test_withChildren_empty() + { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Expected [2] children, got [0]"); + + final DataSource ignored = joinTableToTable.withChildren(Collections.emptyList()); + } + + @Test + public void test_withChildren_two() + { + final DataSource transformed = joinTableToTable.withChildren(ImmutableList.of(fooTable, lookylooLookup)); + + Assert.assertEquals(joinTableToLookup, transformed); + } + + @Test + public void test_equals() + { + EqualsVerifier.forClass(JoinDataSource.class) + .usingGetClass() + .withNonnullFields("left", "right", "rightPrefix", "conditionAnalysis", "joinType") + .verify(); + } + + @Test + public void test_serde() throws Exception + { + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + final JoinDataSource deserialized = (JoinDataSource) jsonMapper.readValue( + jsonMapper.writeValueAsString(joinTableToLookup), + DataSource.class + ); + + Assert.assertEquals(joinTableToLookup, deserialized); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/LookupDataSourceTest.java b/processing/src/test/java/org/apache/druid/query/LookupDataSourceTest.java new file mode 100644 index 000000000000..c68579ff60a0 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/LookupDataSourceTest.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.query; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.Collections; + +public class LookupDataSourceTest +{ + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + private final LookupDataSource lookylooDataSource = new LookupDataSource("lookyloo"); + + @Test + public void test_getTableNames() + { + Assert.assertEquals(Collections.emptySet(), lookylooDataSource.getTableNames()); + } + + @Test + public void test_getChildren() + { + Assert.assertEquals(Collections.emptyList(), lookylooDataSource.getChildren()); + } + + @Test + public void test_isCacheable() + { + Assert.assertFalse(lookylooDataSource.isCacheable()); + } + + @Test + public void test_isGlobal() + { + Assert.assertTrue(lookylooDataSource.isGlobal()); + } + + @Test + public void test_isConcrete() + { + Assert.assertFalse(lookylooDataSource.isConcrete()); + } + + @Test + public void test_withChildren_empty() + { + Assert.assertSame(lookylooDataSource, lookylooDataSource.withChildren(Collections.emptyList())); + } + + @Test + public void test_withChildren_nonEmpty() + { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Cannot accept children"); + + lookylooDataSource.withChildren(ImmutableList.of(new LookupDataSource("bar"))); + } + + @Test + public void test_equals() + { + EqualsVerifier.forClass(LookupDataSource.class).usingGetClass().withNonnullFields("lookupName").verify(); + } + + @Test + public void test_serde() throws Exception + { + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + final LookupDataSource deserialized = (LookupDataSource) jsonMapper.readValue( + jsonMapper.writeValueAsString(lookylooDataSource), + DataSource.class + ); + + Assert.assertEquals(lookylooDataSource, deserialized); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/QueriesTest.java b/processing/src/test/java/org/apache/druid/query/QueriesTest.java index fd5abf238297..16c4783619f8 100644 --- a/processing/src/test/java/org/apache/druid/query/QueriesTest.java +++ b/processing/src/test/java/org/apache/druid/query/QueriesTest.java @@ -20,6 +20,8 @@ package org.apache.druid.query; import com.google.common.collect.ImmutableList; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -27,17 +29,26 @@ import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; import org.apache.druid.query.aggregation.post.ConstantPostAggregator; import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; +import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.junit.Assert; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import java.util.Arrays; import java.util.Collections; import java.util.List; /** + * */ public class QueriesTest { + @Rule + public ExpectedException expectedException = ExpectedException.none(); + @Test public void testVerifyAggregations() { @@ -209,4 +220,114 @@ public void testVerifyAggregationsMultiLevelMissingVal() Assert.assertTrue(exceptionOccured); } + + @Test + public void testWithSpecificSegmentsBasic() + { + final ImmutableList descriptors = ImmutableList.of( + new SegmentDescriptor(Intervals.of("2000/3000"), "0", 0), + new SegmentDescriptor(Intervals.of("2000/3000"), "0", 1) + ); + + Assert.assertEquals( + Druids.newTimeseriesQueryBuilder() + .dataSource("foo") + .intervals( + new MultipleSpecificSegmentSpec( + ImmutableList.of( + new SegmentDescriptor(Intervals.of("2000/3000"), "0", 0), + new SegmentDescriptor(Intervals.of("2000/3000"), "0", 1) + ) + ) + ) + .granularity(Granularities.ALL) + .build(), + Queries.withSpecificSegments( + Druids.newTimeseriesQueryBuilder() + .dataSource("foo") + .intervals("2000/3000") + .granularity(Granularities.ALL) + .build(), + descriptors + ) + ); + } + + @Test + public void testWithSpecificSegmentsSubQueryStack() + { + final ImmutableList descriptors = ImmutableList.of( + new SegmentDescriptor(Intervals.of("2000/3000"), "0", 0), + new SegmentDescriptor(Intervals.of("2000/3000"), "0", 1) + ); + + Assert.assertEquals( + Druids.newTimeseriesQueryBuilder() + .dataSource( + new QueryDataSource( + Druids.newTimeseriesQueryBuilder() + .dataSource( + new QueryDataSource( + Druids.newTimeseriesQueryBuilder() + .dataSource("foo") + .intervals(new MultipleSpecificSegmentSpec(descriptors)) + .granularity(Granularities.ALL) + .build() + ) + ) + .intervals("2000/3000") + .granularity(Granularities.ALL) + .build() + ) + ) + .intervals("2000/3000") + .granularity(Granularities.ALL) + .build(), + Queries.withSpecificSegments( + Druids.newTimeseriesQueryBuilder() + .dataSource( + new QueryDataSource( + Druids.newTimeseriesQueryBuilder() + .dataSource( + new QueryDataSource( + Druids.newTimeseriesQueryBuilder() + .dataSource("foo") + .intervals("2000/3000") + .granularity(Granularities.ALL) + .build() + ) + ) + .intervals("2000/3000") + .granularity(Granularities.ALL) + .build() + ) + ) + .intervals("2000/3000") + .granularity(Granularities.ALL) + .build(), + descriptors + ) + ); + } + + @Test + public void testWithSpecificSegmentsOnUnionIsAnError() + { + final ImmutableList descriptors = ImmutableList.of( + new SegmentDescriptor(Intervals.of("2000/3000"), "0", 0), + new SegmentDescriptor(Intervals.of("2000/3000"), "0", 1) + ); + + final TimeseriesQuery query = + Druids.newTimeseriesQueryBuilder() + .dataSource(new LookupDataSource("lookyloo")) + .intervals("2000/3000") + .granularity(Granularities.ALL) + .build(); + + expectedException.expect(IllegalStateException.class); + expectedException.expectMessage("Unable to apply specific segments to non-table-based dataSource"); + + final Query> ignored = Queries.withSpecificSegments(query, descriptors); + } } diff --git a/processing/src/test/java/org/apache/druid/query/QueryDataSourceTest.java b/processing/src/test/java/org/apache/druid/query/QueryDataSourceTest.java new file mode 100644 index 000000000000..df8c1f6b5647 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/QueryDataSourceTest.java @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.query; + +import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.Collections; + +public class QueryDataSourceTest +{ + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + private final TimeseriesQuery queryOnTable = + Druids.newTimeseriesQueryBuilder() + .dataSource("foo") + .intervals("2000/3000") + .granularity(Granularities.ALL) + .build(); + + private final TimeseriesQuery queryOnLookup = + Druids.newTimeseriesQueryBuilder() + .dataSource(new LookupDataSource("lookyloo")) + .intervals("2000/3000") + .granularity(Granularities.ALL) + .build(); + + private final QueryDataSource queryOnTableDataSource = new QueryDataSource(queryOnTable); + private final QueryDataSource queryOnLookupDataSource = new QueryDataSource(queryOnLookup); + + @Test + public void test_getTableNames_table() + { + Assert.assertEquals(Collections.singleton("foo"), queryOnTableDataSource.getTableNames()); + } + + @Test + public void test_getTableNames_lookup() + { + Assert.assertEquals(Collections.emptySet(), queryOnLookupDataSource.getTableNames()); + } + + @Test + public void test_getChildren_table() + { + Assert.assertEquals(Collections.singletonList(new TableDataSource("foo")), queryOnTableDataSource.getChildren()); + } + + @Test + public void test_getChildren_lookup() + { + Assert.assertEquals( + Collections.singletonList(new LookupDataSource("lookyloo")), + queryOnLookupDataSource.getChildren() + ); + } + + @Test + public void test_isCacheable_table() + { + Assert.assertFalse(queryOnTableDataSource.isCacheable()); + } + + @Test + public void test_isCacheable_lookup() + { + Assert.assertFalse(queryOnLookupDataSource.isCacheable()); + } + + @Test + public void test_isConcrete_table() + { + Assert.assertFalse(queryOnTableDataSource.isConcrete()); + } + + @Test + public void test_isConcrete_lookup() + { + Assert.assertFalse(queryOnLookupDataSource.isConcrete()); + } + + @Test + public void test_isGlobal_table() + { + Assert.assertFalse(queryOnTableDataSource.isGlobal()); + } + + @Test + public void test_isGlobal_lookup() + { + Assert.assertTrue(queryOnLookupDataSource.isGlobal()); + } + + @Test + public void test_withChildren_empty() + { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Must have exactly one child"); + + final DataSource ignored = queryOnLookupDataSource.withChildren(Collections.emptyList()); + } + + @Test + public void test_withChildren_single() + { + final TableDataSource barTable = new TableDataSource("bar"); + + final QueryDataSource transformed = + (QueryDataSource) queryOnLookupDataSource.withChildren(Collections.singletonList(barTable)); + + Assert.assertEquals(barTable, transformed.getQuery().getDataSource()); + } + + @Test + public void test_equals() + { + EqualsVerifier.forClass(QueryDataSource.class).usingGetClass().withNonnullFields("query").verify(); + } + + @Test + public void test_serde() throws Exception + { + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + final QueryDataSource deserialized = (QueryDataSource) jsonMapper.readValue( + jsonMapper.writeValueAsString(queryOnTableDataSource), + DataSource.class + ); + + Assert.assertEquals(queryOnTableDataSource, deserialized); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java index 04f86d8ad249..474f7963c58d 100644 --- a/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java @@ -471,12 +471,14 @@ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) segments )) { Segment segment = holder.getObject().getChunk(0).getObject(); - QueryPlus queryPlusRunning = queryPlus.withQuerySegmentSpec( - new SpecificSegmentSpec( - new SegmentDescriptor( - holder.getInterval(), - holder.getVersion(), - 0 + QueryPlus queryPlusRunning = queryPlus.withQuery( + queryPlus.getQuery().withQuerySegmentSpec( + new SpecificSegmentSpec( + new SegmentDescriptor( + holder.getInterval(), + holder.getVersion(), + 0 + ) ) ) ); diff --git a/processing/src/test/java/org/apache/druid/query/TableDataSourceTest.java b/processing/src/test/java/org/apache/druid/query/TableDataSourceTest.java new file mode 100644 index 000000000000..ef50f3e45d9d --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/TableDataSourceTest.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.query; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.Collections; + +public class TableDataSourceTest +{ + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + private final TableDataSource fooDataSource = new TableDataSource("foo"); + + @Test + public void test_getTableNames() + { + Assert.assertEquals(Collections.singleton("foo"), fooDataSource.getTableNames()); + } + + @Test + public void test_getChildren() + { + Assert.assertEquals(Collections.emptyList(), fooDataSource.getChildren()); + } + + @Test + public void test_isCacheable() + { + Assert.assertTrue(fooDataSource.isCacheable()); + } + + @Test + public void test_isGlobal() + { + Assert.assertFalse(fooDataSource.isGlobal()); + } + + @Test + public void test_isConcrete() + { + Assert.assertTrue(fooDataSource.isConcrete()); + } + + @Test + public void test_withChildren_empty() + { + Assert.assertSame(fooDataSource, fooDataSource.withChildren(Collections.emptyList())); + } + + @Test + public void test_withChildren_nonEmpty() + { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Cannot accept children"); + + fooDataSource.withChildren(ImmutableList.of(new TableDataSource("bar"))); + } + + @Test + public void test_equals() + { + EqualsVerifier.forClass(TableDataSource.class).withNonnullFields("name").verify(); + } + + @Test + public void test_equals_legacy() + { + final LegacyDataSource legacyFoo = new LegacyDataSource("foo"); + final LegacyDataSource legacyBar = new LegacyDataSource("bar"); + + Assert.assertEquals(legacyFoo, fooDataSource); + Assert.assertEquals(fooDataSource, legacyFoo); + + Assert.assertNotEquals(legacyBar, fooDataSource); + Assert.assertNotEquals(fooDataSource, legacyBar); + } + + @Test + public void test_serde() throws Exception + { + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + final TableDataSource deserialized = (TableDataSource) jsonMapper.readValue( + jsonMapper.writeValueAsString(fooDataSource), + DataSource.class + ); + + Assert.assertEquals(fooDataSource, deserialized); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/UnionDataSourceTest.java b/processing/src/test/java/org/apache/druid/query/UnionDataSourceTest.java new file mode 100644 index 000000000000..117225d890b9 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/UnionDataSourceTest.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.query; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.Collections; +import java.util.List; + +public class UnionDataSourceTest +{ + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + private final UnionDataSource unionDataSource = new UnionDataSource( + ImmutableList.of( + new TableDataSource("foo"), + new TableDataSource("bar") + ) + ); + + private final UnionDataSource unionDataSourceWithDuplicates = new UnionDataSource( + ImmutableList.of( + new TableDataSource("bar"), + new TableDataSource("foo"), + new TableDataSource("bar") + ) + ); + + @Test + public void test_getTableNames() + { + Assert.assertEquals(ImmutableSet.of("foo", "bar"), unionDataSource.getTableNames()); + } + + @Test + public void test_getTableNames_withDuplicates() + { + Assert.assertEquals(ImmutableSet.of("foo", "bar"), unionDataSourceWithDuplicates.getTableNames()); + } + + @Test + public void test_getChildren() + { + Assert.assertEquals( + ImmutableList.of(new TableDataSource("foo"), new TableDataSource("bar")), + unionDataSource.getChildren() + ); + } + + @Test + public void test_getChildren_withDuplicates() + { + Assert.assertEquals( + ImmutableList.of(new TableDataSource("bar"), new TableDataSource("foo"), new TableDataSource("bar")), + unionDataSourceWithDuplicates.getChildren() + ); + } + + @Test + public void test_isCacheable() + { + Assert.assertFalse(unionDataSource.isCacheable()); + } + + @Test + public void test_isGlobal() + { + Assert.assertFalse(unionDataSource.isGlobal()); + } + + @Test + public void test_isConcrete() + { + Assert.assertTrue(unionDataSource.isConcrete()); + } + + @Test + public void test_withChildren_empty() + { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("Expected [2] children, got [0]"); + + unionDataSource.withChildren(Collections.emptyList()); + } + + @Test + public void test_withChildren_sameNumber() + { + final List newDataSources = ImmutableList.of( + new TableDataSource("baz"), + new TableDataSource("qux") + ); + + //noinspection unchecked + Assert.assertEquals( + new UnionDataSource(newDataSources), + unionDataSource.withChildren((List) newDataSources) + ); + } + + @Test + public void test_equals() + { + EqualsVerifier.forClass(UnionDataSource.class).usingGetClass().withNonnullFields("dataSources").verify(); + } + + @Test + public void test_serde() throws Exception + { + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + final UnionDataSource deserialized = (UnionDataSource) jsonMapper.readValue( + jsonMapper.writeValueAsString(unionDataSource), + DataSource.class + ); + + Assert.assertEquals(unionDataSource, deserialized); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/UnionQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/UnionQueryRunnerTest.java index 3b9e5e8971fe..a9ce7a9471b2 100644 --- a/processing/src/test/java/org/apache/druid/query/UnionQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/UnionQueryRunnerTest.java @@ -44,7 +44,7 @@ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { // verify that table datasource is passed to baseQueryRunner Assert.assertTrue(queryPlus.getQuery().getDataSource() instanceof TableDataSource); - String dsName = Iterables.getOnlyElement(queryPlus.getQuery().getDataSource().getNames()); + String dsName = Iterables.getOnlyElement(queryPlus.getQuery().getDataSource().getTableNames()); if ("ds1".equals(dsName)) { ds1.compareAndSet(false, true); return Sequences.simple(Arrays.asList(1, 2, 3)); diff --git a/processing/src/test/java/org/apache/druid/query/filter/DimFilterUtilsTest.java b/processing/src/test/java/org/apache/druid/query/filter/DimFilterUtilsTest.java index b8fa4c2b3112..778db21b0a3e 100644 --- a/processing/src/test/java/org/apache/druid/query/filter/DimFilterUtilsTest.java +++ b/processing/src/test/java/org/apache/druid/query/filter/DimFilterUtilsTest.java @@ -20,7 +20,6 @@ package org.apache.druid.query.filter; import com.google.common.base.Function; -import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableRangeSet; import com.google.common.collect.ImmutableSet; @@ -36,6 +35,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; public class DimFilterUtilsTest diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java index ef112f408c52..bdb771c01273 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java @@ -31,6 +31,8 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.CacheStrategy; +import org.apache.druid.query.Druids; +import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.QueryRunnerTestHelper; import org.apache.druid.query.QueryToolChestTestHelper; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -757,6 +759,64 @@ public void testResultsAsArraysDayGran() ); } + @Test + public void testCanPerformSubqueryOnGroupBys() + { + Assert.assertTrue( + new GroupByQueryQueryToolChest(null, null).canPerformSubquery( + new GroupByQuery.Builder() + .setDataSource( + new QueryDataSource( + new GroupByQuery.Builder() + .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) + .setInterval(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) + .setGranularity(Granularities.ALL) + .build() + ) + ) + .setInterval(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) + .setGranularity(Granularities.ALL) + .build() + ) + ); + } + + @Test + public void testCanPerformSubqueryOnTimeseries() + { + Assert.assertFalse( + new GroupByQueryQueryToolChest(null, null).canPerformSubquery( + Druids.newTimeseriesQueryBuilder() + .dataSource(QueryRunnerTestHelper.DATA_SOURCE) + .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) + .granularity(Granularities.ALL) + .build() + ) + ); + } + + @Test + public void testCanPerformSubqueryOnGroupByOfTimeseries() + { + Assert.assertFalse( + new GroupByQueryQueryToolChest(null, null).canPerformSubquery( + new GroupByQuery.Builder() + .setDataSource( + new QueryDataSource( + Druids.newTimeseriesQueryBuilder() + .dataSource(QueryRunnerTestHelper.DATA_SOURCE) + .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) + .granularity(Granularities.ALL) + .build() + ) + ) + .setInterval(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) + .setGranularity(Granularities.ALL) + .build() + ) + ); + } + private AggregatorFactory getComplexAggregatorFactoryForValueType(final ValueType valueType) { switch (valueType) { diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java index b8662e01f1df..037b2b75a973 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java @@ -2983,11 +2983,15 @@ public void testMergeResults() public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { // simulate two daily segments - final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03"))) + final QueryPlus queryPlus1 = queryPlus.withQuery( + queryPlus.getQuery().withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03"))) + ) ); - final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04"))) + final QueryPlus queryPlus2 = queryPlus.withQuery( + queryPlus.getQuery().withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04"))) + ) ); return new MergeSequence( queryPlus.getQuery().getResultOrdering(), @@ -3326,11 +3330,15 @@ private void doTestMergeResultsWithOrderBy( public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { // simulate two daily segments - final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03"))) + final QueryPlus queryPlus1 = queryPlus.withQuery( + queryPlus.getQuery().withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03"))) + ) ); - final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04"))) + final QueryPlus queryPlus2 = queryPlus.withQuery( + queryPlus.getQuery().withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04"))) + ) ); return new MergeSequence( queryPlus.getQuery().getResultOrdering(), @@ -4083,11 +4091,15 @@ public void testPostAggMergedHavingSpec() public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { // simulate two daily segments - final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03"))) + final QueryPlus queryPlus1 = queryPlus.withQuery( + queryPlus.getQuery().withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03"))) + ) ); - final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04"))) + final QueryPlus queryPlus2 = queryPlus.withQuery( + queryPlus.getQuery().withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04"))) + ) ); return new MergeSequence( queryPlus.getQuery().getResultOrdering(), @@ -4349,11 +4361,15 @@ public void testMergedHavingSpec() public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { // simulate two daily segments - final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03"))) + final QueryPlus queryPlus1 = queryPlus.withQuery( + queryPlus.getQuery().withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03"))) + ) ); - final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04"))) + final QueryPlus queryPlus2 = queryPlus.withQuery( + queryPlus.getQuery().withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04"))) + ) ); return new MergeSequence( queryPlus.getQuery().getResultOrdering(), @@ -4426,11 +4442,15 @@ public void testMergedPostAggHavingSpec() public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { // simulate two daily segments - final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03"))) + final QueryPlus queryPlus1 = queryPlus.withQuery( + queryPlus.getQuery().withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03"))) + ) ); - final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04"))) + final QueryPlus queryPlus2 = queryPlus.withQuery( + queryPlus.getQuery().withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04"))) + ) ); return new MergeSequence( queryPlus.getQuery().getResultOrdering(), @@ -9968,11 +9988,15 @@ public void testMergeResultsWithLimitPushDown() public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { // simulate two daily segments - final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03"))) + final QueryPlus queryPlus1 = queryPlus.withQuery( + queryPlus.getQuery().withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03"))) + ) ); - final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04"))) + final QueryPlus queryPlus2 = queryPlus.withQuery( + queryPlus.getQuery().withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04"))) + ) ); return factory.getToolchest().mergeResults( @@ -10034,11 +10058,15 @@ public void testMergeResultsWithLimitPushDownSortByAgg() public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { // simulate two daily segments - final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03"))) + final QueryPlus queryPlus1 = queryPlus.withQuery( + queryPlus.getQuery().withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03"))) + ) ); - final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04"))) + final QueryPlus queryPlus2 = queryPlus.withQuery( + queryPlus.getQuery().withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04"))) + ) ); return factory.getToolchest().mergeResults( @@ -10102,11 +10130,15 @@ public void testMergeResultsWithLimitPushDownSortByDimDim() public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { // simulate two daily segments - final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03"))) + final QueryPlus queryPlus1 = queryPlus.withQuery( + queryPlus.getQuery().withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03"))) + ) ); - final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04"))) + final QueryPlus queryPlus2 = queryPlus.withQuery( + queryPlus.getQuery().withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04"))) + ) ); return factory.getToolchest().mergeResults( @@ -10183,11 +10215,15 @@ public void testMergeResultsWithLimitPushDownSortByDimAggDim() public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { // simulate two daily segments - final QueryPlus queryPlus1 = queryPlus.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03"))) + final QueryPlus queryPlus1 = queryPlus.withQuery( + queryPlus.getQuery().withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03"))) + ) ); - final QueryPlus queryPlus2 = queryPlus.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04"))) + final QueryPlus queryPlus2 = queryPlus.withQuery( + queryPlus.getQuery().withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04"))) + ) ); return factory.getToolchest().mergeResults( diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java index 5b254ea18eb6..5f044ec339f6 100644 --- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java @@ -917,7 +917,7 @@ 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("test_ds", Iterables.getOnlyElement(query.getDataSource().getTableNames())); Assert.assertEquals( Intervals.of("2013-12-04T00:00:00.000Z/2013-12-05T00:00:00.000Z"), query.getIntervals().get(0) @@ -937,7 +937,7 @@ 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("test_ds", Iterables.getOnlyElement(query.getDataSource().getTableNames())); Assert.assertEquals(Intervals.ETERNITY, query.getIntervals().get(0)); Assert.assertTrue(((SegmentMetadataQuery) query).isUsingDefaultInterval()); diff --git a/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java b/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java new file mode 100644 index 000000000000..00c055ecaf75 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java @@ -0,0 +1,476 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.query.planning; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.InlineDataSource; +import org.apache.druid.query.JoinDataSource; +import org.apache.druid.query.LookupDataSource; +import org.apache.druid.query.QueryDataSource; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.UnionDataSource; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.join.JoinConditionAnalysis; +import org.apache.druid.segment.join.JoinType; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.Optional; + +public class DataSourceAnalysisTest +{ + private static final TableDataSource TABLE_FOO = new TableDataSource("foo"); + private static final TableDataSource TABLE_BAR = new TableDataSource("bar"); + private static final LookupDataSource LOOKUP_LOOKYLOO = new LookupDataSource("lookyloo"); + private static final InlineDataSource INLINE = InlineDataSource.fromIterable( + ImmutableList.of("column"), + ImmutableList.of(ValueType.STRING), + ImmutableList.of(new Object[0]) + ); + + @Test + public void testTable() + { + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(TABLE_FOO); + + Assert.assertTrue(analysis.isConcreteBased()); + Assert.assertTrue(analysis.isConcreteTableBased()); + Assert.assertFalse(analysis.isGlobal()); + Assert.assertFalse(analysis.isQuery()); + Assert.assertEquals(TABLE_FOO, analysis.getDataSource()); + Assert.assertEquals(TABLE_FOO, analysis.getBaseDataSource()); + Assert.assertEquals(Optional.of(TABLE_FOO), analysis.getBaseTableDataSource()); + Assert.assertEquals(Optional.empty(), analysis.getBaseQuerySegmentSpec()); + Assert.assertEquals(Collections.emptyList(), analysis.getPreJoinableClauses()); + } + + @Test + public void testUnion() + { + final UnionDataSource unionDataSource = new UnionDataSource(ImmutableList.of(TABLE_FOO, TABLE_BAR)); + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(unionDataSource); + + Assert.assertTrue(analysis.isConcreteBased()); + Assert.assertTrue(analysis.isConcreteTableBased()); + Assert.assertFalse(analysis.isGlobal()); + Assert.assertFalse(analysis.isQuery()); + Assert.assertEquals(unionDataSource, analysis.getDataSource()); + Assert.assertEquals(unionDataSource, analysis.getBaseDataSource()); + Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource()); + Assert.assertEquals(Optional.empty(), analysis.getBaseQuerySegmentSpec()); + Assert.assertEquals(Collections.emptyList(), analysis.getPreJoinableClauses()); + } + + @Test + public void testQueryOnTable() + { + final QueryDataSource queryDataSource = subquery(TABLE_FOO); + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(queryDataSource); + + Assert.assertTrue(analysis.isConcreteBased()); + Assert.assertTrue(analysis.isConcreteTableBased()); + Assert.assertFalse(analysis.isGlobal()); + Assert.assertTrue(analysis.isQuery()); + Assert.assertEquals(queryDataSource, analysis.getDataSource()); + Assert.assertEquals(TABLE_FOO, analysis.getBaseDataSource()); + Assert.assertEquals(Optional.of(TABLE_FOO), analysis.getBaseTableDataSource()); + Assert.assertEquals( + Optional.of(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2000/3000")))), + analysis.getBaseQuerySegmentSpec() + ); + Assert.assertEquals(Collections.emptyList(), analysis.getPreJoinableClauses()); + } + + @Test + public void testQueryOnUnion() + { + final UnionDataSource unionDataSource = new UnionDataSource(ImmutableList.of(TABLE_FOO, TABLE_BAR)); + final QueryDataSource queryDataSource = subquery(unionDataSource); + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(queryDataSource); + + Assert.assertTrue(analysis.isConcreteBased()); + Assert.assertTrue(analysis.isConcreteTableBased()); + Assert.assertFalse(analysis.isGlobal()); + Assert.assertTrue(analysis.isQuery()); + Assert.assertEquals(queryDataSource, analysis.getDataSource()); + Assert.assertEquals(unionDataSource, analysis.getBaseDataSource()); + Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource()); + Assert.assertEquals( + Optional.of(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2000/3000")))), + analysis.getBaseQuerySegmentSpec() + ); + Assert.assertEquals(Collections.emptyList(), analysis.getPreJoinableClauses()); + } + + @Test + public void testLookup() + { + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(LOOKUP_LOOKYLOO); + + Assert.assertFalse(analysis.isConcreteBased()); + Assert.assertFalse(analysis.isConcreteTableBased()); + Assert.assertTrue(analysis.isGlobal()); + Assert.assertFalse(analysis.isQuery()); + Assert.assertEquals(LOOKUP_LOOKYLOO, analysis.getDataSource()); + Assert.assertEquals(LOOKUP_LOOKYLOO, analysis.getBaseDataSource()); + Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource()); + Assert.assertEquals(Optional.empty(), analysis.getBaseQuerySegmentSpec()); + Assert.assertEquals(Collections.emptyList(), analysis.getPreJoinableClauses()); + } + + @Test + public void testQueryOnLookup() + { + final QueryDataSource queryDataSource = new QueryDataSource( + GroupByQuery.builder() + .setDataSource(LOOKUP_LOOKYLOO) + .setInterval(new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2000/3000")))) + .setGranularity(Granularities.ALL) + .build() + ); + + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(queryDataSource); + + Assert.assertFalse(analysis.isConcreteBased()); + Assert.assertFalse(analysis.isConcreteTableBased()); + Assert.assertTrue(analysis.isGlobal()); + Assert.assertTrue(analysis.isQuery()); + Assert.assertEquals(queryDataSource, analysis.getDataSource()); + Assert.assertEquals(LOOKUP_LOOKYLOO, analysis.getBaseDataSource()); + Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource()); + Assert.assertEquals( + Optional.of(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2000/3000")))), + analysis.getBaseQuerySegmentSpec() + ); + Assert.assertEquals(Collections.emptyList(), analysis.getPreJoinableClauses()); + } + + @Test + public void testInline() + { + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(INLINE); + + Assert.assertFalse(analysis.isConcreteBased()); + Assert.assertFalse(analysis.isConcreteTableBased()); + Assert.assertTrue(analysis.isGlobal()); + Assert.assertFalse(analysis.isQuery()); + Assert.assertEquals(INLINE, analysis.getDataSource()); + Assert.assertEquals(INLINE, analysis.getBaseDataSource()); + Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource()); + Assert.assertEquals(Optional.empty(), analysis.getBaseQuerySegmentSpec()); + Assert.assertEquals(Collections.emptyList(), analysis.getPreJoinableClauses()); + } + + @Test + public void testJoinSimpleLeftLeaning() + { + // Join of a table onto a variety of simple joinable objects (lookup, inline, subquery) with a left-leaning + // structure (no right children are joins themselves). + + final JoinDataSource joinDataSource = + join( + join( + join( + TABLE_FOO, + LOOKUP_LOOKYLOO, + "1.", + JoinType.INNER + ), + INLINE, + "2.", + JoinType.LEFT + ), + subquery(LOOKUP_LOOKYLOO), + "3.", + JoinType.FULL + ); + + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(joinDataSource); + + Assert.assertTrue(analysis.isConcreteBased()); + Assert.assertTrue(analysis.isConcreteTableBased()); + Assert.assertFalse(analysis.isGlobal()); + Assert.assertFalse(analysis.isQuery()); + Assert.assertEquals(joinDataSource, analysis.getDataSource()); + Assert.assertEquals(TABLE_FOO, analysis.getBaseDataSource()); + Assert.assertEquals(Optional.of(TABLE_FOO), analysis.getBaseTableDataSource()); + Assert.assertEquals(Optional.empty(), analysis.getBaseQuerySegmentSpec()); + Assert.assertEquals( + ImmutableList.of( + new PreJoinableClause("1.", LOOKUP_LOOKYLOO, JoinType.INNER, joinClause("1.")), + new PreJoinableClause("2.", INLINE, JoinType.LEFT, joinClause("2.")), + new PreJoinableClause("3.", subquery(LOOKUP_LOOKYLOO), JoinType.FULL, joinClause("3.")) + ), + analysis.getPreJoinableClauses() + ); + } + + @Test + public void testJoinSimpleRightLeaning() + { + // Join of a table onto a variety of simple joinable objects (lookup, inline, subquery) with a right-leaning + // structure (no left children are joins themselves). + // + // Note that unlike the left-leaning stack, which is fully flattened, this one will not get flattened at all. + + final JoinDataSource rightLeaningJoinStack = + join( + LOOKUP_LOOKYLOO, + join( + INLINE, + subquery(LOOKUP_LOOKYLOO), + "1.", + JoinType.LEFT + ), + "2.", + JoinType.FULL + ); + + final JoinDataSource joinDataSource = + join( + TABLE_FOO, + rightLeaningJoinStack, + "3.", + JoinType.RIGHT + ); + + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(joinDataSource); + + Assert.assertTrue(analysis.isConcreteBased()); + Assert.assertTrue(analysis.isConcreteTableBased()); + Assert.assertFalse(analysis.isGlobal()); + Assert.assertFalse(analysis.isQuery()); + Assert.assertEquals(joinDataSource, analysis.getDataSource()); + Assert.assertEquals(TABLE_FOO, analysis.getBaseDataSource()); + Assert.assertEquals(Optional.of(TABLE_FOO), analysis.getBaseTableDataSource()); + Assert.assertEquals(Optional.empty(), analysis.getBaseQuerySegmentSpec()); + Assert.assertEquals( + ImmutableList.of( + new PreJoinableClause("3.", rightLeaningJoinStack, JoinType.RIGHT, joinClause("3.")) + ), + analysis.getPreJoinableClauses() + ); + } + + @Test + public void testJoinOverTableSubquery() + { + final JoinDataSource joinDataSource = join( + TABLE_FOO, + subquery(TABLE_FOO), + "1.", + JoinType.INNER + ); + + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(joinDataSource); + + Assert.assertFalse(analysis.isConcreteBased()); + Assert.assertFalse(analysis.isConcreteTableBased()); + Assert.assertFalse(analysis.isGlobal()); + Assert.assertFalse(analysis.isQuery()); + Assert.assertEquals(joinDataSource, analysis.getDataSource()); + Assert.assertEquals(TABLE_FOO, analysis.getBaseDataSource()); + Assert.assertEquals(Optional.of(TABLE_FOO), analysis.getBaseTableDataSource()); + Assert.assertEquals( + ImmutableList.of( + new PreJoinableClause("1.", subquery(TABLE_FOO), JoinType.INNER, joinClause("1.")) + ), + analysis.getPreJoinableClauses() + ); + } + + @Test + public void testJoinTableUnionToLookup() + { + final UnionDataSource unionDataSource = new UnionDataSource(ImmutableList.of(TABLE_FOO, TABLE_BAR)); + final JoinDataSource joinDataSource = join( + unionDataSource, + LOOKUP_LOOKYLOO, + "1.", + JoinType.INNER + ); + + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(joinDataSource); + + Assert.assertTrue(analysis.isConcreteBased()); + Assert.assertTrue(analysis.isConcreteTableBased()); + Assert.assertFalse(analysis.isGlobal()); + Assert.assertFalse(analysis.isQuery()); + Assert.assertEquals(joinDataSource, analysis.getDataSource()); + Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource()); + Assert.assertEquals(unionDataSource, analysis.getBaseDataSource()); + Assert.assertEquals(Optional.empty(), analysis.getBaseQuerySegmentSpec()); + Assert.assertEquals( + ImmutableList.of( + new PreJoinableClause("1.", LOOKUP_LOOKYLOO, JoinType.INNER, joinClause("1.")) + ), + analysis.getPreJoinableClauses() + ); + } + + @Test + public void testJoinUnderTopLevelSubqueries() + { + final QueryDataSource queryDataSource = + subquery( + subquery( + join( + TABLE_FOO, + LOOKUP_LOOKYLOO, + "1.", + JoinType.INNER + ) + ) + ); + + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(queryDataSource); + + Assert.assertTrue(analysis.isConcreteBased()); + Assert.assertTrue(analysis.isConcreteTableBased()); + Assert.assertFalse(analysis.isGlobal()); + Assert.assertTrue(analysis.isQuery()); + Assert.assertEquals(queryDataSource, analysis.getDataSource()); + Assert.assertEquals(TABLE_FOO, analysis.getBaseDataSource()); + Assert.assertEquals(Optional.of(TABLE_FOO), analysis.getBaseTableDataSource()); + Assert.assertEquals( + Optional.of(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2000/3000")))), + analysis.getBaseQuerySegmentSpec() + ); + Assert.assertEquals( + ImmutableList.of( + new PreJoinableClause("1.", LOOKUP_LOOKYLOO, JoinType.INNER, joinClause("1.")) + ), + analysis.getPreJoinableClauses() + ); + } + + @Test + public void testJoinLookupToLookup() + { + final JoinDataSource joinDataSource = join( + LOOKUP_LOOKYLOO, + LOOKUP_LOOKYLOO, + "1.", + JoinType.INNER + ); + + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(joinDataSource); + + Assert.assertFalse(analysis.isConcreteBased()); + Assert.assertFalse(analysis.isConcreteTableBased()); + Assert.assertTrue(analysis.isGlobal()); + Assert.assertFalse(analysis.isQuery()); + Assert.assertEquals(joinDataSource, analysis.getDataSource()); + Assert.assertEquals(LOOKUP_LOOKYLOO, analysis.getBaseDataSource()); + Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource()); + Assert.assertEquals(Optional.empty(), analysis.getBaseQuerySegmentSpec()); + Assert.assertEquals( + ImmutableList.of( + new PreJoinableClause("1.", LOOKUP_LOOKYLOO, JoinType.INNER, joinClause("1.")) + ), + analysis.getPreJoinableClauses() + ); + } + + @Test + public void testJoinLookupToTable() + { + final JoinDataSource joinDataSource = join( + LOOKUP_LOOKYLOO, + TABLE_FOO, + "1.", + JoinType.INNER + ); + + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(joinDataSource); + + Assert.assertFalse(analysis.isConcreteBased()); + Assert.assertFalse(analysis.isConcreteTableBased()); + Assert.assertFalse(analysis.isGlobal()); + Assert.assertFalse(analysis.isQuery()); + Assert.assertEquals(joinDataSource, analysis.getDataSource()); + Assert.assertEquals(LOOKUP_LOOKYLOO, analysis.getBaseDataSource()); + Assert.assertEquals(Optional.empty(), analysis.getBaseTableDataSource()); + Assert.assertEquals(Optional.empty(), analysis.getBaseQuerySegmentSpec()); + Assert.assertEquals( + ImmutableList.of( + new PreJoinableClause("1.", TABLE_FOO, JoinType.INNER, joinClause("1.")) + ), + analysis.getPreJoinableClauses() + ); + } + + /** + * Generate a datasource that joins on a column named "x" on both sides. + */ + private static JoinDataSource join( + final DataSource left, + final DataSource right, + final String rightPrefix, + final JoinType joinType + ) + { + return JoinDataSource.create( + left, + right, + rightPrefix, + joinClause(rightPrefix).getOriginalExpression(), + joinType, + ExprMacroTable.nil() + ); + } + + /** + * Generate a join clause that joins on a column named "x" on both sides. + */ + private static JoinConditionAnalysis joinClause( + final String rightPrefix + ) + { + return JoinConditionAnalysis.forExpression( + StringUtils.format("x == \"%sx\"", rightPrefix), + rightPrefix, + ExprMacroTable.nil() + ); + } + + /** + * Generate a datasource that does a subquery on another datasource. The specific kind of query doesn't matter + * much for the purpose of this test class, so it's always the same. + */ + private static QueryDataSource subquery(final DataSource dataSource) + { + return new QueryDataSource( + GroupByQuery.builder() + .setDataSource(dataSource) + .setInterval(new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2000/3000")))) + .setGranularity(Granularities.ALL) + .build() + ); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/planning/PreJoinableClauseTest.java b/processing/src/test/java/org/apache/druid/query/planning/PreJoinableClauseTest.java new file mode 100644 index 000000000000..80762a758c83 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/planning/PreJoinableClauseTest.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.query.planning; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.segment.join.JoinConditionAnalysis; +import org.apache.druid.segment.join.JoinType; +import org.junit.Assert; +import org.junit.Test; + +public class PreJoinableClauseTest +{ + private final PreJoinableClause clause = new PreJoinableClause( + "j.", + new TableDataSource("foo"), + JoinType.LEFT, + JoinConditionAnalysis.forExpression("x == \"j.x\"", "j.", ExprMacroTable.nil()) + ); + + @Test + public void test_getPrefix() + { + Assert.assertEquals("j.", clause.getPrefix()); + } + + @Test + public void test_getJoinType() + { + Assert.assertEquals(JoinType.LEFT, clause.getJoinType()); + } + + @Test + public void test_getCondition() + { + Assert.assertEquals("x == \"j.x\"", clause.getCondition().getOriginalExpression()); + } + + @Test + public void test_getDataSource() + { + Assert.assertEquals(new TableDataSource("foo"), clause.getDataSource()); + } + + @Test + public void test_equals() + { + EqualsVerifier.forClass(PreJoinableClause.class) + .usingGetClass() + .withNonnullFields("prefix", "dataSource", "joinType", "condition") + .verify(); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java index 6b503a541116..aebcf257b3e5 100644 --- a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java @@ -72,6 +72,7 @@ import java.util.List; /** + * */ @RunWith(Parameterized.class) public class SearchQueryRunnerTest extends InitializedNullHandlingTest @@ -167,11 +168,15 @@ public Sequence> run( ResponseContext responseContext ) { - final QueryPlus> queryPlus1 = queryPlus.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-01-12/2011-02-28"))) + final QueryPlus> queryPlus1 = queryPlus.withQuery( + queryPlus.getQuery().withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-01-12/2011-02-28"))) + ) ); - final QueryPlus> queryPlus2 = queryPlus.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-03-01/2011-04-15"))) + final QueryPlus> queryPlus2 = queryPlus.withQuery( + queryPlus.getQuery().withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-03-01/2011-04-15"))) + ) ); return Sequences.concat(runner.run(queryPlus1, responseContext), runner.run(queryPlus2, responseContext)); } diff --git a/server/src/main/java/org/apache/druid/client/BrokerServerView.java b/server/src/main/java/org/apache/druid/client/BrokerServerView.java index d3f0a6672308..5bf4ee9c911e 100644 --- a/server/src/main/java/org/apache/druid/client/BrokerServerView.java +++ b/server/src/main/java/org/apache/druid/client/BrokerServerView.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Predicate; -import com.google.common.collect.Iterables; import com.google.common.collect.Ordering; import com.google.inject.Inject; import org.apache.druid.client.selector.QueryableDruidServer; @@ -30,26 +29,28 @@ import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.annotations.EscalatedClient; import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.http.client.HttpClient; -import org.apache.druid.query.DataSource; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.QueryWatcher; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.PartitionChunk; -import javax.annotation.Nullable; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; @@ -59,6 +60,7 @@ import java.util.stream.Collectors; /** + * */ @ManageLifecycle public class BrokerServerView implements TimelineServerView @@ -288,14 +290,15 @@ private void serverRemovedSegment(DruidServerMetadata server, DataSegment segmen } } - - @Nullable @Override - public VersionedIntervalTimeline getTimeline(DataSource dataSource) + public Optional> getTimeline(final DataSourceAnalysis analysis) { - String table = Iterables.getOnlyElement(dataSource.getNames()); + final TableDataSource tableDataSource = + analysis.getBaseTableDataSource() + .orElseThrow(() -> new ISE("Cannot handle datasource: %s", analysis.getDataSource())); + synchronized (lock) { - return timelines.get(table); + return Optional.ofNullable(timelines.get(tableDataSource.getName())); } } diff --git a/server/src/main/java/org/apache/druid/client/CacheUtil.java b/server/src/main/java/org/apache/druid/client/CacheUtil.java index d3d9183e1c16..e5825252ac12 100644 --- a/server/src/main/java/org/apache/druid/client/CacheUtil.java +++ b/server/src/main/java/org/apache/druid/client/CacheUtil.java @@ -99,7 +99,8 @@ private static boolean useCache( return QueryContexts.isUseCache(query) && strategy != null && cacheConfig.isUseCache() - && cacheConfig.isQueryCacheable(query); + && cacheConfig.isQueryCacheable(query) + && query.getDataSource().isCacheable(); } private static boolean populateCache( @@ -111,7 +112,8 @@ private static boolean populateCache( return QueryContexts.isPopulateCache(query) && strategy != null && cacheConfig.isPopulateCache() - && cacheConfig.isQueryCacheable(query); + && cacheConfig.isQueryCacheable(query) + && query.getDataSource().isCacheable(); } } diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index 414c0541b648..e003ab8d519e 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; -import com.google.common.base.Optional; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; @@ -53,6 +52,7 @@ import org.apache.druid.query.BySegmentResultValueClass; import org.apache.druid.query.CacheStrategy; import org.apache.druid.query.DruidProcessingConfig; +import org.apache.druid.query.Queries; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryMetrics; @@ -66,7 +66,8 @@ import org.apache.druid.query.aggregation.MetricManipulatorFns; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.filter.DimFilterUtils; -import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; +import org.apache.druid.query.planning.DataSourceAnalysis; +import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.server.QueryResource; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.timeline.DataSegment; @@ -88,6 +89,7 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; @@ -97,6 +99,7 @@ import java.util.stream.Collectors; /** + * */ public class CachingClusteredClient implements QuerySegmentWalker { @@ -231,6 +234,7 @@ private class SpecificQueryRunnable private final int uncoveredIntervalsLimit; private final Query downstreamQuery; private final Map cachePopulatorKeyMap = new HashMap<>(); + private final DataSourceAnalysis dataSourceAnalysis; private final List intervals; SpecificQueryRunnable(final QueryPlus queryPlus, final ResponseContext responseContext) @@ -248,8 +252,11 @@ private class SpecificQueryRunnable // and might blow up in some cases https://github.com/apache/druid/issues/2108 this.uncoveredIntervalsLimit = QueryContexts.getUncoveredIntervalsLimit(query); this.downstreamQuery = query.withOverriddenContext(makeDownstreamQueryContext()); + this.dataSourceAnalysis = DataSourceAnalysis.forDataSource(query.getDataSource()); // For nested queries, we need to look at the intervals of the inner most query. - this.intervals = query.getIntervalsOfInnerMostQuery(); + this.intervals = dataSourceAnalysis.getBaseQuerySegmentSpec() + .map(QuerySegmentSpec::getIntervals) + .orElse(query.getIntervals()); } private ImmutableMap makeDownstreamQueryContext() @@ -269,12 +276,14 @@ private ImmutableMap makeDownstreamQueryContext() Sequence run(final UnaryOperator> timelineConverter) { - @Nullable - TimelineLookup timeline = serverView.getTimeline(query.getDataSource()); - if (timeline == null) { + final Optional> maybeTimeline = serverView.getTimeline( + dataSourceAnalysis + ); + if (!maybeTimeline.isPresent()) { return Sequences.empty(); } - timeline = timelineConverter.apply(timeline); + + final TimelineLookup timeline = timelineConverter.apply(maybeTimeline.get()); if (uncoveredIntervalsLimit > 0) { computeUncoveredIntervals(timeline); } @@ -598,19 +607,17 @@ private void addSequencesFromServer( return; } - final MultipleSpecificSegmentSpec segmentsOfServerSpec = new MultipleSpecificSegmentSpec(segmentsOfServer); - // Divide user-provided maxQueuedBytes by the number of servers, and limit each server to that much. final long maxQueuedBytes = QueryContexts.getMaxQueuedBytes(query, httpClientConfig.getMaxQueuedBytes()); final long maxQueuedBytesPerServer = maxQueuedBytes / segmentsByServer.size(); final Sequence serverResults; if (isBySegment) { - serverResults = getBySegmentServerResults(serverRunner, segmentsOfServerSpec, maxQueuedBytesPerServer); + serverResults = getBySegmentServerResults(serverRunner, segmentsOfServer, maxQueuedBytesPerServer); } else if (!server.segmentReplicatable() || !populateCache) { - serverResults = getSimpleServerResults(serverRunner, segmentsOfServerSpec, maxQueuedBytesPerServer); + serverResults = getSimpleServerResults(serverRunner, segmentsOfServer, maxQueuedBytesPerServer); } else { - serverResults = getAndCacheServerResults(serverRunner, segmentsOfServerSpec, maxQueuedBytesPerServer); + serverResults = getAndCacheServerResults(serverRunner, segmentsOfServer, maxQueuedBytesPerServer); } listOfSequences.add(serverResults); }); @@ -619,13 +626,15 @@ private void addSequencesFromServer( @SuppressWarnings("unchecked") private Sequence getBySegmentServerResults( final QueryRunner serverRunner, - final MultipleSpecificSegmentSpec segmentsOfServerSpec, + final List segmentsOfServer, long maxQueuedBytesPerServer ) { Sequence>> resultsBySegments = serverRunner .run( - queryPlus.withQuerySegmentSpec(segmentsOfServerSpec).withMaxQueuedBytes(maxQueuedBytesPerServer), + queryPlus.withQuery( + Queries.withSpecificSegments(queryPlus.getQuery(), segmentsOfServer) + ).withMaxQueuedBytes(maxQueuedBytesPerServer), responseContext ); // bySegment results need to be de-serialized, see DirectDruidClient.run() @@ -640,27 +649,33 @@ private Sequence getBySegmentServerResults( @SuppressWarnings("unchecked") private Sequence getSimpleServerResults( final QueryRunner serverRunner, - final MultipleSpecificSegmentSpec segmentsOfServerSpec, + final List segmentsOfServer, long maxQueuedBytesPerServer ) { return serverRunner.run( - queryPlus.withQuerySegmentSpec(segmentsOfServerSpec).withMaxQueuedBytes(maxQueuedBytesPerServer), + queryPlus.withQuery( + Queries.withSpecificSegments(queryPlus.getQuery(), segmentsOfServer) + ).withMaxQueuedBytes(maxQueuedBytesPerServer), responseContext ); } private Sequence getAndCacheServerResults( final QueryRunner serverRunner, - final MultipleSpecificSegmentSpec segmentsOfServerSpec, + final List segmentsOfServer, long maxQueuedBytesPerServer ) { @SuppressWarnings("unchecked") final Sequence>> resultsBySegments = serverRunner.run( queryPlus - .withQuery((Query>>) downstreamQuery) - .withQuerySegmentSpec(segmentsOfServerSpec) + .withQuery( + Queries.withSpecificSegments( + (Query>>) downstreamQuery, + segmentsOfServer + ) + ) .withMaxQueuedBytes(maxQueuedBytesPerServer), responseContext ); diff --git a/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java b/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java index 1f1d801d9e36..2517a8f0e9be 100644 --- a/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java +++ b/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java @@ -194,7 +194,7 @@ private void serverRemovedSegment(DruidServerMetadata server, DataSegment segmen public VersionedIntervalTimeline getTimeline(DataSource dataSource) { - String table = Iterables.getOnlyElement(dataSource.getNames()); + String table = Iterables.getOnlyElement(dataSource.getTableNames()); synchronized (lock) { return timelines.get(table); } diff --git a/server/src/main/java/org/apache/druid/client/ResultLevelCacheUtil.java b/server/src/main/java/org/apache/druid/client/ResultLevelCacheUtil.java index 7ca9dc19917f..86b7fe531619 100644 --- a/server/src/main/java/org/apache/druid/client/ResultLevelCacheUtil.java +++ b/server/src/main/java/org/apache/druid/client/ResultLevelCacheUtil.java @@ -73,7 +73,8 @@ private static boolean useResultLevelCache( return QueryContexts.isUseResultLevelCache(query) && strategy != null && cacheConfig.isUseResultLevelCache() - && cacheConfig.isQueryCacheable(query); + && cacheConfig.isQueryCacheable(query) + && query.getDataSource().isCacheable(); } private static boolean populateResultLevelCache( @@ -85,6 +86,7 @@ private static boolean populateResultLevelCache( return QueryContexts.isPopulateResultLevelCache(query) && strategy != null && cacheConfig.isPopulateResultLevelCache() - && cacheConfig.isQueryCacheable(query); + && cacheConfig.isQueryCacheable(query) + && query.getDataSource().isCacheable(); } } diff --git a/server/src/main/java/org/apache/druid/client/ServerViewUtil.java b/server/src/main/java/org/apache/druid/client/ServerViewUtil.java index 8a52cacf2995..b9f1f91a5d35 100644 --- a/server/src/main/java/org/apache/druid/client/ServerViewUtil.java +++ b/server/src/main/java/org/apache/druid/client/ServerViewUtil.java @@ -24,6 +24,7 @@ import org.apache.druid.query.LocatedSegmentDescriptor; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.timeline.TimelineLookup; import org.apache.druid.timeline.TimelineObjectHolder; @@ -33,6 +34,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Optional; /** */ @@ -55,13 +57,14 @@ public static List getTargetLocations( int numCandidates ) { - TimelineLookup timeline = serverView.getTimeline(datasource); - if (timeline == null) { + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(datasource); + final Optional> maybeTimeline = serverView.getTimeline(analysis); + if (!maybeTimeline.isPresent()) { return Collections.emptyList(); } List located = new ArrayList<>(); for (Interval interval : intervals) { - for (TimelineObjectHolder holder : timeline.lookup(interval)) { + for (TimelineObjectHolder holder : maybeTimeline.get().lookup(interval)) { for (PartitionChunk chunk : holder.getObject()) { ServerSelector selector = chunk.getObject(); final SegmentDescriptor descriptor = new SegmentDescriptor( diff --git a/server/src/main/java/org/apache/druid/client/TimelineServerView.java b/server/src/main/java/org/apache/druid/client/TimelineServerView.java index ed1d4dfb7313..477882342425 100644 --- a/server/src/main/java/org/apache/druid/client/TimelineServerView.java +++ b/server/src/main/java/org/apache/druid/client/TimelineServerView.java @@ -20,22 +20,31 @@ package org.apache.druid.client; import org.apache.druid.client.selector.ServerSelector; -import org.apache.druid.query.DataSource; import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.TimelineLookup; -import javax.annotation.Nullable; import java.util.List; +import java.util.Optional; import java.util.concurrent.Executor; /** */ public interface TimelineServerView extends ServerView { - @Nullable - TimelineLookup getTimeline(DataSource dataSource); + /** + * Returns the timeline for a datasource, if it exists. The analysis object passed in must represent a scan-based + * datasource of a single table. + * + * @param analysis data source analysis information + * + * @return timeline, if it exists + * + * @throws IllegalStateException if 'analysis' does not represent a scan-based datasource of a single table + */ + Optional> getTimeline(DataSourceAnalysis analysis); /** * Returns a list of {@link ImmutableDruidServer} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index 6c5287920182..ac9a8da14345 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -20,7 +20,6 @@ package org.apache.druid.segment.realtime.appenderator; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; import org.apache.druid.client.CachingQueryRunner; @@ -42,6 +41,7 @@ import org.apache.druid.query.MetricsEmittingQueryRunner; import org.apache.druid.query.NoopQueryRunner; import org.apache.druid.query.Query; +import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactory; @@ -53,22 +53,26 @@ import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.SinkQueryRunners; import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.query.spec.SpecificSegmentQueryRunner; import org.apache.druid.query.spec.SpecificSegmentSpec; import org.apache.druid.segment.Segment; import org.apache.druid.segment.realtime.FireHydrant; import org.apache.druid.segment.realtime.plumber.Sink; import org.apache.druid.timeline.SegmentId; -import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.PartitionHolder; import org.joda.time.Interval; import java.io.Closeable; +import java.util.Optional; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicLong; +/** + * Query handler for indexing tasks. + */ public class SinkQuerySegmentWalker implements QuerySegmentWalker { private static final EmittingLogger log = new EmittingLogger(SinkQuerySegmentWalker.class); @@ -117,40 +121,17 @@ public QueryRunner getQueryRunnerForIntervals(final Query query, final { final Iterable specs = FunctionalIterable .create(intervals) + .transformCat(sinkTimeline::lookup) .transformCat( - new Function>>() - { - @Override - public Iterable> apply(final Interval interval) - { - return sinkTimeline.lookup(interval); - } - } - ) - .transformCat( - new Function, Iterable>() - { - @Override - public Iterable apply(final TimelineObjectHolder holder) - { - return FunctionalIterable - .create(holder.getObject()) - .transform( - new Function, SegmentDescriptor>() - { - @Override - public SegmentDescriptor apply(final PartitionChunk chunk) - { - return new SegmentDescriptor( - holder.getInterval(), - holder.getVersion(), - chunk.getChunkNumber() - ); - } - } - ); - } - } + holder -> FunctionalIterable + .create(holder.getObject()) + .transform( + chunk -> new SegmentDescriptor( + holder.getInterval(), + holder.getVersion(), + chunk.getChunkNumber() + ) + ) ); return getQueryRunnerForSegments(query, specs); @@ -160,12 +141,12 @@ public SegmentDescriptor apply(final PartitionChunk chunk) 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(); - return new NoopQueryRunner<>(); + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(query.getDataSource()); + final Optional baseTableDataSource = analysis.getBaseTableDataSource(); + + if (!baseTableDataSource.isPresent() || !dataSource.equals(baseTableDataSource.get().getName())) { + // Report error, since we somehow got a query for a datasource we can't handle. + throw new ISE("Cannot handle datasource: %s", analysis.getDataSource()); } final QueryRunnerFactory> factory = conglomerate.findFactory(query); @@ -177,6 +158,16 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final final boolean skipIncrementalSegment = query.getContextValue(CONTEXT_SKIP_INCREMENTAL_SEGMENT, false); final AtomicLong cpuTimeAccumulator = new AtomicLong(0L); + // Make sure this query type can handle the subquery, if present. + if (analysis.isQuery() && !toolChest.canPerformSubquery(((QueryDataSource) analysis.getDataSource()).getQuery())) { + throw new ISE("Cannot handle subquery: %s", analysis.getDataSource()); + } + + // We can't actually handle joins yet; throw an error if they are present. + if (!analysis.getPreJoinableClauses().isEmpty()) { + throw new ISE("Cannot handle 'join' dataSource"); + } + Iterable> perSegmentRunners = Iterables.transform( specs, descriptor -> { diff --git a/server/src/main/java/org/apache/druid/server/ClientInfoResource.java b/server/src/main/java/org/apache/druid/server/ClientInfoResource.java index 24f6c4435d2f..08032a5efe39 100644 --- a/server/src/main/java/org/apache/druid/server/ClientInfoResource.java +++ b/server/src/main/java/org/apache/druid/server/ClientInfoResource.java @@ -37,6 +37,7 @@ import org.apache.druid.query.LocatedSegmentDescriptor; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.metadata.SegmentMetadataQueryConfig; +import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.server.http.security.DatasourceResourceFilter; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthorizationUtils; @@ -64,12 +65,14 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.TreeMap; import java.util.stream.Collectors; import java.util.stream.Stream; /** + * */ @Path("/druid/v2/datasources") public class ClientInfoResource @@ -152,12 +155,12 @@ KEY_METRICS, getDataSourceMetrics(dataSourceName, interval) theInterval = Intervals.of(interval); } - TimelineLookup timeline = timelineServerView.getTimeline(new TableDataSource(dataSourceName)); - Iterable> serversLookup = timeline != null ? timeline.lookup( - theInterval - ) : null; - if (serversLookup == null || Iterables.isEmpty(serversLookup)) { - return Collections.EMPTY_MAP; + final Optional> maybeTimeline = + timelineServerView.getTimeline(DataSourceAnalysis.forDataSource(new TableDataSource(dataSourceName))); + final Optional>> maybeServersLookup = + maybeTimeline.map(timeline -> timeline.lookup(theInterval)); + if (!maybeServersLookup.isPresent() || Iterables.isEmpty(maybeServersLookup.get())) { + return Collections.emptyMap(); } Map servedIntervals = new TreeMap<>( new Comparator() @@ -174,7 +177,7 @@ public int compare(Interval o1, Interval o2) } ); - for (TimelineObjectHolder holder : serversLookup) { + for (TimelineObjectHolder holder : maybeServersLookup.get()) { final Set dimensions = new HashSet<>(); final Set metrics = new HashSet<>(); final PartitionHolder partitionHolder = holder.getObject(); diff --git a/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java index 29a173ccf985..a46c65529103 100644 --- a/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java @@ -25,6 +25,7 @@ import org.apache.druid.client.CachingClusteredClient; import org.apache.druid.client.cache.Cache; import org.apache.druid.client.cache.CacheConfig; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.query.FluentQueryRunnerBuilder; import org.apache.druid.query.PostProcessingOperator; @@ -37,11 +38,10 @@ import org.apache.druid.query.RetryQueryRunner; import org.apache.druid.query.RetryQueryRunnerConfig; import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.server.initialization.ServerConfig; import org.joda.time.Interval; -/** - */ public class ClientQuerySegmentWalker implements QuerySegmentWalker { private final ServiceEmitter emitter; @@ -53,7 +53,6 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker private final Cache cache; private final CacheConfig cacheConfig; - @Inject public ClientQuerySegmentWalker( ServiceEmitter emitter, @@ -79,13 +78,27 @@ public ClientQuerySegmentWalker( @Override public QueryRunner getQueryRunnerForIntervals(Query query, Iterable intervals) { - return makeRunner(query, baseClient.getQueryRunnerForIntervals(query, intervals)); + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(query.getDataSource()); + + if (analysis.isConcreteTableBased()) { + return makeRunner(query, baseClient.getQueryRunnerForIntervals(query, intervals)); + } else { + // In the future, we will check here to see if parts of the query are inlinable, and if that inlining would + // be able to create a concrete table-based query that we can run through the distributed query stack. + throw new ISE("Query dataSource is not table-based, cannot run"); + } } @Override public QueryRunner getQueryRunnerForSegments(Query query, Iterable specs) { - return makeRunner(query, baseClient.getQueryRunnerForSegments(query, specs)); + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(query.getDataSource()); + + if (analysis.isConcreteTableBased()) { + return makeRunner(query, baseClient.getQueryRunnerForSegments(query, specs)); + } else { + throw new ISE("Query dataSource is not table-based, cannot run"); + } } private QueryRunner makeRunner(Query query, QueryRunner baseClientRunner) @@ -93,12 +106,14 @@ private QueryRunner makeRunner(Query query, QueryRunner baseClientR QueryToolChest> toolChest = warehouse.getToolChest(query); // This does not adhere to the fluent workflow. See https://github.com/apache/druid/issues/5517 - return new ResultLevelCachingQueryRunner<>(makeRunner(query, baseClientRunner, toolChest), - toolChest, - query, - objectMapper, - cache, - cacheConfig); + return new ResultLevelCachingQueryRunner<>( + makeRunner(query, baseClientRunner, toolChest), + toolChest, + query, + objectMapper, + cache, + cacheConfig + ); } private QueryRunner makeRunner( @@ -109,9 +124,7 @@ private QueryRunner makeRunner( { PostProcessingOperator postProcessing = objectMapper.convertValue( query.getContextValue("postProcessing"), - new TypeReference>() - { - } + new TypeReference>() {} ); return new FluentQueryRunnerBuilder<>(toolChest) diff --git a/server/src/main/java/org/apache/druid/server/QueryLifecycle.java b/server/src/main/java/org/apache/druid/server/QueryLifecycle.java index 87508107d194..6e876c4d0922 100644 --- a/server/src/main/java/org/apache/druid/server/QueryLifecycle.java +++ b/server/src/main/java/org/apache/druid/server/QueryLifecycle.java @@ -189,7 +189,7 @@ public Access authorize(final AuthenticationResult authenticationResult) AuthorizationUtils.authorizeAllResourceActions( authenticationResult, Iterables.transform( - baseQuery.getDataSource().getNames(), + baseQuery.getDataSource().getTableNames(), AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR ), authorizerMapper @@ -213,7 +213,7 @@ public Access authorize(HttpServletRequest req) AuthorizationUtils.authorizeAllResourceActions( req, Iterables.transform( - baseQuery.getDataSource().getNames(), + baseQuery.getDataSource().getTableNames(), AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR ), authorizerMapper diff --git a/server/src/main/java/org/apache/druid/server/QueryManager.java b/server/src/main/java/org/apache/druid/server/QueryManager.java index a90bf4b077d5..0fd1807d86e3 100644 --- a/server/src/main/java/org/apache/druid/server/QueryManager.java +++ b/server/src/main/java/org/apache/druid/server/QueryManager.java @@ -27,7 +27,6 @@ import org.apache.druid.query.Query; import org.apache.druid.query.QueryWatcher; -import java.util.List; import java.util.Set; public class QueryManager implements QueryWatcher @@ -61,7 +60,7 @@ 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 Set datasources = query.getDataSource().getTableNames(); queries.put(id, future); queryDatasources.putAll(id, datasources); future.addListener( diff --git a/server/src/main/java/org/apache/druid/server/QueryResource.java b/server/src/main/java/org/apache/druid/server/QueryResource.java index 78100aa153a4..e2fb263def4c 100644 --- a/server/src/main/java/org/apache/druid/server/QueryResource.java +++ b/server/src/main/java/org/apache/druid/server/QueryResource.java @@ -190,7 +190,7 @@ public Response doPost( "%s[%s_%s_%s]", currThreadName, query.getType(), - query.getDataSource().getNames(), + query.getDataSource().getTableNames(), queryId ); diff --git a/server/src/main/java/org/apache/druid/server/SegmentManager.java b/server/src/main/java/org/apache/druid/server/SegmentManager.java index dbb48474defc..45b6538bb610 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -23,7 +23,10 @@ import com.google.common.collect.Ordering; import com.google.inject.Inject; import org.apache.druid.common.guava.SettableSupplier; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.Segment; import org.apache.druid.segment.loading.SegmentLoader; @@ -35,8 +38,8 @@ import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.utils.CollectionUtils; -import javax.annotation.Nullable; import java.util.Map; +import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; /** @@ -134,19 +137,30 @@ public boolean isSegmentCached(final DataSegment segment) return segmentLoader.isSegmentLoaded(segment); } - @Nullable - public VersionedIntervalTimeline getTimeline(String dataSource) + /** + * Returns the timeline for a datasource, if it exists. The analysis object passed in must represent a scan-based + * datasource of a single table. + * + * @param analysis data source analysis information + * + * @return timeline, if it exists + * + * @throws IllegalStateException if 'analysis' does not represent a scan-based datasource of a single table + */ + public Optional> getTimeline(DataSourceAnalysis analysis) { - final DataSourceState dataSourceState = dataSources.get(dataSource); - return dataSourceState == null ? null : dataSourceState.getTimeline(); + final TableDataSource tableDataSource = + analysis.getBaseTableDataSource() + .orElseThrow(() -> new ISE("Cannot handle datasource: %s", analysis.getDataSource())); + + return Optional.ofNullable(dataSources.get(tableDataSource.getName())).map(DataSourceState::getTimeline); } /** * Load a single segment. * * @param segment segment to load - * - * @param lazy whether to lazy load columns metadata + * @param lazy whether to lazy load columns metadata * * @return true if the segment was newly loaded, false if it was already loaded * diff --git a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java index 98e6f3e814a3..d82ee0d7dac5 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java @@ -20,8 +20,6 @@ package org.apache.druid.server.coordination; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; -import com.google.common.collect.Iterables; import com.google.inject.Inject; import org.apache.druid.client.CachingQueryRunner; import org.apache.druid.client.cache.Cache; @@ -35,7 +33,6 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.query.BySegmentQueryRunner; import org.apache.druid.query.CPUTimeMetricQueryRunner; -import org.apache.druid.query.DataSource; import org.apache.druid.query.FinalizeResultsQueryRunner; import org.apache.druid.query.MetricsEmittingQueryRunner; import org.apache.druid.query.NoopQueryRunner; @@ -52,7 +49,7 @@ import org.apache.druid.query.ReferenceCountingSegmentQueryRunner; import org.apache.druid.query.ReportTimelineMissingSegmentQueryRunner; import org.apache.druid.query.SegmentDescriptor; -import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.query.spec.SpecificSegmentQueryRunner; import org.apache.druid.query.spec.SpecificSegmentSpec; import org.apache.druid.segment.ReferenceCountingSegment; @@ -60,18 +57,18 @@ import org.apache.druid.server.SetAndVerifyContextQueryRunner; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.timeline.SegmentId; -import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.PartitionHolder; import org.joda.time.Interval; -import javax.annotation.Nullable; import java.util.Collections; +import java.util.Optional; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicLong; /** + * Query handler for Historical processes (see CliHistorical). */ public class ServerManager implements QuerySegmentWalker { @@ -112,105 +109,46 @@ public ServerManager( this.serverConfig = serverConfig; } - private DataSource getInnerMostDataSource(DataSource dataSource) - { - if (dataSource instanceof QueryDataSource) { - return getInnerMostDataSource(((QueryDataSource) dataSource).getQuery().getDataSource()); - } - return dataSource; - } - @Override public QueryRunner getQueryRunnerForIntervals(Query query, Iterable intervals) { - final QueryRunnerFactory> factory = conglomerate.findFactory(query); - if (factory == null) { - throw new ISE("Unknown query type[%s].", query.getClass()); - } - - final QueryToolChest> toolChest = factory.getToolchest(); - final AtomicLong cpuTimeAccumulator = new AtomicLong(0L); - - DataSource dataSource = getInnerMostDataSource(query.getDataSource()); - if (!(dataSource instanceof TableDataSource)) { - throw new UnsupportedOperationException("data source type '" + dataSource.getClass().getName() + "' unsupported"); + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(query.getDataSource()); + + final VersionedIntervalTimeline timeline; + final Optional> maybeTimeline = + segmentManager.getTimeline(analysis); + + if (maybeTimeline.isPresent()) { + timeline = maybeTimeline.get(); + } else { + // Note: this is not correct when there's a right or full outer join going on. + // See https://github.com/apache/druid/issues/9229 for details. + return new NoopQueryRunner<>(); } - String dataSourceName = getDataSourceName(dataSource); - final VersionedIntervalTimeline timeline = segmentManager.getTimeline( - dataSourceName - ); - - if (timeline == null) { - return new NoopQueryRunner(); - } - - FunctionalIterable> queryRunners = FunctionalIterable + FunctionalIterable segmentDescriptors = FunctionalIterable .create(intervals) + .transformCat(timeline::lookup) .transformCat( - new Function>>() - { - @Override - public Iterable> apply(Interval input) - { - return timeline.lookup(input); - } - } - ) - .transformCat( - new Function, Iterable>>() - { - @Override - public Iterable> apply( - @Nullable - final TimelineObjectHolder holder - ) - { - if (holder == null) { - return null; - } - - return FunctionalIterable - .create(holder.getObject()) - .transform( - new Function, QueryRunner>() - { - @Override - public QueryRunner apply(PartitionChunk input) - { - return buildAndDecorateQueryRunner( - factory, - toolChest, - input.getObject(), - new SegmentDescriptor( - holder.getInterval(), - holder.getVersion(), - input.getChunkNumber() - ), - cpuTimeAccumulator - ); - } - } - ); + holder -> { + if (holder == null) { + return null; } + + return FunctionalIterable + .create(holder.getObject()) + .transform( + partitionChunk -> + new SegmentDescriptor( + holder.getInterval(), + holder.getVersion(), + partitionChunk.getChunkNumber() + ) + ); } ); - return CPUTimeMetricQueryRunner.safeBuild( - new FinalizeResultsQueryRunner<>( - toolChest.mergeResults(factory.mergeRunners(exec, queryRunners)), - toolChest - ), - toolChest, - emitter, - cpuTimeAccumulator, - true - ); - } - - private String getDataSourceName(DataSource dataSource) - { - return Iterables.getOnlyElement(dataSource.getNames()); + return getQueryRunnerForSegments(query, segmentDescriptors); } @Override @@ -221,52 +159,63 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable(); + return new NoopQueryRunner<>(); } final QueryToolChest> toolChest = factory.getToolchest(); + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(query.getDataSource()); + final AtomicLong cpuTimeAccumulator = new AtomicLong(0L); - String dataSourceName = getDataSourceName(query.getDataSource()); + final VersionedIntervalTimeline timeline; + final Optional> maybeTimeline = + segmentManager.getTimeline(analysis); - final VersionedIntervalTimeline timeline = segmentManager.getTimeline( - dataSourceName - ); + // Make sure this query type can handle the subquery, if present. + if (analysis.isQuery() && !toolChest.canPerformSubquery(((QueryDataSource) analysis.getDataSource()).getQuery())) { + throw new ISE("Cannot handle subquery: %s", analysis.getDataSource()); + } - if (timeline == null) { - return new NoopQueryRunner(); + if (maybeTimeline.isPresent()) { + timeline = maybeTimeline.get(); + } else { + // Note: this is not correct when there's a right or full outer join going on. + // See https://github.com/apache/druid/issues/9229 for details. + return new NoopQueryRunner<>(); } - final AtomicLong cpuTimeAccumulator = new AtomicLong(0L); + // We can't actually handle joins yet; throw an error if they are present. + if (!analysis.getPreJoinableClauses().isEmpty()) { + throw new ISE("Cannot handle 'join' dataSource"); + } FunctionalIterable> queryRunners = FunctionalIterable .create(specs) .transformCat( - new Function>>() - { - @Override - @SuppressWarnings("unchecked") - public Iterable> apply(SegmentDescriptor input) - { - - final PartitionHolder entry = timeline.findEntry( - input.getInterval(), input.getVersion() - ); - - if (entry == null) { - return Collections.singletonList( - new ReportTimelineMissingSegmentQueryRunner(input)); - } - - final PartitionChunk chunk = entry.getChunk(input.getPartitionNumber()); - if (chunk == null) { - return Collections.singletonList(new ReportTimelineMissingSegmentQueryRunner(input)); - } - - final ReferenceCountingSegment adapter = chunk.getObject(); - return Collections.singletonList( - buildAndDecorateQueryRunner(factory, toolChest, adapter, input, cpuTimeAccumulator) - ); + descriptor -> { + final PartitionHolder entry = timeline.findEntry( + descriptor.getInterval(), + descriptor.getVersion() + ); + + if (entry == null) { + return Collections.singletonList(new ReportTimelineMissingSegmentQueryRunner<>(descriptor)); } + + final PartitionChunk chunk = entry.getChunk(descriptor.getPartitionNumber()); + if (chunk == null) { + return Collections.singletonList(new ReportTimelineMissingSegmentQueryRunner<>(descriptor)); + } + + final ReferenceCountingSegment segment = chunk.getObject(); + return Collections.singletonList( + buildAndDecorateQueryRunner( + factory, + toolChest, + segment, + descriptor, + cpuTimeAccumulator + ) + ); } ); diff --git a/server/src/main/java/org/apache/druid/server/log/LoggingRequestLogger.java b/server/src/main/java/org/apache/druid/server/log/LoggingRequestLogger.java index a0af4cb2fa1b..bcb628a195b8 100644 --- a/server/src/main/java/org/apache/druid/server/log/LoggingRequestLogger.java +++ b/server/src/main/java/org/apache/druid/server/log/LoggingRequestLogger.java @@ -20,21 +20,16 @@ package org.apache.druid.server.log; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.query.DataSource; import org.apache.druid.query.Query; -import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.TableDataSource; -import org.apache.druid.query.UnionDataSource; import org.apache.druid.server.RequestLogLine; import org.slf4j.MDC; import java.io.IOException; import java.util.Map; -import java.util.stream.Collectors; public class LoggingRequestLogger implements RequestLogger { @@ -66,7 +61,7 @@ public void logNativeQuery(RequestLogLine requestLogLine) throws IOException final Query query = requestLogLine.getQuery(); MDC.put("queryId", query.getId()); MDC.put("sqlQueryId", StringUtils.nullToEmptyNonDruidDataString(query.getSqlQueryId())); - MDC.put("dataSource", findInnerDatasource(query).toString()); + MDC.put("dataSource", String.join(",", query.getDataSource().getTableNames())); MDC.put("queryType", query.getType()); MDC.put("isNested", String.valueOf(!(query.getDataSource() instanceof TableDataSource))); MDC.put("hasFilters", Boolean.toString(query.hasFilters())); @@ -119,30 +114,6 @@ public boolean isSetContextMDC() return setContextMDC; } - private Object findInnerDatasource(Query query) - { - DataSource _ds = query.getDataSource(); - if (_ds instanceof TableDataSource) { - return ((TableDataSource) _ds).getName(); - } - if (_ds instanceof QueryDataSource) { - return findInnerDatasource(((QueryDataSource) _ds).getQuery()); - } - if (_ds instanceof UnionDataSource) { - return Joiner.on(",") - .join( - ((UnionDataSource) _ds) - .getDataSources() - .stream() - .map(TableDataSource::getName) - .collect(Collectors.toList()) - ); - } else { - // should not come here - return query.getDataSource(); - } - } - @Override public String toString() { diff --git a/server/src/main/java/org/apache/druid/server/router/TieredBrokerHostSelector.java b/server/src/main/java/org/apache/druid/server/router/TieredBrokerHostSelector.java index 96473e1614e2..08ce78ff0fbc 100644 --- a/server/src/main/java/org/apache/druid/server/router/TieredBrokerHostSelector.java +++ b/server/src/main/java/org/apache/druid/server/router/TieredBrokerHostSelector.java @@ -201,7 +201,7 @@ 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)); + List rules = ruleManager.getRulesWithDefault(Iterables.getFirst(query.getDataSource().getTableNames(), null)); // find the rule that can apply to the entire set of intervals DateTime now = DateTimes.nowUtc(); diff --git a/server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java b/server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java index b630b8ec6c31..dd3961e04f42 100644 --- a/server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java +++ b/server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java @@ -39,6 +39,7 @@ import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.QueryWatcher; import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.segment.TestHelper; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; @@ -114,7 +115,9 @@ public void testSingleServerAddedRemovedSegment() throws Exception Assert.assertTrue(timing.forWaiting().awaitLatch(segmentViewInitLatch)); Assert.assertTrue(timing.forWaiting().awaitLatch(segmentAddedLatch)); - TimelineLookup timeline = brokerServerView.getTimeline(new TableDataSource("test_broker_server_view")); + TimelineLookup timeline = brokerServerView.getTimeline( + DataSourceAnalysis.forDataSource(new TableDataSource("test_broker_server_view")) + ).get(); List serverLookupRes = (List) timeline.lookup( Intervals.of( "2014-10-20T00:00:00Z/P1D" @@ -203,7 +206,9 @@ public DataSegment apply(Pair input) Assert.assertTrue(timing.forWaiting().awaitLatch(segmentViewInitLatch)); Assert.assertTrue(timing.forWaiting().awaitLatch(segmentAddedLatch)); - TimelineLookup timeline = brokerServerView.getTimeline(new TableDataSource("test_broker_server_view")); + TimelineLookup timeline = brokerServerView.getTimeline( + DataSourceAnalysis.forDataSource(new TableDataSource("test_broker_server_view")) + ).get(); assertValues( Arrays.asList( createExpected("2011-04-01/2011-04-02", "v3", druidServers.get(4), segments.get(4)), @@ -224,7 +229,9 @@ public DataSegment apply(Pair input) // renew segmentRemovedLatch since we still have 4 segments to unannounce segmentRemovedLatch = new CountDownLatch(4); - timeline = brokerServerView.getTimeline(new TableDataSource("test_broker_server_view")); + timeline = brokerServerView.getTimeline( + DataSourceAnalysis.forDataSource(new TableDataSource("test_broker_server_view")) + ).get(); assertValues( Arrays.asList( createExpected("2011-04-01/2011-04-02", "v3", druidServers.get(4), segments.get(4)), diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java index a75ad64e080c..3efe7bc5d759 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java @@ -38,7 +38,6 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; -import org.apache.druid.query.DataSource; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.Druids; import org.apache.druid.query.Query; @@ -47,8 +46,10 @@ import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.context.ResponseContext; +import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.TimelineLookup; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.NoneShardSpec; import org.apache.druid.timeline.partition.SingleElementPartitionChunk; @@ -64,11 +65,13 @@ import java.util.Collections; import java.util.Comparator; import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.concurrent.Executor; import java.util.concurrent.ForkJoinPool; /** + * */ public class CachingClusteredClientFunctionalityTest { @@ -245,9 +248,9 @@ public void registerSegmentCallback(Executor exec, SegmentCallback callback) } @Override - public VersionedIntervalTimeline getTimeline(DataSource dataSource) + public Optional> getTimeline(DataSourceAnalysis analysis) { - return timeline; + return Optional.of(timeline); } @Nullable diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index bd91104aac41..9c0588837ffd 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -68,7 +68,6 @@ import org.apache.druid.java.util.common.guava.nary.TrinaryFn; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.BySegmentResultValueClass; -import org.apache.druid.query.DataSource; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.Druids; import org.apache.druid.query.FinalizeResultsQueryRunner; @@ -98,6 +97,7 @@ import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.query.ordering.StringComparators; +import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.query.search.SearchHit; import org.apache.druid.query.search.SearchQuery; import org.apache.druid.query.search.SearchQueryConfig; @@ -148,6 +148,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Random; import java.util.TreeMap; import java.util.concurrent.Callable; @@ -2391,9 +2392,9 @@ public void registerSegmentCallback(Executor exec, SegmentCallback callback) } @Override - public VersionedIntervalTimeline getTimeline(DataSource dataSource) + public Optional> getTimeline(DataSourceAnalysis analysis) { - return timeline; + return Optional.of(timeline); } @Override diff --git a/server/src/test/java/org/apache/druid/server/ClientInfoResourceTest.java b/server/src/test/java/org/apache/druid/server/ClientInfoResourceTest.java index 60b633d22624..f6c880c0e205 100644 --- a/server/src/test/java/org/apache/druid/server/ClientInfoResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/ClientInfoResourceTest.java @@ -30,8 +30,8 @@ import org.apache.druid.client.selector.RandomServerSelectorStrategy; import org.apache.druid.client.selector.ServerSelector; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.query.TableDataSource; import org.apache.druid.query.metadata.SegmentMetadataQueryConfig; +import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.timeline.DataSegment; @@ -48,6 +48,7 @@ import java.util.List; import java.util.Map; +import java.util.Optional; public class ClientInfoResourceTest { @@ -128,7 +129,8 @@ public void setup() EasyMock.expect(serverInventoryView.getInventory()).andReturn(ImmutableList.of(server)).anyTimes(); timelineServerView = EasyMock.createMock(TimelineServerView.class); - EasyMock.expect(timelineServerView.getTimeline(EasyMock.anyObject(TableDataSource.class))).andReturn(timeline); + EasyMock.expect(timelineServerView.getTimeline(EasyMock.anyObject(DataSourceAnalysis.class))) + .andReturn((Optional) Optional.of(timeline)); EasyMock.replay(serverInventoryView, timelineServerView); diff --git a/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java index b6c9c1671fc3..04f796e9918d 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java @@ -24,6 +24,8 @@ import com.google.common.collect.Ordering; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.MapUtils; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.segment.AbstractSegment; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.ReferenceCountingSegment; @@ -49,6 +51,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -382,7 +385,10 @@ public void testRemoveEmptyTimeline() throws SegmentLoadingException @Test public void testGetNonExistingTimeline() { - Assert.assertNull(segmentManager.getTimeline("nonExisting")); + Assert.assertEquals( + Optional.empty(), + segmentManager.getTimeline(DataSourceAnalysis.forDataSource(new TableDataSource("nonExisting"))) + ); } @Test @@ -448,7 +454,10 @@ private void assertResult(List expectedExistingSegments) throws Seg dataSources.forEach( (sourceName, dataSourceState) -> { Assert.assertEquals(expectedDataSourceCounts.get(sourceName).longValue(), dataSourceState.getNumSegments()); - Assert.assertEquals(expectedDataSourceSizes.get(sourceName).longValue(), dataSourceState.getTotalSegmentSize()); + Assert.assertEquals( + expectedDataSourceSizes.get(sourceName).longValue(), + dataSourceState.getTotalSegmentSize() + ); Assert.assertEquals( expectedDataSources.get(sourceName).getAllTimelineEntries(), dataSourceState.getTimeline().getAllTimelineEntries() diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidOuterQueryRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidOuterQueryRel.java index a1b1910a2ef2..934361a8b40d 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidOuterQueryRel.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidOuterQueryRel.java @@ -41,6 +41,7 @@ import javax.annotation.Nullable; import java.util.List; +import java.util.Set; /** * DruidRel that uses a "query" dataSource. @@ -199,9 +200,9 @@ public RelNode copy(final RelTraitSet traitSet, final List inputs) } @Override - public List getDataSourceNames() + public Set getDataSourceNames() { - return ((DruidRel) sourceRel).getDataSourceNames(); + return ((DruidRel) sourceRel).getDataSourceNames(); } @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQueryRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQueryRel.java index 3deb8e415539..fed6d886a6f7 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQueryRel.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQueryRel.java @@ -35,7 +35,7 @@ import org.apache.druid.sql.calcite.table.DruidTable; import javax.annotation.Nonnull; -import java.util.List; +import java.util.Set; /** * DruidRel that uses a "table" dataSource. @@ -123,9 +123,9 @@ public DruidQueryRel asDruidConvention() } @Override - public List getDataSourceNames() + public Set getDataSourceNames() { - return druidTable.getDataSource().getNames(); + return druidTable.getDataSource().getTableNames(); } @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidRel.java index c5549ddb1a11..9f56e2f58142 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidRel.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidRel.java @@ -26,7 +26,7 @@ import org.apache.druid.sql.calcite.planner.PlannerContext; import javax.annotation.Nullable; -import java.util.List; +import java.util.Set; public abstract class DruidRel extends AbstractRelNode { @@ -110,7 +110,7 @@ public PlannerContext getPlannerContext() public abstract T asDruidConvention(); /** - * Get a list of names of datasources read by this DruidRel + * Get the set of names of table datasources read by this DruidRel */ - public abstract List getDataSourceNames(); + public abstract Set getDataSourceNames(); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidSemiJoin.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidSemiJoin.java index 6248a360dfdb..e31437b69622 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidSemiJoin.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidSemiJoin.java @@ -171,13 +171,13 @@ public DruidSemiJoin asDruidConvention() } @Override - public List getDataSourceNames() + public Set getDataSourceNames() { final DruidRel druidRight = (DruidRel) this.right; - Set datasourceNames = new LinkedHashSet<>(); - datasourceNames.addAll(left.getDataSourceNames()); - datasourceNames.addAll(druidRight.getDataSourceNames()); - return new ArrayList<>(datasourceNames); + Set dataSourceNames = new LinkedHashSet<>(); + dataSourceNames.addAll(left.getDataSourceNames()); + dataSourceNames.addAll(druidRight.getDataSourceNames()); + return dataSourceNames; } @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidUnionRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidUnionRel.java index 41f08098b76b..2fe3d2ca0271 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidUnionRel.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidUnionRel.java @@ -37,6 +37,7 @@ import javax.annotation.Nullable; import java.util.ArrayList; import java.util.List; +import java.util.Set; import java.util.stream.Collectors; public class DruidUnionRel extends DruidRel @@ -166,12 +167,11 @@ public RelNode copy(final RelTraitSet traitSet, final List inputs) } @Override - public List getDataSourceNames() + public Set getDataSourceNames() { return rels.stream() .flatMap(rel -> ((DruidRel) rel).getDataSourceNames().stream()) - .distinct() - .collect(Collectors.toList()); + .collect(Collectors.toSet()); } @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/QueryMaker.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/QueryMaker.java index 1910bc27b654..5aacb8773f37 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/QueryMaker.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/QueryMaker.java @@ -37,8 +37,9 @@ import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.math.expr.Evals; import org.apache.druid.query.Query; -import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.QueryToolChest; +import org.apache.druid.query.planning.DataSourceAnalysis; +import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.column.ColumnHolder; @@ -48,6 +49,7 @@ import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.joda.time.DateTime; +import org.joda.time.Interval; import java.io.IOException; import java.util.Collection; @@ -87,8 +89,7 @@ public Sequence runQuery(final DruidQuery druidQuery) final Query query = druidQuery.getQuery(); if (plannerContext.getPlannerConfig().isRequireTimeCondition()) { - final Query innerMostQuery = findInnerMostQuery(query); - if (innerMostQuery.getIntervals().equals(Intervals.ONLY_ETERNITY)) { + if (Intervals.ONLY_ETERNITY.equals(findBaseDataSourceIntervals(query))) { throw new CannotBuildQueryException( "requireTimeCondition is enabled, all queries must include a filter condition on the __time column" ); @@ -121,13 +122,12 @@ public Sequence runQuery(final DruidQuery druidQuery) ); } - private Query findInnerMostQuery(Query outerQuery) + private List findBaseDataSourceIntervals(Query query) { - Query query = outerQuery; - while (query.getDataSource() instanceof QueryDataSource) { - query = ((QueryDataSource) query.getDataSource()).getQuery(); - } - return query; + return DataSourceAnalysis.forDataSource(query.getDataSource()) + .getBaseQuerySegmentSpec() + .map(QuerySegmentSpec::getIntervals) + .orElse(query.getIntervals()); } private Sequence execute(Query query, final List newFields, final List newTypes) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestServerInventoryView.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestServerInventoryView.java index 0f36273035a7..a8e498beb1f0 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestServerInventoryView.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestServerInventoryView.java @@ -26,8 +26,8 @@ import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.client.TimelineServerView; import org.apache.druid.client.selector.ServerSelector; -import org.apache.druid.query.DataSource; import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.timeline.DataSegment; @@ -37,6 +37,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.concurrent.Executor; /** @@ -77,7 +78,7 @@ public TestServerInventoryView(List segments, List rea } @Override - public TimelineLookup getTimeline(DataSource dataSource) + public Optional> getTimeline(DataSourceAnalysis analysis) { throw new UnsupportedOperationException(); }