ClickHouse/src/Processors/QueryPlan/AggregatingStep.h

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

137 lines
5.2 KiB
C++
Raw Normal View History

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>
#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;
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
Names used_keys;
Names missing_keys;
2022-05-10 16:12:03 +00:00
};
using GroupingSetsParamsList = std::vector<GroupingSetsParams>;
Block appendGroupingSetColumn(Block header);
Block generateOutputHeader(const Block & input_header, const Names & keys, bool use_nulls);
class AggregatingProjectionStep;
/// Aggregation. See AggregatingTransform.
2020-06-17 14:21:48 +00:00
class AggregatingStep : public ITransformingStep
{
public:
AggregatingStep(
const DataStream & input_stream_,
Aggregator::Params params_,
2022-05-05 13:56:16 +00:00
GroupingSetsParamsList grouping_sets_params_,
bool final_,
2020-06-17 14:21:48 +00:00
size_t max_block_size_,
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_,
SortDescription group_by_sort_description_,
bool should_produce_results_in_order_of_bucket_number_,
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"; }
void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
2020-06-17 14:21:48 +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;
2021-02-11 08:49:12 +00:00
const Aggregator::Params & getParams() const { return params; }
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(); }
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_);
bool memoryBoundMergingWillBeUsed() const;
2023-01-05 19:22:42 +00:00
void skipMerging() { skip_merging = true; }
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)
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).
std::unique_ptr<AggregatingProjectionStep> convertToAggregatingProjection(const DataStream & input_stream) const;
2020-06-17 14:21:48 +00:00
private:
void updateOutputStream() override;
Aggregator::Params params;
2022-05-05 13:56:16 +00:00
GroupingSetsParamsList grouping_sets_params;
bool final;
2020-06-17 14:21:48 +00:00
size_t max_block_size;
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
/// 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;
bool memory_bound_merging_of_aggregation_results_enabled;
bool explicit_sorting_required_for_aggregation_in_order;
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
};
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
}