Change ActionsDAGPtr to ActionsDAG where possible.

This commit is contained in:
Nikolai Kochetov 2024-07-05 17:49:50 +00:00
parent 7843313f8e
commit 2257f9a2ae
110 changed files with 718 additions and 721 deletions

View File

@ -13,10 +13,10 @@
namespace DB namespace DB
{ {
InterpolateDescription::InterpolateDescription(ActionsDAGPtr actions_, const Aliases & aliases) InterpolateDescription::InterpolateDescription(ActionsDAG actions_, const Aliases & aliases)
: actions(std::move(actions_)) : actions(std::move(actions_))
{ {
for (const auto & name_type : actions->getRequiredColumns()) for (const auto & name_type : actions.getRequiredColumns())
{ {
if (const auto & p = aliases.find(name_type.name); p != aliases.end()) if (const auto & p = aliases.find(name_type.name); p != aliases.end())
required_columns_map[p->second->getColumnName()] = name_type; required_columns_map[p->second->getColumnName()] = name_type;
@ -24,7 +24,7 @@ namespace DB
required_columns_map[name_type.name] = name_type; required_columns_map[name_type.name] = name_type;
} }
for (const ColumnWithTypeAndName & column : actions->getResultColumns()) for (const ColumnWithTypeAndName & column : actions.getResultColumns())
{ {
std::string name = column.name; std::string name = column.name;
if (const auto & p = aliases.find(name); p != aliases.end()) if (const auto & p = aliases.find(name); p != aliases.end())

View File

@ -5,21 +5,20 @@
#include <string> #include <string>
#include <Core/NamesAndTypes.h> #include <Core/NamesAndTypes.h>
#include <Parsers/IAST_fwd.h> #include <Parsers/IAST_fwd.h>
#include <Interpreters/ActionsDAG.h>
namespace DB namespace DB
{ {
class ActionsDAG;
using ActionsDAGPtr = std::unique_ptr<ActionsDAG>;
using Aliases = std::unordered_map<String, ASTPtr>; using Aliases = std::unordered_map<String, ASTPtr>;
/// Interpolate description /// Interpolate description
struct InterpolateDescription struct InterpolateDescription
{ {
explicit InterpolateDescription(ActionsDAGPtr actions, const Aliases & aliases); explicit InterpolateDescription(ActionsDAG actions, const Aliases & aliases);
ActionsDAGPtr actions; ActionsDAG actions;
std::unordered_map<std::string, NameAndTypePair> required_columns_map; /// input column name -> {alias, type} std::unordered_map<std::string, NameAndTypePair> required_columns_map; /// input column name -> {alias, type}
std::unordered_set<std::string> result_columns_set; /// result block columns std::unordered_set<std::string> result_columns_set; /// result block columns

View File

@ -58,11 +58,11 @@ public:
return DataTypeUInt8().createColumnConst(input_rows_count, 1u); return DataTypeUInt8().createColumnConst(input_rows_count, 1u);
} }
void setActions(ActionsDAGPtr actions_) { actions = std::move(actions_); } void setActions(ActionsDAG actions_) { actions = std::move(actions_); }
const ActionsDAGPtr & getActions() const { return actions; } const ActionsDAG & getActions() const { return actions; }
private: private:
ActionsDAGPtr actions; ActionsDAG actions;
}; };
} }

View File

@ -961,9 +961,9 @@ NameSet ActionsDAG::foldActionsByProjection(
} }
ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_map<const Node *, const Node *> & new_inputs, const NodeRawConstPtrs & required_outputs) ActionsDAG ActionsDAG::foldActionsByProjection(const std::unordered_map<const Node *, const Node *> & new_inputs, const NodeRawConstPtrs & required_outputs)
{ {
auto dag = std::make_unique<ActionsDAG>(); ActionsDAG dag;
std::unordered_map<const Node *, const Node *> inputs_mapping; std::unordered_map<const Node *, const Node *> inputs_mapping;
std::unordered_map<const Node *, const Node *> mapping; std::unordered_map<const Node *, const Node *> mapping;
struct Frame struct Frame
@ -1003,9 +1003,9 @@ ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_map<const
{ {
bool should_rename = new_input->result_name != rename->result_name; bool should_rename = new_input->result_name != rename->result_name;
const auto & input_name = should_rename ? rename->result_name : new_input->result_name; const auto & input_name = should_rename ? rename->result_name : new_input->result_name;
mapped_input = &dag->addInput(input_name, new_input->result_type); mapped_input = &dag.addInput(input_name, new_input->result_type);
if (should_rename) if (should_rename)
mapped_input = &dag->addAlias(*mapped_input, new_input->result_name); mapped_input = &dag.addAlias(*mapped_input, new_input->result_name);
} }
node = mapped_input; node = mapped_input;
@ -1034,7 +1034,7 @@ ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_map<const
"Cannot fold actions for projection. Node {} requires input {} which does not belong to projection", "Cannot fold actions for projection. Node {} requires input {} which does not belong to projection",
stack.front().node->result_name, frame.node->result_name); stack.front().node->result_name, frame.node->result_name);
auto & node = dag->nodes.emplace_back(*frame.node); auto & node = dag.nodes.emplace_back(*frame.node);
for (auto & child : node.children) for (auto & child : node.children)
child = mapping[child]; child = mapping[child];
@ -1049,8 +1049,8 @@ ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_map<const
/// Add an alias if the mapped node has a different result name. /// Add an alias if the mapped node has a different result name.
const auto * mapped_output = mapping[output]; const auto * mapped_output = mapping[output];
if (output->result_name != mapped_output->result_name) if (output->result_name != mapped_output->result_name)
mapped_output = &dag->addAlias(*mapped_output, output->result_name); mapped_output = &dag.addAlias(*mapped_output, output->result_name);
dag->outputs.push_back(mapped_output); dag.outputs.push_back(mapped_output);
} }
return dag; return dag;
@ -1411,7 +1411,7 @@ const ActionsDAG::Node & ActionsDAG::materializeNode(const Node & node)
return addAlias(*func, name); return addAlias(*func, name);
} }
ActionsDAGPtr ActionsDAG::makeConvertingActions( ActionsDAG ActionsDAG::makeConvertingActions(
const ColumnsWithTypeAndName & source, const ColumnsWithTypeAndName & source,
const ColumnsWithTypeAndName & result, const ColumnsWithTypeAndName & result,
MatchColumnsMode mode, MatchColumnsMode mode,
@ -1428,7 +1428,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
if (add_casted_columns && mode != MatchColumnsMode::Name) if (add_casted_columns && mode != MatchColumnsMode::Name)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Converting with add_casted_columns supported only for MatchColumnsMode::Name"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Converting with add_casted_columns supported only for MatchColumnsMode::Name");
auto actions_dag = std::make_unique<ActionsDAG>(source); ActionsDAG actions_dag(source);
NodeRawConstPtrs projection(num_result_columns); NodeRawConstPtrs projection(num_result_columns);
FunctionOverloadResolverPtr func_builder_materialize = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionMaterialize>()); FunctionOverloadResolverPtr func_builder_materialize = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionMaterialize>());
@ -1436,9 +1436,9 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
std::unordered_map<std::string_view, std::list<size_t>> inputs; std::unordered_map<std::string_view, std::list<size_t>> inputs;
if (mode == MatchColumnsMode::Name) if (mode == MatchColumnsMode::Name)
{ {
size_t input_nodes_size = actions_dag->inputs.size(); size_t input_nodes_size = actions_dag.inputs.size();
for (size_t pos = 0; pos < input_nodes_size; ++pos) for (size_t pos = 0; pos < input_nodes_size; ++pos)
inputs[actions_dag->inputs[pos]->result_name].push_back(pos); inputs[actions_dag.inputs[pos]->result_name].push_back(pos);
} }
for (size_t result_col_num = 0; result_col_num < num_result_columns; ++result_col_num) for (size_t result_col_num = 0; result_col_num < num_result_columns; ++result_col_num)
@ -1451,7 +1451,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
{ {
case MatchColumnsMode::Position: case MatchColumnsMode::Position:
{ {
src_node = dst_node = actions_dag->inputs[result_col_num]; src_node = dst_node = actions_dag.inputs[result_col_num];
break; break;
} }
@ -1462,7 +1462,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
{ {
const auto * res_const = typeid_cast<const ColumnConst *>(res_elem.column.get()); const auto * res_const = typeid_cast<const ColumnConst *>(res_elem.column.get());
if (ignore_constant_values && res_const) if (ignore_constant_values && res_const)
src_node = dst_node = &actions_dag->addColumn(res_elem); src_node = dst_node = &actions_dag.addColumn(res_elem);
else else
throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, throw Exception(ErrorCodes::THERE_IS_NO_COLUMN,
"Cannot find column `{}` in source stream, there are only columns: [{}]", "Cannot find column `{}` in source stream, there are only columns: [{}]",
@ -1470,7 +1470,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
} }
else else
{ {
src_node = dst_node = actions_dag->inputs[input.front()]; src_node = dst_node = actions_dag.inputs[input.front()];
input.pop_front(); input.pop_front();
} }
break; break;
@ -1483,7 +1483,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
if (const auto * src_const = typeid_cast<const ColumnConst *>(dst_node->column.get())) if (const auto * src_const = typeid_cast<const ColumnConst *>(dst_node->column.get()))
{ {
if (ignore_constant_values) if (ignore_constant_values)
dst_node = &actions_dag->addColumn(res_elem); dst_node = &actions_dag.addColumn(res_elem);
else if (res_const->getField() != src_const->getField()) else if (res_const->getField() != src_const->getField())
throw Exception( throw Exception(
ErrorCodes::ILLEGAL_COLUMN, ErrorCodes::ILLEGAL_COLUMN,
@ -1505,7 +1505,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
column.column = DataTypeString().createColumnConst(0, column.name); column.column = DataTypeString().createColumnConst(0, column.name);
column.type = std::make_shared<DataTypeString>(); column.type = std::make_shared<DataTypeString>();
const auto * right_arg = &actions_dag->addColumn(std::move(column)); const auto * right_arg = &actions_dag.addColumn(std::move(column));
const auto * left_arg = dst_node; const auto * left_arg = dst_node;
CastDiagnostic diagnostic = {dst_node->result_name, res_elem.name}; CastDiagnostic diagnostic = {dst_node->result_name, res_elem.name};
@ -1513,13 +1513,13 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
= createInternalCastOverloadResolver(CastType::nonAccurate, std::move(diagnostic)); = createInternalCastOverloadResolver(CastType::nonAccurate, std::move(diagnostic));
NodeRawConstPtrs children = { left_arg, right_arg }; NodeRawConstPtrs children = { left_arg, right_arg };
dst_node = &actions_dag->addFunction(func_builder_cast, std::move(children), {}); dst_node = &actions_dag.addFunction(func_builder_cast, std::move(children), {});
} }
if (dst_node->column && isColumnConst(*dst_node->column) && !(res_elem.column && isColumnConst(*res_elem.column))) if (dst_node->column && isColumnConst(*dst_node->column) && !(res_elem.column && isColumnConst(*res_elem.column)))
{ {
NodeRawConstPtrs children = {dst_node}; NodeRawConstPtrs children = {dst_node};
dst_node = &actions_dag->addFunction(func_builder_materialize, std::move(children), {}); dst_node = &actions_dag.addFunction(func_builder_materialize, std::move(children), {});
} }
if (dst_node->result_name != res_elem.name) if (dst_node->result_name != res_elem.name)
@ -1538,7 +1538,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
} }
else else
{ {
dst_node = &actions_dag->addAlias(*dst_node, res_elem.name); dst_node = &actions_dag.addAlias(*dst_node, res_elem.name);
projection[result_col_num] = dst_node; projection[result_col_num] = dst_node;
} }
} }
@ -1548,36 +1548,36 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
} }
} }
actions_dag->outputs.swap(projection); actions_dag.outputs.swap(projection);
actions_dag->removeUnusedActions(false); actions_dag.removeUnusedActions(false);
return actions_dag; return actions_dag;
} }
ActionsDAGPtr ActionsDAG::makeAddingColumnActions(ColumnWithTypeAndName column) ActionsDAG ActionsDAG::makeAddingColumnActions(ColumnWithTypeAndName column)
{ {
auto adding_column_action = std::make_unique<ActionsDAG>(); ActionsDAG adding_column_action;
FunctionOverloadResolverPtr func_builder_materialize FunctionOverloadResolverPtr func_builder_materialize
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionMaterialize>()); = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionMaterialize>());
auto column_name = column.name; auto column_name = column.name;
const auto * column_node = &adding_column_action->addColumn(std::move(column)); const auto * column_node = &adding_column_action.addColumn(std::move(column));
NodeRawConstPtrs inputs = {column_node}; NodeRawConstPtrs inputs = {column_node};
const auto & function_node = adding_column_action->addFunction(func_builder_materialize, std::move(inputs), {}); const auto & function_node = adding_column_action.addFunction(func_builder_materialize, std::move(inputs), {});
const auto & alias_node = adding_column_action->addAlias(function_node, std::move(column_name)); const auto & alias_node = adding_column_action.addAlias(function_node, std::move(column_name));
adding_column_action->outputs.push_back(&alias_node); adding_column_action.outputs.push_back(&alias_node);
return adding_column_action; return adding_column_action;
} }
ActionsDAGPtr ActionsDAG::merge(ActionsDAG && first, ActionsDAG && second) ActionsDAG ActionsDAG::merge(ActionsDAG && first, ActionsDAG && second)
{ {
first.mergeInplace(std::move(second)); first.mergeInplace(std::move(second));
/// Some actions could become unused. Do not drop inputs to preserve the header. /// Some actions could become unused. Do not drop inputs to preserve the header.
first.removeUnusedActions(false); first.removeUnusedActions(false);
return std::make_unique<ActionsDAG>(std::move(first)); return std::move(first);
} }
void ActionsDAG::mergeInplace(ActionsDAG && second) void ActionsDAG::mergeInplace(ActionsDAG && second)
@ -1970,15 +1970,15 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set<const Node *> split
second_inputs.push_back(cur.to_second); second_inputs.push_back(cur.to_second);
} }
auto first_actions = std::make_unique<ActionsDAG>(); ActionsDAG first_actions;
first_actions->nodes.swap(first_nodes); first_actions.nodes.swap(first_nodes);
first_actions->outputs.swap(first_outputs); first_actions.outputs.swap(first_outputs);
first_actions->inputs.swap(first_inputs); first_actions.inputs.swap(first_inputs);
auto second_actions = std::make_unique<ActionsDAG>(); ActionsDAG second_actions;
second_actions->nodes.swap(second_nodes); second_actions.nodes.swap(second_nodes);
second_actions->outputs.swap(second_outputs); second_actions.outputs.swap(second_outputs);
second_actions->inputs.swap(second_inputs); second_actions.inputs.swap(second_inputs);
std::unordered_map<const Node *, const Node *> split_nodes_mapping; std::unordered_map<const Node *, const Node *> split_nodes_mapping;
if (create_split_nodes_mapping) if (create_split_nodes_mapping)
@ -2098,7 +2098,7 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsBySortingDescription(const NameS
return res; return res;
} }
bool ActionsDAG::isFilterAlwaysFalseForDefaultValueInputs(const std::string & filter_name, const Block & input_stream_header) bool ActionsDAG::isFilterAlwaysFalseForDefaultValueInputs(const std::string & filter_name, const Block & input_stream_header) const
{ {
const auto * filter_node = tryFindInOutputs(filter_name); const auto * filter_node = tryFindInOutputs(filter_name);
if (!filter_node) if (!filter_node)
@ -2122,7 +2122,7 @@ bool ActionsDAG::isFilterAlwaysFalseForDefaultValueInputs(const std::string & fi
input_node_name_to_default_input_column.emplace(input->result_name, std::move(constant_column_with_type_and_name)); input_node_name_to_default_input_column.emplace(input->result_name, std::move(constant_column_with_type_and_name));
} }
ActionsDAGPtr filter_with_default_value_inputs; std::optional<ActionsDAG> filter_with_default_value_inputs;
try try
{ {
@ -2304,12 +2304,12 @@ ColumnsWithTypeAndName prepareFunctionArguments(const ActionsDAG::NodeRawConstPt
/// ///
/// Result actions add single column with conjunction result (it is always first in outputs). /// Result actions add single column with conjunction result (it is always first in outputs).
/// No other columns are added or removed. /// No other columns are added or removed.
ActionsDAGPtr ActionsDAG::createActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs) std::optional<ActionsDAG> ActionsDAG::createActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs)
{ {
if (conjunction.empty()) if (conjunction.empty())
return nullptr; return {};
auto actions = std::make_unique<ActionsDAG>(); ActionsDAG actions;
FunctionOverloadResolverPtr func_builder_and = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>()); FunctionOverloadResolverPtr func_builder_and = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>());
@ -2350,7 +2350,7 @@ ActionsDAGPtr ActionsDAG::createActionsForConjunction(NodeRawConstPtrs conjuncti
if (cur.next_child_to_visit == cur.node->children.size()) if (cur.next_child_to_visit == cur.node->children.size())
{ {
auto & node = actions->nodes.emplace_back(*cur.node); auto & node = actions.nodes.emplace_back(*cur.node);
nodes_mapping[cur.node] = &node; nodes_mapping[cur.node] = &node;
for (auto & child : node.children) for (auto & child : node.children)
@ -2373,33 +2373,33 @@ ActionsDAGPtr ActionsDAG::createActionsForConjunction(NodeRawConstPtrs conjuncti
for (const auto * predicate : conjunction) for (const auto * predicate : conjunction)
args.emplace_back(nodes_mapping[predicate]); args.emplace_back(nodes_mapping[predicate]);
result_predicate = &actions->addFunction(func_builder_and, std::move(args), {}); result_predicate = &actions.addFunction(func_builder_and, std::move(args), {});
} }
actions->outputs.push_back(result_predicate); actions.outputs.push_back(result_predicate);
for (const auto & col : all_inputs) for (const auto & col : all_inputs)
{ {
const Node * input; const Node * input;
auto & list = required_inputs[col.name]; auto & list = required_inputs[col.name];
if (list.empty()) if (list.empty())
input = &actions->addInput(col); input = &actions.addInput(col);
else else
{ {
input = list.front(); input = list.front();
list.pop_front(); list.pop_front();
actions->inputs.push_back(input); actions.inputs.push_back(input);
} }
/// We should not add result_predicate into the outputs for the second time. /// We should not add result_predicate into the outputs for the second time.
if (input->result_name != result_predicate->result_name) if (input->result_name != result_predicate->result_name)
actions->outputs.push_back(input); actions.outputs.push_back(input);
} }
return actions; return actions;
} }
ActionsDAGPtr ActionsDAG::splitActionsForFilterPushDown( std::optional<ActionsDAG> ActionsDAG::splitActionsForFilterPushDown(
const std::string & filter_name, const std::string & filter_name,
bool removes_filter, bool removes_filter,
const Names & available_inputs, const Names & available_inputs,
@ -2415,7 +2415,7 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilterPushDown(
/// If condition is constant let's do nothing. /// If condition is constant let's do nothing.
/// It means there is nothing to push down or optimization was already applied. /// It means there is nothing to push down or optimization was already applied.
if (predicate->type == ActionType::COLUMN) if (predicate->type == ActionType::COLUMN)
return nullptr; return {};
std::unordered_set<const Node *> allowed_nodes; std::unordered_set<const Node *> allowed_nodes;
@ -2439,7 +2439,7 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilterPushDown(
auto conjunction = getConjunctionNodes(predicate, allowed_nodes); auto conjunction = getConjunctionNodes(predicate, allowed_nodes);
if (conjunction.allowed.empty()) if (conjunction.allowed.empty())
return nullptr; return {};
chassert(predicate->result_type); chassert(predicate->result_type);
@ -2451,13 +2451,13 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilterPushDown(
&& !conjunction.rejected.front()->result_type->equals(*predicate->result_type)) && !conjunction.rejected.front()->result_type->equals(*predicate->result_type))
{ {
/// No further optimization can be done /// No further optimization can be done
return nullptr; return {};
} }
} }
auto actions = createActionsForConjunction(conjunction.allowed, all_inputs); auto actions = createActionsForConjunction(conjunction.allowed, all_inputs);
if (!actions) if (!actions)
return nullptr; return {};
/// Now, when actions are created, update the current DAG. /// Now, when actions are created, update the current DAG.
removeUnusedConjunctions(std::move(conjunction.rejected), predicate, removes_filter); removeUnusedConjunctions(std::move(conjunction.rejected), predicate, removes_filter);
@ -2562,11 +2562,11 @@ ActionsDAG::ActionsForJOINFilterPushDown ActionsDAG::splitActionsForJOINFilterPu
auto left_stream_filter_to_push_down = createActionsForConjunction(left_stream_allowed_conjunctions, left_stream_header.getColumnsWithTypeAndName()); auto left_stream_filter_to_push_down = createActionsForConjunction(left_stream_allowed_conjunctions, left_stream_header.getColumnsWithTypeAndName());
auto right_stream_filter_to_push_down = createActionsForConjunction(right_stream_allowed_conjunctions, right_stream_header.getColumnsWithTypeAndName()); auto right_stream_filter_to_push_down = createActionsForConjunction(right_stream_allowed_conjunctions, right_stream_header.getColumnsWithTypeAndName());
auto replace_equivalent_columns_in_filter = [](const ActionsDAGPtr & filter, auto replace_equivalent_columns_in_filter = [](const ActionsDAG & filter,
const Block & stream_header, const Block & stream_header,
const std::unordered_map<std::string, ColumnWithTypeAndName> & columns_to_replace) const std::unordered_map<std::string, ColumnWithTypeAndName> & columns_to_replace)
{ {
auto updated_filter = ActionsDAG::buildFilterActionsDAG({filter->getOutputs()[0]}, columns_to_replace); auto updated_filter = ActionsDAG::buildFilterActionsDAG({filter.getOutputs()[0]}, columns_to_replace);
chassert(updated_filter->getOutputs().size() == 1); chassert(updated_filter->getOutputs().size() == 1);
/** If result filter to left or right stream has column that is one of the stream inputs, we need distinguish filter column from /** If result filter to left or right stream has column that is one of the stream inputs, we need distinguish filter column from
@ -2587,7 +2587,7 @@ ActionsDAG::ActionsForJOINFilterPushDown ActionsDAG::splitActionsForJOINFilterPu
for (const auto & input : updated_filter->getInputs()) for (const auto & input : updated_filter->getInputs())
updated_filter_inputs[input->result_name].push_back(input); updated_filter_inputs[input->result_name].push_back(input);
for (const auto & input : filter->getInputs()) for (const auto & input : filter.getInputs())
{ {
if (updated_filter_inputs.contains(input->result_name)) if (updated_filter_inputs.contains(input->result_name))
continue; continue;
@ -2625,12 +2625,12 @@ ActionsDAG::ActionsForJOINFilterPushDown ActionsDAG::splitActionsForJOINFilterPu
}; };
if (left_stream_filter_to_push_down) if (left_stream_filter_to_push_down)
left_stream_filter_to_push_down = replace_equivalent_columns_in_filter(left_stream_filter_to_push_down, left_stream_filter_to_push_down = replace_equivalent_columns_in_filter(*left_stream_filter_to_push_down,
left_stream_header, left_stream_header,
equivalent_right_stream_column_to_left_stream_column); equivalent_right_stream_column_to_left_stream_column);
if (right_stream_filter_to_push_down) if (right_stream_filter_to_push_down)
right_stream_filter_to_push_down = replace_equivalent_columns_in_filter(right_stream_filter_to_push_down, right_stream_filter_to_push_down = replace_equivalent_columns_in_filter(*right_stream_filter_to_push_down,
right_stream_header, right_stream_header,
equivalent_left_stream_column_to_right_stream_column); equivalent_left_stream_column_to_right_stream_column);
@ -2859,13 +2859,13 @@ bool ActionsDAG::isSortingPreserved(
return true; return true;
} }
ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( std::optional<ActionsDAG> ActionsDAG::buildFilterActionsDAG(
const NodeRawConstPtrs & filter_nodes, const NodeRawConstPtrs & filter_nodes,
const std::unordered_map<std::string, ColumnWithTypeAndName> & node_name_to_input_node_column, const std::unordered_map<std::string, ColumnWithTypeAndName> & node_name_to_input_node_column,
bool single_output_condition_node) bool single_output_condition_node)
{ {
if (filter_nodes.empty()) if (filter_nodes.empty())
return nullptr; return {};
struct Frame struct Frame
{ {
@ -2873,7 +2873,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
bool visited_children = false; bool visited_children = false;
}; };
auto result_dag = std::make_unique<ActionsDAG>(); ActionsDAG result_dag;
std::unordered_map<std::string, const ActionsDAG::Node *> result_inputs; std::unordered_map<std::string, const ActionsDAG::Node *> result_inputs;
std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> node_to_result_node; std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> node_to_result_node;
@ -2904,7 +2904,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
{ {
auto & result_input = result_inputs[input_node_it->second.name]; auto & result_input = result_inputs[input_node_it->second.name];
if (!result_input) if (!result_input)
result_input = &result_dag->addInput(input_node_it->second); result_input = &result_dag.addInput(input_node_it->second);
node_to_result_node.emplace(node, result_input); node_to_result_node.emplace(node, result_input);
nodes_to_process.pop_back(); nodes_to_process.pop_back();
@ -2931,25 +2931,25 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
{ {
auto & result_input = result_inputs[node->result_name]; auto & result_input = result_inputs[node->result_name];
if (!result_input) if (!result_input)
result_input = &result_dag->addInput({node->column, node->result_type, node->result_name}); result_input = &result_dag.addInput({node->column, node->result_type, node->result_name});
result_node = result_input; result_node = result_input;
break; break;
} }
case ActionsDAG::ActionType::COLUMN: case ActionsDAG::ActionType::COLUMN:
{ {
result_node = &result_dag->addColumn({node->column, node->result_type, node->result_name}); result_node = &result_dag.addColumn({node->column, node->result_type, node->result_name});
break; break;
} }
case ActionsDAG::ActionType::ALIAS: case ActionsDAG::ActionType::ALIAS:
{ {
const auto * child = node->children.front(); const auto * child = node->children.front();
result_node = &result_dag->addAlias(*(node_to_result_node.find(child)->second), node->result_name); result_node = &result_dag.addAlias(*(node_to_result_node.find(child)->second), node->result_name);
break; break;
} }
case ActionsDAG::ActionType::ARRAY_JOIN: case ActionsDAG::ActionType::ARRAY_JOIN:
{ {
const auto * child = node->children.front(); const auto * child = node->children.front();
result_node = &result_dag->addArrayJoin(*(node_to_result_node.find(child)->second), {}); result_node = &result_dag.addArrayJoin(*(node_to_result_node.find(child)->second), {});
break; break;
} }
case ActionsDAG::ActionType::FUNCTION: case ActionsDAG::ActionType::FUNCTION:
@ -2967,13 +2967,11 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
{ {
if (const auto * index_hint = typeid_cast<const FunctionIndexHint *>(adaptor->getFunction().get())) if (const auto * index_hint = typeid_cast<const FunctionIndexHint *>(adaptor->getFunction().get()))
{ {
ActionsDAGPtr index_hint_filter_dag; ActionsDAG index_hint_filter_dag;
const auto & index_hint_args = index_hint->getActions()->getOutputs(); const auto & index_hint_args = index_hint->getActions().getOutputs();
if (index_hint_args.empty()) if (!index_hint_args.empty())
index_hint_filter_dag = std::make_unique<ActionsDAG>(); index_hint_filter_dag = *buildFilterActionsDAG(index_hint_args,
else
index_hint_filter_dag = buildFilterActionsDAG(index_hint_args,
node_name_to_input_node_column, node_name_to_input_node_column,
false /*single_output_condition_node*/); false /*single_output_condition_node*/);
@ -2995,7 +2993,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
auto [arguments, all_const] = getFunctionArguments(function_children); auto [arguments, all_const] = getFunctionArguments(function_children);
auto function_base = function_overload_resolver ? function_overload_resolver->build(arguments) : node->function_base; auto function_base = function_overload_resolver ? function_overload_resolver->build(arguments) : node->function_base;
result_node = &result_dag->addFunctionImpl( result_node = &result_dag.addFunctionImpl(
function_base, function_base,
std::move(function_children), std::move(function_children),
std::move(arguments), std::move(arguments),
@ -3010,7 +3008,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
nodes_to_process.pop_back(); nodes_to_process.pop_back();
} }
auto & result_dag_outputs = result_dag->getOutputs(); auto & result_dag_outputs = result_dag.getOutputs();
result_dag_outputs.reserve(filter_nodes_size); result_dag_outputs.reserve(filter_nodes_size);
for (const auto & node : filter_nodes) for (const auto & node : filter_nodes)
@ -3019,7 +3017,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
if (result_dag_outputs.size() > 1 && single_output_condition_node) if (result_dag_outputs.size() > 1 && single_output_condition_node)
{ {
FunctionOverloadResolverPtr func_builder_and = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>()); FunctionOverloadResolverPtr func_builder_and = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>());
result_dag_outputs = { &result_dag->addFunction(func_builder_and, result_dag_outputs, {}) }; result_dag_outputs = { &result_dag.addFunction(func_builder_and, result_dag_outputs, {}) };
} }
return result_dag; return result_dag;

View File

@ -247,7 +247,7 @@ public:
/// c * d e /// c * d e
/// \ / /// \ /
/// c * d - e /// c * d - e
static ActionsDAGPtr foldActionsByProjection( static ActionsDAG foldActionsByProjection(
const std::unordered_map<const Node *, const Node *> & new_inputs, const std::unordered_map<const Node *, const Node *> & new_inputs,
const NodeRawConstPtrs & required_outputs); const NodeRawConstPtrs & required_outputs);
@ -303,7 +303,7 @@ public:
/// @param ignore_constant_values - Do not check that constants are same. Use value from result_header. /// @param ignore_constant_values - Do not check that constants are same. Use value from result_header.
/// @param add_casted_columns - Create new columns with converted values instead of replacing original. /// @param add_casted_columns - Create new columns with converted values instead of replacing original.
/// @param new_names - Output parameter for new column names when add_casted_columns is used. /// @param new_names - Output parameter for new column names when add_casted_columns is used.
static ActionsDAGPtr makeConvertingActions( static ActionsDAG makeConvertingActions(
const ColumnsWithTypeAndName & source, const ColumnsWithTypeAndName & source,
const ColumnsWithTypeAndName & result, const ColumnsWithTypeAndName & result,
MatchColumnsMode mode, MatchColumnsMode mode,
@ -312,13 +312,13 @@ public:
NameToNameMap * new_names = nullptr); NameToNameMap * new_names = nullptr);
/// Create expression which add const column and then materialize it. /// Create expression which add const column and then materialize it.
static ActionsDAGPtr makeAddingColumnActions(ColumnWithTypeAndName column); static ActionsDAG makeAddingColumnActions(ColumnWithTypeAndName column);
/// Create ActionsDAG which represents expression equivalent to applying first and second actions consequently. /// Create ActionsDAG which represents expression equivalent to applying first and second actions consequently.
/// Is used to replace `(first -> second)` expression chain to single `merge(first, second)` expression. /// Is used to replace `(first -> second)` expression chain to single `merge(first, second)` expression.
/// If first.settings.project_input is set, then outputs of `first` must include inputs of `second`. /// If first.settings.project_input is set, then outputs of `first` must include inputs of `second`.
/// Otherwise, any two actions may be combined. /// Otherwise, any two actions may be combined.
static ActionsDAGPtr merge(ActionsDAG && first, ActionsDAG && second); static ActionsDAG merge(ActionsDAG && first, ActionsDAG && second);
/// The result is similar to merge(*this, second); /// The result is similar to merge(*this, second);
/// Invariant : no nodes are removed from the first (this) DAG. /// Invariant : no nodes are removed from the first (this) DAG.
@ -329,12 +329,7 @@ public:
/// *out_outputs is filled with pointers to the nodes corresponding to second.getOutputs(). /// *out_outputs is filled with pointers to the nodes corresponding to second.getOutputs().
void mergeNodes(ActionsDAG && second, NodeRawConstPtrs * out_outputs = nullptr); void mergeNodes(ActionsDAG && second, NodeRawConstPtrs * out_outputs = nullptr);
struct SplitResult struct SplitResult;
{
ActionsDAGPtr first;
ActionsDAGPtr second;
std::unordered_map<const Node *, const Node *> split_nodes_mapping;
};
/// Split ActionsDAG into two DAGs, where first part contains all nodes from split_nodes and their children. /// Split ActionsDAG into two DAGs, where first part contains all nodes from split_nodes and their children.
/// Execution of first then second parts on block is equivalent to execution of initial DAG. /// Execution of first then second parts on block is equivalent to execution of initial DAG.
@ -362,7 +357,7 @@ public:
* @param filter_name - name of filter node in current DAG. * @param filter_name - name of filter node in current DAG.
* @param input_stream_header - input stream header. * @param input_stream_header - input stream header.
*/ */
bool isFilterAlwaysFalseForDefaultValueInputs(const std::string & filter_name, const Block & input_stream_header); bool isFilterAlwaysFalseForDefaultValueInputs(const std::string & filter_name, const Block & input_stream_header) const;
/// Create actions which may calculate part of filter using only available_inputs. /// Create actions which may calculate part of filter using only available_inputs.
/// If nothing may be calculated, returns nullptr. /// If nothing may be calculated, returns nullptr.
@ -381,19 +376,13 @@ public:
/// columns will be transformed like `x, y, z` -> `z > 0, z, x, y` -(remove filter)-> `z, x, y`. /// columns will be transformed like `x, y, z` -> `z > 0, z, x, y` -(remove filter)-> `z, x, y`.
/// To avoid it, add inputs from `all_inputs` list, /// To avoid it, add inputs from `all_inputs` list,
/// so actions `x, y, z -> z > 0, x, y, z` -(remove filter)-> `x, y, z` will not change columns order. /// so actions `x, y, z -> z > 0, x, y, z` -(remove filter)-> `x, y, z` will not change columns order.
ActionsDAGPtr splitActionsForFilterPushDown( std::optional<ActionsDAG> splitActionsForFilterPushDown(
const std::string & filter_name, const std::string & filter_name,
bool removes_filter, bool removes_filter,
const Names & available_inputs, const Names & available_inputs,
const ColumnsWithTypeAndName & all_inputs); const ColumnsWithTypeAndName & all_inputs);
struct ActionsForJOINFilterPushDown struct ActionsForJOINFilterPushDown;
{
ActionsDAGPtr left_stream_filter_to_push_down;
bool left_stream_filter_removes_filter;
ActionsDAGPtr right_stream_filter_to_push_down;
bool right_stream_filter_removes_filter;
};
/** Split actions for JOIN filter push down. /** Split actions for JOIN filter push down.
* *
@ -440,7 +429,7 @@ public:
* *
* If single_output_condition_node = false, result dag has multiple output nodes. * If single_output_condition_node = false, result dag has multiple output nodes.
*/ */
static ActionsDAGPtr buildFilterActionsDAG( static std::optional<ActionsDAG> buildFilterActionsDAG(
const NodeRawConstPtrs & filter_nodes, const NodeRawConstPtrs & filter_nodes,
const std::unordered_map<std::string, ColumnWithTypeAndName> & node_name_to_input_node_column = {}, const std::unordered_map<std::string, ColumnWithTypeAndName> & node_name_to_input_node_column = {},
bool single_output_condition_node = true); bool single_output_condition_node = true);
@ -472,11 +461,26 @@ private:
void compileFunctions(size_t min_count_to_compile_expression, const std::unordered_set<const Node *> & lazy_executed_nodes = {}); void compileFunctions(size_t min_count_to_compile_expression, const std::unordered_set<const Node *> & lazy_executed_nodes = {});
#endif #endif
static ActionsDAGPtr createActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs); static std::optional<ActionsDAG> createActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs);
void removeUnusedConjunctions(NodeRawConstPtrs rejected_conjunctions, Node * predicate, bool removes_filter); void removeUnusedConjunctions(NodeRawConstPtrs rejected_conjunctions, Node * predicate, bool removes_filter);
}; };
struct ActionsDAG::SplitResult
{
ActionsDAG first;
ActionsDAG second;
std::unordered_map<const Node *, const Node *> split_nodes_mapping;
};
struct ActionsDAG::ActionsForJOINFilterPushDown
{
std::optional<ActionsDAG> left_stream_filter_to_push_down;
bool left_stream_filter_removes_filter;
std::optional<ActionsDAG> right_stream_filter_to_push_down;
bool right_stream_filter_removes_filter;
};
class FindOriginalNodeForOutputName class FindOriginalNodeForOutputName
{ {
using NameToNodeIndex = std::unordered_map<std::string_view, const ActionsDAG::Node *>; using NameToNodeIndex = std::unordered_map<std::string_view, const ActionsDAG::Node *>;

View File

@ -1022,7 +1022,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
dag.project(args); dag.project(args);
auto index_hint = std::make_shared<FunctionIndexHint>(); auto index_hint = std::make_shared<FunctionIndexHint>();
index_hint->setActions(std::make_unique<ActionsDAG>(std::move(dag))); index_hint->setActions(std::move(dag));
// Arguments are removed. We add function instead of constant column to avoid constant folding. // Arguments are removed. We add function instead of constant column to avoid constant folding.
data.addFunction(std::make_unique<FunctionToOverloadResolverAdaptor>(index_hint), {}, column_name); data.addFunction(std::make_unique<FunctionToOverloadResolverAdaptor>(index_hint), {}, column_name);
@ -1285,7 +1285,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
lambda_dag.removeUnusedActions(Names(1, result_name)); lambda_dag.removeUnusedActions(Names(1, result_name));
auto lambda_actions = std::make_shared<ExpressionActions>( auto lambda_actions = std::make_shared<ExpressionActions>(
std::make_unique<ActionsDAG>(std::move(lambda_dag)), std::move(lambda_dag),
ExpressionActionsSettings::fromContext(data.getContext(), CompileExpressions::yes)); ExpressionActionsSettings::fromContext(data.getContext(), CompileExpressions::yes));
DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type; DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type;

View File

@ -49,14 +49,13 @@ namespace ErrorCodes
static std::unordered_set<const ActionsDAG::Node *> processShortCircuitFunctions(const ActionsDAG & actions_dag, ShortCircuitFunctionEvaluation short_circuit_function_evaluation); static std::unordered_set<const ActionsDAG::Node *> processShortCircuitFunctions(const ActionsDAG & actions_dag, ShortCircuitFunctionEvaluation short_circuit_function_evaluation);
ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings_, bool project_inputs_) ExpressionActions::ExpressionActions(ActionsDAG actions_dag_, const ExpressionActionsSettings & settings_, bool project_inputs_)
: project_inputs(project_inputs_) : actions_dag(std::move(actions_dag_))
, project_inputs(project_inputs_)
, settings(settings_) , settings(settings_)
{ {
actions_dag = ActionsDAG::clone(actions_dag_);
/// It's important to determine lazy executed nodes before compiling expressions. /// It's important to determine lazy executed nodes before compiling expressions.
std::unordered_set<const ActionsDAG::Node *> lazy_executed_nodes = processShortCircuitFunctions(*actions_dag, settings.short_circuit_function_evaluation); std::unordered_set<const ActionsDAG::Node *> lazy_executed_nodes = processShortCircuitFunctions(actions_dag, settings.short_circuit_function_evaluation);
#if USE_EMBEDDED_COMPILER #if USE_EMBEDDED_COMPILER
if (settings.can_compile_expressions && settings.compile_expressions == CompileExpressions::yes) if (settings.can_compile_expressions && settings.compile_expressions == CompileExpressions::yes)
@ -68,7 +67,7 @@ ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_, const Expressio
if (settings.max_temporary_columns && num_columns > settings.max_temporary_columns) if (settings.max_temporary_columns && num_columns > settings.max_temporary_columns)
throw Exception(ErrorCodes::TOO_MANY_TEMPORARY_COLUMNS, throw Exception(ErrorCodes::TOO_MANY_TEMPORARY_COLUMNS,
"Too many temporary columns: {}. Maximum: {}", "Too many temporary columns: {}. Maximum: {}",
actions_dag->dumpNames(), settings.max_temporary_columns); actions_dag.dumpNames(), settings.max_temporary_columns);
} }
ExpressionActionsPtr ExpressionActions::clone() const ExpressionActionsPtr ExpressionActions::clone() const
@ -76,12 +75,12 @@ ExpressionActionsPtr ExpressionActions::clone() const
auto copy = std::make_shared<ExpressionActions>(ExpressionActions()); auto copy = std::make_shared<ExpressionActions>(ExpressionActions());
std::unordered_map<const Node *, Node *> copy_map; std::unordered_map<const Node *, Node *> copy_map;
copy->actions_dag = ActionsDAG::clone(actions_dag.get(), copy_map); copy->actions_dag = std::move(*ActionsDAG::clone(&actions_dag, copy_map));
copy->actions = actions; copy->actions = actions;
for (auto & action : copy->actions) for (auto & action : copy->actions)
action.node = copy_map[action.node]; action.node = copy_map[action.node];
for (const auto * input : copy->actions_dag->getInputs()) for (const auto * input : copy->actions_dag.getInputs())
copy->input_positions.emplace(input->result_name, input_positions.at(input->result_name)); copy->input_positions.emplace(input->result_name, input_positions.at(input->result_name));
copy->num_columns = num_columns; copy->num_columns = num_columns;
@ -357,8 +356,8 @@ void ExpressionActions::linearizeActions(const std::unordered_set<const ActionsD
}; };
const auto & nodes = getNodes(); const auto & nodes = getNodes();
const auto & outputs = actions_dag->getOutputs(); const auto & outputs = actions_dag.getOutputs();
const auto & inputs = actions_dag->getInputs(); const auto & inputs = actions_dag.getInputs();
auto reverse_info = getActionsDAGReverseInfo(nodes, outputs); auto reverse_info = getActionsDAGReverseInfo(nodes, outputs);
std::vector<Data> data; std::vector<Data> data;

