Merge pull request #24273 from kitaisreal/expression-actions-compile-only-necessary-places

ExpressionActions compile only necessary places
This commit is contained in:
Maksim Kita 2021-05-20 10:53:30 +03:00 committed by GitHub
commit b0476c1fa2
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
24 changed files with 70 additions and 43 deletions

View File

@ -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) \

View File

@ -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_));
adding_defaults_actions = std::make_shared<ExpressionActions>(std::move(dag), ExpressionActionsSettings::fromContext(context_, CompileExpressions::yes));
}
void AddingDefaultBlockOutputStream::write(const Block & block)

View File

@ -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));
auto actions = std::make_shared<ExpressionActions>(std::move(dag), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes));
actions->execute(evaluate_block);
}

View File

@ -293,6 +293,17 @@ NamesAndTypesList ActionsDAG::getRequiredColumns() const
return result;
}
Names ActionsDAG::getRequiredColumnsNames() const
{
Names result;
result.reserve(inputs.size());
for (const auto & input : inputs)
result.emplace_back(input->result_name);
return result;
}
ColumnsWithTypeAndName ActionsDAG::getResultColumns() const
{
ColumnsWithTypeAndName result;

View File

@ -121,6 +121,7 @@ public:
const NodeRawConstPtrs & getInputs() const { return inputs; }
NamesAndTypesList getRequiredColumns() const;
Names getRequiredColumnsNames() const;
ColumnsWithTypeAndName getResultColumns() const;
NamesAndTypesList getNamesAndTypesList() const;

View File

@ -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()));
ExpressionActionsSettings::fromContext(data.getContext(), CompileExpressions::yes));
DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type;

View File

@ -51,7 +51,7 @@ ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_, const Expressio
actions_dag = actions_dag_->clone();
#if USE_EMBEDDED_COMPILER
if (settings.compile_expressions)
if (settings.can_compile_expressions && settings.compile_expressions == CompileExpressions::yes)
actions_dag->compileExpressions(settings.min_count_to_compile_expression);
#endif

View File

@ -30,7 +30,6 @@ using ArrayJoinActionPtr = std::shared_ptr<ArrayJoinAction>;
class ExpressionActions;
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
/// Sequence of actions on the block.
/// Is used to calculate expressions.
///

View File

@ -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);
}
}

View File

@ -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);
};
}

View File

@ -909,8 +909,8 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere(
auto tmp_actions_dag = std::make_shared<ActionsDAG>(sourceColumns());
getRootActions(select_query->prewhere(), only_types, tmp_actions_dag);
tmp_actions_dag->removeUnusedActions(NameSet{prewhere_column_name});
auto tmp_actions = std::make_shared<ExpressionActions>(tmp_actions_dag, ExpressionActionsSettings::fromContext(getContext()));
auto required_columns = tmp_actions->getRequiredColumns();
auto required_columns = tmp_actions_dag->getRequiredColumnsNames();
NameSet required_source_columns(required_columns.begin(), required_columns.end());
required_source_columns.insert(first_action_names.begin(), first_action_names.end());
@ -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())));
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())));
std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)));
}
}
@ -1345,13 +1345,12 @@ ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool project_r
return actions_dag;
}
ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool project_result)
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()));
getActionsDAG(add_aliases, project_result), ExpressionActionsSettings::fromContext(getContext(), compile_expressions));
}
ExpressionActionsPtr ExpressionAnalyzer::getConstActions(const ColumnsWithTypeAndName & constant_inputs)
{
auto actions = std::make_shared<ActionsDAG>(constant_inputs);

View File

@ -112,7 +112,7 @@ public:
/// If also project_result, than only aliases remain in the output block.
/// Otherwise, only temporary columns will be deleted from the block.
ActionsDAGPtr getActionsDAG(bool add_aliases, bool project_result = true);
ExpressionActionsPtr getActions(bool add_aliases, bool project_result = true);
ExpressionActionsPtr getActions(bool add_aliases, bool project_result = true, CompileExpressions compile_expressions = CompileExpressions::no);
/// Actions that can be performed on an empty block: adding constants and applying functions that depend only on constants.
/// Does not execute subqueries.

View File

@ -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()));
auto actions = std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes));
res.pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr
{

View File

@ -637,9 +637,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
if (analysis_result.prewhere_info)
{
ExpressionActions(
analysis_result.prewhere_info->prewhere_actions,
ExpressionActionsSettings::fromContext(context)).execute(header);
header = analysis_result.prewhere_info->prewhere_actions->updateHeader(header);
if (analysis_result.prewhere_info->remove_prewhere_column)
header.erase(analysis_result.prewhere_info->prewhere_column_name);
}
@ -1901,11 +1899,12 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
// TODO figure out how to make set for projections
query_info.sets = query_analyzer->getPreparedSets();
auto actions_settings = ExpressionActionsSettings::fromContext(context);
auto & prewhere_info = analysis_result.prewhere_info;
if (prewhere_info)
{
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);

View File

@ -124,8 +124,8 @@ static NamesAndTypesList getNames(const ASTFunction & expr, ContextPtr context,
ASTPtr temp_ast = expr.clone();
auto syntax = TreeRewriter(context).analyze(temp_ast, columns);
auto expression = ExpressionAnalyzer(temp_ast, syntax, context).getActions(false);
return expression->getRequiredColumnsWithTypes();
auto required_columns = ExpressionAnalyzer(temp_ast, syntax, context).getActionsDAG(false)->getRequiredColumns();
return required_columns;
}
static NamesAndTypesList modifyPrimaryKeysToNonNullable(const NamesAndTypesList & primary_keys, NamesAndTypesList & columns)

View File

@ -60,6 +60,7 @@ 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());
pipeline.addSimpleTransform([&](const Block & header)

View File

@ -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;
}

