Use DataSourceAnalysis throughout the query stack. (#9239)

Builds on #9235, using the datasource analysis functionality to replace various ad-hoc
approaches. The most interesting changes are in ClientQuerySegmentWalker (brokers),
ServerManager (historicals), and SinkQuerySegmentWalker (indexing tasks).

Other changes related to improving how we analyze queries:

1) Changes TimelineServerView to return an Optional timeline, which I thought made
   the analysis changes cleaner to implement.
2) Added QueryToolChest#canPerformSubquery, which is now used by query entry points to
   determine whether it is safe to pass a subquery dataSource to the query toolchest.
   Fixes an issue introduced in #5471 where subqueries under non-groupBy-typed queries
   were silently ignored, since neither the query entry point nor the toolchest did
   anything special with them.
3) Removes the QueryPlus.withQuerySegmentSpec method, which was mostly being used in
   error-prone ways (ignoring any potential subqueries, and not verifying that the
   underlying data source is actually a table). Replaces with a new function,
   Queries.withSpecificSegments, that includes sanity checks.
This commit is contained in:
Gian Merlino 2020-01-23 14:07:14 -08:00 committed by GitHub
parent 479c09751c
commit f0f68570ec
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
38 changed files with 709 additions and 403 deletions

View File

@ -25,7 +25,6 @@ import com.google.common.base.Preconditions;
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.io.Closer;
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.ResultRow;
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.TearDown;
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.HashMap;
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 class CachingClusteredClientBenchmark
.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 @@ public class CachingClusteredClientBenchmark
.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(selector));
}
@Nullable
@Override
public TimelineLookup<String, ServerSelector> getTimeline(DataSource dataSource)
public Optional<? extends TimelineLookup<String, ServerSelector>> getTimeline(DataSourceAnalysis analysis)
{
final String table = Iterables.getOnlyElement(dataSource.getTableNames());
return timelines.get(table);
return Optional.ofNullable(timelines.get(analysis.getBaseTableDataSource().get().getName()));
}
@Override
@ -563,7 +569,11 @@ public class CachingClusteredClientBenchmark
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);

View File

@ -23,9 +23,11 @@ import com.google.common.collect.ImmutableMap;
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<String, AtomicLong> hitCount = new ConcurrentHashMap<>();
private ConcurrentHashMap<String, AtomicLong> costTime = new ConcurrentHashMap<>();
private ConcurrentHashMap<String, ConcurrentHashMap<Set<String>, 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<Query> optimize(Query query)
{
@ -86,7 +88,7 @@ public class DataSourceOptimizer
// get all derivatives for datasource in query. The derivatives set is sorted by average size of
// per segment granularity.
Set<DerivativeDataSource> derivatives = DerivativeDataSourceManager.getDerivatives(datasourceName);
if (derivatives.isEmpty()) {
return Collections.singletonList(query);
}
@ -96,10 +98,10 @@ public class DataSourceOptimizer
hitCount.putIfAbsent(datasourceName, new AtomicLong(0));
costTime.putIfAbsent(datasourceName, new AtomicLong(0));
totalCount.get(datasourceName).incrementAndGet();
// get all fields which the query required
Set<String> requiredFields = MaterializedViewUtils.getRequiredFields(query);
Set<DerivativeDataSource> derivativesWithRequiredFields = new HashSet<>();
for (DerivativeDataSource derivativeDataSource : derivatives) {
derivativesHitCount.putIfAbsent(derivativeDataSource.getName(), new AtomicLong(0));
@ -115,14 +117,15 @@ public class DataSourceOptimizer
costTime.get(datasourceName).addAndGet(System.currentTimeMillis() - start);
return Collections.singletonList(query);
}
List<Query> queries = new ArrayList<>();
List<Interval> remainingQueryIntervals = (List<Interval>) query.getIntervals();
for (DerivativeDataSource derivativeDataSource : ImmutableSortedSet.copyOf(derivativesWithRequiredFields)) {
final List<Interval> 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 class DataSourceOptimizer
if (derivativeIntervals.isEmpty()) {
continue;
}
remainingQueryIntervals = MaterializedViewUtils.minus(remainingQueryIntervals, derivativeIntervals);
queries.add(
query.withDataSource(new TableDataSource(derivativeDataSource.getName()))
@ -158,13 +161,13 @@ public class DataSourceOptimizer
hitCount.get(datasourceName).incrementAndGet();
costTime.get(datasourceName).addAndGet(System.currentTimeMillis() - start);
return queries;
}
}
finally {
lock.readLock().unlock();
}
}
public List<DataSourceOptimizerStats> getAndResetStats()
public List<DataSourceOptimizerStats> getAndResetStats()
{
ImmutableMap<String, AtomicLong> derivativesHitCountSnapshot;
ImmutableMap<String, AtomicLong> totalCountSnapshot;
@ -183,7 +186,7 @@ public class DataSourceOptimizer
hitCount.clear();
costTime.clear();
missFields.clear();
}
}
finally {
lock.writeLock().unlock();
}

