From 2eac8318f840efbfe03afa24e75a481783e1123b Mon Sep 17 00:00:00 2001 From: Zoltan Haindrich Date: Tue, 5 Nov 2024 16:58:57 +0100 Subject: [PATCH] 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 --- .../CachingClusteredClientBenchmark.java | 74 ++--- .../DatasourceOptimizerTest.java | 4 +- .../movingaverage/MovingAverageQueryTest.java | 12 +- .../indexing/kafka/KafkaIndexTaskTest.java | 38 +-- .../kinesis/KinesisIndexTaskTest.java | 20 +- .../apache/druid/error/DruidException.java | 1 - .../ArenaMemoryAllocatorFactory.java | 7 + .../java/util/common/guava/Sequences.java | 6 +- ...DefaultQueryRunnerFactoryConglomerate.java | 41 ++- .../apache/druid/query/FluentQueryRunner.java | 2 +- .../java/org/apache/druid/query/Query.java | 21 ++ .../org/apache/druid/query/QueryContext.java | 5 + .../org/apache/druid/query/QueryContexts.java | 2 + .../apache/druid/query/QueryDataSource.java | 37 ++- .../org/apache/druid/query/QueryLogic.java | 37 +++ .../query/QueryLogicCompatToolChest.java | 79 +++++ .../org/apache/druid/query/QueryPlus.java | 9 + .../query/QueryRunnerFactoryConglomerate.java | 4 + .../apache/druid/query/QueryToolChest.java | 7 + .../ToolChestBasedResultSerializedRunner.java | 93 ++++++ ...r.java => UnionDataSourceQueryRunner.java} | 4 +- .../druid/query/groupby/GroupByQuery.java | 7 +- .../query/operator/WindowOperatorQuery.java | 7 + .../query/planning/DataSourceAnalysis.java | 16 +- .../apache/druid/query/scan/ScanQuery.java | 6 + .../query/timeseries/TimeseriesQuery.java | 7 +- .../TimeseriesQueryQueryToolChest.java | 6 +- .../apache/druid/query/topn/TopNQuery.java | 3 +- .../query/topn/TopNQueryQueryToolChest.java | 6 +- .../apache/druid/query/union/UnionQuery.java | 308 ++++++++++++++++++ .../druid/query/union/UnionQueryLogic.java | 102 ++++++ .../druid/query/QueryToolChestTestHelper.java | 2 +- ...va => UnionDataSourceQueryRunnerTest.java} | 4 +- .../planning/DataSourceAnalysisTest.java | 19 ++ .../scan/ScanQueryQueryToolChestTest.java | 9 +- .../TimeSeriesUnionQueryRunnerTest.java | 4 +- .../union/UnionQueryQueryToolChestTest.java | 211 ++++++++++++ .../segment/CursorFactoryProjectionTest.java | 6 +- ...AsBrokerQueryComponentSupplierWrapper.java | 89 +---- .../druid/client/CachingClusteredClient.java | 10 +- .../druid/client/DirectDruidClient.java | 10 +- .../client/DirectDruidClientFactory.java | 10 +- .../ConglomerateBackedToolChestWarehouse.java | 36 +- .../org/apache/druid/guice/DruidBinders.java | 37 +++ .../druid/guice/QueryRunnerFactoryModule.java | 5 + .../druid/guice/QueryToolChestModule.java | 4 +- .../client/MessageRelayClientImpl.java | 10 +- .../SegmentMetadataQuerySegmentWalker.java | 14 +- .../server/ClientQuerySegmentWalker.java | 142 +++++--- .../apache/druid/server/QueryLifecycle.java | 10 +- .../druid/server/QueryLifecycleFactory.java | 10 +- .../server/initialization/ServerConfig.java | 5 +- .../druid/client/BrokerServerViewTest.java | 12 +- ...chingClusteredClientFunctionalityTest.java | 22 +- .../CachingClusteredClientPerfTest.java | 21 +- .../client/CachingClusteredClientTest.java | 21 +- .../CachingClusteredClientTestUtils.java | 63 ---- .../druid/client/DirectDruidClientTest.java | 12 +- .../apache/druid/client/SimpleServerView.java | 8 +- ...yRunnerBasedOnClusteredClientTestBase.java | 19 +- .../ResultLevelCachingQueryRunnerTest.java | 2 +- ...inatorSegmentDataCacheConcurrencyTest.java | 1 - ...ordinatorSegmentMetadataCacheTestBase.java | 1 - .../SegmentMetadataCacheTestBase.java | 14 +- ...SegmentMetadataQuerySegmentWalkerTest.java | 27 +- .../TestSegmentMetadataQueryWalker.java | 6 +- .../StreamAppenderatorTester.java | 48 ++- ...nifiedIndexerAppenderatorsManagerTest.java | 2 +- .../server/ClientQuerySegmentWalkerTest.java | 53 +++ .../druid/server/QueryLifecycleTest.java | 28 +- .../druid/server/QueryResourceTest.java | 29 +- .../apache/druid/server/QueryStackTests.java | 281 +++++++++------- .../coordination/ServerManagerTest.java | 18 +- .../org/apache/druid/cli/CliCoordinator.java | 1 + .../sql/calcite/rel/logical/DruidUnion.java | 82 ++++- .../calcite/rule/logical/DruidUnionRule.java | 1 - .../druid/quidem/DruidAvaticaTestDriver.java | 73 +---- .../sql/calcite/BaseCalciteQueryTest.java | 20 +- .../sql/calcite/CalciteArraysQueryTest.java | 2 +- .../sql/calcite/CalciteJoinQueryTest.java | 9 +- .../druid/sql/calcite/CalciteQueryTest.java | 7 +- .../druid/sql/calcite/DecoupledExtension.java | 23 +- .../sql/calcite/DecoupledTestConfig.java | 10 +- .../druid/sql/calcite/NotYetSupported.java | 1 - .../sql/calcite/SqlTestFrameworkConfig.java | 3 +- .../sql/calcite/util/QueryFrameworkUtils.java | 13 +- .../sql/calcite/util/SqlTestFramework.java | 243 ++++++++++++-- .../apache/druid/sql/guice/SqlModuleTest.java | 5 + ...UnnestExtractionFn@NullHandling=default.iq | 4 +- ...testUnnestExtractionFn@NullHandling=sql.iq | 4 +- ...estUnionAllTwoQueriesBothQueriesAreJoin.iq | 147 +++++++++ ...lTwoQueriesLeftQueryIsJoin@all_disabled.iq | 139 ++++++++ ...llTwoQueriesLeftQueryIsJoin@all_enabled.iq | 139 ++++++++ ...ionAllTwoQueriesLeftQueryIsJoin@default.iq | 130 ++++++++ ...yIsJoin@filter-on-value-column_disabled.iq | 139 ++++++++ ...eftQueryIsJoin@filter-rewrites-disabled.iq | 139 ++++++++ ...oQueriesLeftQueryIsJoin@filter-rewrites.iq | 139 ++++++++ ...woQueriesLeftQueryIsJoin@join-to-filter.iq | 139 ++++++++ ...TwoQueriesRightQueryIsJoin@all_disabled.iq | 139 ++++++++ ...lTwoQueriesRightQueryIsJoin@all_enabled.iq | 139 ++++++++ ...onAllTwoQueriesRightQueryIsJoin@default.iq | 130 ++++++++ ...yIsJoin@filter-on-value-column_disabled.iq | 139 ++++++++ ...ghtQueryIsJoin@filter-rewrites-disabled.iq | 139 ++++++++ ...QueriesRightQueryIsJoin@filter-rewrites.iq | 139 ++++++++ ...oQueriesRightQueryIsJoin@join-to-filter.iq | 139 ++++++++ .../testUnionAllQueries.iq | 144 ++++++++ .../testUnionAllQueriesWithLimit.iq | 146 +++++++++ 107 files changed, 4337 insertions(+), 835 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/query/QueryLogic.java create mode 100644 processing/src/main/java/org/apache/druid/query/QueryLogicCompatToolChest.java create mode 100644 processing/src/main/java/org/apache/druid/query/ToolChestBasedResultSerializedRunner.java rename processing/src/main/java/org/apache/druid/query/{UnionQueryRunner.java => UnionDataSourceQueryRunner.java} (97%) create mode 100644 processing/src/main/java/org/apache/druid/query/union/UnionQuery.java create mode 100644 processing/src/main/java/org/apache/druid/query/union/UnionQueryLogic.java rename processing/src/test/java/org/apache/druid/query/{UnionQueryRunnerTest.java => UnionDataSourceQueryRunnerTest.java} (96%) create mode 100644 processing/src/test/java/org/apache/druid/query/union/UnionQueryQueryToolChestTest.java rename processing/src/main/java/org/apache/druid/query/ReflectionQueryToolChestWarehouse.java => server/src/main/java/org/apache/druid/guice/ConglomerateBackedToolChestWarehouse.java (50%) create mode 100644 sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesBothQueriesAreJoin.iq create mode 100644 sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@all_disabled.iq create mode 100644 sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@all_enabled.iq create mode 100644 sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@default.iq create mode 100644 sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@filter-on-value-column_disabled.iq create mode 100644 sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@filter-rewrites-disabled.iq create mode 100644 sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@filter-rewrites.iq create mode 100644 sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@join-to-filter.iq create mode 100644 sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@all_disabled.iq create mode 100644 sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@all_enabled.iq create mode 100644 sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@default.iq create mode 100644 sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@filter-on-value-column_disabled.iq create mode 100644 sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@filter-rewrites-disabled.iq create mode 100644 sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@filter-rewrites.iq create mode 100644 sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@join-to-filter.iq create mode 100644 sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testUnionAllQueries.iq create mode 100644 sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testUnionAllQueriesWithLimit.iq diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java index 8e0715e0fe5..684735ef22d 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -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., QueryRunnerFactory>builder() - .put( - TimeseriesQuery.class, - new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest(), - new TimeseriesQueryEngine(), - QueryRunnerTestHelper.NOOP_QUERYWATCHER - ) + conglomerate = DefaultQueryRunnerFactoryConglomerate.buildFromQueryRunnerFactories(ImmutableMap., 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 > QueryToolChest 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 theRunner = FluentQueryRunner .create( cachingClusteredClient.getQueryRunnerForIntervals(query, query.getIntervals()), - toolChestWarehouse.getToolChest(query) + conglomerate.getToolChest(query) ) .applyPreMergeDecoration() .mergeResults(true) diff --git a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java index 98ab56022db..f67c9794b57 100644 --- a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java +++ b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java @@ -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) diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index b6d787cbc21..68668c9b3aa 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -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) res.accumulate(actualResults, Accumulators.list()); + actualResults = res.accumulate(actualResults, Accumulators.list()); expectedResults = consistentTypeCasting(expectedResults); actualResults = consistentTypeCasting(actualResults); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 23bdeb14acb..db8db1fdb38 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -2891,28 +2891,26 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase @Override protected QueryRunnerFactoryConglomerate makeQueryRunnerConglomerate() { - return new DefaultQueryRunnerFactoryConglomerate( - ImmutableMap., QueryRunnerFactory>builder() - .put( - TimeseriesQuery.class, - new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest(), - new TimeseriesQueryEngine(), - (query, future) -> { - // do nothing - } - ) + return DefaultQueryRunnerFactoryConglomerate.buildFromQueryRunnerFactories(ImmutableMap., 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 diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 2ef39148400..088bb177d0e 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -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 diff --git a/processing/src/main/java/org/apache/druid/error/DruidException.java b/processing/src/main/java/org/apache/druid/error/DruidException.java index 66190d13a91..e83bc6cfa68 100644 --- a/processing/src/main/java/org/apache/druid/error/DruidException.java +++ b/processing/src/main/java/org/apache/druid/error/DruidException.java @@ -523,5 +523,4 @@ public class DruidException extends RuntimeException protected abstract DruidException makeException(DruidExceptionBuilder bob); } - } diff --git a/processing/src/main/java/org/apache/druid/frame/allocation/ArenaMemoryAllocatorFactory.java b/processing/src/main/java/org/apache/druid/frame/allocation/ArenaMemoryAllocatorFactory.java index 80d4dcdadd7..61a20d965e9 100644 --- a/processing/src/main/java/org/apache/druid/frame/allocation/ArenaMemoryAllocatorFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/allocation/ArenaMemoryAllocatorFactory.java @@ -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); + } } diff --git a/processing/src/main/java/org/apache/druid/java/util/common/guava/Sequences.java b/processing/src/main/java/org/apache/druid/java/util/common/guava/Sequences.java index 9f8169434af..e66ba29e4d7 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/guava/Sequences.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/guava/Sequences.java @@ -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 Sequence of(T... values) + { + return simple(Arrays.asList(values)); + } } diff --git a/processing/src/main/java/org/apache/druid/query/DefaultQueryRunnerFactoryConglomerate.java b/processing/src/main/java/org/apache/druid/query/DefaultQueryRunnerFactoryConglomerate.java index 7f004ffa634..a7cae77cb1d 100644 --- a/processing/src/main/java/org/apache/druid/query/DefaultQueryRunnerFactoryConglomerate.java +++ b/processing/src/main/java/org/apache/druid/query/DefaultQueryRunnerFactoryConglomerate.java @@ -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, QueryRunnerFactory> factories; + private final Map, QueryToolChest> toolchests; + private final Map, QueryLogic> querylogics; + + public static DefaultQueryRunnerFactoryConglomerate buildFromQueryRunnerFactories( + Map, QueryRunnerFactory> factories) + { + return new DefaultQueryRunnerFactoryConglomerate( + factories, + Maps.transformValues(factories, f -> f.getToolchest()), + Collections.emptyMap() + ); + } @Inject - public DefaultQueryRunnerFactoryConglomerate(Map, QueryRunnerFactory> factories) + public DefaultQueryRunnerFactoryConglomerate( + Map, QueryRunnerFactory> factories, + Map, QueryToolChest> toolchests, + Map, 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 > QueryRunnerFactory findFactory(QueryType query) { - return (QueryRunnerFactory) factories.get(query.getClass()); + return factories.get(query.getClass()); + } + + @Override + @SuppressWarnings("unchecked") + public > QueryToolChest getToolChest(QueryType query) + { + return toolchests.get(query.getClass()); + } + + @Override + @SuppressWarnings("unchecked") + public > QueryLogic getQueryLogic(QueryType query) + { + return querylogics.get(query.getClass()); } } diff --git a/processing/src/main/java/org/apache/druid/query/FluentQueryRunner.java b/processing/src/main/java/org/apache/druid/query/FluentQueryRunner.java index dcef6fa0651..e6f52d47b54 100644 --- a/processing/src/main/java/org/apache/druid/query/FluentQueryRunner.java +++ b/processing/src/main/java/org/apache/druid/query/FluentQueryRunner.java @@ -64,7 +64,7 @@ public class FluentQueryRunner implements QueryRunner public FluentQueryRunner applyPreMergeDecoration() { - return from(new UnionQueryRunner<>(toolChest.preMergeQueryDecoration(baseRunner))); + return from(new UnionDataSourceQueryRunner<>(toolChest.preMergeQueryDecoration(baseRunner))); } public FluentQueryRunner emitCPUTimeMetric(ServiceEmitter emitter) diff --git a/processing/src/main/java/org/apache/druid/query/Query.java b/processing/src/main/java/org/apache/druid/query/Query.java index 4795ecbb561..3ed1dcbe0ea 100644 --- a/processing/src/main/java/org/apache/druid/query/Query.java +++ b/processing/src/main/java/org/apache/druid/query/Query.java @@ -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 { @@ -79,6 +84,7 @@ public interface Query String TIMESERIES = "timeseries"; String TOPN = "topN"; String WINDOW_OPERATOR = "windowOperator"; + String UNION_QUERY = "union"; DataSource getDataSource(); @@ -285,4 +291,19 @@ public interface Query ) ); } + + default DataSourceAnalysis getDataSourceAnalysis() + { + return getDataSource().getAnalysis().maybeWithBaseQuery(this); + } + + default RowSignature getResultRowSignature() + { + return getResultRowSignature(Finalization.UNKNOWN); + } + + default RowSignature getResultRowSignature(RowSignature.Finalization finalization) + { + return null; + } } diff --git a/processing/src/main/java/org/apache/druid/query/QueryContext.java b/processing/src/main/java/org/apache/druid/query/QueryContext.java index 52395275967..d44e0153f0b 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContext.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContext.java @@ -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); diff --git a/processing/src/main/java/org/apache/druid/query/QueryContexts.java b/processing/src/main/java/org/apache/druid/query/QueryContexts.java index f3fc1a80f83..759a9b49437 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java @@ -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 diff --git a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java index 3d889472ea9..08dc44126fe 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java @@ -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 getTableNames() { - return query.getDataSource().getTableNames(); + Set 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 getChildren() { + return getQueryDataSources(); + } + + private List getQueryDataSources() + { + if (query instanceof UnionQuery) { + return ((UnionQuery) query).getDataSources(); + } return Collections.singletonList(query.getDataSource()); } @Override public DataSource withChildren(List 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 diff --git a/processing/src/main/java/org/apache/druid/query/QueryLogic.java b/processing/src/main/java/org/apache/druid/query/QueryLogic.java new file mode 100644 index 00000000000..5ac1b1c46e2 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/QueryLogic.java @@ -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}. + */ + QueryRunner entryPoint( + Query query, + QuerySegmentWalker walker); +} diff --git a/processing/src/main/java/org/apache/druid/query/QueryLogicCompatToolChest.java b/processing/src/main/java/org/apache/druid/query/QueryLogicCompatToolChest.java new file mode 100644 index 00000000000..c8c47d30e2e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/QueryLogicCompatToolChest.java @@ -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> +{ + private RowSignature resultRowSignature; + + public QueryLogicCompatToolChest(RowSignature resultRowSignature) + { + this.resultRowSignature = resultRowSignature; + } + + @Override + public RowSignature resultArraySignature(Query query) + { + return resultRowSignature; + } + + @Override + public QueryMetrics> makeMetrics(Query query) + { + return new DefaultQueryMetrics<>(); + } + + @Override + public Function makePreComputeManipulatorFn(Query query, MetricManipulationFn fn) + { + return Functions.identity(); + } + + @Override + public TypeReference getResultTypeReference() + { + return null; + } + + @Override + public Sequence resultsAsArrays(Query query, Sequence resultSequence) + { + Sequence res = resultSequence; + return (Sequence) res; + } + + @Override + public Optional> resultsAsFrames(Query query, Sequence resultSequence, + MemoryAllocatorFactory memoryAllocatorFactory, boolean useNestedForUnknownTypes) + { + Sequence res = resultSequence; + return Optional.of((Sequence) res); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/QueryPlus.java b/processing/src/main/java/org/apache/druid/query/QueryPlus.java index 1b18e943909..ae71895b8e4 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryPlus.java +++ b/processing/src/main/java/org/apache/druid/query/QueryPlus.java @@ -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 return query; } + public > C unwrapQuery(Class 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() { diff --git a/processing/src/main/java/org/apache/druid/query/QueryRunnerFactoryConglomerate.java b/processing/src/main/java/org/apache/druid/query/QueryRunnerFactoryConglomerate.java index da89d81c8dd..c2a871930c7 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryRunnerFactoryConglomerate.java +++ b/processing/src/main/java/org/apache/druid/query/QueryRunnerFactoryConglomerate.java @@ -24,4 +24,8 @@ package org.apache.druid.query; public interface QueryRunnerFactoryConglomerate { > QueryRunnerFactory findFactory(QueryType query); + + > QueryToolChest getToolChest(QueryType query); + + > QueryLogic getQueryLogic(QueryType query); } diff --git a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java index 978b4922615..6c67da2544e 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java @@ -422,4 +422,11 @@ public abstract class QueryToolChest boolean canExecuteFully(Query query) + { + DataSource dataSourceFromQuery = query.getDataSource(); + return (!(dataSourceFromQuery instanceof QueryDataSource) + || canPerformSubquery(((QueryDataSource) dataSourceFromQuery).getQuery())); + } } diff --git a/processing/src/main/java/org/apache/druid/query/ToolChestBasedResultSerializedRunner.java b/processing/src/main/java/org/apache/druid/query/ToolChestBasedResultSerializedRunner.java new file mode 100644 index 00000000000..1956eef146c --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/ToolChestBasedResultSerializedRunner.java @@ -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 implements QueryRunner +{ + private final QueryRunner runner; + private final QueryToolChest> toolChest; + + public ToolChestBasedResultSerializedRunner( + Query query, + QuerySegmentWalker walker, + QueryToolChest> toolChest) + { + this.runner = query.getRunner(walker); + this.toolChest = toolChest; + } + + // note: returns a Sequence and not Sequenct + @Override + public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) + { + Query query = queryPlus.getQuery(); + Sequence 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 makes most of the template + // usage okay. + return (Sequence) resultSeq; + } + + private ResultSerializationMode getResultSerializationMode(Query 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; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/UnionQueryRunner.java b/processing/src/main/java/org/apache/druid/query/UnionDataSourceQueryRunner.java similarity index 97% rename from processing/src/main/java/org/apache/druid/query/UnionQueryRunner.java rename to processing/src/main/java/org/apache/druid/query/UnionDataSourceQueryRunner.java index 07b761138f4..5eaa84a5e49 100644 --- a/processing/src/main/java/org/apache/druid/query/UnionQueryRunner.java +++ b/processing/src/main/java/org/apache/druid/query/UnionDataSourceQueryRunner.java @@ -34,11 +34,11 @@ import org.apache.druid.query.planning.DataSourceAnalysis; import java.util.stream.Collectors; import java.util.stream.IntStream; -public class UnionQueryRunner implements QueryRunner +public class UnionDataSourceQueryRunner implements QueryRunner { private final QueryRunner baseRunner; - public UnionQueryRunner( + public UnionDataSourceQueryRunner( QueryRunner baseRunner ) { diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java index 35510d7c5ee..79282735f38 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java @@ -330,11 +330,7 @@ public class GroupByQuery extends BaseQuery 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 * * @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) { diff --git a/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java b/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java index aed132b745e..79a8ebdb19d 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java +++ b/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java @@ -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 return rowSignature; } + @Override + public RowSignature getResultRowSignature(Finalization finalization) + { + return getRowSignature(); + } + @Override public boolean hasFilters() { diff --git a/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java b/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java index f5c600b96ff..23f372c4390 100644 --- a/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java +++ b/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java @@ -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(); + } } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index 037148f50c3..be3d4fa2d5e 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -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 } } + @Override + public RowSignature getResultRowSignature(Finalization finalization) + { + return getRowSignature(); + } /** * Returns the RowSignature. diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java index 4d98ccd9b21..b5e135968a8 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java @@ -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> 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(); } diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 4e06da817ce..acaf1427ce3 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -445,7 +445,7 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest cursorAndCloseable = IterableRowsCursorHelper.getCursorFromSequence( resultsAsArrays(query, resultSequence), rowSignature diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java index 3178ac15a18..aee76989db3 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java @@ -185,7 +185,8 @@ public class TopNQuery extends BaseQuery> topNMetricSpec.initTopNAlgorithmSelector(selector); } - public RowSignature getResultSignature(final RowSignature.Finalization finalization) + @Override + public RowSignature getResultRowSignature(final RowSignature.Finalization finalization) { return RowSignature.builder() .addTimeColumn() diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java index 501684dce40..5500230f04c 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java @@ -517,7 +517,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest cursorAndCloseable = IterableRowsCursorHelper.getCursorFromSequence( resultsAsArrays(query, resultSequence), diff --git a/processing/src/main/java/org/apache/druid/query/union/UnionQuery.java b/processing/src/main/java/org/apache/druid/query/union/UnionQuery.java new file mode 100644 index 00000000000..a999a23fde2 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/union/UnionQuery.java @@ -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 +{ + @JsonProperty("context") + protected final Map context; + + @JsonProperty("queries") + protected final List> queries; + + public UnionQuery(List> queries) + { + this(queries, queries.get(0).getContext()); + } + + @JsonCreator + public UnionQuery( + @JsonProperty("queries") List> queries, + @JsonProperty("context") Map 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 getDataSources() + { + List 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 getRunner(QuerySegmentWalker walker) + { + throw DruidException.defensive("Use QueryToolChest to get a Runner"); + } + + @Override + public List 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 getContext() + { + return context; + } + + @Override + public Ordering getResultOrdering() + { + throw methodNotSupported(); + } + + @Override + public Query withOverriddenContext(Map contextOverrides) + { + List> newQueries = mapQueries(q -> q.withOverriddenContext(contextOverrides)); + return new UnionQuery(newQueries, QueryContexts.override(getContext(), contextOverrides)); + } + + @Override + public Query withQuerySegmentSpec(QuerySegmentSpec spec) + { + throw methodNotSupported(); + } + + @Override + public Query withId(String id) + { + return withOverriddenContext(ImmutableMap.of(BaseQuery.QUERY_ID, id)); + } + + @Override + public String getId() + { + return context().getString(BaseQuery.QUERY_ID); + } + + @Override + public Query 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 withDataSource(DataSource dataSource) + { + throw new RuntimeException("This method is not supported. Use withDataSources instead!"); + } + + public Query withDataSources(List children) + { + Preconditions.checkArgument(queries.size() == children.size(), "Number of children must match number of queries"); + List> 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> mapQueries(Function, Query> mapFn) + { + List> 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 getTableNames() + { + return delegate.getTableNames(); + } + + @Override + public List getChildren() + { + return delegate.getChildren(); + } + + @Override + public DataSource withChildren(List 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 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); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/union/UnionQueryLogic.java b/processing/src/main/java/org/apache/druid/query/union/UnionQueryLogic.java new file mode 100644 index 00000000000..42f031487a2 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/union/UnionQueryLogic.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.query.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 QueryRunner entryPoint(Query query, QuerySegmentWalker walker) + { + return new UnionQueryRunner((UnionQuery) query, conglomerate, walker); + } + + static class UnionQueryRunner implements QueryRunner + { + private final QueryRunnerFactoryConglomerate conglomerate; + private final QuerySegmentWalker walker; + private final List runners; + + public UnionQueryRunner( + UnionQuery query, + QueryRunnerFactoryConglomerate conglomerate, + QuerySegmentWalker walker) + { + this.conglomerate = conglomerate; + this.walker = walker; + this.runners = makeSubQueryRunners(query); + } + + private List makeSubQueryRunners(UnionQuery unionQuery) + { + List 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 run(QueryPlus queryPlus, ResponseContext responseContext) + { + UnionQuery unionQuery = queryPlus.unwrapQuery(UnionQuery.class); + + List> 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); + } + } +} diff --git a/processing/src/test/java/org/apache/druid/query/QueryToolChestTestHelper.java b/processing/src/test/java/org/apache/druid/query/QueryToolChestTestHelper.java index a3bedddc563..3c0d4624223 100644 --- a/processing/src/test/java/org/apache/druid/query/QueryToolChestTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/QueryToolChestTestHelper.java @@ -26,7 +26,7 @@ import java.util.List; public class QueryToolChestTestHelper { - public static void assertArrayResultsEquals(final List expected, final Sequence actual) + public static void assertArrayResultsEquals(List expected, final Sequence actual) { final List actualList = actual.toList(); Assert.assertEquals("number of results", expected.size(), actualList.size()); diff --git a/processing/src/test/java/org/apache/druid/query/UnionQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/UnionDataSourceQueryRunnerTest.java similarity index 96% rename from processing/src/test/java/org/apache/druid/query/UnionQueryRunnerTest.java rename to processing/src/test/java/org/apache/druid/query/UnionDataSourceQueryRunnerTest.java index a9ce7a9471b..e88adaa5bc8 100644 --- a/processing/src/test/java/org/apache/druid/query/UnionQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/UnionDataSourceQueryRunnerTest.java @@ -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( diff --git a/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java b/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java index 1240115221d..fc85b4e3fb6 100644 --- a/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java +++ b/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java @@ -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")); diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java index 76e7f6bdd3e..2e2158f866a 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java @@ -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() diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java index 4cca90c083d..a8644e558e8 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java @@ -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>() { @Override diff --git a/processing/src/test/java/org/apache/druid/query/union/UnionQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/union/UnionQueryQueryToolChestTest.java new file mode 100644 index 00000000000..fbb8a9d358d --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/union/UnionQueryQueryToolChestTest.java @@ -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., 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> 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 results = new ArrayList(); + + 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 makeResultSequence() + { + ScanResultValue result = new ScanResultValue( + QueryRunnerTestHelper.SEGMENT_ID.toString(), + query.getColumns(), + convertResultsToListOfLists() + ); + return Sequences.of(result); + } + + private List> convertResultsToListOfLists() + { + List> resultsRows = new ArrayList>(); + 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 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 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")); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/CursorFactoryProjectionTest.java b/processing/src/test/java/org/apache/druid/segment/CursorFactoryProjectionTest.java index 7eae70cc1f4..c065d3c6aab 100644 --- a/processing/src/test/java/org/apache/druid/segment/CursorFactoryProjectionTest.java +++ b/processing/src/test/java/org/apache/druid/segment/CursorFactoryProjectionTest.java @@ -1110,7 +1110,7 @@ public class CursorFactoryProjectionTest extends InitializedNullHandlingTest final List> 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> 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> 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)); diff --git a/quidem-ut/src/main/java/org/apache/druid/quidem/ExposedAsBrokerQueryComponentSupplierWrapper.java b/quidem-ut/src/main/java/org/apache/druid/quidem/ExposedAsBrokerQueryComponentSupplierWrapper.java index 74149cefcb2..d1fa6a349ba 100644 --- a/quidem-ut/src/main/java/org/apache/druid/quidem/ExposedAsBrokerQueryComponentSupplierWrapper.java +++ b/quidem-ut/src/main/java/org/apache/druid/quidem/ExposedAsBrokerQueryComponentSupplierWrapper.java @@ -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 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(); - } } diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index e4027bcd357..fe64ba7ed42 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -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(); diff --git a/server/src/main/java/org/apache/druid/client/DirectDruidClient.java b/server/src/main/java/org/apache/druid/client/DirectDruidClient.java index 6a502110058..0ae29c2e741 100644 --- a/server/src/main/java/org/apache/druid/client/DirectDruidClient.java +++ b/server/src/main/java/org/apache/druid/client/DirectDruidClient.java @@ -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 implements QueryRunner 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 implements QueryRunner } public DirectDruidClient( - QueryToolChestWarehouse warehouse, + QueryRunnerFactoryConglomerate conglomerate, QueryWatcher queryWatcher, ObjectMapper objectMapper, HttpClient httpClient, @@ -132,7 +132,7 @@ public class DirectDruidClient implements QueryRunner ScheduledExecutorService queryCancellationExecutor ) { - this.warehouse = warehouse; + this.conglomerate = conglomerate; this.queryWatcher = queryWatcher; this.objectMapper = objectMapper; this.httpClient = httpClient; @@ -154,7 +154,7 @@ public class DirectDruidClient implements QueryRunner public Sequence run(final QueryPlus queryPlus, final ResponseContext context) { final Query query = queryPlus.getQuery(); - QueryToolChest> toolChest = warehouse.getToolChest(query); + QueryToolChest> toolChest = conglomerate.getToolChest(query); boolean isBySegment = query.context().isBySegment(); final JavaType queryResultType = isBySegment ? toolChest.getBySegmentResultType() : toolChest.getBaseResultType(); diff --git a/server/src/main/java/org/apache/druid/client/DirectDruidClientFactory.java b/server/src/main/java/org/apache/druid/client/DirectDruidClientFactory.java index 4f29130537b..0761a432d52 100644 --- a/server/src/main/java/org/apache/druid/client/DirectDruidClientFactory.java +++ b/server/src/main/java/org/apache/druid/client/DirectDruidClientFactory.java @@ -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, diff --git a/processing/src/main/java/org/apache/druid/query/ReflectionQueryToolChestWarehouse.java b/server/src/main/java/org/apache/druid/guice/ConglomerateBackedToolChestWarehouse.java similarity index 50% rename from processing/src/main/java/org/apache/druid/query/ReflectionQueryToolChestWarehouse.java rename to server/src/main/java/org/apache/druid/guice/ConglomerateBackedToolChestWarehouse.java index e2b047c1b2d..43db0cf97fc 100644 --- a/processing/src/main/java/org/apache/druid/query/ReflectionQueryToolChestWarehouse.java +++ b/server/src/main/java/org/apache/druid/guice/ConglomerateBackedToolChestWarehouse.java @@ -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> toolChests = new ClassValue>() + @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 > QueryToolChest getToolChest(QueryType query) { - return (QueryToolChest) toolChests.get(query.getClass()); + return conglomerate.getToolChest(query); } } diff --git a/server/src/main/java/org/apache/druid/guice/DruidBinders.java b/server/src/main/java/org/apache/druid/guice/DruidBinders.java index 88f5e65fc0a..63f2061ce13 100644 --- a/server/src/main/java/org/apache/druid/guice/DruidBinders.java +++ b/server/src/main/java/org/apache/druid/guice/DruidBinders.java @@ -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, QueryLogic> queryLogicBinderType(Binder binder) + { + return MapBinder.newMapBinder( + binder, + new TypeLiteral>() {}, + new TypeLiteral() {} + ); + } + + public static QueryLogicBinder queryLogicBinder(Binder binder) + { + return new QueryLogicBinder(binder); + } + + public static class QueryLogicBinder + { + private MapBinder, QueryLogic> queryLogicMapBinder; + private Binder binder; + + public QueryLogicBinder(Binder binder) + { + this.binder = binder; + queryLogicMapBinder = DruidBinders.queryLogicBinderType(binder); + } + + QueryLogicBinder bindQueryLogic( + Class queryTypeClazz, + Class queryLogicClazz) + { + queryLogicMapBinder.addBinding(queryTypeClazz).to(queryLogicClazz); + binder.bind(queryLogicClazz).in(LazySingleton.class); + return this; + } + } + + public static Multibinder> discoveryAnnouncementBinder(Binder binder) { return Multibinder.newSetBinder(binder, new TypeLiteral>() {}); diff --git a/server/src/main/java/org/apache/druid/guice/QueryRunnerFactoryModule.java b/server/src/main/java/org/apache/druid/guice/QueryRunnerFactoryModule.java index 5e71a0d1f08..cc4eb979023 100644 --- a/server/src/main/java/org/apache/druid/guice/QueryRunnerFactoryModule.java +++ b/server/src/main/java/org/apache/druid/guice/QueryRunnerFactoryModule.java @@ -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 diff --git a/server/src/main/java/org/apache/druid/guice/QueryToolChestModule.java b/server/src/main/java/org/apache/druid/guice/QueryToolChestModule.java index da9ab41e299..c8466f887d7 100644 --- a/server/src/main/java/org/apache/druid/guice/QueryToolChestModule.java +++ b/server/src/main/java/org/apache/druid/guice/QueryToolChestModule.java @@ -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); diff --git a/server/src/main/java/org/apache/druid/messages/client/MessageRelayClientImpl.java b/server/src/main/java/org/apache/druid/messages/client/MessageRelayClientImpl.java index 140bd45e1af..b238cdee0aa 100644 --- a/server/src/main/java/org/apache/druid/messages/client/MessageRelayClientImpl.java +++ b/server/src/main/java/org/apache/druid/messages/client/MessageRelayClientImpl.java @@ -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 implements MessageRelayClient 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); diff --git a/server/src/main/java/org/apache/druid/segment/metadata/SegmentMetadataQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/metadata/SegmentMetadataQuerySegmentWalker.java index 00a015eaea8..68f264cebb4 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/SegmentMetadataQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/SegmentMetadataQuerySegmentWalker.java @@ -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 QueryRunner decorateRunner(Query query, QueryRunner baseClusterRunner) { - final QueryToolChest> toolChest = warehouse.getToolChest(query); + final QueryToolChest> toolChest = conglomerate.getToolChest(query); final SetAndVerifyContextQueryRunner baseRunner = new SetAndVerifyContextQueryRunner<>( serverConfig, @@ -141,7 +141,7 @@ public class SegmentMetadataQuerySegmentWalker implements QuerySegmentWalker final TimelineLookup timelineLookup = timelineConverter.apply(timeline); - QueryToolChest> toolChest = warehouse.getToolChest(query); + QueryToolChest> toolChest = conglomerate.getToolChest(query); Set> segmentAndServers = computeSegmentsToQuery(timelineLookup, query, toolChest); queryPlus = queryPlus.withQueryMetrics(toolChest); diff --git a/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java index f4b11f4c8a0..fbe4ad71725 100644 --- a/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java @@ -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 QueryRunner getQueryRunnerForIntervals(final Query query, final Iterable intervals) + public QueryRunner getQueryRunnerForIntervals(Query query, final Iterable intervals) { - final QueryToolChest> 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) queryExecutor.entryPoint(query, this); + } + + final QueryToolChest> 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 boolean canRunQueryUsingLocalWalker(Query query) { - final DataSource dataSourceFromQuery = query.getDataSource(); - final DataSourceAnalysis analysis = dataSourceFromQuery.getAnalysis(); - final QueryToolChest> toolChest = warehouse.getToolChest(query); + final DataSourceAnalysis analysis = query.getDataSourceAnalysis(); + final QueryToolChest> 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 boolean canRunQueryUsingClusterWalker(Query query) { - final DataSource dataSourceFromQuery = query.getDataSource(); - final DataSourceAnalysis analysis = dataSourceFromQuery.getAnalysis(); - final QueryToolChest> toolChest = warehouse.getToolChest(query); + final QueryToolChest> 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 newDataSources = new ArrayList(); + 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 QueryRunner decorateClusterRunner(Query query, QueryRunner baseClusterRunner) { - final QueryToolChest> toolChest = warehouse.getToolChest(query); + final QueryToolChest> toolChest = conglomerate.getToolChest(query); final SetAndVerifyContextQueryRunner 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 nestingInfo = queryDataSourceToSubqueryIds.get(queryDataSource); String subQueryId = nestingInfo.lhs + "." + nestingInfo.rhs; diff --git a/server/src/main/java/org/apache/druid/server/QueryLifecycle.java b/server/src/main/java/org/apache/druid/server/QueryLifecycle.java index a91959ca20b..c4ead8bedce 100644 --- a/server/src/main/java/org/apache/druid/server/QueryLifecycle.java +++ b/server/src/main/java/org/apache/druid/server/QueryLifecycle.java @@ -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 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 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); } /** diff --git a/server/src/main/java/org/apache/druid/server/QueryLifecycleFactory.java b/server/src/main/java/org/apache/druid/server/QueryLifecycleFactory.java index 766307cef5c..1dc32348bc2 100644 --- a/server/src/main/java/org/apache/druid/server/QueryLifecycleFactory.java +++ b/server/src/main/java/org/apache/druid/server/QueryLifecycleFactory.java @@ -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 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, diff --git a/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java b/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java index 4f0b99d125e..8fdef26a410 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java +++ b/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java @@ -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) diff --git a/server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java b/server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java index fd90ff905a2..33b10a8b391 100644 --- a/server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java +++ b/server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java @@ -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) 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) 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) 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) 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) diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java index 89cb2d76f81..52a829267b3 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java @@ -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 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 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 diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientPerfTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientPerfTest.java index b31799cf8b4..c49ee533d03 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientPerfTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientPerfTest.java @@ -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 > QueryToolChest getToolChest(QueryType query) { return new ServerManagerTest.NoopQueryToolChest<>(); } + + @Override + public > QueryRunnerFactory findFactory(QueryType query) + { + return null; + } + + @Override + public > QueryLogic getQueryLogic(QueryType query) + { + return null; + } } private static class MockQueryRunner implements QueryRunner diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index d7464e25a6c..e6d67d7d91d 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -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 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 diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTestUtils.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTestUtils.java index 90e9bc209a1..7489d9a40fa 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTestUtils.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTestUtils.java @@ -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 createWarehouse() - { - final Closer resourceCloser = Closer.create(); - final GroupByQueryRunnerFactory groupByQueryRunnerFactory = GroupByQueryRunnerTest.makeQueryRunnerFactory( - new GroupByQueryConfig(), - resourceCloser.register(TestGroupByBuffers.createDefault()) - ); - return Pair.of( - new MapQueryToolChestWarehouse( - ImmutableMap., 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() - { - } } diff --git a/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java b/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java index 4fffcd6fd35..aa3886ab29d 100644 --- a/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java +++ b/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java @@ -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, diff --git a/server/src/test/java/org/apache/druid/client/SimpleServerView.java b/server/src/test/java/org/apache/druid/client/SimpleServerView.java index c655cf17429..2b50ccbeb11 100644 --- a/server/src/test/java/org/apache/druid/client/SimpleServerView.java +++ b/server/src/test/java/org/apache/druid/client/SimpleServerView.java @@ -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 QueryRunner getQueryRunner(DruidServer server) { final QueryableDruidServer queryableDruidServer = Preconditions.checkNotNull(servers.get(server), "server"); - return (QueryRunner) queryableDruidServer.getQueryRunner(); + return queryableDruidServer.getQueryRunner(); } @Override diff --git a/server/src/test/java/org/apache/druid/query/QueryRunnerBasedOnClusteredClientTestBase.java b/server/src/test/java/org/apache/druid/query/QueryRunnerBasedOnClusteredClientTestBase.java index 308427d40fa..76a63442e3e 100644 --- a/server/src/test/java/org/apache/druid/query/QueryRunnerBasedOnClusteredClientTestBase.java +++ b/server/src/test/java/org/apache/druid/query/QueryRunnerBasedOnClusteredClientTestBase.java @@ -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 > QueryToolChest 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, diff --git a/server/src/test/java/org/apache/druid/query/ResultLevelCachingQueryRunnerTest.java b/server/src/test/java/org/apache/druid/query/ResultLevelCachingQueryRunnerTest.java index 412c97ba2e0..6245509465c 100644 --- a/server/src/test/java/org/apache/druid/query/ResultLevelCachingQueryRunnerTest.java +++ b/server/src/test/java/org/apache/druid/query/ResultLevelCachingQueryRunnerTest.java @@ -241,7 +241,7 @@ public class ResultLevelCachingQueryRunnerTest extends QueryRunnerBasedOnCluster new RetryQueryRunnerConfig(), objectMapper ), - toolChestWarehouse.getToolChest(query), + conglomerate.getToolChest(query), query, objectMapper, cache, diff --git a/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentDataCacheConcurrencyTest.java b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentDataCacheConcurrencyTest.java index 4cc4ac38184..74a201eae27 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentDataCacheConcurrencyTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentDataCacheConcurrencyTest.java @@ -130,7 +130,6 @@ public class CoordinatorSegmentDataCacheConcurrencyTest extends SegmentMetadataC return 0L; } }, - queryToolChestWarehouse, new ServerConfig(), new NoopServiceEmitter(), conglomerate, diff --git a/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTestBase.java b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTestBase.java index 12546df4080..aab0594c64c 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTestBase.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTestBase.java @@ -81,7 +81,6 @@ public class CoordinatorSegmentMetadataCacheTestBase extends SegmentMetadataCach return 0L; } }, - queryToolChestWarehouse, new ServerConfig(), new NoopServiceEmitter(), conglomerate, diff --git a/server/src/test/java/org/apache/druid/segment/metadata/SegmentMetadataCacheTestBase.java b/server/src/test/java/org/apache/druid/segment/metadata/SegmentMetadataCacheTestBase.java index c62577e0125..60195a7a3ef 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/SegmentMetadataCacheTestBase.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/SegmentMetadataCacheTestBase.java @@ -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 > QueryToolChest 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(), diff --git a/server/src/test/java/org/apache/druid/segment/metadata/SegmentMetadataQuerySegmentWalkerTest.java b/server/src/test/java/org/apache/druid/segment/metadata/SegmentMetadataQuerySegmentWalkerTest.java index 7392726b577..99ec5a4068f 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/SegmentMetadataQuerySegmentWalkerTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/SegmentMetadataQuerySegmentWalkerTest.java @@ -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., 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() ); diff --git a/server/src/test/java/org/apache/druid/segment/metadata/TestSegmentMetadataQueryWalker.java b/server/src/test/java/org/apache/druid/segment/metadata/TestSegmentMetadataQueryWalker.java index 95f0d367c8e..6290e7db35f 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/TestSegmentMetadataQueryWalker.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/TestSegmentMetadataQueryWalker.java @@ -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> 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; } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java index 29d758aaed0..c305ada342a 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java @@ -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), diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java index 21f627baa08..66ac774e1e9 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java @@ -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; diff --git a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java index fa5585c374e..7a8e2248367 100644 --- a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java +++ b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java @@ -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. */ diff --git a/server/src/test/java/org/apache/druid/server/QueryLifecycleTest.java b/server/src/test/java/org/apache/druid/server/QueryLifecycleTest.java index a2691297d0b..8bc436ed405 100644 --- a/server/src/test/java/org/apache/druid/server/QueryLifecycleTest.java +++ b/server/src/test/java/org/apache/druid/server/QueryLifecycleTest.java @@ -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, diff --git a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java index 4d827a008f3..bf2c1933d08 100644 --- a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java @@ -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(), diff --git a/server/src/test/java/org/apache/druid/server/QueryStackTests.java b/server/src/test/java/org/apache/druid/server/QueryStackTests.java index 041c4654d92..3801c10f05d 100644 --- a/server/src/test/java/org/apache/druid/server/QueryStackTests.java +++ b/server/src/test/java/org/apache/druid/server/QueryStackTests.java @@ -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 > QueryToolChest 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 minTopNThresholdSupplier - ) - { - return createQueryRunnerFactoryConglomerate(closer, minTopNThresholdSupplier, TestHelper.makeJsonMapper()); - } - - public static QueryRunnerFactoryConglomerate createQueryRunnerFactoryConglomerate( - final Closer closer, - final Supplier 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 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 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., 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, 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, 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., 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 ) diff --git a/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java index 69a7c08f2be..2e20b907184 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java @@ -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 + ., QueryRunnerFactory>builder() + .put(SearchQuery.class, factory) + .build()); serverManager = new ServerManager( - new QueryRunnerFactoryConglomerate() - { - @Override - public > QueryRunnerFactory 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), diff --git a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java index ba7bd43ef8d..b819bb0ded7 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -451,6 +451,7 @@ public class CliCoordinator extends ServerRunnable final MapBinder, 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); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidUnion.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidUnion.java index 96981a751e6..ec50e3b5025 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidUnion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/logical/DruidUnion.java @@ -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 sources) { - List 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 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> 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 sources) + { + for (SourceDesc sourceDesc : sources) { + DataSource dataSource = sourceDesc.dataSource; + if (dataSource instanceof QueryDataSource) { + continue; + } + return false; + } + return true; + } + + private boolean mayUseUnionDataSource(List 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) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidUnionRule.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidUnionRule.java index 2e281c90990..654765b3dcd 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidUnionRule.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/logical/DruidUnionRule.java @@ -30,7 +30,6 @@ import org.checkerframework.checker.nullness.qual.Nullable; public class DruidUnionRule extends ConverterRule { - public DruidUnionRule(Class clazz, RelTrait in, RelTrait out, String descriptionPrefix) { super( diff --git a/sql/src/test/java/org/apache/druid/quidem/DruidAvaticaTestDriver.java b/sql/src/test/java/org/apache/druid/quidem/DruidAvaticaTestDriver.java index debd835fc78..66074055abc 100644 --- a/sql/src/test/java/org/apache/druid/quidem/DruidAvaticaTestDriver.java +++ b/sql/src/test/java/org/apache/druid/quidem/DruidAvaticaTestDriver.java @@ -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(); } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index 9c74d4bbffc..95d3d522893 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -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 Query recursivelyClearContext(final Query query, ObjectMapper queryJsonMapper) { try { - Query newQuery = query.withDataSource(recursivelyClearContext(query.getDataSource(), queryJsonMapper)); + Query newQuery; + if (query instanceof UnionQuery) { + UnionQuery unionQuery = (UnionQuery) query; + newQuery = (Query) 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 recursivelyClearDatasource(final List dataSources, + ObjectMapper queryJsonMapper) + { + List ret = new ArrayList(); + for (DataSource dataSource : dataSources) { + ret.add(recursivelyClearContext(dataSource, queryJsonMapper)); + } + return ret; + } + /** * Override the contexts of all subqueries of a particular datasource. */ diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java index f4816621134..f2ef6941fc3 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java @@ -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() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java index fff819e68ee..68db3805275 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java @@ -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 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 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() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index 26ee0685a13..791e5bf4bb7 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java @@ -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() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledExtension.java b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledExtension.java index 3e89d7a1782..c4ba1f77a67 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledExtension.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledExtension.java @@ -55,16 +55,10 @@ public class DecoupledExtension implements BeforeEachCallback } private static final ImmutableMap CONTEXT_OVERRIDES = ImmutableMap.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); } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledTestConfig.java b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledTestConfig.java index 10d631b665c..ba46dea8008 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledTestConfig.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledTestConfig.java @@ -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() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java b/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java index 8dad971439c..f456bae91f7 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java @@ -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\\."), SORT_REMOVE_CONSTANT_KEYS_CONFLICT(DruidException.class, "not enough rules"), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfig.java b/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfig.java index d13729b61f7..56ac2445d63 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfig.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/SqlTestFrameworkConfig.java @@ -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 PROCESSOR = new ConfigOptionProcessor(NumMergeBuffers.class) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryFrameworkUtils.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryFrameworkUtils.java index 5463f562319..974dfbfd0c7 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryFrameworkUtils.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/QueryFrameworkUtils.java @@ -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 > QueryToolChest getToolChest(final QueryType query) - { - return conglomerate.findFactory(query).getToolchest(); - } - }, + conglomerate, walker, new DefaultGenericQueryMetricsFactory(), new ServiceEmitter("dummy", "dummy", new NoopEmitter()), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java index d59078deff9..a812f9ffa87 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java @@ -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 QueryRunnerFactory> makeRunnerFactories(Injector injector); + + Map, ? 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 QueryRunnerFactory> makeRunnerFactories(Injector injector) + { + return delegate.makeRunnerFactories(injector); + } + + @Override + public Map, ? 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 QueryRunnerFactory> makeRunnerFactories(Injector injector) + { + return Collections.emptyMap(); + } + + @Override + public Map, ? extends QueryToolChest> makeToolChests(Injector injector) + { + return ImmutableMap., 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, 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., 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, QueryToolChest> makeToolchests( + @Named(SQL_TEST_FRAME_WORK) Map, QueryRunnerFactory> factories) + { + return ImmutableMap., QueryToolChest>builder() + .putAll(Maps.transformValues(factories, f -> f.getToolchest())) + .putAll(componentSupplier.makeToolChests(injector)) + .build(); + } + + @Provides + @LazySingleton + public @Named(SQL_TEST_FRAME_WORK) Map, QueryLogic> makeQueryLogics( + UnionQueryLogic unionQueryLogic) + { + return ImmutableMap., 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, QueryRunnerFactory> factories, + @Named(SQL_TEST_FRAME_WORK) Map, QueryToolChest> toolchests, + @Named(SQL_TEST_FRAME_WORK) Map, QueryLogic> querylogics) + { + QueryRunnerFactoryConglomerate conglomerate = new DefaultQueryRunnerFactoryConglomerate(factories, toolchests, querylogics); + return componentSupplier.wrapConglomerate(conglomerate, resourceCloser); } @Provides diff --git a/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java b/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java index bf253cbb1f8..3801ef2888b 100644 --- a/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java @@ -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); diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteArraysQueryTest/testUnnestExtractionFn@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteArraysQueryTest/testUnnestExtractionFn@NullHandling=default.iq index 2acefa4270f..bf46154190a 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteArraysQueryTest/testUnnestExtractionFn@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteArraysQueryTest/testUnnestExtractionFn@NullHandling=default.iq @@ -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 diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteArraysQueryTest/testUnnestExtractionFn@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteArraysQueryTest/testUnnestExtractionFn@NullHandling=sql.iq index ca41ee34e6a..57ce98e52b5 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteArraysQueryTest/testUnnestExtractionFn@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteArraysQueryTest/testUnnestExtractionFn@NullHandling=sql.iq @@ -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 diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesBothQueriesAreJoin.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesBothQueriesAreJoin.iq new file mode 100644 index 00000000000..65b1fac2fad --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesBothQueriesAreJoin.iq @@ -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 diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@all_disabled.iq new file mode 100644 index 00000000000..890e2d86161 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@all_disabled.iq @@ -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 diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@all_enabled.iq new file mode 100644 index 00000000000..f8fef5db517 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@all_enabled.iq @@ -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 diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@default.iq new file mode 100644 index 00000000000..c215c664567 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@default.iq @@ -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 diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@filter-on-value-column_disabled.iq new file mode 100644 index 00000000000..1e5ff447f87 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@filter-on-value-column_disabled.iq @@ -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 diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@filter-rewrites-disabled.iq new file mode 100644 index 00000000000..f5ee8b38c81 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@filter-rewrites-disabled.iq @@ -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 diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@filter-rewrites.iq new file mode 100644 index 00000000000..66213a432d6 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@filter-rewrites.iq @@ -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 diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@join-to-filter.iq new file mode 100644 index 00000000000..01202d21211 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesLeftQueryIsJoin@join-to-filter.iq @@ -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 diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@all_disabled.iq new file mode 100644 index 00000000000..8000f8cc6df --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@all_disabled.iq @@ -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 diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@all_enabled.iq new file mode 100644 index 00000000000..0093d4e581d --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@all_enabled.iq @@ -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 diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@default.iq new file mode 100644 index 00000000000..791603a22fe --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@default.iq @@ -0,0 +1,130 @@ +# testUnionAllTwoQueriesRightQueryIsJoin@default case-crc:2c47c121 +# 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 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", + "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 diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@filter-on-value-column_disabled.iq new file mode 100644 index 00000000000..d17665f24ac --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@filter-on-value-column_disabled.iq @@ -0,0 +1,139 @@ +# testUnionAllTwoQueriesRightQueryIsJoin@filter-on-value-column_disabled case-crc:7fbc9e03 +# 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 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" : "false", + "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" : "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 diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@filter-rewrites-disabled.iq new file mode 100644 index 00000000000..1199eca6b1e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@filter-rewrites-disabled.iq @@ -0,0 +1,139 @@ +# testUnionAllTwoQueriesRightQueryIsJoin@filter-rewrites-disabled case-crc:67c9a73e +# 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 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" : "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" : "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 diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@filter-rewrites.iq new file mode 100644 index 00000000000..80bba8f87d0 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@filter-rewrites.iq @@ -0,0 +1,139 @@ +# testUnionAllTwoQueriesRightQueryIsJoin@filter-rewrites case-crc:cd6030bb +# 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 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" : "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" : "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 diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@join-to-filter.iq new file mode 100644 index 00000000000..02a589ab202 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUnionAllTwoQueriesRightQueryIsJoin@join-to-filter.iq @@ -0,0 +1,139 @@ +# testUnionAllTwoQueriesRightQueryIsJoin@join-to-filter case-crc:6abd3ba0 +# 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 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" : "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" : "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 diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testUnionAllQueries.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testUnionAllQueries.iq new file mode 100644 index 00000000000..e4c98567392 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testUnionAllQueries.iq @@ -0,0 +1,144 @@ +# testUnionAllQueries case-crc:c3323b00 +# 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 UNION ALL SELECT SUM(cnt) FROM foo UNION ALL SELECT COUNT(*) FROM foo; ++--------+ +| EXPR$0 | ++--------+ +| 6 | +| 6 | +| 6 | ++--------+ +(3 rows) + +!ok +LogicalUnion(all=[true]) + LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[SUM($4)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidUnion(all=[true]) + DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[SUM($4)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidTableScan(table=[[druid, foo]], 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" : "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 + } + }, { + "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" : "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 diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testUnionAllQueriesWithLimit.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testUnionAllQueriesWithLimit.iq new file mode 100644 index 00000000000..49fa808fe73 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testUnionAllQueriesWithLimit.iq @@ -0,0 +1,146 @@ +# testUnionAllQueriesWithLimit case-crc:30654774 +# 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 * FROM (SELECT COUNT(*) FROM foo UNION ALL SELECT SUM(cnt) FROM foo UNION ALL SELECT COUNT(*) FROM foo) LIMIT 2; ++--------+ +| EXPR$0 | ++--------+ +| 6 | +| 6 | ++--------+ +(2 rows) + +!ok +LogicalSort(fetch=[2]) + LogicalUnion(all=[true]) + LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[SUM($4)]) + LogicalTableScan(table=[[druid, foo]]) + LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidSort(fetch=[2], druid=[logical]) + DruidUnion(all=[true]) + DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[SUM($4)], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidTableScan(table=[[druid, foo]], 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" : "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 + } + }, { + "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" : "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", + "limit" : 2, + "columns" : [ "a0" ], + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false +} +!nativePlan