mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Updated ExpressionActionSettings
This commit is contained in:
parent
9699299083
commit
c50909468f
@ -102,7 +102,7 @@ class IColumn;
|
||||
\
|
||||
M(Bool, allow_suspicious_low_cardinality_types, false, "In CREATE TABLE statement allows specifying LowCardinality modifier for types of small fixed size (8 or less). Enabling this may increase merge times and memory consumption.", 0) \
|
||||
M(Bool, compile_expressions, true, "Compile some scalar functions and operators to native code.", 0) \
|
||||
M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \
|
||||
M(UInt64, min_count_to_compile_expression, 0, "The number of identical expressions before they are JIT-compiled", 0) \
|
||||
M(UInt64, group_by_two_level_threshold, 100000, "From what number of keys, a two-level aggregation starts. 0 - the threshold is not set.", 0) \
|
||||
M(UInt64, group_by_two_level_threshold_bytes, 50000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.", 0) \
|
||||
M(Bool, distributed_aggregation_memory_efficient, true, "Is the memory-saving mode of distributed aggregation enabled.", 0) \
|
||||
|
@ -15,7 +15,7 @@ AddingDefaultBlockOutputStream::AddingDefaultBlockOutputStream(
|
||||
: output(output_), header(header_)
|
||||
{
|
||||
auto dag = addMissingDefaults(header_, output->getHeader().getNamesAndTypesList(), columns_, context_, null_as_default_);
|
||||
adding_defaults_actions = std::make_shared<ExpressionActions>(std::move(dag), ExpressionActionsSettings::fromContext(context_), CompileExpressions::yes);
|
||||
adding_defaults_actions = std::make_shared<ExpressionActions>(std::move(dag), ExpressionActionsSettings::fromContext(context_, CompileExpressions::yes));
|
||||
}
|
||||
|
||||
void AddingDefaultBlockOutputStream::write(const Block & block)
|
||||
|
@ -174,7 +174,7 @@ Block AddingDefaultsBlockInputStream::readImpl()
|
||||
auto dag = evaluateMissingDefaults(evaluate_block, header.getNamesAndTypesList(), columns, context, false);
|
||||
if (dag)
|
||||
{
|
||||
auto actions = std::make_shared<ExpressionActions>(std::move(dag), ExpressionActionsSettings::fromContext(context), CompileExpressions::yes);
|
||||
auto actions = std::make_shared<ExpressionActions>(std::move(dag), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes));
|
||||
actions->execute(evaluate_block);
|
||||
}
|
||||
|
||||
|
@ -1015,7 +1015,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
||||
|
||||
auto lambda_actions = std::make_shared<ExpressionActions>(
|
||||
lambda_dag,
|
||||
ExpressionActionsSettings::fromContext(data.getContext()), CompileExpressions::yes);
|
||||
ExpressionActionsSettings::fromContext(data.getContext(), CompileExpressions::yes));
|
||||
|
||||
DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type;
|
||||
|
||||
|
@ -45,16 +45,13 @@ namespace ErrorCodes
|
||||
|
||||
ExpressionActions::~ExpressionActions() = default;
|
||||
|
||||
ExpressionActions::ExpressionActions(
|
||||
ActionsDAGPtr actions_dag_,
|
||||
const ExpressionActionsSettings & settings_,
|
||||
CompileExpressions compile_expressions [[maybe_unused]])
|
||||
ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings_)
|
||||
: settings(settings_)
|
||||
{
|
||||
actions_dag = actions_dag_->clone();
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
if (settings.compile_expressions && compile_expressions == CompileExpressions::yes)
|
||||
if (settings.can_compile_expressions && settings.compile_expressions == CompileExpressions::yes)
|
||||
actions_dag->compileExpressions(settings.min_count_to_compile_expression);
|
||||
#endif
|
||||
|
||||
|
@ -30,12 +30,6 @@ using ArrayJoinActionPtr = std::shared_ptr<ArrayJoinAction>;
|
||||
class ExpressionActions;
|
||||
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
||||
|
||||
enum class CompileExpressions: uint8_t
|
||||
{
|
||||
no = 0,
|
||||
yes = 1,
|
||||
};
|
||||
|
||||
/// Sequence of actions on the block.
|
||||
/// Is used to calculate expressions.
|
||||
///
|
||||
@ -89,7 +83,7 @@ private:
|
||||
public:
|
||||
ExpressionActions() = delete;
|
||||
~ExpressionActions();
|
||||
explicit ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings_ = {}, CompileExpressions compile_expressions = CompileExpressions::no);
|
||||
explicit ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings_ = {});
|
||||
ExpressionActions(const ExpressionActions &) = default;
|
||||
ExpressionActions & operator=(const ExpressionActions &) = default;
|
||||
|
||||
|
@ -6,20 +6,21 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ExpressionActionsSettings ExpressionActionsSettings::fromSettings(const Settings & from)
|
||||
ExpressionActionsSettings ExpressionActionsSettings::fromSettings(const Settings & from, CompileExpressions compile_expressions)
|
||||
{
|
||||
ExpressionActionsSettings settings;
|
||||
settings.compile_expressions = from.compile_expressions;
|
||||
settings.can_compile_expressions = from.compile_expressions;
|
||||
settings.min_count_to_compile_expression = from.min_count_to_compile_expression;
|
||||
settings.max_temporary_columns = from.max_temporary_columns;
|
||||
settings.max_temporary_non_const_columns = from.max_temporary_non_const_columns;
|
||||
settings.compile_expressions = compile_expressions;
|
||||
|
||||
return settings;
|
||||
}
|
||||
|
||||
ExpressionActionsSettings ExpressionActionsSettings::fromContext(ContextPtr from)
|
||||
ExpressionActionsSettings ExpressionActionsSettings::fromContext(ContextPtr from, CompileExpressions compile_expressions)
|
||||
{
|
||||
return fromSettings(from->getSettingsRef());
|
||||
return fromSettings(from->getSettingsRef(), compile_expressions);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -9,16 +9,24 @@ namespace DB
|
||||
|
||||
struct Settings;
|
||||
|
||||
enum class CompileExpressions: uint8_t
|
||||
{
|
||||
no = 0,
|
||||
yes = 1,
|
||||
};
|
||||
|
||||
struct ExpressionActionsSettings
|
||||
{
|
||||
bool compile_expressions = false;
|
||||
bool can_compile_expressions = false;
|
||||
size_t min_count_to_compile_expression = 0;
|
||||
|
||||
size_t max_temporary_columns = 0;
|
||||
size_t max_temporary_non_const_columns = 0;
|
||||
|
||||
static ExpressionActionsSettings fromSettings(const Settings & from);
|
||||
static ExpressionActionsSettings fromContext(ContextPtr from);
|
||||
CompileExpressions compile_expressions = CompileExpressions::no;
|
||||
|
||||
static ExpressionActionsSettings fromSettings(const Settings & from, CompileExpressions compile_expressions = CompileExpressions::no);
|
||||
static ExpressionActionsSettings fromContext(ContextPtr from, CompileExpressions compile_expressions = CompileExpressions::no);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1028,7 +1028,7 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain
|
||||
auto actions_dag = std::make_shared<ActionsDAG>(columns_after_join);
|
||||
getRootActions(child, only_types, actions_dag);
|
||||
group_by_elements_actions.emplace_back(
|
||||
std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(getContext()), CompileExpressions::yes));
|
||||
std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)));
|
||||
}
|
||||
}
|
||||
|
||||
@ -1187,7 +1187,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai
|
||||
auto actions_dag = std::make_shared<ActionsDAG>(columns_after_join);
|
||||
getRootActions(child, only_types, actions_dag);
|
||||
order_by_elements_actions.emplace_back(
|
||||
std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(getContext()), CompileExpressions::yes));
|
||||
std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)));
|
||||
}
|
||||
}
|
||||
|
||||
@ -1348,7 +1348,7 @@ ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool project_r
|
||||
ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool project_result, CompileExpressions compile_expressions)
|
||||
{
|
||||
return std::make_shared<ExpressionActions>(
|
||||
getActionsDAG(add_aliases, project_result), ExpressionActionsSettings::fromContext(getContext()), compile_expressions);
|
||||
getActionsDAG(add_aliases, project_result), ExpressionActionsSettings::fromContext(getContext(), compile_expressions));
|
||||
}
|
||||
|
||||
ExpressionActionsPtr ExpressionAnalyzer::getConstActions(const ColumnsWithTypeAndName & constant_inputs)
|
||||
|
@ -325,7 +325,7 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
res.pipeline.getHeader().getColumnsWithTypeAndName(),
|
||||
header.getColumnsWithTypeAndName(),
|
||||
ActionsDAG::MatchColumnsMode::Position);
|
||||
auto actions = std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(getContext()), CompileExpressions::yes);
|
||||
auto actions = std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes));
|
||||
|
||||
res.pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr
|
||||
{
|
||||
|
@ -1897,17 +1897,17 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
|
||||
|
||||
if (prewhere_info)
|
||||
{
|
||||
auto actions_settings = ExpressionActionsSettings::fromContext(context);
|
||||
auto actions_settings = ExpressionActionsSettings::fromContext(context, CompileExpressions::yes);
|
||||
|
||||
query_info.prewhere_info = std::make_shared<PrewhereInfo>();
|
||||
query_info.prewhere_info->prewhere_actions = std::make_shared<ExpressionActions>(prewhere_info->prewhere_actions, actions_settings, CompileExpressions::yes);
|
||||
query_info.prewhere_info->prewhere_actions = std::make_shared<ExpressionActions>(prewhere_info->prewhere_actions, actions_settings);
|
||||
|
||||
if (prewhere_info->row_level_filter_actions)
|
||||
query_info.prewhere_info->row_level_filter = std::make_shared<ExpressionActions>(prewhere_info->row_level_filter_actions, actions_settings, CompileExpressions::yes);
|
||||
query_info.prewhere_info->row_level_filter = std::make_shared<ExpressionActions>(prewhere_info->row_level_filter_actions, actions_settings);
|
||||
if (prewhere_info->alias_actions)
|
||||
query_info.prewhere_info->alias_actions = std::make_shared<ExpressionActions>(prewhere_info->alias_actions, actions_settings, CompileExpressions::yes);
|
||||
query_info.prewhere_info->alias_actions = std::make_shared<ExpressionActions>(prewhere_info->alias_actions, actions_settings);
|
||||
if (prewhere_info->remove_columns_actions)
|
||||
query_info.prewhere_info->remove_columns_actions = std::make_shared<ExpressionActions>(prewhere_info->remove_columns_actions, actions_settings, CompileExpressions::yes);
|
||||
query_info.prewhere_info->remove_columns_actions = std::make_shared<ExpressionActions>(prewhere_info->remove_columns_actions, actions_settings);
|
||||
|
||||
query_info.prewhere_info->prewhere_column_name = prewhere_info->prewhere_column_name;
|
||||
query_info.prewhere_info->remove_prewhere_column = prewhere_info->remove_prewhere_column;
|
||||
|
@ -60,7 +60,8 @@ void ArrayJoinStep::transformPipeline(QueryPipeline & pipeline, const BuildQuery
|
||||
pipeline.getHeader().getColumnsWithTypeAndName(),
|
||||
res_header.getColumnsWithTypeAndName(),
|
||||
ActionsDAG::MatchColumnsMode::Name);
|
||||
auto actions = std::make_shared<ExpressionActions>(actions_dag, settings.getActionsSettings(), CompileExpressions::yes);
|
||||
|
||||
auto actions = std::make_shared<ExpressionActions>(actions_dag, settings.getActionsSettings());
|
||||
|
||||
pipeline.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
|
@ -9,7 +9,7 @@ namespace DB
|
||||
BuildQueryPipelineSettings BuildQueryPipelineSettings::fromSettings(const Settings & from)
|
||||
{
|
||||
BuildQueryPipelineSettings settings;
|
||||
settings.actions_settings = ExpressionActionsSettings::fromSettings(from);
|
||||
settings.actions_settings = ExpressionActionsSettings::fromSettings(from, CompileExpressions::yes);
|
||||
return settings;
|
||||
}
|
||||
|
||||
|
@ -54,7 +54,7 @@ void ExpressionStep::updateInputStream(DataStream input_stream, bool keep_header
|
||||
|
||||
void ExpressionStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings)
|
||||
{
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag, settings.getActionsSettings(), CompileExpressions::yes);
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag, settings.getActionsSettings());
|
||||
|
||||
pipeline.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
@ -67,7 +67,7 @@ void ExpressionStep::transformPipeline(QueryPipeline & pipeline, const BuildQuer
|
||||
pipeline.getHeader().getColumnsWithTypeAndName(),
|
||||
output_stream->header.getColumnsWithTypeAndName(),
|
||||
ActionsDAG::MatchColumnsMode::Name);
|
||||
auto convert_actions = std::make_shared<ExpressionActions>(convert_actions_dag, settings.getActionsSettings(), CompileExpressions::yes);
|
||||
auto convert_actions = std::make_shared<ExpressionActions>(convert_actions_dag, settings.getActionsSettings());
|
||||
|
||||
pipeline.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
|
@ -67,7 +67,7 @@ void FilterStep::updateInputStream(DataStream input_stream, bool keep_header)
|
||||
|
||||
void FilterStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings)
|
||||
{
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag, settings.getActionsSettings(), CompileExpressions::yes);
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag, settings.getActionsSettings());
|
||||
|
||||
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type)
|
||||
{
|
||||
@ -81,7 +81,7 @@ void FilterStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPip
|
||||
pipeline.getHeader().getColumnsWithTypeAndName(),
|
||||
output_stream->header.getColumnsWithTypeAndName(),
|
||||
ActionsDAG::MatchColumnsMode::Name);
|
||||
auto convert_actions = std::make_shared<ExpressionActions>(convert_actions_dag, settings.getActionsSettings(), CompileExpressions::yes);
|
||||
auto convert_actions = std::make_shared<ExpressionActions>(convert_actions_dag, settings.getActionsSettings());
|
||||
|
||||
pipeline.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
|
@ -51,7 +51,7 @@ TotalsHavingStep::TotalsHavingStep(
|
||||
|
||||
void TotalsHavingStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings)
|
||||
{
|
||||
auto expression_actions = actions_dag ? std::make_shared<ExpressionActions>(actions_dag, settings.getActionsSettings(), CompileExpressions::yes) : nullptr;
|
||||
auto expression_actions = actions_dag ? std::make_shared<ExpressionActions>(actions_dag, settings.getActionsSettings()) : nullptr;
|
||||
|
||||
auto totals_having = std::make_shared<TotalsHavingTransform>(
|
||||
pipeline.getHeader(),
|
||||
|
@ -415,8 +415,7 @@ Pipe StorageMerge::createSources(
|
||||
auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column));
|
||||
auto adding_column_actions = std::make_shared<ExpressionActions>(
|
||||
std::move(adding_column_dag),
|
||||
ExpressionActionsSettings::fromContext(modified_context),
|
||||
CompileExpressions::yes);
|
||||
ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes));
|
||||
|
||||
pipe.addSimpleTransform([&](const Block & stream_header)
|
||||
{
|
||||
@ -560,7 +559,7 @@ void StorageMerge::convertingSourceStream(
|
||||
pipe.getHeader().getColumnsWithTypeAndName(),
|
||||
header.getColumnsWithTypeAndName(),
|
||||
ActionsDAG::MatchColumnsMode::Name);
|
||||
auto convert_actions = std::make_shared<ExpressionActions>(convert_actions_dag, ExpressionActionsSettings::fromContext(local_context), CompileExpressions::yes);
|
||||
auto convert_actions = std::make_shared<ExpressionActions>(convert_actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes));
|
||||
|
||||
pipe.addSimpleTransform([&](const Block & stream_header)
|
||||
{
|
||||
|
@ -96,8 +96,7 @@ public:
|
||||
ActionsDAG::MatchColumnsMode::Name);
|
||||
auto convert_actions = std::make_shared<ExpressionActions>(
|
||||
convert_actions_dag,
|
||||
ExpressionActionsSettings::fromSettings(context->getSettingsRef()),
|
||||
CompileExpressions::yes);
|
||||
ExpressionActionsSettings::fromSettings(context->getSettingsRef(), CompileExpressions::yes));
|
||||
|
||||
pipe.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
|
Loading…
Reference in New Issue
Block a user