View File

@ -49,7 +49,6 @@ import org.apache.druid.java.util.common.guava.Sequence;
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.SegmentDescriptor;
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.nio.charset.StandardCharsets;
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 class MovingAverageQueryTest extends InitializedNullHandlingTest
new TimelineServerView()
{
@Override
public TimelineLookup<String, ServerSelector> getTimeline(DataSource dataSource)
public Optional<? extends TimelineLookup<String, ServerSelector>> getTimeline(DataSourceAnalysis analysis)
{
return null;
return Optional.empty();
}
@Override

View File

@ -22,7 +22,6 @@ package org.apache.druid.indexing.overlord;
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.Query;
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 class SingleTaskBackgroundRunner implements TaskRunner, QuerySegmentWalke
private <T> QueryRunner<T> getQueryRunnerImpl(Query<T> query)
{
QueryRunner<T> queryRunner = null;
final String queryDataSource = Iterables.getOnlyElement(query.getDataSource().getTableNames());
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<T> taskQueryRunner = task.getQueryRunner(query);
if (taskQueryRunner != null) {
@ -379,7 +381,7 @@ public class SingleTaskBackgroundRunner implements TaskRunner, QuerySegmentWalke
{
return task.getType();
}
@Override
public String getDataSource()
{

View File

@ -28,6 +28,7 @@ import org.apache.druid.guice.annotations.ExtensionPoint;
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 abstract class BaseQuery<T> implements Query<T>
}
@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

View File

@ -23,8 +23,11 @@ import com.google.common.base.Preconditions;
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,9 +36,6 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
/**
*
*/
@PublicApi
public class Queries
{
@ -131,4 +131,46 @@ public class Queries
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 <T> Query<T> withSpecificSegments(final Query<T> query, final List<SegmentDescriptor> descriptors)
{
final Query<T> 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;
}
}

View File

@ -116,6 +116,12 @@ public interface Query<T>
Query<T> withOverriddenContext(Map<String, Object> 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<T> withQuerySegmentSpec(QuerySegmentSpec spec);
Query<T> withId(String id);
@ -140,14 +146,4 @@ public interface Query<T>
{
return this;
}
default List<Interval> getIntervalsOfInnerMostQuery()
{
if (getDataSource() instanceof QueryDataSource) {
//noinspection unchecked
return ((QueryDataSource) getDataSource()).getQuery().getIntervalsOfInnerMostQuery();
} else {
return getIntervals();
}
}
}

View File

@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableMap;
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 @@ public final class QueryPlus<T>
}
}
/**
* Equivalent of withQuery(getQuery().withQuerySegmentSpec(spec)).
*/
public QueryPlus<T> withQuerySegmentSpec(QuerySegmentSpec spec)
{
return new QueryPlus<>(query.withQuerySegmentSpec(spec), queryMetrics, identity);
}
/**
* Equivalent of withQuery(getQuery().withOverriddenContext(ImmutableMap.of(MAX_QUEUED_BYTES_KEY, maxQueuedBytes))).
*/

View File

@ -22,6 +22,7 @@ package org.apache.druid.query;
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 <T> query result type
* @param query the query to find a Queryable for
* @param <T> 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
*/
<T> QueryRunner<T> getQueryRunnerForIntervals(Query<T> query, Iterable<Interval> intervals);
/**
* Gets the Queryable for a given list of SegmentSpecs.
* Gets the Queryable for a given list of SegmentDescriptors.
*
* @param <T> the query result type
* The descriptors are expected to apply to the base datasource involved in the query, i.e. the one returned by:
*
* <pre>
* DataSourceAnalysis.forDataSource(query.getDataSource()).getBaseDataSource()
* </pre>
*
* @param <T> 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
*/
<T> QueryRunner<T> getQueryRunnerForSegments(Query<T> query, Iterable<SegmentDescriptor> specs);

View File

@ -271,6 +271,20 @@ public abstract class QueryToolChest<ResultType, QueryType extends Query<ResultT
return segments;
}
/**
* Returns whether this toolchest is able to handle the provided subquery.
*
* When this method returns true, the core query stack will pass subquery datasources over to the toolchest and will
* assume they are properly handled.
*
* When this method returns false, the core query stack will throw an error if subqueries are present. In the future,
* instead of throwing an error, the core query stack will handle the subqueries on its own.
*/
public boolean canPerformSubquery(final Query<?> 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}.

View File

