mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Change ActionsDAGPtr to ActionsDAG where possible.
This commit is contained in:
parent
7843313f8e
commit
2257f9a2ae
@ -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())
|
||||||
|
@ -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
|
||||||
|
@ -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;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -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;
|
||||||
|
@ -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 *>;
|
||||||
|
@ -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;
|
||||||
|
@ -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;
|
||||||
|
@ -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; }
|
||||||
|
|
||||||
|
@ -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())
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
@ -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);
|
||||||
|
@ -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");
|
||||||
|
@ -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));
|
||||||
|
|
||||||
|
@ -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;
|
||||||
|
@ -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)));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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());
|
||||||
|
@ -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);
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
@ -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)
|
||||||
{
|
{
|
||||||
|
@ -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);
|
||||||
|
@ -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,
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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)
|
||||||
{
|
{
|
||||||
|
@ -25,7 +25,7 @@ class TableNode;
|
|||||||
|
|
||||||
struct FiltersForTableExpression
|
struct FiltersForTableExpression
|
||||||
{
|
{
|
||||||
ActionsDAGPtr filter_actions;
|
std::optional<ActionsDAG> filter_actions;
|
||||||
PrewhereInfoPtr prewhere_info;
|
PrewhereInfoPtr prewhere_info;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -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)
|
||||||
|
@ -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
|
||||||
|
@ -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.
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
|
|
||||||
|
@ -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());
|
||||||
|
@ -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));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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)
|
||||||
{
|
{
|
||||||
|
@ -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;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -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)
|
||||||
{
|
{
|
||||||
|
@ -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;
|
||||||
};
|
};
|
||||||
|
@ -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;
|
||||||
|
@ -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();
|
||||||
}
|
}
|
||||||
|
@ -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;
|
||||||
|
@ -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]");
|
||||||
|
@ -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)
|
||||||
|
@ -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
|
||||||
|
@ -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() + ")");
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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()))
|
||||||
{
|
{
|
||||||
|
@ -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>(
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
|
@ -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);
|
||||||
|
@ -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
|
||||||
|
@ -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());
|
||||||
|
@ -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
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -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();
|
||||||
|
@ -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");
|
||||||
|
@ -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));
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
@ -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))
|
||||||
{
|
{
|
||||||
|
@ -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));
|
||||||
|
@ -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.
|
||||||
|
@ -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,
|
||||||
|
@ -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;
|
||||||
|
@ -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*/) { }
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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)
|
||||||
|
@ -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())
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
@ -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 {};
|
||||||
}
|
}
|
||||||
|
@ -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)
|
||||||
|
@ -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};
|
||||||
|
@ -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)
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
@ -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});
|
||||||
|
@ -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 {};
|
||||||
|
|
||||||
|
@ -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)
|
||||||
{
|
{
|
||||||
|
@ -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{}),
|
||||||
{}});
|
{}});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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;
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
@ -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,
|
||||||
|
@ -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)
|
||||||
{
|
{
|
||||||
|
@ -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);
|
||||||
|
|
||||||
|
@ -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 =
|
||||||
|
@ -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)
|
||||||
|
@ -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);
|
||||||
|
@ -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);
|
||||||
|
@ -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(
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
@ -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);
|
||||||
|
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
@ -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));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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();
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -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);
|
||||||
|
|
||||||
|
@ -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));
|
||||||
}
|
}
|
||||||
|
@ -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));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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;
|
||||||
};
|
};
|
||||||
|
@ -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);
|
||||||
|
@ -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
Loading…
Reference in New Issue
Block a user