View File

@ -70,7 +70,7 @@ public:
using NameToInputMap = std::unordered_map<std::string_view, std::list<size_t>>; using NameToInputMap = std::unordered_map<std::string_view, std::list<size_t>>;
private: private:
ActionsDAGPtr actions_dag; ActionsDAG actions_dag;
Actions actions; Actions actions;
size_t num_columns = 0; size_t num_columns = 0;
@ -84,13 +84,13 @@ private:
ExpressionActionsSettings settings; ExpressionActionsSettings settings;
public: public:
explicit ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings_ = {}, bool project_inputs_ = false); explicit ExpressionActions(ActionsDAG actions_dag_, const ExpressionActionsSettings & settings_ = {}, bool project_inputs_ = false);
ExpressionActions(ExpressionActions &&) = default; ExpressionActions(ExpressionActions &&) = default;
ExpressionActions & operator=(ExpressionActions &&) = default; ExpressionActions & operator=(ExpressionActions &&) = default;
const Actions & getActions() const { return actions; } const Actions & getActions() const { return actions; }
const std::list<Node> & getNodes() const { return actions_dag->getNodes(); } const std::list<Node> & getNodes() const { return actions_dag.getNodes(); }
const ActionsDAG & getActionsDAG() const { return *actions_dag; } const ActionsDAG & getActionsDAG() const { return actions_dag; }
const ColumnNumbers & getResultPositions() const { return result_positions; } const ColumnNumbers & getResultPositions() const { return result_positions; }
const ExpressionActionsSettings & getSettings() const { return settings; } const ExpressionActionsSettings & getSettings() const { return settings; }

View File

@ -928,7 +928,7 @@ JoinPtr SelectQueryExpressionAnalyzer::appendJoin(
{ {
const ColumnsWithTypeAndName & left_sample_columns = chain.getLastStep().getResultColumns(); const ColumnsWithTypeAndName & left_sample_columns = chain.getLastStep().getResultColumns();
ActionsDAGPtr converting_actions; std::optional<ActionsDAG> converting_actions;
JoinPtr join = makeJoin(*syntax->ast_join, left_sample_columns, converting_actions); JoinPtr join = makeJoin(*syntax->ast_join, left_sample_columns, converting_actions);
if (converting_actions) if (converting_actions)
@ -1039,7 +1039,7 @@ static std::unique_ptr<QueryPlan> buildJoinedPlan(
/// Actions which need to be calculated on joined block. /// Actions which need to be calculated on joined block.
auto joined_block_actions = analyzed_join.createJoinedBlockActions(context); auto joined_block_actions = analyzed_join.createJoinedBlockActions(context);
NamesWithAliases required_columns_with_aliases = analyzed_join.getRequiredColumns( NamesWithAliases required_columns_with_aliases = analyzed_join.getRequiredColumns(
Block(joined_block_actions->getResultColumns()), joined_block_actions->getRequiredColumns().getNames()); Block(joined_block_actions.getResultColumns()), joined_block_actions.getRequiredColumns().getNames());
Names original_right_column_names; Names original_right_column_names;
for (auto & pr : required_columns_with_aliases) for (auto & pr : required_columns_with_aliases)
@ -1060,17 +1060,17 @@ static std::unique_ptr<QueryPlan> buildJoinedPlan(
interpreter->buildQueryPlan(*joined_plan); interpreter->buildQueryPlan(*joined_plan);
{ {
Block original_right_columns = interpreter->getSampleBlock(); Block original_right_columns = interpreter->getSampleBlock();
auto rename_dag = std::make_unique<ActionsDAG>(original_right_columns.getColumnsWithTypeAndName()); ActionsDAG rename_dag(original_right_columns.getColumnsWithTypeAndName());
for (const auto & name_with_alias : required_columns_with_aliases) for (const auto & name_with_alias : required_columns_with_aliases)
{ {
if (name_with_alias.first != name_with_alias.second && original_right_columns.has(name_with_alias.first)) if (name_with_alias.first != name_with_alias.second && original_right_columns.has(name_with_alias.first))
{ {
auto pos = original_right_columns.getPositionByName(name_with_alias.first); auto pos = original_right_columns.getPositionByName(name_with_alias.first);
const auto & alias = rename_dag->addAlias(*rename_dag->getInputs()[pos], name_with_alias.second); const auto & alias = rename_dag.addAlias(*rename_dag.getInputs()[pos], name_with_alias.second);
rename_dag->getOutputs()[pos] = &alias; rename_dag.getOutputs()[pos] = &alias;
} }
} }
rename_dag->appendInputsForUnusedColumns(joined_plan->getCurrentDataStream().header); rename_dag.appendInputsForUnusedColumns(joined_plan->getCurrentDataStream().header);
auto rename_step = std::make_unique<ExpressionStep>(joined_plan->getCurrentDataStream(), std::move(rename_dag)); auto rename_step = std::make_unique<ExpressionStep>(joined_plan->getCurrentDataStream(), std::move(rename_dag));
rename_step->setStepDescription("Rename joined columns"); rename_step->setStepDescription("Rename joined columns");
joined_plan->addStep(std::move(rename_step)); joined_plan->addStep(std::move(rename_step));
@ -1130,14 +1130,14 @@ std::shared_ptr<DirectKeyValueJoin> tryKeyValueJoin(std::shared_ptr<TableJoin> a
JoinPtr SelectQueryExpressionAnalyzer::makeJoin( JoinPtr SelectQueryExpressionAnalyzer::makeJoin(
const ASTTablesInSelectQueryElement & join_element, const ASTTablesInSelectQueryElement & join_element,
const ColumnsWithTypeAndName & left_columns, const ColumnsWithTypeAndName & left_columns,
ActionsDAGPtr & left_convert_actions) std::optional<ActionsDAG> & left_convert_actions)
{ {
/// Two JOINs are not supported with the same subquery, but different USINGs. /// Two JOINs are not supported with the same subquery, but different USINGs.
if (joined_plan) if (joined_plan)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table join was already created for query"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Table join was already created for query");
ActionsDAGPtr right_convert_actions = nullptr; std::optional<ActionsDAG> right_convert_actions;
const auto & analyzed_join = syntax->analyzed_join; const auto & analyzed_join = syntax->analyzed_join;
@ -1145,7 +1145,7 @@ JoinPtr SelectQueryExpressionAnalyzer::makeJoin(
{ {
auto joined_block_actions = analyzed_join->createJoinedBlockActions(getContext()); auto joined_block_actions = analyzed_join->createJoinedBlockActions(getContext());
NamesWithAliases required_columns_with_aliases = analyzed_join->getRequiredColumns( NamesWithAliases required_columns_with_aliases = analyzed_join->getRequiredColumns(
Block(joined_block_actions->getResultColumns()), joined_block_actions->getRequiredColumns().getNames()); Block(joined_block_actions.getResultColumns()), joined_block_actions.getRequiredColumns().getNames());
Names original_right_column_names; Names original_right_column_names;
for (auto & pr : required_columns_with_aliases) for (auto & pr : required_columns_with_aliases)
@ -1162,7 +1162,7 @@ JoinPtr SelectQueryExpressionAnalyzer::makeJoin(
std::tie(left_convert_actions, right_convert_actions) = analyzed_join->createConvertingActions(left_columns, right_columns); std::tie(left_convert_actions, right_convert_actions) = analyzed_join->createConvertingActions(left_columns, right_columns);
if (right_convert_actions) if (right_convert_actions)
{ {
auto converting_step = std::make_unique<ExpressionStep>(joined_plan->getCurrentDataStream(), right_convert_actions); auto converting_step = std::make_unique<ExpressionStep>(joined_plan->getCurrentDataStream(), std::move(*right_convert_actions));
converting_step->setStepDescription("Convert joined columns"); converting_step->setStepDescription("Convert joined columns");
joined_plan->addStep(std::move(converting_step)); joined_plan->addStep(std::move(converting_step));
} }
@ -1354,8 +1354,8 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain
{ {
for (auto & child : asts) for (auto & child : asts)
{ {
auto actions_dag = std::make_unique<ActionsDAG>(columns_after_join); ActionsDAG actions_dag(columns_after_join);
getRootActions(child, only_types, *actions_dag); getRootActions(child, only_types, actions_dag);
group_by_elements_actions.emplace_back( group_by_elements_actions.emplace_back(
std::make_shared<ExpressionActions>(std::move(actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes))); std::make_shared<ExpressionActions>(std::move(actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)));
} }
@ -1471,7 +1471,7 @@ void SelectQueryExpressionAnalyzer::appendGroupByModifiers(ActionsDAG & before_a
ExpressionActionsChain::Step & step = chain.addStep(before_aggregation.getNamesAndTypesList()); ExpressionActionsChain::Step & step = chain.addStep(before_aggregation.getNamesAndTypesList());
step.required_output = std::move(required_output); step.required_output = std::move(required_output);
step.actions()->dag = std::move(*ActionsDAG::makeConvertingActions(source_columns, result_columns, ActionsDAG::MatchColumnsMode::Position)); step.actions()->dag = ActionsDAG::makeConvertingActions(source_columns, result_columns, ActionsDAG::MatchColumnsMode::Position);
} }
void SelectQueryExpressionAnalyzer::appendSelectSkipWindowExpressions(ExpressionActionsChain::Step & step, ASTPtr const & node) void SelectQueryExpressionAnalyzer::appendSelectSkipWindowExpressions(ExpressionActionsChain::Step & step, ASTPtr const & node)
@ -1607,8 +1607,8 @@ ActionsAndProjectInputsFlagPtr SelectQueryExpressionAnalyzer::appendOrderBy(Expr
{ {
for (const auto & child : select_query->orderBy()->children) for (const auto & child : select_query->orderBy()->children)
{ {
auto actions_dag = std::make_unique<ActionsDAG>(columns_after_join); ActionsDAG actions_dag(columns_after_join);
getRootActions(child, only_types, *actions_dag); getRootActions(child, only_types, actions_dag);
order_by_elements_actions.emplace_back( order_by_elements_actions.emplace_back(
std::make_shared<ExpressionActions>(std::move(actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes))); std::make_shared<ExpressionActions>(std::move(actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)));
} }
@ -1737,7 +1737,7 @@ void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const
step.addRequiredOutput(expr->getColumnName()); step.addRequiredOutput(expr->getColumnName());
} }
ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool remove_unused_result) ActionsDAG ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool remove_unused_result)
{ {
ActionsDAG actions_dag(aggregated_columns); ActionsDAG actions_dag(aggregated_columns);
NamesWithAliases result_columns; NamesWithAliases result_columns;
@ -1789,7 +1789,7 @@ ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool remove_un
actions_dag.removeUnusedActions(name_set); actions_dag.removeUnusedActions(name_set);
} }
return std::make_unique<ActionsDAG>(std::move(actions_dag)); return actions_dag;
} }
ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool remove_unused_result, CompileExpressions compile_expressions) ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool remove_unused_result, CompileExpressions compile_expressions)
@ -1798,10 +1798,10 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool remov
getActionsDAG(add_aliases, remove_unused_result), ExpressionActionsSettings::fromContext(getContext(), compile_expressions), add_aliases && remove_unused_result); getActionsDAG(add_aliases, remove_unused_result), ExpressionActionsSettings::fromContext(getContext(), compile_expressions), add_aliases && remove_unused_result);
} }
ActionsDAGPtr ExpressionAnalyzer::getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs) ActionsDAG ExpressionAnalyzer::getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs)
{ {
auto actions = std::make_unique<ActionsDAG>(constant_inputs); ActionsDAG actions(constant_inputs);
getRootActions(query, true /* no_makeset_for_subqueries */, *actions, true /* only_consts */); getRootActions(query, true /* no_makeset_for_subqueries */, actions, true /* only_consts */);
return actions; return actions;
} }
@ -1879,8 +1879,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
if (prewhere_dag_and_flags) if (prewhere_dag_and_flags)
{ {
auto dag = std::make_unique<ActionsDAG>(std::move(prewhere_dag_and_flags->dag)); prewhere_info = std::make_shared<PrewhereInfo>(std::move(prewhere_dag_and_flags->dag), query.prewhere()->getColumnName());
prewhere_info = std::make_shared<PrewhereInfo>(std::move(dag), query.prewhere()->getColumnName());
prewhere_dag_and_flags.reset(); prewhere_dag_and_flags.reset();
} }
@ -1944,7 +1943,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
Block before_prewhere_sample = source_header; Block before_prewhere_sample = source_header;
if (sanitizeBlock(before_prewhere_sample)) if (sanitizeBlock(before_prewhere_sample))
{ {
auto dag = ActionsDAG::clone(&prewhere_dag_and_flags->dag); ActionsDAG dag = std::move(*ActionsDAG::clone(&prewhere_dag_and_flags->dag));
ExpressionActions( ExpressionActions(
std::move(dag), std::move(dag),
ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_prewhere_sample); ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_prewhere_sample);
@ -1980,7 +1979,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
if (sanitizeBlock(before_where_sample)) if (sanitizeBlock(before_where_sample))
{ {
ExpressionActions( ExpressionActions(
ActionsDAG::clone(&before_where->dag), std::move(*ActionsDAG::clone(&before_where->dag)),
ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_where_sample); ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_where_sample);
auto & column_elem auto & column_elem
@ -2054,7 +2053,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
auto & step = chain.lastStep(query_analyzer.aggregated_columns); auto & step = chain.lastStep(query_analyzer.aggregated_columns);
auto & actions = step.actions()->dag; auto & actions = step.actions()->dag;
actions = std::move(*ActionsDAG::merge(std::move(actions), std::move(*converting))); actions = ActionsDAG::merge(std::move(actions), std::move(converting));
} }
} }
@ -2235,7 +2234,7 @@ void ExpressionAnalysisResult::checkActions() const
/// Check that PREWHERE doesn't contain unusual actions. Unusual actions are that can change number of rows. /// Check that PREWHERE doesn't contain unusual actions. Unusual actions are that can change number of rows.
if (hasPrewhere()) if (hasPrewhere())
{ {
auto check_actions = [](const ActionsDAGPtr & actions) auto check_actions = [](const std::optional<ActionsDAG> & actions)
{ {
if (actions) if (actions)
for (const auto & node : actions->getNodes()) for (const auto & node : actions->getNodes())

View File

@ -117,12 +117,12 @@ public:
/// If add_aliases, only the calculated values in the desired order and add aliases. /// If add_aliases, only the calculated values in the desired order and add aliases.
/// If also remove_unused_result, than only aliases remain in the output block. /// If also remove_unused_result, than only aliases remain in the output block.
/// Otherwise, only temporary columns will be deleted from the block. /// Otherwise, only temporary columns will be deleted from the block.
ActionsDAGPtr getActionsDAG(bool add_aliases, bool remove_unused_result = true); ActionsDAG getActionsDAG(bool add_aliases, bool remove_unused_result = true);
ExpressionActionsPtr getActions(bool add_aliases, bool remove_unused_result = true, CompileExpressions compile_expressions = CompileExpressions::no); ExpressionActionsPtr getActions(bool add_aliases, bool remove_unused_result = true, CompileExpressions compile_expressions = CompileExpressions::no);
/// Get actions to evaluate a constant expression. The function adds constants and applies functions that depend only on constants. /// Get actions to evaluate a constant expression. The function adds constants and applies functions that depend only on constants.
/// Does not execute subqueries. /// Does not execute subqueries.
ActionsDAGPtr getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs = {}); ActionsDAG getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs = {});
ExpressionActionsPtr getConstActions(const ColumnsWithTypeAndName & constant_inputs = {}); ExpressionActionsPtr getConstActions(const ColumnsWithTypeAndName & constant_inputs = {});
/** Sets that require a subquery to be create. /** Sets that require a subquery to be create.
@ -367,7 +367,7 @@ private:
JoinPtr makeJoin( JoinPtr makeJoin(
const ASTTablesInSelectQueryElement & join_element, const ASTTablesInSelectQueryElement & join_element,
const ColumnsWithTypeAndName & left_columns, const ColumnsWithTypeAndName & left_columns,
ActionsDAGPtr & left_convert_actions); std::optional<ActionsDAG> & left_convert_actions);
const ASTSelectQuery * getAggregatingQuery() const; const ASTSelectQuery * getAggregatingQuery() const;

View File

@ -295,7 +295,7 @@ private:
{ {
auto joined_block_actions = data.table_join->createJoinedBlockActions(data.getContext()); auto joined_block_actions = data.table_join->createJoinedBlockActions(data.getContext());
NamesWithAliases required_columns_with_aliases = data.table_join->getRequiredColumns( NamesWithAliases required_columns_with_aliases = data.table_join->getRequiredColumns(
Block(joined_block_actions->getResultColumns()), joined_block_actions->getRequiredColumns().getNames()); Block(joined_block_actions.getResultColumns()), joined_block_actions.getRequiredColumns().getNames());
for (auto & pr : required_columns_with_aliases) for (auto & pr : required_columns_with_aliases)
required_columns.push_back(pr.first); required_columns.push_back(pr.first);

View File

@ -99,16 +99,16 @@ static ASTPtr parseAdditionalPostFilter(const Context & context)
"additional filter", settings.max_query_size, settings.max_parser_depth, settings.max_parser_backtracks); "additional filter", settings.max_query_size, settings.max_parser_depth, settings.max_parser_backtracks);
} }
static ActionsDAGPtr makeAdditionalPostFilter(ASTPtr & ast, ContextPtr context, const Block & header) static ActionsDAG makeAdditionalPostFilter(ASTPtr & ast, ContextPtr context, const Block & header)
{ {
auto syntax_result = TreeRewriter(context).analyze(ast, header.getNamesAndTypesList()); auto syntax_result = TreeRewriter(context).analyze(ast, header.getNamesAndTypesList());
String result_column_name = ast->getColumnName(); String result_column_name = ast->getColumnName();
auto dag = ExpressionAnalyzer(ast, syntax_result, context).getActionsDAG(false, false); auto dag = ExpressionAnalyzer(ast, syntax_result, context).getActionsDAG(false, false);
const ActionsDAG::Node * result_node = &dag->findInOutputs(result_column_name); const ActionsDAG::Node * result_node = &dag.findInOutputs(result_column_name);
auto & outputs = dag->getOutputs(); auto & outputs = dag.getOutputs();
outputs.clear(); outputs.clear();
outputs.reserve(dag->getInputs().size() + 1); outputs.reserve(dag.getInputs().size() + 1);
for (const auto * node : dag->getInputs()) for (const auto * node : dag.getInputs())
outputs.push_back(node); outputs.push_back(node);
outputs.push_back(result_node); outputs.push_back(result_node);
@ -126,7 +126,7 @@ void IInterpreterUnionOrSelectQuery::addAdditionalPostFilter(QueryPlan & plan) c
return; return;
auto dag = makeAdditionalPostFilter(ast, context, plan.getCurrentDataStream().header); auto dag = makeAdditionalPostFilter(ast, context, plan.getCurrentDataStream().header);
std::string filter_name = dag->getOutputs().back()->result_name; std::string filter_name = dag.getOutputs().back()->result_name;
auto filter_step = std::make_unique<FilterStep>( auto filter_step = std::make_unique<FilterStep>(
plan.getCurrentDataStream(), std::move(dag), std::move(filter_name), true); plan.getCurrentDataStream(), std::move(dag), std::move(filter_name), true);
filter_step->setStepDescription("Additional result filter"); filter_step->setStepDescription("Additional result filter");

View File

@ -175,7 +175,7 @@ FilterDAGInfoPtr generateFilterActions(
/// Using separate expression analyzer to prevent any possible alias injection /// Using separate expression analyzer to prevent any possible alias injection
auto syntax_result = TreeRewriter(context).analyzeSelect(query_ast, TreeRewriterResult({}, storage, storage_snapshot)); auto syntax_result = TreeRewriter(context).analyzeSelect(query_ast, TreeRewriterResult({}, storage, storage_snapshot));
SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, context, metadata_snapshot, {}, false, {}, prepared_sets); SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, context, metadata_snapshot, {}, false, {}, prepared_sets);
filter_info->actions = std::make_unique<ActionsDAG>(std::move(analyzer.simpleSelectActions()->dag)); filter_info->actions = std::move(analyzer.simpleSelectActions()->dag);
filter_info->column_name = expr_list->children.at(0)->getColumnName(); filter_info->column_name = expr_list->children.at(0)->getColumnName();
filter_info->actions->removeUnusedActions(NameSet{filter_info->column_name}); filter_info->actions->removeUnusedActions(NameSet{filter_info->column_name});
@ -938,7 +938,8 @@ bool InterpreterSelectQuery::adjustParallelReplicasAfterAnalysis()
} }
} }
query_info_copy.filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); if (auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes))
query_info_copy.filter_actions_dag = std::make_shared<const ActionsDAG>(std::move(*filter_actions_dag));
UInt64 rows_to_read = storage_merge_tree->estimateNumberOfRowsToRead(context, storage_snapshot, query_info_copy); UInt64 rows_to_read = storage_merge_tree->estimateNumberOfRowsToRead(context, storage_snapshot, query_info_copy);
/// Note that we treat an estimation of 0 rows as a real estimation /// Note that we treat an estimation of 0 rows as a real estimation
size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica; size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica;
@ -973,7 +974,7 @@ void InterpreterSelectQuery::buildQueryPlan(QueryPlan & query_plan)
ActionsDAG::MatchColumnsMode::Name, ActionsDAG::MatchColumnsMode::Name,
true); true);
auto converting = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), convert_actions_dag); auto converting = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), std::move(convert_actions_dag));
query_plan.addStep(std::move(converting)); query_plan.addStep(std::move(converting));
} }
@ -1297,10 +1298,10 @@ static InterpolateDescriptionPtr getInterpolateDescription(
auto syntax_result = TreeRewriter(context).analyze(exprs, source_columns); auto syntax_result = TreeRewriter(context).analyze(exprs, source_columns);
ExpressionAnalyzer analyzer(exprs, syntax_result, context); ExpressionAnalyzer analyzer(exprs, syntax_result, context);
ActionsDAGPtr actions = analyzer.getActionsDAG(true); ActionsDAG actions = analyzer.getActionsDAG(true);
ActionsDAGPtr conv_dag = ActionsDAG::makeConvertingActions(actions->getResultColumns(), ActionsDAG conv_dag = ActionsDAG::makeConvertingActions(actions.getResultColumns(),
result_columns, ActionsDAG::MatchColumnsMode::Position, true); result_columns, ActionsDAG::MatchColumnsMode::Position, true);
ActionsDAGPtr merge_dag = ActionsDAG::merge(std::move(* ActionsDAG::clone(actions)), std::move(*conv_dag)); ActionsDAG merge_dag = ActionsDAG::merge(std::move(actions), std::move(conv_dag));
interpolate_descr = std::make_shared<InterpolateDescription>(std::move(merge_dag), aliases); interpolate_descr = std::make_shared<InterpolateDescription>(std::move(merge_dag), aliases);
} }
@ -1485,7 +1486,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
{ {
auto row_level_security_step = std::make_unique<FilterStep>( auto row_level_security_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(), query_plan.getCurrentDataStream(),
expressions.filter_info->actions, std::move(*ActionsDAG::clone(&*expressions.filter_info->actions)),
expressions.filter_info->column_name, expressions.filter_info->column_name,
expressions.filter_info->do_remove_column); expressions.filter_info->do_remove_column);
@ -1499,7 +1500,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
{ {
auto row_level_filter_step = std::make_unique<FilterStep>( auto row_level_filter_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(), query_plan.getCurrentDataStream(),
expressions.prewhere_info->row_level_filter, std::move(*ActionsDAG::clone(&*expressions.prewhere_info->row_level_filter)),
expressions.prewhere_info->row_level_column_name, expressions.prewhere_info->row_level_column_name,
true); true);
@ -1509,7 +1510,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
auto prewhere_step = std::make_unique<FilterStep>( auto prewhere_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(), query_plan.getCurrentDataStream(),
expressions.prewhere_info->prewhere_actions, std::move(*ActionsDAG::clone(&*expressions.prewhere_info->prewhere_actions)),
expressions.prewhere_info->prewhere_column_name, expressions.prewhere_info->prewhere_column_name,
expressions.prewhere_info->remove_prewhere_column); expressions.prewhere_info->remove_prewhere_column);
@ -1611,7 +1612,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
{ {
auto row_level_security_step = std::make_unique<FilterStep>( auto row_level_security_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(), query_plan.getCurrentDataStream(),
expressions.filter_info->actions, std::move(*ActionsDAG::clone(&*expressions.filter_info->actions)),
expressions.filter_info->column_name, expressions.filter_info->column_name,
expressions.filter_info->do_remove_column); expressions.filter_info->do_remove_column);
@ -1623,7 +1624,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
{ {
auto filter_step = std::make_unique<FilterStep>( auto filter_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(), query_plan.getCurrentDataStream(),
new_filter_info->actions, std::move(*ActionsDAG::clone(&*new_filter_info->actions)),
new_filter_info->column_name, new_filter_info->column_name,
new_filter_info->do_remove_column); new_filter_info->do_remove_column);
@ -2045,7 +2046,7 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c
pipe.addSimpleTransform([&](const Block & header) pipe.addSimpleTransform([&](const Block & header)
{ {
return std::make_shared<FilterTransform>(header, return std::make_shared<FilterTransform>(header,
std::make_shared<ExpressionActions>(ActionsDAG::clone(prewhere_info.row_level_filter)), std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(&*prewhere_info.row_level_filter))),
prewhere_info.row_level_column_name, true); prewhere_info.row_level_column_name, true);
}); });
} }
@ -2053,7 +2054,7 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c
pipe.addSimpleTransform([&](const Block & header) pipe.addSimpleTransform([&](const Block & header)
{ {
return std::make_shared<FilterTransform>( return std::make_shared<FilterTransform>(
header, std::make_shared<ExpressionActions>(ActionsDAG::clone(prewhere_info.prewhere_actions)), header, std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(&*prewhere_info.prewhere_actions))),
prewhere_info.prewhere_column_name, prewhere_info.remove_prewhere_column); prewhere_info.prewhere_column_name, prewhere_info.remove_prewhere_column);
}); });
} }
@ -2106,7 +2107,7 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis
else else
{ {
/// Add row level security actions to prewhere. /// Add row level security actions to prewhere.
analysis.prewhere_info->row_level_filter = std::move(analysis.filter_info->actions); analysis.prewhere_info->row_level_filter = std::move(*analysis.filter_info->actions);
analysis.prewhere_info->row_level_column_name = std::move(analysis.filter_info->column_name); analysis.prewhere_info->row_level_column_name = std::move(analysis.filter_info->column_name);
analysis.filter_info = nullptr; analysis.filter_info = nullptr;
} }
@ -2323,7 +2324,7 @@ std::optional<UInt64> InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle
if (!filter_actions_dag) if (!filter_actions_dag)
return {}; return {};
return storage->totalRowsByPartitionPredicate(filter_actions_dag, context); return storage->totalRowsByPartitionPredicate(*filter_actions_dag, context);
} }
} }
@ -2573,7 +2574,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
/// Aliases in table declaration. /// Aliases in table declaration.
if (processing_stage == QueryProcessingStage::FetchColumns && alias_actions) if (processing_stage == QueryProcessingStage::FetchColumns && alias_actions)
{ {
auto table_aliases = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), alias_actions); auto table_aliases = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), std::move(*ActionsDAG::clone(&*alias_actions)));
table_aliases->setStepDescription("Add table aliases"); table_aliases->setStepDescription("Add table aliases");
query_plan.addStep(std::move(table_aliases)); query_plan.addStep(std::move(table_aliases));
} }
@ -2581,9 +2582,9 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter) void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter)
{ {
auto dag = ActionsDAG::clone(&expression->dag); auto dag = std::move(*ActionsDAG::clone(&expression->dag));
if (expression->project_input) if (expression->project_input)
dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); dag.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
auto where_step = std::make_unique<FilterStep>( auto where_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(), std::move(dag), getSelectQuery().where()->getColumnName(), remove_filter); query_plan.getCurrentDataStream(), std::move(dag), getSelectQuery().where()->getColumnName(), remove_filter);
@ -2755,9 +2756,9 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool
void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter) void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter)
{ {
auto dag = ActionsDAG::clone(&expression->dag); auto dag = std::move(*ActionsDAG::clone(&expression->dag));
if (expression->project_input) if (expression->project_input)
dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); dag.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
auto having_step auto having_step
= std::make_unique<FilterStep>(query_plan.getCurrentDataStream(), std::move(dag), getSelectQuery().having()->getColumnName(), remove_filter); = std::make_unique<FilterStep>(query_plan.getCurrentDataStream(), std::move(dag), getSelectQuery().having()->getColumnName(), remove_filter);
@ -2770,10 +2771,10 @@ void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const Actions
void InterpreterSelectQuery::executeTotalsAndHaving( void InterpreterSelectQuery::executeTotalsAndHaving(
QueryPlan & query_plan, bool has_having, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter, bool overflow_row, bool final) QueryPlan & query_plan, bool has_having, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter, bool overflow_row, bool final)
{ {
ActionsDAGPtr dag; std::optional<ActionsDAG> dag;
if (expression) if (expression)
{ {
dag = ActionsDAG::clone(&expression->dag); dag = std::move(*ActionsDAG::clone(&expression->dag));
if (expression->project_input) if (expression->project_input)
dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
} }
@ -2822,9 +2823,9 @@ void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const Act
if (!expression) if (!expression)
return; return;
auto dag = ActionsDAG::clone(&expression->dag); ActionsDAG dag = std::move(*ActionsDAG::clone(&expression->dag));
if (expression->project_input) if (expression->project_input)
dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); dag.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
auto expression_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), std::move(dag)); auto expression_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), std::move(dag));

View File

@ -240,7 +240,7 @@ private:
Block source_header; Block source_header;
/// Actions to calculate ALIAS if required. /// Actions to calculate ALIAS if required.
ActionsDAGPtr alias_actions; std::optional<ActionsDAG> alias_actions;
/// The subquery interpreter, if the subquery /// The subquery interpreter, if the subquery
std::unique_ptr<InterpreterSelectWithUnionQuery> interpreter_subquery; std::unique_ptr<InterpreterSelectWithUnionQuery> interpreter_subquery;

View File

@ -1197,7 +1197,7 @@ void MutationsInterpreter::Source::read(
const auto & names = first_stage.filter_column_names; const auto & names = first_stage.filter_column_names;
size_t num_filters = names.size(); size_t num_filters = names.size();
ActionsDAGPtr filter; std::optional<ActionsDAG> filter;
if (!first_stage.filter_column_names.empty()) if (!first_stage.filter_column_names.empty())
{ {
ActionsDAG::NodeRawConstPtrs nodes(num_filters); ActionsDAG::NodeRawConstPtrs nodes(num_filters);
@ -1278,19 +1278,19 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v
if (i < stage.filter_column_names.size()) if (i < stage.filter_column_names.size())
{ {
auto dag = ActionsDAG::clone(&step->actions()->dag); auto dag = std::move(*ActionsDAG::clone(&step->actions()->dag));
if (step->actions()->project_input) if (step->actions()->project_input)
dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header); dag.appendInputsForUnusedColumns(plan.getCurrentDataStream().header);
/// Execute DELETEs. /// Execute DELETEs.
plan.addStep(std::make_unique<FilterStep>(plan.getCurrentDataStream(), dag, stage.filter_column_names[i], false)); plan.addStep(std::make_unique<FilterStep>(plan.getCurrentDataStream(), std::move(dag), stage.filter_column_names[i], false));
} }
else else
{ {
auto dag = ActionsDAG::clone(&step->actions()->dag); auto dag = std::move(*ActionsDAG::clone(&step->actions()->dag));
if (step->actions()->project_input) if (step->actions()->project_input)
dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header); dag.appendInputsForUnusedColumns(plan.getCurrentDataStream().header);
/// Execute UPDATE or final projection. /// Execute UPDATE or final projection.
plan.addStep(std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), dag)); plan.addStep(std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), std::move(dag)));
} }
} }

View File

@ -182,7 +182,7 @@ static NamesAndTypesList getNames(const ASTFunction & expr, ContextPtr context,
ASTPtr temp_ast = expr.clone(); ASTPtr temp_ast = expr.clone();
auto syntax = TreeRewriter(context).analyze(temp_ast, columns); auto syntax = TreeRewriter(context).analyze(temp_ast, columns);
auto required_columns = ExpressionAnalyzer(temp_ast, syntax, context).getActionsDAG(false)->getRequiredColumns(); auto required_columns = ExpressionAnalyzer(temp_ast, syntax, context).getActionsDAG(false).getRequiredColumns();
return required_columns; return required_columns;
} }

View File

@ -462,19 +462,19 @@ static void makeColumnNameUnique(const ColumnsWithTypeAndName & source_columns,
} }
} }
static ActionsDAGPtr createWrapWithTupleActions( static std::optional<ActionsDAG> createWrapWithTupleActions(
const ColumnsWithTypeAndName & source_columns, const ColumnsWithTypeAndName & source_columns,
std::unordered_set<std::string_view> && column_names_to_wrap, std::unordered_set<std::string_view> && column_names_to_wrap,
NameToNameMap & new_names) NameToNameMap & new_names)
{ {
if (column_names_to_wrap.empty()) if (column_names_to_wrap.empty())
return nullptr; return {};
auto actions_dag = std::make_unique<ActionsDAG>(source_columns); ActionsDAG actions_dag(source_columns);
FunctionOverloadResolverPtr func_builder = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionTuple>()); FunctionOverloadResolverPtr func_builder = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionTuple>());
for (const auto * input_node : actions_dag->getInputs()) for (const auto * input_node : actions_dag.getInputs())
{ {
const auto & column_name = input_node->result_name; const auto & column_name = input_node->result_name;
auto it = column_names_to_wrap.find(column_name); auto it = column_names_to_wrap.find(column_name);
@ -485,9 +485,9 @@ static ActionsDAGPtr createWrapWithTupleActions(
String node_name = "__wrapNullsafe(" + column_name + ")"; String node_name = "__wrapNullsafe(" + column_name + ")";
makeColumnNameUnique(source_columns, node_name); makeColumnNameUnique(source_columns, node_name);
const auto & dst_node = actions_dag->addFunction(func_builder, {input_node}, node_name); const auto & dst_node = actions_dag.addFunction(func_builder, {input_node}, node_name);
new_names[column_name] = dst_node.result_name; new_names[column_name] = dst_node.result_name;
actions_dag->addOrReplaceInOutputs(dst_node); actions_dag.addOrReplaceInOutputs(dst_node);
} }
if (!column_names_to_wrap.empty()) if (!column_names_to_wrap.empty())
@ -537,21 +537,23 @@ std::pair<NameSet, NameSet> TableJoin::getKeysForNullSafeComparion(const Columns
return {left_keys_to_wrap, right_keys_to_wrap}; return {left_keys_to_wrap, right_keys_to_wrap};
} }
static void mergeDags(ActionsDAGPtr & result_dag, ActionsDAGPtr && new_dag) static void mergeDags(std::optional<ActionsDAG> & result_dag, std::optional<ActionsDAG> && new_dag)
{ {
if (!new_dag)
return;
if (result_dag) if (result_dag)
result_dag->mergeInplace(std::move(*new_dag)); result_dag->mergeInplace(std::move(*new_dag));
else else
result_dag = std::move(new_dag); result_dag = std::move(new_dag);
} }
std::pair<ActionsDAGPtr, ActionsDAGPtr> std::pair<std::optional<ActionsDAG>, std::optional<ActionsDAG>>
TableJoin::createConvertingActions( TableJoin::createConvertingActions(
const ColumnsWithTypeAndName & left_sample_columns, const ColumnsWithTypeAndName & left_sample_columns,
const ColumnsWithTypeAndName & right_sample_columns) const ColumnsWithTypeAndName & right_sample_columns)
{ {
ActionsDAGPtr left_dag = nullptr; std::optional<ActionsDAG> left_dag;
ActionsDAGPtr right_dag = nullptr; std::optional<ActionsDAG> right_dag;
/** If the types are not equal, we need to convert them to a common type. /** If the types are not equal, we need to convert them to a common type.
* Example: * Example:
* SELECT * FROM t1 JOIN t2 ON t1.a = t2.b * SELECT * FROM t1 JOIN t2 ON t1.a = t2.b
@ -693,7 +695,7 @@ void TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig
} }
} }
static ActionsDAGPtr changeKeyTypes(const ColumnsWithTypeAndName & cols_src, static std::optional<ActionsDAG> changeKeyTypes(const ColumnsWithTypeAndName & cols_src,
const TableJoin::NameToTypeMap & type_mapping, const TableJoin::NameToTypeMap & type_mapping,
bool add_new_cols, bool add_new_cols,
NameToNameMap & key_column_rename) NameToNameMap & key_column_rename)
@ -710,7 +712,7 @@ static ActionsDAGPtr changeKeyTypes(const ColumnsWithTypeAndName & cols_src,
} }
} }
if (!has_some_to_do) if (!has_some_to_do)
return nullptr; return {};
return ActionsDAG::makeConvertingActions( return ActionsDAG::makeConvertingActions(
/* source= */ cols_src, /* source= */ cols_src,
@ -721,7 +723,7 @@ static ActionsDAGPtr changeKeyTypes(const ColumnsWithTypeAndName & cols_src,
/* new_names= */ &key_column_rename); /* new_names= */ &key_column_rename);
} }
static ActionsDAGPtr changeTypesToNullable( static std::optional<ActionsDAG> changeTypesToNullable(
const ColumnsWithTypeAndName & cols_src, const ColumnsWithTypeAndName & cols_src,
const NameSet & exception_cols) const NameSet & exception_cols)
{ {
@ -737,7 +739,7 @@ static ActionsDAGPtr changeTypesToNullable(
} }
if (!has_some_to_do) if (!has_some_to_do)
return nullptr; return {};
return ActionsDAG::makeConvertingActions( return ActionsDAG::makeConvertingActions(
/* source= */ cols_src, /* source= */ cols_src,
@ -748,29 +750,29 @@ static ActionsDAGPtr changeTypesToNullable(
/* new_names= */ nullptr); /* new_names= */ nullptr);
} }
ActionsDAGPtr TableJoin::applyKeyConvertToTable( std::optional<ActionsDAG> TableJoin::applyKeyConvertToTable(
const ColumnsWithTypeAndName & cols_src, const ColumnsWithTypeAndName & cols_src,
const NameToTypeMap & type_mapping, const NameToTypeMap & type_mapping,
JoinTableSide table_side, JoinTableSide table_side,
NameToNameMap & key_column_rename) NameToNameMap & key_column_rename)
{ {
if (type_mapping.empty()) if (type_mapping.empty())
return nullptr; return {};
/// Create DAG to convert key columns /// Create DAG to convert key columns
ActionsDAGPtr convert_dag = changeKeyTypes(cols_src, type_mapping, !hasUsing(), key_column_rename); auto convert_dag = changeKeyTypes(cols_src, type_mapping, !hasUsing(), key_column_rename);
applyRename(table_side, key_column_rename); applyRename(table_side, key_column_rename);
return convert_dag; return convert_dag;
} }
ActionsDAGPtr TableJoin::applyNullsafeWrapper( std::optional<ActionsDAG> TableJoin::applyNullsafeWrapper(
const ColumnsWithTypeAndName & cols_src, const ColumnsWithTypeAndName & cols_src,
const NameSet & columns_for_nullsafe_comparison, const NameSet & columns_for_nullsafe_comparison,
JoinTableSide table_side, JoinTableSide table_side,
NameToNameMap & key_column_rename) NameToNameMap & key_column_rename)
{ {
if (columns_for_nullsafe_comparison.empty()) if (columns_for_nullsafe_comparison.empty())
return nullptr; return {};
std::unordered_set<std::string_view> column_names_to_wrap; std::unordered_set<std::string_view> column_names_to_wrap;
for (const auto & name : columns_for_nullsafe_comparison) for (const auto & name : columns_for_nullsafe_comparison)
@ -784,7 +786,7 @@ ActionsDAGPtr TableJoin::applyNullsafeWrapper(
} }
/// Create DAG to wrap keys with tuple for null-safe comparison /// Create DAG to wrap keys with tuple for null-safe comparison
ActionsDAGPtr null_safe_wrap_dag = createWrapWithTupleActions(cols_src, std::move(column_names_to_wrap), key_column_rename); auto null_safe_wrap_dag = createWrapWithTupleActions(cols_src, std::move(column_names_to_wrap), key_column_rename);
for (auto & clause : clauses) for (auto & clause : clauses)
{ {
for (size_t i : clause.nullsafe_compare_key_indexes) for (size_t i : clause.nullsafe_compare_key_indexes)
@ -799,7 +801,7 @@ ActionsDAGPtr TableJoin::applyNullsafeWrapper(
return null_safe_wrap_dag; return null_safe_wrap_dag;
} }
ActionsDAGPtr TableJoin::applyJoinUseNullsConversion( std::optional<ActionsDAG> TableJoin::applyJoinUseNullsConversion(
const ColumnsWithTypeAndName & cols_src, const ColumnsWithTypeAndName & cols_src,
const NameToNameMap & key_column_rename) const NameToNameMap & key_column_rename)
{ {
@ -809,8 +811,7 @@ ActionsDAGPtr TableJoin::applyJoinUseNullsConversion(
exclude_columns.insert(it.second); exclude_columns.insert(it.second);
/// Create DAG to make columns nullable if needed /// Create DAG to make columns nullable if needed
ActionsDAGPtr add_nullable_dag = changeTypesToNullable(cols_src, exclude_columns); return changeTypesToNullable(cols_src, exclude_columns);
return add_nullable_dag;
} }
void TableJoin::setStorageJoin(std::shared_ptr<const IKeyValueEntity> storage) void TableJoin::setStorageJoin(std::shared_ptr<const IKeyValueEntity> storage)
@ -957,7 +958,7 @@ bool TableJoin::allowParallelHashJoin() const
return true; return true;
} }
ActionsDAGPtr TableJoin::createJoinedBlockActions(ContextPtr context) const ActionsDAG TableJoin::createJoinedBlockActions(ContextPtr context) const
{ {
ASTPtr expression_list = rightKeysList(); ASTPtr expression_list = rightKeysList();
auto syntax_result = TreeRewriter(context).analyze(expression_list, columnsFromJoinedTable()); auto syntax_result = TreeRewriter(context).analyze(expression_list, columnsFromJoinedTable());

View File

@ -202,19 +202,19 @@ private:
Names requiredJoinedNames() const; Names requiredJoinedNames() const;
/// Create converting actions and change key column names if required /// Create converting actions and change key column names if required
ActionsDAGPtr applyKeyConvertToTable( std::optional<ActionsDAG> applyKeyConvertToTable(
const ColumnsWithTypeAndName & cols_src, const ColumnsWithTypeAndName & cols_src,
const NameToTypeMap & type_mapping, const NameToTypeMap & type_mapping,
JoinTableSide table_side, JoinTableSide table_side,
NameToNameMap & key_column_rename); NameToNameMap & key_column_rename);
ActionsDAGPtr applyNullsafeWrapper( std::optional<ActionsDAG> applyNullsafeWrapper(
const ColumnsWithTypeAndName & cols_src, const ColumnsWithTypeAndName & cols_src,
const NameSet & columns_for_nullsafe_comparison, const NameSet & columns_for_nullsafe_comparison,
JoinTableSide table_side, JoinTableSide table_side,
NameToNameMap & key_column_rename); NameToNameMap & key_column_rename);
ActionsDAGPtr applyJoinUseNullsConversion( std::optional<ActionsDAG> applyJoinUseNullsConversion(
const ColumnsWithTypeAndName & cols_src, const ColumnsWithTypeAndName & cols_src,
const NameToNameMap & key_column_rename); const NameToNameMap & key_column_rename);
@ -264,7 +264,7 @@ public:
TemporaryDataOnDiskScopePtr getTempDataOnDisk() { return tmp_data; } TemporaryDataOnDiskScopePtr getTempDataOnDisk() { return tmp_data; }
ActionsDAGPtr createJoinedBlockActions(ContextPtr context) const; ActionsDAG createJoinedBlockActions(ContextPtr context) const;
const std::vector<JoinAlgorithm> & getEnabledJoinAlgorithms() const { return join_algorithm; } const std::vector<JoinAlgorithm> & getEnabledJoinAlgorithms() const { return join_algorithm; }
@ -379,7 +379,7 @@ public:
/// Calculate converting actions, rename key columns in required /// Calculate converting actions, rename key columns in required
/// For `USING` join we will convert key columns inplace and affect into types in the result table /// For `USING` join we will convert key columns inplace and affect into types in the result table
/// For `JOIN ON` we will create new columns with converted keys to join by. /// For `JOIN ON` we will create new columns with converted keys to join by.
std::pair<ActionsDAGPtr, ActionsDAGPtr> std::pair<std::optional<ActionsDAG>, std::optional<ActionsDAG>>
createConvertingActions( createConvertingActions(
const ColumnsWithTypeAndName & left_sample_columns, const ColumnsWithTypeAndName & left_sample_columns,
const ColumnsWithTypeAndName & right_sample_columns); const ColumnsWithTypeAndName & right_sample_columns);

View File

@ -14,15 +14,15 @@
namespace DB namespace DB
{ {
ActionsDAGPtr addMissingDefaults( ActionsDAG addMissingDefaults(
const Block & header, const Block & header,
const NamesAndTypesList & required_columns, const NamesAndTypesList & required_columns,
const ColumnsDescription & columns, const ColumnsDescription & columns,
ContextPtr context, ContextPtr context,
bool null_as_default) bool null_as_default)
{ {
auto actions = std::make_unique<ActionsDAG>(header.getColumnsWithTypeAndName()); ActionsDAG actions(header.getColumnsWithTypeAndName());
auto & index = actions->getOutputs(); auto & index = actions.getOutputs();
/// For missing columns of nested structure, you need to create not a column of empty arrays, but a column of arrays of correct lengths. /// For missing columns of nested structure, you need to create not a column of empty arrays, but a column of arrays of correct lengths.
/// First, remember the offset columns for all arrays in the block. /// First, remember the offset columns for all arrays in the block.
@ -40,7 +40,7 @@ ActionsDAGPtr addMissingDefaults(
if (group.empty()) if (group.empty())
group.push_back(nullptr); group.push_back(nullptr);
group.push_back(actions->getInputs()[i]); group.push_back(actions.getInputs()[i]);
} }
} }
@ -62,11 +62,11 @@ ActionsDAGPtr addMissingDefaults(
{ {
const auto & nested_type = array_type->getNestedType(); const auto & nested_type = array_type->getNestedType();
ColumnPtr nested_column = nested_type->createColumnConstWithDefaultValue(0); ColumnPtr nested_column = nested_type->createColumnConstWithDefaultValue(0);
const auto & constant = actions->addColumn({nested_column, nested_type, column.name}); const auto & constant = actions.addColumn({nested_column, nested_type, column.name});
auto & group = nested_groups[offsets_name]; auto & group = nested_groups[offsets_name];
group[0] = &constant; group[0] = &constant;
index.push_back(&actions->addFunction(func_builder_replicate, group, constant.result_name)); index.push_back(&actions.addFunction(func_builder_replicate, group, constant.result_name));
continue; continue;
} }
@ -75,17 +75,17 @@ ActionsDAGPtr addMissingDefaults(
* it can be full (or the interpreter may decide that it is constant everywhere). * it can be full (or the interpreter may decide that it is constant everywhere).
*/ */
auto new_column = column.type->createColumnConstWithDefaultValue(0); auto new_column = column.type->createColumnConstWithDefaultValue(0);
const auto * col = &actions->addColumn({new_column, column.type, column.name}); const auto * col = &actions.addColumn({new_column, column.type, column.name});
index.push_back(&actions->materializeNode(*col)); index.push_back(&actions.materializeNode(*col));
} }
/// Computes explicitly specified values by default and materialized columns. /// Computes explicitly specified values by default and materialized columns.
if (auto dag = evaluateMissingDefaults(actions->getResultColumns(), required_columns, columns, context, true, null_as_default)) if (auto dag = evaluateMissingDefaults(actions.getResultColumns(), required_columns, columns, context, true, null_as_default))
actions = ActionsDAG::merge(std::move(*actions), std::move(*dag)); actions = ActionsDAG::merge(std::move(actions), std::move(*dag));
/// Removes unused columns and reorders result. /// Removes unused columns and reorders result.
actions->removeUnusedActions(required_columns.getNames(), false); actions.removeUnusedActions(required_columns.getNames(), false);
actions->addMaterializingOutputActions(); actions.addMaterializingOutputActions();
return actions; return actions;
} }

View File

@ -24,7 +24,7 @@ using ActionsDAGPtr = std::unique_ptr<ActionsDAG>;
* Also can substitute NULL with DEFAULT value in case of INSERT SELECT query (null_as_default) if according setting is 1. * Also can substitute NULL with DEFAULT value in case of INSERT SELECT query (null_as_default) if according setting is 1.
* All three types of columns are materialized (not constants). * All three types of columns are materialized (not constants).
*/ */
ActionsDAGPtr addMissingDefaults( ActionsDAG addMissingDefaults(
const Block & header, const NamesAndTypesList & required_columns, const Block & header, const NamesAndTypesList & required_columns,
const ColumnsDescription & columns, ContextPtr context, bool null_as_default = false); const ColumnsDescription & columns, ContextPtr context, bool null_as_default = false);
} }

View File

@ -89,7 +89,7 @@ std::optional<EvaluateConstantExpressionResult> evaluateConstantExpressionImpl(c
ColumnPtr result_column; ColumnPtr result_column;
DataTypePtr result_type; DataTypePtr result_type;
String result_name = ast->getColumnName(); String result_name = ast->getColumnName();
for (const auto & action_node : actions->getOutputs()) for (const auto & action_node : actions.getOutputs())
{ {
if ((action_node->result_name == result_name) && action_node->column) if ((action_node->result_name == result_name) && action_node->column)
{ {

View File

@ -152,22 +152,20 @@ ASTPtr convertRequiredExpressions(Block & block, const NamesAndTypesList & requi
return conversion_expr_list; return conversion_expr_list;
} }
ActionsDAGPtr createExpressions( std::optional<ActionsDAG> createExpressions(
const Block & header, const Block & header,
ASTPtr expr_list, ASTPtr expr_list,
bool save_unneeded_columns, bool save_unneeded_columns,
ContextPtr context) ContextPtr context)
{ {
if (!expr_list) if (!expr_list)
return nullptr; return {};
auto syntax_result = TreeRewriter(context).analyze(expr_list, header.getNamesAndTypesList()); auto syntax_result = TreeRewriter(context).analyze(expr_list, header.getNamesAndTypesList());
auto expression_analyzer = ExpressionAnalyzer{expr_list, syntax_result, context}; auto expression_analyzer = ExpressionAnalyzer{expr_list, syntax_result, context};
auto dag = std::make_unique<ActionsDAG>(header.getNamesAndTypesList()); ActionsDAG dag(header.getNamesAndTypesList());
auto actions = expression_analyzer.getActionsDAG(true, !save_unneeded_columns); auto actions = expression_analyzer.getActionsDAG(true, !save_unneeded_columns);
dag = ActionsDAG::merge(std::move(*dag), std::move(*actions)); return ActionsDAG::merge(std::move(dag), std::move(actions));
return dag;
} }
} }
@ -180,7 +178,7 @@ void performRequiredConversions(Block & block, const NamesAndTypesList & require
if (auto dag = createExpressions(block, conversion_expr_list, true, context)) if (auto dag = createExpressions(block, conversion_expr_list, true, context))
{ {
auto expression = std::make_shared<ExpressionActions>(std::move(dag), ExpressionActionsSettings::fromContext(context)); auto expression = std::make_shared<ExpressionActions>(std::move(*dag), ExpressionActionsSettings::fromContext(context));
expression->execute(block); expression->execute(block);
} }
} }
@ -195,7 +193,7 @@ bool needConvertAnyNullToDefault(const Block & header, const NamesAndTypesList &
return false; return false;
} }
ActionsDAGPtr evaluateMissingDefaults( std::optional<ActionsDAG> evaluateMissingDefaults(
const Block & header, const Block & header,
const NamesAndTypesList & required_columns, const NamesAndTypesList & required_columns,
const ColumnsDescription & columns, const ColumnsDescription & columns,
@ -204,7 +202,7 @@ ActionsDAGPtr evaluateMissingDefaults(
bool null_as_default) bool null_as_default)
{ {
if (!columns.hasDefaults() && (!null_as_default || !needConvertAnyNullToDefault(header, required_columns, columns))) if (!columns.hasDefaults() && (!null_as_default || !needConvertAnyNullToDefault(header, required_columns, columns)))
return nullptr; return {};
ASTPtr expr_list = defaultRequiredExpressions(header, required_columns, columns, null_as_default); ASTPtr expr_list = defaultRequiredExpressions(header, required_columns, columns, null_as_default);
return createExpressions(header, expr_list, save_unneeded_columns, context); return createExpressions(header, expr_list, save_unneeded_columns, context);

View File

@ -5,9 +5,6 @@
#include <Common/COW.h> #include <Common/COW.h>
#include <memory> #include <memory>
#include <string>
#include <unordered_map>
namespace DB namespace DB
{ {
@ -24,12 +21,11 @@ struct StorageInMemoryMetadata;
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>; using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
class ActionsDAG; class ActionsDAG;
using ActionsDAGPtr = std::unique_ptr<ActionsDAG>;
/// Create actions which adds missing defaults to block according to required_columns using columns description /// Create actions which adds missing defaults to block according to required_columns using columns description
/// or substitute NULL into DEFAULT value in case of INSERT SELECT query (null_as_default) if according setting is 1. /// or substitute NULL into DEFAULT value in case of INSERT SELECT query (null_as_default) if according setting is 1.
/// Return nullptr if no actions required. /// Return nullptr if no actions required.
ActionsDAGPtr evaluateMissingDefaults( std::optional<ActionsDAG> evaluateMissingDefaults(
const Block & header, const Block & header,
const NamesAndTypesList & required_columns, const NamesAndTypesList & required_columns,
const ColumnsDescription & columns, const ColumnsDescription & columns,

View File

@ -332,12 +332,12 @@ void addExpressionStep(QueryPlan & query_plan,
const std::string & step_description, const std::string & step_description,
UsefulSets & useful_sets) UsefulSets & useful_sets)
{ {
auto actions = ActionsDAG::clone(&expression_actions->dag); auto actions = std::move(*ActionsDAG::clone(&expression_actions->dag));
if (expression_actions->project_input) if (expression_actions->project_input)
actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); actions.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
auto expression_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), actions); auto expression_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), std::move(actions));
appendSetsFromActionsDAG(*expression_step->getExpression(), useful_sets); appendSetsFromActionsDAG(expression_step->getExpression(), useful_sets);
expression_step->setStepDescription(step_description); expression_step->setStepDescription(step_description);
query_plan.addStep(std::move(expression_step)); query_plan.addStep(std::move(expression_step));
} }
@ -347,15 +347,15 @@ void addFilterStep(QueryPlan & query_plan,
const std::string & step_description, const std::string & step_description,
UsefulSets & useful_sets) UsefulSets & useful_sets)
{ {
auto actions = ActionsDAG::clone(&filter_analysis_result.filter_actions->dag); auto actions = std::move(*ActionsDAG::clone(&filter_analysis_result.filter_actions->dag));
if (filter_analysis_result.filter_actions->project_input) if (filter_analysis_result.filter_actions->project_input)
actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); actions.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
auto where_step = std::make_unique<FilterStep>(query_plan.getCurrentDataStream(), auto where_step = std::make_unique<FilterStep>(query_plan.getCurrentDataStream(),
actions, std::move(actions),
filter_analysis_result.filter_column_name, filter_analysis_result.filter_column_name,
filter_analysis_result.remove_filter_column); filter_analysis_result.remove_filter_column);
appendSetsFromActionsDAG(*where_step->getExpression(), useful_sets); appendSetsFromActionsDAG(where_step->getExpression(), useful_sets);
where_step->setStepDescription(step_description); where_step->setStepDescription(step_description);
query_plan.addStep(std::move(where_step)); query_plan.addStep(std::move(where_step));
} }
@ -552,10 +552,10 @@ void addTotalsHavingStep(QueryPlan & query_plan,
const auto & having_analysis_result = expression_analysis_result.getHaving(); const auto & having_analysis_result = expression_analysis_result.getHaving();
bool need_finalize = !query_node.isGroupByWithRollup() && !query_node.isGroupByWithCube(); bool need_finalize = !query_node.isGroupByWithRollup() && !query_node.isGroupByWithCube();
ActionsDAGPtr actions; std::optional<ActionsDAG> actions;
if (having_analysis_result.filter_actions) if (having_analysis_result.filter_actions)
{ {
actions = ActionsDAG::clone(&having_analysis_result.filter_actions->dag); actions = std::move(*ActionsDAG::clone(&having_analysis_result.filter_actions->dag));
if (having_analysis_result.filter_actions->project_input) if (having_analysis_result.filter_actions->project_input)
actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
} }
@ -564,7 +564,7 @@ void addTotalsHavingStep(QueryPlan & query_plan,
query_plan.getCurrentDataStream(), query_plan.getCurrentDataStream(),
aggregation_analysis_result.aggregate_descriptions, aggregation_analysis_result.aggregate_descriptions,
query_analysis_result.aggregate_overflow_row, query_analysis_result.aggregate_overflow_row,
actions, std::move(actions),
having_analysis_result.filter_column_name, having_analysis_result.filter_column_name,
having_analysis_result.remove_filter_column, having_analysis_result.remove_filter_column,
settings.totals_mode, settings.totals_mode,
@ -715,13 +715,13 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan,
if (query_node.hasInterpolate()) if (query_node.hasInterpolate())
{ {
auto interpolate_actions_dag = std::make_unique<ActionsDAG>(); ActionsDAG interpolate_actions_dag;
auto query_plan_columns = query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); auto query_plan_columns = query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName();
for (auto & query_plan_column : query_plan_columns) for (auto & query_plan_column : query_plan_columns)
{ {
/// INTERPOLATE actions dag input columns must be non constant /// INTERPOLATE actions dag input columns must be non constant
query_plan_column.column = nullptr; query_plan_column.column = nullptr;
interpolate_actions_dag->addInput(query_plan_column); interpolate_actions_dag.addInput(query_plan_column);
} }
auto & interpolate_list_node = query_node.getInterpolate()->as<ListNode &>(); auto & interpolate_list_node = query_node.getInterpolate()->as<ListNode &>();
@ -729,12 +729,12 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan,
if (interpolate_list_nodes.empty()) if (interpolate_list_nodes.empty())
{ {
for (const auto * input_node : interpolate_actions_dag->getInputs()) for (const auto * input_node : interpolate_actions_dag.getInputs())
{ {
if (column_names_with_fill.contains(input_node->result_name)) if (column_names_with_fill.contains(input_node->result_name))
continue; continue;
interpolate_actions_dag->getOutputs().push_back(input_node); interpolate_actions_dag.getOutputs().push_back(input_node);
} }
} }
else else
@ -744,12 +744,12 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan,
auto & interpolate_node_typed = interpolate_node->as<InterpolateNode &>(); auto & interpolate_node_typed = interpolate_node->as<InterpolateNode &>();
PlannerActionsVisitor planner_actions_visitor(planner_context); PlannerActionsVisitor planner_actions_visitor(planner_context);
auto expression_to_interpolate_expression_nodes = planner_actions_visitor.visit(*interpolate_actions_dag, auto expression_to_interpolate_expression_nodes = planner_actions_visitor.visit(interpolate_actions_dag,
interpolate_node_typed.getExpression()); interpolate_node_typed.getExpression());
if (expression_to_interpolate_expression_nodes.size() != 1) if (expression_to_interpolate_expression_nodes.size() != 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression to interpolate expected to have single action node"); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression to interpolate expected to have single action node");
auto interpolate_expression_nodes = planner_actions_visitor.visit(*interpolate_actions_dag, auto interpolate_expression_nodes = planner_actions_visitor.visit(interpolate_actions_dag,
interpolate_node_typed.getInterpolateExpression()); interpolate_node_typed.getInterpolateExpression());
if (interpolate_expression_nodes.size() != 1) if (interpolate_expression_nodes.size() != 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Interpolate expression expected to have single action node"); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Interpolate expression expected to have single action node");
@ -760,16 +760,16 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan,
const auto * interpolate_expression = interpolate_expression_nodes[0]; const auto * interpolate_expression = interpolate_expression_nodes[0];
if (!interpolate_expression->result_type->equals(*expression_to_interpolate->result_type)) if (!interpolate_expression->result_type->equals(*expression_to_interpolate->result_type))
{ {
interpolate_expression = &interpolate_actions_dag->addCast(*interpolate_expression, interpolate_expression = &interpolate_actions_dag.addCast(*interpolate_expression,
expression_to_interpolate->result_type, expression_to_interpolate->result_type,
interpolate_expression->result_name); interpolate_expression->result_name);
} }
const auto * alias_node = &interpolate_actions_dag->addAlias(*interpolate_expression, expression_to_interpolate_name); const auto * alias_node = &interpolate_actions_dag.addAlias(*interpolate_expression, expression_to_interpolate_name);
interpolate_actions_dag->getOutputs().push_back(alias_node); interpolate_actions_dag.getOutputs().push_back(alias_node);
} }
interpolate_actions_dag->removeUnusedActions(); interpolate_actions_dag.removeUnusedActions();
} }
Aliases empty_aliases; Aliases empty_aliases;
@ -1130,7 +1130,7 @@ void addAdditionalFilterStepIfNeeded(QueryPlan & query_plan,
return; return;
auto filter_step = std::make_unique<FilterStep>(query_plan.getCurrentDataStream(), auto filter_step = std::make_unique<FilterStep>(query_plan.getCurrentDataStream(),
filter_info.actions, std::move(*filter_info.actions),
filter_info.column_name, filter_info.column_name,
filter_info.do_remove_column); filter_info.do_remove_column);
filter_step->setStepDescription("additional result filter"); filter_step->setStepDescription("additional result filter");
@ -1418,7 +1418,7 @@ void Planner::buildPlanForQueryNode()
if (it != table_filters.end()) if (it != table_filters.end())
{ {
const auto & filters = it->second; const auto & filters = it->second;
table_expression_data.setFilterActions(ActionsDAG::clone(filters.filter_actions)); table_expression_data.setFilterActions(ActionsDAG::clone(&*filters.filter_actions));
table_expression_data.setPrewhereInfo(filters.prewhere_info); table_expression_data.setPrewhereInfo(filters.prewhere_info);
} }
} }