@ -30,7 +30,6 @@ import org.apache.druid.java.util.common.guava.YieldingAccumulator;
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 class RetryQueryRunner<T> implements QueryRunner<T>
log.info("[%,d] missing segments found. Retry attempt [%,d]", missingSegments.size(), i);
context.put(ResponseContext.Key.MISSING_SEGMENTS, new ArrayList<>());
final QueryPlus<T> retryQueryPlus = queryPlus.withQuerySegmentSpec(
new MultipleSpecificSegmentSpec(
missingSegments
)
final QueryPlus<T> retryQueryPlus = queryPlus.withQuery(
Queries.withSpecificSegments(queryPlus.getQuery(), missingSegments)
);
Sequence<T> retrySequence = baseRunner.run(retryQueryPlus, context);
listOfSequences.add(retrySequence);

View File

@ -92,8 +92,11 @@ public class TimewarpOperator<T> implements PostProcessingOperator<T>
);
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<T, T>()

View File

@ -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.HashMap;
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 @@ public class DimFilterUtils
* {@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 <T> This can be any type, as long as transform function is provided to convert this to ShardSpec
* @param <T> 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 <T> Set<T> filterShards(DimFilter dimFilter, Iterable<T> input, Function<T, ShardSpec> converter)
{
return filterShards(dimFilter, input, converter, new HashMap<String, Optional<RangeSet<String>>>());
return filterShards(dimFilter, input, converter, new HashMap<>());
}
/**
@ -106,15 +108,20 @@ public class DimFilterUtils
* 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 <T> This can be any type, as long as transform function is provided to convert this to ShardSpec
* @param <T> 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 <T> Set<T> filterShards(DimFilter dimFilter, Iterable<T> input, Function<T, ShardSpec> converter,
Map<String, Optional<RangeSet<String>>> dimensionRangeCache)
public static <T> Set<T> filterShards(
final DimFilter dimFilter,
final Iterable<T> input,
final Function<T, ShardSpec> converter,
final Map<String, Optional<RangeSet<String>>> dimensionRangeCache
)
{
Set<T> retSet = new LinkedHashSet<>();
@ -127,7 +134,7 @@ public class DimFilterUtils
List<String> dimensions = shard.getDomainDimensions();
for (String dimension : dimensions) {
Optional<RangeSet<String>> 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());

View File

@ -663,6 +663,26 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<ResultRow, GroupB
};
}
@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<String> resultArrayFields(final GroupByQuery query)
{

View File

@ -30,6 +30,7 @@ import org.apache.druid.query.SegmentDescriptor;
import org.joda.time.Interval;
import java.util.List;
import java.util.Objects;
/**
*/
@ -93,24 +94,13 @@ public class MultipleSpecificSegmentSpec implements QuerySegmentSpec
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);
}
}

View File

@ -28,6 +28,7 @@ import org.apache.druid.java.util.common.guava.Sequences;
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.io.IOException;
import java.util.Collections;
/**
*
*/
public class SpecificSegmentQueryRunner<T> implements QueryRunner<T>
{
@ -56,7 +58,13 @@ public class SpecificSegmentQueryRunner<T> implements QueryRunner<T>
@Override
public Sequence<T> run(final QueryPlus<T> input, final ResponseContext responseContext)
{
final QueryPlus<T> queryPlus = input.withQuerySegmentSpec(specificSpec);
final QueryPlus<T> queryPlus = input.withQuery(
Queries.withSpecificSegments(
input.getQuery(),
Collections.singletonList(specificSpec.getDescriptor())
)
);
final Query<T> query = queryPlus.getQuery();
final Thread currThread = Thread.currentThread();

View File

@ -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.PostAggregator;
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 class QueriesTest
Assert.assertTrue(exceptionOccured);
}
@Test
public void testWithSpecificSegmentsBasic()
{
final ImmutableList<SegmentDescriptor> 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<SegmentDescriptor> 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<SegmentDescriptor> 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<Result<TimeseriesResultValue>> ignored = Queries.withSpecificSegments(query, descriptors);
}
}

View File

@ -471,12 +471,14 @@ public class QueryRunnerTestHelper
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
)
)
)
);

View File

@ -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.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
public class DimFilterUtilsTest

View File