View File

@ -55,6 +55,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());
pipeline.addSimpleTransform([&](const Block & header)
{
return std::make_shared<ExpressionTransform>(header, expression);
@ -80,7 +81,7 @@ void ExpressionStep::describeActions(FormatSettings & settings) const
String prefix(settings.offset, ' ');
bool first = true;
auto expression = std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings{});
auto expression = std::make_shared<ExpressionActions>(actions_dag);
for (const auto & action : expression->getActions())
{
settings.out << prefix << (first ? "Actions: "
@ -97,7 +98,7 @@ void ExpressionStep::describeActions(FormatSettings & settings) const
void ExpressionStep::describeActions(JSONBuilder::JSONMap & map) const
{
auto expression = std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings{});
auto expression = std::make_shared<ExpressionActions>(actions_dag);
map.add("Expression", expression->toTree());
}

View File

@ -68,6 +68,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());
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type)
{
bool on_totals = stream_type == QueryPipeline::StreamType::Totals;
@ -99,7 +100,7 @@ void FilterStep::describeActions(FormatSettings & settings) const
settings.out << '\n';
bool first = true;
auto expression = std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings{});
auto expression = std::make_shared<ExpressionActions>(actions_dag);
for (const auto & action : expression->getActions())
{
settings.out << prefix << (first ? "Actions: "
@ -119,7 +120,7 @@ void FilterStep::describeActions(JSONBuilder::JSONMap & map) const
map.add("Filter Column", filter_column_name);
map.add("Removes Filter", remove_filter_column);
auto expression = std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings{});
auto expression = std::make_shared<ExpressionActions>(actions_dag);
map.add("Expression", expression->toTree());
}

View File

@ -51,10 +51,16 @@ TotalsHavingStep::TotalsHavingStep(
void TotalsHavingStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings)
{
auto expression_actions = actions_dag ? std::make_shared<ExpressionActions>(actions_dag, settings.getActionsSettings()) : nullptr;
auto totals_having = std::make_shared<TotalsHavingTransform>(
pipeline.getHeader(), overflow_row,
(actions_dag ? std::make_shared<ExpressionActions>(actions_dag, settings.getActionsSettings()) : nullptr),
filter_column_name, totals_mode, auto_include_threshold, final);
pipeline.getHeader(),
overflow_row,
expression_actions,
filter_column_name,
totals_mode,
auto_include_threshold,
final);
pipeline.addTotalsHavingTransform(std::move(totals_having));
}
@ -85,7 +91,7 @@ void TotalsHavingStep::describeActions(FormatSettings & settings) const
if (actions_dag)
{
bool first = true;
auto expression = std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings{});
auto expression = std::make_shared<ExpressionActions>(actions_dag);
for (const auto & action : expression->getActions())
{
settings.out << prefix << (first ? "Actions: "
@ -102,7 +108,7 @@ void TotalsHavingStep::describeActions(JSONBuilder::JSONMap & map) const
if (actions_dag)
{
map.add("Filter column", filter_column_name);
auto expression = std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings{});
auto expression = std::make_shared<ExpressionActions>(actions_dag);
map.add("Expression", expression->toTree());
}
}

View File

@ -52,7 +52,7 @@ ConstraintsExpressions ConstraintsDescription::getExpressions(const DB::ContextP
auto * constraint_ptr = constraint->as<ASTConstraintDeclaration>();
ASTPtr expr = constraint_ptr->expr->clone();
auto syntax_result = TreeRewriter(context).analyze(expr, source_columns_);
res.push_back(ExpressionAnalyzer(constraint_ptr->expr->clone(), syntax_result, context).getActions(false));
res.push_back(ExpressionAnalyzer(constraint_ptr->expr->clone(), syntax_result, context).getActions(false, true, CompileExpressions::yes));
}
return res;
}

View File

@ -415,7 +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));
ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes));
pipe.addSimpleTransform([&](const Block & stream_header)
{
@ -559,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));
auto convert_actions = std::make_shared<ExpressionActions>(convert_actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes));
pipe.addSimpleTransform([&](const Block & stream_header)
{

View File

@ -103,7 +103,7 @@ public:
ActionsDAG::MatchColumnsMode::Name);
auto convert_actions = std::make_shared<ExpressionActions>(
convert_actions_dag,
ExpressionActionsSettings::fromSettings(context->getSettingsRef()));
ExpressionActionsSettings::fromSettings(context->getSettingsRef(), CompileExpressions::yes));
pipe.addSimpleTransform([&](const Block & header)
{

View File

@ -199,7 +199,7 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr contex
auto syntax_result = TreeRewriter(context).analyze(expression_ast, block.getNamesAndTypesList());
ExpressionAnalyzer analyzer(expression_ast, syntax_result, context);
buildSets(expression_ast, analyzer);
ExpressionActionsPtr actions = analyzer.getActions(false);
ExpressionActionsPtr actions = analyzer.getActions(false /* add alises */, true /* project result */, CompileExpressions::yes);
Block block_with_filter = block;
actions->execute(block_with_filter);