Support Union in Decoupled planning (#17354)

* introduces `UnionQuery`
* some changes to enable a `UnionQuery` to have multiple input datasources
* `UnionQuery` execution is driven by the `QueryLogic` - which could later enable to reduce some complexity in `ClientQuerySegmentWalker`
* to run the subqueries of `UnionQuery` there was a need to access the `conglomerate` from the `Runner`; to enable that some refactors were done
* renamed `UnionQueryRunner` to `UnionDataSourceQueryRunner`
* `QueryRunnerFactoryConglomerate` have taken the place of `QueryToolChestWarehouse` which shaves of some unnecessary things here and there
* small cleanup/refactors
This commit is contained in:
Zoltan Haindrich 2024-11-05 16:58:57 +01:00 committed by GitHub
parent ba76264244
commit 2eac8318f8
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
107 changed files with 4337 additions and 835 deletions

View File

@ -68,8 +68,6 @@ import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QueryRunnerTestHelper;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.Result;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.context.ResponseContext;
@ -167,7 +165,6 @@ public class CachingClusteredClientBenchmark
@Param({"all", "minute"})
private String queryGranularity;
private QueryToolChestWarehouse toolChestWarehouse;
private QueryRunnerFactoryConglomerate conglomerate;
private CachingClusteredClient cachingClusteredClient;
private ExecutorService processingPool;
@ -258,48 +255,37 @@ public class CachingClusteredClientBenchmark
}
};
conglomerate = new DefaultQueryRunnerFactoryConglomerate(
ImmutableMap.<Class<? extends Query>, QueryRunnerFactory>builder()
.put(
TimeseriesQuery.class,
new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
conglomerate = DefaultQueryRunnerFactoryConglomerate.buildFromQueryRunnerFactories(ImmutableMap.<Class<? extends Query>, QueryRunnerFactory>builder()
.put(
TimeseriesQuery.class,
new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
.put(
TopNQuery.class,
new TopNQueryRunnerFactory(
new StupidPool<>(
"TopNQueryRunnerFactory-bufferPool",
() -> ByteBuffer.allocate(PROCESSING_BUFFER_SIZE)
),
new TopNQueryQueryToolChest(new TopNQueryConfig()),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
)
.put(
TopNQuery.class,
new TopNQueryRunnerFactory(
new StupidPool<>(
"TopNQueryRunnerFactory-bufferPool",
() -> ByteBuffer.allocate(PROCESSING_BUFFER_SIZE)
),
new TopNQueryQueryToolChest(new TopNQueryConfig()),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
.put(
GroupByQuery.class,
makeGroupByQueryRunnerFactory(
GroupByQueryRunnerTest.DEFAULT_MAPPER,
new GroupByQueryConfig()
{
},
processingConfig
)
)
.put(
GroupByQuery.class,
makeGroupByQueryRunnerFactory(
GroupByQueryRunnerTest.DEFAULT_MAPPER,
new GroupByQueryConfig()
{
},
processingConfig
)
.build()
);
toolChestWarehouse = new QueryToolChestWarehouse()
{
@Override
public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(final QueryType query)
{
return conglomerate.findFactory(query).getToolchest();
}
};
)
.build());
SimpleServerView serverView = new SimpleServerView();
int serverSuffx = 1;
@ -319,7 +305,7 @@ public class CachingClusteredClientBenchmark
true
);
cachingClusteredClient = new CachingClusteredClient(
toolChestWarehouse,
conglomerate,
serverView,
MapCache.create(0),
JSON_MAPPER,
@ -468,7 +454,7 @@ public class CachingClusteredClientBenchmark
QueryRunner<T> theRunner = FluentQueryRunner
.create(
cachingClusteredClient.getQueryRunnerForIntervals(query, query.getIntervals()),
toolChestWarehouse.getToolChest(query)
conglomerate.getToolChest(query)
)
.applyPreMergeDecoration()
.mergeResults(true)

View File

@ -44,8 +44,8 @@ import org.apache.druid.java.util.http.client.HttpClient;
import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator;
import org.apache.druid.metadata.TestDerbyConnector;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QueryRunnerTestHelper;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.QueryWatcher;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
@ -307,7 +307,7 @@ public class DatasourceOptimizerTest extends CuratorTestBase
DirectDruidClientFactory druidClientFactory = new DirectDruidClientFactory(
new NoopServiceEmitter(),
EasyMock.createMock(QueryToolChestWarehouse.class),
EasyMock.createMock(QueryRunnerFactoryConglomerate.class),
EasyMock.createMock(QueryWatcher.class),
getSmileMapper(),
EasyMock.createMock(HttpClient.class)

View File

@ -56,8 +56,8 @@ import org.apache.druid.query.BrokerParallelMergeConfig;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.Result;
import org.apache.druid.query.RetryQueryRunnerConfig;
import org.apache.druid.query.SegmentDescriptor;
@ -104,7 +104,7 @@ import java.util.concurrent.ForkJoinPool;
public class MovingAverageQueryTest extends InitializedNullHandlingTest
{
private final ObjectMapper jsonMapper;
private final QueryToolChestWarehouse warehouse;
private final QueryRunnerFactoryConglomerate conglomerate;
private final RetryQueryRunnerConfig retryConfig;
private final ServerConfig serverConfig;
@ -167,7 +167,7 @@ public class MovingAverageQueryTest extends InitializedNullHandlingTest
Injector injector = Initialization.makeInjectorWithModules(baseInjector, modules);
jsonMapper = injector.getInstance(ObjectMapper.class);
warehouse = injector.getInstance(QueryToolChestWarehouse.class);
conglomerate = injector.getInstance(QueryRunnerFactoryConglomerate.class);
retryConfig = injector.getInstance(RetryQueryRunnerConfig.class);
serverConfig = injector.getInstance(ServerConfig.class);
@ -321,7 +321,7 @@ public class MovingAverageQueryTest extends InitializedNullHandlingTest
};
CachingClusteredClient baseClient = new CachingClusteredClient(
warehouse,
conglomerate,
new TimelineServerView()
{
@Override
@ -375,7 +375,7 @@ public class MovingAverageQueryTest extends InitializedNullHandlingTest
new NoopServiceEmitter(),
baseClient,
null /* local client; unused in this test, so pass in null */,
warehouse,
conglomerate,
new MapJoinableFactory(ImmutableSet.of(), ImmutableMap.of()),
retryConfig,
jsonMapper,
@ -392,7 +392,7 @@ public class MovingAverageQueryTest extends InitializedNullHandlingTest
final Sequence<?> res = query.getRunner(walker).run(queryPlus);
List actualResults = new ArrayList();
actualResults = (List<MapBasedRow>) res.accumulate(actualResults, Accumulators.list());
actualResults = res.accumulate(actualResults, Accumulators.list());
expectedResults = consistentTypeCasting(expectedResults);
actualResults = consistentTypeCasting(actualResults);

View File

@ -2891,28 +2891,26 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
@Override
protected QueryRunnerFactoryConglomerate makeQueryRunnerConglomerate()
{
return new DefaultQueryRunnerFactoryConglomerate(
ImmutableMap.<Class<? extends Query>, QueryRunnerFactory>builder()
.put(
TimeseriesQuery.class,
new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(),
new TimeseriesQueryEngine(),
(query, future) -> {
// do nothing
}
)
return DefaultQueryRunnerFactoryConglomerate.buildFromQueryRunnerFactories(ImmutableMap.<Class<? extends Query>, QueryRunnerFactory>builder()
.put(
TimeseriesQuery.class,
new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(),
new TimeseriesQueryEngine(),
(query, future) -> {
// do nothing
}
)
.put(
ScanQuery.class,
new ScanQueryRunnerFactory(
new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()),
new ScanQueryEngine(),
new ScanQueryConfig()
)
)
.put(
ScanQuery.class,
new ScanQueryRunnerFactory(
new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()),
new ScanQueryEngine(),
new ScanQueryConfig()
)
.build()
);
)
.build());
}
private void makeToolboxFactory() throws IOException

View File

@ -2418,18 +2418,16 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase
@Override
protected QueryRunnerFactoryConglomerate makeQueryRunnerConglomerate()
{
return new DefaultQueryRunnerFactoryConglomerate(
ImmutableMap.of(
TimeseriesQuery.class,
new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(),
new TimeseriesQueryEngine(),
(query, future) -> {
// do nothing
}
)
return DefaultQueryRunnerFactoryConglomerate.buildFromQueryRunnerFactories(ImmutableMap.of(
TimeseriesQuery.class,
new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(),
new TimeseriesQueryEngine(),
(query, future) -> {
// do nothing
}
)
);
));
}
private void makeToolboxFactory() throws IOException

View File

@ -523,5 +523,4 @@ public class DruidException extends RuntimeException
protected abstract DruidException makeException(DruidExceptionBuilder bob);
}
}

View File

@ -24,6 +24,8 @@ package org.apache.druid.frame.allocation;
*/
public class ArenaMemoryAllocatorFactory implements MemoryAllocatorFactory
{
private static final int FRAME_SIZE = 8_000_000;
private final int capacity;
public ArenaMemoryAllocatorFactory(final int capacity)
@ -42,4 +44,9 @@ public class ArenaMemoryAllocatorFactory implements MemoryAllocatorFactory
{
return capacity;
}
public static MemoryAllocatorFactory makeDefault()
{
return new ArenaMemoryAllocatorFactory(FRAME_SIZE);
}
}

View File

@ -22,7 +22,6 @@ package org.apache.druid.java.util.common.guava;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import java.io.Closeable;
import java.util.Arrays;
import java.util.Collections;
@ -147,4 +146,9 @@ public class Sequences
return Yielders.done(initValue, null);
}
}
public static <T> Sequence<T> of(T... values)
{
return simple(Arrays.asList(values));
}
}

View File

@ -19,29 +19,58 @@
package org.apache.druid.query;
import com.google.common.collect.Maps;
import com.google.inject.Inject;
import java.util.Collections;
import java.util.IdentityHashMap;
import java.util.Map;
/**
*/
public class DefaultQueryRunnerFactoryConglomerate implements QueryRunnerFactoryConglomerate
{
private final Map<Class<? extends Query>, QueryRunnerFactory> factories;
private final Map<Class<? extends Query>, QueryToolChest> toolchests;
private final Map<Class<? extends Query>, QueryLogic> querylogics;
public static DefaultQueryRunnerFactoryConglomerate buildFromQueryRunnerFactories(
Map<Class<? extends Query>, QueryRunnerFactory> factories)
{
return new DefaultQueryRunnerFactoryConglomerate(
factories,
Maps.transformValues(factories, f -> f.getToolchest()),
Collections.emptyMap()
);
}
@Inject
public DefaultQueryRunnerFactoryConglomerate(Map<Class<? extends Query>, QueryRunnerFactory> factories)
public DefaultQueryRunnerFactoryConglomerate(
Map<Class<? extends Query>, QueryRunnerFactory> factories,
Map<Class<? extends Query>, QueryToolChest> toolchests,
Map<Class<? extends Query>, QueryLogic> querylogics)
{
// Accesses to IdentityHashMap should be faster than to HashMap or ImmutableMap.
// Class doesn't override Object.equals().
this.factories = new IdentityHashMap<>(factories);
this.toolchests = new IdentityHashMap<>(toolchests);
this.querylogics = new IdentityHashMap<>(querylogics);
}
@Override
@SuppressWarnings("unchecked")
public <T, QueryType extends Query<T>> QueryRunnerFactory<T, QueryType> findFactory(QueryType query)
{
return (QueryRunnerFactory<T, QueryType>) factories.get(query.getClass());
return factories.get(query.getClass());
}
@Override
@SuppressWarnings("unchecked")
public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(QueryType query)
{
return toolchests.get(query.getClass());
}
@Override
@SuppressWarnings("unchecked")
public <T, QueryType extends Query<T>> QueryLogic getQueryLogic(QueryType query)
{
return querylogics.get(query.getClass());
}
}

View File

@ -64,7 +64,7 @@ public class FluentQueryRunner<T> implements QueryRunner<T>
public FluentQueryRunner<T> applyPreMergeDecoration()
{
return from(new UnionQueryRunner<>(toolChest.preMergeQueryDecoration(baseRunner)));
return from(new UnionDataSourceQueryRunner<>(toolChest.preMergeQueryDecoration(baseRunner)));
}
public FluentQueryRunner<T> emitCPUTimeMetric(ServiceEmitter emitter)

View File

@ -33,6 +33,7 @@ import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery;
import org.apache.druid.query.operator.WindowOperatorQuery;
import org.apache.druid.query.planning.DataSourceAnalysis;
import org.apache.druid.query.scan.ScanQuery;
import org.apache.druid.query.search.SearchQuery;
import org.apache.druid.query.select.SelectQuery;
@ -40,8 +41,11 @@ import org.apache.druid.query.spec.QuerySegmentSpec;
import org.apache.druid.query.timeboundary.TimeBoundaryQuery;
import org.apache.druid.query.timeseries.TimeseriesQuery;
import org.apache.druid.query.topn.TopNQuery;
import org.apache.druid.query.union.UnionQuery;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.column.RowSignature.Finalization;
import org.apache.druid.utils.CollectionUtils;
import org.joda.time.DateTimeZone;
import org.joda.time.Duration;
@ -66,6 +70,7 @@ import java.util.UUID;
@JsonSubTypes.Type(name = Query.TIMESERIES, value = TimeseriesQuery.class),
@JsonSubTypes.Type(name = Query.TOPN, value = TopNQuery.class),
@JsonSubTypes.Type(name = Query.WINDOW_OPERATOR, value = WindowOperatorQuery.class),
@JsonSubTypes.Type(name = Query.UNION_QUERY, value = UnionQuery.class),
})
public interface Query<T>
{
@ -79,6 +84,7 @@ public interface Query<T>
String TIMESERIES = "timeseries";
String TOPN = "topN";
String WINDOW_OPERATOR = "windowOperator";
String UNION_QUERY = "union";
DataSource getDataSource();
@ -285,4 +291,19 @@ public interface Query<T>
)
);
}
default DataSourceAnalysis getDataSourceAnalysis()
{
return getDataSource().getAnalysis().maybeWithBaseQuery(this);
}
default RowSignature getResultRowSignature()
{
return getResultRowSignature(Finalization.UNKNOWN);
}
default RowSignature getResultRowSignature(RowSignature.Finalization finalization)
{
return null;
}
}

View File

