mirror of
https://github.com/apache/druid.git
synced 2025-02-16 23:15:16 +00:00
remove group-by v1 (#14866)
* remove group-by v1 * docs * remove unused configs, fix test * fix test * adjustments * why not * adjust * review stuff
This commit is contained in:
parent
0c76df1c7d
commit
36e659a501
@ -52,13 +52,10 @@ import org.apache.druid.query.context.ResponseContext;
|
||||
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 class GroupByTypeInterfaceBenchmark
|
||||
);
|
||||
final GroupByQueryConfig config = new GroupByQueryConfig()
|
||||
{
|
||||
@Override
|
||||
public String getDefaultStrategy()
|
||||
{
|
||||
return defaultStrategy;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getBufferGrouperInitialBuckets()
|
||||
@ -365,8 +354,6 @@ public class GroupByTypeInterfaceBenchmark
|
||||
}
|
||||
};
|
||||
config.setSingleThreaded(false);
|
||||
config.setMaxIntermediateRows(Integer.MAX_VALUE);
|
||||
config.setMaxResults(Integer.MAX_VALUE);
|
||||
|
||||
DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig()
|
||||
{
|
||||
@ -385,27 +372,19 @@ public class GroupByTypeInterfaceBenchmark
|
||||
};
|
||||
|
||||
final Supplier<GroupByQueryConfig> 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)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -77,14 +77,11 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
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 class CachingClusteredClientBenchmark
|
||||
GroupByQueryRunnerTest.DEFAULT_MAPPER,
|
||||
new GroupByQueryConfig()
|
||||
{
|
||||
@Override
|
||||
public String getDefaultStrategy()
|
||||
{
|
||||
return GroupByStrategySelector.STRATEGY_V2;
|
||||
}
|
||||
},
|
||||
processingConfig
|
||||
)
|
||||
@ -364,25 +356,17 @@ public class CachingClusteredClientBenchmark
|
||||
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)
|
||||
|
@ -63,15 +63,12 @@ import org.apache.druid.query.expression.TestExprMacroTable;
|
||||
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 class GroupByBenchmark
|
||||
);
|
||||
final GroupByQueryConfig config = new GroupByQueryConfig()
|
||||
{
|
||||
@Override
|
||||
public String getDefaultStrategy()
|
||||
{
|
||||
return defaultStrategy;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getBufferGrouperInitialBuckets()
|
||||
@ -480,8 +469,6 @@ public class GroupByBenchmark
|
||||
}
|
||||
};
|
||||
config.setSingleThreaded(false);
|
||||
config.setMaxIntermediateRows(Integer.MAX_VALUE);
|
||||
config.setMaxResults(Integer.MAX_VALUE);
|
||||
|
||||
DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig()
|
||||
{
|
||||
@ -500,27 +487,19 @@ public class GroupByBenchmark
|
||||
};
|
||||
|
||||
final Supplier<GroupByQueryConfig> 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)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -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|
|
||||
|
@ -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.
|
||||
|
||||
<a name="connection-pool"></a>
|
||||
|
||||
|
@ -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.
|
||||
|
||||
|
@ -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
|
||||
|
@ -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.
|
||||
|
||||
|
@ -128,9 +128,6 @@ public abstract class CompressedBigDecimalAggregatorFactoryBase
|
||||
@Override
|
||||
public abstract AggregateCombiner<CompressedBigDecimal> makeAggregateCombiner();
|
||||
|
||||
@Override
|
||||
public abstract List<AggregatorFactory> getRequiredColumns();
|
||||
|
||||
@Override
|
||||
public abstract String toString();
|
||||
|
||||
|
@ -28,9 +28,6 @@ import org.apache.druid.query.aggregation.BufferAggregator;
|
||||
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 class CompressedBigDecimalMaxAggregatorFactory extends CompressedBigDecim
|
||||
return new CompressedBigDecimalMaxAggregateCombiner();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new CompressedBigDecimalMaxAggregatorFactory(
|
||||
fieldName,
|
||||
fieldName,
|
||||
size,
|
||||
scale,
|
||||
strictNumberParsing
|
||||
));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
|
@ -29,8 +29,6 @@ import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
|
||||
/**
|
||||
@ -118,18 +116,6 @@ public class CompressedBigDecimalMinAggregatorFactory extends CompressedBigDecim
|
||||
return new CompressedBigDecimalMinAggregateCombiner();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new CompressedBigDecimalMinAggregatorFactory(
|
||||
name,
|
||||
fieldName,
|
||||
size,
|
||||
scale,
|
||||
strictNumberParsing
|
||||
));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
|
@ -30,8 +30,6 @@ import org.apache.druid.segment.ColumnValueSelector;
|
||||
|
||||
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 class CompressedBigDecimalSumAggregatorFactory extends CompressedBigDecim
|
||||
return new CompressedBigDecimalSumAggregateCombiner();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new CompressedBigDecimalSumAggregatorFactory(
|
||||
name,
|
||||
fieldName,
|
||||
size,
|
||||
scale,
|
||||
strictNumberParsing
|
||||
));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
|
@ -26,7 +26,6 @@ import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.math.BigDecimal;
|
||||
import java.util.Arrays;
|
||||
|
||||
|
||||
public class CompressedBigDecimalMaxFactoryTest extends CompressedBigDecimalFactoryTestBase
|
||||
@ -50,10 +49,6 @@ public class CompressedBigDecimalMaxFactoryTest extends CompressedBigDecimalFact
|
||||
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<compressedBigDecimal>', 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());
|
||||
|
@ -26,7 +26,6 @@ import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.math.BigDecimal;
|
||||
import java.util.Arrays;
|
||||
|
||||
|
||||
public class CompressedBigDecimalMinFactoryTest extends CompressedBigDecimalFactoryTestBase
|
||||
@ -50,10 +49,7 @@ public class CompressedBigDecimalMinFactoryTest extends CompressedBigDecimalFact
|
||||
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<compressedBigDecimal>', 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());
|
||||
|
@ -26,7 +26,6 @@ import org.junit.Test;
|
||||
|
||||
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 class CompressedBigDecimalSumFactoryTest extends CompressedBigDecimalFact
|
||||
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<compressedBigDecimal>', 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());
|
||||
|
@ -141,14 +141,6 @@ public class DistinctCountAggregatorFactory extends AggregatorFactory
|
||||
return new LongSumAggregatorFactory(name, name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(
|
||||
new DistinctCountAggregatorFactory(fieldName, fieldName, bitMapFactory)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -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,
|
||||
|
@ -167,19 +167,6 @@ public class MomentSketchAggregatorFactory extends AggregatorFactory
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(
|
||||
new MomentSketchAggregatorFactory(
|
||||
fieldName,
|
||||
fieldName,
|
||||
k,
|
||||
compress
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
private MomentSketchWrapper deserializeFromByteArray(byte[] bytes)
|
||||
{
|
||||
return MomentSketchWrapper.fromByteArray(bytes);
|
||||
|
@ -105,15 +105,6 @@ public class AveragerFactoryWrapper<T, R> extends AggregatorFactory
|
||||
throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper.");
|
||||
}
|
||||
|
||||
/**
|
||||
* Not implemented. Throws UnsupportedOperationException.
|
||||
*/
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper.");
|
||||
}
|
||||
|
||||
/**
|
||||
* Not implemented. Throws UnsupportedOperationException.
|
||||
*/
|
||||
|
@ -162,18 +162,6 @@ public class TDigestSketchAggregatorFactory extends AggregatorFactory
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(
|
||||
new TDigestSketchAggregatorFactory(
|
||||
fieldName,
|
||||
fieldName,
|
||||
compression
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object serializedSketch)
|
||||
{
|
||||
|
@ -22,11 +22,9 @@ package org.apache.druid.query.aggregation;
|
||||
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 class TimestampMaxAggregatorFactory extends TimestampAggregatorFactory
|
||||
return new TimestampMaxAggregatorFactory(name, name, timeFormat);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return ImmutableList.of(
|
||||
new TimestampMaxAggregatorFactory(name, fieldName, timeFormat)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorFactory withName(String newName)
|
||||
{
|
||||
|
@ -22,11 +22,8 @@ package org.apache.druid.query.aggregation;
|
||||
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 class TimestampMinAggregatorFactory extends TimestampAggregatorFactory
|
||||
return new TimestampMinAggregatorFactory(name, name, timeFormat);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return ImmutableList.of(
|
||||
new TimestampMinAggregatorFactory(name, fieldName, timeFormat)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorFactory withName(String newName)
|
||||
{
|
||||
|
@ -38,9 +38,8 @@ import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.groupby.GroupByQueryConfig;
|
||||
import org.apache.druid.query.groupby.GroupByQueryQueryToolChest;
|
||||
import org.apache.druid.query.groupby.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 GroupingEngine groupingEngine = new GroupingEngine(
|
||||
new DruidProcessingConfig()
|
||||
{
|
||||
@Override
|
||||
public String getFormatString()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
final GroupByStrategySelector strategySelector = new GroupByStrategySelector(
|
||||
@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(
|
||||
|
@ -133,27 +133,6 @@ public abstract class HllSketchAggregatorFactory extends AggregatorFactory
|
||||
return Collections.singletonList(fieldName);
|
||||
}
|
||||
|
||||
/**
|
||||
* Used by groupBy v1 to create a "transfer aggregator".
|
||||
*
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(
|
||||
new HllSketchBuildAggregatorFactory(
|
||||
fieldName,
|
||||
fieldName,
|
||||
lgK,
|
||||
tgtHllType.toString(),
|
||||
stringEncoding,
|
||||
shouldFinalize,
|
||||
round
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HllSketchHolder deserialize(final Object object)
|
||||
{
|
||||
|
@ -38,9 +38,7 @@ import org.apache.druid.segment.vector.VectorObjectSelector;
|
||||
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<KllDoublesSketch, Double>
|
||||
{
|
||||
@ -81,19 +79,6 @@ public class KllDoublesSketchAggregatorFactory extends KllSketchAggregatorFactor
|
||||
return COMPARATOR;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(
|
||||
new KllDoublesSketchAggregatorFactory(
|
||||
getFieldName(),
|
||||
getFieldName(),
|
||||
getK(),
|
||||
getMaxStreamLength()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorFactory getMergingFactory(final AggregatorFactory other)
|
||||
throws AggregatorFactoryNotMergeableException
|
||||
|
@ -38,9 +38,7 @@ import org.apache.druid.segment.vector.VectorObjectSelector;
|
||||
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<KllFloatsSketch, Float>
|
||||
{
|
||||
@ -81,19 +79,6 @@ public class KllFloatsSketchAggregatorFactory extends KllSketchAggregatorFactory
|
||||
return COMPARATOR;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(
|
||||
new KllFloatsSketchAggregatorFactory(
|
||||
getFieldName(),
|
||||
getFieldName(),
|
||||
getK(),
|
||||
getMaxStreamLength()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorFactory getMergingFactory(final AggregatorFactory other)
|
||||
throws AggregatorFactoryNotMergeableException
|
||||
|
@ -359,20 +359,6 @@ public class DoublesSketchAggregatorFactory extends AggregatorFactory
|
||||
return DoublesSketch.getUpdatableStorageBytes(k, maxStreamLength);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(
|
||||
new DoublesSketchAggregatorFactory(
|
||||
fieldName,
|
||||
fieldName,
|
||||
k,
|
||||
maxStreamLength,
|
||||
shouldFinalize
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorFactory getCombiningFactory()
|
||||
{
|
||||
|
@ -29,8 +29,6 @@ import org.apache.druid.query.aggregation.AggregatorUtil;
|
||||
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 class SketchMergeAggregatorFactory extends SketchAggregatorFactory
|
||||
this.errorBoundsStdDev = errorBoundsStdDev;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(
|
||||
new SketchMergeAggregatorFactory(
|
||||
fieldName,
|
||||
fieldName,
|
||||
size,
|
||||
shouldFinalize,
|
||||
isInputThetaSketch,
|
||||
errorBoundsStdDev
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorFactory getCombiningFactory()
|
||||
{
|
||||
|
@ -279,20 +279,6 @@ public class ArrayOfDoublesSketchAggregatorFactory extends AggregatorFactory
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(
|
||||
new ArrayOfDoublesSketchAggregatorFactory(
|
||||
fieldName,
|
||||
fieldName,
|
||||
nominalEntries,
|
||||
metricColumns,
|
||||
numberOfValues
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorFactory getCombiningFactory()
|
||||
{
|
||||
|
@ -79,31 +79,6 @@ public class HllSketchAggregatorFactoryTest
|
||||
Assert.assertEquals(STRING_ENCODING, target.getStringEncoding());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetRequiredColumns()
|
||||
{
|
||||
List<AggregatorFactory> 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<AggregatorFactory> 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()
|
||||
{
|
||||
|
@ -89,7 +89,7 @@ public class HllSketchAggregatorTest extends InitializedNullHandlingTest
|
||||
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});
|
||||
}
|
||||
}
|
||||
|
@ -31,7 +31,6 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
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 class KllDoublesSketchAggregatorTest extends InitializedNullHandlingTest
|
||||
@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<ResultRow> 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<ResultRow> 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();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -31,7 +31,6 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
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 class KllFloatsSketchAggregatorTest extends InitializedNullHandlingTest
|
||||
@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<ResultRow> 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<ResultRow> 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();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -31,7 +31,6 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
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 class DoublesSketchAggregatorTest extends InitializedNullHandlingTest
|
||||
@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<ResultRow> 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<ResultRow> 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();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -136,12 +136,6 @@ public class BloomFilterAggregatorFactory extends AggregatorFactory
|
||||
return new BloomFilterMergeAggregatorFactory(name, name, maxNumEntries);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new BloomFilterAggregatorFactory(name, field, maxNumEntries));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -31,8 +31,6 @@ import org.apache.druid.segment.ColumnValueSelector;
|
||||
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 class BloomFilterMergeAggregatorFactory extends BloomFilterAggregatorFact
|
||||
return makeMergeAggregator(metricFactory);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new BloomFilterMergeAggregatorFactory(getName(), fieldName, getMaxNumEntries()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
|
@ -33,9 +33,7 @@ import java.io.IOException;
|
||||
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
|
||||
{
|
||||
|
@ -34,7 +34,6 @@ import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.groupby.GroupByQueryConfig;
|
||||
import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
|
||||
import org.apache.druid.query.groupby.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 class BloomFilterGroupByQueryTest extends InitializedNullHandlingTest
|
||||
config,
|
||||
tempFolder
|
||||
);
|
||||
isV2 = config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2);
|
||||
}
|
||||
|
||||
@Parameterized.Parameters(name = "{0}")
|
||||
@ -123,10 +120,6 @@ public class BloomFilterGroupByQueryTest extends InitializedNullHandlingTest
|
||||
@Test
|
||||
public void testNestedQuery() throws Exception
|
||||
{
|
||||
if (!isV2) {
|
||||
return;
|
||||
}
|
||||
|
||||
String query = "{"
|
||||
+ "\"queryType\": \"groupBy\","
|
||||
+ "\"dataSource\": {"
|
||||
@ -160,10 +153,6 @@ public class BloomFilterGroupByQueryTest extends InitializedNullHandlingTest
|
||||
@Test
|
||||
public void testNestedQueryComplex() throws Exception
|
||||
{
|
||||
if (!isV2) {
|
||||
return;
|
||||
}
|
||||
|
||||
String query = "{"
|
||||
+ "\"queryType\": \"groupBy\","
|
||||
+ "\"dataSource\": {"
|
||||
|
@ -212,22 +212,6 @@ public class ApproximateHistogramAggregatorFactory extends AggregatorFactory
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(
|
||||
new ApproximateHistogramAggregatorFactory(
|
||||
fieldName,
|
||||
fieldName,
|
||||
resolution,
|
||||
numBuckets,
|
||||
lowerLimit,
|
||||
upperLimit,
|
||||
finalizeAsBase64Binary
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -222,22 +222,6 @@ public class FixedBucketsHistogramAggregatorFactory extends AggregatorFactory
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(
|
||||
new FixedBucketsHistogramAggregatorFactory(
|
||||
fieldName,
|
||||
fieldName,
|
||||
numBuckets,
|
||||
lowerLimit,
|
||||
upperLimit,
|
||||
outlierHandlingMode,
|
||||
finalizeAsBase64Binary
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -35,7 +35,6 @@ import org.apache.druid.query.groupby.ResultRow;
|
||||
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 class ApproximateHistogramGroupByQueryTest extends InitializedNullHandlin
|
||||
{
|
||||
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 class ApproximateHistogramGroupByQueryTest extends InitializedNullHandlin
|
||||
}
|
||||
};
|
||||
|
||||
v1Config.setMaxIntermediateRows(10000);
|
||||
v1SingleThreadedConfig.setMaxIntermediateRows(10000);
|
||||
|
||||
final List<Object[]> constructors = new ArrayList<>();
|
||||
final List<GroupByQueryConfig> configs = ImmutableList.of(
|
||||
v1Config,
|
||||
v1SingleThreadedConfig,
|
||||
v2Config
|
||||
);
|
||||
|
||||
|
@ -35,7 +35,6 @@ import org.apache.druid.query.groupby.ResultRow;
|
||||
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 class FixedBucketsHistogramGroupByQueryTest extends InitializedNullHandli
|
||||
{
|
||||
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 class FixedBucketsHistogramGroupByQueryTest extends InitializedNullHandli
|
||||
}
|
||||
};
|
||||
|
||||
v1Config.setMaxIntermediateRows(10000);
|
||||
v1SingleThreadedConfig.setMaxIntermediateRows(10000);
|
||||
|
||||
final List<Object[]> constructors = new ArrayList<>();
|
||||
final List<GroupByQueryConfig> configs = ImmutableList.of(
|
||||
v1Config,
|
||||
v1SingleThreadedConfig,
|
||||
v2Config
|
||||
);
|
||||
|
||||
|
@ -23,7 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
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 class IndexerFrameContext implements FrameContext
|
||||
}
|
||||
|
||||
@Override
|
||||
public GroupByStrategySelector groupByStrategySelector()
|
||||
public GroupingEngine groupingEngine()
|
||||
{
|
||||
return context.injector().getInstance(GroupByStrategySelector.class);
|
||||
return context.injector().getInstance(GroupingEngine.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -22,7 +22,7 @@ package org.apache.druid.msq.kernel;
|
||||
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();
|
||||
|
||||
|
@ -36,12 +36,12 @@ import org.apache.druid.msq.querykit.QueryKitUtils;
|
||||
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<Long>
|
||||
|
||||
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 class GroupByPostShuffleFrameProcessor implements FrameProcessor<Long>
|
||||
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 =
|
||||
|
@ -43,7 +43,7 @@ import org.apache.druid.msq.kernel.ProcessorsAndChannels;
|
||||
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 class GroupByPostShuffleFrameProcessorFactory extends BaseFrameProcessorF
|
||||
{
|
||||
// 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<OutputChannel> outputChannels = new Int2ObjectAVLTreeMap<>();
|
||||
|
||||
for (final ReadablePartition partition : slice.getPartitions()) {
|
||||
@ -112,7 +112,7 @@ public class GroupByPostShuffleFrameProcessorFactory extends BaseFrameProcessorF
|
||||
|
||||
return new GroupByPostShuffleFrameProcessor(
|
||||
query,
|
||||
strategySelector,
|
||||
engine,
|
||||
readableInput.getChannel(),
|
||||
outputChannel.getWritableChannel(),
|
||||
stageDefinition.createFrameWriterFactory(outputChannel.getFrameMemoryAllocator()),
|
||||
|
@ -42,9 +42,9 @@ import org.apache.druid.msq.input.ReadableInput;
|
||||
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 @@ import java.io.IOException;
|
||||
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 class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor
|
||||
final GroupByQuery query,
|
||||
final ReadableInput baseInput,
|
||||
final Int2ObjectMap<ReadableInput> sideChannels,
|
||||
final GroupByStrategySelector strategySelector,
|
||||
final GroupingEngine groupingEngine,
|
||||
final ResourceHolder<WritableFrameChannel> outputChannelHolder,
|
||||
final ResourceHolder<FrameWriterFactory> frameWriterFactoryHolder,
|
||||
final long memoryReservedForBroadcastJoin
|
||||
@ -89,7 +89,7 @@ public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor
|
||||
memoryReservedForBroadcastJoin
|
||||
);
|
||||
this.query = query;
|
||||
this.strategySelector = strategySelector;
|
||||
this.groupingEngine = groupingEngine;
|
||||
this.frameWriterColumnSelectorFactory = RowBasedGrouperHelper.createResultRowBasedColumnSelectorFactory(
|
||||
query,
|
||||
() -> resultYielder.get(),
|
||||
@ -104,12 +104,11 @@ public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor
|
||||
final ResourceHolder<Segment> segmentHolder = closer.register(segment.getOrLoad());
|
||||
|
||||
final Sequence<ResultRow> 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 @@ public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor
|
||||
final FrameSegment frameSegment = new FrameSegment(frame, inputFrameReader, SegmentId.dummy("x"));
|
||||
|
||||
final Sequence<ResultRow> 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()) {
|
||||
|
@ -63,7 +63,7 @@ public class GroupByPreShuffleFrameProcessorFactory extends BaseLeafFrameProcess
|
||||
query,
|
||||
baseInput,
|
||||
sideChannels,
|
||||
frameContext.groupByStrategySelector(),
|
||||
frameContext.groupingEngine(),
|
||||
outputChannelHolder,
|
||||
frameWriterFactoryHolder,
|
||||
frameContext.memoryParameters().getBroadcastJoinMemory()
|
||||
|
@ -117,12 +117,6 @@ public class PassthroughAggregatorFactory extends AggregatorFactory
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -53,8 +53,8 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
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 class CalciteMSQTestsHelper
|
||||
.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,
|
||||
|
@ -121,8 +121,8 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
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;
|
||||
@ -392,6 +392,11 @@ public class MSQTestBase extends BaseCalciteQueryTest
|
||||
};
|
||||
|
||||
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<Set<NodeRole>>()
|
||||
@ -404,10 +409,6 @@ public class MSQTestBase extends BaseCalciteQueryTest
|
||||
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");
|
||||
|
@ -249,12 +249,6 @@ public class VarianceAggregatorFactory extends AggregatorFactory
|
||||
return new VarianceFoldingAggregatorFactory(name, name, estimator);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new VarianceAggregatorFactory(fieldName, fieldName, estimator, inputType));
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorFactory getMergingFactory(AggregatorFactory other) throws AggregatorFactoryNotMergeableException
|
||||
{
|
||||
|
@ -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<T> implements QueryRunner<T>
|
||||
{
|
||||
private static final Logger log = new Logger(GroupByMergedQueryRunner.class);
|
||||
private final Iterable<QueryRunner<T>> queryables;
|
||||
private final Supplier<GroupByQueryConfig> configSupplier;
|
||||
private final QueryWatcher queryWatcher;
|
||||
private final QueryProcessingPool queryProcessingPool;
|
||||
|
||||
public GroupByMergedQueryRunner(
|
||||
QueryProcessingPool queryProcessingPool,
|
||||
Supplier<GroupByQueryConfig> configSupplier,
|
||||
QueryWatcher queryWatcher,
|
||||
Iterable<QueryRunner<T>> queryables
|
||||
)
|
||||
{
|
||||
this.queryProcessingPool = queryProcessingPool;
|
||||
this.queryWatcher = queryWatcher;
|
||||
this.queryables = Iterables.unmodifiableIterable(Iterables.filter(queryables, Predicates.notNull()));
|
||||
this.configSupplier = configSupplier;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Sequence<T> run(final QueryPlus<T> queryPlus, final ResponseContext responseContext)
|
||||
{
|
||||
final GroupByQuery query = (GroupByQuery) queryPlus.getQuery();
|
||||
final GroupByQueryConfig querySpecificConfig = configSupplier.get().withOverrides(query);
|
||||
final boolean isSingleThreaded = querySpecificConfig.isSingleThreaded();
|
||||
final Pair<IncrementalIndex, Accumulator<IncrementalIndex, T>> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair(
|
||||
query,
|
||||
null,
|
||||
querySpecificConfig
|
||||
);
|
||||
final Pair<Queue, Accumulator<Queue, T>> bySegmentAccumulatorPair = GroupByQueryHelper.createBySegmentAccumulatorPair();
|
||||
final QueryContext queryContext = query.context();
|
||||
final boolean bySegment = queryContext.isBySegment();
|
||||
final int priority = queryContext.getPriority();
|
||||
final QueryPlus<T> threadSafeQueryPlus = queryPlus.withoutThreadUnsafeState();
|
||||
final List<ListenableFuture<Void>> futures =
|
||||
Lists.newArrayList(
|
||||
Iterables.transform(
|
||||
queryables,
|
||||
new Function<QueryRunner<T>, ListenableFuture<Void>>()
|
||||
{
|
||||
@Override
|
||||
public ListenableFuture<Void> apply(final QueryRunner<T> input)
|
||||
{
|
||||
if (input == null) {
|
||||
throw new ISE("Null queryRunner! Looks to be some segment unmapping action happening");
|
||||
}
|
||||
|
||||
ListenableFuture<Void> future = queryProcessingPool.submitRunnerTask(
|
||||
new AbstractPrioritizedQueryRunnerCallable<Void, T>(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<Row, T>()
|
||||
{
|
||||
@Override
|
||||
public T apply(Row input)
|
||||
{
|
||||
return (T) input;
|
||||
}
|
||||
}
|
||||
)
|
||||
), indexAccumulatorPair.lhs
|
||||
);
|
||||
}
|
||||
|
||||
private void waitForFutureCompletion(
|
||||
GroupByQuery query,
|
||||
List<ListenableFuture<Void>> futures,
|
||||
IncrementalIndex closeOnFailure
|
||||
)
|
||||
{
|
||||
ListenableFuture<List<Void>> 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());
|
||||
}
|
||||
}
|
||||
}
|
@ -30,7 +30,7 @@ import org.apache.druid.java.util.common.StringUtils;
|
||||
*
|
||||
* <ul>
|
||||
* <li>When the query is rejected by QueryScheduler.</li>
|
||||
* <li>When the query cannot acquire enough merge buffers for groupBy v2</li>
|
||||
* <li>When the query cannot acquire enough merge buffers for groupBy</li>
|
||||
* </ul>
|
||||
* <p>
|
||||
* As a {@link QueryException} it is expected to be serialied to a json response, but will be mapped to
|
||||
|
@ -30,7 +30,7 @@ import org.apache.druid.guice.annotations.ExtensionPoint;
|
||||
* <p>
|
||||
* 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.
|
||||
* <p>
|
||||
* Query execution task also includes a reference to {@link QueryRunner} so that any state required to decide the priority
|
||||
|
@ -194,17 +194,15 @@ public abstract class AggregatorFactory implements Cacheable
|
||||
}
|
||||
|
||||
/**
|
||||
* 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<AggregatorFactory> getRequiredColumns();
|
||||
@Deprecated
|
||||
public List<AggregatorFactory> 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
|
||||
|
@ -29,7 +29,6 @@ import org.apache.druid.segment.column.ColumnType;
|
||||
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 class CountAggregatorFactory extends AggregatorFactory
|
||||
return new LongSumAggregatorFactory(name, name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new CountAggregatorFactory(name));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -29,8 +29,6 @@ import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
@ -111,12 +109,6 @@ public class DoubleMaxAggregatorFactory extends SimpleDoubleAggregatorFactory
|
||||
return new DoubleMaxAggregatorFactory(name, name, null, macroTable);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new DoubleMaxAggregatorFactory(fieldName, fieldName, expression, macroTable));
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorFactory withName(String newName)
|
||||
{
|
||||
|
@ -29,8 +29,6 @@ import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
@ -111,12 +109,6 @@ public class DoubleMinAggregatorFactory extends SimpleDoubleAggregatorFactory
|
||||
return new DoubleMinAggregatorFactory(name, name, null, macroTable);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new DoubleMinAggregatorFactory(fieldName, fieldName, expression, macroTable));
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorFactory withName(String newName)
|
||||
{
|
||||
|
@ -29,8 +29,6 @@ import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
@ -111,12 +109,6 @@ public class DoubleSumAggregatorFactory extends SimpleDoubleAggregatorFactory
|
||||
return new DoubleSumAggregatorFactory(name, name, null, macroTable);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new DoubleSumAggregatorFactory(fieldName, fieldName, expression, macroTable));
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorFactory withName(String newName)
|
||||
{
|
||||
|
@ -50,7 +50,6 @@ import org.apache.druid.segment.virtual.ExpressionSelectors;
|
||||
|
||||
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 class ExpressionLambdaAggregatorFactory extends AggregatorFactory
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(
|
||||
new ExpressionLambdaAggregatorFactory(
|
||||
name,
|
||||
fields,
|
||||
accumulatorId,
|
||||
initialValueExpressionString,
|
||||
initialCombineValueExpressionString,
|
||||
isNullUnlessAggregated,
|
||||
shouldAggregateNullInputs,
|
||||
shouldCombineAggregateNullInputs,
|
||||
foldExpressionString,
|
||||
combineExpressionString,
|
||||
compareExpressionString,
|
||||
finalizeExpressionString,
|
||||
maxSizeBytes,
|
||||
macroTable
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnType getIntermediateType()
|
||||
{
|
||||
|
@ -284,12 +284,6 @@ public class FilteredAggregatorFactory extends AggregatorFactory
|
||||
return dimFilter;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return delegate.getRequiredColumns();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(final Object o)
|
||||
{
|
||||
|
@ -29,8 +29,6 @@ import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
@ -111,12 +109,6 @@ public class FloatMaxAggregatorFactory extends SimpleFloatAggregatorFactory
|
||||
return new FloatMaxAggregatorFactory(name, name, null, macroTable);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new FloatMaxAggregatorFactory(fieldName, fieldName, expression, macroTable));
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorFactory withName(String newName)
|
||||
{
|
||||
|
@ -29,8 +29,6 @@ import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
@ -111,12 +109,6 @@ public class FloatMinAggregatorFactory extends SimpleFloatAggregatorFactory
|
||||
return new FloatMinAggregatorFactory(name, name, null, macroTable);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new FloatMinAggregatorFactory(fieldName, fieldName, expression, macroTable));
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorFactory withName(String newName)
|
||||
{
|
||||
|
@ -29,8 +29,6 @@ import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
@ -111,12 +109,6 @@ public class FloatSumAggregatorFactory extends SimpleFloatAggregatorFactory
|
||||
return new FloatSumAggregatorFactory(name, name, null, macroTable);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new FloatSumAggregatorFactory(fieldName, fieldName, expression, macroTable));
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorFactory withName(String newName)
|
||||
{
|
||||
|
@ -180,12 +180,6 @@ public class GroupingAggregatorFactory extends AggregatorFactory
|
||||
return new GroupingAggregatorFactory(name, groupings, keyDimensions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new GroupingAggregatorFactory(name, groupings, keyDimensions));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -138,13 +138,6 @@ public class HistogramAggregatorFactory extends AggregatorFactory
|
||||
return new HistogramAggregatorFactory(name, name, breaksList);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(
|
||||
new HistogramAggregatorFactory(fieldName, fieldName, breaksList));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -25,8 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
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 class JavaScriptAggregatorFactory extends AggregatorFactory
|
||||
throw new AggregatorFactoryNotMergeableException(this, other);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return ImmutableList.copyOf(
|
||||
Lists.transform(
|
||||
fieldNames,
|
||||
new com.google.common.base.Function<String, AggregatorFactory>()
|
||||
{
|
||||
@Override
|
||||
public AggregatorFactory apply(String input)
|
||||
{
|
||||
return new JavaScriptAggregatorFactory(input, Collections.singletonList(input), fnCombine, fnReset, fnCombine, config);
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -29,8 +29,6 @@ import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
@ -111,12 +109,6 @@ public class LongMaxAggregatorFactory extends SimpleLongAggregatorFactory
|
||||
return new LongMaxAggregatorFactory(name, name, null, macroTable);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new LongMaxAggregatorFactory(fieldName, fieldName, expression, macroTable));
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorFactory withName(String newName)
|
||||
{
|
||||
|
@ -29,8 +29,6 @@ import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
@ -111,12 +109,6 @@ public class LongMinAggregatorFactory extends SimpleLongAggregatorFactory
|
||||
return new LongMinAggregatorFactory(name, name, null, macroTable);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new LongMinAggregatorFactory(fieldName, fieldName, expression, macroTable));
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorFactory withName(String newName)
|
||||
{
|
||||
|
@ -29,8 +29,6 @@ import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
@ -117,12 +115,6 @@ public class LongSumAggregatorFactory extends SimpleLongAggregatorFactory
|
||||
return new LongSumAggregatorFactory(name, name, null, macroTable);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new LongSumAggregatorFactory(fieldName, fieldName, expression, macroTable));
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
|
@ -107,12 +107,6 @@ public class SuppressedAggregatorFactory extends AggregatorFactory
|
||||
return delegate.getMergingFactory(other);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return delegate.getRequiredColumns();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -158,12 +158,6 @@ public class DoubleAnyAggregatorFactory extends AggregatorFactory
|
||||
return new DoubleAnyAggregatorFactory(name, name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new DoubleAnyAggregatorFactory(fieldName, fieldName));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -156,12 +156,6 @@ public class FloatAnyAggregatorFactory extends AggregatorFactory
|
||||
return new FloatAnyAggregatorFactory(name, name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new FloatAnyAggregatorFactory(fieldName, fieldName));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -154,12 +154,6 @@ public class LongAnyAggregatorFactory extends AggregatorFactory
|
||||
return new LongAnyAggregatorFactory(name, name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new LongAnyAggregatorFactory(fieldName, fieldName));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -125,12 +125,6 @@ public class StringAnyAggregatorFactory extends AggregatorFactory
|
||||
return new StringAnyAggregatorFactory(name, name, maxStringBytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new StringAnyAggregatorFactory(fieldName, fieldName, maxStringBytes));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -53,7 +53,6 @@ import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||
|
||||
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 class CardinalityAggregatorFactory extends AggregatorFactory
|
||||
return new HyperUniquesAggregatorFactory(name, name, false, round);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> 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)
|
||||
|
@ -41,7 +41,6 @@ import org.apache.druid.segment.column.ColumnType;
|
||||
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 class DoubleFirstAggregatorFactory extends AggregatorFactory
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new DoubleFirstAggregatorFactory(fieldName, fieldName, timeColumn));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -41,7 +41,6 @@ import org.apache.druid.segment.column.ColumnType;
|
||||
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 class FloatFirstAggregatorFactory extends AggregatorFactory
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new FloatFirstAggregatorFactory(fieldName, fieldName, timeColumn));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
|
@ -41,7 +41,6 @@ import org.apache.druid.segment.column.ColumnType;
|
||||
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 class LongFirstAggregatorFactory extends AggregatorFactory
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new LongFirstAggregatorFactory(fieldName, fieldName, timeColumn));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -42,7 +42,6 @@ import org.apache.druid.segment.column.ColumnType;
|
||||
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 class StringFirstAggregatorFactory extends AggregatorFactory
|
||||
return new StringFirstAggregatorFactory(name, name, timeColumn, maxStringBytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new StringFirstAggregatorFactory(fieldName, fieldName, timeColumn, maxStringBytes));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -187,17 +187,6 @@ public class HyperUniquesAggregatorFactory extends AggregatorFactory
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new HyperUniquesAggregatorFactory(
|
||||
fieldName,
|
||||
fieldName,
|
||||
isInputHyperUnique,
|
||||
round
|
||||
));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -32,7 +32,6 @@ import org.apache.druid.query.aggregation.BufferAggregator;
|
||||
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 org.apache.druid.segment.vector.VectorValueSelector;
|
||||
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 class DoubleLastAggregatorFactory extends AggregatorFactory
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new LongFirstAggregatorFactory(fieldName, fieldName, timeColumn));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -32,7 +32,6 @@ import org.apache.druid.query.aggregation.BufferAggregator;
|
||||
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 org.apache.druid.segment.vector.VectorValueSelector;
|
||||
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 class FloatLastAggregatorFactory extends AggregatorFactory
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new LongFirstAggregatorFactory(fieldName, fieldName, timeColumn));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -48,7 +48,6 @@ import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
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 class LongLastAggregatorFactory extends AggregatorFactory
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new LongLastAggregatorFactory(fieldName, fieldName, timeColumn));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -48,7 +48,6 @@ import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
|
||||
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 class StringLastAggregatorFactory extends AggregatorFactory
|
||||
return new StringLastAggregatorFactory(name, name, timeColumn, maxStringBytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new StringLastAggregatorFactory(fieldName, fieldName, timeColumn, maxStringBytes));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -160,12 +160,6 @@ public class DoubleMeanAggregatorFactory extends AggregatorFactory
|
||||
return new DoubleMeanAggregatorFactory(name, name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(new DoubleMeanAggregatorFactory(fieldName, fieldName));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object object)
|
||||
{
|
||||
|
@ -73,7 +73,6 @@ import org.joda.time.DateTime;
|
||||
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.List;
|
||||
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 class GroupByQuery extends BaseQuery<ResultRow>
|
||||
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));
|
||||
|
@ -25,7 +25,6 @@ import org.apache.druid.java.util.common.logger.Logger;
|
||||
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 class GroupByQueryConfig
|
||||
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 class GroupByQueryConfig
|
||||
{
|
||||
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 class GroupByQueryConfig
|
||||
public String toString()
|
||||
{
|
||||
return "GroupByQueryConfig{" +
|
||||
"defaultStrategy='" + defaultStrategy + '\'' +
|
||||
", singleThreaded=" + singleThreaded +
|
||||
", maxIntermediateRows=" + maxIntermediateRows +
|
||||
", maxResults=" + maxResults +
|
||||
"singleThreaded=" + singleThreaded +
|
||||
", bufferGrouperMaxSize=" + bufferGrouperMaxSize +
|
||||
", bufferGrouperMaxLoadFactor=" + bufferGrouperMaxLoadFactor +
|
||||
", bufferGrouperInitialBuckets=" + bufferGrouperInitialBuckets +
|
||||
|
@ -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<GroupByQueryConfig> config;
|
||||
private final NonBlockingPool<ByteBuffer> intermediateResultsBufferPool;
|
||||
|
||||
@Inject
|
||||
public GroupByQueryEngine(
|
||||
Supplier<GroupByQueryConfig> config,
|
||||
@Global NonBlockingPool<ByteBuffer> intermediateResultsBufferPool
|
||||
)
|
||||
{
|
||||
this.config = config;
|
||||
this.intermediateResultsBufferPool = intermediateResultsBufferPool;
|
||||
}
|
||||
|
||||
public Sequence<Row> 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<Interval> 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<Cursor> cursors = storageAdapter.makeCursors(
|
||||
filter,
|
||||
intervals.get(0),
|
||||
query.getVirtualColumns(),
|
||||
query.getGranularity(),
|
||||
false,
|
||||
groupByQueryMetrics
|
||||
);
|
||||
|
||||
final ResourceHolder<ByteBuffer> bufferHolder = intermediateResultsBufferPool.take();
|
||||
|
||||
return Sequences.concat(
|
||||
Sequences.withBaggage(
|
||||
Sequences.map(
|
||||
cursors,
|
||||
new Function<Cursor, Sequence<Row>>()
|
||||
{
|
||||
@Override
|
||||
public Sequence<Row> apply(final Cursor cursor)
|
||||
{
|
||||
return new BaseSequence<>(
|
||||
new BaseSequence.IteratorMaker<Row, RowIterator>()
|
||||
{
|
||||
@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<ByteBuffer, Integer> 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<ByteBuffer, Integer> 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<ByteBuffer, Integer> getPositions()
|
||||
{
|
||||
return positions;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
private List<ByteBuffer> updateValues(ByteBuffer key, List<DimensionSelector> 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<ByteBuffer> retVal = null;
|
||||
for (int i = 0; i < rowSize; i++) {
|
||||
ByteBuffer newKey = key.duplicate();
|
||||
int dimValue = row.get(i);
|
||||
newKey.putInt(dimValue);
|
||||
List<ByteBuffer> 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<Row>
|
||||
{
|
||||
private final GroupByQuery query;
|
||||
private final Cursor cursor;
|
||||
private final ByteBuffer metricsBuffer;
|
||||
private final int maxIntermediateRows;
|
||||
|
||||
private final List<DimensionSelector> dimensions;
|
||||
private final ArrayList<String> dimNames;
|
||||
private final BufferAggregator[] aggregators;
|
||||
private final String[] metricNames;
|
||||
private final int[] sizesRequired;
|
||||
|
||||
@Nullable
|
||||
private List<ByteBuffer> unprocessedKeys;
|
||||
private Iterator<Row> 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<DimensionSpec> 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<AggregatorFactory> 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<ByteBuffer> 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<Map.Entry<ByteBuffer, Integer>, Row>()
|
||||
{
|
||||
private final DateTime timestamp = cursor.getTime();
|
||||
private final int[] increments = positionMaintainer.getIncrements();
|
||||
|
||||
@Override
|
||||
public Row apply(@Nullable Map.Entry<ByteBuffer, Integer> input)
|
||||
{
|
||||
Map<String, Object> 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();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -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 <T> Pair<IncrementalIndex, Accumulator<IncrementalIndex, T>> 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<AggregatorFactory> aggs;
|
||||
if (combine) {
|
||||
aggs = Lists.transform(
|
||||
query.getAggregatorSpecs(),
|
||||
new Function<AggregatorFactory, AggregatorFactory>()
|
||||
{
|
||||
@Override
|
||||
public AggregatorFactory apply(AggregatorFactory input)
|
||||
{
|
||||
return input.getCombiningFactory();
|
||||
}
|
||||
}
|
||||
);
|
||||
} else {
|
||||
aggs = query.getAggregatorSpecs();
|
||||
}
|
||||
|
||||
final List<String> dimensions = Lists.transform(
|
||||
query.getDimensions(),
|
||||
new Function<DimensionSpec, String>()
|
||||
{
|
||||
@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<DimensionSchema> 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<IncrementalIndex, T> accumulator = new Accumulator<IncrementalIndex, T>()
|
||||
{
|
||||
@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 <T> Pair<Queue, Accumulator<Queue, T>> createBySegmentAccumulatorPair()
|
||||
{
|
||||
// In parallel query runner multiple threads add to this queue concurrently
|
||||
Queue init = new ConcurrentLinkedQueue<>();
|
||||
Accumulator<Queue, T> accumulator = new Accumulator<Queue, T>()
|
||||
{
|
||||
@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<ResultRow> rows
|
||||
)
|
||||
{
|
||||
final Pair<IncrementalIndex, Accumulator<IncrementalIndex, ResultRow>> indexAccumulatorPair =
|
||||
GroupByQueryHelper.createIndexAccumulatorPair(query, subquery, config);
|
||||
|
||||
return rows.accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs);
|
||||
}
|
||||
|
||||
// Used by GroupByStrategyV1
|
||||
public static Sequence<ResultRow> 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;
|
||||
}
|
||||
}
|
@ -32,7 +32,6 @@ import com.google.common.annotations.VisibleForTesting;
|
||||
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.context.ResponseContext;
|
||||
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<ResultRow, GroupB
|
||||
private static final TypeReference<ResultRow> TYPE_REFERENCE = new TypeReference<ResultRow>()
|
||||
{
|
||||
};
|
||||
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<GroupByQueryConfig> queryConfigSupplier,
|
||||
GroupByQueryMetricsFactory queryMetricsFactory
|
||||
)
|
||||
{
|
||||
this.strategySelector = strategySelector;
|
||||
this.groupingEngine = groupingEngine;
|
||||
this.queryConfig = queryConfigSupplier.get();
|
||||
this.queryMetricsFactory = queryMetricsFactory;
|
||||
}
|
||||
@ -133,23 +128,20 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<ResultRow, GroupB
|
||||
}
|
||||
|
||||
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<ResultRow> createMergeFn(Query<ResultRow> query)
|
||||
{
|
||||
return strategySelector.strategize((GroupByQuery) query).createMergeFn(query);
|
||||
return groupingEngine.createMergeFn(query);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<ResultRow> createResultComparator(Query<ResultRow> query)
|
||||
{
|
||||
return strategySelector.strategize((GroupByQuery) query).createResultComparator(query);
|
||||
return groupingEngine.createResultComparator(query);
|
||||
}
|
||||
|
||||
private Sequence<ResultRow> initAndMergeGroupByResults(
|
||||
@ -158,11 +150,9 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<ResultRow, GroupB
|
||||
ResponseContext context
|
||||
)
|
||||
{
|
||||
final GroupByStrategy groupByStrategy = strategySelector.strategize(query);
|
||||
final GroupByQueryResource resource = groupByStrategy.prepareResource(query);
|
||||
final GroupByQueryResources resource = groupingEngine.prepareResource(query);
|
||||
try {
|
||||
final Sequence<ResultRow> mergedSequence = mergeGroupByResults(
|
||||
groupByStrategy,
|
||||
query,
|
||||
resource,
|
||||
runner,
|
||||
@ -179,23 +169,21 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<ResultRow, GroupB
|
||||
}
|
||||
|
||||
private Sequence<ResultRow> mergeGroupByResults(
|
||||
GroupByStrategy groupByStrategy,
|
||||
final GroupByQuery query,
|
||||
GroupByQueryResource resource,
|
||||
GroupByQueryResources resource,
|
||||
QueryRunner<ResultRow> 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<ResultRow> mergeGroupByResultsWithoutPushDown(
|
||||
GroupByStrategy groupByStrategy,
|
||||
GroupByQuery query,
|
||||
GroupByQueryResource resource,
|
||||
GroupByQueryResources resource,
|
||||
QueryRunner<ResultRow> runner,
|
||||
ResponseContext context
|
||||
)
|
||||
@ -228,15 +216,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<ResultRow, GroupB
|
||||
}
|
||||
|
||||
final Sequence<ResultRow> 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 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<ResultRow, GroupB
|
||||
final Sequence<ResultRow> 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<ResultRow> mergeResultsWithNestedQueryPushDown(
|
||||
GroupByStrategy groupByStrategy,
|
||||
GroupByQuery query,
|
||||
GroupByQueryResource resource,
|
||||
GroupByQueryResources resource,
|
||||
QueryRunner<ResultRow> runner,
|
||||
ResponseContext context
|
||||
)
|
||||
{
|
||||
Sequence<ResultRow> pushDownQueryResults = groupByStrategy.mergeResults(runner, query, context);
|
||||
Sequence<ResultRow> pushDownQueryResults = groupingEngine.mergeResults(runner, query, context);
|
||||
final Sequence<ResultRow> 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 @@ public class GroupByQueryQueryToolChest extends QueryToolChest<ResultRow, GroupB
|
||||
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 class GroupByQueryQueryToolChest extends QueryToolChest<ResultRow, GroupB
|
||||
@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
|
||||
|
@ -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<List<String>> 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<String> queryDimOutputNames = query.getDimensions().stream().map(DimensionSpec::getOutputName).collect(
|
||||
Collectors.toList());
|
||||
for (List<String> 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<ReferenceCountingResourceHolder<ByteBuffer>> mergeBufferHolders;
|
||||
private final Deque<ByteBuffer> mergeBuffers;
|
||||
|
||||
public GroupByQueryResources()
|
||||
{
|
||||
this.mergeBufferHolders = null;
|
||||
this.mergeBuffers = new ArrayDeque<>();
|
||||
}
|
||||
|
||||
public GroupByQueryResources(List<ReferenceCountingResourceHolder<ByteBuffer>> 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<ByteBuffer> getMergeBuffer()
|
||||
{
|
||||
final ByteBuffer buffer = mergeBuffers.pop();
|
||||
return new ResourceHolder<ByteBuffer>()
|
||||
{
|
||||
@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);
|
||||
}
|
||||
}
|
||||
}
|
@ -30,7 +30,6 @@ import org.apache.druid.query.QueryRunner;
|
||||
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 @@ import org.apache.druid.segment.StorageAdapter;
|
||||
*/
|
||||
public class GroupByQueryRunnerFactory implements QueryRunnerFactory<ResultRow, GroupByQuery>
|
||||
{
|
||||
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<ResultRow> createRunner(final Segment segment)
|
||||
{
|
||||
return new GroupByQueryRunner(segment, strategySelector);
|
||||
return new GroupByQueryRunner(segment, groupingEngine);
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -69,9 +68,7 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<ResultRow,
|
||||
@Override
|
||||
public Sequence<ResultRow> run(QueryPlus<ResultRow> queryPlus, ResponseContext responseContext)
|
||||
{
|
||||
QueryRunner<ResultRow> rowQueryRunner = strategySelector
|
||||
.strategize((GroupByQuery) queryPlus.getQuery())
|
||||
.mergeRunners(queryProcessingPool, queryRunners);
|
||||
QueryRunner<ResultRow> rowQueryRunner = groupingEngine.mergeRunners(queryProcessingPool, queryRunners);
|
||||
return rowQueryRunner.run(queryPlus, responseContext);
|
||||
}
|
||||
};
|
||||
@ -86,12 +83,12 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<ResultRow,
|
||||
private static class GroupByQueryRunner implements QueryRunner<ResultRow>
|
||||
{
|
||||
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 class GroupByQueryRunnerFactory implements QueryRunnerFactory<ResultRow,
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
@ -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.granularity.Granularity;
|
||||
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.aggregation.PostAggregator;
|
||||
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.epinephelinae.GroupByRowProcessor;
|
||||
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.Set;
|
||||
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<GroupByQueryConfig> configSupplier;
|
||||
private final NonBlockingPool<ByteBuffer> bufferPool;
|
||||
@ -104,7 +93,7 @@ public class GroupByStrategyV2 implements GroupByStrategy
|
||||
private final QueryWatcher queryWatcher;
|
||||
|
||||
@Inject
|
||||
public GroupByStrategyV2(
|
||||
public GroupingEngine(
|
||||
DruidProcessingConfig processingConfig,
|
||||
Supplier<GroupByQueryConfig> configSupplier,
|
||||
@Global NonBlockingPool<ByteBuffer> bufferPool,
|
||||
@ -123,10 +112,19 @@ public class GroupByStrategyV2 implements GroupByStrategy
|
||||
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 class GroupByStrategyV2 implements GroupByStrategy
|
||||
+ mergeBufferPool.maxSize() + " merge buffers were configured"
|
||||
);
|
||||
} else if (requiredMergeBufferNum == 0) {
|
||||
return new GroupByQueryResource();
|
||||
return new GroupByQueryResources();
|
||||
} else {
|
||||
final List<ReferenceCountingResourceHolder<ByteBuffer>> mergeBufferHolders;
|
||||
final QueryContext context = query.context();
|
||||
@ -151,61 +149,49 @@ public class GroupByStrategyV2 implements GroupByStrategy
|
||||
)
|
||||
);
|
||||
} 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<ResultRow> createResultComparator(Query<ResultRow> 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<ResultRow> createMergeFn(Query<ResultRow> 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<ResultRow> mergeResults(
|
||||
final QueryRunner<ResultRow> baseRunner,
|
||||
final GroupByQuery query,
|
||||
@ -223,7 +209,6 @@ public class GroupByStrategyV2 implements GroupByStrategy
|
||||
// Set up downstream context.
|
||||
final ImmutableMap.Builder<String, Object> 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 class GroupByStrategyV2 implements GroupByStrategy
|
||||
}
|
||||
}
|
||||
|
||||
private void moveOrReplicateTimestampInRow(
|
||||
GroupByQuery query,
|
||||
int timestampResultFieldIndexInOriginalDimensions,
|
||||
ResultRow before,
|
||||
ResultRow after
|
||||
/**
|
||||
* 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).
|
||||
* <p>
|
||||
* 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<ResultRow> mergeRunners(
|
||||
final QueryProcessingPool queryProcessingPool,
|
||||
final Iterable<QueryRunner<ResultRow>> queryRunners
|
||||
)
|
||||
{
|
||||
// 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 new GroupByMergingQueryRunnerV2(
|
||||
configSupplier.get(),
|
||||
processingConfig,
|
||||
queryProcessingPool,
|
||||
queryWatcher,
|
||||
queryRunners,
|
||||
processingConfig.getNumThreads(),
|
||||
mergeBufferPool,
|
||||
processingConfig.intermediateComputeSizeBytes(),
|
||||
spillMapper,
|
||||
processingConfig.getTmpDir()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
/**
|
||||
* 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<ResultRow> process(
|
||||
GroupByQuery query,
|
||||
StorageAdapter storageAdapter,
|
||||
@Nullable GroupByQueryMetrics groupByQueryMetrics
|
||||
)
|
||||
{
|
||||
return GroupByQueryEngineV2.process(
|
||||
query,
|
||||
storageAdapter,
|
||||
bufferPool,
|
||||
configSupplier.get().withOverrides(query),
|
||||
processingConfig,
|
||||
groupByQueryMetrics
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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<ResultRow> applyPostProcessing(Sequence<ResultRow> results, GroupByQuery query)
|
||||
{
|
||||
// Don't apply limit here for inner results, that will be pushed down to the BufferHashGrouper
|
||||
@ -421,11 +444,22 @@ public class GroupByStrategyV2 implements GroupByStrategy
|
||||
}
|
||||
}
|
||||
|
||||
@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<ResultRow> processSubqueryResult(
|
||||
GroupByQuery subquery,
|
||||
GroupByQuery query,
|
||||
GroupByQueryResource resource,
|
||||
GroupByQueryResources resource,
|
||||
Sequence<ResultRow> subqueryResult,
|
||||
boolean wasQueryPushedDown
|
||||
)
|
||||
@ -462,7 +496,7 @@ public class GroupByStrategyV2 implements GroupByStrategy
|
||||
mergeResults(
|
||||
(queryPlus, responseContext) -> finalResultSupplier.results(null),
|
||||
query,
|
||||
null
|
||||
ResponseContext.createEmpty()
|
||||
),
|
||||
finalResultSupplier
|
||||
);
|
||||
@ -472,10 +506,18 @@ public class GroupByStrategyV2 implements GroupByStrategy
|
||||
}
|
||||
}
|
||||
|
||||
@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<ResultRow> processSubtotalsSpec(
|
||||
GroupByQuery query,
|
||||
GroupByQueryResource resource,
|
||||
GroupByQueryResources resource,
|
||||
Sequence<ResultRow> queryResult
|
||||
)
|
||||
{
|
||||
@ -631,7 +673,7 @@ public class GroupByStrategyV2 implements GroupByStrategy
|
||||
)
|
||||
),
|
||||
subtotalQuery,
|
||||
null
|
||||
ResponseContext.createEmpty()
|
||||
);
|
||||
}
|
||||
catch (Throwable e) {
|
||||
@ -639,6 +681,34 @@ public class GroupByStrategyV2 implements GroupByStrategy
|
||||
}
|
||||
}
|
||||
|
||||
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<String> getAggregatorAndPostAggregatorNames(GroupByQuery query)
|
||||
{
|
||||
Set<String> aggsAndPostAggs = new HashSet();
|
||||
@ -656,74 +726,4 @@ public class GroupByStrategyV2 implements GroupByStrategy
|
||||
|
||||
return aggsAndPostAggs;
|
||||
}
|
||||
|
||||
private static int numMergeBuffersNeededForSubtotalsSpec(GroupByQuery query)
|
||||
{
|
||||
List<List<String>> 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<String> queryDimOutputNames = query.getDimensions().stream().map(DimensionSpec::getOutputName).collect(
|
||||
Collectors.toList());
|
||||
for (List<String> subtotalSpec : subtotalSpecs) {
|
||||
if (!Utils.isPrefix(subtotalSpec, queryDimOutputNames)) {
|
||||
return 2;
|
||||
}
|
||||
}
|
||||
|
||||
return Math.max(1, numMergeBuffersNeededForSubQuerySubtotal);
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueryRunner<ResultRow> mergeRunners(
|
||||
final QueryProcessingPool queryProcessingPool,
|
||||
final Iterable<QueryRunner<ResultRow>> queryRunners
|
||||
)
|
||||
{
|
||||
return new GroupByMergingQueryRunnerV2(
|
||||
configSupplier.get(),
|
||||
processingConfig,
|
||||
queryProcessingPool,
|
||||
queryWatcher,
|
||||
queryRunners,
|
||||
processingConfig.getNumThreads(),
|
||||
mergeBufferPool,
|
||||
processingConfig.intermediateComputeSizeBytes(),
|
||||
spillMapper,
|
||||
processingConfig.getTmpDir()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Sequence<ResultRow> 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;
|
||||
}
|
||||
}
|
@ -34,7 +34,7 @@ import java.util.function.BinaryOperator;
|
||||
* 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<ResultRow>
|
||||
{
|
||||
|
@ -80,7 +80,7 @@ import java.util.concurrent.TimeoutException;
|
||||
* 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<ResultRow>
|
||||
{
|
||||
|
@ -42,6 +42,7 @@ import org.apache.druid.query.filter.Filter;
|
||||
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.column.StringGroupByColumnSe
|
||||
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.DateTime;
|
||||
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 @@ import java.util.stream.Stream;
|
||||
* 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 class GroupByQueryEngineV2
|
||||
|
||||
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
|
||||
|
@ -33,9 +33,9 @@ import org.apache.druid.query.ResourceLimitExceededException;
|
||||
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 @@ import java.util.UUID;
|
||||
|
||||
/**
|
||||
* 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 class GroupByRowProcessor
|
||||
final Sequence<ResultRow> rows,
|
||||
final GroupByQueryConfig config,
|
||||
final DruidProcessingConfig processingConfig,
|
||||
final GroupByQueryResource resource,
|
||||
final GroupByQueryResources resource,
|
||||
final ObjectMapper spillMapper,
|
||||
final String processingTmpDir,
|
||||
final int mergeBufferSize
|
||||
|
@ -280,12 +280,6 @@ public class DefaultLimitSpec implements LimitSpec
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public LimitSpec merge(LimitSpec other)
|
||||
{
|
||||
return this;
|
||||
}
|
||||
|
||||
private ColumnType getOrderByType(final OrderByColumnSpec columnSpec, final List<DimensionSpec> dimensions)
|
||||
{
|
||||
for (DimensionSpec dimSpec : dimensions) {
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
x
Reference in New Issue
Block a user