View File

@ -757,12 +757,12 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
lambda_arguments_names_and_types.emplace_back(lambda_argument_name, std::move(lambda_argument_type)); lambda_arguments_names_and_types.emplace_back(lambda_argument_name, std::move(lambda_argument_type));
} }
auto lambda_actions_dag = std::make_unique<ActionsDAG>(); ActionsDAG lambda_actions_dag;
actions_stack.emplace_back(*lambda_actions_dag, node); actions_stack.emplace_back(lambda_actions_dag, node);
auto [lambda_expression_node_name, levels] = visitImpl(lambda_node.getExpression()); auto [lambda_expression_node_name, levels] = visitImpl(lambda_node.getExpression());
lambda_actions_dag->getOutputs().push_back(actions_stack.back().getNodeOrThrow(lambda_expression_node_name)); lambda_actions_dag.getOutputs().push_back(actions_stack.back().getNodeOrThrow(lambda_expression_node_name));
lambda_actions_dag->removeUnusedActions(Names(1, lambda_expression_node_name)); lambda_actions_dag.removeUnusedActions(Names(1, lambda_expression_node_name));
auto expression_actions_settings = ExpressionActionsSettings::fromContext(planner_context->getQueryContext(), CompileExpressions::yes); auto expression_actions_settings = ExpressionActionsSettings::fromContext(planner_context->getQueryContext(), CompileExpressions::yes);
auto lambda_actions = std::make_shared<ExpressionActions>(std::move(lambda_actions_dag), expression_actions_settings); auto lambda_actions = std::make_shared<ExpressionActions>(std::move(lambda_actions_dag), expression_actions_settings);
@ -879,14 +879,14 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
const auto & function_node = node->as<FunctionNode &>(); const auto & function_node = node->as<FunctionNode &>();
auto function_node_name = action_node_name_helper.calculateActionNodeName(node); auto function_node_name = action_node_name_helper.calculateActionNodeName(node);
auto index_hint_actions_dag = std::make_unique<ActionsDAG>(); ActionsDAG index_hint_actions_dag;
auto & index_hint_actions_dag_outputs = index_hint_actions_dag->getOutputs(); auto & index_hint_actions_dag_outputs = index_hint_actions_dag.getOutputs();
std::unordered_set<std::string_view> index_hint_actions_dag_output_node_names; std::unordered_set<std::string_view> index_hint_actions_dag_output_node_names;
PlannerActionsVisitor actions_visitor(planner_context); PlannerActionsVisitor actions_visitor(planner_context);
for (const auto & argument : function_node.getArguments()) for (const auto & argument : function_node.getArguments())
{ {
auto index_hint_argument_expression_dag_nodes = actions_visitor.visit(*index_hint_actions_dag, argument); auto index_hint_argument_expression_dag_nodes = actions_visitor.visit(index_hint_actions_dag, argument);
for (auto & expression_dag_node : index_hint_argument_expression_dag_nodes) for (auto & expression_dag_node : index_hint_argument_expression_dag_nodes)
{ {

View File

@ -25,7 +25,7 @@ class TableNode;
struct FiltersForTableExpression struct FiltersForTableExpression
{ {
ActionsDAGPtr filter_actions; std::optional<ActionsDAG> filter_actions;
PrewhereInfoPtr prewhere_info; PrewhereInfoPtr prewhere_info;
}; };

View File

@ -591,19 +591,19 @@ UInt64 mainQueryNodeBlockSizeByLimit(const SelectQueryInfo & select_query_info)
std::unique_ptr<ExpressionStep> createComputeAliasColumnsStep( std::unique_ptr<ExpressionStep> createComputeAliasColumnsStep(
const std::unordered_map<std::string, ActionsDAGPtr> & alias_column_expressions, const DataStream & current_data_stream) const std::unordered_map<std::string, ActionsDAGPtr> & alias_column_expressions, const DataStream & current_data_stream)
{ {
ActionsDAGPtr merged_alias_columns_actions_dag = std::make_unique<ActionsDAG>(current_data_stream.header.getColumnsWithTypeAndName()); ActionsDAG merged_alias_columns_actions_dag(current_data_stream.header.getColumnsWithTypeAndName());
ActionsDAG::NodeRawConstPtrs action_dag_outputs = merged_alias_columns_actions_dag->getInputs(); ActionsDAG::NodeRawConstPtrs action_dag_outputs = merged_alias_columns_actions_dag.getInputs();
for (const auto & [column_name, alias_column_actions_dag] : alias_column_expressions) for (const auto & [column_name, alias_column_actions_dag] : alias_column_expressions)
{ {
const auto & current_outputs = alias_column_actions_dag->getOutputs(); const auto & current_outputs = alias_column_actions_dag->getOutputs();
action_dag_outputs.insert(action_dag_outputs.end(), current_outputs.begin(), current_outputs.end()); action_dag_outputs.insert(action_dag_outputs.end(), current_outputs.begin(), current_outputs.end());
merged_alias_columns_actions_dag->mergeNodes(std::move(*alias_column_actions_dag)); merged_alias_columns_actions_dag.mergeNodes(std::move(*alias_column_actions_dag));
} }
for (const auto * output_node : action_dag_outputs) for (const auto * output_node : action_dag_outputs)
merged_alias_columns_actions_dag->addOrReplaceInOutputs(*output_node); merged_alias_columns_actions_dag.addOrReplaceInOutputs(*output_node);
merged_alias_columns_actions_dag->removeUnusedActions(false); merged_alias_columns_actions_dag.removeUnusedActions(false);
auto alias_column_step = std::make_unique<ExpressionStep>(current_data_stream, std::move(merged_alias_columns_actions_dag)); auto alias_column_step = std::make_unique<ExpressionStep>(current_data_stream, std::move(merged_alias_columns_actions_dag));
alias_column_step->setStepDescription("Compute alias columns"); alias_column_step->setStepDescription("Compute alias columns");
@ -776,7 +776,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
if (prewhere_actions) if (prewhere_actions)
{ {
prewhere_info = std::make_shared<PrewhereInfo>(); prewhere_info = std::make_shared<PrewhereInfo>();
prewhere_info->prewhere_actions = ActionsDAG::clone(prewhere_actions); prewhere_info->prewhere_actions = std::move(*ActionsDAG::clone(prewhere_actions));
prewhere_info->prewhere_column_name = prewhere_actions->getOutputs().at(0)->result_name; prewhere_info->prewhere_column_name = prewhere_actions->getOutputs().at(0)->result_name;
prewhere_info->remove_prewhere_column = true; prewhere_info->remove_prewhere_column = true;
prewhere_info->need_filter = true; prewhere_info->need_filter = true;
@ -805,14 +805,14 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
if (!prewhere_info->prewhere_actions) if (!prewhere_info->prewhere_actions)
{ {
prewhere_info->prewhere_actions = std::move(filter_info.actions); prewhere_info->prewhere_actions = std::move(*filter_info.actions);
prewhere_info->prewhere_column_name = filter_info.column_name; prewhere_info->prewhere_column_name = filter_info.column_name;
prewhere_info->remove_prewhere_column = filter_info.do_remove_column; prewhere_info->remove_prewhere_column = filter_info.do_remove_column;
prewhere_info->need_filter = true; prewhere_info->need_filter = true;
} }
else if (!prewhere_info->row_level_filter) else if (!prewhere_info->row_level_filter)
{ {
prewhere_info->row_level_filter = std::move(filter_info.actions); prewhere_info->row_level_filter = std::move(*filter_info.actions);
prewhere_info->row_level_column_name = filter_info.column_name; prewhere_info->row_level_column_name = filter_info.column_name;
prewhere_info->need_filter = true; prewhere_info->need_filter = true;
} }
@ -831,7 +831,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
auto row_policy_filter_info auto row_policy_filter_info
= buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context, used_row_policies); = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context, used_row_policies);
if (row_policy_filter_info.actions) if (row_policy_filter_info.actions)
table_expression_data.setRowLevelFilterActions(ActionsDAG::clone(row_policy_filter_info.actions)); table_expression_data.setRowLevelFilterActions(ActionsDAG::clone(&*row_policy_filter_info.actions));
add_filter(row_policy_filter_info, "Row-level security filter"); add_filter(row_policy_filter_info, "Row-level security filter");
if (query_context->getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY) if (query_context->getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY)
@ -964,15 +964,14 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
query_plan.addStep(std::move(alias_column_step)); query_plan.addStep(std::move(alias_column_step));
} }
for (const auto & filter_info_and_description : where_filters) for (auto && [filter_info, description] : where_filters)
{ {
const auto & [filter_info, description] = filter_info_and_description;
if (query_plan.isInitialized() && if (query_plan.isInitialized() &&
from_stage == QueryProcessingStage::FetchColumns && from_stage == QueryProcessingStage::FetchColumns &&
filter_info.actions) filter_info.actions)
{ {
auto filter_step = std::make_unique<FilterStep>(query_plan.getCurrentDataStream(), auto filter_step = std::make_unique<FilterStep>(query_plan.getCurrentDataStream(),
filter_info.actions, std::move(*filter_info.actions),
filter_info.column_name, filter_info.column_name,
filter_info.do_remove_column); filter_info.do_remove_column);
filter_step->setStepDescription(description); filter_step->setStepDescription(description);
@ -1063,19 +1062,19 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
if (from_stage == QueryProcessingStage::FetchColumns) if (from_stage == QueryProcessingStage::FetchColumns)
{ {
auto rename_actions_dag = std::make_unique<ActionsDAG>(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); ActionsDAG rename_actions_dag(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName());
ActionsDAG::NodeRawConstPtrs updated_actions_dag_outputs; ActionsDAG::NodeRawConstPtrs updated_actions_dag_outputs;
for (auto & output_node : rename_actions_dag->getOutputs()) for (auto & output_node : rename_actions_dag.getOutputs())
{ {
const auto * column_identifier = table_expression_data.getColumnIdentifierOrNull(output_node->result_name); const auto * column_identifier = table_expression_data.getColumnIdentifierOrNull(output_node->result_name);
if (!column_identifier) if (!column_identifier)
continue; continue;
updated_actions_dag_outputs.push_back(&rename_actions_dag->addAlias(*output_node, *column_identifier)); updated_actions_dag_outputs.push_back(&rename_actions_dag.addAlias(*output_node, *column_identifier));
} }
rename_actions_dag->getOutputs() = std::move(updated_actions_dag_outputs); rename_actions_dag.getOutputs() = std::move(updated_actions_dag_outputs);
auto rename_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), std::move(rename_actions_dag)); auto rename_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), std::move(rename_actions_dag));
rename_step->setStepDescription("Change column names to column identifiers"); rename_step->setStepDescription("Change column names to column identifiers");
@ -1117,9 +1116,9 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextPtr & planner_context, const FunctionOverloadResolverPtr & to_nullable_function) void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextPtr & planner_context, const FunctionOverloadResolverPtr & to_nullable_function)
{ {
auto cast_actions_dag = std::make_unique<ActionsDAG>(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); ActionsDAG cast_actions_dag(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName());
for (auto & output_node : cast_actions_dag->getOutputs()) for (auto & output_node : cast_actions_dag.getOutputs())
{ {
if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(output_node->result_name)) if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(output_node->result_name))
{ {
@ -1128,11 +1127,11 @@ void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextP
type_to_check = type_to_check_low_cardinality->getDictionaryType(); type_to_check = type_to_check_low_cardinality->getDictionaryType();
if (type_to_check->canBeInsideNullable()) if (type_to_check->canBeInsideNullable())
output_node = &cast_actions_dag->addFunction(to_nullable_function, {output_node}, output_node->result_name); output_node = &cast_actions_dag.addFunction(to_nullable_function, {output_node}, output_node->result_name);
} }
} }
cast_actions_dag->appendInputsForUnusedColumns(plan_to_add_cast.getCurrentDataStream().header); cast_actions_dag.appendInputsForUnusedColumns(plan_to_add_cast.getCurrentDataStream().header);
auto cast_join_columns_step = std::make_unique<ExpressionStep>(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag)); auto cast_join_columns_step = std::make_unique<ExpressionStep>(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag));
cast_join_columns_step->setStepDescription("Cast JOIN columns to Nullable"); cast_join_columns_step->setStepDescription("Cast JOIN columns to Nullable");
plan_to_add_cast.addStep(std::move(cast_join_columns_step)); plan_to_add_cast.addStep(std::move(cast_join_columns_step));
@ -1178,16 +1177,16 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
join_table_expression, join_table_expression,
planner_context); planner_context);
join_clauses_and_actions.left_join_expressions_actions->appendInputsForUnusedColumns(left_plan.getCurrentDataStream().header); join_clauses_and_actions.left_join_expressions_actions.appendInputsForUnusedColumns(left_plan.getCurrentDataStream().header);
auto left_join_expressions_actions_step = std::make_unique<ExpressionStep>(left_plan.getCurrentDataStream(), join_clauses_and_actions.left_join_expressions_actions); auto left_join_expressions_actions_step = std::make_unique<ExpressionStep>(left_plan.getCurrentDataStream(), std::move(join_clauses_and_actions.left_join_expressions_actions));
left_join_expressions_actions_step->setStepDescription("JOIN actions"); left_join_expressions_actions_step->setStepDescription("JOIN actions");
appendSetsFromActionsDAG(*left_join_expressions_actions_step->getExpression(), left_join_tree_query_plan.useful_sets); appendSetsFromActionsDAG(left_join_expressions_actions_step->getExpression(), left_join_tree_query_plan.useful_sets);
left_plan.addStep(std::move(left_join_expressions_actions_step)); left_plan.addStep(std::move(left_join_expressions_actions_step));
join_clauses_and_actions.right_join_expressions_actions->appendInputsForUnusedColumns(right_plan.getCurrentDataStream().header); join_clauses_and_actions.right_join_expressions_actions.appendInputsForUnusedColumns(right_plan.getCurrentDataStream().header);
auto right_join_expressions_actions_step = std::make_unique<ExpressionStep>(right_plan.getCurrentDataStream(), join_clauses_and_actions.right_join_expressions_actions); auto right_join_expressions_actions_step = std::make_unique<ExpressionStep>(right_plan.getCurrentDataStream(), std::move(join_clauses_and_actions.right_join_expressions_actions));
right_join_expressions_actions_step->setStepDescription("JOIN actions"); right_join_expressions_actions_step->setStepDescription("JOIN actions");
appendSetsFromActionsDAG(*right_join_expressions_actions_step->getExpression(), right_join_tree_query_plan.useful_sets); appendSetsFromActionsDAG(right_join_expressions_actions_step->getExpression(), right_join_tree_query_plan.useful_sets);
right_plan.addStep(std::move(right_join_expressions_actions_step)); right_plan.addStep(std::move(right_join_expressions_actions_step));
} }
@ -1225,19 +1224,19 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
auto join_cast_plan_output_nodes = [&](QueryPlan & plan_to_add_cast, std::unordered_map<std::string, DataTypePtr> & plan_column_name_to_cast_type) auto join_cast_plan_output_nodes = [&](QueryPlan & plan_to_add_cast, std::unordered_map<std::string, DataTypePtr> & plan_column_name_to_cast_type)
{ {
auto cast_actions_dag = std::make_unique<ActionsDAG>(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName()); ActionsDAG cast_actions_dag(plan_to_add_cast.getCurrentDataStream().header.getColumnsWithTypeAndName());
for (auto & output_node : cast_actions_dag->getOutputs()) for (auto & output_node : cast_actions_dag.getOutputs())
{ {
auto it = plan_column_name_to_cast_type.find(output_node->result_name); auto it = plan_column_name_to_cast_type.find(output_node->result_name);
if (it == plan_column_name_to_cast_type.end()) if (it == plan_column_name_to_cast_type.end())
continue; continue;
const auto & cast_type = it->second; const auto & cast_type = it->second;
output_node = &cast_actions_dag->addCast(*output_node, cast_type, output_node->result_name); output_node = &cast_actions_dag.addCast(*output_node, cast_type, output_node->result_name);
} }
cast_actions_dag->appendInputsForUnusedColumns(plan_to_add_cast.getCurrentDataStream().header); cast_actions_dag.appendInputsForUnusedColumns(plan_to_add_cast.getCurrentDataStream().header);
auto cast_join_columns_step auto cast_join_columns_step
= std::make_unique<ExpressionStep>(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag)); = std::make_unique<ExpressionStep>(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag));
cast_join_columns_step->setStepDescription("Cast JOIN USING columns"); cast_join_columns_step->setStepDescription("Cast JOIN USING columns");
@ -1385,7 +1384,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
{ {
ExpressionActionsPtr & mixed_join_expression = table_join->getMixedJoinExpression(); ExpressionActionsPtr & mixed_join_expression = table_join->getMixedJoinExpression();
mixed_join_expression = std::make_shared<ExpressionActions>( mixed_join_expression = std::make_shared<ExpressionActions>(
std::move(join_clauses_and_actions.mixed_join_expressions_actions), std::move(*join_clauses_and_actions.mixed_join_expressions_actions),
ExpressionActionsSettings::fromContext(planner_context->getQueryContext())); ExpressionActionsSettings::fromContext(planner_context->getQueryContext()));
appendSetsFromActionsDAG(mixed_join_expression->getActionsDAG(), left_join_tree_query_plan.useful_sets); appendSetsFromActionsDAG(mixed_join_expression->getActionsDAG(), left_join_tree_query_plan.useful_sets);
@ -1542,12 +1541,12 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
result_plan.unitePlans(std::move(join_step), {std::move(plans)}); result_plan.unitePlans(std::move(join_step), {std::move(plans)});
} }
auto drop_unused_columns_after_join_actions_dag = std::make_unique<ActionsDAG>(result_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); ActionsDAG drop_unused_columns_after_join_actions_dag(result_plan.getCurrentDataStream().header.getColumnsWithTypeAndName());
ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs; ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs;
std::unordered_set<std::string_view> drop_unused_columns_after_join_actions_dag_updated_outputs_names; std::unordered_set<std::string_view> drop_unused_columns_after_join_actions_dag_updated_outputs_names;
std::optional<size_t> first_skipped_column_node_index; std::optional<size_t> first_skipped_column_node_index;
auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag->getOutputs(); auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag.getOutputs();
size_t drop_unused_columns_after_join_actions_dag_outputs_size = drop_unused_columns_after_join_actions_dag_outputs.size(); size_t drop_unused_columns_after_join_actions_dag_outputs_size = drop_unused_columns_after_join_actions_dag_outputs.size();
for (size_t i = 0; i < drop_unused_columns_after_join_actions_dag_outputs_size; ++i) for (size_t i = 0; i < drop_unused_columns_after_join_actions_dag_outputs_size; ++i)
@ -1619,7 +1618,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_
auto plan = std::move(join_tree_query_plan.query_plan); auto plan = std::move(join_tree_query_plan.query_plan);
auto plan_output_columns = plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); auto plan_output_columns = plan.getCurrentDataStream().header.getColumnsWithTypeAndName();
ActionsDAGPtr array_join_action_dag = std::make_unique<ActionsDAG>(plan_output_columns); ActionsDAG array_join_action_dag(plan_output_columns);
PlannerActionsVisitor actions_visitor(planner_context); PlannerActionsVisitor actions_visitor(planner_context);
std::unordered_set<std::string> array_join_expressions_output_nodes; std::unordered_set<std::string> array_join_expressions_output_nodes;
@ -1630,28 +1629,28 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_
array_join_column_names.insert(array_join_column_identifier); array_join_column_names.insert(array_join_column_identifier);
auto & array_join_expression_column = array_join_expression->as<ColumnNode &>(); auto & array_join_expression_column = array_join_expression->as<ColumnNode &>();
auto expression_dag_index_nodes = actions_visitor.visit(*array_join_action_dag, array_join_expression_column.getExpressionOrThrow()); auto expression_dag_index_nodes = actions_visitor.visit(array_join_action_dag, array_join_expression_column.getExpressionOrThrow());
for (auto & expression_dag_index_node : expression_dag_index_nodes) for (auto & expression_dag_index_node : expression_dag_index_nodes)
{ {
const auto * array_join_column_node = &array_join_action_dag->addAlias(*expression_dag_index_node, array_join_column_identifier); const auto * array_join_column_node = &array_join_action_dag.addAlias(*expression_dag_index_node, array_join_column_identifier);
array_join_action_dag->getOutputs().push_back(array_join_column_node); array_join_action_dag.getOutputs().push_back(array_join_column_node);
array_join_expressions_output_nodes.insert(array_join_column_node->result_name); array_join_expressions_output_nodes.insert(array_join_column_node->result_name);
} }
} }
array_join_action_dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header); array_join_action_dag.appendInputsForUnusedColumns(plan.getCurrentDataStream().header);
auto array_join_actions = std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), std::move(array_join_action_dag)); auto array_join_actions = std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), std::move(array_join_action_dag));
array_join_actions->setStepDescription("ARRAY JOIN actions"); array_join_actions->setStepDescription("ARRAY JOIN actions");
appendSetsFromActionsDAG(*array_join_actions->getExpression(), join_tree_query_plan.useful_sets); appendSetsFromActionsDAG(array_join_actions->getExpression(), join_tree_query_plan.useful_sets);
plan.addStep(std::move(array_join_actions)); plan.addStep(std::move(array_join_actions));
auto drop_unused_columns_before_array_join_actions_dag = std::make_unique<ActionsDAG>(plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); ActionsDAG drop_unused_columns_before_array_join_actions_dag(plan.getCurrentDataStream().header.getColumnsWithTypeAndName());
ActionsDAG::NodeRawConstPtrs drop_unused_columns_before_array_join_actions_dag_updated_outputs; ActionsDAG::NodeRawConstPtrs drop_unused_columns_before_array_join_actions_dag_updated_outputs;
std::unordered_set<std::string_view> drop_unused_columns_before_array_join_actions_dag_updated_outputs_names; std::unordered_set<std::string_view> drop_unused_columns_before_array_join_actions_dag_updated_outputs_names;
auto & drop_unused_columns_before_array_join_actions_dag_outputs = drop_unused_columns_before_array_join_actions_dag->getOutputs(); auto & drop_unused_columns_before_array_join_actions_dag_outputs = drop_unused_columns_before_array_join_actions_dag.getOutputs();
size_t drop_unused_columns_before_array_join_actions_dag_outputs_size = drop_unused_columns_before_array_join_actions_dag_outputs.size(); size_t drop_unused_columns_before_array_join_actions_dag_outputs_size = drop_unused_columns_before_array_join_actions_dag_outputs.size();
for (size_t i = 0; i < drop_unused_columns_before_array_join_actions_dag_outputs_size; ++i) for (size_t i = 0; i < drop_unused_columns_before_array_join_actions_dag_outputs_size; ++i)