@ -31,6 +31,8 @@ import org.apache.druid.java.util.common.StringUtils;
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 class GroupByQueryQueryToolChestTest
);
}
@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) {

View File

@ -2983,11 +2983,15 @@ public class GroupByQueryRunnerTest extends InitializedNullHandlingTest
public Sequence<ResultRow> run(QueryPlus<ResultRow> 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 @@ public class GroupByQueryRunnerTest extends InitializedNullHandlingTest
public Sequence<ResultRow> run(QueryPlus<ResultRow> 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 class GroupByQueryRunnerTest extends InitializedNullHandlingTest
public Sequence<ResultRow> run(QueryPlus<ResultRow> 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 class GroupByQueryRunnerTest extends InitializedNullHandlingTest
public Sequence<ResultRow> run(QueryPlus<ResultRow> 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 class GroupByQueryRunnerTest extends InitializedNullHandlingTest
public Sequence<ResultRow> run(QueryPlus<ResultRow> 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 class GroupByQueryRunnerTest extends InitializedNullHandlingTest
public Sequence<ResultRow> run(QueryPlus<ResultRow> queryPlus, ResponseContext responseContext)
{
// simulate two daily segments
final QueryPlus<ResultRow> queryPlus1 = queryPlus.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03")))
final QueryPlus<ResultRow> queryPlus1 = queryPlus.withQuery(
queryPlus.getQuery().withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03")))
)
);
final QueryPlus<ResultRow> queryPlus2 = queryPlus.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04")))
final QueryPlus<ResultRow> 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 class GroupByQueryRunnerTest extends InitializedNullHandlingTest
public Sequence<ResultRow> run(QueryPlus<ResultRow> queryPlus, ResponseContext responseContext)
{
// simulate two daily segments
final QueryPlus<ResultRow> queryPlus1 = queryPlus.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03")))
final QueryPlus<ResultRow> queryPlus1 = queryPlus.withQuery(
queryPlus.getQuery().withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03")))
)
);
final QueryPlus<ResultRow> queryPlus2 = queryPlus.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04")))
final QueryPlus<ResultRow> 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 class GroupByQueryRunnerTest extends InitializedNullHandlingTest
public Sequence<ResultRow> run(QueryPlus<ResultRow> queryPlus, ResponseContext responseContext)
{
// simulate two daily segments
final QueryPlus<ResultRow> queryPlus1 = queryPlus.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03")))
final QueryPlus<ResultRow> queryPlus1 = queryPlus.withQuery(
queryPlus.getQuery().withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03")))
)
);
final QueryPlus<ResultRow> queryPlus2 = queryPlus.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04")))
final QueryPlus<ResultRow> 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 class GroupByQueryRunnerTest extends InitializedNullHandlingTest
public Sequence<ResultRow> run(QueryPlus<ResultRow> queryPlus, ResponseContext responseContext)
{
// simulate two daily segments
final QueryPlus<ResultRow> queryPlus1 = queryPlus.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03")))
final QueryPlus<ResultRow> queryPlus1 = queryPlus.withQuery(
queryPlus.getQuery().withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-02/2011-04-03")))
)
);
final QueryPlus<ResultRow> queryPlus2 = queryPlus.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04")))
final QueryPlus<ResultRow> queryPlus2 = queryPlus.withQuery(
queryPlus.getQuery().withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-03/2011-04-04")))
)
);
return factory.getToolchest().mergeResults(

View File

@ -72,6 +72,7 @@ import java.util.Collections;
import java.util.List;
/**
*
*/
@RunWith(Parameterized.class)
public class SearchQueryRunnerTest extends InitializedNullHandlingTest
@ -167,11 +168,15 @@ public class SearchQueryRunnerTest extends InitializedNullHandlingTest
ResponseContext responseContext
)
{
final QueryPlus<Result<SearchResultValue>> queryPlus1 = queryPlus.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-01-12/2011-02-28")))
final QueryPlus<Result<SearchResultValue>> queryPlus1 = queryPlus.withQuery(
queryPlus.getQuery().withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-01-12/2011-02-28")))
)
);
final QueryPlus<Result<SearchResultValue>> queryPlus2 = queryPlus.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-03-01/2011-04-15")))
final QueryPlus<Result<SearchResultValue>> 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));
}

View File

@ -21,7 +21,6 @@ package org.apache.druid.client;
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.client.selector.TierSelectorStrategy;
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;
@ -289,14 +290,15 @@ public class BrokerServerView implements TimelineServerView
}
}
@Nullable
@Override
public VersionedIntervalTimeline<String, ServerSelector> getTimeline(DataSource dataSource)
public Optional<VersionedIntervalTimeline<String, ServerSelector>> getTimeline(final DataSourceAnalysis analysis)
{
String table = Iterables.getOnlyElement(dataSource.getTableNames());
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()));
}
}

View File

