mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 01:22:04 +00:00
Add TotalsHavingStep.
This commit is contained in:
parent
1ad4f2c0fd
commit
c5d9379df0
@ -89,6 +89,7 @@
|
||||
#include <Processors/QueryPlan/AddingDelayedStreamStep.h>
|
||||
#include <Processors/QueryPlan/AggregatingStep.h>
|
||||
#include <Processors/QueryPlan/CreatingSetsStep.h>
|
||||
#include <Processors/QueryPlan/TotalsHavingStep.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -1508,12 +1509,13 @@ void InterpreterSelectQuery::executeTotalsAndHaving(QueryPipeline & pipeline, bo
|
||||
{
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
|
||||
auto totals_having = std::make_shared<TotalsHavingTransform>(
|
||||
pipeline.getHeader(), overflow_row, expression,
|
||||
TotalsHavingStep totals_having_step(
|
||||
DataStream{.header = pipeline.getHeader()},
|
||||
overflow_row, expression,
|
||||
has_having ? getSelectQuery().having()->getColumnName() : "",
|
||||
settings.totals_mode, settings.totals_auto_threshold, final);
|
||||
|
||||
pipeline.addTotalsHavingTransform(std::move(totals_having));
|
||||
totals_having_step.transformPipeline(pipeline);
|
||||
}
|
||||
|
||||
|
||||
|
38
src/Processors/QueryPlan/TotalsHavingStep.cpp
Normal file
38
src/Processors/QueryPlan/TotalsHavingStep.cpp
Normal file
@ -0,0 +1,38 @@
|
||||
#include <Processors/QueryPlan/TotalsHavingStep.h>
|
||||
#include <Processors/Transforms/DistinctTransform.h>
|
||||
#include <Processors/QueryPipeline.h>
|
||||
#include <Processors/Transforms/TotalsHavingTransform.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
TotalsHavingStep::TotalsHavingStep(
|
||||
const DataStream & input_stream_,
|
||||
bool overflow_row_,
|
||||
const ExpressionActionsPtr & expression_,
|
||||
const std::string & filter_column_,
|
||||
TotalsMode totals_mode_,
|
||||
double auto_include_threshold_,
|
||||
bool final_)
|
||||
: ITransformingStep(
|
||||
input_stream_,
|
||||
DataStream{.header = TotalsHavingTransform::transformHeader(input_stream_.header, expression_, final_)})
|
||||
, overflow_row(overflow_row_)
|
||||
, expression(expression_)
|
||||
, filter_column_name(filter_column_)
|
||||
, totals_mode(totals_mode_)
|
||||
, auto_include_threshold(auto_include_threshold_)
|
||||
, final(final_)
|
||||
{
|
||||
}
|
||||
|
||||
void TotalsHavingStep::transformPipeline(QueryPipeline & pipeline)
|
||||
{
|
||||
auto totals_having = std::make_shared<TotalsHavingTransform>(
|
||||
pipeline.getHeader(), overflow_row, expression,
|
||||
filter_column_name, totals_mode, auto_include_threshold, final);
|
||||
|
||||
pipeline.addTotalsHavingTransform(std::move(totals_having));
|
||||
}
|
||||
|
||||
}
|
39
src/Processors/QueryPlan/TotalsHavingStep.h
Normal file
39
src/Processors/QueryPlan/TotalsHavingStep.h
Normal file
@ -0,0 +1,39 @@
|
||||
#pragma once
|
||||
#include <Processors/QueryPlan/ITransformingStep.h>
|
||||
#include <DataStreams/SizeLimits.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ExpressionActions;
|
||||
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
||||
|
||||
enum class TotalsMode;
|
||||
|
||||
class TotalsHavingStep : public ITransformingStep
|
||||
{
|
||||
public:
|
||||
TotalsHavingStep(
|
||||
const DataStream & input_stream_,
|
||||
bool overflow_row_,
|
||||
const ExpressionActionsPtr & expression_,
|
||||
const std::string & filter_column_,
|
||||
TotalsMode totals_mode_,
|
||||
double auto_include_threshold_,
|
||||
bool final_);
|
||||
|
||||
String getName() const override { return "TotalsHaving"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
|
||||
private:
|
||||
bool overflow_row;
|
||||
ExpressionActionsPtr expression;
|
||||
String filter_column_name;
|
||||
TotalsMode totals_mode;
|
||||
double auto_include_threshold;
|
||||
bool final;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -32,7 +32,7 @@ void finalizeChunk(Chunk & chunk)
|
||||
chunk.setColumns(std::move(columns), num_rows);
|
||||
}
|
||||
|
||||
static Block createOutputHeader(Block block, const ExpressionActionsPtr & expression, bool final)
|
||||
Block TotalsHavingTransform::transformHeader(Block block, const ExpressionActionsPtr & expression, bool final)
|
||||
{
|
||||
if (final)
|
||||
finalizeBlock(block);
|
||||
@ -51,7 +51,7 @@ TotalsHavingTransform::TotalsHavingTransform(
|
||||
TotalsMode totals_mode_,
|
||||
double auto_include_threshold_,
|
||||
bool final_)
|
||||
: ISimpleTransform(header, createOutputHeader(header, expression_, final_), true)
|
||||
: ISimpleTransform(header, transformHeader(header, expression_, final_), true)
|
||||
, overflow_row(overflow_row_)
|
||||
, expression(expression_)
|
||||
, filter_column_name(filter_column_)
|
||||
|
@ -37,6 +37,8 @@ public:
|
||||
Status prepare() override;
|
||||
void work() override;
|
||||
|
||||
static Block transformHeader(Block block, const ExpressionActionsPtr & expression, bool final);
|
||||
|
||||
protected:
|
||||
void transform(Chunk & chunk) override;
|
||||
|
||||
|
@ -155,6 +155,7 @@ SRCS(
|
||||
QueryPlan/ReadFromPreparedSource.cpp
|
||||
QueryPlan/ReadFromStorageStep.cpp
|
||||
QueryPlan/ReadNothingStep.cpp
|
||||
QueryPlan/TotalsHavingStep
|
||||
QueryPlan/QueryPlan.cpp
|
||||
)
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user