View File

@ -177,13 +177,13 @@ std::set<JoinTableSide> extractJoinTableSidesFromExpression(//const ActionsDAG::
} }
const ActionsDAG::Node * appendExpression( const ActionsDAG::Node * appendExpression(
ActionsDAGPtr & dag, ActionsDAG & dag,
const QueryTreeNodePtr & expression, const QueryTreeNodePtr & expression,
const PlannerContextPtr & planner_context, const PlannerContextPtr & planner_context,
const JoinNode & join_node) const JoinNode & join_node)
{ {
PlannerActionsVisitor join_expression_visitor(planner_context); PlannerActionsVisitor join_expression_visitor(planner_context);
auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(*dag, expression); auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(dag, expression);
if (join_expression_dag_node_raw_pointers.size() != 1) if (join_expression_dag_node_raw_pointers.size() != 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(ErrorCodes::LOGICAL_ERROR,
"JOIN {} ON clause contains multiple expressions", "JOIN {} ON clause contains multiple expressions",
@ -193,9 +193,9 @@ const ActionsDAG::Node * appendExpression(
} }
void buildJoinClause( void buildJoinClause(
ActionsDAGPtr & left_dag, ActionsDAG & left_dag,
ActionsDAGPtr & right_dag, ActionsDAG & right_dag,
ActionsDAGPtr & mixed_dag, ActionsDAG & mixed_dag,
const PlannerContextPtr & planner_context, const PlannerContextPtr & planner_context,
const QueryTreeNodePtr & join_expression, const QueryTreeNodePtr & join_expression,
const TableExpressionSet & left_table_expressions, const TableExpressionSet & left_table_expressions,
@ -376,8 +376,8 @@ JoinClausesAndActions buildJoinClausesAndActions(
const JoinNode & join_node, const JoinNode & join_node,
const PlannerContextPtr & planner_context) const PlannerContextPtr & planner_context)
{ {
ActionsDAGPtr left_join_actions = std::make_unique<ActionsDAG>(left_table_expression_columns); ActionsDAG left_join_actions(left_table_expression_columns);
ActionsDAGPtr right_join_actions = std::make_unique<ActionsDAG>(right_table_expression_columns); ActionsDAG right_join_actions(right_table_expression_columns);
ColumnsWithTypeAndName mixed_table_expression_columns; ColumnsWithTypeAndName mixed_table_expression_columns;
for (const auto & left_column : left_table_expression_columns) for (const auto & left_column : left_table_expression_columns)
{ {
@ -387,7 +387,7 @@ JoinClausesAndActions buildJoinClausesAndActions(
{ {
mixed_table_expression_columns.push_back(right_column); mixed_table_expression_columns.push_back(right_column);
} }
ActionsDAGPtr mixed_join_actions = std::make_unique<ActionsDAG>(mixed_table_expression_columns); ActionsDAG mixed_join_actions(mixed_table_expression_columns);
/** It is possible to have constant value in JOIN ON section, that we need to ignore during DAG construction. /** It is possible to have constant value in JOIN ON section, that we need to ignore during DAG construction.
* If we do not ignore it, this function will be replaced by underlying constant. * If we do not ignore it, this function will be replaced by underlying constant.
@ -498,12 +498,12 @@ JoinClausesAndActions buildJoinClausesAndActions(
{ {
const ActionsDAG::Node * dag_filter_condition_node = nullptr; const ActionsDAG::Node * dag_filter_condition_node = nullptr;
if (left_filter_condition_nodes.size() > 1) if (left_filter_condition_nodes.size() > 1)
dag_filter_condition_node = &left_join_actions->addFunction(and_function, left_filter_condition_nodes, {}); dag_filter_condition_node = &left_join_actions.addFunction(and_function, left_filter_condition_nodes, {});
else else
dag_filter_condition_node = left_filter_condition_nodes[0]; dag_filter_condition_node = left_filter_condition_nodes[0];
join_clause.getLeftFilterConditionNodes() = {dag_filter_condition_node}; join_clause.getLeftFilterConditionNodes() = {dag_filter_condition_node};
left_join_actions->addOrReplaceInOutputs(*dag_filter_condition_node); left_join_actions.addOrReplaceInOutputs(*dag_filter_condition_node);
add_necessary_name_if_needed(JoinTableSide::Left, dag_filter_condition_node->result_name); add_necessary_name_if_needed(JoinTableSide::Left, dag_filter_condition_node->result_name);
} }
@ -514,12 +514,12 @@ JoinClausesAndActions buildJoinClausesAndActions(
const ActionsDAG::Node * dag_filter_condition_node = nullptr; const ActionsDAG::Node * dag_filter_condition_node = nullptr;
if (right_filter_condition_nodes.size() > 1) if (right_filter_condition_nodes.size() > 1)
dag_filter_condition_node = &right_join_actions->addFunction(and_function, right_filter_condition_nodes, {}); dag_filter_condition_node = &right_join_actions.addFunction(and_function, right_filter_condition_nodes, {});
else else
dag_filter_condition_node = right_filter_condition_nodes[0]; dag_filter_condition_node = right_filter_condition_nodes[0];
join_clause.getRightFilterConditionNodes() = {dag_filter_condition_node}; join_clause.getRightFilterConditionNodes() = {dag_filter_condition_node};
right_join_actions->addOrReplaceInOutputs(*dag_filter_condition_node); right_join_actions.addOrReplaceInOutputs(*dag_filter_condition_node);
add_necessary_name_if_needed(JoinTableSide::Right, dag_filter_condition_node->result_name); add_necessary_name_if_needed(JoinTableSide::Right, dag_filter_condition_node->result_name);
} }
@ -556,10 +556,10 @@ JoinClausesAndActions buildJoinClausesAndActions(
} }
if (!left_key_node->result_type->equals(*common_type)) if (!left_key_node->result_type->equals(*common_type))
left_key_node = &left_join_actions->addCast(*left_key_node, common_type, {}); left_key_node = &left_join_actions.addCast(*left_key_node, common_type, {});
if (!right_key_node->result_type->equals(*common_type)) if (!right_key_node->result_type->equals(*common_type))
right_key_node = &right_join_actions->addCast(*right_key_node, common_type, {}); right_key_node = &right_join_actions.addCast(*right_key_node, common_type, {});
} }
if (join_clause.isNullsafeCompareKey(i) && left_key_node->result_type->isNullable() && right_key_node->result_type->isNullable()) if (join_clause.isNullsafeCompareKey(i) && left_key_node->result_type->isNullable() && right_key_node->result_type->isNullable())
@ -576,24 +576,24 @@ JoinClausesAndActions buildJoinClausesAndActions(
* SELECT * FROM t1 JOIN t2 ON tuple(t1.a) == tuple(t2.b) * SELECT * FROM t1 JOIN t2 ON tuple(t1.a) == tuple(t2.b)
*/ */
auto wrap_nullsafe_function = FunctionFactory::instance().get("tuple", planner_context->getQueryContext()); auto wrap_nullsafe_function = FunctionFactory::instance().get("tuple", planner_context->getQueryContext());
left_key_node = &left_join_actions->addFunction(wrap_nullsafe_function, {left_key_node}, {}); left_key_node = &left_join_actions.addFunction(wrap_nullsafe_function, {left_key_node}, {});
right_key_node = &right_join_actions->addFunction(wrap_nullsafe_function, {right_key_node}, {}); right_key_node = &right_join_actions.addFunction(wrap_nullsafe_function, {right_key_node}, {});
} }
left_join_actions->addOrReplaceInOutputs(*left_key_node); left_join_actions.addOrReplaceInOutputs(*left_key_node);
right_join_actions->addOrReplaceInOutputs(*right_key_node); right_join_actions.addOrReplaceInOutputs(*right_key_node);
add_necessary_name_if_needed(JoinTableSide::Left, left_key_node->result_name); add_necessary_name_if_needed(JoinTableSide::Left, left_key_node->result_name);
add_necessary_name_if_needed(JoinTableSide::Right, right_key_node->result_name); add_necessary_name_if_needed(JoinTableSide::Right, right_key_node->result_name);
} }
} }
result.left_join_expressions_actions = ActionsDAG::clone(left_join_actions); result.left_join_expressions_actions = std::move(left_join_actions);
result.left_join_tmp_expression_actions = std::move(left_join_actions); //result.left_join_tmp_expression_actions = std::move(left_join_actions);
result.left_join_expressions_actions->removeUnusedActions(join_left_actions_names); result.left_join_expressions_actions.removeUnusedActions(join_left_actions_names);
result.right_join_expressions_actions = ActionsDAG::clone(right_join_actions); result.right_join_expressions_actions = std::move(right_join_actions);
result.right_join_tmp_expression_actions = std::move(right_join_actions); //result.right_join_tmp_expression_actions = std::move(right_join_actions);
result.right_join_expressions_actions->removeUnusedActions(join_right_actions_names); result.right_join_expressions_actions.removeUnusedActions(join_right_actions_names);
if (is_inequal_join) if (is_inequal_join)
{ {
@ -601,16 +601,16 @@ JoinClausesAndActions buildJoinClausesAndActions(
/// So, for each column, we recalculate the value of the whole expression from JOIN ON to check if rows should be joined. /// So, for each column, we recalculate the value of the whole expression from JOIN ON to check if rows should be joined.
if (result.join_clauses.size() > 1) if (result.join_clauses.size() > 1)
{ {
auto mixed_join_expressions_actions = std::make_unique<ActionsDAG>(mixed_table_expression_columns); ActionsDAG mixed_join_expressions_actions(mixed_table_expression_columns);
PlannerActionsVisitor join_expression_visitor(planner_context); PlannerActionsVisitor join_expression_visitor(planner_context);
auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(*mixed_join_expressions_actions, join_expression); auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(mixed_join_expressions_actions, join_expression);
if (join_expression_dag_node_raw_pointers.size() != 1) if (join_expression_dag_node_raw_pointers.size() != 1)
throw Exception( throw Exception(
ErrorCodes::LOGICAL_ERROR, "JOIN {} ON clause contains multiple expressions", join_node.formatASTForErrorMessage()); ErrorCodes::LOGICAL_ERROR, "JOIN {} ON clause contains multiple expressions", join_node.formatASTForErrorMessage());
mixed_join_expressions_actions->addOrReplaceInOutputs(*join_expression_dag_node_raw_pointers[0]); mixed_join_expressions_actions.addOrReplaceInOutputs(*join_expression_dag_node_raw_pointers[0]);
Names required_names{join_expression_dag_node_raw_pointers[0]->result_name}; Names required_names{join_expression_dag_node_raw_pointers[0]->result_name};
mixed_join_expressions_actions->removeUnusedActions(required_names); mixed_join_expressions_actions.removeUnusedActions(required_names);
result.mixed_join_expressions_actions = std::move(mixed_join_expressions_actions); result.mixed_join_expressions_actions = std::move(mixed_join_expressions_actions);
} }
else else

View File

@ -182,15 +182,15 @@ struct JoinClausesAndActions
/// Join clauses. Actions dag nodes point into join_expression_actions. /// Join clauses. Actions dag nodes point into join_expression_actions.
JoinClauses join_clauses; JoinClauses join_clauses;
/// Whole JOIN ON section expressions /// Whole JOIN ON section expressions
ActionsDAGPtr left_join_tmp_expression_actions; // ActionsDAGPtr left_join_tmp_expression_actions;
ActionsDAGPtr right_join_tmp_expression_actions; // ActionsDAGPtr right_join_tmp_expression_actions;
/// Left join expressions actions /// Left join expressions actions
ActionsDAGPtr left_join_expressions_actions; ActionsDAG left_join_expressions_actions;
/// Right join expressions actions /// Right join expressions actions
ActionsDAGPtr right_join_expressions_actions; ActionsDAG right_join_expressions_actions;
/// Originally used for inequal join. it's the total join expression. /// Originally used for inequal join. it's the total join expression.
/// If there is no inequal join conditions, it's null. /// If there is no inequal join conditions, it's null.
ActionsDAGPtr mixed_join_expressions_actions; std::optional<ActionsDAG> mixed_join_expressions_actions;
}; };
/** Calculate join clauses and actions for JOIN ON section. /** Calculate join clauses and actions for JOIN ON section.

View File

@ -442,22 +442,22 @@ FilterDAGInfo buildFilterInfo(QueryTreeNodePtr filter_query_tree,
collectSourceColumns(filter_query_tree, planner_context, false /*keep_alias_columns*/); collectSourceColumns(filter_query_tree, planner_context, false /*keep_alias_columns*/);
collectSets(filter_query_tree, *planner_context); collectSets(filter_query_tree, *planner_context);
auto filter_actions_dag = std::make_unique<ActionsDAG>(); ActionsDAG filter_actions_dag;
PlannerActionsVisitor actions_visitor(planner_context, false /*use_column_identifier_as_action_node_name*/); PlannerActionsVisitor actions_visitor(planner_context, false /*use_column_identifier_as_action_node_name*/);
auto expression_nodes = actions_visitor.visit(*filter_actions_dag, filter_query_tree); auto expression_nodes = actions_visitor.visit(filter_actions_dag, filter_query_tree);
if (expression_nodes.size() != 1) if (expression_nodes.size() != 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS, throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Filter actions must return single output node. Actual {}", "Filter actions must return single output node. Actual {}",
expression_nodes.size()); expression_nodes.size());
auto & filter_actions_outputs = filter_actions_dag->getOutputs(); auto & filter_actions_outputs = filter_actions_dag.getOutputs();
filter_actions_outputs = std::move(expression_nodes); filter_actions_outputs = std::move(expression_nodes);
std::string filter_node_name = filter_actions_outputs[0]->result_name; std::string filter_node_name = filter_actions_outputs[0]->result_name;
bool remove_filter_column = true; bool remove_filter_column = true;
for (const auto & filter_input_node : filter_actions_dag->getInputs()) for (const auto & filter_input_node : filter_actions_dag.getInputs())
if (table_expression_required_names_without_filter.contains(filter_input_node->result_name)) if (table_expression_required_names_without_filter.contains(filter_input_node->result_name))
filter_actions_outputs.push_back(filter_input_node); filter_actions_outputs.push_back(filter_input_node);
@ -498,7 +498,7 @@ void appendSetsFromActionsDAG(const ActionsDAG & dag, UsefulSets & useful_sets)
{ {
if (const auto * index_hint = typeid_cast<const FunctionIndexHint *>(adaptor->getFunction().get())) if (const auto * index_hint = typeid_cast<const FunctionIndexHint *>(adaptor->getFunction().get()))
{ {
appendSetsFromActionsDAG(*index_hint->getActions(), useful_sets); appendSetsFromActionsDAG(index_hint->getActions(), useful_sets);
} }
} }
} }

View File

@ -303,15 +303,15 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B
const auto & header = ports[set_counter]->getHeader(); const auto & header = ports[set_counter]->getHeader();
/// Here we create a DAG which fills missing keys and adds `__grouping_set` column /// Here we create a DAG which fills missing keys and adds `__grouping_set` column
auto dag = std::make_unique<ActionsDAG>(header.getColumnsWithTypeAndName()); ActionsDAG dag(header.getColumnsWithTypeAndName());
ActionsDAG::NodeRawConstPtrs outputs; ActionsDAG::NodeRawConstPtrs outputs;
outputs.reserve(output_header.columns() + 1); outputs.reserve(output_header.columns() + 1);
auto grouping_col = ColumnConst::create(ColumnUInt64::create(1, set_counter), 0); auto grouping_col = ColumnConst::create(ColumnUInt64::create(1, set_counter), 0);
const auto * grouping_node = &dag->addColumn( const auto * grouping_node = &dag.addColumn(
{ColumnPtr(std::move(grouping_col)), std::make_shared<DataTypeUInt64>(), "__grouping_set"}); {ColumnPtr(std::move(grouping_col)), std::make_shared<DataTypeUInt64>(), "__grouping_set"});
grouping_node = &dag->materializeNode(*grouping_node); grouping_node = &dag.materializeNode(*grouping_node);
outputs.push_back(grouping_node); outputs.push_back(grouping_node);
const auto & missing_columns = grouping_sets_params[set_counter].missing_keys; const auto & missing_columns = grouping_sets_params[set_counter].missing_keys;
@ -332,21 +332,21 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B
column_with_default->finalize(); column_with_default->finalize();
auto column = ColumnConst::create(std::move(column_with_default), 0); auto column = ColumnConst::create(std::move(column_with_default), 0);
const auto * node = &dag->addColumn({ColumnPtr(std::move(column)), col.type, col.name}); const auto * node = &dag.addColumn({ColumnPtr(std::move(column)), col.type, col.name});
node = &dag->materializeNode(*node); node = &dag.materializeNode(*node);
outputs.push_back(node); outputs.push_back(node);
} }
else else
{ {
const auto * column_node = dag->getOutputs()[header.getPositionByName(col.name)]; const auto * column_node = dag.getOutputs()[header.getPositionByName(col.name)];
if (used_it != used_keys.end() && group_by_use_nulls && column_node->result_type->canBeInsideNullable()) if (used_it != used_keys.end() && group_by_use_nulls && column_node->result_type->canBeInsideNullable())
outputs.push_back(&dag->addFunction(to_nullable_function, { column_node }, col.name)); outputs.push_back(&dag.addFunction(to_nullable_function, { column_node }, col.name));
else else
outputs.push_back(column_node); outputs.push_back(column_node);
} }
} }
dag->getOutputs().swap(outputs); dag.getOutputs().swap(outputs);
auto expression = std::make_shared<ExpressionActions>(std::move(dag), settings.getActionsSettings()); auto expression = std::make_shared<ExpressionActions>(std::move(dag), settings.getActionsSettings());
auto transform = std::make_shared<ExpressionTransform>(header, expression); auto transform = std::make_shared<ExpressionTransform>(header, expression);

View File

@ -36,27 +36,27 @@ CubeStep::CubeStep(const DataStream & input_stream_, Aggregator::Params params_,
ProcessorPtr addGroupingSetForTotals(const Block & header, const Names & keys, bool use_nulls, const BuildQueryPipelineSettings & settings, UInt64 grouping_set_number) ProcessorPtr addGroupingSetForTotals(const Block & header, const Names & keys, bool use_nulls, const BuildQueryPipelineSettings & settings, UInt64 grouping_set_number)
{ {
auto dag = std::make_unique<ActionsDAG>(header.getColumnsWithTypeAndName()); ActionsDAG dag(header.getColumnsWithTypeAndName());
auto & outputs = dag->getOutputs(); auto & outputs = dag.getOutputs();
if (use_nulls) if (use_nulls)
{ {
auto to_nullable = FunctionFactory::instance().get("toNullable", nullptr); auto to_nullable = FunctionFactory::instance().get("toNullable", nullptr);
for (const auto & key : keys) for (const auto & key : keys)
{ {
const auto * node = dag->getOutputs()[header.getPositionByName(key)]; const auto * node = dag.getOutputs()[header.getPositionByName(key)];
if (node->result_type->canBeInsideNullable()) if (node->result_type->canBeInsideNullable())
{ {
dag->addOrReplaceInOutputs(dag->addFunction(to_nullable, { node }, node->result_name)); dag.addOrReplaceInOutputs(dag.addFunction(to_nullable, { node }, node->result_name));
} }
} }
} }
auto grouping_col = ColumnUInt64::create(1, grouping_set_number); auto grouping_col = ColumnUInt64::create(1, grouping_set_number);
const auto * grouping_node = &dag->addColumn( const auto * grouping_node = &dag.addColumn(
{ColumnPtr(std::move(grouping_col)), std::make_shared<DataTypeUInt64>(), "__grouping_set"}); {ColumnPtr(std::move(grouping_col)), std::make_shared<DataTypeUInt64>(), "__grouping_set"});
grouping_node = &dag->materializeNode(*grouping_node); grouping_node = &dag.materializeNode(*grouping_node);
outputs.insert(outputs.begin(), grouping_node); outputs.insert(outputs.begin(), grouping_node);
auto expression = std::make_shared<ExpressionActions>(std::move(dag), settings.getActionsSettings()); auto expression = std::make_shared<ExpressionActions>(std::move(dag), settings.getActionsSettings());

View File

@ -32,7 +32,7 @@ void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missi
}; };
auto convert_actions_dag = get_converting_dag(plan.getCurrentDataStream().header, header); auto convert_actions_dag = get_converting_dag(plan.getCurrentDataStream().header, header);
auto converting = std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), convert_actions_dag); auto converting = std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), std::move(convert_actions_dag));
plan.addStep(std::move(converting)); plan.addStep(std::move(converting));
} }

View File