@ -22,7 +22,6 @@ package org.apache.druid.client;
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.java.util.emitter.EmittingLogger;
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.SegmentDescriptor;
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.Iterator;
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.function.UnaryOperator;
import java.util.stream.Collectors;
/**
*
*/
public class CachingClusteredClient implements QuerySegmentWalker
{
@ -231,6 +234,7 @@ public class CachingClusteredClient implements QuerySegmentWalker
private final int uncoveredIntervalsLimit;
private final Query<T> downstreamQuery;
private final Map<String, Cache.NamedKey> cachePopulatorKeyMap = new HashMap<>();
private final DataSourceAnalysis dataSourceAnalysis;
private final List<Interval> intervals;
SpecificQueryRunnable(final QueryPlus<T> queryPlus, final ResponseContext responseContext)
@ -248,8 +252,11 @@ public class CachingClusteredClient implements QuerySegmentWalker
// 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<String, Object> makeDownstreamQueryContext()
@ -269,12 +276,14 @@ public class CachingClusteredClient implements QuerySegmentWalker
Sequence<T> run(final UnaryOperator<TimelineLookup<String, ServerSelector>> timelineConverter)
{
@Nullable
TimelineLookup<String, ServerSelector> timeline = serverView.getTimeline(query.getDataSource());
if (timeline == null) {
final Optional<? extends TimelineLookup<String, ServerSelector>> maybeTimeline = serverView.getTimeline(
dataSourceAnalysis
);
if (!maybeTimeline.isPresent()) {
return Sequences.empty();
}
timeline = timelineConverter.apply(timeline);
final TimelineLookup<String, ServerSelector> timeline = timelineConverter.apply(maybeTimeline.get());
if (uncoveredIntervalsLimit > 0) {
computeUncoveredIntervals(timeline);
}
@ -598,19 +607,17 @@ public class CachingClusteredClient implements QuerySegmentWalker
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<T> 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 @@ public class CachingClusteredClient implements QuerySegmentWalker
@SuppressWarnings("unchecked")
private Sequence<T> getBySegmentServerResults(
final QueryRunner serverRunner,
final MultipleSpecificSegmentSpec segmentsOfServerSpec,
final List<SegmentDescriptor> segmentsOfServer,
long maxQueuedBytesPerServer
)
{
Sequence<Result<BySegmentResultValueClass<T>>> 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 @@ public class CachingClusteredClient implements QuerySegmentWalker
@SuppressWarnings("unchecked")
private Sequence<T> getSimpleServerResults(
final QueryRunner serverRunner,
final MultipleSpecificSegmentSpec segmentsOfServerSpec,
final List<SegmentDescriptor> segmentsOfServer,
long maxQueuedBytesPerServer
)
{
return serverRunner.run(
queryPlus.withQuerySegmentSpec(segmentsOfServerSpec).withMaxQueuedBytes(maxQueuedBytesPerServer),
queryPlus.withQuery(
Queries.withSpecificSegments(queryPlus.getQuery(), segmentsOfServer)
).withMaxQueuedBytes(maxQueuedBytesPerServer),
responseContext
);
}
private Sequence<T> getAndCacheServerResults(
final QueryRunner serverRunner,
final MultipleSpecificSegmentSpec segmentsOfServerSpec,
final List<SegmentDescriptor> segmentsOfServer,
long maxQueuedBytesPerServer
)
{
@SuppressWarnings("unchecked")
final Sequence<Result<BySegmentResultValueClass<T>>> resultsBySegments = serverRunner.run(
queryPlus
.withQuery((Query<Result<BySegmentResultValueClass<T>>>) downstreamQuery)
.withQuerySegmentSpec(segmentsOfServerSpec)
.withQuery(
Queries.withSpecificSegments(
(Query<Result<BySegmentResultValueClass<T>>>) downstreamQuery,
segmentsOfServer
)
)
.withMaxQueuedBytes(maxQueuedBytesPerServer),
responseContext
);

View File

@ -24,6 +24,7 @@ import org.apache.druid.query.DataSource;
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 org.joda.time.Interval;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
/**
*/
@ -55,13 +57,14 @@ public class ServerViewUtil
int numCandidates
)
{
TimelineLookup<String, ServerSelector> timeline = serverView.getTimeline(datasource);
if (timeline == null) {
final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(datasource);
final Optional<? extends TimelineLookup<String, ServerSelector>> maybeTimeline = serverView.getTimeline(analysis);
if (!maybeTimeline.isPresent()) {
return Collections.emptyList();
}
List<LocatedSegmentDescriptor> located = new ArrayList<>();
for (Interval interval : intervals) {
for (TimelineObjectHolder<String, ServerSelector> holder : timeline.lookup(interval)) {
for (TimelineObjectHolder<String, ServerSelector> holder : maybeTimeline.get().lookup(interval)) {
for (PartitionChunk<ServerSelector> chunk : holder.getObject()) {
ServerSelector selector = chunk.getObject();
final SegmentDescriptor descriptor = new SegmentDescriptor(

View File

@ -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<String, ServerSelector> 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<? extends TimelineLookup<String, ServerSelector>> getTimeline(DataSourceAnalysis analysis);
/**
* Returns a list of {@link ImmutableDruidServer}

View File

@ -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.FinalizeResultsQueryRunner;
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;
@ -60,16 +60,19 @@ 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);
@ -118,40 +121,17 @@ public class SinkQuerySegmentWalker implements QuerySegmentWalker
{
final Iterable<SegmentDescriptor> specs = FunctionalIterable
.create(intervals)
.transformCat(sinkTimeline::lookup)
.transformCat(
new Function<Interval, Iterable<TimelineObjectHolder<String, Sink>>>()
{
@Override
public Iterable<TimelineObjectHolder<String, Sink>> apply(final Interval interval)
{
return sinkTimeline.lookup(interval);
}
}
)
.transformCat(
new Function<TimelineObjectHolder<String, Sink>, Iterable<SegmentDescriptor>>()
{
@Override
public Iterable<SegmentDescriptor> apply(final TimelineObjectHolder<String, Sink> holder)
{
return FunctionalIterable
.create(holder.getObject())
.transform(
new Function<PartitionChunk<Sink>, SegmentDescriptor>()
{
@Override
public SegmentDescriptor apply(final PartitionChunk<Sink> 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);
@ -161,16 +141,15 @@ public class SinkQuerySegmentWalker implements QuerySegmentWalker
public <T> QueryRunner<T> getQueryRunnerForSegments(final Query<T> query, final Iterable<SegmentDescriptor> 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<TableDataSource> 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());
}
// Sanity check: we cannot actually handle joins yet, so detect them and throw an error.
final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(query.getDataSource());
if (!analysis.getPreJoinableClauses().isEmpty()) {
throw new ISE("Cannot handle join dataSource");
}
@ -184,6 +163,11 @@ public class SinkQuerySegmentWalker implements QuerySegmentWalker
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());
}
Iterable<QueryRunner<T>> perSegmentRunners = Iterables.transform(
specs,
descriptor -> {

View File

@ -37,6 +37,7 @@ import org.apache.druid.java.util.common.logger.Logger;
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.Comparator;
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 @@ public class ClientInfoResource
theInterval = Intervals.of(interval);
}
TimelineLookup<String, ServerSelector> timeline = timelineServerView.getTimeline(new TableDataSource(dataSourceName));
Iterable<TimelineObjectHolder<String, ServerSelector>> serversLookup = timeline != null ? timeline.lookup(
theInterval
) : null;
if (serversLookup == null || Iterables.isEmpty(serversLookup)) {
return Collections.EMPTY_MAP;
final Optional<? extends TimelineLookup<String, ServerSelector>> maybeTimeline =
timelineServerView.getTimeline(DataSourceAnalysis.forDataSource(new TableDataSource(dataSourceName)));
final Optional<Iterable<TimelineObjectHolder<String, ServerSelector>>> maybeServersLookup =
maybeTimeline.map(timeline -> timeline.lookup(theInterval));
if (!maybeServersLookup.isPresent() || Iterables.isEmpty(maybeServersLookup.get())) {
return Collections.emptyMap();
}
Map<Interval, Object> servedIntervals = new TreeMap<>(
new Comparator<Interval>()
@ -174,7 +177,7 @@ public class ClientInfoResource
}
);
for (TimelineObjectHolder<String, ServerSelector> holder : serversLookup) {
for (TimelineObjectHolder<String, ServerSelector> holder : maybeServersLookup.get()) {
final Set<Object> dimensions = new HashSet<>();
final Set<Object> metrics = new HashSet<>();
final PartitionHolder<ServerSelector> partitionHolder = holder.getObject();

View File

@ -43,7 +43,7 @@ import org.apache.druid.server.initialization.ServerConfig;
import org.joda.time.Interval;
/**
*
* Query handler for Broker processes (see CliBroker).
*/
public class ClientQuerySegmentWalker implements QuerySegmentWalker
{
@ -56,7 +56,6 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
private final Cache cache;
private final CacheConfig cacheConfig;
@Inject
public ClientQuerySegmentWalker(
ServiceEmitter emitter,
@ -82,25 +81,27 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
@Override
public <T> QueryRunner<T> getQueryRunnerForIntervals(Query<T> query, Iterable<Interval> intervals)
{
// Sanity check: we cannot actually handle joins yet, so detect them and throw an error.
final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(query.getDataSource());
if (!analysis.getPreJoinableClauses().isEmpty()) {
throw new ISE("Cannot handle join dataSource");
}
return makeRunner(query, baseClient.getQueryRunnerForIntervals(query, intervals));
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 <T> QueryRunner<T> getQueryRunnerForSegments(Query<T> query, Iterable<SegmentDescriptor> specs)
{
// Sanity check: we cannot actually handle joins yet, so detect them and throw an error.
final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(query.getDataSource());
if (!analysis.getPreJoinableClauses().isEmpty()) {
throw new ISE("Cannot handle join dataSource");
}
return makeRunner(query, baseClient.getQueryRunnerForSegments(query, specs));
if (analysis.isConcreteTableBased()) {
return makeRunner(query, baseClient.getQueryRunnerForSegments(query, specs));
} else {
throw new ISE("Query dataSource is not table-based, cannot run");
}
}
private <T> QueryRunner<T> makeRunner(Query<T> query, QueryRunner<T> baseClientRunner)
@ -126,9 +127,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
{
PostProcessingOperator<T> postProcessing = objectMapper.convertValue(
query.<String>getContextValue("postProcessing"),
new TypeReference<PostProcessingOperator<T>>()
{
}
new TypeReference<PostProcessingOperator<T>>() {}
);
return new FluentQueryRunnerBuilder<>(toolChest)

View File

@ -23,7 +23,10 @@ import com.google.common.annotations.VisibleForTesting;
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.PartitionHolder;
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 class SegmentManager
return segmentLoader.isSegmentLoaded(segment);
}
@Nullable
public VersionedIntervalTimeline<String, ReferenceCountingSegment> 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<VersionedIntervalTimeline<String, ReferenceCountingSegment>> 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
*

View File

@ -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.EmittingLogger;
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,6 @@ import org.apache.druid.query.QueryToolChest;
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;
@ -61,17 +57,19 @@ import org.apache.druid.server.SegmentManager;
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
{
private static final EmittingLogger log = new EmittingLogger(ServerManager.class);
@ -111,110 +109,49 @@ public class ServerManager implements QuerySegmentWalker
this.serverConfig = serverConfig;
}
private DataSource getInnerMostDataSource(DataSource dataSource)
{
if (dataSource instanceof QueryDataSource) {
return getInnerMostDataSource(((QueryDataSource) dataSource).getQuery().getDataSource());
}
return dataSource;
}
@Override
public <T> QueryRunner<T> getQueryRunnerForIntervals(Query<T> query, Iterable<Interval> intervals)
{
final QueryRunnerFactory<T, Query<T>> factory = conglomerate.findFactory(query);
if (factory == null) {
throw new ISE("Unknown query type[%s].", query.getClass());
}
// Sanity check: we cannot actually handle joins yet, so detect them and throw an error.
final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(query.getDataSource());
if (!analysis.getPreJoinableClauses().isEmpty()) {
throw new ISE("Cannot handle join dataSource");
}
final QueryToolChest<T, Query<T>> toolChest = factory.getToolchest();
final AtomicLong cpuTimeAccumulator = new AtomicLong(0L);
final VersionedIntervalTimeline<String, ReferenceCountingSegment> timeline;
final Optional<VersionedIntervalTimeline<String, ReferenceCountingSegment>> maybeTimeline =
segmentManager.getTimeline(analysis);
DataSource dataSource = getInnerMostDataSource(query.getDataSource());
if (!(dataSource instanceof TableDataSource)) {
throw new UnsupportedOperationException("data source type '" + dataSource.getClass().getName() + "' unsupported");
}
String dataSourceName = getDataSourceName(dataSource);
final VersionedIntervalTimeline<String, ReferenceCountingSegment> timeline = segmentManager.getTimeline(
dataSourceName
);
if (timeline == null) {
return new NoopQueryRunner<T>();
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<>();
}
FunctionalIterable<QueryRunner<T>> queryRunners = FunctionalIterable
FunctionalIterable<SegmentDescriptor> segmentDescriptors = FunctionalIterable
.create(intervals)
.transformCat(timeline::lookup)
.transformCat(
new Function<Interval, Iterable<TimelineObjectHolder<String, ReferenceCountingSegment>>>()
{
@Override
public Iterable<TimelineObjectHolder<String, ReferenceCountingSegment>> apply(Interval input)
{
return timeline.lookup(input);
holder -> {
if (holder == null) {
return null;
}
}
)
.transformCat(
new Function<TimelineObjectHolder<String, ReferenceCountingSegment>, Iterable<QueryRunner<T>>>()
{
@Override
public Iterable<QueryRunner<T>> apply(
@Nullable final TimelineObjectHolder<String, ReferenceCountingSegment> holder
)
{
if (holder == null) {
return null;
}
return FunctionalIterable
.create(holder.getObject())
.transform(
new Function<PartitionChunk<ReferenceCountingSegment>, QueryRunner<T>>()
{
@Override
public QueryRunner<T> apply(PartitionChunk<ReferenceCountingSegment> input)
{
return buildAndDecorateQueryRunner(
factory,
toolChest,
input.getObject(),
new SegmentDescriptor(
holder.getInterval(),
holder.getVersion(),
input.getChunkNumber()
),
cpuTimeAccumulator
);
}
}
);
}
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.getTableNames());
return getQueryRunnerForSegments(query, segmentDescriptors);
}
@Override
@ -225,7 +162,7 @@ public class ServerManager implements QuerySegmentWalker
log.makeAlert("Unknown query type, [%s]", query.getClass())
.addData("dataSource", query.getDataSource())
.emit();
return new NoopQueryRunner<T>();
return new NoopQueryRunner<>();
}
// Sanity check: we cannot actually handle joins yet, so detect them and throw an error.
@ -235,48 +172,53 @@ public class ServerManager implements QuerySegmentWalker
}
final QueryToolChest<T, Query<T>> toolChest = factory.getToolchest();
final AtomicLong cpuTimeAccumulator = new AtomicLong(0L);
String dataSourceName = getDataSourceName(query.getDataSource());
final VersionedIntervalTimeline<String, ReferenceCountingSegment> timeline;
final Optional<VersionedIntervalTimeline<String, ReferenceCountingSegment>> maybeTimeline =
segmentManager.getTimeline(analysis);
final VersionedIntervalTimeline<String, ReferenceCountingSegment> timeline = segmentManager.getTimeline(
dataSourceName
);
if (timeline == null) {
return new NoopQueryRunner<T>();
// 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());
}
final AtomicLong cpuTimeAccumulator = new AtomicLong(0L);
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<>();
}
FunctionalIterable<QueryRunner<T>> queryRunners = FunctionalIterable
.create(specs)
.transformCat(
new Function<SegmentDescriptor, Iterable<QueryRunner<T>>>()
{
@Override
@SuppressWarnings("unchecked")
public Iterable<QueryRunner<T>> apply(SegmentDescriptor input)
{
descriptor -> {
final PartitionHolder<ReferenceCountingSegment> entry = timeline.findEntry(
descriptor.getInterval(),
descriptor.getVersion()
);
final PartitionHolder<ReferenceCountingSegment> entry = timeline.findEntry(
input.getInterval(), input.getVersion()
);
if (entry == null) {
return Collections.singletonList(
new ReportTimelineMissingSegmentQueryRunner<T>(input));
}
final PartitionChunk<ReferenceCountingSegment> chunk = entry.getChunk(input.getPartitionNumber());
if (chunk == null) {
return Collections.singletonList(new ReportTimelineMissingSegmentQueryRunner<T>(input));
}
final ReferenceCountingSegment adapter = chunk.getObject();
return Collections.singletonList(
buildAndDecorateQueryRunner(factory, toolChest, adapter, input, cpuTimeAccumulator)
);
if (entry == null) {
return Collections.singletonList(new ReportTimelineMissingSegmentQueryRunner<>(descriptor));
}
final PartitionChunk<ReferenceCountingSegment> 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
)
);
}
);

View File

@ -39,6 +39,7 @@ import org.apache.druid.java.util.http.client.HttpClient;
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 class BrokerServerViewTest extends CuratorTestBase
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<TimelineObjectHolder> serverLookupRes = (List<TimelineObjectHolder>) timeline.lookup(
Intervals.of(
"2014-10-20T00:00:00Z/P1D"
@ -203,7 +206,9 @@ public class BrokerServerViewTest extends CuratorTestBase
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 class BrokerServerViewTest extends CuratorTestBase
// 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)),

View File

@ -38,7 +38,6 @@ import org.apache.druid.java.util.common.Intervals;
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.QueryRunner;
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.io.IOException;
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 class CachingClusteredClientFunctionalityTest
}
@Override
public VersionedIntervalTimeline<String, ServerSelector> getTimeline(DataSource dataSource)
public Optional<? extends TimelineLookup<String, ServerSelector>> getTimeline(DataSourceAnalysis analysis)
{
return timeline;
return Optional.of(timeline);
}
@Nullable

View File

@ -68,7 +68,6 @@ import org.apache.druid.java.util.common.guava.Sequences;
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.GroupByQuery;
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.Collections;
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 class CachingClusteredClientTest
}
@Override
public VersionedIntervalTimeline<String, ServerSelector> getTimeline(DataSource dataSource)
public Optional<VersionedIntervalTimeline<String, ServerSelector>> getTimeline(DataSourceAnalysis analysis)
{
return timeline;
return Optional.of(timeline);
}
@Override

View File

@ -30,8 +30,8 @@ import org.apache.druid.client.selector.HighestPriorityTierSelectorStrategy;
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 org.junit.Test;
import java.util.List;
import java.util.Map;
import java.util.Optional;
public class ClientInfoResourceTest
{
@ -128,7 +129,8 @@ public class ClientInfoResourceTest
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);

View File

@ -24,6 +24,8 @@ import com.google.common.collect.ImmutableMap;
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.ArrayList;
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 class SegmentManagerTest
@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 @@ public class SegmentManagerTest
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()

View File

@ -37,8 +37,9 @@ import org.apache.druid.java.util.common.guava.Sequence;
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.server.security.AuthenticationResult;
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 class QueryMaker
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 class QueryMaker
);
}
private Query<?> findInnerMostQuery(Query outerQuery)
private List<Interval> 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 <T> Sequence<Object[]> execute(Query<T> query, final List<String> newFields, final List<SqlTypeName> newTypes)

View File

@ -26,8 +26,8 @@ import org.apache.druid.client.ImmutableDruidDataSource;
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 javax.annotation.Nullable;
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 class TestServerInventoryView implements TimelineServerView
}
@Override
public TimelineLookup<String, ServerSelector> getTimeline(DataSource dataSource)
public Optional<? extends TimelineLookup<String, ServerSelector>> getTimeline(DataSourceAnalysis analysis)
{
throw new UnsupportedOperationException();
}