@ -368,6 +368,11 @@ public class QueryContext
return getBoolean(QueryContexts.USE_NESTED_FOR_UNKNOWN_TYPE_IN_SUBQUERY, defaultUseNestedForUnkownTypeInSubquery);
}
public boolean isUseNestedForUnknownTypeInSubquery()
{
return isUseNestedForUnknownTypeInSubquery(QueryContexts.DEFAULT_USE_NESTED_FOR_UNKNOWN_TYPE_IN_SUBQUERY);
}
public int getUncoveredIntervalsLimit()
{
return getUncoveredIntervalsLimit(QueryContexts.DEFAULT_UNCOVERED_INTERVALS_LIMIT);

View File

@ -133,6 +133,8 @@ public class QueryContexts
public static final int DEFAULT_IN_FUNCTION_EXPR_THRESHOLD = 2;
public static final boolean DEFAULT_ENABLE_TIME_BOUNDARY_PLANNING = false;
public static final boolean DEFAULT_CATALOG_VALIDATION_ENABLED = true;
public static final boolean DEFAULT_USE_NESTED_FOR_UNKNOWN_TYPE_IN_SUBQUERY = false;
@SuppressWarnings("unused") // Used by Jackson serialization
public enum Vectorize

View File

@ -23,12 +23,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 org.apache.druid.query.planning.DataSourceAnalysis;
import org.apache.druid.query.union.UnionQuery;
import org.apache.druid.segment.SegmentReference;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
@ -49,7 +50,11 @@ public class QueryDataSource implements DataSource
@Override
public Set<String> getTableNames()
{
return query.getDataSource().getTableNames();
Set<String> names = new HashSet<>();
for (DataSource ds : getQueryDataSources()) {
names.addAll(ds.getTableNames());
}
return names;
}
@JsonProperty
@ -61,17 +66,28 @@ public class QueryDataSource implements DataSource
@Override
public List<DataSource> getChildren()
{
return getQueryDataSources();
}
private List<DataSource> getQueryDataSources()
{
if (query instanceof UnionQuery) {
return ((UnionQuery) query).getDataSources();
}
return Collections.singletonList(query.getDataSource());
}
@Override
public DataSource withChildren(List<DataSource> children)
{
if (children.size() != 1) {
throw new IAE("Must have exactly one child");
if (query instanceof UnionQuery) {
return new QueryDataSource(((UnionQuery) query).withDataSources(children));
} else {
if (children.size() != 1) {
throw new IAE("Must have exactly one child");
}
return new QueryDataSource(query.withDataSource(children.get(0)));
}
return new QueryDataSource(query.withDataSource(Iterables.getOnlyElement(children)));
}
@Override
@ -118,14 +134,7 @@ public class QueryDataSource implements DataSource
public DataSourceAnalysis getAnalysis()
{
final Query<?> subQuery = this.getQuery();
if (!(subQuery instanceof BaseQuery)) {
// We must verify that the subQuery is a BaseQuery, because it is required to make
// "DataSourceAnalysis.getBaseQuerySegmentSpec" work properly.
// All built-in 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());
}
final DataSource current = subQuery.getDataSource();
return current.getAnalysis().maybeWithBaseQuery(subQuery);
return subQuery.getDataSourceAnalysis();
}
@Override

View File

@ -0,0 +1,37 @@
/*
* 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;
/**
* Provides facilities to executes the given query type by utilizing the
* {@link QuerySegmentWalker} to run subqueries if necessary.
*
*/
public interface QueryLogic
{
/**
* Builds a {@link QueryRunner} for the given query and walker.
*
* The returned runner must respect {@link ResultSerializationMode}.
*/
<T> QueryRunner<Object> entryPoint(
Query<T> query,
QuerySegmentWalker walker);
}

View File

@ -0,0 +1,79 @@
/*
* 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.core.type.TypeReference;
import com.google.common.base.Function;
import com.google.common.base.Functions;
import org.apache.druid.frame.allocation.MemoryAllocatorFactory;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.query.aggregation.MetricManipulationFn;
import org.apache.druid.segment.column.RowSignature;
import java.util.Optional;
public class QueryLogicCompatToolChest extends QueryToolChest<Object, Query<Object>>
{
private RowSignature resultRowSignature;
public QueryLogicCompatToolChest(RowSignature resultRowSignature)
{
this.resultRowSignature = resultRowSignature;
}
@Override
public RowSignature resultArraySignature(Query<Object> query)
{
return resultRowSignature;
}
@Override
public QueryMetrics<? super Query<Object>> makeMetrics(Query<Object> query)
{
return new DefaultQueryMetrics<>();
}
@Override
public Function<Object, Object> makePreComputeManipulatorFn(Query<Object> query, MetricManipulationFn fn)
{
return Functions.identity();
}
@Override
public TypeReference<Object> getResultTypeReference()
{
return null;
}
@Override
public Sequence<Object[]> resultsAsArrays(Query<Object> query, Sequence<Object> resultSequence)
{
Sequence<?> res = resultSequence;
return (Sequence<Object[]>) res;
}
@Override
public Optional<Sequence<FrameSignaturePair>> resultsAsFrames(Query<Object> query, Sequence<Object> resultSequence,
MemoryAllocatorFactory memoryAllocatorFactory, boolean useNestedForUnknownTypes)
{
Sequence<?> res = resultSequence;
return Optional.of((Sequence<FrameSignaturePair>) res);
}
}

View File

@ -21,6 +21,7 @@ package org.apache.druid.query;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.error.DruidException;
import org.apache.druid.guice.annotations.PublicApi;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.query.context.ResponseContext;
@ -59,6 +60,14 @@ public final class QueryPlus<T>
return query;
}
public <C extends Query<T>> C unwrapQuery(Class<C> clazz)
{
if (clazz.isInstance(query)) {
return (C) query;
}
throw DruidException.defensive("Encountered unexpected query type [%s] instead of [%s]", query.getClass(), clazz);
}
@Nullable
public QueryMetrics<?> getQueryMetrics()
{

View File

@ -24,4 +24,8 @@ package org.apache.druid.query;
public interface QueryRunnerFactoryConglomerate
{
<T, QueryType extends Query<T>> QueryRunnerFactory<T, QueryType> findFactory(QueryType query);
<T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(QueryType query);
<T, QueryType extends Query<T>> QueryLogic getQueryLogic(QueryType query);
}

View File

@ -422,4 +422,11 @@ public abstract class QueryToolChest<ResultType, QueryType extends Query<ResultT
{
return Optional.empty();
}
public <T> boolean canExecuteFully(Query<T> query)
{
DataSource dataSourceFromQuery = query.getDataSource();
return (!(dataSourceFromQuery instanceof QueryDataSource)
|| canPerformSubquery(((QueryDataSource) dataSourceFromQuery).getQuery()));
}
}

View File

@ -0,0 +1,93 @@
/*
* 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 org.apache.druid.error.DruidException;
import org.apache.druid.frame.allocation.ArenaMemoryAllocatorFactory;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.query.context.ResponseContext;
/**
* Implements a QueryRunner that takes {@link ResultSerializationMode} into
* account.
*
* It uses the toolchest service methods to provide the appropriate
* serialization.
*/
public class ToolChestBasedResultSerializedRunner<T> implements QueryRunner<T>
{
private final QueryRunner<T> runner;
private final QueryToolChest<T, Query<T>> toolChest;
public ToolChestBasedResultSerializedRunner(
Query<T> query,
QuerySegmentWalker walker,
QueryToolChest<T, Query<T>> toolChest)
{
this.runner = query.getRunner(walker);
this.toolChest = toolChest;
}
// note: returns a Sequence<Object> and not Sequenct<T>
@Override
public Sequence<T> run(QueryPlus<T> queryPlus, ResponseContext responseContext)
{
Query<T> query = queryPlus.getQuery();
Sequence<T> seq = runner.run(queryPlus, responseContext);
boolean useNestedForUnknownTypeInSubquery = query.context().isUseNestedForUnknownTypeInSubquery();
ResultSerializationMode serializationMode = getResultSerializationMode(query);
Sequence<?> resultSeq;
switch (serializationMode) {
case ROWS:
resultSeq = toolChest.resultsAsArrays(query, seq);
break;
case FRAMES:
resultSeq = toolChest.resultsAsFrames(
query,
seq,
ArenaMemoryAllocatorFactory.makeDefault(),
useNestedForUnknownTypeInSubquery
).orElseThrow(() -> DruidException.defensive("Unable to materialize the results as frames."));
break;
default:
throw DruidException.defensive("Not supported serializationMode [%s].", serializationMode);
}
// this cast is not valid; however QueryRunner<T> makes most of the template
// usage okay.
return (Sequence<T>) resultSeq;
}
private ResultSerializationMode getResultSerializationMode(Query<T> query)
{
ResultSerializationMode serializationMode = query.context().getEnum(
ResultSerializationMode.CTX_SERIALIZATION_PARAMETER,
ResultSerializationMode.class,
null
);
if (serializationMode == null) {
throw DruidException.defensive(
"Serialization mode [%s] is not setup correctly!", ResultSerializationMode.CTX_SERIALIZATION_PARAMETER
);
}
return serializationMode;
}
}

View File

@ -34,11 +34,11 @@ import org.apache.druid.query.planning.DataSourceAnalysis;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
public class UnionQueryRunner<T> implements QueryRunner<T>
public class UnionDataSourceQueryRunner<T> implements QueryRunner<T>
{
private final QueryRunner<T> baseRunner;
public UnionQueryRunner(
public UnionDataSourceQueryRunner(
QueryRunner<T> baseRunner
)
{

View File

@ -330,11 +330,7 @@ public class GroupByQuery extends BaseQuery<ResultRow>
return subtotalsSpec;
}
/**
* Equivalent to {@code getResultRowSignature(Finalization.UNKNOWN)}.
*
* @see ResultRow for documentation about the order that fields will be in
*/
@Override
public RowSignature getResultRowSignature()
{
return resultRowSignature;
@ -350,6 +346,7 @@ public class GroupByQuery extends BaseQuery<ResultRow>
*
* @see ResultRow for documentation about the order that fields will be in
*/
@Override
public RowSignature getResultRowSignature(final RowSignature.Finalization finalization)
{
if (finalization == RowSignature.Finalization.UNKNOWN) {

View File

@ -34,6 +34,7 @@ import org.apache.druid.query.rowsandcols.RowsAndColumns;
import org.apache.druid.query.scan.ScanQuery;
import org.apache.druid.query.spec.QuerySegmentSpec;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.column.RowSignature.Finalization;
import javax.annotation.Nullable;
import java.util.ArrayList;
@ -163,6 +164,12 @@ public class WindowOperatorQuery extends BaseQuery<RowsAndColumns>
return rowSignature;
}
@Override
public RowSignature getResultRowSignature(Finalization finalization)
{
return getRowSignature();
}
@Override
public boolean hasFilters()
{

View File

@ -32,6 +32,7 @@ import org.apache.druid.query.spec.QuerySegmentSpec;
import org.apache.druid.segment.join.JoinPrefixUtils;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
@ -181,7 +182,7 @@ public class DataSourceAnalysis
*/
public DataSourceAnalysis maybeWithBaseQuery(Query<?> query)
{
if (!getBaseQuery().isPresent()) {
if (!getBaseQuery().isPresent() && query instanceof BaseQuery) {
return new DataSourceAnalysis(baseDataSource, query, joinBaseTableFilter, preJoinableClauses);
}
return this;
@ -294,4 +295,17 @@ public class DataSourceAnalysis
", preJoinableClauses=" + preJoinableClauses +
'}';
}
/**
* {@link DataSource#isGlobal()}.
*/
public boolean isGlobal()
{
for (PreJoinableClause preJoinableClause : preJoinableClauses) {
if (!preJoinableClause.getDataSource().isGlobal()) {
return false;
}
}
return baseDataSource.isGlobal();
}
}

View File

@ -48,6 +48,7 @@ import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.column.RowSignature.Builder;
import org.apache.druid.segment.column.RowSignature.Finalization;
import javax.annotation.Nullable;
import java.util.Collections;
@ -611,6 +612,11 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
}
}
@Override
public RowSignature getResultRowSignature(Finalization finalization)
{
return getRowSignature();
}
/**
* Returns the RowSignature.

View File

@ -43,6 +43,7 @@ import org.apache.druid.query.spec.QuerySegmentSpec;
import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.column.RowSignature.Finalization;
import javax.annotation.Nullable;
import java.util.ArrayList;
@ -181,15 +182,17 @@ public class TimeseriesQuery extends BaseQuery<Result<TimeseriesResultValue>>
return context().getBoolean(SKIP_EMPTY_BUCKETS, false);
}
public RowSignature getResultSignature(final RowSignature.Finalization finalization)
@Override
public RowSignature getResultRowSignature(Finalization finalization)
{
final Finalization finalization1 = finalization;
final RowSignature.Builder builder = RowSignature.builder();
builder.addTimeColumn();
String timestampResultField = getTimestampResultField();
if (StringUtils.isNotEmpty(timestampResultField)) {
builder.add(timestampResultField, ColumnType.LONG);
}
builder.addAggregators(aggregatorSpecs, finalization);
builder.addAggregators(aggregatorSpecs, finalization1);
builder.addPostAggregators(postAggregatorSpecs);
return builder.build();
}

View File

@ -445,7 +445,7 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest<Result<Timeser
@Override
public RowSignature resultArraySignature(TimeseriesQuery query)
{
return query.getResultSignature(RowSignature.Finalization.UNKNOWN);
return query.getResultRowSignature(RowSignature.Finalization.UNKNOWN);
}
@Override
@ -486,9 +486,7 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest<Result<Timeser
)
{
final RowSignature rowSignature =
query.getResultSignature(
query.context().isFinalize(true) ? RowSignature.Finalization.YES : RowSignature.Finalization.NO
);
query.getResultRowSignature(query.context().isFinalize(true) ? RowSignature.Finalization.YES : RowSignature.Finalization.NO);
final Pair<Cursor, Closeable> cursorAndCloseable = IterableRowsCursorHelper.getCursorFromSequence(
resultsAsArrays(query, resultSequence),
rowSignature

View File

@ -185,7 +185,8 @@ public class TopNQuery extends BaseQuery<Result<TopNResultValue>>
topNMetricSpec.initTopNAlgorithmSelector(selector);
}
public RowSignature getResultSignature(final RowSignature.Finalization finalization)
@Override
public RowSignature getResultRowSignature(final RowSignature.Finalization finalization)
{
return RowSignature.builder()
.addTimeColumn()

View File

@ -517,7 +517,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
@Override
public RowSignature resultArraySignature(TopNQuery query)
{
return query.getResultSignature(RowSignature.Finalization.UNKNOWN);
return query.getResultRowSignature(RowSignature.Finalization.UNKNOWN);
}
@Override
@ -563,9 +563,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest<Result<TopNResultVal
boolean useNestedForUnknownTypes
)
{
final RowSignature rowSignature = query.getResultSignature(
query.context().isFinalize(true) ? RowSignature.Finalization.YES : RowSignature.Finalization.NO
);
final RowSignature rowSignature = query.getResultRowSignature(query.context().isFinalize(true) ? RowSignature.Finalization.YES : RowSignature.Finalization.NO);
final Pair<Cursor, Closeable> cursorAndCloseable = IterableRowsCursorHelper.getCursorFromSequence(
resultsAsArrays(query, resultSequence),

View File

@ -0,0 +1,308 @@
/*
* 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.union;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Ordering;
import org.apache.druid.error.DruidException;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.query.BaseQuery;
import org.apache.druid.query.DataSource;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.query.UnionDataSource;
import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.query.planning.DataSourceAnalysis;
import org.apache.druid.query.spec.QuerySegmentSpec;
import org.apache.druid.segment.SegmentReference;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.column.RowSignature.Finalization;
import org.joda.time.DateTimeZone;
import org.joda.time.Duration;
import org.joda.time.Interval;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
public class UnionQuery implements Query<Object>
{
@JsonProperty("context")
protected final Map<String, Object> context;
@JsonProperty("queries")
protected final List<Query<?>> queries;
public UnionQuery(List<Query<?>> queries)
{
this(queries, queries.get(0).getContext());
}
@JsonCreator
public UnionQuery(
@JsonProperty("queries") List<Query<?>> queries,
@JsonProperty("context") Map<String, Object> context)
{
Preconditions.checkArgument(queries.size() > 1, "union with fewer than 2 queries makes no sense");
this.queries = queries;
this.context = context;
}
@Override
public DataSource getDataSource()
{
throw DruidException.defensive("This is not supported. Use getDataSources instead!");
}
public List<DataSource> getDataSources()
{
List<DataSource> dataSources = new ArrayList<>();
for (Query<?> query : queries) {
dataSources.add(query.getDataSource());
}
return dataSources;
}
@Override
public boolean hasFilters()
{
return false;
}
@Override
public DimFilter getFilter()
{
return null;
}
@Override
public String getType()
{
return getClass().getSimpleName();
}
@Override
public QueryRunner<Object> getRunner(QuerySegmentWalker walker)
{
throw DruidException.defensive("Use QueryToolChest to get a Runner");
}
@Override
public List<Interval> getIntervals()
{
return Collections.emptyList();
}
@Override
public Duration getDuration()
{
throw methodNotSupported();
}
@Override
public Granularity getGranularity()
{
return Granularities.ALL;
}
@Override
public DateTimeZone getTimezone()
{
throw methodNotSupported();
}
@Override
public Map<String, Object> getContext()
{
return context;
}
@Override
public Ordering<Object> getResultOrdering()
{
throw methodNotSupported();
}
@Override
public Query<Object> withOverriddenContext(Map<String, Object> contextOverrides)
{
List<Query<?>> newQueries = mapQueries(q -> q.withOverriddenContext(contextOverrides));
return new UnionQuery(newQueries, QueryContexts.override(getContext(), contextOverrides));
}
@Override
public Query<Object> withQuerySegmentSpec(QuerySegmentSpec spec)
{
throw methodNotSupported();
}
@Override
public Query<Object> withId(String id)
{
return withOverriddenContext(ImmutableMap.of(BaseQuery.QUERY_ID, id));
}
@Override
public String getId()
{
return context().getString(BaseQuery.QUERY_ID);
}
@Override
public Query<Object> withSubQueryId(String subQueryId)
{
return withOverriddenContext(ImmutableMap.of(BaseQuery.SUB_QUERY_ID, subQueryId));
}
@Override
public String getSubQueryId()
{
return context().getString(BaseQuery.SUB_QUERY_ID);
}
@Override
public Query<Object> withDataSource(DataSource dataSource)
{
throw new RuntimeException("This method is not supported. Use withDataSources instead!");
}
public Query<Object> withDataSources(List<DataSource> children)
{
Preconditions.checkArgument(queries.size() == children.size(), "Number of children must match number of queries");
List<Query<?>> newQueries = new ArrayList<>();
for (int i = 0; i < queries.size(); i++) {
newQueries.add(queries.get(i).withDataSource(children.get(i)));
}
return new UnionQuery(newQueries, context);
}
List<Query<?>> mapQueries(Function<Query<?>, Query<?>> mapFn)
{
List<Query<?>> newQueries = new ArrayList<>();
for (Query<?> query : queries) {
newQueries.add(mapFn.apply(query));
}
return newQueries;
}
@Override
public String toString()
{
return "UnionQuery [context=" + context + ", queries=" + queries + "]";
}
@Override
public DataSourceAnalysis getDataSourceAnalysis()
{
OpaqueDataSourceCover ds = new OpaqueDataSourceCover(new UnionDataSource(getDataSources()));
return new DataSourceAnalysis(ds, null, null, Collections.emptyList());
}
private static class OpaqueDataSourceCover implements DataSource
{
private DataSource delegate;
public OpaqueDataSourceCover(DataSource delegate)
{
this.delegate = delegate;
}
@Override
public Set<String> getTableNames()
{
return delegate.getTableNames();
}
@Override
public List<DataSource> getChildren()
{
return delegate.getChildren();
}
@Override
public DataSource withChildren(List<DataSource> children)
{
throw methodNotSupported();
}
@Override
public boolean isCacheable(boolean isBroker)
{
return delegate.isCacheable(isBroker);
}
@Override
public boolean isGlobal()
{
return delegate.isGlobal();
}
@Override
public boolean isConcrete()
{
return delegate.isConcrete();
}
@Override
public Function<SegmentReference, SegmentReference> createSegmentMapFunction(Query query, AtomicLong cpuTimeAcc)
{
throw methodNotSupported();
}
@Override
public DataSource withUpdatedDataSource(DataSource newSource)
{
throw methodNotSupported();
}
@Override
public byte[] getCacheKey()
{
return delegate.getCacheKey();
}
@Override
public DataSourceAnalysis getAnalysis()
{
throw methodNotSupported();
}
}
public static DruidException methodNotSupported()
{
return DruidException.defensive("Method not supported. This method is not expected to be called!");
}
@Override
public RowSignature getResultRowSignature(Finalization finalization)
{
return queries.get(0).getResultRowSignature(finalization);
}
}

View File

@ -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.union;
import com.google.inject.Inject;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryLogic;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.query.ToolChestBasedResultSerializedRunner;
import org.apache.druid.query.context.ResponseContext;
import java.util.ArrayList;
import java.util.List;
public class UnionQueryLogic implements QueryLogic
{
protected QueryRunnerFactoryConglomerate conglomerate;
@Inject
public void initialize(QueryRunnerFactoryConglomerate conglomerate)
{
this.conglomerate = conglomerate;
}
@Override
public <T> QueryRunner<Object> entryPoint(Query<T> query, QuerySegmentWalker walker)
{
return new UnionQueryRunner((UnionQuery) query, conglomerate, walker);
}
static class UnionQueryRunner implements QueryRunner<Object>
{
private final QueryRunnerFactoryConglomerate conglomerate;
private final QuerySegmentWalker walker;
private final List<QueryRunner> runners;
public UnionQueryRunner(
UnionQuery query,
QueryRunnerFactoryConglomerate conglomerate,
QuerySegmentWalker walker)
{
this.conglomerate = conglomerate;
this.walker = walker;
this.runners = makeSubQueryRunners(query);
}
private List<QueryRunner> makeSubQueryRunners(UnionQuery unionQuery)
{
List<QueryRunner> runners = new ArrayList<>();
for (Query<?> query : unionQuery.queries) {
runners.add(buildRunnerFor(query));
}
return runners;
}
private QueryRunner<?> buildRunnerFor(Query<?> query)
{
QueryLogic queryLogic = conglomerate.getQueryLogic(query);
if (queryLogic != null) {
return queryLogic.entryPoint(query, walker);
}
return new ToolChestBasedResultSerializedRunner(query, walker, conglomerate.getToolChest(query));
}
@Override
public Sequence<Object> run(QueryPlus<Object> queryPlus, ResponseContext responseContext)
{
UnionQuery unionQuery = queryPlus.unwrapQuery(UnionQuery.class);
List<Sequence<Object>> seqs = new ArrayList<>();
for (int i = 0; i < runners.size(); i++) {
Query<?> q = unionQuery.queries.get(i);
QueryRunner runner = runners.get(i);
Sequence run = runner.run(queryPlus.withQuery(q), responseContext);
seqs.add(run);
}
return Sequences.concat(seqs);
}
}
}

View File

@ -26,7 +26,7 @@ import java.util.List;
public class QueryToolChestTestHelper
{
public static void assertArrayResultsEquals(final List<Object[]> expected, final Sequence<Object[]> actual)
public static void assertArrayResultsEquals(List<Object[]> expected, final Sequence<Object[]> actual)
{
final List<Object[]> actualList = actual.toList();
Assert.assertEquals("number of results", expected.size(), actualList.size());

View File

@ -30,7 +30,7 @@ import java.util.Arrays;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
public class UnionQueryRunnerTest
public class UnionDataSourceQueryRunnerTest
{
@Test
public void testUnionQueryRunner()
@ -56,7 +56,7 @@ public class UnionQueryRunnerTest
}
}
};
UnionQueryRunner runner = new UnionQueryRunner(baseRunner);
UnionDataSourceQueryRunner runner = new UnionDataSourceQueryRunner(baseRunner);
// Make a dummy query with Union datasource
Query q = Druids.newTimeseriesQueryBuilder()
.dataSource(

View File

@ -73,6 +73,7 @@ public class DataSourceAnalysisTest
Assert.assertEquals(Optional.empty(), analysis.getBaseQuery());
Assert.assertEquals(Optional.empty(), analysis.getBaseQuerySegmentSpec());
Assert.assertEquals(Collections.emptyList(), analysis.getPreJoinableClauses());
Assert.assertFalse(analysis.isGlobal());
Assert.assertFalse(analysis.isJoin());
Assert.assertTrue(analysis.isBaseColumn("foo"));
}
@ -92,6 +93,7 @@ public class DataSourceAnalysisTest
Assert.assertEquals(Optional.empty(), analysis.getBaseQuery());
Assert.assertEquals(Optional.empty(), analysis.getBaseQuerySegmentSpec());
Assert.assertEquals(Collections.emptyList(), analysis.getPreJoinableClauses());
Assert.assertEquals(unionDataSource.isGlobal(), analysis.isGlobal());
Assert.assertFalse(analysis.isJoin());
Assert.assertTrue(analysis.isBaseColumn("foo"));
}
@ -114,6 +116,7 @@ public class DataSourceAnalysisTest
analysis.getBaseQuerySegmentSpec()
);
Assert.assertEquals(Collections.emptyList(), analysis.getPreJoinableClauses());
Assert.assertFalse(analysis.isGlobal());
Assert.assertFalse(analysis.isJoin());
Assert.assertFalse(analysis.isBaseColumn("foo"));
}
@ -137,6 +140,7 @@ public class DataSourceAnalysisTest
analysis.getBaseQuerySegmentSpec()
);
Assert.assertEquals(Collections.emptyList(), analysis.getPreJoinableClauses());
Assert.assertFalse(analysis.isGlobal());
Assert.assertFalse(analysis.isJoin());
Assert.assertFalse(analysis.isBaseColumn("foo"));
}
@ -155,6 +159,7 @@ public class DataSourceAnalysisTest
Assert.assertEquals(Optional.empty(), analysis.getBaseQuery());
Assert.assertEquals(Optional.empty(), analysis.getBaseQuerySegmentSpec());
Assert.assertEquals(Collections.emptyList(), analysis.getPreJoinableClauses());
Assert.assertTrue(analysis.isGlobal());
Assert.assertFalse(analysis.isJoin());
Assert.assertTrue(analysis.isBaseColumn("foo"));
}
@ -177,6 +182,7 @@ public class DataSourceAnalysisTest
analysis.getBaseQuerySegmentSpec()
);
Assert.assertEquals(Collections.emptyList(), analysis.getPreJoinableClauses());
Assert.assertTrue(analysis.isGlobal());
Assert.assertFalse(analysis.isJoin());
Assert.assertFalse(analysis.isBaseColumn("foo"));
}
@ -195,6 +201,8 @@ public class DataSourceAnalysisTest
Assert.assertEquals(Optional.empty(), analysis.getBaseQuery());
Assert.assertEquals(Optional.empty(), analysis.getBaseQuerySegmentSpec());
Assert.assertEquals(Collections.emptyList(), analysis.getPreJoinableClauses());
Assert.assertEquals(INLINE.isGlobal(), analysis.isGlobal());
Assert.assertTrue(analysis.isGlobal());
Assert.assertFalse(analysis.isJoin());
Assert.assertTrue(analysis.isBaseColumn("foo"));
}
@ -243,6 +251,8 @@ public class DataSourceAnalysisTest
),
analysis.getPreJoinableClauses()
);
Assert.assertEquals(joinDataSource.isGlobal(), analysis.isGlobal());
Assert.assertFalse(analysis.isGlobal());
Assert.assertTrue(analysis.isJoin());
Assert.assertTrue(analysis.isBaseColumn("foo"));
Assert.assertFalse(analysis.isBaseColumn("1.foo"));
@ -292,6 +302,8 @@ public class DataSourceAnalysisTest
),
analysis.getPreJoinableClauses()
);
Assert.assertEquals(joinDataSource.isGlobal(), analysis.isGlobal());
Assert.assertFalse(analysis.isGlobal());
Assert.assertTrue(analysis.isJoin());
Assert.assertTrue(analysis.isBaseColumn("foo"));
Assert.assertFalse(analysis.isBaseColumn("1.foo"));
@ -345,6 +357,7 @@ public class DataSourceAnalysisTest
),
analysis.getPreJoinableClauses()
);
Assert.assertFalse(analysis.isGlobal());
Assert.assertTrue(analysis.isJoin());
Assert.assertTrue(analysis.isBaseColumn("foo"));
Assert.assertTrue(analysis.isBaseColumn("1.foo"));
@ -394,6 +407,7 @@ public class DataSourceAnalysisTest
),
analysis.getPreJoinableClauses()
);
Assert.assertFalse(analysis.isGlobal());
Assert.assertTrue(analysis.isJoin());
Assert.assertTrue(analysis.isBaseColumn("foo"));
Assert.assertTrue(analysis.isBaseColumn("1.foo"));
@ -427,6 +441,7 @@ public class DataSourceAnalysisTest
),
analysis.getPreJoinableClauses()
);
Assert.assertFalse(analysis.isGlobal());
Assert.assertTrue(analysis.isJoin());
Assert.assertTrue(analysis.isBaseColumn("foo"));
Assert.assertFalse(analysis.isBaseColumn("1.foo"));
@ -460,6 +475,7 @@ public class DataSourceAnalysisTest
),
analysis.getPreJoinableClauses()
);
Assert.assertFalse(analysis.isGlobal());
Assert.assertTrue(analysis.isJoin());
Assert.assertTrue(analysis.isBaseColumn("foo"));
Assert.assertFalse(analysis.isBaseColumn("1.foo"));
@ -514,6 +530,7 @@ public class DataSourceAnalysisTest
),
analysis.getPreJoinableClauses()
);
Assert.assertFalse(analysis.isGlobal());
Assert.assertTrue(analysis.isJoin());
Assert.assertFalse(analysis.isBaseColumn("foo"));
Assert.assertFalse(analysis.isBaseColumn("1.foo"));
@ -546,6 +563,7 @@ public class DataSourceAnalysisTest
),
analysis.getPreJoinableClauses()
);
Assert.assertTrue(analysis.isGlobal());
Assert.assertTrue(analysis.isJoin());
Assert.assertTrue(analysis.isBaseColumn("foo"));
Assert.assertFalse(analysis.isBaseColumn("1.foo"));
@ -578,6 +596,7 @@ public class DataSourceAnalysisTest
),
analysis.getPreJoinableClauses()
);
Assert.assertFalse(analysis.isGlobal());
Assert.assertTrue(analysis.isJoin());
Assert.assertTrue(analysis.isBaseColumn("foo"));
Assert.assertFalse(analysis.isBaseColumn("1.foo"));

View File

@ -71,9 +71,12 @@ public class ScanQueryQueryToolChestTest
new Object[]{3.5, "str4"}
);
private final ScanQueryQueryToolChest toolChest = new ScanQueryQueryToolChest(
DefaultGenericQueryMetricsFactory.instance()
);
private final ScanQueryQueryToolChest toolChest = makeTestScanQueryToolChest();
public static ScanQueryQueryToolChest makeTestScanQueryToolChest()
{
return new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance());
}
@Test
public void test_resultArraySignature_columnsNotSpecified()

View File

@ -32,7 +32,7 @@ import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.Result;
import org.apache.druid.query.TableDataSource;
import org.apache.druid.query.UnionDataSource;
import org.apache.druid.query.UnionQueryRunner;
import org.apache.druid.query.UnionDataSourceQueryRunner;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.segment.TestHelper;
@ -172,7 +172,7 @@ public class TimeSeriesUnionQueryRunnerTest extends InitializedNullHandlingTest
);
QueryRunner mergingrunner = toolChest.mergeResults(
new UnionQueryRunner<>(
new UnionDataSourceQueryRunner<>(
new QueryRunner<Result<TimeseriesResultValue>>()
{
@Override

View File

@ -0,0 +1,211 @@
/*
* 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.union;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate;
import org.apache.druid.query.Druids;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryRunnerTestHelper;
import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryToolChestTestHelper;
import org.apache.druid.query.ResultSerializationMode;
import org.apache.druid.query.scan.ScanQuery;
import org.apache.druid.query.scan.ScanQueryQueryToolChest;
import org.apache.druid.query.scan.ScanQueryQueryToolChestTest;
import org.apache.druid.query.scan.ScanResultValue;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.junit.Assert;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
import org.mockito.Mockito;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.argThat;
public class UnionQueryQueryToolChestTest
{
@BeforeAll
public static void setUpClass()
{
NullHandling.initializeForTests();
}
final UnionQueryLogic queryLogic;
private ScanQueryQueryToolChest scanToolChest;
public UnionQueryQueryToolChestTest()
{
queryLogic = new UnionQueryLogic();
scanToolChest = ScanQueryQueryToolChestTest.makeTestScanQueryToolChest();
DefaultQueryRunnerFactoryConglomerate conglomerate = new DefaultQueryRunnerFactoryConglomerate(
Collections.emptyMap(),
ImmutableMap.<Class<? extends Query>, QueryToolChest>builder()
.put(ScanQuery.class, scanToolChest)
.build(),
Collections.emptyMap()
);
queryLogic.initialize(conglomerate);
}
@Test
public void testResultArraySignatureWithTimestampResultField()
{
RowSignature sig = RowSignature.builder()
.add("a", ColumnType.STRING)
.add("b", ColumnType.STRING)
.build();
TestScanQuery scan1 = new TestScanQuery("foo", sig)
.appendRow("a", "a")
.appendRow("a", "b");
TestScanQuery scan2 = new TestScanQuery("bar", sig)
.appendRow("x", "x")
.appendRow("x", "y");
List<Query<?>> queries = ImmutableList.of(
scan1.query,
scan2.query
);
UnionQuery query = new UnionQuery(queries);
Assert.assertEquals(
sig,
query.getResultRowSignature()
);
}
static class TestScanQuery
{
final ScanQuery query;
final List<Object[]> results = new ArrayList<Object[]>();
public TestScanQuery(String sourceName, RowSignature signature)
{
this.query = Druids.newScanQueryBuilder()
.dataSource(sourceName)
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2000/3000"))))
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.columns(signature.getColumnNames())
.columnTypes(signature.getColumnTypes())
.build();
}
public TestScanQuery appendRow(Object... row)
{
results.add(row);
return this;
}
public Sequence<ScanResultValue> makeResultSequence()
{
ScanResultValue result = new ScanResultValue(
QueryRunnerTestHelper.SEGMENT_ID.toString(),
query.getColumns(),
convertResultsToListOfLists()
);
return Sequences.of(result);
}
private List<List<Object>> convertResultsToListOfLists()
{
List<List<Object>> resultsRows = new ArrayList<List<Object>>();
for (Object[] objects : results) {
resultsRows.add(Arrays.asList(objects));
}
return resultsRows;
}
private boolean matchQuery(ScanQuery query)
{
return query != null && serializedAsRows(this.query).equals(serializedAsRows(query));
}
public Sequence<Object[]> makeResultsAsArrays()
{
ScanQueryQueryToolChest scanToolChest = ScanQueryQueryToolChestTest.makeTestScanQueryToolChest();
return scanToolChest.resultsAsArrays(query, makeResultSequence());
}
}
@Test
void testQueryRunner()
{
RowSignature sig = RowSignature.builder()
.add("a", ColumnType.STRING)
.add("b", ColumnType.STRING)
.build();
TestScanQuery scan1 = new TestScanQuery("foo", sig)
.appendRow("a", "a")
.appendRow("a", "b");
TestScanQuery scan2 = new TestScanQuery("bar", sig)
.appendRow("x", "x")
.appendRow("x", "y");
UnionQuery query = new UnionQuery(
ImmutableList.of(
scan1.query,
scan2.query
)
);
query = (UnionQuery) serializedAsRows(query);
QuerySegmentWalker walker = Mockito.mock(QuerySegmentWalker.class);
Mockito.when(walker.getQueryRunnerForIntervals(argThat(scan1::matchQuery), any()))
.thenReturn((q, ctx) -> (Sequence) scan1.makeResultSequence());
Mockito.when(walker.getQueryRunnerForIntervals(argThat(scan2::matchQuery), any()))
.thenReturn((q, ctx) -> (Sequence) scan2.makeResultSequence());
QueryRunner<Object> unionRunner = queryLogic.entryPoint(query, walker);
Sequence results = unionRunner.run(QueryPlus.wrap(query), null);
QueryToolChestTestHelper.assertArrayResultsEquals(
Sequences.concat(
scan1.makeResultsAsArrays(),
scan2.makeResultsAsArrays()
).toList(),
results
);
}
private static Query<?> serializedAsRows(Query<?> query)
{
return query
.withOverriddenContext(ImmutableMap.of(ResultSerializationMode.CTX_SERIALIZATION_PARAMETER, "rows"));
}
}

View File

@ -1110,7 +1110,7 @@ public class CursorFactoryProjectionTest extends InitializedNullHandlingTest
final List<Result<TimeseriesResultValue>> results = resultRows.toList();
Assert.assertEquals(2, results.size());
final RowSignature querySignature = query.getResultSignature(RowSignature.Finalization.YES);
final RowSignature querySignature = query.getResultRowSignature(RowSignature.Finalization.YES);
Assert.assertArrayEquals(new Object[]{TIMESTAMP, 16L}, getResultArray(results.get(0), querySignature));
Assert.assertArrayEquals(new Object[]{TIMESTAMP.plusHours(1), 3L}, getResultArray(results.get(1), querySignature));
}
@ -1147,7 +1147,7 @@ public class CursorFactoryProjectionTest extends InitializedNullHandlingTest
final List<Result<TimeseriesResultValue>> results = resultRows.toList();
Assert.assertEquals(1, results.size());
final RowSignature querySignature = query.getResultSignature(RowSignature.Finalization.YES);
final RowSignature querySignature = query.getResultRowSignature(RowSignature.Finalization.YES);
Assert.assertArrayEquals(new Object[]{TIMESTAMP, 19L}, getResultArray(results.get(0), querySignature));
}
@ -1183,7 +1183,7 @@ public class CursorFactoryProjectionTest extends InitializedNullHandlingTest
final List<Result<TimeseriesResultValue>> results = resultRows.toList();
Assert.assertEquals(8, results.size());
final RowSignature querySignature = query.getResultSignature(RowSignature.Finalization.YES);
final RowSignature querySignature = query.getResultRowSignature(RowSignature.Finalization.YES);
Assert.assertArrayEquals(new Object[]{TIMESTAMP, 1L}, getResultArray(results.get(0), querySignature));
Assert.assertArrayEquals(new Object[]{TIMESTAMP.plusMinutes(2), 1L}, getResultArray(results.get(1), querySignature));
Assert.assertArrayEquals(new Object[]{TIMESTAMP.plusMinutes(4), 2L}, getResultArray(results.get(2), querySignature));

View File

@ -19,11 +19,9 @@
package org.apache.druid.quidem;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.inject.AbstractModule;
import com.google.inject.Injector;
import com.google.inject.Key;
import com.google.inject.Module;
import com.google.inject.Provides;
@ -44,7 +42,6 @@ import org.apache.druid.guice.AnnouncerModule;
import org.apache.druid.guice.BrokerProcessingModule;
import org.apache.druid.guice.BrokerServiceModule;
import org.apache.druid.guice.CoordinatorDiscoveryModule;
import org.apache.druid.guice.DruidInjectorBuilder;
import org.apache.druid.guice.ExpressionModule;
import org.apache.druid.guice.ExtensionsModule;
import org.apache.druid.guice.JacksonConfigManagerModule;
@ -73,20 +70,14 @@ import org.apache.druid.initialization.CoreInjectorBuilder;
import org.apache.druid.initialization.Log4jShutterDownerModule;
import org.apache.druid.initialization.ServerInjectorBuilder;
import org.apache.druid.initialization.TombstoneDataStorageModule;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.metadata.storage.derby.DerbyMetadataStorageDruidModule;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.RetryQueryRunnerConfig;
import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider;
import org.apache.druid.rpc.guice.ServiceClientModule;
import org.apache.druid.segment.join.JoinableFactoryWrapper;
import org.apache.druid.segment.writeout.SegmentWriteOutMediumModule;
import org.apache.druid.server.BrokerQueryResource;
import org.apache.druid.server.ClientInfoResource;
import org.apache.druid.server.DruidNode;
import org.apache.druid.server.QueryLifecycleFactory;
import org.apache.druid.server.ResponseContextConfig;
import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker;
import org.apache.druid.server.SubqueryGuardrailHelper;
import org.apache.druid.server.SubqueryGuardrailHelperProvider;
import org.apache.druid.server.coordination.ServerType;
@ -101,50 +92,33 @@ import org.apache.druid.server.metrics.QueryCountStatsProvider;
import org.apache.druid.server.metrics.SubqueryCountStatsProvider;
import org.apache.druid.server.router.TieredBrokerConfig;
import org.apache.druid.server.security.TLSCertificateCheckerModule;
import org.apache.druid.sql.calcite.run.SqlEngine;
import org.apache.druid.sql.calcite.schema.BrokerSegmentMetadataCache;
import org.apache.druid.sql.calcite.schema.DruidSchemaName;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.apache.druid.sql.calcite.util.SqlTestFramework;
import org.apache.druid.sql.calcite.util.SqlTestFramework.Builder;
import org.apache.druid.sql.calcite.util.SqlTestFramework.PlannerComponentSupplier;
import org.apache.druid.sql.calcite.util.SqlTestFramework.QueryComponentSupplier;
import org.apache.druid.sql.calcite.util.SqlTestFramework.QueryComponentSupplierDelegate;
import org.apache.druid.sql.guice.SqlModule;
import org.apache.druid.storage.StorageConnectorModule;
import org.apache.druid.timeline.PruneLoadSpec;
import org.eclipse.jetty.server.Server;
import java.io.IOException;
import java.util.List;
import java.util.Properties;
/**
* A wrapper class to expose a {@link QueryComponentSupplier} as a Broker service.
*/
public class ExposedAsBrokerQueryComponentSupplierWrapper implements QueryComponentSupplier
public class ExposedAsBrokerQueryComponentSupplierWrapper extends QueryComponentSupplierDelegate
{
private QueryComponentSupplier delegate;
public ExposedAsBrokerQueryComponentSupplierWrapper(QueryComponentSupplier delegate)
{
this.delegate = delegate;
}
@Override
public void gatherProperties(Properties properties)
{
delegate.gatherProperties(properties);
}
@Override
public void configureGuice(DruidInjectorBuilder builder)
{
super(delegate);
}
@Override
public void configureGuice(CoreInjectorBuilder builder, List<Module> overrideModules)
{
delegate.configureGuice(builder);
super.configureGuice(builder);
installForServerModules(builder);
builder.add(new QueryRunnerFactoryModule());
@ -154,55 +128,6 @@ public class ExposedAsBrokerQueryComponentSupplierWrapper implements QueryCompon
builder.add(QuidemCaptureModule.class);
}
@Override
public QueryRunnerFactoryConglomerate createCongolmerate(Builder builder, Closer closer, ObjectMapper om)
{
return delegate.createCongolmerate(builder, closer, om);
}
@Override
public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker(QueryRunnerFactoryConglomerate conglomerate,
JoinableFactoryWrapper joinableFactory, Injector injector)
{
return delegate.createQuerySegmentWalker(conglomerate, joinableFactory, injector);
}
@Override
public SqlEngine createEngine(QueryLifecycleFactory qlf, ObjectMapper objectMapper, Injector injector)
{
return delegate.createEngine(qlf, objectMapper, injector);
}
@Override
public void configureJsonMapper(ObjectMapper mapper)
{
delegate.configureJsonMapper(mapper);
}
@Override
public JoinableFactoryWrapper createJoinableFactoryWrapper(LookupExtractorFactoryContainerProvider lookupProvider)
{
return delegate.createJoinableFactoryWrapper(lookupProvider);
}
@Override
public void finalizeTestFramework(SqlTestFramework sqlTestFramework)
{
delegate.finalizeTestFramework(sqlTestFramework);
}
@Override
public void close() throws IOException
{
delegate.close();
}
@Override
public PlannerComponentSupplier getPlannerComponentSupplier()
{
return delegate.getPlannerComponentSupplier();
}
public static class BrokerTestModule extends AbstractModule
{
@Override
@ -336,10 +261,4 @@ public class ExposedAsBrokerQueryComponentSupplierWrapper implements QueryCompon
}
);
}
@Override
public Boolean isExplainSupported()
{
return delegate.isExplainSupported();
}
}

View File

@ -65,9 +65,9 @@ import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.QueryMetrics;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.Result;
import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.query.aggregation.MetricManipulatorFns;
@ -117,7 +117,7 @@ import java.util.stream.Collectors;
public class CachingClusteredClient implements QuerySegmentWalker
{
private static final EmittingLogger log = new EmittingLogger(CachingClusteredClient.class);
private final QueryToolChestWarehouse warehouse;
private final QueryRunnerFactoryConglomerate conglomerate;
private final TimelineServerView serverView;
private final Cache cache;
private final ObjectMapper objectMapper;
@ -131,7 +131,7 @@ public class CachingClusteredClient implements QuerySegmentWalker
@Inject
public CachingClusteredClient(
QueryToolChestWarehouse warehouse,
QueryRunnerFactoryConglomerate conglomerate,
TimelineServerView serverView,
Cache cache,
@Smile ObjectMapper objectMapper,
@ -144,7 +144,7 @@ public class CachingClusteredClient implements QuerySegmentWalker
ServiceEmitter emitter
)
{
this.warehouse = warehouse;
this.conglomerate = conglomerate;
this.serverView = serverView;
this.cache = cache;
this.objectMapper = objectMapper;
@ -274,7 +274,7 @@ public class CachingClusteredClient implements QuerySegmentWalker
this.queryPlus = queryPlus;
this.responseContext = responseContext;
this.query = queryPlus.getQuery();
this.toolChest = warehouse.getToolChest(query);
this.toolChest = conglomerate.getToolChest(query);
this.strategy = toolChest.getCacheStrategy(query, objectMapper);
this.dataSourceAnalysis = query.getDataSource().getAnalysis();

View File

@ -47,9 +47,9 @@ import org.apache.druid.query.QueryContext;
import org.apache.druid.query.QueryMetrics;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QueryTimeoutException;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.QueryWatcher;
import org.apache.druid.query.ResourceLimitExceededException;
import org.apache.druid.query.aggregation.MetricManipulatorFns;
@ -93,7 +93,7 @@ public class DirectDruidClient<T> implements QueryRunner<T>
private static final Logger log = new Logger(DirectDruidClient.class);
private static final int VAL_TO_REDUCE_REMAINING_RESPONSES = -1;
private final QueryToolChestWarehouse warehouse;
private final QueryRunnerFactoryConglomerate conglomerate;
private final QueryWatcher queryWatcher;
private final ObjectMapper objectMapper;
private final HttpClient httpClient;
@ -122,7 +122,7 @@ public class DirectDruidClient<T> implements QueryRunner<T>
}
public DirectDruidClient(
QueryToolChestWarehouse warehouse,
QueryRunnerFactoryConglomerate conglomerate,
QueryWatcher queryWatcher,
ObjectMapper objectMapper,
HttpClient httpClient,
@ -132,7 +132,7 @@ public class DirectDruidClient<T> implements QueryRunner<T>
ScheduledExecutorService queryCancellationExecutor
)
{
this.warehouse = warehouse;
this.conglomerate = conglomerate;
this.queryWatcher = queryWatcher;
this.objectMapper = objectMapper;
this.httpClient = httpClient;
@ -154,7 +154,7 @@ public class DirectDruidClient<T> implements QueryRunner<T>
public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext context)
{
final Query<T> query = queryPlus.getQuery();
QueryToolChest<T, Query<T>> toolChest = warehouse.getToolChest(query);
QueryToolChest<T, Query<T>> toolChest = conglomerate.getToolChest(query);
boolean isBySegment = query.context().isBySegment();
final JavaType queryResultType = isBySegment ? toolChest.getBySegmentResultType() : toolChest.getBaseResultType();

View File

@ -27,7 +27,7 @@ import org.apache.druid.guice.annotations.Smile;
import org.apache.druid.java.util.common.concurrent.ScheduledExecutors;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.java.util.http.client.HttpClient;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QueryWatcher;
import org.apache.druid.utils.JvmUtils;
@ -40,7 +40,7 @@ import java.util.concurrent.ScheduledExecutorService;
public class DirectDruidClientFactory
{
private final ServiceEmitter emitter;
private final QueryToolChestWarehouse warehouse;
private final QueryRunnerFactoryConglomerate conglomerate;
private final QueryWatcher queryWatcher;
private final ObjectMapper smileMapper;
private final HttpClient httpClient;
@ -49,14 +49,14 @@ public class DirectDruidClientFactory
@Inject
public DirectDruidClientFactory(
final ServiceEmitter emitter,
final QueryToolChestWarehouse warehouse,
final QueryRunnerFactoryConglomerate conglomerate,
final QueryWatcher queryWatcher,
final @Smile ObjectMapper smileMapper,
final @EscalatedClient HttpClient httpClient
)
{
this.emitter = emitter;
this.warehouse = warehouse;
this.conglomerate = conglomerate;
this.queryWatcher = queryWatcher;
this.smileMapper = smileMapper;
this.httpClient = httpClient;
@ -68,7 +68,7 @@ public class DirectDruidClientFactory
public DirectDruidClient makeDirectClient(DruidServer server)
{
return new DirectDruidClient(
warehouse,
conglomerate,
queryWatcher,
smileMapper,
httpClient,

View File

@ -17,37 +17,27 @@
* under the License.
*/
package org.apache.druid.query;
package org.apache.druid.guice;
import org.apache.druid.java.util.common.logger.Logger;
import com.google.inject.Inject;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryToolChestWarehouse;
/**
*/
@Deprecated
public class ReflectionQueryToolChestWarehouse implements QueryToolChestWarehouse
public class ConglomerateBackedToolChestWarehouse implements QueryToolChestWarehouse
{
private static final Logger log = new Logger(ReflectionQueryToolChestWarehouse.class);
private QueryRunnerFactoryConglomerate conglomerate;
private final ClassValue<QueryToolChest<?, ?>> toolChests = new ClassValue<QueryToolChest<?, ?>>()
@Inject
public ConglomerateBackedToolChestWarehouse(QueryRunnerFactoryConglomerate conglomerate)
{
@Override
protected QueryToolChest<?, ?> computeValue(Class<?> type)
{
try {
final Class<?> queryToolChestClass = Class.forName(type.getName() + "QueryToolChest");
return (QueryToolChest<?, ?>) queryToolChestClass.newInstance();
}
catch (Exception e) {
log.warn(e, "Unable to load interface[QueryToolChest] for input class[%s]", type);
throw new RuntimeException(e);
}
}
};
this.conglomerate = conglomerate;
}
@Override
@SuppressWarnings("unchecked")
public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(QueryType query)
{
return (QueryToolChest<T, QueryType>) toolChests.get(query.getClass());
return conglomerate.getToolChest(query);
}
}

View File

@ -26,6 +26,7 @@ import com.google.inject.multibindings.Multibinder;
import org.apache.druid.java.util.metrics.Monitor;
import org.apache.druid.query.DataSource;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryLogic;
import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.segment.SegmentWrangler;
@ -52,6 +53,42 @@ public class DruidBinders
);
}
public static MapBinder<Class<? extends Query>, QueryLogic> queryLogicBinderType(Binder binder)
{
return MapBinder.newMapBinder(
binder,
new TypeLiteral<Class<? extends Query>>() {},
new TypeLiteral<QueryLogic>() {}
);
}
public static QueryLogicBinder queryLogicBinder(Binder binder)
{
return new QueryLogicBinder(binder);
}
public static class QueryLogicBinder
{
private MapBinder<Class<? extends Query>, QueryLogic> queryLogicMapBinder;
private Binder binder;
public QueryLogicBinder(Binder binder)
{
this.binder = binder;
queryLogicMapBinder = DruidBinders.queryLogicBinderType(binder);
}
QueryLogicBinder bindQueryLogic(
Class<? extends Query> queryTypeClazz,
Class<? extends QueryLogic> queryLogicClazz)
{
queryLogicMapBinder.addBinding(queryTypeClazz).to(queryLogicClazz);
binder.bind(queryLogicClazz).in(LazySingleton.class);
return this;
}
}
public static Multibinder<KeyHolder<DruidNode>> discoveryAnnouncementBinder(Binder binder)
{
return Multibinder.newSetBinder(binder, new TypeLiteral<KeyHolder<DruidNode>>() {});

View File

@ -46,6 +46,8 @@ import org.apache.druid.query.timeseries.TimeseriesQuery;
import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory;
import org.apache.druid.query.topn.TopNQuery;
import org.apache.druid.query.topn.TopNQueryRunnerFactory;
import org.apache.druid.query.union.UnionQuery;
import org.apache.druid.query.union.UnionQueryLogic;
import org.apache.druid.server.QueryScheduler;
import org.apache.druid.server.QuerySchedulerProvider;
@ -87,6 +89,9 @@ public class QueryRunnerFactoryModule extends QueryToolChestModule
queryFactoryBinder.addBinding(entry.getKey()).to(entry.getValue());
binder.bind(entry.getValue()).in(LazySingleton.class);
}
DruidBinders.queryLogicBinder(binder)
.bindQueryLogic(UnionQuery.class, UnionQueryLogic.class);
}
@LazySingleton

View File

@ -27,7 +27,6 @@ import com.google.inject.multibindings.MapBinder;
import org.apache.druid.query.DefaultGenericQueryMetricsFactory;
import org.apache.druid.query.DefaultQueryConfig;
import org.apache.druid.query.GenericQueryMetricsFactory;
import org.apache.druid.query.MapQueryToolChestWarehouse;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryToolChestWarehouse;
@ -62,7 +61,6 @@ import org.apache.druid.query.topn.TopNQuery;
import org.apache.druid.query.topn.TopNQueryConfig;
import org.apache.druid.query.topn.TopNQueryMetricsFactory;
import org.apache.druid.query.topn.TopNQueryQueryToolChest;
import java.util.Map;
/**
@ -98,7 +96,7 @@ public class QueryToolChestModule implements Module
binder.bind(entry.getValue()).in(LazySingleton.class);
}
binder.bind(QueryToolChestWarehouse.class).to(MapQueryToolChestWarehouse.class);
binder.bind(QueryToolChestWarehouse.class).to(ConglomerateBackedToolChestWarehouse.class);
JsonConfigProvider.bind(binder, "druid.query.default", DefaultQueryConfig.class);
JsonConfigProvider.bind(binder, "druid.query.groupBy", GroupByQueryConfig.class);

View File

@ -26,6 +26,7 @@ import org.apache.druid.common.guava.FutureUtils;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.jackson.JacksonUtils;
import org.apache.druid.java.util.http.client.response.BytesFullResponseHandler;
import org.apache.druid.java.util.http.client.response.BytesFullResponseHolder;
import org.apache.druid.messages.MessageBatch;
import org.apache.druid.rpc.RequestBuilder;
import org.apache.druid.rpc.ServiceClient;
@ -68,11 +69,12 @@ public class MessageRelayClientImpl<MessageType> implements MessageRelayClient<M
startWatermark
);
ListenableFuture<BytesFullResponseHolder> asyncRequest = serviceClient.asyncRequest(
new RequestBuilder(HttpMethod.GET, path),
new BytesFullResponseHandler()
);
return FutureUtils.transform(
serviceClient.asyncRequest(
new RequestBuilder(HttpMethod.GET, path),
new BytesFullResponseHandler()
),
asyncRequest,
holder -> {
if (holder.getResponse().getStatus().getCode() == HttpStatus.NO_CONTENT_204) {
return new MessageBatch<>(Collections.emptyList(), epoch, startWatermark);

View File

@ -35,9 +35,9 @@ import org.apache.druid.query.Queries;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.server.SetAndVerifyContextQueryRunner;
@ -67,22 +67,22 @@ public class SegmentMetadataQuerySegmentWalker implements QuerySegmentWalker
private static final EmittingLogger log = new EmittingLogger(SegmentMetadataQuerySegmentWalker.class);
private final CoordinatorServerView serverView;
private final DruidHttpClientConfig httpClientConfig;
private final QueryToolChestWarehouse warehouse;
private final ServerConfig serverConfig;
private final ServiceEmitter emitter;
protected final QueryRunnerFactoryConglomerate conglomerate;
protected final ServerConfig serverConfig;
@Inject
public SegmentMetadataQuerySegmentWalker(
final CoordinatorServerView serverView,
final DruidHttpClientConfig httpClientConfig,
final QueryToolChestWarehouse warehouse,
final QueryRunnerFactoryConglomerate conglomerate,
final ServerConfig serverConfig,
final ServiceEmitter emitter
)
{
this.serverView = serverView;
this.httpClientConfig = httpClientConfig;
this.warehouse = warehouse;
this.conglomerate = conglomerate;
this.emitter = emitter;
this.serverConfig = serverConfig;
}
@ -112,7 +112,7 @@ public class SegmentMetadataQuerySegmentWalker implements QuerySegmentWalker
private <T> QueryRunner<T> decorateRunner(Query<T> query, QueryRunner<T> baseClusterRunner)
{
final QueryToolChest<T, Query<T>> toolChest = warehouse.getToolChest(query);
final QueryToolChest<T, Query<T>> toolChest = conglomerate.getToolChest(query);
final SetAndVerifyContextQueryRunner<T> baseRunner = new SetAndVerifyContextQueryRunner<>(
serverConfig,
@ -141,7 +141,7 @@ public class SegmentMetadataQuerySegmentWalker implements QuerySegmentWalker
final TimelineLookup<String, SegmentLoadInfo> timelineLookup = timelineConverter.apply(timeline);
QueryToolChest<T, Query<T>> toolChest = warehouse.getToolChest(query);
QueryToolChest<T, Query<T>> toolChest = conglomerate.getToolChest(query);
Set<Pair<SegmentDescriptor, SegmentLoadInfo>> segmentAndServers = computeSegmentsToQuery(timelineLookup, query, toolChest);
queryPlus = queryPlus.withQueryMetrics(toolChest);

View File

@ -21,6 +21,7 @@ package org.apache.druid.server;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.inject.Inject;
import org.apache.commons.lang3.StringUtils;
@ -47,13 +48,16 @@ import org.apache.druid.query.GlobalTableDataSource;
import org.apache.druid.query.InlineDataSource;
import org.apache.druid.query.PostProcessingOperator;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryContext;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.QueryDataSource;
import org.apache.druid.query.QueryLogic;
import org.apache.druid.query.QueryLogicCompatToolChest;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.ResourceLimitExceededException;
import org.apache.druid.query.ResultLevelCachingQueryRunner;
import org.apache.druid.query.ResultSerializationMode;
@ -103,7 +107,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
private final ServiceEmitter emitter;
private final QuerySegmentWalker clusterClient;
private final QuerySegmentWalker localClient;
private final QueryToolChestWarehouse warehouse;
private final QueryRunnerFactoryConglomerate conglomerate;
private final JoinableFactory joinableFactory;
private final RetryQueryRunnerConfig retryConfig;
private final ObjectMapper objectMapper;
@ -117,7 +121,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
ServiceEmitter emitter,
QuerySegmentWalker clusterClient,
QuerySegmentWalker localClient,
QueryToolChestWarehouse warehouse,
QueryRunnerFactoryConglomerate conglomerate,
JoinableFactory joinableFactory,
RetryQueryRunnerConfig retryConfig,
ObjectMapper objectMapper,
@ -131,7 +135,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
this.emitter = emitter;
this.clusterClient = clusterClient;
this.localClient = localClient;
this.warehouse = warehouse;
this.conglomerate = conglomerate;
this.joinableFactory = joinableFactory;
this.retryConfig = retryConfig;
this.objectMapper = objectMapper;
@ -147,7 +151,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
ServiceEmitter emitter,
CachingClusteredClient clusterClient,
LocalQuerySegmentWalker localClient,
QueryToolChestWarehouse warehouse,
QueryRunnerFactoryConglomerate conglomerate,
JoinableFactory joinableFactory,
RetryQueryRunnerConfig retryConfig,
ObjectMapper objectMapper,
@ -162,7 +166,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
emitter,
clusterClient,
(QuerySegmentWalker) localClient,
warehouse,
conglomerate,
joinableFactory,
retryConfig,
objectMapper,
@ -175,9 +179,27 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
}
@Override
public <T> QueryRunner<T> getQueryRunnerForIntervals(final Query<T> query, final Iterable<Interval> intervals)
public <T> QueryRunner<T> getQueryRunnerForIntervals(Query<T> query, final Iterable<Interval> intervals)
{
final QueryToolChest<T, Query<T>> toolChest = warehouse.getToolChest(query);
QueryContext context = query.context();
final int maxSubqueryRows = context.getMaxSubqueryRows(serverConfig.getMaxSubqueryRows());
final String maxSubqueryMemoryString = context.getMaxSubqueryMemoryBytes(serverConfig.getMaxSubqueryBytes());
final long maxSubqueryMemory = subqueryGuardrailHelper.convertSubqueryLimitStringToLong(maxSubqueryMemoryString);
final boolean useNestedForUnknownTypeInSubquery = context
.isUseNestedForUnknownTypeInSubquery(serverConfig.isuseNestedForUnknownTypeInSubquery());
final QueryLogic queryExecutor = conglomerate.getQueryLogic(query);
if (queryExecutor != null) {
query = query.withOverriddenContext(
ImmutableMap.of(
QueryContexts.USE_NESTED_FOR_UNKNOWN_TYPE_IN_SUBQUERY,
useNestedForUnknownTypeInSubquery
)
);
return (QueryRunner<T>) queryExecutor.entryPoint(query, this);
}
final QueryToolChest<T, Query<T>> toolChest = conglomerate.getToolChest(query);
// transform TableDataSource to GlobalTableDataSource when eligible
// before further transformation to potentially inline
@ -192,15 +214,8 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
newQuery = ResourceIdPopulatingQueryRunner.populateResourceId(newQuery);
final DataSource freeTradeDataSource = globalizeIfPossible(newQuery.getDataSource());
// do an inlining dry run to see if any inlining is necessary, without actually running the queries.
final int maxSubqueryRows = query.context().getMaxSubqueryRows(serverConfig.getMaxSubqueryRows());
final String maxSubqueryMemoryString = query.context()
.getMaxSubqueryMemoryBytes(serverConfig.getMaxSubqueryBytes());
final long maxSubqueryMemory = subqueryGuardrailHelper.convertSubqueryLimitStringToLong(maxSubqueryMemoryString);
final boolean useNestedForUnknownTypeInSubquery = query.context()
.isUseNestedForUnknownTypeInSubquery(serverConfig.isuseNestedForUnknownTypeInSubquery());
final DataSource inlineDryRun = inlineIfNecessary(
freeTradeDataSource,
toolChest,
@ -218,7 +233,6 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
// Dry run didn't go well.
throw new ISE("Cannot handle subquery structure for dataSource: %s", query.getDataSource());
}
// Now that we know the structure is workable, actually do the inlining (if necessary).
AtomicLong memoryLimitAcc = new AtomicLong(0);
DataSource maybeInlinedDataSource = inlineIfNecessary(
@ -289,17 +303,15 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
*/
private <T> boolean canRunQueryUsingLocalWalker(Query<T> query)
{
final DataSource dataSourceFromQuery = query.getDataSource();
final DataSourceAnalysis analysis = dataSourceFromQuery.getAnalysis();
final QueryToolChest<T, Query<T>> toolChest = warehouse.getToolChest(query);
final DataSourceAnalysis analysis = query.getDataSourceAnalysis();
final QueryToolChest<T, Query<T>> toolChest = conglomerate.getToolChest(query);
// 1) Must be based on a concrete datasource that is not a table.
// 2) Must be based on globally available data (so we have a copy here on the Broker).
// 3) If there is an outer query, it must be handleable by the query toolchest (the local walker does not handle
// subqueries on its own).
return analysis.isConcreteBased() && !analysis.isConcreteAndTableBased() && dataSourceFromQuery.isGlobal()
&& (!(dataSourceFromQuery instanceof QueryDataSource)
|| toolChest.canPerformSubquery(((QueryDataSource) dataSourceFromQuery).getQuery()));
return analysis.isConcreteBased() && !analysis.isConcreteAndTableBased() && analysis.isGlobal()
&& toolChest.canExecuteFully(query);
}
/**
@ -308,16 +320,14 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
*/
private <T> boolean canRunQueryUsingClusterWalker(Query<T> query)
{
final DataSource dataSourceFromQuery = query.getDataSource();
final DataSourceAnalysis analysis = dataSourceFromQuery.getAnalysis();
final QueryToolChest<T, Query<T>> toolChest = warehouse.getToolChest(query);
final QueryToolChest<T, Query<T>> toolChest = conglomerate.getToolChest(query);
final DataSourceAnalysis analysis = query.getDataSourceAnalysis();
// 1) Must be based on a concrete table (the only shape the Druid cluster can handle).
// 2) If there is an outer query, it must be handleable by the query toolchest (the cluster walker does not handle
// subqueries on its own).
return analysis.isConcreteAndTableBased()
&& (!(dataSourceFromQuery instanceof QueryDataSource)
|| toolChest.canPerformSubquery(((QueryDataSource) dataSourceFromQuery).getQuery()));
&& toolChest.canExecuteFully(query);
}
@ -375,7 +385,43 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
if (dataSource instanceof QueryDataSource) {
// This datasource is a subquery.
final Query subQuery = ((QueryDataSource) dataSource).getQuery();
final QueryToolChest toolChest = warehouse.getToolChest(subQuery);
final QueryToolChest toolChest = conglomerate.getToolChest(subQuery);
final QueryLogic subQueryLogic = conglomerate.getQueryLogic(subQuery);
if (subQueryLogic != null) {
final Sequence<?> queryResults;
if (dryRun) {
queryResults = Sequences.empty();
} else {
Query subQueryWithSerialization = subQuery.withOverriddenContext(
Collections.singletonMap(
ResultSerializationMode.CTX_SERIALIZATION_PARAMETER,
ClientQuerySegmentWalkerUtils.getLimitType(maxSubqueryMemory, cannotMaterializeToFrames.get())
.serializationMode()
.toString()
)
);
queryResults = subQueryLogic
.entryPoint(subQueryWithSerialization, this)
.run(QueryPlus.wrap(subQueryWithSerialization), DirectDruidClient.makeResponseContextForQuery());
}
return toInlineDataSource(
subQuery,
queryResults,
(QueryToolChest) new QueryLogicCompatToolChest(subQuery.getResultRowSignature()),
subqueryRowLimitAccumulator,
subqueryMemoryLimitAccumulator,
cannotMaterializeToFrames,
maxSubqueryRows,
maxSubqueryMemory,
useNestedForUnknownTypeInSubquery,
subqueryStatsProvider,
!dryRun,
emitter
);
}
if (toolChestIfOutermost != null && toolChestIfOutermost.canPerformSubquery(subQuery)) {
// Strip outer queries that are handleable by the toolchest, and inline subqueries that may be underneath
@ -443,6 +489,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
.toString()
)
);
queryResults = subQueryWithSerialization
.getRunner(this)
.run(QueryPlus.wrap(subQueryWithSerialization), DirectDruidClient.makeResponseContextForQuery());
@ -451,7 +498,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
return toInlineDataSource(
subQuery,
queryResults,
warehouse.getToolChest(subQuery),
toolChest,
subqueryRowLimitAccumulator,
subqueryMemoryLimitAccumulator,
cannotMaterializeToFrames,
@ -464,21 +511,26 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
);
} else {
// Cannot inline subquery. Attempt to inline one level deeper, and then try again.
List<DataSource> newDataSources = new ArrayList<DataSource>();
for (DataSource ds : dataSource.getChildren()) {
newDataSources.add(
inlineIfNecessary(
ds,
null,
subqueryRowLimitAccumulator,
subqueryMemoryLimitAccumulator,
cannotMaterializeToFrames,
maxSubqueryRows,
maxSubqueryMemory,
useNestedForUnknownTypeInSubquery,
dryRun
)
);
}
return inlineIfNecessary(
dataSource.withChildren(
Collections.singletonList(
inlineIfNecessary(
Iterables.getOnlyElement(dataSource.getChildren()),
null,
subqueryRowLimitAccumulator,
subqueryMemoryLimitAccumulator,
cannotMaterializeToFrames,
maxSubqueryRows,
maxSubqueryMemory,
useNestedForUnknownTypeInSubquery,
dryRun
)
)
newDataSources
),
toolChestIfOutermost,
subqueryRowLimitAccumulator,
@ -521,7 +573,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
*/
private <T> QueryRunner<T> decorateClusterRunner(Query<T> query, QueryRunner<T> baseClusterRunner)
{
final QueryToolChest<T, Query<T>> toolChest = warehouse.getToolChest(query);
final QueryToolChest<T, Query<T>> toolChest = conglomerate.getToolChest(query);
final SetAndVerifyContextQueryRunner<T> baseRunner = new SetAndVerifyContextQueryRunner<>(
serverConfig,
@ -623,7 +675,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker
)
{
if (currentDataSource instanceof QueryDataSource
&& queryDataSourceToSubqueryIds.containsKey((QueryDataSource) currentDataSource)) {
&& queryDataSourceToSubqueryIds.containsKey(currentDataSource)) {
QueryDataSource queryDataSource = (QueryDataSource) currentDataSource;
Pair<Integer, Integer> nestingInfo = queryDataSourceToSubqueryIds.get(queryDataSource);
String subQueryId = nestingInfo.lhs + "." + nestingInfo.rhs;

View File

@ -42,10 +42,10 @@ import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.QueryInterruptedException;
import org.apache.druid.query.QueryMetrics;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.query.QueryTimeoutException;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.server.QueryResource.ResourceIOReaderWriter;
import org.apache.druid.server.log.RequestLogger;
@ -86,7 +86,7 @@ public class QueryLifecycle
{
private static final Logger log = new Logger(QueryLifecycle.class);
private final QueryToolChestWarehouse warehouse;
private final QueryRunnerFactoryConglomerate conglomerate;
private final QuerySegmentWalker texasRanger;
private final GenericQueryMetricsFactory queryMetricsFactory;
private final ServiceEmitter emitter;
@ -107,7 +107,7 @@ public class QueryLifecycle
private Set<String> userContextKeys;
public QueryLifecycle(
final QueryToolChestWarehouse warehouse,
final QueryRunnerFactoryConglomerate conglomerate,
final QuerySegmentWalker texasRanger,
final GenericQueryMetricsFactory queryMetricsFactory,
final ServiceEmitter emitter,
@ -119,7 +119,7 @@ public class QueryLifecycle
final long startNs
)
{
this.warehouse = warehouse;
this.conglomerate = conglomerate;
this.texasRanger = texasRanger;
this.queryMetricsFactory = queryMetricsFactory;
this.emitter = emitter;
@ -207,7 +207,7 @@ public class QueryLifecycle
Map<String, Object> mergedUserAndConfigContext = QueryContexts.override(defaultQueryConfig.getContext(), baseQuery.getContext());
mergedUserAndConfigContext.put(BaseQuery.QUERY_ID, queryId);
this.baseQuery = baseQuery.withOverriddenContext(mergedUserAndConfigContext);
this.toolChest = warehouse.getToolChest(this.baseQuery);
this.toolChest = conglomerate.getToolChest(this.baseQuery);
}
/**

View File

@ -25,8 +25,8 @@ import org.apache.druid.guice.LazySingleton;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.query.DefaultQueryConfig;
import org.apache.druid.query.GenericQueryMetricsFactory;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.server.log.RequestLogger;
import org.apache.druid.server.security.AuthConfig;
import org.apache.druid.server.security.AuthorizerMapper;
@ -34,7 +34,7 @@ import org.apache.druid.server.security.AuthorizerMapper;
@LazySingleton
public class QueryLifecycleFactory
{
private final QueryToolChestWarehouse warehouse;
private final QueryRunnerFactoryConglomerate conglomerate;
private final QuerySegmentWalker texasRanger;
private final GenericQueryMetricsFactory queryMetricsFactory;
private final ServiceEmitter emitter;
@ -45,7 +45,7 @@ public class QueryLifecycleFactory
@Inject
public QueryLifecycleFactory(
final QueryToolChestWarehouse warehouse,
final QueryRunnerFactoryConglomerate conglomerate,
final QuerySegmentWalker texasRanger,
final GenericQueryMetricsFactory queryMetricsFactory,
final ServiceEmitter emitter,
@ -55,7 +55,7 @@ public class QueryLifecycleFactory
final Supplier<DefaultQueryConfig> queryConfigSupplier
)
{
this.warehouse = warehouse;
this.conglomerate = conglomerate;
this.texasRanger = texasRanger;
this.queryMetricsFactory = queryMetricsFactory;
this.emitter = emitter;
@ -68,7 +68,7 @@ public class QueryLifecycleFactory
public QueryLifecycle factorize()
{
return new QueryLifecycle(
warehouse,
conglomerate,
texasRanger,
queryMetricsFactory,
emitter,

View File

@ -26,6 +26,7 @@ import org.apache.druid.common.exception.ErrorResponseTransformStrategy;
import org.apache.druid.common.exception.NoErrorResponseTransformStrategy;
import org.apache.druid.java.util.common.HumanReadableBytes;
import org.apache.druid.java.util.common.HumanReadableBytesRange;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.server.SubqueryGuardrailHelper;
import org.apache.druid.utils.JvmUtils;
import org.joda.time.Period;
@ -46,8 +47,6 @@ public class ServerConfig
{
public static final int DEFAULT_GZIP_INFLATE_BUFFER_SIZE = 4096;
private static final boolean DEFAULT_USE_NESTED_FOR_UNKNOWN_TYPE_IN_SUBQUERY = false;
/**
* The ServerConfig is normally created using {@link org.apache.druid.guice.JsonConfigProvider} binding.
*
@ -143,7 +142,7 @@ public class ServerConfig
private String maxSubqueryBytes = SubqueryGuardrailHelper.LIMIT_DISABLED_VALUE;
@JsonProperty
private boolean useNestedForUnknownTypeInSubquery = DEFAULT_USE_NESTED_FOR_UNKNOWN_TYPE_IN_SUBQUERY;
private boolean useNestedForUnknownTypeInSubquery = QueryContexts.DEFAULT_USE_NESTED_FOR_UNKNOWN_TYPE_IN_SUBQUERY;
@JsonProperty
@Min(1)

View File

@ -38,7 +38,7 @@ import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.http.client.HttpClient;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QueryWatcher;
import org.apache.druid.query.TableDataSource;
import org.apache.druid.segment.TestHelper;
@ -182,7 +182,7 @@ public class BrokerServerViewTest extends CuratorTestBase
createExpected("2011-04-02/2011-04-06", "v2", druidServers.get(2), segments.get(2)),
createExpected("2011-04-06/2011-04-09", "v3", druidServers.get(3), segments.get(3))
),
(List<TimelineObjectHolder>) timeline.lookup(
timeline.lookup(
Intervals.of(
"2011-04-01/2011-04-09"
)
@ -206,7 +206,7 @@ public class BrokerServerViewTest extends CuratorTestBase
createExpected("2011-04-03/2011-04-06", "v1", druidServers.get(1), segments.get(1)),
createExpected("2011-04-06/2011-04-09", "v3", druidServers.get(3), segments.get(3))
),
(List<TimelineObjectHolder>) timeline.lookup(
timeline.lookup(
Intervals.of(
"2011-04-01/2011-04-09"
)
@ -285,7 +285,7 @@ public class BrokerServerViewTest extends CuratorTestBase
createExpected("2011-04-02/2011-04-06", "v2", druidServers.get(2), segments.get(2)),
createExpected("2011-04-06/2011-04-09", "v3", druidServers.get(3), segments.get(3))
),
(List<TimelineObjectHolder>) timeline.lookup(
timeline.lookup(
Intervals.of(
"2011-04-01/2011-04-09"
)
@ -316,7 +316,7 @@ public class BrokerServerViewTest extends CuratorTestBase
createExpected("2011-04-02/2011-04-06", "v2", druidServers.get(2), segments.get(2)),
createExpected("2011-04-06/2011-04-09", "v3", druidServers.get(3), segments.get(3))
),
(List<TimelineObjectHolder>) timeline.lookup(
timeline.lookup(
Intervals.of(
"2011-04-01/2011-04-09"
)
@ -655,7 +655,7 @@ public class BrokerServerViewTest extends CuratorTestBase
DirectDruidClientFactory druidClientFactory = new DirectDruidClientFactory(
new NoopServiceEmitter(),
EasyMock.createMock(QueryToolChestWarehouse.class),
EasyMock.createMock(QueryRunnerFactoryConglomerate.class),
EasyMock.createMock(QueryWatcher.class),
getSmileMapper(),
EasyMock.createMock(HttpClient.class)

View File

@ -35,15 +35,12 @@ import org.apache.druid.client.selector.ServerSelector;
import org.apache.druid.client.selector.TierSelectorStrategy;
import org.apache.druid.guice.http.DruidHttpClientConfig;
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.BrokerParallelMergeConfig;
import org.apache.druid.query.Druids;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryPlus;
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;
@ -57,13 +54,13 @@ import org.apache.druid.timeline.partition.NoneShardSpec;
import org.apache.druid.timeline.partition.SingleElementPartitionChunk;
import org.easymock.EasyMock;
import org.joda.time.Interval;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.ClassRule;
import org.junit.Test;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
@ -78,21 +75,14 @@ import java.util.concurrent.ForkJoinPool;
public class CachingClusteredClientFunctionalityTest
{
private static final ObjectMapper OBJECT_MAPPER = CachingClusteredClientTestUtils.createObjectMapper();
private static final Pair<QueryToolChestWarehouse, Closer> WAREHOUSE_AND_CLOSER =
CachingClusteredClientTestUtils.createWarehouse();
private static final QueryToolChestWarehouse WAREHOUSE = WAREHOUSE_AND_CLOSER.lhs;
private static final Closer RESOURCE_CLOSER = WAREHOUSE_AND_CLOSER.rhs;
private CachingClusteredClient client;
private VersionedIntervalTimeline<String, ServerSelector> timeline;
private TimelineServerView serverView;
private Cache cache;
@AfterClass
public static void tearDownClass() throws IOException
{
RESOURCE_CLOSER.close();
}
@ClassRule
public static QueryStackTests.Junit4ConglomerateRule conglomerateRule = new QueryStackTests.Junit4ConglomerateRule();
@Before
public void setUp()
@ -175,7 +165,7 @@ public class CachingClusteredClientFunctionalityTest
for (String interval : intervals) {
expectedList.add(Intervals.of(interval));
}
Assert.assertEquals((Object) expectedList, context.getUncoveredIntervals());
Assert.assertEquals(expectedList, context.getUncoveredIntervals());
Assert.assertEquals(uncoveredIntervalsOverflowed, context.get(ResponseContext.Keys.UNCOVERED_INTERVALS_OVERFLOWED));
}
@ -241,7 +231,7 @@ public class CachingClusteredClientFunctionalityTest
)
{
return new CachingClusteredClient(
WAREHOUSE,
conglomerateRule.getConglomerate(),
new TimelineServerView()
{
@Override

View File

@ -37,10 +37,12 @@ import org.apache.druid.query.BaseQuery;
import org.apache.druid.query.BrokerParallelMergeConfig;
import org.apache.druid.query.DataSource;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryLogic;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.query.TableDataSource;
import org.apache.druid.query.context.ResponseContext;
@ -128,7 +130,7 @@ public class CachingClusteredClientPerfTest
Mockito.doReturn(Optional.of(timeline)).when(serverView).getTimeline(any());
Mockito.doReturn(new MockQueryRunner()).when(serverView).getQueryRunner(any());
CachingClusteredClient cachingClusteredClient = new CachingClusteredClient(
new MockQueryToolChestWareHouse(),
new MockQueryRunnerFactoryConglomerate(),
serverView,
MapCache.create(1024),
TestHelper.makeJsonMapper(),
@ -170,14 +172,25 @@ public class CachingClusteredClientPerfTest
.build();
}
private static class MockQueryToolChestWareHouse implements QueryToolChestWarehouse
private static class MockQueryRunnerFactoryConglomerate implements QueryRunnerFactoryConglomerate
{
@Override
public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(QueryType query)
{
return new ServerManagerTest.NoopQueryToolChest<>();
}
@Override
public <T, QueryType extends Query<T>> QueryRunnerFactory<T, QueryType> findFactory(QueryType query)
{
return null;
}
@Override
public <T, QueryType extends Query<T>> QueryLogic getQueryLogic(QueryType query)
{
return null;
}
}
private static class MockQueryRunner implements QueryRunner<SegmentDescriptor>

View File

@ -63,7 +63,6 @@ import org.apache.druid.java.util.common.guava.MergeIterable;
import org.apache.druid.java.util.common.guava.Sequence;
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.BrokerParallelMergeConfig;
import org.apache.druid.query.BySegmentResultValueClass;
import org.apache.druid.query.Druids;
@ -74,7 +73,6 @@ import org.apache.druid.query.QueryContext;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.Result;
import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.query.aggregation.AggregatorFactory;
@ -116,6 +114,7 @@ import org.apache.druid.query.topn.TopNQueryQueryToolChest;
import org.apache.druid.query.topn.TopNResultValue;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.server.QueryScheduler;
import org.apache.druid.server.QueryStackTests;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.server.initialization.ServerConfig;
import org.apache.druid.server.metrics.NoopServiceEmitter;
@ -138,15 +137,14 @@ import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.joda.time.Interval;
import org.joda.time.Period;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.ClassRule;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
@ -252,10 +250,9 @@ public class CachingClusteredClientTest
private static final DateTimeZone TIMEZONE = DateTimes.inferTzFromString("America/Los_Angeles");
private static final Granularity PT1H_TZ_GRANULARITY = new PeriodGranularity(new Period("PT1H"), null, TIMEZONE);
private static final String TOP_DIM = "a_dim";
private static final Pair<QueryToolChestWarehouse, Closer> WAREHOUSE_AND_CLOSER =
CachingClusteredClientTestUtils.createWarehouse();
private static final QueryToolChestWarehouse WAREHOUSE = WAREHOUSE_AND_CLOSER.lhs;
private static final Closer RESOURCE_CLOSER = WAREHOUSE_AND_CLOSER.rhs;
@ClassRule
public static QueryStackTests.Junit4ConglomerateRule conglomerateRule = new QueryStackTests.Junit4ConglomerateRule();
private final Random random;
@ -287,12 +284,6 @@ public class CachingClusteredClientTest
);
}
@AfterClass
public static void tearDownClass() throws IOException
{
RESOURCE_CLOSER.close();
}
@Before
public void setUp()
{
@ -2637,7 +2628,7 @@ public class CachingClusteredClientTest
)
{
return new CachingClusteredClient(
WAREHOUSE,
conglomerateRule.getConglomerate(),
new TimelineServerView()
{
@Override

View File

@ -21,69 +21,10 @@ package org.apache.druid.client;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.query.MapQueryToolChestWarehouse;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.query.groupby.GroupByQueryConfig;
import org.apache.druid.query.groupby.GroupByQueryRunnerFactory;
import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
import org.apache.druid.query.groupby.TestGroupByBuffers;
import org.apache.druid.query.search.SearchQuery;
import org.apache.druid.query.search.SearchQueryConfig;
import org.apache.druid.query.search.SearchQueryQueryToolChest;
import org.apache.druid.query.timeboundary.TimeBoundaryQuery;
import org.apache.druid.query.timeboundary.TimeBoundaryQueryQueryToolChest;
import org.apache.druid.query.timeseries.TimeseriesQuery;
import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest;
import org.apache.druid.query.topn.TopNQuery;
import org.apache.druid.query.topn.TopNQueryConfig;
import org.apache.druid.query.topn.TopNQueryQueryToolChest;
public final class CachingClusteredClientTestUtils
{
/**
* Returns a new {@link QueryToolChestWarehouse} for unit tests and a resourceCloser which should be closed at the end
* of the test.
*/
public static Pair<QueryToolChestWarehouse, Closer> createWarehouse()
{
final Closer resourceCloser = Closer.create();
final GroupByQueryRunnerFactory groupByQueryRunnerFactory = GroupByQueryRunnerTest.makeQueryRunnerFactory(
new GroupByQueryConfig(),
resourceCloser.register(TestGroupByBuffers.createDefault())
);
return Pair.of(
new MapQueryToolChestWarehouse(
ImmutableMap.<Class<? extends Query>, QueryToolChest>builder()
.put(
TimeseriesQuery.class,
new TimeseriesQueryQueryToolChest()
)
.put(
TopNQuery.class,
new TopNQueryQueryToolChest(new TopNQueryConfig())
)
.put(
SearchQuery.class,
new SearchQueryQueryToolChest(new SearchQueryConfig())
)
.put(
GroupByQuery.class,
groupByQueryRunnerFactory.getToolchest()
)
.put(TimeBoundaryQuery.class, new TimeBoundaryQueryQueryToolChest())
.build()
),
resourceCloser
);
}
public static ObjectMapper createObjectMapper()
{
final SmileFactory factory = new SmileFactory();
@ -91,8 +32,4 @@ public final class CachingClusteredClientTestUtils
factory.setCodec(objectMapper);
return objectMapper;
}
private CachingClusteredClientTestUtils()
{
}
}

View File

@ -46,9 +46,9 @@ import org.apache.druid.query.QueryInterruptedException;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunnerTestHelper;
import org.apache.druid.query.QueryTimeoutException;
import org.apache.druid.query.ReflectionQueryToolChestWarehouse;
import org.apache.druid.query.Result;
import org.apache.druid.query.timeboundary.TimeBoundaryQuery;
import org.apache.druid.server.QueryStackTests;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.server.metrics.NoopServiceEmitter;
import org.apache.druid.timeline.DataSegment;
@ -62,6 +62,7 @@ import org.joda.time.Duration;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.ClassRule;
import org.junit.Test;
import java.io.ByteArrayInputStream;
@ -78,6 +79,9 @@ import java.util.concurrent.TimeUnit;
public class DirectDruidClientTest
{
@ClassRule
public static QueryStackTests.Junit4ConglomerateRule conglomerateRule = new QueryStackTests.Junit4ConglomerateRule();
private final String hostName = "localhost:8080";
private final DataSegment dataSegment = new DataSegment(
@ -109,7 +113,7 @@ public class DirectDruidClientTest
);
queryCancellationExecutor = Execs.scheduledSingleThreaded("query-cancellation-executor");
client = new DirectDruidClient(
new ReflectionQueryToolChestWarehouse(),
conglomerateRule.getConglomerate(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER,
new DefaultObjectMapper(),
httpClient,
@ -181,7 +185,7 @@ public class DirectDruidClientTest
EasyMock.replay(httpClient);
DirectDruidClient client2 = new DirectDruidClient(
new ReflectionQueryToolChestWarehouse(),
conglomerateRule.getConglomerate(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER,
new DefaultObjectMapper(),
httpClient,
@ -443,7 +447,7 @@ public class DirectDruidClientTest
});
DirectDruidClient client2 = new DirectDruidClient(
new ReflectionQueryToolChestWarehouse(),
conglomerateRule.getConglomerate(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER,
mockObjectMapper,
httpClient,

View File

@ -30,7 +30,7 @@ import org.apache.druid.client.selector.TierSelectorStrategy;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.http.client.HttpClient;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QueryWatcher;
import org.apache.druid.query.TableDataSource;
import org.apache.druid.query.planning.DataSourceAnalysis;
@ -66,14 +66,14 @@ public class SimpleServerView implements TimelineServerView
private final DirectDruidClientFactory clientFactory;
public SimpleServerView(
QueryToolChestWarehouse warehouse,
QueryRunnerFactoryConglomerate conglomerate,
ObjectMapper objectMapper,
HttpClient httpClient
)
{
this.clientFactory = new DirectDruidClientFactory(
new NoopServiceEmitter(),
warehouse,
conglomerate,
NOOP_QUERY_WATCHER,
objectMapper,
httpClient
@ -143,7 +143,7 @@ public class SimpleServerView implements TimelineServerView
public <T> QueryRunner<T> getQueryRunner(DruidServer server)
{
final QueryableDruidServer queryableDruidServer = Preconditions.checkNotNull(servers.get(server), "server");
return (QueryRunner<T>) queryableDruidServer.getQueryRunner();
return queryableDruidServer.getQueryRunner();
}
@Override

View File

@ -92,9 +92,7 @@ public abstract class QueryRunnerBasedOnClusteredClientTestBase
private static final boolean USE_PARALLEL_MERGE_POOL_CONFIGURED = false;
protected final ObjectMapper objectMapper = new DefaultObjectMapper();
protected final QueryToolChestWarehouse toolChestWarehouse;
private final QueryRunnerFactoryConglomerate conglomerate;
protected final QueryRunnerFactoryConglomerate conglomerate;
protected TestHttpClient httpClient;
protected SimpleServerView simpleServerView;
@ -107,17 +105,8 @@ public abstract class QueryRunnerBasedOnClusteredClientTestBase
{
conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(
CLOSER,
() -> TopNQueryConfig.DEFAULT_MIN_TOPN_THRESHOLD
TopNQueryConfig.DEFAULT_MIN_TOPN_THRESHOLD
);
toolChestWarehouse = new QueryToolChestWarehouse()
{
@Override
public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(final QueryType query)
{
return conglomerate.findFactory(query).getToolchest();
}
};
}
@AfterClass
@ -131,9 +120,9 @@ public abstract class QueryRunnerBasedOnClusteredClientTestBase
{
segmentGenerator = new SegmentGenerator();
httpClient = new TestHttpClient(objectMapper);
simpleServerView = new SimpleServerView(toolChestWarehouse, objectMapper, httpClient);
simpleServerView = new SimpleServerView(conglomerate, objectMapper, httpClient);
cachingClusteredClient = new CachingClusteredClient(
toolChestWarehouse,
conglomerate,
simpleServerView,
MapCache.create(0),
objectMapper,

View File

@ -241,7 +241,7 @@ public class ResultLevelCachingQueryRunnerTest extends QueryRunnerBasedOnCluster
new RetryQueryRunnerConfig(),
objectMapper
),
toolChestWarehouse.getToolChest(query),
conglomerate.getToolChest(query),
query,
objectMapper,
cache,

View File

@ -130,7 +130,6 @@ public class CoordinatorSegmentDataCacheConcurrencyTest extends SegmentMetadataC
return 0L;
}
},
queryToolChestWarehouse,
new ServerConfig(),
new NoopServiceEmitter(),
conglomerate,

View File

@ -81,7 +81,6 @@ public class CoordinatorSegmentMetadataCacheTestBase extends SegmentMetadataCach
return 0L;
}
},
queryToolChestWarehouse,
new ServerConfig(),
new NoopServiceEmitter(),
conglomerate,

View File

@ -31,11 +31,8 @@ import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.query.DefaultGenericQueryMetricsFactory;
import org.apache.druid.query.DefaultQueryConfig;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
@ -90,7 +87,6 @@ public abstract class SegmentMetadataCacheTestBase extends InitializedNullHandli
public QueryRunnerFactoryConglomerate conglomerate;
public Closer resourceCloser;
public QueryToolChestWarehouse queryToolChestWarehouse;
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
@ -112,14 +108,6 @@ public abstract class SegmentMetadataCacheTestBase extends InitializedNullHandli
{
resourceCloser = Closer.create();
conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(resourceCloser);
queryToolChestWarehouse = new QueryToolChestWarehouse()
{
@Override
public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(final QueryType query)
{
return conglomerate.findFactory(query).getToolchest();
}
};
}
public void setUpData() throws Exception
@ -299,7 +287,7 @@ public abstract class SegmentMetadataCacheTestBase extends InitializedNullHandli
public QueryLifecycleFactory getQueryLifecycleFactory(QuerySegmentWalker walker)
{
return new QueryLifecycleFactory(
queryToolChestWarehouse,
conglomerate,
walker,
new DefaultGenericQueryMetricsFactory(),
new NoopServiceEmitter(),

View File

@ -19,7 +19,6 @@
package org.apache.druid.segment.metadata;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import org.apache.druid.client.CachingClusteredClientTest.ServerExpectation;
import org.apache.druid.client.CachingClusteredClientTest.ServerExpectations;
@ -38,22 +37,17 @@ import org.apache.druid.java.util.common.guava.Yielder;
import org.apache.druid.java.util.common.guava.Yielders;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.query.DataSource;
import org.apache.druid.query.MapQueryToolChestWarehouse;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.query.TableDataSource;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.metadata.SegmentMetadataQueryConfig;
import org.apache.druid.query.metadata.SegmentMetadataQueryQueryToolChest;
import org.apache.druid.query.metadata.metadata.AllColumnIncluderator;
import org.apache.druid.query.metadata.metadata.SegmentAnalysis;
import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery;
import org.apache.druid.query.spec.MultipleSpecificSegmentSpec;
import org.apache.druid.server.QueryStackTests;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.server.initialization.ServerConfig;
import org.apache.druid.server.metrics.NoopServiceEmitter;
@ -67,6 +61,7 @@ import org.easymock.IAnswer;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Before;
import org.junit.ClassRule;
import org.junit.Test;
import org.mockito.Mockito;
@ -86,24 +81,16 @@ import java.util.stream.Collectors;
public class SegmentMetadataQuerySegmentWalkerTest
{
private final String DATASOURCE = "testDatasource";
private QueryToolChestWarehouse warehouse;
private DruidHttpClientConfig httpClientConfig;
private DruidServer[] servers;
private Random random;
@ClassRule
public static QueryStackTests.Junit4ConglomerateRule conglomerateRule = new QueryStackTests.Junit4ConglomerateRule();
@Before
public void setUp()
{
warehouse = new MapQueryToolChestWarehouse(
ImmutableMap.<Class<? extends Query>, QueryToolChest>builder()
.put(
SegmentMetadataQuery.class,
new SegmentMetadataQueryQueryToolChest(
new SegmentMetadataQueryConfig("P1W")
)
).build());
httpClientConfig = new DruidHttpClientConfig()
{
@Override
@ -169,7 +156,7 @@ public class SegmentMetadataQuerySegmentWalkerTest
SegmentMetadataQuerySegmentWalker walker = new SegmentMetadataQuerySegmentWalker(
new TestCoordinatorServerView(timelines, queryRunnerMap),
httpClientConfig,
warehouse,
conglomerateRule.getConglomerate(),
new ServerConfig(),
new NoopServiceEmitter()
);
@ -264,7 +251,7 @@ public class SegmentMetadataQuerySegmentWalkerTest
SegmentMetadataQuerySegmentWalker walker = new SegmentMetadataQuerySegmentWalker(
new TestCoordinatorServerView(timelines, queryRunnerMap),
httpClientConfig,
warehouse,
conglomerateRule.getConglomerate(),
new ServerConfig(),
new NoopServiceEmitter()
);

View File

@ -33,7 +33,6 @@ import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.segment.QueryableIndex;
@ -47,13 +46,11 @@ import java.util.stream.Collectors;
public class TestSegmentMetadataQueryWalker extends SegmentMetadataQuerySegmentWalker
{
private final QueryRunnerFactoryConglomerate conglomerate;
private final Map<SegmentDescriptor, Pair<QueryableIndex, DataSegment>> queryableIndexMap;
public TestSegmentMetadataQueryWalker(
CoordinatorServerView serverView,
DruidHttpClientConfig httpClientConfig,
QueryToolChestWarehouse warehouse,
ServerConfig serverConfig,
ServiceEmitter emitter,
QueryRunnerFactoryConglomerate conglomerate,
@ -63,11 +60,10 @@ public class TestSegmentMetadataQueryWalker extends SegmentMetadataQuerySegmentW
super(
serverView,
httpClientConfig,
warehouse,
conglomerate,
serverConfig,
emitter
);
this.conglomerate = conglomerate;
this.queryableIndexMap = queryableIndexMap;
}

View File

@ -224,20 +224,18 @@ public class StreamAppenderatorTester implements AutoCloseable
objectMapper,
indexIO,
indexMerger,
new DefaultQueryRunnerFactoryConglomerate(
ImmutableMap.of(
TimeseriesQuery.class, new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
),
ScanQuery.class, new ScanQueryRunnerFactory(
new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()),
new ScanQueryEngine(),
new ScanQueryConfig()
)
DefaultQueryRunnerFactoryConglomerate.buildFromQueryRunnerFactories(ImmutableMap.of(
TimeseriesQuery.class, new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
),
ScanQuery.class, new ScanQueryRunnerFactory(
new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()),
new ScanQueryEngine(),
new ScanQueryConfig()
)
),
)),
announcer,
emitter,
new ForwardingQueryProcessingPool(queryExecutor),
@ -268,20 +266,18 @@ public class StreamAppenderatorTester implements AutoCloseable
objectMapper,
indexIO,
indexMerger,
new DefaultQueryRunnerFactoryConglomerate(
ImmutableMap.of(
TimeseriesQuery.class, new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
),
ScanQuery.class, new ScanQueryRunnerFactory(
new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()),
new ScanQueryEngine(),
new ScanQueryConfig()
)
DefaultQueryRunnerFactoryConglomerate.buildFromQueryRunnerFactories(ImmutableMap.of(
TimeseriesQuery.class, new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
),
ScanQuery.class, new ScanQueryRunnerFactory(
new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()),
new ScanQueryEngine(),
new ScanQueryConfig()
)
),
)),
new NoopDataSegmentAnnouncer(),
emitter,
new ForwardingQueryProcessingPool(queryExecutor),

View File

@ -83,7 +83,7 @@ public class UnifiedIndexerAppenderatorsManagerTest extends InitializedNullHandl
new CachePopulatorStats(),
TestHelper.makeJsonMapper(),
new NoopServiceEmitter(),
() -> new DefaultQueryRunnerFactoryConglomerate(ImmutableMap.of())
() -> DefaultQueryRunnerFactoryConglomerate.buildFromQueryRunnerFactories(ImmutableMap.of())
);
private AppenderatorConfig appenderatorConfig;

View File

@ -69,6 +69,7 @@ import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.query.timeseries.TimeseriesQuery;
import org.apache.druid.query.topn.TopNQuery;
import org.apache.druid.query.topn.TopNQueryBuilder;
import org.apache.druid.query.union.UnionQuery;
import org.apache.druid.segment.FrameBasedInlineSegmentWrangler;
import org.apache.druid.segment.InlineSegmentWrangler;
import org.apache.druid.segment.MapSegmentWrangler;
@ -1550,6 +1551,58 @@ public class ClientQuerySegmentWalkerTest
Assert.assertEquals(1, scheduler.getTotalReleased().get());
}
@Test
public void testUnionQuery()
{
TimeseriesQuery subQuery1 = (TimeseriesQuery) Druids.newTimeseriesQueryBuilder()
.dataSource(FOO)
.granularity(Granularities.ALL)
.intervals(Collections.singletonList(INTERVAL))
.aggregators(new LongSumAggregatorFactory("sum", "n"))
.context(ImmutableMap.of(TimeseriesQuery.CTX_GRAND_TOTAL, false))
.build()
.withId(DUMMY_QUERY_ID);
TimeseriesQuery subQuery2 = (TimeseriesQuery) Druids.newTimeseriesQueryBuilder()
.dataSource(BAR)
.granularity(Granularities.ALL)
.intervals(Collections.singletonList(INTERVAL))
.aggregators(new LongSumAggregatorFactory("sum", "n"))
.context(ImmutableMap.of(TimeseriesQuery.CTX_GRAND_TOTAL, false))
.build()
.withId(DUMMY_QUERY_ID);
final Query query = Druids.newScanQueryBuilder()
.columns("sum")
.intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.ETERNITY)))
.dataSource(
new UnionQuery(ImmutableList.of(subQuery1, subQuery2))
).build();
testQuery(
query,
ImmutableList.of(
ExpectedQuery.cluster(subQuery1.withSubQueryId("1.1")),
ExpectedQuery.cluster(subQuery2.withSubQueryId("1.1")),
ExpectedQuery.local(
query.withDataSource(
InlineDataSource.fromIterable(
ImmutableList.of(
new Object[] {946684800000L, 10L},
new Object[] {946684800000L, 10L}
),
RowSignature.builder().add("__time", ColumnType.LONG).add("sum", ColumnType.LONG).build()
)
)
)
),
ImmutableList.of(new Object[] {10L}, new Object[] {10L})
);
Assert.assertEquals(3, scheduler.getTotalRun().get());
Assert.assertEquals(2, scheduler.getTotalPrioritizedAndLaned().get());
Assert.assertEquals(3, scheduler.getTotalAcquired().get());
Assert.assertEquals(3, scheduler.getTotalReleased().get());
}
/**
* Initialize (or reinitialize) our {@link #walker} and {@link #closer} with default scheduler.
*/

View File

@ -32,9 +32,9 @@ import org.apache.druid.query.GenericQueryMetricsFactory;
import org.apache.druid.query.QueryContextTest;
import org.apache.druid.query.QueryMetrics;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.timeseries.TimeseriesQuery;
import org.apache.druid.server.log.RequestLogger;
@ -70,7 +70,7 @@ public class QueryLifecycleTest
.intervals(ImmutableList.of(Intervals.ETERNITY))
.aggregators(new CountAggregatorFactory("chocula"))
.build();
QueryToolChestWarehouse toolChestWarehouse;
QueryRunnerFactoryConglomerate conglomerate;
QuerySegmentWalker texasRanger;
GenericQueryMetricsFactory metricsFactory;
ServiceEmitter emitter;
@ -90,7 +90,7 @@ public class QueryLifecycleTest
@Before
public void setup()
{
toolChestWarehouse = EasyMock.createMock(QueryToolChestWarehouse.class);
conglomerate = EasyMock.createMock(QueryRunnerFactoryConglomerate.class);
texasRanger = EasyMock.createMock(QuerySegmentWalker.class);
metricsFactory = EasyMock.createMock(GenericQueryMetricsFactory.class);
emitter = EasyMock.createMock(ServiceEmitter.class);
@ -110,7 +110,7 @@ public class QueryLifecycleTest
long nanos = System.nanoTime();
long millis = System.currentTimeMillis();
return new QueryLifecycle(
toolChestWarehouse,
conglomerate,
texasRanger,
metricsFactory,
emitter,
@ -127,7 +127,7 @@ public class QueryLifecycleTest
public void teardown()
{
EasyMock.verify(
toolChestWarehouse,
conglomerate,
texasRanger,
metricsFactory,
emitter,
@ -146,7 +146,7 @@ public class QueryLifecycleTest
{
EasyMock.expect(queryConfig.getContext()).andReturn(ImmutableMap.of()).anyTimes();
EasyMock.expect(authenticationResult.getIdentity()).andReturn(IDENTITY).anyTimes();
EasyMock.expect(toolChestWarehouse.getToolChest(EasyMock.anyObject()))
EasyMock.expect(conglomerate.getToolChest(EasyMock.anyObject()))
.andReturn(toolChest)
.once();
EasyMock.expect(texasRanger.getQueryRunnerForIntervals(EasyMock.anyObject(), EasyMock.anyObject()))
@ -168,7 +168,7 @@ public class QueryLifecycleTest
EasyMock.expect(queryConfig.getContext()).andReturn(ImmutableMap.of()).anyTimes();
EasyMock.expect(authenticationResult.getIdentity()).andReturn(IDENTITY).anyTimes();
EasyMock.expect(toolChestWarehouse.getToolChest(EasyMock.anyObject()))
EasyMock.expect(conglomerate.getToolChest(EasyMock.anyObject()))
.andReturn(toolChest)
.once();
@ -194,7 +194,7 @@ public class QueryLifecycleTest
EasyMock.expect(authorizer.authorize(authenticationResult, new Resource("baz", ResourceType.QUERY_CONTEXT), Action.WRITE))
.andReturn(Access.OK).times(2);
EasyMock.expect(toolChestWarehouse.getToolChest(EasyMock.anyObject()))
EasyMock.expect(conglomerate.getToolChest(EasyMock.anyObject()))
.andReturn(toolChest)
.times(2);
@ -242,7 +242,7 @@ public class QueryLifecycleTest
.andReturn(Access.DENIED)
.times(2);
EasyMock.expect(toolChestWarehouse.getToolChest(EasyMock.anyObject()))
EasyMock.expect(conglomerate.getToolChest(EasyMock.anyObject()))
.andReturn(toolChest)
.times(2);
@ -277,7 +277,7 @@ public class QueryLifecycleTest
.andReturn(Access.OK)
.times(2);
EasyMock.expect(toolChestWarehouse.getToolChest(EasyMock.anyObject()))
EasyMock.expect(conglomerate.getToolChest(EasyMock.anyObject()))
.andReturn(toolChest)
.times(2);
@ -323,7 +323,7 @@ public class QueryLifecycleTest
.andReturn(Access.OK)
.times(2);
EasyMock.expect(toolChestWarehouse.getToolChest(EasyMock.anyObject()))
EasyMock.expect(conglomerate.getToolChest(EasyMock.anyObject()))
.andReturn(toolChest)
.times(2);
@ -373,7 +373,7 @@ public class QueryLifecycleTest
.andReturn(Access.DENIED)
.times(2);
EasyMock.expect(toolChestWarehouse.getToolChest(EasyMock.anyObject()))
EasyMock.expect(conglomerate.getToolChest(EasyMock.anyObject()))
.andReturn(toolChest)
.times(2);
@ -417,7 +417,7 @@ public class QueryLifecycleTest
.andReturn(Access.OK)
.times(2);
EasyMock.expect(toolChestWarehouse.getToolChest(EasyMock.anyObject()))
EasyMock.expect(conglomerate.getToolChest(EasyMock.anyObject()))
.andReturn(toolChest)
.times(2);
@ -460,7 +460,7 @@ public class QueryLifecycleTest
private void replayAll()
{
EasyMock.replay(
toolChestWarehouse,
conglomerate,
texasRanger,
metricsFactory,
emitter,

View File

@ -51,7 +51,7 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.query.BadJsonQueryException;
import org.apache.druid.query.DefaultGenericQueryMetricsFactory;
import org.apache.druid.query.DefaultQueryConfig;
import org.apache.druid.query.MapQueryToolChestWarehouse;
import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryCapacityExceededException;
import org.apache.druid.query.QueryException;
@ -59,7 +59,6 @@ import org.apache.druid.query.QueryInterruptedException;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.query.QueryTimeoutException;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.QueryUnsupportedException;
import org.apache.druid.query.ResourceLimitExceededException;
import org.apache.druid.query.Result;
@ -125,7 +124,7 @@ import java.util.function.Consumer;
public class QueryResourceTest
{
private static final QueryToolChestWarehouse WAREHOUSE = new MapQueryToolChestWarehouse(ImmutableMap.of());
private static final DefaultQueryRunnerFactoryConglomerate CONGLOMERATE = DefaultQueryRunnerFactoryConglomerate.buildFromQueryRunnerFactories(ImmutableMap.of());
private static final AuthenticationResult AUTHENTICATION_RESULT =
new AuthenticationResult("druid", "druid", null, null);
@ -239,7 +238,7 @@ public class QueryResourceTest
{
return new QueryResource(
new QueryLifecycleFactory(
WAREHOUSE,
CONGLOMERATE,
TEST_SEGMENT_WALKER,
new DefaultGenericQueryMetricsFactory(),
new NoopServiceEmitter(),
@ -274,7 +273,7 @@ public class QueryResourceTest
DefaultQueryConfig overrideConfig = new DefaultQueryConfig(ImmutableMap.of(overrideConfigKey, overrideConfigValue));
queryResource = new QueryResource(
new QueryLifecycleFactory(
WAREHOUSE,
CONGLOMERATE,
TEST_SEGMENT_WALKER,
new DefaultGenericQueryMetricsFactory(),
new NoopServiceEmitter(),
@ -327,7 +326,7 @@ public class QueryResourceTest
DefaultQueryConfig overrideConfig = new DefaultQueryConfig(ImmutableMap.of(overrideConfigKey, overrideConfigValue));
queryResource = new QueryResource(
new QueryLifecycleFactory(
WAREHOUSE,
CONGLOMERATE,
new QuerySegmentWalker()
{
@Override
@ -397,7 +396,7 @@ public class QueryResourceTest
{
queryResource = new QueryResource(
new QueryLifecycleFactory(
WAREHOUSE,
CONGLOMERATE,
new QuerySegmentWalker()
{
@Override
@ -478,7 +477,7 @@ public class QueryResourceTest
{
queryResource = new QueryResource(
new QueryLifecycleFactory(
WAREHOUSE,
CONGLOMERATE,
TEST_SEGMENT_WALKER,
new DefaultGenericQueryMetricsFactory(),
new NoopServiceEmitter(),
@ -549,7 +548,7 @@ public class QueryResourceTest
public QueryLifecycle factorize()
{
return new QueryLifecycle(
WAREHOUSE,
CONGLOMERATE,
querySegmentWalker,
new DefaultGenericQueryMetricsFactory(),
new NoopServiceEmitter(),
@ -602,7 +601,7 @@ public class QueryResourceTest
DefaultQueryConfig overrideConfig = new DefaultQueryConfig(ImmutableMap.of(overrideConfigKey, overrideConfigValue));
queryResource = new QueryResource(
new QueryLifecycleFactory(
WAREHOUSE,
CONGLOMERATE,
TEST_SEGMENT_WALKER,
new DefaultGenericQueryMetricsFactory(),
new NoopServiceEmitter(),
@ -839,7 +838,7 @@ public class QueryResourceTest
queryResource = new QueryResource(
new QueryLifecycleFactory(
WAREHOUSE,
CONGLOMERATE,
TEST_SEGMENT_WALKER,
new DefaultGenericQueryMetricsFactory(),
new NoopServiceEmitter(),
@ -914,7 +913,7 @@ public class QueryResourceTest
final QueryResource timeoutQueryResource = new QueryResource(
new QueryLifecycleFactory(
WAREHOUSE,
CONGLOMERATE,
timeoutSegmentWalker,
new DefaultGenericQueryMetricsFactory(),
new NoopServiceEmitter(),
@ -1010,7 +1009,7 @@ public class QueryResourceTest
queryResource = new QueryResource(
new QueryLifecycleFactory(
WAREHOUSE,
CONGLOMERATE,
TEST_SEGMENT_WALKER,
new DefaultGenericQueryMetricsFactory(),
new NoopServiceEmitter(),
@ -1117,7 +1116,7 @@ public class QueryResourceTest
queryResource = new QueryResource(
new QueryLifecycleFactory(
WAREHOUSE,
CONGLOMERATE,
TEST_SEGMENT_WALKER,
new DefaultGenericQueryMetricsFactory(),
new NoopServiceEmitter(),
@ -1469,7 +1468,7 @@ public class QueryResourceTest
queryResource = new QueryResource(
new QueryLifecycleFactory(
WAREHOUSE,
CONGLOMERATE,
texasRanger,
new DefaultGenericQueryMetricsFactory(),
new NoopServiceEmitter(),

View File

@ -20,8 +20,10 @@
package org.apache.druid.server;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Suppliers;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Maps;
import com.google.inject.Injector;
import org.apache.druid.client.cache.Cache;
import org.apache.druid.client.cache.CacheConfig;
@ -45,8 +47,6 @@ import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QueryRunnerTestHelper;
import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.RetryQueryRunnerConfig;
import org.apache.druid.query.TestBufferPool;
import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable;
@ -67,6 +67,11 @@ import org.apache.druid.query.scan.ScanQueryConfig;
import org.apache.druid.query.scan.ScanQueryEngine;
import org.apache.druid.query.scan.ScanQueryQueryToolChest;
import org.apache.druid.query.scan.ScanQueryRunnerFactory;
import org.apache.druid.query.search.SearchQuery;
import org.apache.druid.query.search.SearchQueryConfig;
import org.apache.druid.query.search.SearchQueryQueryToolChest;
import org.apache.druid.query.search.SearchQueryRunnerFactory;
import org.apache.druid.query.search.SearchStrategySelector;
import org.apache.druid.query.timeboundary.TimeBoundaryQuery;
import org.apache.druid.query.timeboundary.TimeBoundaryQueryRunnerFactory;
import org.apache.druid.query.timeseries.TimeseriesQuery;
@ -77,6 +82,8 @@ import org.apache.druid.query.topn.TopNQuery;
import org.apache.druid.query.topn.TopNQueryConfig;
import org.apache.druid.query.topn.TopNQueryQueryToolChest;
import org.apache.druid.query.topn.TopNQueryRunnerFactory;
import org.apache.druid.query.union.UnionQuery;
import org.apache.druid.query.union.UnionQueryLogic;
import org.apache.druid.segment.ReferenceCountingSegment;
import org.apache.druid.segment.SegmentWrangler;
import org.apache.druid.segment.TestHelper;
@ -95,18 +102,52 @@ import org.apache.druid.sql.calcite.util.CacheTestHelperModule.ResultCacheMode;
import org.apache.druid.timeline.VersionedIntervalTimeline;
import org.apache.druid.utils.JvmUtils;
import org.junit.Assert;
import org.junit.rules.ExternalResource;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.Collections;
import java.util.Map;
import java.util.Set;
import java.util.function.Supplier;
/**
* Utilities for creating query-stack objects for tests.
*/
public class QueryStackTests
{
public static class Junit4ConglomerateRule extends ExternalResource
{
private Closer closer;
private QueryRunnerFactoryConglomerate conglomerate;
@Override
protected void before()
{
closer = Closer.create();
conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(closer);
}
@Override
protected void after()
{
try {
closer.close();
}
catch (IOException e) {
throw new RuntimeException(e);
}
conglomerate = null;
closer = null;
}
public QueryRunnerFactoryConglomerate getConglomerate()
{
return conglomerate;
}
}
public static final QueryScheduler DEFAULT_NOOP_SCHEDULER = new QueryScheduler(
0,
ManualQueryPrioritizationStrategy.INSTANCE,
@ -137,14 +178,7 @@ public class QueryStackTests
emitter,
clusterWalker,
localWalker,
new QueryToolChestWarehouse()
{
@Override
public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(final QueryType query)
{
return conglomerate.findFactory(query).getToolchest();
}
},
conglomerate,
joinableFactory,
new RetryQueryRunnerConfig(),
injector.getInstance(ObjectMapper.class),
@ -235,21 +269,12 @@ public class QueryStackTests
*/
public static QueryRunnerFactoryConglomerate createQueryRunnerFactoryConglomerate(final Closer closer)
{
return createQueryRunnerFactoryConglomerate(closer, () -> TopNQueryConfig.DEFAULT_MIN_TOPN_THRESHOLD);
return createQueryRunnerFactoryConglomerate(closer, TopNQueryConfig.DEFAULT_MIN_TOPN_THRESHOLD);
}
public static QueryRunnerFactoryConglomerate createQueryRunnerFactoryConglomerate(
final Closer closer,
final Supplier<Integer> minTopNThresholdSupplier
)
{
return createQueryRunnerFactoryConglomerate(closer, minTopNThresholdSupplier, TestHelper.makeJsonMapper());
}
public static QueryRunnerFactoryConglomerate createQueryRunnerFactoryConglomerate(
final Closer closer,
final Supplier<Integer> minTopNThresholdSupplier,
final ObjectMapper jsonMapper
final Integer minTopNThreshold
)
{
return createQueryRunnerFactoryConglomerate(
@ -257,8 +282,8 @@ public class QueryStackTests
getProcessingConfig(
DEFAULT_NUM_MERGE_BUFFERS
),
minTopNThresholdSupplier,
jsonMapper
minTopNThreshold,
TestHelper.makeJsonMapper()
);
}
@ -270,116 +295,144 @@ public class QueryStackTests
return createQueryRunnerFactoryConglomerate(
closer,
processingConfig,
() -> TopNQueryConfig.DEFAULT_MIN_TOPN_THRESHOLD
);
}
public static QueryRunnerFactoryConglomerate createQueryRunnerFactoryConglomerate(
final Closer closer,
final DruidProcessingConfig processingConfig,
final ObjectMapper jsonMapper
)
{
return createQueryRunnerFactoryConglomerate(
closer,
processingConfig,
() -> TopNQueryConfig.DEFAULT_MIN_TOPN_THRESHOLD,
jsonMapper
);
}
public static QueryRunnerFactoryConglomerate createQueryRunnerFactoryConglomerate(
final Closer closer,
final DruidProcessingConfig processingConfig,
final Supplier<Integer> minTopNThresholdSupplier
)
{
return createQueryRunnerFactoryConglomerate(
closer,
processingConfig,
minTopNThresholdSupplier,
TopNQueryConfig.DEFAULT_MIN_TOPN_THRESHOLD,
TestHelper.makeJsonMapper()
);
}
public static QueryRunnerFactoryConglomerate createQueryRunnerFactoryConglomerate(
final Closer closer,
final DruidProcessingConfig processingConfig,
final Supplier<Integer> minTopNThresholdSupplier,
final ObjectMapper jsonMapper
)
public static TestBufferPool makeTestBufferPool(final Closer closer)
{
final TestBufferPool testBufferPool = TestBufferPool.offHeap(COMPUTE_BUFFER_SIZE, Integer.MAX_VALUE);
closer.register(() -> {
// Verify that all objects have been returned to the pool.
Assert.assertEquals(0, testBufferPool.getOutstandingObjectCount());
});
return testBufferPool;
}
public static TestGroupByBuffers makeGroupByBuffers(final Closer closer, final DruidProcessingConfig processingConfig)
{
final TestGroupByBuffers groupByBuffers =
closer.register(TestGroupByBuffers.createFromProcessingConfig(processingConfig));
return groupByBuffers;
}
final GroupByQueryRunnerFactory groupByQueryRunnerFactory =
GroupByQueryRunnerTest.makeQueryRunnerFactory(
jsonMapper,
new GroupByQueryConfig()
{
},
groupByBuffers,
processingConfig
);
public static QueryRunnerFactoryConglomerate createQueryRunnerFactoryConglomerate(
final Closer closer,
final DruidProcessingConfig processingConfig,
final Integer minTopNThreshold,
final ObjectMapper jsonMapper
)
{
final TestBufferPool testBufferPool = makeTestBufferPool(closer);
final TestGroupByBuffers groupByBuffers = makeGroupByBuffers(closer, processingConfig);
final QueryRunnerFactoryConglomerate conglomerate = new DefaultQueryRunnerFactoryConglomerate(
ImmutableMap.<Class<? extends Query>, QueryRunnerFactory>builder()
.put(
SegmentMetadataQuery.class,
new SegmentMetadataQueryRunnerFactory(
new SegmentMetadataQueryQueryToolChest(
new SegmentMetadataQueryConfig("P1W")
),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
)
.put(
ScanQuery.class,
new ScanQueryRunnerFactory(
new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()),
new ScanQueryEngine(),
new ScanQueryConfig()
)
)
.put(
TimeseriesQuery.class,
new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
)
.put(
TopNQuery.class,
new TopNQueryRunnerFactory(
testBufferPool,
new TopNQueryQueryToolChest(new TopNQueryConfig()
{
@Override
public int getMinTopNThreshold()
{
return minTopNThresholdSupplier.get();
}
}),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
)
.put(GroupByQuery.class, groupByQueryRunnerFactory)
.put(TimeBoundaryQuery.class, new TimeBoundaryQueryRunnerFactory(QueryRunnerTestHelper.NOOP_QUERYWATCHER))
.put(WindowOperatorQuery.class, new WindowOperatorQueryQueryRunnerFactory())
.build()
return createQueryRunnerFactoryConglomerate(
processingConfig,
minTopNThreshold,
jsonMapper,
testBufferPool,
groupByBuffers);
}
public static QueryRunnerFactoryConglomerate createQueryRunnerFactoryConglomerate(
final DruidProcessingConfig processingConfig,
final Integer minTopNThreshold,
final ObjectMapper jsonMapper,
final TestBufferPool testBufferPool,
final TestGroupByBuffers groupByBuffers)
{
ImmutableMap<Class<? extends Query>, QueryRunnerFactory> factories = makeDefaultQueryRunnerFactories(
processingConfig,
minTopNThreshold,
jsonMapper,
testBufferPool,
groupByBuffers
);
UnionQueryLogic unionQueryLogic = new UnionQueryLogic();
final QueryRunnerFactoryConglomerate conglomerate = new DefaultQueryRunnerFactoryConglomerate(
factories,
Maps.transformValues(factories, f -> f.getToolchest()),
ImmutableMap.of(UnionQuery.class, unionQueryLogic)
);
unionQueryLogic.initialize(conglomerate);
return conglomerate;
}
@SuppressWarnings("rawtypes")
public static ImmutableMap<Class<? extends Query>, QueryRunnerFactory> makeDefaultQueryRunnerFactories(
final DruidProcessingConfig processingConfig,
final Integer minTopNThreshold,
final ObjectMapper jsonMapper,
final TestBufferPool testBufferPool,
final TestGroupByBuffers groupByBuffers)
{
final GroupByQueryRunnerFactory groupByQueryRunnerFactory = GroupByQueryRunnerTest.makeQueryRunnerFactory(
jsonMapper,
new GroupByQueryConfig()
{
},
groupByBuffers,
processingConfig
);
return ImmutableMap.<Class<? extends Query>, QueryRunnerFactory>builder()
.put(
SegmentMetadataQuery.class,
new SegmentMetadataQueryRunnerFactory(
new SegmentMetadataQueryQueryToolChest(
new SegmentMetadataQueryConfig("P1W")
),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
)
.put(
SearchQuery.class,
new SearchQueryRunnerFactory(
new SearchStrategySelector(Suppliers.ofInstance(new SearchQueryConfig())),
new SearchQueryQueryToolChest(new SearchQueryConfig()),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
)
.put(
ScanQuery.class,
new ScanQueryRunnerFactory(
new ScanQueryQueryToolChest(DefaultGenericQueryMetricsFactory.instance()),
new ScanQueryEngine(),
new ScanQueryConfig()
)
)
.put(
TimeseriesQuery.class,
new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
)
.put(
TopNQuery.class,
new TopNQueryRunnerFactory(
testBufferPool,
new TopNQueryQueryToolChest(new TopNQueryConfig()
{
@Override
public int getMinTopNThreshold()
{
return minTopNThreshold;
}
}),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
)
.put(GroupByQuery.class, groupByQueryRunnerFactory)
.put(TimeBoundaryQuery.class, new TimeBoundaryQueryRunnerFactory(QueryRunnerTestHelper.NOOP_QUERYWATCHER))
.put(WindowOperatorQuery.class, new WindowOperatorQueryQueryRunnerFactory())
.build();
}
public static JoinableFactory makeJoinableFactoryForLookup(
LookupExtractorFactoryContainerProvider lookupProvider
)

View File

@ -48,6 +48,7 @@ import org.apache.druid.query.BaseQuery;
import org.apache.druid.query.ConcatQueryRunner;
import org.apache.druid.query.DataSource;
import org.apache.druid.query.DefaultQueryMetrics;
import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate;
import org.apache.druid.query.Druids;
import org.apache.druid.query.ForwardingQueryProcessingPool;
import org.apache.druid.query.NoopQueryRunner;
@ -187,19 +188,12 @@ public class ServerManagerTest
queryNotifyLatch = new CountDownLatch(1);
factory = new MyQueryRunnerFactory(queryWaitLatch, queryWaitYieldLatch, queryNotifyLatch);
serverManagerExec = Execs.multiThreaded(2, "ServerManagerTest-%d");
QueryRunnerFactoryConglomerate conglomerate = DefaultQueryRunnerFactoryConglomerate.buildFromQueryRunnerFactories(ImmutableMap
.<Class<? extends Query>, QueryRunnerFactory>builder()
.put(SearchQuery.class, factory)
.build());
serverManager = new ServerManager(
new QueryRunnerFactoryConglomerate()
{
@Override
public <T, QueryType extends Query<T>> QueryRunnerFactory<T, QueryType> findFactory(QueryType query)
{
if (query instanceof SearchQuery) {
return (QueryRunnerFactory) factory;
} else {
return null;
}
}
},
conglomerate,
new NoopServiceEmitter(),
new ForwardingQueryProcessingPool(serverManagerExec),
new ForegroundCachePopulator(new DefaultObjectMapper(), new CachePopulatorStats(), -1),

View File

@ -451,6 +451,7 @@ public class CliCoordinator extends ServerRunnable
final MapBinder<Class<? extends Query>, QueryRunnerFactory> queryFactoryBinder =
DruidBinders.queryRunnerFactoryBinder(binder);
queryFactoryBinder.addBinding(SegmentMetadataQuery.class).to(SegmentMetadataQueryRunnerFactory.class);
DruidBinders.queryLogicBinder(binder);
binder.bind(SegmentMetadataQueryRunnerFactory.class).in(LazySingleton.class);
binder.bind(GenericQueryMetricsFactory.class).to(DefaultGenericQueryMetricsFactory.class);

View File

@ -31,11 +31,15 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery;
import org.apache.druid.error.DruidException;
import org.apache.druid.query.DataSource;
import org.apache.druid.query.InlineDataSource;
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.union.UnionQuery;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.apache.druid.sql.calcite.planner.querygen.SourceDescProducer;
import java.util.ArrayList;
import java.util.List;
@ -66,24 +70,72 @@ public class DruidUnion extends Union implements DruidLogicalNode, SourceDescPro
@Override
public SourceDesc getSourceDesc(PlannerContext plannerContext, List<SourceDesc> sources)
{
List<DataSource> dataSources = new ArrayList<>();
RowSignature signature = null;
for (SourceDesc sourceDesc : sources) {
checkDataSourceSupported(sourceDesc.dataSource);
dataSources.add(sourceDesc.dataSource);
if (signature == null) {
signature = sourceDesc.rowSignature;
} else {
if (!signature.equals(sourceDesc.rowSignature)) {
throw DruidException.defensive(
"Row signature mismatch in Union inputs [%s] and [%s]",
signature,
sourceDesc.rowSignature
);
if (mayUseUnionDataSource(sources)) {
List<DataSource> dataSources = new ArrayList<>();
RowSignature signature = null;
for (SourceDesc sourceDesc : sources) {
checkDataSourceSupported(sourceDesc.dataSource);
dataSources.add(sourceDesc.dataSource);
if (signature == null) {
signature = sourceDesc.rowSignature;
} else {
if (!signature.equals(sourceDesc.rowSignature)) {
throw DruidException.defensive(
"Row signature mismatch in Union inputs [%s] and [%s]",
signature,
sourceDesc.rowSignature
);
}
}
}
return new SourceDesc(new UnionDataSource(dataSources), signature);
}
return new SourceDesc(new UnionDataSource(dataSources), signature);
if (mayUseUnionQuery(sources)) {
RowSignature signature = null;
List<Query<?>> queries = new ArrayList<>();
for (SourceDesc sourceDesc : sources) {
QueryDataSource qds = (QueryDataSource) sourceDesc.dataSource;
queries.add(qds.getQuery());
if (signature == null) {
signature = sourceDesc.rowSignature;
} else {
if (!signature.equals(sourceDesc.rowSignature)) {
throw DruidException.defensive(
"Row signature mismatch in Union inputs [%s] and [%s]",
signature,
sourceDesc.rowSignature
);
}
}
}
return new SourceDesc(new QueryDataSource(new UnionQuery(queries)), signature);
}
throw DruidException.defensive("Union with input [%s] is not supported. This should not happen.", sources);
}
private boolean mayUseUnionQuery(List<SourceDesc> sources)
{
for (SourceDesc sourceDesc : sources) {
DataSource dataSource = sourceDesc.dataSource;
if (dataSource instanceof QueryDataSource) {
continue;
}
return false;
}
return true;
}
private boolean mayUseUnionDataSource(List<SourceDesc> sources)
{
for (SourceDesc sourceDesc : sources) {
DataSource dataSource = sourceDesc.dataSource;
if (dataSource instanceof TableDataSource || dataSource instanceof InlineDataSource) {
continue;
}
return false;
}
return true;
}
private void checkDataSourceSupported(DataSource dataSource)

View File

@ -30,7 +30,6 @@ import org.checkerframework.checker.nullness.qual.Nullable;
public class DruidUnionRule extends ConverterRule
{
public DruidUnionRule(Class<? extends RelNode> clazz, RelTrait in, RelTrait out, String descriptionPrefix)
{
super(

View File

@ -21,7 +21,6 @@ package org.apache.druid.quidem;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Binder;
import com.google.inject.Injector;
import com.google.inject.Provides;
import com.google.inject.name.Named;
import com.google.inject.name.Names;
@ -33,10 +32,7 @@ import org.apache.druid.java.util.common.FileUtils;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider;
import org.apache.druid.segment.join.JoinableFactoryWrapper;
import org.apache.druid.server.DruidNode;
import org.apache.druid.server.QueryLifecycleFactory;
import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker;
import org.apache.druid.sql.avatica.AvaticaMonitor;
import org.apache.druid.sql.avatica.DruidAvaticaJsonHandler;
@ -45,13 +41,10 @@ import org.apache.druid.sql.calcite.SqlTestFrameworkConfig;
import org.apache.druid.sql.calcite.SqlTestFrameworkConfig.ConfigurationInstance;
import org.apache.druid.sql.calcite.SqlTestFrameworkConfig.SqlTestFrameworkConfigStore;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.run.SqlEngine;
import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.apache.druid.sql.calcite.util.SqlTestFramework;
import org.apache.druid.sql.calcite.util.SqlTestFramework.Builder;
import org.apache.druid.sql.calcite.util.SqlTestFramework.PlannerComponentSupplier;
import org.apache.druid.sql.calcite.util.SqlTestFramework.QueryComponentSupplier;
import org.apache.druid.sql.calcite.util.SqlTestFramework.QueryComponentSupplierDelegate;
import org.apache.druid.sql.hook.DruidHookDispatcher;
import org.apache.http.client.utils.URIBuilder;
import org.eclipse.jetty.server.Server;
@ -206,27 +199,20 @@ public class DruidAvaticaTestDriver implements Driver
}
}
static class AvaticaBasedTestConnectionSupplier implements QueryComponentSupplier
static class AvaticaBasedTestConnectionSupplier extends QueryComponentSupplierDelegate
{
private QueryComponentSupplier delegate;
private AvaticaBasedConnectionModule connectionModule;
public AvaticaBasedTestConnectionSupplier(QueryComponentSupplier delegate)
{
this.delegate = delegate;
super(delegate);
this.connectionModule = new AvaticaBasedConnectionModule();
}
@Override
public void gatherProperties(Properties properties)
{
delegate.gatherProperties(properties);
}
@Override
public void configureGuice(DruidInjectorBuilder builder)
{
delegate.configureGuice(builder);
super.configureGuice(builder);
builder.addModule(connectionModule);
builder.addModule(
binder -> {
@ -237,60 +223,11 @@ public class DruidAvaticaTestDriver implements Driver
);
}
@Override
public QueryRunnerFactoryConglomerate createCongolmerate(Builder builder, Closer closer, ObjectMapper jsonMapper)
{
return delegate.createCongolmerate(builder, closer, jsonMapper);
}
@Override
public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker(QueryRunnerFactoryConglomerate conglomerate,
JoinableFactoryWrapper joinableFactory, Injector injector)
{
return delegate.createQuerySegmentWalker(conglomerate, joinableFactory, injector);
}
@Override
public SqlEngine createEngine(QueryLifecycleFactory qlf, ObjectMapper objectMapper, Injector injector)
{
return delegate.createEngine(qlf, objectMapper, injector);
}
@Override
public void configureJsonMapper(ObjectMapper mapper)
{
delegate.configureJsonMapper(mapper);
}
@Override
public JoinableFactoryWrapper createJoinableFactoryWrapper(LookupExtractorFactoryContainerProvider lookupProvider)
{
return delegate.createJoinableFactoryWrapper(lookupProvider);
}
@Override
public void finalizeTestFramework(SqlTestFramework sqlTestFramework)
{
delegate.finalizeTestFramework(sqlTestFramework);
}
@Override
public void close() throws IOException
{
connectionModule.close();
delegate.close();
}
@Override
public PlannerComponentSupplier getPlannerComponentSupplier()
{
return delegate.getPlannerComponentSupplier();
}
@Override
public Boolean isExplainSupported()
{
return delegate.isExplainSupported();
super.close();
}
}

View File

@ -78,6 +78,7 @@ import org.apache.druid.query.scan.ScanQuery;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.query.spec.QuerySegmentSpec;
import org.apache.druid.query.timeseries.TimeseriesQuery;
import org.apache.druid.query.union.UnionQuery;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
@ -1352,7 +1353,14 @@ public class BaseCalciteQueryTest extends CalciteTestBase
public static <T> Query<?> recursivelyClearContext(final Query<T> query, ObjectMapper queryJsonMapper)
{
try {
Query<T> newQuery = query.withDataSource(recursivelyClearContext(query.getDataSource(), queryJsonMapper));
Query<T> newQuery;
if (query instanceof UnionQuery) {
UnionQuery unionQuery = (UnionQuery) query;
newQuery = (Query<T>) unionQuery
.withDataSources(recursivelyClearDatasource(unionQuery.getDataSources(), queryJsonMapper));
} else {
newQuery = query.withDataSource(recursivelyClearContext(query.getDataSource(), queryJsonMapper));
}
final JsonNode newQueryNode = queryJsonMapper.valueToTree(newQuery);
((ObjectNode) newQueryNode).remove("context");
return queryJsonMapper.treeToValue(newQueryNode, Query.class);
@ -1362,6 +1370,16 @@ public class BaseCalciteQueryTest extends CalciteTestBase
}
}
private static List<DataSource> recursivelyClearDatasource(final List<DataSource> dataSources,
ObjectMapper queryJsonMapper)
{
List<DataSource> ret = new ArrayList<DataSource>();
for (DataSource dataSource : dataSources) {
ret.add(recursivelyClearContext(dataSource, queryJsonMapper));
}
return ret;
}
/**
* Override the contexts of all subqueries of a particular datasource.
*/

View File

@ -7200,7 +7200,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
);
}
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.UNNEST_DIFFERENT_RESULTSET, separateDefaultModeTest = true)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.UNNEST_SUBSTRING_EMPTY, separateDefaultModeTest = true)
@Test
public void testUnnestExtractionFn()
{

View File

@ -4326,7 +4326,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
});
}
@NotYetSupported(Modes.UNION_WITH_COMPLEX_OPERAND)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.UNION_ALL_QUERY)
@MethodSource("provideQueryContexts")
@ParameterizedTest(name = "{0}")
public void testUnionAllTwoQueriesLeftQueryIsJoin(Map<String, Object> queryContext)
@ -4339,6 +4339,8 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
cannotVectorize();
}
testQuery(
"(SELECT COUNT(*) FROM foo INNER JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k) UNION ALL SELECT SUM(cnt) FROM foo",
queryContext,
@ -4371,7 +4373,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
);
}
@NotYetSupported(Modes.UNION_WITH_COMPLEX_OPERAND)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.UNION_ALL_QUERY)
@MethodSource("provideQueryContexts")
@ParameterizedTest(name = "{0}")
public void testUnionAllTwoQueriesRightQueryIsJoin(Map<String, Object> queryContext)
@ -4416,7 +4418,8 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
);
}
@NotYetSupported(Modes.UNION_WITH_COMPLEX_OPERAND)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.UNION_ALL_QUERY)
@Test
public void testUnionAllTwoQueriesBothQueriesAreJoin()
{

View File

@ -3114,11 +3114,14 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@NotYetSupported(Modes.UNION_WITH_COMPLEX_OPERAND)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.UNION_ALL_QUERY)
@Test
public void testUnionAllQueries()
{
msqIncompatible();
skipVectorize();
testQuery(
"SELECT COUNT(*) FROM foo UNION ALL SELECT SUM(cnt) FROM foo UNION ALL SELECT COUNT(*) FROM foo",
ImmutableList.of(
@ -3148,7 +3151,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@NotYetSupported(Modes.UNION_WITH_COMPLEX_OPERAND)
@DecoupledTestConfig(quidemReason = QuidemTestCaseReason.UNION_ALL_QUERY)
@Test
public void testUnionAllQueriesWithLimit()
{

View File

@ -55,16 +55,10 @@ public class DecoupledExtension implements BeforeEachCallback
}
private static final ImmutableMap<String, Object> CONTEXT_OVERRIDES = ImmutableMap.<String, Object>builder()
.putAll(BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT)
.put(
PlannerConfig.CTX_NATIVE_QUERY_SQL_PLANNING_MODE,
PlannerConfig.NATIVE_QUERY_SQL_PLANNING_MODE_DECOUPLED
)
.put(
QueryContexts.ENABLE_DEBUG,
true
)
.build();
.putAll(BaseCalciteQueryTest.QUERY_CONTEXT_DEFAULT)
.put(PlannerConfig.CTX_NATIVE_QUERY_SQL_PLANNING_MODE, PlannerConfig.NATIVE_QUERY_SQL_PLANNING_MODE_DECOUPLED)
.put(QueryContexts.ENABLE_DEBUG, true)
.build();
public QueryTestBuilder testBuilder()
{
@ -124,10 +118,9 @@ public class DecoupledExtension implements BeforeEachCallback
}
};
return builder.cannotVectorize(
baseTest.cannotVectorize ||
(!ExpressionProcessing.allowVectorizeFallback() && baseTest.cannotVectorizeUnlessFallback)
)
.skipVectorize(baseTest.skipVectorize);
boolean cannotVectorize = baseTest.cannotVectorize
|| (!ExpressionProcessing.allowVectorizeFallback() && baseTest.cannotVectorizeUnlessFallback);
return builder.cannotVectorize(cannotVectorize)
.skipVectorize(baseTest.skipVectorize);
}
}

View File

@ -157,7 +157,15 @@ public @interface DecoupledTestConfig
/**
* New plan UNNEST-s a different resultset.
*/
UNNEST_DIFFERENT_RESULTSET;
UNNEST_DIFFERENT_RESULTSET,
/**
* Uses a UNION ALL query.
*/
UNION_ALL_QUERY,
/**
* This is due to substring('',1') is null.
*/
UNNEST_SUBSTRING_EMPTY;
public boolean isPresent()
{

View File

@ -89,7 +89,6 @@ public @interface NotYetSupported
RESULT_MISMATCH(AssertionError.class, "(assertResulEquals|AssertionError: column content mismatch)"),
LONG_CASTING(AssertionError.class, "expected: java.lang.Long"),
UNSUPPORTED_NULL_ORDERING(DruidException.class, "(A|DE)SCENDING ordering with NULLS (LAST|FIRST)"),
UNION_WITH_COMPLEX_OPERAND(DruidException.class, "Only Table and Values are supported as inputs for Union"),
UNION_MORE_STRICT_ROWTYPE_CHECK(DruidException.class, "Row signature mismatch in Union inputs"),
SORT_REMOVE_TROUBLE(DruidException.class, "Calcite assertion violated.*Sort\\.<init>"),
SORT_REMOVE_CONSTANT_KEYS_CONFLICT(DruidException.class, "not enough rules"),

View File

@ -31,6 +31,7 @@ import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.query.topn.TopNQueryConfig;
import org.apache.druid.quidem.DruidAvaticaTestDriver;
import org.apache.druid.server.QueryStackTests;
import org.apache.druid.sql.calcite.util.CacheTestHelperModule.ResultCacheMode;
import org.apache.druid.sql.calcite.util.SqlTestFramework;
import org.apache.druid.sql.calcite.util.SqlTestFramework.QueryComponentSupplier;
@ -94,7 +95,7 @@ public class SqlTestFrameworkConfig
{
@Retention(RetentionPolicy.RUNTIME)
@Target({ElementType.METHOD, ElementType.TYPE})
@NumMergeBuffers(0)
@NumMergeBuffers(QueryStackTests.DEFAULT_NUM_MERGE_BUFFERS)
public @interface NumMergeBuffers
{
ConfigOptionProcessor<Integer> PROCESSOR = new ConfigOptionProcessor<Integer>(NumMergeBuffers.class)

View File

@ -31,11 +31,8 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.query.DefaultGenericQueryMetricsFactory;
import org.apache.druid.query.DefaultQueryConfig;
import org.apache.druid.query.GlobalTableDataSource;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider;
import org.apache.druid.segment.join.JoinableFactory;
import org.apache.druid.segment.loading.SegmentCacheManager;
@ -76,6 +73,7 @@ import org.apache.druid.sql.calcite.view.ViewManager;
import org.easymock.EasyMock;
import javax.annotation.Nullable;
import java.util.HashSet;
import java.util.Set;
import java.util.stream.Collectors;
@ -90,14 +88,7 @@ public class QueryFrameworkUtils
)
{
return new QueryLifecycleFactory(
new QueryToolChestWarehouse()
{
@Override
public <T, QueryType extends Query<T>> QueryToolChest<T, QueryType> getToolChest(final QueryType query)
{
return conglomerate.findFactory(query).getToolchest();
}
},
conglomerate,
walker,
new DefaultGenericQueryMetricsFactory(),
new ServiceEmitter("dummy", "dummy", new NoopEmitter()),

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Maps;
import com.google.inject.Binder;
import com.google.inject.Injector;
import com.google.inject.Module;
@ -39,11 +40,21 @@ import org.apache.druid.initialization.ServiceInjectorBuilder;
import org.apache.druid.java.util.common.RE;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate;
import org.apache.druid.query.DruidProcessingConfig;
import org.apache.druid.query.GlobalTableDataSource;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryLogic;
import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.TestBufferPool;
import org.apache.druid.query.groupby.TestGroupByBuffers;
import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider;
import org.apache.druid.query.topn.TopNQueryConfig;
import org.apache.druid.query.union.UnionQuery;
import org.apache.druid.query.union.UnionQueryLogic;
import org.apache.druid.quidem.TestSqlModule;
import org.apache.druid.segment.DefaultColumnFormatConfig;
import org.apache.druid.segment.join.JoinableFactoryWrapper;
@ -81,7 +92,9 @@ import java.io.IOException;
import java.net.URI;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
@ -146,6 +159,7 @@ public class SqlTestFramework
*/
void gatherProperties(Properties properties);
/**
* Configure modules needed for tests. This is the preferred way to configure
* Jackson: include the production module in this method that includes the
@ -153,12 +167,6 @@ public class SqlTestFramework
*/
void configureGuice(DruidInjectorBuilder builder);
QueryRunnerFactoryConglomerate createCongolmerate(
Builder builder,
Closer closer,
ObjectMapper jsonMapper
);
SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker(
QueryRunnerFactoryConglomerate conglomerate,
JoinableFactoryWrapper joinableFactory,
@ -204,6 +212,107 @@ public class SqlTestFramework
* MSQ right now needs a full query run.
*/
Boolean isExplainSupported();
QueryRunnerFactoryConglomerate wrapConglomerate(QueryRunnerFactoryConglomerate conglomerate, Closer resourceCloser);
Map<? extends Class<? extends Query>, ? extends QueryRunnerFactory> makeRunnerFactories(Injector injector);
Map<? extends Class<? extends Query>, ? extends QueryToolChest> makeToolChests(Injector injector);
}
public abstract static class QueryComponentSupplierDelegate implements QueryComponentSupplier
{
private final QueryComponentSupplier delegate;
public QueryComponentSupplierDelegate(QueryComponentSupplier delegate)
{
this.delegate = delegate;
}
@Override
public void gatherProperties(Properties properties)
{
delegate.gatherProperties(properties);
}
@Override
public void configureGuice(DruidInjectorBuilder builder)
{
delegate.configureGuice(builder);
}
@Override
public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker(
QueryRunnerFactoryConglomerate conglomerate,
JoinableFactoryWrapper joinableFactory,
Injector injector)
{
return delegate.createQuerySegmentWalker(conglomerate, joinableFactory, injector);
}
@Override
public SqlEngine createEngine(
QueryLifecycleFactory qlf,
ObjectMapper objectMapper,
Injector injector)
{
return delegate.createEngine(qlf, objectMapper, injector);
}
@Override
public void configureJsonMapper(ObjectMapper mapper)
{
delegate.configureJsonMapper(mapper);
}
@Override
public JoinableFactoryWrapper createJoinableFactoryWrapper(LookupExtractorFactoryContainerProvider lookupProvider)
{
return delegate.createJoinableFactoryWrapper(lookupProvider);
}
@Override
public void finalizeTestFramework(SqlTestFramework sqlTestFramework)
{
delegate.finalizeTestFramework(sqlTestFramework);
}
@Override
public PlannerComponentSupplier getPlannerComponentSupplier()
{
return delegate.getPlannerComponentSupplier();
}
@Override
public void close() throws IOException
{
delegate.close();
}
@Override
public Boolean isExplainSupported()
{
return delegate.isExplainSupported();
}
@Override
public QueryRunnerFactoryConglomerate wrapConglomerate(QueryRunnerFactoryConglomerate conglomerate,
Closer resourceCloser)
{
return delegate.wrapConglomerate(conglomerate, resourceCloser);
}
@Override
public Map<? extends Class<? extends Query>, ? extends QueryRunnerFactory> makeRunnerFactories(Injector injector)
{
return delegate.makeRunnerFactories(injector);
}
@Override
public Map<? extends Class<? extends Query>, ? extends QueryToolChest> makeToolChests(Injector injector)
{
return delegate.makeToolChests(injector);
}
}
public interface PlannerComponentSupplier
@ -258,28 +367,6 @@ public class SqlTestFramework
{
}
@Override
public QueryRunnerFactoryConglomerate createCongolmerate(
Builder builder,
Closer resourceCloser,
ObjectMapper jsonMapper
)
{
if (builder.mergeBufferCount == 0) {
return QueryStackTests.createQueryRunnerFactoryConglomerate(
resourceCloser,
() -> builder.minTopNThreshold,
jsonMapper
);
} else {
return QueryStackTests.createQueryRunnerFactoryConglomerate(
resourceCloser,
QueryStackTests.getProcessingConfig(builder.mergeBufferCount),
jsonMapper
);
}
}
@Override
public SpecificSegmentsQuerySegmentWalker createQuerySegmentWalker(
final QueryRunnerFactoryConglomerate conglomerate,
@ -348,6 +435,26 @@ public class SqlTestFramework
{
return true;
}
@Override
public QueryRunnerFactoryConglomerate wrapConglomerate(QueryRunnerFactoryConglomerate conglomerate,
Closer resourceCloser)
{
return conglomerate;
}
@Override
public Map<? extends Class<? extends Query>, ? extends QueryRunnerFactory> makeRunnerFactories(Injector injector)
{
return Collections.emptyMap();
}
@Override
public Map<? extends Class<? extends Query>, ? extends QueryToolChest> makeToolChests(Injector injector)
{
return ImmutableMap.<Class<? extends Query>, QueryToolChest>builder()
.build();
}
}
public static class StandardPlannerComponentSupplier implements PlannerComponentSupplier
@ -548,6 +655,8 @@ public class SqlTestFramework
}
}
public static final String SQL_TEST_FRAME_WORK = "sqlTestFrameWork";
/**
* Guice module to create the various query framework items. By creating items within
* a module, later items can depend on those created earlier by grabbing them from the
@ -592,11 +701,85 @@ public class SqlTestFramework
binder.bind(DefaultColumnFormatConfig.class).toInstance(new DefaultColumnFormatConfig(null, null));
}
@Provides
@LazySingleton
public QueryRunnerFactoryConglomerate conglomerate()
public @Named(SQL_TEST_FRAME_WORK) Map<Class<? extends Query>, QueryRunnerFactory> makeRunnerFactories(
ObjectMapper jsonMapper,
final TestBufferPool testBufferPool,
final TestGroupByBuffers groupByBuffers,
@Named(SqlTestFramework.SQL_TEST_FRAME_WORK) DruidProcessingConfig processingConfig)
{
return componentSupplier.createCongolmerate(builder, resourceCloser, queryJsonMapper());
return ImmutableMap.<Class<? extends Query>, QueryRunnerFactory>builder()
.putAll(
QueryStackTests
.makeDefaultQueryRunnerFactories(
processingConfig,
builder.minTopNThreshold,
jsonMapper,
testBufferPool,
groupByBuffers
)
)
.putAll(componentSupplier.makeRunnerFactories(injector))
.build();
}
@Provides
@LazySingleton
public @Named(SQL_TEST_FRAME_WORK) Map<Class<? extends Query>, QueryToolChest> makeToolchests(
@Named(SQL_TEST_FRAME_WORK) Map<Class<? extends Query>, QueryRunnerFactory> factories)
{
return ImmutableMap.<Class<? extends Query>, QueryToolChest>builder()
.putAll(Maps.transformValues(factories, f -> f.getToolchest()))
.putAll(componentSupplier.makeToolChests(injector))
.build();
}
@Provides
@LazySingleton
public @Named(SQL_TEST_FRAME_WORK) Map<Class<? extends Query>, QueryLogic> makeQueryLogics(
UnionQueryLogic unionQueryLogic)
{
return ImmutableMap.<Class<? extends Query>, QueryLogic>builder()
.put(UnionQuery.class, unionQueryLogic)
.build();
}
/*
* Ideally this should not have a Named annotation, but it clashes with {@link DruidProcessingModule}.
*/
@Named(SQL_TEST_FRAME_WORK)
@Provides
@LazySingleton
public DruidProcessingConfig makeProcessingConfig()
{
return QueryStackTests.getProcessingConfig(builder.mergeBufferCount);
}
@Provides
@LazySingleton
public TestBufferPool makeTestBufferPool()
{
return QueryStackTests.makeTestBufferPool(resourceCloser);
}
@Provides
@LazySingleton
public TestGroupByBuffers makeTestGroupByBuffers(@Named(SQL_TEST_FRAME_WORK) DruidProcessingConfig processingConfig)
{
return QueryStackTests.makeGroupByBuffers(resourceCloser, processingConfig);
}
@Provides
@LazySingleton
public QueryRunnerFactoryConglomerate conglomerate(
@Named(SQL_TEST_FRAME_WORK) Map<Class<? extends Query>, QueryRunnerFactory> factories,
@Named(SQL_TEST_FRAME_WORK) Map<Class<? extends Query>, QueryToolChest> toolchests,
@Named(SQL_TEST_FRAME_WORK) Map<Class<? extends Query>, QueryLogic> querylogics)
{
QueryRunnerFactoryConglomerate conglomerate = new DefaultQueryRunnerFactoryConglomerate(factories, toolchests, querylogics);
return componentSupplier.wrapConglomerate(conglomerate, resourceCloser);
}
@Provides

View File

@ -48,6 +48,7 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.query.DefaultQueryConfig;
import org.apache.druid.query.GenericQueryMetricsFactory;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider;
@ -119,6 +120,9 @@ public class SqlModuleTest
@Mock
private SegmentCacheManager segmentCacheManager;
@Mock
private QueryRunnerFactoryConglomerate conglomerate;
private Injector injector;
@Before
@ -200,6 +204,7 @@ public class SqlModuleTest
binder.bind(GenericQueryMetricsFactory.class).toInstance(genericQueryMetricsFactory);
binder.bind(QuerySegmentWalker.class).toInstance(querySegmentWalker);
binder.bind(QueryToolChestWarehouse.class).toInstance(queryToolChestWarehouse);
binder.bind(QueryRunnerFactoryConglomerate.class).toInstance(conglomerate);
binder.bind(LookupExtractorFactoryContainerProvider.class).toInstance(lookupExtractorFactoryContainerProvider);
binder.bind(JoinableFactory.class).toInstance(joinableFactory);
binder.bind(SegmentCacheManager.class).toInstance(segmentCacheManager);

View File

@ -1,5 +1,5 @@
# testUnnestExtractionFn@NullHandling=default case-crc:404dc668
# quidem testcase reason: UNNEST_DIFFERENT_RESULTSET
# testUnnestExtractionFn@NullHandling=default case-crc:33ee4d88
# quidem testcase reason: UNNEST_SUBSTRING_EMPTY
!set debug true
!set defaultTimeout 300000
!set maxScatterGatherBytes 9223372036854775807

View File

@ -1,5 +1,5 @@
# testUnnestExtractionFn@NullHandling=sql case-crc:404dc668
# quidem testcase reason: UNNEST_DIFFERENT_RESULTSET
# testUnnestExtractionFn@NullHandling=sql case-crc:33ee4d88
# quidem testcase reason: UNNEST_SUBSTRING_EMPTY
!set debug true
!set defaultTimeout 300000
!set maxScatterGatherBytes 9223372036854775807

View File

@ -0,0 +1,147 @@
# testUnionAllTwoQueriesBothQueriesAreJoin case-crc:0924562a
# quidem testcase reason: UNION_ALL_QUERY
!set debug true
!set defaultTimeout 300000
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
(SELECT COUNT(*) FROM foo LEFT JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k UNION ALL SELECT COUNT(*) FROM foo INNER JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k) ;
+--------+
| EXPR$0 |
+--------+
| 1 |
| 6 |
+--------+
(2 rows)
!ok
LogicalUnion(all=[true])
LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
LogicalJoin(condition=[=($0, $1)], joinType=[left])
LogicalProject(dim1=[$1])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(k=[$0])
LogicalTableScan(table=[[lookup, lookyloo]])
LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
LogicalJoin(condition=[=($0, $1)], joinType=[inner])
LogicalProject(dim1=[$1])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(k=[$0])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidUnion(all=[true])
DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical])
DruidJoin(condition=[=($0, $1)], joinType=[left])
DruidProject(dim1=[$1], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(k=[$0], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical])
DruidJoin(condition=[=($0, $1)], joinType=[inner])
DruidProject(dim1=[$1], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(k=[$0], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "union",
"queries" : [ {
"queryType" : "timeseries",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"dim1\" == \"j0.k\")",
"joinType" : "LEFT"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"aggregations" : [ {
"type" : "count",
"name" : "a0"
} ],
"context" : {
"debug" : "true",
"defaultTimeout" : "300000",
"maxScatterGatherBytes" : "9223372036854775807",
"outputformat" : "MYSQL",
"plannerStrategy" : "DECOUPLED",
"sqlCurrentTimestamp" : "2000-01-01T00:00:00Z",
"sqlQueryId" : "dummy",
"sqlStringifyArrays" : false
}
}, {
"queryType" : "timeseries",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"dim1\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"aggregations" : [ {
"type" : "count",
"name" : "a0"
} ],
"context" : {
"debug" : "true",
"defaultTimeout" : "300000",
"maxScatterGatherBytes" : "9223372036854775807",
"outputformat" : "MYSQL",
"plannerStrategy" : "DECOUPLED",
"sqlCurrentTimestamp" : "2000-01-01T00:00:00Z",
"sqlQueryId" : "dummy",
"sqlStringifyArrays" : false
}
} ],
"context" : null
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "a0" ],
"columnTypes" : [ "LONG" ],
"granularity" : {
"type" : "all"
},
"legacy" : false
}
!nativePlan

View File

@ -0,0 +1,139 @@
# testUnionAllTwoQueriesLeftQueryIsJoin@all_disabled case-crc:7ee1bcbd
# quidem testcase reason: UNION_ALL_QUERY
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters false
!set enableRewriteJoinToFilter false
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
(SELECT COUNT(*) FROM foo INNER JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k) UNION ALL SELECT SUM(cnt) FROM foo;
+--------+
| EXPR$0 |
+--------+
| 1 |
| 6 |
+--------+
(2 rows)
!ok
LogicalUnion(all=[true])
LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
LogicalJoin(condition=[=($0, $1)], joinType=[inner])
LogicalProject(dim1=[$1])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(k=[$0])
LogicalTableScan(table=[[lookup, lookyloo]])
LogicalAggregate(group=[{}], EXPR$0=[SUM($4)])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidUnion(all=[true])
DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical])
DruidJoin(condition=[=($0, $1)], joinType=[inner])
DruidProject(dim1=[$1], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(k=[$0], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
DruidAggregate(group=[{}], EXPR$0=[SUM($4)], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "union",
"queries" : [ {
"queryType" : "timeseries",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"dim1\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"aggregations" : [ {
"type" : "count",
"name" : "a0"
} ],
"context" : {
"debug" : "true",
"defaultTimeout" : "300000",
"enableJoinFilterRewrite" : "false",
"enableJoinFilterRewriteValueColumnFilters" : "false",
"enableRewriteJoinToFilter" : "false",
"maxScatterGatherBytes" : "9223372036854775807",
"outputformat" : "MYSQL",
"plannerStrategy" : "DECOUPLED",
"sqlCurrentTimestamp" : "2000-01-01T00:00:00Z",
"sqlQueryId" : "dummy",
"sqlStringifyArrays" : false
}
}, {
"queryType" : "timeseries",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"aggregations" : [ {
"type" : "longSum",
"name" : "a0",
"fieldName" : "cnt"
} ],
"context" : {
"debug" : "true",
"defaultTimeout" : "300000",
"enableJoinFilterRewrite" : "false",
"enableJoinFilterRewriteValueColumnFilters" : "false",
"enableRewriteJoinToFilter" : "false",
"maxScatterGatherBytes" : "9223372036854775807",
"outputformat" : "MYSQL",
"plannerStrategy" : "DECOUPLED",
"sqlCurrentTimestamp" : "2000-01-01T00:00:00Z",
"sqlQueryId" : "dummy",
"sqlStringifyArrays" : false
}
} ],
"context" : null
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "a0" ],
"columnTypes" : [ "LONG" ],
"granularity" : {
"type" : "all"
},
"legacy" : false
}
!nativePlan

View File

@ -0,0 +1,139 @@
# testUnionAllTwoQueriesLeftQueryIsJoin@all_enabled case-crc:6ac8a0fc
# quidem testcase reason: UNION_ALL_QUERY
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
(SELECT COUNT(*) FROM foo INNER JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k) UNION ALL SELECT SUM(cnt) FROM foo;
+--------+
| EXPR$0 |
+--------+
| 1 |
| 6 |
+--------+
(2 rows)
!ok
LogicalUnion(all=[true])
LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
LogicalJoin(condition=[=($0, $1)], joinType=[inner])
LogicalProject(dim1=[$1])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(k=[$0])
LogicalTableScan(table=[[lookup, lookyloo]])
LogicalAggregate(group=[{}], EXPR$0=[SUM($4)])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidUnion(all=[true])
DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical])
DruidJoin(condition=[=($0, $1)], joinType=[inner])
DruidProject(dim1=[$1], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(k=[$0], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
DruidAggregate(group=[{}], EXPR$0=[SUM($4)], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "union",
"queries" : [ {
"queryType" : "timeseries",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"dim1\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"aggregations" : [ {
"type" : "count",
"name" : "a0"
} ],
"context" : {
"debug" : "true",
"defaultTimeout" : "300000",
"enableJoinFilterRewrite" : "true",
"enableJoinFilterRewriteValueColumnFilters" : "true",
"enableRewriteJoinToFilter" : "true",
"maxScatterGatherBytes" : "9223372036854775807",
"outputformat" : "MYSQL",
"plannerStrategy" : "DECOUPLED",
"sqlCurrentTimestamp" : "2000-01-01T00:00:00Z",
"sqlQueryId" : "dummy",
"sqlStringifyArrays" : false
}
}, {
"queryType" : "timeseries",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"aggregations" : [ {
"type" : "longSum",
"name" : "a0",
"fieldName" : "cnt"
} ],
"context" : {
"debug" : "true",
"defaultTimeout" : "300000",
"enableJoinFilterRewrite" : "true",
"enableJoinFilterRewriteValueColumnFilters" : "true",
"enableRewriteJoinToFilter" : "true",
"maxScatterGatherBytes" : "9223372036854775807",
"outputformat" : "MYSQL",
"plannerStrategy" : "DECOUPLED",
"sqlCurrentTimestamp" : "2000-01-01T00:00:00Z",
"sqlQueryId" : "dummy",
"sqlStringifyArrays" : false
}
} ],
"context" : null
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "a0" ],
"columnTypes" : [ "LONG" ],
"granularity" : {
"type" : "all"
},
"legacy" : false
}
!nativePlan

View File

@ -0,0 +1,130 @@
# testUnionAllTwoQueriesLeftQueryIsJoin@default case-crc:4327c7fa
# quidem testcase reason: UNION_ALL_QUERY
!set debug true
!set defaultTimeout 300000
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
(SELECT COUNT(*) FROM foo INNER JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k) UNION ALL SELECT SUM(cnt) FROM foo;
+--------+
| EXPR$0 |
+--------+
| 1 |
| 6 |
+--------+
(2 rows)
!ok
LogicalUnion(all=[true])
LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
LogicalJoin(condition=[=($0, $1)], joinType=[inner])
LogicalProject(dim1=[$1])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(k=[$0])
LogicalTableScan(table=[[lookup, lookyloo]])
LogicalAggregate(group=[{}], EXPR$0=[SUM($4)])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidUnion(all=[true])
DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical])
DruidJoin(condition=[=($0, $1)], joinType=[inner])
DruidProject(dim1=[$1], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(k=[$0], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
DruidAggregate(group=[{}], EXPR$0=[SUM($4)], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "union",
"queries" : [ {
"queryType" : "timeseries",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"dim1\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"aggregations" : [ {
"type" : "count",
"name" : "a0"
} ],
"context" : {
"debug" : "true",
"defaultTimeout" : "300000",
"maxScatterGatherBytes" : "9223372036854775807",
"outputformat" : "MYSQL",
"plannerStrategy" : "DECOUPLED",
"sqlCurrentTimestamp" : "2000-01-01T00:00:00Z",
"sqlQueryId" : "dummy",
"sqlStringifyArrays" : false
}
}, {
"queryType" : "timeseries",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"aggregations" : [ {
"type" : "longSum",
"name" : "a0",
"fieldName" : "cnt"
} ],
"context" : {
"debug" : "true",
"defaultTimeout" : "300000",
"maxScatterGatherBytes" : "9223372036854775807",
"outputformat" : "MYSQL",
"plannerStrategy" : "DECOUPLED",
"sqlCurrentTimestamp" : "2000-01-01T00:00:00Z",
"sqlQueryId" : "dummy",
"sqlStringifyArrays" : false
}
} ],
"context" : null
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "a0" ],
"columnTypes" : [ "LONG" ],
"granularity" : {
"type" : "all"
},
"legacy" : false
}
!nativePlan

View File

@ -0,0 +1,139 @@
# testUnionAllTwoQueriesLeftQueryIsJoin@filter-on-value-column_disabled case-crc:10dc98d8
# quidem testcase reason: UNION_ALL_QUERY
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters false
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
(SELECT COUNT(*) FROM foo INNER JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k) UNION ALL SELECT SUM(cnt) FROM foo;
+--------+
| EXPR$0 |
+--------+
| 1 |
| 6 |
+--------+
(2 rows)
!ok
LogicalUnion(all=[true])
LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
LogicalJoin(condition=[=($0, $1)], joinType=[inner])
LogicalProject(dim1=[$1])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(k=[$0])
LogicalTableScan(table=[[lookup, lookyloo]])
LogicalAggregate(group=[{}], EXPR$0=[SUM($4)])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidUnion(all=[true])
DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical])
DruidJoin(condition=[=($0, $1)], joinType=[inner])
DruidProject(dim1=[$1], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(k=[$0], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
DruidAggregate(group=[{}], EXPR$0=[SUM($4)], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "union",
"queries" : [ {
"queryType" : "timeseries",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"dim1\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"aggregations" : [ {
"type" : "count",
"name" : "a0"
} ],
"context" : {
"debug" : "true",
"defaultTimeout" : "300000",
"enableJoinFilterRewrite" : "true",
"enableJoinFilterRewriteValueColumnFilters" : "false",
"enableRewriteJoinToFilter" : "true",
"maxScatterGatherBytes" : "9223372036854775807",
"outputformat" : "MYSQL",
"plannerStrategy" : "DECOUPLED",
"sqlCurrentTimestamp" : "2000-01-01T00:00:00Z",
"sqlQueryId" : "dummy",
"sqlStringifyArrays" : false
}
}, {
"queryType" : "timeseries",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"aggregations" : [ {
"type" : "longSum",
"name" : "a0",
"fieldName" : "cnt"
} ],
"context" : {
"debug" : "true",
"defaultTimeout" : "300000",
"enableJoinFilterRewrite" : "true",
"enableJoinFilterRewriteValueColumnFilters" : "false",
"enableRewriteJoinToFilter" : "true",
"maxScatterGatherBytes" : "9223372036854775807",
"outputformat" : "MYSQL",
"plannerStrategy" : "DECOUPLED",
"sqlCurrentTimestamp" : "2000-01-01T00:00:00Z",
"sqlQueryId" : "dummy",
"sqlStringifyArrays" : false
}
} ],
"context" : null
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "a0" ],
"columnTypes" : [ "LONG" ],
"granularity" : {
"type" : "all"
},
"legacy" : false
}
!nativePlan

View File

@ -0,0 +1,139 @@
# testUnionAllTwoQueriesLeftQueryIsJoin@filter-rewrites-disabled case-crc:08a9a1e5
# quidem testcase reason: UNION_ALL_QUERY
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
(SELECT COUNT(*) FROM foo INNER JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k) UNION ALL SELECT SUM(cnt) FROM foo;
+--------+
| EXPR$0 |
+--------+
| 1 |
| 6 |
+--------+
(2 rows)
!ok
LogicalUnion(all=[true])
LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
LogicalJoin(condition=[=($0, $1)], joinType=[inner])
LogicalProject(dim1=[$1])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(k=[$0])
LogicalTableScan(table=[[lookup, lookyloo]])
LogicalAggregate(group=[{}], EXPR$0=[SUM($4)])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidUnion(all=[true])
DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical])
DruidJoin(condition=[=($0, $1)], joinType=[inner])
DruidProject(dim1=[$1], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(k=[$0], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
DruidAggregate(group=[{}], EXPR$0=[SUM($4)], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "union",
"queries" : [ {
"queryType" : "timeseries",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"dim1\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"aggregations" : [ {
"type" : "count",
"name" : "a0"
} ],
"context" : {
"debug" : "true",
"defaultTimeout" : "300000",
"enableJoinFilterRewrite" : "false",
"enableJoinFilterRewriteValueColumnFilters" : "true",
"enableRewriteJoinToFilter" : "true",
"maxScatterGatherBytes" : "9223372036854775807",
"outputformat" : "MYSQL",
"plannerStrategy" : "DECOUPLED",
"sqlCurrentTimestamp" : "2000-01-01T00:00:00Z",
"sqlQueryId" : "dummy",
"sqlStringifyArrays" : false
}
}, {
"queryType" : "timeseries",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"aggregations" : [ {
"type" : "longSum",
"name" : "a0",
"fieldName" : "cnt"
} ],
"context" : {
"debug" : "true",
"defaultTimeout" : "300000",
"enableJoinFilterRewrite" : "false",
"enableJoinFilterRewriteValueColumnFilters" : "true",
"enableRewriteJoinToFilter" : "true",
"maxScatterGatherBytes" : "9223372036854775807",
"outputformat" : "MYSQL",
"plannerStrategy" : "DECOUPLED",
"sqlCurrentTimestamp" : "2000-01-01T00:00:00Z",
"sqlQueryId" : "dummy",
"sqlStringifyArrays" : false
}
} ],
"context" : null
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "a0" ],
"columnTypes" : [ "LONG" ],
"granularity" : {
"type" : "all"
},
"legacy" : false
}
!nativePlan

View File

@ -0,0 +1,139 @@
# testUnionAllTwoQueriesLeftQueryIsJoin@filter-rewrites case-crc:a2003660
# quidem testcase reason: UNION_ALL_QUERY
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters true
!set enableRewriteJoinToFilter false
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
(SELECT COUNT(*) FROM foo INNER JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k) UNION ALL SELECT SUM(cnt) FROM foo;
+--------+
| EXPR$0 |
+--------+
| 1 |
| 6 |
+--------+
(2 rows)
!ok
LogicalUnion(all=[true])
LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
LogicalJoin(condition=[=($0, $1)], joinType=[inner])
LogicalProject(dim1=[$1])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(k=[$0])
LogicalTableScan(table=[[lookup, lookyloo]])
LogicalAggregate(group=[{}], EXPR$0=[SUM($4)])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidUnion(all=[true])
DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical])
DruidJoin(condition=[=($0, $1)], joinType=[inner])
DruidProject(dim1=[$1], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(k=[$0], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
DruidAggregate(group=[{}], EXPR$0=[SUM($4)], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "union",
"queries" : [ {
"queryType" : "timeseries",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"dim1\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"aggregations" : [ {
"type" : "count",
"name" : "a0"
} ],
"context" : {
"debug" : "true",
"defaultTimeout" : "300000",
"enableJoinFilterRewrite" : "true",
"enableJoinFilterRewriteValueColumnFilters" : "true",
"enableRewriteJoinToFilter" : "false",
"maxScatterGatherBytes" : "9223372036854775807",
"outputformat" : "MYSQL",
"plannerStrategy" : "DECOUPLED",
"sqlCurrentTimestamp" : "2000-01-01T00:00:00Z",
"sqlQueryId" : "dummy",
"sqlStringifyArrays" : false
}
}, {
"queryType" : "timeseries",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"aggregations" : [ {
"type" : "longSum",
"name" : "a0",
"fieldName" : "cnt"
} ],
"context" : {
"debug" : "true",
"defaultTimeout" : "300000",
"enableJoinFilterRewrite" : "true",
"enableJoinFilterRewriteValueColumnFilters" : "true",
"enableRewriteJoinToFilter" : "false",
"maxScatterGatherBytes" : "9223372036854775807",
"outputformat" : "MYSQL",
"plannerStrategy" : "DECOUPLED",
"sqlCurrentTimestamp" : "2000-01-01T00:00:00Z",
"sqlQueryId" : "dummy",
"sqlStringifyArrays" : false
}
} ],
"context" : null
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "a0" ],
"columnTypes" : [ "LONG" ],
"granularity" : {
"type" : "all"
},
"legacy" : false
}
!nativePlan

View File

@ -0,0 +1,139 @@
# testUnionAllTwoQueriesLeftQueryIsJoin@join-to-filter case-crc:05dd3d7b
# quidem testcase reason: UNION_ALL_QUERY
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters false
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
(SELECT COUNT(*) FROM foo INNER JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k) UNION ALL SELECT SUM(cnt) FROM foo;
+--------+
| EXPR$0 |
+--------+
| 1 |
| 6 |
+--------+
(2 rows)
!ok
LogicalUnion(all=[true])
LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
LogicalJoin(condition=[=($0, $1)], joinType=[inner])
LogicalProject(dim1=[$1])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(k=[$0])
LogicalTableScan(table=[[lookup, lookyloo]])
LogicalAggregate(group=[{}], EXPR$0=[SUM($4)])
LogicalTableScan(table=[[druid, foo]])
!logicalPlan
DruidUnion(all=[true])
DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical])
DruidJoin(condition=[=($0, $1)], joinType=[inner])
DruidProject(dim1=[$1], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(k=[$0], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
DruidAggregate(group=[{}], EXPR$0=[SUM($4)], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "union",
"queries" : [ {
"queryType" : "timeseries",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"dim1\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"aggregations" : [ {
"type" : "count",
"name" : "a0"
} ],
"context" : {
"debug" : "true",
"defaultTimeout" : "300000",
"enableJoinFilterRewrite" : "false",
"enableJoinFilterRewriteValueColumnFilters" : "false",
"enableRewriteJoinToFilter" : "true",
"maxScatterGatherBytes" : "9223372036854775807",
"outputformat" : "MYSQL",
"plannerStrategy" : "DECOUPLED",
"sqlCurrentTimestamp" : "2000-01-01T00:00:00Z",
"sqlQueryId" : "dummy",
"sqlStringifyArrays" : false
}
}, {
"queryType" : "timeseries",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"aggregations" : [ {
"type" : "longSum",
"name" : "a0",
"fieldName" : "cnt"
} ],
"context" : {
"debug" : "true",
"defaultTimeout" : "300000",
"enableJoinFilterRewrite" : "false",
"enableJoinFilterRewriteValueColumnFilters" : "false",
"enableRewriteJoinToFilter" : "true",
"maxScatterGatherBytes" : "9223372036854775807",
"outputformat" : "MYSQL",
"plannerStrategy" : "DECOUPLED",
"sqlCurrentTimestamp" : "2000-01-01T00:00:00Z",
"sqlQueryId" : "dummy",
"sqlStringifyArrays" : false
}
} ],
"context" : null
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "a0" ],
"columnTypes" : [ "LONG" ],
"granularity" : {
"type" : "all"
},
"legacy" : false
}
!nativePlan

View File

@ -0,0 +1,139 @@
# testUnionAllTwoQueriesRightQueryIsJoin@all_disabled case-crc:1181ba66
# quidem testcase reason: UNION_ALL_QUERY
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite false
!set enableJoinFilterRewriteValueColumnFilters false
!set enableRewriteJoinToFilter false
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
(SELECT SUM(cnt) FROM foo UNION ALL SELECT COUNT(*) FROM foo INNER JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k) ;
+--------+
| EXPR$0 |
+--------+
| 1 |
| 6 |
+--------+
(2 rows)
!ok
LogicalUnion(all=[true])
LogicalAggregate(group=[{}], EXPR$0=[SUM($4)])
LogicalTableScan(table=[[druid, foo]])
LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
LogicalJoin(condition=[=($0, $1)], joinType=[inner])
LogicalProject(dim1=[$1])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(k=[$0])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidUnion(all=[true])
DruidAggregate(group=[{}], EXPR$0=[SUM($4)], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical])
DruidJoin(condition=[=($0, $1)], joinType=[inner])
DruidProject(dim1=[$1], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(k=[$0], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "union",
"queries" : [ {
"queryType" : "timeseries",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"aggregations" : [ {
"type" : "longSum",
"name" : "a0",
"fieldName" : "cnt"
} ],
"context" : {
"debug" : "true",
"defaultTimeout" : "300000",
"enableJoinFilterRewrite" : "false",
"enableJoinFilterRewriteValueColumnFilters" : "false",
"enableRewriteJoinToFilter" : "false",
"maxScatterGatherBytes" : "9223372036854775807",
"outputformat" : "MYSQL",
"plannerStrategy" : "DECOUPLED",
"sqlCurrentTimestamp" : "2000-01-01T00:00:00Z",
"sqlQueryId" : "dummy",
"sqlStringifyArrays" : false
}
}, {
"queryType" : "timeseries",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"dim1\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"aggregations" : [ {
"type" : "count",
"name" : "a0"
} ],
"context" : {
"debug" : "true",
"defaultTimeout" : "300000",
"enableJoinFilterRewrite" : "false",
"enableJoinFilterRewriteValueColumnFilters" : "false",
"enableRewriteJoinToFilter" : "false",
"maxScatterGatherBytes" : "9223372036854775807",
"outputformat" : "MYSQL",
"plannerStrategy" : "DECOUPLED",
"sqlCurrentTimestamp" : "2000-01-01T00:00:00Z",
"sqlQueryId" : "dummy",
"sqlStringifyArrays" : false
}
} ],
"context" : null
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "a0" ],
"columnTypes" : [ "LONG" ],
"granularity" : {
"type" : "all"
},
"legacy" : false
}
!nativePlan

View File

@ -0,0 +1,139 @@
# testUnionAllTwoQueriesRightQueryIsJoin@all_enabled case-crc:05a8a627
# quidem testcase reason: UNION_ALL_QUERY
!set debug true
!set defaultTimeout 300000
!set enableJoinFilterRewrite true
!set enableJoinFilterRewriteValueColumnFilters true
!set enableRewriteJoinToFilter true
!set maxScatterGatherBytes 9223372036854775807
!set plannerStrategy DECOUPLED
!set sqlCurrentTimestamp 2000-01-01T00:00:00Z
!set sqlQueryId dummy
!set outputformat mysql
!use druidtest:///
(SELECT SUM(cnt) FROM foo UNION ALL SELECT COUNT(*) FROM foo INNER JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k) ;
+--------+
| EXPR$0 |
+--------+
| 1 |
| 6 |
+--------+
(2 rows)
!ok
LogicalUnion(all=[true])
LogicalAggregate(group=[{}], EXPR$0=[SUM($4)])
LogicalTableScan(table=[[druid, foo]])
LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
LogicalJoin(condition=[=($0, $1)], joinType=[inner])
LogicalProject(dim1=[$1])
LogicalTableScan(table=[[druid, foo]])
LogicalProject(k=[$0])
LogicalTableScan(table=[[lookup, lookyloo]])
!logicalPlan
DruidUnion(all=[true])
DruidAggregate(group=[{}], EXPR$0=[SUM($4)], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical])
DruidJoin(condition=[=($0, $1)], joinType=[inner])
DruidProject(dim1=[$1], druid=[logical])
DruidTableScan(table=[[druid, foo]], druid=[logical])
DruidProject(k=[$0], druid=[logical])
DruidTableScan(table=[[lookup, lookyloo]], druid=[logical])
!druidPlan
{
"queryType" : "scan",
"dataSource" : {
"type" : "query",
"query" : {
"queryType" : "union",
"queries" : [ {
"queryType" : "timeseries",
"dataSource" : {
"type" : "table",
"name" : "foo"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"aggregations" : [ {
"type" : "longSum",
"name" : "a0",
"fieldName" : "cnt"
} ],
"context" : {
"debug" : "true",
"defaultTimeout" : "300000",
"enableJoinFilterRewrite" : "true",
"enableJoinFilterRewriteValueColumnFilters" : "true",
"enableRewriteJoinToFilter" : "true",
"maxScatterGatherBytes" : "9223372036854775807",
"outputformat" : "MYSQL",
"plannerStrategy" : "DECOUPLED",
"sqlCurrentTimestamp" : "2000-01-01T00:00:00Z",
"sqlQueryId" : "dummy",
"sqlStringifyArrays" : false
}
}, {
"queryType" : "timeseries",
"dataSource" : {
"type" : "join",
"left" : {
"type" : "table",
"name" : "foo"
},
"right" : {
"type" : "lookup",
"lookup" : "lookyloo"
},
"rightPrefix" : "j0.",
"condition" : "(\"dim1\" == \"j0.k\")",
"joinType" : "INNER"
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"granularity" : {
"type" : "all"
},
"aggregations" : [ {
"type" : "count",
"name" : "a0"
} ],
"context" : {
"debug" : "true",
"defaultTimeout" : "300000",
"enableJoinFilterRewrite" : "true",
"enableJoinFilterRewriteValueColumnFilters" : "true",
"enableRewriteJoinToFilter" : "true",
"maxScatterGatherBytes" : "9223372036854775807",
"outputformat" : "MYSQL",
"plannerStrategy" : "DECOUPLED",
"sqlCurrentTimestamp" : "2000-01-01T00:00:00Z",
"sqlQueryId" : "dummy",
"sqlStringifyArrays" : false
}
} ],
"context" : null
}
},
"intervals" : {
"type" : "intervals",
"intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ]
},
"resultFormat" : "compactedList",
"columns" : [ "a0" ],
"columnTypes" : [ "LONG" ],
"granularity" : {
"type" : "all"
},
"legacy" : false
}
!nativePlan

Some files were not shown because too many files have changed in this diff Show More