@ -10,33 +10,33 @@
namespace DB namespace DB
{ {
static ITransformingStep::Traits getTraits(const ActionsDAGPtr & actions, const Block & header, const SortDescription & sort_description) static ITransformingStep::Traits getTraits(const ActionsDAG & actions, const Block & header, const SortDescription & sort_description)
{ {
return ITransformingStep::Traits return ITransformingStep::Traits
{ {
{ {
.returns_single_stream = false, .returns_single_stream = false,
.preserves_number_of_streams = true, .preserves_number_of_streams = true,
.preserves_sorting = actions->isSortingPreserved(header, sort_description), .preserves_sorting = actions.isSortingPreserved(header, sort_description),
}, },
{ {
.preserves_number_of_rows = !actions->hasArrayJoin(), .preserves_number_of_rows = !actions.hasArrayJoin(),
} }
}; };
} }
ExpressionStep::ExpressionStep(const DataStream & input_stream_, const ActionsDAGPtr & actions_dag_) ExpressionStep::ExpressionStep(const DataStream & input_stream_, ActionsDAG actions_dag_)
: ITransformingStep( : ITransformingStep(
input_stream_, input_stream_,
ExpressionTransform::transformHeader(input_stream_.header, *actions_dag_), ExpressionTransform::transformHeader(input_stream_.header, actions_dag_),
getTraits(actions_dag_, input_stream_.header, input_stream_.sort_description)) getTraits(actions_dag_, input_stream_.header, input_stream_.sort_description))
, actions_dag(ActionsDAG::clone(actions_dag_)) , actions_dag(std::move(actions_dag_))
{ {
} }
void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings)
{ {
auto expression = std::make_shared<ExpressionActions>(ActionsDAG::clone(actions_dag), settings.getActionsSettings()); auto expression = std::make_shared<ExpressionActions>(std::move(actions_dag), settings.getActionsSettings());
pipeline.addSimpleTransform([&](const Block & header) pipeline.addSimpleTransform([&](const Block & header)
{ {
@ -61,25 +61,25 @@ void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const Bu
void ExpressionStep::describeActions(FormatSettings & settings) const void ExpressionStep::describeActions(FormatSettings & settings) const
{ {
String prefix(settings.offset, settings.indent_char); String prefix(settings.offset, settings.indent_char);
auto expression = std::make_shared<ExpressionActions>(ActionsDAG::clone(actions_dag)); auto expression = std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(&actions_dag)));
expression->describeActions(settings.out, prefix); expression->describeActions(settings.out, prefix);
} }
void ExpressionStep::describeActions(JSONBuilder::JSONMap & map) const void ExpressionStep::describeActions(JSONBuilder::JSONMap & map) const
{ {
auto expression = std::make_shared<ExpressionActions>(ActionsDAG::clone(actions_dag)); auto expression = std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(&actions_dag)));
map.add("Expression", expression->toTree()); map.add("Expression", expression->toTree());
} }
void ExpressionStep::updateOutputStream() void ExpressionStep::updateOutputStream()
{ {
output_stream = createOutputStream( output_stream = createOutputStream(
input_streams.front(), ExpressionTransform::transformHeader(input_streams.front().header, *actions_dag), getDataStreamTraits()); input_streams.front(), ExpressionTransform::transformHeader(input_streams.front().header, actions_dag), getDataStreamTraits());
if (!getDataStreamTraits().preserves_sorting) if (!getDataStreamTraits().preserves_sorting)
return; return;
FindAliasForInputName alias_finder(*actions_dag); FindAliasForInputName alias_finder(actions_dag);
const auto & input_sort_description = getInputStreams().front().sort_description; const auto & input_sort_description = getInputStreams().front().sort_description;
for (size_t i = 0, s = input_sort_description.size(); i < s; ++i) for (size_t i = 0, s = input_sort_description.size(); i < s; ++i)
{ {

View File

@ -1,12 +1,10 @@
#pragma once #pragma once
#include <Processors/QueryPlan/ITransformingStep.h> #include <Processors/QueryPlan/ITransformingStep.h>
#include <Interpreters/ActionsDAG.h>
namespace DB namespace DB
{ {
class ActionsDAG;
using ActionsDAGPtr = std::unique_ptr<ActionsDAG>;
class ExpressionTransform; class ExpressionTransform;
class JoiningTransform; class JoiningTransform;
@ -15,21 +13,22 @@ class ExpressionStep : public ITransformingStep
{ {
public: public:
explicit ExpressionStep(const DataStream & input_stream_, const ActionsDAGPtr & actions_dag_); explicit ExpressionStep(const DataStream & input_stream_, ActionsDAG actions_dag_);
String getName() const override { return "Expression"; } String getName() const override { return "Expression"; }
void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override;
void describeActions(FormatSettings & settings) const override; void describeActions(FormatSettings & settings) const override;
const ActionsDAGPtr & getExpression() const { return actions_dag; } ActionsDAG & getExpression() { return actions_dag; }
const ActionsDAG & getExpression() const { return actions_dag; }
void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(JSONBuilder::JSONMap & map) const override;
private: private:
void updateOutputStream() override; void updateOutputStream() override;
ActionsDAGPtr actions_dag; ActionsDAG actions_dag;
}; };
} }

View File

@ -9,9 +9,9 @@
namespace DB namespace DB
{ {
static ITransformingStep::Traits getTraits(const ActionsDAGPtr & expression, const Block & header, const SortDescription & sort_description, bool remove_filter_column, const String & filter_column_name) static ITransformingStep::Traits getTraits(const ActionsDAG & expression, const Block & header, const SortDescription & sort_description, bool remove_filter_column, const String & filter_column_name)
{ {
bool preserves_sorting = expression->isSortingPreserved(header, sort_description, remove_filter_column ? filter_column_name : ""); bool preserves_sorting = expression.isSortingPreserved(header, sort_description, remove_filter_column ? filter_column_name : "");
if (remove_filter_column) if (remove_filter_column)
{ {
preserves_sorting &= std::find_if( preserves_sorting &= std::find_if(
@ -35,22 +35,22 @@ static ITransformingStep::Traits getTraits(const ActionsDAGPtr & expression, con
FilterStep::FilterStep( FilterStep::FilterStep(
const DataStream & input_stream_, const DataStream & input_stream_,
const ActionsDAGPtr & actions_dag_, ActionsDAG actions_dag_,
String filter_column_name_, String filter_column_name_,
bool remove_filter_column_) bool remove_filter_column_)
: ITransformingStep( : ITransformingStep(
input_stream_, input_stream_,
FilterTransform::transformHeader( FilterTransform::transformHeader(
input_stream_.header, input_stream_.header,
actions_dag_.get(), &actions_dag_,
filter_column_name_, filter_column_name_,
remove_filter_column_), remove_filter_column_),
getTraits(actions_dag_, input_stream_.header, input_stream_.sort_description, remove_filter_column_, filter_column_name_)) getTraits(actions_dag_, input_stream_.header, input_stream_.sort_description, remove_filter_column_, filter_column_name_))
, actions_dag(std::move(actions_dag_))
, filter_column_name(std::move(filter_column_name_)) , filter_column_name(std::move(filter_column_name_))
, remove_filter_column(remove_filter_column_) , remove_filter_column(remove_filter_column_)
{ {
actions_dag = ActionsDAG::clone(actions_dag_); actions_dag.removeAliasesForFilter(filter_column_name);
actions_dag->removeAliasesForFilter(filter_column_name);
} }
void FilterStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) void FilterStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings)
@ -87,7 +87,7 @@ void FilterStep::describeActions(FormatSettings & settings) const
settings.out << " (removed)"; settings.out << " (removed)";
settings.out << '\n'; settings.out << '\n';
auto expression = std::make_shared<ExpressionActions>(ActionsDAG::clone(actions_dag)); auto expression = std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(&actions_dag)));
expression->describeActions(settings.out, prefix); expression->describeActions(settings.out, prefix);
} }
@ -96,7 +96,7 @@ void FilterStep::describeActions(JSONBuilder::JSONMap & map) const
map.add("Filter Column", filter_column_name); map.add("Filter Column", filter_column_name);
map.add("Removes Filter", remove_filter_column); map.add("Removes Filter", remove_filter_column);
auto expression = std::make_shared<ExpressionActions>(ActionsDAG::clone(actions_dag)); auto expression = std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(&actions_dag)));
map.add("Expression", expression->toTree()); map.add("Expression", expression->toTree());
} }
@ -104,13 +104,13 @@ void FilterStep::updateOutputStream()
{ {
output_stream = createOutputStream( output_stream = createOutputStream(
input_streams.front(), input_streams.front(),
FilterTransform::transformHeader(input_streams.front().header, actions_dag.get(), filter_column_name, remove_filter_column), FilterTransform::transformHeader(input_streams.front().header, &actions_dag, filter_column_name, remove_filter_column),
getDataStreamTraits()); getDataStreamTraits());
if (!getDataStreamTraits().preserves_sorting) if (!getDataStreamTraits().preserves_sorting)
return; return;
FindAliasForInputName alias_finder(*actions_dag); FindAliasForInputName alias_finder(actions_dag);
const auto & input_sort_description = getInputStreams().front().sort_description; const auto & input_sort_description = getInputStreams().front().sort_description;
for (size_t i = 0, s = input_sort_description.size(); i < s; ++i) for (size_t i = 0, s = input_sort_description.size(); i < s; ++i)
{ {

View File

@ -1,19 +1,17 @@
#pragma once #pragma once
#include <Processors/QueryPlan/ITransformingStep.h> #include <Processors/QueryPlan/ITransformingStep.h>
#include <Interpreters/ActionsDAG.h>
namespace DB namespace DB
{ {
class ActionsDAG;
using ActionsDAGPtr = std::unique_ptr<ActionsDAG>;
/// Implements WHERE, HAVING operations. See FilterTransform. /// Implements WHERE, HAVING operations. See FilterTransform.
class FilterStep : public ITransformingStep class FilterStep : public ITransformingStep
{ {
public: public:
FilterStep( FilterStep(
const DataStream & input_stream_, const DataStream & input_stream_,
const ActionsDAGPtr & actions_dag_, ActionsDAG actions_dag_,
String filter_column_name_, String filter_column_name_,
bool remove_filter_column_); bool remove_filter_column_);
@ -23,15 +21,15 @@ public:
void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(JSONBuilder::JSONMap & map) const override;
void describeActions(FormatSettings & settings) const override; void describeActions(FormatSettings & settings) const override;
const ActionsDAGPtr & getExpression() const { return actions_dag; } const ActionsDAG & getExpression() const { return actions_dag; }
ActionsDAGPtr & getExpression() { return actions_dag; } ActionsDAG & getExpression() { return actions_dag; }
const String & getFilterColumnName() const { return filter_column_name; } const String & getFilterColumnName() const { return filter_column_name; }
bool removesFilterColumn() const { return remove_filter_column; } bool removesFilterColumn() const { return remove_filter_column; }
private: private:
void updateOutputStream() override; void updateOutputStream() override;
ActionsDAGPtr actions_dag; ActionsDAG actions_dag;
String filter_column_name; String filter_column_name;
bool remove_filter_column; bool remove_filter_column;
}; };

View File

@ -45,10 +45,10 @@ size_t tryConvertOuterJoinToInnerJoin(QueryPlan::Node * parent_node, QueryPlan::
bool right_stream_safe = true; bool right_stream_safe = true;
if (check_left_stream) if (check_left_stream)
left_stream_safe = filter_dag->isFilterAlwaysFalseForDefaultValueInputs(filter_column_name, left_stream_input_header); left_stream_safe = filter_dag.isFilterAlwaysFalseForDefaultValueInputs(filter_column_name, left_stream_input_header);
if (check_right_stream) if (check_right_stream)
right_stream_safe = filter_dag->isFilterAlwaysFalseForDefaultValueInputs(filter_column_name, right_stream_input_header); right_stream_safe = filter_dag.isFilterAlwaysFalseForDefaultValueInputs(filter_column_name, right_stream_input_header);
if (!left_stream_safe || !right_stream_safe) if (!left_stream_safe || !right_stream_safe)
return 0; return 0;

View File

@ -79,9 +79,9 @@ size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node)
steps_to_update.push_back(step); steps_to_update.push_back(step);
if (const auto * const expr = typeid_cast<const ExpressionStep *>(step); expr) if (const auto * const expr = typeid_cast<const ExpressionStep *>(step); expr)
dag_stack.push_back(expr->getExpression().get()); dag_stack.push_back(&expr->getExpression());
else if (const auto * const filter = typeid_cast<const FilterStep *>(step); filter) else if (const auto * const filter = typeid_cast<const FilterStep *>(step); filter)
dag_stack.push_back(filter->getExpression().get()); dag_stack.push_back(&filter->getExpression());
node = node->children.front(); node = node->children.front();
} }

View File

@ -101,7 +101,7 @@ static NameSet findIdentifiersOfNode(const ActionsDAG::Node * node)
return res; return res;
} }
static ActionsDAGPtr splitFilter(QueryPlan::Node * parent_node, const Names & available_inputs, size_t child_idx = 0) static std::optional<ActionsDAG> splitFilter(QueryPlan::Node * parent_node, const Names & available_inputs, size_t child_idx = 0)
{ {
QueryPlan::Node * child_node = parent_node->children.front(); QueryPlan::Node * child_node = parent_node->children.front();
checkChildrenSize(child_node, child_idx + 1); checkChildrenSize(child_node, child_idx + 1);
@ -110,16 +110,16 @@ static ActionsDAGPtr splitFilter(QueryPlan::Node * parent_node, const Names & av
auto & child = child_node->step; auto & child = child_node->step;
auto * filter = assert_cast<FilterStep *>(parent.get()); auto * filter = assert_cast<FilterStep *>(parent.get());
const auto & expression = filter->getExpression(); auto & expression = filter->getExpression();
const auto & filter_column_name = filter->getFilterColumnName(); const auto & filter_column_name = filter->getFilterColumnName();
bool removes_filter = filter->removesFilterColumn(); bool removes_filter = filter->removesFilterColumn();
const auto & all_inputs = child->getInputStreams()[child_idx].header.getColumnsWithTypeAndName(); const auto & all_inputs = child->getInputStreams()[child_idx].header.getColumnsWithTypeAndName();
return expression->splitActionsForFilterPushDown(filter_column_name, removes_filter, available_inputs, all_inputs); return expression.splitActionsForFilterPushDown(filter_column_name, removes_filter, available_inputs, all_inputs);
} }
static size_t static size_t
addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, const ActionsDAGPtr & split_filter, addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, ActionsDAG split_filter,
bool can_remove_filter = true, size_t child_idx = 0, bool update_parent_filter = true) bool can_remove_filter = true, size_t child_idx = 0, bool update_parent_filter = true)
{ {
QueryPlan::Node * child_node = parent_node->children.front(); QueryPlan::Node * child_node = parent_node->children.front();
@ -129,14 +129,14 @@ addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes,
auto & child = child_node->step; auto & child = child_node->step;
auto * filter = assert_cast<FilterStep *>(parent.get()); auto * filter = assert_cast<FilterStep *>(parent.get());
const auto & expression = filter->getExpression(); auto & expression = filter->getExpression();
const auto & filter_column_name = filter->getFilterColumnName(); const auto & filter_column_name = filter->getFilterColumnName();
const auto * filter_node = expression->tryFindInOutputs(filter_column_name); const auto * filter_node = expression.tryFindInOutputs(filter_column_name);
if (update_parent_filter && !filter_node && !filter->removesFilterColumn()) if (update_parent_filter && !filter_node && !filter->removesFilterColumn())
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(ErrorCodes::LOGICAL_ERROR,
"Filter column {} was removed from ActionsDAG but it is needed in result. DAG:\n{}", "Filter column {} was removed from ActionsDAG but it is needed in result. DAG:\n{}",
filter_column_name, expression->dumpDAG()); filter_column_name, expression.dumpDAG());
/// Add new Filter step before Child. /// Add new Filter step before Child.
/// Expression/Filter -> Child -> Something /// Expression/Filter -> Child -> Something
@ -147,10 +147,10 @@ addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes,
/// Expression/Filter -> Child -> Filter -> Something /// Expression/Filter -> Child -> Filter -> Something
/// New filter column is the first one. /// New filter column is the first one.
String split_filter_column_name = split_filter->getOutputs().front()->result_name; String split_filter_column_name = split_filter.getOutputs().front()->result_name;
node.step = std::make_unique<FilterStep>( node.step = std::make_unique<FilterStep>(
node.children.at(0)->step->getOutputStream(), split_filter, std::move(split_filter_column_name), can_remove_filter); node.children.at(0)->step->getOutputStream(), std::move(split_filter), std::move(split_filter_column_name), can_remove_filter);
if (auto * transforming_step = dynamic_cast<ITransformingStep *>(child.get())) if (auto * transforming_step = dynamic_cast<ITransformingStep *>(child.get()))
{ {
@ -176,7 +176,7 @@ addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes,
{ {
/// This means that all predicates of filter were pushed down. /// This means that all predicates of filter were pushed down.
/// Replace current actions to expression, as we don't need to filter anything. /// Replace current actions to expression, as we don't need to filter anything.
parent = std::make_unique<ExpressionStep>(child->getOutputStream(), expression); parent = std::make_unique<ExpressionStep>(child->getOutputStream(), std::move(expression));
} }
else else
{ {
@ -192,7 +192,7 @@ tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, con
bool can_remove_filter = true, size_t child_idx = 0) bool can_remove_filter = true, size_t child_idx = 0)
{ {
if (auto split_filter = splitFilter(parent_node, allowed_inputs, child_idx)) if (auto split_filter = splitFilter(parent_node, allowed_inputs, child_idx))
return addNewFilterStepOrThrow(parent_node, nodes, split_filter, can_remove_filter, child_idx); return addNewFilterStepOrThrow(parent_node, nodes, std::move(*split_filter), can_remove_filter, child_idx);
return 0; return 0;
} }
@ -332,7 +332,7 @@ static size_t tryPushDownOverJoinStep(QueryPlan::Node * parent_node, QueryPlan::
Names left_stream_available_columns_to_push_down = get_available_columns_for_filter(true /*push_to_left_stream*/, left_stream_filter_push_down_input_columns_available); Names left_stream_available_columns_to_push_down = get_available_columns_for_filter(true /*push_to_left_stream*/, left_stream_filter_push_down_input_columns_available);
Names right_stream_available_columns_to_push_down = get_available_columns_for_filter(false /*push_to_left_stream*/, right_stream_filter_push_down_input_columns_available); Names right_stream_available_columns_to_push_down = get_available_columns_for_filter(false /*push_to_left_stream*/, right_stream_filter_push_down_input_columns_available);
auto join_filter_push_down_actions = filter->getExpression()->splitActionsForJOINFilterPushDown(filter->getFilterColumnName(), auto join_filter_push_down_actions = filter->getExpression().splitActionsForJOINFilterPushDown(filter->getFilterColumnName(),
filter->removesFilterColumn(), filter->removesFilterColumn(),
left_stream_available_columns_to_push_down, left_stream_available_columns_to_push_down,
left_stream_input_header, left_stream_input_header,
@ -346,42 +346,44 @@ static size_t tryPushDownOverJoinStep(QueryPlan::Node * parent_node, QueryPlan::
if (join_filter_push_down_actions.left_stream_filter_to_push_down) if (join_filter_push_down_actions.left_stream_filter_to_push_down)
{ {
const auto & result_name = join_filter_push_down_actions.left_stream_filter_to_push_down->getOutputs()[0]->result_name;
updated_steps += addNewFilterStepOrThrow(parent_node, updated_steps += addNewFilterStepOrThrow(parent_node,
nodes, nodes,
join_filter_push_down_actions.left_stream_filter_to_push_down, std::move(*join_filter_push_down_actions.left_stream_filter_to_push_down),
join_filter_push_down_actions.left_stream_filter_removes_filter, join_filter_push_down_actions.left_stream_filter_removes_filter,
0 /*child_idx*/, 0 /*child_idx*/,
false /*update_parent_filter*/); false /*update_parent_filter*/);
LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"), LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"),
"Pushed down filter {} to the {} side of join", "Pushed down filter {} to the {} side of join",
join_filter_push_down_actions.left_stream_filter_to_push_down->getOutputs()[0]->result_name, result_name,
JoinKind::Left); JoinKind::Left);
} }
if (join_filter_push_down_actions.right_stream_filter_to_push_down && allow_push_down_to_right) if (join_filter_push_down_actions.right_stream_filter_to_push_down && allow_push_down_to_right)
{ {
const auto & result_name = join_filter_push_down_actions.right_stream_filter_to_push_down->getOutputs()[0]->result_name;
updated_steps += addNewFilterStepOrThrow(parent_node, updated_steps += addNewFilterStepOrThrow(parent_node,
nodes, nodes,
join_filter_push_down_actions.right_stream_filter_to_push_down, std::move(*join_filter_push_down_actions.right_stream_filter_to_push_down),
join_filter_push_down_actions.right_stream_filter_removes_filter, join_filter_push_down_actions.right_stream_filter_removes_filter,
1 /*child_idx*/, 1 /*child_idx*/,
false /*update_parent_filter*/); false /*update_parent_filter*/);
LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"), LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"),
"Pushed down filter {} to the {} side of join", "Pushed down filter {} to the {} side of join",
join_filter_push_down_actions.right_stream_filter_to_push_down->getOutputs()[0]->result_name, result_name,
JoinKind::Right); JoinKind::Right);
} }
if (updated_steps > 0) if (updated_steps > 0)
{ {
const auto & filter_column_name = filter->getFilterColumnName(); const auto & filter_column_name = filter->getFilterColumnName();
const auto & filter_expression = filter->getExpression(); auto & filter_expression = filter->getExpression();
const auto * filter_node = filter_expression->tryFindInOutputs(filter_column_name); const auto * filter_node = filter_expression.tryFindInOutputs(filter_column_name);
if (!filter_node && !filter->removesFilterColumn()) if (!filter_node && !filter->removesFilterColumn())
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(ErrorCodes::LOGICAL_ERROR,
"Filter column {} was removed from ActionsDAG but it is needed in result. DAG:\n{}", "Filter column {} was removed from ActionsDAG but it is needed in result. DAG:\n{}",
filter_column_name, filter_expression->dumpDAG()); filter_column_name, filter_expression.dumpDAG());
/// Filter column was replaced to constant. /// Filter column was replaced to constant.
@ -391,7 +393,7 @@ static size_t tryPushDownOverJoinStep(QueryPlan::Node * parent_node, QueryPlan::
{ {
/// This means that all predicates of filter were pushed down. /// This means that all predicates of filter were pushed down.
/// Replace current actions to expression, as we don't need to filter anything. /// Replace current actions to expression, as we don't need to filter anything.
parent = std::make_unique<ExpressionStep>(child->getOutputStream(), filter_expression); parent = std::make_unique<ExpressionStep>(child->getOutputStream(), std::move(filter_expression));
} }
else else
{ {
@ -416,7 +418,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes
if (!filter) if (!filter)
return 0; return 0;
if (filter->getExpression()->hasStatefulFunctions()) if (filter->getExpression().hasStatefulFunctions())
return 0; return 0;
if (auto * aggregating = typeid_cast<AggregatingStep *>(child.get())) if (auto * aggregating = typeid_cast<AggregatingStep *>(child.get()))
@ -430,7 +432,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes
return 0; return 0;
const auto & actions = filter->getExpression(); const auto & actions = filter->getExpression();
const auto & filter_node = actions->findInOutputs(filter->getFilterColumnName()); const auto & filter_node = actions.findInOutputs(filter->getFilterColumnName());
auto identifiers_in_predicate = findIdentifiersOfNode(&filter_node); auto identifiers_in_predicate = findIdentifiersOfNode(&filter_node);
@ -597,7 +599,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes
filter_node.step = std::make_unique<FilterStep>( filter_node.step = std::make_unique<FilterStep>(
filter_node.children.front()->step->getOutputStream(), filter_node.children.front()->step->getOutputStream(),
ActionsDAG::clone(filter->getExpression()), std::move(*ActionsDAG::clone(&filter->getExpression())),
filter->getFilterColumnName(), filter->getFilterColumnName(),
filter->removesFilterColumn()); filter->removesFilterColumn());
} }
@ -611,7 +613,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes
if (auto * read_from_merge = typeid_cast<ReadFromMerge *>(child.get())) if (auto * read_from_merge = typeid_cast<ReadFromMerge *>(child.get()))
{ {
FilterDAGInfo info{ActionsDAG::clone(filter->getExpression()), filter->getFilterColumnName(), filter->removesFilterColumn()}; FilterDAGInfo info{std::move(*ActionsDAG::clone(&filter->getExpression())), filter->getFilterColumnName(), filter->removesFilterColumn()};
read_from_merge->addFilter(std::move(info)); read_from_merge->addFilter(std::move(info));
std::swap(*parent_node, *child_node); std::swap(*parent_node, *child_node);
return 1; return 1;

View File

@ -28,10 +28,10 @@ size_t tryLiftUpArrayJoin(QueryPlan::Node * parent_node, QueryPlan::Nodes & node
const auto & expression = expression_step ? expression_step->getExpression() const auto & expression = expression_step ? expression_step->getExpression()
: filter_step->getExpression(); : filter_step->getExpression();
auto split_actions = expression->splitActionsBeforeArrayJoin(array_join->columns); auto split_actions = expression.splitActionsBeforeArrayJoin(array_join->columns);
/// No actions can be moved before ARRAY JOIN. /// No actions can be moved before ARRAY JOIN.
if (split_actions.first->trivial()) if (split_actions.first.trivial())
return 0; return 0;
auto description = parent->getStepDescription(); auto description = parent->getStepDescription();
@ -49,9 +49,9 @@ size_t tryLiftUpArrayJoin(QueryPlan::Node * parent_node, QueryPlan::Nodes & node
array_join_step->updateInputStream(node.step->getOutputStream()); array_join_step->updateInputStream(node.step->getOutputStream());
if (expression_step) if (expression_step)
parent = std::make_unique<ExpressionStep>(array_join_step->getOutputStream(), split_actions.second); parent = std::make_unique<ExpressionStep>(array_join_step->getOutputStream(), std::move(split_actions.second));
else else
parent = std::make_unique<FilterStep>(array_join_step->getOutputStream(), split_actions.second, parent = std::make_unique<FilterStep>(array_join_step->getOutputStream(), std::move(split_actions.second),
filter_step->getFilterColumnName(), filter_step->removesFilterColumn()); filter_step->getFilterColumnName(), filter_step->removesFilterColumn());
parent->setStepDescription(description + " [split]"); parent->setStepDescription(description + " [split]");

View File

@ -66,13 +66,13 @@ size_t tryExecuteFunctionsAfterSorting(QueryPlan::Node * parent_node, QueryPlan:
NameSet sort_columns; NameSet sort_columns;
for (const auto & col : sorting_step->getSortDescription()) for (const auto & col : sorting_step->getSortDescription())
sort_columns.insert(col.column_name); sort_columns.insert(col.column_name);
auto [needed_for_sorting, unneeded_for_sorting, _] = expression_step->getExpression()->splitActionsBySortingDescription(sort_columns); auto [needed_for_sorting, unneeded_for_sorting, _] = expression_step->getExpression().splitActionsBySortingDescription(sort_columns);
// No calculations can be postponed. // No calculations can be postponed.
if (unneeded_for_sorting->trivial()) if (unneeded_for_sorting.trivial())
return 0; return 0;
if (!areNodesConvertableToBlock(needed_for_sorting->getOutputs()) || !areNodesConvertableToBlock(unneeded_for_sorting->getInputs())) if (!areNodesConvertableToBlock(needed_for_sorting.getOutputs()) || !areNodesConvertableToBlock(unneeded_for_sorting.getInputs()))
return 0; return 0;
// Sorting (parent_node) -> Expression (child_node) // Sorting (parent_node) -> Expression (child_node)

View File

@ -49,7 +49,7 @@ size_t tryLiftUpUnion(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes)
expr_node.step = std::make_unique<ExpressionStep>( expr_node.step = std::make_unique<ExpressionStep>(
expr_node.children.front()->step->getOutputStream(), expr_node.children.front()->step->getOutputStream(),
ActionsDAG::clone(expression->getExpression())); std::move(*ActionsDAG::clone(&expression->getExpression())));
} }
/// - Expression - Something /// - Expression - Something

View File

@ -38,18 +38,18 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &)
if (parent_expr && child_expr) if (parent_expr && child_expr)
{ {
const auto & child_actions = child_expr->getExpression(); auto & child_actions = child_expr->getExpression();
const auto & parent_actions = parent_expr->getExpression(); auto & parent_actions = parent_expr->getExpression();
/// We cannot combine actions with arrayJoin and stateful function because we not always can reorder them. /// We cannot combine actions with arrayJoin and stateful function because we not always can reorder them.
/// Example: select rowNumberInBlock() from (select arrayJoin([1, 2])) /// Example: select rowNumberInBlock() from (select arrayJoin([1, 2]))
/// Such a query will return two zeroes if we combine actions together. /// Such a query will return two zeroes if we combine actions together.
if (child_actions->hasArrayJoin() && parent_actions->hasStatefulFunctions()) if (child_actions.hasArrayJoin() && parent_actions.hasStatefulFunctions())
return 0; return 0;
auto merged = ActionsDAG::merge(std::move(*child_actions), std::move(*parent_actions)); auto merged = ActionsDAG::merge(std::move(child_actions), std::move(parent_actions));
auto expr = std::make_unique<ExpressionStep>(child_expr->getInputStreams().front(), merged); auto expr = std::make_unique<ExpressionStep>(child_expr->getInputStreams().front(), std::move(merged));
expr->setStepDescription("(" + parent_expr->getStepDescription() + " + " + child_expr->getStepDescription() + ")"); expr->setStepDescription("(" + parent_expr->getStepDescription() + " + " + child_expr->getStepDescription() + ")");
parent_node->step = std::move(expr); parent_node->step = std::move(expr);
@ -58,16 +58,16 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &)
} }
else if (parent_filter && child_expr) else if (parent_filter && child_expr)
{ {
const auto & child_actions = child_expr->getExpression(); auto & child_actions = child_expr->getExpression();
const auto & parent_actions = parent_filter->getExpression(); auto & parent_actions = parent_filter->getExpression();
if (child_actions->hasArrayJoin() && parent_actions->hasStatefulFunctions()) if (child_actions.hasArrayJoin() && parent_actions.hasStatefulFunctions())
return 0; return 0;
auto merged = ActionsDAG::merge(std::move(*child_actions), std::move(*parent_actions)); auto merged = ActionsDAG::merge(std::move(child_actions), std::move(parent_actions));
auto filter = std::make_unique<FilterStep>(child_expr->getInputStreams().front(), auto filter = std::make_unique<FilterStep>(child_expr->getInputStreams().front(),
merged, std::move(merged),
parent_filter->getFilterColumnName(), parent_filter->getFilterColumnName(),
parent_filter->removesFilterColumn()); parent_filter->removesFilterColumn());
filter->setStepDescription("(" + parent_filter->getStepDescription() + " + " + child_expr->getStepDescription() + ")"); filter->setStepDescription("(" + parent_filter->getStepDescription() + " + " + child_expr->getStepDescription() + ")");
@ -78,32 +78,31 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &)
} }
else if (parent_filter && child_filter) else if (parent_filter && child_filter)
{ {
const auto & child_actions = child_filter->getExpression(); auto & child_actions = child_filter->getExpression();
const auto & parent_actions = parent_filter->getExpression(); auto & parent_actions = parent_filter->getExpression();
if (child_actions->hasArrayJoin()) if (child_actions.hasArrayJoin())
return 0; return 0;
auto actions = ActionsDAG::clone(child_actions); const auto & child_filter_node = child_actions.findInOutputs(child_filter->getFilterColumnName());
const auto & child_filter_node = actions->findInOutputs(child_filter->getFilterColumnName());
if (child_filter->removesFilterColumn()) if (child_filter->removesFilterColumn())
removeFromOutputs(*actions, child_filter_node); removeFromOutputs(child_actions, child_filter_node);
actions->mergeInplace(std::move(*ActionsDAG::clone(parent_actions))); child_actions.mergeInplace(std::move(parent_actions));
const auto & parent_filter_node = actions->findInOutputs(parent_filter->getFilterColumnName()); const auto & parent_filter_node = child_actions.findInOutputs(parent_filter->getFilterColumnName());
if (parent_filter->removesFilterColumn()) if (parent_filter->removesFilterColumn())
removeFromOutputs(*actions, parent_filter_node); removeFromOutputs(child_actions, parent_filter_node);
FunctionOverloadResolverPtr func_builder_and = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>()); FunctionOverloadResolverPtr func_builder_and = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>());
const auto & condition = actions->addFunction(func_builder_and, {&child_filter_node, &parent_filter_node}, {}); const auto & condition = child_actions.addFunction(func_builder_and, {&child_filter_node, &parent_filter_node}, {});
auto & outputs = actions->getOutputs(); auto & outputs = child_actions.getOutputs();
outputs.insert(outputs.begin(), &condition); outputs.insert(outputs.begin(), &condition);
actions->removeUnusedActions(false); child_actions.removeUnusedActions(false);
auto filter = std::make_unique<FilterStep>(child_filter->getInputStreams().front(), auto filter = std::make_unique<FilterStep>(child_filter->getInputStreams().front(),
actions, std::move(child_actions),
condition.result_name, condition.result_name,
true); true);
filter->setStepDescription("(" + parent_filter->getStepDescription() + " + " + child_filter->getStepDescription() + ")"); filter->setStepDescription("(" + parent_filter->getStepDescription() + " + " + child_filter->getStepDescription() + ")");

View File

@ -83,10 +83,11 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &)
Names queried_columns = source_step_with_filter->requiredSourceColumns(); Names queried_columns = source_step_with_filter->requiredSourceColumns();
const auto & source_filter_actions_dag = source_step_with_filter->getFilterActionsDAG();
MergeTreeWhereOptimizer where_optimizer{ MergeTreeWhereOptimizer where_optimizer{
std::move(column_compressed_sizes), std::move(column_compressed_sizes),
storage_metadata, storage_metadata,
storage.getConditionSelectivityEstimatorByPredicate(storage_snapshot, source_step_with_filter->getFilterActionsDAG(), context), storage.getConditionSelectivityEstimatorByPredicate(storage_snapshot, source_filter_actions_dag ? &*source_filter_actions_dag : nullptr, context),
queried_columns, queried_columns,
storage.supportedPrewhereColumns(), storage.supportedPrewhereColumns(),
getLogger("QueryPlanOptimizePrewhere")}; getLogger("QueryPlanOptimizePrewhere")};
@ -113,15 +114,15 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &)
if (prewhere_info->remove_prewhere_column) if (prewhere_info->remove_prewhere_column)
{ {
removeFromOutput(*filter_expression, filter_column_name); removeFromOutput(filter_expression, filter_column_name);
auto & outputs = filter_expression->getOutputs(); auto & outputs = filter_expression.getOutputs();
size_t size = outputs.size(); size_t size = outputs.size();
outputs.insert(outputs.end(), optimize_result.prewhere_nodes.begin(), optimize_result.prewhere_nodes.end()); outputs.insert(outputs.end(), optimize_result.prewhere_nodes.begin(), optimize_result.prewhere_nodes.end());
filter_expression->removeUnusedActions(false); filter_expression.removeUnusedActions(false);
outputs.resize(size); outputs.resize(size);
} }
auto split_result = filter_expression->split(optimize_result.prewhere_nodes, true, true); auto split_result = filter_expression.split(optimize_result.prewhere_nodes, true, true);
/// This is the leak of abstraction. /// This is the leak of abstraction.
/// Splited actions may have inputs which are needed only for PREWHERE. /// Splited actions may have inputs which are needed only for PREWHERE.
@ -137,15 +138,15 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &)
/// So, here we restore removed inputs for PREWHERE actions /// So, here we restore removed inputs for PREWHERE actions
{ {
std::unordered_set<const ActionsDAG::Node *> first_outputs( std::unordered_set<const ActionsDAG::Node *> first_outputs(
split_result.first->getOutputs().begin(), split_result.first->getOutputs().end()); split_result.first.getOutputs().begin(), split_result.first.getOutputs().end());
for (const auto * input : split_result.first->getInputs()) for (const auto * input : split_result.first.getInputs())
{ {
if (!first_outputs.contains(input)) if (!first_outputs.contains(input))
{ {
split_result.first->getOutputs().push_back(input); split_result.first.getOutputs().push_back(input);
/// Add column to second actions as input. /// Add column to second actions as input.
/// Do not add it to result, so it would be removed. /// Do not add it to result, so it would be removed.
split_result.second->addInput(input->result_name, input->result_type); split_result.second.addInput(input->result_name, input->result_type);
} }
} }
} }

View File

@ -18,16 +18,16 @@ void optimizePrimaryKeyConditionAndLimit(const Stack & stack)
const auto & storage_prewhere_info = source_step_with_filter->getPrewhereInfo(); const auto & storage_prewhere_info = source_step_with_filter->getPrewhereInfo();
if (storage_prewhere_info) if (storage_prewhere_info)
{ {
source_step_with_filter->addFilter(ActionsDAG::clone(storage_prewhere_info->prewhere_actions), storage_prewhere_info->prewhere_column_name); source_step_with_filter->addFilter(ActionsDAG::clone(&*storage_prewhere_info->prewhere_actions), storage_prewhere_info->prewhere_column_name);
if (storage_prewhere_info->row_level_filter) if (storage_prewhere_info->row_level_filter)
source_step_with_filter->addFilter(ActionsDAG::clone(storage_prewhere_info->row_level_filter), storage_prewhere_info->row_level_column_name); source_step_with_filter->addFilter(ActionsDAG::clone(&*storage_prewhere_info->row_level_filter), storage_prewhere_info->row_level_column_name);
} }
for (auto iter = stack.rbegin() + 1; iter != stack.rend(); ++iter) for (auto iter = stack.rbegin() + 1; iter != stack.rend(); ++iter)
{ {
if (auto * filter_step = typeid_cast<FilterStep *>(iter->node->step.get())) if (auto * filter_step = typeid_cast<FilterStep *>(iter->node->step.get()))
{ {
source_step_with_filter->addFilter(ActionsDAG::clone(filter_step->getExpression()), filter_step->getFilterColumnName()); source_step_with_filter->addFilter(ActionsDAG::clone(&filter_step->getExpression()), filter_step->getFilterColumnName());
} }
else if (auto * limit_step = typeid_cast<LimitStep *>(iter->node->step.get())) else if (auto * limit_step = typeid_cast<LimitStep *>(iter->node->step.get()))
{ {

View File

@ -170,12 +170,12 @@ static void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filt
} }
} }
static void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) static void appendExpression(ActionsDAGPtr & dag, const ActionsDAG & expression)
{ {
if (dag) if (dag)
dag->mergeInplace(std::move(*ActionsDAG::clone(expression))); dag->mergeInplace(std::move(*ActionsDAG::clone(&expression)));
else else
dag = ActionsDAG::clone(expression); dag = ActionsDAG::clone(&expression);
} }
/// This function builds a common DAG which is a merge of DAGs from Filter and Expression steps chain. /// This function builds a common DAG which is a merge of DAGs from Filter and Expression steps chain.
@ -193,7 +193,7 @@ void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns &
if (prewhere_info->prewhere_actions) if (prewhere_info->prewhere_actions)
{ {
//std::cerr << "====== Adding prewhere " << std::endl; //std::cerr << "====== Adding prewhere " << std::endl;
appendExpression(dag, prewhere_info->prewhere_actions); appendExpression(dag, *prewhere_info->prewhere_actions);
if (const auto * filter_expression = dag->tryFindInOutputs(prewhere_info->prewhere_column_name)) if (const auto * filter_expression = dag->tryFindInOutputs(prewhere_info->prewhere_column_name))
appendFixedColumnsFromFilterExpression(*filter_expression, fixed_columns); appendFixedColumnsFromFilterExpression(*filter_expression, fixed_columns);
} }
@ -211,7 +211,7 @@ void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns &
const auto & actions = expression->getExpression(); const auto & actions = expression->getExpression();
/// Should ignore limit because arrayJoin() can reduce the number of rows in case of empty array. /// Should ignore limit because arrayJoin() can reduce the number of rows in case of empty array.
if (actions->hasArrayJoin()) if (actions.hasArrayJoin())
limit = 0; limit = 0;
appendExpression(dag, actions); appendExpression(dag, actions);
@ -1066,13 +1066,13 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node,
for (const auto & actions_dag : window_desc.partition_by_actions) for (const auto & actions_dag : window_desc.partition_by_actions)
{ {
order_by_elements_actions.emplace_back( order_by_elements_actions.emplace_back(
std::make_shared<ExpressionActions>(ActionsDAG::clone(actions_dag.get()), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(actions_dag.get())), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes)));
} }
for (const auto & actions_dag : window_desc.order_by_actions) for (const auto & actions_dag : window_desc.order_by_actions)
{ {
order_by_elements_actions.emplace_back( order_by_elements_actions.emplace_back(
std::make_shared<ExpressionActions>(ActionsDAG::clone(actions_dag.get()), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes))); std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(actions_dag.get())), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes)));
} }
auto order_optimizer = std::make_shared<ReadInOrderOptimizer>( auto order_optimizer = std::make_shared<ReadInOrderOptimizer>(

View File

@ -273,7 +273,7 @@ static void appendAggregateFunctions(
} }
} }
ActionsDAGPtr analyzeAggregateProjection( std::optional<ActionsDAG> analyzeAggregateProjection(
const AggregateProjectionInfo & info, const AggregateProjectionInfo & info,
const QueryDAG & query, const QueryDAG & query,
const DAGIndex & query_index, const DAGIndex & query_index,
@ -393,7 +393,7 @@ ActionsDAGPtr analyzeAggregateProjection(
// LOG_TRACE(getLogger("optimizeUseProjections"), "Folding actions by projection"); // LOG_TRACE(getLogger("optimizeUseProjections"), "Folding actions by projection");
auto proj_dag = query.dag->foldActionsByProjection(new_inputs, query_key_nodes); auto proj_dag = query.dag->foldActionsByProjection(new_inputs, query_key_nodes);
appendAggregateFunctions(*proj_dag, aggregates, *matched_aggregates); appendAggregateFunctions(proj_dag, aggregates, *matched_aggregates);
return proj_dag; return proj_dag;
} }
@ -405,7 +405,7 @@ struct AggregateProjectionCandidate : public ProjectionCandidate
/// Actions which need to be applied to columns from projection /// Actions which need to be applied to columns from projection
/// in order to get all the columns required for aggregation. /// in order to get all the columns required for aggregation.
ActionsDAGPtr dag; ActionsDAG dag;
}; };
struct MinMaxProjectionCandidate struct MinMaxProjectionCandidate
@ -480,13 +480,13 @@ AggregateProjectionCandidates getAggregateProjectionCandidates(
if (auto proj_dag = analyzeAggregateProjection(info, dag, query_index, keys, aggregates)) if (auto proj_dag = analyzeAggregateProjection(info, dag, query_index, keys, aggregates))
{ {
// LOG_TRACE(getLogger("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); // LOG_TRACE(getLogger("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG());
AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(proj_dag)}; AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(*proj_dag)};
// LOG_TRACE(getLogger("optimizeUseProjections"), "Projection sample block {}", sample_block.dumpStructure()); // LOG_TRACE(getLogger("optimizeUseProjections"), "Projection sample block {}", sample_block.dumpStructure());
auto block = reading.getMergeTreeData().getMinMaxCountProjectionBlock( auto block = reading.getMergeTreeData().getMinMaxCountProjectionBlock(
metadata, metadata,
candidate.dag->getRequiredColumnsNames(), candidate.dag.getRequiredColumnsNames(),
(dag.filter_node ? dag.dag.get() : nullptr), (dag.filter_node ? &*dag.dag : nullptr),
parts, parts,
max_added_blocks.get(), max_added_blocks.get(),
context); context);
@ -536,7 +536,7 @@ AggregateProjectionCandidates getAggregateProjectionCandidates(
if (auto proj_dag = analyzeAggregateProjection(info, dag, query_index, keys, aggregates)) if (auto proj_dag = analyzeAggregateProjection(info, dag, query_index, keys, aggregates))
{ {
// LOG_TRACE(getLogger("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG()); // LOG_TRACE(getLogger("optimizeUseProjections"), "Projection analyzed DAG {}", proj_dag->dumpDAG());
AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(proj_dag)}; AggregateProjectionCandidate candidate{.info = std::move(info), .dag = std::move(*proj_dag)};
candidate.projection = projection; candidate.projection = projection;
candidates.real.emplace_back(std::move(candidate)); candidates.real.emplace_back(std::move(candidate));
} }
@ -664,7 +664,7 @@ std::optional<String> optimizeUseAggregateProjections(QueryPlan::Node & node, Qu
/// Selecting best candidate. /// Selecting best candidate.
for (auto & candidate : candidates.real) for (auto & candidate : candidates.real)
{ {
auto required_column_names = candidate.dag->getRequiredColumnsNames(); auto required_column_names = candidate.dag.getRequiredColumnsNames();
bool analyzed = analyzeProjectionCandidate( bool analyzed = analyzeProjectionCandidate(
candidate, candidate,
@ -675,7 +675,7 @@ std::optional<String> optimizeUseAggregateProjections(QueryPlan::Node & node, Qu
query_info, query_info,
context, context,
max_added_blocks, max_added_blocks,
candidate.dag.get()); &candidate.dag);
if (!analyzed) if (!analyzed)
continue; continue;
@ -765,7 +765,7 @@ std::optional<String> optimizeUseAggregateProjections(QueryPlan::Node & node, Qu
projection_reading = reader.readFromParts( projection_reading = reader.readFromParts(
/* parts = */ {}, /* parts = */ {},
/* alter_conversions = */ {}, /* alter_conversions = */ {},
best_candidate->dag->getRequiredColumnsNames(), best_candidate->dag.getRequiredColumnsNames(),
proj_snapshot, proj_snapshot,
projection_query_info, projection_query_info,
context, context,
@ -777,7 +777,7 @@ std::optional<String> optimizeUseAggregateProjections(QueryPlan::Node & node, Qu
if (!projection_reading) if (!projection_reading)
{ {
auto header = proj_snapshot->getSampleBlockForColumns(best_candidate->dag->getRequiredColumnsNames()); auto header = proj_snapshot->getSampleBlockForColumns(best_candidate->dag.getRequiredColumnsNames());
Pipe pipe(std::make_shared<NullSource>(std::move(header))); Pipe pipe(std::make_shared<NullSource>(std::move(header)));
projection_reading = std::make_unique<ReadFromPreparedSource>(std::move(pipe)); projection_reading = std::make_unique<ReadFromPreparedSource>(std::move(pipe));
} }
@ -808,17 +808,19 @@ std::optional<String> optimizeUseAggregateProjections(QueryPlan::Node & node, Qu
if (best_candidate) if (best_candidate)
{ {
aggregate_projection_node = &nodes.emplace_back(); aggregate_projection_node = &nodes.emplace_back();
if (candidates.has_filter) if (candidates.has_filter)
{ {
const auto & result_name = best_candidate->dag.getOutputs().front()->result_name;
aggregate_projection_node->step = std::make_unique<FilterStep>( aggregate_projection_node->step = std::make_unique<FilterStep>(
projection_reading_node.step->getOutputStream(), projection_reading_node.step->getOutputStream(),
best_candidate->dag, std::move(best_candidate->dag),
best_candidate->dag->getOutputs().front()->result_name, result_name,
true); true);
} }
else else
aggregate_projection_node->step aggregate_projection_node->step
= std::make_unique<ExpressionStep>(projection_reading_node.step->getOutputStream(), best_candidate->dag); = std::make_unique<ExpressionStep>(projection_reading_node.step->getOutputStream(), std::move(best_candidate->dag));
aggregate_projection_node->children.push_back(&projection_reading_node); aggregate_projection_node->children.push_back(&projection_reading_node);
} }

View File

@ -23,7 +23,7 @@ struct NormalProjectionCandidate : public ProjectionCandidate
{ {
}; };
static ActionsDAGPtr makeMaterializingDAG(const Block & proj_header, const Block main_header) static std::optional<ActionsDAG> makeMaterializingDAG(const Block & proj_header, const Block main_header)
{ {
/// Materialize constants in case we don't have it in output header. /// Materialize constants in case we don't have it in output header.
/// This may happen e.g. if we have PREWHERE. /// This may happen e.g. if we have PREWHERE.
@ -31,7 +31,7 @@ static ActionsDAGPtr makeMaterializingDAG(const Block & proj_header, const Block
size_t num_columns = main_header.columns(); size_t num_columns = main_header.columns();
/// This is a error; will have block structure mismatch later. /// This is a error; will have block structure mismatch later.
if (proj_header.columns() != num_columns) if (proj_header.columns() != num_columns)
return nullptr; return {};
std::vector<size_t> const_positions; std::vector<size_t> const_positions;
for (size_t i = 0; i < num_columns; ++i) for (size_t i = 0; i < num_columns; ++i)
@ -45,17 +45,17 @@ static ActionsDAGPtr makeMaterializingDAG(const Block & proj_header, const Block
} }
if (const_positions.empty()) if (const_positions.empty())
return nullptr; return {};
ActionsDAGPtr dag = std::make_unique<ActionsDAG>(); ActionsDAG dag;
auto & outputs = dag->getOutputs(); auto & outputs = dag.getOutputs();
for (const auto & col : proj_header.getColumnsWithTypeAndName()) for (const auto & col : proj_header.getColumnsWithTypeAndName())
outputs.push_back(&dag->addInput(col)); outputs.push_back(&dag.addInput(col));
for (auto pos : const_positions) for (auto pos : const_positions)
{ {
auto & output = outputs[pos]; auto & output = outputs[pos];
output = &dag->materializeNode(*output); output = &dag.materializeNode(*output);
} }
return dag; return dag;
@ -172,7 +172,7 @@ std::optional<String> optimizeUseNormalProjections(Stack & stack, QueryPlan::Nod
query_info, query_info,
context, context,
max_added_blocks, max_added_blocks,
query.filter_node ? query.dag.get() : nullptr); query.filter_node ? &*query.dag : nullptr);
if (!analyzed) if (!analyzed)
continue; continue;
@ -242,14 +242,14 @@ std::optional<String> optimizeUseNormalProjections(Stack & stack, QueryPlan::Nod
{ {
expr_or_filter_node.step = std::make_unique<FilterStep>( expr_or_filter_node.step = std::make_unique<FilterStep>(
projection_reading_node.step->getOutputStream(), projection_reading_node.step->getOutputStream(),
query.dag, std::move(*query.dag),
query.filter_node->result_name, query.filter_node->result_name,
true); true);
} }
else else
expr_or_filter_node.step = std::make_unique<ExpressionStep>( expr_or_filter_node.step = std::make_unique<ExpressionStep>(
projection_reading_node.step->getOutputStream(), projection_reading_node.step->getOutputStream(),
query.dag); std::move(*query.dag));
expr_or_filter_node.children.push_back(&projection_reading_node); expr_or_filter_node.children.push_back(&projection_reading_node);
next_node = &expr_or_filter_node; next_node = &expr_or_filter_node;
@ -267,7 +267,7 @@ std::optional<String> optimizeUseNormalProjections(Stack & stack, QueryPlan::Nod
if (auto materializing = makeMaterializingDAG(proj_stream->header, main_stream.header)) if (auto materializing = makeMaterializingDAG(proj_stream->header, main_stream.header))
{ {
auto converting = std::make_unique<ExpressionStep>(*proj_stream, materializing); auto converting = std::make_unique<ExpressionStep>(*proj_stream, std::move(*materializing));
proj_stream = &converting->getOutputStream(); proj_stream = &converting->getOutputStream();
auto & expr_node = nodes.emplace_back(); auto & expr_node = nodes.emplace_back();
expr_node.step = std::move(converting); expr_node.step = std::move(converting);

View File

@ -64,12 +64,12 @@ std::shared_ptr<PartitionIdToMaxBlock> getMaxAddedBlocks(ReadFromMergeTree * rea
return {}; return {};
} }
void QueryDAG::appendExpression(const ActionsDAGPtr & expression) void QueryDAG::appendExpression(const ActionsDAG & expression)
{ {
if (dag) if (dag)
dag->mergeInplace(std::move(*ActionsDAG::clone(expression))); dag->mergeInplace(std::move(*ActionsDAG::clone(&expression)));
else else
dag = ActionsDAG::clone(expression); dag = std::move(*ActionsDAG::clone(&expression));
} }
const ActionsDAG::Node * findInOutputs(ActionsDAG & dag, const std::string & name, bool remove) const ActionsDAG::Node * findInOutputs(ActionsDAG & dag, const std::string & name, bool remove)
@ -120,7 +120,7 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs &
{ {
if (prewhere_info->row_level_filter) if (prewhere_info->row_level_filter)
{ {
appendExpression(prewhere_info->row_level_filter); appendExpression(*prewhere_info->row_level_filter);
if (const auto * filter_expression = findInOutputs(*dag, prewhere_info->row_level_column_name, false)) if (const auto * filter_expression = findInOutputs(*dag, prewhere_info->row_level_column_name, false))
filter_nodes.push_back(filter_expression); filter_nodes.push_back(filter_expression);
else else
@ -129,7 +129,7 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs &
if (prewhere_info->prewhere_actions) if (prewhere_info->prewhere_actions)
{ {
appendExpression(prewhere_info->prewhere_actions); appendExpression(*prewhere_info->prewhere_actions);
if (const auto * filter_expression if (const auto * filter_expression
= findInOutputs(*dag, prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column)) = findInOutputs(*dag, prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column))
filter_nodes.push_back(filter_expression); filter_nodes.push_back(filter_expression);
@ -149,7 +149,7 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs &
if (auto * expression = typeid_cast<ExpressionStep *>(step)) if (auto * expression = typeid_cast<ExpressionStep *>(step))
{ {
const auto & actions = expression->getExpression(); const auto & actions = expression->getExpression();
if (actions->hasArrayJoin()) if (actions.hasArrayJoin())
return false; return false;
appendExpression(actions); appendExpression(actions);
@ -159,7 +159,7 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs &
if (auto * filter = typeid_cast<FilterStep *>(step)) if (auto * filter = typeid_cast<FilterStep *>(step))
{ {
const auto & actions = filter->getExpression(); const auto & actions = filter->getExpression();
if (actions->hasArrayJoin()) if (actions.hasArrayJoin())
return false; return false;
appendExpression(actions); appendExpression(actions);

View File

@ -25,14 +25,14 @@ std::shared_ptr<PartitionIdToMaxBlock> getMaxAddedBlocks(ReadFromMergeTree * rea
/// Additionally, for all the Filter steps, we collect filter conditions into filter_nodes. /// Additionally, for all the Filter steps, we collect filter conditions into filter_nodes.
struct QueryDAG struct QueryDAG
{ {
ActionsDAGPtr dag; std::optional<ActionsDAG> dag;
const ActionsDAG::Node * filter_node = nullptr; const ActionsDAG::Node * filter_node = nullptr;
bool build(QueryPlan::Node & node); bool build(QueryPlan::Node & node);
private: private:
bool buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs & filter_nodes); bool buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs & filter_nodes);
void appendExpression(const ActionsDAGPtr & expression); void appendExpression(const ActionsDAG & expression);
}; };
struct ProjectionCandidate struct ProjectionCandidate

View File

@ -132,10 +132,10 @@ namespace
return true; return true;
if (const auto * const expr = typeid_cast<const ExpressionStep *>(step); expr) if (const auto * const expr = typeid_cast<const ExpressionStep *>(step); expr)
return !expr->getExpression()->hasArrayJoin(); return !expr->getExpression().hasArrayJoin();
if (const auto * const filter = typeid_cast<const FilterStep *>(step); filter) if (const auto * const filter = typeid_cast<const FilterStep *>(step); filter)
return !filter->getExpression()->hasArrayJoin(); return !filter->getExpression().hasArrayJoin();
if (typeid_cast<const LimitStep *>(step) || typeid_cast<const LimitByStep *>(step) || typeid_cast<const SortingStep *>(step) if (typeid_cast<const LimitStep *>(step) || typeid_cast<const LimitByStep *>(step) || typeid_cast<const SortingStep *>(step)
|| typeid_cast<const WindowStep *>(step)) || typeid_cast<const WindowStep *>(step))
@ -183,9 +183,9 @@ namespace
} }
if (const auto * const expr = typeid_cast<const ExpressionStep *>(current_step); expr) if (const auto * const expr = typeid_cast<const ExpressionStep *>(current_step); expr)
dag_stack.push_back(expr->getExpression().get()); dag_stack.push_back(&expr->getExpression());
else if (const auto * const filter = typeid_cast<const FilterStep *>(current_step); filter) else if (const auto * const filter = typeid_cast<const FilterStep *>(current_step); filter)
dag_stack.push_back(filter->getExpression().get()); dag_stack.push_back(&filter->getExpression());
node = node->children.front(); node = node->children.front();
if (inner_distinct_step = typeid_cast<DistinctStep *>(node->step.get()); inner_distinct_step) if (inner_distinct_step = typeid_cast<DistinctStep *>(node->step.get()); inner_distinct_step)
@ -236,9 +236,9 @@ namespace
} }
if (const auto * const expr = typeid_cast<const ExpressionStep *>(current_step); expr) if (const auto * const expr = typeid_cast<const ExpressionStep *>(current_step); expr)
dag_stack.push_back(expr->getExpression().get()); dag_stack.push_back(&expr->getExpression());
else if (const auto * const filter = typeid_cast<const FilterStep *>(current_step); filter) else if (const auto * const filter = typeid_cast<const FilterStep *>(current_step); filter)
dag_stack.push_back(filter->getExpression().get()); dag_stack.push_back(&filter->getExpression());
node = node->children.front(); node = node->children.front();
inner_distinct_step = typeid_cast<DistinctStep *>(node->step.get()); inner_distinct_step = typeid_cast<DistinctStep *>(node->step.get());

View File

@ -213,12 +213,12 @@ private:
logStep("checking for stateful function", node); logStep("checking for stateful function", node);
if (const auto * expr = typeid_cast<const ExpressionStep *>(step); expr) if (const auto * expr = typeid_cast<const ExpressionStep *>(step); expr)
{ {
if (expr->getExpression()->hasStatefulFunctions()) if (expr->getExpression().hasStatefulFunctions())
return false; return false;
} }
else if (const auto * filter = typeid_cast<const FilterStep *>(step); filter) else if (const auto * filter = typeid_cast<const FilterStep *>(step); filter)
{ {
if (filter->getExpression()->hasStatefulFunctions()) if (filter->getExpression().hasStatefulFunctions())
return false; return false;
} }
else else

View File

@ -17,13 +17,13 @@ size_t trySplitFilter(QueryPlan::Node * node, QueryPlan::Nodes & nodes)
const std::string & filter_column_name = filter_step->getFilterColumnName(); const std::string & filter_column_name = filter_step->getFilterColumnName();
/// Do not split if there are function like runningDifference. /// Do not split if there are function like runningDifference.
if (expr->hasStatefulFunctions()) if (expr.hasStatefulFunctions())
return 0; return 0;
bool filter_name_clashs_with_input = false; bool filter_name_clashs_with_input = false;
if (filter_step->removesFilterColumn()) if (filter_step->removesFilterColumn())
{ {
for (const auto * input : expr->getInputs()) for (const auto * input : expr.getInputs())
{ {
if (input->result_name == filter_column_name) if (input->result_name == filter_column_name)
{ {
@ -33,14 +33,14 @@ size_t trySplitFilter(QueryPlan::Node * node, QueryPlan::Nodes & nodes)
} }
} }
auto split = expr->splitActionsForFilter(filter_column_name); auto split = expr.splitActionsForFilter(filter_column_name);
if (split.second->trivial()) if (split.second.trivial())
return 0; return 0;
bool remove_filter = false; bool remove_filter = false;
if (filter_step->removesFilterColumn()) if (filter_step->removesFilterColumn())
remove_filter = split.second->removeUnusedResult(filter_column_name); remove_filter = split.second.removeUnusedResult(filter_column_name);
auto description = filter_step->getStepDescription(); auto description = filter_step->getStepDescription();
@ -53,11 +53,11 @@ size_t trySplitFilter(QueryPlan::Node * node, QueryPlan::Nodes & nodes)
{ {
split_filter_name = "__split_filter"; split_filter_name = "__split_filter";
for (auto & filter_output : split.first->getOutputs()) for (auto & filter_output : split.first.getOutputs())
{ {
if (filter_output->result_name == filter_column_name) if (filter_output->result_name == filter_column_name)
{ {
filter_output = &split.first->addAlias(*filter_output, split_filter_name); filter_output = &split.first.addAlias(*filter_output, split_filter_name);
break; break;
} }
} }

View File

@ -146,16 +146,16 @@ bool allOutputsDependsOnlyOnAllowedNodes(
/// 3. We match partition key actions with group by key actions to find col1', ..., coln' in partition key actions. /// 3. We match partition key actions with group by key actions to find col1', ..., coln' in partition key actions.
/// 4. We check that partition key is indeed a deterministic function of col1', ..., coln'. /// 4. We check that partition key is indeed a deterministic function of col1', ..., coln'.
bool isPartitionKeySuitsGroupByKey( bool isPartitionKeySuitsGroupByKey(
const ReadFromMergeTree & reading, const ActionsDAGPtr & group_by_actions, const AggregatingStep & aggregating) const ReadFromMergeTree & reading, const ActionsDAG & group_by_actions, const AggregatingStep & aggregating)
{ {
if (aggregating.isGroupingSets()) if (aggregating.isGroupingSets())
return false; return false;
if (group_by_actions->hasArrayJoin() || group_by_actions->hasStatefulFunctions() || group_by_actions->hasNonDeterministic()) if (group_by_actions.hasArrayJoin() || group_by_actions.hasStatefulFunctions() || group_by_actions.hasNonDeterministic())
return false; return false;
/// We are interested only in calculations required to obtain group by keys (and not aggregate function arguments for example). /// We are interested only in calculations required to obtain group by keys (and not aggregate function arguments for example).
auto key_nodes = group_by_actions->findInOutpus(aggregating.getParams().keys); auto key_nodes = group_by_actions.findInOutpus(aggregating.getParams().keys);
auto group_by_key_actions = ActionsDAG::cloneSubDAG(key_nodes, /*remove_aliases=*/ true); auto group_by_key_actions = ActionsDAG::cloneSubDAG(key_nodes, /*remove_aliases=*/ true);
const auto & gb_key_required_columns = group_by_key_actions->getRequiredColumnsNames(); const auto & gb_key_required_columns = group_by_key_actions->getRequiredColumnsNames();

View File

@ -943,7 +943,7 @@ SplitPartsWithRangesByPrimaryKeyResult splitPartsWithRangesByPrimaryKey(
auto syntax_result = TreeRewriter(context).analyze(filter_function, primary_key.expression->getRequiredColumnsWithTypes()); auto syntax_result = TreeRewriter(context).analyze(filter_function, primary_key.expression->getRequiredColumnsWithTypes());
auto actions = ExpressionAnalyzer(filter_function, syntax_result, context).getActionsDAG(false); auto actions = ExpressionAnalyzer(filter_function, syntax_result, context).getActionsDAG(false);
reorderColumns(*actions, result.merging_pipes[i].getHeader(), filter_function->getColumnName()); reorderColumns(actions, result.merging_pipes[i].getHeader(), filter_function->getColumnName());
ExpressionActionsPtr expression_actions = std::make_shared<ExpressionActions>(std::move(actions)); ExpressionActionsPtr expression_actions = std::make_shared<ExpressionActions>(std::move(actions));
auto description = fmt::format( auto description = fmt::format(
"filter values in ({}, {}]", i ? ::toString(borders[i - 1]) : "-inf", i < borders.size() ? ::toString(borders[i]) : "+inf"); "filter values in ({}, {}]", i ? ::toString(borders[i - 1]) : "-inf", i < borders.size() ? ::toString(borders[i]) : "+inf");

View File

@ -799,7 +799,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_
info.use_uncompressed_cache); info.use_uncompressed_cache);
}; };
auto sorting_expr = std::make_shared<ExpressionActions>(ActionsDAG::clone(&metadata_for_reading->getSortingKey().expression->getActionsDAG())); auto sorting_expr = std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(&metadata_for_reading->getSortingKey().expression->getActionsDAG())));
SplitPartsWithRangesByPrimaryKeyResult split_ranges_result = splitPartsWithRangesByPrimaryKey( SplitPartsWithRangesByPrimaryKeyResult split_ranges_result = splitPartsWithRangesByPrimaryKey(
metadata_for_reading->getPrimaryKey(), metadata_for_reading->getPrimaryKey(),
@ -848,16 +848,16 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_
info.use_uncompressed_cache); info.use_uncompressed_cache);
} }
static ActionsDAGPtr createProjection(const Block & header) static ActionsDAG createProjection(const Block & header)
{ {
return std::make_unique<ActionsDAG>(header.getNamesAndTypesList()); return ActionsDAG(header.getNamesAndTypesList());
} }
Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder(
RangesInDataParts && parts_with_ranges, RangesInDataParts && parts_with_ranges,
size_t num_streams, size_t num_streams,
const Names & column_names, const Names & column_names,
ActionsDAGPtr & out_projection, std::optional<ActionsDAG> & out_projection,
const InputOrderInfoPtr & input_order_info) const InputOrderInfoPtr & input_order_info)
{ {
const auto & settings = context->getSettingsRef(); const auto & settings = context->getSettingsRef();
@ -1171,7 +1171,7 @@ bool ReadFromMergeTree::doNotMergePartsAcrossPartitionsFinal() const
} }
Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
RangesInDataParts && parts_with_ranges, size_t num_streams, const Names & origin_column_names, const Names & column_names, ActionsDAGPtr & out_projection) RangesInDataParts && parts_with_ranges, size_t num_streams, const Names & origin_column_names, const Names & column_names, std::optional<ActionsDAG> & out_projection)
{ {
const auto & settings = context->getSettingsRef(); const auto & settings = context->getSettingsRef();
const auto & data_settings = data.getSettings(); const auto & data_settings = data.getSettings();
@ -1212,7 +1212,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
/// we will store lonely parts with level > 0 to use parallel select on them. /// we will store lonely parts with level > 0 to use parallel select on them.
RangesInDataParts non_intersecting_parts_by_primary_key; RangesInDataParts non_intersecting_parts_by_primary_key;
auto sorting_expr = std::make_shared<ExpressionActions>(ActionsDAG::clone(&metadata_for_reading->getSortingKey().expression->getActionsDAG())); auto sorting_expr = std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(&metadata_for_reading->getSortingKey().expression->getActionsDAG())));
if (prewhere_info) if (prewhere_info)
{ {
@ -1333,7 +1333,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
if (!merging_pipes.empty() && !no_merging_pipes.empty()) if (!merging_pipes.empty() && !no_merging_pipes.empty())
{ {
out_projection = nullptr; /// We do projection here out_projection = {}; /// We do projection here
Pipes pipes; Pipes pipes;
pipes.resize(2); pipes.resize(2);
pipes[0] = Pipe::unitePipes(std::move(merging_pipes)); pipes[0] = Pipe::unitePipes(std::move(merging_pipes));
@ -1519,7 +1519,8 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes)
/// (1) SourceStepWithFilter::filter_nodes, (2) query_info.filter_actions_dag. Make sure there are consistent. /// (1) SourceStepWithFilter::filter_nodes, (2) query_info.filter_actions_dag. Make sure there are consistent.
/// TODO: Get rid of filter_actions_dag in query_info after we move analysis of /// TODO: Get rid of filter_actions_dag in query_info after we move analysis of
/// parallel replicas and unused shards into optimization, similar to projection analysis. /// parallel replicas and unused shards into optimization, similar to projection analysis.
query_info.filter_actions_dag = std::move(filter_actions_dag); if (filter_actions_dag)
query_info.filter_actions_dag = std::make_shared<const ActionsDAG>(std::move(*filter_actions_dag));
buildIndexes( buildIndexes(
indexes, indexes,
@ -1833,7 +1834,7 @@ bool ReadFromMergeTree::isQueryWithSampling() const
} }
Pipe ReadFromMergeTree::spreadMarkRanges( Pipe ReadFromMergeTree::spreadMarkRanges(
RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, ActionsDAGPtr & result_projection) RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, std::optional<ActionsDAG> & result_projection)
{ {
const bool final = isQueryWithFinal(); const bool final = isQueryWithFinal();
Names column_names_to_read = result.column_names_to_read; Names column_names_to_read = result.column_names_to_read;
@ -1894,7 +1895,7 @@ Pipe ReadFromMergeTree::spreadMarkRanges(
} }
} }
Pipe ReadFromMergeTree::groupStreamsByPartition(AnalysisResult & result, ActionsDAGPtr & result_projection) Pipe ReadFromMergeTree::groupStreamsByPartition(AnalysisResult & result, std::optional<ActionsDAG> & result_projection)
{ {
auto && parts_with_ranges = std::move(result.parts_with_ranges); auto && parts_with_ranges = std::move(result.parts_with_ranges);
@ -1983,7 +1984,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons
/// Projection, that needed to drop columns, which have appeared by execution /// Projection, that needed to drop columns, which have appeared by execution
/// of some extra expressions, and to allow execute the same expressions later. /// of some extra expressions, and to allow execute the same expressions later.
/// NOTE: It may lead to double computation of expressions. /// NOTE: It may lead to double computation of expressions.
ActionsDAGPtr result_projection; std::optional<ActionsDAG> result_projection;
Pipe pipe = output_each_partition_through_separate_port Pipe pipe = output_each_partition_through_separate_port
? groupStreamsByPartition(result, result_projection) ? groupStreamsByPartition(result, result_projection)
@ -2000,7 +2001,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons
if (result.sampling.use_sampling) if (result.sampling.use_sampling)
{ {
auto sampling_actions = std::make_shared<ExpressionActions>(ActionsDAG::clone(result.sampling.filter_expression.get())); auto sampling_actions = std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(result.sampling.filter_expression.get())));
pipe.addSimpleTransform([&](const Block & header) pipe.addSimpleTransform([&](const Block & header)
{ {
return std::make_shared<FilterTransform>( return std::make_shared<FilterTransform>(
@ -2013,12 +2014,12 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons
Block cur_header = pipe.getHeader(); Block cur_header = pipe.getHeader();
auto append_actions = [&result_projection](ActionsDAGPtr actions) auto append_actions = [&result_projection](ActionsDAG actions)
{ {
if (!result_projection) if (!result_projection)
result_projection = std::move(actions); result_projection = std::move(actions);
else else
result_projection = ActionsDAG::merge(std::move(*result_projection), std::move(*actions)); result_projection = ActionsDAG::merge(std::move(*result_projection), std::move(actions));
}; };
if (result_projection) if (result_projection)
@ -2038,7 +2039,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons
if (result_projection) if (result_projection)
{ {
auto projection_actions = std::make_shared<ExpressionActions>(ActionsDAG::clone(result_projection)); auto projection_actions = std::make_shared<ExpressionActions>(std::move(*result_projection));
pipe.addSimpleTransform([&](const Block & header) pipe.addSimpleTransform([&](const Block & header)
{ {
return std::make_shared<ExpressionTransform>(header, projection_actions); return std::make_shared<ExpressionTransform>(header, projection_actions);
@ -2133,7 +2134,7 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const
format_settings.out << " (removed)"; format_settings.out << " (removed)";
format_settings.out << '\n'; format_settings.out << '\n';
auto expression = std::make_shared<ExpressionActions>(ActionsDAG::clone(prewhere_info->prewhere_actions)); auto expression = std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions)));
expression->describeActions(format_settings.out, prefix); expression->describeActions(format_settings.out, prefix);
} }
@ -2142,7 +2143,7 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const
format_settings.out << prefix << "Row level filter" << '\n'; format_settings.out << prefix << "Row level filter" << '\n';
format_settings.out << prefix << "Row level filter column: " << prewhere_info->row_level_column_name << '\n'; format_settings.out << prefix << "Row level filter column: " << prewhere_info->row_level_column_name << '\n';
auto expression = std::make_shared<ExpressionActions>(ActionsDAG::clone(prewhere_info->row_level_filter)); auto expression = std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter)));
expression->describeActions(format_settings.out, prefix); expression->describeActions(format_settings.out, prefix);
} }
} }
@ -2168,7 +2169,7 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const
std::unique_ptr<JSONBuilder::JSONMap> prewhere_filter_map = std::make_unique<JSONBuilder::JSONMap>(); std::unique_ptr<JSONBuilder::JSONMap> prewhere_filter_map = std::make_unique<JSONBuilder::JSONMap>();
prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name); prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name);
prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column); prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column);
auto expression = std::make_shared<ExpressionActions>(ActionsDAG::clone(prewhere_info->prewhere_actions)); auto expression = std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions)));
prewhere_filter_map->add("Prewhere filter expression", expression->toTree()); prewhere_filter_map->add("Prewhere filter expression", expression->toTree());
prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map)); prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map));
@ -2178,7 +2179,7 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const
{ {
std::unique_ptr<JSONBuilder::JSONMap> row_level_filter_map = std::make_unique<JSONBuilder::JSONMap>(); std::unique_ptr<JSONBuilder::JSONMap> row_level_filter_map = std::make_unique<JSONBuilder::JSONMap>();
row_level_filter_map->add("Row level filter column", prewhere_info->row_level_column_name); row_level_filter_map->add("Row level filter column", prewhere_info->row_level_column_name);
auto expression = std::make_shared<ExpressionActions>(ActionsDAG::clone(prewhere_info->row_level_filter)); auto expression = std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter)));
row_level_filter_map->add("Row level filter expression", expression->toTree()); row_level_filter_map->add("Row level filter expression", expression->toTree());
prewhere_info_map->add("Row level filter", std::move(row_level_filter_map)); prewhere_info_map->add("Row level filter", std::move(row_level_filter_map));

View File

@ -243,9 +243,9 @@ private:
Pipe readFromPoolParallelReplicas(RangesInDataParts parts_with_range, Names required_columns, PoolSettings pool_settings); Pipe readFromPoolParallelReplicas(RangesInDataParts parts_with_range, Names required_columns, PoolSettings pool_settings);
Pipe readInOrder(RangesInDataParts parts_with_ranges, Names required_columns, PoolSettings pool_settings, ReadType read_type, UInt64 limit); Pipe readInOrder(RangesInDataParts parts_with_ranges, Names required_columns, PoolSettings pool_settings, ReadType read_type, UInt64 limit);
Pipe spreadMarkRanges(RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, ActionsDAGPtr & result_projection); Pipe spreadMarkRanges(RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, std::optional<ActionsDAG> & result_projection);
Pipe groupStreamsByPartition(AnalysisResult & result, ActionsDAGPtr & result_projection); Pipe groupStreamsByPartition(AnalysisResult & result, std::optional<ActionsDAG> & result_projection);
Pipe spreadMarkRangesAmongStreams(RangesInDataParts && parts_with_ranges, size_t num_streams, const Names & column_names); Pipe spreadMarkRangesAmongStreams(RangesInDataParts && parts_with_ranges, size_t num_streams, const Names & column_names);
@ -253,13 +253,13 @@ private:
RangesInDataParts && parts_with_ranges, RangesInDataParts && parts_with_ranges,
size_t num_streams, size_t num_streams,
const Names & column_names, const Names & column_names,
ActionsDAGPtr & out_projection, std::optional<ActionsDAG> & out_projection,
const InputOrderInfoPtr & input_order_info); const InputOrderInfoPtr & input_order_info);
bool doNotMergePartsAcrossPartitionsFinal() const; bool doNotMergePartsAcrossPartitionsFinal() const;
Pipe spreadMarkRangesAmongStreamsFinal( Pipe spreadMarkRangesAmongStreamsFinal(
RangesInDataParts && parts, size_t num_streams, const Names & origin_column_names, const Names & column_names, ActionsDAGPtr & out_projection); RangesInDataParts && parts, size_t num_streams, const Names & origin_column_names, const Names & column_names, std::optional<ActionsDAG> & out_projection);
ReadFromMergeTree::AnalysisResult getAnalysisResult() const; ReadFromMergeTree::AnalysisResult getAnalysisResult() const;

View File

@ -441,7 +441,7 @@ Pipe ReadFromSystemNumbersStep::makePipe()
chassert(numbers_storage.step != UInt64{0}); chassert(numbers_storage.step != UInt64{0});
/// Build rpn of query filters /// Build rpn of query filters
KeyCondition condition(filter_actions_dag.get(), context, column_names, key_expression); KeyCondition condition(filter_actions_dag ? &*filter_actions_dag : nullptr, context, column_names, key_expression);
if (condition.extractPlainRanges(ranges)) if (condition.extractPlainRanges(ranges))
{ {

View File

@ -110,7 +110,7 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con
format_settings.out << " (removed)"; format_settings.out << " (removed)";
format_settings.out << '\n'; format_settings.out << '\n';
auto expression = std::make_shared<ExpressionActions>(ActionsDAG::clone(prewhere_info->prewhere_actions)); auto expression = std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions)));
expression->describeActions(format_settings.out, prefix); expression->describeActions(format_settings.out, prefix);
} }
@ -119,7 +119,7 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con
format_settings.out << prefix << "Row level filter" << '\n'; format_settings.out << prefix << "Row level filter" << '\n';
format_settings.out << prefix << "Row level filter column: " << prewhere_info->row_level_column_name << '\n'; format_settings.out << prefix << "Row level filter column: " << prewhere_info->row_level_column_name << '\n';
auto expression = std::make_shared<ExpressionActions>(ActionsDAG::clone(prewhere_info->row_level_filter)); auto expression = std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter)));
expression->describeActions(format_settings.out, prefix); expression->describeActions(format_settings.out, prefix);
} }
} }
@ -137,7 +137,7 @@ void SourceStepWithFilter::describeActions(JSONBuilder::JSONMap & map) const
std::unique_ptr<JSONBuilder::JSONMap> prewhere_filter_map = std::make_unique<JSONBuilder::JSONMap>(); std::unique_ptr<JSONBuilder::JSONMap> prewhere_filter_map = std::make_unique<JSONBuilder::JSONMap>();
prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name); prewhere_filter_map->add("Prewhere filter column", prewhere_info->prewhere_column_name);
prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column); prewhere_filter_map->add("Prewhere filter remove filter column", prewhere_info->remove_prewhere_column);
auto expression = std::make_shared<ExpressionActions>(ActionsDAG::clone(prewhere_info->prewhere_actions)); auto expression = std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions)));
prewhere_filter_map->add("Prewhere filter expression", expression->toTree()); prewhere_filter_map->add("Prewhere filter expression", expression->toTree());
prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map)); prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map));
@ -147,7 +147,7 @@ void SourceStepWithFilter::describeActions(JSONBuilder::JSONMap & map) const
{ {
std::unique_ptr<JSONBuilder::JSONMap> row_level_filter_map = std::make_unique<JSONBuilder::JSONMap>(); std::unique_ptr<JSONBuilder::JSONMap> row_level_filter_map = std::make_unique<JSONBuilder::JSONMap>();
row_level_filter_map->add("Row level filter column", prewhere_info->row_level_column_name); row_level_filter_map->add("Row level filter column", prewhere_info->row_level_column_name);
auto expression = std::make_shared<ExpressionActions>(ActionsDAG::clone(prewhere_info->row_level_filter)); auto expression = std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter)));
row_level_filter_map->add("Row level filter expression", expression->toTree()); row_level_filter_map->add("Row level filter expression", expression->toTree());
prewhere_info_map->add("Row level filter", std::move(row_level_filter_map)); prewhere_info_map->add("Row level filter", std::move(row_level_filter_map));

