From b9bbde5c0a1f7421bfa0dd3498f13885a4264dfe Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Mon, 22 Apr 2024 14:10:44 +0530 Subject: [PATCH] Fix deadlock that can occur while merging group by results (#15420) This PR prevents such a deadlock from happening by acquiring the merge buffers in a single place and passing it down to the runner that might need it. --- .../GroupByTypeInterfaceBenchmark.java | 7 +- .../CachingClusteredClientBenchmark.java | 9 +- .../benchmark/query/GroupByBenchmark.java | 7 +- .../DistinctCountGroupByQueryTest.java | 4 +- .../MaterializedViewQueryQueryToolChest.java | 25 +- ...terializedViewQueryQueryToolChestTest.java | 6 +- .../segment/MapVirtualColumnGroupByTest.java | 10 +- .../ApproximateHistogramGroupByQueryTest.java | 6 +- ...FixedBucketsHistogramGroupByQueryTest.java | 6 +- .../druid/testing/utils/ITRetryUtil.java | 9 +- .../apache/druid/query/FluentQueryRunner.java | 9 +- .../org/apache/druid/query/QueryContext.java | 4 + .../org/apache/druid/query/QueryContexts.java | 4 +- .../apache/druid/query/QueryResourceId.java | 119 ++++++++++ .../apache/druid/query/QueryToolChest.java | 24 ++ .../groupby/GroupByQueryQueryToolChest.java | 51 +++- .../query/groupby/GroupByQueryResources.java | 119 ++++++++-- .../groupby/GroupByQueryRunnerFactory.java | 3 + .../GroupByResourcesReservationPool.java | 141 +++++++++++ .../druid/query/groupby/GroupingEngine.java | 61 +++-- .../GroupByMergingQueryRunner.java | 78 +++---- .../epinephelinae/GroupByQueryEngine.java | 3 + .../epinephelinae/GroupByRowProcessor.java | 2 +- .../druid/query/QueryResourceIdTest.java | 55 +++++ .../druid/query/QueryRunnerTestHelper.java | 4 +- .../aggregation/AggregationTestHelper.java | 10 +- .../query/context/ResponseContextTest.java | 4 +- ...ByLimitPushDownInsufficientBufferTest.java | 83 ++++--- ...roupByLimitPushDownMultiNodeMergeTest.java | 174 +++++++++----- .../groupby/GroupByMultiSegmentTest.java | 17 +- .../groupby/GroupByQueryMergeBufferTest.java | 25 +- .../GroupByQueryQueryToolChestTest.java | 110 ++++----- .../GroupByQueryRunnerFactoryTest.java | 15 +- .../GroupByQueryRunnerFailureTest.java | 14 +- .../query/groupby/GroupByQueryRunnerTest.java | 218 ++++++++++++------ .../groupby/GroupByQueryRunnerTestHelper.java | 22 +- .../GroupByTimeseriesQueryRunnerTest.java | 1 + .../groupby/NestedQueryPushDownTest.java | 33 ++- .../groupby/UnnestGroupByQueryRunnerTest.java | 7 +- .../query/search/SearchQueryRunnerTest.java | 2 +- .../druid/guice/BrokerProcessingModule.java | 13 ++ .../druid/guice/DruidProcessingModule.java | 13 ++ .../druid/guice/RouterProcessingModule.java | 16 ++ .../appenderator/SinkQuerySegmentWalker.java | 28 ++- .../server/ClientQuerySegmentWalker.java | 43 +--- .../druid/server/LocalQuerySegmentWalker.java | 2 +- .../server/QuerySwappingQueryRunner.java | 62 +++++ .../ResourceIdPopulatingQueryRunner.java | 68 ++++++ .../server/coordination/ServerManager.java | 36 +-- .../client/CachingClusteredClientTest.java | 2 +- .../druid/discovery/DataServerClientTest.java | 2 +- .../server/ClientQuerySegmentWalkerTest.java | 61 +++-- .../druid/server/QuerySchedulerTest.java | 7 +- .../apache/druid/server/QueryStackTests.java | 3 +- .../SpecificSegmentsQuerySegmentWalker.java | 14 +- .../server/TestClusterQuerySegmentWalker.java | 33 ++- .../coordination/ServerManagerTest.java | 2 +- .../org/apache/druid/cli/DumpSegment.java | 5 +- .../org/apache/druid/cli/DumpSegmentTest.java | 2 + .../sql/calcite/util/TestDataBuilder.java | 4 +- 60 files changed, 1427 insertions(+), 490 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/query/QueryResourceId.java create mode 100644 processing/src/main/java/org/apache/druid/query/groupby/GroupByResourcesReservationPool.java create mode 100644 processing/src/test/java/org/apache/druid/query/QueryResourceIdTest.java create mode 100644 server/src/main/java/org/apache/druid/server/QuerySwappingQueryRunner.java create mode 100644 server/src/main/java/org/apache/druid/server/ResourceIdPopulatingQueryRunner.java diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java index 0fe549834f07..95d59856395f 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java @@ -54,6 +54,7 @@ import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryQueryToolChest; import org.apache.druid.query.groupby.GroupByQueryRunnerFactory; +import org.apache.druid.query.groupby.GroupByResourcesReservationPool; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; @@ -372,11 +373,13 @@ public String getFormatString() }; final Supplier configSupplier = Suppliers.ofInstance(config); + final GroupByResourcesReservationPool groupByResourcesReservationPool = + new GroupByResourcesReservationPool(mergePool, config); final GroupingEngine groupingEngine = new GroupingEngine( druidProcessingConfig, configSupplier, bufferPool, - mergePool, + groupByResourcesReservationPool, TestHelper.makeJsonMapper(), new ObjectMapper(new SmileFactory()), QueryBenchmarkUtil.NOOP_QUERYWATCHER @@ -384,7 +387,7 @@ public String getFormatString() factory = new GroupByQueryRunnerFactory( groupingEngine, - new GroupByQueryQueryToolChest(groupingEngine) + new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool) ); } 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 5adfcee0de8e..24afa1e84772 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 @@ -80,6 +80,7 @@ import org.apache.druid.query.groupby.GroupByQueryQueryToolChest; import org.apache.druid.query.groupby.GroupByQueryRunnerFactory; import org.apache.druid.query.groupby.GroupByQueryRunnerTest; +import org.apache.druid.query.groupby.GroupByResourcesReservationPool; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.planning.DataSourceAnalysis; @@ -356,16 +357,18 @@ private static GroupByQueryRunnerFactory makeGroupByQueryRunnerFactory( bufferSupplier, processingConfig.getNumMergeBuffers() ); + final GroupByResourcesReservationPool groupByResourcesReservationPool = + new GroupByResourcesReservationPool(mergeBufferPool, config); final GroupingEngine groupingEngine = new GroupingEngine( processingConfig, configSupplier, bufferPool, - mergeBufferPool, + groupByResourcesReservationPool, mapper, mapper, QueryRunnerTestHelper.NOOP_QUERYWATCHER ); - final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(groupingEngine); + final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool); return new GroupByQueryRunnerFactory(groupingEngine, toolChest); } @@ -469,7 +472,7 @@ private List runQuery() toolChestWarehouse.getToolChest(query) ) .applyPreMergeDecoration() - .mergeResults() + .mergeResults(true) .applyPostMergeDecoration(); //noinspection unchecked diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java index efb6bed6cb6f..217946f4ecd9 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java @@ -65,6 +65,7 @@ import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryQueryToolChest; import org.apache.druid.query.groupby.GroupByQueryRunnerFactory; +import org.apache.druid.query.groupby.GroupByResourcesReservationPool; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; @@ -487,11 +488,13 @@ public String getFormatString() }; final Supplier configSupplier = Suppliers.ofInstance(config); + final GroupByResourcesReservationPool groupByResourcesReservationPool = + new GroupByResourcesReservationPool(mergePool, config); final GroupingEngine groupingEngine = new GroupingEngine( druidProcessingConfig, configSupplier, bufferPool, - mergePool, + groupByResourcesReservationPool, TestHelper.makeJsonMapper(), new ObjectMapper(new SmileFactory()), QueryBenchmarkUtil.NOOP_QUERYWATCHER @@ -499,7 +502,7 @@ public String getFormatString() factory = new GroupByQueryRunnerFactory( groupingEngine, - new GroupByQueryQueryToolChest(groupingEngine) + new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool) ); } diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java index eaef749c324e..557b68e89375 100644 --- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java +++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java @@ -137,9 +137,9 @@ public void testGroupByWithDistinctCountAgg() throws Exception Iterable results = FluentQueryRunner .create(factory.createRunner(incrementalIndexSegment), factory.getToolchest()) .applyPreMergeDecoration() - .mergeResults() + .mergeResults(true) .applyPostMergeDecoration() - .run(QueryPlus.wrap(query)) + .run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(query))) .toList(); List expectedResults = Arrays.asList( diff --git a/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChest.java b/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChest.java index d6916e719ef4..3b59e2909907 100644 --- a/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChest.java +++ b/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChest.java @@ -48,16 +48,35 @@ public MaterializedViewQueryQueryToolChest( { this.warehouse = warehouse; } - + @Override public QueryRunner mergeResults(QueryRunner runner) { - return new QueryRunner() { + return new QueryRunner() + { + @Override + public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) + { + Query realQuery = getRealQuery(queryPlus.getQuery()); + return warehouse.getToolChest(realQuery) + .mergeResults(runner) + .run(queryPlus.withQuery(realQuery), responseContext); + } + }; + } + + @Override + public QueryRunner mergeResults(QueryRunner runner, boolean willMergeRunner) + { + return new QueryRunner() + { @Override public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { Query realQuery = getRealQuery(queryPlus.getQuery()); - return warehouse.getToolChest(realQuery).mergeResults(runner).run(queryPlus.withQuery(realQuery), responseContext); + return warehouse.getToolChest(realQuery) + .mergeResults(runner, willMergeRunner) + .run(queryPlus.withQuery(realQuery), responseContext); } }; } diff --git a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChestTest.java b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChestTest.java index 36c43ef475d4..5fe15b63b630 100644 --- a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChestTest.java +++ b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChestTest.java @@ -127,7 +127,7 @@ public void testDecorateObjectMapper() throws IOException QueryToolChest queryToolChest = new MaterializedViewQueryQueryToolChest(new MapQueryToolChestWarehouse( ImmutableMap., QueryToolChest>builder() - .put(GroupByQuery.class, new GroupByQueryQueryToolChest(null)) + .put(GroupByQuery.class, new GroupByQueryQueryToolChest(null, null)) .build() )); @@ -186,7 +186,7 @@ public void testDecorateObjectMapperMaterializedViewQuery() throws IOException QueryToolChest materializedViewQueryQueryToolChest = new MaterializedViewQueryQueryToolChest(new MapQueryToolChestWarehouse( ImmutableMap., QueryToolChest>builder() - .put(GroupByQuery.class, new GroupByQueryQueryToolChest(null)) + .put(GroupByQuery.class, new GroupByQueryQueryToolChest(null, null)) .build() )); @@ -245,7 +245,7 @@ public void testGetRealQuery() MaterializedViewQueryQueryToolChest materializedViewQueryQueryToolChest = new MaterializedViewQueryQueryToolChest(new MapQueryToolChestWarehouse( ImmutableMap., QueryToolChest>builder() - .put(GroupByQuery.class, new GroupByQueryQueryToolChest(null)) + .put(GroupByQuery.class, new GroupByQueryQueryToolChest(null, null)) .build() )); diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java index ebbee1f8a547..8e1bba405ee3 100644 --- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java +++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java @@ -42,6 +42,7 @@ import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryQueryToolChest; import org.apache.druid.query.groupby.GroupByQueryRunnerFactory; +import org.apache.druid.query.groupby.GroupByResourcesReservationPool; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; @@ -67,6 +68,9 @@ public class MapVirtualColumnGroupByTest extends InitializedNullHandlingTest public void setup() throws IOException { final IncrementalIndex incrementalIndex = MapVirtualColumnTestBase.generateIndex(); + final GroupByQueryConfig config = new GroupByQueryConfig(); + final GroupByResourcesReservationPool groupByResourcesReservationPool = + new GroupByResourcesReservationPool(new DefaultBlockingPool<>(() -> ByteBuffer.allocate(1024), 1), config); final GroupingEngine groupingEngine = new GroupingEngine( new DruidProcessingConfig() { @@ -94,9 +98,9 @@ public int getNumThreads() return 1; } }, - GroupByQueryConfig::new, + () -> config, new StupidPool<>("map-virtual-column-groupby-test", () -> ByteBuffer.allocate(1024)), - new DefaultBlockingPool<>(() -> ByteBuffer.allocate(1024), 1), + groupByResourcesReservationPool, TestHelper.makeJsonMapper(), new DefaultObjectMapper(), QueryRunnerTestHelper.NOOP_QUERYWATCHER @@ -104,7 +108,7 @@ public int getNumThreads() final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( groupingEngine, - new GroupByQueryQueryToolChest(groupingEngine) + new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool) ); runner = QueryRunnerTestHelper.makeQueryRunner( diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java index ecb2c85c7fb3..5be792966a97 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java @@ -184,7 +184,8 @@ public void testGroupByWithApproximateHistogramAgg() ) ); - Iterable results = runner.run(QueryPlus.wrap(query)).toList(); + Iterable results = runner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(query))) + .toList(); TestHelper.assertExpectedObjects(expectedResults, results, "approx-histo"); } @@ -231,7 +232,8 @@ public void testGroupByWithSameNameComplexPostAgg() ) ); - Iterable results = runner.run(QueryPlus.wrap(query)).toList(); + Iterable results = runner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(query))) + .toList(); TestHelper.assertExpectedObjects(expectedResults, results, "approx-histo"); } } diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramGroupByQueryTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramGroupByQueryTest.java index b22982a2e7ce..49f91ada0068 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramGroupByQueryTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramGroupByQueryTest.java @@ -186,7 +186,8 @@ public void testGroupByWithFixedHistogramAgg() ) ); - Iterable results = runner.run(QueryPlus.wrap(query)).toList(); + Iterable results = runner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(query))) + .toList(); TestHelper.assertExpectedObjects(expectedResults, results, "fixed-histo"); } @@ -233,7 +234,8 @@ public void testGroupByWithSameNameComplexPostAgg() ) ); - Iterable results = runner.run(QueryPlus.wrap(query)).toList(); + Iterable results = runner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(query))) + .toList(); TestHelper.assertExpectedObjects(expectedResults, results, "fixed-histo"); } } diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/ITRetryUtil.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/ITRetryUtil.java index a306d81b707d..c159b7197c0b 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/utils/ITRetryUtil.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/ITRetryUtil.java @@ -69,7 +69,10 @@ public static void retryUntil( LOG.info( "Attempt[%d/%d] did not pass: Task %s still not complete. Next retry in %d ms", - currentTry, retryCount, taskMessage, delayInMillis + currentTry, + retryCount, + taskMessage, + delayInMillis ); try { Thread.sleep(delayInMillis); @@ -83,10 +86,10 @@ public static void retryUntil( if (currentTry > retryCount) { if (lastException != null) { throw new ISE( + lastException, "Max number of retries[%d] exceeded for Task[%s]. Failing.", retryCount, - taskMessage, - lastException + taskMessage ); } else { throw new ISE( 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 8e01cc95a63f..e8875761ad42 100644 --- a/processing/src/main/java/org/apache/druid/query/FluentQueryRunner.java +++ b/processing/src/main/java/org/apache/druid/query/FluentQueryRunner.java @@ -90,9 +90,14 @@ public FluentQueryRunner postProcess(PostProcessingOperator postProcessing return from(postProcessing != null ? postProcessing.postProcess(baseRunner) : baseRunner); } - public FluentQueryRunner mergeResults() + /** + * Delegates to {@link QueryToolChest#mergeResults(QueryRunner, boolean)}. + * + * @see QueryToolChest#mergeResults(QueryRunner, boolean) + */ + public FluentQueryRunner mergeResults(boolean willMergeRunner) { - return from(toolChest.mergeResults(baseRunner)); + return from(toolChest.mergeResults(baseRunner, willMergeRunner)); } public FluentQueryRunner map(final Function, QueryRunner> mapFn) 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 3364512052bc..5c08678b8884 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContext.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContext.java @@ -591,6 +591,10 @@ public boolean isWindowingStrictValidation() ); } + public QueryResourceId getQueryResourceId() + { + return new QueryResourceId(getString(QueryContexts.QUERY_RESOURCE_ID)); + } public String getBrokerServiceName() { 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 0359b13e2f0c..2ea31d339485 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java @@ -86,7 +86,9 @@ public class QueryContexts public static final String UNCOVERED_INTERVALS_LIMIT_KEY = "uncoveredIntervalsLimit"; public static final String MIN_TOP_N_THRESHOLD = "minTopNThreshold"; public static final String WINDOWING_STRICT_VALIDATION = "windowingStrictValidation"; - + // Unique identifier for the query, that is used to map the global shared resources (specifically merge buffers) to the + // query's runtime + public static final String QUERY_RESOURCE_ID = "queryResourceId"; // SQL query context keys public static final String CTX_SQL_QUERY_ID = BaseQuery.SQL_QUERY_ID; diff --git a/processing/src/main/java/org/apache/druid/query/QueryResourceId.java b/processing/src/main/java/org/apache/druid/query/QueryResourceId.java new file mode 100644 index 000000000000..da988de4b793 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/QueryResourceId.java @@ -0,0 +1,119 @@ +/* + * 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.google.common.base.Preconditions; + +import java.util.Objects; + +/** + * Wrapper class on the queryResourceId string. The object must be addressable on an associative map, therefore it must implement + * equals and hashCode. + *

+ * Query's resource id is used to allocate the resources, and identify the resources allocated to a query in a global pool. + * Queries USUALLY do not share any resources - each query is assigned its own thread, and buffer pool. However, some resources + * are shared globally - the GroupBy query's merge buffers being a prime example of those (and the primary utiliser of the + * query's resource id). Such resources MUST be allocated once to prevent deadlocks, and can be used throughout the query stack, till + * the query holds those resources, or till its completion. A query holding a global resources must not request for more of the same + * resource, or else it becomes a candidate for deadlocks. + *

+ * Each query has a unique resource id, that is assigned to it when it enters the queryable server. This is distinct from + * the existing queryId, subqueryId and sqlQueryId in the following ways: + * 1. It is not assigned by the user, it is assigned internally for usage by the Druid server + * 2. The query's resource id will be unique to the query in the system. The queryId can be non-unique amongst the queries + * that are running in the system. Druid must ensure that the queryResourceId isn't unique. If the user (somehow) + * assigns the queryResourceId to the query, it must be overwritten internally. + * 3. During the query server <-> data server communication, the queryResourceId assigned to a particular query can (and will) + * differ in the query servers and the data servers. This is particularly helpful in case of union queries, where a + * single query in the broker can be treated as two separate queries and executed simultaneously in the historicals. + *

+ * The queryId is assigned to the query, and populated in the query context at the time it hits the queryable server. In Druid, + * there are three queryable servers (classes are not linkable from this method): + * 1. {@link org.apache.druid.server.ClientQuerySegmentWalker} - For brokers + * 2. {@link org.apache.druid.server.coordination.ServerManager} - For historicals + * 3. {@link org.apache.druid.segment.realtime.appenderator.SinkQuerySegmentWalker} - For peons & indexer's tasks + *

+ * These three classes are one of the first places the query reaches when it begins processing, therefore it is + * guaranteed that if the resource id is allotted at only these places, no one will overwrite the resource id + * during the execution. + *

+ * Note: Historicals and Peons could have used the same query id allotted by the brokers, however they assign their own because: + * 1. The user can directly choose to query the data server (while debugging etc.) + * 2. UNIONs are treated as multiple separate queries when the broker sends them to the historicals. Therefore, we + * require a unique id for each part of the union, and hence we need to reassign the resource id to the query's part, + * or else they'll end up sharing the same resource id, as mentioned before + *

+ * Notable places where QueryResourceId is used: + *

+ * 1. {@link org.apache.druid.query.groupby.GroupByResourcesReservationPool} Primary user of the query resource id. + *

+ * 2. {@link org.apache.druid.server.ClientQuerySegmentWalker} Allocates the query resource id on the brokers + *

+ * 3. {@link org.apache.druid.server.coordination.ServerManager} Allocates the query resource id on the historicals + *

+ * 4. {@link org.apache.druid.segment.realtime.appenderator.SinkQuerySegmentWalker} Allocates the query resource id on the peons + * (MMs) and indexers + *

+ * 5. {@link org.apache.druid.server.ResourceIdPopulatingQueryRunner} Populates the query resource id. ({@link org.apache.druid.server.ClientQuerySegmentWalker} + * allocates the query resource id directly, since it also does a bunch of transforms to the query) + *

+ * 6. {@link org.apache.druid.query.groupby.GroupByQueryQueryToolChest} Allocates, and associates one of the global resources, + * merge buffers, with the query's resource id. It also cleans it up, once the query is completed. Apart from that, + * it is also a consumer of the merge buffers it allocates. + *

+ * 7. {@link org.apache.druid.query.groupby.epinephelinae.GroupByMergingQueryRunner} One of the consumer of the merge buffers, + * allocated at the beginning of the query + * + * @see org.apache.druid.query.groupby.GroupByResourcesReservationPool + */ +public class QueryResourceId +{ + private final String queryResourceId; + + public QueryResourceId(String queryResourceId) + { + this.queryResourceId = Preconditions.checkNotNull(queryResourceId, "queryResourceId must be present"); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + QueryResourceId that = (QueryResourceId) o; + return Objects.equals(queryResourceId, that.queryResourceId); + } + + @Override + public int hashCode() + { + return Objects.hash(queryResourceId); + } + + @Override + public String toString() + { + return queryResourceId; + } +} 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 0bf3a5a9e1a1..b0678f247c9d 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java @@ -118,6 +118,30 @@ public QueryRunner mergeResults(QueryRunner runner) return new ResultMergeQueryRunner<>(runner, this::createResultComparator, this::createMergeFn); } + /** + * Like {@link #mergeResults(QueryRunner)}, but with an additional flag that indicates the type of runner that is passeed to the call. + * + * willMergeRunner specifies that the input runner to the mergeResults would be the one created by the corresponding + * {@link QueryRunnerFactory#mergeRunners}. + * While it depends on the input runner, it is usually true since most of the time the same server is generating a runner + * that it wants to merge. The notable deviation from this norm is when the broker is accumulating the results from the + * data servers and needs to merge them together. In this case willMergeRunner is false. + * + * Currently, the sole consumer of this parameter is {@link org.apache.druid.query.groupby.GroupByQueryQueryToolChest}, where + * it is used to determine if the mergeResults is called with {@link org.apache.druid.query.groupby.epinephelinae.GroupByMergingQueryRunner} + * to estimate the number of merge buffers required for the query to succeed. It is set false on the brokers, because they + * (mostly) fetch the results from the historicals, while the data servers set it to false (because they call this method + * with {@link QueryRunnerFactory#mergeRunners}. + * + * By default, the willMergeRunners is ignored, and the {@link #mergeResults(QueryRunner)} is called. For the toolchests + * that override this method must ensure that {@link #mergeResults(QueryRunner)} delegates to it (else it will use the + * default implementation for {@link #mergeResults(QueryRunner)}) which would be undesirable. + */ + public QueryRunner mergeResults(QueryRunner runner, boolean willMergeRunner) + { + return mergeResults(runner); + } + /** * Creates a merge function that is used to merge intermediate aggregates from historicals in broker. This merge * function is used in the default {@link ResultMergeQueryRunner} provided by diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java index 66ba25d6b347..81ec050ce088 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -43,12 +43,14 @@ import org.apache.druid.frame.write.FrameWriterFactory; import org.apache.druid.frame.write.FrameWriterUtils; import org.apache.druid.frame.write.FrameWriters; +import org.apache.druid.guice.annotations.Merging; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.MappedSequence; 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.io.Closer; import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.query.CacheStrategy; import org.apache.druid.query.DataSource; @@ -57,6 +59,7 @@ import org.apache.druid.query.Query; import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.QueryResourceId; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.SubqueryQueryRunner; @@ -86,7 +89,7 @@ import java.util.function.BinaryOperator; /** - * + * Toolchest for GroupBy queries */ public class GroupByQueryQueryToolChest extends QueryToolChest { @@ -102,27 +105,45 @@ public class GroupByQueryQueryToolChest extends QueryToolChest queryConfigSupplier, - GroupByQueryMetricsFactory queryMetricsFactory + GroupByQueryMetricsFactory queryMetricsFactory, + @Merging GroupByResourcesReservationPool groupByResourcesReservationPool ) { this.groupingEngine = groupingEngine; this.queryConfig = queryConfigSupplier.get(); this.queryMetricsFactory = queryMetricsFactory; + this.groupByResourcesReservationPool = groupByResourcesReservationPool; } @Override public QueryRunner mergeResults(final QueryRunner runner) + { + return mergeResults(runner, true); + } + + + @Override + public QueryRunner mergeResults(final QueryRunner runner, boolean willMergeRunner) { return (queryPlus, responseContext) -> { if (queryPlus.getQuery().context().isBySegment()) { @@ -130,7 +151,7 @@ public QueryRunner mergeResults(final QueryRunner runner) } final GroupByQuery groupByQuery = (GroupByQuery) queryPlus.getQuery(); - return initAndMergeGroupByResults(groupByQuery, runner, responseContext); + return initAndMergeGroupByResults(groupByQuery, runner, responseContext, willMergeRunner); }; } @@ -149,10 +170,21 @@ public Comparator createResultComparator(Query query) private Sequence initAndMergeGroupByResults( final GroupByQuery query, QueryRunner runner, - ResponseContext context + ResponseContext context, + boolean willMergeRunner ) { - final GroupByQueryResources resource = groupingEngine.prepareResource(query); + // Reserve the group by resources (merge buffers) required for executing the query + final QueryResourceId queryResourceId = query.context().getQueryResourceId(); + groupByResourcesReservationPool.reserve(queryResourceId, query, willMergeRunner); + + final GroupByQueryResources resource = groupByResourcesReservationPool.fetch(queryResourceId); + if (resource == null) { + throw DruidException.defensive( + "Did not associate any resources with the given query resource id [%s]", + queryResourceId + ); + } try { final Sequence mergedSequence = mergeGroupByResults( query, @@ -160,8 +192,11 @@ private Sequence initAndMergeGroupByResults( runner, context ); + Closer closer = Closer.create(); - return Sequences.withBaggage(mergedSequence, resource); + // Clean up the resources reserved during the execution of the query + closer.register(() -> groupByResourcesReservationPool.clean(queryResourceId)); + return Sequences.withBaggage(mergedSequence, closer); } catch (Exception e) { // Error creating the Sequence; release resources. diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryResources.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryResources.java index 4a2a8bdf8aa1..348e49d91a45 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryResources.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryResources.java @@ -22,12 +22,15 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.druid.collections.ReferenceCountingResourceHolder; import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.collect.Utils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.DataSource; import org.apache.druid.query.Query; import org.apache.druid.query.QueryDataSource; +import org.apache.druid.query.QueryRunner; import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.segment.Segment; import javax.annotation.Nullable; import java.io.Closeable; @@ -40,6 +43,16 @@ /** * This class contains resources required for a groupBy query execution. * Currently, it contains only merge buffers, but any additional resources can be added in the future. + * + * It contains merge buffers for the execution of + * a) {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)} - Required for merging the results of the subqueries + * and the subtotals. + * b) {@link org.apache.druid.query.groupby.epinephelinae.GroupByMergingQueryRunner} - Required for merging the results + * of the individual runners created by {@link GroupByQueryRunnerFactory#createRunner(Segment)} + * + * The resources should be acquired once throughout the execution of the query (with caveats such as union being treated + * as separate queries on the data servers) or it should release all the resources before re-acquiring them (if needed), + * to prevent deadlocks. */ public class GroupByQueryResources implements Closeable { @@ -91,40 +104,86 @@ private static int numMergeBuffersNeededForSubtotalsSpec(GroupByQuery query) return Math.max(1, numMergeBuffersNeededForSubQuerySubtotal); } + /** + * Counts the number of merge buffers required for {@link GroupByQueryQueryToolChest#mergeResults}. For a given query, + * it is dependent on the structure of the group by query. + */ @VisibleForTesting - public static int countRequiredMergeBufferNum(GroupByQuery query) + public static int countRequiredMergeBufferNumForToolchestMerge(GroupByQuery query) { return countRequiredMergeBufferNumWithoutSubtotal(query, 1) + numMergeBuffersNeededForSubtotalsSpec(query); } + /** + * Count the number of merge buffers required for {@link org.apache.druid.query.groupby.epinephelinae.GroupByMergingQueryRunner} + * It can be either 1 or 2, depending on the query's config + */ + public static int countRequiredMergeBufferNumForMergingQueryRunner(GroupByQueryConfig config, GroupByQuery query) + { + GroupByQueryConfig querySpecificConfig = config.withOverrides(query); + return querySpecificConfig.getNumParallelCombineThreads() > 1 ? 2 : 1; + } + @Nullable - private final List> mergeBufferHolders; - private final Deque mergeBuffers; + private final List> toolchestMergeBuffersHolders; + + private final Deque toolchestMergeBuffers = new ArrayDeque<>(); + + @Nullable + private final List> mergingQueryRunnerMergeBuffersHolders; + + private final Deque mergingQueryRunnerMergeBuffers = new ArrayDeque<>(); + + public GroupByQueryResources( + @Nullable List> toolchestMergeBuffersHolders, + @Nullable List> mergingQueryRunnerMergeBuffersHolders + ) + { + this.toolchestMergeBuffersHolders = toolchestMergeBuffersHolders; + if (toolchestMergeBuffersHolders != null) { + toolchestMergeBuffersHolders.forEach(holder -> toolchestMergeBuffers.add(holder.get())); + } + this.mergingQueryRunnerMergeBuffersHolders = mergingQueryRunnerMergeBuffersHolders; + if (mergingQueryRunnerMergeBuffersHolders != null) { + mergingQueryRunnerMergeBuffersHolders.forEach(holder -> mergingQueryRunnerMergeBuffers.add(holder.get())); + } + } - public GroupByQueryResources() + /** + * Returns a merge buffer associate with the {@link GroupByQueryQueryToolChest#mergeResults} + */ + public ResourceHolder getToolchestMergeBuffer() { - this.mergeBufferHolders = null; - this.mergeBuffers = new ArrayDeque<>(); + return getMergeBuffer(toolchestMergeBuffers); } - public GroupByQueryResources(List> mergeBufferHolders) + /** + * Returns a merge buffer associated with the {@link org.apache.druid.query.groupby.epinephelinae.GroupByMergingQueryRunner} + */ + public ResourceHolder getMergingQueryRunnerMergeBuffer() { - this.mergeBufferHolders = mergeBufferHolders; - this.mergeBuffers = new ArrayDeque<>(mergeBufferHolders.size()); - mergeBufferHolders.forEach(holder -> mergeBuffers.add(holder.get())); + return getMergeBuffer(mergingQueryRunnerMergeBuffers); + } + + /** + * Returns the number of the currently unused merge buffers reserved for {@link org.apache.druid.query.groupby.epinephelinae.GroupByMergingQueryRunner} + */ + public int getNumMergingQueryRunnerMergeBuffers() + { + return mergingQueryRunnerMergeBuffers.size(); } /** * Get a merge buffer from the pre-acquired resources. * * @return a resource holder containing a merge buffer - * - * @throws IllegalStateException if this resource is initialized with empty merge buffers, or - * there isn't any available merge buffers */ - public ResourceHolder getMergeBuffer() + private static ResourceHolder getMergeBuffer(Deque acquiredBufferPool) { - final ByteBuffer buffer = mergeBuffers.pop(); + if (acquiredBufferPool.size() == 0) { + throw DruidException.defensive("Insufficient free merge buffers present."); + } + final ByteBuffer buffer = acquiredBufferPool.pop(); return new ResourceHolder() { @Override @@ -136,19 +195,39 @@ public ByteBuffer get() @Override public void close() { - mergeBuffers.add(buffer); + acquiredBufferPool.add(buffer); } }; } + /** + * Closes the query resource. It must be called to release back the acquired merge buffers back into the global + * merging pool from where all the merge buffers are acquired. The references to the merge buffers will become invalid + * once this method is called. The user must ensure that the callers are not using the stale references to the merge + * buffers after this method is called, as reading them would give incorrect results and writing there would interfere + * with other users of the merge buffers + */ @Override public void close() { - if (mergeBufferHolders != null) { - if (mergeBuffers.size() != mergeBufferHolders.size()) { - log.warn("%d resources are not returned yet", mergeBufferHolders.size() - mergeBuffers.size()); + if (toolchestMergeBuffersHolders != null) { + if (toolchestMergeBuffers.size() != toolchestMergeBuffersHolders.size()) { + log.warn( + "%d toolchest merge buffers are not returned yet", + toolchestMergeBuffersHolders.size() - toolchestMergeBuffers.size() + ); + } + toolchestMergeBuffersHolders.forEach(ReferenceCountingResourceHolder::close); + } + + if (mergingQueryRunnerMergeBuffersHolders != null) { + if (mergingQueryRunnerMergeBuffers.size() != mergingQueryRunnerMergeBuffersHolders.size()) { + log.warn( + "%d merging query runner merge buffers are not returned yet", + mergingQueryRunnerMergeBuffersHolders.size() - mergingQueryRunnerMergeBuffers.size() + ); } - mergeBufferHolders.forEach(ReferenceCountingResourceHolder::close); + mergingQueryRunnerMergeBuffersHolders.forEach(ReferenceCountingResourceHolder::close); } } } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java index f68f5a6bad7e..adf43e8cc16c 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java @@ -57,6 +57,9 @@ public QueryRunner createRunner(final Segment segment) return new GroupByQueryRunner(segment, groupingEngine); } + /** + * @see GroupingEngine#mergeRunners(QueryProcessingPool, Iterable) + */ @Override public QueryRunner mergeRunners( final QueryProcessingPool queryProcessingPool, diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByResourcesReservationPool.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByResourcesReservationPool.java new file mode 100644 index 000000000000..49a698b42ae7 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByResourcesReservationPool.java @@ -0,0 +1,141 @@ +/* + * 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.groupby; + +import org.apache.druid.collections.BlockingPool; +import org.apache.druid.error.DruidException; +import org.apache.druid.guice.annotations.Merging; +import org.apache.druid.query.QueryResourceId; + +import javax.annotation.Nullable; +import javax.inject.Inject; +import java.nio.ByteBuffer; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Reserves the {@link GroupByQueryResources} for a given group by query and maps them to the query's resource ID. + * The merge buffers can be required for the group by query in a couple of places: + * 1. {@link GroupByQueryQueryToolChest} + * 2. {@link GroupByResourcesReservationPool} + * However, acquiring them separately can lead to deadlocks when multiple queries are fired. Therefore, instead of + * acquiring them separately, we acquire them once during the query execution, in {@link GroupByQueryQueryToolChest} and + * use those resources till the query is active. + *

+ * ALLOCATION + * The merge buffers are allocated and associated with a given resource id in this pool. Multiple attempts to insert the same resource id will fail, + * therefore we know that there will only be resources allocated only once, as long as the query id doesn't change during the execution of the query. + * The pool is cleaned once close() is called on the reserved resources, and the mapping is removed, thus ensuring that the mapping doesn't keep growing + * during the execution of the queries. + * The call to allocate the merge buffers in the pool is done by mergeResults, and it allocates the resources required for its execution as well as the + * execution of the GroupByMergingQueryRunner if willMergeRunners=true. The GroupByMergingQueryRunner doesn't allocate any resources, it assumes that the resources + * have been preallocated, and just takes them from the pool. + * Once the required merge buffers are allocated from the pool, they cannot be used by the other queries till the close() method is called on the GroupByQueryResource. + * This is usually done with a call to the GroupByResourcesReservationPool#clean() which does this and also cleans up the mapping. + * While the GroupByQueryResource is unclosed, the merge buffers can be taken and given back to it as needed during the execution of the query. As such, the resources are not + * released back to the global pool, and only given back to signify that the work of that execution unit is complete, and it can be reused (or closed safely). Closing the GroupByQueryResources + * when all the merge buffers are not acquired back from the individual execution units log a warning, but doesn't throw. + * The resources get freed up, and if the execution unit was actually using the resources for something, it can error out. + *

+ * ASSUMPTIONS + * There's an attempt to link various places where the merge buffers are acquired ({@link org.apache.druid.query.QueryToolChest#mergeResults}) + * and merge buffers are utilized ({@link org.apache.druid.query.QueryToolChest#mergeResults} and {@link GroupByQueryRunnerFactory.GroupByQueryRunner#mergeRunners}). + * However, Druid's code doesn't provide any explicit contract between the arguments of these methods, and input to {@code mergeResults} can be any runner, + * and it should function the same. While this provides flexibility and reusability to the methods, this also necessitates that there are some assumptions + * that the code makes implicitly, to know what type of runner is passed to mergeResults - so that the mergeResults can allocate + * the merge buffers required for the runner appropriately. + *

+ * 1. For a given query, and a given server, only a single top-level mergeResults call will be made, that will collect the results from the various runners. + * The code will break down if there are multiple, nested mergeResults calls made (unnested calls are fine, though they don't happen) + * 2. There can be multiple mergeRunners, because GroupByMergingQueryRunner only needs the merge buffers for the top-level query runner, + * nested ones execute via an unoptimized way. + * 3. There's some knowledge to the mergeResults that the query runner passed to it is the one created by the corresponding toolchest's + * mergeRunners (which is the typical use case). This is encoded in the argument {@code willMergeRunner}, and is to be set by the callers. + * The only production use case where this isn't true is when the broker is merging the results gathered from the historical) + *

+ * TESTING + * Unit tests mimic the broker-historical interaction in many places, which can lead to the code not working as intended because the assumptions don't hold. + * In many test cases, there are two nested mergeResults calls, the outer call mimics what the broker does, while the inner one mimics what the historical does, + * and the assumption (1) fails. Therefore, the testing code should assign a unique resource id b/w each mergeResults call, and also make sure that the top level mergeResults + * would have willMergeRunner = false, since it's being called on top of a mergeResults's runner, while the inner one would have willMergeRunner = true because its being + * called on actual runners (as it happens in the brokers, and the historicals) + */ +public class GroupByResourcesReservationPool +{ + /** + * Map of query's resource id -> group by resources reserved for the query to execute + */ + final ConcurrentHashMap pool = new ConcurrentHashMap<>(); + + /** + * Buffer pool from where the merge buffers are picked and reserved + */ + final BlockingPool mergeBufferPool; + + /** + * Group by query config of the server + */ + final GroupByQueryConfig groupByQueryConfig; + + @Inject + public GroupByResourcesReservationPool( + @Merging BlockingPool mergeBufferPool, + GroupByQueryConfig groupByQueryConfig + ) + { + this.mergeBufferPool = mergeBufferPool; + this.groupByQueryConfig = groupByQueryConfig; + } + + /** + * Reserves appropriate resources, and maps it to the queryResourceId (usually the query's resource id) in the internal map + */ + public void reserve(QueryResourceId queryResourceId, GroupByQuery groupByQuery, boolean willMergeRunner) + { + if (queryResourceId == null) { + throw DruidException.defensive("Query resource id must be populated"); + } + pool.compute(queryResourceId, (id, existingResource) -> { + if (existingResource != null) { + throw DruidException.defensive("Resource with the given identifier [%s] is already present", id); + } + return GroupingEngine.prepareResource(groupByQuery, mergeBufferPool, willMergeRunner, groupByQueryConfig); + }); + } + + /** + * Fetches resources corresponding to the given resource id + */ + @Nullable + public GroupByQueryResources fetch(QueryResourceId queryResourceId) + { + return pool.get(queryResourceId); + } + + /** + * Removes the entry corresponding to the unique id from the map, and cleans up the resources. + */ + public void clean(QueryResourceId queryResourceId) + { + GroupByQueryResources resources = pool.remove(queryResourceId); + if (resources != null) { + resources.close(); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java index 0c419055c5fe..5fa26d3fa2b4 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java @@ -112,7 +112,7 @@ public class GroupingEngine private final DruidProcessingConfig processingConfig; private final Supplier configSupplier; private final NonBlockingPool bufferPool; - private final BlockingPool mergeBufferPool; + GroupByResourcesReservationPool groupByResourcesReservationPool; private final ObjectMapper jsonMapper; private final ObjectMapper spillMapper; private final QueryWatcher queryWatcher; @@ -122,7 +122,7 @@ public GroupingEngine( DruidProcessingConfig processingConfig, Supplier configSupplier, @Global NonBlockingPool bufferPool, - @Merging BlockingPool mergeBufferPool, + @Merging GroupByResourcesReservationPool groupByResourcesReservationPool, @Json ObjectMapper jsonMapper, @Smile ObjectMapper spillMapper, QueryWatcher queryWatcher @@ -131,25 +131,36 @@ public GroupingEngine( this.processingConfig = processingConfig; this.configSupplier = configSupplier; this.bufferPool = bufferPool; - this.mergeBufferPool = mergeBufferPool; + this.groupByResourcesReservationPool = groupByResourcesReservationPool; this.jsonMapper = jsonMapper; this.spillMapper = spillMapper; this.queryWatcher = queryWatcher; } /** - * Initializes resources required to run {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)} for a - * particular query. That method is also the primary caller of this method. - * - * Used by {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)}. - * - * @param query a groupBy query to be processed - * - * @return broker resource + * Initializes resources required to run {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)} and + * {@link GroupByMergingQueryRunner} for a particular query. The resources are to be acquired once throughout the + * execution of the query, or need to be re-acquired (if needed). Users must ensure that throughout the execution, + * a query already holding the resources shouldn't request for more resources, because that can cause deadlocks. */ - public GroupByQueryResources prepareResource(GroupByQuery query) + public static GroupByQueryResources prepareResource( + GroupByQuery query, + BlockingPool mergeBufferPool, + boolean usesGroupByMergingQueryRunner, + GroupByQueryConfig groupByQueryConfig + ) { - final int requiredMergeBufferNum = GroupByQueryResources.countRequiredMergeBufferNum(query); + + final int requiredMergeBufferNumForToolchestMerge = + GroupByQueryResources.countRequiredMergeBufferNumForToolchestMerge(query); + + final int requiredMergeBufferNumForMergingQueryRunner = + usesGroupByMergingQueryRunner + ? GroupByQueryResources.countRequiredMergeBufferNumForMergingQueryRunner(groupByQueryConfig, query) + : 0; + + final int requiredMergeBufferNum = + requiredMergeBufferNumForToolchestMerge + requiredMergeBufferNumForMergingQueryRunner; if (requiredMergeBufferNum > mergeBufferPool.maxSize()) { throw new ResourceLimitExceededException( @@ -157,7 +168,7 @@ public GroupByQueryResources prepareResource(GroupByQuery query) + mergeBufferPool.maxSize() + " merge buffers were configured" ); } else if (requiredMergeBufferNum == 0) { - return new GroupByQueryResources(); + return new GroupByQueryResources(null, null); } else { final List> mergeBufferHolders; final QueryContext context = query.context(); @@ -174,7 +185,10 @@ public GroupByQueryResources prepareResource(GroupByQuery query) ) ); } else { - return new GroupByQueryResources(mergeBufferHolders); + return new GroupByQueryResources( + mergeBufferHolders.subList(0, requiredMergeBufferNumForToolchestMerge), + mergeBufferHolders.subList(requiredMergeBufferNumForToolchestMerge, requiredMergeBufferNum) + ); } } } @@ -402,12 +416,17 @@ public Sequence mergeResults( } /** - * Merge a variety of single-segment query runners into a combined runner. Used by + * Merges a variety of single-segment query runners into a combined runner. Used by * {@link GroupByQueryRunnerFactory#mergeRunners(QueryProcessingPool, Iterable)}. In * that sense, it is intended to go along with {@link #process(GroupByQuery, StorageAdapter, GroupByQueryMetrics)} (the runners created * by that method will be fed into this method). - *

- * This method is only called on data servers, like Historicals (not the Broker). + * + * This is primarily called on the data servers, to merge the results from processing on the segments. This method can + * also be called on the brokers if the query is operating on the local data sources, like the inline + * datasources. + * + * It uses {@link GroupByMergingQueryRunner} which requires the merge buffers to be passed in the responseContext + * of the query that is run. * * @param queryProcessingPool {@link QueryProcessingPool} service used for parallel execution of the query runners * @param queryRunners collection of query runners to merge @@ -424,8 +443,8 @@ public QueryRunner mergeRunners( queryProcessingPool, queryWatcher, queryRunners, + groupByResourcesReservationPool, processingConfig.getNumThreads(), - mergeBufferPool, processingConfig.intermediateComputeSizeBytes(), spillMapper, processingConfig.getTmpDir() @@ -542,7 +561,7 @@ public Sequence applyPostProcessing(Sequence results, Grou * * @param subquery inner query * @param query outer query - * @param resource resources returned by {@link #prepareResource(GroupByQuery)} + * @param resource resources returned by {@link #prepareResource(GroupByQuery, BlockingPool, boolean, GroupByQueryConfig)} * @param subqueryResult result rows from the subquery * @param wasQueryPushedDown true if the outer query was pushed down (so we only need to merge the outer query's * results, not run it from scratch like a normal outer query) @@ -603,7 +622,7 @@ public Sequence processSubqueryResult( * Called by {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)} when it needs to generate subtotals. * * @param query query that has a "subtotalsSpec" - * @param resource resources returned by {@link #prepareResource(GroupByQuery)} + * @param resource resources returned by {@link #prepareResource(GroupByQuery, BlockingPool, boolean, GroupByQueryConfig)} * @param queryResult result rows from the main query * * @return results for each list of subtotals in the query, concatenated together diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunner.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunner.java index 749cb37c4aef..6e2064d5f1f1 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunner.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunner.java @@ -30,9 +30,10 @@ import com.google.common.collect.Lists; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.collections.BlockingPool; import org.apache.druid.collections.ReferenceCountingResourceHolder; +import org.apache.druid.collections.ResourceHolder; import org.apache.druid.common.guava.GuavaUtils; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; @@ -48,6 +49,7 @@ import org.apache.druid.query.QueryInterruptedException; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryProcessingPool; +import org.apache.druid.query.QueryResourceId; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryTimeoutException; import org.apache.druid.query.QueryWatcher; @@ -55,12 +57,15 @@ import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.groupby.GroupByQueryResources; +import org.apache.druid.query.groupby.GroupByResourcesReservationPool; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.epinephelinae.RowBasedGrouperHelper.RowBasedKey; import java.io.Closeable; import java.io.File; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.List; import java.util.UUID; import java.util.concurrent.CancellationException; @@ -73,8 +78,9 @@ * using a buffer provided by {@code mergeBufferPool} and a parallel executor provided by {@code exec}. Outputs a * fully aggregated stream of {@link ResultRow} objects. Does not apply post-aggregators. * - * The input {@code queryables} are expected to come from a {@link GroupByQueryEngine}. This code runs on data - * servers, like Historicals. + * The input {@code queryables} are expected to come from a {@link GroupByQueryEngine}. This code primarily runs on data + * servers like Historicals and Realtime Tasks. This can also run on Brokers, if the query is operating on local + * data sources, like inlined data, where the broker itself acts like a data server * * This class has some resemblance to {@link GroupByRowProcessor}. See the javadoc of that class for a discussion of * similarities and differences. @@ -90,10 +96,10 @@ public class GroupByMergingQueryRunner implements QueryRunner private final GroupByQueryConfig config; private final DruidProcessingConfig processingConfig; private final Iterable> queryables; + private final GroupByResourcesReservationPool groupByResourcesReservationPool; private final QueryProcessingPool queryProcessingPool; private final QueryWatcher queryWatcher; private final int concurrencyHint; - private final BlockingPool mergeBufferPool; private final ObjectMapper spillMapper; private final String processingTmpDir; private final int mergeBufferSize; @@ -104,8 +110,8 @@ public GroupByMergingQueryRunner( QueryProcessingPool queryProcessingPool, QueryWatcher queryWatcher, Iterable> queryables, + GroupByResourcesReservationPool groupByResourcesReservationPool, int concurrencyHint, - BlockingPool mergeBufferPool, int mergeBufferSize, ObjectMapper spillMapper, String processingTmpDir @@ -116,8 +122,8 @@ public GroupByMergingQueryRunner( this.queryProcessingPool = queryProcessingPool; this.queryWatcher = queryWatcher; this.queryables = Iterables.unmodifiableIterable(Iterables.filter(queryables, Predicates.notNull())); + this.groupByResourcesReservationPool = groupByResourcesReservationPool; this.concurrencyHint = concurrencyHint; - this.mergeBufferPool = mergeBufferPool; this.spillMapper = spillMapper; this.processingTmpDir = processingTmpDir; this.mergeBufferSize = mergeBufferSize; @@ -185,11 +191,8 @@ public CloseableGrouperIterator make() // If parallelCombine is enabled, we need two merge buffers for parallel aggregating and parallel combining final int numMergeBuffers = querySpecificConfig.getNumParallelCombineThreads() > 1 ? 2 : 1; - final List> mergeBufferHolders = getMergeBuffersHolder( - numMergeBuffers, - hasTimeout, - timeoutAt - ); + final List> mergeBufferHolders = + getMergeBuffersHolder(query, numMergeBuffers); resources.registerAll(mergeBufferHolders); final ReferenceCountingResourceHolder mergeBufferHolder = mergeBufferHolders.get(0); @@ -309,41 +312,32 @@ public void cleanup(CloseableGrouperIterator iterFromMak ); } - private List> getMergeBuffersHolder( - int numBuffers, - boolean hasTimeout, - long timeoutAt - ) + private List> getMergeBuffersHolder(GroupByQuery query, int numBuffers) { - try { - if (numBuffers > mergeBufferPool.maxSize()) { - throw new ResourceLimitExceededException( - "Query needs " + numBuffers + " merge buffers, but only " - + mergeBufferPool.maxSize() + " merge buffers were configured. " - + "Try raising druid.processing.numMergeBuffers." - ); - } - final List> mergeBufferHolder; - // This will potentially block if there are no merge buffers left in the pool. - if (hasTimeout) { - final long timeout = timeoutAt - System.currentTimeMillis(); - if (timeout <= 0) { - throw new QueryTimeoutException(); - } - if ((mergeBufferHolder = mergeBufferPool.takeBatch(numBuffers, timeout)).isEmpty()) { - throw new QueryTimeoutException("Cannot acquire enough merge buffers"); - } - } else { - mergeBufferHolder = mergeBufferPool.takeBatch(numBuffers); - } - return mergeBufferHolder; + QueryResourceId queryResourceId = query.context().getQueryResourceId(); + GroupByQueryResources resource = groupByResourcesReservationPool.fetch(queryResourceId); + if (resource == null) { + throw DruidException.defensive( + "Expected merge buffers to be reserved in the reservation pool for the query id [%s] however while executing " + + "the GroupByMergingQueryRunner, however none were provided.", + queryResourceId + ); } - catch (QueryTimeoutException | ResourceLimitExceededException e) { - throw e; + if (numBuffers > resource.getNumMergingQueryRunnerMergeBuffers()) { + // Defensive exception, because we should have acquired the correct number of merge buffers beforehand, or + // thrown an RLE in the caller of the runner + throw DruidException.defensive( + "Query needs [%d] merge buffers for GroupByMergingQueryRunner, however only [%d] were provided.", + numBuffers, + resource.getNumMergingQueryRunnerMergeBuffers() + ); } - catch (Exception e) { - throw new QueryInterruptedException(e); + final List> mergeBufferHolders = new ArrayList<>(); + for (int i = 0; i < numBuffers; ++i) { + ResourceHolder mergeBufferHolder = resource.getMergingQueryRunnerMergeBuffer(); + mergeBufferHolders.add(new ReferenceCountingResourceHolder<>(mergeBufferHolder.get(), mergeBufferHolder)); } + return mergeBufferHolders; } private void waitForFutureCompletion( diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java index 9ffc006799ce..eff19df12189 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java @@ -88,6 +88,9 @@ * Used for non-vectorized processing by * {@link GroupingEngine#process(GroupByQuery, StorageAdapter, GroupByQueryMetrics)}. * + * This code runs on data servers, like Historicals and Peons, and also Brokers, if they operate on local datasources like + * inlined data wherein the broker needs to process some portion of data like the data server + * * @see org.apache.druid.query.groupby.epinephelinae.vector.VectorGroupByEngine for vectorized version of this class */ public class GroupByQueryEngine diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByRowProcessor.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByRowProcessor.java index ea6540d6f961..c040c5b64658 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByRowProcessor.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByRowProcessor.java @@ -121,7 +121,7 @@ public static ResultSupplier process( @Override public ByteBuffer get() { - final ResourceHolder mergeBufferHolder = resource.getMergeBuffer(); + final ResourceHolder mergeBufferHolder = resource.getToolchestMergeBuffer(); closeOnExit.register(mergeBufferHolder); return mergeBufferHolder.get(); } diff --git a/processing/src/test/java/org/apache/druid/query/QueryResourceIdTest.java b/processing/src/test/java/org/apache/druid/query/QueryResourceIdTest.java new file mode 100644 index 000000000000..875656993bb9 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/QueryResourceIdTest.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.query; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +public class QueryResourceIdTest +{ + + @Test + public void testConstructorWithNullString() + { + Assert.assertThrows(NullPointerException.class, () -> new QueryResourceId(null)); + } + + @Test + public void testEqualsAndHashCode() + { + EqualsVerifier.forClass(QueryResourceId.class) + .withNonnullFields("queryResourceId") + .usingGetClass() + .verify(); + } + + @Test + public void testAddressableOnAssociativeMap() + { + Map map = new HashMap<>(); + map.put(new QueryResourceId("abc"), 1); + Assert.assertEquals(1, (int) map.get(new QueryResourceId("abc"))); + + } +} diff --git a/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java index aed644cb07d5..f4699639af1f 100644 --- a/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java @@ -455,7 +455,7 @@ public static > ArrayList> mapQuery retVal.add( FluentQueryRunner.create(baseRunner, toolchest) .applyPreMergeDecoration() - .mergeResults() + .mergeResults(true) .applyPostMergeDecoration() .setToString(baseRunner.toString()) ); @@ -568,7 +568,7 @@ public static QueryRunner makeFilteringQueryRunner( toolChest ) .applyPreMergeDecoration() - .mergeResults() + .mergeResults(true) .applyPostMergeDecoration(); } diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java index cc62209ae91e..686e01e86173 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java @@ -38,7 +38,6 @@ import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; @@ -46,6 +45,7 @@ import org.apache.druid.java.util.common.guava.YieldingAccumulator; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.DefaultGenericQueryMetricsFactory; +import org.apache.druid.query.DirectQueryProcessingPool; import org.apache.druid.query.FinalizeResultsQueryRunner; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; @@ -58,6 +58,7 @@ 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.GroupByQueryRunnerTestHelper; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.TestGroupByBuffers; @@ -702,7 +703,7 @@ public Sequence runQueryOnSegmentsObjs(final List segments, fina toolChest.mergeResults( toolChest.preMergeQueryDecoration( factory.mergeRunners( - Execs.directExecutor(), + DirectQueryProcessingPool.INSTANCE, Lists.transform( segments, new Function() @@ -724,13 +725,14 @@ public QueryRunner apply(final Segment segment) } ) ) - ) + ), + true ) ), toolChest ); - return baseRunner.run(QueryPlus.wrap(query)); + return baseRunner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(query))); } public QueryRunner makeStringSerdeQueryRunner( diff --git a/processing/src/test/java/org/apache/druid/query/context/ResponseContextTest.java b/processing/src/test/java/org/apache/druid/query/context/ResponseContextTest.java index 03ed376f9385..0ae182b2f305 100644 --- a/processing/src/test/java/org/apache/druid/query/context/ResponseContextTest.java +++ b/processing/src/test/java/org/apache/druid/query/context/ResponseContextTest.java @@ -232,9 +232,7 @@ public void serializeWithCorrectnessTest() throws JsonProcessingException "string-value")), ctx1.serializeWith(mapper, Integer.MAX_VALUE).getResult()); - final ResponseContext ctx2 = ResponseContext.createEmpty(); - // Add two non-header fields, and one that will be in the header - ctx2.putEntityTag("not in header"); + final ResponseContext ctx2 = ResponseContext.createEmpty(); // Add two non-header fields, and one that will be in the header ctx2.putEntityTag("not in header"); ctx2.addCpuNanos(100); ctx2.add(EXTN_COUNTER_KEY, 100); Assert.assertEquals( diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java index 2dfe2b7dfbfb..dcbcb79724f5 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java @@ -43,6 +43,7 @@ import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.BySegmentQueryRunner; +import org.apache.druid.query.DirectQueryProcessingPool; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.FinalizeResultsQueryRunner; import org.apache.druid.query.Query; @@ -257,10 +258,10 @@ private void setupGroupByFactory() executorService = Execs.multiThreaded(3, "GroupByThreadPool[%d]"); final TestBufferPool bufferPool = TestBufferPool.offHeap(10_000_000, Integer.MAX_VALUE); + final TestBufferPool bufferPool2 = TestBufferPool.offHeap(10_000_000, Integer.MAX_VALUE); - // limit of 2 is required since we simulate both historical merge and broker merge in the same process + // Since the test has nested 'mergeResults' calls, we require 2 merge buffers for each mergeResults call. final TestBufferPool mergePool = TestBufferPool.offHeap(10_000_000, 2); - // limit of 2 is required since we simulate both historical merge and broker merge in the same process final TestBufferPool tooSmallMergePool = TestBufferPool.onHeap(255, 2); resourceCloser.register(() -> { @@ -324,11 +325,19 @@ public String getFormatString() }; final Supplier configSupplier = Suppliers.ofInstance(config); + GroupByResourcesReservationPool groupByResourcesReservationPool = new GroupByResourcesReservationPool( + mergePool, + config + ); + GroupByResourcesReservationPool tooSmallGroupByResourcesReservationPool = new GroupByResourcesReservationPool( + tooSmallMergePool, + config + ); final GroupingEngine groupingEngine = new GroupingEngine( druidProcessingConfig, configSupplier, bufferPool, - mergePool, + groupByResourcesReservationPool, TestHelper.makeJsonMapper(), new ObjectMapper(new SmileFactory()), NOOP_QUERYWATCHER @@ -337,22 +346,21 @@ public String getFormatString() final GroupingEngine tooSmallEngine = new GroupingEngine( tooSmallDruidProcessingConfig, configSupplier, - bufferPool, - tooSmallMergePool, + bufferPool2, + tooSmallGroupByResourcesReservationPool, TestHelper.makeJsonMapper(), new ObjectMapper(new SmileFactory()), NOOP_QUERYWATCHER ); - groupByFactory = new GroupByQueryRunnerFactory( groupingEngine, - new GroupByQueryQueryToolChest(groupingEngine) + new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool) ); tooSmallGroupByFactory = new GroupByQueryRunnerFactory( tooSmallEngine, - new GroupByQueryQueryToolChest(tooSmallEngine) + new GroupByQueryQueryToolChest(tooSmallEngine, tooSmallGroupByResourcesReservationPool) ); } @@ -380,16 +388,19 @@ public void testPartialLimitPushDownMerge() // one segment's results use limit push down, the other doesn't because of insufficient buffer capacity QueryToolChest toolChest = groupByFactory.getToolchest(); + QueryToolChest tooSmallToolChest = tooSmallGroupByFactory.getToolchest(); QueryRunner theRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( - groupByFactory.mergeRunners(executorService, getRunner1()) + groupByFactory.mergeRunners(DirectQueryProcessingPool.INSTANCE, getRunner1()), + true ), (QueryToolChest) toolChest ); QueryRunner theRunner2 = new FinalizeResultsQueryRunner<>( - toolChest.mergeResults( - tooSmallGroupByFactory.mergeRunners(executorService, getRunner2()) + tooSmallToolChest.mergeResults( + tooSmallGroupByFactory.mergeRunners(DirectQueryProcessingPool.INSTANCE, getRunner2()), + true ), (QueryToolChest) toolChest ); @@ -404,13 +415,24 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r return Sequences .simple( ImmutableList.of( - theRunner.run(queryPlus, responseContext), - theRunner2.run(queryPlus, responseContext) + Sequences.simple( + theRunner.run( + GroupByQueryRunnerTestHelper.populateResourceId(queryPlus), + responseContext + ).toList() + ), + Sequences.simple( + theRunner2.run( + GroupByQueryRunnerTestHelper.populateResourceId(queryPlus), + responseContext + ).toList() + ) ) ) .flatMerge(Function.identity(), queryPlus.getQuery().getResultOrdering()); } - } + }, + true ), (QueryToolChest) toolChest ); @@ -434,7 +456,10 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r .setGranularity(Granularities.ALL) .build(); - Sequence queryResult = theRunner3.run(QueryPlus.wrap(query), ResponseContext.createEmpty()); + Sequence queryResult = theRunner3.run( + QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(query)), + ResponseContext.createEmpty() + ); List results = queryResult.toList(); ResultRow expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( @@ -468,19 +493,19 @@ public void testPartialLimitPushDownMergeForceAggs() // one segment's results use limit push down, the other doesn't because of insufficient buffer capacity QueryToolChest toolChest = groupByFactory.getToolchest(); - QueryRunner theRunner = new FinalizeResultsQueryRunner<>( - toolChest.mergeResults( + QueryRunner theRunner = new FinalizeResultsQueryRunner( + (queryPlus, responseContext) -> toolChest.mergeResults( groupByFactory.mergeRunners(executorService, getRunner1()) - ), + ).run(GroupByQueryRunnerTestHelper.populateResourceId(queryPlus), responseContext), (QueryToolChest) toolChest ); - - QueryRunner theRunner2 = new FinalizeResultsQueryRunner<>( - toolChest.mergeResults( + QueryToolChest tooSmalltoolChest = tooSmallGroupByFactory.getToolchest(); + QueryRunner theRunner2 = new FinalizeResultsQueryRunner( + (queryPlus, responseContext) -> tooSmalltoolChest.mergeResults( tooSmallGroupByFactory.mergeRunners(executorService, getRunner2()) - ), - (QueryToolChest) toolChest + ).run(GroupByQueryRunnerTestHelper.populateResourceId(queryPlus), responseContext), + (QueryToolChest) tooSmalltoolChest ); QueryRunner theRunner3 = new FinalizeResultsQueryRunner<>( @@ -493,13 +518,14 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r return Sequences .simple( ImmutableList.of( - theRunner.run(queryPlus, responseContext), - theRunner2.run(queryPlus, responseContext) + Sequences.simple(theRunner.run(queryPlus, responseContext).toList()), + Sequences.simple(theRunner2.run(queryPlus, responseContext).toList()) ) ) .flatMerge(Function.identity(), queryPlus.getQuery().getResultOrdering()); } - } + }, + true ), (QueryToolChest) toolChest ); @@ -531,7 +557,10 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r ) .build(); - Sequence queryResult = theRunner3.run(QueryPlus.wrap(query), ResponseContext.createEmpty()); + Sequence queryResult = theRunner3.run( + QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(query)), + ResponseContext.createEmpty() + ); List results = queryResult.toList(); ResultRow expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java index 28ff970efdf4..9b9fea03231e 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java @@ -44,6 +44,7 @@ import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.BySegmentQueryRunner; +import org.apache.druid.query.DirectQueryProcessingPool; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.FinalizeResultsQueryRunner; import org.apache.druid.query.Query; @@ -80,6 +81,7 @@ import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; import org.joda.time.DateTimeZone; import org.joda.time.Period; @@ -98,7 +100,7 @@ import java.util.concurrent.ExecutorService; import java.util.function.Function; -public class GroupByLimitPushDownMultiNodeMergeTest +public class GroupByLimitPushDownMultiNodeMergeTest extends InitializedNullHandlingTest { public static final ObjectMapper JSON_MAPPER; @@ -106,8 +108,9 @@ public class GroupByLimitPushDownMultiNodeMergeTest private static final IndexIO INDEX_IO; private File tmpDir; - private QueryRunnerFactory groupByFactory; - private QueryRunnerFactory groupByFactory2; + private QueryRunnerFactory groupByFactoryBroker; + private QueryRunnerFactory groupByFactoryHistorical; + private QueryRunnerFactory groupByFactoryHistorical2; private List incrementalIndices = new ArrayList<>(); private List groupByIndices = new ArrayList<>(); private ExecutorService executorService; @@ -529,15 +532,16 @@ private void setupGroupByFactory() executorService = Execs.multiThreaded(3, "GroupByThreadPool[%d]"); final TestBufferPool bufferPool = TestBufferPool.offHeap(10_000_000, Integer.MAX_VALUE); - final TestBufferPool mergePool = TestBufferPool.offHeap(10_000_000, 2); - // limit of 2 is required since we simulate both historical merge and broker merge in the same process - final TestBufferPool mergePool2 = TestBufferPool.offHeap(10_000_000, 2); + final TestBufferPool mergePoolBroker = TestBufferPool.offHeap(10_000_000, 1); + final TestBufferPool mergePoolHistorical = TestBufferPool.offHeap(10_000_000, 1); + final TestBufferPool mergePoolHistorical2 = TestBufferPool.offHeap(10_000_000, 1); resourceCloser.register(() -> { // Verify that all objects have been returned to the pools. Assert.assertEquals(0, bufferPool.getOutstandingObjectCount()); - Assert.assertEquals(0, mergePool.getOutstandingObjectCount()); - Assert.assertEquals(0, mergePool2.getOutstandingObjectCount()); + Assert.assertEquals(0, mergePoolBroker.getOutstandingObjectCount()); + Assert.assertEquals(0, mergePoolHistorical.getOutstandingObjectCount()); + Assert.assertEquals(0, mergePoolHistorical2.getOutstandingObjectCount()); }); final GroupByQueryConfig config = new GroupByQueryConfig() @@ -574,34 +578,54 @@ public String getFormatString() }; final Supplier configSupplier = Suppliers.ofInstance(config); - final GroupingEngine groupingEngine = new GroupingEngine( + GroupByResourcesReservationPool groupByResourcesReservationPoolBroker = + new GroupByResourcesReservationPool(mergePoolBroker, config); + GroupByResourcesReservationPool groupByResourcesReservationPoolHistorical = + new GroupByResourcesReservationPool(mergePoolHistorical, config); + GroupByResourcesReservationPool groupByResourcesReservationPoolHistorical2 = + new GroupByResourcesReservationPool(mergePoolHistorical2, config); + + final GroupingEngine groupingEngineBroker = new GroupingEngine( druidProcessingConfig, configSupplier, bufferPool, - mergePool, + groupByResourcesReservationPoolBroker, TestHelper.makeJsonMapper(), new ObjectMapper(new SmileFactory()), NOOP_QUERYWATCHER ); - - final GroupingEngine groupingEngine2 = new GroupingEngine( + final GroupingEngine groupingEngineHistorical = new GroupingEngine( + druidProcessingConfig, + configSupplier, + bufferPool, + groupByResourcesReservationPoolHistorical, + TestHelper.makeJsonMapper(), + new ObjectMapper(new SmileFactory()), + NOOP_QUERYWATCHER + ); + final GroupingEngine groupingEngineHistorical2 = new GroupingEngine( druidProcessingConfig, configSupplier, bufferPool, - mergePool2, + groupByResourcesReservationPoolHistorical2, TestHelper.makeJsonMapper(), new ObjectMapper(new SmileFactory()), NOOP_QUERYWATCHER ); - groupByFactory = new GroupByQueryRunnerFactory( - groupingEngine, - new GroupByQueryQueryToolChest(groupingEngine) + groupByFactoryBroker = new GroupByQueryRunnerFactory( + groupingEngineBroker, + new GroupByQueryQueryToolChest(groupingEngineBroker, groupByResourcesReservationPoolBroker) ); - groupByFactory2 = new GroupByQueryRunnerFactory( - groupingEngine2, - new GroupByQueryQueryToolChest(groupingEngine2) + groupByFactoryHistorical = new GroupByQueryRunnerFactory( + groupingEngineHistorical, + new GroupByQueryQueryToolChest(groupingEngineHistorical, groupByResourcesReservationPoolHistorical) + ); + + groupByFactoryHistorical2 = new GroupByQueryRunnerFactory( + groupingEngineHistorical2, + new GroupByQueryQueryToolChest(groupingEngineHistorical2, groupByResourcesReservationPoolHistorical2) ); } @@ -626,23 +650,28 @@ public void tearDown() throws Exception @Test public void testDescendingNumerics() { - QueryToolChest toolChest = groupByFactory.getToolchest(); + QueryToolChest toolChestHistorical = groupByFactoryHistorical.getToolchest(); QueryRunner theRunner = new FinalizeResultsQueryRunner<>( - toolChest.mergeResults( - groupByFactory.mergeRunners(executorService, getRunner1(2)) + toolChestHistorical.mergeResults( + groupByFactoryHistorical.mergeRunners(DirectQueryProcessingPool.INSTANCE, getRunner1(2)), + true ), - (QueryToolChest) toolChest + (QueryToolChest) toolChestHistorical ); + QueryToolChest toolChestHistorical2 = groupByFactoryHistorical2.getToolchest(); QueryRunner theRunner2 = new FinalizeResultsQueryRunner<>( - toolChest.mergeResults( - groupByFactory2.mergeRunners(executorService, getRunner2(3)) + toolChestHistorical2.mergeResults( + groupByFactoryHistorical2.mergeRunners(DirectQueryProcessingPool.INSTANCE, getRunner2(3)), + true ), - (QueryToolChest) toolChest + (QueryToolChest) toolChestHistorical2 ); + QueryToolChest toolChestBroker = groupByFactoryHistorical.getToolchest(); + QueryRunner finalRunner = new FinalizeResultsQueryRunner<>( - toolChest.mergeResults( + toolChestBroker.mergeResults( new QueryRunner() { @Override @@ -651,15 +680,16 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r return Sequences .simple( ImmutableList.of( - theRunner.run(queryPlus, responseContext), - theRunner2.run(queryPlus, responseContext) + theRunner.run(GroupByQueryRunnerTestHelper.populateResourceId(queryPlus), responseContext), + theRunner2.run(GroupByQueryRunnerTestHelper.populateResourceId(queryPlus), responseContext) ) ) .flatMerge(Function.identity(), queryPlus.getQuery().getResultOrdering()); } - } + }, + false ), - (QueryToolChest) toolChest + (QueryToolChest) toolChestBroker ); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec( @@ -715,7 +745,10 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r .setGranularity(Granularities.ALL) .build(); - Sequence queryResult = finalRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty()); + Sequence queryResult = finalRunner.run( + QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(query)), + ResponseContext.createEmpty() + ); List results = queryResult.toList(); ResultRow expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( @@ -763,23 +796,28 @@ public void testPartialLimitPushDownMerge() { // one segment's results use limit push down, the other doesn't because of insufficient buffer capacity - QueryToolChest toolChest = groupByFactory.getToolchest(); + QueryToolChest toolChestHistorical = groupByFactoryHistorical.getToolchest(); QueryRunner theRunner = new FinalizeResultsQueryRunner<>( - toolChest.mergeResults( - groupByFactory.mergeRunners(executorService, getRunner1(0)) + toolChestHistorical.mergeResults( + groupByFactoryHistorical.mergeRunners(DirectQueryProcessingPool.INSTANCE, getRunner1(0)), + true ), - (QueryToolChest) toolChest + (QueryToolChest) toolChestHistorical ); + QueryToolChest toolChestHistorical2 = groupByFactoryHistorical2.getToolchest(); QueryRunner theRunner2 = new FinalizeResultsQueryRunner<>( - toolChest.mergeResults( - groupByFactory2.mergeRunners(executorService, getRunner2(1)) + toolChestHistorical2.mergeResults( + groupByFactoryHistorical2.mergeRunners(DirectQueryProcessingPool.INSTANCE, getRunner2(1)), + true ), - (QueryToolChest) toolChest + (QueryToolChest) toolChestHistorical2 ); + QueryToolChest toolchestBroker = groupByFactoryBroker.getToolchest(); + QueryRunner finalRunner = new FinalizeResultsQueryRunner<>( - toolChest.mergeResults( + toolchestBroker.mergeResults( new QueryRunner() { @Override @@ -788,15 +826,16 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r return Sequences .simple( ImmutableList.of( - theRunner.run(queryPlus, responseContext), - theRunner2.run(queryPlus, responseContext) + theRunner.run(GroupByQueryRunnerTestHelper.populateResourceId(queryPlus), responseContext), + theRunner2.run(GroupByQueryRunnerTestHelper.populateResourceId(queryPlus), responseContext) ) ) .flatMerge(Function.identity(), queryPlus.getQuery().getResultOrdering()); } - } + }, + false ), - (QueryToolChest) toolChest + (QueryToolChest) toolchestBroker ); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec( @@ -840,7 +879,10 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r .setGranularity(Granularities.ALL) .build(); - Sequence queryResult = finalRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty()); + Sequence queryResult = finalRunner.run( + QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(query)), + ResponseContext.createEmpty() + ); List results = queryResult.toList(); ResultRow expectedRow0 = GroupByQueryRunnerTestHelper.createExpectedRow( @@ -904,23 +946,27 @@ public void testForcePushLimitDownAccuracyWhenSortHasNonGroupingFields() private List testForcePushLimitDownAccuracyWhenSortHasNonGroupingFieldsHelper(Map context) { - QueryToolChest toolChest = groupByFactory.getToolchest(); + QueryToolChest toolChestHistorical = groupByFactoryHistorical.getToolchest(); QueryRunner theRunner = new FinalizeResultsQueryRunner<>( - toolChest.mergeResults( - groupByFactory.mergeRunners(executorService, getRunner1(4)) + toolChestHistorical.mergeResults( + groupByFactoryHistorical.mergeRunners(DirectQueryProcessingPool.INSTANCE, getRunner1(4)), + true ), - (QueryToolChest) toolChest + (QueryToolChest) toolChestHistorical ); + QueryToolChest toolChestHistorical2 = groupByFactoryHistorical2.getToolchest(); QueryRunner theRunner2 = new FinalizeResultsQueryRunner<>( - toolChest.mergeResults( - groupByFactory2.mergeRunners(executorService, getRunner2(5)) + toolChestHistorical2.mergeResults( + groupByFactoryHistorical2.mergeRunners(DirectQueryProcessingPool.INSTANCE, getRunner2(5)), + true ), - (QueryToolChest) toolChest + (QueryToolChest) toolChestHistorical2 ); + QueryToolChest toolchestBroker = groupByFactoryBroker.getToolchest(); QueryRunner finalRunner = new FinalizeResultsQueryRunner<>( - toolChest.mergeResults( + toolchestBroker.mergeResults( new QueryRunner() { @Override @@ -929,15 +975,16 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r return Sequences .simple( ImmutableList.of( - theRunner.run(queryPlus, responseContext), - theRunner2.run(queryPlus, responseContext) + theRunner.run(GroupByQueryRunnerTestHelper.populateResourceId(queryPlus), responseContext), + theRunner2.run(GroupByQueryRunnerTestHelper.populateResourceId(queryPlus), responseContext) ) ) .flatMerge(Function.identity(), queryPlus.getQuery().getResultOrdering()); } - } + }, + false ), - (QueryToolChest) toolChest + (QueryToolChest) toolchestBroker ); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec( @@ -964,7 +1011,10 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r .setGranularity(Granularities.ALL) .build(); - Sequence queryResult = finalRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty()); + Sequence queryResult = finalRunner.run( + QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(query)), + ResponseContext.createEmpty() + ); return queryResult.toList(); } @@ -973,11 +1023,11 @@ private List> getRunner1(int qIndexNumber) List> runners = new ArrayList<>(); QueryableIndex index = groupByIndices.get(qIndexNumber); QueryRunner runner = makeQueryRunner( - groupByFactory, + groupByFactoryHistorical, SegmentId.dummy(index.toString()), new QueryableIndexSegment(index, SegmentId.dummy(index.toString())) ); - runners.add(groupByFactory.getToolchest().preMergeQueryDecoration(runner)); + runners.add(groupByFactoryHistorical.getToolchest().preMergeQueryDecoration(runner)); return runners; } @@ -986,11 +1036,11 @@ private List> getRunner2(int qIndexNumber) List> runners = new ArrayList<>(); QueryableIndex index2 = groupByIndices.get(qIndexNumber); QueryRunner tooSmallRunner = makeQueryRunner( - groupByFactory2, + groupByFactoryHistorical2, SegmentId.dummy(index2.toString()), new QueryableIndexSegment(index2, SegmentId.dummy(index2.toString())) ); - runners.add(groupByFactory2.getToolchest().preMergeQueryDecoration(tooSmallRunner)); + runners.add(groupByFactoryHistorical2.getToolchest().preMergeQueryDecoration(tooSmallRunner)); return runners; } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java index 0284edd41279..d461876c9bdf 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java @@ -40,6 +40,7 @@ import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.BySegmentQueryRunner; +import org.apache.druid.query.DirectQueryProcessingPool; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.FinalizeResultsQueryRunner; import org.apache.druid.query.Query; @@ -69,6 +70,7 @@ import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; import org.junit.After; import org.junit.Assert; @@ -84,7 +86,7 @@ import java.util.Map; import java.util.concurrent.ExecutorService; -public class GroupByMultiSegmentTest +public class GroupByMultiSegmentTest extends InitializedNullHandlingTest { public static final ObjectMapper JSON_MAPPER; @@ -239,11 +241,12 @@ public String getFormatString() }; final Supplier configSupplier = Suppliers.ofInstance(config); + final GroupByResourcesReservationPool groupByResourcesReservationPool = new GroupByResourcesReservationPool(mergePool, config); final GroupingEngine groupingEngine = new GroupingEngine( druidProcessingConfig, configSupplier, bufferPool, - mergePool, + groupByResourcesReservationPool, TestHelper.makeJsonMapper(), new ObjectMapper(new SmileFactory()), NOOP_QUERYWATCHER @@ -251,7 +254,7 @@ public String getFormatString() groupByFactory = new GroupByQueryRunnerFactory( groupingEngine, - new GroupByQueryQueryToolChest(groupingEngine) + new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool) ); } @@ -279,7 +282,8 @@ public void testHavingAndNoLimitPushDown() QueryToolChest toolChest = groupByFactory.getToolchest(); QueryRunner theRunner = new FinalizeResultsQueryRunner<>( toolChest.mergeResults( - groupByFactory.mergeRunners(executorService, makeGroupByMultiRunners()) + groupByFactory.mergeRunners(DirectQueryProcessingPool.INSTANCE, makeGroupByMultiRunners()), + true ), (QueryToolChest) toolChest ); @@ -305,7 +309,10 @@ public void testHavingAndNoLimitPushDown() .setGranularity(Granularities.ALL) .build(); - Sequence queryResult = theRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty()); + Sequence queryResult = theRunner.run( + QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(query)), + ResponseContext.createEmpty() + ); List results = queryResult.toList(); ResultRow expectedRow = GroupByQueryRunnerTestHelper.createExpectedRow( diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java index e080af234b67..7c5b5e908444 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java @@ -124,17 +124,22 @@ private static GroupByQueryRunnerFactory makeQueryRunnerFactory( ) { final Supplier configSupplier = Suppliers.ofInstance(config); + final GroupByResourcesReservationPool groupByResourcesReservationPool = + new GroupByResourcesReservationPool(MERGE_BUFFER_POOL, config); final GroupingEngine groupingEngine = new GroupingEngine( PROCESSING_CONFIG, configSupplier, BUFFER_POOL, - MERGE_BUFFER_POOL, + groupByResourcesReservationPool, TestHelper.makeJsonMapper(), mapper, QueryRunnerTestHelper.NOOP_QUERYWATCHER ); - final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(groupingEngine); + final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest( + groupingEngine, + groupByResourcesReservationPool + ); return new GroupByQueryRunnerFactory(groupingEngine, toolChest); } @@ -197,7 +202,7 @@ public void testSimpleGroupBy() .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, TIMEOUT)) .build(); - Assert.assertEquals(0, GroupByQueryResources.countRequiredMergeBufferNum(query)); + Assert.assertEquals(0, GroupByQueryResources.countRequiredMergeBufferNumForToolchestMerge(query)); GroupByQueryRunnerTestHelper.runQuery(FACTORY, runner, query); Assert.assertEquals(3, MERGE_BUFFER_POOL.getMinRemainBufferNum()); @@ -226,7 +231,7 @@ public void testNestedGroupBy() .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, TIMEOUT)) .build(); - Assert.assertEquals(1, GroupByQueryResources.countRequiredMergeBufferNum(query)); + Assert.assertEquals(1, GroupByQueryResources.countRequiredMergeBufferNumForToolchestMerge(query)); GroupByQueryRunnerTestHelper.runQuery(FACTORY, runner, query); Assert.assertEquals(2, MERGE_BUFFER_POOL.getMinRemainBufferNum()); @@ -266,7 +271,7 @@ public void testDoubleNestedGroupBy() .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, TIMEOUT)) .build(); - Assert.assertEquals(2, GroupByQueryResources.countRequiredMergeBufferNum(query)); + Assert.assertEquals(2, GroupByQueryResources.countRequiredMergeBufferNumForToolchestMerge(query)); GroupByQueryRunnerTestHelper.runQuery(FACTORY, runner, query); // This should be 1 because the broker needs 2 buffers and the queryable node needs one. @@ -320,7 +325,7 @@ public void testTripleNestedGroupBy() .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, TIMEOUT)) .build(); - Assert.assertEquals(2, GroupByQueryResources.countRequiredMergeBufferNum(query)); + Assert.assertEquals(2, GroupByQueryResources.countRequiredMergeBufferNumForToolchestMerge(query)); GroupByQueryRunnerTestHelper.runQuery(FACTORY, runner, query); // This should be 1 because the broker needs 2 buffers and the queryable node needs one. @@ -349,7 +354,7 @@ public void testSimpleGroupByWithSubtotals() .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, TIMEOUT)) .build(); - Assert.assertEquals(1, GroupByQueryResources.countRequiredMergeBufferNum(query)); + Assert.assertEquals(1, GroupByQueryResources.countRequiredMergeBufferNumForToolchestMerge(query)); GroupByQueryRunnerTestHelper.runQuery(FACTORY, runner, query); // 1 for subtotal and 1 for GroupByQueryRunnerFactory#mergeRunners @@ -378,7 +383,7 @@ public void testSimpleGroupByWithSubtotalsWithoutPrefixMatch() .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, TIMEOUT)) .build(); - Assert.assertEquals(2, GroupByQueryResources.countRequiredMergeBufferNum(query)); + Assert.assertEquals(2, GroupByQueryResources.countRequiredMergeBufferNumForToolchestMerge(query)); GroupByQueryRunnerTestHelper.runQuery(FACTORY, runner, query); // 2 needed by subtotal and 1 for GroupByQueryRunnerFactory#mergeRunners @@ -420,7 +425,7 @@ public void testNestedGroupByWithSubtotals() .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, TIMEOUT)) .build(); - Assert.assertEquals(3, GroupByQueryResources.countRequiredMergeBufferNum(query)); + Assert.assertEquals(3, GroupByQueryResources.countRequiredMergeBufferNumForToolchestMerge(query)); GroupByQueryRunnerTestHelper.runQuery(FACTORY, runner, query); // 2 for subtotal, 1 for nested group by and 1 for GroupByQueryRunnerFactory#mergeRunners @@ -466,7 +471,7 @@ public void testNestedGroupByWithNestedSubtotals() .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, TIMEOUT)) .build(); - Assert.assertEquals(3, GroupByQueryResources.countRequiredMergeBufferNum(query)); + Assert.assertEquals(3, GroupByQueryResources.countRequiredMergeBufferNumForToolchestMerge(query)); GroupByQueryRunnerTestHelper.runQuery(FACTORY, runner, query); // 2 for subtotal, 1 for nested group by and 1 for GroupByQueryRunnerFactory#mergeRunners diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java index 726d04f65c86..0c89afb75e23 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java @@ -129,13 +129,11 @@ public void testResultLevelCacheKeyWithPostAggregate() .setGranularity(QueryRunnerTestHelper.DAY_GRAN) .build(); - final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( - null - ).getCacheStrategy(query1); + final CacheStrategy strategy1 = + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1); - final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( - null - ).getCacheStrategy(query2); + final CacheStrategy strategy2 = + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2); Assert.assertTrue(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2))); Assert.assertFalse(Arrays.equals( @@ -191,13 +189,11 @@ public void testResultLevelCacheKeyWithLimitSpec() ) .build(); - final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( - null - ).getCacheStrategy(query1); + final CacheStrategy strategy1 = + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1); - final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( - null - ).getCacheStrategy(query2); + final CacheStrategy strategy2 = + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2); Assert.assertTrue(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2))); Assert.assertFalse(Arrays.equals( @@ -255,13 +251,11 @@ public void testResultLevelCacheKeyWithHavingSpec() .setHavingSpec(new GreaterThanHavingSpec(QueryRunnerTestHelper.UNIQUE_METRIC, 10)) .build(); - final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( - null - ).getCacheStrategy(query1); + final CacheStrategy strategy1 = + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1); - final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( - null - ).getCacheStrategy(query2); + final CacheStrategy strategy2 = + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2); Assert.assertTrue(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2))); Assert.assertFalse(Arrays.equals( @@ -341,13 +335,11 @@ public void testResultLevelCacheKeyWithAndHavingSpec() .setHavingSpec(andHavingSpec2) .build(); - final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( - null - ).getCacheStrategy(query1); + final CacheStrategy strategy1 = + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1); - final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( - null - ).getCacheStrategy(query2); + final CacheStrategy strategy2 = + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2); Assert.assertTrue(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2))); Assert.assertFalse(Arrays.equals( @@ -434,13 +426,11 @@ public void testResultLevelCacheKeyWithHavingDimFilterHavingSpec() .setHavingSpec(havingSpec2) .build(); - final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( - null - ).getCacheStrategy(query1); + final CacheStrategy strategy1 = + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1); - final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( - null - ).getCacheStrategy(query2); + final CacheStrategy strategy2 = + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2); Assert.assertTrue(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2))); Assert.assertFalse(Arrays.equals( @@ -499,13 +489,11 @@ public void testResultLevelCacheKeyWithSubTotalsSpec() )) .build(); - final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( - null - ).getCacheStrategy(query1); + final CacheStrategy strategy1 = + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1); - final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( - null - ).getCacheStrategy(query2); + final CacheStrategy strategy2 = + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2); Assert.assertTrue(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2))); Assert.assertFalse(Arrays.equals( @@ -550,9 +538,7 @@ public void testMultiColumnCacheStrategy() throws Exception .build(); CacheStrategy strategy = - new GroupByQueryQueryToolChest(null).getCacheStrategy( - query1 - ); + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1); // test timestamps that result in integer size millis final ResultRow result1 = ResultRow.of( @@ -589,7 +575,7 @@ public void testResultSerde() throws Exception .setGranularity(QueryRunnerTestHelper.DAY_GRAN) .build(); - final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(null); + final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(null, null); final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); final ObjectMapper arraysObjectMapper = toolChest.decorateObjectMapper( @@ -682,6 +668,7 @@ public boolean isIntermediateResultAsMapCompat() return true; } }, + null, null ); @@ -798,7 +785,7 @@ public void testResultArraySignatureAllGran() .add("uniques", null) .add("const", ColumnType.LONG) .build(), - new GroupByQueryQueryToolChest(null).resultArraySignature(query) + new GroupByQueryQueryToolChest(null, null).resultArraySignature(query) ); } @@ -823,7 +810,7 @@ public void testResultArraySignatureDayGran() .add("uniques", null) .add("const", ColumnType.LONG) .build(), - new GroupByQueryQueryToolChest(null).resultArraySignature(query) + new GroupByQueryQueryToolChest(null, null).resultArraySignature(query) ); } @@ -844,7 +831,7 @@ public void testResultsAsArraysAllGran() new Object[]{"foo", 1L, 2L, 3L, 1L}, new Object[]{"bar", 4L, 5L, 6L, 1L} ), - new GroupByQueryQueryToolChest(null).resultsAsArrays( + new GroupByQueryQueryToolChest(null, null).resultsAsArrays( query, Sequences.simple( ImmutableList.of( @@ -873,7 +860,7 @@ public void testResultsAsArraysDayGran() new Object[]{DateTimes.of("2000-01-01").getMillis(), "foo", 1L, 2L, 3L, 1L}, new Object[]{DateTimes.of("2000-01-02").getMillis(), "bar", 4L, 5L, 6L, 1L} ), - new GroupByQueryQueryToolChest(null).resultsAsArrays( + new GroupByQueryQueryToolChest(null, null).resultsAsArrays( query, Sequences.simple( ImmutableList.of( @@ -889,7 +876,7 @@ public void testResultsAsArraysDayGran() public void testCanPerformSubqueryOnGroupBys() { Assert.assertTrue( - new GroupByQueryQueryToolChest(null).canPerformSubquery( + new GroupByQueryQueryToolChest(null, null).canPerformSubquery( new GroupByQuery.Builder() .setDataSource( new QueryDataSource( @@ -911,7 +898,7 @@ public void testCanPerformSubqueryOnGroupBys() public void testCanPerformSubqueryOnTimeseries() { Assert.assertFalse( - new GroupByQueryQueryToolChest(null).canPerformSubquery( + new GroupByQueryQueryToolChest(null, null).canPerformSubquery( Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -925,7 +912,7 @@ public void testCanPerformSubqueryOnTimeseries() public void testCanPerformSubqueryOnGroupByOfTimeseries() { Assert.assertFalse( - new GroupByQueryQueryToolChest(null).canPerformSubquery( + new GroupByQueryQueryToolChest(null, null).canPerformSubquery( new GroupByQuery.Builder() .setDataSource( new QueryDataSource( @@ -998,9 +985,7 @@ private void doTestCacheStrategy(final ColumnType valueType, final Object dimVal .build(); CacheStrategy strategy = - new GroupByQueryQueryToolChest(null).getCacheStrategy( - query1 - ); + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1); // test timestamps that result in integer size millis final ResultRow result1 = ResultRow.of( @@ -1092,13 +1077,11 @@ public void testQueryCacheKeyWithLimitSpec() .setGranularity(QueryRunnerTestHelper.DAY_GRAN) .build(); - final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( - null - ).getCacheStrategy(query1); + final CacheStrategy strategy1 = + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1); - final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( - null - ).getCacheStrategy(query2); + final CacheStrategy strategy2 = + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2); Assert.assertFalse(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2))); Assert.assertFalse(Arrays.equals( @@ -1130,13 +1113,11 @@ public void testQueryCacheKeyWithLimitSpecPushDownUsingContext() .overrideContext(ImmutableMap.of(GroupByQueryConfig.CTX_KEY_APPLY_LIMIT_PUSH_DOWN, "false")) .build(); - final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( - null - ).getCacheStrategy(query1); + final CacheStrategy strategy1 = + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1); - final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( - null - ).getCacheStrategy(query2); + final CacheStrategy strategy2 = + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2); Assert.assertFalse(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2))); Assert.assertTrue( @@ -1183,16 +1164,17 @@ public String getFormatString() bufferSupplier, processingConfig.getNumMergeBuffers() ); + GroupByResourcesReservationPool groupByResourcesReservationPool = new GroupByResourcesReservationPool(mergeBufferPool, queryConfig); final GroupingEngine groupingEngine = new GroupingEngine( processingConfig, queryConfigSupplier, bufferPool, - mergeBufferPool, + groupByResourcesReservationPool, TestHelper.makeJsonMapper(), new ObjectMapper(new SmileFactory()), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); - final GroupByQueryQueryToolChest queryToolChest = new GroupByQueryQueryToolChest(groupingEngine); + final GroupByQueryQueryToolChest queryToolChest = new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool); CacheStrategy cacheStrategy = queryToolChest.getCacheStrategy(query); Assert.assertTrue( "result level cache on broker server for GroupByStrategyV2 should be enabled", diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java index 266691ee2943..bc8563552cd1 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java @@ -96,6 +96,7 @@ public void testMergeRunnersEnsureGroupMerging() @Override public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { + //noinspection unchecked return factory.getToolchest().mergeResults( new QueryRunner() { @@ -108,8 +109,8 @@ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) query.getResultOrdering(), Sequences.simple( Arrays.asList( - factory.createRunner(createSegment()).run(queryPlus, responseContext), - factory.createRunner(createSegment()).run(queryPlus, responseContext) + Sequences.simple(factory.createRunner(createSegment()).run(queryPlus, responseContext).toList()), + Sequences.simple(factory.createRunner(createSegment()).run(queryPlus, responseContext).toList()) ) ) ); @@ -118,13 +119,17 @@ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) throw new RuntimeException(e); } } - } - ).run(queryPlus, responseContext); + }, + false + ).run(GroupByQueryRunnerTestHelper.populateResourceId(queryPlus), responseContext); } } ); - Sequence result = mergedRunner.run(QueryPlus.wrap(query), ResponseContext.createEmpty()); + Sequence result = mergedRunner.run( + QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(query)), + ResponseContext.createEmpty() + ); List expectedResults = Arrays.asList( GroupByQueryRunnerTestHelper.createExpectedRow(query, "1970-01-01T00:00:00.000Z", "tags", "t1", "count", 2L), diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java index 6fc0cdf548f5..ad6157dc927d 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java @@ -31,7 +31,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.DruidProcessingConfig; -import org.apache.druid.query.QueryCapacityExceededException; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.QueryRunner; @@ -95,16 +94,17 @@ private static GroupByQueryRunnerFactory makeQueryRunnerFactory( ) { final Supplier configSupplier = Suppliers.ofInstance(config); + GroupByResourcesReservationPool groupByResourcesReservationPool = new GroupByResourcesReservationPool(MERGE_BUFFER_POOL, config); final GroupingEngine groupingEngine = new GroupingEngine( DEFAULT_PROCESSING_CONFIG, configSupplier, BUFFER_POOL, - MERGE_BUFFER_POOL, + groupByResourcesReservationPool, TestHelper.makeJsonMapper(), mapper, QueryRunnerTestHelper.NOOP_QUERYWATCHER ); - final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(groupingEngine); + final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool); return new GroupByQueryRunnerFactory(groupingEngine, toolChest); } @@ -151,8 +151,8 @@ public GroupByQueryRunnerFailureTest(QueryRunner runner) @Test(timeout = 60_000L) public void testNotEnoughMergeBuffersOnQueryable() { - expectedException.expect(QueryTimeoutException.class); - expectedException.expectMessage("Cannot acquire enough merge buffers"); + expectedException.expect(ResourceLimitExceededException.class); + expectedException.expectMessage("Query needs 2 merge buffers, but only 1 merge buffers were configured"); final GroupByQuery query = GroupByQuery .builder() @@ -239,8 +239,8 @@ public void testInsufficientResourcesOnBroker() List> holder = null; try { holder = MERGE_BUFFER_POOL.takeBatch(1, 10); - expectedException.expect(QueryCapacityExceededException.class); - expectedException.expectMessage("Cannot acquire 1 merge buffers. Try again after current running queries are finished."); + expectedException.expect(ResourceLimitExceededException.class); + expectedException.expectMessage("Query needs 2 merge buffers, but only 1 merge buffers were configured"); GroupByQueryRunnerTestHelper.runQuery(FACTORY, runner, query); } finally { diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java index 6f6538f6fd50..0b4d8994bf3d 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java @@ -349,16 +349,23 @@ public static GroupByQueryRunnerFactory makeQueryRunnerFactory( ); } final Supplier configSupplier = Suppliers.ofInstance(config); + final GroupByResourcesReservationPool groupByResourcesReservationPool = + new GroupByResourcesReservationPool(bufferPools.getMergePool(), config); final GroupingEngine groupingEngine = new GroupingEngine( processingConfig, configSupplier, bufferPools.getProcessingPool(), - bufferPools.getMergePool(), + groupByResourcesReservationPool, TestHelper.makeJsonMapper(), mapper, QueryRunnerTestHelper.NOOP_QUERYWATCHER ); - final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(groupingEngine); + final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest( + groupingEngine, + () -> config, + DefaultGroupByQueryMetricsFactory.instance(), + groupByResourcesReservationPool + ); return new GroupByQueryRunnerFactory(groupingEngine, toolChest); } @@ -3768,11 +3775,15 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r return new MergeSequence( queryPlus.getQuery().getResultOrdering(), Sequences.simple( - Arrays.asList(runner.run(queryPlus1, responseContext), runner.run(queryPlus2, responseContext)) + Arrays.asList( + Sequences.simple(runner.run(queryPlus1, responseContext).toList()), + Sequences.simple(runner.run(queryPlus2, responseContext).toList()) + ) ) ); } - } + }, + true ); List expectedResults = Arrays.asList( @@ -3787,8 +3798,10 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r makeRow(fullQuery, "2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) ); - ResponseContext context = ResponseContext.createEmpty(); - TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(fullQuery)), "merged"); + TestHelper.assertExpectedObjects( + expectedResults, + mergedRunner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(fullQuery))), "merged" + ); List allGranExpectedResults = Arrays.asList( makeRow(allGranQuery, "2011-04-02", "alias", "automotive", "rows", 2L, "idx", 269L), @@ -3804,7 +3817,7 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r TestHelper.assertExpectedObjects( allGranExpectedResults, - mergedRunner.run(QueryPlus.wrap(allGranQuery)), + mergedRunner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(allGranQuery))), "merged" ); } @@ -3928,7 +3941,7 @@ private void doTestMergeResultsWithValidLimit(final int limit, final int offset) TestHelper.assertExpectedObjects( Iterables.limit(Iterables.skip(expectedResults, offset), limit), - mergeRunner.run(QueryPlus.wrap(fullQuery)), + mergeRunner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(fullQuery))), StringUtils.format("limit: %d", limit) ); } @@ -3966,11 +3979,11 @@ public void testMergeResultsAcrossMultipleDaysWithLimitAndOrderBy() makeRow(fullQuery, "2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) ); - QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); + QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner, true); TestHelper.assertExpectedObjects( Iterables.limit(expectedResults, limit), - mergeRunner.run(QueryPlus.wrap(fullQuery)), + mergeRunner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(fullQuery))), StringUtils.format("limit: %d", limit) ); } @@ -4020,7 +4033,7 @@ public void testMergeResultsAcrossMultipleDaysWithLimitAndOrderByUsingMathExpres TestHelper.assertExpectedObjects( Iterables.limit(expectedResults, limit), - mergeRunner.run(QueryPlus.wrap(fullQuery)), + mergeRunner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(fullQuery))), StringUtils.format("limit: %d", limit) ); } @@ -4114,15 +4127,23 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r return new MergeSequence( queryPlus.getQuery().getResultOrdering(), Sequences.simple( - Arrays.asList(runner.run(queryPlus1, responseContext), runner.run(queryPlus2, responseContext)) + Arrays.asList( + Sequences.simple(runner.run(queryPlus1, responseContext).toList()), + Sequences.simple(runner.run(queryPlus2, responseContext).toList()) + ) ) ); } - } + }, + true ); final GroupByQuery query = baseQuery.withLimitSpec(limitSpec); - TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(query)), "merged"); + TestHelper.assertExpectedObjects( + expectedResults, + mergedRunner.run(GroupByQueryRunnerTestHelper.populateResourceId(QueryPlus.wrap(query))), + "merged" + ); } @Test @@ -4151,12 +4172,18 @@ public void testGroupByOrderLimit() makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L) ); - QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query)), "no-limit"); + QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner, true); + TestHelper.assertExpectedObjects( + expectedResults, + mergeRunner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(query))), + "no-limit" + ); TestHelper.assertExpectedObjects( Iterables.limit(expectedResults, 5), - mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build())), + mergeRunner.run(QueryPlus.wrap(builder.setLimit(5) + .overrideContext(GroupByQueryRunnerTestHelper.defaultResourceIdMap()) + .build())), "limited" ); @@ -4244,10 +4271,16 @@ public void testGroupByWithOrderLimit2() ); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query)), "no-limit"); + TestHelper.assertExpectedObjects( + expectedResults, + mergeRunner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(query))), + "no-limit" + ); TestHelper.assertExpectedObjects( Iterables.limit(expectedResults, 5), - mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build())), + mergeRunner.run(QueryPlus.wrap(builder.setLimit(5) + .overrideContext(GroupByQueryRunnerTestHelper.defaultResourceIdMap()) + .build())), "limited" ); } @@ -4281,10 +4314,16 @@ public void testGroupByWithOrderLimit3() ); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query)), "no-limit"); + TestHelper.assertExpectedObjects( + expectedResults, + mergeRunner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(query))), + "no-limit" + ); TestHelper.assertExpectedObjects( Iterables.limit(expectedResults, 5), - mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build())), + mergeRunner.run(QueryPlus.wrap(builder.setLimit(5) + .overrideContext(GroupByQueryRunnerTestHelper.defaultResourceIdMap()) + .build())), "limited" ); } @@ -4324,10 +4363,16 @@ public void testGroupByOrderLimitNumeric() ); QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query)), "no-limit"); + TestHelper.assertExpectedObjects( + expectedResults, + mergeRunner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(query))), + "no-limit" + ); TestHelper.assertExpectedObjects( Iterables.limit(expectedResults, 5), - mergeRunner.run(QueryPlus.wrap(builder.setLimit(5).build())), + mergeRunner.run(QueryPlus.wrap(builder.setLimit(5) + .overrideContext(GroupByQueryRunnerTestHelper.defaultResourceIdMap()) + .build())), "limited" ); } @@ -4875,14 +4920,22 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r return new MergeSequence( queryPlus.getQuery().getResultOrdering(), Sequences.simple( - Arrays.asList(runner.run(queryPlus1, responseContext), runner.run(queryPlus2, responseContext)) + Arrays.asList( + Sequences.simple(runner.run(queryPlus1, responseContext).toList()), + Sequences.simple(runner.run(queryPlus2, responseContext).toList()) + ) ) ); } - } + }, + true ); - TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(fullQuery)), "merged"); + TestHelper.assertExpectedObjects( + expectedResults, + mergedRunner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(fullQuery))), + "merged" + ); } @Test @@ -5317,14 +5370,21 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r return new MergeSequence( queryPlus.getQuery().getResultOrdering(), Sequences.simple( - Arrays.asList(runner.run(queryPlus1, responseContext), runner.run(queryPlus2, responseContext)) + Arrays.asList( + Sequences.simple(runner.run(queryPlus1, responseContext).toList()), + Sequences.simple(runner.run(queryPlus2, responseContext).toList()) + ) ) ); } } ); - TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(QueryPlus.wrap(fullQuery)), "merged"); + TestHelper.assertExpectedObjects( + expectedResults, + mergedRunner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(fullQuery))), + "merged" + ); } @Test @@ -5398,22 +5458,27 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r return new MergeSequence( queryPlus.getQuery().getResultOrdering(), Sequences.simple( - Arrays.asList(runner.run(queryPlus1, responseContext), runner.run(queryPlus2, responseContext)) + Arrays.asList( + Sequences.simple(runner.run(queryPlus1, responseContext).toList()), + Sequences.simple(runner.run(queryPlus2, responseContext).toList()) + ) ) ); } } ); - ResponseContext context = ResponseContext.createEmpty(); // add an extra layer of merging, simulate broker forwarding query to historical TestHelper.assertExpectedObjects( expectedResults, factory.getToolchest().postMergeQueryDecoration( factory.getToolchest().mergeResults( - factory.getToolchest().preMergeQueryDecoration(mergedRunner) + (queryPlus, responseContext) -> + factory.getToolchest() + .preMergeQueryDecoration(mergedRunner) + .run(GroupByQueryRunnerTestHelper.populateResourceId(queryPlus), responseContext) ) - ).run(QueryPlus.wrap(query)), + ).run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(query))), "merged" ); @@ -5421,9 +5486,12 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r expectedResults, factory.getToolchest().postMergeQueryDecoration( factory.getToolchest().mergeResults( - factory.getToolchest().preMergeQueryDecoration(mergedRunner) + (queryPlus, responseContext) -> + factory.getToolchest() + .preMergeQueryDecoration(mergedRunner) + .run(GroupByQueryRunnerTestHelper.populateResourceId(queryPlus), responseContext) ) - ).run(QueryPlus.wrap(expressionQuery)), + ).run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(expressionQuery))), "merged" ); } @@ -5485,8 +5553,12 @@ public void testGroupByWithRegEx() makeRow(query, "2011-04-01", "quality", "automotive", "rows", 2L) ); - QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query)), "no-limit"); + QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner, true); + TestHelper.assertExpectedObjects( + expectedResults, + mergeRunner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(query))), + "no-limit" + ); } @Test @@ -5540,8 +5612,12 @@ public void testGroupByWithNonexistentDimension() makeRow(query, "2011-04-01", "billy", null, "quality", "travel", "rows", 2L) ); - QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); - TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(QueryPlus.wrap(query)), "no-limit"); + QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner, true); + TestHelper.assertExpectedObjects( + expectedResults, + mergeRunner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(query))), + "no-limit" + ); } // A subquery identical to the query should yield identical results @@ -11155,15 +11231,14 @@ public void testMergeLimitPushDownResultsWithLongDimensionNotInLimitSpec() queryPlus3.getQuery().getResultOrdering(), Sequences.simple( Arrays.asList( - runner.run(queryPlus1, responseContext1), - runner.run(queryPlus2, responseContext1) + Sequences.simple(runner.run(queryPlus1, responseContext1).toList()), + Sequences.simple(runner.run(queryPlus2, responseContext1).toList()) ) ) ) - ).run(queryPlus, responseContext); + ).run(GroupByQueryRunnerTestHelper.populateResourceId(queryPlus), ResponseContext.createEmpty()); } ); - Map context = new HashMap<>(); List allGranExpectedResults = Arrays.asList( makeRow(allGranQuery, "2011-04-02", "qualityLen", 4L, "rows", 2L), makeRow(allGranQuery, "2011-04-02", "qualityLen", 6L, "rows", 4L), @@ -11176,7 +11251,7 @@ public void testMergeLimitPushDownResultsWithLongDimensionNotInLimitSpec() TestHelper.assertExpectedObjects( allGranExpectedResults, - mergedRunner.run(QueryPlus.wrap(allGranQuery)), + mergedRunner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(allGranQuery))), "merged" ); } @@ -11223,16 +11298,15 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r queryPlus3.getQuery().getResultOrdering(), Sequences.simple( Arrays.asList( - runner.run(queryPlus1, responseContext1), - runner.run(queryPlus2, responseContext1) + Sequences.simple(runner.run(queryPlus1, responseContext1).toList()), + Sequences.simple(runner.run(queryPlus2, responseContext1).toList()) ) ) ) - ).run(queryPlus, responseContext); + ).run(GroupByQueryRunnerTestHelper.populateResourceId(queryPlus), responseContext); } } ); - Map context = new HashMap<>(); List allGranExpectedResults = Arrays.asList( makeRow(allGranQuery, "2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L), makeRow(allGranQuery, "2011-04-02", "alias", "technology", "rows", 2L, "idx", 177L), @@ -11243,7 +11317,7 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r TestHelper.assertExpectedObjects( allGranExpectedResults, - mergedRunner.run(QueryPlus.wrap(allGranQuery)), + mergedRunner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(allGranQuery))), "merged" ); } @@ -11290,14 +11364,15 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r queryPlus3.getQuery().getResultOrdering(), Sequences.simple( Arrays.asList( - runner.run(queryPlus1, responseContext1), - runner.run(queryPlus2, responseContext1) + Sequences.simple(runner.run(queryPlus1, responseContext1).toList()), + Sequences.simple(runner.run(queryPlus2, responseContext1).toList()) ) ) ) - ).run(queryPlus, responseContext); + ).run(GroupByQueryRunnerTestHelper.populateResourceId(queryPlus), responseContext); } - } + }, + true ); List allGranExpectedResults = Arrays.asList( @@ -11308,7 +11383,8 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r makeRow(allGranQuery, "2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L) ); - Iterable results = mergedRunner.run(QueryPlus.wrap(allGranQuery)).toList(); + Iterable results = + mergedRunner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(allGranQuery))).toList(); TestHelper.assertExpectedObjects(allGranExpectedResults, results, "merged"); } @@ -11350,17 +11426,22 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r ) ); - return factory.getToolchest().mergeResults( + return Sequences.simple(factory.getToolchest().mergeResults( (queryPlus3, responseContext1) -> new MergeSequence<>( queryPlus3.getQuery().getResultOrdering(), Sequences.simple( Arrays.asList( - runner.run(queryPlus1, responseContext1), - runner.run(queryPlus2, responseContext1) + Sequences.simple(runner.run(queryPlus1, responseContext1).toList()), + Sequences.simple(runner.run(queryPlus2, responseContext1).toList()) ) ) ) - ).run(queryPlus, responseContext); + ).run( + queryPlus.withQuery( + GroupByQueryRunnerTestHelper.populateResourceId(queryPlus.getQuery()) + ), + responseContext + ).toList()); } } ); @@ -11371,7 +11452,9 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r makeRow(allGranQuery, "2011-04-02", "alias", "travel", "rows", 2L, "idx", 243L) ); - Iterable results = mergedRunner.run(QueryPlus.wrap(allGranQuery)).toList(); + Iterable results = mergedRunner.run( + QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(allGranQuery)) + ).toList(); TestHelper.assertExpectedObjects(allGranExpectedResults, results, "merged"); } @@ -11422,12 +11505,12 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r queryPlus3.getQuery().getResultOrdering(), Sequences.simple( Arrays.asList( - runner.run(queryPlus1, responseContext1), - runner.run(queryPlus2, responseContext1) + Sequences.simple(runner.run(queryPlus1, responseContext1).toList()), + Sequences.simple(runner.run(queryPlus2, responseContext1).toList()) ) ) ) - ).run(queryPlus, responseContext); + ).run(GroupByQueryRunnerTestHelper.populateResourceId(queryPlus), responseContext); } } ); @@ -11440,7 +11523,9 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r makeRow(allGranQuery, "2011-04-02", "alias", "premium", "market", "spot", "rows", 2L, "idx", 257L) ); - Iterable results = mergedRunner.run(QueryPlus.wrap(allGranQuery)).toList(); + Iterable results = + mergedRunner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(allGranQuery))) + .toList(); TestHelper.assertExpectedObjects(allGranExpectedResults, results, "merged"); } @@ -11504,12 +11589,12 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r queryPlus3.getQuery().getResultOrdering(), Sequences.simple( Arrays.asList( - runner.run(queryPlus1, responseContext1), - runner.run(queryPlus2, responseContext1) + Sequences.simple(runner.run(queryPlus1, responseContext1).toList()), + Sequences.simple(runner.run(queryPlus2, responseContext1).toList()) ) ) ) - ).run(queryPlus, responseContext); + ).run(GroupByQueryRunnerTestHelper.populateResourceId(queryPlus), responseContext); } } ); @@ -11522,7 +11607,8 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r makeRow(allGranQuery, "2011-04-02", "alias", "premium", "market", "spot", "rows", 2L, "idx", 257L) ); - Iterable results = mergedRunner.run(QueryPlus.wrap(allGranQuery)).toList(); + Iterable results = + mergedRunner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(allGranQuery))).toList(); TestHelper.assertExpectedObjects(allGranExpectedResults, results, "merged"); } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTestHelper.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTestHelper.java index 5001ccdc917c..049b00ca1159 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTestHelper.java @@ -26,6 +26,7 @@ import org.apache.druid.query.FinalizeResultsQueryRunner; import org.apache.druid.query.MetricsEmittingQueryRunner; 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.QueryRunnerFactory; @@ -36,7 +37,10 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; +import java.util.Map; +import java.util.UUID; /** * @@ -51,7 +55,7 @@ public static Iterable runQuery(QueryRunnerFactory factory, QueryRunner r toolChest ); - Sequence queryResult = theRunner.run(QueryPlus.wrap(query)); + Sequence queryResult = theRunner.run(QueryPlus.wrap(populateResourceId(query))); return queryResult.toList(); } @@ -76,7 +80,7 @@ public static Iterable runQueryWithEmitter( toolChest ); - return theRunner.run(QueryPlus.wrap(query)).toList(); + return theRunner.run(QueryPlus.wrap(populateResourceId(query))).toList(); } public static ResultRow createExpectedRow(final GroupByQuery query, final String timestamp, Object... vals) @@ -136,4 +140,18 @@ public static List createExpectedRows( return expected; } + public static QueryPlus populateResourceId(QueryPlus queryPlus) + { + return queryPlus.withQuery(populateResourceId(queryPlus.getQuery())); + } + + public static Query populateResourceId(Query query) + { + return query.withOverriddenContext(defaultResourceIdMap()); + } + + public static Map defaultResourceIdMap() + { + return Collections.singletonMap(QueryContexts.QUERY_RESOURCE_ID, UUID.randomUUID().toString()); + } } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java index ee89ae40fa9f..d8a609800bb7 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java @@ -106,6 +106,7 @@ public static Iterable constructorFeeder() @Override public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { + queryPlus = GroupByQueryRunnerTestHelper.populateResourceId(queryPlus); final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); TimeseriesQuery tsQuery = (TimeseriesQuery) queryPlus.getQuery(); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java index 06802b55694c..5a187582278c 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java @@ -43,6 +43,7 @@ import org.apache.druid.js.JavaScriptConfig; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.BySegmentQueryRunner; +import org.apache.druid.query.DirectQueryProcessingPool; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.FinalizeResultsQueryRunner; import org.apache.druid.query.Query; @@ -286,11 +287,13 @@ public String getFormatString() }; final Supplier configSupplier = Suppliers.ofInstance(config); + GroupByResourcesReservationPool groupByResourcesReservationPool = new GroupByResourcesReservationPool(mergePool, config); + GroupByResourcesReservationPool groupByResourcesReservationPool2 = new GroupByResourcesReservationPool(mergePool2, config); final GroupingEngine engine1 = new GroupingEngine( druidProcessingConfig, configSupplier, bufferPool, - mergePool, + groupByResourcesReservationPool, TestHelper.makeJsonMapper(), new ObjectMapper(new SmileFactory()), NOOP_QUERYWATCHER @@ -299,7 +302,7 @@ public String getFormatString() druidProcessingConfig, configSupplier, bufferPool, - mergePool2, + groupByResourcesReservationPool2, TestHelper.makeJsonMapper(), new ObjectMapper(new SmileFactory()), NOOP_QUERYWATCHER @@ -307,12 +310,12 @@ public String getFormatString() groupByFactory = new GroupByQueryRunnerFactory( engine1, - new GroupByQueryQueryToolChest(engine1) + new GroupByQueryQueryToolChest(engine1, groupByResourcesReservationPool) ); groupByFactory2 = new GroupByQueryRunnerFactory( engine2, - new GroupByQueryQueryToolChest(engine2) + new GroupByQueryQueryToolChest(engine2, groupByResourcesReservationPool2) ); } @@ -700,17 +703,20 @@ private Sequence runNestedQueryWithForcePushDown(GroupByQuery nestedQ { ResponseContext context = ResponseContext.createEmpty(); QueryToolChest toolChest = groupByFactory.getToolchest(); + QueryToolChest toolChest2 = groupByFactory2.getToolchest(); GroupByQuery pushDownQuery = nestedQuery; QueryRunner segment1Runner = new FinalizeResultsQueryRunner( toolChest.mergeResults( - groupByFactory.mergeRunners(executorService, getQueryRunnerForSegment1()) + groupByFactory.mergeRunners(DirectQueryProcessingPool.INSTANCE, getQueryRunnerForSegment1()), + true ), (QueryToolChest) toolChest ); QueryRunner segment2Runner = new FinalizeResultsQueryRunner( - toolChest.mergeResults( - groupByFactory2.mergeRunners(executorService, getQueryRunnerForSegment2()) + toolChest2.mergeResults( + groupByFactory2.mergeRunners(DirectQueryProcessingPool.INSTANCE, getQueryRunnerForSegment2()), + true ), (QueryToolChest) toolChest ); @@ -721,14 +727,16 @@ private Sequence runNestedQueryWithForcePushDown(GroupByQuery nestedQ .simple( ImmutableList.of( Sequences.map( - segment1Runner.run(queryPlus, responseContext), + segment1Runner + .run(GroupByQueryRunnerTestHelper.populateResourceId(queryPlus), responseContext), toolChest.makePreComputeManipulatorFn( (GroupByQuery) queryPlus.getQuery(), MetricManipulatorFns.deserializing() ) ), Sequences.map( - segment2Runner.run(queryPlus, responseContext), + segment2Runner + .run(GroupByQueryRunnerTestHelper.populateResourceId(queryPlus), responseContext), toolChest.makePreComputeManipulatorFn( (GroupByQuery) queryPlus.getQuery(), MetricManipulatorFns.deserializing() @@ -736,7 +744,8 @@ private Sequence runNestedQueryWithForcePushDown(GroupByQuery nestedQ ) ) ) - .flatMerge(Function.identity(), queryPlus.getQuery().getResultOrdering()) + .flatMerge(Function.identity(), queryPlus.getQuery().getResultOrdering()), + true ), (QueryToolChest) toolChest ); @@ -748,12 +757,12 @@ private Sequence runNestedQueryWithForcePushDown(GroupByQuery nestedQ )); Sequence pushDownQueryResults = groupingEngine.mergeResults( queryRunnerForSegments, - queryWithPushDownDisabled, + (GroupByQuery) GroupByQueryRunnerTestHelper.populateResourceId(queryWithPushDownDisabled), context ); return toolChest.mergeResults((queryPlus, responseContext) -> pushDownQueryResults) - .run(QueryPlus.wrap(nestedQuery), context); + .run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(nestedQuery)), context); } @Test diff --git a/processing/src/test/java/org/apache/druid/query/groupby/UnnestGroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/UnnestGroupByQueryRunnerTest.java index 62caef2493a2..13a33191e8ee 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/UnnestGroupByQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/UnnestGroupByQueryRunnerTest.java @@ -158,16 +158,19 @@ public static GroupByQueryRunnerFactory makeQueryRunnerFactory( ); } final Supplier configSupplier = Suppliers.ofInstance(config); + GroupByResourcesReservationPool groupByResourcesReservationPool = + new GroupByResourcesReservationPool(bufferPools.getMergePool(), config); final GroupingEngine groupingEngine = new GroupingEngine( processingConfig, configSupplier, bufferPools.getProcessingPool(), - bufferPools.getMergePool(), + groupByResourcesReservationPool, TestHelper.makeJsonMapper(), mapper, QueryRunnerTestHelper.NOOP_QUERYWATCHER ); - final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(groupingEngine); + final GroupByQueryQueryToolChest toolChest = + new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool); return new GroupByQueryRunnerFactory(groupingEngine, toolChest); } diff --git a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java index 8f987c76f4d1..306ed04f634c 100644 --- a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java @@ -109,7 +109,7 @@ public SearchQueryRunnerTest( this.runner = runner; this.decoratedRunner = FluentQueryRunner.create(runner, TOOL_CHEST) .applyPreMergeDecoration() - .mergeResults() + .mergeResults(true) .applyPostMergeDecoration(); } diff --git a/server/src/main/java/org/apache/druid/guice/BrokerProcessingModule.java b/server/src/main/java/org/apache/druid/guice/BrokerProcessingModule.java index 0fcb4785191e..d70a2157e154 100644 --- a/server/src/main/java/org/apache/druid/guice/BrokerProcessingModule.java +++ b/server/src/main/java/org/apache/druid/guice/BrokerProcessingModule.java @@ -46,6 +46,8 @@ import org.apache.druid.query.ExecutorServiceMonitor; import org.apache.druid.query.ForwardingQueryProcessingPool; import org.apache.druid.query.QueryProcessingPool; +import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.groupby.GroupByResourcesReservationPool; import org.apache.druid.server.metrics.MetricsModule; import org.apache.druid.utils.JvmUtils; @@ -132,6 +134,17 @@ public BlockingPool getMergeBufferPool(DruidProcessingConfig config) ); } + @Provides + @LazySingleton + @Merging + public GroupByResourcesReservationPool getGroupByResourcesReservationPool( + @Merging BlockingPool mergeBufferPool, + GroupByQueryConfig groupByQueryConfig + ) + { + return new GroupByResourcesReservationPool(mergeBufferPool, groupByQueryConfig); + } + @Provides @ManageLifecycle public LifecycleForkJoinPoolProvider getMergeProcessingPoolProvider(BrokerParallelMergeConfig config) diff --git a/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java b/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java index 76cc855cd224..a2daa25e214a 100644 --- a/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java +++ b/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java @@ -46,6 +46,8 @@ import org.apache.druid.query.MetricsEmittingQueryProcessingPool; import org.apache.druid.query.PrioritizedExecutorService; import org.apache.druid.query.QueryProcessingPool; +import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.groupby.GroupByResourcesReservationPool; import org.apache.druid.server.metrics.MetricsModule; import org.apache.druid.utils.JvmUtils; @@ -133,6 +135,17 @@ public BlockingPool getMergeBufferPool(DruidProcessingConfig config) ); } + @Provides + @LazySingleton + @Merging + public GroupByResourcesReservationPool getGroupByResourcesReservationPool( + @Merging BlockingPool mergeBufferPool, + GroupByQueryConfig groupByQueryConfig + ) + { + return new GroupByResourcesReservationPool(mergeBufferPool, groupByQueryConfig); + } + private void verifyDirectMemory(DruidProcessingConfig config) { try { diff --git a/server/src/main/java/org/apache/druid/guice/RouterProcessingModule.java b/server/src/main/java/org/apache/druid/guice/RouterProcessingModule.java index dc68cbb4bffa..f76b5ed940d6 100644 --- a/server/src/main/java/org/apache/druid/guice/RouterProcessingModule.java +++ b/server/src/main/java/org/apache/druid/guice/RouterProcessingModule.java @@ -34,6 +34,8 @@ import org.apache.druid.query.ExecutorServiceMonitor; import org.apache.druid.query.ForwardingQueryProcessingPool; import org.apache.druid.query.QueryProcessingPool; +import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.groupby.GroupByResourcesReservationPool; import org.apache.druid.server.metrics.MetricsModule; import java.nio.ByteBuffer; @@ -87,4 +89,18 @@ public BlockingPool getMergeBufferPool(DruidProcessingConfig config) } return DummyBlockingPool.instance(); } + + /** + * Reservation pool injected with a dummy pool + */ + @Provides + @LazySingleton + @Merging + public GroupByResourcesReservationPool getGroupByResourcesReservationPool( + @Merging BlockingPool mergeBufferPool, + GroupByQueryConfig groupByQueryConfig + ) + { + return new GroupByResourcesReservationPool(mergeBufferPool, groupByQueryConfig); + } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index 1bf07fa41463..d02e200cfcbd 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -61,6 +61,7 @@ import org.apache.druid.segment.realtime.FireHydrant; import org.apache.druid.segment.realtime.plumber.Sink; import org.apache.druid.segment.realtime.plumber.SinkSegmentReference; +import org.apache.druid.server.ResourceIdPopulatingQueryRunner; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.PartitionChunk; @@ -333,18 +334,21 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final ); } - // 1) Merge results using the toolChest, finalize if necessary. - // 2) Measure CPU time of that operation. - // 3) Release all sink segment references. - return QueryRunnerHelper.makeClosingQueryRunner( - CPUTimeMetricQueryRunner.safeBuild( - new FinalizeResultsQueryRunner<>(toolChest.mergeResults(mergedRunner), toolChest), - toolChest, - emitter, - cpuTimeAccumulator, - true - ), - () -> CloseableUtils.closeAll(allSegmentReferences) + // 1) Populate resource id to the query + // 2) Merge results using the toolChest, finalize if necessary. + // 3) Measure CPU time of that operation. + // 4) Release all sink segment references. + return new ResourceIdPopulatingQueryRunner<>( + QueryRunnerHelper.makeClosingQueryRunner( + CPUTimeMetricQueryRunner.safeBuild( + new FinalizeResultsQueryRunner<>(toolChest.mergeResults(mergedRunner, true), toolChest), + toolChest, + emitter, + cpuTimeAccumulator, + true + ), + () -> CloseableUtils.closeAll(allSegmentReferences) + ) ); } catch (Throwable e) { 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 c8525771ceb5..1d3b38b2fdbd 100644 --- a/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java @@ -58,7 +58,6 @@ import org.apache.druid.query.RetryQueryRunnerConfig; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.TableDataSource; -import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.join.JoinableFactory; @@ -171,7 +170,7 @@ public ClientQuerySegmentWalker( } @Override - public QueryRunner getQueryRunnerForIntervals(Query query, Iterable intervals) + public QueryRunner getQueryRunnerForIntervals(final Query query, final Iterable intervals) { final QueryToolChest> toolChest = warehouse.getToolChest(query); @@ -185,6 +184,8 @@ public QueryRunner getQueryRunnerForIntervals(Query query, Iterable QueryRunner getQueryRunnerForIntervals(Query query, Iterable QueryRunner getQueryRunnerForSegments(Query query, Iterable specs) + public QueryRunner getQueryRunnerForSegments(final Query query, final Iterable specs) { + // Inlining isn't done for segments-based queries, but we still globalify the table datasources if possible - final Query freeTradeQuery = query.withDataSource(globalizeIfPossible(query.getDataSource())); + Query freeTradeQuery = query.withDataSource(globalizeIfPossible(query.getDataSource())); + + freeTradeQuery = ResourceIdPopulatingQueryRunner.populateResourceId(freeTradeQuery); if (canRunQueryUsingClusterWalker(query)) { return new QuerySwappingQueryRunner<>( @@ -519,7 +523,7 @@ private QueryRunner decorateClusterRunner(Query query, QueryRunner return FluentQueryRunner .create(baseRunner, toolChest) .applyPreMergeDecoration() - .mergeResults() + .mergeResults(false) .applyPostMergeDecoration() .emitCPUTimeMetric(emitter) .postProcess( @@ -843,33 +847,4 @@ private static String rowLimitExceededMessage(final int rowLimitUsed) ); } - /** - * A {@link QueryRunner} which validates that a *specific* query is passed in, and then swaps it with another one. - * Useful since the inlining we do relies on passing the modified query to the underlying {@link QuerySegmentWalker}, - * and callers of {@link #getQueryRunnerForIntervals} aren't able to do this themselves. - */ - private static class QuerySwappingQueryRunner implements QueryRunner - { - private final QueryRunner baseRunner; - private final Query query; - private final Query newQuery; - - public QuerySwappingQueryRunner(QueryRunner baseRunner, Query query, Query newQuery) - { - this.baseRunner = baseRunner; - this.query = query; - this.newQuery = newQuery; - } - - @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) - { - //noinspection ObjectEquality - if (queryPlus.getQuery() != query) { - throw new ISE("Unexpected query received"); - } - - return baseRunner.run(queryPlus.withQuery(newQuery), responseContext); - } - } } diff --git a/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java index 1d0eab3351e9..ae6b67deb5f7 100644 --- a/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java @@ -115,7 +115,7 @@ public QueryRunner getQueryRunnerForIntervals(final Query query, final return FluentQueryRunner .create(scheduler.wrapQueryRunner(baseRunner), queryRunnerFactory.getToolchest()) .applyPreMergeDecoration() - .mergeResults() + .mergeResults(true) .applyPostMergeDecoration() .emitCPUTimeMetric(emitter, cpuAccumulator); } diff --git a/server/src/main/java/org/apache/druid/server/QuerySwappingQueryRunner.java b/server/src/main/java/org/apache/druid/server/QuerySwappingQueryRunner.java new file mode 100644 index 000000000000..51c140be037c --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/QuerySwappingQueryRunner.java @@ -0,0 +1,62 @@ +/* + * 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.server; + +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QuerySegmentWalker; +import org.apache.druid.query.context.ResponseContext; + +/** + * A {@link QueryRunner} which validates that a *specific* query is passed in, and then swaps it with another one. + * Useful in passing the modified query to the underlying runners, since the {@link QuerySegmentWalker#} + * Useful since walkers might need to enrich the query with additional parameters and the callers actually calling the + * `run()` won't know about this modification. + * + * It validates that the query passed to the `run()` was the same query that was passed while creating the runner, + * to ensure that the enrichment was done to the correct query. + */ +public class QuerySwappingQueryRunner implements QueryRunner +{ + private final QueryRunner baseRunner; + private final Query query; + private final Query newQuery; + + public QuerySwappingQueryRunner(QueryRunner baseRunner, Query query, Query newQuery) + { + this.baseRunner = baseRunner; + this.query = query; + this.newQuery = newQuery; + } + + @Override + public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) + { + //noinspection ObjectEquality + if (queryPlus.getQuery() != query) { + throw DruidException.defensive("Unexpected query received"); + } + + return baseRunner.run(queryPlus.withQuery(newQuery), responseContext); + } +} diff --git a/server/src/main/java/org/apache/druid/server/ResourceIdPopulatingQueryRunner.java b/server/src/main/java/org/apache/druid/server/ResourceIdPopulatingQueryRunner.java new file mode 100644 index 000000000000..a9e4250bf0b0 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/ResourceIdPopulatingQueryRunner.java @@ -0,0 +1,68 @@ +/* + * 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.server; + +import org.apache.druid.java.util.common.guava.Sequence; +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.context.ResponseContext; + +import java.util.Collections; +import java.util.UUID; + +/** + * Populates {@link QueryContexts#QUERY_RESOURCE_ID} in the query context + */ +public class ResourceIdPopulatingQueryRunner implements QueryRunner +{ + private final QueryRunner baseRunner; + + public ResourceIdPopulatingQueryRunner(QueryRunner baseRunner) + { + this.baseRunner = baseRunner; + } + + /** + * Assigns a random resource id to the given query + */ + public static Query populateResourceId(Query query) + { + return query.withOverriddenContext(Collections.singletonMap( + QueryContexts.QUERY_RESOURCE_ID, + UUID.randomUUID().toString() + )); + } + + @Override + public Sequence run( + final QueryPlus queryPlus, + final ResponseContext responseContext + ) + { + return baseRunner.run( + queryPlus.withQuery( + populateResourceId(queryPlus.getQuery()) + ), + responseContext + ); + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java index c932f1bcc097..d47feea1ce79 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java @@ -61,6 +61,7 @@ import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.join.JoinableFactoryWrapper; +import org.apache.druid.server.ResourceIdPopulatingQueryRunner; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.SetAndVerifyContextQueryRunner; import org.apache.druid.server.initialization.ServerConfig; @@ -164,15 +165,16 @@ public QueryRunner getQueryRunnerForIntervals(Query query, Iterable QueryRunner getQueryRunnerForSegments(Query query, Iterable specs) + public QueryRunner getQueryRunnerForSegments(Query theQuery, Iterable specs) { - final DataSource dataSourceFromQuery = query.getDataSource(); - final QueryRunnerFactory> factory = conglomerate.findFactory(query); + final Query newQuery = ResourceIdPopulatingQueryRunner.populateResourceId(theQuery); + final DataSource dataSourceFromQuery = newQuery.getDataSource(); + final QueryRunnerFactory> factory = conglomerate.findFactory(newQuery); if (factory == null) { final QueryUnsupportedException e = new QueryUnsupportedException( - StringUtils.format("Unknown query type, [%s]", query.getClass()) + StringUtils.format("Unknown query type, [%s]", newQuery.getClass()) ); - log.makeAlert(e, "Error while executing a query[%s]", query.getId()) + log.makeAlert(e, "Error while executing a query[%s]", newQuery.getId()) .addData("dataSource", dataSourceFromQuery) .emit(); throw e; @@ -199,7 +201,7 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable segmentMapFn = dataSourceFromQuery - .createSegmentMapFunction(query, cpuTimeAccumulator); + .createSegmentMapFunction(newQuery, cpuTimeAccumulator); // We compute the datasource's cache key here itself so it doesn't need to be re-computed for every segment final Optional cacheKeyPrefix = Optional.ofNullable(dataSourceFromQuery.getCacheKey()); @@ -209,7 +211,7 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable Collections.singletonList( buildQueryRunnerForSegment( - query, + newQuery, descriptor, factory, toolChest, @@ -221,15 +223,17 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable( - toolChest.mergeResults(factory.mergeRunners(queryProcessingPool, queryRunners)), - toolChest - ), - toolChest, - emitter, - cpuTimeAccumulator, - true + return new ResourceIdPopulatingQueryRunner<>( + CPUTimeMetricQueryRunner.safeBuild( + new FinalizeResultsQueryRunner<>( + toolChest.mergeResults(factory.mergeRunners(queryProcessingPool, queryRunners), true), + toolChest + ), + toolChest, + emitter, + cpuTimeAccumulator, + true + ) ); } 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 91cd39e54a59..818c6b77a532 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -1070,7 +1070,7 @@ private FluentQueryRunner makeTopNQueryRunner() return FluentQueryRunner .create(getDefaultQueryRunner(), new TopNQueryQueryToolChest(new TopNQueryConfig())) .applyPreMergeDecoration() - .mergeResults() + .mergeResults(true) .applyPostMergeDecoration(); } diff --git a/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java b/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java index a5ba6e75bd2d..7afdc948a7a2 100644 --- a/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java +++ b/server/src/test/java/org/apache/druid/discovery/DataServerClientTest.java @@ -106,7 +106,7 @@ public void testFetchSegmentFromDataServer() throws JsonProcessingException jsonMapper.writeValueAsBytes(Collections.singletonList(scanResultValue)) ); - ResponseContext responseContext = new DefaultResponseContext(); + ResponseContext responseContext = DefaultResponseContext.createEmpty(); Sequence result = target.run( query, responseContext, 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 ae808e2eb20d..c3863708a66d 100644 --- a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java +++ b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java @@ -407,6 +407,40 @@ public void testGroupByOnGroupByOnTable() Assert.assertEquals(1, scheduler.getTotalReleased().get()); } + @Test + public void testGroupByOnGroupByOnInlineTable() + { + final GroupByQuery subquery = + (GroupByQuery) GroupByQuery.builder() + .setDataSource(FOO_INLINE) + .setGranularity(Granularities.ALL) + .setInterval(Collections.singletonList(INTERVAL)) + .setDimensions(DefaultDimensionSpec.of("s")) + .build() + .withId("queryId"); + + final GroupByQuery query = + (GroupByQuery) GroupByQuery.builder() + .setDataSource(new QueryDataSource(subquery)) + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ONLY_ETERNITY) + .setAggregatorSpecs(new CountAggregatorFactory("cnt")) + .build() + .withId(DUMMY_QUERY_ID); + + testQuery( + query, + // GroupBy handles its own subqueries; only the inner one will go to the cluster. Also, it gets a subquery id + ImmutableList.of(ExpectedQuery.local(query.withDataSource(new QueryDataSource(subquery.withSubQueryId("1.1"))))), + ImmutableList.of(new Object[]{3L}) + ); + + Assert.assertEquals(1, scheduler.getTotalRun().get()); + Assert.assertEquals(0, scheduler.getTotalPrioritizedAndLaned().get()); + Assert.assertEquals(1, scheduler.getTotalAcquired().get()); + Assert.assertEquals(1, scheduler.getTotalReleased().get()); + } + @Test public void testGroupByOnUnionOfTwoTables() { @@ -1505,6 +1539,7 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable modifiedQuery; // Need to blast various parameters that will vary and aren't important to test for. - modifiedQuery = query.withOverriddenContext( - ImmutableMap.builder() - .put(DirectDruidClient.QUERY_FAIL_TIME, 0L) - .put(QueryContexts.DEFAULT_TIMEOUT_KEY, 0L) - .put(QueryContexts.FINALIZE_KEY, true) - .put(QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, 0L) - .put(GroupByQuery.CTX_KEY_SORT_BY_DIMS_FIRST, false) - .put(GroupByQueryConfig.CTX_KEY_ARRAY_RESULT_ROWS, true) - .put(GroupByQueryConfig.CTX_KEY_APPLY_LIMIT_PUSH_DOWN, true) - .put(GroupingEngine.CTX_KEY_OUTERMOST, true) - .put(GroupingEngine.CTX_KEY_FUDGE_TIMESTAMP, "1979") - .build() - ); + ImmutableMap.Builder contextBuilder = ImmutableMap.builder(); + contextBuilder.put(DirectDruidClient.QUERY_FAIL_TIME, 0L) + .put(QueryContexts.DEFAULT_TIMEOUT_KEY, 0L) + .put(QueryContexts.FINALIZE_KEY, true) + .put(QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, 0L) + .put(GroupByQuery.CTX_KEY_SORT_BY_DIMS_FIRST, false) + .put(GroupByQueryConfig.CTX_KEY_ARRAY_RESULT_ROWS, true) + .put(GroupByQueryConfig.CTX_KEY_APPLY_LIMIT_PUSH_DOWN, true) + .put(GroupingEngine.CTX_KEY_OUTERMOST, true) + .put(GroupingEngine.CTX_KEY_FUDGE_TIMESTAMP, "1979") + .put(QueryContexts.QUERY_RESOURCE_ID, "dummy"); + + modifiedQuery = query.withOverriddenContext(contextBuilder.build()); if (modifiedQuery.getDataSource() instanceof FrameBasedInlineDataSource) { // Do round-trip serialization in order to replace FrameBasedInlineDataSource with InlineDataSource, so diff --git a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java index e8443d12f7ae..d7af61370be6 100644 --- a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java +++ b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java @@ -58,6 +58,7 @@ import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryRunnerTest; +import org.apache.druid.query.groupby.GroupByQueryRunnerTestHelper; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.having.HavingSpec; import org.apache.druid.query.topn.TopNQuery; @@ -791,10 +792,12 @@ private ListenableFuture makeMergingQueryFuture( toolChest ) .applyPreMergeDecoration() - .mergeResults() + .mergeResults(true) .applyPostMergeDecoration(); - final int actualNumRows = consumeAndCloseSequence(runner.run(QueryPlus.wrap(query))); + final int actualNumRows = consumeAndCloseSequence( + runner.run(QueryPlus.wrap(GroupByQueryRunnerTestHelper.populateResourceId(query))) + ); Assert.assertEquals(actualNumRows, numRows); } catch (IOException ex) { 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 9be963c31606..494ef763a783 100644 --- a/server/src/test/java/org/apache/druid/server/QueryStackTests.java +++ b/server/src/test/java/org/apache/druid/server/QueryStackTests.java @@ -161,10 +161,11 @@ public static TestClusterQuerySegmentWalker createClusterQuerySegmentWalker( Map> timelines, QueryRunnerFactoryConglomerate conglomerate, @Nullable QueryScheduler scheduler, + GroupByQueryConfig groupByQueryConfig, Injector injector ) { - return new TestClusterQuerySegmentWalker(timelines, conglomerate, scheduler, injector.getInstance(EtagProvider.KEY)); + return new TestClusterQuerySegmentWalker(timelines, conglomerate, scheduler, groupByQueryConfig, injector.getInstance(EtagProvider.KEY)); } public static LocalQuerySegmentWalker createLocalQuerySegmentWalker( diff --git a/server/src/test/java/org/apache/druid/server/SpecificSegmentsQuerySegmentWalker.java b/server/src/test/java/org/apache/druid/server/SpecificSegmentsQuerySegmentWalker.java index a60d9fe325a2..cf301ceb81e7 100644 --- a/server/src/test/java/org/apache/druid/server/SpecificSegmentsQuerySegmentWalker.java +++ b/server/src/test/java/org/apache/druid/server/SpecificSegmentsQuerySegmentWalker.java @@ -32,6 +32,7 @@ import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.lookup.LookupExtractorFactoryContainer; import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; import org.apache.druid.segment.FrameBasedInlineSegmentWrangler; @@ -104,7 +105,9 @@ public static SpecificSegmentsQuerySegmentWalker createWalker( final QueryRunnerFactoryConglomerate conglomerate, final SegmentWrangler segmentWrangler, final JoinableFactoryWrapper joinableFactoryWrapper, - final QueryScheduler scheduler) + final QueryScheduler scheduler, + final GroupByQueryConfig groupByQueryConfig + ) { Map> timelines = new HashMap<>(); return new SpecificSegmentsQuerySegmentWalker( @@ -115,6 +118,7 @@ public static SpecificSegmentsQuerySegmentWalker createWalker( timelines, conglomerate, scheduler, + groupByQueryConfig, injector ), QueryStackTests.createLocalQuerySegmentWalker( @@ -134,7 +138,10 @@ public static SpecificSegmentsQuerySegmentWalker createWalker( * Create an instance without any lookups and with a default {@link JoinableFactory} that handles only inline * datasources. */ - public static SpecificSegmentsQuerySegmentWalker createWalker(final Injector injector, final QueryRunnerFactoryConglomerate conglomerate) + public static SpecificSegmentsQuerySegmentWalker createWalker( + final Injector injector, + final QueryRunnerFactoryConglomerate conglomerate + ) { return createWalker( injector, @@ -150,7 +157,8 @@ public static SpecificSegmentsQuerySegmentWalker createWalker(final Injector inj .build() ), new JoinableFactoryWrapper(QueryStackTests.makeJoinableFactoryForLookup(LOOKUP_EXTRACTOR_FACTORY_CONTAINER_PROVIDER)), - QueryStackTests.DEFAULT_NOOP_SCHEDULER + QueryStackTests.DEFAULT_NOOP_SCHEDULER, + new GroupByQueryConfig() ); } diff --git a/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java b/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java index 7c4133404ec9..21178fe140a7 100644 --- a/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java +++ b/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java @@ -33,6 +33,7 @@ import org.apache.druid.query.Queries; import org.apache.druid.query.Query; import org.apache.druid.query.QueryDataSource; +import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactory; import org.apache.druid.query.QueryRunnerFactoryConglomerate; @@ -42,6 +43,8 @@ import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.context.ResponseContext.Keys; +import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.groupby.GroupByQueryRunnerTestHelper; import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.query.spec.SpecificSegmentQueryRunner; import org.apache.druid.query.spec.SpecificSegmentSpec; @@ -75,18 +78,21 @@ public class TestClusterQuerySegmentWalker implements QuerySegmentWalker private final QueryRunnerFactoryConglomerate conglomerate; @Nullable private final QueryScheduler scheduler; + private final GroupByQueryConfig groupByQueryConfig; private final EtagProvider etagProvider; TestClusterQuerySegmentWalker( Map> timelines, QueryRunnerFactoryConglomerate conglomerate, @Nullable QueryScheduler scheduler, + GroupByQueryConfig groupByQueryConfig, EtagProvider etagProvider ) { this.timelines = timelines; this.conglomerate = conglomerate; this.scheduler = scheduler; + this.groupByQueryConfig = groupByQueryConfig; this.etagProvider = etagProvider; } @@ -149,8 +155,12 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final toolChest.postMergeQueryDecoration( toolChest.mergeResults( toolChest.preMergeQueryDecoration( - makeTableRunner(toolChest, factory, getSegmentsForTable(dataSourceName, specs), segmentMapFn) - ) + (queryPlus, responseContext) -> { + return makeTableRunner(toolChest, factory, getSegmentsForTable(dataSourceName, specs), segmentMapFn) + .run(GroupByQueryRunnerTestHelper.populateResourceId(queryPlus), responseContext); + } + ), + false ) ), toolChest @@ -160,26 +170,28 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final // Wrap baseRunner in a runner that rewrites the QuerySegmentSpec to mention the specific segments. // This mimics what CachingClusteredClient on the Broker does, and is required for certain queries (like Scan) // to function properly. SegmentServerSelector does not currently mimic CachingClusteredClient, it is using - // the LocalQuerySegmentWalker constructor instead since this walker is not mimic remote DruidServer objects + // the LocalQuerySegmentWalker constructor instead since this walker does not mimic remote DruidServer objects // to actually serve the queries return (theQuery, responseContext) -> { + QueryPlus newQuery = GroupByQueryRunnerTestHelper.populateResourceId(theQuery); responseContext.initializeRemainingResponses(); - String etag = etagProvider.getEtagFor(theQuery.getQuery()); + String etag = etagProvider.getEtagFor(newQuery.getQuery()); if (etag != null) { responseContext.put(Keys.ETAG, etag); } responseContext.addRemainingResponse( - theQuery.getQuery().getMostSpecificId(), 0); + newQuery.getQuery().getMostSpecificId(), 0); + if (scheduler != null) { Set segments = new HashSet<>(); specs.forEach(spec -> segments.add(new SegmentServerSelector(spec))); return scheduler.run( - scheduler.prioritizeAndLaneQuery(theQuery, segments), + scheduler.prioritizeAndLaneQuery(newQuery, segments), new LazySequence<>( () -> baseRunner.run( - theQuery.withQuery(Queries.withSpecificSegments( - theQuery.getQuery(), + newQuery.withQuery(Queries.withSpecificSegments( + newQuery.getQuery(), ImmutableList.copyOf(specs) )), responseContext @@ -188,7 +200,7 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final ); } else { return baseRunner.run( - theQuery.withQuery(Queries.withSpecificSegments(theQuery.getQuery(), ImmutableList.copyOf(specs))), + newQuery.withQuery(Queries.withSpecificSegments(newQuery.getQuery(), ImmutableList.copyOf(specs))), responseContext ); } @@ -227,7 +239,8 @@ private QueryRunner makeTableRunner( new SpecificSegmentSpec(segment.getDescriptor()) ) ) - ) + ), + true ), toolChest ); 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 9b9bf015efe9..5bc94c08e0be 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 @@ -615,7 +615,7 @@ public String getType() @Override public Query withOverriddenContext(Map contextOverride) { - return null; + return this; } @Override diff --git a/services/src/main/java/org/apache/druid/cli/DumpSegment.java b/services/src/main/java/org/apache/druid/cli/DumpSegment.java index afbd58ab9546..3c663e22488f 100644 --- a/services/src/main/java/org/apache/druid/cli/DumpSegment.java +++ b/services/src/main/java/org/apache/druid/cli/DumpSegment.java @@ -93,6 +93,7 @@ import org.apache.druid.segment.nested.NestedFieldDictionaryEncodedColumn; import org.apache.druid.segment.nested.NestedPathFinder; import org.apache.druid.segment.nested.NestedPathPart; +import org.apache.druid.server.ResourceIdPopulatingQueryRunner; import org.apache.druid.timeline.SegmentId; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; @@ -746,8 +747,8 @@ static Sequence executeQuery(final Injector injector, final QueryableInde final QueryRunner runner = factory.createRunner(new QueryableIndexSegment(index, SegmentId.dummy("segment"))); return factory .getToolchest() - .mergeResults(factory.mergeRunners(DirectQueryProcessingPool.INSTANCE, ImmutableList.of(runner))) - .run(QueryPlus.wrap(query), ResponseContext.createEmpty()); + .mergeResults(factory.mergeRunners(DirectQueryProcessingPool.INSTANCE, ImmutableList.of(runner)), true) + .run(QueryPlus.wrap(ResourceIdPopulatingQueryRunner.populateResourceId(query)), ResponseContext.createEmpty()); } private static void evaluateSequenceForSideEffects(final Sequence sequence) diff --git a/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java b/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java index 868eec46fd05..9369717bf1f6 100644 --- a/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java +++ b/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java @@ -104,10 +104,12 @@ public void testExecuteQuery() QueryRunner mergeRunner = Mockito.mock(QueryRunner.class); Query query = Mockito.mock(Query.class); Sequence expected = Sequences.simple(Collections.singletonList(123)); + Mockito.when(query.withOverriddenContext(ArgumentMatchers.any())).thenReturn(query); Mockito.when(injector.getInstance(QueryRunnerFactoryConglomerate.class)).thenReturn(conglomerate); Mockito.when(conglomerate.findFactory(ArgumentMatchers.any())).thenReturn(factory); Mockito.when(factory.createRunner(ArgumentMatchers.any())).thenReturn(runner); Mockito.when(factory.getToolchest().mergeResults(factory.mergeRunners(DirectQueryProcessingPool.INSTANCE, ImmutableList.of(runner)))).thenReturn(mergeRunner); + Mockito.when(factory.getToolchest().mergeResults(factory.mergeRunners(DirectQueryProcessingPool.INSTANCE, ImmutableList.of(runner)), true)).thenReturn(mergeRunner); Mockito.when(mergeRunner.run(ArgumentMatchers.any(), ArgumentMatchers.any())).thenReturn(expected); Sequence actual = DumpSegment.executeQuery(injector, null, query); Assert.assertSame(expected, actual); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java index 4148353e603a..f1aba574448a 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestDataBuilder.java @@ -55,6 +55,7 @@ import org.apache.druid.query.aggregation.last.DoubleLastAggregatorFactory; import org.apache.druid.query.aggregation.last.FloatLastAggregatorFactory; import org.apache.druid.query.aggregation.last.LongLastAggregatorFactory; +import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.IndexSpec; @@ -836,7 +837,8 @@ public static SpecificSegmentsQuerySegmentWalker createMockWalker( conglomerate, injector.getInstance(SegmentWrangler.class), joinableFactoryWrapper, - scheduler + scheduler, + injector.getInstance(GroupByQueryConfig.class) ).add( DataSegment.builder() .dataSource(CalciteTests.DATASOURCE1)