From 58dcfb6be79bc523b3a85e9bec86eece79ca98be Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 17 Jan 2020 16:38:43 -0800 Subject: [PATCH 1/3] Remove the deprecated interval-chunking stuff. See https://github.com/apache/druid/pull/6591, https://github.com/apache/druid/pull/4004#issuecomment-284171911 for details. --- .../FilteredAggregatorBenchmark.java | 4 +- .../GroupByTypeInterfaceBenchmark.java | 5 +- .../benchmark/TopNTypeInterfaceBenchmark.java | 5 +- .../CachingClusteredClientBenchmark.java | 19 +- .../benchmark/query/GroupByBenchmark.java | 5 +- .../benchmark/query/QueryBenchmarkUtil.java | 31 +--- .../benchmark/query/SearchBenchmark.java | 5 +- .../benchmark/query/TimeseriesBenchmark.java | 4 +- .../druid/benchmark/query/TopNBenchmark.java | 5 +- .../timecompare/TimeCompareBenchmark.java | 7 +- docs/configuration/index.md | 2 +- docs/design/extensions-contrib/dropwizard.md | 5 - docs/operations/basic-cluster-tuning.md | 8 - docs/operations/metrics.md | 1 - docs/querying/groupbyquery.md | 2 - docs/querying/query-context.md | 1 - ...asedDruidToTimelineEventConverterTest.java | 1 - .../resources/defaultMetricDimensions.json | 5 - .../graphite/WhiteListBasedConverterTest.java | 3 +- ...terializedViewQueryQueryToolChestTest.java | 7 +- .../src/main/resources/defaultMetrics.json | 3 - .../resources/defaultMetricDimensions.json | 1 - .../segment/MapVirtualColumnGroupByTest.java | 5 +- .../segment/MapVirtualColumnTopNTest.java | 5 +- .../ApproximateHistogramTopNQueryTest.java | 10 +- .../FixedBucketsHistogramTopNQueryTest.java | 10 +- .../indexing/kafka/KafkaIndexTaskTest.java | 17 +- .../kinesis/KinesisIndexTaskTest.java | 21 +-- .../variance/VarianceTopNQueryTest.java | 5 +- ...penderatorDriverRealtimeIndexTaskTest.java | 18 +- .../common/task/RealtimeIndexTaskTest.java | 17 +- .../druid/query/DefaultQueryMetrics.java | 7 - .../query/IntervalChunkingQueryRunner.java | 161 ----------------- .../IntervalChunkingQueryRunnerDecorator.java | 55 ------ .../org/apache/druid/query/QueryContexts.java | 9 - .../org/apache/druid/query/QueryMetrics.java | 5 - .../groupby/GroupByQueryQueryToolChest.java | 26 +-- .../groupby/strategy/GroupByStrategy.java | 10 -- .../groupby/strategy/GroupByStrategyV1.java | 11 -- .../groupby/strategy/GroupByStrategyV2.java | 17 -- .../search/DefaultSearchQueryMetrics.java | 6 - .../search/SearchQueryQueryToolChest.java | 164 ++++++++---------- .../TimeseriesQueryQueryToolChest.java | 30 ++-- .../query/topn/TopNQueryQueryToolChest.java | 65 +++---- .../druid/query/DefaultQueryMetricsTest.java | 5 - .../IntervalChunkingQueryRunnerTest.java | 110 ------------ .../druid/query/MultiValuedDimensionTest.java | 15 +- .../druid/query/QueryRunnerTestHelper.java | 52 ++---- .../apache/druid/query/TestQueryRunners.java | 13 +- .../aggregation/AggregationTestHelper.java | 9 +- ...ByLimitPushDownInsufficientBufferTest.java | 30 +--- ...roupByLimitPushDownMultiNodeMergeTest.java | 40 +---- .../groupby/GroupByMultiSegmentTest.java | 25 +-- .../groupby/GroupByQueryMergeBufferTest.java | 5 +- .../GroupByQueryQueryToolChestTest.java | 41 ++--- .../GroupByQueryRunnerFailureTest.java | 5 +- .../query/groupby/GroupByQueryRunnerTest.java | 97 +---------- .../groupby/NestedQueryPushDownTest.java | 33 +--- .../query/search/SearchQueryRunnerTest.java | 5 +- .../search/SearchQueryRunnerWithCaseTest.java | 5 +- .../TimeSeriesUnionQueryRunnerTest.java | 4 +- .../TimeseriesQueryRunnerBonusTest.java | 2 +- .../timeseries/TimeseriesQueryRunnerTest.java | 20 +-- .../topn/TopNQueryQueryToolChestTest.java | 5 +- .../query/topn/TopNQueryRunnerBenchmark.java | 2 +- .../druid/query/topn/TopNQueryRunnerTest.java | 127 +------------- .../druid/query/topn/TopNUnionQueryTest.java | 10 +- .../IndexMergerV9WithSpatialIndexTest.java | 12 +- .../segment/data/IncrementalIndexTest.java | 4 +- .../filter/SpatialFilterBonusTest.java | 9 +- .../segment/filter/SpatialFilterTest.java | 9 +- .../OnheapIncrementalIndexBenchmark.java | 2 +- .../client/CachingClusteredClientTest.java | 79 +++------ .../CachingClusteredClientTestUtils.java | 15 +- .../druid/client/CachingQueryRunnerTest.java | 9 +- .../appenderator/AppenderatorTester.java | 9 +- .../druid/sql/calcite/util/CalciteTests.java | 7 +- website/.spelling | 2 - 78 files changed, 248 insertions(+), 1372 deletions(-) delete mode 100644 processing/src/main/java/org/apache/druid/query/IntervalChunkingQueryRunner.java delete mode 100644 processing/src/main/java/org/apache/druid/query/IntervalChunkingQueryRunnerDecorator.java delete mode 100644 processing/src/test/java/org/apache/druid/query/IntervalChunkingQueryRunnerTest.java diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java index cb84ab4bd606..f60708596a8c 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java @@ -197,9 +197,7 @@ public void setup() throws IOException qIndex = INDEX_IO.loadIndex(indexFile); factory = new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest( - QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator() - ), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryBenchmarkUtil.NOOP_QUERYWATCHER ); diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java index bf8c164a555b..647a787f67b6 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java @@ -409,10 +409,7 @@ public String getFormatString() factory = new GroupByQueryRunnerFactory( strategySelector, - new GroupByQueryQueryToolChest( - strategySelector, - QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator() - ) + new GroupByQueryQueryToolChest(strategySelector) ); } diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java index 505ef275196c..69f48664555f 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java @@ -301,10 +301,7 @@ public void setup() throws IOException 0, Integer.MAX_VALUE ), - new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator() - ), + new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryBenchmarkUtil.NOOP_QUERYWATCHER ); } diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java index 6fa37a896a72..2d4d0f5a6501 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -260,9 +260,7 @@ public boolean useParallelMergePool() .put( TimeseriesQuery.class, new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) @@ -274,10 +272,7 @@ public boolean useParallelMergePool() "TopNQueryRunnerFactory-bufferPool", () -> ByteBuffer.allocate(PROCESSING_BUFFER_SIZE) ), - new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ), + new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) ) @@ -374,14 +369,8 @@ private static GroupByQueryRunnerFactory makeGroupByQueryRunnerFactory( QueryRunnerTestHelper.NOOP_QUERYWATCHER ) ); - final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest( - strategySelector, - QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() - ); - return new GroupByQueryRunnerFactory( - strategySelector, - toolChest - ); + final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(strategySelector); + return new GroupByQueryRunnerFactory(strategySelector, toolChest); } @TearDown(Level.Trial) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java index 8cff9e5f1f2a..b1f910be1b73 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java @@ -572,10 +572,7 @@ public String getFormatString() factory = new GroupByQueryRunnerFactory( strategySelector, - new GroupByQueryQueryToolChest( - strategySelector, - QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator() - ) + new GroupByQueryQueryToolChest(strategySelector) ); } diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/QueryBenchmarkUtil.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/QueryBenchmarkUtil.java index 43cf6d20fa64..d3f10f40444e 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/QueryBenchmarkUtil.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/QueryBenchmarkUtil.java @@ -19,19 +19,14 @@ package org.apache.druid.benchmark.query; -import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.BySegmentQueryRunner; import org.apache.druid.query.FinalizeResultsQueryRunner; -import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; import org.apache.druid.query.Query; -import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactory; import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.QueryWatcher; -import org.apache.druid.query.context.ResponseContext; import org.apache.druid.segment.Segment; import org.apache.druid.timeline.SegmentId; @@ -53,29 +48,5 @@ public static > QueryRunner makeQueryRunner( ); } - public static IntervalChunkingQueryRunnerDecorator noopIntervalChunkingQueryRunnerDecorator() - { - return new IntervalChunkingQueryRunnerDecorator(null, null, null) { - @Override - public QueryRunner decorate(final QueryRunner delegate, QueryToolChest> toolChest) - { - return new QueryRunner() { - @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) - { - return delegate.run(queryPlus, responseContext); - } - }; - } - }; - } - - public static final QueryWatcher NOOP_QUERYWATCHER = new QueryWatcher() - { - @Override - public void registerQuery(Query query, ListenableFuture future) - { - - } - }; + public static final QueryWatcher NOOP_QUERYWATCHER = (query, future) -> {}; } diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SearchBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SearchBenchmark.java index c86eafab599a..64746c72e047 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SearchBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SearchBenchmark.java @@ -373,10 +373,7 @@ public void setup() throws IOException final SearchQueryConfig config = new SearchQueryConfig().withOverrides(query); factory = new SearchQueryRunnerFactory( new SearchStrategySelector(Suppliers.ofInstance(config)), - new SearchQueryQueryToolChest( - config, - QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator() - ), + new SearchQueryQueryToolChest(config), QueryBenchmarkUtil.NOOP_QUERYWATCHER ); } diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java index 272902b6ebee..3077abf5a1f4 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java @@ -296,9 +296,7 @@ public void setup() throws IOException } factory = new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest( - QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator() - ), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryBenchmarkUtil.NOOP_QUERYWATCHER ); diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/TopNBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/TopNBenchmark.java index 9bc3b3442649..afaf122f0ccf 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/TopNBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/TopNBenchmark.java @@ -279,10 +279,7 @@ public void setup() throws IOException 0, Integer.MAX_VALUE ), - new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator() - ), + new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryBenchmarkUtil.NOOP_QUERYWATCHER ); } diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java index acfec5b7ba91..2ac67afa95e4 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java @@ -230,10 +230,7 @@ private void setupQueries() 0, Integer.MAX_VALUE ), - new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator() - ), + new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryBenchmarkUtil.NOOP_QUERYWATCHER ); } @@ -274,7 +271,7 @@ private void setupQueries() timeseriesQuery = timeseriesQueryBuilder.build(); timeseriesFactory = new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest(QueryBenchmarkUtil.noopIntervalChunkingQueryRunnerDecorator()), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryBenchmarkUtil.NOOP_QUERYWATCHER ); diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 2a322ba5987e..7abcf0871e96 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1505,7 +1505,7 @@ Druid broker can optionally retry queries internally for transient errors. ##### Processing -The broker uses processing configs for nested groupBy queries. And, if you use groupBy v1, long-interval queries (of any type) can be broken into shorter interval queries and processed in parallel inside this thread pool. For more details, see "chunkPeriod" in the [query context](../querying/query-context.md) doc. +The broker uses processing configs for nested groupBy queries. |Property|Description|Default| |--------|-----------|-------| diff --git a/docs/design/extensions-contrib/dropwizard.md b/docs/design/extensions-contrib/dropwizard.md index 41ec1007cd9e..b0f6cdff92a0 100644 --- a/docs/design/extensions-contrib/dropwizard.md +++ b/docs/design/extensions-contrib/dropwizard.md @@ -123,11 +123,6 @@ Latest default metrics mapping can be found [here] (https://github.com/apache/dr "type": "timer", "timeUnit": "MILLISECONDS" }, - "query/intervalChunk/time": { - "dimensions": [], - "type": "timer", - "timeUnit": "MILLISECONDS" - }, "query/segment/time": { "dimensions": [], "type": "timer", diff --git a/docs/operations/basic-cluster-tuning.md b/docs/operations/basic-cluster-tuning.md index ffa9d6795ac4..341da8386ff8 100644 --- a/docs/operations/basic-cluster-tuning.md +++ b/docs/operations/basic-cluster-tuning.md @@ -144,14 +144,6 @@ On the Broker, the amount of direct memory needed depends on how many merge buff - `druid.processing.numThreads`: set this to 1 (the minimum allowed) - `druid.processing.numMergeBuffers`: set this to the same value as on Historicals or a bit higher -##### Note on the deprecated `chunkPeriod` - -There is one exception to the Broker not needing processing threads and processing buffers: - -If the deprecated `chunkPeriod` property in the [query context](../querying/query-context.md) is set, GroupBy V1 queries will use processing threads and processing buffers on the Broker. - -Both `chunkPeriod` and GroupBy V1 are deprecated (use GroupBy V2 instead) and will be removed in the future, we do not recommend using them. The presence of the deprecated `chunkPeriod` feature is why a minimum of 1 processing thread must be configured, even if it's unused. - #### Connection pool sizing Please see the [General Connection Pool Guidelines](#connection-pool) section for an overview of connection pool configuration. diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md index 1e66fc7ed67c..5c0da46f0905 100644 --- a/docs/operations/metrics.md +++ b/docs/operations/metrics.md @@ -54,7 +54,6 @@ Available Metrics |`query/node/bytes`|number of bytes returned from querying individual historical/realtime processes.|id, status, server.| | |`query/node/ttfb`|Time to first byte. Milliseconds elapsed until Broker starts receiving the response from individual historical/realtime processes.|id, status, server.|< 1s| |`query/node/backpressure`|Milliseconds that the channel to this process has spent suspended due to backpressure.|id, status, server.| | -|`query/intervalChunk/time`|Only emitted if interval chunking is enabled. Milliseconds required to query an interval chunk. This metric is deprecated and will be removed in the future because interval chunking is deprecated. See [Query Context](../querying/query-context.md).|id, status, chunkInterval (if interval chunking is enabled).|< 1s| |`query/count`|number of total queries|This metric is only available if the QueryCountStatsMonitor module is included.|| |`query/success/count`|number of queries successfully processed|This metric is only available if the QueryCountStatsMonitor module is included.|| |`query/failed/count`|number of failed queries|This metric is only available if the QueryCountStatsMonitor module is included.|| diff --git a/docs/querying/groupbyquery.md b/docs/querying/groupbyquery.md index a8ffb64f62ba..8c2ea49c4c4a 100644 --- a/docs/querying/groupbyquery.md +++ b/docs/querying/groupbyquery.md @@ -257,8 +257,6 @@ by using a finite-sized merge buffer pool. By default, the number of merge buffe threads. You can adjust this as necessary to balance concurrency and memory usage. - groupBy v1 supports caching on either the Broker or Historical processes, whereas groupBy v2 only supports caching on Historical processes. -- groupBy v1 supports using [chunkPeriod](query-context.html) to parallelize merging on the Broker, whereas groupBy v2 -ignores chunkPeriod. - groupBy v2 supports both array-based aggregation and hash-based aggregation. The array-based aggregation is used only when the grouping key is a single indexed string column. In array-based aggregation, the dictionary-encoded value is used as the index, so the aggregated values in the array can be accessed directly without finding buckets based on hashing. diff --git a/docs/querying/query-context.md b/docs/querying/query-context.md index 0ba16e8d6abd..28cfb3bd5228 100644 --- a/docs/querying/query-context.md +++ b/docs/querying/query-context.md @@ -36,7 +36,6 @@ The query context is used for various query configuration parameters. The follow |populateResultLevelCache | `true` | Flag indicating whether to save the results of the query to the result level cache. Primarily used for debugging. When set to false, it disables saving the results of this query to the query cache. When set to true, Druid uses `druid.broker.cache.populateResultLevelCache` to determine whether or not to save the results of this query to the result-level query cache | |bySegment | `false` | Return "by segment" results. Primarily used for debugging, setting it to `true` returns results associated with the data segment they came from | |finalize | `true` | Flag indicating whether to "finalize" aggregation results. Primarily used for debugging. For instance, the `hyperUnique` aggregator will return the full HyperLogLog sketch instead of the estimated cardinality when this flag is set to `false` | -|chunkPeriod | `P0D` (off) | At the Broker process level, long interval queries (of any type) may be broken into shorter interval queries to parallelize merging more than normal. Broken up queries will use a larger share of cluster resources, but, if you use groupBy "v1, it may be able to complete faster as a result. Use ISO 8601 periods. For example, if this property is set to `P1M` (one month), then a query covering a year would be broken into 12 smaller queries. The broker uses its query processing executor service to initiate processing for query chunks, so make sure `druid.processing.numThreads` is configured appropriately on the broker. [groupBy queries](groupbyquery.html) do not support chunkPeriod by default, although they do if using the legacy "v1" engine. This context is deprecated since it's only useful for groupBy "v1", and will be removed in the future releases.| |maxScatterGatherBytes| `druid.server.http.maxScatterGatherBytes` | Maximum number of bytes gathered from data processes such as Historicals and realtime processes to execute a query. This parameter can be used to further reduce `maxScatterGatherBytes` limit at query time. See [Broker configuration](../configuration/index.html#broker) for more details.| |maxQueuedBytes | `druid.broker.http.maxQueuedBytes` | Maximum number of bytes queued per query before exerting backpressure on the channel to the data server. Similar to `maxScatterGatherBytes`, except unlike that configuration, this one will trigger backpressure rather than query failure. Zero means disabled.| |serializeDateTimeAsLong| `false` | If true, DateTime is serialized as long in the result returned by Broker and the data transportation between Broker and compute process| diff --git a/extensions-contrib/ambari-metrics-emitter/src/test/java/org/apache/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverterTest.java b/extensions-contrib/ambari-metrics-emitter/src/test/java/org/apache/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverterTest.java index 746011a752ce..c2b8c2c4b796 100644 --- a/extensions-contrib/ambari-metrics-emitter/src/test/java/org/apache/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverterTest.java +++ b/extensions-contrib/ambari-metrics-emitter/src/test/java/org/apache/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverterTest.java @@ -71,7 +71,6 @@ public void setUp() "query/time, true", "query/node/ttfb, true", "query/segmentAndCache/time, true", - "query/intervalChunk/time, false", "query/time/balaba, true", "query/tim, false", "segment/added/bytes, true", diff --git a/extensions-contrib/dropwizard-emitter/src/main/resources/defaultMetricDimensions.json b/extensions-contrib/dropwizard-emitter/src/main/resources/defaultMetricDimensions.json index ed071968a4f2..f6f347fdb09f 100644 --- a/extensions-contrib/dropwizard-emitter/src/main/resources/defaultMetricDimensions.json +++ b/extensions-contrib/dropwizard-emitter/src/main/resources/defaultMetricDimensions.json @@ -28,11 +28,6 @@ "type": "timer", "timeUnit": "MILLISECONDS" }, - "query/intervalChunk/time": { - "dimensions": [], - "type": "timer", - "timeUnit": "MILLISECONDS" - }, "query/segment/time": { "dimensions": [], "type": "timer", diff --git a/extensions-contrib/graphite-emitter/src/test/java/org/apache/druid/emitter/graphite/WhiteListBasedConverterTest.java b/extensions-contrib/graphite-emitter/src/test/java/org/apache/druid/emitter/graphite/WhiteListBasedConverterTest.java index eda7764231fa..bdc04010eedc 100644 --- a/extensions-contrib/graphite-emitter/src/test/java/org/apache/druid/emitter/graphite/WhiteListBasedConverterTest.java +++ b/extensions-contrib/graphite-emitter/src/test/java/org/apache/druid/emitter/graphite/WhiteListBasedConverterTest.java @@ -75,7 +75,6 @@ public void setUp() "query/time, true", "query/node/ttfb, true", "query/segmentAndCache/time, true", - "query/intervalChunk/time, false", "query/time/balaba, true", "query/tim, false", "segment/added/bytes, false", @@ -136,7 +135,7 @@ public void testWhiteListedStringArrayDimension() throws IOException ); ServiceMetricEvent event = new ServiceMetricEvent.Builder() - .setDimension("gcName", new String[] {"g1"}) + .setDimension("gcName", new String[]{"g1"}) .build(createdTime, "jvm/gc/cpu", 10) .build(serviceName, hostname); 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 cff9ddcb8dd0..cf1649c91020 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 @@ -55,12 +55,7 @@ public void testMakePostComputeManipulatorFn() QueryToolChest materializedViewQueryQueryToolChest = new MaterializedViewQueryQueryToolChest(new MapQueryToolChestWarehouse( ImmutableMap., QueryToolChest>builder() - .put( - TimeseriesQuery.class, - new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ) - ) + .put(TimeseriesQuery.class, new TimeseriesQueryQueryToolChest()) .build() )); diff --git a/extensions-contrib/opentsdb-emitter/src/main/resources/defaultMetrics.json b/extensions-contrib/opentsdb-emitter/src/main/resources/defaultMetrics.json index 847e702acd1d..d00b79265420 100644 --- a/extensions-contrib/opentsdb-emitter/src/main/resources/defaultMetrics.json +++ b/extensions-contrib/opentsdb-emitter/src/main/resources/defaultMetrics.json @@ -16,9 +16,6 @@ "query/node/ttfb": [ "server" ], - "query/intervalChunk/time": [ - "chunkInterval" - ], "query/success/count": [], "query/failed/count": [], "query/interrupted/count": [], diff --git a/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json b/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json index 30cbeff7e507..2c7817b516a1 100644 --- a/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json +++ b/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json @@ -5,7 +5,6 @@ "query/node/ttfb" : { "dimensions" : ["server"], "type" : "timer"}, "query/node/bytes" : { "dimensions" : ["server"], "type" : "count"}, "query/node/backpressure": { "dimensions" : ["server"], "type" : "timer"}, - "query/intervalChunk/time" : { "dimensions" : [], "type" : "timer"}, "query/segment/time" : { "dimensions" : [], "type" : "timer"}, "query/wait/time" : { "dimensions" : [], "type" : "timer"}, 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 d335025239fd..e16a906375e7 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 @@ -110,10 +110,7 @@ public int getNumThreads() final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( strategySelector, - new GroupByQueryQueryToolChest( - strategySelector, - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ) + new GroupByQueryQueryToolChest(strategySelector) ); runner = QueryRunnerTestHelper.makeQueryRunner( diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java index 345698564f35..81e169b811ca 100644 --- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java +++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java @@ -67,10 +67,7 @@ public void setup() throws IOException final TopNQueryRunnerFactory factory = new TopNQueryRunnerFactory( new StupidPool<>("map-virtual-column-test", () -> ByteBuffer.allocate(1024)), - new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ), + new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java index d61114a06ffd..d09be5b1dab0 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java @@ -79,20 +79,14 @@ public static Iterable constructorFeeder() QueryRunnerTestHelper.makeQueryRunners( new TopNQueryRunnerFactory( defaultPool, - new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ), + new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) ), QueryRunnerTestHelper.makeQueryRunners( new TopNQueryRunnerFactory( customPool, - new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ), + new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) ) diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramTopNQueryTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramTopNQueryTest.java index b493745f6b00..cf1d2bbde7d8 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramTopNQueryTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramTopNQueryTest.java @@ -79,20 +79,14 @@ public static Iterable constructorFeeder() QueryRunnerTestHelper.makeQueryRunners( new TopNQueryRunnerFactory( defaultPool, - new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ), + new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) ), QueryRunnerTestHelper.makeQueryRunners( new TopNQueryRunnerFactory( customPool, - new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ), + new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) ) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index aba8bbcdcad5..6bc4991375d6 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -87,13 +87,10 @@ import org.apache.druid.query.DefaultGenericQueryMetricsFactory; import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; import org.apache.druid.query.Druids; -import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; -import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactory; import org.apache.druid.query.QueryRunnerFactoryConglomerate; -import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.scan.ScanQuery; @@ -2508,24 +2505,12 @@ private static DataSchema cloneDataSchema(final DataSchema dataSchema) private QueryRunnerFactoryConglomerate makeTimeseriesAndScanConglomerate() { - IntervalChunkingQueryRunnerDecorator queryRunnerDecorator = new IntervalChunkingQueryRunnerDecorator( - null, - null, - null - ) - { - @Override - public QueryRunner decorate(QueryRunner delegate, QueryToolChest> toolChest) - { - return delegate; - } - }; return new DefaultQueryRunnerFactoryConglomerate( ImmutableMap., QueryRunnerFactory>builder() .put( TimeseriesQuery.class, new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest(queryRunnerDecorator), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), (query, future) -> { // do nothing diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index c214def64369..43b4f06c9c68 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -88,11 +88,7 @@ import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; -import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; -import org.apache.druid.query.Query; -import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactoryConglomerate; -import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.timeseries.TimeseriesQuery; @@ -2753,26 +2749,11 @@ private static DataSchema cloneDataSchema(final DataSchema dataSchema) private QueryRunnerFactoryConglomerate makeTimeseriesOnlyConglomerate() { - IntervalChunkingQueryRunnerDecorator queryRunnerDecorator = new IntervalChunkingQueryRunnerDecorator( - null, - null, - null - ) - { - @Override - public QueryRunner decorate( - QueryRunner delegate, - QueryToolChest> toolChest - ) - { - return delegate; - } - }; return new DefaultQueryRunnerFactoryConglomerate( ImmutableMap.of( TimeseriesQuery.class, new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest(queryRunnerDecorator), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), (query, future) -> { // do nothing diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java index 8003ad1c3949..cbdb8242d6c2 100644 --- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java +++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java @@ -136,10 +136,7 @@ private Sequence> assertExpectedResults( TopNQuery query ) { - final TopNQueryQueryToolChest chest = new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ); + final TopNQueryQueryToolChest chest = new TopNQueryQueryToolChest(new TopNQueryConfig()); final QueryRunner> mergeRunner = chest.mergeResults(runner); final Sequence> retval = mergeRunner.run(QueryPlus.wrap(query)); TestHelper.assertExpectedResults(expectedResults, retval); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index 0c873dc578ca..969f76953702 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -95,12 +95,8 @@ import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; import org.apache.druid.query.Druids; -import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; -import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; -import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactoryConglomerate; -import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.Result; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -1530,23 +1526,11 @@ public SegmentPublishResult announceHistoricalSegments( taskActionToolbox, new TaskAuditLogConfig(false) ); - IntervalChunkingQueryRunnerDecorator queryRunnerDecorator = new IntervalChunkingQueryRunnerDecorator( - null, - null, - null - ) - { - @Override - public QueryRunner decorate(QueryRunner delegate, QueryToolChest> toolChest) - { - return delegate; - } - }; final QueryRunnerFactoryConglomerate conglomerate = new DefaultQueryRunnerFactoryConglomerate( ImmutableMap.of( TimeseriesQuery.class, new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest(queryRunnerDecorator), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), (query, future) -> { // do nothing diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java index 121f085aeaaa..e3967e16cb1a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -79,12 +79,9 @@ import org.apache.druid.metadata.EntryExistsException; import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; import org.apache.druid.query.Druids; -import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; -import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactoryConglomerate; -import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.QueryWatcher; import org.apache.druid.query.Result; import org.apache.druid.query.SegmentDescriptor; @@ -902,23 +899,11 @@ private TaskToolbox makeToolbox( taskActionToolbox, new TaskAuditLogConfig(false) ); - IntervalChunkingQueryRunnerDecorator queryRunnerDecorator = new IntervalChunkingQueryRunnerDecorator( - null, - null, - null - ) - { - @Override - public QueryRunner decorate(QueryRunner delegate, QueryToolChest> toolChest) - { - return delegate; - } - }; final QueryRunnerFactoryConglomerate conglomerate = new DefaultQueryRunnerFactoryConglomerate( ImmutableMap.of( TimeseriesQuery.class, new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest(queryRunnerDecorator), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), new QueryWatcher() { diff --git a/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java index 7effff35c8a2..bffdf6779029 100644 --- a/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java +++ b/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java @@ -228,13 +228,6 @@ public QueryMetrics reportSegmentAndCacheTime(long timeNs) return reportMillisTimeMetric("query/segmentAndCache/time", timeNs); } - @Deprecated - @Override - public QueryMetrics reportIntervalChunkTime(long timeNs) - { - return reportMillisTimeMetric("query/intervalChunk/time", timeNs); - } - @Override public QueryMetrics reportCpuTime(long timeNs) { diff --git a/processing/src/main/java/org/apache/druid/query/IntervalChunkingQueryRunner.java b/processing/src/main/java/org/apache/druid/query/IntervalChunkingQueryRunner.java deleted file mode 100644 index 5bf029ec015a..000000000000 --- a/processing/src/main/java/org/apache/druid/query/IntervalChunkingQueryRunner.java +++ /dev/null @@ -1,161 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.query; - -import com.google.common.base.Function; -import com.google.common.collect.Lists; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.granularity.PeriodGranularity; -import org.apache.druid.java.util.common.guava.FunctionalIterable; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.query.context.ResponseContext; -import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; -import org.joda.time.DateTime; -import org.joda.time.Interval; -import org.joda.time.Period; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.concurrent.ExecutorService; - -/** - * This class is deprecated and will removed in the future. - * See https://github.com/apache/druid/pull/4004#issuecomment-284171911 for details about deprecation. - */ -@Deprecated -public class IntervalChunkingQueryRunner implements QueryRunner -{ - private final QueryRunner baseRunner; - - private final QueryToolChest> toolChest; - private final ExecutorService executor; - private final QueryWatcher queryWatcher; - private final ServiceEmitter emitter; - - public IntervalChunkingQueryRunner( - QueryRunner baseRunner, - QueryToolChest> toolChest, - ExecutorService executor, - QueryWatcher queryWatcher, - ServiceEmitter emitter - ) - { - this.baseRunner = baseRunner; - this.toolChest = toolChest; - this.executor = executor; - this.queryWatcher = queryWatcher; - this.emitter = emitter; - } - - @Override - public Sequence run(final QueryPlus queryPlus, final ResponseContext responseContext) - { - final Period chunkPeriod = getChunkPeriod(queryPlus.getQuery()); - - // Check for non-empty chunkPeriod, avoiding toStandardDuration since that cannot handle periods like P1M. - if (DateTimes.EPOCH.plus(chunkPeriod).getMillis() == DateTimes.EPOCH.getMillis()) { - return baseRunner.run(queryPlus, responseContext); - } - - List chunkIntervals = Lists.newArrayList( - FunctionalIterable - .create(queryPlus.getQuery().getIntervals()) - .transformCat( - new Function>() - { - @Override - public Iterable apply(Interval input) - { - return splitInterval(input, chunkPeriod); - } - } - ) - ); - - if (chunkIntervals.size() <= 1) { - return baseRunner.run(queryPlus, responseContext); - } - - return Sequences.concat( - Lists.newArrayList( - FunctionalIterable.create(chunkIntervals).transform( - new Function>() - { - @Override - public Sequence apply(Interval singleInterval) - { - return new AsyncQueryRunner( - //Note: it is assumed that toolChest.mergeResults(..) gives a query runner that is - //not lazy i.e. it does most of its work on call to run() method - toolChest.mergeResults( - new MetricsEmittingQueryRunner( - emitter, - toolChest, - baseRunner, - QueryMetrics::reportIntervalChunkTime, - queryMetrics -> queryMetrics.chunkInterval(singleInterval) - ).withWaitMeasuredFromNow() - ), - executor, queryWatcher - ).run( - queryPlus.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(Collections.singletonList(singleInterval))), - responseContext - ); - } - } - ) - ) - ); - } - - private static Iterable splitInterval(Interval interval, Period period) - { - if (interval.getEndMillis() == interval.getStartMillis()) { - return Collections.singletonList(interval); - } - - List intervals = new ArrayList<>(); - Iterator timestamps = new PeriodGranularity(period, null, null).getIterable(interval).iterator(); - - DateTime start = DateTimes.max(timestamps.next().getStart(), interval.getStart()); - while (timestamps.hasNext()) { - DateTime end = timestamps.next().getStart(); - intervals.add(new Interval(start, end)); - start = end; - } - - if (start.compareTo(interval.getEnd()) < 0) { - intervals.add(new Interval(start, interval.getEnd())); - } - - return intervals; - } - - private Period getChunkPeriod(Query query) - { - final String p = QueryContexts.getChunkPeriod(query); - return Period.parse(p); - } -} diff --git a/processing/src/main/java/org/apache/druid/query/IntervalChunkingQueryRunnerDecorator.java b/processing/src/main/java/org/apache/druid/query/IntervalChunkingQueryRunnerDecorator.java deleted file mode 100644 index bec8a05d04f4..000000000000 --- a/processing/src/main/java/org/apache/druid/query/IntervalChunkingQueryRunnerDecorator.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.query; - -import com.google.inject.Inject; -import org.apache.druid.guice.annotations.Processing; -import org.apache.druid.guice.annotations.PublicApi; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; - -import java.util.concurrent.ExecutorService; - -/** - * This class is deprecated and will removed in the future. - * See https://github.com/apache/druid/pull/4004#issuecomment-284171911 for details about deprecation. - */ -@Deprecated -public class IntervalChunkingQueryRunnerDecorator -{ - private final ExecutorService executor; - private final QueryWatcher queryWatcher; - private final ServiceEmitter emitter; - - @Inject - public IntervalChunkingQueryRunnerDecorator(@Processing ExecutorService executor, QueryWatcher queryWatcher, - ServiceEmitter emitter) - { - this.executor = executor; - this.queryWatcher = queryWatcher; - this.emitter = emitter; - } - - @PublicApi - public QueryRunner decorate(QueryRunner delegate, QueryToolChest> toolChest) - { - return new IntervalChunkingQueryRunner(delegate, (QueryToolChest>) toolChest, - executor, queryWatcher, emitter); - } -} 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 562336556750..726032fe0c84 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java @@ -43,9 +43,6 @@ public class QueryContexts public static final String BROKER_PARALLEL_MERGE_INITIAL_YIELD_ROWS_KEY = "parallelMergeInitialYieldRows"; public static final String BROKER_PARALLEL_MERGE_SMALL_BATCH_ROWS_KEY = "parallelMergeSmallBatchRows"; public static final String BROKER_PARALLELISM = "parallelMergeParallelism"; - - @Deprecated - public static final String CHUNK_PERIOD_KEY = "chunkPeriod"; public static final String VECTORIZE_KEY = "vectorize"; public static final String VECTOR_SIZE_KEY = "vectorSize"; @@ -221,12 +218,6 @@ public static int getParallelMergeParallelism(Query query, int defaultVal return parseInt(query, BROKER_PARALLELISM, defaultValue); } - @Deprecated - public static String getChunkPeriod(Query query) - { - return query.getContextValue(CHUNK_PERIOD_KEY, "P0D"); - } - public static Query withMaxScatterGatherBytes(Query query, long maxScatterGatherBytesLimit) { Object obj = query.getContextValue(MAX_SCATTER_GATHER_BYTES_KEY); diff --git a/processing/src/main/java/org/apache/druid/query/QueryMetrics.java b/processing/src/main/java/org/apache/druid/query/QueryMetrics.java index ec3c82495501..535dccfb2857 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryMetrics.java +++ b/processing/src/main/java/org/apache/druid/query/QueryMetrics.java @@ -281,11 +281,6 @@ public interface QueryMetrics> */ QueryMetrics reportSegmentAndCacheTime(long timeNs); - /** - * Registers "interval chunk time" metric. - */ - QueryMetrics reportIntervalChunkTime(long timeNs); - /** * Registers "cpu time" metric. */ 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 4274e8e19439..acfe77688e6a 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 @@ -42,7 +42,6 @@ import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.CacheStrategy; import org.apache.druid.query.DataSource; -import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryDataSource; @@ -91,28 +90,21 @@ public class GroupByQueryQueryToolChest extends QueryToolChest run(QueryPlus queryPlus, ResponseContext r } } - return strategySelector.strategize(delegateGroupByQuery) - .createIntervalChunkingRunner( - intervalChunkingQueryRunnerDecorator, - runner, - GroupByQueryQueryToolChest.this - ) - .run( - queryPlus.withQuery(delegateGroupByQuery.withDimensionSpecs(dimensionSpecs)), - responseContext - ); + return runner.run( + queryPlus.withQuery(delegateGroupByQuery.withDimensionSpecs(dimensionSpecs)), + responseContext + ); } } ); diff --git a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java index dca39eb56e20..4a892a47ec6a 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java @@ -22,7 +22,6 @@ import com.google.common.util.concurrent.ListeningExecutorService; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactory; @@ -71,15 +70,6 @@ public interface GroupByStrategy */ boolean doMergeResults(GroupByQuery query); - /** - * Decorate a runner with an interval chunking decorator. - */ - QueryRunner createIntervalChunkingRunner( - IntervalChunkingQueryRunnerDecorator decorator, - QueryRunner runner, - GroupByQueryQueryToolChest toolChest - ); - /** * Runs a provided {@link QueryRunner} on a provided {@link GroupByQuery}, which is assumed to return rows that are * properly sorted (by timestamp and dimensions) but not necessarily fully merged (that is, there may be adjacent diff --git a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV1.java b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV1.java index be9ba0a1394a..18eaab21579c 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV1.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV1.java @@ -33,7 +33,6 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.GroupByMergedQueryRunner; -import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryWatcher; @@ -92,16 +91,6 @@ public boolean isCacheable(boolean willMergeRunners) return true; } - @Override - public QueryRunner createIntervalChunkingRunner( - final IntervalChunkingQueryRunnerDecorator decorator, - final QueryRunner runner, - final GroupByQueryQueryToolChest toolChest - ) - { - return decorator.decorate(runner, toolChest); - } - @Override public boolean doMergeResults(final GroupByQuery query) { diff --git a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java index ceae02cd98b8..587704bbf2a0 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java @@ -41,7 +41,6 @@ import org.apache.druid.query.DataSource; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.InsufficientResourcesException; -import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; import org.apache.druid.query.Query; import org.apache.druid.query.QueryConfig; import org.apache.druid.query.QueryContexts; @@ -58,7 +57,6 @@ import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; -import org.apache.druid.query.groupby.GroupByQueryQueryToolChest; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.epinephelinae.GroupByBinaryFnV2; import org.apache.druid.query.groupby.epinephelinae.GroupByMergingQueryRunnerV2; @@ -173,21 +171,6 @@ public boolean doMergeResults(final GroupByQuery query) return true; } - @Override - public QueryRunner createIntervalChunkingRunner( - final IntervalChunkingQueryRunnerDecorator decorator, - final QueryRunner runner, - final GroupByQueryQueryToolChest toolChest - ) - { - // No chunkPeriod-based interval chunking for groupBy v2. - // 1) It concats query chunks for consecutive intervals, which won't generate correct results. - // 2) Merging instead of concating isn't a good idea, since it requires all chunks to run simultaneously, - // which may take more resources than the cluster has. - // See also https://github.com/apache/druid/pull/4004 - return runner; - } - @Override public Comparator createResultComparator(Query queryParam) { diff --git a/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java b/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java index b10ab934f784..a50bc95899bc 100644 --- a/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java +++ b/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java @@ -208,12 +208,6 @@ public QueryMetrics reportSegmentAndCacheTime(long timeNs) return delegateQueryMetrics.reportSegmentAndCacheTime(timeNs); } - @Override - public QueryMetrics reportIntervalChunkTime(long timeNs) - { - return delegateQueryMetrics.reportIntervalChunkTime(timeNs); - } - @Override public QueryMetrics reportCpuTime(long timeNs) { diff --git a/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java index 5465f879d8a8..08ee67b2918e 100644 --- a/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java @@ -34,7 +34,6 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.CacheStrategy; -import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryPlus; @@ -57,6 +56,7 @@ import java.util.function.BinaryOperator; /** + * */ public class SearchQueryQueryToolChest extends QueryToolChest, SearchQuery> { @@ -69,28 +69,21 @@ public class SearchQueryQueryToolChest extends QueryToolChest, Object, SearchQuery> getCacheStr { private final List dimensionSpecs = query.getDimensions() != null ? query.getDimensions() : Collections.emptyList(); - private final List dimOutputNames = dimensionSpecs.size() > 0 ? - Lists.transform(dimensionSpecs, DimensionSpec::getOutputName) : Collections.emptyList(); + private final List dimOutputNames = dimensionSpecs.size() > 0 + ? + Lists.transform(dimensionSpecs, DimensionSpec::getOutputName) + : Collections.emptyList(); @Override public boolean isCacheable(SearchQuery query, boolean willMergeRunners) @@ -210,8 +205,8 @@ public Function, Object> prepareForCache(boolean isRes public Object apply(Result input) { return dimensionSpecs.size() > 0 - ? Lists.newArrayList(input.getTimestamp().getMillis(), input.getValue(), dimOutputNames) - : Lists.newArrayList(input.getTimestamp().getMillis(), input.getValue()); + ? Lists.newArrayList(input.getTimestamp().getMillis(), input.getValue(), dimOutputNames) + : Lists.newArrayList(input.getTimestamp().getMillis(), input.getValue()); } }; } @@ -230,8 +225,10 @@ public Result apply(Object input) final Map outputNameMap = new HashMap<>(); if (hasOutputName(result)) { List cachedOutputNames = (List) result.get(2); - Preconditions.checkArgument(cachedOutputNames.size() == dimOutputNames.size(), - "cache hit, but number of dimensions mismatch"); + Preconditions.checkArgument( + cachedOutputNames.size() == dimOutputNames.size(), + "cache hit, but number of dimensions mismatch" + ); needsRename = false; for (int idx = 0; idx < cachedOutputNames.size(); idx++) { String cachedOutputName = cachedOutputNames.get(idx); @@ -244,63 +241,63 @@ public Result apply(Object input) } return !needsRename - ? new Result<>( - DateTimes.utc(((Number) result.get(0)).longValue()), - new SearchResultValue( - Lists.transform( - (List) result.get(1), - new Function() - { - @Override - public SearchHit apply(@Nullable Object input) - { - if (input instanceof Map) { - return new SearchHit( - (String) ((Map) input).get("dimension"), - (String) ((Map) input).get("value"), - (Integer) ((Map) input).get("count") - ); - } else if (input instanceof SearchHit) { - return (SearchHit) input; - } else { - throw new IAE("Unknown format [%s]", input.getClass()); - } - } + ? new Result<>( + DateTimes.utc(((Number) result.get(0)).longValue()), + new SearchResultValue( + Lists.transform( + (List) result.get(1), + new Function() + { + @Override + public SearchHit apply(@Nullable Object input) + { + if (input instanceof Map) { + return new SearchHit( + (String) ((Map) input).get("dimension"), + (String) ((Map) input).get("value"), + (Integer) ((Map) input).get("count") + ); + } else if (input instanceof SearchHit) { + return (SearchHit) input; + } else { + throw new IAE("Unknown format [%s]", input.getClass()); } - ) + } + } ) ) - : new Result<>( - DateTimes.utc(((Number) result.get(0)).longValue()), - new SearchResultValue( - Lists.transform( - (List) result.get(1), - new Function() - { - @Override - public SearchHit apply(@Nullable Object input) - { - String dim; - String val; - Integer count; - if (input instanceof Map) { - dim = outputNameMap.get((String) ((Map) input).get("dimension")); - val = (String) ((Map) input).get("value"); - count = (Integer) ((Map) input).get("count"); - } else if (input instanceof SearchHit) { - SearchHit cached = (SearchHit) input; - dim = outputNameMap.get(cached.getDimension()); - val = cached.getValue(); - count = cached.getCount(); - } else { - throw new IAE("Unknown format [%s]", input.getClass()); - } - return new SearchHit(dim, val, count); - } - } - ) - ) - ); + ) + : new Result<>( + DateTimes.utc(((Number) result.get(0)).longValue()), + new SearchResultValue( + Lists.transform( + (List) result.get(1), + new Function() + { + @Override + public SearchHit apply(@Nullable Object input) + { + String dim; + String val; + Integer count; + if (input instanceof Map) { + dim = outputNameMap.get((String) ((Map) input).get("dimension")); + val = (String) ((Map) input).get("value"); + count = (Integer) ((Map) input).get("count"); + } else if (input instanceof SearchHit) { + SearchHit cached = (SearchHit) input; + dim = outputNameMap.get(cached.getDimension()); + val = cached.getValue(); + count = cached.getCount(); + } else { + throw new IAE("Unknown format [%s]", input.getClass()); + } + return new SearchHit(dim, val, count); + } + } + ) + ) + ); } }; } @@ -324,25 +321,14 @@ private boolean hasOutputName(List cachedEntry) public QueryRunner> preMergeQueryDecoration(final QueryRunner> runner) { return new SearchThresholdAdjustingQueryRunner( - intervalChunkingQueryRunnerDecorator.decorate( - new QueryRunner>() - { - @Override - public Sequence> run( - QueryPlus> queryPlus, - ResponseContext responseContext - ) - { - SearchQuery searchQuery = (SearchQuery) queryPlus.getQuery(); - if (searchQuery.getDimensionsFilter() != null) { - searchQuery = searchQuery.withDimFilter(searchQuery.getDimensionsFilter().optimize()); - queryPlus = queryPlus.withQuery(searchQuery); - } - return runner.run(queryPlus, responseContext); - } - }, - this - ), + (queryPlus, responseContext) -> { + SearchQuery searchQuery = (SearchQuery) queryPlus.getQuery(); + if (searchQuery.getDimensionsFilter() != null) { + searchQuery = searchQuery.withDimFilter(searchQuery.getDimensionsFilter().optimize()); + queryPlus = queryPlus.withQuery(searchQuery); + } + return runner.run(queryPlus, responseContext); + }, config ); } diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 1a6ca22e6858..82566c4ebafd 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -35,7 +35,6 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.CacheStrategy; -import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; @@ -77,23 +76,17 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest apply(Object input) @Override public QueryRunner> preMergeQueryDecoration(final QueryRunner> runner) { - return intervalChunkingQueryRunnerDecorator.decorate( - (queryPlus, responseContext) -> { - TimeseriesQuery timeseriesQuery = (TimeseriesQuery) queryPlus.getQuery(); - if (timeseriesQuery.getDimensionsFilter() != null) { - timeseriesQuery = timeseriesQuery.withDimFilter(timeseriesQuery.getDimensionsFilter().optimize()); - queryPlus = queryPlus.withQuery(timeseriesQuery); - } - return runner.run(queryPlus, responseContext); - }, this); + return (queryPlus, responseContext) -> { + TimeseriesQuery timeseriesQuery = (TimeseriesQuery) queryPlus.getQuery(); + if (timeseriesQuery.getDimensionsFilter() != null) { + timeseriesQuery = timeseriesQuery.withDimFilter(timeseriesQuery.getDimensionsFilter().optimize()); + queryPlus = queryPlus.withQuery(timeseriesQuery); + } + return runner.run(queryPlus, responseContext); + }; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java index 369537474e97..2d6c5e851f61 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java @@ -32,7 +32,6 @@ import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.BySegmentResultValue; import org.apache.druid.query.CacheStrategy; -import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryPlus; @@ -73,28 +72,21 @@ public class TopNQueryQueryToolChest extends QueryToolChest apply(Object input) @Override public QueryRunner> preMergeQueryDecoration(final QueryRunner> runner) { - return intervalChunkingQueryRunnerDecorator.decorate( - new QueryRunner>() - { - @Override - public Sequence> run( - QueryPlus> queryPlus, - ResponseContext responseContext - ) - { - TopNQuery topNQuery = (TopNQuery) queryPlus.getQuery(); - if (topNQuery.getDimensionsFilter() != null) { - topNQuery = topNQuery.withDimFilter(topNQuery.getDimensionsFilter().optimize()); - } - final TopNQuery delegateTopNQuery = topNQuery; - if (TopNQueryEngine.canApplyExtractionInPost(delegateTopNQuery)) { - final DimensionSpec dimensionSpec = delegateTopNQuery.getDimensionSpec(); - QueryPlus> delegateQueryPlus = queryPlus.withQuery( - delegateTopNQuery.withDimensionSpec( - new DefaultDimensionSpec( - dimensionSpec.getDimension(), - dimensionSpec.getOutputName() - ) - ) - ); - return runner.run(delegateQueryPlus, responseContext); - } else { - return runner.run(queryPlus.withQuery(delegateTopNQuery), responseContext); - } - } - }, - this - ); + return (queryPlus, responseContext) -> { + TopNQuery topNQuery = (TopNQuery) queryPlus.getQuery(); + if (topNQuery.getDimensionsFilter() != null) { + topNQuery = topNQuery.withDimFilter(topNQuery.getDimensionsFilter().optimize()); + } + final TopNQuery delegateTopNQuery = topNQuery; + if (TopNQueryEngine.canApplyExtractionInPost(delegateTopNQuery)) { + final DimensionSpec dimensionSpec = delegateTopNQuery.getDimensionSpec(); + QueryPlus> delegateQueryPlus = queryPlus.withQuery( + delegateTopNQuery.withDimensionSpec( + new DefaultDimensionSpec( + dimensionSpec.getDimension(), + dimensionSpec.getOutputName() + ) + ) + ); + return runner.run(delegateQueryPlus, responseContext); + } else { + return runner.run(queryPlus.withQuery(delegateTopNQuery), responseContext); + } + }; } @Override diff --git a/processing/src/test/java/org/apache/druid/query/DefaultQueryMetricsTest.java b/processing/src/test/java/org/apache/druid/query/DefaultQueryMetricsTest.java index 424b72bd9c0d..c463dc4052ea 100644 --- a/processing/src/test/java/org/apache/druid/query/DefaultQueryMetricsTest.java +++ b/processing/src/test/java/org/apache/druid/query/DefaultQueryMetricsTest.java @@ -120,11 +120,6 @@ public static void testQueryMetricsDefaultMetricNamesAndUnits( Assert.assertEquals("query/segmentAndCache/time", actualEvent.get("metric")); Assert.assertEquals(4L, actualEvent.get("value")); - queryMetrics.reportIntervalChunkTime(5000001).emit(serviceEmitter); - actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); - Assert.assertEquals("query/intervalChunk/time", actualEvent.get("metric")); - Assert.assertEquals(5L, actualEvent.get("value")); - queryMetrics.reportCpuTime(6000001).emit(serviceEmitter); actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); Assert.assertEquals("query/cpu/time", actualEvent.get("metric")); diff --git a/processing/src/test/java/org/apache/druid/query/IntervalChunkingQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/IntervalChunkingQueryRunnerTest.java deleted file mode 100644 index 87e90c936060..000000000000 --- a/processing/src/test/java/org/apache/druid/query/IntervalChunkingQueryRunnerTest.java +++ /dev/null @@ -1,110 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.query; - -import com.google.common.collect.ImmutableMap; -import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.query.Druids.TimeseriesQueryBuilder; -import org.apache.druid.query.aggregation.CountAggregatorFactory; -import org.apache.druid.query.context.ResponseContext; -import org.easymock.EasyMock; -import org.junit.Before; -import org.junit.Test; - -import java.util.Collections; -import java.util.concurrent.ExecutorService; - -public class IntervalChunkingQueryRunnerTest -{ - private IntervalChunkingQueryRunnerDecorator decorator; - private ExecutorService executors; - private QueryRunner baseRunner; - private QueryToolChest toolChest; - - private final TimeseriesQueryBuilder queryBuilder; - - public IntervalChunkingQueryRunnerTest() - { - queryBuilder = Druids.newTimeseriesQueryBuilder() - .dataSource("test") - .aggregators(Collections.singletonList(new CountAggregatorFactory("count"))); - } - - @Before - public void setup() - { - executors = EasyMock.createMock(ExecutorService.class); - ServiceEmitter emitter = EasyMock.createNiceMock(ServiceEmitter.class); - decorator = new IntervalChunkingQueryRunnerDecorator(executors, - QueryRunnerTestHelper.NOOP_QUERYWATCHER, emitter); - baseRunner = EasyMock.createMock(QueryRunner.class); - toolChest = EasyMock.createNiceMock(QueryToolChest.class); - } - - @Test - public void testDefaultNoChunking() - { - QueryPlus queryPlus = QueryPlus.wrap(queryBuilder.intervals("2014/2016").build()); - final ResponseContext context = ResponseContext.createEmpty(); - - EasyMock.expect(baseRunner.run(queryPlus, context)).andReturn(Sequences.empty()); - EasyMock.replay(baseRunner); - - QueryRunner runner = decorator.decorate(baseRunner, toolChest); - runner.run(queryPlus, context); - - EasyMock.verify(baseRunner); - } - - @Test - public void testChunking() - { - Query query = queryBuilder.intervals("2015-01-01T00:00:00.000/2015-01-11T00:00:00.000").context(ImmutableMap.of("chunkPeriod", "P1D")).build(); - - executors.execute(EasyMock.anyObject(Runnable.class)); - EasyMock.expectLastCall().times(10); - - EasyMock.replay(executors); - EasyMock.replay(toolChest); - - QueryRunner runner = decorator.decorate(baseRunner, toolChest); - runner.run(QueryPlus.wrap(query), ResponseContext.createEmpty()); - - EasyMock.verify(executors); - } - - @Test - public void testChunkingOnMonths() - { - Query query = queryBuilder.intervals("2015-01-01T00:00:00.000/2015-02-11T00:00:00.000").context(ImmutableMap.of("chunkPeriod", "P1M")).build(); - - executors.execute(EasyMock.anyObject(Runnable.class)); - EasyMock.expectLastCall().times(2); - - EasyMock.replay(executors); - EasyMock.replay(toolChest); - - QueryRunner runner = decorator.decorate(baseRunner, toolChest); - runner.run(QueryPlus.wrap(query), ResponseContext.createEmpty()); - - EasyMock.verify(executors); - } -} diff --git a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java index c97e8f67fa85..da4a25809b7b 100644 --- a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java +++ b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java @@ -1023,10 +1023,7 @@ public void testTopNWithDimFilterAndWithFilteredDimSpec() try (CloseableStupidPool pool = TestQueryRunners.createDefaultNonBlockingPool()) { QueryRunnerFactory factory = new TopNQueryRunnerFactory( pool, - new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ), + new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); QueryRunner> runner = QueryRunnerTestHelper.makeQueryRunner( @@ -1076,10 +1073,7 @@ public void testTopNExpression() try (CloseableStupidPool pool = TestQueryRunners.createDefaultNonBlockingPool()) { QueryRunnerFactory factory = new TopNQueryRunnerFactory( pool, - new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ), + new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); QueryRunner> runner = QueryRunnerTestHelper.makeQueryRunner( @@ -1136,10 +1130,7 @@ public void testTopNExpressionAutoTransform() try (CloseableStupidPool pool = TestQueryRunners.createDefaultNonBlockingPool()) { QueryRunnerFactory factory = new TopNQueryRunnerFactory( pool, - new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ), + new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); QueryRunner> runner = QueryRunnerTestHelper.makeQueryRunner( 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 bb7ddd4b7dfc..04f86d8ad249 100644 --- a/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java @@ -26,14 +26,11 @@ import com.google.common.collect.Lists; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.MergeSequence; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.java.util.emitter.core.NoopEmitter; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.js.JavaScriptConfig; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; @@ -92,7 +89,9 @@ public class QueryRunnerTestHelper public static final Interval FULL_ON_INTERVAL = Intervals.of("1970-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z"); public static final SegmentId SEGMENT_ID = SegmentId.of(DATA_SOURCE, FULL_ON_INTERVAL, "dummy_version", 0); public static final UnionDataSource UNION_DATA_SOURCE = new UnionDataSource( - Stream.of(DATA_SOURCE, DATA_SOURCE, DATA_SOURCE, DATA_SOURCE).map(TableDataSource::new).collect(Collectors.toList()) + Stream.of(DATA_SOURCE, DATA_SOURCE, DATA_SOURCE, DATA_SOURCE) + .map(TableDataSource::new) + .collect(Collectors.toList()) ); public static final Granularity DAY_GRAN = Granularities.DAY; @@ -118,7 +117,10 @@ public class QueryRunnerTestHelper public static final CountAggregatorFactory ROWS_COUNT = new CountAggregatorFactory("rows"); public static final LongSumAggregatorFactory INDEX_LONG_SUM = new LongSumAggregatorFactory("index", INDEX_METRIC); public static final LongSumAggregatorFactory TIME_LONG_SUM = new LongSumAggregatorFactory("sumtime", TIME_DIMENSION); - public static final DoubleSumAggregatorFactory INDEX_DOUBLE_SUM = new DoubleSumAggregatorFactory("index", INDEX_METRIC); + public static final DoubleSumAggregatorFactory INDEX_DOUBLE_SUM = new DoubleSumAggregatorFactory( + "index", + INDEX_METRIC + ); public static final String JS_COMBINE_A_PLUS_B = "function combine(a, b) { return a + b; }"; public static final String JS_RESET_0 = "function reset() { return 0; }"; public static final JavaScriptAggregatorFactory JS_INDEX_SUM_IF_PLACEMENTISH_A = new JavaScriptAggregatorFactory( @@ -166,7 +168,7 @@ public class QueryRunnerTestHelper public static final FieldAccessPostAggregator ROWS_POST_AGG = new FieldAccessPostAggregator("rows", "rows"); public static final FieldAccessPostAggregator INDEX_POST_AGG = new FieldAccessPostAggregator("index", "index"); public static final ArithmeticPostAggregator ADD_ROWS_INDEX_CONSTANT = new ArithmeticPostAggregator( - ADD_ROWS_INDEX_CONSTANT_METRIC, + ADD_ROWS_INDEX_CONSTANT_METRIC, "+", Lists.newArrayList(CONSTANT, ROWS_POST_AGG, INDEX_POST_AGG) ); @@ -464,7 +466,10 @@ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) segments.addAll(timeline.lookup(interval)); } List> sequences = new ArrayList<>(); - for (TimelineObjectHolder holder : toolChest.filterSegments(query, segments)) { + for (TimelineObjectHolder holder : toolChest.filterSegments( + query, + segments + )) { Segment segment = holder.getObject().getChunk(0).getObject(); QueryPlus queryPlusRunning = queryPlus.withQuerySegmentSpec( new SpecificSegmentSpec( @@ -486,37 +491,6 @@ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) .applyPostMergeDecoration(); } - public static IntervalChunkingQueryRunnerDecorator noopIntervalChunkingQueryRunnerDecorator() - { - return new IntervalChunkingQueryRunnerDecorator(null, null, null) - { - @Override - public QueryRunner decorate( - final QueryRunner delegate, - QueryToolChest> toolChest - ) - { - return new QueryRunner() - { - @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) - { - return delegate.run(queryPlus, responseContext); - } - }; - } - }; - } - - public static IntervalChunkingQueryRunnerDecorator sameThreadIntervalChunkingQueryRunnerDecorator() - { - return new IntervalChunkingQueryRunnerDecorator( - Execs.directExecutor(), - QueryRunnerTestHelper.NOOP_QUERYWATCHER, - new ServiceEmitter("dummy", "dummy", new NoopEmitter()) - ); - } - public static Map of(Object... keyvalues) { ImmutableMap.Builder builder = ImmutableMap.builder(); @@ -529,7 +503,7 @@ public static Map of(Object... keyvalues) public static TimeseriesQueryRunnerFactory newTimeseriesQueryRunnerFactory() { return new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest(noopIntervalChunkingQueryRunnerDecorator()), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); diff --git a/processing/src/test/java/org/apache/druid/query/TestQueryRunners.java b/processing/src/test/java/org/apache/druid/query/TestQueryRunners.java index 4b4b4d3e0863..9d8e7dfe4e5e 100644 --- a/processing/src/test/java/org/apache/druid/query/TestQueryRunners.java +++ b/processing/src/test/java/org/apache/druid/query/TestQueryRunners.java @@ -55,10 +55,7 @@ public static QueryRunner makeTopNQueryRunner(Segment adapter, NonBlockin { QueryRunnerFactory factory = new TopNQueryRunnerFactory( pool, - new TopNQueryQueryToolChest( - TOPN_CONFIG, - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ), + new TopNQueryQueryToolChest(TOPN_CONFIG), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); return new FinalizeResultsQueryRunner( @@ -70,8 +67,7 @@ public static QueryRunner makeTopNQueryRunner(Segment adapter, NonBlockin public static QueryRunner makeTimeSeriesQueryRunner(Segment adapter) { QueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); @@ -87,10 +83,7 @@ public static QueryRunner makeSearchQueryRunner(Segment adapter) final SearchQueryConfig config = new SearchQueryConfig(); QueryRunnerFactory factory = new SearchQueryRunnerFactory( new SearchStrategySelector(Suppliers.ofInstance(config)), - new SearchQueryQueryToolChest( - config, - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ), + new SearchQueryQueryToolChest(config), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); return new FinalizeResultsQueryRunner( 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 51e2a2cd5e6d..379b04a64a4b 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 @@ -177,9 +177,7 @@ public static AggregationTestHelper createTimeseriesQueryAggregationTestHelper( { ObjectMapper mapper = TestHelper.makeJsonMapper(); - TimeseriesQueryQueryToolChest toolchest = new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ); + TimeseriesQueryQueryToolChest toolchest = new TimeseriesQueryQueryToolChest(); TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( toolchest, @@ -218,10 +216,7 @@ public static AggregationTestHelper createTopNQueryAggregationTestHelper( { ObjectMapper mapper = TestHelper.makeJsonMapper(); - TopNQueryQueryToolChest toolchest = new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ); + TopNQueryQueryToolChest toolchest = new TopNQueryQueryToolChest(new TopNQueryConfig()); final CloseableStupidPool pool = new CloseableStupidPool<>( "TopNQueryRunnerFactory-bufferPool", 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 6894f71ad55c..25877f10dfdd 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 @@ -47,7 +47,6 @@ import org.apache.druid.query.BySegmentQueryRunner; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.FinalizeResultsQueryRunner; -import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; import org.apache.druid.query.Query; import org.apache.druid.query.QueryConfig; import org.apache.druid.query.QueryPlus; @@ -396,18 +395,12 @@ public String getFormatString() groupByFactory = new GroupByQueryRunnerFactory( strategySelector, - new GroupByQueryQueryToolChest( - strategySelector, - noopIntervalChunkingQueryRunnerDecorator() - ) + new GroupByQueryQueryToolChest(strategySelector) ); tooSmallGroupByFactory = new GroupByQueryRunnerFactory( tooSmallStrategySelector, - new GroupByQueryQueryToolChest( - tooSmallStrategySelector, - noopIntervalChunkingQueryRunnerDecorator() - ) + new GroupByQueryQueryToolChest(tooSmallStrategySelector) ); } @@ -688,23 +681,4 @@ public void registerQuery(Query query, ListenableFuture future) } }; - - public static IntervalChunkingQueryRunnerDecorator noopIntervalChunkingQueryRunnerDecorator() - { - return new IntervalChunkingQueryRunnerDecorator(null, null, null) - { - @Override - public QueryRunner decorate(final QueryRunner delegate, QueryToolChest> toolChest) - { - return new QueryRunner() - { - @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) - { - return delegate.run(queryPlus, responseContext); - } - }; - } - }; - } } 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 4044e2ff9d76..85f507134f60 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 @@ -26,7 +26,6 @@ import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.collections.CloseableDefaultBlockingPool; import org.apache.druid.collections.CloseableStupidPool; import org.apache.druid.data.input.InputRow; @@ -48,7 +47,6 @@ import org.apache.druid.query.BySegmentQueryRunner; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.FinalizeResultsQueryRunner; -import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; import org.apache.druid.query.Query; import org.apache.druid.query.QueryConfig; import org.apache.druid.query.QueryPlus; @@ -429,18 +427,12 @@ public String getFormatString() groupByFactory = new GroupByQueryRunnerFactory( strategySelector, - new GroupByQueryQueryToolChest( - strategySelector, - noopIntervalChunkingQueryRunnerDecorator() - ) + new GroupByQueryQueryToolChest(strategySelector) ); groupByFactory2 = new GroupByQueryRunnerFactory( strategySelector2, - new GroupByQueryQueryToolChest( - strategySelector2, - noopIntervalChunkingQueryRunnerDecorator() - ) + new GroupByQueryQueryToolChest(strategySelector2) ); } @@ -784,31 +776,5 @@ public static > QueryRunner makeQueryRunner( ); } - public static final QueryWatcher NOOP_QUERYWATCHER = new QueryWatcher() - { - @Override - public void registerQuery(Query query, ListenableFuture future) - { - - } - }; - - public static IntervalChunkingQueryRunnerDecorator noopIntervalChunkingQueryRunnerDecorator() - { - return new IntervalChunkingQueryRunnerDecorator(null, null, null) - { - @Override - public QueryRunner decorate(final QueryRunner delegate, QueryToolChest> toolChest) - { - return new QueryRunner() - { - @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) - { - return delegate.run(queryPlus, responseContext); - } - }; - } - }; - } + public static final QueryWatcher NOOP_QUERYWATCHER = (query, future) -> {}; } 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 0c4445abca27..35fca34e1c3c 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 @@ -44,7 +44,6 @@ import org.apache.druid.query.BySegmentQueryRunner; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.FinalizeResultsQueryRunner; -import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; import org.apache.druid.query.Query; import org.apache.druid.query.QueryConfig; import org.apache.druid.query.QueryPlus; @@ -286,10 +285,7 @@ public String getFormatString() groupByFactory = new GroupByQueryRunnerFactory( strategySelector, - new GroupByQueryQueryToolChest( - strategySelector, - noopIntervalChunkingQueryRunnerDecorator() - ) + new GroupByQueryQueryToolChest(strategySelector) ); } @@ -420,23 +416,4 @@ public void registerQuery(Query query, ListenableFuture future) } }; - - public static IntervalChunkingQueryRunnerDecorator noopIntervalChunkingQueryRunnerDecorator() - { - return new IntervalChunkingQueryRunnerDecorator(null, null, null) - { - @Override - public QueryRunner decorate(final QueryRunner delegate, QueryToolChest> toolChest) - { - return new QueryRunner() - { - @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) - { - return delegate.run(queryPlus, responseContext); - } - }; - } - }; - } } 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 8823db873516..221bc321b4aa 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 @@ -156,10 +156,7 @@ private static GroupByQueryRunnerFactory makeQueryRunnerFactory( QueryRunnerTestHelper.NOOP_QUERYWATCHER ) ); - final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest( - strategySelector, - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ); + final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(strategySelector); return new GroupByQueryRunnerFactory(strategySelector, toolChest); } 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 c27b14b974b1..ef112f408c52 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 @@ -113,13 +113,11 @@ public void testResultLevelCacheKeyWithPostAggregate() .build(); final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( - null, - QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() + null ).getCacheStrategy(query1); final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( - null, - QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() + null ).getCacheStrategy(query2); Assert.assertTrue(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2))); @@ -177,13 +175,11 @@ public void testResultLevelCacheKeyWithLimitSpec() .build(); final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( - null, - QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() + null ).getCacheStrategy(query1); final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( - null, - QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() + null ).getCacheStrategy(query2); Assert.assertTrue(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2))); @@ -243,13 +239,11 @@ public void testResultLevelCacheKeyWithHavingSpec() .build(); final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( - null, - QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() + null ).getCacheStrategy(query1); final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( - null, - QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() + null ).getCacheStrategy(query2); Assert.assertTrue(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2))); @@ -331,13 +325,11 @@ public void testResultLevelCacheKeyWithAndHavingSpec() .build(); final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( - null, - QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() + null ).getCacheStrategy(query1); final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( - null, - QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() + null ).getCacheStrategy(query2); Assert.assertTrue(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2))); @@ -426,13 +418,11 @@ public void testResultLevelCacheKeyWithHavingDimFilterHavingSpec() .build(); final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( - null, - QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() + null ).getCacheStrategy(query1); final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( - null, - QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() + null ).getCacheStrategy(query2); Assert.assertTrue(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2))); @@ -493,13 +483,11 @@ public void testResultLevelCacheKeyWithSubTotalsSpec() .build(); final CacheStrategy strategy1 = new GroupByQueryQueryToolChest( - null, - QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() + null ).getCacheStrategy(query1); final CacheStrategy strategy2 = new GroupByQueryQueryToolChest( - null, - QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() + null ).getCacheStrategy(query2); Assert.assertTrue(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2))); @@ -584,10 +572,7 @@ public void testResultSerde() throws Exception .setGranularity(QueryRunnerTestHelper.DAY_GRAN) .build(); - final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest( - null, - QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() - ); + final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(null); final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); final ObjectMapper arraysObjectMapper = toolChest.decorateObjectMapper( 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 02196e6dc85d..142f361b4e8c 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 @@ -118,10 +118,7 @@ private static GroupByQueryRunnerFactory makeQueryRunnerFactory( QueryRunnerTestHelper.NOOP_QUERYWATCHER ) ); - final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest( - strategySelector, - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ); + final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(strategySelector); return new GroupByQueryRunnerFactory(strategySelector, toolChest); } 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 fb61fc375d3e..b8662e01f1df 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 @@ -402,10 +402,7 @@ public ByteBuffer get() QueryRunnerTestHelper.NOOP_QUERYWATCHER ) ); - final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest( - strategySelector, - QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() - ); + final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(strategySelector); final Closer closer = Closer.create(); closer.register(bufferPool); closer.register(mergeBufferPool); @@ -1177,34 +1174,6 @@ public void testGroupByWithSortDimsFirst() TestHelper.assertExpectedObjects(expectedResults, results, "sort-by-dimensions-first"); } - @Test - public void testGroupByWithChunkPeriod() - { - GroupByQuery query = makeQueryBuilder() - .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) - .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) - .setDimensions(new DefaultDimensionSpec("quality", "alias")) - .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT, new LongSumAggregatorFactory("idx", "index")) - .setGranularity(QueryRunnerTestHelper.ALL_GRAN) - .overrideContext(ImmutableMap.of("chunkPeriod", "P1D")) - .build(); - - List expectedResults = Arrays.asList( - makeRow(query, "2011-04-01", "alias", "automotive", "rows", 2L, "idx", 282L), - makeRow(query, "2011-04-01", "alias", "business", "rows", 2L, "idx", 230L), - makeRow(query, "2011-04-01", "alias", "entertainment", "rows", 2L, "idx", 324L), - makeRow(query, "2011-04-01", "alias", "health", "rows", 2L, "idx", 233L), - makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 5317L), - makeRow(query, "2011-04-01", "alias", "news", "rows", 2L, "idx", 235L), - makeRow(query, "2011-04-01", "alias", "premium", "rows", 6L, "idx", 5405L), - makeRow(query, "2011-04-01", "alias", "technology", "rows", 2L, "idx", 175L), - makeRow(query, "2011-04-01", "alias", "travel", "rows", 2L, "idx", 245L) - ); - - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); - TestHelper.assertExpectedObjects(expectedResults, results, "chunk-period"); - } - @Test public void testGroupByNoAggregators() { @@ -4732,70 +4701,6 @@ public void testSubqueryWithMultipleIntervalsInOuterQuery() TestHelper.assertExpectedObjects(expectedResults, results, "subquery-multiple-intervals"); } - @Test - public void testSubqueryWithMultipleIntervalsInOuterQueryAndChunkPeriod() - { - GroupByQuery subquery = makeQueryBuilder() - .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) - .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) - .setDimensions(new DefaultDimensionSpec("quality", "alias")) - .setDimFilter(new JavaScriptDimFilter( - "quality", - "function(dim){ return true; }", - null, - JavaScriptConfig.getEnabledInstance() - )) - .setAggregatorSpecs( - QueryRunnerTestHelper.ROWS_COUNT, - new LongSumAggregatorFactory("idx", "index"), - new LongSumAggregatorFactory("indexMaxPlusTen", "indexMaxPlusTen") - ) - .setGranularity(QueryRunnerTestHelper.DAY_GRAN) - .overrideContext(ImmutableMap.of("chunkPeriod", "P1D")) - .build(); - - GroupByQuery query = makeQueryBuilder() - .setDataSource(subquery) - .setQuerySegmentSpec( - new MultipleIntervalSegmentSpec( - ImmutableList.of( - Intervals.of("2011-04-01T00:00:00.000Z/2011-04-01T23:58:00.000Z"), - Intervals.of("2011-04-02T00:00:00.000Z/2011-04-03T00:00:00.000Z") - ) - ) - ) - .setDimensions(new DefaultDimensionSpec("alias", "alias")) - .setAggregatorSpecs(new LongSumAggregatorFactory("rows", "rows"), new LongSumAggregatorFactory("idx", "idx")) - .setGranularity(QueryRunnerTestHelper.DAY_GRAN) - .build(); - - List expectedResults = Arrays.asList( - makeRow(query, "2011-04-01", "alias", "automotive", "rows", 1L, "idx", 135L), - makeRow(query, "2011-04-01", "alias", "business", "rows", 1L, "idx", 118L), - makeRow(query, "2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 158L), - makeRow(query, "2011-04-01", "alias", "health", "rows", 1L, "idx", 120L), - makeRow(query, "2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 2870L), - makeRow(query, "2011-04-01", "alias", "news", "rows", 1L, "idx", 121L), - makeRow(query, "2011-04-01", "alias", "premium", "rows", 3L, "idx", 2900L), - makeRow(query, "2011-04-01", "alias", "technology", "rows", 1L, "idx", 78L), - makeRow(query, "2011-04-01", "alias", "travel", "rows", 1L, "idx", 119L), - - makeRow(query, "2011-04-02", "alias", "automotive", "rows", 1L, "idx", 147L), - makeRow(query, "2011-04-02", "alias", "business", "rows", 1L, "idx", 112L), - makeRow(query, "2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 166L), - makeRow(query, "2011-04-02", "alias", "health", "rows", 1L, "idx", 113L), - makeRow(query, "2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 2447L), - makeRow(query, "2011-04-02", "alias", "news", "rows", 1L, "idx", 114L), - makeRow(query, "2011-04-02", "alias", "premium", "rows", 3L, "idx", 2505L), - makeRow(query, "2011-04-02", "alias", "technology", "rows", 1L, "idx", 97L), - makeRow(query, "2011-04-02", "alias", "travel", "rows", 1L, "idx", 126L) - ); - - // Subqueries are handled by the ToolChest - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); - TestHelper.assertExpectedObjects(expectedResults, results, "subquery-multiple-intervals"); - } - @Test public void testSubqueryWithExtractionFnInOuterQuery() { 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 3d290601f821..a294d5743e08 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 @@ -26,7 +26,6 @@ import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.collections.BlockingPool; import org.apache.druid.collections.DefaultBlockingPool; import org.apache.druid.collections.NonBlockingPool; @@ -49,7 +48,6 @@ import org.apache.druid.query.BySegmentQueryRunner; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.FinalizeResultsQueryRunner; -import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; import org.apache.druid.query.Query; import org.apache.druid.query.QueryConfig; import org.apache.druid.query.QueryPlus; @@ -357,18 +355,12 @@ public String getFormatString() groupByFactory = new GroupByQueryRunnerFactory( strategySelector, - new GroupByQueryQueryToolChest( - strategySelector, - noopIntervalChunkingQueryRunnerDecorator() - ) + new GroupByQueryQueryToolChest(strategySelector) ); groupByFactory2 = new GroupByQueryRunnerFactory( strategySelector2, - new GroupByQueryQueryToolChest( - strategySelector2, - noopIntervalChunkingQueryRunnerDecorator() - ) + new GroupByQueryQueryToolChest(strategySelector2) ); } @@ -925,24 +917,5 @@ private static > QueryRunner makeQueryRunnerFor ); } - public static final QueryWatcher NOOP_QUERYWATCHER = new QueryWatcher() - { - @Override - public void registerQuery(Query query, ListenableFuture future) - { - - } - }; - - public static IntervalChunkingQueryRunnerDecorator noopIntervalChunkingQueryRunnerDecorator() - { - return new IntervalChunkingQueryRunnerDecorator(null, null, null) - { - @Override - public QueryRunner decorate(final QueryRunner delegate, QueryToolChest> toolChest) - { - return (queryPlus, responseContext) -> delegate.run(queryPlus, responseContext); - } - }; - } + public static final QueryWatcher NOOP_QUERYWATCHER = (query, future) -> {}; } 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 d759c57b0266..6b503a541116 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 @@ -78,10 +78,7 @@ public class SearchQueryRunnerTest extends InitializedNullHandlingTest { private static final Logger LOG = new Logger(SearchQueryRunnerTest.class); private static final SearchQueryConfig CONFIG = new SearchQueryConfig(); - private static final SearchQueryQueryToolChest TOOL_CHEST = new SearchQueryQueryToolChest( - CONFIG, - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ); + private static final SearchQueryQueryToolChest TOOL_CHEST = new SearchQueryQueryToolChest(CONFIG); private static final SearchStrategySelector SELECTOR = new SearchStrategySelector(Suppliers.ofInstance(CONFIG)); @Parameterized.Parameters(name = "{0}") diff --git a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerWithCaseTest.java b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerWithCaseTest.java index 35ad5f814f1b..0bcb40fdcf52 100644 --- a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerWithCaseTest.java +++ b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerWithCaseTest.java @@ -115,10 +115,7 @@ static SearchQueryRunnerFactory makeRunnerFactory(final SearchQueryConfig config { return new SearchQueryRunnerFactory( new SearchStrategySelector(Suppliers.ofInstance(config)), - new SearchQueryQueryToolChest( - config, - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ), + new SearchQueryQueryToolChest(config), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); } diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java index d920bd1b17f8..812bf1e2629b 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java @@ -61,7 +61,7 @@ public static Iterable constructorFeeder() return QueryRunnerTestHelper.cartesian( QueryRunnerTestHelper.makeUnionQueryRunners( new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) @@ -143,7 +143,7 @@ public void testUnionResultMerging() ) .descending(descending) .build(); - QueryToolChest toolChest = new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()); + QueryToolChest toolChest = new TimeseriesQueryQueryToolChest(); final List> ds1 = Lists.newArrayList( new Result<>( DateTimes.of("2011-04-02"), diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java index b1ca4c728a50..a0d33e10ea42 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java @@ -111,7 +111,7 @@ public void testOneRowAtATime() throws Exception private List> runTimeseriesCount(IncrementalIndex index) { final QueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java index d0fc01e73cc2..4c0154b120c3 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -97,9 +97,7 @@ public static Iterable constructorFeeder() // runners QueryRunnerTestHelper.makeQueryRunners( new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) @@ -544,9 +542,7 @@ public void testTimeseriesGrandTotal() ); // Must create a toolChest so we can run mergeResults (which applies grand totals). - QueryToolChest, TimeseriesQuery> toolChest = new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ); + QueryToolChest, TimeseriesQuery> toolChest = new TimeseriesQueryQueryToolChest(); // Must wrapped in a results finalizer to stop the runner's builtin finalizer from being called. final FinalizeResultsQueryRunner finalRunner = new FinalizeResultsQueryRunner( @@ -595,9 +591,7 @@ public void testTimeseriesIntervalOutOfRanges() ); // Must create a toolChest so we can run mergeResults (which creates the zeroed-out row). - QueryToolChest, TimeseriesQuery> toolChest = new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ); + QueryToolChest, TimeseriesQuery> toolChest = new TimeseriesQueryQueryToolChest(); // Must wrapped in a results finalizer to stop the runner's builtin finalizer from being called. final FinalizeResultsQueryRunner finalRunner = new FinalizeResultsQueryRunner( @@ -2533,9 +2527,7 @@ public void testTimeSeriesWithSelectionFilterLookupExtractionFn() Iterable> results = runner.run(QueryPlus.wrap(query)).toList(); TestHelper.assertExpectedResults(expectedResults, results); - QueryToolChest, TimeseriesQuery> toolChest = new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ); + QueryToolChest, TimeseriesQuery> toolChest = new TimeseriesQueryQueryToolChest(); QueryRunner> optimizedRunner = toolChest.postMergeQueryDecoration( toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner))); Iterable> results2 = new FinalizeResultsQueryRunner(optimizedRunner, toolChest) @@ -2564,9 +2556,7 @@ public void testTimeseriesWithLimit() .build(); // Must create a toolChest so we can run mergeResults. - QueryToolChest, TimeseriesQuery> toolChest = new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ); + QueryToolChest, TimeseriesQuery> toolChest = new TimeseriesQueryQueryToolChest(); // Must wrapped in a results finalizer to stop the runner's builtin finalizer from being called. final FinalizeResultsQueryRunner finalRunner = new FinalizeResultsQueryRunner( diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryQueryToolChestTest.java index 30226542946f..760d1c268538 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryQueryToolChestTest.java @@ -257,10 +257,7 @@ public void testComputeResultLevelCacheKeyWithDifferentPostAgg() public void testMinTopNThreshold() { TopNQueryConfig config = new TopNQueryConfig(); - final TopNQueryQueryToolChest chest = new TopNQueryQueryToolChest( - config, - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ); + final TopNQueryQueryToolChest chest = new TopNQueryQueryToolChest(config); try (CloseableStupidPool pool = TestQueryRunners.createDefaultNonBlockingPool()) { QueryRunnerFactory factory = new TopNQueryRunnerFactory( pool, diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerBenchmark.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerBenchmark.java index 4fdc3d6abf27..cf8a3eb6e2e7 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerBenchmark.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerBenchmark.java @@ -96,7 +96,7 @@ public ByteBuffer get() } } ), - new TopNQueryQueryToolChest(new TopNQueryConfig(), QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()), + new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); TEST_CASE_MAP.put( diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java index 0ba5860dc74a..b9ee8d35bf33 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java @@ -160,10 +160,7 @@ public static List>> queryRunners() QueryRunnerTestHelper.makeQueryRunners( new TopNQueryRunnerFactory( defaultPool, - new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() - ), + new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) ) @@ -172,10 +169,7 @@ public static List>> queryRunners() QueryRunnerTestHelper.makeQueryRunners( new TopNQueryRunnerFactory( customPool, - new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() - ), + new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) ) @@ -258,10 +252,7 @@ private Sequence> runWithMerge(TopNQuery query) private Sequence> runWithMerge(TopNQuery query, ResponseContext context) { - final TopNQueryQueryToolChest chest = new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ); + final TopNQueryQueryToolChest chest = new TopNQueryQueryToolChest(new TopNQueryConfig()); final QueryRunner> mergeRunner = new FinalizeResultsQueryRunner( chest.mergeResults(runner), chest @@ -920,118 +911,6 @@ public void testTopNOverFirstLastAggregator() assertExpectedResults(expectedResults, query); } - @Test - public void testTopNOverFirstLastAggregatorChunkPeriod() - { - TopNQuery query = new TopNQueryBuilder() - .dataSource(QueryRunnerTestHelper.DATA_SOURCE) - .granularity(QueryRunnerTestHelper.MONTH_GRAN) - .dimension(QueryRunnerTestHelper.MARKET_DIMENSION) - .metric("last") - .threshold(3) - .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) - .aggregators( - new LongFirstAggregatorFactory("first", "index"), - new LongLastAggregatorFactory("last", "index") - ) - .context(ImmutableMap.of("chunkPeriod", "P1D")) - .build(); - - List> expectedResults = Arrays.asList( - new Result<>( - DateTimes.of("2011-01-01T00:00:00.000Z"), - new TopNResultValue( - Arrays.>asList( - ImmutableMap.builder() - .put("market", "total_market") - .put("first", 1000L) - .put("last", 1127L) - .build(), - ImmutableMap.builder() - .put("market", "upfront") - .put("first", 800L) - .put("last", 943L) - .build(), - ImmutableMap.builder() - .put("market", "spot") - .put("first", 100L) - .put("last", 155L) - .build() - ) - ) - ), - new Result<>( - DateTimes.of("2011-02-01T00:00:00.000Z"), - new TopNResultValue( - Arrays.>asList( - ImmutableMap.builder() - .put("market", "total_market") - .put("first", 1203L) - .put("last", 1292L) - .build(), - ImmutableMap.builder() - .put("market", "upfront") - .put("first", 1667L) - .put("last", 1101L) - .build(), - ImmutableMap.builder() - .put("market", "spot") - .put("first", 132L) - .put("last", 114L) - .build() - ) - ) - ), - new Result<>( - DateTimes.of("2011-03-01T00:00:00.000Z"), - new TopNResultValue( - Arrays.>asList( - ImmutableMap.builder() - .put("market", "total_market") - .put("first", 1124L) - .put("last", 1366L) - .build(), - ImmutableMap.builder() - .put("market", "upfront") - .put("first", 1166L) - .put("last", 1063L) - .build(), - ImmutableMap.builder() - .put("market", "spot") - .put("first", 153L) - .put("last", 125L) - .build() - ) - ) - ), - new Result<>( - DateTimes.of("2011-04-01T00:00:00.000Z"), - new TopNResultValue( - Arrays.>asList( - ImmutableMap.builder() - .put("market", "total_market") - .put("first", 1314L) - .put("last", 1029L) - .build(), - ImmutableMap.builder() - .put("market", "upfront") - .put("first", 1447L) - .put("last", 780L) - .build(), - ImmutableMap.builder() - .put("market", "spot") - .put("first", 135L) - .put("last", 120L) - .build() - ) - ) - ) - ); - - final Sequence> retval = runWithPreMergeAndMerge(query); - TestHelper.assertExpectedResults(expectedResults, retval); - } - @Test public void testTopNOverFirstLastFloatAggregatorUsingDoubleColumn() { diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java index 42e6a8e8244c..7a2607662380 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java @@ -70,20 +70,14 @@ public static Iterable constructorFeeder() QueryRunnerTestHelper.makeUnionQueryRunners( new TopNQueryRunnerFactory( defaultPool, - new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ), + new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) ), QueryRunnerTestHelper.makeUnionQueryRunners( new TopNQueryRunnerFactory( customPool, - new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ), + new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) ) diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java index fcd418073bd2..1c68a21568d3 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java @@ -571,9 +571,7 @@ public void testSpatialQuery() ); try { TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); @@ -625,9 +623,7 @@ public void testSpatialQueryWithOtherSpatialDim() ); try { TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); @@ -714,9 +710,7 @@ public void testSpatialQueryMorePoints() ); try { TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); diff --git a/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java b/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java index 11d5bdc67774..dfc3e533b289 100644 --- a/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java @@ -431,7 +431,7 @@ public void testSingleThreadedIndexingAndQuery() throws Exception final Segment incrementalIndexSegment = new IncrementalIndexSegment(index, null); final QueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); @@ -528,7 +528,7 @@ public void testConcurrentAddRead() throws InterruptedException, ExecutionExcept final List> queryFutures = Lists.newArrayListWithExpectedSize(concurrentThreads); final Segment incrementalIndexSegment = new IncrementalIndexSegment(index, null); final QueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java index bf29d87b0418..9328904fc7bf 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java @@ -505,8 +505,7 @@ public void testSpatialQuery() ); try { TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); @@ -592,8 +591,7 @@ public void testSpatialQueryMorePoints() ); try { TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); @@ -685,8 +683,7 @@ public void testSpatialQueryFilteredAggregator() ); try { TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java index 71425369c093..3ab78fc986ac 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java @@ -561,8 +561,7 @@ public void testSpatialQuery() ); try { TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); @@ -614,8 +613,7 @@ public void testSpatialQueryWithOtherSpatialDim() ); try { TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); @@ -702,8 +700,7 @@ public void testSpatialQueryMorePoints() ); try { TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java b/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java index 17607728b735..55dba3f7ceeb 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java @@ -344,7 +344,7 @@ public void testConcurrentAddRead() final List> queryFutures = new ArrayList<>(); final Segment incrementalIndexSegment = new IncrementalIndexSegment(incrementalIndex, null); final QueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ); 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 cd9e6d071fd6..a377c1c92478 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -429,11 +429,7 @@ public void run() .postAggregators(POST_AGGS) .context(CONTEXT); - QueryRunner runner = new FinalizeResultsQueryRunner( - getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ) - ); + QueryRunner runner = new FinalizeResultsQueryRunner(getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest()); testQueryCaching( runner, @@ -468,11 +464,7 @@ public void testTimeseriesCaching() .postAggregators(POST_AGGS) .context(CONTEXT); - QueryRunner runner = new FinalizeResultsQueryRunner( - getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ) - ); + QueryRunner runner = new FinalizeResultsQueryRunner(getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest()); testQueryCaching( runner, @@ -594,10 +586,7 @@ public void testTimeseriesMergingOutOfOrderPartitions() .postAggregators(POST_AGGS) .context(CONTEXT); - QueryRunner runner = new FinalizeResultsQueryRunner( - getDefaultQueryRunner(), - new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()) - ); + QueryRunner runner = new FinalizeResultsQueryRunner(getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest()); testQueryCaching( runner, @@ -655,11 +644,7 @@ public void testTimeseriesCachingTimeZone() .postAggregators(POST_AGGS) .context(CONTEXT); - QueryRunner runner = new FinalizeResultsQueryRunner( - getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ) - ); + QueryRunner runner = new FinalizeResultsQueryRunner(getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest()); testQueryCaching( runner, @@ -699,10 +684,9 @@ public void testDisableUseCache() .aggregators(AGGS) .postAggregators(POST_AGGS) .context(CONTEXT); - QueryRunner runner = new FinalizeResultsQueryRunner( - getDefaultQueryRunner(), - new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()) - ); + + QueryRunner runner = new FinalizeResultsQueryRunner(getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest()); + testQueryCaching( runner, 1, @@ -775,10 +759,7 @@ public void testTopNCaching() QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), - new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ) + new TopNQueryQueryToolChest(new TopNQueryConfig()) ); testQueryCaching( @@ -851,10 +832,7 @@ public void testTopNCachingTimeZone() QueryRunner runner = new FinalizeResultsQueryRunner( getDefaultQueryRunner(), - new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ) + new TopNQueryQueryToolChest(new TopNQueryConfig()) ); testQueryCaching( @@ -955,10 +933,8 @@ public void testTopNCachingEmptyResults() .context(CONTEXT); QueryRunner runner = new FinalizeResultsQueryRunner( - getDefaultQueryRunner(), new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ) + getDefaultQueryRunner(), + new TopNQueryQueryToolChest(new TopNQueryConfig()) ); testQueryCaching( runner, @@ -1027,11 +1003,10 @@ public void testTopNOnPostAggMetricCaching() .context(CONTEXT); QueryRunner runner = new FinalizeResultsQueryRunner( - getDefaultQueryRunner(), new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ) + getDefaultQueryRunner(), + new TopNQueryQueryToolChest(new TopNQueryConfig()) ); + testQueryCaching( runner, builder.build(), @@ -1127,11 +1102,10 @@ public void testSearchCaching() ); QueryRunner runner = new FinalizeResultsQueryRunner( - getDefaultQueryRunner(), new SearchQueryQueryToolChest( - new SearchQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ) + getDefaultQueryRunner(), + new SearchQueryQueryToolChest(new SearchQueryConfig()) ); + TestHelper.assertExpectedResults( makeSearchResults( TOP_DIM, @@ -1196,11 +1170,10 @@ public void testSearchCachingRenamedOutput() ); QueryRunner runner = new FinalizeResultsQueryRunner( - getDefaultQueryRunner(), new SearchQueryQueryToolChest( - new SearchQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ) + getDefaultQueryRunner(), + new SearchQueryQueryToolChest(new SearchQueryConfig()) ); + ResponseContext context = ResponseContext.createEmpty(); TestHelper.assertExpectedResults( makeSearchResults( @@ -1437,11 +1410,7 @@ public void testTimeSeriesWithFilter() .postAggregators(POST_AGGS) .context(CONTEXT); - QueryRunner runner = new FinalizeResultsQueryRunner( - getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ) - ); + QueryRunner runner = new FinalizeResultsQueryRunner(getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest()); /* For dim0 (2011-01-01/2011-01-05), the combined range is {[1,1], [222,333]}, so segments [-inf,1], [1,2], [2,3], and @@ -1510,11 +1479,7 @@ public void testSingleDimensionPruning() final Interval interval2 = Intervals.of("2011-01-07/2011-01-08"); final Interval interval3 = Intervals.of("2011-01-08/2011-01-09"); - QueryRunner runner = new FinalizeResultsQueryRunner( - getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ) - ); + QueryRunner runner = new FinalizeResultsQueryRunner(getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest()); final DruidServer lastServer = servers[random.nextInt(servers.length)]; ServerSelector selector1 = makeMockSingleDimensionSelector(lastServer, "dim1", null, "b", 1); diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTestUtils.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTestUtils.java index 16f75be51976..5a9450697415 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTestUtils.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTestUtils.java @@ -27,7 +27,6 @@ import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.MapQueryToolChestWarehouse; import org.apache.druid.query.Query; -import org.apache.druid.query.QueryRunnerTestHelper; import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.groupby.GroupByQuery; @@ -65,23 +64,15 @@ public static Pair createWarehouse( ImmutableMap., QueryToolChest>builder() .put( TimeseriesQuery.class, - new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ) + new TimeseriesQueryQueryToolChest() ) .put( TopNQuery.class, - new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ) + new TopNQueryQueryToolChest(new TopNQueryConfig()) ) .put( SearchQuery.class, - new SearchQueryQueryToolChest( - new SearchQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ) + new SearchQueryQueryToolChest(new SearchQueryConfig()) ) .put( GroupByQuery.class, diff --git a/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java b/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java index 8fd83381f0b4..ef6bc0a79212 100644 --- a/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java @@ -140,10 +140,7 @@ public void testCloseAndPopulate() throws Exception .aggregators(AGGS) .granularity(Granularities.ALL); - QueryToolChest toolchest = new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ); + QueryToolChest toolchest = new TopNQueryQueryToolChest(new TopNQueryConfig()); testCloseAndPopulate(expectedRes, expectedCacheRes, builder.build(), toolchest); testUseCache(expectedCacheRes, builder.build(), toolchest); @@ -189,9 +186,7 @@ public void testTimeseries() throws Exception expectedResults = Lists.newArrayList(row1, row2); } - QueryToolChest toolChest = new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ); + QueryToolChest toolChest = new TimeseriesQueryQueryToolChest(); testCloseAndPopulate(expectedResults, expectedResults, query, toolChest); testUseCache(expectedResults, query, toolChest); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java index fe4b61283e00..1584df56f2ed 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java @@ -37,7 +37,6 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.query.DefaultGenericQueryMetricsFactory; import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; -import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; import org.apache.druid.query.QueryRunnerTestHelper; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; @@ -240,13 +239,7 @@ public Map makeLoadSpec(URI uri) new DefaultQueryRunnerFactoryConglomerate( ImmutableMap.of( TimeseriesQuery.class, new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest( - new IntervalChunkingQueryRunnerDecorator( - queryExecutor, - QueryRunnerTestHelper.NOOP_QUERYWATCHER, - emitter - ) - ), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java index 402bb3ab5d32..2f1626d2da58 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java @@ -610,7 +610,7 @@ public int getNumMergeBuffers() .put( TimeseriesQuery.class, new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator()), + new TimeseriesQueryQueryToolChest(), new TimeseriesQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) @@ -619,10 +619,7 @@ public int getNumMergeBuffers() TopNQuery.class, new TopNQueryRunnerFactory( stupidPool, - new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() - ), + new TopNQueryQueryToolChest(new TopNQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) ) diff --git a/website/.spelling b/website/.spelling index cdf9afd275d3..1128762fcb3f 100644 --- a/website/.spelling +++ b/website/.spelling @@ -1234,7 +1234,6 @@ TimeseriesQuery D1 D2 D3 -chunkPeriod druid.query.groupBy.defaultStrategy druid.query.groupBy.maxMergingDictionarySize druid.query.groupBy.maxOnDiskStorage @@ -1560,7 +1559,6 @@ allowAll array_mod batch_index_task cgroup -chunkPeriod classloader com.metamx common.runtime.properties From 89cc830dbb1fa5af7e9a6d0448ed5bc53ceddebc Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Sat, 18 Jan 2020 06:53:32 -0800 Subject: [PATCH 2/3] Remove unused import. --- .../java/org/apache/druid/client/CachingClusteredClientTest.java | 1 - 1 file changed, 1 deletion(-) 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 a377c1c92478..bd91104aac41 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -75,7 +75,6 @@ import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; -import org.apache.druid.query.QueryRunnerTestHelper; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.Result; import org.apache.druid.query.SegmentDescriptor; From c1dd21c70361ed25a6a4390e126948d3fb90b1fb Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Sun, 19 Jan 2020 10:17:48 -0800 Subject: [PATCH 3/3] Remove chunkInterval too. --- .../java/org/apache/druid/query/DefaultQueryMetrics.java | 6 ------ .../src/main/java/org/apache/druid/query/QueryMetrics.java | 3 --- .../druid/query/search/DefaultSearchQueryMetrics.java | 7 ------- website/.spelling | 1 - 4 files changed, 17 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java index bffdf6779029..145f8dac3527 100644 --- a/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java +++ b/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java @@ -156,12 +156,6 @@ public void segment(String segmentIdentifier) setDimension("segment", segmentIdentifier); } - @Override - public void chunkInterval(Interval interval) - { - setDimension("chunkInterval", interval.toString()); - } - @Override public void preFilters(List preFilters) { diff --git a/processing/src/main/java/org/apache/druid/query/QueryMetrics.java b/processing/src/main/java/org/apache/druid/query/QueryMetrics.java index 535dccfb2857..fa342aa8b56a 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryMetrics.java +++ b/processing/src/main/java/org/apache/druid/query/QueryMetrics.java @@ -25,7 +25,6 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.search.SearchQueryMetricsFactory; -import org.joda.time.Interval; import java.util.List; @@ -224,8 +223,6 @@ public interface QueryMetrics> void segment(String segmentIdentifier); - void chunkInterval(Interval interval); - void preFilters(List preFilters); void postFilters(List postFilters); diff --git a/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java b/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java index a50bc95899bc..67ffead23cd8 100644 --- a/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java +++ b/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java @@ -26,7 +26,6 @@ import org.apache.druid.query.Query; import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.filter.Filter; -import org.joda.time.Interval; import java.util.List; @@ -136,12 +135,6 @@ public void segment(String segmentIdentifier) delegateQueryMetrics.segment(segmentIdentifier); } - @Override - public void chunkInterval(Interval interval) - { - delegateQueryMetrics.chunkInterval(interval); - } - @Override public void preFilters(List preFilters) { diff --git a/website/.spelling b/website/.spelling index 1128762fcb3f..213c19008507 100644 --- a/website/.spelling +++ b/website/.spelling @@ -1114,7 +1114,6 @@ SysMonitor TaskCountStatsMonitor bufferCapacity bufferPoolName -chunkInterval cms cpuName cpuTime