View File

@ -33,8 +33,8 @@ public:
{ {
} }
const ActionsDAGPtr & getFilterActionsDAG() const { return filter_actions_dag; } const std::optional<ActionsDAG> & getFilterActionsDAG() const { return filter_actions_dag; }
ActionsDAGPtr detachFilterActionsDAG() { return std::move(filter_actions_dag); } std::optional<ActionsDAG> detachFilterActionsDAG() { return std::move(filter_actions_dag); }
const SelectQueryInfo & getQueryInfo() const { return query_info; } const SelectQueryInfo & getQueryInfo() const { return query_info; }
const PrewhereInfoPtr & getPrewhereInfo() const { return prewhere_info; } const PrewhereInfoPtr & getPrewhereInfo() const { return prewhere_info; }
@ -81,7 +81,7 @@ protected:
ContextPtr context; ContextPtr context;
std::optional<size_t> limit; std::optional<size_t> limit;
ActionsDAGPtr filter_actions_dag; std::optional<ActionsDAG> filter_actions_dag;
private: private:
/// Will be cleared after applyFilters() is called. /// Will be cleared after applyFilters() is called.

View File

@ -28,7 +28,7 @@ TotalsHavingStep::TotalsHavingStep(
const DataStream & input_stream_, const DataStream & input_stream_,
const AggregateDescriptions & aggregates_, const AggregateDescriptions & aggregates_,
bool overflow_row_, bool overflow_row_,
const ActionsDAGPtr & actions_dag_, std::optional<ActionsDAG> actions_dag_,
const std::string & filter_column_, const std::string & filter_column_,
bool remove_filter_, bool remove_filter_,
TotalsMode totals_mode_, TotalsMode totals_mode_,
@ -38,7 +38,7 @@ TotalsHavingStep::TotalsHavingStep(
input_stream_, input_stream_,
TotalsHavingTransform::transformHeader( TotalsHavingTransform::transformHeader(
input_stream_.header, input_stream_.header,
actions_dag_.get(), actions_dag_ ? &*actions_dag_ : nullptr,
filter_column_, filter_column_,
remove_filter_, remove_filter_,
final_, final_,
@ -46,7 +46,7 @@ TotalsHavingStep::TotalsHavingStep(
getTraits(!filter_column_.empty())) getTraits(!filter_column_.empty()))
, aggregates(aggregates_) , aggregates(aggregates_)
, overflow_row(overflow_row_) , overflow_row(overflow_row_)
, actions_dag(ActionsDAG::clone(actions_dag_)) , actions_dag(std::move(actions_dag_))
, filter_column_name(filter_column_) , filter_column_name(filter_column_)
, remove_filter(remove_filter_) , remove_filter(remove_filter_)
, totals_mode(totals_mode_) , totals_mode(totals_mode_)
@ -57,7 +57,7 @@ TotalsHavingStep::TotalsHavingStep(
void TotalsHavingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) void TotalsHavingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings)
{ {
auto expression_actions = actions_dag ? std::make_shared<ExpressionActions>(ActionsDAG::clone(actions_dag), settings.getActionsSettings()) : nullptr; auto expression_actions = actions_dag ? std::make_shared<ExpressionActions>(std::move(*actions_dag), settings.getActionsSettings()) : nullptr;
auto totals_having = std::make_shared<TotalsHavingTransform>( auto totals_having = std::make_shared<TotalsHavingTransform>(
pipeline.getHeader(), pipeline.getHeader(),
@ -100,7 +100,7 @@ void TotalsHavingStep::describeActions(FormatSettings & settings) const
if (actions_dag) if (actions_dag)
{ {
bool first = true; bool first = true;
auto expression = std::make_shared<ExpressionActions>(ActionsDAG::clone(actions_dag)); auto expression = std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(getActions())));
for (const auto & action : expression->getActions()) for (const auto & action : expression->getActions())
{ {
settings.out << prefix << (first ? "Actions: " settings.out << prefix << (first ? "Actions: "
@ -117,7 +117,7 @@ void TotalsHavingStep::describeActions(JSONBuilder::JSONMap & map) const
if (actions_dag) if (actions_dag)
{ {
map.add("Filter column", filter_column_name); map.add("Filter column", filter_column_name);
auto expression = std::make_shared<ExpressionActions>(ActionsDAG::clone(actions_dag)); auto expression = std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(getActions())));
map.add("Expression", expression->toTree()); map.add("Expression", expression->toTree());
} }
} }
@ -128,7 +128,7 @@ void TotalsHavingStep::updateOutputStream()
input_streams.front(), input_streams.front(),
TotalsHavingTransform::transformHeader( TotalsHavingTransform::transformHeader(
input_streams.front().header, input_streams.front().header,
actions_dag.get(), getActions(),
filter_column_name, filter_column_name,
remove_filter, remove_filter,
final, final,

View File

@ -1,6 +1,7 @@
#pragma once #pragma once
#include <Processors/QueryPlan/ITransformingStep.h> #include <Processors/QueryPlan/ITransformingStep.h>
#include <Processors/Transforms/finalizeChunk.h> #include <Processors/Transforms/finalizeChunk.h>
#include <Interpreters/ActionsDAG.h>
namespace DB namespace DB
{ {
@ -18,7 +19,7 @@ public:
const DataStream & input_stream_, const DataStream & input_stream_,
const AggregateDescriptions & aggregates_, const AggregateDescriptions & aggregates_,
bool overflow_row_, bool overflow_row_,
const ActionsDAGPtr & actions_dag_, std::optional<ActionsDAG> actions_dag_,
const std::string & filter_column_, const std::string & filter_column_,
bool remove_filter_, bool remove_filter_,
TotalsMode totals_mode_, TotalsMode totals_mode_,
@ -32,7 +33,7 @@ public:
void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(JSONBuilder::JSONMap & map) const override;
void describeActions(FormatSettings & settings) const override; void describeActions(FormatSettings & settings) const override;
const ActionsDAGPtr & getActions() const { return actions_dag; } const ActionsDAG * getActions() const { return actions_dag ? &*actions_dag : nullptr; }
private: private:
void updateOutputStream() override; void updateOutputStream() override;
@ -40,7 +41,7 @@ private:
const AggregateDescriptions aggregates; const AggregateDescriptions aggregates;
bool overflow_row; bool overflow_row;
ActionsDAGPtr actions_dag; std::optional<ActionsDAG> actions_dag;
String filter_column_name; String filter_column_name;
bool remove_filter; bool remove_filter;
TotalsMode totals_mode; TotalsMode totals_mode;

View File

@ -16,13 +16,13 @@ protected:
/// Represents pushed down filters in source /// Represents pushed down filters in source
std::shared_ptr<const KeyCondition> key_condition; std::shared_ptr<const KeyCondition> key_condition;
void setKeyConditionImpl(const ActionsDAG * filter_actions_dag, ContextPtr context, const Block & keys) void setKeyConditionImpl(const std::optional<ActionsDAG> & filter_actions_dag, ContextPtr context, const Block & keys)
{ {
key_condition = std::make_shared<const KeyCondition>( key_condition = std::make_shared<const KeyCondition>(
filter_actions_dag, filter_actions_dag ? &*filter_actions_dag : nullptr,
context, context,
keys.getNames(), keys.getNames(),
std::make_shared<ExpressionActions>(std::make_unique<ActionsDAG>(keys.getColumnsWithTypeAndName()))); std::make_shared<ExpressionActions>(ActionsDAG(keys.getColumnsWithTypeAndName())));
} }
public: public:
@ -33,6 +33,6 @@ public:
virtual void setKeyCondition(const std::shared_ptr<const KeyCondition> & key_condition_) { key_condition = key_condition_; } virtual void setKeyCondition(const std::shared_ptr<const KeyCondition> & key_condition_) { key_condition = key_condition_; }
/// Set key_condition created by filter_actions_dag and context. /// Set key_condition created by filter_actions_dag and context.
virtual void setKeyCondition(const ActionsDAGPtr & /*filter_actions_dag*/, ContextPtr /*context*/) { } virtual void setKeyCondition(const std::optional<ActionsDAG> & /*filter_actions_dag*/, ContextPtr /*context*/) { }
}; };
} }

View File

@ -178,7 +178,7 @@ void AddingDefaultsTransform::transform(Chunk & chunk)
auto dag = evaluateMissingDefaults(evaluate_block, header.getNamesAndTypesList(), columns, context, false); auto dag = evaluateMissingDefaults(evaluate_block, header.getNamesAndTypesList(), columns, context, false);
if (dag) if (dag)
{ {
auto actions = std::make_shared<ExpressionActions>(std::move(dag), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes), true); auto actions = std::make_shared<ExpressionActions>(std::move(*dag), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes), true);
actions->execute(evaluate_block); actions->execute(evaluate_block);
} }

View File

@ -203,7 +203,7 @@ FillingTransform::FillingTransform(
, use_with_fill_by_sorting_prefix(use_with_fill_by_sorting_prefix_) , use_with_fill_by_sorting_prefix(use_with_fill_by_sorting_prefix_)
{ {
if (interpolate_description) if (interpolate_description)
interpolate_actions = std::make_shared<ExpressionActions>(ActionsDAG::clone(interpolate_description->actions)); interpolate_actions = std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(&interpolate_description->actions)));
std::vector<bool> is_fill_column(header_.columns()); std::vector<bool> is_fill_column(header_.columns());
for (size_t i = 0, size = fill_description.size(); i < size; ++i) for (size_t i = 0, size = fill_description.size(); i < size; ++i)

View File

@ -516,7 +516,7 @@ void StorageHive::initMinMaxIndexExpression()
partition_names = partition_name_types.getNames(); partition_names = partition_name_types.getNames();
partition_types = partition_name_types.getTypes(); partition_types = partition_name_types.getTypes();
partition_minmax_idx_expr = std::make_shared<ExpressionActions>( partition_minmax_idx_expr = std::make_shared<ExpressionActions>(
std::make_unique<ActionsDAG>(partition_name_types), ExpressionActionsSettings::fromContext(getContext())); ActionsDAG(partition_name_types), ExpressionActionsSettings::fromContext(getContext()));
} }
NamesAndTypesList all_name_types = metadata_snapshot->getColumns().getAllPhysical(); NamesAndTypesList all_name_types = metadata_snapshot->getColumns().getAllPhysical();
@ -526,7 +526,7 @@ void StorageHive::initMinMaxIndexExpression()
hivefile_name_types.push_back(column); hivefile_name_types.push_back(column);
} }
hivefile_minmax_idx_expr = std::make_shared<ExpressionActions>( hivefile_minmax_idx_expr = std::make_shared<ExpressionActions>(
std::make_unique<ActionsDAG>(hivefile_name_types), ExpressionActionsSettings::fromContext(getContext())); ActionsDAG(hivefile_name_types), ExpressionActionsSettings::fromContext(getContext()));
} }
ASTPtr StorageHive::extractKeyExpressionList(const ASTPtr & node) ASTPtr StorageHive::extractKeyExpressionList(const ASTPtr & node)
@ -583,7 +583,7 @@ static HiveFilePtr createHiveFile(
HiveFiles StorageHive::collectHiveFilesFromPartition( HiveFiles StorageHive::collectHiveFilesFromPartition(
const Apache::Hadoop::Hive::Partition & partition, const Apache::Hadoop::Hive::Partition & partition,
const ActionsDAGPtr & filter_actions_dag, const ActionsDAG * filter_actions_dag,
const HiveTableMetadataPtr & hive_table_metadata, const HiveTableMetadataPtr & hive_table_metadata,
const HDFSFSPtr & fs, const HDFSFSPtr & fs,
const ContextPtr & context_, const ContextPtr & context_,
@ -647,7 +647,7 @@ HiveFiles StorageHive::collectHiveFilesFromPartition(
for (size_t i = 0; i < partition_names.size(); ++i) for (size_t i = 0; i < partition_names.size(); ++i)
ranges.emplace_back(fields[i]); ranges.emplace_back(fields[i]);
const KeyCondition partition_key_condition(filter_actions_dag.get(), getContext(), partition_names, partition_minmax_idx_expr); const KeyCondition partition_key_condition(filter_actions_dag, getContext(), partition_names, partition_minmax_idx_expr);
if (!partition_key_condition.checkInHyperrectangle(ranges, partition_types).can_be_true) if (!partition_key_condition.checkInHyperrectangle(ranges, partition_types).can_be_true)
return {}; return {};
} }
@ -681,7 +681,7 @@ StorageHive::listDirectory(const String & path, const HiveTableMetadataPtr & hiv
HiveFilePtr StorageHive::getHiveFileIfNeeded( HiveFilePtr StorageHive::getHiveFileIfNeeded(
const FileInfo & file_info, const FileInfo & file_info,
const FieldVector & fields, const FieldVector & fields,
const ActionsDAGPtr & filter_actions_dag, const ActionsDAG * filter_actions_dag,
const HiveTableMetadataPtr & hive_table_metadata, const HiveTableMetadataPtr & hive_table_metadata,
const ContextPtr & context_, const ContextPtr & context_,
PruneLevel prune_level) const PruneLevel prune_level) const
@ -715,7 +715,7 @@ HiveFilePtr StorageHive::getHiveFileIfNeeded(
if (prune_level >= PruneLevel::File) if (prune_level >= PruneLevel::File)
{ {
const KeyCondition hivefile_key_condition(filter_actions_dag.get(), getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); const KeyCondition hivefile_key_condition(filter_actions_dag, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr);
if (hive_file->useFileMinMaxIndex()) if (hive_file->useFileMinMaxIndex())
{ {
/// Load file level minmax index and apply /// Load file level minmax index and apply
@ -828,7 +828,7 @@ void ReadFromHive::createFiles()
if (hive_files) if (hive_files)
return; return;
hive_files = storage->collectHiveFiles(num_streams, filter_actions_dag, hive_table_metadata, fs, context); hive_files = storage->collectHiveFiles(num_streams, filter_actions_dag ? &*filter_actions_dag : nullptr, hive_table_metadata, fs, context);
LOG_INFO(log, "Collect {} hive files to read", hive_files->size()); LOG_INFO(log, "Collect {} hive files to read", hive_files->size());
} }
@ -950,7 +950,7 @@ void ReadFromHive::initializePipeline(QueryPipelineBuilder & pipeline, const Bui
HiveFiles StorageHive::collectHiveFiles( HiveFiles StorageHive::collectHiveFiles(
size_t max_threads, size_t max_threads,
const ActionsDAGPtr & filter_actions_dag, const ActionsDAG * filter_actions_dag,
const HiveTableMetadataPtr & hive_table_metadata, const HiveTableMetadataPtr & hive_table_metadata,
const HDFSFSPtr & fs, const HDFSFSPtr & fs,
const ContextPtr & context_, const ContextPtr & context_,
@ -1023,12 +1023,12 @@ SinkToStoragePtr StorageHive::write(const ASTPtr & /*query*/, const StorageMetad
std::optional<UInt64> StorageHive::totalRows(const Settings & settings) const std::optional<UInt64> StorageHive::totalRows(const Settings & settings) const
{ {
/// query_info is not used when prune_level == PruneLevel::None /// query_info is not used when prune_level == PruneLevel::None
return totalRowsImpl(settings, nullptr, getContext(), PruneLevel::None); return totalRowsImpl(settings, {}, getContext(), PruneLevel::None);
} }
std::optional<UInt64> StorageHive::totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) const std::optional<UInt64> StorageHive::totalRowsByPartitionPredicate(const ActionsDAG & filter_actions_dag, ContextPtr context_) const
{ {
return totalRowsImpl(context_->getSettingsRef(), filter_actions_dag, context_, PruneLevel::Partition); return totalRowsImpl(context_->getSettingsRef(), &filter_actions_dag, context_, PruneLevel::Partition);
} }
void StorageHive::checkAlterIsPossible(const AlterCommands & commands, ContextPtr /*local_context*/) const void StorageHive::checkAlterIsPossible(const AlterCommands & commands, ContextPtr /*local_context*/) const
@ -1043,7 +1043,7 @@ void StorageHive::checkAlterIsPossible(const AlterCommands & commands, ContextPt
} }
std::optional<UInt64> std::optional<UInt64>
StorageHive::totalRowsImpl(const Settings & settings, const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, PruneLevel prune_level) const StorageHive::totalRowsImpl(const Settings & settings, const ActionsDAG * filter_actions_dag, ContextPtr context_, PruneLevel prune_level) const
{ {
/// Row-based format like Text doesn't support totalRowsByPartitionPredicate /// Row-based format like Text doesn't support totalRowsByPartitionPredicate
if (!supportsSubsetOfColumns()) if (!supportsSubsetOfColumns())

View File

@ -57,7 +57,7 @@ public:
bool supportsSubsetOfColumns() const; bool supportsSubsetOfColumns() const;
std::optional<UInt64> totalRows(const Settings & settings) const override; std::optional<UInt64> totalRows(const Settings & settings) const override;
std::optional<UInt64> totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) const override; std::optional<UInt64> totalRowsByPartitionPredicate(const ActionsDAG & filter_actions_dag, ContextPtr context_) const override;
void checkAlterIsPossible(const AlterCommands & commands, ContextPtr local_context) const override; void checkAlterIsPossible(const AlterCommands & commands, ContextPtr local_context) const override;
protected: protected:
@ -90,7 +90,7 @@ private:
HiveFiles collectHiveFiles( HiveFiles collectHiveFiles(
size_t max_threads, size_t max_threads,
const ActionsDAGPtr & filter_actions_dag, const ActionsDAG * filter_actions_dag,
const HiveTableMetadataPtr & hive_table_metadata, const HiveTableMetadataPtr & hive_table_metadata,
const HDFSFSPtr & fs, const HDFSFSPtr & fs,
const ContextPtr & context_, const ContextPtr & context_,
@ -98,7 +98,7 @@ private:
HiveFiles collectHiveFilesFromPartition( HiveFiles collectHiveFilesFromPartition(
const Apache::Hadoop::Hive::Partition & partition, const Apache::Hadoop::Hive::Partition & partition,
const ActionsDAGPtr & filter_actions_dag, const ActionsDAG * filter_actions_dag,
const HiveTableMetadataPtr & hive_table_metadata, const HiveTableMetadataPtr & hive_table_metadata,
const HDFSFSPtr & fs, const HDFSFSPtr & fs,
const ContextPtr & context_, const ContextPtr & context_,
@ -107,7 +107,7 @@ private:
HiveFilePtr getHiveFileIfNeeded( HiveFilePtr getHiveFileIfNeeded(
const FileInfo & file_info, const FileInfo & file_info,
const FieldVector & fields, const FieldVector & fields,
const ActionsDAGPtr & filter_actions_dag, const ActionsDAG * filter_actions_dag,
const HiveTableMetadataPtr & hive_table_metadata, const HiveTableMetadataPtr & hive_table_metadata,
const ContextPtr & context_, const ContextPtr & context_,
PruneLevel prune_level = PruneLevel::Max) const; PruneLevel prune_level = PruneLevel::Max) const;
@ -115,7 +115,7 @@ private:
void lazyInitialize(); void lazyInitialize();
std::optional<UInt64> std::optional<UInt64>
totalRowsImpl(const Settings & settings, const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, PruneLevel prune_level) const; totalRowsImpl(const Settings & settings, const ActionsDAG * filter_actions_dag, ContextPtr context_, PruneLevel prune_level) const;
String hive_metastore_url; String hive_metastore_url;

View File

@ -236,7 +236,7 @@ StorageID IStorage::getStorageID() const
return storage_id; return storage_id;
} }
ConditionSelectivityEstimator IStorage::getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAGPtr &, ContextPtr) const ConditionSelectivityEstimator IStorage::getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAG *, ContextPtr) const
{ {
return {}; return {};
} }

View File

@ -135,7 +135,7 @@ public:
/// Returns true if the storage supports queries with the PREWHERE section. /// Returns true if the storage supports queries with the PREWHERE section.
virtual bool supportsPrewhere() const { return false; } virtual bool supportsPrewhere() const { return false; }
virtual ConditionSelectivityEstimator getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAGPtr &, ContextPtr) const; virtual ConditionSelectivityEstimator getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAG *, ContextPtr) const;
/// Returns which columns supports PREWHERE, or empty std::nullopt if all columns is supported. /// Returns which columns supports PREWHERE, or empty std::nullopt if all columns is supported.
/// This is needed for engines whose aggregates data from multiple tables, like Merge. /// This is needed for engines whose aggregates data from multiple tables, like Merge.
@ -682,7 +682,7 @@ public:
virtual std::optional<UInt64> totalRows(const Settings &) const { return {}; } virtual std::optional<UInt64> totalRows(const Settings &) const { return {}; }
/// Same as above but also take partition predicate into account. /// Same as above but also take partition predicate into account.
virtual std::optional<UInt64> totalRowsByPartitionPredicate(const ActionsDAGPtr &, ContextPtr) const { return {}; } virtual std::optional<UInt64> totalRowsByPartitionPredicate(const ActionsDAG &, ContextPtr) const { return {}; }
/// If it is possible to quickly determine exact number of bytes for the table on storage: /// If it is possible to quickly determine exact number of bytes for the table on storage:
/// - memory (approximated, resident) /// - memory (approximated, resident)

View File

@ -231,7 +231,7 @@ bool traverseDAGFilter(
} }
std::pair<FieldVectorPtr, bool> getFilterKeys( std::pair<FieldVectorPtr, bool> getFilterKeys(
const String & primary_key, const DataTypePtr & primary_key_type, const ActionsDAGPtr & filter_actions_dag, const ContextPtr & context) const String & primary_key, const DataTypePtr & primary_key_type, const std::optional<ActionsDAG> & filter_actions_dag, const ContextPtr & context)
{ {
if (!filter_actions_dag) if (!filter_actions_dag)
return {{}, true}; return {{}, true};

View File

@ -22,7 +22,7 @@ std::pair<FieldVectorPtr, bool> getFilterKeys(
const std::string & primary_key, const DataTypePtr & primary_key_type, const SelectQueryInfo & query_info, const ContextPtr & context); const std::string & primary_key, const DataTypePtr & primary_key_type, const SelectQueryInfo & query_info, const ContextPtr & context);
std::pair<FieldVectorPtr, bool> getFilterKeys( std::pair<FieldVectorPtr, bool> getFilterKeys(
const String & primary_key, const DataTypePtr & primary_key_type, const ActionsDAGPtr & filter_actions_dag, const ContextPtr & context); const String & primary_key, const DataTypePtr & primary_key_type, const std::optional<ActionsDAG> & filter_actions_dag, const ContextPtr & context);
template <typename K, typename V> template <typename K, typename V>
void fillColumns(const K & key, const V & value, size_t key_pos, const Block & header, MutableColumns & columns) void fillColumns(const K & key, const V & value, size_t key_pos, const Block & header, MutableColumns & columns)

View File

@ -160,7 +160,7 @@ KeyDescription KeyDescription::buildEmptyKey()
{ {
KeyDescription result; KeyDescription result;
result.expression_list_ast = std::make_shared<ASTExpressionList>(); result.expression_list_ast = std::make_shared<ASTExpressionList>();
result.expression = std::make_shared<ExpressionActions>(std::make_unique<ActionsDAG>(), ExpressionActionsSettings{}); result.expression = std::make_shared<ExpressionActions>(ActionsDAG(), ExpressionActionsSettings{});
return result; return result;
} }

View File

@ -163,8 +163,8 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns
if (dag) if (dag)
{ {
dag->addMaterializingOutputActions(); dag->addMaterializingOutputActions();
auto actions = std::make_shared< auto actions = std::make_shared<ExpressionActions>(
ExpressionActions>(std::move(dag), std::move(*dag),
ExpressionActionsSettings::fromSettings(data_part_info_for_read->getContext()->getSettingsRef())); ExpressionActionsSettings::fromSettings(data_part_info_for_read->getContext()->getSettingsRef()));
actions->execute(additional_columns); actions->execute(additional_columns);
} }

View File

@ -628,7 +628,7 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown(
if (const auto * index_hint = typeid_cast<const FunctionIndexHint *>(adaptor->getFunction().get())) if (const auto * index_hint = typeid_cast<const FunctionIndexHint *>(adaptor->getFunction().get()))
{ {
const auto & index_hint_dag = index_hint->getActions(); const auto & index_hint_dag = index_hint->getActions();
children = index_hint_dag->getOutputs(); children = index_hint_dag.getOutputs();
for (auto & arg : children) for (auto & arg : children)
arg = &cloneASTWithInversionPushDown(*arg, inverted_dag, to_inverted, context, need_inversion); arg = &cloneASTWithInversionPushDown(*arg, inverted_dag, to_inverted, context, need_inversion);
@ -729,7 +729,7 @@ Block KeyCondition::getBlockWithConstants(
if (syntax_analyzer_result) if (syntax_analyzer_result)
{ {
auto actions = ExpressionAnalyzer(query, syntax_analyzer_result, context).getConstActionsDAG(); auto actions = ExpressionAnalyzer(query, syntax_analyzer_result, context).getConstActionsDAG();
for (const auto & action_node : actions->getOutputs()) for (const auto & action_node : actions.getOutputs())
{ {
if (action_node->column) if (action_node->column)
result.insert(ColumnWithTypeAndName{action_node->column, action_node->result_type, action_node->result_name}); result.insert(ColumnWithTypeAndName{action_node->column, action_node->result_type, action_node->result_name});

View File

@ -472,7 +472,7 @@ StoragePolicyPtr MergeTreeData::getStoragePolicy() const
} }
ConditionSelectivityEstimator MergeTreeData::getConditionSelectivityEstimatorByPredicate( ConditionSelectivityEstimator MergeTreeData::getConditionSelectivityEstimatorByPredicate(
const StorageSnapshotPtr & storage_snapshot, const ActionsDAGPtr & filter_dag, ContextPtr local_context) const const StorageSnapshotPtr & storage_snapshot, const ActionsDAG * filter_dag, ContextPtr local_context) const
{ {
if (!local_context->getSettings().allow_statistics_optimize) if (!local_context->getSettings().allow_statistics_optimize)
return {}; return {};
@ -487,7 +487,7 @@ ConditionSelectivityEstimator MergeTreeData::getConditionSelectivityEstimatorByP
ASTPtr expression_ast; ASTPtr expression_ast;
ConditionSelectivityEstimator result; ConditionSelectivityEstimator result;
PartitionPruner partition_pruner(storage_snapshot->metadata, filter_dag.get(), local_context); PartitionPruner partition_pruner(storage_snapshot->metadata, filter_dag, local_context);
if (partition_pruner.isUseless()) if (partition_pruner.isUseless())
{ {
@ -746,7 +746,7 @@ ExpressionActionsPtr MergeTreeData::getMinMaxExpr(const KeyDescription & partiti
if (!partition_key.column_names.empty()) if (!partition_key.column_names.empty())
partition_key_columns = partition_key.expression->getRequiredColumnsWithTypes(); partition_key_columns = partition_key.expression->getRequiredColumnsWithTypes();
return std::make_shared<ExpressionActions>(std::make_unique<ActionsDAG>(partition_key_columns), settings); return std::make_shared<ExpressionActions>(ActionsDAG(partition_key_columns), settings);
} }
Names MergeTreeData::getMinMaxColumnsNames(const KeyDescription & partition_key) Names MergeTreeData::getMinMaxColumnsNames(const KeyDescription & partition_key)
@ -1134,7 +1134,7 @@ Block MergeTreeData::getBlockWithVirtualsForFilter(
std::optional<UInt64> MergeTreeData::totalRowsByPartitionPredicateImpl( std::optional<UInt64> MergeTreeData::totalRowsByPartitionPredicateImpl(
const ActionsDAGPtr & filter_actions_dag, ContextPtr local_context, const DataPartsVector & parts) const const ActionsDAG & filter_actions_dag, ContextPtr local_context, const DataPartsVector & parts) const
{ {
if (parts.empty()) if (parts.empty())
return 0; return 0;
@ -1142,7 +1142,7 @@ std::optional<UInt64> MergeTreeData::totalRowsByPartitionPredicateImpl(
auto metadata_snapshot = getInMemoryMetadataPtr(); auto metadata_snapshot = getInMemoryMetadataPtr();
auto virtual_columns_block = getBlockWithVirtualsForFilter(metadata_snapshot, {parts[0]}); auto virtual_columns_block = getBlockWithVirtualsForFilter(metadata_snapshot, {parts[0]});
auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), nullptr); auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag.getOutputs().at(0), nullptr);
if (!filter_dag) if (!filter_dag)
return {}; return {};

View File

@ -426,7 +426,7 @@ public:
bool supportsPrewhere() const override { return true; } bool supportsPrewhere() const override { return true; }
ConditionSelectivityEstimator getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAGPtr &, ContextPtr) const override; ConditionSelectivityEstimator getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAG *, ContextPtr) const override;
bool supportsFinal() const override; bool supportsFinal() const override;
@ -1227,7 +1227,7 @@ protected:
boost::iterator_range<DataPartIteratorByStateAndInfo> range, const ColumnsDescription & storage_columns); boost::iterator_range<DataPartIteratorByStateAndInfo> range, const ColumnsDescription & storage_columns);
std::optional<UInt64> totalRowsByPartitionPredicateImpl( std::optional<UInt64> totalRowsByPartitionPredicateImpl(
const ActionsDAGPtr & filter_actions_dag, ContextPtr context, const DataPartsVector & parts) const; const ActionsDAG & filter_actions_dag, ContextPtr context, const DataPartsVector & parts) const;
static decltype(auto) getStateModifier(DataPartState state) static decltype(auto) getStateModifier(DataPartState state)
{ {

View File

@ -428,7 +428,7 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling(
ASTPtr query = sampling.filter_function; ASTPtr query = sampling.filter_function;
auto syntax_result = TreeRewriter(context).analyze(query, available_real_columns); auto syntax_result = TreeRewriter(context).analyze(query, available_real_columns);
sampling.filter_expression = ExpressionAnalyzer(sampling.filter_function, syntax_result, context).getActionsDAG(false); sampling.filter_expression = std::make_shared<const ActionsDAG>(ExpressionAnalyzer(sampling.filter_function, syntax_result, context).getActionsDAG(false));
} }
} }
@ -466,7 +466,7 @@ void MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset(
dag.get(), dag.get(),
context, context,
sample.getNames(), sample.getNames(),
std::make_shared<ExpressionActions>(std::make_unique<ActionsDAG>(sample.getColumnsWithTypeAndName()), ExpressionActionsSettings{}), std::make_shared<ExpressionActions>(ActionsDAG(sample.getColumnsWithTypeAndName()), ExpressionActionsSettings{}),
{}}); {}});
} }

View File

@ -265,15 +265,15 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet(
if (!set->buildOrderedSetInplace(context)) if (!set->buildOrderedSetInplace(context))
return; return;
auto filter_actions_dag = ActionsDAG::clone(filter_dag); auto filter_actions_dag = std::move(*ActionsDAG::clone(filter_dag));
const auto * filter_actions_dag_node = filter_actions_dag->getOutputs().at(0); const auto * filter_actions_dag_node = filter_actions_dag.getOutputs().at(0);
std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> node_to_result_node; std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> node_to_result_node;
filter_actions_dag->getOutputs()[0] = &traverseDAG(*filter_actions_dag_node, filter_actions_dag, context, node_to_result_node); filter_actions_dag.getOutputs()[0] = &traverseDAG(*filter_actions_dag_node, filter_actions_dag, context, node_to_result_node);
filter_actions_dag->removeUnusedActions(); filter_actions_dag.removeUnusedActions();
actions_output_column_name = filter_actions_dag->getOutputs().at(0)->result_name; actions_output_column_name = filter_actions_dag.getOutputs().at(0)->result_name;
actions = std::make_shared<ExpressionActions>(std::move(filter_actions_dag)); actions = std::make_shared<ExpressionActions>(std::move(filter_actions_dag));
} }
@ -306,7 +306,7 @@ bool MergeTreeIndexConditionSet::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx
} }
static const ActionsDAG::NodeRawConstPtrs & getArguments(const ActionsDAG::Node & node, const ActionsDAGPtr & result_dag_or_null, ActionsDAG::NodeRawConstPtrs * storage) static const ActionsDAG::NodeRawConstPtrs & getArguments(const ActionsDAG::Node & node, ActionsDAG * result_dag_or_null, ActionsDAG::NodeRawConstPtrs * storage)
{ {
chassert(node.type == ActionsDAG::ActionType::FUNCTION); chassert(node.type == ActionsDAG::ActionType::FUNCTION);
if (node.function_base->getName() != "indexHint") if (node.function_base->getName() != "indexHint")
@ -316,17 +316,17 @@ static const ActionsDAG::NodeRawConstPtrs & getArguments(const ActionsDAG::Node
const auto & adaptor = typeid_cast<const FunctionToFunctionBaseAdaptor &>(*node.function_base); const auto & adaptor = typeid_cast<const FunctionToFunctionBaseAdaptor &>(*node.function_base);
const auto & index_hint = typeid_cast<const FunctionIndexHint &>(*adaptor.getFunction()); const auto & index_hint = typeid_cast<const FunctionIndexHint &>(*adaptor.getFunction());
if (!result_dag_or_null) if (!result_dag_or_null)
return index_hint.getActions()->getOutputs(); return index_hint.getActions().getOutputs();
/// Import the DAG and map argument pointers. /// Import the DAG and map argument pointers.
ActionsDAGPtr actions_clone = ActionsDAG::clone(index_hint.getActions()); auto actions_clone = std::move(*ActionsDAG::clone(&index_hint.getActions()));
chassert(storage); chassert(storage);
result_dag_or_null->mergeNodes(std::move(*actions_clone), storage); result_dag_or_null->mergeNodes(std::move(actions_clone), storage);
return *storage; return *storage;
} }
const ActionsDAG::Node & MergeTreeIndexConditionSet::traverseDAG(const ActionsDAG::Node & node, const ActionsDAG::Node & MergeTreeIndexConditionSet::traverseDAG(const ActionsDAG::Node & node,
ActionsDAGPtr & result_dag, ActionsDAG & result_dag,
const ContextPtr & context, const ContextPtr & context,
std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> & node_to_result_node) const std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> & node_to_result_node) const
{ {
@ -348,7 +348,7 @@ const ActionsDAG::Node & MergeTreeIndexConditionSet::traverseDAG(const ActionsDA
atom_node_ptr->type == ActionsDAG::ActionType::FUNCTION) atom_node_ptr->type == ActionsDAG::ActionType::FUNCTION)
{ {
auto bit_wrapper_function = FunctionFactory::instance().get("__bitWrapperFunc", context); auto bit_wrapper_function = FunctionFactory::instance().get("__bitWrapperFunc", context);
result_node = &result_dag->addFunction(bit_wrapper_function, {atom_node_ptr}, {}); result_node = &result_dag.addFunction(bit_wrapper_function, {atom_node_ptr}, {});
} }
} }
else else
@ -359,14 +359,14 @@ const ActionsDAG::Node & MergeTreeIndexConditionSet::traverseDAG(const ActionsDA
unknown_field_column_with_type.type = std::make_shared<DataTypeUInt8>(); unknown_field_column_with_type.type = std::make_shared<DataTypeUInt8>();
unknown_field_column_with_type.column = unknown_field_column_with_type.type->createColumnConst(1, UNKNOWN_FIELD); unknown_field_column_with_type.column = unknown_field_column_with_type.type->createColumnConst(1, UNKNOWN_FIELD);
result_node = &result_dag->addColumn(unknown_field_column_with_type); result_node = &result_dag.addColumn(unknown_field_column_with_type);
} }
node_to_result_node.emplace(&node, result_node); node_to_result_node.emplace(&node, result_node);
return *result_node; return *result_node;
} }
const ActionsDAG::Node * MergeTreeIndexConditionSet::atomFromDAG(const ActionsDAG::Node & node, ActionsDAGPtr & result_dag, const ContextPtr & context) const const ActionsDAG::Node * MergeTreeIndexConditionSet::atomFromDAG(const ActionsDAG::Node & node, ActionsDAG & result_dag, const ContextPtr & context) const
{ {
/// Function, literal or column /// Function, literal or column
@ -386,7 +386,7 @@ const ActionsDAG::Node * MergeTreeIndexConditionSet::atomFromDAG(const ActionsDA
const auto * result_node = node_to_check; const auto * result_node = node_to_check;
if (node.type != ActionsDAG::ActionType::INPUT) if (node.type != ActionsDAG::ActionType::INPUT)
result_node = &result_dag->addInput(column_name, node.result_type); result_node = &result_dag.addInput(column_name, node.result_type);
return result_node; return result_node;
} }
@ -407,11 +407,11 @@ const ActionsDAG::Node * MergeTreeIndexConditionSet::atomFromDAG(const ActionsDA
return nullptr; return nullptr;
} }
return &result_dag->addFunction(node.function_base, children, {}); return &result_dag.addFunction(node.function_base, children, {});
} }
const ActionsDAG::Node * MergeTreeIndexConditionSet::operatorFromDAG(const ActionsDAG::Node & node, const ActionsDAG::Node * MergeTreeIndexConditionSet::operatorFromDAG(const ActionsDAG::Node & node,
ActionsDAGPtr & result_dag, ActionsDAG & result_dag,
const ContextPtr & context, const ContextPtr & context,
std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> & node_to_result_node) const std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> & node_to_result_node) const
{ {
@ -429,7 +429,7 @@ const ActionsDAG::Node * MergeTreeIndexConditionSet::operatorFromDAG(const Actio
auto function_name = node_to_check->function->getName(); auto function_name = node_to_check->function->getName();
ActionsDAG::NodeRawConstPtrs temp_ptrs_to_argument; ActionsDAG::NodeRawConstPtrs temp_ptrs_to_argument;
const auto & arguments = getArguments(*node_to_check, result_dag, &temp_ptrs_to_argument); const auto & arguments = getArguments(*node_to_check, &result_dag, &temp_ptrs_to_argument);
size_t arguments_size = arguments.size(); size_t arguments_size = arguments.size();
if (function_name == "not") if (function_name == "not")
@ -440,7 +440,7 @@ const ActionsDAG::Node * MergeTreeIndexConditionSet::operatorFromDAG(const Actio
const ActionsDAG::Node * argument = &traverseDAG(*arguments[0], result_dag, context, node_to_result_node); const ActionsDAG::Node * argument = &traverseDAG(*arguments[0], result_dag, context, node_to_result_node);
auto bit_swap_last_two_function = FunctionFactory::instance().get("__bitSwapLastTwo", context); auto bit_swap_last_two_function = FunctionFactory::instance().get("__bitSwapLastTwo", context);
return &result_dag->addFunction(bit_swap_last_two_function, {argument}, {}); return &result_dag.addFunction(bit_swap_last_two_function, {argument}, {});
} }
else if (function_name == "and" || function_name == "indexHint" || function_name == "or") else if (function_name == "and" || function_name == "indexHint" || function_name == "or")
{ {
@ -468,7 +468,7 @@ const ActionsDAG::Node * MergeTreeIndexConditionSet::operatorFromDAG(const Actio
const auto * before_last_argument = children.back(); const auto * before_last_argument = children.back();
children.pop_back(); children.pop_back();
last_argument = &result_dag->addFunction(function, {before_last_argument, last_argument}, {}); last_argument = &result_dag.addFunction(function, {before_last_argument, last_argument}, {});
} }
return last_argument; return last_argument;

View File

@ -93,16 +93,16 @@ public:
~MergeTreeIndexConditionSet() override = default; ~MergeTreeIndexConditionSet() override = default;
private: private:
const ActionsDAG::Node & traverseDAG(const ActionsDAG::Node & node, const ActionsDAG::Node & traverseDAG(const ActionsDAG::Node & node,
ActionsDAGPtr & result_dag, ActionsDAG & result_dag,
const ContextPtr & context, const ContextPtr & context,
std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> & node_to_result_node) const; std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> & node_to_result_node) const;
const ActionsDAG::Node * atomFromDAG(const ActionsDAG::Node & node, const ActionsDAG::Node * atomFromDAG(const ActionsDAG::Node & node,
ActionsDAGPtr & result_dag, ActionsDAG & result_dag,
const ContextPtr & context) const; const ContextPtr & context) const;
const ActionsDAG::Node * operatorFromDAG(const ActionsDAG::Node & node, const ActionsDAG::Node * operatorFromDAG(const ActionsDAG::Node & node,
ActionsDAGPtr & result_dag, ActionsDAG & result_dag,
const ContextPtr & context, const ContextPtr & context,
std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> & node_to_result_node) const; std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> & node_to_result_node) const;

View File

@ -80,7 +80,7 @@ PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr pr
PrewhereExprStep row_level_filter_step PrewhereExprStep row_level_filter_step
{ {
.type = PrewhereExprStep::Filter, .type = PrewhereExprStep::Filter,
.actions = std::make_shared<ExpressionActions>(ActionsDAG::clone(prewhere_info->row_level_filter), actions_settings), .actions = std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(&*prewhere_info->row_level_filter)), actions_settings),
.filter_column_name = prewhere_info->row_level_column_name, .filter_column_name = prewhere_info->row_level_column_name,
.remove_filter_column = true, .remove_filter_column = true,
.need_filter = true, .need_filter = true,
@ -96,7 +96,7 @@ PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr pr
PrewhereExprStep prewhere_step PrewhereExprStep prewhere_step
{ {
.type = PrewhereExprStep::Filter, .type = PrewhereExprStep::Filter,
.actions = std::make_shared<ExpressionActions>(ActionsDAG::clone(prewhere_info->prewhere_actions), actions_settings), .actions = std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(&*prewhere_info->prewhere_actions)), actions_settings),
.filter_column_name = prewhere_info->prewhere_column_name, .filter_column_name = prewhere_info->prewhere_column_name,
.remove_filter_column = prewhere_info->remove_prewhere_column, .remove_filter_column = prewhere_info->remove_prewhere_column,
.need_filter = prewhere_info->need_filter, .need_filter = prewhere_info->need_filter,

View File

@ -349,7 +349,7 @@ public:
MergeTreeData::DataPartPtr data_part_, MergeTreeData::DataPartPtr data_part_,
Names columns_to_read_, Names columns_to_read_,
bool apply_deleted_mask_, bool apply_deleted_mask_,
ActionsDAGPtr filter_, std::optional<ActionsDAG> filter_,
ContextPtr context_, ContextPtr context_,
LoggerPtr log_) LoggerPtr log_)
: ISourceStep(DataStream{.header = storage_snapshot_->getSampleBlockForColumns(columns_to_read_)}) : ISourceStep(DataStream{.header = storage_snapshot_->getSampleBlockForColumns(columns_to_read_)})
@ -376,7 +376,7 @@ public:
{ {
const auto & primary_key = storage_snapshot->metadata->getPrimaryKey(); const auto & primary_key = storage_snapshot->metadata->getPrimaryKey();
const Names & primary_key_column_names = primary_key.column_names; const Names & primary_key_column_names = primary_key.column_names;
KeyCondition key_condition(filter.get(), context, primary_key_column_names, primary_key.expression); KeyCondition key_condition(&*filter, context, primary_key_column_names, primary_key.expression);
LOG_DEBUG(log, "Key condition: {}", key_condition.toString()); LOG_DEBUG(log, "Key condition: {}", key_condition.toString());
if (!key_condition.alwaysFalse()) if (!key_condition.alwaysFalse())
@ -417,7 +417,7 @@ private:
MergeTreeData::DataPartPtr data_part; MergeTreeData::DataPartPtr data_part;
Names columns_to_read; Names columns_to_read;
bool apply_deleted_mask; bool apply_deleted_mask;
ActionsDAGPtr filter; std::optional<ActionsDAG> filter;
ContextPtr context; ContextPtr context;
LoggerPtr log; LoggerPtr log;
}; };
@ -430,7 +430,7 @@ void createReadFromPartStep(
MergeTreeData::DataPartPtr data_part, MergeTreeData::DataPartPtr data_part,
Names columns_to_read, Names columns_to_read,
bool apply_deleted_mask, bool apply_deleted_mask,
ActionsDAGPtr filter, std::optional<ActionsDAG> filter,
ContextPtr context, ContextPtr context,
LoggerPtr log) LoggerPtr log)
{ {

View File

@ -38,7 +38,7 @@ void createReadFromPartStep(
MergeTreeData::DataPartPtr data_part, MergeTreeData::DataPartPtr data_part,
Names columns_to_read, Names columns_to_read,
bool apply_deleted_mask, bool apply_deleted_mask,
ActionsDAGPtr filter, std::optional<ActionsDAG> filter,
ContextPtr context, ContextPtr context,
LoggerPtr log); LoggerPtr log);

View File

@ -349,7 +349,7 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction
PrewhereExprStep new_step PrewhereExprStep new_step
{ {
.type = PrewhereExprStep::Filter, .type = PrewhereExprStep::Filter,
.actions = std::make_shared<ExpressionActions>(std::move(step.actions), actions_settings), .actions = std::make_shared<ExpressionActions>(std::move(*step.actions), actions_settings),
.filter_column_name = step.column_name, .filter_column_name = step.column_name,
/// Don't remove if it's in the list of original outputs /// Don't remove if it's in the list of original outputs
.remove_filter_column = .remove_filter_column =

View File

@ -112,7 +112,7 @@ void MergeTreeWhereOptimizer::optimize(SelectQueryInfo & select_query_info, cons
LOG_DEBUG(log, "MergeTreeWhereOptimizer: condition \"{}\" moved to PREWHERE", select.prewhere()->formatForLogging(log_queries_cut_to_length)); LOG_DEBUG(log, "MergeTreeWhereOptimizer: condition \"{}\" moved to PREWHERE", select.prewhere()->formatForLogging(log_queries_cut_to_length));
} }
MergeTreeWhereOptimizer::FilterActionsOptimizeResult MergeTreeWhereOptimizer::optimize(const ActionsDAGPtr & filter_dag, MergeTreeWhereOptimizer::FilterActionsOptimizeResult MergeTreeWhereOptimizer::optimize(const ActionsDAG & filter_dag,
const std::string & filter_column_name, const std::string & filter_column_name,
const ContextPtr & context, const ContextPtr & context,
bool is_final) bool is_final)
@ -126,7 +126,7 @@ MergeTreeWhereOptimizer::FilterActionsOptimizeResult MergeTreeWhereOptimizer::op
where_optimizer_context.use_statistics = context->getSettingsRef().allow_statistics_optimize; where_optimizer_context.use_statistics = context->getSettingsRef().allow_statistics_optimize;
RPNBuilderTreeContext tree_context(context); RPNBuilderTreeContext tree_context(context);
RPNBuilderTreeNode node(&filter_dag->findInOutputs(filter_column_name), tree_context); RPNBuilderTreeNode node(&filter_dag.findInOutputs(filter_column_name), tree_context);
auto optimize_result = optimizeImpl(node, where_optimizer_context); auto optimize_result = optimizeImpl(node, where_optimizer_context);
if (!optimize_result) if (!optimize_result)

View File

@ -52,7 +52,7 @@ public:
bool fully_moved_to_prewhere = false; bool fully_moved_to_prewhere = false;
}; };
FilterActionsOptimizeResult optimize(const ActionsDAGPtr & filter_dag, FilterActionsOptimizeResult optimize(const ActionsDAG & filter_dag,
const std::string & filter_column_name, const std::string & filter_column_name,
const ContextPtr & context, const ContextPtr & context,
bool is_final); bool is_final);

