2020-06-17 14:21:48 +00:00
|
|
|
#pragma once
|
|
|
|
#include <Processors/QueryPlan/ITransformingStep.h>
|
2021-10-15 20:18:20 +00:00
|
|
|
#include <QueryPipeline/SizeLimits.h>
|
2020-06-17 14:21:48 +00:00
|
|
|
#include <Storages/SelectQueryInfo.h>
|
2020-06-18 18:29:33 +00:00
|
|
|
#include <Interpreters/Aggregator.h>
|
2020-06-17 14:21:48 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2022-05-10 16:12:03 +00:00
|
|
|
struct GroupingSetsParams
|
|
|
|
{
|
|
|
|
GroupingSetsParams() = default;
|
|
|
|
|
2022-06-27 11:16:52 +00:00
|
|
|
GroupingSetsParams(Names used_keys_, Names missing_keys_) : used_keys(std::move(used_keys_)), missing_keys(std::move(missing_keys_)) { }
|
2022-05-10 16:12:03 +00:00
|
|
|
|
2022-06-27 11:16:52 +00:00
|
|
|
Names used_keys;
|
|
|
|
Names missing_keys;
|
2022-05-10 16:12:03 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
using GroupingSetsParamsList = std::vector<GroupingSetsParams>;
|
|
|
|
|
2022-05-16 17:33:38 +00:00
|
|
|
Block appendGroupingSetColumn(Block header);
|
2022-07-01 16:24:05 +00:00
|
|
|
Block generateOutputHeader(const Block & input_header, const Names & keys, bool use_nulls);
|
2022-05-16 17:33:38 +00:00
|
|
|
|
2023-02-01 17:55:06 +00:00
|
|
|
class AggregatingProjectionStep;
|
|
|
|
|
2020-06-22 10:18:28 +00:00
|
|
|
/// Aggregation. See AggregatingTransform.
|
2020-06-17 14:21:48 +00:00
|
|
|
class AggregatingStep : public ITransformingStep
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
AggregatingStep(
|
|
|
|
const DataStream & input_stream_,
|
2020-06-18 18:29:33 +00:00
|
|
|
Aggregator::Params params_,
|
2022-05-05 13:56:16 +00:00
|
|
|
GroupingSetsParamsList grouping_sets_params_,
|
2020-06-18 18:29:33 +00:00
|
|
|
bool final_,
|
2020-06-17 14:21:48 +00:00
|
|
|
size_t max_block_size_,
|
2021-10-19 13:36:31 +00:00
|
|
|
size_t aggregation_in_order_max_block_bytes_,
|
2020-06-17 14:21:48 +00:00
|
|
|
size_t merge_threads_,
|
|
|
|
size_t temporary_data_merge_threads_,
|
|
|
|
bool storage_has_evenly_distributed_read_,
|
2022-06-30 15:14:26 +00:00
|
|
|
bool group_by_use_nulls_,
|
2022-11-23 18:38:12 +00:00
|
|
|
SortDescription sort_description_for_merging_,
|
2022-06-28 23:16:30 +00:00
|
|
|
SortDescription group_by_sort_description_,
|
2022-11-27 23:41:31 +00:00
|
|
|
bool should_produce_results_in_order_of_bucket_number_,
|
2023-02-14 13:02:54 +00:00
|
|
|
bool memory_bound_merging_of_aggregation_results_enabled_,
|
|
|
|
bool explicit_sorting_required_for_aggregation_in_order_);
|
2020-06-17 14:21:48 +00:00
|
|
|
|
2022-12-19 16:33:10 +00:00
|
|
|
static Block appendGroupingColumn(Block block, const Names & keys, bool has_grouping, bool use_nulls);
|
|
|
|
|
2020-06-17 14:21:48 +00:00
|
|
|
String getName() const override { return "Aggregating"; }
|
|
|
|
|
2021-09-14 16:28:41 +00:00
|
|
|
void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
|
2020-06-17 14:21:48 +00:00
|
|
|
|
2021-04-14 14:07:56 +00:00
|
|
|
void describeActions(JSONBuilder::JSONMap & map) const override;
|
|
|
|
|
2020-06-27 14:02:24 +00:00
|
|
|
void describeActions(FormatSettings &) const override;
|
2020-06-25 09:39:17 +00:00
|
|
|
void describePipeline(FormatSettings & settings) const override;
|
2020-06-23 16:06:56 +00:00
|
|
|
|
2021-02-11 08:49:12 +00:00
|
|
|
const Aggregator::Params & getParams() const { return params; }
|
|
|
|
|
2023-02-22 10:54:19 +00:00
|
|
|
const auto & getGroupingSetsParamsList() const { return grouping_sets_params; }
|
|
|
|
|
2022-11-23 18:38:12 +00:00
|
|
|
bool inOrder() const { return !sort_description_for_merging.empty(); }
|
2023-02-14 13:02:54 +00:00
|
|
|
bool explicitSortingRequired() const { return explicit_sorting_required_for_aggregation_in_order; }
|
2022-11-23 18:38:12 +00:00
|
|
|
bool isGroupingSets() const { return !grouping_sets_params.empty(); }
|
|
|
|
void applyOrder(SortDescription sort_description_for_merging_, SortDescription group_by_sort_description_);
|
2022-11-30 14:31:39 +00:00
|
|
|
bool memoryBoundMergingWillBeUsed() const;
|
2023-01-05 19:22:42 +00:00
|
|
|
void skipMerging() { skip_merging = true; }
|
2022-11-27 23:41:31 +00:00
|
|
|
|
2023-02-01 17:55:06 +00:00
|
|
|
bool canUseProjection() const;
|
2023-02-21 20:01:59 +00:00
|
|
|
/// When we apply aggregate projection (which is full), this step will only merge data.
|
|
|
|
/// Argument input_stream replaces current single input.
|
2023-02-21 19:50:17 +00:00
|
|
|
/// Probably we should replace this step to MergingAggregated later? (now, aggregation-in-order will not work)
|
2023-01-31 20:33:01 +00:00
|
|
|
void requestOnlyMergeForAggregateProjection(const DataStream & input_stream);
|
2023-02-21 20:01:59 +00:00
|
|
|
/// When we apply aggregate projection (which is partial), this step should be replaced to AggregatingProjection.
|
|
|
|
/// Argument input_stream would be the second input (from projection).
|
2023-02-01 17:55:06 +00:00
|
|
|
std::unique_ptr<AggregatingProjectionStep> convertToAggregatingProjection(const DataStream & input_stream) const;
|
2023-01-31 20:33:01 +00:00
|
|
|
|
2020-06-17 14:21:48 +00:00
|
|
|
private:
|
2022-06-27 11:16:52 +00:00
|
|
|
void updateOutputStream() override;
|
|
|
|
|
2020-06-18 18:29:33 +00:00
|
|
|
Aggregator::Params params;
|
2022-05-05 13:56:16 +00:00
|
|
|
GroupingSetsParamsList grouping_sets_params;
|
2020-06-18 18:29:33 +00:00
|
|
|
bool final;
|
2020-06-17 14:21:48 +00:00
|
|
|
size_t max_block_size;
|
2021-10-19 13:36:31 +00:00
|
|
|
size_t aggregation_in_order_max_block_bytes;
|
2020-06-17 14:21:48 +00:00
|
|
|
size_t merge_threads;
|
|
|
|
size_t temporary_data_merge_threads;
|
2023-01-05 19:22:42 +00:00
|
|
|
bool skip_merging = false; // if we aggregate partitioned data merging is not needed
|
2020-06-17 14:21:48 +00:00
|
|
|
|
|
|
|
bool storage_has_evenly_distributed_read;
|
2022-06-30 15:14:26 +00:00
|
|
|
bool group_by_use_nulls;
|
2020-06-17 14:21:48 +00:00
|
|
|
|
2022-12-06 12:34:07 +00:00
|
|
|
/// Both sort descriptions are needed for aggregate-in-order optimisation.
|
|
|
|
/// Both sort descriptions are subset of GROUP BY key columns (or monotonic functions over it).
|
|
|
|
/// Sort description for merging is a sort description for input and a prefix of group_by_sort_description.
|
|
|
|
/// group_by_sort_description contains all GROUP BY keys and is used for final merging of aggregated data.
|
2022-11-23 18:38:12 +00:00
|
|
|
SortDescription sort_description_for_merging;
|
2020-06-17 14:21:48 +00:00
|
|
|
SortDescription group_by_sort_description;
|
2020-06-25 09:39:17 +00:00
|
|
|
|
2022-11-27 23:41:31 +00:00
|
|
|
/// These settings are used to determine if we should resize pipeline to 1 at the end.
|
2023-01-22 19:39:24 +00:00
|
|
|
const bool should_produce_results_in_order_of_bucket_number;
|
2022-11-27 23:41:31 +00:00
|
|
|
bool memory_bound_merging_of_aggregation_results_enabled;
|
2023-02-14 13:02:54 +00:00
|
|
|
bool explicit_sorting_required_for_aggregation_in_order;
|
2022-06-28 23:16:30 +00:00
|
|
|
|
2020-06-25 09:39:17 +00:00
|
|
|
Processors aggregating_in_order;
|
|
|
|
Processors aggregating_sorted;
|
|
|
|
Processors finalizing;
|
|
|
|
|
|
|
|
Processors aggregating;
|
2020-06-17 14:21:48 +00:00
|
|
|
};
|
|
|
|
|
2023-02-01 17:55:06 +00:00
|
|
|
class AggregatingProjectionStep : public IQueryPlanStep
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
AggregatingProjectionStep(
|
|
|
|
DataStreams input_streams_,
|
|
|
|
Aggregator::Params params_,
|
|
|
|
bool final_,
|
|
|
|
size_t merge_threads_,
|
|
|
|
size_t temporary_data_merge_threads_
|
|
|
|
);
|
|
|
|
|
|
|
|
String getName() const override { return "AggregatingProjection"; }
|
|
|
|
QueryPipelineBuilderPtr updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings &) override;
|
|
|
|
|
|
|
|
private:
|
|
|
|
Aggregator::Params params;
|
|
|
|
bool final;
|
|
|
|
size_t merge_threads;
|
|
|
|
size_t temporary_data_merge_threads;
|
|
|
|
|
|
|
|
Processors aggregating;
|
|
|
|
};
|
|
|
|
|
2020-06-17 14:21:48 +00:00
|
|
|
}
|