mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
better ExpressionAction::execute()
This commit is contained in:
parent
bfe30a9723
commit
2d7d389b77
@ -323,8 +323,20 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings,
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void ExpressionAction::execute(Block & block, ExtraBlockPtr & not_processed) const
|
||||||
|
{
|
||||||
|
switch (type)
|
||||||
|
{
|
||||||
|
case JOIN:
|
||||||
|
join->joinBlock(block, not_processed);
|
||||||
|
break;
|
||||||
|
|
||||||
void ExpressionAction::execute(Block & block, bool dry_run, ExtraBlockPtr & not_processed) const
|
default:
|
||||||
|
throw Exception("Unexpected expression call", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void ExpressionAction::execute(Block & block, bool dry_run) const
|
||||||
{
|
{
|
||||||
size_t input_rows_count = block.rows();
|
size_t input_rows_count = block.rows();
|
||||||
|
|
||||||
@ -362,10 +374,7 @@ void ExpressionAction::execute(Block & block, bool dry_run, ExtraBlockPtr & not_
|
|||||||
}
|
}
|
||||||
|
|
||||||
case JOIN:
|
case JOIN:
|
||||||
{
|
throw Exception("Unexpected JOIN expression call", ErrorCodes::LOGICAL_ERROR);
|
||||||
join->joinBlock(block, not_processed);
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
|
|
||||||
case PROJECT:
|
case PROJECT:
|
||||||
{
|
{
|
||||||
@ -681,7 +690,7 @@ void ExpressionActions::execute(Block & block, ExtraBlockPtr & not_processed) co
|
|||||||
if (actions.size() != 1)
|
if (actions.size() != 1)
|
||||||
throw Exception("Continuation over multiple expressions is not supported", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Continuation over multiple expressions is not supported", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
actions[0].execute(block, false, not_processed);
|
actions[0].execute(block, not_processed);
|
||||||
checkLimits(block);
|
checkLimits(block);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -139,13 +139,8 @@ private:
|
|||||||
void executeOnTotals(Block & block) const;
|
void executeOnTotals(Block & block) const;
|
||||||
|
|
||||||
/// Executes action on block (modify it). Block could be splitted in case of JOIN. Then not_processed block is created.
|
/// Executes action on block (modify it). Block could be splitted in case of JOIN. Then not_processed block is created.
|
||||||
void execute(Block & block, bool dry_run, ExtraBlockPtr & not_processed) const;
|
void execute(Block & block, ExtraBlockPtr & not_processed) const;
|
||||||
|
void execute(Block & block, bool dry_run) const;
|
||||||
void execute(Block & block, bool dry_run) const
|
|
||||||
{
|
|
||||||
ExtraBlockPtr extra;
|
|
||||||
execute(block, dry_run, extra);
|
|
||||||
}
|
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
@ -32,6 +32,7 @@
|
|||||||
#include <Processors/Pipe.h>
|
#include <Processors/Pipe.h>
|
||||||
#include <Processors/Sources/SourceFromInputStream.h>
|
#include <Processors/Sources/SourceFromInputStream.h>
|
||||||
#include <Processors/Transforms/ExpressionTransform.h>
|
#include <Processors/Transforms/ExpressionTransform.h>
|
||||||
|
#include <Processors/Transforms/InflatingExpressionTransform.h>
|
||||||
#include <Processors/Transforms/AggregatingTransform.h>
|
#include <Processors/Transforms/AggregatingTransform.h>
|
||||||
#include <Processors/QueryPlan/ReadFromStorageStep.h>
|
#include <Processors/QueryPlan/ReadFromStorageStep.h>
|
||||||
#include <Processors/QueryPlan/ExpressionStep.h>
|
#include <Processors/QueryPlan/ExpressionStep.h>
|
||||||
@ -873,7 +874,8 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
|
|||||||
Block join_result_sample;
|
Block join_result_sample;
|
||||||
JoinPtr join = expressions.join->getTableJoinAlgo();
|
JoinPtr join = expressions.join->getTableJoinAlgo();
|
||||||
|
|
||||||
join_result_sample = ExpressionTransform::transformHeader(query_plan.getCurrentDataStream().header, expressions.join);
|
join_result_sample = InflatingExpressionTransform::transformHeader(
|
||||||
|
query_plan.getCurrentDataStream().header, expressions.join);
|
||||||
|
|
||||||
QueryPlanStepPtr join_step = std::make_unique<InflatingExpressionStep>(
|
QueryPlanStepPtr join_step = std::make_unique<InflatingExpressionStep>(
|
||||||
query_plan.getCurrentDataStream(),
|
query_plan.getCurrentDataStream(),
|
||||||
|
@ -31,7 +31,7 @@ static void filterDistinctColumns(const Block & res_header, NameSet & distinct_c
|
|||||||
ExpressionStep::ExpressionStep(const DataStream & input_stream_, ExpressionActionsPtr expression_, bool default_totals_)
|
ExpressionStep::ExpressionStep(const DataStream & input_stream_, ExpressionActionsPtr expression_, bool default_totals_)
|
||||||
: ITransformingStep(
|
: ITransformingStep(
|
||||||
input_stream_,
|
input_stream_,
|
||||||
ExpressionTransform::transformHeader(input_stream_.header, expression_),
|
Transform::transformHeader(input_stream_.header, expression_),
|
||||||
getTraits(expression_))
|
getTraits(expression_))
|
||||||
, expression(std::move(expression_))
|
, expression(std::move(expression_))
|
||||||
, default_totals(default_totals_)
|
, default_totals(default_totals_)
|
||||||
@ -55,14 +55,14 @@ void ExpressionStep::transformPipeline(QueryPipeline & pipeline)
|
|||||||
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type)
|
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type)
|
||||||
{
|
{
|
||||||
bool on_totals = stream_type == QueryPipeline::StreamType::Totals;
|
bool on_totals = stream_type == QueryPipeline::StreamType::Totals;
|
||||||
return std::make_shared<ExpressionTransform>(header, expression, on_totals, add_default_totals);
|
return std::make_shared<Transform>(header, expression, on_totals, add_default_totals);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
InflatingExpressionStep::InflatingExpressionStep(const DataStream & input_stream_, ExpressionActionsPtr expression_, bool default_totals_)
|
InflatingExpressionStep::InflatingExpressionStep(const DataStream & input_stream_, ExpressionActionsPtr expression_, bool default_totals_)
|
||||||
: ITransformingStep(
|
: ITransformingStep(
|
||||||
input_stream_,
|
input_stream_,
|
||||||
ExpressionTransform::transformHeader(input_stream_.header, expression_),
|
Transform::transformHeader(input_stream_.header, expression_),
|
||||||
getTraits(expression_))
|
getTraits(expression_))
|
||||||
, expression(std::move(expression_))
|
, expression(std::move(expression_))
|
||||||
, default_totals(default_totals_)
|
, default_totals(default_totals_)
|
||||||
@ -84,7 +84,7 @@ void InflatingExpressionStep::transformPipeline(QueryPipeline & pipeline)
|
|||||||
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type)
|
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type)
|
||||||
{
|
{
|
||||||
bool on_totals = stream_type == QueryPipeline::StreamType::Totals;
|
bool on_totals = stream_type == QueryPipeline::StreamType::Totals;
|
||||||
return std::make_shared<InflatingExpressionTransform>(header, expression, on_totals, add_default_totals);
|
return std::make_shared<Transform>(header, expression, on_totals, add_default_totals);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -7,9 +7,14 @@ namespace DB
|
|||||||
class ExpressionActions;
|
class ExpressionActions;
|
||||||
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
||||||
|
|
||||||
|
class ExpressionTransform;
|
||||||
|
class InflatingExpressionTransform;
|
||||||
|
|
||||||
class ExpressionStep : public ITransformingStep
|
class ExpressionStep : public ITransformingStep
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
using Transform = ExpressionTransform;
|
||||||
|
|
||||||
explicit ExpressionStep(const DataStream & input_stream_, ExpressionActionsPtr expression_, bool default_totals_ = false);
|
explicit ExpressionStep(const DataStream & input_stream_, ExpressionActionsPtr expression_, bool default_totals_ = false);
|
||||||
String getName() const override { return "Expression"; }
|
String getName() const override { return "Expression"; }
|
||||||
|
|
||||||
@ -24,6 +29,8 @@ private:
|
|||||||
class InflatingExpressionStep : public ITransformingStep
|
class InflatingExpressionStep : public ITransformingStep
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
using Transform = InflatingExpressionTransform;
|
||||||
|
|
||||||
explicit InflatingExpressionStep(const DataStream & input_stream_, ExpressionActionsPtr expression_, bool default_totals_ = false);
|
explicit InflatingExpressionStep(const DataStream & input_stream_, ExpressionActionsPtr expression_, bool default_totals_ = false);
|
||||||
String getName() const override { return "Expression"; }
|
String getName() const override { return "Expression"; }
|
||||||
|
|
||||||
|
@ -5,9 +5,10 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
static Block transformHeader(Block header, const ExpressionActionsPtr & expression)
|
Block InflatingExpressionTransform::transformHeader(Block header, const ExpressionActionsPtr & expression)
|
||||||
{
|
{
|
||||||
expression->execute(header, true);
|
ExtraBlockPtr tmp;
|
||||||
|
expression->execute(header, tmp);
|
||||||
return header;
|
return header;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -16,6 +16,8 @@ public:
|
|||||||
|
|
||||||
String getName() const override { return "InflatingExpressionTransform"; }
|
String getName() const override { return "InflatingExpressionTransform"; }
|
||||||
|
|
||||||
|
static Block transformHeader(Block header, const ExpressionActionsPtr & expression);
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
void transform(Chunk & chunk) override;
|
void transform(Chunk & chunk) override;
|
||||||
bool needInputData() const override { return !not_processed; }
|
bool needInputData() const override { return !not_processed; }
|
||||||
|
Loading…
Reference in New Issue
Block a user