View File

@ -398,7 +398,7 @@ size_t RPNBuilderFunctionTreeNode::getArgumentsSize() const
{ {
const auto * adaptor = typeid_cast<const FunctionToFunctionBaseAdaptor *>(dag_node->function_base.get()); const auto * adaptor = typeid_cast<const FunctionToFunctionBaseAdaptor *>(dag_node->function_base.get());
const auto * index_hint = typeid_cast<const FunctionIndexHint *>(adaptor->getFunction().get()); const auto * index_hint = typeid_cast<const FunctionIndexHint *>(adaptor->getFunction().get());
return index_hint->getActions()->getOutputs().size(); return index_hint->getActions().getOutputs().size();
} }
return dag_node->children.size(); return dag_node->children.size();
@ -426,7 +426,7 @@ RPNBuilderTreeNode RPNBuilderFunctionTreeNode::getArgumentAt(size_t index) const
{ {
const auto & adaptor = typeid_cast<const FunctionToFunctionBaseAdaptor &>(*dag_node->function_base); const auto & adaptor = typeid_cast<const FunctionToFunctionBaseAdaptor &>(*dag_node->function_base);
const auto & index_hint = typeid_cast<const FunctionIndexHint &>(*adaptor.getFunction()); const auto & index_hint = typeid_cast<const FunctionIndexHint &>(*adaptor.getFunction());
return RPNBuilderTreeNode(index_hint.getActions()->getOutputs()[index], tree_context); return RPNBuilderTreeNode(index_hint.getActions().getOutputs()[index], tree_context);
} }
return RPNBuilderTreeNode(dag_node->children[index], tree_context); return RPNBuilderTreeNode(dag_node->children[index], tree_context);

