diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java index 87a421df5fcb..2d902c12163f 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java @@ -52,13 +52,10 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; -import org.apache.druid.query.groupby.GroupByQueryEngine; import org.apache.druid.query.groupby.GroupByQueryQueryToolChest; import org.apache.druid.query.groupby.GroupByQueryRunnerFactory; +import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.ResultRow; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV1; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV2; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.IndexIO; @@ -128,9 +125,6 @@ public class GroupByTypeInterfaceBenchmark @Param({"100000"}) private int rowsPerSegment; - @Param({"v2"}) - private String defaultStrategy; - @Param({"all"}) private String queryGranularity; @@ -346,11 +340,6 @@ public void setup() throws IOException ); final GroupByQueryConfig config = new GroupByQueryConfig() { - @Override - public String getDefaultStrategy() - { - return defaultStrategy; - } @Override public int getBufferGrouperInitialBuckets() @@ -365,8 +354,6 @@ public HumanReadableBytes getMaxOnDiskStorage() } }; config.setSingleThreaded(false); - config.setMaxIntermediateRows(Integer.MAX_VALUE); - config.setMaxResults(Integer.MAX_VALUE); DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig() { @@ -385,27 +372,19 @@ public String getFormatString() }; final Supplier configSupplier = Suppliers.ofInstance(config); - final GroupByStrategySelector strategySelector = new GroupByStrategySelector( + final GroupingEngine groupingEngine = new GroupingEngine( + druidProcessingConfig, configSupplier, - new GroupByStrategyV1( - configSupplier, - new GroupByQueryEngine(configSupplier, bufferPool), - QueryBenchmarkUtil.NOOP_QUERYWATCHER - ), - new GroupByStrategyV2( - druidProcessingConfig, - configSupplier, - bufferPool, - mergePool, - TestHelper.makeJsonMapper(), - new ObjectMapper(new SmileFactory()), - QueryBenchmarkUtil.NOOP_QUERYWATCHER - ) + bufferPool, + mergePool, + TestHelper.makeJsonMapper(), + new ObjectMapper(new SmileFactory()), + QueryBenchmarkUtil.NOOP_QUERYWATCHER ); factory = new GroupByQueryRunnerFactory( - strategySelector, - new GroupByQueryQueryToolChest(strategySelector) + groupingEngine, + new GroupByQueryQueryToolChest(groupingEngine) ); } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java index e38ee8862dcb..5adfcee0de8e 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -77,14 +77,11 @@ import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; -import org.apache.druid.query.groupby.GroupByQueryEngine; import org.apache.druid.query.groupby.GroupByQueryQueryToolChest; import org.apache.druid.query.groupby.GroupByQueryRunnerFactory; import org.apache.druid.query.groupby.GroupByQueryRunnerTest; +import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.ResultRow; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV1; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV2; import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; @@ -287,11 +284,6 @@ public int getNumThreads() GroupByQueryRunnerTest.DEFAULT_MAPPER, new GroupByQueryConfig() { - @Override - public String getDefaultStrategy() - { - return GroupByStrategySelector.STRATEGY_V2; - } }, processingConfig ) @@ -364,25 +356,17 @@ private static GroupByQueryRunnerFactory makeGroupByQueryRunnerFactory( bufferSupplier, processingConfig.getNumMergeBuffers() ); - final GroupByStrategySelector strategySelector = new GroupByStrategySelector( + final GroupingEngine groupingEngine = new GroupingEngine( + processingConfig, configSupplier, - new GroupByStrategyV1( - configSupplier, - new GroupByQueryEngine(configSupplier, bufferPool), - QueryRunnerTestHelper.NOOP_QUERYWATCHER - ), - new GroupByStrategyV2( - processingConfig, - configSupplier, - bufferPool, - mergeBufferPool, - mapper, - mapper, - QueryRunnerTestHelper.NOOP_QUERYWATCHER - ) + bufferPool, + mergeBufferPool, + mapper, + mapper, + QueryRunnerTestHelper.NOOP_QUERYWATCHER ); - final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(strategySelector); - return new GroupByQueryRunnerFactory(strategySelector, toolChest); + final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(groupingEngine); + return new GroupByQueryRunnerFactory(groupingEngine, toolChest); } @TearDown(Level.Trial) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java index 08c51b9edf2b..d355dd2d0058 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java @@ -63,15 +63,12 @@ import org.apache.druid.query.filter.BoundDimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; -import org.apache.druid.query.groupby.GroupByQueryEngine; import org.apache.druid.query.groupby.GroupByQueryQueryToolChest; import org.apache.druid.query.groupby.GroupByQueryRunnerFactory; +import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV1; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV2; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; @@ -139,9 +136,6 @@ public class GroupByBenchmark @Param({"basic.A", "basic.nested"}) private String schemaAndQuery; - @Param({"v1", "v2"}) - private String defaultStrategy; - @Param({"all", "day"}) private String queryGranularity; @@ -461,11 +455,6 @@ public void setup() ); final GroupByQueryConfig config = new GroupByQueryConfig() { - @Override - public String getDefaultStrategy() - { - return defaultStrategy; - } @Override public int getBufferGrouperInitialBuckets() @@ -480,8 +469,6 @@ public HumanReadableBytes getMaxOnDiskStorage() } }; config.setSingleThreaded(false); - config.setMaxIntermediateRows(Integer.MAX_VALUE); - config.setMaxResults(Integer.MAX_VALUE); DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig() { @@ -500,27 +487,19 @@ public String getFormatString() }; final Supplier configSupplier = Suppliers.ofInstance(config); - final GroupByStrategySelector strategySelector = new GroupByStrategySelector( + final GroupingEngine groupingEngine = new GroupingEngine( + druidProcessingConfig, configSupplier, - new GroupByStrategyV1( - configSupplier, - new GroupByQueryEngine(configSupplier, bufferPool), - QueryBenchmarkUtil.NOOP_QUERYWATCHER - ), - new GroupByStrategyV2( - druidProcessingConfig, - configSupplier, - bufferPool, - mergePool, - TestHelper.makeJsonMapper(), - new ObjectMapper(new SmileFactory()), - QueryBenchmarkUtil.NOOP_QUERYWATCHER - ) + bufferPool, + mergePool, + TestHelper.makeJsonMapper(), + new ObjectMapper(new SmileFactory()), + QueryBenchmarkUtil.NOOP_QUERYWATCHER ); factory = new GroupByQueryRunnerFactory( - strategySelector, - new GroupByQueryQueryToolChest(strategySelector) + groupingEngine, + new GroupByQueryQueryToolChest(groupingEngine) ); } diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 048564580ef8..36023ecf2786 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1502,7 +1502,7 @@ Processing properties set on the MiddleManager will be passed through to Peons. |`druid.processing.buffer.sizeBytes`|This specifies a buffer size (less than 2GiB) for the storage of intermediate results. The computation engine in both the Historical and Realtime processes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed. [Human-readable format](human-readable-byte.md) is supported.|auto (max 1 GiB)| |`druid.processing.buffer.poolCacheMaxCount`|Processing buffer pool caches the buffers for later use. This is the maximum count that the cache will grow to. Note that pool can create more buffers than it can cache if necessary.|`Integer.MAX_VALUE`| |`druid.processing.formatString`|Realtime and Historical processes use this format string to name their processing threads.|processing-%s| -|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`| +|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`| |`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)| |`druid.processing.fifo`|Enables the processing queue to treat tasks of equal priority in a FIFO manner.|`true`| |`druid.processing.tmpDir`|Path where temporary files created while processing a query should be stored. If specified, this configuration takes priority over the default `java.io.tmpdir` path.|path represented by `java.io.tmpdir`| @@ -1651,7 +1651,7 @@ Druid uses Jetty to serve HTTP requests. |`druid.processing.buffer.sizeBytes`|This specifies a buffer size (less than 2GiB) for the storage of intermediate results. The computation engine in the Indexer processes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed. [Human-readable format](human-readable-byte.md) is supported.|auto (max 1GiB)| |`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE| |`druid.processing.formatString`|Indexer processes use this format string to name their processing threads.|processing-%s| -|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`| +|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`| |`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)| |`druid.processing.fifo`|If the processing queue should treat tasks of equal priority in a FIFO manner|`true`| |`druid.processing.tmpDir`|Path where temporary files created while processing a query should be stored. If specified, this configuration takes priority over the default `java.io.tmpdir` path.|path represented by `java.io.tmpdir`| @@ -1760,7 +1760,7 @@ Druid uses Jetty to serve HTTP requests. |`druid.processing.buffer.sizeBytes`|This specifies a buffer size (less than 2GiB), for the storage of intermediate results. The computation engine in both the Historical and Realtime processes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed. [Human-readable format](human-readable-byte.md) is supported.|auto (max 1GiB)| |`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE| |`druid.processing.formatString`|Realtime and Historical processes use this format string to name their processing threads.|processing-%s| -|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`| +|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`| |`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)| |`druid.processing.fifo`|If the processing queue should treat tasks of equal priority in a FIFO manner|`true`| |`druid.processing.tmpDir`|Path where temporary files created while processing a query should be stored. If specified, this configuration takes priority over the default `java.io.tmpdir` path.|path represented by `java.io.tmpdir`| @@ -1939,7 +1939,7 @@ The broker uses processing configs for nested groupBy queries. |`druid.processing.buffer.sizeBytes`|This specifies a buffer size (less than 2GiB) for the storage of intermediate results. The computation engine in both the Historical and Realtime processes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed. [Human-readable format](human-readable-byte.md) is supported.|auto (max 1GiB)| |`druid.processing.buffer.poolCacheInitialCount`|initializes the number of buffers allocated on the intermediate results pool. Note that pool can create more buffers if necessary.|`0`| |`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE| -|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`| +|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`| |`druid.processing.fifo`|If the processing queue should treat tasks of equal priority in a FIFO manner|`true`| |`druid.processing.tmpDir`|Path where temporary files created while processing a query should be stored. If specified, this configuration takes priority over the default `java.io.tmpdir` path.|path represented by `java.io.tmpdir`| |`druid.processing.merge.useParallelMergePool`|Enable automatic parallel merging for Brokers on a dedicated async ForkJoinPool. If `false`, instead merges will be done serially on the `HTTP` thread pool.|`true`| @@ -2027,8 +2027,8 @@ You can optionally only configure caching to be enabled on the Broker by setting See [cache configuration](#cache-configuration) for how to configure cache settings. :::info - Note: Even if cache is enabled, for [groupBy v2](../querying/groupbyquery.md#strategies) queries, segment level cache do not work on Brokers. - See [Differences between v1 and v2](../querying/groupbyquery.md#differences-between-v1-and-v2) and [Query caching](../querying/caching.md) for more information. + Note: Even if cache is enabled, for [groupBy](../querying/groupbyquery.md) queries, segment level cache does not work on Brokers. + See [Query caching](../querying/caching.md) for more information. ::: #### Segment Discovery @@ -2203,8 +2203,6 @@ context). If query does have `maxQueuedBytes` in the context, then that value is This section describes the configurations for groupBy queries. You can set the runtime properties in the `runtime.properties` file on Broker, Historical, and MiddleManager processes. You can set the query context parameters through the [query context](../querying/query-context.md). -#### Configurations for groupBy v2 - Supported runtime properties: |Property|Description|Default| @@ -2225,29 +2223,11 @@ Supported query contexts: ### Advanced configurations -#### Common configurations for all groupBy strategies - Supported runtime properties: |Property|Description|Default| |--------|-----------|-------| -|`druid.query.groupBy.defaultStrategy`|Default groupBy query strategy.|v2| |`druid.query.groupBy.singleThreaded`|Merge results using a single thread.|false| - -Supported query contexts: - -|Key|Description| -|---|-----------| -|`groupByStrategy`|Overrides the value of `druid.query.groupBy.defaultStrategy` for this query.| -|`groupByIsSingleThreaded`|Overrides the value of `druid.query.groupBy.singleThreaded` for this query.| - - -#### GroupBy v2 configurations - -Supported runtime properties: - -|Property|Description|Default| -|--------|-----------|-------| |`druid.query.groupBy.bufferGrouperInitialBuckets`|Initial number of buckets in the off-heap hash table used for grouping results. Set to 0 to use a reasonable default (1024).|0| |`druid.query.groupBy.bufferGrouperMaxLoadFactor`|Maximum load factor of the off-heap hash table used for grouping results. When the load factor exceeds this size, the table will be grown or spilled to disk. Set to 0 to use a reasonable default (0.7).|0| |`druid.query.groupBy.forceHashAggregation`|Force to use hash-based aggregation.|false| @@ -2258,6 +2238,7 @@ Supported query contexts: |Key|Description|Default| |---|-----------|-------| +|`groupByIsSingleThreaded`|Overrides the value of `druid.query.groupBy.singleThreaded` for this query.| |`bufferGrouperInitialBuckets`|Overrides the value of `druid.query.groupBy.bufferGrouperInitialBuckets` for this query.|None| |`bufferGrouperMaxLoadFactor`|Overrides the value of `druid.query.groupBy.bufferGrouperMaxLoadFactor` for this query.|None| |`forceHashAggregation`|Overrides the value of `druid.query.groupBy.forceHashAggregation`|None| @@ -2266,24 +2247,6 @@ Supported query contexts: |`sortByDimsFirst`|Sort the results first by dimension values and then by timestamp.|false| |`forceLimitPushDown`|When all fields in the orderby are part of the grouping key, the broker will push limit application down to the Historical processes. When the sorting order uses fields that are not in the grouping key, applying this optimization can result in approximate results with unknown accuracy, so this optimization is disabled by default in that case. Enabling this context flag turns on limit push down for limit/orderbys that contain non-grouping key columns.|false| - -#### GroupBy v1 configurations - -Supported runtime properties: - -|Property|Description|Default| -|--------|-----------|-------| -|`druid.query.groupBy.maxIntermediateRows`|Maximum number of intermediate rows for the per-segment grouping engine. This is a tuning parameter that does not impose a hard limit; rather, it potentially shifts merging work from the per-segment engine to the overall merging index. Queries that exceed this limit will not fail.|50000| -|`druid.query.groupBy.maxResults`|Maximum number of results. Queries that exceed this limit will fail.|500000| - -Supported query contexts: - -|Key|Description|Default| -|---|-----------|-------| -|`maxIntermediateRows`|Ignored by groupBy v2. Can be used to lower the value of `druid.query.groupBy.maxIntermediateRows` for a groupBy v1 query.|None| -|`maxResults`|Ignored by groupBy v2. Can be used to lower the value of `druid.query.groupBy.maxResults` for a groupBy v1 query.|None| -|`useOffheap`|Ignored by groupBy v2, and no longer supported for groupBy v1. Enabling this option with groupBy v1 will result in an error. For off-heap aggregation, switch to groupBy v2, which always operates off-heap.|false| - #### Expression processing configurations |Key|Description|Default| diff --git a/docs/development/extensions-core/kafka-ingestion.md b/docs/development/extensions-core/kafka-ingestion.md index 46426e55f27c..329967747bfa 100644 --- a/docs/development/extensions-core/kafka-ingestion.md +++ b/docs/development/extensions-core/kafka-ingestion.md @@ -135,11 +135,12 @@ The following example demonstrates a supervisor spec for Kafka that uses the `JS If you want to parse the Kafka metadata fields in addition to the Kafka payload value contents, you can use the `kafka` input format. The `kafka` input format wraps around the payload parsing input format and augments the data it outputs with the Kafka event timestamp, -the Kafka event headers, and the key field that itself can be parsed using any available InputFormat. +the Kafka topic name, the Kafka event headers, and the key field that itself can be parsed using any available InputFormat. For example, consider the following structure for a Kafka message that represents a fictitious wiki edit in a development environment: - **Kafka timestamp**: `1680795276351` +- **Kafka topic**: `wiki-edits` - **Kafka headers**: - `env=development` - `zone=z1` @@ -153,6 +154,7 @@ You would configure it as follows: - `valueFormat`: Define how to parse the payload value. Set this to the payload parsing input format (`{ "type": "json" }`). - `timestampColumnName`: Supply a custom name for the Kafka timestamp in the Druid schema to avoid conflicts with columns from the payload. The default is `kafka.timestamp`. +- `topicColumnName`: Supply a custom name for the Kafka topic in the Druid schema to avoid conflicts with columns from the payload. The default is `kafka.topic`. This field is useful when ingesting data from multiple topics into same datasource. - `headerFormat`: The default value `string` decodes strings in UTF-8 encoding from the Kafka header. Other supported encoding formats include the following: - `ISO-8859-1`: ISO Latin Alphabet No. 1, that is, ISO-LATIN-1. @@ -174,7 +176,7 @@ You would configure it as follows: Note that for `tsv`,`csv`, and `regex` formats, you need to provide a `columns` array to make a valid input format. Only the first one is used, and its name will be ignored in favor of `keyColumnName`. - `keyColumnName`: Supply the name for the Kafka key column to avoid conflicts with columns from the payload. The default is `kafka.key`. -Putting it together, the following input format (that uses the default values for `timestampColumnName`, `headerColumnPrefix`, and `keyColumnName`) +Putting it together, the following input format (that uses the default values for `timestampColumnName`, `topicColumnName`, `headerColumnPrefix`, and `keyColumnName`) ```json { @@ -203,6 +205,7 @@ would parse the example message as follows: "delta": 31, "namespace": "Main", "kafka.timestamp": 1680795276351, + "kafka.topic": "wiki-edits", "kafka.header.env": "development", "kafka.header.zone": "z1", "kafka.key": "wiki-edit" @@ -213,7 +216,7 @@ For more information on data formats, see [Data formats](../../ingestion/data-fo Finally, add these Kafka metadata columns to the `dimensionsSpec` or set your `dimensionsSpec` to auto-detect columns. -The following supervisor spec demonstrates how to ingest the Kafka header, key, and timestamp into Druid dimensions: +The following supervisor spec demonstrates how to ingest the Kafka header, key, timestamp, and topic into Druid dimensions: ``` { @@ -270,15 +273,16 @@ After Druid ingests the data, you can query the Kafka metadata columns as follow SELECT "kafka.header.env", "kafka.key", - "kafka.timestamp" + "kafka.timestamp", + "kafka.topic" FROM "wikiticker" ``` This query returns: -| `kafka.header.env` | `kafka.key` | `kafka.timestamp` | -|--------------------|-----------|---------------| -| `development` | `wiki-edit` | `1680795276351` | +| `kafka.header.env` | `kafka.key` | `kafka.timestamp` | `kafka.topic` | +|--------------------|-----------|---------------|---------------| +| `development` | `wiki-edit` | `1680795276351` | `wiki-edits` | For more information, see [`kafka` data format](../../ingestion/data-formats.md#kafka). diff --git a/docs/ingestion/data-formats.md b/docs/ingestion/data-formats.md index e3ad22a9163b..7dd1b10c7fa9 100644 --- a/docs/ingestion/data-formats.md +++ b/docs/ingestion/data-formats.md @@ -560,17 +560,19 @@ Configure the Kafka `inputFormat` as follows: | `type` | String | Set value to `kafka`. | yes | | `valueFormat` | [InputFormat](#input-format) | Any [InputFormat](#input-format) to parse the Kafka value payload. For details about specifying the input format, see [Specifying data format](../development/extensions-core/kafka-supervisor-reference.md#specifying-data-format). | yes | | `timestampColumnName` | String | Name of the column for the kafka record's timestamp.| no (default = "kafka.timestamp") | +| `topicColumnName` | String |Name of the column for the kafka record's topic. It is useful when ingesting data from multiple topics.| no (default = "kafka.timestamp") | | `headerColumnPrefix` | String | Custom prefix for all the header columns. | no (default = "kafka.header.") | | `headerFormat` | Object | `headerFormat` specifies how to parse the Kafka headers. Supports String types. Because Kafka header values are bytes, the parser decodes them as UTF-8 encoded strings. To change this behavior, implement your own parser based on the encoding style. Change the 'encoding' type in `KafkaStringHeaderFormat` to match your custom implementation. | no | | `keyFormat` | [InputFormat](#input-format) | Any [input format](#input-format) to parse the Kafka key. It only processes the first entry of the `inputFormat` field. For details, see [Specifying data format](../development/extensions-core/kafka-supervisor-reference.md#specifying-data-format). | no | | `keyColumnName` | String | Name of the column for the kafka record's key.| no (default = "kafka.key") | + The Kafka input format augments the payload with information from the Kafka timestamp, headers, and key. If there are conflicts between column names in the payload and those created from the metadata, the payload takes precedence. This ensures that upgrading a Kafka ingestion to use the Kafka input format (by taking its existing input format and setting it as the `valueFormat`) can be done without losing any of the payload data. -Here is a minimal example that only augments the parsed payload with the Kafka timestamp column: +Here is a minimal example that only augments the parsed payload with the Kafka timestamp column and kafka topic column: ``` "ioConfig": { @@ -594,6 +596,7 @@ Here is a complete example: "type": "json" } "timestampColumnName": "kafka.timestamp", + "topicColumnName": "kafka.topic", "headerFormat": { "type": "string", "encoding": "UTF-8" diff --git a/docs/operations/basic-cluster-tuning.md b/docs/operations/basic-cluster-tuning.md index 9d54afed27d5..538ae33d75f2 100644 --- a/docs/operations/basic-cluster-tuning.md +++ b/docs/operations/basic-cluster-tuning.md @@ -326,13 +326,13 @@ The TopN and GroupBy queries use these buffers to store intermediate computed re ### GroupBy merging buffers -If you plan to issue GroupBy V2 queries, `druid.processing.numMergeBuffers` is an important configuration property. +If you plan to issue GroupBy queries, `druid.processing.numMergeBuffers` is an important configuration property. -GroupBy V2 queries use an additional pool of off-heap buffers for merging query results. These buffers have the same size as the processing buffers described above, set by the `druid.processing.buffer.sizeBytes` property. +GroupBy queries use an additional pool of off-heap buffers for merging query results. These buffers have the same size as the processing buffers described above, set by the `druid.processing.buffer.sizeBytes` property. -Non-nested GroupBy V2 queries require 1 merge buffer per query, while a nested GroupBy V2 query requires 2 merge buffers (regardless of the depth of nesting). +Non-nested GroupBy queries require 1 merge buffer per query, while a nested GroupBy query requires 2 merge buffers (regardless of the depth of nesting). -The number of merge buffers determines the number of GroupBy V2 queries that can be processed concurrently. +The number of merge buffers determines the number of GroupBy queries that can be processed concurrently. diff --git a/docs/querying/caching.md b/docs/querying/caching.md index 62cdb30fa25c..a84e3d25eee4 100644 --- a/docs/querying/caching.md +++ b/docs/querying/caching.md @@ -101,12 +101,11 @@ Caching does not solve all types of query performance issues. For each cache typ **Per-segment caching** doesn't work for the following: - queries containing a sub-query in them. However the output of sub-queries may be cached. See [Query execution](./query-execution.md) for more details on sub-queries execution. - queries with joins do not support any caching on the broker. -- GroupBy v2 queries do not support any caching on broker. +- GroupBy queries do not support segment level caching on broker. - queries with `bySegment` set in the query context are not cached on the broker. **Whole-query caching** doesn't work for the following: - queries that involve an inline datasource or a lookup datasource. -- GroupBy v2 queries. - queries with joins. - queries with a union datasource. diff --git a/docs/querying/groupbyquery.md b/docs/querying/groupbyquery.md index 81da0f0e2323..d6980476c0bd 100644 --- a/docs/querying/groupbyquery.md +++ b/docs/querying/groupbyquery.md @@ -240,49 +240,9 @@ The response for the query above would look something like: ## Implementation details -### Strategies - -GroupBy queries can be executed using two different strategies. The default strategy for a cluster is determined by the -"druid.query.groupBy.defaultStrategy" runtime property on the Broker. This can be overridden using "groupByStrategy" in -the query context. If neither the context field nor the property is set, the "v2" strategy will be used. - -- "v2", the default, is designed to offer better performance and memory management. This strategy generates -per-segment results using a fully off-heap map. Data processes merge the per-segment results using a fully off-heap -concurrent facts map combined with an on-heap string dictionary. This may optionally involve spilling to disk. Data -processes return sorted results to the Broker, which merges result streams using an N-way merge. The broker materializes -the results if necessary (e.g. if the query sorts on columns other than its dimensions). Otherwise, it streams results -back as they are merged. - -- "v1", a legacy engine, generates per-segment results on data processes (Historical, realtime, MiddleManager) using a map which -is partially on-heap (dimension keys and the map itself) and partially off-heap (the aggregated values). Data processes then -merge the per-segment results using Druid's indexing mechanism. This merging is multi-threaded by default, but can -optionally be single-threaded. The Broker merges the final result set using Druid's indexing mechanism again. The broker -merging is always single-threaded. Because the Broker merges results using the indexing mechanism, it must materialize -the full result set before returning any results. On both the data processes and the Broker, the merging index is fully -on-heap by default, but it can optionally store aggregated values off-heap. - -### Differences between v1 and v2 - -Query API and results are compatible between the two engines; however, there are some differences from a cluster -configuration perspective: - -- groupBy v1 controls resource usage using a row-based limit (maxResults) whereas groupBy v2 uses bytes-based limits. -In addition, groupBy v1 merges results on-heap, whereas groupBy v2 merges results off-heap. These factors mean that -memory tuning and resource limits behave differently between v1 and v2. In particular, due to this, some queries -that can complete successfully in one engine may exceed resource limits and fail with the other engine. See the -"Memory tuning and resource limits" section for more details. -- groupBy v1 imposes no limit on the number of concurrently running queries, whereas groupBy v2 controls memory usage -by using a finite-sized merge buffer pool. By default, the number of merge buffers is 1/4 the number of processing -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 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. - ### Memory tuning and resource limits -When using groupBy v2, four parameters control resource usage and limits: +When using groupBy, four parameters control resource usage and limits: - `druid.processing.buffer.sizeBytes`: size of the off-heap hash table used for aggregation, per query, in bytes. At most `druid.processing.numMergeBuffers` of these will be created at once, which also serves as an upper limit on the @@ -306,7 +266,7 @@ sorted and flushed to disk. Then, both in-memory structures will be cleared out then go on to exceed `maxOnDiskStorage` will fail with a "Resource limit exceeded" error indicating that they ran out of disk space. -With groupBy v2, cluster operators should make sure that the off-heap hash tables and on-heap merging dictionaries +With groupBy, cluster operators should make sure that the off-heap hash tables and on-heap merging dictionaries will not exceed available memory for the maximum possible concurrent query load (given by `druid.processing.numMergeBuffers`). See the [basic cluster tuning guide](../operations/basic-cluster-tuning.md) for more details about direct memory usage, organized by Druid process type. @@ -315,24 +275,18 @@ Brokers do not need merge buffers for basic groupBy queries. Queries with subque Historicals and ingestion tasks need one merge buffer for each groupBy query, unless [parallel combination](groupbyquery.md#parallel-combine) is enabled, in which case they need two merge buffers per query. -When using groupBy v1, all aggregation is done on-heap, and resource limits are done through the parameter -`druid.query.groupBy.maxResults`. This is a cap on the maximum number of results in a result set. Queries that exceed -this limit will fail with a "Resource limit exceeded" error indicating they exceeded their row limit. Cluster -operators should make sure that the on-heap aggregations will not exceed available JVM heap space for the expected -concurrent query load. - -### Performance tuning for groupBy v2 +### Performance tuning for groupBy #### Limit pushdown optimization -Druid pushes down the `limit` spec in groupBy queries to the segments on Historicals wherever possible to early prune unnecessary intermediate results and minimize the amount of data transferred to Brokers. By default, this technique is applied only when all fields in the `orderBy` spec is a subset of the grouping keys. This is because the `limitPushDown` doesn't guarantee the exact results if the `orderBy` spec includes any fields that are not in the grouping keys. However, you can enable this technique even in such cases if you can sacrifice some accuracy for fast query processing like in topN queries. See `forceLimitPushDown` in [advanced groupBy v2 configurations](#groupby-v2-configurations). +Druid pushes down the `limit` spec in groupBy queries to the segments on Historicals wherever possible to early prune unnecessary intermediate results and minimize the amount of data transferred to Brokers. By default, this technique is applied only when all fields in the `orderBy` spec is a subset of the grouping keys. This is because the `limitPushDown` doesn't guarantee the exact results if the `orderBy` spec includes any fields that are not in the grouping keys. However, you can enable this technique even in such cases if you can sacrifice some accuracy for fast query processing like in topN queries. See `forceLimitPushDown` in [advanced configurations](#advanced-configurations). #### Optimizing hash table -The groupBy v2 engine uses an open addressing hash table for aggregation. The hash table is initialized with a given initial bucket number and gradually grows on buffer full. On hash collisions, the linear probing technique is used. +The groupBy engine uses an open addressing hash table for aggregation. The hash table is initialized with a given initial bucket number and gradually grows on buffer full. On hash collisions, the linear probing technique is used. -The default number of initial buckets is 1024 and the default max load factor of the hash table is 0.7. If you can see too many collisions in the hash table, you can adjust these numbers. See `bufferGrouperInitialBuckets` and `bufferGrouperMaxLoadFactor` in [Advanced groupBy v2 configurations](#groupby-v2-configurations). +The default number of initial buckets is 1024 and the default max load factor of the hash table is 0.7. If you can see too many collisions in the hash table, you can adjust these numbers. See `bufferGrouperInitialBuckets` and `bufferGrouperMaxLoadFactor` in [advanced configurations](#advanced-configurations). #### Parallel combine @@ -350,16 +304,16 @@ longer time than timeseries or topN queries, they should release processing thre However, you might care about the performance of some really heavy groupBy queries. Usually, the performance bottleneck of heavy groupBy queries is merging sorted aggregates. In such cases, you can use processing threads for it as well. This is called _parallel combine_. To enable parallel combine, see `numParallelCombineThreads` in -[Advanced groupBy v2 configurations](#groupby-v2-configurations). Note that parallel combine can be enabled only when +[advanced configurations](#advanced-configurations). Note that parallel combine can be enabled only when data is actually spilled (see [Memory tuning and resource limits](#memory-tuning-and-resource-limits)). -Once parallel combine is enabled, the groupBy v2 engine can create a combining tree for merging sorted aggregates. Each +Once parallel combine is enabled, the groupBy engine can create a combining tree for merging sorted aggregates. Each intermediate node of the tree is a thread merging aggregates from the child nodes. The leaf node threads read and merge aggregates from hash tables including spilled ones. Usually, leaf processes are slower than intermediate nodes because they need to read data from disk. As a result, less threads are used for intermediate nodes by default. You can change the -degree of intermediate nodes. See `intermediateCombineDegree` in [Advanced groupBy v2 configurations](#groupby-v2-configurations). +degree of intermediate nodes. See `intermediateCombineDegree` in [advanced configurations](#advanced-configurations). -Please note that each Historical needs two merge buffers to process a groupBy v2 query with parallel combine: one for +Please note that each Historical needs two merge buffers to process a groupBy query with parallel combine: one for computing intermediate aggregates from each segment and another for combining intermediate aggregates in parallel. @@ -377,18 +331,14 @@ results acceptable. ### Nested groupBys -Nested groupBys (dataSource of type "query") are performed differently for "v1" and "v2". The Broker first runs the -inner groupBy query in the usual way. "v1" strategy then materializes the inner query's results on-heap with Druid's -indexing mechanism, and runs the outer query on these materialized results. "v2" strategy runs the outer query on the -inner query's results stream with off-heap fact map and on-heap string dictionary that can spill to disk. Both -strategy perform the outer query on the Broker in a single-threaded fashion. +Nested groupBys (dataSource of type "query") are performed with the Broker first running the inner groupBy query in the +usual way. Next, the outer query is run on the inner query's results stream with off-heap fact map and on-heap string +dictionary that can spill to disk. The outer query is run on the Broker in a single-threaded fashion. ### Configurations This section describes the configurations for groupBy queries. You can set the runtime properties in the `runtime.properties` file on Broker, Historical, and MiddleManager processes. You can set the query context parameters through the [query context](query-context.md). -#### Configurations for groupBy v2 - Supported runtime properties: |Property|Description|Default| @@ -405,30 +355,12 @@ Supported query contexts: ### Advanced configurations -#### Common configurations for all groupBy strategies - Supported runtime properties: |Property|Description|Default| |--------|-----------|-------| -|`druid.query.groupBy.defaultStrategy`|Default groupBy query strategy.|v2| |`druid.query.groupBy.singleThreaded`|Merge results using a single thread.|false| |`druid.query.groupBy.intermediateResultAsMapCompat`|Whether Brokers are able to understand map-based result rows. Setting this to `true` adds some overhead to all groupBy queries. It is required for compatibility with data servers running versions older than 0.16.0, which introduced [array-based result rows](#array-based-result-rows).|false| - -Supported query contexts: - -|Key|Description| -|---|-----------| -|`groupByStrategy`|Overrides the value of `druid.query.groupBy.defaultStrategy` for this query.| -|`groupByIsSingleThreaded`|Overrides the value of `druid.query.groupBy.singleThreaded` for this query.| - - -#### GroupBy v2 configurations - -Supported runtime properties: - -|Property|Description|Default| -|--------|-----------|-------| |`druid.query.groupBy.bufferGrouperInitialBuckets`|Initial number of buckets in the off-heap hash table used for grouping results. Set to 0 to use a reasonable default (1024).|0| |`druid.query.groupBy.bufferGrouperMaxLoadFactor`|Maximum load factor of the off-heap hash table used for grouping results. When the load factor exceeds this size, the table will be grown or spilled to disk. Set to 0 to use a reasonable default (0.7).|0| |`druid.query.groupBy.forceHashAggregation`|Force to use hash-based aggregation.|false| @@ -440,6 +372,7 @@ Supported query contexts: |Key|Description|Default| |---|-----------|-------| +|`groupByIsSingleThreaded`|Overrides the value of `druid.query.groupBy.singleThreaded` for this query.|None| |`bufferGrouperInitialBuckets`|Overrides the value of `druid.query.groupBy.bufferGrouperInitialBuckets` for this query.|None| |`bufferGrouperMaxLoadFactor`|Overrides the value of `druid.query.groupBy.bufferGrouperMaxLoadFactor` for this query.|None| |`forceHashAggregation`|Overrides the value of `druid.query.groupBy.forceHashAggregation`|None| @@ -452,23 +385,6 @@ Supported query contexts: |`groupByEnableMultiValueUnnesting`|Safety flag to enable/disable the implicit unnesting on multi value column's as part of the grouping key. 'true' indicates multi-value grouping keys are unnested. 'false' returns an error if a multi value column is found as part of the grouping key.|true| -#### GroupBy v1 configurations - -Supported runtime properties: - -|Property|Description|Default| -|--------|-----------|-------| -|`druid.query.groupBy.maxIntermediateRows`|Maximum number of intermediate rows for the per-segment grouping engine. This is a tuning parameter that does not impose a hard limit; rather, it potentially shifts merging work from the per-segment engine to the overall merging index. Queries that exceed this limit will not fail.|50000| -|`druid.query.groupBy.maxResults`|Maximum number of results. Queries that exceed this limit will fail.|500000| - -Supported query contexts: - -|Key|Description|Default| -|---|-----------|-------| -|`maxIntermediateRows`|Ignored by groupBy v2. Can be used to lower the value of `druid.query.groupBy.maxIntermediateRows` for a groupBy v1 query.|None| -|`maxResults`|Ignored by groupBy v2. Can be used to lower the value of `druid.query.groupBy.maxResults` for a groupBy v1 query.|None| -|`useOffheap`|Ignored by groupBy v2, and no longer supported for groupBy v1. Enabling this option with groupBy v1 will result in an error. For off-heap aggregation, switch to groupBy v2, which always operates off-heap.|false| - #### Array based result rows Internally Druid always uses an array based representation of groupBy result rows, but by default this is translated diff --git a/docs/querying/troubleshooting.md b/docs/querying/troubleshooting.md index 042cfb65dfbc..4b9a83d8a291 100644 --- a/docs/querying/troubleshooting.md +++ b/docs/querying/troubleshooting.md @@ -63,6 +63,6 @@ To mitigate query failure due to web server timeout: Set the max idle time in the `druid.server.http.maxIdleTime` property in the `historical/runtime.properties` file. You must restart the Druid cluster for this change to take effect. See [Configuration reference](../configuration/index.md) for more information on configuring the server. -* If the timeout occurs because the data servers have not pushed any results to the Broker, consider optimizing data server performance. Significant slowdown in the data servers may be a result of spilling too much data to disk in [groupBy v2 queries](groupbyquery.md#performance-tuning-for-groupby-v2), large [`IN` filters](filters.md#in-filter) in the query, or an under scaled cluster. Analyze your [Druid query metrics](../operations/metrics.md#query-metrics) to determine the bottleneck. +* If the timeout occurs because the data servers have not pushed any results to the Broker, consider optimizing data server performance. Significant slowdown in the data servers may be a result of spilling too much data to disk in [groupBy queries](groupbyquery.md#performance-tuning-for-groupby), large [`IN` filters](filters.md#in-filter) in the query, or an under scaled cluster. Analyze your [Druid query metrics](../operations/metrics.md#query-metrics) to determine the bottleneck. * If the timeout is caused by Broker backpressure, consider optimizing Broker performance. Check whether the connection is fast enough between the Broker and deep storage. diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorFactoryBase.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorFactoryBase.java index 201ceee405e3..da1dd19cfaa1 100644 --- a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorFactoryBase.java +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorFactoryBase.java @@ -128,9 +128,6 @@ public Comparator getComparator() @Override public abstract AggregateCombiner makeAggregateCombiner(); - @Override - public abstract List getRequiredColumns(); - @Override public abstract String toString(); diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregatorFactory.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregatorFactory.java index 47fbda0ec633..bce2bf257084 100644 --- a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregatorFactory.java +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregatorFactory.java @@ -28,9 +28,6 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; -import java.util.Collections; -import java.util.List; - public class CompressedBigDecimalMaxAggregatorFactory extends CompressedBigDecimalAggregatorFactoryBase { private static final byte CACHE_TYPE_ID = 0x37; @@ -113,18 +110,6 @@ public AggregateCombiner makeAggregateCombiner() return new CompressedBigDecimalMaxAggregateCombiner(); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new CompressedBigDecimalMaxAggregatorFactory( - fieldName, - fieldName, - size, - scale, - strictNumberParsing - )); - } - @Override public String toString() { diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregatorFactory.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregatorFactory.java index 8d3590a7329b..091af3453532 100644 --- a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregatorFactory.java +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregatorFactory.java @@ -29,8 +29,6 @@ import org.apache.druid.segment.ColumnValueSelector; import javax.annotation.Nonnull; -import java.util.Collections; -import java.util.List; /** @@ -118,18 +116,6 @@ public AggregateCombiner makeAggregateCombiner() return new CompressedBigDecimalMinAggregateCombiner(); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new CompressedBigDecimalMinAggregatorFactory( - name, - fieldName, - size, - scale, - strictNumberParsing - )); - } - @Override public String toString() { diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregatorFactory.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregatorFactory.java index bbe5d360813f..d7e08a2daf42 100644 --- a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregatorFactory.java +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregatorFactory.java @@ -30,8 +30,6 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; -import java.util.Collections; -import java.util.List; public class CompressedBigDecimalSumAggregatorFactory extends CompressedBigDecimalAggregatorFactoryBase { @@ -118,18 +116,6 @@ public AggregateCombiner makeAggregateCombiner() return new CompressedBigDecimalSumAggregateCombiner(); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new CompressedBigDecimalSumAggregatorFactory( - name, - fieldName, - size, - scale, - strictNumberParsing - )); - } - @Override public String toString() { diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxFactoryTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxFactoryTest.java index 0f1d548cec58..9eef55d4ee7e 100644 --- a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxFactoryTest.java +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxFactoryTest.java @@ -26,7 +26,6 @@ import java.io.IOException; import java.math.BigDecimal; -import java.util.Arrays; public class CompressedBigDecimalMaxFactoryTest extends CompressedBigDecimalFactoryTestBase @@ -50,10 +49,6 @@ public void testCompressedBigDecimalMaxAggregatorFactory() Assert.assertEquals("5", aggregatorFactory.deserialize(new BigDecimal(5)).toString()); Assert.assertEquals("5.0", aggregatorFactory.deserialize(5d).toString()); Assert.assertEquals("5", aggregatorFactory.deserialize("5").toString()); - Assert.assertEquals( - "[CompressedBigDecimalMaxAggregatorFactory{name='fieldName', type='COMPLEX', fieldName='fieldName', requiredFields='[fieldName]', size='9', scale='0', strictNumberParsing='false'}]", - Arrays.toString(aggregatorFactory.getRequiredColumns().toArray()) - ); Assert.assertNull(aggregatorFactory.combine(null, null)); Assert.assertEquals("4", aggregatorFactory.combine(new BigDecimal(4), null).toString()); Assert.assertEquals("4", aggregatorFactory.combine(null, new BigDecimal(4)).toString()); diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinFactoryTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinFactoryTest.java index 1c445011ff6b..7a9aa0f82b09 100644 --- a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinFactoryTest.java +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinFactoryTest.java @@ -26,7 +26,6 @@ import java.io.IOException; import java.math.BigDecimal; -import java.util.Arrays; public class CompressedBigDecimalMinFactoryTest extends CompressedBigDecimalFactoryTestBase @@ -50,10 +49,7 @@ public void testCompressedBigDecimalMinAggregatorFactory() Assert.assertEquals("5", aggregatorFactory.deserialize(new BigDecimal(5)).toString()); Assert.assertEquals("5.0", aggregatorFactory.deserialize(5d).toString()); Assert.assertEquals("5", aggregatorFactory.deserialize("5").toString()); - Assert.assertEquals( - "[CompressedBigDecimalMinAggregatorFactory{name='name', type='COMPLEX', fieldName='fieldName', requiredFields='[fieldName]', size='9', scale='0', strictNumberParsing='false'}]", - Arrays.toString(aggregatorFactory.getRequiredColumns().toArray()) - ); + // default is to initialize to Assert.assertNull(aggregatorFactory.combine(null, null)); Assert.assertEquals("4", aggregatorFactory.combine(new BigDecimal(4), null).toString()); diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumFactoryTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumFactoryTest.java index c26a2568f7f1..572772507b09 100644 --- a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumFactoryTest.java +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumFactoryTest.java @@ -26,7 +26,6 @@ import java.io.IOException; import java.math.BigDecimal; -import java.util.Arrays; /** * test CompressedBigDecimalSumFactory and various aggregators and combiner produced @@ -52,10 +51,7 @@ public void testCompressedBigDecimalAggregatorFactory() Assert.assertEquals("5", aggregatorFactory.deserialize(new BigDecimal(5)).toString()); Assert.assertEquals("5.0", aggregatorFactory.deserialize(5d).toString()); Assert.assertEquals("5", aggregatorFactory.deserialize("5").toString()); - Assert.assertEquals( - "[CompressedBigDecimalSumAggregatorFactory{name='name', type='COMPLEX', fieldName='fieldName', requiredFields='[fieldName]', size='9', scale='0', strictNumberParsing='false'}]", - Arrays.toString(aggregatorFactory.getRequiredColumns().toArray()) - ); + Assert.assertEquals("0", aggregatorFactory.combine(null, null).toString()); Assert.assertEquals("4", aggregatorFactory.combine(new BigDecimal(4), null).toString()); Assert.assertEquals("4", aggregatorFactory.combine(null, new BigDecimal(4)).toString()); diff --git a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java b/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java index ce6f2f30bc9c..5d4b09ef4928 100644 --- a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java +++ b/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java @@ -141,14 +141,6 @@ public AggregatorFactory getCombiningFactory() return new LongSumAggregatorFactory(name, name); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList( - new DistinctCountAggregatorFactory(fieldName, fieldName, bitMapFactory) - ); - } - @Override public Object deserialize(Object object) { diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java index 3aaf91ead95c..c7946255b07e 100644 --- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java +++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java @@ -65,7 +65,6 @@ public class DistinctCountGroupByQueryTest extends InitializedNullHandlingTest public void setup() { final GroupByQueryConfig config = new GroupByQueryConfig(); - config.setMaxIntermediateRows(10000); this.resourceCloser = Closer.create(); this.factory = GroupByQueryRunnerTest.makeQueryRunnerFactory( config, diff --git a/extensions-contrib/momentsketch/src/main/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchAggregatorFactory.java b/extensions-contrib/momentsketch/src/main/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchAggregatorFactory.java index ebaab352aa32..f873a023adc1 100644 --- a/extensions-contrib/momentsketch/src/main/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchAggregatorFactory.java +++ b/extensions-contrib/momentsketch/src/main/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchAggregatorFactory.java @@ -167,19 +167,6 @@ public AggregatorFactory getMergingFactory(AggregatorFactory other) throws Aggre } } - @Override - public List getRequiredColumns() - { - return Collections.singletonList( - new MomentSketchAggregatorFactory( - fieldName, - fieldName, - k, - compress - ) - ); - } - private MomentSketchWrapper deserializeFromByteArray(byte[] bytes) { return MomentSketchWrapper.fromByteArray(bytes); diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java index 941114abb741..69538c366c0f 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java @@ -105,15 +105,6 @@ public AggregatorFactory getCombiningFactory() throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); } - /** - * Not implemented. Throws UnsupportedOperationException. - */ - @Override - public List getRequiredColumns() - { - throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); - } - /** * Not implemented. Throws UnsupportedOperationException. */ diff --git a/extensions-contrib/opentelemetry-emitter/pom.xml b/extensions-contrib/opentelemetry-emitter/pom.xml index 2f1cd06055c7..58415e7ea3ef 100644 --- a/extensions-contrib/opentelemetry-emitter/pom.xml +++ b/extensions-contrib/opentelemetry-emitter/pom.xml @@ -39,7 +39,7 @@ 32.0.1-jre - 1.41.3 + 1.57.2 diff --git a/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchAggregatorFactory.java b/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchAggregatorFactory.java index 58d0dcc42113..da753777e037 100644 --- a/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchAggregatorFactory.java +++ b/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchAggregatorFactory.java @@ -162,18 +162,6 @@ public AggregatorFactory getMergingFactory(AggregatorFactory other) throws Aggre } } - @Override - public List getRequiredColumns() - { - return Collections.singletonList( - new TDigestSketchAggregatorFactory( - fieldName, - fieldName, - compression - ) - ); - } - @Override public Object deserialize(Object serializedSketch) { diff --git a/extensions-contrib/time-min-max/src/main/java/org/apache/druid/query/aggregation/TimestampMaxAggregatorFactory.java b/extensions-contrib/time-min-max/src/main/java/org/apache/druid/query/aggregation/TimestampMaxAggregatorFactory.java index aec9d1bf1347..902a60fdf6a2 100644 --- a/extensions-contrib/time-min-max/src/main/java/org/apache/druid/query/aggregation/TimestampMaxAggregatorFactory.java +++ b/extensions-contrib/time-min-max/src/main/java/org/apache/druid/query/aggregation/TimestampMaxAggregatorFactory.java @@ -22,11 +22,9 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; import com.google.common.collect.Ordering; import javax.annotation.Nullable; -import java.util.List; public class TimestampMaxAggregatorFactory extends TimestampAggregatorFactory { @@ -47,14 +45,6 @@ public AggregatorFactory getCombiningFactory() return new TimestampMaxAggregatorFactory(name, name, timeFormat); } - @Override - public List getRequiredColumns() - { - return ImmutableList.of( - new TimestampMaxAggregatorFactory(name, fieldName, timeFormat) - ); - } - @Override public AggregatorFactory withName(String newName) { diff --git a/extensions-contrib/time-min-max/src/main/java/org/apache/druid/query/aggregation/TimestampMinAggregatorFactory.java b/extensions-contrib/time-min-max/src/main/java/org/apache/druid/query/aggregation/TimestampMinAggregatorFactory.java index d463a4669817..7b699e0a5bef 100644 --- a/extensions-contrib/time-min-max/src/main/java/org/apache/druid/query/aggregation/TimestampMinAggregatorFactory.java +++ b/extensions-contrib/time-min-max/src/main/java/org/apache/druid/query/aggregation/TimestampMinAggregatorFactory.java @@ -22,11 +22,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; import com.google.common.collect.Ordering; -import java.util.List; - public class TimestampMinAggregatorFactory extends TimestampAggregatorFactory { @JsonCreator @@ -46,14 +43,6 @@ public AggregatorFactory getCombiningFactory() return new TimestampMinAggregatorFactory(name, name, timeFormat); } - @Override - public List getRequiredColumns() - { - return ImmutableList.of( - new TimestampMinAggregatorFactory(name, fieldName, timeFormat) - ); - } - @Override public AggregatorFactory withName(String newName) { 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 cd746aaff78e..fb1ca3717819 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 @@ -38,9 +38,8 @@ import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryQueryToolChest; import org.apache.druid.query.groupby.GroupByQueryRunnerFactory; +import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.ResultRow; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV2; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -67,49 +66,44 @@ public class MapVirtualColumnGroupByTest extends InitializedNullHandlingTest public void setup() throws IOException { final IncrementalIndex incrementalIndex = MapVirtualColumnTestBase.generateIndex(); - - final GroupByStrategySelector strategySelector = new GroupByStrategySelector( + final GroupingEngine groupingEngine = new GroupingEngine( + new DruidProcessingConfig() + { + @Override + public String getFormatString() + { + return null; + } + + @Override + public int intermediateComputeSizeBytes() + { + return 10 * 1024 * 1024; + } + + @Override + public int getNumMergeBuffers() + { + return 1; + } + + @Override + public int getNumThreads() + { + return 1; + } + }, GroupByQueryConfig::new, - null, - new GroupByStrategyV2( - new DruidProcessingConfig() - { - @Override - public String getFormatString() - { - return null; - } - - @Override - public int intermediateComputeSizeBytes() - { - return 10 * 1024 * 1024; - } - - @Override - public int getNumMergeBuffers() - { - return 1; - } - - @Override - public int getNumThreads() - { - return 1; - } - }, - GroupByQueryConfig::new, - new StupidPool<>("map-virtual-column-groupby-test", () -> ByteBuffer.allocate(1024)), - new DefaultBlockingPool<>(() -> ByteBuffer.allocate(1024), 1), - TestHelper.makeJsonMapper(), - new DefaultObjectMapper(), - QueryRunnerTestHelper.NOOP_QUERYWATCHER - ) + new StupidPool<>("map-virtual-column-groupby-test", () -> ByteBuffer.allocate(1024)), + new DefaultBlockingPool<>(() -> ByteBuffer.allocate(1024), 1), + TestHelper.makeJsonMapper(), + new DefaultObjectMapper(), + QueryRunnerTestHelper.NOOP_QUERYWATCHER ); final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( - strategySelector, - new GroupByQueryQueryToolChest(strategySelector) + groupingEngine, + new GroupByQueryQueryToolChest(groupingEngine) ); runner = QueryRunnerTestHelper.makeQueryRunner( diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java index 4bc734dc0051..a54a63de4d2b 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java @@ -133,27 +133,6 @@ public List requiredFields() return Collections.singletonList(fieldName); } - /** - * Used by groupBy v1 to create a "transfer aggregator". - * - * {@inheritDoc} - */ - @Override - public List getRequiredColumns() - { - return Collections.singletonList( - new HllSketchBuildAggregatorFactory( - fieldName, - fieldName, - lgK, - tgtHllType.toString(), - stringEncoding, - shouldFinalize, - round - ) - ); - } - @Override public HllSketchHolder deserialize(final Object object) { diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchAggregatorFactory.java index b18153067a2d..815227adf55b 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchAggregatorFactory.java @@ -38,9 +38,7 @@ import org.apache.druid.segment.vector.VectorValueSelector; import javax.annotation.Nullable; -import java.util.Collections; import java.util.Comparator; -import java.util.List; public class KllDoublesSketchAggregatorFactory extends KllSketchAggregatorFactory { @@ -81,19 +79,6 @@ public Comparator getComparator() return COMPARATOR; } - @Override - public List getRequiredColumns() - { - return Collections.singletonList( - new KllDoublesSketchAggregatorFactory( - getFieldName(), - getFieldName(), - getK(), - getMaxStreamLength() - ) - ); - } - @Override public AggregatorFactory getMergingFactory(final AggregatorFactory other) throws AggregatorFactoryNotMergeableException diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchAggregatorFactory.java index c5506a3b86ab..9cc61524615c 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchAggregatorFactory.java @@ -38,9 +38,7 @@ import org.apache.druid.segment.vector.VectorValueSelector; import javax.annotation.Nullable; -import java.util.Collections; import java.util.Comparator; -import java.util.List; public class KllFloatsSketchAggregatorFactory extends KllSketchAggregatorFactory { @@ -81,19 +79,6 @@ public Comparator getComparator() return COMPARATOR; } - @Override - public List getRequiredColumns() - { - return Collections.singletonList( - new KllFloatsSketchAggregatorFactory( - getFieldName(), - getFieldName(), - getK(), - getMaxStreamLength() - ) - ); - } - @Override public AggregatorFactory getMergingFactory(final AggregatorFactory other) throws AggregatorFactoryNotMergeableException diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java index 5168a558440e..2830cd95d9a2 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java @@ -359,20 +359,6 @@ public int getMaxIntermediateSize() return DoublesSketch.getUpdatableStorageBytes(k, maxStreamLength); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList( - new DoublesSketchAggregatorFactory( - fieldName, - fieldName, - k, - maxStreamLength, - shouldFinalize - ) - ); - } - @Override public AggregatorFactory getCombiningFactory() { diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchMergeAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchMergeAggregatorFactory.java index 41869d5ea509..addf76d38dfb 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchMergeAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchMergeAggregatorFactory.java @@ -29,8 +29,6 @@ import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; -import java.util.Collections; -import java.util.List; public class SketchMergeAggregatorFactory extends SketchAggregatorFactory { @@ -55,21 +53,6 @@ public SketchMergeAggregatorFactory( this.errorBoundsStdDev = errorBoundsStdDev; } - @Override - public List getRequiredColumns() - { - return Collections.singletonList( - new SketchMergeAggregatorFactory( - fieldName, - fieldName, - size, - shouldFinalize, - isInputThetaSketch, - errorBoundsStdDev - ) - ); - } - @Override public AggregatorFactory getCombiningFactory() { diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchAggregatorFactory.java index 2732cff10166..3fe939b4ca87 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchAggregatorFactory.java @@ -279,20 +279,6 @@ public AggregatorFactory withName(String newName) ); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList( - new ArrayOfDoublesSketchAggregatorFactory( - fieldName, - fieldName, - nominalEntries, - metricColumns, - numberOfValues - ) - ); - } - @Override public AggregatorFactory getCombiningFactory() { diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactoryTest.java index 2c4ff635faa4..c265e99f8d67 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactoryTest.java @@ -79,31 +79,6 @@ public void testStringEncoding() Assert.assertEquals(STRING_ENCODING, target.getStringEncoding()); } - @Test - public void testGetRequiredColumns() - { - List aggregatorFactories = target.getRequiredColumns(); - Assert.assertEquals(1, aggregatorFactories.size()); - HllSketchAggregatorFactory aggregatorFactory = (HllSketchAggregatorFactory) aggregatorFactories.get(0); - Assert.assertEquals(FIELD_NAME, aggregatorFactory.getName()); - Assert.assertEquals(FIELD_NAME, aggregatorFactory.getFieldName()); - Assert.assertEquals(LG_K, aggregatorFactory.getLgK()); - Assert.assertEquals(TGT_HLL_TYPE, aggregatorFactory.getTgtHllType()); - Assert.assertEquals(HllSketchAggregatorFactory.DEFAULT_SHOULD_FINALIZE, aggregatorFactory.isShouldFinalize()); - Assert.assertEquals(ROUND, aggregatorFactory.isRound()); - } - - - @Test - public void testWithName() - { - List aggregatorFactories = target.getRequiredColumns(); - Assert.assertEquals(1, aggregatorFactories.size()); - HllSketchAggregatorFactory aggregatorFactory = (HllSketchAggregatorFactory) aggregatorFactories.get(0); - Assert.assertEquals(aggregatorFactory, aggregatorFactory.withName(aggregatorFactory.getName())); - Assert.assertEquals("newTest", aggregatorFactory.withName("newTest").getName()); - } - @Test public void testFinalizeComputationNull() { diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorTest.java index b8acb0ce2c22..0cd4d8cf39b5 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorTest.java @@ -89,7 +89,7 @@ public static Collection constructorFeeder() for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) { for (String vectorize : new String[]{"false", "force"}) { for (StringEncoding stringEncoding : StringEncoding.values()) { - if (!("v1".equals(config.getDefaultStrategy()) && "force".equals(vectorize))) { + if (!("force".equals(vectorize))) { constructors.add(new Object[]{config, vectorize, stringEncoding}); } } diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchAggregatorTest.java index 0aec67c2efe5..f0efcab2dcfb 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchAggregatorTest.java @@ -31,7 +31,6 @@ import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryRunnerTest; import org.apache.druid.query.groupby.ResultRow; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.After; import org.junit.Assert; @@ -541,92 +540,47 @@ public void timeSeriesQueryInputAsFloat() throws Exception @Test public void testSuccessWhenMaxStreamLengthHit() throws Exception { - if (GroupByStrategySelector.STRATEGY_V1.equals(config.getDefaultStrategy())) { - helper.createIndexAndRunQueryOnSegment( - new File(this.getClass().getClassLoader().getResource("kll/kll_doubles_sketch_build_data.tsv").getFile()), - String.join( - "\n", - "{", - " \"type\": \"string\",", - " \"parseSpec\": {", - " \"format\": \"tsv\",", - " \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMddHH\"},", - " \"dimensionsSpec\": {", - " \"dimensions\": [\"sequenceNumber\", \"product\"],", - " \"dimensionExclusions\": [],", - " \"spatialDimensions\": []", - " },", - " \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\"]", - " }", - "}" - ), - "[{\"type\": \"doubleSum\", \"name\": \"value\", \"fieldName\": \"value\"}]", - 0, // minTimestamp - Granularities.NONE, - 10, // maxRowCount - String.join( - "\n", - "{", - " \"queryType\": \"groupBy\",", - " \"dataSource\": \"test_datasource\",", - " \"granularity\": \"ALL\",", - " \"dimensions\": [],", - " \"aggregations\": [", - " {\"type\": \"KllDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"value\", \"k\": 200, \"maxStreamLength\": 10}", - " ],", - " \"postAggregations\": [", - " {\"type\": \"KllDoublesSketchToQuantile\", \"name\": \"quantile\", \"fraction\": 0.5, \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},", - " {\"type\": \"KllDoublesSketchToQuantiles\", \"name\": \"quantiles\", \"fractions\": [0, 0.5, 1], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},", - " {\"type\": \"KllDoublesSketchToHistogram\", \"name\": \"histogram\", \"splitPoints\": [0.25, 0.5, 0.75], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}}", - " ],", - " \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]", - "}" - ) - ); - } else { - Sequence seq = helper.createIndexAndRunQueryOnSegment( - new File(this.getClass().getClassLoader().getResource("kll/kll_doubles_sketch_build_data.tsv").getFile()), - String.join( - "\n", - "{", - " \"type\": \"string\",", - " \"parseSpec\": {", - " \"format\": \"tsv\",", - " \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMddHH\"},", - " \"dimensionsSpec\": {", - " \"dimensions\": [\"sequenceNumber\", \"product\"],", - " \"dimensionExclusions\": [],", - " \"spatialDimensions\": []", - " },", - " \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\"]", - " }", - "}" - ), - "[{\"type\": \"doubleSum\", \"name\": \"value\", \"fieldName\": \"value\"}]", - 0, // minTimestamp - Granularities.NONE, - 10, // maxRowCount - String.join( - "\n", - "{", - " \"queryType\": \"groupBy\",", - " \"dataSource\": \"test_datasource\",", - " \"granularity\": \"ALL\",", - " \"dimensions\": [],", - " \"aggregations\": [", - " {\"type\": \"KllDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"value\", \"k\": 200, \"maxStreamLength\": 10}", - " ],", - " \"postAggregations\": [", - " {\"type\": \"KllDoublesSketchToQuantile\", \"name\": \"quantile\", \"fraction\": 0.5, \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},", - " {\"type\": \"KllDoublesSketchToQuantiles\", \"name\": \"quantiles\", \"fractions\": [0, 0.5, 1], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},", - " {\"type\": \"KllDoublesSketchToHistogram\", \"name\": \"histogram\", \"splitPoints\": [0.25, 0.5, 0.75], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}}", - " ],", - " \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]", - "}" - ) - ); - seq.toList(); - } + Sequence seq = helper.createIndexAndRunQueryOnSegment( + new File(this.getClass().getClassLoader().getResource("kll/kll_doubles_sketch_build_data.tsv").getFile()), + String.join( + "\n", + "{", + " \"type\": \"string\",", + " \"parseSpec\": {", + " \"format\": \"tsv\",", + " \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMddHH\"},", + " \"dimensionsSpec\": {", + " \"dimensions\": [\"sequenceNumber\", \"product\"],", + " \"dimensionExclusions\": [],", + " \"spatialDimensions\": []", + " },", + " \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\"]", + " }", + "}" + ), + "[{\"type\": \"doubleSum\", \"name\": \"value\", \"fieldName\": \"value\"}]", + 0, // minTimestamp + Granularities.NONE, + 10, // maxRowCount + String.join( + "\n", + "{", + " \"queryType\": \"groupBy\",", + " \"dataSource\": \"test_datasource\",", + " \"granularity\": \"ALL\",", + " \"dimensions\": [],", + " \"aggregations\": [", + " {\"type\": \"KllDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"value\", \"k\": 200, \"maxStreamLength\": 10}", + " ],", + " \"postAggregations\": [", + " {\"type\": \"KllDoublesSketchToQuantile\", \"name\": \"quantile\", \"fraction\": 0.5, \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},", + " {\"type\": \"KllDoublesSketchToQuantiles\", \"name\": \"quantiles\", \"fractions\": [0, 0.5, 1], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},", + " {\"type\": \"KllDoublesSketchToHistogram\", \"name\": \"histogram\", \"splitPoints\": [0.25, 0.5, 0.75], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}}", + " ],", + " \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]", + "}" + ) + ); + seq.toList(); } - } diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchAggregatorTest.java index 11e9aa9615d7..a7fd46af258f 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchAggregatorTest.java @@ -31,7 +31,6 @@ import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryRunnerTest; import org.apache.druid.query.groupby.ResultRow; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.After; import org.junit.Assert; @@ -541,92 +540,47 @@ public void timeSeriesQueryInputAsFloat() throws Exception @Test public void testSuccessWhenMaxStreamLengthHit() throws Exception { - if (GroupByStrategySelector.STRATEGY_V1.equals(config.getDefaultStrategy())) { - helper.createIndexAndRunQueryOnSegment( - new File(this.getClass().getClassLoader().getResource("kll/kll_floats_sketch_build_data.tsv").getFile()), - String.join( - "\n", - "{", - " \"type\": \"string\",", - " \"parseSpec\": {", - " \"format\": \"tsv\",", - " \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMddHH\"},", - " \"dimensionsSpec\": {", - " \"dimensions\": [\"sequenceNumber\", \"product\"],", - " \"dimensionExclusions\": [],", - " \"spatialDimensions\": []", - " },", - " \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\"]", - " }", - "}" - ), - "[{\"type\": \"doubleSum\", \"name\": \"value\", \"fieldName\": \"value\"}]", - 0, // minTimestamp - Granularities.NONE, - 10, // maxRowCount - String.join( - "\n", - "{", - " \"queryType\": \"groupBy\",", - " \"dataSource\": \"test_datasource\",", - " \"granularity\": \"ALL\",", - " \"dimensions\": [],", - " \"aggregations\": [", - " {\"type\": \"KllFloatsSketch\", \"name\": \"sketch\", \"fieldName\": \"value\", \"k\": 200, \"maxStreamLength\": 10}", - " ],", - " \"postAggregations\": [", - " {\"type\": \"KllFloatsSketchToQuantile\", \"name\": \"quantile\", \"fraction\": 0.5, \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},", - " {\"type\": \"KllFloatsSketchToQuantiles\", \"name\": \"quantiles\", \"fractions\": [0, 0.5, 1], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},", - " {\"type\": \"KllFloatsSketchToHistogram\", \"name\": \"histogram\", \"splitPoints\": [0.25, 0.5, 0.75], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}}", - " ],", - " \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]", - "}" - ) - ); - } else { - Sequence seq = helper.createIndexAndRunQueryOnSegment( - new File(this.getClass().getClassLoader().getResource("kll/kll_floats_sketch_build_data.tsv").getFile()), - String.join( - "\n", - "{", - " \"type\": \"string\",", - " \"parseSpec\": {", - " \"format\": \"tsv\",", - " \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMddHH\"},", - " \"dimensionsSpec\": {", - " \"dimensions\": [\"sequenceNumber\", \"product\"],", - " \"dimensionExclusions\": [],", - " \"spatialDimensions\": []", - " },", - " \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\"]", - " }", - "}" - ), - "[{\"type\": \"doubleSum\", \"name\": \"value\", \"fieldName\": \"value\"}]", - 0, // minTimestamp - Granularities.NONE, - 10, // maxRowCount - String.join( - "\n", - "{", - " \"queryType\": \"groupBy\",", - " \"dataSource\": \"test_datasource\",", - " \"granularity\": \"ALL\",", - " \"dimensions\": [],", - " \"aggregations\": [", - " {\"type\": \"KllFloatsSketch\", \"name\": \"sketch\", \"fieldName\": \"value\", \"k\": 200, \"maxStreamLength\": 10}", - " ],", - " \"postAggregations\": [", - " {\"type\": \"KllFloatsSketchToQuantile\", \"name\": \"quantile\", \"fraction\": 0.5, \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},", - " {\"type\": \"KllFloatsSketchToQuantiles\", \"name\": \"quantiles\", \"fractions\": [0, 0.5, 1], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},", - " {\"type\": \"KllFloatsSketchToHistogram\", \"name\": \"histogram\", \"splitPoints\": [0.25, 0.5, 0.75], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}}", - " ],", - " \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]", - "}" - ) - ); - seq.toList(); - } + Sequence seq = helper.createIndexAndRunQueryOnSegment( + new File(this.getClass().getClassLoader().getResource("kll/kll_floats_sketch_build_data.tsv").getFile()), + String.join( + "\n", + "{", + " \"type\": \"string\",", + " \"parseSpec\": {", + " \"format\": \"tsv\",", + " \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMddHH\"},", + " \"dimensionsSpec\": {", + " \"dimensions\": [\"sequenceNumber\", \"product\"],", + " \"dimensionExclusions\": [],", + " \"spatialDimensions\": []", + " },", + " \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\"]", + " }", + "}" + ), + "[{\"type\": \"doubleSum\", \"name\": \"value\", \"fieldName\": \"value\"}]", + 0, // minTimestamp + Granularities.NONE, + 10, // maxRowCount + String.join( + "\n", + "{", + " \"queryType\": \"groupBy\",", + " \"dataSource\": \"test_datasource\",", + " \"granularity\": \"ALL\",", + " \"dimensions\": [],", + " \"aggregations\": [", + " {\"type\": \"KllFloatsSketch\", \"name\": \"sketch\", \"fieldName\": \"value\", \"k\": 200, \"maxStreamLength\": 10}", + " ],", + " \"postAggregations\": [", + " {\"type\": \"KllFloatsSketchToQuantile\", \"name\": \"quantile\", \"fraction\": 0.5, \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},", + " {\"type\": \"KllFloatsSketchToQuantiles\", \"name\": \"quantiles\", \"fractions\": [0, 0.5, 1], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},", + " {\"type\": \"KllFloatsSketchToHistogram\", \"name\": \"histogram\", \"splitPoints\": [0.25, 0.5, 0.75], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}}", + " ],", + " \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]", + "}" + ) + ); + seq.toList(); } - } diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java index 2c05509e7d1a..5173c492dd7c 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorTest.java @@ -31,7 +31,6 @@ import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryRunnerTest; import org.apache.druid.query.groupby.ResultRow; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.After; import org.junit.Assert; @@ -540,92 +539,47 @@ public void timeSeriesQueryInputAsFloat() throws Exception @Test public void testSuccessWhenMaxStreamLengthHit() throws Exception { - if (GroupByStrategySelector.STRATEGY_V1.equals(config.getDefaultStrategy())) { - helper.createIndexAndRunQueryOnSegment( - new File(this.getClass().getClassLoader().getResource("quantiles/doubles_build_data.tsv").getFile()), - String.join( - "\n", - "{", - " \"type\": \"string\",", - " \"parseSpec\": {", - " \"format\": \"tsv\",", - " \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMddHH\"},", - " \"dimensionsSpec\": {", - " \"dimensions\": [\"sequenceNumber\", \"product\"],", - " \"dimensionExclusions\": [],", - " \"spatialDimensions\": []", - " },", - " \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\"]", - " }", - "}" - ), - "[{\"type\": \"doubleSum\", \"name\": \"value\", \"fieldName\": \"value\"}]", - 0, // minTimestamp - Granularities.NONE, - 10, // maxRowCount - String.join( - "\n", - "{", - " \"queryType\": \"groupBy\",", - " \"dataSource\": \"test_datasource\",", - " \"granularity\": \"ALL\",", - " \"dimensions\": [],", - " \"aggregations\": [", - " {\"type\": \"quantilesDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"value\", \"k\": 128, \"maxStreamLength\": 10}", - " ],", - " \"postAggregations\": [", - " {\"type\": \"quantilesDoublesSketchToQuantile\", \"name\": \"quantile\", \"fraction\": 0.5, \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},", - " {\"type\": \"quantilesDoublesSketchToQuantiles\", \"name\": \"quantiles\", \"fractions\": [0, 0.5, 1], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},", - " {\"type\": \"quantilesDoublesSketchToHistogram\", \"name\": \"histogram\", \"splitPoints\": [0.25, 0.5, 0.75], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}}", - " ],", - " \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]", - "}" - ) - ); - } else { - Sequence seq = helper.createIndexAndRunQueryOnSegment( - new File(this.getClass().getClassLoader().getResource("quantiles/doubles_build_data.tsv").getFile()), - String.join( - "\n", - "{", - " \"type\": \"string\",", - " \"parseSpec\": {", - " \"format\": \"tsv\",", - " \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMddHH\"},", - " \"dimensionsSpec\": {", - " \"dimensions\": [\"sequenceNumber\", \"product\"],", - " \"dimensionExclusions\": [],", - " \"spatialDimensions\": []", - " },", - " \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\"]", - " }", - "}" - ), - "[{\"type\": \"doubleSum\", \"name\": \"value\", \"fieldName\": \"value\"}]", - 0, // minTimestamp - Granularities.NONE, - 10, // maxRowCount - String.join( - "\n", - "{", - " \"queryType\": \"groupBy\",", - " \"dataSource\": \"test_datasource\",", - " \"granularity\": \"ALL\",", - " \"dimensions\": [],", - " \"aggregations\": [", - " {\"type\": \"quantilesDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"value\", \"k\": 128, \"maxStreamLength\": 10}", - " ],", - " \"postAggregations\": [", - " {\"type\": \"quantilesDoublesSketchToQuantile\", \"name\": \"quantile\", \"fraction\": 0.5, \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},", - " {\"type\": \"quantilesDoublesSketchToQuantiles\", \"name\": \"quantiles\", \"fractions\": [0, 0.5, 1], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},", - " {\"type\": \"quantilesDoublesSketchToHistogram\", \"name\": \"histogram\", \"splitPoints\": [0.25, 0.5, 0.75], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}}", - " ],", - " \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]", - "}" - ) - ); - seq.toList(); - } + Sequence seq = helper.createIndexAndRunQueryOnSegment( + new File(this.getClass().getClassLoader().getResource("quantiles/doubles_build_data.tsv").getFile()), + String.join( + "\n", + "{", + " \"type\": \"string\",", + " \"parseSpec\": {", + " \"format\": \"tsv\",", + " \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMddHH\"},", + " \"dimensionsSpec\": {", + " \"dimensions\": [\"sequenceNumber\", \"product\"],", + " \"dimensionExclusions\": [],", + " \"spatialDimensions\": []", + " },", + " \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\"]", + " }", + "}" + ), + "[{\"type\": \"doubleSum\", \"name\": \"value\", \"fieldName\": \"value\"}]", + 0, // minTimestamp + Granularities.NONE, + 10, // maxRowCount + String.join( + "\n", + "{", + " \"queryType\": \"groupBy\",", + " \"dataSource\": \"test_datasource\",", + " \"granularity\": \"ALL\",", + " \"dimensions\": [],", + " \"aggregations\": [", + " {\"type\": \"quantilesDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"value\", \"k\": 128, \"maxStreamLength\": 10}", + " ],", + " \"postAggregations\": [", + " {\"type\": \"quantilesDoublesSketchToQuantile\", \"name\": \"quantile\", \"fraction\": 0.5, \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},", + " {\"type\": \"quantilesDoublesSketchToQuantiles\", \"name\": \"quantiles\", \"fractions\": [0, 0.5, 1], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},", + " {\"type\": \"quantilesDoublesSketchToHistogram\", \"name\": \"histogram\", \"splitPoints\": [0.25, 0.5, 0.75], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}}", + " ],", + " \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]", + "}" + ) + ); + seq.toList(); } - } diff --git a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorFactory.java b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorFactory.java index 374f9ef3e6fb..b67f27bcece4 100644 --- a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorFactory.java +++ b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorFactory.java @@ -136,12 +136,6 @@ public AggregatorFactory getCombiningFactory() return new BloomFilterMergeAggregatorFactory(name, name, maxNumEntries); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new BloomFilterAggregatorFactory(name, field, maxNumEntries)); - } - @Override public Object deserialize(Object object) { diff --git a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterMergeAggregatorFactory.java b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterMergeAggregatorFactory.java index 233dc287c031..706561ea81e5 100644 --- a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterMergeAggregatorFactory.java +++ b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterMergeAggregatorFactory.java @@ -31,8 +31,6 @@ import org.apache.druid.segment.NilColumnValueSelector; import java.nio.ByteBuffer; -import java.util.Collections; -import java.util.List; public class BloomFilterMergeAggregatorFactory extends BloomFilterAggregatorFactory { @@ -56,12 +54,6 @@ public BufferAggregator factorizeBuffered(final ColumnSelectorFactory metricFact return makeMergeAggregator(metricFactory); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new BloomFilterMergeAggregatorFactory(getName(), fieldName, getMaxNumEntries())); - } - @Override public byte[] getCacheKey() { diff --git a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterSerde.java b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterSerde.java index 69494bcb6e57..7a488e05cec4 100644 --- a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterSerde.java +++ b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterSerde.java @@ -33,9 +33,7 @@ import java.nio.ByteBuffer; /** - * Dummy {@link ComplexMetricSerde} that exists so {@link BloomFilterAggregatorFactory} has something to register so - * {@link org.apache.druid.query.groupby.GroupByQueryEngine} will work, but isn't actually used because bloom filter - * aggregators are currently only implemented for use at query time + * Exists to provide an object-strategy. Bloom filters are query time only so does not fully implement this interface. */ public class BloomFilterSerde extends ComplexMetricSerde { diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterGroupByQueryTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterGroupByQueryTest.java index 8b664ff3464a..39c5aac3cbc9 100644 --- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterGroupByQueryTest.java +++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterGroupByQueryTest.java @@ -34,7 +34,6 @@ import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryRunnerTest; import org.apache.druid.query.groupby.ResultRow; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.segment.TestHelper; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.After; @@ -65,7 +64,6 @@ public class BloomFilterGroupByQueryTest extends InitializedNullHandlingTest } private AggregationTestHelper helper; - private boolean isV2; @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); @@ -77,7 +75,6 @@ public BloomFilterGroupByQueryTest(final GroupByQueryConfig config) config, tempFolder ); - isV2 = config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2); } @Parameterized.Parameters(name = "{0}") @@ -123,10 +120,6 @@ public void testQuery() throws Exception @Test public void testNestedQuery() throws Exception { - if (!isV2) { - return; - } - String query = "{" + "\"queryType\": \"groupBy\"," + "\"dataSource\": {" @@ -160,10 +153,6 @@ public void testNestedQuery() throws Exception @Test public void testNestedQueryComplex() throws Exception { - if (!isV2) { - return; - } - String query = "{" + "\"queryType\": \"groupBy\"," + "\"dataSource\": {" diff --git a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java index 38291d963456..1b7c2dd02990 100644 --- a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java +++ b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java @@ -212,22 +212,6 @@ public AggregatorFactory getMergingFactory(AggregatorFactory other) throws Aggre } } - @Override - public List getRequiredColumns() - { - return Collections.singletonList( - new ApproximateHistogramAggregatorFactory( - fieldName, - fieldName, - resolution, - numBuckets, - lowerLimit, - upperLimit, - finalizeAsBase64Binary - ) - ); - } - @Override public Object deserialize(Object object) { diff --git a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramAggregatorFactory.java b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramAggregatorFactory.java index 964a20dd9772..bb01ba9220b0 100644 --- a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramAggregatorFactory.java +++ b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramAggregatorFactory.java @@ -222,22 +222,6 @@ public AggregatorFactory getMergingFactory(AggregatorFactory other) ); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList( - new FixedBucketsHistogramAggregatorFactory( - fieldName, - fieldName, - numBuckets, - lowerLimit, - upperLimit, - outlierHandlingMode, - finalizeAsBase64Binary - ) - ); - } - @Override public Object deserialize(Object object) { diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java index af916db937c1..ecb2c85c7fb3 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramGroupByQueryTest.java @@ -35,7 +35,6 @@ import org.apache.druid.query.groupby.TestGroupByBuffers; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.segment.TestHelper; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.After; @@ -81,47 +80,8 @@ public static Iterable constructorFeeder() { setUpClass(); - final GroupByQueryConfig v1Config = new GroupByQueryConfig() - { - @Override - public String getDefaultStrategy() - { - return GroupByStrategySelector.STRATEGY_V1; - } - - @Override - public String toString() - { - return "v1"; - } - }; - final GroupByQueryConfig v1SingleThreadedConfig = new GroupByQueryConfig() - { - @Override - public boolean isSingleThreaded() - { - return true; - } - - @Override - public String getDefaultStrategy() - { - return GroupByStrategySelector.STRATEGY_V1; - } - - @Override - public String toString() - { - return "v1SingleThreaded"; - } - }; final GroupByQueryConfig v2Config = new GroupByQueryConfig() { - @Override - public String getDefaultStrategy() - { - return GroupByStrategySelector.STRATEGY_V2; - } @Override public String toString() @@ -130,13 +90,8 @@ public String toString() } }; - v1Config.setMaxIntermediateRows(10000); - v1SingleThreadedConfig.setMaxIntermediateRows(10000); - final List constructors = new ArrayList<>(); final List configs = ImmutableList.of( - v1Config, - v1SingleThreadedConfig, v2Config ); diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramGroupByQueryTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramGroupByQueryTest.java index 7394daee4f71..b22982a2e7ce 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramGroupByQueryTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramGroupByQueryTest.java @@ -35,7 +35,6 @@ import org.apache.druid.query.groupby.TestGroupByBuffers; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.segment.TestHelper; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.After; @@ -81,47 +80,8 @@ public static Iterable constructorFeeder() { setUpClass(); - final GroupByQueryConfig v1Config = new GroupByQueryConfig() - { - @Override - public String getDefaultStrategy() - { - return GroupByStrategySelector.STRATEGY_V1; - } - - @Override - public String toString() - { - return "v1"; - } - }; - final GroupByQueryConfig v1SingleThreadedConfig = new GroupByQueryConfig() - { - @Override - public boolean isSingleThreaded() - { - return true; - } - - @Override - public String getDefaultStrategy() - { - return GroupByStrategySelector.STRATEGY_V1; - } - - @Override - public String toString() - { - return "v1SingleThreaded"; - } - }; final GroupByQueryConfig v2Config = new GroupByQueryConfig() { - @Override - public String getDefaultStrategy() - { - return GroupByStrategySelector.STRATEGY_V2; - } @Override public String toString() @@ -130,13 +90,8 @@ public String toString() } }; - v1Config.setMaxIntermediateRows(10000); - v1SingleThreadedConfig.setMaxIntermediateRows(10000); - final List constructors = new ArrayList<>(); final List configs = ImmutableList.of( - v1Config, - v1SingleThreadedConfig, v2Config ); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 19733dea0ac1..d883a587e9b5 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -55,6 +55,7 @@ import org.apache.druid.frame.key.RowKeyReader; import org.apache.druid.frame.processor.FrameProcessorExecutor; import org.apache.druid.frame.util.DurableStorageUtils; +import org.apache.druid.frame.write.InvalidNullByteException; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.LockGranularity; @@ -104,6 +105,7 @@ import org.apache.druid.msq.indexing.error.InsertCannotBeEmptyFault; import org.apache.druid.msq.indexing.error.InsertLockPreemptedFault; import org.apache.druid.msq.indexing.error.InsertTimeOutOfBoundsFault; +import org.apache.druid.msq.indexing.error.InvalidNullByteFault; import org.apache.druid.msq.indexing.error.MSQErrorReport; import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.indexing.error.MSQFault; @@ -412,9 +414,15 @@ public TaskStatus runTask(final Closer closer) final String selfHost = MSQTasks.getHostFromSelfNode(selfDruidNode); final MSQErrorReport controllerError = exceptionEncountered != null - ? MSQErrorReport.fromException(id(), selfHost, null, exceptionEncountered) + ? MSQErrorReport.fromException( + id(), + selfHost, + null, + exceptionEncountered, + task.getQuerySpec().getColumnMappings() + ) : null; - final MSQErrorReport workerError = workerErrorRef.get(); + MSQErrorReport workerError = workerErrorRef.get(); taskStateForReport = TaskState.FAILED; errorForReport = MSQTasks.makeErrorReport(id(), selfHost, controllerError, workerError); @@ -748,7 +756,10 @@ public void workerError(MSQErrorReport errorReport) !workerTaskLauncher.isTaskLatest(errorReport.getTaskId())) { log.info("Ignoring task %s", errorReport.getTaskId()); } else { - workerErrorRef.compareAndSet(null, errorReport); + workerErrorRef.compareAndSet( + null, + mapQueryColumnNameToOutputColumnName(errorReport) + ); } } @@ -2651,6 +2662,39 @@ static ClusterStatisticsMergeMode finalizeClusterStatisticsMergeMode( return mergeMode; } + /** + * Maps the query column names (used internally while generating the query plan) to output column names (the one used + * by the user in the SQL query) for certain errors reported by workers (where they have limited knowledge of the + * ColumnMappings). For remaining errors not relying on the query column names, it returns it as is. + */ + @Nullable + private MSQErrorReport mapQueryColumnNameToOutputColumnName( + @Nullable final MSQErrorReport workerErrorReport + ) + { + + if (workerErrorReport == null) { + return null; + } else if (workerErrorReport.getFault() instanceof InvalidNullByteFault) { + InvalidNullByteFault inbf = (InvalidNullByteFault) workerErrorReport.getFault(); + return MSQErrorReport.fromException( + workerErrorReport.getTaskId(), + workerErrorReport.getHost(), + workerErrorReport.getStageNumber(), + InvalidNullByteException.builder() + .source(inbf.getSource()) + .rowNumber(inbf.getRowNumber()) + .column(inbf.getColumn()) + .value(inbf.getValue()) + .position(inbf.getPosition()) + .build(), + task.getQuerySpec().getColumnMappings() + ); + } else { + return workerErrorReport; + } + } + /** * Interface used by {@link #contactWorkersForStage}. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java index 162fe5a36d97..c35832992f93 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java @@ -23,7 +23,7 @@ import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.kernel.FrameContext; import org.apache.druid.msq.querykit.DataSegmentProvider; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; +import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.SegmentWrangler; @@ -59,9 +59,9 @@ public SegmentWrangler segmentWrangler() } @Override - public GroupByStrategySelector groupByStrategySelector() + public GroupingEngine groupingEngine() { - return context.injector().getInstance(GroupByStrategySelector.class); + return context.injector().getInstance(GroupingEngine.class); } @Override diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InvalidNullByteFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InvalidNullByteFault.java index 3b2a1881fa1c..0e85e6817d41 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InvalidNullByteFault.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/InvalidNullByteFault.java @@ -63,7 +63,8 @@ public InvalidNullByteFault( { super( CODE, - "Invalid null byte at source [%s], rowNumber [%d], column[%s], value[%s], position[%d]. Consider sanitizing the string using REPLACE(\"%s\", U&'\\0000', '') AS %s", + "Invalid null byte at source [%s], rowNumber [%d], column[%s], value[%s], position[%d]. " + + "Consider sanitizing the input string column using REPLACE(\"%s\", U&'\\0000', '') AS %s", source, rowNumber, column, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQErrorReport.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQErrorReport.java index 09d7c258ec09..ffad2c0c80f8 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQErrorReport.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQErrorReport.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; +import it.unimi.dsi.fastutil.ints.IntList; import org.apache.druid.frame.processor.FrameRowTooLargeException; import org.apache.druid.frame.write.InvalidNullByteException; import org.apache.druid.frame.write.UnsupportedColumnTypeException; @@ -31,6 +32,7 @@ import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.msq.statistics.TooManyBucketsException; import org.apache.druid.query.groupby.epinephelinae.UnexpectedMultiValueDimensionException; +import org.apache.druid.sql.calcite.planner.ColumnMappings; import javax.annotation.Nullable; import java.util.Objects; @@ -78,12 +80,23 @@ public static MSQErrorReport fromException( @Nullable final Integer stageNumber, final Throwable e ) + { + return fromException(taskId, host, stageNumber, e, null); + } + + public static MSQErrorReport fromException( + final String taskId, + @Nullable final String host, + @Nullable final Integer stageNumber, + final Throwable e, + @Nullable final ColumnMappings columnMappings + ) { return new MSQErrorReport( taskId, host, stageNumber, - getFaultFromException(e), + getFaultFromException(e, columnMappings), Throwables.getStackTraceAsString(e) ); } @@ -159,12 +172,17 @@ public String toString() '}'; } + public static MSQFault getFaultFromException(@Nullable final Throwable e) + { + return getFaultFromException(e, null); + } + /** * Magical code that extracts a useful fault from an exception, even if that exception is not necessarily a * {@link MSQException}. This method walks through the causal chain, and also "knows" about various exception * types thrown by other Druid code. */ - public static MSQFault getFaultFromException(@Nullable final Throwable e) + public static MSQFault getFaultFromException(@Nullable final Throwable e, @Nullable final ColumnMappings columnMappings) { // Unwrap exception wrappers to find an underlying fault. The assumption here is that the topmost recognizable // exception should be used to generate the fault code for the entire report. @@ -195,10 +213,21 @@ public static MSQFault getFaultFromException(@Nullable final Throwable e) return new RowTooLargeFault(((FrameRowTooLargeException) cause).getMaxFrameSize()); } else if (cause instanceof InvalidNullByteException) { InvalidNullByteException invalidNullByteException = (InvalidNullByteException) cause; + String columnName = invalidNullByteException.getColumn(); + if (columnMappings != null) { + IntList outputColumnsForQueryColumn = columnMappings.getOutputColumnsForQueryColumn(columnName); + + // outputColumnsForQueryColumn.size should always be 1 due to hasUniqueOutputColumnNames check that is done + if (outputColumnsForQueryColumn.size() >= 1) { + int outputColumn = outputColumnsForQueryColumn.getInt(0); + columnName = columnMappings.getOutputColumnName(outputColumn); + } + } + return new InvalidNullByteFault( invalidNullByteException.getSource(), invalidNullByteException.getRowNumber(), - invalidNullByteException.getColumn(), + columnName, invalidNullByteException.getValue(), invalidNullByteException.getPosition() ); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/ParseExceptionUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/ParseExceptionUtils.java index bfc1769868cf..0911059b364f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/ParseExceptionUtils.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/ParseExceptionUtils.java @@ -43,7 +43,10 @@ public class ParseExceptionUtils public static String generateReadableInputSourceNameFromMappedSegment(Segment segment) { if (segment instanceof ExternalSegment) { - return StringUtils.format("external input source: %s", ((ExternalSegment) segment).externalInputSource().toString()); + return StringUtils.format( + "external input source: %s", + ((ExternalSegment) segment).externalInputSource().toString() + ); } else if (segment instanceof LookupSegment) { return StringUtils.format("lookup input source: %s", segment.getId().getDataSource()); } else if (segment instanceof QueryableIndexSegment) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java index dc4b7fbb6ca8..2339ac5537a0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.querykit.DataSegmentProvider; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; +import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.SegmentWrangler; @@ -38,7 +38,7 @@ public interface FrameContext { SegmentWrangler segmentWrangler(); - GroupByStrategySelector groupByStrategySelector(); + GroupingEngine groupingEngine(); RowIngestionMeters rowIngestionMeters(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessor.java index 7aa7a72c5583..6356d00b0563 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessor.java @@ -36,12 +36,12 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.epinephelinae.RowBasedGrouperHelper; import org.apache.druid.query.groupby.having.AlwaysHavingSpec; import org.apache.druid.query.groupby.having.DimFilterHavingSpec; import org.apache.druid.query.groupby.having.HavingSpec; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; @@ -81,7 +81,7 @@ public class GroupByPostShuffleFrameProcessor implements FrameProcessor public GroupByPostShuffleFrameProcessor( final GroupByQuery query, - final GroupByStrategySelector strategySelector, + final GroupingEngine groupingEngine, final ReadableFrameChannel inputChannel, final WritableFrameChannel outputChannel, final FrameWriterFactory frameWriterFactory, @@ -94,8 +94,8 @@ public GroupByPostShuffleFrameProcessor( this.outputChannel = outputChannel; this.frameReader = frameReader; this.frameWriterFactory = frameWriterFactory; - this.compareFn = strategySelector.strategize(query).createResultComparator(query); - this.mergeFn = strategySelector.strategize(query).createMergeFn(query); + this.compareFn = groupingEngine.createResultComparator(query); + this.mergeFn = groupingEngine.createMergeFn(query); this.finalizeFn = makeFinalizeFn(query); this.havingSpec = cloneHavingSpec(query); this.columnSelectorFactoryForFrameWriter = diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessorFactory.java index 73206da16608..ab7f4fc26ba8 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessorFactory.java @@ -43,7 +43,7 @@ import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.querykit.BaseFrameProcessorFactory; import org.apache.druid.query.groupby.GroupByQuery; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; +import org.apache.druid.query.groupby.GroupingEngine; import javax.annotation.Nullable; import java.io.IOException; @@ -85,7 +85,7 @@ public ProcessorsAndChannels, Long> makeProcessors( { // Expecting a single input slice from some prior stage. final StageInputSlice slice = (StageInputSlice) Iterables.getOnlyElement(inputSlices); - final GroupByStrategySelector strategySelector = frameContext.groupByStrategySelector(); + final GroupingEngine engine = frameContext.groupingEngine(); final Int2ObjectSortedMap outputChannels = new Int2ObjectAVLTreeMap<>(); for (final ReadablePartition partition : slice.getPartitions()) { @@ -112,7 +112,7 @@ public ProcessorsAndChannels, Long> makeProcessors( return new GroupByPostShuffleFrameProcessor( query, - strategySelector, + engine, readableInput.getChannel(), outputChannel.getWritableChannel(), stageDefinition.createFrameWriterFactory(outputChannel.getFrameMemoryAllocator()), diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java index dbf89096ab27..67e53cd41e7d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java @@ -42,9 +42,9 @@ import org.apache.druid.msq.input.table.SegmentWithDescriptor; import org.apache.druid.msq.querykit.BaseLeafFrameProcessor; import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.epinephelinae.RowBasedGrouperHelper; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.SpecificSegmentSpec; import org.apache.druid.segment.ColumnSelectorFactory; @@ -61,7 +61,7 @@ public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor { private final GroupByQuery query; - private final GroupByStrategySelector strategySelector; + private final GroupingEngine groupingEngine; private final ColumnSelectorFactory frameWriterColumnSelectorFactory; private final Closer closer = Closer.create(); @@ -74,7 +74,7 @@ public GroupByPreShuffleFrameProcessor( final GroupByQuery query, final ReadableInput baseInput, final Int2ObjectMap sideChannels, - final GroupByStrategySelector strategySelector, + final GroupingEngine groupingEngine, final ResourceHolder outputChannelHolder, final ResourceHolder frameWriterFactoryHolder, final long memoryReservedForBroadcastJoin @@ -89,7 +89,7 @@ public GroupByPreShuffleFrameProcessor( memoryReservedForBroadcastJoin ); this.query = query; - this.strategySelector = strategySelector; + this.groupingEngine = groupingEngine; this.frameWriterColumnSelectorFactory = RowBasedGrouperHelper.createResultRowBasedColumnSelectorFactory( query, () -> resultYielder.get(), @@ -104,12 +104,11 @@ protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment final ResourceHolder segmentHolder = closer.register(segment.getOrLoad()); final Sequence rowSequence = - strategySelector.strategize(query) - .process( - query.withQuerySegmentSpec(new SpecificSegmentSpec(segment.getDescriptor())), - mapSegment(segmentHolder.get()).asStorageAdapter(), - null - ); + groupingEngine.process( + query.withQuerySegmentSpec(new SpecificSegmentSpec(segment.getDescriptor())), + mapSegment(segmentHolder.get()).asStorageAdapter(), + null + ); resultYielder = Yielders.each(rowSequence); } @@ -137,12 +136,11 @@ protected ReturnOrAwait runWithInputChannel( final FrameSegment frameSegment = new FrameSegment(frame, inputFrameReader, SegmentId.dummy("x")); final Sequence rowSequence = - strategySelector.strategize(query) - .process( - query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Intervals.ONLY_ETERNITY)), - mapSegment(frameSegment).asStorageAdapter(), - null - ); + groupingEngine.process( + query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Intervals.ONLY_ETERNITY)), + mapSegment(frameSegment).asStorageAdapter(), + null + ); resultYielder = Yielders.each(rowSequence); } else if (inputChannel.isFinished()) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessorFactory.java index 15e9f24190a3..4c0601d6c90c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessorFactory.java @@ -63,7 +63,7 @@ protected FrameProcessor makeProcessor( query, baseInput, sideChannels, - frameContext.groupByStrategySelector(), + frameContext.groupingEngine(), outputChannelHolder, frameWriterFactoryHolder, frameContext.memoryParameters().getBroadcastJoinMemory() diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/PassthroughAggregatorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/PassthroughAggregatorFactory.java index 4acf50cfb7a8..f590a2edd80b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/PassthroughAggregatorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/PassthroughAggregatorFactory.java @@ -117,12 +117,6 @@ public AggregatorFactory getCombiningFactory() return this; } - @Override - public List getRequiredColumns() - { - throw new UnsupportedOperationException(); - } - @Override public Object deserialize(Object object) { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQParseExceptionsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQParseExceptionsTest.java index 3f4c953bcb52..61a89f0d6fc2 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQParseExceptionsTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQParseExceptionsTest.java @@ -103,6 +103,44 @@ public void testIngestWithNullByte() throws IOException .verifyResults(); } + @Test + public void testIngestWithNullByteInSqlExpression() + { + + RowSignature rowSignature = RowSignature.builder() + .add("desc", ColumnType.STRING) + .add("text", ColumnType.STRING) + .build(); + + testIngestQuery() + .setSql("" + + "WITH \"ext\" AS (SELECT *\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{\"type\":\"inline\",\"data\":\"{\\\"desc\\\":\\\"Row with NULL\\\",\\\"text\\\":\\\"There is a null in\\\\u0000 here somewhere\\\"}\\n\"}',\n" + + " '{\"type\":\"json\"}'\n" + + " )\n" + + ") EXTEND (\"desc\" VARCHAR, \"text\" VARCHAR))\n" + + "SELECT\n" + + " \"desc\",\n" + + " REPLACE(\"text\", 'a', 'A') AS \"text\"\n" + + "FROM \"ext\"\n" + + "") + .setExpectedRowSignature(rowSignature) + .setExpectedDataSource("foo1") + .setExpectedMSQFault( + new InvalidNullByteFault( + "external input source: InlineInputSource{data='{\"desc\":\"Row with NULL\",\"text\":\"There is a null in\\u0000 here somewhere\"}\n'}", + 1, + "text", + "There is A null in\u0000 here somewhere", + 18 + ) + ) + .setQueryContext(DEFAULT_MSQ_CONTEXT) + .verifyResults(); + } + @Test public void testIngestWithSanitizedNullByte() throws IOException { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java index c2616c0514e7..0c840b14a88e 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java @@ -53,8 +53,8 @@ import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryRunnerTest; +import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.TestGroupByBuffers; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; @@ -164,9 +164,11 @@ public String getFormatString() .toInstance((segmentId, channelCounters, isReindex) -> getSupplierForSegment(segmentId)); GroupByQueryConfig groupByQueryConfig = new GroupByQueryConfig(); - binder.bind(GroupByStrategySelector.class) - .toInstance(GroupByQueryRunnerTest.makeQueryRunnerFactory(groupByQueryConfig, groupByBuffers) - .getStrategySelector()); + GroupingEngine groupingEngine = GroupByQueryRunnerTest.makeQueryRunnerFactory( + groupByQueryConfig, + groupByBuffers + ).getGroupingEngine(); + binder.bind(GroupingEngine.class).toInstance(groupingEngine); }; return ImmutableList.of( customBindings, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index e3c198ad045a..8d2c304c4945 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -121,8 +121,8 @@ import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryRunnerTest; +import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.TestGroupByBuffers; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.IndexIO; @@ -399,6 +399,11 @@ public String getFormatString() }; GroupByQueryConfig groupByQueryConfig = new GroupByQueryConfig(); + GroupingEngine groupingEngine = GroupByQueryRunnerTest.makeQueryRunnerFactory( + groupByQueryConfig, + groupByBuffers + ).getGroupingEngine(); + binder.bind(GroupingEngine.class).toInstance(groupingEngine); binder.bind(DruidProcessingConfig.class).toInstance(druidProcessingConfig); binder.bind(new TypeLiteral>() @@ -411,10 +416,6 @@ public String getFormatString() binder.bind(IndexIO.class).toInstance(indexIO); binder.bind(SpecificSegmentsQuerySegmentWalker.class).toInstance(qf.walker()); - binder.bind(GroupByStrategySelector.class) - .toInstance(GroupByQueryRunnerTest.makeQueryRunnerFactory(groupByQueryConfig, groupByBuffers) - .getStrategySelector()); - LocalDataSegmentPusherConfig config = new LocalDataSegmentPusherConfig(); try { config.storageDirectory = tmpFolder.newFolder("localsegments"); diff --git a/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorFactory.java b/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorFactory.java index 40d06bbbe02a..b860ea0a80ca 100644 --- a/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorFactory.java +++ b/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorFactory.java @@ -249,12 +249,6 @@ public AggregatorFactory getCombiningFactory() return new VarianceFoldingAggregatorFactory(name, name, estimator); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new VarianceAggregatorFactory(fieldName, fieldName, estimator, inputType)); - } - @Override public AggregatorFactory getMergingFactory(AggregatorFactory other) throws AggregatorFactoryNotMergeableException { diff --git a/licenses.yaml b/licenses.yaml index 2608758cf320..feb85c522dd2 100644 --- a/licenses.yaml +++ b/licenses.yaml @@ -3812,7 +3812,7 @@ name: ANTLR 4 Runtime license_category: binary module: java-core license_name: BSD-3-Clause License -version: 4.5.1 +version: 4.5.3 copyright: The ANTLR Project license_file_path: licenses/bin/antlr4-runtime.BSD3 libraries: diff --git a/owasp-dependency-check-suppressions.xml b/owasp-dependency-check-suppressions.xml index 1eb7c5a72d79..4fcf56053345 100644 --- a/owasp-dependency-check-suppressions.xml +++ b/owasp-dependency-check-suppressions.xml @@ -359,6 +359,9 @@ ]]> CVE-2022-45855 CVE-2022-42009 + + CVE-2022-25168 + CVE-2021-33036 CVE-2023-25613 CVE-2023-2976 + + CVE-2023-1370 + CVE-2023-37475 @@ -785,4 +792,12 @@ ^pkg:maven/.*/.*@.*$ CVE-2021-4277 + + + + ^pkg:maven/com\.squareup\.okio/okio@1..*$ + CVE-2023-3635 + diff --git a/pom.xml b/pom.xml index de5d45cea157..b7ebfa723013 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ org.apache apache - 21 + 25 org.apache.druid @@ -779,12 +779,12 @@ org.antlr antlr4-runtime - 4.5.1 + 4.5.3 org.antlr antlr4-coordinator - 4.5.1 + 4.5.3 commons-cli @@ -1634,7 +1634,7 @@ org.cyclonedx cyclonedx-maven-plugin - 2.7.5 + 2.7.9 package @@ -1744,7 +1744,7 @@ org.antlr antlr4-maven-plugin - 4.5.1 + 4.5.3 org.apache.maven.plugins diff --git a/processing/src/main/java/org/apache/druid/math/expr/BinaryOperatorExpr.java b/processing/src/main/java/org/apache/druid/math/expr/BinaryEvalOpExprBase.java similarity index 100% rename from processing/src/main/java/org/apache/druid/math/expr/BinaryOperatorExpr.java rename to processing/src/main/java/org/apache/druid/math/expr/BinaryEvalOpExprBase.java diff --git a/processing/src/main/java/org/apache/druid/math/expr/BinaryLogicalOperatorExpr.java b/processing/src/main/java/org/apache/druid/math/expr/BinaryLogicalOperatorExpr.java index a65999c1ae99..13bb4e7f52f1 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/BinaryLogicalOperatorExpr.java +++ b/processing/src/main/java/org/apache/druid/math/expr/BinaryLogicalOperatorExpr.java @@ -29,6 +29,12 @@ import javax.annotation.Nullable; import java.util.Objects; +@SuppressWarnings("unused") +final class BinaryLogicalOperatorExpr +{ + // phony class to enable maven to track the compilation of this class +} + // logical operators live here @SuppressWarnings("ClassName") class BinLtExpr extends BinaryBooleanOpExprBase diff --git a/processing/src/main/java/org/apache/druid/math/expr/BinaryMathOperatorExpr.java b/processing/src/main/java/org/apache/druid/math/expr/BinaryMathOperatorExpr.java index 17160c24baee..a11b48af541b 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/BinaryMathOperatorExpr.java +++ b/processing/src/main/java/org/apache/druid/math/expr/BinaryMathOperatorExpr.java @@ -31,6 +31,12 @@ // math operators live here +@SuppressWarnings("unused") +final class BinaryMathOperatorExpr +{ + // phony class to enable maven to track the compilation of this class +} + @SuppressWarnings("ClassName") final class BinPlusExpr extends BinaryEvalOpExprBase { diff --git a/processing/src/main/java/org/apache/druid/math/expr/FunctionalExpr.java b/processing/src/main/java/org/apache/druid/math/expr/FunctionalExpr.java index 3eef9c7bacb5..b83efb2b2bcc 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/FunctionalExpr.java +++ b/processing/src/main/java/org/apache/druid/math/expr/FunctionalExpr.java @@ -30,6 +30,12 @@ import java.util.Set; import java.util.stream.Collectors; +@SuppressWarnings("unused") +final class FunctionalExpr +{ + // phony class to enable maven to track the compilation of this class +} + @SuppressWarnings("ClassName") class LambdaExpr implements Expr { diff --git a/processing/src/main/java/org/apache/druid/math/expr/UnaryOperatorExpr.java b/processing/src/main/java/org/apache/druid/math/expr/UnaryOperatorExpr.java index d260f0b887db..684f3ac2520d 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/UnaryOperatorExpr.java +++ b/processing/src/main/java/org/apache/druid/math/expr/UnaryOperatorExpr.java @@ -32,6 +32,13 @@ import java.math.BigInteger; import java.util.Objects; + +@SuppressWarnings("unused") +final class UnaryOperatorExpr +{ + // phony class to enable maven to track the compilation of this class +} + /** * Base type for all single argument operators, with a single {@link Expr} child for the operand. */ diff --git a/processing/src/main/java/org/apache/druid/query/GroupByMergedQueryRunner.java b/processing/src/main/java/org/apache/druid/query/GroupByMergedQueryRunner.java deleted file mode 100644 index d0d231b0c3df..000000000000 --- a/processing/src/main/java/org/apache/druid/query/GroupByMergedQueryRunner.java +++ /dev/null @@ -1,207 +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.base.Predicates; -import com.google.common.base.Supplier; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.common.guava.GuavaUtils; -import org.apache.druid.data.input.Row; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.guava.Accumulator; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.query.context.ResponseContext; -import org.apache.druid.query.groupby.GroupByQuery; -import org.apache.druid.query.groupby.GroupByQueryConfig; -import org.apache.druid.query.groupby.GroupByQueryHelper; -import org.apache.druid.segment.incremental.IncrementalIndex; - -import java.util.List; -import java.util.Queue; -import java.util.concurrent.CancellationException; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - - -public class GroupByMergedQueryRunner implements QueryRunner -{ - private static final Logger log = new Logger(GroupByMergedQueryRunner.class); - private final Iterable> queryables; - private final Supplier configSupplier; - private final QueryWatcher queryWatcher; - private final QueryProcessingPool queryProcessingPool; - - public GroupByMergedQueryRunner( - QueryProcessingPool queryProcessingPool, - Supplier configSupplier, - QueryWatcher queryWatcher, - Iterable> queryables - ) - { - this.queryProcessingPool = queryProcessingPool; - this.queryWatcher = queryWatcher; - this.queryables = Iterables.unmodifiableIterable(Iterables.filter(queryables, Predicates.notNull())); - this.configSupplier = configSupplier; - } - - @Override - public Sequence run(final QueryPlus queryPlus, final ResponseContext responseContext) - { - final GroupByQuery query = (GroupByQuery) queryPlus.getQuery(); - final GroupByQueryConfig querySpecificConfig = configSupplier.get().withOverrides(query); - final boolean isSingleThreaded = querySpecificConfig.isSingleThreaded(); - final Pair> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair( - query, - null, - querySpecificConfig - ); - final Pair> bySegmentAccumulatorPair = GroupByQueryHelper.createBySegmentAccumulatorPair(); - final QueryContext queryContext = query.context(); - final boolean bySegment = queryContext.isBySegment(); - final int priority = queryContext.getPriority(); - final QueryPlus threadSafeQueryPlus = queryPlus.withoutThreadUnsafeState(); - final List> futures = - Lists.newArrayList( - Iterables.transform( - queryables, - new Function, ListenableFuture>() - { - @Override - public ListenableFuture apply(final QueryRunner input) - { - if (input == null) { - throw new ISE("Null queryRunner! Looks to be some segment unmapping action happening"); - } - - ListenableFuture future = queryProcessingPool.submitRunnerTask( - new AbstractPrioritizedQueryRunnerCallable(priority, input) - { - @Override - public Void call() - { - try { - if (bySegment) { - input.run(threadSafeQueryPlus, responseContext) - .accumulate(bySegmentAccumulatorPair.lhs, bySegmentAccumulatorPair.rhs); - } else { - input.run(threadSafeQueryPlus, responseContext) - .accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); - } - - return null; - } - catch (QueryInterruptedException e) { - throw new RuntimeException(e); - } - catch (Exception e) { - log.error(e, "Exception with one of the sequences!"); - Throwables.propagateIfPossible(e); - throw new RuntimeException(e); - } - } - } - ); - - if (isSingleThreaded) { - waitForFutureCompletion(query, ImmutableList.of(future), indexAccumulatorPair.lhs); - } - - return future; - } - } - ) - ); - - if (!isSingleThreaded) { - waitForFutureCompletion(query, futures, indexAccumulatorPair.lhs); - } - - if (bySegment) { - return Sequences.simple(bySegmentAccumulatorPair.lhs); - } - - return Sequences.withBaggage( - Sequences.simple( - Iterables.transform( - indexAccumulatorPair.lhs.iterableWithPostAggregations(null, query.isDescending()), - new Function() - { - @Override - public T apply(Row input) - { - return (T) input; - } - } - ) - ), indexAccumulatorPair.lhs - ); - } - - private void waitForFutureCompletion( - GroupByQuery query, - List> futures, - IncrementalIndex closeOnFailure - ) - { - ListenableFuture> future = Futures.allAsList(futures); - try { - queryWatcher.registerQueryFuture(query, future); - final QueryContext context = query.context(); - if (context.hasTimeout()) { - future.get(context.getTimeout(), TimeUnit.MILLISECONDS); - } else { - future.get(); - } - } - catch (InterruptedException e) { - log.warn(e, "Query interrupted, cancelling pending results, query id [%s]", query.getId()); - GuavaUtils.cancelAll(true, future, futures); - closeOnFailure.close(); - throw new QueryInterruptedException(e); - } - catch (CancellationException e) { - closeOnFailure.close(); - throw new QueryInterruptedException(e); - } - catch (TimeoutException e) { - closeOnFailure.close(); - log.info("Query timeout, cancelling pending results for query id [%s]", query.getId()); - GuavaUtils.cancelAll(true, future, futures); - throw new QueryTimeoutException(StringUtils.nonStrictFormat("Query [%s] timed out", query.getId())); - } - catch (ExecutionException e) { - GuavaUtils.cancelAll(true, future, futures); - closeOnFailure.close(); - Throwables.propagateIfPossible(e.getCause()); - throw new RuntimeException(e.getCause()); - } - } -} diff --git a/processing/src/main/java/org/apache/druid/query/QueryCapacityExceededException.java b/processing/src/main/java/org/apache/druid/query/QueryCapacityExceededException.java index 694fbb780cb6..82e6ef8eaeef 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryCapacityExceededException.java +++ b/processing/src/main/java/org/apache/druid/query/QueryCapacityExceededException.java @@ -30,7 +30,7 @@ * *
    *
  • When the query is rejected by QueryScheduler.
  • - *
  • When the query cannot acquire enough merge buffers for groupBy v2
  • + *
  • When the query cannot acquire enough merge buffers for groupBy
  • *
*

* As a {@link QueryException} it is expected to be serialied to a json response, but will be mapped to diff --git a/processing/src/main/java/org/apache/druid/query/QueryProcessingPool.java b/processing/src/main/java/org/apache/druid/query/QueryProcessingPool.java index 785c31cad792..da423c6cddd0 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryProcessingPool.java +++ b/processing/src/main/java/org/apache/druid/query/QueryProcessingPool.java @@ -30,7 +30,7 @@ *

* This interface extends {@link ListeningExecutorService} as well. It has a separate * method to submit query execution tasks so that implementations can differentiate those tasks from any regular async - * tasks. One example is {@link org.apache.druid.query.groupby.strategy.GroupByStrategyV2#mergeRunners(QueryProcessingPool, Iterable)} + * tasks. One example is {@link org.apache.druid.query.groupby.GroupingEngine#mergeRunners(QueryProcessingPool, Iterable)} * where different kind of tasks are submitted to same processing pool. *

* Query execution task also includes a reference to {@link QueryRunner} so that any state required to decide the priority diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java index 3e54f449d6c9..8567c4b6d6c6 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java @@ -194,17 +194,15 @@ public AggregatorFactory getMergingFactory(AggregatorFactory other) throws Aggre } /** - * Used by {@link org.apache.druid.query.groupby.strategy.GroupByStrategyV1} when running nested groupBys, to - * "transfer" values from this aggreagtor to an incremental index that the outer query will run on. This method - * only exists due to the design of GroupByStrategyV1, and should probably not be used for anything else. If you are - * here because you are looking for a way to get the input fields required by this aggregator, and thought - * "getRequiredColumns" sounded right, please use {@link #requiredFields()} instead. - * - * @return AggregatorFactories that can be used to "transfer" values from this aggregator into an incremental index - * - * @see #requiredFields() a similarly-named method that is perhaps the one you want instead. + * This was previously used by group-by v1 and will be removed in a future release */ - public abstract List getRequiredColumns(); + @Deprecated + public List getRequiredColumns() + { + throw new UnsupportedOperationException( + "Do not call or implement this method, it is deprecated and will be removed in a future releases." + ); + } /** * A method that knows how to "deserialize" the object from whatever form it might have been put into diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/CountAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/CountAggregatorFactory.java index 24d8a9ce1172..7089789f90a5 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/CountAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/CountAggregatorFactory.java @@ -29,7 +29,6 @@ import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; -import java.util.Collections; import java.util.Comparator; import java.util.List; @@ -97,12 +96,6 @@ public AggregatorFactory getCombiningFactory() return new LongSumAggregatorFactory(name, name); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new CountAggregatorFactory(name)); - } - @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/DoubleMaxAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/DoubleMaxAggregatorFactory.java index 10c93c4e80cf..a4b2ea87e63d 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/DoubleMaxAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/DoubleMaxAggregatorFactory.java @@ -29,8 +29,6 @@ import org.apache.druid.segment.vector.VectorValueSelector; import javax.annotation.Nullable; -import java.util.Collections; -import java.util.List; /** */ @@ -111,12 +109,6 @@ public AggregatorFactory getCombiningFactory() return new DoubleMaxAggregatorFactory(name, name, null, macroTable); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new DoubleMaxAggregatorFactory(fieldName, fieldName, expression, macroTable)); - } - @Override public AggregatorFactory withName(String newName) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/DoubleMinAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/DoubleMinAggregatorFactory.java index 29ffb1a23f33..241d7911af6a 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/DoubleMinAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/DoubleMinAggregatorFactory.java @@ -29,8 +29,6 @@ import org.apache.druid.segment.vector.VectorValueSelector; import javax.annotation.Nullable; -import java.util.Collections; -import java.util.List; /** */ @@ -111,12 +109,6 @@ public AggregatorFactory getCombiningFactory() return new DoubleMinAggregatorFactory(name, name, null, macroTable); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new DoubleMinAggregatorFactory(fieldName, fieldName, expression, macroTable)); - } - @Override public AggregatorFactory withName(String newName) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/DoubleSumAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/DoubleSumAggregatorFactory.java index 86d846f0b048..6a433b7bf8fb 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/DoubleSumAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/DoubleSumAggregatorFactory.java @@ -29,8 +29,6 @@ import org.apache.druid.segment.vector.VectorValueSelector; import javax.annotation.Nullable; -import java.util.Collections; -import java.util.List; /** */ @@ -111,12 +109,6 @@ public AggregatorFactory getCombiningFactory() return new DoubleSumAggregatorFactory(name, name, null, macroTable); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new DoubleSumAggregatorFactory(fieldName, fieldName, expression, macroTable)); - } - @Override public AggregatorFactory withName(String newName) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregatorFactory.java index df1040c29d2f..160dc5c19d7b 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregatorFactory.java @@ -50,7 +50,6 @@ import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Objects; @@ -397,29 +396,6 @@ public AggregatorFactory getCombiningFactory() ); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList( - new ExpressionLambdaAggregatorFactory( - name, - fields, - accumulatorId, - initialValueExpressionString, - initialCombineValueExpressionString, - isNullUnlessAggregated, - shouldAggregateNullInputs, - shouldCombineAggregateNullInputs, - foldExpressionString, - combineExpressionString, - compareExpressionString, - finalizeExpressionString, - maxSizeBytes, - macroTable - ) - ); - } - @Override public ColumnType getIntermediateType() { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregatorFactory.java index 722bfe12ff8a..6e4925b62c0f 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregatorFactory.java @@ -284,12 +284,6 @@ public DimFilter getFilter() return dimFilter; } - @Override - public List getRequiredColumns() - { - return delegate.getRequiredColumns(); - } - @Override public boolean equals(final Object o) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/FloatMaxAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/FloatMaxAggregatorFactory.java index 09c67e22829b..4bfeba615cdb 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/FloatMaxAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/FloatMaxAggregatorFactory.java @@ -29,8 +29,6 @@ import org.apache.druid.segment.vector.VectorValueSelector; import javax.annotation.Nullable; -import java.util.Collections; -import java.util.List; /** */ @@ -111,12 +109,6 @@ public AggregatorFactory getCombiningFactory() return new FloatMaxAggregatorFactory(name, name, null, macroTable); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new FloatMaxAggregatorFactory(fieldName, fieldName, expression, macroTable)); - } - @Override public AggregatorFactory withName(String newName) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/FloatMinAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/FloatMinAggregatorFactory.java index bdf74005c9ef..d720658a6b3f 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/FloatMinAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/FloatMinAggregatorFactory.java @@ -29,8 +29,6 @@ import org.apache.druid.segment.vector.VectorValueSelector; import javax.annotation.Nullable; -import java.util.Collections; -import java.util.List; /** */ @@ -111,12 +109,6 @@ public AggregatorFactory getCombiningFactory() return new FloatMinAggregatorFactory(name, name, null, macroTable); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new FloatMinAggregatorFactory(fieldName, fieldName, expression, macroTable)); - } - @Override public AggregatorFactory withName(String newName) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/FloatSumAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/FloatSumAggregatorFactory.java index 6cc674a37a0d..397b55abed78 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/FloatSumAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/FloatSumAggregatorFactory.java @@ -29,8 +29,6 @@ import org.apache.druid.segment.vector.VectorValueSelector; import javax.annotation.Nullable; -import java.util.Collections; -import java.util.List; /** */ @@ -111,12 +109,6 @@ public AggregatorFactory getCombiningFactory() return new FloatSumAggregatorFactory(name, name, null, macroTable); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new FloatSumAggregatorFactory(fieldName, fieldName, expression, macroTable)); - } - @Override public AggregatorFactory withName(String newName) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/GroupingAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/GroupingAggregatorFactory.java index 18b188389b40..8f8f7be4a14d 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/GroupingAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/GroupingAggregatorFactory.java @@ -180,12 +180,6 @@ public AggregatorFactory getCombiningFactory() return new GroupingAggregatorFactory(name, groupings, keyDimensions); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new GroupingAggregatorFactory(name, groupings, keyDimensions)); - } - @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/HistogramAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/HistogramAggregatorFactory.java index c3bbeae15196..5e8a225008f5 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/HistogramAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/HistogramAggregatorFactory.java @@ -138,13 +138,6 @@ public AggregatorFactory getCombiningFactory() return new HistogramAggregatorFactory(name, name, breaksList); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList( - new HistogramAggregatorFactory(fieldName, fieldName, breaksList)); - } - @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/JavaScriptAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/JavaScriptAggregatorFactory.java index 2a43fb3c2493..55119a9e4ccc 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/JavaScriptAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/JavaScriptAggregatorFactory.java @@ -25,8 +25,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; import it.unimi.dsi.fastutil.objects.ObjectArrays; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.js.JavaScriptConfig; @@ -176,24 +174,6 @@ public AggregatorFactory getMergingFactory(AggregatorFactory other) throws Aggre throw new AggregatorFactoryNotMergeableException(this, other); } - @Override - public List getRequiredColumns() - { - return ImmutableList.copyOf( - Lists.transform( - fieldNames, - new com.google.common.base.Function() - { - @Override - public AggregatorFactory apply(String input) - { - return new JavaScriptAggregatorFactory(input, Collections.singletonList(input), fnCombine, fnReset, fnCombine, config); - } - } - ) - ); - } - @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/LongMaxAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/LongMaxAggregatorFactory.java index 926ad339c7e4..1304d272ede9 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/LongMaxAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/LongMaxAggregatorFactory.java @@ -29,8 +29,6 @@ import org.apache.druid.segment.vector.VectorValueSelector; import javax.annotation.Nullable; -import java.util.Collections; -import java.util.List; /** */ @@ -111,12 +109,6 @@ public AggregatorFactory getCombiningFactory() return new LongMaxAggregatorFactory(name, name, null, macroTable); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new LongMaxAggregatorFactory(fieldName, fieldName, expression, macroTable)); - } - @Override public AggregatorFactory withName(String newName) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/LongMinAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/LongMinAggregatorFactory.java index f16e2fcc8b6a..3073217986cf 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/LongMinAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/LongMinAggregatorFactory.java @@ -29,8 +29,6 @@ import org.apache.druid.segment.vector.VectorValueSelector; import javax.annotation.Nullable; -import java.util.Collections; -import java.util.List; /** */ @@ -111,12 +109,6 @@ public AggregatorFactory getCombiningFactory() return new LongMinAggregatorFactory(name, name, null, macroTable); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new LongMinAggregatorFactory(fieldName, fieldName, expression, macroTable)); - } - @Override public AggregatorFactory withName(String newName) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/LongSumAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/LongSumAggregatorFactory.java index 4e22ad106f85..9d1c5fcdd84a 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/LongSumAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/LongSumAggregatorFactory.java @@ -29,8 +29,6 @@ import org.apache.druid.segment.vector.VectorValueSelector; import javax.annotation.Nullable; -import java.util.Collections; -import java.util.List; /** */ @@ -117,12 +115,6 @@ public AggregatorFactory getCombiningFactory() return new LongSumAggregatorFactory(name, name, null, macroTable); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new LongSumAggregatorFactory(fieldName, fieldName, expression, macroTable)); - } - @Override public byte[] getCacheKey() { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/SuppressedAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/SuppressedAggregatorFactory.java index eb9605e8e3c8..1f066f1b12b0 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/SuppressedAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/SuppressedAggregatorFactory.java @@ -107,12 +107,6 @@ public AggregatorFactory getMergingFactory(AggregatorFactory other) throws Aggre return delegate.getMergingFactory(other); } - @Override - public List getRequiredColumns() - { - return delegate.getRequiredColumns(); - } - @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java index ca4c1d65d6a1..86f85455a6da 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java @@ -158,12 +158,6 @@ public AggregatorFactory getCombiningFactory() return new DoubleAnyAggregatorFactory(name, name); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new DoubleAnyAggregatorFactory(fieldName, fieldName)); - } - @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/any/FloatAnyAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/any/FloatAnyAggregatorFactory.java index 97ecab10228a..35495c9e30eb 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/any/FloatAnyAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/any/FloatAnyAggregatorFactory.java @@ -156,12 +156,6 @@ public AggregatorFactory getCombiningFactory() return new FloatAnyAggregatorFactory(name, name); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new FloatAnyAggregatorFactory(fieldName, fieldName)); - } - @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/any/LongAnyAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/any/LongAnyAggregatorFactory.java index 6d6197f06ede..9af417a600db 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/any/LongAnyAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/any/LongAnyAggregatorFactory.java @@ -154,12 +154,6 @@ public AggregatorFactory getCombiningFactory() return new LongAnyAggregatorFactory(name, name); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new LongAnyAggregatorFactory(fieldName, fieldName)); - } - @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/any/StringAnyAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/any/StringAnyAggregatorFactory.java index 6deadfbb131e..307de0650c3a 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/any/StringAnyAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/any/StringAnyAggregatorFactory.java @@ -125,12 +125,6 @@ public AggregatorFactory getCombiningFactory() return new StringAnyAggregatorFactory(name, name, maxStringBytes); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new StringAnyAggregatorFactory(fieldName, fieldName, maxStringBytes)); - } - @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java index 39af58c8b351..5f2297b65521 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java @@ -53,7 +53,6 @@ import javax.annotation.Nullable; import java.nio.ByteBuffer; -import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Objects; @@ -233,22 +232,6 @@ public AggregatorFactory getCombiningFactory() return new HyperUniquesAggregatorFactory(name, name, false, round); } - @Override - public List getRequiredColumns() - { - return fields.stream() - .map( - field -> - new CardinalityAggregatorFactory( - field.getOutputName(), - null, - Collections.singletonList(field), - byRow, - round - ) - ) - .collect(Collectors.toList()); - } @Override public Object deserialize(Object object) diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/first/DoubleFirstAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/first/DoubleFirstAggregatorFactory.java index d575b263f0fb..d3b0e4449530 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/first/DoubleFirstAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/first/DoubleFirstAggregatorFactory.java @@ -41,7 +41,6 @@ import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.Arrays; -import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -223,12 +222,6 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) }; } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new DoubleFirstAggregatorFactory(fieldName, fieldName, timeColumn)); - } - @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/first/FloatFirstAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/first/FloatFirstAggregatorFactory.java index be6a0f6aad97..809cc8183ed5 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/first/FloatFirstAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/first/FloatFirstAggregatorFactory.java @@ -41,7 +41,6 @@ import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.Arrays; -import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -220,11 +219,6 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) }; } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new FloatFirstAggregatorFactory(fieldName, fieldName, timeColumn)); - } @Override public Object deserialize(Object object) diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/first/LongFirstAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/first/LongFirstAggregatorFactory.java index 695d01b3a4a7..c767cf0e09c7 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/first/LongFirstAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/first/LongFirstAggregatorFactory.java @@ -41,7 +41,6 @@ import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.Arrays; -import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -218,12 +217,6 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) }; } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new LongFirstAggregatorFactory(fieldName, fieldName, timeColumn)); - } - @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/first/StringFirstAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/first/StringFirstAggregatorFactory.java index f7624f4541b3..63e48e878a84 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/first/StringFirstAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/first/StringFirstAggregatorFactory.java @@ -42,7 +42,6 @@ import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.Arrays; -import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -178,12 +177,6 @@ public AggregatorFactory getCombiningFactory() return new StringFirstAggregatorFactory(name, name, timeColumn, maxStringBytes); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new StringFirstAggregatorFactory(fieldName, fieldName, timeColumn, maxStringBytes)); - } - @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java index 3cb970450373..c1c55a826b10 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java @@ -187,17 +187,6 @@ public AggregatorFactory getMergingFactory(AggregatorFactory other) throws Aggre } } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new HyperUniquesAggregatorFactory( - fieldName, - fieldName, - isInputHyperUnique, - round - )); - } - @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/last/DoubleLastAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/last/DoubleLastAggregatorFactory.java index ee6c330249d0..d3770fb3ae47 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/last/DoubleLastAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/last/DoubleLastAggregatorFactory.java @@ -32,7 +32,6 @@ import org.apache.druid.query.aggregation.VectorAggregator; import org.apache.druid.query.aggregation.any.NumericNilVectorAggregator; import org.apache.druid.query.aggregation.first.DoubleFirstAggregatorFactory; -import org.apache.druid.query.aggregation.first.LongFirstAggregatorFactory; import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.BaseDoubleColumnValueSelector; @@ -49,7 +48,6 @@ import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.Arrays; -import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -250,12 +248,6 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) }; } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new LongFirstAggregatorFactory(fieldName, fieldName, timeColumn)); - } - @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/last/FloatLastAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/last/FloatLastAggregatorFactory.java index f9fb0b3a4c4d..dff50b095bcf 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/last/FloatLastAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/last/FloatLastAggregatorFactory.java @@ -32,7 +32,6 @@ import org.apache.druid.query.aggregation.VectorAggregator; import org.apache.druid.query.aggregation.any.NumericNilVectorAggregator; import org.apache.druid.query.aggregation.first.FloatFirstAggregatorFactory; -import org.apache.druid.query.aggregation.first.LongFirstAggregatorFactory; import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.BaseFloatColumnValueSelector; @@ -49,7 +48,6 @@ import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.Arrays; -import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -246,12 +244,6 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) }; } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new LongFirstAggregatorFactory(fieldName, fieldName, timeColumn)); - } - @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/last/LongLastAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/last/LongLastAggregatorFactory.java index c08967461879..29d9ad2a06ea 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/last/LongLastAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/last/LongLastAggregatorFactory.java @@ -48,7 +48,6 @@ import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.Arrays; -import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -244,12 +243,6 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) }; } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new LongLastAggregatorFactory(fieldName, fieldName, timeColumn)); - } - @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/last/StringLastAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/last/StringLastAggregatorFactory.java index 2f135f96cca1..7e414638d2f7 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/last/StringLastAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/last/StringLastAggregatorFactory.java @@ -48,7 +48,6 @@ import javax.annotation.Nullable; import java.nio.ByteBuffer; -import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -193,12 +192,6 @@ public AggregatorFactory getCombiningFactory() return new StringLastAggregatorFactory(name, name, timeColumn, maxStringBytes); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new StringLastAggregatorFactory(fieldName, fieldName, timeColumn, maxStringBytes)); - } - @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregatorFactory.java index dc939e143237..58cbd502fef0 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregatorFactory.java @@ -160,12 +160,6 @@ public AggregatorFactory getCombiningFactory() return new DoubleMeanAggregatorFactory(name, name); } - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new DoubleMeanAggregatorFactory(fieldName, fieldName)); - } - @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java index 9a63a796d971..045083170ca7 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java @@ -73,7 +73,6 @@ import org.joda.time.Interval; import javax.annotation.Nullable; - import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -83,7 +82,6 @@ import java.util.Map; import java.util.Objects; import java.util.Set; -import java.util.UUID; import java.util.stream.Collectors; /** @@ -1196,11 +1194,6 @@ public Builder setContext(Map context) return this; } - public Builder randomQueryId() - { - return queryId(UUID.randomUUID().toString()); - } - public Builder queryId(String queryId) { context = BaseQuery.computeOverriddenContext(context, ImmutableMap.of(BaseQuery.QUERY_ID, queryId)); diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryConfig.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryConfig.java index 380cf825f603..6b94ba3bc53d 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryConfig.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryConfig.java @@ -25,7 +25,6 @@ import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryContexts; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.utils.JvmUtils; /** @@ -37,7 +36,6 @@ public class GroupByQueryConfig public static final long AUTOMATIC = 0; - public static final String CTX_KEY_STRATEGY = "groupByStrategy"; public static final String CTX_KEY_FORCE_LIMIT_PUSH_DOWN = "forceLimitPushDown"; public static final String CTX_KEY_APPLY_LIMIT_PUSH_DOWN = "applyLimitPushDown"; public static final String CTX_KEY_APPLY_LIMIT_PUSH_DOWN_TO_SEGMENT = "applyLimitPushDownToSegment"; @@ -47,8 +45,6 @@ public class GroupByQueryConfig public static final String CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING = "groupByEnableMultiValueUnnesting"; public static final String CTX_KEY_BUFFER_GROUPER_MAX_SIZE = "bufferGrouperMaxSize"; private static final String CTX_KEY_IS_SINGLE_THREADED = "groupByIsSingleThreaded"; - private static final String CTX_KEY_MAX_INTERMEDIATE_ROWS = "maxIntermediateRows"; - private static final String CTX_KEY_MAX_RESULTS = "maxResults"; private static final String CTX_KEY_BUFFER_GROUPER_INITIAL_BUCKETS = "bufferGrouperInitialBuckets"; private static final String CTX_KEY_BUFFER_GROUPER_MAX_LOAD_FACTOR = "bufferGrouperMaxLoadFactor"; private static final String CTX_KEY_MAX_ON_DISK_STORAGE = "maxOnDiskStorage"; @@ -71,18 +67,9 @@ public class GroupByQueryConfig private static final long MIN_AUTOMATIC_DICTIONARY_SIZE = 1; private static final long MAX_AUTOMATIC_DICTIONARY_SIZE = 1_000_000_000; - @JsonProperty - private String defaultStrategy = GroupByStrategySelector.STRATEGY_V2; - @JsonProperty private boolean singleThreaded = false; - @JsonProperty - private int maxIntermediateRows = 50000; - - @JsonProperty - private int maxResults = 500000; - @JsonProperty // Not documented, only used for tests to force spilling private int bufferGrouperMaxSize = Integer.MAX_VALUE; @@ -139,11 +126,6 @@ public class GroupByQueryConfig @JsonProperty private boolean enableMultiValueUnnesting = true; - public String getDefaultStrategy() - { - return defaultStrategy; - } - public boolean isSingleThreaded() { return singleThreaded; @@ -154,26 +136,6 @@ public void setSingleThreaded(boolean singleThreaded) this.singleThreaded = singleThreaded; } - public int getMaxIntermediateRows() - { - return maxIntermediateRows; - } - - public void setMaxIntermediateRows(int maxIntermediateRows) - { - this.maxIntermediateRows = maxIntermediateRows; - } - - public int getMaxResults() - { - return maxResults; - } - - public void setMaxResults(int maxResults) - { - this.maxResults = maxResults; - } - public int getBufferGrouperMaxSize() { return bufferGrouperMaxSize; @@ -337,16 +299,7 @@ public GroupByQueryConfig withOverrides(final GroupByQuery query) { final GroupByQueryConfig newConfig = new GroupByQueryConfig(); final QueryContext queryContext = query.context(); - newConfig.defaultStrategy = queryContext.getString(CTX_KEY_STRATEGY, getDefaultStrategy()); newConfig.singleThreaded = queryContext.getBoolean(CTX_KEY_IS_SINGLE_THREADED, isSingleThreaded()); - newConfig.maxIntermediateRows = Math.min( - queryContext.getInt(CTX_KEY_MAX_INTERMEDIATE_ROWS, getMaxIntermediateRows()), - getMaxIntermediateRows() - ); - newConfig.maxResults = Math.min( - queryContext.getInt(CTX_KEY_MAX_RESULTS, getMaxResults()), - getMaxResults() - ); newConfig.bufferGrouperMaxSize = Math.min( queryContext.getInt(CTX_KEY_BUFFER_GROUPER_MAX_SIZE, getBufferGrouperMaxSize()), getBufferGrouperMaxSize() @@ -403,10 +356,7 @@ public GroupByQueryConfig withOverrides(final GroupByQuery query) public String toString() { return "GroupByQueryConfig{" + - "defaultStrategy='" + defaultStrategy + '\'' + - ", singleThreaded=" + singleThreaded + - ", maxIntermediateRows=" + maxIntermediateRows + - ", maxResults=" + maxResults + + "singleThreaded=" + singleThreaded + ", bufferGrouperMaxSize=" + bufferGrouperMaxSize + ", bufferGrouperMaxLoadFactor=" + bufferGrouperMaxLoadFactor + ", bufferGrouperInitialBuckets=" + bufferGrouperInitialBuckets + diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryEngine.java deleted file mode 100644 index e4236f411d02..000000000000 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryEngine.java +++ /dev/null @@ -1,460 +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.groupby; - -import com.google.common.base.Function; -import com.google.common.base.Supplier; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.inject.Inject; -import org.apache.druid.collections.NonBlockingPool; -import org.apache.druid.collections.ResourceHolder; -import org.apache.druid.data.input.MapBasedRow; -import org.apache.druid.data.input.Row; -import org.apache.druid.guice.annotations.Global; -import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.UOE; -import org.apache.druid.java.util.common.guava.BaseSequence; -import org.apache.druid.java.util.common.guava.FunctionalIterator; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.java.util.common.parsers.CloseableIterator; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.BufferAggregator; -import org.apache.druid.query.aggregation.PostAggregator; -import org.apache.druid.query.dimension.DimensionSpec; -import org.apache.druid.query.filter.Filter; -import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.DimensionDictionarySelector; -import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.column.ValueType; -import org.apache.druid.segment.data.IndexedInts; -import org.apache.druid.segment.filter.Filters; -import org.apache.druid.utils.CloseableUtils; -import org.joda.time.DateTime; -import org.joda.time.Interval; - -import javax.annotation.Nullable; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.TreeMap; - -/** - * - */ -public class GroupByQueryEngine -{ - private static final int MISSING_VALUE = -1; - - private final Supplier config; - private final NonBlockingPool intermediateResultsBufferPool; - - @Inject - public GroupByQueryEngine( - Supplier config, - @Global NonBlockingPool intermediateResultsBufferPool - ) - { - this.config = config; - this.intermediateResultsBufferPool = intermediateResultsBufferPool; - } - - public Sequence process( - final GroupByQuery query, - final StorageAdapter storageAdapter, - @Nullable final GroupByQueryMetrics groupByQueryMetrics - ) - { - if (storageAdapter == null) { - throw new ISE( - "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." - ); - } - if (!query.context().getBoolean(GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING, true)) { - throw new UOE( - "GroupBy v1 does not support %s as false. Set %s to true or use groupBy v2", - GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING, - GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING - ); - } - final List intervals = query.getQuerySegmentSpec().getIntervals(); - if (intervals.size() != 1) { - throw new IAE("Should only have one interval, got[%s]", intervals); - } - - Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getDimFilter())); - - final Sequence cursors = storageAdapter.makeCursors( - filter, - intervals.get(0), - query.getVirtualColumns(), - query.getGranularity(), - false, - groupByQueryMetrics - ); - - final ResourceHolder bufferHolder = intermediateResultsBufferPool.take(); - - return Sequences.concat( - Sequences.withBaggage( - Sequences.map( - cursors, - new Function>() - { - @Override - public Sequence apply(final Cursor cursor) - { - return new BaseSequence<>( - new BaseSequence.IteratorMaker() - { - @Override - public RowIterator make() - { - return new RowIterator(query, cursor, bufferHolder.get(), config.get()); - } - - @Override - public void cleanup(RowIterator iterFromMake) - { - CloseableUtils.closeAndWrapExceptions(iterFromMake); - } - } - ); - } - } - ), - bufferHolder - ) - ); - } - - private static class RowUpdater - { - private final ByteBuffer metricValues; - private final BufferAggregator[] aggregators; - private final PositionMaintainer positionMaintainer; - - private final Map positions = new TreeMap<>(); - // GroupBy queries tend to do a lot of reads from this. We co-store a hash map to make those reads go faster. - private final Map positionsHash = new HashMap<>(); - - public RowUpdater( - ByteBuffer metricValues, - BufferAggregator[] aggregators, - PositionMaintainer positionMaintainer - ) - { - this.metricValues = metricValues; - this.aggregators = aggregators; - this.positionMaintainer = positionMaintainer; - } - - public int getNumRows() - { - return positions.size(); - } - - public Map getPositions() - { - return positions; - } - - @Nullable - private List updateValues(ByteBuffer key, List dims) - { - if (dims.size() > 0) { - final DimensionSelector dimSelector = dims.get(0); - final IndexedInts row = dimSelector.getRow(); - final int rowSize = row.size(); - if (rowSize == 0) { - ByteBuffer newKey = key.duplicate(); - newKey.putInt(MISSING_VALUE); - return updateValues(newKey, dims.subList(1, dims.size())); - } else { - List retVal = null; - for (int i = 0; i < rowSize; i++) { - ByteBuffer newKey = key.duplicate(); - int dimValue = row.get(i); - newKey.putInt(dimValue); - List unaggregatedBuffers = updateValues(newKey, dims.subList(1, dims.size())); - if (unaggregatedBuffers != null) { - if (retVal == null) { - retVal = new ArrayList<>(); - } - retVal.addAll(unaggregatedBuffers); - } - } - return retVal; - } - } else { - key.clear(); - Integer position = positionsHash.get(key); - int[] increments = positionMaintainer.getIncrements(); - int thePosition; - - if (position == null) { - ByteBuffer keyCopy = ByteBuffer.allocate(key.limit()); - keyCopy.put(key.asReadOnlyBuffer()); - keyCopy.clear(); - - position = positionMaintainer.getNext(); - if (position == null) { - return Collections.singletonList(keyCopy); - } - - positions.put(keyCopy, position); - positionsHash.put(keyCopy, position); - thePosition = position; - for (int i = 0; i < aggregators.length; ++i) { - aggregators[i].init(metricValues, thePosition); - thePosition += increments[i]; - } - } - - thePosition = position; - for (int i = 0; i < aggregators.length; ++i) { - aggregators[i].aggregate(metricValues, thePosition); - thePosition += increments[i]; - } - return null; - } - } - } - - private static class PositionMaintainer - { - private final int[] increments; - private final int increment; - private final int max; - - private long nextVal; - - public PositionMaintainer( - int start, - int[] increments, - int max - ) - { - this.nextVal = (long) start; - this.increments = increments; - - int theIncrement = 0; - for (int inc : increments) { - theIncrement += inc; - } - increment = theIncrement; - - this.max = max - increment; // Make sure there is enough room for one more increment - } - - @Nullable - public Integer getNext() - { - if (nextVal > max) { - return null; - } else { - int retVal = (int) nextVal; - nextVal += increment; - return retVal; - } - } - - public int getIncrement() - { - return increment; - } - - public int[] getIncrements() - { - return increments; - } - } - - private static class RowIterator implements CloseableIterator - { - private final GroupByQuery query; - private final Cursor cursor; - private final ByteBuffer metricsBuffer; - private final int maxIntermediateRows; - - private final List dimensions; - private final ArrayList dimNames; - private final BufferAggregator[] aggregators; - private final String[] metricNames; - private final int[] sizesRequired; - - @Nullable - private List unprocessedKeys; - private Iterator delegate; - - public RowIterator(GroupByQuery query, final Cursor cursor, ByteBuffer metricsBuffer, GroupByQueryConfig config) - { - final GroupByQueryConfig querySpecificConfig = config.withOverrides(query); - - this.query = query; - this.cursor = cursor; - this.metricsBuffer = metricsBuffer; - this.maxIntermediateRows = querySpecificConfig.getMaxIntermediateRows(); - - unprocessedKeys = null; - delegate = Collections.emptyIterator(); - List dimensionSpecs = query.getDimensions(); - dimensions = Lists.newArrayListWithExpectedSize(dimensionSpecs.size()); - dimNames = Lists.newArrayListWithExpectedSize(dimensionSpecs.size()); - - for (final DimensionSpec dimSpec : dimensionSpecs) { - if (!dimSpec.getOutputType().is(ValueType.STRING)) { - throw new UnsupportedOperationException( - "GroupBy v1 only supports dimensions with an outputType of STRING." - ); - } - - final DimensionSelector selector = cursor.getColumnSelectorFactory().makeDimensionSelector(dimSpec); - if (selector.getValueCardinality() == DimensionDictionarySelector.CARDINALITY_UNKNOWN) { - throw new UnsupportedOperationException( - "GroupBy v1 does not support dimension selectors with unknown cardinality."); - } - dimensions.add(selector); - dimNames.add(dimSpec.getOutputName()); - } - - List aggregatorSpecs = query.getAggregatorSpecs(); - aggregators = new BufferAggregator[aggregatorSpecs.size()]; - metricNames = new String[aggregatorSpecs.size()]; - sizesRequired = new int[aggregatorSpecs.size()]; - for (int i = 0; i < aggregatorSpecs.size(); ++i) { - AggregatorFactory aggregatorSpec = aggregatorSpecs.get(i); - aggregators[i] = aggregatorSpec.factorizeBuffered(cursor.getColumnSelectorFactory()); - metricNames[i] = aggregatorSpec.getName(); - sizesRequired[i] = aggregatorSpec.getMaxIntermediateSizeWithNulls(); - } - } - - @Override - public boolean hasNext() - { - return delegate.hasNext() || !cursor.isDone(); - } - - @Override - public Row next() - { - if (delegate.hasNext()) { - return delegate.next(); - } - - if (unprocessedKeys == null && cursor.isDone()) { - throw new NoSuchElementException(); - } - - final PositionMaintainer positionMaintainer = new PositionMaintainer(0, sizesRequired, metricsBuffer.remaining()); - final RowUpdater rowUpdater = new RowUpdater(metricsBuffer, aggregators, positionMaintainer); - if (unprocessedKeys != null) { - for (ByteBuffer key : unprocessedKeys) { - final List unprocUnproc = rowUpdater.updateValues(key, ImmutableList.of()); - if (unprocUnproc != null) { - throw new ISE("Not enough memory to process the request."); - } - } - cursor.advance(); - } - while (!cursor.isDone() && rowUpdater.getNumRows() < maxIntermediateRows) { - ByteBuffer key = ByteBuffer.allocate(dimensions.size() * Integer.BYTES); - - unprocessedKeys = rowUpdater.updateValues(key, dimensions); - if (unprocessedKeys != null) { - break; - } - - cursor.advance(); - } - - if (rowUpdater.getPositions().isEmpty() && unprocessedKeys != null) { - throw new ISE( - "Not enough memory to process even a single item. Required [%,d] memory, but only have[%,d]", - positionMaintainer.getIncrement(), metricsBuffer.remaining() - ); - } - - delegate = FunctionalIterator - .create(rowUpdater.getPositions().entrySet().iterator()) - .transform( - new Function, Row>() - { - private final DateTime timestamp = cursor.getTime(); - private final int[] increments = positionMaintainer.getIncrements(); - - @Override - public Row apply(@Nullable Map.Entry input) - { - Map theEvent = Maps.newLinkedHashMap(); - - ByteBuffer keyBuffer = input.getKey().duplicate(); - for (int i = 0; i < dimensions.size(); ++i) { - final DimensionSelector dimSelector = dimensions.get(i); - final int dimVal = keyBuffer.getInt(); - if (MISSING_VALUE != dimVal) { - theEvent.put(dimNames.get(i), dimSelector.lookupName(dimVal)); - } - } - - int position = input.getValue(); - for (int i = 0; i < aggregators.length; ++i) { - theEvent.put(metricNames[i], aggregators[i].get(metricsBuffer, position)); - position += increments[i]; - } - - for (PostAggregator postAggregator : query.getPostAggregatorSpecs()) { - theEvent.put(postAggregator.getName(), postAggregator.compute(theEvent)); - } - - return new MapBasedRow(timestamp, theEvent); - } - } - ); - - return delegate.next(); - } - - @Override - public void remove() - { - throw new UnsupportedOperationException(); - } - - @Override - public void close() - { - // cleanup - for (BufferAggregator agg : aggregators) { - agg.close(); - } - } - } -} diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryHelper.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryHelper.java deleted file mode 100644 index 9f65fac85fba..000000000000 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryHelper.java +++ /dev/null @@ -1,247 +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.groupby; - -import com.google.common.base.Function; -import com.google.common.collect.Lists; -import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.data.input.MapBasedRow; -import org.apache.druid.data.input.Row; -import org.apache.druid.data.input.impl.DimensionSchema; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; -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.Accumulator; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.query.ResourceLimitExceededException; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.PostAggregator; -import org.apache.druid.query.dimension.DimensionSpec; -import org.apache.druid.segment.incremental.AppendableIndexBuilder; -import org.apache.druid.segment.incremental.IncrementalIndex; -import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IndexSizeExceededException; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; - -import javax.annotation.Nullable; -import java.util.ArrayList; -import java.util.List; -import java.util.Queue; -import java.util.concurrent.ConcurrentLinkedQueue; - -public class GroupByQueryHelper -{ - public static final String CTX_KEY_SORT_RESULTS = "sortResults"; - - public static Pair> createIndexAccumulatorPair( - final GroupByQuery query, - @Nullable final GroupByQuery subquery, - final GroupByQueryConfig config - ) - { - final GroupByQueryConfig querySpecificConfig = config.withOverrides(query); - final Granularity gran = query.getGranularity(); - final long timeStart = query.getIntervals().get(0).getStartMillis(); - final boolean combine = subquery == null; - - long granTimeStart = timeStart; - if (!(Granularities.ALL.equals(gran))) { - granTimeStart = gran.bucketStart(timeStart); - } - - final List aggs; - if (combine) { - aggs = Lists.transform( - query.getAggregatorSpecs(), - new Function() - { - @Override - public AggregatorFactory apply(AggregatorFactory input) - { - return input.getCombiningFactory(); - } - } - ); - } else { - aggs = query.getAggregatorSpecs(); - } - - final List dimensions = Lists.transform( - query.getDimensions(), - new Function() - { - @Override - public String apply(DimensionSpec input) - { - return input.getOutputName(); - } - } - ); - final IncrementalIndex index; - - final boolean sortResults = query.context().getBoolean(CTX_KEY_SORT_RESULTS, true); - - // All groupBy dimensions are strings, for now. - final List dimensionSchemas = new ArrayList<>(); - for (DimensionSpec dimension : query.getDimensions()) { - dimensionSchemas.add(new StringDimensionSchema(dimension.getOutputName())); - } - - final IncrementalIndexSchema indexSchema = new IncrementalIndexSchema.Builder() - .withDimensionsSpec(new DimensionsSpec(dimensionSchemas)) - .withMetrics(aggs.toArray(new AggregatorFactory[0])) - .withQueryGranularity(gran) - .withMinTimestamp(granTimeStart) - .build(); - - - final AppendableIndexBuilder indexBuilder; - - if (query.context().getBoolean("useOffheap", false)) { - throw new UnsupportedOperationException( - "The 'useOffheap' option is no longer available for groupBy v1. Please move to the newer groupBy engine, " - + "which always operates off-heap, by removing any custom 'druid.query.groupBy.defaultStrategy' runtime " - + "properties and 'groupByStrategy' query context parameters that you have set." - ); - } else { - indexBuilder = new OnheapIncrementalIndex.Builder(); - } - - index = indexBuilder - .setIndexSchema(indexSchema) - .setDeserializeComplexMetrics(false) - .setConcurrentEventAdd(true) - .setSortFacts(sortResults) - .setMaxRowCount(querySpecificConfig.getMaxResults()) - .build(); - - Accumulator accumulator = new Accumulator() - { - @Override - public IncrementalIndex accumulate(IncrementalIndex accumulated, T in) - { - final MapBasedRow mapBasedRow; - - if (in instanceof MapBasedRow) { - mapBasedRow = (MapBasedRow) in; - } else if (in instanceof ResultRow) { - final ResultRow row = (ResultRow) in; - mapBasedRow = row.toMapBasedRow(combine ? query : subquery); - } else { - throw new ISE("Unable to accumulate something of type [%s]", in.getClass()); - } - - try { - accumulated.add( - new MapBasedInputRow( - mapBasedRow.getTimestamp(), - dimensions, - mapBasedRow.getEvent() - ) - ); - } - catch (IndexSizeExceededException e) { - throw new ResourceLimitExceededException(e.getMessage()); - } - - return accumulated; - } - }; - return new Pair<>(index, accumulator); - } - - public static Pair> createBySegmentAccumulatorPair() - { - // In parallel query runner multiple threads add to this queue concurrently - Queue init = new ConcurrentLinkedQueue<>(); - Accumulator accumulator = new Accumulator() - { - @Override - public Queue accumulate(Queue accumulated, T in) - { - if (in == null) { - throw new ISE("Cannot have null result"); - } - accumulated.offer(in); - return accumulated; - } - }; - return new Pair<>(init, accumulator); - } - - // Used by GroupByStrategyV1 - public static IncrementalIndex makeIncrementalIndex( - GroupByQuery query, - @Nullable GroupByQuery subquery, - GroupByQueryConfig config, - Sequence rows - ) - { - final Pair> indexAccumulatorPair = - GroupByQueryHelper.createIndexAccumulatorPair(query, subquery, config); - - return rows.accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); - } - - // Used by GroupByStrategyV1 - public static Sequence postAggregate(final GroupByQuery query, IncrementalIndex index) - { - return Sequences.map( - Sequences.simple(index.iterableWithPostAggregations(query.getPostAggregatorSpecs(), query.isDescending())), - input -> { - final ResultRow resultRow = toResultRow(query, input); - - if (query.getResultRowHasTimestamp()) { - resultRow.set(0, query.getGranularity().toDateTime(resultRow.getLong(0)).getMillis()); - } - - return resultRow; - } - ); - } - - public static ResultRow toResultRow(final GroupByQuery query, final Row row) - { - final ResultRow resultRow = ResultRow.create(query.getResultRowSizeWithPostAggregators()); - int i = 0; - - if (query.getResultRowHasTimestamp()) { - resultRow.set(i++, row.getTimestampFromEpoch()); - } - - for (DimensionSpec dimensionSpec : query.getDimensions()) { - resultRow.set(i++, row.getRaw(dimensionSpec.getOutputName())); - } - - for (AggregatorFactory aggregatorFactory : query.getAggregatorSpecs()) { - resultRow.set(i++, row.getRaw(aggregatorFactory.getName())); - } - - for (PostAggregator postAggregator : query.getPostAggregatorSpecs()) { - resultRow.set(i++, row.getRaw(postAggregator.getName())); - } - - return resultRow; - } -} 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 ac1eec040252..73205d2b75fa 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 @@ -32,7 +32,6 @@ import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.base.Supplier; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.inject.Inject; import org.apache.druid.data.input.Row; @@ -68,9 +67,6 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.extraction.ExtractionFn; -import org.apache.druid.query.groupby.resource.GroupByQueryResource; -import org.apache.druid.query.groupby.strategy.GroupByStrategy; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.column.RowSignature; @@ -100,26 +96,25 @@ public class GroupByQueryQueryToolChest extends QueryToolChest TYPE_REFERENCE = new TypeReference() { }; - public static final String GROUP_BY_MERGE_KEY = "groupByMerge"; - private final GroupByStrategySelector strategySelector; + private final GroupingEngine groupingEngine; private final GroupByQueryConfig queryConfig; private final GroupByQueryMetricsFactory queryMetricsFactory; @VisibleForTesting - public GroupByQueryQueryToolChest(GroupByStrategySelector strategySelector) + public GroupByQueryQueryToolChest(GroupingEngine groupingEngine) { - this(strategySelector, GroupByQueryConfig::new, DefaultGroupByQueryMetricsFactory.instance()); + this(groupingEngine, GroupByQueryConfig::new, DefaultGroupByQueryMetricsFactory.instance()); } @Inject public GroupByQueryQueryToolChest( - GroupByStrategySelector strategySelector, + GroupingEngine groupingEngine, Supplier queryConfigSupplier, GroupByQueryMetricsFactory queryMetricsFactory ) { - this.strategySelector = strategySelector; + this.groupingEngine = groupingEngine; this.queryConfig = queryConfigSupplier.get(); this.queryMetricsFactory = queryMetricsFactory; } @@ -133,23 +128,20 @@ public QueryRunner mergeResults(final QueryRunner runner) } final GroupByQuery groupByQuery = (GroupByQuery) queryPlus.getQuery(); - if (strategySelector.strategize(groupByQuery).doMergeResults(groupByQuery)) { - return initAndMergeGroupByResults(groupByQuery, runner, responseContext); - } - return runner.run(queryPlus, responseContext); + return initAndMergeGroupByResults(groupByQuery, runner, responseContext); }; } @Override public BinaryOperator createMergeFn(Query query) { - return strategySelector.strategize((GroupByQuery) query).createMergeFn(query); + return groupingEngine.createMergeFn(query); } @Override public Comparator createResultComparator(Query query) { - return strategySelector.strategize((GroupByQuery) query).createResultComparator(query); + return groupingEngine.createResultComparator(query); } private Sequence initAndMergeGroupByResults( @@ -158,11 +150,9 @@ private Sequence initAndMergeGroupByResults( ResponseContext context ) { - final GroupByStrategy groupByStrategy = strategySelector.strategize(query); - final GroupByQueryResource resource = groupByStrategy.prepareResource(query); + final GroupByQueryResources resource = groupingEngine.prepareResource(query); try { final Sequence mergedSequence = mergeGroupByResults( - groupByStrategy, query, resource, runner, @@ -179,23 +169,21 @@ private Sequence initAndMergeGroupByResults( } private Sequence mergeGroupByResults( - GroupByStrategy groupByStrategy, final GroupByQuery query, - GroupByQueryResource resource, + GroupByQueryResources resource, QueryRunner runner, ResponseContext context ) { - if (isNestedQueryPushDown(query, groupByStrategy)) { - return mergeResultsWithNestedQueryPushDown(groupByStrategy, query, resource, runner, context); + if (isNestedQueryPushDown(query)) { + return mergeResultsWithNestedQueryPushDown(query, resource, runner, context); } - return mergeGroupByResultsWithoutPushDown(groupByStrategy, query, resource, runner, context); + return mergeGroupByResultsWithoutPushDown(query, resource, runner, context); } private Sequence mergeGroupByResultsWithoutPushDown( - GroupByStrategy groupByStrategy, GroupByQuery query, - GroupByQueryResource resource, + GroupByQueryResources resource, QueryRunner runner, ResponseContext context ) @@ -228,15 +216,7 @@ private Sequence mergeGroupByResultsWithoutPushDown( } final Sequence subqueryResult = mergeGroupByResults( - groupByStrategy, - subquery.withOverriddenContext( - ImmutableMap.of( - //setting sort to false avoids unnecessary sorting while merging results. we only need to sort - //in the end when returning results to user. (note this is only respected by groupBy v1) - GroupByQueryHelper.CTX_KEY_SORT_RESULTS, - false - ) - ), + subquery, resource, runner, context @@ -245,52 +225,57 @@ private Sequence mergeGroupByResultsWithoutPushDown( final Sequence finalizingResults = finalizeSubqueryResults(subqueryResult, subquery); if (query.getSubtotalsSpec() != null) { - return groupByStrategy.processSubtotalsSpec( + return groupingEngine.processSubtotalsSpec( query, resource, - groupByStrategy.processSubqueryResult(subquery, query, resource, finalizingResults, false) + groupingEngine.processSubqueryResult(subquery, query, resource, finalizingResults, false) ); } else { - return groupByStrategy.applyPostProcessing(groupByStrategy.processSubqueryResult( - subquery, - query, - resource, - finalizingResults, - false - ), query); + return groupingEngine.applyPostProcessing( + groupingEngine.processSubqueryResult( + subquery, + query, + resource, + finalizingResults, + false + ), + query + ); } } else { if (query.getSubtotalsSpec() != null) { - return groupByStrategy.processSubtotalsSpec( + return groupingEngine.processSubtotalsSpec( query, resource, - groupByStrategy.mergeResults(runner, query.withSubtotalsSpec(null), context) + groupingEngine.mergeResults(runner, query.withSubtotalsSpec(null), context) ); } else { - return groupByStrategy.applyPostProcessing(groupByStrategy.mergeResults(runner, query, context), query); + return groupingEngine.applyPostProcessing(groupingEngine.mergeResults(runner, query, context), query); } } } private Sequence mergeResultsWithNestedQueryPushDown( - GroupByStrategy groupByStrategy, GroupByQuery query, - GroupByQueryResource resource, + GroupByQueryResources resource, QueryRunner runner, ResponseContext context ) { - Sequence pushDownQueryResults = groupByStrategy.mergeResults(runner, query, context); + Sequence pushDownQueryResults = groupingEngine.mergeResults(runner, query, context); final Sequence finalizedResults = finalizeSubqueryResults(pushDownQueryResults, query); GroupByQuery rewrittenQuery = rewriteNestedQueryForPushDown(query); - return groupByStrategy.applyPostProcessing(groupByStrategy.processSubqueryResult( - query, - rewrittenQuery, - resource, - finalizedResults, - true - ), query); + return groupingEngine.applyPostProcessing( + groupingEngine.processSubqueryResult( + query, + rewrittenQuery, + resource, + finalizedResults, + true + ), + query + ); } /** @@ -328,12 +313,11 @@ private Sequence finalizeSubqueryResults(Sequence subquery return finalizingResults; } - public static boolean isNestedQueryPushDown(GroupByQuery q, GroupByStrategy strategy) + public static boolean isNestedQueryPushDown(GroupByQuery q) { return q.getDataSource() instanceof QueryDataSource && q.context().getBoolean(GroupByQueryConfig.CTX_KEY_FORCE_PUSH_DOWN_NESTED_QUERY, false) - && q.getSubtotalsSpec() == null - && strategy.supportsNestedQueryPushDown(); + && q.getSubtotalsSpec() == null; } @Override @@ -528,7 +512,9 @@ public CacheStrategy getCacheStrategy(final Gro @Override public boolean isCacheable(GroupByQuery query, boolean willMergeRunners, boolean bySegment) { - return strategySelector.strategize(query).isCacheable(willMergeRunners, bySegment); + //disable segment-level cache on borker, + //see PR https://github.com/apache/druid/issues/3820 + return willMergeRunners || !bySegment; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryResources.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryResources.java new file mode 100644 index 000000000000..4a2a8bdf8aa1 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryResources.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.query.groupby; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.druid.collections.ReferenceCountingResourceHolder; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.java.util.common.collect.Utils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryDataSource; +import org.apache.druid.query.dimension.DimensionSpec; + +import javax.annotation.Nullable; +import java.io.Closeable; +import java.nio.ByteBuffer; +import java.util.ArrayDeque; +import java.util.Deque; +import java.util.List; +import java.util.stream.Collectors; + +/** + * This class contains resources required for a groupBy query execution. + * Currently, it contains only merge buffers, but any additional resources can be added in the future. + */ +public class GroupByQueryResources implements Closeable +{ + private static final Logger log = new Logger(GroupByQueryResources.class); + + private static final int MAX_MERGE_BUFFER_NUM_WITHOUT_SUBTOTAL = 2; + + private static int countRequiredMergeBufferNumWithoutSubtotal(Query query, int foundNum) + { + // Note: A broker requires merge buffers for processing the groupBy layers beyond the inner-most one. + // For example, the number of required merge buffers for a nested groupBy (groupBy -> groupBy -> table) is 1. + // If the broker processes an outer groupBy which reads input from an inner groupBy, + // it requires two merge buffers for inner and outer groupBys to keep the intermediate result of inner groupBy + // until the outer groupBy processing completes. + // This is same for subsequent groupBy layers, and thus the maximum number of required merge buffers becomes 2. + + final DataSource dataSource = query.getDataSource(); + if (foundNum == MAX_MERGE_BUFFER_NUM_WITHOUT_SUBTOTAL + 1 || !(dataSource instanceof QueryDataSource)) { + return foundNum - 1; + } else { + return countRequiredMergeBufferNumWithoutSubtotal(((QueryDataSource) dataSource).getQuery(), foundNum + 1); + } + } + + private static int numMergeBuffersNeededForSubtotalsSpec(GroupByQuery query) + { + List> subtotalSpecs = query.getSubtotalsSpec(); + final DataSource dataSource = query.getDataSource(); + int numMergeBuffersNeededForSubQuerySubtotal = 0; + if (dataSource instanceof QueryDataSource) { + Query subQuery = ((QueryDataSource) dataSource).getQuery(); + if (subQuery instanceof GroupByQuery) { + numMergeBuffersNeededForSubQuerySubtotal = numMergeBuffersNeededForSubtotalsSpec((GroupByQuery) subQuery); + } + + } + if (subtotalSpecs == null || subtotalSpecs.size() == 0) { + return numMergeBuffersNeededForSubQuerySubtotal; + } + + List queryDimOutputNames = query.getDimensions().stream().map(DimensionSpec::getOutputName).collect( + Collectors.toList()); + for (List subtotalSpec : subtotalSpecs) { + if (!Utils.isPrefix(subtotalSpec, queryDimOutputNames)) { + return 2; + } + } + + return Math.max(1, numMergeBuffersNeededForSubQuerySubtotal); + } + + @VisibleForTesting + public static int countRequiredMergeBufferNum(GroupByQuery query) + { + return countRequiredMergeBufferNumWithoutSubtotal(query, 1) + numMergeBuffersNeededForSubtotalsSpec(query); + } + + @Nullable + private final List> mergeBufferHolders; + private final Deque mergeBuffers; + + public GroupByQueryResources() + { + this.mergeBufferHolders = null; + this.mergeBuffers = new ArrayDeque<>(); + } + + public GroupByQueryResources(List> mergeBufferHolders) + { + this.mergeBufferHolders = mergeBufferHolders; + this.mergeBuffers = new ArrayDeque<>(mergeBufferHolders.size()); + mergeBufferHolders.forEach(holder -> mergeBuffers.add(holder.get())); + } + + /** + * Get a merge buffer from the pre-acquired resources. + * + * @return a resource holder containing a merge buffer + * + * @throws IllegalStateException if this resource is initialized with empty merge buffers, or + * there isn't any available merge buffers + */ + public ResourceHolder getMergeBuffer() + { + final ByteBuffer buffer = mergeBuffers.pop(); + return new ResourceHolder() + { + @Override + public ByteBuffer get() + { + return buffer; + } + + @Override + public void close() + { + mergeBuffers.add(buffer); + } + }; + } + + @Override + public void close() + { + if (mergeBufferHolders != null) { + if (mergeBuffers.size() != mergeBufferHolders.size()) { + log.warn("%d resources are not returned yet", mergeBufferHolders.size() - mergeBuffers.size()); + } + mergeBufferHolders.forEach(ReferenceCountingResourceHolder::close); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java index 6553f4c4eec4..f68f5a6bad7e 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java @@ -30,7 +30,6 @@ import org.apache.druid.query.QueryRunnerFactory; import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.context.ResponseContext; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; @@ -39,23 +38,23 @@ */ public class GroupByQueryRunnerFactory implements QueryRunnerFactory { - private final GroupByStrategySelector strategySelector; + private final GroupingEngine groupingEngine; private final GroupByQueryQueryToolChest toolChest; @Inject public GroupByQueryRunnerFactory( - GroupByStrategySelector strategySelector, + GroupingEngine groupingEngine, GroupByQueryQueryToolChest toolChest ) { - this.strategySelector = strategySelector; + this.groupingEngine = groupingEngine; this.toolChest = toolChest; } @Override public QueryRunner createRunner(final Segment segment) { - return new GroupByQueryRunner(segment, strategySelector); + return new GroupByQueryRunner(segment, groupingEngine); } @Override @@ -69,9 +68,7 @@ public QueryRunner mergeRunners( @Override public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { - QueryRunner rowQueryRunner = strategySelector - .strategize((GroupByQuery) queryPlus.getQuery()) - .mergeRunners(queryProcessingPool, queryRunners); + QueryRunner rowQueryRunner = groupingEngine.mergeRunners(queryProcessingPool, queryRunners); return rowQueryRunner.run(queryPlus, responseContext); } }; @@ -86,12 +83,12 @@ public QueryToolChest getToolchest() private static class GroupByQueryRunner implements QueryRunner { private final StorageAdapter adapter; - private final GroupByStrategySelector strategySelector; + private final GroupingEngine groupingEngine; - public GroupByQueryRunner(Segment segment, final GroupByStrategySelector strategySelector) + public GroupByQueryRunner(Segment segment, final GroupingEngine groupingEngine) { this.adapter = segment.asStorageAdapter(); - this.strategySelector = strategySelector; + this.groupingEngine = groupingEngine; } @Override @@ -102,15 +99,13 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r throw new ISE("Got a [%s] which isn't a %s", query.getClass(), GroupByQuery.class); } - return strategySelector - .strategize((GroupByQuery) query) - .process((GroupByQuery) query, adapter, (GroupByQueryMetrics) queryPlus.getQueryMetrics()); + return groupingEngine.process((GroupByQuery) query, adapter, (GroupByQueryMetrics) queryPlus.getQueryMetrics()); } } @VisibleForTesting - public GroupByStrategySelector getStrategySelector() + public GroupingEngine getGroupingEngine() { - return strategySelector; + return groupingEngine; } } 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/GroupingEngine.java similarity index 84% rename from processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java rename to processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java index df599952845d..112f6ea25ed3 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java @@ -17,10 +17,9 @@ * under the License. */ -package org.apache.druid.query.groupby.strategy; +package org.apache.druid.query.groupby; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; @@ -41,13 +40,11 @@ import org.apache.druid.java.util.common.guava.LazySequence; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.query.DataSource; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.Query; import org.apache.druid.query.QueryCapacityExceededException; import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryContexts; -import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryProcessingPool; import org.apache.druid.query.QueryRunner; @@ -59,10 +56,6 @@ import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.dimension.DefaultDimensionSpec; 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.GroupByQueryMetrics; -import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.epinephelinae.GroupByBinaryFnV2; import org.apache.druid.query.groupby.epinephelinae.GroupByMergingQueryRunnerV2; import org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngineV2; @@ -70,7 +63,6 @@ import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.LimitSpec; import org.apache.druid.query.groupby.orderby.NoopLimitSpec; -import org.apache.druid.query.groupby.resource.GroupByQueryResource; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; @@ -87,14 +79,11 @@ import java.util.function.BinaryOperator; import java.util.stream.Collectors; -public class GroupByStrategyV2 implements GroupByStrategy +public class GroupingEngine { public static final String CTX_KEY_FUDGE_TIMESTAMP = "fudgeTimestamp"; public static final String CTX_KEY_OUTERMOST = "groupByOutermost"; - // see countRequiredMergeBufferNumWithoutSubtotal() for explanation - private static final int MAX_MERGE_BUFFER_NUM_WITHOUT_SUBTOTAL = 2; - private final DruidProcessingConfig processingConfig; private final Supplier configSupplier; private final NonBlockingPool bufferPool; @@ -104,7 +93,7 @@ public class GroupByStrategyV2 implements GroupByStrategy private final QueryWatcher queryWatcher; @Inject - public GroupByStrategyV2( + public GroupingEngine( DruidProcessingConfig processingConfig, Supplier configSupplier, @Global NonBlockingPool bufferPool, @@ -123,10 +112,19 @@ public GroupByStrategyV2( this.queryWatcher = queryWatcher; } - @Override - public GroupByQueryResource prepareResource(GroupByQuery query) + /** + * Initializes resources required to run {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)} for a + * particular query. That method is also the primary caller of this method. + * + * Used by {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)}. + * + * @param query a groupBy query to be processed + * + * @return broker resource + */ + public GroupByQueryResources prepareResource(GroupByQuery query) { - final int requiredMergeBufferNum = countRequiredMergeBufferNum(query); + final int requiredMergeBufferNum = GroupByQueryResources.countRequiredMergeBufferNum(query); if (requiredMergeBufferNum > mergeBufferPool.maxSize()) { throw new ResourceLimitExceededException( @@ -134,7 +132,7 @@ public GroupByQueryResource prepareResource(GroupByQuery query) + mergeBufferPool.maxSize() + " merge buffers were configured" ); } else if (requiredMergeBufferNum == 0) { - return new GroupByQueryResource(); + return new GroupByQueryResources(); } else { final List> mergeBufferHolders; final QueryContext context = query.context(); @@ -151,61 +149,49 @@ public GroupByQueryResource prepareResource(GroupByQuery query) ) ); } else { - return new GroupByQueryResource(mergeBufferHolders); + return new GroupByQueryResources(mergeBufferHolders); } } } - @VisibleForTesting - public static int countRequiredMergeBufferNum(GroupByQuery query) - { - return countRequiredMergeBufferNumWithoutSubtotal(query, 1) + numMergeBuffersNeededForSubtotalsSpec(query); - } - - private static int countRequiredMergeBufferNumWithoutSubtotal(Query query, int foundNum) - { - // Note: A broker requires merge buffers for processing the groupBy layers beyond the inner-most one. - // For example, the number of required merge buffers for a nested groupBy (groupBy -> groupBy -> table) is 1. - // If the broker processes an outer groupBy which reads input from an inner groupBy, - // it requires two merge buffers for inner and outer groupBys to keep the intermediate result of inner groupBy - // until the outer groupBy processing completes. - // This is same for subsequent groupBy layers, and thus the maximum number of required merge buffers becomes 2. - - final DataSource dataSource = query.getDataSource(); - if (foundNum == MAX_MERGE_BUFFER_NUM_WITHOUT_SUBTOTAL + 1 || !(dataSource instanceof QueryDataSource)) { - return foundNum - 1; - } else { - return countRequiredMergeBufferNumWithoutSubtotal(((QueryDataSource) dataSource).getQuery(), foundNum + 1); - } - } - - @Override - public boolean isCacheable(boolean willMergeRunners, boolean bySegment) - { - //disable segment-level cache on borker, - //see PR https://github.com/apache/druid/issues/3820 - return willMergeRunners || !bySegment; - } - - @Override - public boolean doMergeResults(final GroupByQuery query) - { - return true; - } - - @Override + /** + * See {@link org.apache.druid.query.QueryToolChest#createResultComparator(Query)}, allows + * {@link GroupByQueryQueryToolChest} to delegate implementation to the strategy + */ public Comparator createResultComparator(Query queryParam) { return ((GroupByQuery) queryParam).getRowOrdering(true); } - @Override + /** + * See {@link org.apache.druid.query.QueryToolChest#createMergeFn(Query)} for details, allows + * {@link GroupByQueryQueryToolChest} to delegate implementation to the strategy + */ public BinaryOperator createMergeFn(Query queryParam) { return new GroupByBinaryFnV2((GroupByQuery) queryParam); } - @Override + /** + * 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 + * rows with the same timestamp and dimensions) and without PostAggregators computed. This method will fully merge + * the rows, apply PostAggregators, and return the resulting {@link Sequence}. + * + * The query will be modified before passing it down to the base runner. For example, "having" clauses will be + * removed and various context parameters will be adjusted. + * + * Despite the similar name, this method is much reduced in scope compared to + * {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)}. That method does delegate to this one at some points, + * but has a truckload of other responsibility, including computing outer query results (if there are subqueries), + * computing subtotals (like GROUPING SETS), and computing the havingSpec and limitSpec. + * + * @param baseRunner base query runner + * @param query the groupBy query to run inside the base query runner + * @param responseContext the response context to pass to the base query runner + * + * @return merged result sequence + */ public Sequence mergeResults( final QueryRunner baseRunner, final GroupByQuery query, @@ -223,7 +209,6 @@ public Sequence mergeResults( // Set up downstream context. final ImmutableMap.Builder context = ImmutableMap.builder(); context.put(QueryContexts.FINALIZE_KEY, false); - context.put(GroupByQueryConfig.CTX_KEY_STRATEGY, GroupByStrategySelector.STRATEGY_V2); context.put(CTX_KEY_OUTERMOST, false); Granularity granularity = query.getGranularity(); @@ -382,35 +367,73 @@ public Sequence mergeResults( } } - private void moveOrReplicateTimestampInRow( + /** + * Merge a variety of single-segment query runners into a combined runner. Used by + * {@link GroupByQueryRunnerFactory#mergeRunners(QueryProcessingPool, Iterable)}. In + * that sense, it is intended to go along with {@link #process(GroupByQuery, StorageAdapter, GroupByQueryMetrics)} (the runners created + * by that method will be fed into this method). + *

+ * This method is only called on data servers, like Historicals (not the Broker). + * + * @param queryProcessingPool {@link QueryProcessingPool} service used for parallel execution of the query runners + * @param queryRunners collection of query runners to merge + * @return merged query runner + */ + public QueryRunner mergeRunners( + final QueryProcessingPool queryProcessingPool, + final Iterable> queryRunners + ) + { + return new GroupByMergingQueryRunnerV2( + configSupplier.get(), + processingConfig, + queryProcessingPool, + queryWatcher, + queryRunners, + processingConfig.getNumThreads(), + mergeBufferPool, + processingConfig.intermediateComputeSizeBytes(), + spillMapper, + processingConfig.getTmpDir() + ); + } + + /** + * Process a groupBy query on a single {@link StorageAdapter}. This is used by + * {@link GroupByQueryRunnerFactory#createRunner} to create per-segment + * QueryRunners. + * + * This method is only called on data servers, like Historicals (not the Broker). + * + * @param query the groupBy query + * @param storageAdapter storage adatper for the segment in question + * + * @return result sequence for the storage adapter + */ + public Sequence process( GroupByQuery query, - int timestampResultFieldIndexInOriginalDimensions, - ResultRow before, - ResultRow after + StorageAdapter storageAdapter, + @Nullable GroupByQueryMetrics groupByQueryMetrics ) { - // d1 is the __time - // when query.granularity=all: convert [__time, d0] to [d0, d1] (actually, [d0, __time]) - // when query.granularity!=all: convert [__time, d0] to [__time, d0, d1] (actually, [__time, d0, __time]) - // overall, insert the removed d1 at the position where it is removed and remove the first __time if granularity=all - Object theTimestamp = before.get(0); - int expectedDimensionStartInAfterRow = 0; - if (query.getResultRowHasTimestamp()) { - expectedDimensionStartInAfterRow = 1; - after.set(0, theTimestamp); - } - int timestampResultFieldIndexInAfterRow = timestampResultFieldIndexInOriginalDimensions + expectedDimensionStartInAfterRow; - for (int i = expectedDimensionStartInAfterRow; i < timestampResultFieldIndexInAfterRow; i++) { - // 0 in beforeRow is the timestamp, so plus 1 is the start of dimension in beforeRow - after.set(i, before.get(i + 1)); - } - after.set(timestampResultFieldIndexInAfterRow, theTimestamp); - for (int i = timestampResultFieldIndexInAfterRow + 1; i < before.length() + expectedDimensionStartInAfterRow; i++) { - after.set(i, before.get(i - expectedDimensionStartInAfterRow)); - } + return GroupByQueryEngineV2.process( + query, + storageAdapter, + bufferPool, + configSupplier.get().withOverrides(query), + processingConfig, + groupByQueryMetrics + ); } - @Override + /** + * Apply the {@link GroupByQuery} "postProcessingFn", which is responsible for HavingSpec and LimitSpec. + * + * @param results sequence of results + * @param query the groupBy query + * + * @return post-processed results, with HavingSpec and LimitSpec applied + */ public Sequence applyPostProcessing(Sequence results, GroupByQuery query) { // Don't apply limit here for inner results, that will be pushed down to the BufferHashGrouper @@ -421,11 +444,22 @@ public Sequence applyPostProcessing(Sequence results, Grou } } - @Override + /** + * Called by {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)} when it needs to process a subquery. + * + * @param subquery inner query + * @param query outer query + * @param resource resources returned by {@link #prepareResource(GroupByQuery)} + * @param subqueryResult result rows from the subquery + * @param wasQueryPushedDown true if the outer query was pushed down (so we only need to merge the outer query's + * results, not run it from scratch like a normal outer query) + * + * @return results of the outer query + */ public Sequence processSubqueryResult( GroupByQuery subquery, GroupByQuery query, - GroupByQueryResource resource, + GroupByQueryResources resource, Sequence subqueryResult, boolean wasQueryPushedDown ) @@ -462,7 +496,7 @@ public Sequence processSubqueryResult( mergeResults( (queryPlus, responseContext) -> finalResultSupplier.results(null), query, - null + ResponseContext.createEmpty() ), finalResultSupplier ); @@ -472,10 +506,18 @@ public Sequence processSubqueryResult( } } - @Override + /** + * Called by {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)} when it needs to generate subtotals. + * + * @param query query that has a "subtotalsSpec" + * @param resource resources returned by {@link #prepareResource(GroupByQuery)} + * @param queryResult result rows from the main query + * + * @return results for each list of subtotals in the query, concatenated together + */ public Sequence processSubtotalsSpec( GroupByQuery query, - GroupByQueryResource resource, + GroupByQueryResources resource, Sequence queryResult ) { @@ -631,7 +673,7 @@ private Sequence processSubtotalsResultAndOptionallyClose( ) ), subtotalQuery, - null + ResponseContext.createEmpty() ); } catch (Throwable e) { @@ -639,6 +681,34 @@ private Sequence processSubtotalsResultAndOptionallyClose( } } + private void moveOrReplicateTimestampInRow( + GroupByQuery query, + int timestampResultFieldIndexInOriginalDimensions, + ResultRow before, + ResultRow after + ) + { + // d1 is the __time + // when query.granularity=all: convert [__time, d0] to [d0, d1] (actually, [d0, __time]) + // when query.granularity!=all: convert [__time, d0] to [__time, d0, d1] (actually, [__time, d0, __time]) + // overall, insert the removed d1 at the position where it is removed and remove the first __time if granularity=all + Object theTimestamp = before.get(0); + int expectedDimensionStartInAfterRow = 0; + if (query.getResultRowHasTimestamp()) { + expectedDimensionStartInAfterRow = 1; + after.set(0, theTimestamp); + } + int timestampResultFieldIndexInAfterRow = timestampResultFieldIndexInOriginalDimensions + expectedDimensionStartInAfterRow; + for (int i = expectedDimensionStartInAfterRow; i < timestampResultFieldIndexInAfterRow; i++) { + // 0 in beforeRow is the timestamp, so plus 1 is the start of dimension in beforeRow + after.set(i, before.get(i + 1)); + } + after.set(timestampResultFieldIndexInAfterRow, theTimestamp); + for (int i = timestampResultFieldIndexInAfterRow + 1; i < before.length() + expectedDimensionStartInAfterRow; i++) { + after.set(i, before.get(i - expectedDimensionStartInAfterRow)); + } + } + private Set getAggregatorAndPostAggregatorNames(GroupByQuery query) { Set aggsAndPostAggs = new HashSet(); @@ -656,74 +726,4 @@ private Set getAggregatorAndPostAggregatorNames(GroupByQuery query) return aggsAndPostAggs; } - - private static int numMergeBuffersNeededForSubtotalsSpec(GroupByQuery query) - { - List> subtotalSpecs = query.getSubtotalsSpec(); - final DataSource dataSource = query.getDataSource(); - int numMergeBuffersNeededForSubQuerySubtotal = 0; - if (dataSource instanceof QueryDataSource) { - Query subQuery = ((QueryDataSource) dataSource).getQuery(); - if (subQuery instanceof GroupByQuery) { - numMergeBuffersNeededForSubQuerySubtotal = numMergeBuffersNeededForSubtotalsSpec((GroupByQuery) subQuery); - } - - } - if (subtotalSpecs == null || subtotalSpecs.size() == 0) { - return numMergeBuffersNeededForSubQuerySubtotal; - } - - List queryDimOutputNames = query.getDimensions().stream().map(DimensionSpec::getOutputName).collect( - Collectors.toList()); - for (List subtotalSpec : subtotalSpecs) { - if (!Utils.isPrefix(subtotalSpec, queryDimOutputNames)) { - return 2; - } - } - - return Math.max(1, numMergeBuffersNeededForSubQuerySubtotal); - } - - @Override - public QueryRunner mergeRunners( - final QueryProcessingPool queryProcessingPool, - final Iterable> queryRunners - ) - { - return new GroupByMergingQueryRunnerV2( - configSupplier.get(), - processingConfig, - queryProcessingPool, - queryWatcher, - queryRunners, - processingConfig.getNumThreads(), - mergeBufferPool, - processingConfig.intermediateComputeSizeBytes(), - spillMapper, - processingConfig.getTmpDir() - ); - } - - @Override - public Sequence process( - GroupByQuery query, - StorageAdapter storageAdapter, - @Nullable GroupByQueryMetrics groupByQueryMetrics - ) - { - return GroupByQueryEngineV2.process( - query, - storageAdapter, - bufferPool, - configSupplier.get().withOverrides(query), - processingConfig, - groupByQueryMetrics - ); - } - - @Override - public boolean supportsNestedQueryPushDown() - { - return true; - } } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByBinaryFnV2.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByBinaryFnV2.java index 74cae7a618ff..bf6ef5d189f8 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByBinaryFnV2.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByBinaryFnV2.java @@ -34,7 +34,7 @@ * and dimensions. This code runs on Brokers as well as data servers, like Historicals. * * Used by - * {@link org.apache.druid.query.groupby.strategy.GroupByStrategyV2#mergeResults}. + * {@link org.apache.druid.query.groupby.GroupingEngine#mergeResults}. */ public class GroupByBinaryFnV2 implements BinaryOperator { diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java index 1ad44bc84eb2..fbffbf311fda 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java @@ -80,7 +80,7 @@ * similarities and differences. * * Used by - * {@link org.apache.druid.query.groupby.strategy.GroupByStrategyV2#mergeRunners(QueryProcessingPool, Iterable)} + * {@link org.apache.druid.query.groupby.GroupingEngine#mergeRunners(QueryProcessingPool, Iterable)} */ public class GroupByMergingQueryRunnerV2 implements QueryRunner { diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java index d5141ff415ab..445cb0708da2 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java @@ -42,6 +42,7 @@ import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryMetrics; +import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.epinephelinae.column.ArrayDoubleGroupByColumnSelectorStrategy; import org.apache.druid.query.groupby.epinephelinae.column.ArrayLongGroupByColumnSelectorStrategy; @@ -57,7 +58,6 @@ import org.apache.druid.query.groupby.epinephelinae.vector.VectorGroupByEngine; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV2; import org.apache.druid.query.ordering.StringComparator; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; @@ -76,7 +76,6 @@ import org.joda.time.Interval; import javax.annotation.Nullable; - import java.io.Closeable; import java.nio.ByteBuffer; import java.util.Iterator; @@ -92,7 +91,7 @@ * This code runs on data servers, like Historicals. * * Used by - * {@link GroupByStrategyV2#process(GroupByQuery, StorageAdapter, GroupByQueryMetrics)}. + * {@link GroupingEngine#process(GroupByQuery, StorageAdapter, GroupByQueryMetrics)}. */ public class GroupByQueryEngineV2 { @@ -141,7 +140,7 @@ public static Sequence process( try { final String fudgeTimestampString = NullHandling.emptyToNullIfNeeded( - query.context().getString(GroupByStrategyV2.CTX_KEY_FUDGE_TIMESTAMP) + query.context().getString(GroupingEngine.CTX_KEY_FUDGE_TIMESTAMP) ); final DateTime fudgeTimestamp = fudgeTimestampString == null diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByRowProcessor.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByRowProcessor.java index 76125c2c504b..3f4ce0717650 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByRowProcessor.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByRowProcessor.java @@ -33,9 +33,9 @@ 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.GroupByQueryResources; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.epinephelinae.RowBasedGrouperHelper.RowBasedKey; -import org.apache.druid.query.groupby.resource.GroupByQueryResource; import javax.annotation.Nullable; import java.io.Closeable; @@ -47,9 +47,9 @@ /** * Utility class that knows how to do higher-level groupBys: i.e. group a {@link Sequence} of {@link ResultRow} - * originating from a subquery. It uses a buffer provided by a {@link GroupByQueryResource}. The output rows may not + * originating from a subquery. It uses a buffer provided by a {@link GroupByQueryResources}. The output rows may not * be perfectly grouped and will not have PostAggregators applied, so they should be fed into - * {@link org.apache.druid.query.groupby.strategy.GroupByStrategy#mergeResults}. + * {@link org.apache.druid.query.groupby.GroupingEngine#mergeResults}. * * This class has two primary uses: processing nested groupBys, and processing subtotals. * @@ -90,7 +90,7 @@ public static ResultSupplier process( final Sequence rows, final GroupByQueryConfig config, final DruidProcessingConfig processingConfig, - final GroupByQueryResource resource, + final GroupByQueryResources resource, final ObjectMapper spillMapper, final String processingTmpDir, final int mergeBufferSize diff --git a/processing/src/main/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpec.java b/processing/src/main/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpec.java index b49fa0d2a8cc..e4071b5501a9 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpec.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpec.java @@ -280,12 +280,6 @@ public Function, Sequence> build(final GroupByQue } } - @Override - public LimitSpec merge(LimitSpec other) - { - return this; - } - private ColumnType getOrderByType(final OrderByColumnSpec columnSpec, final List dimensions) { for (DimensionSpec dimSpec : dimensions) { diff --git a/processing/src/main/java/org/apache/druid/query/groupby/orderby/LimitSpec.java b/processing/src/main/java/org/apache/druid/query/groupby/orderby/LimitSpec.java index fbdd0f1d38fc..834c224c02a0 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/orderby/LimitSpec.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/orderby/LimitSpec.java @@ -53,8 +53,6 @@ static LimitSpec nullToNoopLimitSpec(@Nullable LimitSpec limitSpec) */ Function, Sequence> build(GroupByQuery query); - LimitSpec merge(LimitSpec other); - /** * Discard sorting columns not contained in given set. This is used when generating new queries, e.g. to process * subtotal spec in GroupBy query. diff --git a/processing/src/main/java/org/apache/druid/query/groupby/orderby/NoopLimitSpec.java b/processing/src/main/java/org/apache/druid/query/groupby/orderby/NoopLimitSpec.java index 84f4cfaee013..cc5de903a539 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/orderby/NoopLimitSpec.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/orderby/NoopLimitSpec.java @@ -53,12 +53,6 @@ public Function, Sequence> build(GroupByQuery que return Functions.identity(); } - @Override - public LimitSpec merge(LimitSpec other) - { - return other; - } - @Override public LimitSpec filterColumns(Set names) { diff --git a/processing/src/main/java/org/apache/druid/query/groupby/resource/GroupByQueryResource.java b/processing/src/main/java/org/apache/druid/query/groupby/resource/GroupByQueryResource.java deleted file mode 100644 index 89ba61bfb46f..000000000000 --- a/processing/src/main/java/org/apache/druid/query/groupby/resource/GroupByQueryResource.java +++ /dev/null @@ -1,93 +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.groupby.resource; - -import org.apache.druid.collections.ReferenceCountingResourceHolder; -import org.apache.druid.collections.ResourceHolder; -import org.apache.druid.java.util.common.logger.Logger; - -import java.io.Closeable; -import java.nio.ByteBuffer; -import java.util.ArrayDeque; -import java.util.Deque; -import java.util.List; - -/** - * This class contains resources required for a groupBy query execution. - * Currently, it contains only merge buffers, but any additional resources can be added in the future. - */ -public class GroupByQueryResource implements Closeable -{ - private static final Logger log = new Logger(GroupByQueryResource.class); - - private final List> mergeBufferHolders; - private final Deque mergeBuffers; - - public GroupByQueryResource() - { - this.mergeBufferHolders = null; - this.mergeBuffers = new ArrayDeque<>(); - } - - public GroupByQueryResource(List> mergeBufferHolders) - { - this.mergeBufferHolders = mergeBufferHolders; - this.mergeBuffers = new ArrayDeque<>(mergeBufferHolders.size()); - mergeBufferHolders.forEach(holder -> mergeBuffers.add(holder.get())); - } - - /** - * Get a merge buffer from the pre-acquired resources. - * - * @return a resource holder containing a merge buffer - * - * @throws IllegalStateException if this resource is initialized with empty merge buffers, or - * there isn't any available merge buffers - */ - public ResourceHolder getMergeBuffer() - { - final ByteBuffer buffer = mergeBuffers.pop(); - return new ResourceHolder() - { - @Override - public ByteBuffer get() - { - return buffer; - } - - @Override - public void close() - { - mergeBuffers.add(buffer); - } - }; - } - - @Override - public void close() - { - if (mergeBufferHolders != null) { - if (mergeBuffers.size() != mergeBufferHolders.size()) { - log.warn("%d resources are not returned yet", mergeBufferHolders.size() - mergeBuffers.size()); - } - mergeBufferHolders.forEach(ReferenceCountingResourceHolder::close); - } - } -} 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 deleted file mode 100644 index 230e824cab1e..000000000000 --- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java +++ /dev/null @@ -1,205 +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.groupby.strategy; - -import org.apache.druid.java.util.common.UOE; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.query.Query; -import org.apache.druid.query.QueryProcessingPool; -import org.apache.druid.query.QueryRunner; -import org.apache.druid.query.QueryRunnerFactory; -import org.apache.druid.query.context.ResponseContext; -import org.apache.druid.query.groupby.GroupByQuery; -import org.apache.druid.query.groupby.GroupByQueryMetrics; -import org.apache.druid.query.groupby.GroupByQueryQueryToolChest; -import org.apache.druid.query.groupby.ResultRow; -import org.apache.druid.query.groupby.resource.GroupByQueryResource; -import org.apache.druid.segment.StorageAdapter; - -import javax.annotation.Nullable; - -import java.util.Comparator; -import java.util.function.BinaryOperator; - -public interface GroupByStrategy -{ - /** - * Initializes resources required to run {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)} for a - * particular query. That method is also the primary caller of this method. - * - * Used by {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)}. - * - * @param query a groupBy query to be processed - * - * @return broker resource - */ - GroupByQueryResource prepareResource(GroupByQuery query); - - /** - * Indicates if results from this query are cacheable or not. - * - * Used by {@link GroupByQueryQueryToolChest#getCacheStrategy(GroupByQuery)}. - * - * @param willMergeRunners indicates that {@link QueryRunnerFactory#mergeRunners(QueryProcessingPool, Iterable)} will be - * called on the cached by-segment results. Can be used to distinguish if we are running on - * a broker or data node. - * @param bySegment segment level or result level cache - * - * @return true if this strategy is cacheable, otherwise false. - */ - boolean isCacheable(boolean willMergeRunners, boolean bySegment); - - /** - * Indicates if this query should undergo "mergeResults" or not. Checked by - * {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)}. - */ - boolean doMergeResults(GroupByQuery query); - - /** - * 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 - * rows with the same timestamp and dimensions) and without PostAggregators computed. This method will fully merge - * the rows, apply PostAggregators, and return the resulting {@link Sequence}. - * - * The query will be modified before passing it down to the base runner. For example, "having" clauses will be - * removed and various context parameters will be adjusted. - * - * Despite the similar name, this method is much reduced in scope compared to - * {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)}. That method does delegate to this one at some points, - * but has a truckload of other responsibility, including computing outer query results (if there are subqueries), - * computing subtotals (like GROUPING SETS), and computing the havingSpec and limitSpec. - * - * @param baseRunner base query runner - * @param query the groupBy query to run inside the base query runner - * @param responseContext the response context to pass to the base query runner - * - * @return merged result sequence - */ - Sequence mergeResults( - QueryRunner baseRunner, - GroupByQuery query, - ResponseContext responseContext - ); - - /** - * See {@link org.apache.druid.query.QueryToolChest#createMergeFn(Query)} for details, allows - * {@link GroupByQueryQueryToolChest} to delegate implementation to the strategy - */ - @Nullable - default BinaryOperator createMergeFn(Query query) - { - return null; - } - - /** - * See {@link org.apache.druid.query.QueryToolChest#createResultComparator(Query)}, allows - * {@link GroupByQueryQueryToolChest} to delegate implementation to the strategy - */ - @Nullable - default Comparator createResultComparator(Query queryParam) - { - throw new UOE("%s doesn't provide a result comparator", this.getClass().getName()); - } - - /** - * Apply the {@link GroupByQuery} "postProcessingFn", which is responsible for HavingSpec and LimitSpec. - * - * @param results sequence of results - * @param query the groupBy query - * - * @return post-processed results, with HavingSpec and LimitSpec applied - */ - Sequence applyPostProcessing(Sequence results, GroupByQuery query); - - /** - * Called by {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)} when it needs to process a subquery. - * - * @param subquery inner query - * @param query outer query - * @param resource resources returned by {@link #prepareResource(GroupByQuery)} - * @param subqueryResult result rows from the subquery - * @param wasQueryPushedDown true if the outer query was pushed down (so we only need to merge the outer query's - * results, not run it from scratch like a normal outer query) - * - * @return results of the outer query - */ - Sequence processSubqueryResult( - GroupByQuery subquery, - GroupByQuery query, - GroupByQueryResource resource, - Sequence subqueryResult, - boolean wasQueryPushedDown - ); - - /** - * Called by {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)} when it needs to generate subtotals. - * - * @param query query that has a "subtotalsSpec" - * @param resource resources returned by {@link #prepareResource(GroupByQuery)} - * @param queryResult result rows from the main query - * - * @return results for each list of subtotals in the query, concatenated together - */ - Sequence processSubtotalsSpec( - GroupByQuery query, - GroupByQueryResource resource, - Sequence queryResult - ); - - /** - * Merge a variety of single-segment query runners into a combined runner. Used by - * {@link org.apache.druid.query.groupby.GroupByQueryRunnerFactory#mergeRunners(QueryProcessingPool, Iterable)}. In - * that sense, it is intended to go along with {@link #process(GroupByQuery, StorageAdapter, GroupByQueryMetrics)} (the runners created - * by that method will be fed into this method). - *

- * This method is only called on data servers, like Historicals (not the Broker). - * - * @param queryProcessingPool {@link QueryProcessingPool} service used for parallel execution of the query runners - * @param queryRunners collection of query runners to merge - * @return merged query runner - */ - QueryRunner mergeRunners(QueryProcessingPool queryProcessingPool, Iterable> queryRunners); - - /** - * Process a groupBy query on a single {@link StorageAdapter}. This is used by - * {@link org.apache.druid.query.groupby.GroupByQueryRunnerFactory#createRunner} to create per-segment - * QueryRunners. - * - * This method is only called on data servers, like Historicals (not the Broker). - * - * @param query the groupBy query - * @param storageAdapter storage adatper for the segment in question - * - * @return result sequence for the storage adapter - */ - Sequence process( - GroupByQuery query, - StorageAdapter storageAdapter, - @Nullable GroupByQueryMetrics groupByQueryMetrics); - - /** - * Returns whether this strategy supports pushing down outer queries. This is used by - * {@link GroupByQueryQueryToolChest#mergeResults(QueryRunner)} when it decides whether or not to push down an - * outer query from the Broker to data servers, like Historicals. - * - * Can be removed when the "v1" groupBy strategy is removed. ("v1" returns false, and "v2" returns true.) - */ - boolean supportsNestedQueryPushDown(); -} diff --git a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategySelector.java b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategySelector.java deleted file mode 100644 index 6c22c7ff293b..000000000000 --- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategySelector.java +++ /dev/null @@ -1,70 +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.groupby.strategy; - -import com.google.common.base.Supplier; -import com.google.inject.Inject; -import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.query.groupby.GroupByQuery; -import org.apache.druid.query.groupby.GroupByQueryConfig; - -public class GroupByStrategySelector -{ - public static final String STRATEGY_V2 = "v2"; - public static final String STRATEGY_V1 = "v1"; - - private final GroupByQueryConfig config; - private final GroupByStrategyV1 strategyV1; - private final GroupByStrategyV2 strategyV2; - - @Inject - public GroupByStrategySelector( - Supplier configSupplier, - GroupByStrategyV1 strategyV1, - GroupByStrategyV2 strategyV2 - ) - { - this.config = configSupplier.get(); - this.strategyV1 = strategyV1; - this.strategyV2 = strategyV2; - } - - public GroupByStrategy strategize(GroupByQuery query) - { - final String strategyString = config.withOverrides(query).getDefaultStrategy(); - - switch (strategyString) { - case STRATEGY_V2: - return strategyV2; - - case STRATEGY_V1: - // Fail early if subtotals were asked from GroupBy V1 - if (query.getSubtotalsSpec() != null) { - throw new IAE("GroupBy Strategy [%s] does not support subtotalsSpec.", STRATEGY_V1); - } - - return strategyV1; - - default: - throw new ISE("No such strategy[%s]", strategyString); - } - } -} 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 deleted file mode 100644 index c82d29d16922..000000000000 --- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV1.java +++ /dev/null @@ -1,293 +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.groupby.strategy; - -import com.google.common.base.Function; -import com.google.common.base.Predicate; -import com.google.common.base.Supplier; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; -import com.google.inject.Inject; -import org.apache.druid.java.util.common.IAE; -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.QueryContexts; -import org.apache.druid.query.QueryPlus; -import org.apache.druid.query.QueryProcessingPool; -import org.apache.druid.query.QueryRunner; -import org.apache.druid.query.QueryWatcher; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.context.ResponseContext; -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.GroupByQueryEngine; -import org.apache.druid.query.groupby.GroupByQueryHelper; -import org.apache.druid.query.groupby.GroupByQueryMetrics; -import org.apache.druid.query.groupby.GroupByQueryQueryToolChest; -import org.apache.druid.query.groupby.ResultRow; -import org.apache.druid.query.groupby.orderby.NoopLimitSpec; -import org.apache.druid.query.groupby.resource.GroupByQueryResource; -import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; -import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.incremental.IncrementalIndex; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; -import org.joda.time.Interval; - -import javax.annotation.Nullable; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.Set; - -public class GroupByStrategyV1 implements GroupByStrategy -{ - private final Supplier configSupplier; - private final GroupByQueryEngine engine; - private final QueryWatcher queryWatcher; - - @Inject - public GroupByStrategyV1( - Supplier configSupplier, - GroupByQueryEngine engine, - QueryWatcher queryWatcher - ) - { - this.configSupplier = configSupplier; - this.engine = engine; - this.queryWatcher = queryWatcher; - } - - @Override - public GroupByQueryResource prepareResource(GroupByQuery query) - { - return new GroupByQueryResource(); - } - - @Override - public boolean isCacheable(boolean willMergeRunners, boolean bySegment) - { - return true; - } - - @Override - public boolean doMergeResults(final GroupByQuery query) - { - return query.context().getBoolean(GroupByQueryQueryToolChest.GROUP_BY_MERGE_KEY, true); - } - - @Override - public Sequence mergeResults( - final QueryRunner baseRunner, - final GroupByQuery query, - final ResponseContext responseContext - ) - { - final IncrementalIndex index = GroupByQueryHelper.makeIncrementalIndex( - query, - null, - configSupplier.get(), - baseRunner.run( - QueryPlus.wrap( - new GroupByQuery.Builder(query) - // Don't do post aggs until the end of this method. - .setPostAggregatorSpecs(ImmutableList.of()) - // Don't do "having" clause until the end of this method. - .setHavingSpec(null) - .setLimitSpec(NoopLimitSpec.instance()) - .overrideContext( - ImmutableMap.builder() - .put(GroupByQueryConfig.CTX_KEY_STRATEGY, GroupByStrategySelector.STRATEGY_V1) - .put(QueryContexts.FINALIZE_KEY, false) - - // Always request array result rows when passing the query down. - .put(GroupByQueryConfig.CTX_KEY_ARRAY_RESULT_ROWS, true) - - // Set sort to false avoids unnecessary sorting while merging results. we only need to sort - // in the end when returning results to user. (note this is only respected by groupBy v1) - .put(GroupByQueryHelper.CTX_KEY_SORT_RESULTS, false) - - // No merging needed at historicals because GroupByQueryRunnerFactory.mergeRunners(..) would - // return merged results. (note this is only respected by groupBy v1) - .put(GroupByQueryQueryToolChest.GROUP_BY_MERGE_KEY, false) - .build() - ) - .build() - ), - responseContext - ) - ); - - return Sequences.withBaggage(GroupByQueryHelper.postAggregate(query, index), index); - } - - @Override - public Sequence applyPostProcessing(Sequence results, GroupByQuery query) - { - return query.postProcess(results); - } - - @Override - public Sequence processSubqueryResult( - GroupByQuery subquery, - GroupByQuery query, - GroupByQueryResource resource, - Sequence subqueryResult, - boolean wasQueryPushedDown - ) - { - final Set aggs = new HashSet<>(); - - // Nested group-bys work by first running the inner query and then materializing the results in an incremental - // index which the outer query is then run against. To build the incremental index, we use the fieldNames from - // the aggregators for the outer query to define the column names so that the index will match the query. If - // there are multiple types of aggregators in the outer query referencing the same fieldName, we will try to build - // multiple columns of the same name using different aggregator types and will fail. Here, we permit multiple - // aggregators of the same type referencing the same fieldName (and skip creating identical columns for the - // subsequent ones) and return an error if the aggregator types are different. - final Set dimensionNames = new HashSet<>(); - for (DimensionSpec dimension : subquery.getDimensions()) { - dimensionNames.add(dimension.getOutputName()); - } - for (AggregatorFactory aggregatorFactory : query.getAggregatorSpecs()) { - for (final AggregatorFactory transferAgg : aggregatorFactory.getRequiredColumns()) { - if (dimensionNames.contains(transferAgg.getName())) { - // This transferAgg is already represented in the subquery's dimensions. Assume that the outer aggregator - // *probably* wants the dimension and just ignore it. This is a gross workaround for cases like having - // a cardinality aggregator in the outer query. It is necessary because what this block of code is trying to - // do is use aggregators to "transfer" values from the inner results to an incremental index, but aggregators - // can't transfer all kinds of values (strings are a common one). If you don't like it, use groupBy v2, which - // doesn't have this problem. - continue; - } - if (Iterables.any(aggs, new Predicate() - { - @Override - public boolean apply(AggregatorFactory agg) - { - return agg.getName().equals(transferAgg.getName()) && !agg.equals(transferAgg); - } - })) { - throw new IAE("Inner aggregator can currently only be referenced by a single type of outer aggregator" + - " for '%s'", transferAgg.getName()); - } - - aggs.add(transferAgg); - } - } - - // We need the inner incremental index to have all the columns required by the outer query - final GroupByQuery innerQuery = new GroupByQuery.Builder(subquery) - .setAggregatorSpecs(ImmutableList.copyOf(aggs)) - .setInterval(subquery.getIntervals()) - .setPostAggregatorSpecs(new ArrayList<>()) - .build(); - - final GroupByQuery outerQuery = new GroupByQuery.Builder(query) - .setLimitSpec(query.getLimitSpec().merge(subquery.getLimitSpec())) - .build(); - - final IncrementalIndex innerQueryResultIndex = GroupByQueryHelper.makeIncrementalIndex( - innerQuery.withOverriddenContext( - ImmutableMap.of( - GroupByQueryHelper.CTX_KEY_SORT_RESULTS, true - ) - ), - subquery, - configSupplier.get(), - subqueryResult - ); - - //Outer query might have multiple intervals, but they are expected to be non-overlapping and sorted which - //is ensured by QuerySegmentSpec. - //GroupByQueryEngine can only process one interval at a time, so we need to call it once per interval - //and concatenate the results. - final IncrementalIndex outerQueryResultIndex = GroupByQueryHelper.makeIncrementalIndex( - outerQuery, - null, - configSupplier.get(), - Sequences.concat( - Sequences.map( - Sequences.simple(outerQuery.getIntervals()), - new Function>() - { - @Override - public Sequence apply(Interval interval) - { - return process( - outerQuery.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(ImmutableList.of(interval)) - ), - new IncrementalIndexStorageAdapter(innerQueryResultIndex), - null - ); - } - } - ) - ) - ); - - innerQueryResultIndex.close(); - - return Sequences.withBaggage( - outerQuery.postProcess(GroupByQueryHelper.postAggregate(query, outerQueryResultIndex)), - outerQueryResultIndex - ); - } - - @Override - public Sequence processSubtotalsSpec( - GroupByQuery query, - GroupByQueryResource resource, - Sequence queryResult - ) - { - throw new UnsupportedOperationException("subtotalsSpec is not supported for v1 groupBy strategy."); - } - - @Override - public QueryRunner mergeRunners( - final QueryProcessingPool queryProcessingPool, - final Iterable> queryRunners - ) - { - return new GroupByMergedQueryRunner<>(queryProcessingPool, configSupplier, queryWatcher, queryRunners); - } - - @Override - public Sequence process( - final GroupByQuery query, - final StorageAdapter storageAdapter, - @Nullable final GroupByQueryMetrics groupByQueryMetrics - ) - { - return Sequences.map( - engine.process(query, storageAdapter, groupByQueryMetrics), - row -> GroupByQueryHelper.toResultRow(query, row) - ); - } - - @Override - public boolean supportsNestedQueryPushDown() - { - return false; - } -} diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java index 35ccd8961da5..8e253f8e9034 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java @@ -264,8 +264,8 @@ protected AddToFactsResult addToFacts( if (IncrementalIndexRow.EMPTY_ROW_INDEX == prev) { numEntries.incrementAndGet(); } else { - // This would happen in a race condition where there are multiple write threads - // which could be possible in case of GroupBy v1 strategy + // this should never happen. Previously, this would happen in a race condition involving multiple write threads + // for GroupBy v1 strategy, but it is no more, so this code needs the concurrency model reworked in the future parseExceptionMessages.clear(); aggs = concurrentGet(prev); aggSizeForRow = doAggregate(metrics, aggs, rowContainer, row, parseExceptionMessages); 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 cb764d3967e6..a7abce4943c9 100644 --- a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java +++ b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java @@ -49,7 +49,6 @@ import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.query.spec.LegacySegmentSpec; import org.apache.druid.query.topn.TopNQuery; import org.apache.druid.query.topn.TopNQueryBuilder; @@ -135,9 +134,7 @@ public MultiValuedDimensionTest(final GroupByQueryConfig config, SegmentWriteOut this.config = config; this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; - this.context = config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1) - ? ImmutableMap.of() - : ImmutableMap.of("forceHashAggregation", forceHashAggregation); + this.context = ImmutableMap.of("forceHashAggregation", forceHashAggregation); } @Before @@ -397,10 +394,6 @@ public void testGroupByWithDimFilterAndWithFilteredDimSpec() @Test public void testGroupByExpression() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(RuntimeException.class); - expectedException.expectMessage("GroupBy v1 does not support dimension selectors with unknown cardinality."); - } GroupByQuery query = GroupByQuery .builder() .setDataSource("xx") @@ -451,10 +444,6 @@ public void testGroupByExpression() @Test public void testGroupByExpressionMultiMulti() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(RuntimeException.class); - expectedException.expectMessage("GroupBy v1 does not support dimension selectors with unknown cardinality."); - } GroupByQuery query = GroupByQuery .builder() .setDataSource("xx") @@ -508,10 +497,6 @@ public void testGroupByExpressionMultiMultiBackwardsCompat0dot22andOlder() { try { ExpressionProcessing.initializeForHomogenizeNullMultiValueStrings(); - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(RuntimeException.class); - expectedException.expectMessage("GroupBy v1 does not support dimension selectors with unknown cardinality."); - } GroupByQuery query = GroupByQuery .builder() .setDataSource("xx") @@ -557,10 +542,6 @@ public void testGroupByExpressionMultiMultiBackwardsCompat0dot22andOlder() @Test public void testGroupByExpressionMultiMultiAuto() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(RuntimeException.class); - expectedException.expectMessage("GroupBy v1 does not support dimension selectors with unknown cardinality."); - } GroupByQuery query = GroupByQuery .builder() .setDataSource("xx") @@ -602,10 +583,6 @@ public void testGroupByExpressionMultiMultiAuto() @Test public void testGroupByExpressionMultiMultiAutoAuto() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(RuntimeException.class); - expectedException.expectMessage("GroupBy v1 does not support dimension selectors with unknown cardinality."); - } GroupByQuery query = GroupByQuery .builder() .setDataSource("xx") @@ -695,10 +672,6 @@ public void testGroupByExpressionMultiMultiAutoAutoDupeIdentifier() @Test public void testGroupByExpressionMultiMultiAutoAutoWithFilter() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(RuntimeException.class); - expectedException.expectMessage("GroupBy v1 does not support dimension selectors with unknown cardinality."); - } GroupByQuery query = GroupByQuery .builder() .setDataSource("xx") @@ -786,10 +759,6 @@ public void testGroupByExpressionAuto() @Test public void testGroupByExpressionArrayExpressionFilter() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(RuntimeException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); - } GroupByQuery query = GroupByQuery .builder() .setDataSource("xx") @@ -836,10 +805,6 @@ public void testGroupByExpressionArrayExpressionFilter() @Test public void testGroupByExpressionArrayFnArg() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(RuntimeException.class); - expectedException.expectMessage("GroupBy v1 does not support dimension selectors with unknown cardinality."); - } GroupByQuery query = GroupByQuery .builder() .setDataSource("xx") @@ -879,10 +844,6 @@ public void testGroupByExpressionArrayFnArg() @Test public void testGroupByExpressionAutoArrayFnArg() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(RuntimeException.class); - expectedException.expectMessage("GroupBy v1 does not support dimension selectors with unknown cardinality."); - } GroupByQuery query = GroupByQuery .builder() .setDataSource("xx") @@ -922,10 +883,6 @@ public void testGroupByExpressionAutoArrayFnArg() @Test public void testGroupByExpressionFoldArrayToString() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(RuntimeException.class); - expectedException.expectMessage("GroupBy v1 does not support dimension selectors with unknown cardinality."); - } GroupByQuery query = GroupByQuery .builder() .setDataSource("xx") @@ -973,10 +930,6 @@ public void testGroupByExpressionFoldArrayToString() @Test public void testGroupByExpressionFoldArrayToStringWithConcats() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(RuntimeException.class); - expectedException.expectMessage("GroupBy v1 does not support dimension selectors with unknown cardinality."); - } GroupByQuery query = GroupByQuery .builder() .setDataSource("xx") 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 941505cf499d..391a490e2da0 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 @@ -57,9 +57,6 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV1; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV2; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; @@ -275,11 +272,6 @@ private void setupGroupByFactory() final GroupByQueryConfig config = new GroupByQueryConfig() { - @Override - public String getDefaultStrategy() - { - return "v2"; - } @Override public int getBufferGrouperInitialBuckets() @@ -294,9 +286,6 @@ public HumanReadableBytes getMaxOnDiskStorage() } }; config.setSingleThreaded(false); - config.setMaxIntermediateRows(Integer.MAX_VALUE); - config.setMaxResults(Integer.MAX_VALUE); - DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig() { @Override @@ -336,50 +325,35 @@ public String getFormatString() }; final Supplier configSupplier = Suppliers.ofInstance(config); - final GroupByStrategySelector strategySelector = new GroupByStrategySelector( + final GroupingEngine groupingEngine = new GroupingEngine( + druidProcessingConfig, configSupplier, - new GroupByStrategyV1( - configSupplier, - new GroupByQueryEngine(configSupplier, bufferPool), - NOOP_QUERYWATCHER - ), - new GroupByStrategyV2( - druidProcessingConfig, - configSupplier, - bufferPool, - mergePool, - TestHelper.makeJsonMapper(), - new ObjectMapper(new SmileFactory()), - NOOP_QUERYWATCHER - ) + bufferPool, + mergePool, + TestHelper.makeJsonMapper(), + new ObjectMapper(new SmileFactory()), + NOOP_QUERYWATCHER ); - final GroupByStrategySelector tooSmallStrategySelector = new GroupByStrategySelector( + final GroupingEngine tooSmallEngine = new GroupingEngine( + tooSmallDruidProcessingConfig, configSupplier, - new GroupByStrategyV1( - configSupplier, - new GroupByQueryEngine(configSupplier, bufferPool), - NOOP_QUERYWATCHER - ), - new GroupByStrategyV2( - tooSmallDruidProcessingConfig, - configSupplier, - bufferPool, - tooSmallMergePool, - TestHelper.makeJsonMapper(), - new ObjectMapper(new SmileFactory()), - NOOP_QUERYWATCHER - ) + bufferPool, + tooSmallMergePool, + TestHelper.makeJsonMapper(), + new ObjectMapper(new SmileFactory()), + NOOP_QUERYWATCHER ); + groupByFactory = new GroupByQueryRunnerFactory( - strategySelector, - new GroupByQueryQueryToolChest(strategySelector) + groupingEngine, + new GroupByQueryQueryToolChest(groupingEngine) ); tooSmallGroupByFactory = new GroupByQueryRunnerFactory( - tooSmallStrategySelector, - new GroupByQueryQueryToolChest(tooSmallStrategySelector) + tooSmallEngine, + new GroupByQueryQueryToolChest(tooSmallEngine) ); } 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 2b50fcb4a5e8..8cd451eb009e 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 @@ -62,9 +62,6 @@ import org.apache.druid.query.extraction.TimeFormatExtractionFn; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV1; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV2; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; @@ -546,11 +543,6 @@ private void setupGroupByFactory() final GroupByQueryConfig config = new GroupByQueryConfig() { - @Override - public String getDefaultStrategy() - { - return "v2"; - } @Override public int getBufferGrouperInitialBuckets() @@ -565,8 +557,6 @@ public HumanReadableBytes getMaxOnDiskStorage() } }; config.setSingleThreaded(false); - config.setMaxIntermediateRows(Integer.MAX_VALUE); - config.setMaxResults(Integer.MAX_VALUE); DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig() { @@ -585,50 +575,34 @@ public String getFormatString() }; final Supplier configSupplier = Suppliers.ofInstance(config); - final GroupByStrategySelector strategySelector = new GroupByStrategySelector( + final GroupingEngine groupingEngine = new GroupingEngine( + druidProcessingConfig, configSupplier, - new GroupByStrategyV1( - configSupplier, - new GroupByQueryEngine(configSupplier, bufferPool), - NOOP_QUERYWATCHER - ), - new GroupByStrategyV2( - druidProcessingConfig, - configSupplier, - bufferPool, - mergePool, - TestHelper.makeJsonMapper(), - new ObjectMapper(new SmileFactory()), - NOOP_QUERYWATCHER - ) + bufferPool, + mergePool, + TestHelper.makeJsonMapper(), + new ObjectMapper(new SmileFactory()), + NOOP_QUERYWATCHER ); - final GroupByStrategySelector strategySelector2 = new GroupByStrategySelector( + final GroupingEngine groupingEngine2 = new GroupingEngine( + druidProcessingConfig, configSupplier, - new GroupByStrategyV1( - configSupplier, - new GroupByQueryEngine(configSupplier, bufferPool), - NOOP_QUERYWATCHER - ), - new GroupByStrategyV2( - druidProcessingConfig, - configSupplier, - bufferPool, - mergePool2, - TestHelper.makeJsonMapper(), - new ObjectMapper(new SmileFactory()), - NOOP_QUERYWATCHER - ) + bufferPool, + mergePool2, + TestHelper.makeJsonMapper(), + new ObjectMapper(new SmileFactory()), + NOOP_QUERYWATCHER ); groupByFactory = new GroupByQueryRunnerFactory( - strategySelector, - new GroupByQueryQueryToolChest(strategySelector) + groupingEngine, + new GroupByQueryQueryToolChest(groupingEngine) ); groupByFactory2 = new GroupByQueryRunnerFactory( - strategySelector2, - new GroupByQueryQueryToolChest(strategySelector2) + groupingEngine2, + new GroupByQueryQueryToolChest(groupingEngine2) ); } 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 27c60c5d0fe8..05aa06db0a54 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 @@ -55,9 +55,6 @@ import org.apache.druid.query.groupby.having.GreaterThanHavingSpec; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV1; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV2; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.IndexIO; @@ -211,11 +208,6 @@ private void setupGroupByFactory() final GroupByQueryConfig config = new GroupByQueryConfig() { - @Override - public String getDefaultStrategy() - { - return "v2"; - } @Override public int getBufferGrouperInitialBuckets() @@ -230,8 +222,6 @@ public HumanReadableBytes getMaxOnDiskStorage() } }; config.setSingleThreaded(false); - config.setMaxIntermediateRows(Integer.MAX_VALUE); - config.setMaxResults(Integer.MAX_VALUE); DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig() { @@ -250,27 +240,19 @@ public String getFormatString() }; final Supplier configSupplier = Suppliers.ofInstance(config); - final GroupByStrategySelector strategySelector = new GroupByStrategySelector( + final GroupingEngine groupingEngine = new GroupingEngine( + druidProcessingConfig, configSupplier, - new GroupByStrategyV1( - configSupplier, - new GroupByQueryEngine(configSupplier, bufferPool), - NOOP_QUERYWATCHER - ), - new GroupByStrategyV2( - druidProcessingConfig, - configSupplier, - bufferPool, - mergePool, - TestHelper.makeJsonMapper(), - new ObjectMapper(new SmileFactory()), - NOOP_QUERYWATCHER - ) + bufferPool, + mergePool, + TestHelper.makeJsonMapper(), + new ObjectMapper(new SmileFactory()), + NOOP_QUERYWATCHER ); groupByFactory = new GroupByQueryRunnerFactory( - strategySelector, - new GroupByQueryQueryToolChest(strategySelector) + groupingEngine, + new GroupByQueryQueryToolChest(groupingEngine) ); } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryConfigTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryConfigTest.java index 0c7a57ca0306..b034210bcf70 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryConfigTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryConfigTest.java @@ -34,10 +34,7 @@ public class GroupByQueryConfigTest private final ImmutableMap CONFIG_MAP = ImmutableMap .builder() .put("singleThreaded", "true") - .put("defaultStrategy", "v2") .put("bufferGrouperInitialBuckets", "1") - .put("maxIntermediateRows", "2") - .put("maxResults", "3") .put("defaultOnDiskStorage", "1M") .put("maxOnDiskStorage", "4M") .put("maxSelectorDictionarySize", "5") @@ -51,10 +48,7 @@ public void testSerde() final GroupByQueryConfig config = MAPPER.convertValue(CONFIG_MAP, GroupByQueryConfig.class); Assert.assertEquals(true, config.isSingleThreaded()); - Assert.assertEquals("v2", config.getDefaultStrategy()); Assert.assertEquals(1, config.getBufferGrouperInitialBuckets()); - Assert.assertEquals(2, config.getMaxIntermediateRows()); - Assert.assertEquals(3, config.getMaxResults()); Assert.assertEquals(4_000_000, config.getMaxOnDiskStorage().getBytes()); Assert.assertEquals(1_000_000, config.getDefaultOnDiskStorage().getBytes()); Assert.assertEquals(5, config.getConfiguredMaxSelectorDictionarySize()); @@ -76,10 +70,7 @@ public void testNoOverrides() ); Assert.assertEquals(true, config2.isSingleThreaded()); - Assert.assertEquals("v2", config2.getDefaultStrategy()); Assert.assertEquals(1, config2.getBufferGrouperInitialBuckets()); - Assert.assertEquals(2, config2.getMaxIntermediateRows()); - Assert.assertEquals(3, config2.getMaxResults()); Assert.assertEquals(1_000_000, config2.getMaxOnDiskStorage().getBytes()); Assert.assertEquals(5, config2.getConfiguredMaxSelectorDictionarySize()); Assert.assertEquals(6_000_000, config2.getConfiguredMaxMergingDictionarySize()); @@ -98,7 +89,6 @@ public void testOverrides() .setGranularity(Granularities.ALL) .setContext( ImmutableMap.builder() - .put("groupByStrategy", "v1") .put("maxOnDiskStorage", "3M") .put("maxResults", 2) .put("maxSelectorDictionarySize", 3) @@ -110,10 +100,7 @@ public void testOverrides() ); Assert.assertEquals(true, config2.isSingleThreaded()); - Assert.assertEquals("v1", config2.getDefaultStrategy()); Assert.assertEquals(1, config2.getBufferGrouperInitialBuckets()); - Assert.assertEquals(2, config2.getMaxIntermediateRows()); - Assert.assertEquals(2, config2.getMaxResults()); Assert.assertEquals(3_000_000, config2.getMaxOnDiskStorage().getBytes()); Assert.assertEquals(5 /* Can't override */, config2.getConfiguredMaxSelectorDictionarySize()); Assert.assertEquals(6_000_000 /* Can't override */, config2.getConfiguredMaxMergingDictionarySize()); 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 e31249d5b165..e080af234b67 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 @@ -37,9 +37,6 @@ import org.apache.druid.query.QueryRunnerTestHelper; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV1; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV2; import org.apache.druid.segment.TestHelper; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.AfterClass; @@ -128,25 +125,17 @@ private static GroupByQueryRunnerFactory makeQueryRunnerFactory( { final Supplier configSupplier = Suppliers.ofInstance(config); - final GroupByStrategySelector strategySelector = new GroupByStrategySelector( + final GroupingEngine groupingEngine = new GroupingEngine( + PROCESSING_CONFIG, configSupplier, - new GroupByStrategyV1( - configSupplier, - new GroupByQueryEngine(configSupplier, BUFFER_POOL), - QueryRunnerTestHelper.NOOP_QUERYWATCHER - ), - new GroupByStrategyV2( - PROCESSING_CONFIG, - configSupplier, - BUFFER_POOL, - MERGE_BUFFER_POOL, - TestHelper.makeJsonMapper(), - mapper, - QueryRunnerTestHelper.NOOP_QUERYWATCHER - ) + BUFFER_POOL, + MERGE_BUFFER_POOL, + TestHelper.makeJsonMapper(), + mapper, + QueryRunnerTestHelper.NOOP_QUERYWATCHER ); - final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(strategySelector); - return new GroupByQueryRunnerFactory(strategySelector, toolChest); + final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(groupingEngine); + return new GroupByQueryRunnerFactory(groupingEngine, toolChest); } private static final CloseableStupidPool BUFFER_POOL = new CloseableStupidPool<>( @@ -163,11 +152,6 @@ private static GroupByQueryRunnerFactory makeQueryRunnerFactory( GroupByQueryRunnerTest.DEFAULT_MAPPER, new GroupByQueryConfig() { - @Override - public String getDefaultStrategy() - { - return "v2"; - } } ); @@ -213,7 +197,7 @@ public void testSimpleGroupBy() .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, TIMEOUT)) .build(); - Assert.assertEquals(0, GroupByStrategyV2.countRequiredMergeBufferNum(query)); + Assert.assertEquals(0, GroupByQueryResources.countRequiredMergeBufferNum(query)); GroupByQueryRunnerTestHelper.runQuery(FACTORY, runner, query); Assert.assertEquals(3, MERGE_BUFFER_POOL.getMinRemainBufferNum()); @@ -242,7 +226,7 @@ public void testNestedGroupBy() .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, TIMEOUT)) .build(); - Assert.assertEquals(1, GroupByStrategyV2.countRequiredMergeBufferNum(query)); + Assert.assertEquals(1, GroupByQueryResources.countRequiredMergeBufferNum(query)); GroupByQueryRunnerTestHelper.runQuery(FACTORY, runner, query); Assert.assertEquals(2, MERGE_BUFFER_POOL.getMinRemainBufferNum()); @@ -282,7 +266,7 @@ public void testDoubleNestedGroupBy() .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, TIMEOUT)) .build(); - Assert.assertEquals(2, GroupByStrategyV2.countRequiredMergeBufferNum(query)); + Assert.assertEquals(2, GroupByQueryResources.countRequiredMergeBufferNum(query)); GroupByQueryRunnerTestHelper.runQuery(FACTORY, runner, query); // This should be 1 because the broker needs 2 buffers and the queryable node needs one. @@ -336,7 +320,7 @@ public void testTripleNestedGroupBy() .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, TIMEOUT)) .build(); - Assert.assertEquals(2, GroupByStrategyV2.countRequiredMergeBufferNum(query)); + Assert.assertEquals(2, GroupByQueryResources.countRequiredMergeBufferNum(query)); GroupByQueryRunnerTestHelper.runQuery(FACTORY, runner, query); // This should be 1 because the broker needs 2 buffers and the queryable node needs one. @@ -365,7 +349,7 @@ public void testSimpleGroupByWithSubtotals() .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, TIMEOUT)) .build(); - Assert.assertEquals(1, GroupByStrategyV2.countRequiredMergeBufferNum(query)); + Assert.assertEquals(1, GroupByQueryResources.countRequiredMergeBufferNum(query)); GroupByQueryRunnerTestHelper.runQuery(FACTORY, runner, query); // 1 for subtotal and 1 for GroupByQueryRunnerFactory#mergeRunners @@ -394,7 +378,7 @@ public void testSimpleGroupByWithSubtotalsWithoutPrefixMatch() .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, TIMEOUT)) .build(); - Assert.assertEquals(2, GroupByStrategyV2.countRequiredMergeBufferNum(query)); + Assert.assertEquals(2, GroupByQueryResources.countRequiredMergeBufferNum(query)); GroupByQueryRunnerTestHelper.runQuery(FACTORY, runner, query); // 2 needed by subtotal and 1 for GroupByQueryRunnerFactory#mergeRunners @@ -436,7 +420,7 @@ public void testNestedGroupByWithSubtotals() .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, TIMEOUT)) .build(); - Assert.assertEquals(3, GroupByStrategyV2.countRequiredMergeBufferNum(query)); + Assert.assertEquals(3, GroupByQueryResources.countRequiredMergeBufferNum(query)); GroupByQueryRunnerTestHelper.runQuery(FACTORY, runner, query); // 2 for subtotal, 1 for nested group by and 1 for GroupByQueryRunnerFactory#mergeRunners @@ -482,7 +466,7 @@ public void testNestedGroupByWithNestedSubtotals() .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, TIMEOUT)) .build(); - Assert.assertEquals(3, GroupByStrategyV2.countRequiredMergeBufferNum(query)); + Assert.assertEquals(3, GroupByQueryResources.countRequiredMergeBufferNum(query)); GroupByQueryRunnerTestHelper.runQuery(FACTORY, runner, query); // 2 for subtotal, 1 for nested group by and 1 for GroupByQueryRunnerFactory#mergeRunners diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java index 2e877f61cfa3..f8483ff1215c 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 @@ -70,9 +70,6 @@ import org.apache.druid.query.groupby.having.OrHavingSpec; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV1; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV2; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnType; @@ -1180,24 +1177,16 @@ public String getFormatString() bufferSupplier, processingConfig.getNumMergeBuffers() ); - final GroupByStrategySelector strategySelector = new GroupByStrategySelector( + final GroupingEngine groupingEngine = new GroupingEngine( + processingConfig, queryConfigSupplier, - new GroupByStrategyV1( - queryConfigSupplier, - new GroupByQueryEngine(queryConfigSupplier, bufferPool), - QueryRunnerTestHelper.NOOP_QUERYWATCHER - ), - new GroupByStrategyV2( - processingConfig, - queryConfigSupplier, - bufferPool, - mergeBufferPool, - TestHelper.makeJsonMapper(), - new ObjectMapper(new SmileFactory()), - QueryRunnerTestHelper.NOOP_QUERYWATCHER - ) + bufferPool, + mergeBufferPool, + TestHelper.makeJsonMapper(), + new ObjectMapper(new SmileFactory()), + QueryRunnerTestHelper.NOOP_QUERYWATCHER ); - final GroupByQueryQueryToolChest queryToolChest = new GroupByQueryQueryToolChest(strategySelector); + final GroupByQueryQueryToolChest queryToolChest = new GroupByQueryQueryToolChest(groupingEngine); CacheStrategy cacheStrategy = queryToolChest.getCacheStrategy(query); Assert.assertTrue( "result level cache on broker server for GroupByStrategyV2 should be enabled", diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java index acfc31892f87..6fc0cdf548f5 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 @@ -40,9 +40,6 @@ import org.apache.druid.query.ResourceLimitExceededException; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV1; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV2; import org.apache.druid.segment.TestHelper; import org.junit.AfterClass; import org.junit.Rule; @@ -98,26 +95,17 @@ private static GroupByQueryRunnerFactory makeQueryRunnerFactory( ) { final Supplier configSupplier = Suppliers.ofInstance(config); - - final GroupByStrategySelector strategySelector = new GroupByStrategySelector( + final GroupingEngine groupingEngine = new GroupingEngine( + DEFAULT_PROCESSING_CONFIG, configSupplier, - new GroupByStrategyV1( - configSupplier, - new GroupByQueryEngine(configSupplier, BUFFER_POOL), - QueryRunnerTestHelper.NOOP_QUERYWATCHER - ), - new GroupByStrategyV2( - DEFAULT_PROCESSING_CONFIG, - configSupplier, - BUFFER_POOL, - MERGE_BUFFER_POOL, - TestHelper.makeJsonMapper(), - mapper, - QueryRunnerTestHelper.NOOP_QUERYWATCHER - ) + BUFFER_POOL, + MERGE_BUFFER_POOL, + TestHelper.makeJsonMapper(), + mapper, + QueryRunnerTestHelper.NOOP_QUERYWATCHER ); - final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(strategySelector); - return new GroupByQueryRunnerFactory(strategySelector, toolChest); + final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(groupingEngine); + return new GroupByQueryRunnerFactory(groupingEngine, toolChest); } private static final CloseableStupidPool BUFFER_POOL = new CloseableStupidPool<>( @@ -133,11 +121,6 @@ private static GroupByQueryRunnerFactory makeQueryRunnerFactory( GroupByQueryRunnerTest.DEFAULT_MAPPER, new GroupByQueryConfig() { - @Override - public String getDefaultStrategy() - { - return "v2"; - } } ); @@ -286,11 +269,6 @@ public void testTimeoutExceptionOnQueryable() GroupByQueryRunnerTest.DEFAULT_MAPPER, new GroupByQueryConfig() { - @Override - public String getDefaultStrategy() - { - return "v2"; - } @Override public boolean isSingleThreaded() 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 0adf5c77d990..269067ff1f9b 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 @@ -39,7 +39,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.DurationGranularity; import org.apache.druid.java.util.common.granularity.Granularities; @@ -124,9 +123,6 @@ import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.LimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV1; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV2; import org.apache.druid.query.lookup.LookupExtractionFn; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.search.ContainsSearchQuerySpec; @@ -214,48 +210,8 @@ public int getNumThreads() public static List testConfigs() { - final GroupByQueryConfig v1Config = new GroupByQueryConfig() - { - @Override - public String toString() - { - return "v1"; - } - - @Override - public String getDefaultStrategy() - { - return GroupByStrategySelector.STRATEGY_V1; - } - }; - - final GroupByQueryConfig v1SingleThreadedConfig = new GroupByQueryConfig() - { - @Override - public boolean isSingleThreaded() - { - return true; - } - - @Override - public String getDefaultStrategy() - { - return GroupByStrategySelector.STRATEGY_V1; - } - - @Override - public String toString() - { - return "v1SingleThreaded"; - } - }; final GroupByQueryConfig v2Config = new GroupByQueryConfig() { - @Override - public String getDefaultStrategy() - { - return GroupByStrategySelector.STRATEGY_V2; - } @Override public int getBufferGrouperInitialBuckets() @@ -272,11 +228,6 @@ public String toString() }; final GroupByQueryConfig v2SmallBufferConfig = new GroupByQueryConfig() { - @Override - public String getDefaultStrategy() - { - return GroupByStrategySelector.STRATEGY_V2; - } @Override public int getBufferGrouperMaxSize() @@ -298,11 +249,6 @@ public String toString() }; final GroupByQueryConfig v2SmallDictionaryConfig = new GroupByQueryConfig() { - @Override - public String getDefaultStrategy() - { - return GroupByStrategySelector.STRATEGY_V2; - } @Override public long getConfiguredMaxSelectorDictionarySize() @@ -330,11 +276,6 @@ public String toString() }; final GroupByQueryConfig v2ParallelCombineConfig = new GroupByQueryConfig() { - @Override - public String getDefaultStrategy() - { - return GroupByStrategySelector.STRATEGY_V2; - } @Override public int getNumParallelCombineThreads() @@ -349,12 +290,7 @@ public String toString() } }; - v1Config.setMaxIntermediateRows(10000); - v1SingleThreadedConfig.setMaxIntermediateRows(10000); - return ImmutableList.of( - v1Config, - v1SingleThreadedConfig, v2Config, v2SmallBufferConfig, v2SmallDictionaryConfig, @@ -415,25 +351,17 @@ public static GroupByQueryRunnerFactory makeQueryRunnerFactory( ); } final Supplier configSupplier = Suppliers.ofInstance(config); - final GroupByStrategySelector strategySelector = new GroupByStrategySelector( + final GroupingEngine groupingEngine = new GroupingEngine( + processingConfig, configSupplier, - new GroupByStrategyV1( - configSupplier, - new GroupByQueryEngine(configSupplier, bufferPools.getProcessingPool()), - QueryRunnerTestHelper.NOOP_QUERYWATCHER - ), - new GroupByStrategyV2( - processingConfig, - configSupplier, - bufferPools.getProcessingPool(), - bufferPools.getMergePool(), - TestHelper.makeJsonMapper(), - mapper, - QueryRunnerTestHelper.NOOP_QUERYWATCHER - ) + bufferPools.getProcessingPool(), + bufferPools.getMergePool(), + TestHelper.makeJsonMapper(), + mapper, + QueryRunnerTestHelper.NOOP_QUERYWATCHER ); - final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(strategySelector); - return new GroupByQueryRunnerFactory(strategySelector, toolChest); + final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(groupingEngine); + return new GroupByQueryRunnerFactory(groupingEngine, toolChest); } @Parameterized.Parameters(name = "{0}") @@ -450,9 +378,7 @@ public static Collection constructorFeeder() final String testName = StringUtils.format("config=%s, runner=%s, vectorize=%s", config, runner, vectorize); // Add vectorization tests for any indexes that support it. - if (!vectorize || - (QueryRunnerTestHelper.isTestRunnerVectorizable(runner) && - config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2))) { + if (!vectorize || (QueryRunnerTestHelper.isTestRunnerVectorizable(runner))) { constructors.add(new Object[]{testName, config, factory, runner, vectorize}); } } @@ -1235,9 +1161,6 @@ public void testGroupByWithOutputNameCollisions() @Test public void testGroupByWithSortDimsFirst() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - return; - } GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -1351,13 +1274,7 @@ public void testMultiValueDimensionNotAllowed() { final String dimName = "placementish"; - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UOE.class); - expectedException.expectMessage(StringUtils.format( - "GroupBy v1 does not support %s as false", - GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING - )); - } else if (!vectorize) { + if (!vectorize) { expectedException.expect(RuntimeException.class); expectedException.expectCause(CoreMatchers.instanceOf(ExecutionException.class)); expectedException.expectCause( @@ -1410,13 +1327,6 @@ public void describeTo(Description description) @Test public void testMultiValueDimensionAsArray() { - // array types don't work with group by v1 - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage( - "GroupBy v1 only supports dimensions with an outputType of STRING."); - } - // Cannot vectorize due to multi-value dimensions. cannotVectorize(); @@ -1453,13 +1363,6 @@ public void testMultiValueDimensionAsArray() @Test public void testSingleValueDimensionAsArray() { - // array types don't work with group by v1 - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage( - "GroupBy v1 only supports dimensions with an outputType of STRING"); - } - // Cannot vectorize due to multi-value dimensions. cannotVectorize(); @@ -1494,14 +1397,6 @@ public void testSingleValueDimensionAsArray() @Test public void testMultiValueDimensionAsArrayWithOtherDims() { - // array types don't work with group by v1 - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage( - "GroupBy v1 only supports dimensions with an outputType of STRING"); - } - - // Cannot vectorize due to multi-value dimensions. cannotVectorize(); @@ -1689,10 +1584,7 @@ public void testMultiValueDimensionAsArrayWithOtherDims() @Test public void testMultiValueDimensionAsStringArrayWithoutExpression() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING"); - } else if (!vectorize) { + if (!vectorize) { expectedException.expect(RuntimeException.class); expectedException.expectMessage("Not supported for multi-value dimensions"); } @@ -1714,10 +1606,7 @@ public void testMultiValueDimensionAsStringArrayWithoutExpression() @Test public void testSingleValueDimensionAsStringArrayWithoutExpression() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING"); - } else if (!vectorize) { + if (!vectorize) { // cannot add exact class cast message due to discrepancies between various JDK versions expectedException.expect(RuntimeException.class); } @@ -1756,10 +1645,7 @@ public void testSingleValueDimensionAsStringArrayWithoutExpression() @Test public void testNumericDimAsStringArrayWithoutExpression() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING"); - } else if (!vectorize) { + if (!vectorize) { // cannot add exact class cast message due to discrepancies between various JDK versions expectedException.expect(RuntimeException.class); } @@ -1782,10 +1668,7 @@ public void testNumericDimAsStringArrayWithoutExpression() @Test public void testMultiValueVirtualDimAsString() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING"); - } else if (!vectorize) { + if (!vectorize) { // cannot add exact class cast message due to discrepancies between various JDK versions expectedException.expect(RuntimeException.class); } @@ -1816,10 +1699,6 @@ public void testMultiValueVirtualDimAsString() @Test public void testExtractionStringSpecWithMultiValueVirtualDimAsInput() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 does not support dimension selectors with unknown cardinality"); - } cannotVectorize(); GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) @@ -1878,10 +1757,7 @@ public void testExtractionStringSpecWithMultiValueVirtualDimAsInput() @Test public void testExtractionStringArraySpecWithMultiValueVirtualDimAsInput() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING"); - } else if (!vectorize) { + if (!vectorize) { expectedException.expect(RuntimeException.class); expectedException.expectMessage("Not supported for multi-value dimensions"); } @@ -1912,10 +1788,7 @@ public void testExtractionStringArraySpecWithMultiValueVirtualDimAsInput() @Test public void testVirtualColumnNumericTypeAsStringArray() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING"); - } else if (!vectorize) { + if (!vectorize) { // cannot add exact class cast message due to discrepancies between various JDK versions expectedException.expect(RuntimeException.class); } @@ -1945,10 +1818,6 @@ public void testVirtualColumnNumericTypeAsStringArray() @Test public void testNestedGroupByWithStringArray() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING"); - } cannotVectorize(); GroupByQuery inner = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) @@ -1995,10 +1864,6 @@ public void testNestedGroupByWithStringArray() @Test public void testNestedGroupByWithLongArrays() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING"); - } cannotVectorize(); GroupByQuery inner = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) @@ -2039,10 +1904,6 @@ public void testNestedGroupByWithLongArrays() @Test public void testGroupByWithLongArrays() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING"); - } cannotVectorize(); GroupByQuery outer = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) @@ -2106,10 +1967,6 @@ public void testGroupByWithLongArrays() @Test public void testGroupByWithLongArraysDesc() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING"); - } cannotVectorize(); GroupByQuery outer = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) @@ -2174,10 +2031,6 @@ public void testGroupByWithLongArraysDesc() @Test public void testGroupByWithDoubleArrays() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING"); - } cannotVectorize(); GroupByQuery outer = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) @@ -2242,10 +2095,6 @@ public void testGroupByWithDoubleArrays() @Test public void testGroupByWithDoubleArraysDesc() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING"); - } cannotVectorize(); GroupByQuery outer = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) @@ -2870,50 +2719,45 @@ public void testGroupByMaxRowsLimitContextOverride() .overrideContext(ImmutableMap.of("maxResults", 1)) .build(); - List expectedResults = null; - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(ResourceLimitExceededException.class); - } else { - 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), + 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) - ); - } + 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) + ); Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "override-maxResults"); @@ -2970,50 +2814,9 @@ public void testGroupByMaxOnDiskStorageContextOverride() .build(); List expectedResults = null; - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { - expectedException.expect(ResourceLimitExceededException.class); - expectedException.expectMessage("Not enough merge buffer memory to execute this query"); - } else { - 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), + expectedException.expect(ResourceLimitExceededException.class); + expectedException.expectMessage("Not enough merge buffer memory to execute this query"); - 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) - ); - } Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, "overide-maxOnDiskStorage"); @@ -3032,54 +2835,12 @@ public void testNotEnoughDiskSpaceThroughContextOverride() .build(); List expectedResults = null; - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { - expectedException.expect(ResourceLimitExceededException.class); - if (config.getMaxOnDiskStorage().getBytes() > 0) { - // The error message always mentions disk if you have spilling enabled (maxOnDiskStorage > 0) - expectedException.expectMessage("Not enough disk space to execute this query"); - } else { - expectedException.expectMessage("Not enough merge buffer memory to execute this query"); - } + expectedException.expect(ResourceLimitExceededException.class); + if (config.getMaxOnDiskStorage().getBytes() > 0) { + // The error message always mentions disk if you have spilling enabled (maxOnDiskStorage > 0) + expectedException.expectMessage("Not enough disk space to execute this query"); } else { - 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) - ); + expectedException.expectMessage("Not enough merge buffer memory to execute this query"); } Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); @@ -3116,20 +2877,9 @@ public void testSubqueryWithOuterMaxOnDiskStorageContextOverride() .overrideContext(ImmutableMap.of("maxOnDiskStorage", 0, "bufferGrouperMaxSize", 0)) .build(); - // v1 strategy throws an exception for this query because it tries to merge the noop outer - // and default inner limit specs, then apply the resulting spec to the outer query, which - // fails because the inner limit spec refers to columns that don't exist in the outer - // query. I'm not sure why it does this. - - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(ISE.class); - expectedException.expectMessage("Unknown column in order clause"); - GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); - } else { - expectedException.expect(ResourceLimitExceededException.class); - expectedException.expectMessage("Not enough merge buffer memory to execute this query"); - GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); - } + expectedException.expect(ResourceLimitExceededException.class); + expectedException.expectMessage("Not enough merge buffer memory to execute this query"); + GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); } @Test @@ -6134,18 +5884,12 @@ public void testDoubleMeanQuery() ) .build(); - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(ISE.class); - expectedException.expectMessage("Unable to handle complex type"); - GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + Row result = Iterables.getOnlyElement(results).toMapBasedRow(query); + if (NullHandling.replaceWithDefault()) { + Assert.assertEquals(39.2307d, result.getMetric("meanOnDouble").doubleValue(), 0.0001d); } else { - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); - Row result = Iterables.getOnlyElement(results).toMapBasedRow(query); - if (NullHandling.replaceWithDefault()) { - Assert.assertEquals(39.2307d, result.getMetric("meanOnDouble").doubleValue(), 0.0001d); - } else { - Assert.assertEquals(51.0d, result.getMetric("meanOnDouble").doubleValue(), 0.0001d); - } + Assert.assertEquals(51.0d, result.getMetric("meanOnDouble").doubleValue(), 0.0001d); } } @@ -7142,22 +6886,11 @@ public void testSubqueryWithOuterCountAggregator() .setGranularity(QueryRunnerTestHelper.ALL_GRAN) .build(); - // v1 strategy throws an exception for this query because it tries to merge the noop outer - // and default inner limit specs, then apply the resulting spec to the outer query, which - // fails because the inner limit spec refers to columns that don't exist in the outer - // query. I'm not sure why it does this. - - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(ISE.class); - expectedException.expectMessage("Unknown column in order clause"); - GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); - } else { - List expectedResults = Collections.singletonList( - makeRow(query, "2011-04-01", "count", 18L) - ); - Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); - TestHelper.assertExpectedObjects(expectedResults, results, "subquery-count-agg"); - } + List expectedResults = Collections.singletonList( + makeRow(query, "2011-04-01", "count", 18L) + ); + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, "subquery-count-agg"); } @Test @@ -7506,10 +7239,6 @@ public void testSubqueryWithFirstLast() @Test public void testGroupByWithSubtotalsSpecOfDimensionsPrefixes() { - if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { - return; - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -7624,10 +7353,6 @@ public void testGroupByWithSubtotalsSpecOfDimensionsPrefixes() @Test public void testGroupByWithSubtotalsSpecGeneral() { - if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { - return; - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -7984,10 +7709,6 @@ public void testGroupByWithSubtotalsSpecGeneral() @Test public void testGroupByWithSubtotalsSpecWithRenamedDimensionAndFilter() { - if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { - return; - } - GroupByQuery query = GroupByQuery .builder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) @@ -8082,10 +7803,6 @@ public void testGroupByWithSubtotalsSpecWithRenamedDimensionAndFilter() @Test public void testGroupByWithSubtotalsSpecWithLongDimensionColumn() { - if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { - return; - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -8377,10 +8094,6 @@ public void testGroupByWithSubtotalsSpecWithLongDimensionColumn() @Test public void testGroupByWithSubtotalsSpecWithOrderLimit() { - if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { - return; - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -8416,10 +8129,6 @@ public void testGroupByWithSubtotalsSpecWithOrderLimit() @Test public void testGroupByWithSubtotalsSpecWithOrderLimitAndOffset() { - if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { - return; - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -8455,10 +8164,6 @@ public void testGroupByWithSubtotalsSpecWithOrderLimitAndOffset() @Test public void testGroupByWithSubtotalsSpecWithOrderLimitForcePushdown() { - if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { - return; - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10056,11 +9761,6 @@ public void testGroupByCardinalityAggOnHyperUnique() @Test public void testGroupByLongColumn() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10136,11 +9836,6 @@ public void testGroupByComplexColumn() @Test public void testGroupByLongColumnDescending() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10189,11 +9884,6 @@ public void testGroupByLongColumnWithExFn() // Cannot vectorize due to extraction dimension spec. cannotVectorize(); - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 does not support dimension selectors with unknown cardinality."); - } - String jsFn = "function(str) { return 'super-' + str; }"; ExtractionFn jsExtractionFn = new JavaScriptExtractionFn(jsFn, false, JavaScriptConfig.getEnabledInstance()); @@ -10235,11 +9925,6 @@ public void testGroupByLongColumnWithExFn() @Test public void testGroupByLongTimeColumn() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10322,11 +10007,6 @@ public void testGroupByLongTimeColumnWithExFn() @Test public void testGroupByFloatColumn() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10373,11 +10053,6 @@ public void testGroupByFloatColumn() @Test public void testGroupByFloatColumnDescending() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10423,11 +10098,6 @@ public void testGroupByFloatColumnDescending() @Test public void testGroupByDoubleColumnDescending() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10476,11 +10146,6 @@ public void testGroupByFloatColumnWithExFn() // Cannot vectorize due to extraction dimension spec. cannotVectorize(); - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 does not support dimension selectors with unknown cardinality."); - } - String jsFn = "function(str) { return 'super-' + str; }"; ExtractionFn jsExtractionFn = new JavaScriptExtractionFn(jsFn, false, JavaScriptConfig.getEnabledInstance()); @@ -10525,11 +10190,6 @@ public void testGroupByFloatColumnWithExFn() @Test public void testGroupByWithHavingSpecOnLongAndFloat() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10582,11 +10242,6 @@ public void testGroupByWithHavingSpecOnLongAndFloat() @Test public void testGroupByLongAndFloatOutputAsString() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 does not support dimension selectors with unknown cardinality."); - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10632,11 +10287,6 @@ public void testGroupByLongAndFloatOutputAsString() @Test public void testGroupByNumericStringsAsNumeric() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 does not support dimension selectors with unknown cardinality."); - } - GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10693,11 +10343,6 @@ public void testGroupByNumericStringsAsNumericWithDecoration() // Cannot vectorize due to regex-filtered dimension spec. cannotVectorize(); - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); - } - // rows with `technology` have `170000` in the qualityNumericString field RegexFilteredDimensionSpec regexSpec = new RegexFilteredDimensionSpec( new DefaultDimensionSpec("qualityNumericString", "ql", ColumnType.LONG), @@ -10749,11 +10394,6 @@ public void testGroupByDecorationOnNumerics() // Cannot vectorize due to filtered dimension spec. cannotVectorize(); - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); - } - RegexFilteredDimensionSpec regexSpec = new RegexFilteredDimensionSpec( new DefaultDimensionSpec("qualityLong", "ql", ColumnType.LONG), "1700" @@ -10817,11 +10457,6 @@ public void testGroupByDecorationOnNumerics() @Test public void testGroupByNestedWithInnerQueryNumerics() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); - } - GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -10895,11 +10530,6 @@ public void testGroupByNestedWithInnerQueryOutputNullNumerics() { cannotVectorize(); - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); - } - // Following extractionFn will generate null value for one kind of quality ExtractionFn extractionFn = new SearchQuerySpecDimExtractionFn(new ContainsSearchQuerySpec("1200", false)); GroupByQuery subquery = makeQueryBuilder() @@ -10979,11 +10609,6 @@ public void testGroupByNestedWithInnerQueryOutputNullNumerics() @Test public void testGroupByNestedWithInnerQueryNumericsWithLongTime() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); - } - GroupByQuery subQuery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -11070,11 +10695,6 @@ public void testGroupByStringOutputAsLong() // Cannot vectorize due to extraction dimension spec. cannotVectorize(); - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); - } - ExtractionFn strlenFn = StrlenExtractionFn.instance(); GroupByQuery query = makeQueryBuilder() @@ -11192,11 +10812,6 @@ public void testGroupByNestedOuterExtractionFnOnFloatInner() // Cannot vectorize due to extraction dimension spec. cannotVectorize(); - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); - } - String jsFn = "function(obj) { return obj; }"; ExtractionFn jsExtractionFn = new JavaScriptExtractionFn(jsFn, false, JavaScriptConfig.getEnabledInstance()); @@ -11240,11 +10855,6 @@ public void testGroupByNestedDoubleTimeExtractionFnWithLongOutputTypes() // Cannot vectorize due to extraction dimension spec. cannotVectorize(); - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); - } - GroupByQuery subquery = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -11285,9 +10895,6 @@ public void testGroupByNestedDoubleTimeExtractionFnWithLongOutputTypes() @Test public void testGroupByLimitPushDown() { - if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { - return; - } GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setGranularity(QueryRunnerTestHelper.ALL_GRAN).setDimensions(new DefaultDimensionSpec( @@ -11333,9 +10940,6 @@ public void testGroupByLimitPushDown() @Test public void testGroupByLimitPushDownWithOffset() { - if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { - return; - } GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setGranularity(QueryRunnerTestHelper.ALL_GRAN).setDimensions(new DefaultDimensionSpec( @@ -11385,9 +10989,6 @@ public void testGroupByLimitPushDownWithLongDimensionNotInLimitSpec() // Cannot vectorize due to extraction dimension spec. cannotVectorize(); - if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { - return; - } GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setGranularity(QueryRunnerTestHelper.ALL_GRAN).setDimensions( @@ -11464,9 +11065,6 @@ public void testMergeLimitPushDownResultsWithLongDimensionNotInLimitSpec() // Cannot vectorize due to extraction dimension spec. cannotVectorize(); - if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { - return; - } GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -11536,9 +11134,6 @@ public void testMergeLimitPushDownResultsWithLongDimensionNotInLimitSpec() @Test public void testMergeResultsWithLimitPushDown() { - if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { - return; - } GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -11606,9 +11201,6 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r @Test public void testMergeResultsWithLimitPushDownSortByAgg() { - if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { - return; - } GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -11673,9 +11265,6 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r @Test public void testMergeResultsWithLimitPushDownSortByAggWithOffset() { - if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { - return; - } GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -11739,9 +11328,6 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r @Test public void testMergeResultsWithLimitPushDownSortByDimDim() { - if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { - return; - } GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -11811,9 +11397,6 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r @Test public void testMergeResultsWithLimitPushDownSortByDimAggDim() { - if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { - return; - } GroupByQuery.Builder builder = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setInterval("2011-04-02/2011-04-04") @@ -11896,9 +11479,6 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r @Test public void testGroupByLimitPushDownPostAggNotSupported() { - if (!config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { - return; - } expectedException.expect(UnsupportedOperationException.class); expectedException.expectMessage("Limit push down when sorting by a post aggregator is not supported."); @@ -12139,11 +11719,6 @@ public void testTypeConversionWithMergingChainedExecutionRunner() // Cannot vectorize due to extraction dimension spec. cannotVectorize(); - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -12207,10 +11782,6 @@ public void testGroupByNoMatchingPrefilter() @Test public void testGroupByOnNullableLong() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -12248,10 +11819,6 @@ public void testGroupByOnNullableLong() @Test public void testGroupByOnNullableDouble() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -12289,10 +11856,6 @@ public void testGroupByOnNullableDouble() @Test public void testGroupByOnNullableDoubleNoLimitPushdown() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -12334,10 +11897,6 @@ public void testGroupByOnNullableDoubleNoLimitPushdown() @Test public void testGroupByOnNullableFloat() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -12375,10 +11934,6 @@ public void testGroupByOnNullableFloat() @Test public void testGroupByOnVirtualColumn() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -12422,10 +11977,6 @@ public void testGroupByOnVirtualColumn() @Test public void testGroupByOnVirtualColumnTimeFloor() { - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(UnsupportedOperationException.class); - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -12784,13 +12335,6 @@ public void testGroupByWithExpressionAggregatorWithArrays() // expression agg not yet vectorized cannotVectorize(); - // array types don't work with group by v1 - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(IllegalStateException.class); - expectedException.expectMessage( - "Unable to handle type[ARRAY] for AggregatorFactory[class org.apache.druid.query.aggregation.ExpressionLambdaAggregatorFactory]"); - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -13078,13 +12622,6 @@ public void testGroupByExpressionAggregatorArrayMultiValue() // expression agg not yet vectorized cannotVectorize(); - // array types don't work with group by v1 - if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { - expectedException.expect(IllegalStateException.class); - expectedException.expectMessage( - "Unable to handle type[ARRAY] for AggregatorFactory[class org.apache.druid.query.aggregation.ExpressionLambdaAggregatorFactory]"); - } - GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -13469,7 +13006,7 @@ private Map makeContext() private void cannotVectorize() { - if (vectorize && config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { + if (vectorize) { expectedException.expect(RuntimeException.class); expectedException.expectMessage("Cannot vectorize!"); } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java index 4e96ebe6cdb9..6851a2022f8a 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java @@ -93,7 +93,6 @@ public static Iterable constructorFeeder() { setUpClass(); GroupByQueryConfig config = new GroupByQueryConfig(); - config.setMaxIntermediateRows(10000); final GroupByQueryRunnerFactory factory = GroupByQueryRunnerTest.makeQueryRunnerFactory(config, BUFFER_POOLS); final List constructors = new ArrayList<>(); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java index dae2363fdd96..12bf586527ab 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java @@ -37,7 +37,6 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.segment.Segment; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -111,11 +110,9 @@ public static Collection constructorFeeder() NestedDataTestUtils.getSegmentGenerators(NestedDataTestUtils.SIMPLE_DATA_FILE); for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) { - if (!GroupByStrategySelector.STRATEGY_V1.equals(config.getDefaultStrategy())) { - for (BiFunction> generatorFn : segmentsGenerators) { - for (String vectorize : new String[]{"false", "true", "force"}) { - constructors.add(new Object[]{config, generatorFn, vectorize}); - } + for (BiFunction> generatorFn : segmentsGenerators) { + for (String vectorize : new String[]{"false", "true", "force"}) { + constructors.add(new Object[]{config, generatorFn, vectorize}); } } } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedGroupByArrayQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedGroupByArrayQueryTest.java index 4d09df32e31e..a43509de9212 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/NestedGroupByArrayQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedGroupByArrayQueryTest.java @@ -33,7 +33,6 @@ import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngineV2; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.segment.Segment; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -106,10 +105,6 @@ public static Collection constructorFeeder() NestedDataTestUtils.getSegmentGenerators(NestedDataTestUtils.ARRAY_TYPES_DATA_FILE); for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) { - if (GroupByStrategySelector.STRATEGY_V1.equals(config.getDefaultStrategy())) { - // group by v1 doesn't support array stuff - continue; - } for (BiFunction> generatorFn : segmentsGenerators) { // skip force because arrays don't really support vectorize engine, but we want the coverage for once they do... for (String vectorize : new String[]{"false", "true"}) { 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 def6630144de..8bf2785e2b7d 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 @@ -61,10 +61,6 @@ import org.apache.druid.query.extraction.RegexDimExtractionFn; import org.apache.druid.query.filter.JavaScriptDimFilter; import org.apache.druid.query.groupby.having.GreaterThanHavingSpec; -import org.apache.druid.query.groupby.strategy.GroupByStrategy; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV1; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV2; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.IndexIO; @@ -259,11 +255,6 @@ private void setupGroupByFactory() final GroupByQueryConfig config = new GroupByQueryConfig() { - @Override - public String getDefaultStrategy() - { - return "v2"; - } @Override public int getBufferGrouperInitialBuckets() @@ -278,8 +269,6 @@ public HumanReadableBytes getMaxOnDiskStorage() } }; config.setSingleThreaded(false); - config.setMaxIntermediateRows(Integer.MAX_VALUE); - config.setMaxResults(Integer.MAX_VALUE); DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig() { @@ -298,50 +287,33 @@ public String getFormatString() }; final Supplier configSupplier = Suppliers.ofInstance(config); - final GroupByStrategySelector strategySelector = new GroupByStrategySelector( + final GroupingEngine engine1 = new GroupingEngine( + druidProcessingConfig, configSupplier, - new GroupByStrategyV1( - configSupplier, - new GroupByQueryEngine(configSupplier, bufferPool), - NOOP_QUERYWATCHER - ), - new GroupByStrategyV2( - druidProcessingConfig, - configSupplier, - bufferPool, - mergePool, - TestHelper.makeJsonMapper(), - new ObjectMapper(new SmileFactory()), - NOOP_QUERYWATCHER - ) - ); - - final GroupByStrategySelector strategySelector2 = new GroupByStrategySelector( + bufferPool, + mergePool, + TestHelper.makeJsonMapper(), + new ObjectMapper(new SmileFactory()), + NOOP_QUERYWATCHER + ); + final GroupingEngine engine2 = new GroupingEngine( + druidProcessingConfig, configSupplier, - new GroupByStrategyV1( - configSupplier, - new GroupByQueryEngine(configSupplier, bufferPool), - NOOP_QUERYWATCHER - ), - new GroupByStrategyV2( - druidProcessingConfig, - configSupplier, - bufferPool, - mergePool2, - TestHelper.makeJsonMapper(), - new ObjectMapper(new SmileFactory()), - NOOP_QUERYWATCHER - ) + bufferPool, + mergePool2, + TestHelper.makeJsonMapper(), + new ObjectMapper(new SmileFactory()), + NOOP_QUERYWATCHER ); groupByFactory = new GroupByQueryRunnerFactory( - strategySelector, - new GroupByQueryQueryToolChest(strategySelector) + engine1, + new GroupByQueryQueryToolChest(engine1) ); groupByFactory2 = new GroupByQueryRunnerFactory( - strategySelector2, - new GroupByQueryQueryToolChest(strategySelector2) + engine2, + new GroupByQueryQueryToolChest(engine2) ); } @@ -769,14 +741,13 @@ private Sequence runNestedQueryWithForcePushDown(GroupByQuery nestedQ ), (QueryToolChest) toolChest ); - GroupByStrategy strategy = ((GroupByQueryRunnerFactory) groupByFactory).getStrategySelector() - .strategize(nestedQuery); + GroupingEngine groupingEngine = ((GroupByQueryRunnerFactory) groupByFactory).getGroupingEngine(); // Historicals execute the query with force push down flag as false GroupByQuery queryWithPushDownDisabled = pushDownQuery.withOverriddenContext(ImmutableMap.of( GroupByQueryConfig.CTX_KEY_FORCE_PUSH_DOWN_NESTED_QUERY, false )); - Sequence pushDownQueryResults = strategy.mergeResults( + Sequence pushDownQueryResults = groupingEngine.mergeResults( queryRunnerForSegments, queryWithPushDownDisabled, context diff --git a/processing/src/test/java/org/apache/druid/query/groupby/UnnestGroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/UnnestGroupByQueryRunnerTest.java index 826d612f678c..d0a7d94c3967 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/UnnestGroupByQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/UnnestGroupByQueryRunnerTest.java @@ -41,9 +41,6 @@ import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.extraction.StringFormatExtractionFn; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV1; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV2; import org.apache.druid.segment.IncrementalIndexSegment; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestIndex; @@ -95,11 +92,6 @@ public static List testConfigs() final GroupByQueryConfig v2Config = new GroupByQueryConfig() { - @Override - public String getDefaultStrategy() - { - return GroupByStrategySelector.STRATEGY_V2; - } @Override public int getBufferGrouperInitialBuckets() @@ -164,25 +156,17 @@ public static GroupByQueryRunnerFactory makeQueryRunnerFactory( ); } final Supplier configSupplier = Suppliers.ofInstance(config); - final GroupByStrategySelector strategySelector = new GroupByStrategySelector( + final GroupingEngine groupingEngine = new GroupingEngine( + processingConfig, configSupplier, - new GroupByStrategyV1( - configSupplier, - new GroupByQueryEngine(configSupplier, bufferPools.getProcessingPool()), - QueryRunnerTestHelper.NOOP_QUERYWATCHER - ), - new GroupByStrategyV2( - processingConfig, - configSupplier, - bufferPools.getProcessingPool(), - bufferPools.getMergePool(), - TestHelper.makeJsonMapper(), - mapper, - QueryRunnerTestHelper.NOOP_QUERYWATCHER - ) + bufferPools.getProcessingPool(), + bufferPools.getMergePool(), + TestHelper.makeJsonMapper(), + mapper, + QueryRunnerTestHelper.NOOP_QUERYWATCHER ); - final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(strategySelector); - return new GroupByQueryRunnerFactory(strategySelector, toolChest); + final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(groupingEngine); + return new GroupByQueryRunnerFactory(groupingEngine, toolChest); } @Parameterized.Parameters(name = "{0}") @@ -197,10 +181,7 @@ public static Collection constructorFeeder() for (boolean vectorize : ImmutableList.of(false)) { // Add vectorization tests for any indexes that support it. - if (!vectorize || - config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { - constructors.add(new Object[]{config, factory, vectorize}); - } + constructors.add(new Object[]{config, factory, vectorize}); } } @@ -734,7 +715,7 @@ private Map makeContext() private void cannotVectorize() { - if (vectorize && config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2)) { + if (vectorize) { expectedException.expect(RuntimeException.class); expectedException.expectMessage("Cannot vectorize!"); } diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java index 796efc754933..503eedb5db88 100644 --- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java +++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java @@ -546,12 +546,6 @@ public AggregatorFactory getCombiningFactory() return null; } - @Override - public List getRequiredColumns() - { - return null; - } - @Override public Object deserialize(Object object) { diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index 6cec0392dd3c..a7045b329b75 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -22,21 +22,19 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.google.common.base.Predicate; import com.google.common.base.Predicates; -import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import org.apache.druid.collections.CloseableStupidPool; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.data.input.MapBasedRow; -import org.apache.druid.data.input.Row; import org.apache.druid.guice.NestedDataModule; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.js.JavaScriptConfig; +import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.JavaScriptAggregatorFactory; @@ -52,7 +50,8 @@ import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; -import org.apache.druid.query.groupby.GroupByQueryEngine; +import org.apache.druid.query.groupby.ResultRow; +import org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngineV2; import org.apache.druid.query.topn.TopNQueryBuilder; import org.apache.druid.query.topn.TopNQueryEngine; import org.apache.druid.query.topn.TopNResultValue; @@ -139,21 +138,8 @@ public void testSanity() throws Exception () -> ByteBuffer.allocate(50000) ) ) { - final GroupByQueryEngine engine = new GroupByQueryEngine( - Suppliers.ofInstance( - new GroupByQueryConfig() - { - @Override - public int getMaxIntermediateRows() - { - return 5; - } - } - ), - pool - ); - final Sequence rows = engine.process( + final Sequence rows = GroupByQueryEngineV2.process( GroupByQuery.builder() .setDataSource("test") .setGranularity(Granularities.ALL) @@ -161,20 +147,24 @@ public int getMaxIntermediateRows() .addDimension("billy") .addDimension("sally") .addAggregator(new LongSumAggregatorFactory("cnt", "cnt")) + .addOrderByColumn("billy") .build(), new IncrementalIndexStorageAdapter(index), + pool, + new GroupByQueryConfig(), + new DruidProcessingConfig(), null ); - final List results = rows.toList(); + final List results = rows.toList(); Assert.assertEquals(2, results.size()); - MapBasedRow row = (MapBasedRow) results.get(0); - Assert.assertEquals(ImmutableMap.of("sally", "bo", "cnt", 1L), row.getEvent()); + ResultRow row = results.get(0); + Assert.assertArrayEquals(new Object[]{NullHandling.defaultStringValue(), "bo", 1L}, row.getArray()); - row = (MapBasedRow) results.get(1); - Assert.assertEquals(ImmutableMap.of("billy", "hi", "cnt", 1L), row.getEvent()); + row = results.get(1); + Assert.assertArrayEquals(new Object[]{"hi", NullHandling.defaultStringValue(), 1L}, row.getArray()); } } @@ -206,21 +196,8 @@ public void testObjectColumnSelectorOnVaryingColumnSchema() throws Exception () -> ByteBuffer.allocate(50000) ) ) { - final GroupByQueryEngine engine = new GroupByQueryEngine( - Suppliers.ofInstance( - new GroupByQueryConfig() - { - @Override - public int getMaxIntermediateRows() - { - return 5; - } - } - ), - pool - ); - final Sequence rows = engine.process( + final Sequence rows = GroupByQueryEngineV2.process( GroupByQuery.builder() .setDataSource("test") .setGranularity(Granularities.ALL) @@ -240,22 +217,26 @@ public int getMaxIntermediateRows() JavaScriptConfig.getEnabledInstance() ) ) + .addOrderByColumn("billy") .build(), new IncrementalIndexStorageAdapter(index), + pool, + new GroupByQueryConfig(), + new DruidProcessingConfig(), null ); - final List results = rows.toList(); + final List results = rows.toList(); Assert.assertEquals(2, results.size()); - MapBasedRow row = (MapBasedRow) results.get(0); - Assert.assertEquals(ImmutableMap.of("billy", "hi", "cnt", 1L, "fieldLength", 2.0), row.getEvent()); + ResultRow row = results.get(0); + Assert.assertArrayEquals(new Object[]{"hi", NullHandling.defaultStringValue(), 1L, 2.0}, row.getArray()); - row = (MapBasedRow) results.get(1); - Assert.assertEquals( - ImmutableMap.of("billy", "hip", "sally", "hop", "cnt", 1L, "fieldLength", 6.0), - row.getEvent() + row = results.get(1); + Assert.assertArrayEquals( + new Object[]{"hip", "hop", 1L, 6.0}, + row.getArray() ); } } @@ -388,21 +369,8 @@ public void testFilterByNull() throws Exception () -> ByteBuffer.allocate(50000) ) ) { - final GroupByQueryEngine engine = new GroupByQueryEngine( - Suppliers.ofInstance( - new GroupByQueryConfig() - { - @Override - public int getMaxIntermediateRows() - { - return 5; - } - } - ), - pool - ); - final Sequence rows = engine.process( + final Sequence rows = GroupByQueryEngineV2.process( GroupByQuery.builder() .setDataSource("test") .setGranularity(Granularities.ALL) @@ -413,15 +381,18 @@ public int getMaxIntermediateRows() .setDimFilter(DimFilters.dimEquals("sally", (String) null)) .build(), new IncrementalIndexStorageAdapter(index), + pool, + new GroupByQueryConfig(), + new DruidProcessingConfig(), null ); - final List results = rows.toList(); + final List results = rows.toList(); Assert.assertEquals(1, results.size()); - MapBasedRow row = (MapBasedRow) results.get(0); - Assert.assertEquals(ImmutableMap.of("billy", "hi", "cnt", 1L), row.getEvent()); + ResultRow row = results.get(0); + Assert.assertArrayEquals(new Object[]{"hi", NullHandling.defaultStringValue(), 1L}, row.getArray()); } } diff --git a/server/src/main/java/org/apache/druid/guice/QueryRunnerFactoryModule.java b/server/src/main/java/org/apache/druid/guice/QueryRunnerFactoryModule.java index a2fe8f8d2a58..5e71a0d1f082 100644 --- a/server/src/main/java/org/apache/druid/guice/QueryRunnerFactoryModule.java +++ b/server/src/main/java/org/apache/druid/guice/QueryRunnerFactoryModule.java @@ -31,7 +31,6 @@ import org.apache.druid.query.datasourcemetadata.DataSourceMetadataQuery; import org.apache.druid.query.datasourcemetadata.DataSourceMetadataQueryRunnerFactory; import org.apache.druid.query.groupby.GroupByQuery; -import org.apache.druid.query.groupby.GroupByQueryEngine; import org.apache.druid.query.groupby.GroupByQueryRunnerFactory; import org.apache.druid.query.metadata.SegmentMetadataQueryRunnerFactory; import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; @@ -88,8 +87,6 @@ public void configure(Binder binder) queryFactoryBinder.addBinding(entry.getKey()).to(entry.getValue()); binder.bind(entry.getValue()).in(LazySingleton.class); } - - binder.bind(GroupByQueryEngine.class).in(LazySingleton.class); } @LazySingleton 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 4436a5c5899a..7755ae2c122c 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -29,8 +29,6 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; -import com.google.common.hash.HashFunction; -import com.google.common.hash.Hashing; import com.google.common.util.concurrent.ForwardingListeningExecutorService; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; @@ -50,7 +48,6 @@ import org.apache.druid.client.selector.RandomServerSelectorStrategy; import org.apache.druid.client.selector.ServerSelector; import org.apache.druid.guice.http.DruidHttpClientConfig; -import org.apache.druid.hll.HyperLogLogCollector; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; @@ -84,7 +81,6 @@ import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; -import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; import org.apache.druid.query.aggregation.post.ConstantPostAggregator; import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; @@ -97,9 +93,7 @@ import org.apache.druid.query.filter.OrDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.groupby.GroupByQuery; -import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.ResultRow; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.query.search.SearchHit; @@ -154,7 +148,6 @@ import javax.annotation.Nullable; import java.io.IOException; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -180,10 +173,7 @@ public class CachingClusteredClientTest { private static final ImmutableMap CONTEXT = ImmutableMap.of( - QueryContexts.FINALIZE_KEY, false, - - // GroupBy v2 won't cache on the broker, so test with v1. - GroupByQueryConfig.CTX_KEY_STRATEGY, GroupByStrategySelector.STRATEGY_V1 + QueryContexts.FINALIZE_KEY, false ); private static final MultipleIntervalSegmentSpec SEG_SPEC = new MultipleIntervalSegmentSpec(ImmutableList.of()); private static final String DATA_SOURCE = "test"; @@ -1244,112 +1234,6 @@ public void testSearchCachingRenamedOutput() ); } - @Test - public void testGroupByCaching() - { - List aggsWithUniques = ImmutableList.builder() - .addAll(AGGS) - .add(new HyperUniquesAggregatorFactory("uniques", "uniques")) - .build(); - - final HashFunction hashFn = Hashing.murmur3_128(); - - GroupByQuery.Builder builder = new GroupByQuery.Builder() - .setDataSource(DATA_SOURCE) - .setQuerySegmentSpec(SEG_SPEC) - .setDimFilter(DIM_FILTER) - .setGranularity(GRANULARITY).setDimensions(new DefaultDimensionSpec("a", "a")) - .setAggregatorSpecs(aggsWithUniques) - .setPostAggregatorSpecs(POST_AGGS) - .setContext(CONTEXT); - - final HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector(); - collector.add(hashFn.hashString("abc123", StandardCharsets.UTF_8).asBytes()); - collector.add(hashFn.hashString("123abc", StandardCharsets.UTF_8).asBytes()); - - final GroupByQuery query = builder.randomQueryId().build(); - - testQueryCaching( - getDefaultQueryRunner(), - query, - Intervals.of("2011-01-01/2011-01-02"), - makeGroupByResults( - query, - DateTimes.of("2011-01-01"), - ImmutableMap.of("a", "a", "rows", 1, "imps", 1, "impers", 1, "uniques", collector) - ), - - Intervals.of("2011-01-02/2011-01-03"), - makeGroupByResults( - query, - DateTimes.of("2011-01-02"), - ImmutableMap.of("a", "b", "rows", 2, "imps", 2, "impers", 2, "uniques", collector) - ), - - Intervals.of("2011-01-05/2011-01-10"), - makeGroupByResults( - query, - DateTimes.of("2011-01-05"), - ImmutableMap.of("a", "c", "rows", 3, "imps", 3, "impers", 3, "uniques", collector), - DateTimes.of("2011-01-06"), - ImmutableMap.of("a", "d", "rows", 4, "imps", 4, "impers", 4, "uniques", collector), - DateTimes.of("2011-01-07"), - ImmutableMap.of("a", "e", "rows", 5, "imps", 5, "impers", 5, "uniques", collector), - DateTimes.of("2011-01-08"), - ImmutableMap.of("a", "f", "rows", 6, "imps", 6, "impers", 6, "uniques", collector), - DateTimes.of("2011-01-09"), - ImmutableMap.of("a", "g", "rows", 7, "imps", 7, "impers", 7, "uniques", collector) - ), - - Intervals.of("2011-01-05/2011-01-10"), - makeGroupByResults( - query, - DateTimes.of("2011-01-05T01"), - ImmutableMap.of("a", "c", "rows", 3, "imps", 3, "impers", 3, "uniques", collector), - DateTimes.of("2011-01-06T01"), - ImmutableMap.of("a", "d", "rows", 4, "imps", 4, "impers", 4, "uniques", collector), - DateTimes.of("2011-01-07T01"), - ImmutableMap.of("a", "e", "rows", 5, "imps", 5, "impers", 5, "uniques", collector), - DateTimes.of("2011-01-08T01"), - ImmutableMap.of("a", "f", "rows", 6, "imps", 6, "impers", 6, "uniques", collector), - DateTimes.of("2011-01-09T01"), - ImmutableMap.of("a", "g", "rows", 7, "imps", 7, "impers", 7, "uniques", collector) - ) - ); - - QueryRunner runner = new FinalizeResultsQueryRunner( - getDefaultQueryRunner(), - WAREHOUSE.getToolChest(query) - ); - TestHelper.assertExpectedObjects( - makeGroupByResults( - query, - DateTimes.of("2011-01-05T"), - ImmutableMap.of("a", "c", "rows", 3, "imps", 3, "impers", 3, "uniques", collector), - DateTimes.of("2011-01-05T01"), - ImmutableMap.of("a", "c", "rows", 3, "imps", 3, "impers", 3, "uniques", collector), - DateTimes.of("2011-01-06T"), - ImmutableMap.of("a", "d", "rows", 4, "imps", 4, "impers", 4, "uniques", collector), - DateTimes.of("2011-01-06T01"), - ImmutableMap.of("a", "d", "rows", 4, "imps", 4, "impers", 4, "uniques", collector), - DateTimes.of("2011-01-07T"), - ImmutableMap.of("a", "e", "rows", 5, "imps", 5, "impers", 5, "uniques", collector), - DateTimes.of("2011-01-07T01"), - ImmutableMap.of("a", "e", "rows", 5, "imps", 5, "impers", 5, "uniques", collector), - DateTimes.of("2011-01-08T"), - ImmutableMap.of("a", "f", "rows", 6, "imps", 6, "impers", 6, "uniques", collector), - DateTimes.of("2011-01-08T01"), - ImmutableMap.of("a", "f", "rows", 6, "imps", 6, "impers", 6, "uniques", collector), - DateTimes.of("2011-01-09T"), - ImmutableMap.of("a", "g", "rows", 7, "imps", 7, "impers", 7, "uniques", collector), - DateTimes.of("2011-01-09T01"), - ImmutableMap.of("a", "g", "rows", 7, "imps", 7, "impers", 7, "uniques", collector) - ), - runner.run(QueryPlus.wrap(builder.randomQueryId().setInterval("2011-01-05/2011-01-10").build())), - "" - ); - } - @Test public void testTimeBoundaryCaching() { @@ -3165,103 +3049,6 @@ public void testTimeBoundaryCachingWhenTimeIsInteger() ); } - @Test - public void testGroupByCachingRenamedAggs() - { - GroupByQuery.Builder builder = new GroupByQuery.Builder() - .setDataSource(DATA_SOURCE) - .setQuerySegmentSpec(SEG_SPEC) - .setDimFilter(DIM_FILTER) - .setGranularity(GRANULARITY).setDimensions(new DefaultDimensionSpec("a", "output")) - .setAggregatorSpecs(AGGS) - .setContext(CONTEXT); - - final GroupByQuery query1 = builder.randomQueryId().build(); - testQueryCaching( - getDefaultQueryRunner(), - query1, - Intervals.of("2011-01-01/2011-01-02"), - makeGroupByResults( - query1, - DateTimes.of("2011-01-01"), - ImmutableMap.of("output", "a", "rows", 1, "imps", 1, "impers", 1) - ), - - Intervals.of("2011-01-02/2011-01-03"), - makeGroupByResults( - query1, - DateTimes.of("2011-01-02"), - ImmutableMap.of("output", "b", "rows", 2, "imps", 2, "impers", 2) - ), - - Intervals.of("2011-01-05/2011-01-10"), - makeGroupByResults( - query1, - DateTimes.of("2011-01-05"), ImmutableMap.of("output", "c", "rows", 3, "imps", 3, "impers", 3), - DateTimes.of("2011-01-06"), ImmutableMap.of("output", "d", "rows", 4, "imps", 4, "impers", 4), - DateTimes.of("2011-01-07"), ImmutableMap.of("output", "e", "rows", 5, "imps", 5, "impers", 5), - DateTimes.of("2011-01-08"), ImmutableMap.of("output", "f", "rows", 6, "imps", 6, "impers", 6), - DateTimes.of("2011-01-09"), ImmutableMap.of("output", "g", "rows", 7, "imps", 7, "impers", 7) - ), - - Intervals.of("2011-01-05/2011-01-10"), - makeGroupByResults( - query1, - DateTimes.of("2011-01-05T01"), ImmutableMap.of("output", "c", "rows", 3, "imps", 3, "impers", 3), - DateTimes.of("2011-01-06T01"), ImmutableMap.of("output", "d", "rows", 4, "imps", 4, "impers", 4), - DateTimes.of("2011-01-07T01"), ImmutableMap.of("output", "e", "rows", 5, "imps", 5, "impers", 5), - DateTimes.of("2011-01-08T01"), ImmutableMap.of("output", "f", "rows", 6, "imps", 6, "impers", 6), - DateTimes.of("2011-01-09T01"), ImmutableMap.of("output", "g", "rows", 7, "imps", 7, "impers", 7) - ) - ); - - QueryRunner runner = new FinalizeResultsQueryRunner( - getDefaultQueryRunner(), - WAREHOUSE.getToolChest(query1) - ); - final ResponseContext context = initializeResponseContext(); - TestHelper.assertExpectedObjects( - makeGroupByResults( - query1, - DateTimes.of("2011-01-05T"), ImmutableMap.of("output", "c", "rows", 3, "imps", 3, "impers", 3), - DateTimes.of("2011-01-05T01"), ImmutableMap.of("output", "c", "rows", 3, "imps", 3, "impers", 3), - DateTimes.of("2011-01-06T"), ImmutableMap.of("output", "d", "rows", 4, "imps", 4, "impers", 4), - DateTimes.of("2011-01-06T01"), ImmutableMap.of("output", "d", "rows", 4, "imps", 4, "impers", 4), - DateTimes.of("2011-01-07T"), ImmutableMap.of("output", "e", "rows", 5, "imps", 5, "impers", 5), - DateTimes.of("2011-01-07T01"), ImmutableMap.of("output", "e", "rows", 5, "imps", 5, "impers", 5), - DateTimes.of("2011-01-08T"), ImmutableMap.of("output", "f", "rows", 6, "imps", 6, "impers", 6), - DateTimes.of("2011-01-08T01"), ImmutableMap.of("output", "f", "rows", 6, "imps", 6, "impers", 6), - DateTimes.of("2011-01-09T"), ImmutableMap.of("output", "g", "rows", 7, "imps", 7, "impers", 7), - DateTimes.of("2011-01-09T01"), ImmutableMap.of("output", "g", "rows", 7, "imps", 7, "impers", 7) - ), - runner.run(QueryPlus.wrap(builder.randomQueryId().setInterval("2011-01-05/2011-01-10").build()), context), - "" - ); - - final GroupByQuery query2 = builder - .setInterval("2011-01-05/2011-01-10").setDimensions(new DefaultDimensionSpec("a", "output2")) - .setAggregatorSpecs(RENAMED_AGGS) - .randomQueryId() - .build(); - TestHelper.assertExpectedObjects( - makeGroupByResults( - query2, - DateTimes.of("2011-01-05T"), ImmutableMap.of("output2", "c", "rows", 3, "imps", 3, "impers2", 3), - DateTimes.of("2011-01-05T01"), ImmutableMap.of("output2", "c", "rows", 3, "imps", 3, "impers2", 3), - DateTimes.of("2011-01-06T"), ImmutableMap.of("output2", "d", "rows", 4, "imps", 4, "impers2", 4), - DateTimes.of("2011-01-06T01"), ImmutableMap.of("output2", "d", "rows", 4, "imps", 4, "impers2", 4), - DateTimes.of("2011-01-07T"), ImmutableMap.of("output2", "e", "rows", 5, "imps", 5, "impers2", 5), - DateTimes.of("2011-01-07T01"), ImmutableMap.of("output2", "e", "rows", 5, "imps", 5, "impers2", 5), - DateTimes.of("2011-01-08T"), ImmutableMap.of("output2", "f", "rows", 6, "imps", 6, "impers2", 6), - DateTimes.of("2011-01-08T01"), ImmutableMap.of("output2", "f", "rows", 6, "imps", 6, "impers2", 6), - DateTimes.of("2011-01-09T"), ImmutableMap.of("output2", "g", "rows", 7, "imps", 7, "impers2", 7), - DateTimes.of("2011-01-09T01"), ImmutableMap.of("output2", "g", "rows", 7, "imps", 7, "impers2", 7) - ), - runner.run(QueryPlus.wrap(query2), context), - "renamed aggregators test" - ); - } - @Test public void testIfNoneMatch() { diff --git a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java index a8aaf10da00b..7180b03983fa 100644 --- a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java +++ b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java @@ -57,8 +57,7 @@ import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; -import org.apache.druid.query.groupby.GroupByQueryHelper; -import org.apache.druid.query.groupby.strategy.GroupByStrategyV2; +import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.topn.TopNQuery; @@ -1492,12 +1491,10 @@ private static class ExpectedQuery .put(QueryContexts.FINALIZE_KEY, true) .put(QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, 0L) .put(GroupByQuery.CTX_KEY_SORT_BY_DIMS_FIRST, false) - .put(GroupByQueryHelper.CTX_KEY_SORT_RESULTS, false) .put(GroupByQueryConfig.CTX_KEY_ARRAY_RESULT_ROWS, true) - .put(GroupByQueryConfig.CTX_KEY_STRATEGY, "X") .put(GroupByQueryConfig.CTX_KEY_APPLY_LIMIT_PUSH_DOWN, true) - .put(GroupByStrategyV2.CTX_KEY_OUTERMOST, true) - .put(GroupByStrategyV2.CTX_KEY_FUDGE_TIMESTAMP, "1979") + .put(GroupingEngine.CTX_KEY_OUTERMOST, true) + .put(GroupingEngine.CTX_KEY_FUDGE_TIMESTAMP, "1979") .build() ); diff --git a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java index 1dae540412aa..8c361e28ffea 100644 --- a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java +++ b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java @@ -60,7 +60,6 @@ import org.apache.druid.query.groupby.GroupByQueryRunnerTest; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.having.HavingSpec; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.query.topn.TopNQuery; import org.apache.druid.query.topn.TopNQueryBuilder; import org.apache.druid.server.initialization.ServerConfig; @@ -387,11 +386,6 @@ public void testExplodingWrapperDoesNotLeakLocks() QueryRunnerFactory factory = GroupByQueryRunnerTest.makeQueryRunnerFactory( new GroupByQueryConfig() { - @Override - public String getDefaultStrategy() - { - return GroupByStrategySelector.STRATEGY_V2; - } @Override public String toString() diff --git a/server/src/test/java/org/apache/druid/server/QueryStackTests.java b/server/src/test/java/org/apache/druid/server/QueryStackTests.java index 69305adc12db..1c9f15acf79d 100644 --- a/server/src/test/java/org/apache/druid/server/QueryStackTests.java +++ b/server/src/test/java/org/apache/druid/server/QueryStackTests.java @@ -46,7 +46,6 @@ import org.apache.druid.query.groupby.GroupByQueryRunnerFactory; import org.apache.druid.query.groupby.GroupByQueryRunnerTest; import org.apache.druid.query.groupby.TestGroupByBuffers; -import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; import org.apache.druid.query.metadata.SegmentMetadataQueryConfig; import org.apache.druid.query.metadata.SegmentMetadataQueryQueryToolChest; @@ -291,11 +290,6 @@ public static QueryRunnerFactoryConglomerate createQueryRunnerFactoryConglomerat GroupByQueryRunnerTest.DEFAULT_MAPPER, new GroupByQueryConfig() { - @Override - public String getDefaultStrategy() - { - return GroupByStrategySelector.STRATEGY_V2; - } }, groupByBuffers, processingConfig diff --git a/website/.spelling b/website/.spelling index 32f7786b1b60..cc4e02fcf293 100644 --- a/website/.spelling +++ b/website/.spelling @@ -1258,6 +1258,7 @@ KafkaStringHeaderFormat kafka.header. kafka.key kafka.timestamp +kafka.topic keyColumnName keyFormat listDelimiter