View File

@ -77,9 +77,9 @@ StorageObjectStorageSource::~StorageObjectStorageSource()
create_reader_pool->wait(); create_reader_pool->wait();
} }
void StorageObjectStorageSource::setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) void StorageObjectStorageSource::setKeyCondition(const std::optional<ActionsDAG> & filter_actions_dag, ContextPtr context_)
{ {
setKeyConditionImpl(filter_actions_dag.get(), context_, read_from_format_info.format_header); setKeyConditionImpl(filter_actions_dag, context_, read_from_format_info.format_header);
} }
std::string StorageObjectStorageSource::getUniqueStoragePathIdentifier( std::string StorageObjectStorageSource::getUniqueStoragePathIdentifier(

View File

@ -45,7 +45,7 @@ public:
String getName() const override { return name; } String getName() const override { return name; }
void setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) override; void setKeyCondition(const std::optional<ActionsDAG> & filter_actions_dag, ContextPtr context_) override;
Chunk generate() override; Chunk generate() override;

View File

@ -79,7 +79,7 @@ void readFinalFromNestedStorage(
auto step = std::make_unique<FilterStep>( auto step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(), query_plan.getCurrentDataStream(),
actions, std::move(actions),
filter_column_name, filter_column_name,
false); false);

View File

@ -46,9 +46,9 @@ struct PrewhereInfo
{ {
/// Actions for row level security filter. Applied separately before prewhere_actions. /// Actions for row level security filter. Applied separately before prewhere_actions.
/// This actions are separate because prewhere condition should not be executed over filtered rows. /// This actions are separate because prewhere condition should not be executed over filtered rows.
ActionsDAGPtr row_level_filter; std::optional<ActionsDAG> row_level_filter;
/// Actions which are executed on block in order to get filter column for prewhere step. /// Actions which are executed on block in order to get filter column for prewhere step.
ActionsDAGPtr prewhere_actions; std::optional<ActionsDAG> prewhere_actions;
String row_level_column_name; String row_level_column_name;
String prewhere_column_name; String prewhere_column_name;
bool remove_prewhere_column = false; bool remove_prewhere_column = false;
@ -56,7 +56,7 @@ struct PrewhereInfo
bool generated_by_optimizer = false; bool generated_by_optimizer = false;
PrewhereInfo() = default; PrewhereInfo() = default;
explicit PrewhereInfo(ActionsDAGPtr prewhere_actions_, String prewhere_column_name_) explicit PrewhereInfo(std::optional<ActionsDAG> prewhere_actions_, String prewhere_column_name_)
: prewhere_actions(std::move(prewhere_actions_)), prewhere_column_name(std::move(prewhere_column_name_)) {} : prewhere_actions(std::move(prewhere_actions_)), prewhere_column_name(std::move(prewhere_column_name_)) {}
std::string dump() const; std::string dump() const;
@ -66,10 +66,10 @@ struct PrewhereInfo
PrewhereInfoPtr prewhere_info = std::make_shared<PrewhereInfo>(); PrewhereInfoPtr prewhere_info = std::make_shared<PrewhereInfo>();
if (row_level_filter) if (row_level_filter)
prewhere_info->row_level_filter = ActionsDAG::clone(row_level_filter); prewhere_info->row_level_filter = std::move(*ActionsDAG::clone(&*row_level_filter));
if (prewhere_actions) if (prewhere_actions)
prewhere_info->prewhere_actions = ActionsDAG::clone(prewhere_actions); prewhere_info->prewhere_actions = std::move(*ActionsDAG::clone(&*prewhere_actions));
prewhere_info->row_level_column_name = row_level_column_name; prewhere_info->row_level_column_name = row_level_column_name;
prewhere_info->prewhere_column_name = prewhere_column_name; prewhere_info->prewhere_column_name = prewhere_column_name;
@ -93,7 +93,7 @@ struct FilterInfo
/// Same as FilterInfo, but with ActionsDAG. /// Same as FilterInfo, but with ActionsDAG.
struct FilterDAGInfo struct FilterDAGInfo
{ {
ActionsDAGPtr actions; std::optional<ActionsDAG> actions;
String column_name; String column_name;
bool do_remove_column = false; bool do_remove_column = false;

View File

@ -312,7 +312,7 @@ void StorageBuffer::read(
if (src_table_query_info.prewhere_info->row_level_filter) if (src_table_query_info.prewhere_info->row_level_filter)
{ {
src_table_query_info.prewhere_info->row_level_filter = ActionsDAG::merge( src_table_query_info.prewhere_info->row_level_filter = ActionsDAG::merge(
std::move(*ActionsDAG::clone(actions_dag)), std::move(*ActionsDAG::clone(&actions_dag)),
std::move(*src_table_query_info.prewhere_info->row_level_filter)); std::move(*src_table_query_info.prewhere_info->row_level_filter));
src_table_query_info.prewhere_info->row_level_filter->removeUnusedActions(); src_table_query_info.prewhere_info->row_level_filter->removeUnusedActions();
@ -321,7 +321,7 @@ void StorageBuffer::read(
if (src_table_query_info.prewhere_info->prewhere_actions) if (src_table_query_info.prewhere_info->prewhere_actions)
{ {
src_table_query_info.prewhere_info->prewhere_actions = ActionsDAG::merge( src_table_query_info.prewhere_info->prewhere_actions = ActionsDAG::merge(
std::move(*ActionsDAG::clone(actions_dag)), std::move(*ActionsDAG::clone(&actions_dag)),
std::move(*src_table_query_info.prewhere_info->prewhere_actions)); std::move(*src_table_query_info.prewhere_info->prewhere_actions));
src_table_query_info.prewhere_info->prewhere_actions->removeUnusedActions(); src_table_query_info.prewhere_info->prewhere_actions->removeUnusedActions();
@ -353,7 +353,7 @@ void StorageBuffer::read(
header.getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(),
ActionsDAG::MatchColumnsMode::Name); ActionsDAG::MatchColumnsMode::Name);
auto converting = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), actions_dag); auto converting = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), std::move(actions_dag));
converting->setStepDescription("Convert destination table columns to Buffer table structure"); converting->setStepDescription("Convert destination table columns to Buffer table structure");
query_plan.addStep(std::move(converting)); query_plan.addStep(std::move(converting));
@ -432,7 +432,7 @@ void StorageBuffer::read(
{ {
return std::make_shared<FilterTransform>( return std::make_shared<FilterTransform>(
header, header,
std::make_shared<ExpressionActions>(ActionsDAG::clone(query_info.prewhere_info->row_level_filter), actions_settings), std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(&*query_info.prewhere_info->row_level_filter)), actions_settings),
query_info.prewhere_info->row_level_column_name, query_info.prewhere_info->row_level_column_name,
false); false);
}); });
@ -442,7 +442,7 @@ void StorageBuffer::read(
{ {
return std::make_shared<FilterTransform>( return std::make_shared<FilterTransform>(
header, header,
std::make_shared<ExpressionActions>(ActionsDAG::clone(query_info.prewhere_info->prewhere_actions), actions_settings), std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(&*query_info.prewhere_info->prewhere_actions)), actions_settings),
query_info.prewhere_info->prewhere_column_name, query_info.prewhere_info->prewhere_column_name,
query_info.prewhere_info->remove_prewhere_column); query_info.prewhere_info->remove_prewhere_column);
}); });
@ -472,7 +472,7 @@ void StorageBuffer::read(
result_header.getColumnsWithTypeAndName(), result_header.getColumnsWithTypeAndName(),
ActionsDAG::MatchColumnsMode::Name); ActionsDAG::MatchColumnsMode::Name);
auto converting = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), convert_actions_dag); auto converting = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), std::move(convert_actions_dag));
query_plan.addStep(std::move(converting)); query_plan.addStep(std::move(converting));
} }

View File

@ -1074,7 +1074,7 @@ std::optional<QueryPipeline> StorageDistributed::distributedWriteBetweenDistribu
return pipeline; return pipeline;
} }
static ActionsDAGPtr getFilterFromQuery(const ASTPtr & ast, ContextPtr context) static std::optional<ActionsDAG> getFilterFromQuery(const ASTPtr & ast, ContextPtr context)
{ {
QueryPlan plan; QueryPlan plan;
SelectQueryOptions options; SelectQueryOptions options;
@ -1118,7 +1118,7 @@ static ActionsDAGPtr getFilterFromQuery(const ASTPtr & ast, ContextPtr context)
} }
if (!source) if (!source)
return nullptr; return {};
return source->detachFilterActionsDAG(); return source->detachFilterActionsDAG();
} }

View File

@ -1233,9 +1233,9 @@ StorageFileSource::~StorageFileSource()
beforeDestroy(); beforeDestroy();
} }
void StorageFileSource::setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) void StorageFileSource::setKeyCondition(const std::optional<ActionsDAG> & filter_actions_dag, ContextPtr context_)
{ {
setKeyConditionImpl(filter_actions_dag.get(), context_, block_for_format); setKeyConditionImpl(filter_actions_dag, context_, block_for_format);
} }

View File

@ -265,7 +265,7 @@ private:
return storage->getName(); return storage->getName();
} }
void setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) override; void setKeyCondition(const std::optional<ActionsDAG> & filter_actions_dag, ContextPtr context_) override;
bool tryGetCountFromCache(const struct stat & file_stat); bool tryGetCountFromCache(const struct stat & file_stat);

View File

@ -273,8 +273,8 @@ void StorageMaterializedView::read(
* They may be added in case of distributed query with JOIN. * They may be added in case of distributed query with JOIN.
* In that case underlying table returns joined columns as well. * In that case underlying table returns joined columns as well.
*/ */
converting_actions->removeUnusedActions(); converting_actions.removeUnusedActions();
auto converting_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), converting_actions); auto converting_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), std::move(converting_actions));
converting_step->setStepDescription("Convert target table structure to MaterializedView structure"); converting_step->setStepDescription("Convert target table structure to MaterializedView structure");
query_plan.addStep(std::move(converting_step)); query_plan.addStep(std::move(converting_step));
} }

View File

@ -407,7 +407,7 @@ void ReadFromMerge::addFilter(FilterDAGInfo filter)
{ {
output_stream->header = FilterTransform::transformHeader( output_stream->header = FilterTransform::transformHeader(
output_stream->header, output_stream->header,
filter.actions.get(), filter.actions ? &*filter.actions : nullptr,
filter.column_name, filter.column_name,
filter.do_remove_column); filter.do_remove_column);
pushed_down_filters.push_back(std::move(filter)); pushed_down_filters.push_back(std::move(filter));
@ -628,7 +628,7 @@ std::vector<ReadFromMerge::ChildPlan> ReadFromMerge::createChildrenPlans(SelectQ
auto alias_actions = ExpressionAnalyzer(required_columns_expr_list, syntax_result, context).getActionsDAG(true); auto alias_actions = ExpressionAnalyzer(required_columns_expr_list, syntax_result, context).getActionsDAG(true);
column_names_as_aliases = alias_actions->getRequiredColumns().getNames(); column_names_as_aliases = alias_actions.getRequiredColumns().getNames();
if (column_names_as_aliases.empty()) if (column_names_as_aliases.empty())
column_names_as_aliases.push_back(ExpressionActions::getSmallestColumn(storage_metadata_snapshot->getColumns().getAllPhysical()).name); column_names_as_aliases.push_back(ExpressionActions::getSmallestColumn(storage_metadata_snapshot->getColumns().getAllPhysical()).name);
} }
@ -662,7 +662,7 @@ std::vector<ReadFromMerge::ChildPlan> ReadFromMerge::createChildrenPlans(SelectQ
{ {
auto filter_step = std::make_unique<FilterStep>( auto filter_step = std::make_unique<FilterStep>(
child.plan.getCurrentDataStream(), child.plan.getCurrentDataStream(),
ActionsDAG::clone(filter_info.actions), std::move(*ActionsDAG::clone(&*filter_info.actions)),
filter_info.column_name, filter_info.column_name,
filter_info.do_remove_column); filter_info.do_remove_column);
@ -1060,7 +1060,7 @@ void ReadFromMerge::addVirtualColumns(
column.column = column.type->createColumnConst(0, Field(database_name)); column.column = column.type->createColumnConst(0, Field(database_name));
auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column));
auto expression_step = std::make_unique<ExpressionStep>(child.plan.getCurrentDataStream(), adding_column_dag); auto expression_step = std::make_unique<ExpressionStep>(child.plan.getCurrentDataStream(), std::move(adding_column_dag));
child.plan.addStep(std::move(expression_step)); child.plan.addStep(std::move(expression_step));
plan_header = child.plan.getCurrentDataStream().header; plan_header = child.plan.getCurrentDataStream().header;
} }
@ -1074,7 +1074,7 @@ void ReadFromMerge::addVirtualColumns(
column.column = column.type->createColumnConst(0, Field(table_name)); column.column = column.type->createColumnConst(0, Field(table_name));
auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column));
auto expression_step = std::make_unique<ExpressionStep>(child.plan.getCurrentDataStream(), adding_column_dag); auto expression_step = std::make_unique<ExpressionStep>(child.plan.getCurrentDataStream(), std::move(adding_column_dag));
child.plan.addStep(std::move(expression_step)); child.plan.addStep(std::move(expression_step));
plan_header = child.plan.getCurrentDataStream().header; plan_header = child.plan.getCurrentDataStream().header;
} }
@ -1089,7 +1089,7 @@ void ReadFromMerge::addVirtualColumns(
column.column = column.type->createColumnConst(0, Field(database_name)); column.column = column.type->createColumnConst(0, Field(database_name));
auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column));
auto expression_step = std::make_unique<ExpressionStep>(child.plan.getCurrentDataStream(), adding_column_dag); auto expression_step = std::make_unique<ExpressionStep>(child.plan.getCurrentDataStream(), std::move(adding_column_dag));
child.plan.addStep(std::move(expression_step)); child.plan.addStep(std::move(expression_step));
plan_header = child.plan.getCurrentDataStream().header; plan_header = child.plan.getCurrentDataStream().header;
} }
@ -1102,7 +1102,7 @@ void ReadFromMerge::addVirtualColumns(
column.column = column.type->createColumnConst(0, Field(table_name)); column.column = column.type->createColumnConst(0, Field(table_name));
auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column));
auto expression_step = std::make_unique<ExpressionStep>(child.plan.getCurrentDataStream(), adding_column_dag); auto expression_step = std::make_unique<ExpressionStep>(child.plan.getCurrentDataStream(), std::move(adding_column_dag));
child.plan.addStep(std::move(expression_step)); child.plan.addStep(std::move(expression_step));
plan_header = child.plan.getCurrentDataStream().header; plan_header = child.plan.getCurrentDataStream().header;
} }
@ -1240,7 +1240,7 @@ ReadFromMerge::RowPolicyData::RowPolicyData(RowPolicyFilterPtr row_policy_filter
auto expression_analyzer = ExpressionAnalyzer{expr, syntax_result, local_context}; auto expression_analyzer = ExpressionAnalyzer{expr, syntax_result, local_context};
actions_dag = expression_analyzer.getActionsDAG(false /* add_aliases */, false /* project_result */); actions_dag = expression_analyzer.getActionsDAG(false /* add_aliases */, false /* project_result */);
filter_actions = std::make_shared<ExpressionActions>(ActionsDAG::clone(actions_dag), filter_actions = std::make_shared<ExpressionActions>(std::move(*ActionsDAG::clone(&actions_dag)),
ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes));
const auto & required_columns = filter_actions->getRequiredColumnsWithTypes(); const auto & required_columns = filter_actions->getRequiredColumnsWithTypes();
const auto & sample_block_columns = filter_actions->getSampleBlock().getNamesAndTypesList(); const auto & sample_block_columns = filter_actions->getSampleBlock().getNamesAndTypesList();
@ -1278,12 +1278,12 @@ void ReadFromMerge::RowPolicyData::extendNames(Names & names) const
void ReadFromMerge::RowPolicyData::addStorageFilter(SourceStepWithFilter * step) const void ReadFromMerge::RowPolicyData::addStorageFilter(SourceStepWithFilter * step) const
{ {
step->addFilter(ActionsDAG::clone(actions_dag), filter_column_name); step->addFilter(ActionsDAG::clone(&actions_dag), filter_column_name);
} }
void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPlan & plan) const void ReadFromMerge::RowPolicyData::addFilterTransform(QueryPlan & plan) const
{ {
auto filter_step = std::make_unique<FilterStep>(plan.getCurrentDataStream(), ActionsDAG::clone(actions_dag), filter_column_name, true /* remove filter column */); auto filter_step = std::make_unique<FilterStep>(plan.getCurrentDataStream(), std::move(*ActionsDAG::clone(&actions_dag)), filter_column_name, true /* remove filter column */);
plan.addStep(std::move(filter_step)); plan.addStep(std::move(filter_step));
} }
@ -1476,7 +1476,7 @@ void ReadFromMerge::convertAndFilterSourceStream(
{ {
pipe_columns.emplace_back(NameAndTypePair(alias.name, alias.type)); pipe_columns.emplace_back(NameAndTypePair(alias.name, alias.type));
auto actions_dag = std::make_unique<ActionsDAG>(pipe_columns); ActionsDAG actions_dag(pipe_columns);
QueryTreeNodePtr query_tree = buildQueryTree(alias.expression, local_context); QueryTreeNodePtr query_tree = buildQueryTree(alias.expression, local_context);
query_tree->setAlias(alias.name); query_tree->setAlias(alias.name);
@ -1485,12 +1485,12 @@ void ReadFromMerge::convertAndFilterSourceStream(
query_analysis_pass.run(query_tree, local_context); query_analysis_pass.run(query_tree, local_context);
PlannerActionsVisitor actions_visitor(modified_query_info.planner_context, false /*use_column_identifier_as_action_node_name*/); PlannerActionsVisitor actions_visitor(modified_query_info.planner_context, false /*use_column_identifier_as_action_node_name*/);
const auto & nodes = actions_visitor.visit(*actions_dag, query_tree); const auto & nodes = actions_visitor.visit(actions_dag, query_tree);
if (nodes.size() != 1) if (nodes.size() != 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected to have 1 output but got {}", nodes.size()); throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected to have 1 output but got {}", nodes.size());
actions_dag->addOrReplaceInOutputs(actions_dag->addAlias(*nodes.front(), alias.name)); actions_dag.addOrReplaceInOutputs(actions_dag.addAlias(*nodes.front(), alias.name));
auto expression_step = std::make_unique<ExpressionStep>(child.plan.getCurrentDataStream(), std::move(actions_dag)); auto expression_step = std::make_unique<ExpressionStep>(child.plan.getCurrentDataStream(), std::move(actions_dag));
child.plan.addStep(std::move(expression_step)); child.plan.addStep(std::move(expression_step));
} }
@ -1506,7 +1506,7 @@ void ReadFromMerge::convertAndFilterSourceStream(
auto dag = std::make_shared<ActionsDAG>(pipe_columns); auto dag = std::make_shared<ActionsDAG>(pipe_columns);
auto actions_dag = expression_analyzer.getActionsDAG(true, false); auto actions_dag = expression_analyzer.getActionsDAG(true, false);
auto expression_step = std::make_unique<ExpressionStep>(child.plan.getCurrentDataStream(), actions_dag); auto expression_step = std::make_unique<ExpressionStep>(child.plan.getCurrentDataStream(), std::move(actions_dag));
child.plan.addStep(std::move(expression_step)); child.plan.addStep(std::move(expression_step));
} }
} }
@ -1524,7 +1524,7 @@ void ReadFromMerge::convertAndFilterSourceStream(
header.getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(),
convert_actions_match_columns_mode); convert_actions_match_columns_mode);
auto expression_step = std::make_unique<ExpressionStep>(child.plan.getCurrentDataStream(), convert_actions_dag); auto expression_step = std::make_unique<ExpressionStep>(child.plan.getCurrentDataStream(), std::move(convert_actions_dag));
child.plan.addStep(std::move(expression_step)); child.plan.addStep(std::move(expression_step));
} }

View File

@ -225,7 +225,7 @@ private:
private: private:
std::string filter_column_name; // complex filter, may contain logic operations std::string filter_column_name; // complex filter, may contain logic operations
ActionsDAGPtr actions_dag; ActionsDAG actions_dag;
ExpressionActionsPtr filter_actions; ExpressionActionsPtr filter_actions;
StorageMetadataPtr storage_metadata_snapshot; StorageMetadataPtr storage_metadata_snapshot;
}; };

View File

@ -245,7 +245,7 @@ std::optional<UInt64> StorageMergeTree::totalRows(const Settings &) const
return getTotalActiveSizeInRows(); return getTotalActiveSizeInRows();
} }
std::optional<UInt64> StorageMergeTree::totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr local_context) const std::optional<UInt64> StorageMergeTree::totalRowsByPartitionPredicate(const ActionsDAG & filter_actions_dag, ContextPtr local_context) const
{ {
auto parts = getVisibleDataPartsVector(local_context); auto parts = getVisibleDataPartsVector(local_context);
return totalRowsByPartitionPredicateImpl(filter_actions_dag, local_context, parts); return totalRowsByPartitionPredicateImpl(filter_actions_dag, local_context, parts);

View File

@ -65,7 +65,7 @@ public:
size_t num_streams) override; size_t num_streams) override;
std::optional<UInt64> totalRows(const Settings &) const override; std::optional<UInt64> totalRows(const Settings &) const override;
std::optional<UInt64> totalRowsByPartitionPredicate(const ActionsDAGPtr & filter_actions_dag, ContextPtr) const override; std::optional<UInt64> totalRowsByPartitionPredicate(const ActionsDAG & filter_actions_dag, ContextPtr) const override;
std::optional<UInt64> totalBytes(const Settings &) const override; std::optional<UInt64> totalBytes(const Settings &) const override;
std::optional<UInt64> totalBytesUncompressed(const Settings &) const override; std::optional<UInt64> totalBytesUncompressed(const Settings &) const override;

Some files were not shown because too many files have changed in this diff Show More