mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Merge branch 'master' into fix-json-recursion
This commit is contained in:
commit
08934e1282
21
.github/workflows/master.yml
vendored
21
.github/workflows/master.yml
vendored
@ -93,21 +93,21 @@ jobs:
|
||||
with:
|
||||
stage: Builds_2
|
||||
data: ${{ needs.RunConfig.outputs.data }}
|
||||
Tests_2:
|
||||
Tests_2_ww:
|
||||
needs: [RunConfig, Builds_2]
|
||||
if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_2_ww') }}
|
||||
uses: ./.github/workflows/reusable_test_stage.yml
|
||||
with:
|
||||
stage: Tests_2_ww
|
||||
data: ${{ needs.RunConfig.outputs.data }}
|
||||
Tests_2:
|
||||
# Test_3 should not wait for Test_1/Test_2 and should not be blocked by them on master branch since all jobs need to run there.
|
||||
needs: [RunConfig, Builds_1]
|
||||
if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_2') }}
|
||||
uses: ./.github/workflows/reusable_test_stage.yml
|
||||
with:
|
||||
stage: Tests_2
|
||||
data: ${{ needs.RunConfig.outputs.data }}
|
||||
Tests_3:
|
||||
# Test_3 should not wait for Test_1/Test_2 and should not be blocked by them on master branch since all jobs need to run there.
|
||||
needs: [RunConfig, Builds_1]
|
||||
if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_3') }}
|
||||
uses: ./.github/workflows/reusable_test_stage.yml
|
||||
with:
|
||||
stage: Tests_3
|
||||
data: ${{ needs.RunConfig.outputs.data }}
|
||||
|
||||
################################# Reports #################################
|
||||
# Reports should run even if Builds_1/2 fail - run them separately, not in Tests_1/2/3
|
||||
@ -123,7 +123,7 @@ jobs:
|
||||
|
||||
FinishCheck:
|
||||
if: ${{ !cancelled() }}
|
||||
needs: [RunConfig, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2, Tests_3]
|
||||
needs: [RunConfig, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2_ww, Tests_2]
|
||||
runs-on: [self-hosted, style-checker-aarch64]
|
||||
steps:
|
||||
- name: Check out repository code
|
||||
@ -133,6 +133,7 @@ jobs:
|
||||
cd "$GITHUB_WORKSPACE/tests/ci"
|
||||
python3 finish_check.py --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }}
|
||||
- name: Check Workflow results
|
||||
if: ${{ !cancelled() }}
|
||||
run: |
|
||||
export WORKFLOW_RESULT_FILE="/tmp/workflow_results.json"
|
||||
cat > "$WORKFLOW_RESULT_FILE" << 'EOF'
|
||||
|
20
.github/workflows/pull_request.yml
vendored
20
.github/workflows/pull_request.yml
vendored
@ -123,20 +123,20 @@ jobs:
|
||||
stage: Builds_2
|
||||
data: ${{ needs.RunConfig.outputs.data }}
|
||||
# stage for running non-required checks without being blocked by required checks (Test_1) if corresponding settings is selected
|
||||
Tests_2:
|
||||
Tests_2_ww:
|
||||
needs: [RunConfig, Builds_1]
|
||||
if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_2_ww') }}
|
||||
uses: ./.github/workflows/reusable_test_stage.yml
|
||||
with:
|
||||
stage: Tests_2_ww
|
||||
data: ${{ needs.RunConfig.outputs.data }}
|
||||
Tests_2:
|
||||
needs: [RunConfig, Builds_1, Tests_1]
|
||||
if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_2') }}
|
||||
uses: ./.github/workflows/reusable_test_stage.yml
|
||||
with:
|
||||
stage: Tests_2
|
||||
data: ${{ needs.RunConfig.outputs.data }}
|
||||
Tests_3:
|
||||
needs: [RunConfig, Builds_1, Tests_1]
|
||||
if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_3') }}
|
||||
uses: ./.github/workflows/reusable_test_stage.yml
|
||||
with:
|
||||
stage: Tests_3
|
||||
data: ${{ needs.RunConfig.outputs.data }}
|
||||
|
||||
################################# Reports #################################
|
||||
# Reports should run even if Builds_1/2 fail - run them separately (not in Tests_1/2/3)
|
||||
@ -154,7 +154,7 @@ jobs:
|
||||
if: ${{ !cancelled() }}
|
||||
# Test_2 or Test_3 do not have the jobs required for Mergeable check,
|
||||
# however, set them as "needs" to get all checks results before the automatic merge occurs.
|
||||
needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2, Tests_3]
|
||||
needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2_ww, Tests_2]
|
||||
runs-on: [self-hosted, style-checker-aarch64]
|
||||
steps:
|
||||
- name: Check out repository code
|
||||
@ -178,7 +178,7 @@ jobs:
|
||||
#
|
||||
FinishCheck:
|
||||
if: ${{ !failure() && !cancelled() }}
|
||||
needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2, Tests_3]
|
||||
needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2_ww, Tests_2]
|
||||
runs-on: [self-hosted, style-checker-aarch64]
|
||||
steps:
|
||||
- name: Check out repository code
|
||||
|
@ -54,7 +54,7 @@ CREATE TABLE keeper_map_table
|
||||
`v2` String,
|
||||
`v3` Float32
|
||||
)
|
||||
ENGINE = KeeperMap(/keeper_map_table, 4)
|
||||
ENGINE = KeeperMap('/keeper_map_table', 4)
|
||||
PRIMARY KEY key
|
||||
```
|
||||
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -13,10 +13,10 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
InterpolateDescription::InterpolateDescription(ActionsDAGPtr actions_, const Aliases & aliases)
|
||||
: actions(actions_)
|
||||
InterpolateDescription::InterpolateDescription(ActionsDAG actions_, const Aliases & aliases)
|
||||
: 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())
|
||||
required_columns_map[p->second->getColumnName()] = name_type;
|
||||
@ -24,7 +24,7 @@ namespace DB
|
||||
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;
|
||||
if (const auto & p = aliases.find(name); p != aliases.end())
|
||||
|
@ -5,21 +5,20 @@
|
||||
#include <string>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Interpreters/ActionsDAG.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ActionsDAG;
|
||||
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
|
||||
using Aliases = std::unordered_map<String, ASTPtr>;
|
||||
|
||||
/// Interpolate description
|
||||
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_set<std::string> result_columns_set; /// result block columns
|
||||
|
@ -186,7 +186,7 @@ class IColumn;
|
||||
M(Bool, allow_suspicious_ttl_expressions, false, "Reject TTL expressions that don't depend on any of table's columns. It indicates a user error most of the time.", 0) \
|
||||
M(Bool, allow_suspicious_variant_types, false, "In CREATE TABLE statement allows specifying Variant type with similar variant types (for example, with different numeric or date types). Enabling this setting may introduce some ambiguity when working with values with similar types.", 0) \
|
||||
M(Bool, allow_suspicious_primary_key, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)", 0) \
|
||||
M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \
|
||||
M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code. Due to a bug in the LLVM compiler infrastructure, on AArch64 machines, it is known to lead to a nullptr dereference and, consequently, server crash. Do not enable this setting.", 0) \
|
||||
M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \
|
||||
M(Bool, compile_aggregate_expressions, true, "Compile aggregate functions to native code.", 0) \
|
||||
M(UInt64, min_count_to_compile_aggregate_expression, 3, "The number of identical aggregate expressions before they are JIT-compiled", 0) \
|
||||
|
@ -810,6 +810,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
|
||||
{
|
||||
last_caller_id = FileSegment::getCallerId();
|
||||
|
||||
chassert(file_offset_of_buffer_end <= read_until_position);
|
||||
if (file_offset_of_buffer_end == read_until_position)
|
||||
return false;
|
||||
|
||||
@ -1051,7 +1052,11 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
|
||||
|
||||
if (download_current_segment && download_current_segment_succeeded)
|
||||
chassert(file_segment.getCurrentWriteOffset() >= file_offset_of_buffer_end);
|
||||
chassert(file_offset_of_buffer_end <= read_until_position);
|
||||
|
||||
chassert(
|
||||
file_offset_of_buffer_end <= read_until_position,
|
||||
fmt::format("Expected {} <= {} (size: {}, read range: {})",
|
||||
file_offset_of_buffer_end, read_until_position, size, current_read_range.toString()));
|
||||
}
|
||||
|
||||
swap(*implementation_buffer);
|
||||
|
@ -2,14 +2,12 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Interpreters/ActionsDAG.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ActionsDAG;
|
||||
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
|
||||
|
||||
/** The `indexHint` function takes any number of any arguments and always returns one.
|
||||
*
|
||||
* This function has a special meaning (see ExpressionAnalyzer, KeyCondition)
|
||||
@ -64,11 +62,11 @@ public:
|
||||
return DataTypeUInt8().createColumnConst(input_rows_count, 1u);
|
||||
}
|
||||
|
||||
void setActions(ActionsDAGPtr actions_) { actions = std::move(actions_); }
|
||||
const ActionsDAGPtr & getActions() const { return actions; }
|
||||
void setActions(ActionsDAG actions_) { actions = std::move(actions_); }
|
||||
const ActionsDAG & getActions() const { return actions; }
|
||||
|
||||
private:
|
||||
ActionsDAGPtr actions;
|
||||
ActionsDAG actions;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -624,9 +624,9 @@ void ActionsDAG::removeAliasesForFilter(const std::string & filter_name)
|
||||
}
|
||||
}
|
||||
|
||||
ActionsDAGPtr ActionsDAG::cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases)
|
||||
ActionsDAG ActionsDAG::cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases)
|
||||
{
|
||||
auto actions = std::make_shared<ActionsDAG>();
|
||||
ActionsDAG actions;
|
||||
std::unordered_map<const Node *, Node *> copy_map;
|
||||
|
||||
struct Frame
|
||||
@ -661,21 +661,21 @@ ActionsDAGPtr ActionsDAG::cloneSubDAG(const NodeRawConstPtrs & outputs, bool rem
|
||||
if (remove_aliases && frame.node->type == ActionType::ALIAS)
|
||||
copy_node = copy_map[frame.node->children.front()];
|
||||
else
|
||||
copy_node = &actions->nodes.emplace_back(*frame.node);
|
||||
copy_node = &actions.nodes.emplace_back(*frame.node);
|
||||
|
||||
if (frame.node->type == ActionType::INPUT)
|
||||
actions->inputs.push_back(copy_node);
|
||||
actions.inputs.push_back(copy_node);
|
||||
|
||||
stack.pop();
|
||||
}
|
||||
}
|
||||
|
||||
for (auto & node : actions->nodes)
|
||||
for (auto & node : actions.nodes)
|
||||
for (auto & child : node.children)
|
||||
child = copy_map[child];
|
||||
|
||||
for (const auto * output : outputs)
|
||||
actions->outputs.push_back(copy_map[output]);
|
||||
actions.outputs.push_back(copy_map[output]);
|
||||
|
||||
return 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 *> mapping;
|
||||
struct Frame
|
||||
@ -1003,9 +1003,9 @@ ActionsDAGPtr ActionsDAG::foldActionsByProjection(const std::unordered_map<const
|
||||
{
|
||||
bool should_rename = new_input->result_name != rename->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)
|
||||
mapped_input = &dag->addAlias(*mapped_input, new_input->result_name);
|
||||
mapped_input = &dag.addAlias(*mapped_input, new_input->result_name);
|
||||
}
|
||||
|
||||
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",
|
||||
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)
|
||||
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.
|
||||
const auto * mapped_output = mapping[output];
|
||||
if (output->result_name != mapped_output->result_name)
|
||||
mapped_output = &dag->addAlias(*mapped_output, output->result_name);
|
||||
dag->outputs.push_back(mapped_output);
|
||||
mapped_output = &dag.addAlias(*mapped_output, output->result_name);
|
||||
dag.outputs.push_back(mapped_output);
|
||||
}
|
||||
|
||||
return dag;
|
||||
@ -1246,27 +1246,31 @@ bool ActionsDAG::removeUnusedResult(const std::string & column_name)
|
||||
return true;
|
||||
}
|
||||
|
||||
ActionsDAGPtr ActionsDAG::clone() const
|
||||
ActionsDAG ActionsDAG::clone() const
|
||||
{
|
||||
auto actions = std::make_shared<ActionsDAG>();
|
||||
std::unordered_map<const Node *, Node *> old_to_new_nodes;
|
||||
return clone(old_to_new_nodes);
|
||||
}
|
||||
|
||||
std::unordered_map<const Node *, Node *> copy_map;
|
||||
ActionsDAG ActionsDAG::clone(std::unordered_map<const Node *, Node *> & old_to_new_nodes) const
|
||||
{
|
||||
ActionsDAG actions;
|
||||
|
||||
for (const auto & node : nodes)
|
||||
{
|
||||
auto & copy_node = actions->nodes.emplace_back(node);
|
||||
copy_map[&node] = ©_node;
|
||||
auto & copy_node = actions.nodes.emplace_back(node);
|
||||
old_to_new_nodes[&node] = ©_node;
|
||||
}
|
||||
|
||||
for (auto & node : actions->nodes)
|
||||
for (auto & node : actions.nodes)
|
||||
for (auto & child : node.children)
|
||||
child = copy_map[child];
|
||||
child = old_to_new_nodes[child];
|
||||
|
||||
for (const auto & output_node : outputs)
|
||||
actions->outputs.push_back(copy_map[output_node]);
|
||||
actions.outputs.push_back(old_to_new_nodes[output_node]);
|
||||
|
||||
for (const auto & input_node : inputs)
|
||||
actions->inputs.push_back(copy_map[input_node]);
|
||||
actions.inputs.push_back(old_to_new_nodes[input_node]);
|
||||
|
||||
return actions;
|
||||
}
|
||||
@ -1404,7 +1408,7 @@ const ActionsDAG::Node & ActionsDAG::materializeNode(const Node & node)
|
||||
return addAlias(*func, name);
|
||||
}
|
||||
|
||||
ActionsDAGPtr ActionsDAG::makeConvertingActions(
|
||||
ActionsDAG ActionsDAG::makeConvertingActions(
|
||||
const ColumnsWithTypeAndName & source,
|
||||
const ColumnsWithTypeAndName & result,
|
||||
MatchColumnsMode mode,
|
||||
@ -1421,7 +1425,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
|
||||
if (add_casted_columns && mode != MatchColumnsMode::Name)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Converting with add_casted_columns supported only for MatchColumnsMode::Name");
|
||||
|
||||
auto actions_dag = std::make_shared<ActionsDAG>(source);
|
||||
ActionsDAG actions_dag(source);
|
||||
NodeRawConstPtrs projection(num_result_columns);
|
||||
|
||||
FunctionOverloadResolverPtr func_builder_materialize = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionMaterialize>());
|
||||
@ -1429,9 +1433,9 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
|
||||
std::unordered_map<std::string_view, std::list<size_t>> inputs;
|
||||
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)
|
||||
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)
|
||||
@ -1444,7 +1448,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
|
||||
{
|
||||
case MatchColumnsMode::Position:
|
||||
{
|
||||
src_node = dst_node = actions_dag->inputs[result_col_num];
|
||||
src_node = dst_node = actions_dag.inputs[result_col_num];
|
||||
break;
|
||||
}
|
||||
|
||||
@ -1455,7 +1459,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
|
||||
{
|
||||
const auto * res_const = typeid_cast<const ColumnConst *>(res_elem.column.get());
|
||||
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
|
||||
throw Exception(ErrorCodes::THERE_IS_NO_COLUMN,
|
||||
"Cannot find column `{}` in source stream, there are only columns: [{}]",
|
||||
@ -1463,7 +1467,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
|
||||
}
|
||||
else
|
||||
{
|
||||
src_node = dst_node = actions_dag->inputs[input.front()];
|
||||
src_node = dst_node = actions_dag.inputs[input.front()];
|
||||
input.pop_front();
|
||||
}
|
||||
break;
|
||||
@ -1476,7 +1480,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
|
||||
if (const auto * src_const = typeid_cast<const ColumnConst *>(dst_node->column.get()))
|
||||
{
|
||||
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())
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
@ -1498,7 +1502,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
|
||||
column.column = DataTypeString().createColumnConst(0, column.name);
|
||||
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;
|
||||
|
||||
CastDiagnostic diagnostic = {dst_node->result_name, res_elem.name};
|
||||
@ -1506,13 +1510,13 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
|
||||
= createInternalCastOverloadResolver(CastType::nonAccurate, std::move(diagnostic));
|
||||
|
||||
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)))
|
||||
{
|
||||
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)
|
||||
@ -1531,7 +1535,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
|
||||
}
|
||||
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;
|
||||
}
|
||||
}
|
||||
@ -1541,36 +1545,36 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
|
||||
}
|
||||
}
|
||||
|
||||
actions_dag->outputs.swap(projection);
|
||||
actions_dag->removeUnusedActions(false);
|
||||
actions_dag.outputs.swap(projection);
|
||||
actions_dag.removeUnusedActions(false);
|
||||
|
||||
return actions_dag;
|
||||
}
|
||||
|
||||
ActionsDAGPtr ActionsDAG::makeAddingColumnActions(ColumnWithTypeAndName column)
|
||||
ActionsDAG ActionsDAG::makeAddingColumnActions(ColumnWithTypeAndName column)
|
||||
{
|
||||
auto adding_column_action = std::make_shared<ActionsDAG>();
|
||||
ActionsDAG adding_column_action;
|
||||
FunctionOverloadResolverPtr func_builder_materialize
|
||||
= std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionMaterialize>());
|
||||
|
||||
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};
|
||||
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 & 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));
|
||||
|
||||
adding_column_action->outputs.push_back(&alias_node);
|
||||
adding_column_action.outputs.push_back(&alias_node);
|
||||
return adding_column_action;
|
||||
}
|
||||
|
||||
ActionsDAGPtr ActionsDAG::merge(ActionsDAG && first, ActionsDAG && second)
|
||||
ActionsDAG ActionsDAG::merge(ActionsDAG && first, ActionsDAG && second)
|
||||
{
|
||||
first.mergeInplace(std::move(second));
|
||||
|
||||
/// Some actions could become unused. Do not drop inputs to preserve the header.
|
||||
first.removeUnusedActions(false);
|
||||
|
||||
return std::make_shared<ActionsDAG>(std::move(first));
|
||||
return std::move(first);
|
||||
}
|
||||
|
||||
void ActionsDAG::mergeInplace(ActionsDAG && second)
|
||||
@ -1963,15 +1967,15 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set<const Node *> split
|
||||
second_inputs.push_back(cur.to_second);
|
||||
}
|
||||
|
||||
auto first_actions = std::make_shared<ActionsDAG>();
|
||||
first_actions->nodes.swap(first_nodes);
|
||||
first_actions->outputs.swap(first_outputs);
|
||||
first_actions->inputs.swap(first_inputs);
|
||||
ActionsDAG first_actions;
|
||||
first_actions.nodes.swap(first_nodes);
|
||||
first_actions.outputs.swap(first_outputs);
|
||||
first_actions.inputs.swap(first_inputs);
|
||||
|
||||
auto second_actions = std::make_shared<ActionsDAG>();
|
||||
second_actions->nodes.swap(second_nodes);
|
||||
second_actions->outputs.swap(second_outputs);
|
||||
second_actions->inputs.swap(second_inputs);
|
||||
ActionsDAG second_actions;
|
||||
second_actions.nodes.swap(second_nodes);
|
||||
second_actions.outputs.swap(second_outputs);
|
||||
second_actions.inputs.swap(second_inputs);
|
||||
|
||||
std::unordered_map<const Node *, const Node *> split_nodes_mapping;
|
||||
if (create_split_nodes_mapping)
|
||||
@ -2091,7 +2095,7 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsBySortingDescription(const NameS
|
||||
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);
|
||||
if (!filter_node)
|
||||
@ -2115,7 +2119,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));
|
||||
}
|
||||
|
||||
ActionsDAGPtr filter_with_default_value_inputs;
|
||||
std::optional<ActionsDAG> filter_with_default_value_inputs;
|
||||
|
||||
try
|
||||
{
|
||||
@ -2297,12 +2301,12 @@ ColumnsWithTypeAndName prepareFunctionArguments(const ActionsDAG::NodeRawConstPt
|
||||
///
|
||||
/// Result actions add single column with conjunction result (it is always first in outputs).
|
||||
/// 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())
|
||||
return nullptr;
|
||||
return {};
|
||||
|
||||
auto actions = std::make_shared<ActionsDAG>();
|
||||
ActionsDAG actions;
|
||||
|
||||
FunctionOverloadResolverPtr func_builder_and = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>());
|
||||
|
||||
@ -2343,7 +2347,7 @@ ActionsDAGPtr ActionsDAG::createActionsForConjunction(NodeRawConstPtrs conjuncti
|
||||
|
||||
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;
|
||||
|
||||
for (auto & child : node.children)
|
||||
@ -2366,33 +2370,33 @@ ActionsDAGPtr ActionsDAG::createActionsForConjunction(NodeRawConstPtrs conjuncti
|
||||
for (const auto * predicate : conjunction)
|
||||
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)
|
||||
{
|
||||
const Node * input;
|
||||
auto & list = required_inputs[col.name];
|
||||
if (list.empty())
|
||||
input = &actions->addInput(col);
|
||||
input = &actions.addInput(col);
|
||||
else
|
||||
{
|
||||
input = list.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.
|
||||
if (input->result_name != result_predicate->result_name)
|
||||
actions->outputs.push_back(input);
|
||||
actions.outputs.push_back(input);
|
||||
}
|
||||
|
||||
return actions;
|
||||
}
|
||||
|
||||
ActionsDAGPtr ActionsDAG::splitActionsForFilterPushDown(
|
||||
std::optional<ActionsDAG> ActionsDAG::splitActionsForFilterPushDown(
|
||||
const std::string & filter_name,
|
||||
bool removes_filter,
|
||||
const Names & available_inputs,
|
||||
@ -2408,7 +2412,7 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilterPushDown(
|
||||
/// If condition is constant let's do nothing.
|
||||
/// It means there is nothing to push down or optimization was already applied.
|
||||
if (predicate->type == ActionType::COLUMN)
|
||||
return nullptr;
|
||||
return {};
|
||||
|
||||
std::unordered_set<const Node *> allowed_nodes;
|
||||
|
||||
@ -2432,7 +2436,7 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilterPushDown(
|
||||
auto conjunction = getConjunctionNodes(predicate, allowed_nodes);
|
||||
|
||||
if (conjunction.allowed.empty())
|
||||
return nullptr;
|
||||
return {};
|
||||
|
||||
chassert(predicate->result_type);
|
||||
|
||||
@ -2444,13 +2448,13 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilterPushDown(
|
||||
&& !conjunction.rejected.front()->result_type->equals(*predicate->result_type))
|
||||
{
|
||||
/// No further optimization can be done
|
||||
return nullptr;
|
||||
return {};
|
||||
}
|
||||
}
|
||||
|
||||
auto actions = createActionsForConjunction(conjunction.allowed, all_inputs);
|
||||
if (!actions)
|
||||
return nullptr;
|
||||
return {};
|
||||
|
||||
/// Now, when actions are created, update the current DAG.
|
||||
removeUnusedConjunctions(std::move(conjunction.rejected), predicate, removes_filter);
|
||||
@ -2555,11 +2559,11 @@ ActionsDAG::ActionsForJOINFilterPushDown ActionsDAG::splitActionsForJOINFilterPu
|
||||
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 replace_equivalent_columns_in_filter = [](const ActionsDAGPtr & filter,
|
||||
auto replace_equivalent_columns_in_filter = [](const ActionsDAG & filter,
|
||||
const Block & stream_header,
|
||||
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);
|
||||
|
||||
/** If result filter to left or right stream has column that is one of the stream inputs, we need distinguish filter column from
|
||||
@ -2580,7 +2584,7 @@ ActionsDAG::ActionsForJOINFilterPushDown ActionsDAG::splitActionsForJOINFilterPu
|
||||
for (const auto & input : updated_filter->getInputs())
|
||||
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))
|
||||
continue;
|
||||
@ -2618,12 +2622,12 @@ ActionsDAG::ActionsForJOINFilterPushDown ActionsDAG::splitActionsForJOINFilterPu
|
||||
};
|
||||
|
||||
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,
|
||||
equivalent_right_stream_column_to_left_stream_column);
|
||||
|
||||
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,
|
||||
equivalent_left_stream_column_to_right_stream_column);
|
||||
|
||||
@ -2852,13 +2856,13 @@ bool ActionsDAG::isSortingPreserved(
|
||||
return true;
|
||||
}
|
||||
|
||||
ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
|
||||
std::optional<ActionsDAG> ActionsDAG::buildFilterActionsDAG(
|
||||
const NodeRawConstPtrs & filter_nodes,
|
||||
const std::unordered_map<std::string, ColumnWithTypeAndName> & node_name_to_input_node_column,
|
||||
bool single_output_condition_node)
|
||||
{
|
||||
if (filter_nodes.empty())
|
||||
return nullptr;
|
||||
return {};
|
||||
|
||||
struct Frame
|
||||
{
|
||||
@ -2866,7 +2870,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
|
||||
bool visited_children = false;
|
||||
};
|
||||
|
||||
auto result_dag = std::make_shared<ActionsDAG>();
|
||||
ActionsDAG result_dag;
|
||||
std::unordered_map<std::string, const ActionsDAG::Node *> result_inputs;
|
||||
std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> node_to_result_node;
|
||||
|
||||
@ -2897,7 +2901,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
|
||||
{
|
||||
auto & result_input = result_inputs[input_node_it->second.name];
|
||||
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);
|
||||
nodes_to_process.pop_back();
|
||||
@ -2924,25 +2928,25 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
|
||||
{
|
||||
auto & result_input = result_inputs[node->result_name];
|
||||
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;
|
||||
break;
|
||||
}
|
||||
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;
|
||||
}
|
||||
case ActionsDAG::ActionType::ALIAS:
|
||||
{
|
||||
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;
|
||||
}
|
||||
case ActionsDAG::ActionType::ARRAY_JOIN:
|
||||
{
|
||||
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;
|
||||
}
|
||||
case ActionsDAG::ActionType::FUNCTION:
|
||||
@ -2960,13 +2964,11 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
|
||||
{
|
||||
if (const auto * index_hint = typeid_cast<const FunctionIndexHint *>(adaptor->getFunction().get()))
|
||||
{
|
||||
ActionsDAGPtr index_hint_filter_dag;
|
||||
const auto & index_hint_args = index_hint->getActions()->getOutputs();
|
||||
ActionsDAG index_hint_filter_dag;
|
||||
const auto & index_hint_args = index_hint->getActions().getOutputs();
|
||||
|
||||
if (index_hint_args.empty())
|
||||
index_hint_filter_dag = std::make_shared<ActionsDAG>();
|
||||
else
|
||||
index_hint_filter_dag = buildFilterActionsDAG(index_hint_args,
|
||||
if (!index_hint_args.empty())
|
||||
index_hint_filter_dag = *buildFilterActionsDAG(index_hint_args,
|
||||
node_name_to_input_node_column,
|
||||
false /*single_output_condition_node*/);
|
||||
|
||||
@ -2988,7 +2990,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
|
||||
auto [arguments, all_const] = getFunctionArguments(function_children);
|
||||
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,
|
||||
std::move(function_children),
|
||||
std::move(arguments),
|
||||
@ -3003,7 +3005,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
|
||||
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);
|
||||
|
||||
for (const auto & node : filter_nodes)
|
||||
@ -3012,7 +3014,7 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
|
||||
if (result_dag_outputs.size() > 1 && single_output_condition_node)
|
||||
{
|
||||
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;
|
||||
@ -3108,10 +3110,9 @@ ActionsDAG::NodeRawConstPtrs ActionsDAG::filterNodesByAllowedInputs(
|
||||
return nodes;
|
||||
}
|
||||
|
||||
FindOriginalNodeForOutputName::FindOriginalNodeForOutputName(const ActionsDAGPtr & actions_)
|
||||
:actions(actions_)
|
||||
FindOriginalNodeForOutputName::FindOriginalNodeForOutputName(const ActionsDAG & actions_)
|
||||
{
|
||||
const auto & actions_outputs = actions->getOutputs();
|
||||
const auto & actions_outputs = actions_.getOutputs();
|
||||
for (const auto * output_node : actions_outputs)
|
||||
{
|
||||
/// find input node which refers to the output node
|
||||
@ -3147,10 +3148,9 @@ const ActionsDAG::Node * FindOriginalNodeForOutputName::find(const String & outp
|
||||
return it->second;
|
||||
}
|
||||
|
||||
FindAliasForInputName::FindAliasForInputName(const ActionsDAGPtr & actions_)
|
||||
:actions(actions_)
|
||||
FindAliasForInputName::FindAliasForInputName(const ActionsDAG & actions_)
|
||||
{
|
||||
const auto & actions_outputs = actions->getOutputs();
|
||||
const auto & actions_outputs = actions_.getOutputs();
|
||||
for (const auto * output_node : actions_outputs)
|
||||
{
|
||||
/// find input node which corresponds to alias
|
||||
|
@ -11,9 +11,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ActionsDAG;
|
||||
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
|
||||
|
||||
class IExecutableFunction;
|
||||
using ExecutableFunctionPtr = std::shared_ptr<IExecutableFunction>;
|
||||
|
||||
@ -247,7 +244,7 @@ public:
|
||||
/// c * d e
|
||||
/// \ /
|
||||
/// c * d - e
|
||||
static ActionsDAGPtr foldActionsByProjection(
|
||||
static ActionsDAG foldActionsByProjection(
|
||||
const std::unordered_map<const Node *, const Node *> & new_inputs,
|
||||
const NodeRawConstPtrs & required_outputs);
|
||||
|
||||
@ -261,9 +258,10 @@ public:
|
||||
void compileExpressions(size_t min_count_to_compile_expression, const std::unordered_set<const Node *> & lazy_executed_nodes = {});
|
||||
#endif
|
||||
|
||||
ActionsDAGPtr clone() const;
|
||||
ActionsDAG clone(std::unordered_map<const Node *, Node *> & old_to_new_nodes) const;
|
||||
ActionsDAG clone() const;
|
||||
|
||||
static ActionsDAGPtr cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases);
|
||||
static ActionsDAG cloneSubDAG(const NodeRawConstPtrs & outputs, bool remove_aliases);
|
||||
|
||||
/// Execute actions for header. Input block must have empty columns.
|
||||
/// Result should be equal to the execution of ExpressionActions built from this DAG.
|
||||
@ -301,7 +299,7 @@ public:
|
||||
/// @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 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 & result,
|
||||
MatchColumnsMode mode,
|
||||
@ -310,13 +308,13 @@ public:
|
||||
NameToNameMap * new_names = nullptr);
|
||||
|
||||
/// 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.
|
||||
/// 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`.
|
||||
/// 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);
|
||||
/// Invariant : no nodes are removed from the first (this) DAG.
|
||||
@ -327,12 +325,7 @@ public:
|
||||
/// *out_outputs is filled with pointers to the nodes corresponding to second.getOutputs().
|
||||
void mergeNodes(ActionsDAG && second, NodeRawConstPtrs * out_outputs = nullptr);
|
||||
|
||||
struct SplitResult
|
||||
{
|
||||
ActionsDAGPtr first;
|
||||
ActionsDAGPtr second;
|
||||
std::unordered_map<const Node *, const Node *> split_nodes_mapping;
|
||||
};
|
||||
struct SplitResult;
|
||||
|
||||
/// 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.
|
||||
@ -360,7 +353,7 @@ public:
|
||||
* @param filter_name - name of filter node in current DAG.
|
||||
* @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.
|
||||
/// If nothing may be calculated, returns nullptr.
|
||||
@ -379,19 +372,13 @@ public:
|
||||
/// 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,
|
||||
/// 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,
|
||||
bool removes_filter,
|
||||
const Names & available_inputs,
|
||||
const ColumnsWithTypeAndName & all_inputs);
|
||||
|
||||
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;
|
||||
};
|
||||
struct ActionsForJOINFilterPushDown;
|
||||
|
||||
/** Split actions for JOIN filter push down.
|
||||
*
|
||||
@ -438,7 +425,7 @@ public:
|
||||
*
|
||||
* 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 std::unordered_map<std::string, ColumnWithTypeAndName> & node_name_to_input_node_column = {},
|
||||
bool single_output_condition_node = true);
|
||||
@ -470,21 +457,35 @@ private:
|
||||
void compileFunctions(size_t min_count_to_compile_expression, const std::unordered_set<const Node *> & lazy_executed_nodes = {});
|
||||
#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);
|
||||
};
|
||||
|
||||
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
|
||||
{
|
||||
using NameToNodeIndex = std::unordered_map<std::string_view, const ActionsDAG::Node *>;
|
||||
|
||||
public:
|
||||
explicit FindOriginalNodeForOutputName(const ActionsDAGPtr & actions);
|
||||
explicit FindOriginalNodeForOutputName(const ActionsDAG & actions);
|
||||
const ActionsDAG::Node * find(const String & output_name);
|
||||
|
||||
private:
|
||||
ActionsDAGPtr actions;
|
||||
NameToNodeIndex index;
|
||||
};
|
||||
|
||||
@ -493,11 +494,10 @@ class FindAliasForInputName
|
||||
using NameToNodeIndex = std::unordered_map<std::string_view, const ActionsDAG::Node *>;
|
||||
|
||||
public:
|
||||
explicit FindAliasForInputName(const ActionsDAGPtr & actions);
|
||||
explicit FindAliasForInputName(const ActionsDAG & actions);
|
||||
const ActionsDAG::Node * find(const String & name);
|
||||
|
||||
private:
|
||||
ActionsDAGPtr actions;
|
||||
NameToNodeIndex index;
|
||||
};
|
||||
|
||||
|
@ -1024,7 +1024,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
||||
dag.project(args);
|
||||
|
||||
auto index_hint = std::make_shared<FunctionIndexHint>();
|
||||
index_hint->setActions(std::make_shared<ActionsDAG>(std::move(dag)));
|
||||
index_hint->setActions(std::move(dag));
|
||||
|
||||
// Arguments are removed. We add function instead of constant column to avoid constant folding.
|
||||
data.addFunction(std::make_unique<FunctionToOverloadResolverAdaptor>(index_hint), {}, column_name);
|
||||
@ -1287,7 +1287,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
||||
lambda_dag.removeUnusedActions(Names(1, result_name));
|
||||
|
||||
auto lambda_actions = std::make_shared<ExpressionActions>(
|
||||
std::make_shared<ActionsDAG>(std::move(lambda_dag)),
|
||||
std::move(lambda_dag),
|
||||
ExpressionActionsSettings::fromContext(data.getContext(), CompileExpressions::yes));
|
||||
|
||||
DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type;
|
||||
|
@ -18,12 +18,6 @@ namespace DB
|
||||
class ASTExpressionList;
|
||||
class ASTFunction;
|
||||
|
||||
class ExpressionActions;
|
||||
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
||||
|
||||
class ActionsDAG;
|
||||
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
|
||||
|
||||
class IFunctionOverloadResolver;
|
||||
using FunctionOverloadResolverPtr = std::shared_ptr<IFunctionOverloadResolver>;
|
||||
|
||||
@ -32,7 +26,7 @@ FutureSetPtr makeExplicitSet(
|
||||
const ASTFunction * node, const ActionsDAG & actions, ContextPtr context, PreparedSets & prepared_sets);
|
||||
|
||||
/** For ActionsVisitor
|
||||
* A stack of ExpressionActions corresponding to nested lambda expressions.
|
||||
* A stack of ActionsDAG corresponding to nested lambda expressions.
|
||||
* The new action should be added to the highest possible level.
|
||||
* For example, in the expression "select arrayMap(x -> x + column1 * column2, array1)"
|
||||
* calculation of the product must be done outside the lambda expression (it does not depend on x),
|
||||
|
@ -49,18 +49,17 @@ namespace ErrorCodes
|
||||
|
||||
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_)
|
||||
: project_inputs(project_inputs_)
|
||||
ExpressionActions::ExpressionActions(ActionsDAG actions_dag_, const ExpressionActionsSettings & settings_, bool project_inputs_)
|
||||
: actions_dag(std::move(actions_dag_))
|
||||
, project_inputs(project_inputs_)
|
||||
, settings(settings_)
|
||||
{
|
||||
actions_dag = actions_dag_->clone();
|
||||
|
||||
/// 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 (settings.can_compile_expressions && settings.compile_expressions == CompileExpressions::yes)
|
||||
actions_dag->compileExpressions(settings.min_count_to_compile_expression, lazy_executed_nodes);
|
||||
actions_dag.compileExpressions(settings.min_count_to_compile_expression, lazy_executed_nodes);
|
||||
#endif
|
||||
|
||||
linearizeActions(lazy_executed_nodes);
|
||||
@ -68,12 +67,32 @@ ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_, const Expressio
|
||||
if (settings.max_temporary_columns && num_columns > settings.max_temporary_columns)
|
||||
throw Exception(ErrorCodes::TOO_MANY_TEMPORARY_COLUMNS,
|
||||
"Too many temporary columns: {}. Maximum: {}",
|
||||
actions_dag->dumpNames(), settings.max_temporary_columns);
|
||||
actions_dag.dumpNames(), settings.max_temporary_columns);
|
||||
}
|
||||
|
||||
ExpressionActionsPtr ExpressionActions::clone() const
|
||||
{
|
||||
return std::make_shared<ExpressionActions>(*this);
|
||||
auto copy = std::make_shared<ExpressionActions>(ExpressionActions());
|
||||
|
||||
std::unordered_map<const Node *, Node *> copy_map;
|
||||
copy->actions_dag = actions_dag.clone(copy_map);
|
||||
copy->actions = actions;
|
||||
for (auto & action : copy->actions)
|
||||
action.node = copy_map[action.node];
|
||||
|
||||
for (const auto * input : copy->actions_dag.getInputs())
|
||||
copy->input_positions.emplace(input->result_name, input_positions.at(input->result_name));
|
||||
|
||||
copy->num_columns = num_columns;
|
||||
|
||||
copy->required_columns = required_columns;
|
||||
copy->result_positions = result_positions;
|
||||
copy->sample_block = sample_block;
|
||||
|
||||
copy->project_inputs = project_inputs;
|
||||
copy->settings = settings;
|
||||
|
||||
return copy;
|
||||
}
|
||||
|
||||
namespace
|
||||
@ -341,8 +360,8 @@ void ExpressionActions::linearizeActions(const std::unordered_set<const ActionsD
|
||||
};
|
||||
|
||||
const auto & nodes = getNodes();
|
||||
const auto & outputs = actions_dag->getOutputs();
|
||||
const auto & inputs = actions_dag->getInputs();
|
||||
const auto & outputs = actions_dag.getOutputs();
|
||||
const auto & inputs = actions_dag.getInputs();
|
||||
|
||||
auto reverse_info = getActionsDAGReverseInfo(nodes, outputs);
|
||||
std::vector<Data> data;
|
||||
|
@ -70,7 +70,7 @@ public:
|
||||
using NameToInputMap = std::unordered_map<std::string_view, std::list<size_t>>;
|
||||
|
||||
private:
|
||||
ActionsDAGPtr actions_dag;
|
||||
ActionsDAG actions_dag;
|
||||
Actions actions;
|
||||
size_t num_columns = 0;
|
||||
|
||||
@ -84,14 +84,13 @@ private:
|
||||
ExpressionActionsSettings settings;
|
||||
|
||||
public:
|
||||
ExpressionActions() = delete;
|
||||
explicit ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings_ = {}, bool project_inputs_ = false);
|
||||
ExpressionActions(const ExpressionActions &) = default;
|
||||
ExpressionActions & operator=(const ExpressionActions &) = default;
|
||||
explicit ExpressionActions(ActionsDAG actions_dag_, const ExpressionActionsSettings & settings_ = {}, bool project_inputs_ = false);
|
||||
ExpressionActions(ExpressionActions &&) = default;
|
||||
ExpressionActions & operator=(ExpressionActions &&) = default;
|
||||
|
||||
const Actions & getActions() const { return actions; }
|
||||
const std::list<Node> & getNodes() const { return actions_dag->getNodes(); }
|
||||
const ActionsDAG & getActionsDAG() const { return *actions_dag; }
|
||||
const std::list<Node> & getNodes() const { return actions_dag.getNodes(); }
|
||||
const ActionsDAG & getActionsDAG() const { return actions_dag; }
|
||||
const ColumnNumbers & getResultPositions() const { return result_positions; }
|
||||
const ExpressionActionsSettings & getSettings() const { return settings; }
|
||||
|
||||
@ -131,6 +130,7 @@ public:
|
||||
ExpressionActionsPtr clone() const;
|
||||
|
||||
private:
|
||||
ExpressionActions() = default;
|
||||
void checkLimits(const ColumnsWithTypeAndName & columns) const;
|
||||
|
||||
void linearizeActions(const std::unordered_set<const Node *> & lazy_executed_nodes);
|
||||
|
@ -658,7 +658,7 @@ void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_,
|
||||
with_alias->getColumnName(), 1 /* direction */,
|
||||
1 /* nulls_direction */));
|
||||
|
||||
auto actions_dag = std::make_shared<ActionsDAG>(aggregated_columns);
|
||||
auto actions_dag = std::make_unique<ActionsDAG>(aggregated_columns);
|
||||
getRootActions(column_ast, false, *actions_dag);
|
||||
desc.partition_by_actions.push_back(std::move(actions_dag));
|
||||
}
|
||||
@ -679,7 +679,7 @@ void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_,
|
||||
order_by_element.direction,
|
||||
order_by_element.nulls_direction));
|
||||
|
||||
auto actions_dag = std::make_shared<ActionsDAG>(aggregated_columns);
|
||||
auto actions_dag = std::make_unique<ActionsDAG>(aggregated_columns);
|
||||
getRootActions(column_ast, false, *actions_dag);
|
||||
desc.order_by_actions.push_back(std::move(actions_dag));
|
||||
}
|
||||
@ -823,13 +823,14 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAG & actions)
|
||||
makeWindowDescriptionFromAST(*current_context, window_descriptions,
|
||||
desc, &definition);
|
||||
|
||||
auto full_sort_description = desc.full_sort_description;
|
||||
|
||||
auto [it, inserted] = window_descriptions.insert(
|
||||
{default_window_name, desc});
|
||||
{default_window_name, std::move(desc)});
|
||||
|
||||
if (!inserted)
|
||||
{
|
||||
assert(it->second.full_sort_description
|
||||
== desc.full_sort_description);
|
||||
assert(it->second.full_sort_description == full_sort_description);
|
||||
}
|
||||
|
||||
it->second.window_functions.push_back(window_function);
|
||||
@ -927,7 +928,7 @@ JoinPtr SelectQueryExpressionAnalyzer::appendJoin(
|
||||
{
|
||||
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);
|
||||
|
||||
if (converting_actions)
|
||||
@ -1039,7 +1040,7 @@ static std::unique_ptr<QueryPlan> buildJoinedPlan(
|
||||
/// Actions which need to be calculated on joined block.
|
||||
auto joined_block_actions = analyzed_join.createJoinedBlockActions(context);
|
||||
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;
|
||||
for (auto & pr : required_columns_with_aliases)
|
||||
@ -1060,17 +1061,17 @@ static std::unique_ptr<QueryPlan> buildJoinedPlan(
|
||||
interpreter->buildQueryPlan(*joined_plan);
|
||||
{
|
||||
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)
|
||||
{
|
||||
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);
|
||||
const auto & alias = rename_dag->addAlias(*rename_dag->getInputs()[pos], name_with_alias.second);
|
||||
rename_dag->getOutputs()[pos] = &alias;
|
||||
const auto & alias = rename_dag.addAlias(*rename_dag.getInputs()[pos], name_with_alias.second);
|
||||
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));
|
||||
rename_step->setStepDescription("Rename joined columns");
|
||||
joined_plan->addStep(std::move(rename_step));
|
||||
@ -1130,14 +1131,14 @@ std::shared_ptr<DirectKeyValueJoin> tryKeyValueJoin(std::shared_ptr<TableJoin> a
|
||||
JoinPtr SelectQueryExpressionAnalyzer::makeJoin(
|
||||
const ASTTablesInSelectQueryElement & join_element,
|
||||
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.
|
||||
|
||||
if (joined_plan)
|
||||
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;
|
||||
|
||||
@ -1145,7 +1146,7 @@ JoinPtr SelectQueryExpressionAnalyzer::makeJoin(
|
||||
{
|
||||
auto joined_block_actions = analyzed_join->createJoinedBlockActions(getContext());
|
||||
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;
|
||||
for (auto & pr : required_columns_with_aliases)
|
||||
@ -1162,7 +1163,7 @@ JoinPtr SelectQueryExpressionAnalyzer::makeJoin(
|
||||
std::tie(left_convert_actions, right_convert_actions) = analyzed_join->createConvertingActions(left_columns, right_columns);
|
||||
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");
|
||||
joined_plan->addStep(std::move(converting_step));
|
||||
}
|
||||
@ -1354,10 +1355,10 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain
|
||||
{
|
||||
for (auto & child : asts)
|
||||
{
|
||||
auto actions_dag = std::make_shared<ActionsDAG>(columns_after_join);
|
||||
getRootActions(child, only_types, *actions_dag);
|
||||
ActionsDAG actions_dag(columns_after_join);
|
||||
getRootActions(child, only_types, actions_dag);
|
||||
group_by_elements_actions.emplace_back(
|
||||
std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)));
|
||||
std::make_shared<ExpressionActions>(std::move(actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)));
|
||||
}
|
||||
}
|
||||
|
||||
@ -1471,7 +1472,7 @@ void SelectQueryExpressionAnalyzer::appendGroupByModifiers(ActionsDAG & before_a
|
||||
ExpressionActionsChain::Step & step = chain.addStep(before_aggregation.getNamesAndTypesList());
|
||||
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)
|
||||
@ -1607,10 +1608,10 @@ ActionsAndProjectInputsFlagPtr SelectQueryExpressionAnalyzer::appendOrderBy(Expr
|
||||
{
|
||||
for (const auto & child : select_query->orderBy()->children)
|
||||
{
|
||||
auto actions_dag = std::make_shared<ActionsDAG>(columns_after_join);
|
||||
getRootActions(child, only_types, *actions_dag);
|
||||
ActionsDAG actions_dag(columns_after_join);
|
||||
getRootActions(child, only_types, actions_dag);
|
||||
order_by_elements_actions.emplace_back(
|
||||
std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)));
|
||||
std::make_shared<ExpressionActions>(std::move(actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)));
|
||||
}
|
||||
}
|
||||
|
||||
@ -1737,7 +1738,7 @@ void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const
|
||||
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);
|
||||
NamesWithAliases result_columns;
|
||||
@ -1789,7 +1790,7 @@ ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool remove_un
|
||||
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)
|
||||
@ -1798,17 +1799,17 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool remov
|
||||
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_shared<ActionsDAG>(constant_inputs);
|
||||
getRootActions(query, true /* no_makeset_for_subqueries */, *actions, true /* only_consts */);
|
||||
ActionsDAG actions(constant_inputs);
|
||||
getRootActions(query, true /* no_makeset_for_subqueries */, actions, true /* only_consts */);
|
||||
return actions;
|
||||
}
|
||||
|
||||
ExpressionActionsPtr ExpressionAnalyzer::getConstActions(const ColumnsWithTypeAndName & constant_inputs)
|
||||
{
|
||||
auto actions = getConstActionsDAG(constant_inputs);
|
||||
return std::make_shared<ExpressionActions>(actions, ExpressionActionsSettings::fromContext(getContext()));
|
||||
return std::make_shared<ExpressionActions>(std::move(actions), ExpressionActionsSettings::fromContext(getContext()));
|
||||
}
|
||||
|
||||
std::unique_ptr<QueryPlan> SelectQueryExpressionAnalyzer::getJoinedPlan()
|
||||
@ -1879,8 +1880,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
|
||||
|
||||
if (prewhere_dag_and_flags)
|
||||
{
|
||||
auto dag = std::make_shared<ActionsDAG>(std::move(prewhere_dag_and_flags->dag));
|
||||
prewhere_info = std::make_shared<PrewhereInfo>(std::move(dag), query.prewhere()->getColumnName());
|
||||
prewhere_info = std::make_shared<PrewhereInfo>(std::move(prewhere_dag_and_flags->dag), query.prewhere()->getColumnName());
|
||||
prewhere_dag_and_flags.reset();
|
||||
}
|
||||
|
||||
@ -1923,7 +1923,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
|
||||
|
||||
if (storage && additional_filter)
|
||||
{
|
||||
Names columns_for_additional_filter = additional_filter->actions->getRequiredColumnsNames();
|
||||
Names columns_for_additional_filter = additional_filter->actions.getRequiredColumnsNames();
|
||||
additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(),
|
||||
columns_for_additional_filter.begin(), columns_for_additional_filter.end());
|
||||
}
|
||||
@ -1944,10 +1944,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
|
||||
Block before_prewhere_sample = source_header;
|
||||
if (sanitizeBlock(before_prewhere_sample))
|
||||
{
|
||||
auto dag = prewhere_dag_and_flags->dag.clone();
|
||||
ExpressionActions(
|
||||
dag,
|
||||
ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_prewhere_sample);
|
||||
before_prewhere_sample = prewhere_dag_and_flags->dag.updateHeader(before_prewhere_sample);
|
||||
auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName());
|
||||
/// If the filter column is a constant, record it.
|
||||
if (column_elem.column)
|
||||
@ -1979,9 +1976,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
|
||||
before_where_sample = source_header;
|
||||
if (sanitizeBlock(before_where_sample))
|
||||
{
|
||||
ExpressionActions(
|
||||
before_where->dag.clone(),
|
||||
ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_where_sample);
|
||||
before_where_sample = before_where->dag.updateHeader(before_where_sample);
|
||||
|
||||
auto & column_elem
|
||||
= before_where_sample.getByName(query.where()->getColumnName());
|
||||
@ -2054,7 +2049,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
|
||||
|
||||
auto & step = chain.lastStep(query_analyzer.aggregated_columns);
|
||||
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,12 +2230,11 @@ void ExpressionAnalysisResult::checkActions() const
|
||||
/// Check that PREWHERE doesn't contain unusual actions. Unusual actions are that can change number of rows.
|
||||
if (hasPrewhere())
|
||||
{
|
||||
auto check_actions = [](const ActionsDAGPtr & actions)
|
||||
auto check_actions = [](ActionsDAG & actions)
|
||||
{
|
||||
if (actions)
|
||||
for (const auto & node : actions->getNodes())
|
||||
if (node.type == ActionsDAG::ActionType::ARRAY_JOIN)
|
||||
throw Exception(ErrorCodes::ILLEGAL_PREWHERE, "PREWHERE cannot contain ARRAY JOIN action");
|
||||
for (const auto & node : actions.getNodes())
|
||||
if (node.type == ActionsDAG::ActionType::ARRAY_JOIN)
|
||||
throw Exception(ErrorCodes::ILLEGAL_PREWHERE, "PREWHERE cannot contain ARRAY JOIN action");
|
||||
};
|
||||
|
||||
check_actions(prewhere_info->prewhere_actions);
|
||||
|
@ -38,9 +38,6 @@ using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
|
||||
class ArrayJoinAction;
|
||||
using ArrayJoinActionPtr = std::shared_ptr<ArrayJoinAction>;
|
||||
|
||||
class ActionsDAG;
|
||||
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
|
||||
|
||||
/// Create columns in block or return false if not possible
|
||||
bool sanitizeBlock(Block & block, bool throw_if_cannot_create_column = false);
|
||||
|
||||
@ -117,12 +114,12 @@ public:
|
||||
/// 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.
|
||||
/// 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);
|
||||
|
||||
/// Get actions to evaluate a constant expression. The function adds constants and applies functions that depend only on constants.
|
||||
/// Does not execute subqueries.
|
||||
ActionsDAGPtr getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs = {});
|
||||
ActionsDAG getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs = {});
|
||||
ExpressionActionsPtr getConstActions(const ColumnsWithTypeAndName & constant_inputs = {});
|
||||
|
||||
/** Sets that require a subquery to be create.
|
||||
@ -367,7 +364,7 @@ private:
|
||||
JoinPtr makeJoin(
|
||||
const ASTTablesInSelectQueryElement & join_element,
|
||||
const ColumnsWithTypeAndName & left_columns,
|
||||
ActionsDAGPtr & left_convert_actions);
|
||||
std::optional<ActionsDAG> & left_convert_actions);
|
||||
|
||||
const ASTSelectQuery * getAggregatingQuery() const;
|
||||
|
||||
|
@ -296,7 +296,7 @@ private:
|
||||
{
|
||||
auto joined_block_actions = data.table_join->createJoinedBlockActions(data.getContext());
|
||||
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)
|
||||
required_columns.push_back(pr.first);
|
||||
|
@ -124,16 +124,16 @@ static ASTPtr parseAdditionalPostFilter(const Context & context)
|
||||
"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());
|
||||
String result_column_name = ast->getColumnName();
|
||||
auto dag = ExpressionAnalyzer(ast, syntax_result, context).getActionsDAG(false, false);
|
||||
const ActionsDAG::Node * result_node = &dag->findInOutputs(result_column_name);
|
||||
auto & outputs = dag->getOutputs();
|
||||
const ActionsDAG::Node * result_node = &dag.findInOutputs(result_column_name);
|
||||
auto & outputs = dag.getOutputs();
|
||||
outputs.clear();
|
||||
outputs.reserve(dag->getInputs().size() + 1);
|
||||
for (const auto * node : dag->getInputs())
|
||||
outputs.reserve(dag.getInputs().size() + 1);
|
||||
for (const auto * node : dag.getInputs())
|
||||
outputs.push_back(node);
|
||||
|
||||
outputs.push_back(result_node);
|
||||
@ -151,7 +151,7 @@ void IInterpreterUnionOrSelectQuery::addAdditionalPostFilter(QueryPlan & plan) c
|
||||
return;
|
||||
|
||||
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>(
|
||||
plan.getCurrentDataStream(), std::move(dag), std::move(filter_name), true);
|
||||
filter_step->setStepDescription("Additional result filter");
|
||||
|
@ -390,7 +390,7 @@ Chain InterpreterInsertQuery::buildPreSinkChain(
|
||||
context_ptr,
|
||||
null_as_default);
|
||||
|
||||
auto adding_missing_defaults_actions = std::make_shared<ExpressionActions>(adding_missing_defaults_dag);
|
||||
auto adding_missing_defaults_actions = std::make_shared<ExpressionActions>(std::move(adding_missing_defaults_dag));
|
||||
|
||||
/// Actually we don't know structure of input blocks from query/table,
|
||||
/// because some clients break insertion protocol (columns != header)
|
||||
@ -536,7 +536,7 @@ QueryPipeline InterpreterInsertQuery::buildInsertSelectPipeline(ASTInsertQuery &
|
||||
pipeline.getHeader().getColumnsWithTypeAndName(),
|
||||
query_sample_block.getColumnsWithTypeAndName(),
|
||||
ActionsDAG::MatchColumnsMode::Position);
|
||||
auto actions = std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes));
|
||||
auto actions = std::make_shared<ExpressionActions>(std::move(actions_dag), ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes));
|
||||
|
||||
pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr
|
||||
{
|
||||
|
@ -177,15 +177,15 @@ FilterDAGInfoPtr generateFilterActions(
|
||||
/// Using separate expression analyzer to prevent any possible alias injection
|
||||
auto syntax_result = TreeRewriter(context).analyzeSelect(query_ast, TreeRewriterResult({}, storage, storage_snapshot));
|
||||
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->actions->removeUnusedActions(NameSet{filter_info->column_name});
|
||||
filter_info->actions.removeUnusedActions(NameSet{filter_info->column_name});
|
||||
|
||||
for (const auto * node : filter_info->actions->getInputs())
|
||||
filter_info->actions->getOutputs().push_back(node);
|
||||
for (const auto * node : filter_info->actions.getInputs())
|
||||
filter_info->actions.getOutputs().push_back(node);
|
||||
|
||||
auto required_columns_from_filter = filter_info->actions->getRequiredColumns();
|
||||
auto required_columns_from_filter = filter_info->actions.getRequiredColumns();
|
||||
|
||||
for (const auto & column : required_columns_from_filter)
|
||||
{
|
||||
@ -937,7 +937,7 @@ bool InterpreterSelectQuery::adjustParallelReplicasAfterAnalysis()
|
||||
{
|
||||
{
|
||||
const auto & node
|
||||
= query_info_copy.prewhere_info->prewhere_actions->findInOutputs(query_info_copy.prewhere_info->prewhere_column_name);
|
||||
= query_info_copy.prewhere_info->prewhere_actions.findInOutputs(query_info_copy.prewhere_info->prewhere_column_name);
|
||||
added_filter_nodes.nodes.push_back(&node);
|
||||
}
|
||||
|
||||
@ -949,7 +949,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);
|
||||
/// 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;
|
||||
@ -984,7 +985,7 @@ void InterpreterSelectQuery::buildQueryPlan(QueryPlan & query_plan)
|
||||
ActionsDAG::MatchColumnsMode::Name,
|
||||
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));
|
||||
}
|
||||
|
||||
@ -1057,7 +1058,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
|
||||
|
||||
if (analysis_result.prewhere_info)
|
||||
{
|
||||
header = analysis_result.prewhere_info->prewhere_actions->updateHeader(header);
|
||||
header = analysis_result.prewhere_info->prewhere_actions.updateHeader(header);
|
||||
if (analysis_result.prewhere_info->remove_prewhere_column)
|
||||
header.erase(analysis_result.prewhere_info->prewhere_column_name);
|
||||
}
|
||||
@ -1308,12 +1309,12 @@ static InterpolateDescriptionPtr getInterpolateDescription(
|
||||
|
||||
auto syntax_result = TreeRewriter(context).analyze(exprs, source_columns);
|
||||
ExpressionAnalyzer analyzer(exprs, syntax_result, context);
|
||||
ActionsDAGPtr actions = analyzer.getActionsDAG(true);
|
||||
ActionsDAGPtr conv_dag = ActionsDAG::makeConvertingActions(actions->getResultColumns(),
|
||||
ActionsDAG actions = analyzer.getActionsDAG(true);
|
||||
ActionsDAG conv_dag = ActionsDAG::makeConvertingActions(actions.getResultColumns(),
|
||||
result_columns, ActionsDAG::MatchColumnsMode::Position, true);
|
||||
ActionsDAGPtr merge_dag = ActionsDAG::merge(std::move(*actions->clone()), std::move(*conv_dag));
|
||||
ActionsDAG merge_dag = ActionsDAG::merge(std::move(actions), std::move(conv_dag));
|
||||
|
||||
interpolate_descr = std::make_shared<InterpolateDescription>(merge_dag, aliases);
|
||||
interpolate_descr = std::make_shared<InterpolateDescription>(std::move(merge_dag), aliases);
|
||||
}
|
||||
|
||||
return interpolate_descr;
|
||||
@ -1496,7 +1497,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
|
||||
{
|
||||
auto row_level_security_step = std::make_unique<FilterStep>(
|
||||
query_plan.getCurrentDataStream(),
|
||||
expressions.filter_info->actions,
|
||||
expressions.filter_info->actions.clone(),
|
||||
expressions.filter_info->column_name,
|
||||
expressions.filter_info->do_remove_column);
|
||||
|
||||
@ -1510,7 +1511,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
|
||||
{
|
||||
auto row_level_filter_step = std::make_unique<FilterStep>(
|
||||
query_plan.getCurrentDataStream(),
|
||||
expressions.prewhere_info->row_level_filter,
|
||||
expressions.prewhere_info->row_level_filter->clone(),
|
||||
expressions.prewhere_info->row_level_column_name,
|
||||
true);
|
||||
|
||||
@ -1520,7 +1521,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
|
||||
|
||||
auto prewhere_step = std::make_unique<FilterStep>(
|
||||
query_plan.getCurrentDataStream(),
|
||||
expressions.prewhere_info->prewhere_actions,
|
||||
expressions.prewhere_info->prewhere_actions.clone(),
|
||||
expressions.prewhere_info->prewhere_column_name,
|
||||
expressions.prewhere_info->remove_prewhere_column);
|
||||
|
||||
@ -1622,7 +1623,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
|
||||
{
|
||||
auto row_level_security_step = std::make_unique<FilterStep>(
|
||||
query_plan.getCurrentDataStream(),
|
||||
expressions.filter_info->actions,
|
||||
expressions.filter_info->actions.clone(),
|
||||
expressions.filter_info->column_name,
|
||||
expressions.filter_info->do_remove_column);
|
||||
|
||||
@ -1630,11 +1631,11 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
|
||||
query_plan.addStep(std::move(row_level_security_step));
|
||||
}
|
||||
|
||||
const auto add_filter_step = [&](const auto & new_filter_info, const std::string & description)
|
||||
const auto add_filter_step = [&](auto & new_filter_info, const std::string & description)
|
||||
{
|
||||
auto filter_step = std::make_unique<FilterStep>(
|
||||
query_plan.getCurrentDataStream(),
|
||||
new_filter_info->actions,
|
||||
std::move(new_filter_info->actions),
|
||||
new_filter_info->column_name,
|
||||
new_filter_info->do_remove_column);
|
||||
|
||||
@ -2056,18 +2057,20 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c
|
||||
|
||||
if (prewhere_info.row_level_filter)
|
||||
{
|
||||
auto row_level_actions = std::make_shared<ExpressionActions>(prewhere_info.row_level_filter->clone());
|
||||
pipe.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
return std::make_shared<FilterTransform>(header,
|
||||
std::make_shared<ExpressionActions>(prewhere_info.row_level_filter),
|
||||
row_level_actions,
|
||||
prewhere_info.row_level_column_name, true);
|
||||
});
|
||||
}
|
||||
|
||||
auto filter_actions = std::make_shared<ExpressionActions>(prewhere_info.prewhere_actions.clone());
|
||||
pipe.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
return std::make_shared<FilterTransform>(
|
||||
header, std::make_shared<ExpressionActions>(prewhere_info.prewhere_actions),
|
||||
header, filter_actions,
|
||||
prewhere_info.prewhere_column_name, prewhere_info.remove_prewhere_column);
|
||||
});
|
||||
}
|
||||
@ -2111,8 +2114,8 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis
|
||||
if (does_storage_support_prewhere && shouldMoveToPrewhere())
|
||||
{
|
||||
/// Execute row level filter in prewhere as a part of "move to prewhere" optimization.
|
||||
analysis.prewhere_info = std::make_shared<PrewhereInfo>(analysis.filter_info->actions, analysis.filter_info->column_name);
|
||||
analysis.prewhere_info->remove_prewhere_column = analysis.filter_info->do_remove_column;
|
||||
analysis.prewhere_info = std::make_shared<PrewhereInfo>(std::move(analysis.filter_info->actions), analysis.filter_info->column_name);
|
||||
analysis.prewhere_info->remove_prewhere_column = std::move(analysis.filter_info->do_remove_column);
|
||||
analysis.prewhere_info->need_filter = true;
|
||||
analysis.filter_info = nullptr;
|
||||
}
|
||||
@ -2120,8 +2123,8 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis
|
||||
else
|
||||
{
|
||||
/// Add row level security actions to prewhere.
|
||||
analysis.prewhere_info->row_level_filter = analysis.filter_info->actions;
|
||||
analysis.prewhere_info->row_level_column_name = analysis.filter_info->column_name;
|
||||
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.filter_info = nullptr;
|
||||
}
|
||||
}
|
||||
@ -2154,7 +2157,7 @@ void InterpreterSelectQuery::addPrewhereAliasActions()
|
||||
if (prewhere_info)
|
||||
{
|
||||
/// Get some columns directly from PREWHERE expression actions
|
||||
auto prewhere_required_columns = prewhere_info->prewhere_actions->getRequiredColumns().getNames();
|
||||
auto prewhere_required_columns = prewhere_info->prewhere_actions.getRequiredColumns().getNames();
|
||||
columns.insert(prewhere_required_columns.begin(), prewhere_required_columns.end());
|
||||
|
||||
if (prewhere_info->row_level_filter)
|
||||
@ -2226,7 +2229,7 @@ void InterpreterSelectQuery::addPrewhereAliasActions()
|
||||
if (prewhere_info)
|
||||
{
|
||||
NameSet columns_to_remove(columns_to_remove_after_prewhere.begin(), columns_to_remove_after_prewhere.end());
|
||||
Block prewhere_actions_result = prewhere_info->prewhere_actions->getResultColumns();
|
||||
Block prewhere_actions_result = prewhere_info->prewhere_actions.getResultColumns();
|
||||
|
||||
/// Populate required columns with the columns, added by PREWHERE actions and not removed afterwards.
|
||||
/// XXX: looks hacky that we already know which columns after PREWHERE we won't need for sure.
|
||||
@ -2265,7 +2268,7 @@ void InterpreterSelectQuery::addPrewhereAliasActions()
|
||||
{
|
||||
/// Don't remove columns which are needed to be aliased.
|
||||
for (const auto & name : required_columns)
|
||||
prewhere_info->prewhere_actions->tryRestoreColumn(name);
|
||||
prewhere_info->prewhere_actions.tryRestoreColumn(name);
|
||||
|
||||
/// Add physical columns required by prewhere actions.
|
||||
for (const auto & column : required_columns_from_prewhere)
|
||||
@ -2323,7 +2326,7 @@ std::optional<UInt64> InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle
|
||||
if (analysis_result.hasPrewhere())
|
||||
{
|
||||
auto & prewhere_info = analysis_result.prewhere_info;
|
||||
filter_nodes.push_back(&prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name));
|
||||
filter_nodes.push_back(&prewhere_info->prewhere_actions.findInOutputs(prewhere_info->prewhere_column_name));
|
||||
|
||||
if (prewhere_info->row_level_filter)
|
||||
filter_nodes.push_back(&prewhere_info->row_level_filter->findInOutputs(prewhere_info->row_level_column_name));
|
||||
@ -2337,7 +2340,7 @@ std::optional<UInt64> InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle
|
||||
if (!filter_actions_dag)
|
||||
return {};
|
||||
|
||||
return storage->totalRowsByPartitionPredicate(filter_actions_dag, context);
|
||||
return storage->totalRowsByPartitionPredicate(*filter_actions_dag, context);
|
||||
}
|
||||
}
|
||||
|
||||
@ -2587,7 +2590,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
|
||||
/// Aliases in table declaration.
|
||||
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(), alias_actions->clone());
|
||||
table_aliases->setStepDescription("Add table aliases");
|
||||
query_plan.addStep(std::move(table_aliases));
|
||||
}
|
||||
@ -2597,7 +2600,7 @@ void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsA
|
||||
{
|
||||
auto dag = expression->dag.clone();
|
||||
if (expression->project_input)
|
||||
dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
|
||||
dag.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
|
||||
|
||||
auto where_step = std::make_unique<FilterStep>(
|
||||
query_plan.getCurrentDataStream(), std::move(dag), getSelectQuery().where()->getColumnName(), remove_filter);
|
||||
@ -2771,7 +2774,7 @@ void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const Actions
|
||||
{
|
||||
auto dag = expression->dag.clone();
|
||||
if (expression->project_input)
|
||||
dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
|
||||
dag.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
|
||||
|
||||
auto having_step
|
||||
= std::make_unique<FilterStep>(query_plan.getCurrentDataStream(), std::move(dag), getSelectQuery().having()->getColumnName(), remove_filter);
|
||||
@ -2784,7 +2787,7 @@ void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const Actions
|
||||
void InterpreterSelectQuery::executeTotalsAndHaving(
|
||||
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)
|
||||
{
|
||||
dag = expression->dag.clone();
|
||||
@ -2838,7 +2841,7 @@ void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const Act
|
||||
|
||||
auto dag = expression->dag.clone();
|
||||
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));
|
||||
|
||||
|
@ -239,7 +239,7 @@ private:
|
||||
Block source_header;
|
||||
|
||||
/// Actions to calculate ALIAS if required.
|
||||
ActionsDAGPtr alias_actions;
|
||||
std::optional<ActionsDAG> alias_actions;
|
||||
|
||||
/// The subquery interpreter, if the subquery
|
||||
std::unique_ptr<InterpreterSelectWithUnionQuery> interpreter_subquery;
|
||||
|
@ -1200,7 +1200,7 @@ void MutationsInterpreter::Source::read(
|
||||
const auto & names = first_stage.filter_column_names;
|
||||
size_t num_filters = names.size();
|
||||
|
||||
ActionsDAGPtr filter;
|
||||
std::optional<ActionsDAG> filter;
|
||||
if (!first_stage.filter_column_names.empty())
|
||||
{
|
||||
ActionsDAG::NodeRawConstPtrs nodes(num_filters);
|
||||
@ -1214,7 +1214,7 @@ void MutationsInterpreter::Source::read(
|
||||
MergeTreeSequentialSourceType::Mutation,
|
||||
plan, *data, storage_snapshot,
|
||||
part, required_columns,
|
||||
apply_deleted_mask_, filter, context_,
|
||||
apply_deleted_mask_, std::move(filter), context_,
|
||||
getLogger("MutationsInterpreter"));
|
||||
}
|
||||
else
|
||||
@ -1283,17 +1283,17 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v
|
||||
{
|
||||
auto dag = step->actions()->dag.clone();
|
||||
if (step->actions()->project_input)
|
||||
dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header);
|
||||
dag.appendInputsForUnusedColumns(plan.getCurrentDataStream().header);
|
||||
/// 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
|
||||
{
|
||||
auto dag = step->actions()->dag.clone();
|
||||
if (step->actions()->project_input)
|
||||
dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header);
|
||||
dag.appendInputsForUnusedColumns(plan.getCurrentDataStream().header);
|
||||
/// 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();
|
||||
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;
|
||||
}
|
||||
|
||||
|
@ -462,19 +462,19 @@ static void makeColumnNameUnique(const ColumnsWithTypeAndName & source_columns,
|
||||
}
|
||||
}
|
||||
|
||||
static ActionsDAGPtr createWrapWithTupleActions(
|
||||
static std::optional<ActionsDAG> createWrapWithTupleActions(
|
||||
const ColumnsWithTypeAndName & source_columns,
|
||||
std::unordered_set<std::string_view> && column_names_to_wrap,
|
||||
NameToNameMap & new_names)
|
||||
{
|
||||
if (column_names_to_wrap.empty())
|
||||
return nullptr;
|
||||
return {};
|
||||
|
||||
auto actions_dag = std::make_shared<ActionsDAG>(source_columns);
|
||||
ActionsDAG actions_dag(source_columns);
|
||||
|
||||
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;
|
||||
auto it = column_names_to_wrap.find(column_name);
|
||||
@ -485,9 +485,9 @@ static ActionsDAGPtr createWrapWithTupleActions(
|
||||
String node_name = "__wrapNullsafe(" + column_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;
|
||||
actions_dag->addOrReplaceInOutputs(dst_node);
|
||||
actions_dag.addOrReplaceInOutputs(dst_node);
|
||||
}
|
||||
|
||||
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};
|
||||
}
|
||||
|
||||
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)
|
||||
result_dag->mergeInplace(std::move(*new_dag));
|
||||
else
|
||||
result_dag = std::move(new_dag);
|
||||
}
|
||||
|
||||
std::pair<ActionsDAGPtr, ActionsDAGPtr>
|
||||
std::pair<std::optional<ActionsDAG>, std::optional<ActionsDAG>>
|
||||
TableJoin::createConvertingActions(
|
||||
const ColumnsWithTypeAndName & left_sample_columns,
|
||||
const ColumnsWithTypeAndName & right_sample_columns)
|
||||
{
|
||||
ActionsDAGPtr left_dag = nullptr;
|
||||
ActionsDAGPtr right_dag = nullptr;
|
||||
std::optional<ActionsDAG> left_dag;
|
||||
std::optional<ActionsDAG> right_dag;
|
||||
/** If the types are not equal, we need to convert them to a common type.
|
||||
* Example:
|
||||
* SELECT * FROM t1 JOIN t2 ON t1.a = t2.b
|
||||
@ -616,7 +618,7 @@ TableJoin::createConvertingActions(
|
||||
mergeDags(right_dag, std::move(new_right_dag));
|
||||
}
|
||||
|
||||
return {left_dag, right_dag};
|
||||
return {std::move(left_dag), std::move(right_dag)};
|
||||
}
|
||||
|
||||
template <typename LeftNamesAndTypes, typename RightNamesAndTypes>
|
||||
@ -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,
|
||||
bool add_new_cols,
|
||||
NameToNameMap & key_column_rename)
|
||||
@ -710,7 +712,7 @@ static ActionsDAGPtr changeKeyTypes(const ColumnsWithTypeAndName & cols_src,
|
||||
}
|
||||
}
|
||||
if (!has_some_to_do)
|
||||
return nullptr;
|
||||
return {};
|
||||
|
||||
return ActionsDAG::makeConvertingActions(
|
||||
/* source= */ cols_src,
|
||||
@ -721,7 +723,7 @@ static ActionsDAGPtr changeKeyTypes(const ColumnsWithTypeAndName & cols_src,
|
||||
/* new_names= */ &key_column_rename);
|
||||
}
|
||||
|
||||
static ActionsDAGPtr changeTypesToNullable(
|
||||
static std::optional<ActionsDAG> changeTypesToNullable(
|
||||
const ColumnsWithTypeAndName & cols_src,
|
||||
const NameSet & exception_cols)
|
||||
{
|
||||
@ -737,7 +739,7 @@ static ActionsDAGPtr changeTypesToNullable(
|
||||
}
|
||||
|
||||
if (!has_some_to_do)
|
||||
return nullptr;
|
||||
return {};
|
||||
|
||||
return ActionsDAG::makeConvertingActions(
|
||||
/* source= */ cols_src,
|
||||
@ -748,29 +750,29 @@ static ActionsDAGPtr changeTypesToNullable(
|
||||
/* new_names= */ nullptr);
|
||||
}
|
||||
|
||||
ActionsDAGPtr TableJoin::applyKeyConvertToTable(
|
||||
std::optional<ActionsDAG> TableJoin::applyKeyConvertToTable(
|
||||
const ColumnsWithTypeAndName & cols_src,
|
||||
const NameToTypeMap & type_mapping,
|
||||
JoinTableSide table_side,
|
||||
NameToNameMap & key_column_rename)
|
||||
{
|
||||
if (type_mapping.empty())
|
||||
return nullptr;
|
||||
return {};
|
||||
|
||||
/// 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);
|
||||
return convert_dag;
|
||||
}
|
||||
|
||||
ActionsDAGPtr TableJoin::applyNullsafeWrapper(
|
||||
std::optional<ActionsDAG> TableJoin::applyNullsafeWrapper(
|
||||
const ColumnsWithTypeAndName & cols_src,
|
||||
const NameSet & columns_for_nullsafe_comparison,
|
||||
JoinTableSide table_side,
|
||||
NameToNameMap & key_column_rename)
|
||||
{
|
||||
if (columns_for_nullsafe_comparison.empty())
|
||||
return nullptr;
|
||||
return {};
|
||||
|
||||
std::unordered_set<std::string_view> column_names_to_wrap;
|
||||
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
|
||||
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 (size_t i : clause.nullsafe_compare_key_indexes)
|
||||
@ -799,7 +801,7 @@ ActionsDAGPtr TableJoin::applyNullsafeWrapper(
|
||||
return null_safe_wrap_dag;
|
||||
}
|
||||
|
||||
ActionsDAGPtr TableJoin::applyJoinUseNullsConversion(
|
||||
std::optional<ActionsDAG> TableJoin::applyJoinUseNullsConversion(
|
||||
const ColumnsWithTypeAndName & cols_src,
|
||||
const NameToNameMap & key_column_rename)
|
||||
{
|
||||
@ -809,8 +811,7 @@ ActionsDAGPtr TableJoin::applyJoinUseNullsConversion(
|
||||
exclude_columns.insert(it.second);
|
||||
|
||||
/// Create DAG to make columns nullable if needed
|
||||
ActionsDAGPtr add_nullable_dag = changeTypesToNullable(cols_src, exclude_columns);
|
||||
return add_nullable_dag;
|
||||
return changeTypesToNullable(cols_src, exclude_columns);
|
||||
}
|
||||
|
||||
void TableJoin::setStorageJoin(std::shared_ptr<const IKeyValueEntity> storage)
|
||||
@ -957,7 +958,7 @@ bool TableJoin::allowParallelHashJoin() const
|
||||
return true;
|
||||
}
|
||||
|
||||
ActionsDAGPtr TableJoin::createJoinedBlockActions(ContextPtr context) const
|
||||
ActionsDAG TableJoin::createJoinedBlockActions(ContextPtr context) const
|
||||
{
|
||||
ASTPtr expression_list = rightKeysList();
|
||||
auto syntax_result = TreeRewriter(context).analyze(expression_list, columnsFromJoinedTable());
|
||||
|
@ -201,19 +201,19 @@ private:
|
||||
Names requiredJoinedNames() const;
|
||||
|
||||
/// Create converting actions and change key column names if required
|
||||
ActionsDAGPtr applyKeyConvertToTable(
|
||||
std::optional<ActionsDAG> applyKeyConvertToTable(
|
||||
const ColumnsWithTypeAndName & cols_src,
|
||||
const NameToTypeMap & type_mapping,
|
||||
JoinTableSide table_side,
|
||||
NameToNameMap & key_column_rename);
|
||||
|
||||
ActionsDAGPtr applyNullsafeWrapper(
|
||||
std::optional<ActionsDAG> applyNullsafeWrapper(
|
||||
const ColumnsWithTypeAndName & cols_src,
|
||||
const NameSet & columns_for_nullsafe_comparison,
|
||||
JoinTableSide table_side,
|
||||
NameToNameMap & key_column_rename);
|
||||
|
||||
ActionsDAGPtr applyJoinUseNullsConversion(
|
||||
std::optional<ActionsDAG> applyJoinUseNullsConversion(
|
||||
const ColumnsWithTypeAndName & cols_src,
|
||||
const NameToNameMap & key_column_rename);
|
||||
|
||||
@ -263,7 +263,7 @@ public:
|
||||
|
||||
TemporaryDataOnDiskScopePtr getTempDataOnDisk() { return tmp_data; }
|
||||
|
||||
ActionsDAGPtr createJoinedBlockActions(ContextPtr context) const;
|
||||
ActionsDAG createJoinedBlockActions(ContextPtr context) const;
|
||||
|
||||
const std::vector<JoinAlgorithm> & getEnabledJoinAlgorithms() const { return join_algorithm; }
|
||||
|
||||
@ -378,7 +378,7 @@ public:
|
||||
/// 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 `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(
|
||||
const ColumnsWithTypeAndName & left_sample_columns,
|
||||
const ColumnsWithTypeAndName & right_sample_columns);
|
||||
|
@ -14,7 +14,6 @@ namespace DB
|
||||
class ASTFunction;
|
||||
|
||||
class ActionsDAG;
|
||||
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
|
||||
|
||||
struct WindowFunctionDescription
|
||||
{
|
||||
@ -93,8 +92,8 @@ struct WindowDescription
|
||||
// then by ORDER BY. This field holds this combined sort order.
|
||||
SortDescription full_sort_description;
|
||||
|
||||
std::vector<ActionsDAGPtr> partition_by_actions;
|
||||
std::vector<ActionsDAGPtr> order_by_actions;
|
||||
std::vector<std::shared_ptr<const ActionsDAG>> partition_by_actions;
|
||||
std::vector<std::shared_ptr<const ActionsDAG>> order_by_actions;
|
||||
|
||||
WindowFrame frame;
|
||||
|
||||
|
@ -14,15 +14,15 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ActionsDAGPtr addMissingDefaults(
|
||||
ActionsDAG addMissingDefaults(
|
||||
const Block & header,
|
||||
const NamesAndTypesList & required_columns,
|
||||
const ColumnsDescription & columns,
|
||||
ContextPtr context,
|
||||
bool null_as_default)
|
||||
{
|
||||
auto actions = std::make_shared<ActionsDAG>(header.getColumnsWithTypeAndName());
|
||||
auto & index = actions->getOutputs();
|
||||
ActionsDAG actions(header.getColumnsWithTypeAndName());
|
||||
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.
|
||||
/// First, remember the offset columns for all arrays in the block.
|
||||
@ -40,7 +40,7 @@ ActionsDAGPtr addMissingDefaults(
|
||||
if (group.empty())
|
||||
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();
|
||||
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];
|
||||
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;
|
||||
}
|
||||
@ -75,17 +75,17 @@ ActionsDAGPtr addMissingDefaults(
|
||||
* it can be full (or the interpreter may decide that it is constant everywhere).
|
||||
*/
|
||||
auto new_column = column.type->createColumnConstWithDefaultValue(0);
|
||||
const auto * col = &actions->addColumn({new_column, column.type, column.name});
|
||||
index.push_back(&actions->materializeNode(*col));
|
||||
const auto * col = &actions.addColumn({new_column, column.type, column.name});
|
||||
index.push_back(&actions.materializeNode(*col));
|
||||
}
|
||||
|
||||
/// Computes explicitly specified values by default and materialized columns.
|
||||
if (auto dag = evaluateMissingDefaults(actions->getResultColumns(), required_columns, columns, context, true, null_as_default))
|
||||
actions = ActionsDAG::merge(std::move(*actions), std::move(*dag));
|
||||
if (auto dag = evaluateMissingDefaults(actions.getResultColumns(), required_columns, columns, context, true, null_as_default))
|
||||
actions = ActionsDAG::merge(std::move(actions), std::move(*dag));
|
||||
|
||||
/// Removes unused columns and reorders result.
|
||||
actions->removeUnusedActions(required_columns.getNames(), false);
|
||||
actions->addMaterializingOutputActions();
|
||||
actions.removeUnusedActions(required_columns.getNames(), false);
|
||||
actions.addMaterializingOutputActions();
|
||||
|
||||
return actions;
|
||||
}
|
||||
|
@ -2,11 +2,6 @@
|
||||
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
|
||||
#include <memory>
|
||||
#include <string>
|
||||
#include <unordered_map>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -15,7 +10,6 @@ class NamesAndTypesList;
|
||||
class ColumnsDescription;
|
||||
|
||||
class ActionsDAG;
|
||||
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
|
||||
|
||||
/** Adds three types of columns into block
|
||||
* 1. Columns, that are missed inside request, but present in table without defaults (missed columns)
|
||||
@ -24,7 +18,7 @@ using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
|
||||
* 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).
|
||||
*/
|
||||
ActionsDAGPtr addMissingDefaults(
|
||||
ActionsDAG addMissingDefaults(
|
||||
const Block & header, const NamesAndTypesList & required_columns,
|
||||
const ColumnsDescription & columns, ContextPtr context, bool null_as_default = false);
|
||||
}
|
||||
|
@ -91,7 +91,7 @@ std::optional<EvaluateConstantExpressionResult> evaluateConstantExpressionImpl(c
|
||||
ColumnPtr result_column;
|
||||
DataTypePtr result_type;
|
||||
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)
|
||||
{
|
||||
@ -679,9 +679,9 @@ std::optional<ConstantVariants> evaluateExpressionOverConstantCondition(
|
||||
size_t max_elements)
|
||||
{
|
||||
auto inverted_dag = KeyCondition::cloneASTWithInversionPushDown({predicate}, context);
|
||||
auto matches = matchTrees(expr, *inverted_dag, false);
|
||||
auto matches = matchTrees(expr, inverted_dag, false);
|
||||
|
||||
auto predicates = analyze(inverted_dag->getOutputs().at(0), matches, context, max_elements);
|
||||
auto predicates = analyze(inverted_dag.getOutputs().at(0), matches, context, max_elements);
|
||||
|
||||
if (!predicates)
|
||||
return {};
|
||||
|
@ -152,22 +152,20 @@ ASTPtr convertRequiredExpressions(Block & block, const NamesAndTypesList & requi
|
||||
return conversion_expr_list;
|
||||
}
|
||||
|
||||
ActionsDAGPtr createExpressions(
|
||||
std::optional<ActionsDAG> createExpressions(
|
||||
const Block & header,
|
||||
ASTPtr expr_list,
|
||||
bool save_unneeded_columns,
|
||||
ContextPtr context)
|
||||
{
|
||||
if (!expr_list)
|
||||
return nullptr;
|
||||
return {};
|
||||
|
||||
auto syntax_result = TreeRewriter(context).analyze(expr_list, header.getNamesAndTypesList());
|
||||
auto expression_analyzer = ExpressionAnalyzer{expr_list, syntax_result, context};
|
||||
auto dag = std::make_shared<ActionsDAG>(header.getNamesAndTypesList());
|
||||
ActionsDAG dag(header.getNamesAndTypesList());
|
||||
auto actions = expression_analyzer.getActionsDAG(true, !save_unneeded_columns);
|
||||
dag = ActionsDAG::merge(std::move(*dag), std::move(*actions));
|
||||
|
||||
return dag;
|
||||
return ActionsDAG::merge(std::move(dag), std::move(actions));
|
||||
}
|
||||
|
||||
}
|
||||
@ -180,7 +178,7 @@ void performRequiredConversions(Block & block, const NamesAndTypesList & require
|
||||
|
||||
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);
|
||||
}
|
||||
}
|
||||
@ -195,7 +193,7 @@ bool needConvertAnyNullToDefault(const Block & header, const NamesAndTypesList &
|
||||
return false;
|
||||
}
|
||||
|
||||
ActionsDAGPtr evaluateMissingDefaults(
|
||||
std::optional<ActionsDAG> evaluateMissingDefaults(
|
||||
const Block & header,
|
||||
const NamesAndTypesList & required_columns,
|
||||
const ColumnsDescription & columns,
|
||||
@ -204,7 +202,7 @@ ActionsDAGPtr evaluateMissingDefaults(
|
||||
bool null_as_default)
|
||||
{
|
||||
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);
|
||||
return createExpressions(header, expr_list, save_unneeded_columns, context);
|
||||
|
@ -5,9 +5,6 @@
|
||||
#include <Common/COW.h>
|
||||
|
||||
#include <memory>
|
||||
#include <string>
|
||||
#include <unordered_map>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -24,12 +21,11 @@ struct StorageInMemoryMetadata;
|
||||
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
|
||||
|
||||
class ActionsDAG;
|
||||
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
|
||||
|
||||
/// 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.
|
||||
/// Return nullptr if no actions required.
|
||||
ActionsDAGPtr evaluateMissingDefaults(
|
||||
std::optional<ActionsDAG> evaluateMissingDefaults(
|
||||
const Block & header,
|
||||
const NamesAndTypesList & required_columns,
|
||||
const ColumnsDescription & columns,
|
||||
|
@ -88,16 +88,16 @@ public:
|
||||
|
||||
auto column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(node);
|
||||
|
||||
ActionsDAGPtr alias_column_actions_dag = std::make_shared<ActionsDAG>();
|
||||
ActionsDAG alias_column_actions_dag;
|
||||
PlannerActionsVisitor actions_visitor(planner_context, false);
|
||||
auto outputs = actions_visitor.visit(*alias_column_actions_dag, column_node->getExpression());
|
||||
auto outputs = actions_visitor.visit(alias_column_actions_dag, column_node->getExpression());
|
||||
if (outputs.size() != 1)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Expected single output in actions dag for alias column {}. Actual {}", column_node->dumpTree(), outputs.size());
|
||||
const auto & column_name = column_node->getColumnName();
|
||||
const auto & alias_node = alias_column_actions_dag->addAlias(*outputs[0], column_name);
|
||||
alias_column_actions_dag->addOrReplaceInOutputs(alias_node);
|
||||
table_expression_data.addAliasColumn(column_node->getColumn(), column_identifier, alias_column_actions_dag, select_added_columns);
|
||||
const auto & alias_node = alias_column_actions_dag.addAlias(*outputs[0], column_name);
|
||||
alias_column_actions_dag.addOrReplaceInOutputs(alias_node);
|
||||
table_expression_data.addAliasColumn(column_node->getColumn(), column_identifier, std::move(alias_column_actions_dag), select_added_columns);
|
||||
}
|
||||
|
||||
return;
|
||||
@ -335,22 +335,22 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr
|
||||
collect_source_columns_visitor.setKeepAliasColumns(false);
|
||||
collect_source_columns_visitor.visit(query_node_typed.getPrewhere());
|
||||
|
||||
auto prewhere_actions_dag = std::make_shared<ActionsDAG>();
|
||||
ActionsDAG prewhere_actions_dag;
|
||||
|
||||
QueryTreeNodePtr query_tree_node = query_node_typed.getPrewhere();
|
||||
|
||||
PlannerActionsVisitor visitor(planner_context, false /*use_column_identifier_as_action_node_name*/);
|
||||
auto expression_nodes = visitor.visit(*prewhere_actions_dag, query_tree_node);
|
||||
auto expression_nodes = visitor.visit(prewhere_actions_dag, query_tree_node);
|
||||
if (expression_nodes.size() != 1)
|
||||
throw Exception(ErrorCodes::ILLEGAL_PREWHERE,
|
||||
"Invalid PREWHERE. Expected single boolean expression. In query {}",
|
||||
query_node->formatASTForErrorMessage());
|
||||
|
||||
prewhere_actions_dag->getOutputs().push_back(expression_nodes.back());
|
||||
prewhere_actions_dag.getOutputs().push_back(expression_nodes.back());
|
||||
|
||||
for (const auto & prewhere_input_node : prewhere_actions_dag->getInputs())
|
||||
for (const auto & prewhere_input_node : prewhere_actions_dag.getInputs())
|
||||
if (required_column_names_without_prewhere.contains(prewhere_input_node->result_name))
|
||||
prewhere_actions_dag->getOutputs().push_back(prewhere_input_node);
|
||||
prewhere_actions_dag.getOutputs().push_back(prewhere_input_node);
|
||||
|
||||
table_expression_data.setPrewhereFilterActions(std::move(prewhere_actions_dag));
|
||||
}
|
||||
|
@ -12,7 +12,6 @@
|
||||
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/CastOverloadResolver.h>
|
||||
#include <Functions/indexHint.h>
|
||||
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||
@ -216,9 +215,11 @@ FiltersForTableExpressionMap collectFiltersForAnalysis(const QueryTreeNodePtr &
|
||||
if (!read_from_dummy)
|
||||
continue;
|
||||
|
||||
auto filter_actions = read_from_dummy->getFilterActionsDAG();
|
||||
const auto & table_node = dummy_storage_to_table.at(&read_from_dummy->getStorage());
|
||||
res[table_node] = FiltersForTableExpression{std::move(filter_actions), read_from_dummy->getPrewhereInfo()};
|
||||
if (auto filter_actions = read_from_dummy->detachFilterActionsDAG())
|
||||
{
|
||||
const auto & table_node = dummy_storage_to_table.at(&read_from_dummy->getStorage());
|
||||
res[table_node] = FiltersForTableExpression{std::move(filter_actions), read_from_dummy->getPrewhereInfo()};
|
||||
}
|
||||
}
|
||||
|
||||
return res;
|
||||
@ -332,34 +333,34 @@ public:
|
||||
};
|
||||
|
||||
void addExpressionStep(QueryPlan & query_plan,
|
||||
const ActionsAndProjectInputsFlagPtr & expression_actions,
|
||||
ActionsAndProjectInputsFlagPtr & expression_actions,
|
||||
const std::string & step_description,
|
||||
std::vector<ActionsDAGPtr> & result_actions_to_execute)
|
||||
UsefulSets & useful_sets)
|
||||
{
|
||||
auto actions = expression_actions->dag.clone();
|
||||
auto actions = std::move(expression_actions->dag);
|
||||
if (expression_actions->project_input)
|
||||
actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
|
||||
actions.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
|
||||
|
||||
result_actions_to_execute.push_back(actions);
|
||||
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);
|
||||
expression_step->setStepDescription(step_description);
|
||||
query_plan.addStep(std::move(expression_step));
|
||||
}
|
||||
|
||||
void addFilterStep(QueryPlan & query_plan,
|
||||
const FilterAnalysisResult & filter_analysis_result,
|
||||
FilterAnalysisResult & filter_analysis_result,
|
||||
const std::string & step_description,
|
||||
std::vector<ActionsDAGPtr> & result_actions_to_execute)
|
||||
UsefulSets & useful_sets)
|
||||
{
|
||||
auto actions = filter_analysis_result.filter_actions->dag.clone();
|
||||
auto actions = std::move(filter_analysis_result.filter_actions->dag);
|
||||
if (filter_analysis_result.filter_actions->project_input)
|
||||
actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
|
||||
actions.appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
|
||||
|
||||
result_actions_to_execute.push_back(actions);
|
||||
auto where_step = std::make_unique<FilterStep>(query_plan.getCurrentDataStream(),
|
||||
actions,
|
||||
std::move(actions),
|
||||
filter_analysis_result.filter_column_name,
|
||||
filter_analysis_result.remove_filter_column);
|
||||
appendSetsFromActionsDAG(where_step->getExpression(), useful_sets);
|
||||
where_step->setStepDescription(step_description);
|
||||
query_plan.addStep(std::move(where_step));
|
||||
}
|
||||
@ -543,39 +544,41 @@ void addMergingAggregatedStep(QueryPlan & query_plan,
|
||||
}
|
||||
|
||||
void addTotalsHavingStep(QueryPlan & query_plan,
|
||||
const PlannerExpressionsAnalysisResult & expression_analysis_result,
|
||||
PlannerExpressionsAnalysisResult & expression_analysis_result,
|
||||
const QueryAnalysisResult & query_analysis_result,
|
||||
const PlannerContextPtr & planner_context,
|
||||
const QueryNode & query_node,
|
||||
std::vector<ActionsDAGPtr> & result_actions_to_execute)
|
||||
UsefulSets & useful_sets)
|
||||
{
|
||||
const auto & query_context = planner_context->getQueryContext();
|
||||
const auto & settings = query_context->getSettingsRef();
|
||||
|
||||
const auto & aggregation_analysis_result = expression_analysis_result.getAggregation();
|
||||
const auto & having_analysis_result = expression_analysis_result.getHaving();
|
||||
auto & aggregation_analysis_result = expression_analysis_result.getAggregation();
|
||||
auto & having_analysis_result = expression_analysis_result.getHaving();
|
||||
bool need_finalize = !query_node.isGroupByWithRollup() && !query_node.isGroupByWithCube();
|
||||
|
||||
ActionsDAGPtr actions;
|
||||
std::optional<ActionsDAG> actions;
|
||||
if (having_analysis_result.filter_actions)
|
||||
{
|
||||
actions = having_analysis_result.filter_actions->dag.clone();
|
||||
actions = std::move(having_analysis_result.filter_actions->dag);
|
||||
if (having_analysis_result.filter_actions->project_input)
|
||||
actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
|
||||
|
||||
result_actions_to_execute.push_back(actions);
|
||||
}
|
||||
|
||||
auto totals_having_step = std::make_unique<TotalsHavingStep>(
|
||||
query_plan.getCurrentDataStream(),
|
||||
aggregation_analysis_result.aggregate_descriptions,
|
||||
query_analysis_result.aggregate_overflow_row,
|
||||
actions,
|
||||
std::move(actions),
|
||||
having_analysis_result.filter_column_name,
|
||||
having_analysis_result.remove_filter_column,
|
||||
settings.totals_mode,
|
||||
settings.totals_auto_threshold,
|
||||
need_finalize);
|
||||
|
||||
if (having_analysis_result.filter_actions)
|
||||
appendSetsFromActionsDAG(*totals_having_step->getActions(), useful_sets);
|
||||
|
||||
query_plan.addStep(std::move(totals_having_step));
|
||||
}
|
||||
|
||||
@ -717,13 +720,13 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan,
|
||||
|
||||
if (query_node.hasInterpolate())
|
||||
{
|
||||
auto interpolate_actions_dag = std::make_shared<ActionsDAG>();
|
||||
ActionsDAG interpolate_actions_dag;
|
||||
auto query_plan_columns = query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName();
|
||||
for (auto & query_plan_column : query_plan_columns)
|
||||
{
|
||||
/// INTERPOLATE actions dag input columns must be non constant
|
||||
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 &>();
|
||||
@ -731,12 +734,12 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan,
|
||||
|
||||
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))
|
||||
continue;
|
||||
|
||||
interpolate_actions_dag->getOutputs().push_back(input_node);
|
||||
interpolate_actions_dag.getOutputs().push_back(input_node);
|
||||
}
|
||||
}
|
||||
else
|
||||
@ -746,12 +749,12 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan,
|
||||
auto & interpolate_node_typed = interpolate_node->as<InterpolateNode &>();
|
||||
|
||||
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());
|
||||
if (expression_to_interpolate_expression_nodes.size() != 1)
|
||||
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());
|
||||
if (interpolate_expression_nodes.size() != 1)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Interpolate expression expected to have single action node");
|
||||
@ -762,16 +765,16 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan,
|
||||
const auto * interpolate_expression = interpolate_expression_nodes[0];
|
||||
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,
|
||||
interpolate_expression->result_name);
|
||||
}
|
||||
|
||||
const auto * alias_node = &interpolate_actions_dag->addAlias(*interpolate_expression, expression_to_interpolate_name);
|
||||
interpolate_actions_dag->getOutputs().push_back(alias_node);
|
||||
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->removeUnusedActions();
|
||||
interpolate_actions_dag.removeUnusedActions();
|
||||
}
|
||||
|
||||
Aliases empty_aliases;
|
||||
@ -883,12 +886,12 @@ bool addPreliminaryLimitOptimizationStepIfNeeded(QueryPlan & query_plan,
|
||||
* WINDOW functions.
|
||||
*/
|
||||
void addPreliminarySortOrDistinctOrLimitStepsIfNeeded(QueryPlan & query_plan,
|
||||
const PlannerExpressionsAnalysisResult & expressions_analysis_result,
|
||||
PlannerExpressionsAnalysisResult & expressions_analysis_result,
|
||||
const QueryAnalysisResult & query_analysis_result,
|
||||
const PlannerContextPtr & planner_context,
|
||||
const PlannerQueryProcessingInfo & query_processing_info,
|
||||
const QueryTreeNodePtr & query_tree,
|
||||
std::vector<ActionsDAGPtr> & result_actions_to_execute)
|
||||
UsefulSets & useful_sets)
|
||||
{
|
||||
const auto & query_node = query_tree->as<QueryNode &>();
|
||||
|
||||
@ -919,8 +922,8 @@ void addPreliminarySortOrDistinctOrLimitStepsIfNeeded(QueryPlan & query_plan,
|
||||
|
||||
if (expressions_analysis_result.hasLimitBy())
|
||||
{
|
||||
const auto & limit_by_analysis_result = expressions_analysis_result.getLimitBy();
|
||||
addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", result_actions_to_execute);
|
||||
auto & limit_by_analysis_result = expressions_analysis_result.getLimitBy();
|
||||
addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", useful_sets);
|
||||
addLimitByStep(query_plan, limit_by_analysis_result, query_node);
|
||||
}
|
||||
|
||||
@ -930,12 +933,12 @@ void addPreliminarySortOrDistinctOrLimitStepsIfNeeded(QueryPlan & query_plan,
|
||||
|
||||
void addWindowSteps(QueryPlan & query_plan,
|
||||
const PlannerContextPtr & planner_context,
|
||||
const WindowAnalysisResult & window_analysis_result)
|
||||
WindowAnalysisResult & window_analysis_result)
|
||||
{
|
||||
const auto & query_context = planner_context->getQueryContext();
|
||||
const auto & settings = query_context->getSettingsRef();
|
||||
|
||||
auto window_descriptions = window_analysis_result.window_descriptions;
|
||||
auto & window_descriptions = window_analysis_result.window_descriptions;
|
||||
sortWindowDescriptions(window_descriptions);
|
||||
|
||||
size_t window_descriptions_size = window_descriptions.size();
|
||||
@ -1057,47 +1060,15 @@ void addOffsetStep(QueryPlan & query_plan, const QueryAnalysisResult & query_ana
|
||||
}
|
||||
}
|
||||
|
||||
void collectSetsFromActionsDAG(const ActionsDAGPtr & dag, std::unordered_set<const FutureSet *> & useful_sets)
|
||||
{
|
||||
for (const auto & node : dag->getNodes())
|
||||
{
|
||||
if (node.column)
|
||||
{
|
||||
const IColumn * column = node.column.get();
|
||||
if (const auto * column_const = typeid_cast<const ColumnConst *>(column))
|
||||
column = &column_const->getDataColumn();
|
||||
|
||||
if (const auto * column_set = typeid_cast<const ColumnSet *>(column))
|
||||
useful_sets.insert(column_set->getData().get());
|
||||
}
|
||||
|
||||
if (node.type == ActionsDAG::ActionType::FUNCTION && node.function_base->getName() == "indexHint")
|
||||
{
|
||||
ActionsDAG::NodeRawConstPtrs children;
|
||||
if (const auto * adaptor = typeid_cast<const FunctionToFunctionBaseAdaptor *>(node.function_base.get()))
|
||||
{
|
||||
if (const auto * index_hint = typeid_cast<const FunctionIndexHint *>(adaptor->getFunction().get()))
|
||||
{
|
||||
collectSetsFromActionsDAG(index_hint->getActions(), useful_sets);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void addBuildSubqueriesForSetsStepIfNeeded(
|
||||
QueryPlan & query_plan,
|
||||
const SelectQueryOptions & select_query_options,
|
||||
const PlannerContextPtr & planner_context,
|
||||
const std::vector<ActionsDAGPtr> & result_actions_to_execute)
|
||||
const UsefulSets & useful_sets)
|
||||
{
|
||||
auto subqueries = planner_context->getPreparedSets().getSubqueries();
|
||||
std::unordered_set<const FutureSet *> useful_sets;
|
||||
|
||||
for (const auto & actions_to_execute : result_actions_to_execute)
|
||||
collectSetsFromActionsDAG(actions_to_execute, useful_sets);
|
||||
|
||||
auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set.get()); };
|
||||
auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set); };
|
||||
auto it = std::remove_if(subqueries.begin(), subqueries.end(), std::move(predicate));
|
||||
subqueries.erase(it, subqueries.end());
|
||||
|
||||
@ -1159,11 +1130,11 @@ void addAdditionalFilterStepIfNeeded(QueryPlan & query_plan,
|
||||
auto fake_table_expression = std::make_shared<TableNode>(std::move(storage), query_context);
|
||||
|
||||
auto filter_info = buildFilterInfo(additional_result_filter_ast, fake_table_expression, planner_context, std::move(fake_name_set));
|
||||
if (!filter_info.actions || !query_plan.isInitialized())
|
||||
if (!query_plan.isInitialized())
|
||||
return;
|
||||
|
||||
auto filter_step = std::make_unique<FilterStep>(query_plan.getCurrentDataStream(),
|
||||
filter_info.actions,
|
||||
std::move(filter_info.actions),
|
||||
filter_info.column_name,
|
||||
filter_info.do_remove_column);
|
||||
filter_step->setStepDescription("additional result filter");
|
||||
@ -1443,7 +1414,7 @@ void Planner::buildPlanForQueryNode()
|
||||
checkStoragesSupportTransactions(planner_context);
|
||||
|
||||
const auto & table_filters = planner_context->getGlobalPlannerContext()->filters_for_table_expressions;
|
||||
if (!select_query_options.only_analyze && !table_filters.empty()) // && top_level)
|
||||
if (!select_query_options.only_analyze && !table_filters.empty())
|
||||
{
|
||||
for (auto & [table_node, table_expression_data] : planner_context->getTableExpressionNodeToData())
|
||||
{
|
||||
@ -1451,7 +1422,7 @@ void Planner::buildPlanForQueryNode()
|
||||
if (it != table_filters.end())
|
||||
{
|
||||
const auto & filters = it->second;
|
||||
table_expression_data.setFilterActions(filters.filter_actions);
|
||||
table_expression_data.setFilterActions(filters.filter_actions->clone());
|
||||
table_expression_data.setPrewhereInfo(filters.prewhere_info);
|
||||
}
|
||||
}
|
||||
@ -1542,15 +1513,15 @@ void Planner::buildPlanForQueryNode()
|
||||
planner_context,
|
||||
query_processing_info);
|
||||
|
||||
std::vector<ActionsDAGPtr> result_actions_to_execute = std::move(join_tree_query_plan.actions_dags);
|
||||
auto useful_sets = std::move(join_tree_query_plan.useful_sets);
|
||||
|
||||
for (auto & [_, table_expression_data] : planner_context->getTableExpressionNodeToData())
|
||||
{
|
||||
if (table_expression_data.getPrewhereFilterActions())
|
||||
result_actions_to_execute.push_back(table_expression_data.getPrewhereFilterActions());
|
||||
appendSetsFromActionsDAG(*table_expression_data.getPrewhereFilterActions(), useful_sets);
|
||||
|
||||
if (table_expression_data.getRowLevelFilterActions())
|
||||
result_actions_to_execute.push_back(table_expression_data.getRowLevelFilterActions());
|
||||
appendSetsFromActionsDAG(*table_expression_data.getRowLevelFilterActions(), useful_sets);
|
||||
}
|
||||
|
||||
if (query_processing_info.isIntermediateStage())
|
||||
@ -1561,7 +1532,7 @@ void Planner::buildPlanForQueryNode()
|
||||
planner_context,
|
||||
query_processing_info,
|
||||
query_tree,
|
||||
result_actions_to_execute);
|
||||
useful_sets);
|
||||
|
||||
if (expression_analysis_result.hasAggregation())
|
||||
{
|
||||
@ -1573,13 +1544,13 @@ void Planner::buildPlanForQueryNode()
|
||||
if (query_processing_info.isFirstStage())
|
||||
{
|
||||
if (expression_analysis_result.hasWhere())
|
||||
addFilterStep(query_plan, expression_analysis_result.getWhere(), "WHERE", result_actions_to_execute);
|
||||
addFilterStep(query_plan, expression_analysis_result.getWhere(), "WHERE", useful_sets);
|
||||
|
||||
if (expression_analysis_result.hasAggregation())
|
||||
{
|
||||
const auto & aggregation_analysis_result = expression_analysis_result.getAggregation();
|
||||
auto & aggregation_analysis_result = expression_analysis_result.getAggregation();
|
||||
if (aggregation_analysis_result.before_aggregation_actions)
|
||||
addExpressionStep(query_plan, aggregation_analysis_result.before_aggregation_actions, "Before GROUP BY", result_actions_to_execute);
|
||||
addExpressionStep(query_plan, aggregation_analysis_result.before_aggregation_actions, "Before GROUP BY", useful_sets);
|
||||
|
||||
addAggregationStep(query_plan, aggregation_analysis_result, query_analysis_result, planner_context, select_query_info);
|
||||
}
|
||||
@ -1596,9 +1567,9 @@ void Planner::buildPlanForQueryNode()
|
||||
* window functions, we can't execute ORDER BY and DISTINCT
|
||||
* now, on shard (first_stage).
|
||||
*/
|
||||
const auto & window_analysis_result = expression_analysis_result.getWindow();
|
||||
auto & window_analysis_result = expression_analysis_result.getWindow();
|
||||
if (window_analysis_result.before_window_actions)
|
||||
addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before WINDOW", result_actions_to_execute);
|
||||
addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before WINDOW", useful_sets);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -1606,8 +1577,8 @@ void Planner::buildPlanForQueryNode()
|
||||
* Projection expressions, preliminary DISTINCT and before ORDER BY expressions
|
||||
* now, on shards (first_stage).
|
||||
*/
|
||||
const auto & projection_analysis_result = expression_analysis_result.getProjection();
|
||||
addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", result_actions_to_execute);
|
||||
auto & projection_analysis_result = expression_analysis_result.getProjection();
|
||||
addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", useful_sets);
|
||||
|
||||
if (query_node.isDistinct())
|
||||
{
|
||||
@ -1622,8 +1593,8 @@ void Planner::buildPlanForQueryNode()
|
||||
|
||||
if (expression_analysis_result.hasSort())
|
||||
{
|
||||
const auto & sort_analysis_result = expression_analysis_result.getSort();
|
||||
addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", result_actions_to_execute);
|
||||
auto & sort_analysis_result = expression_analysis_result.getSort();
|
||||
addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", useful_sets);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1634,7 +1605,7 @@ void Planner::buildPlanForQueryNode()
|
||||
planner_context,
|
||||
query_processing_info,
|
||||
query_tree,
|
||||
result_actions_to_execute);
|
||||
useful_sets);
|
||||
}
|
||||
|
||||
if (query_processing_info.isSecondStage() || query_processing_info.isFromAggregationState())
|
||||
@ -1656,14 +1627,14 @@ void Planner::buildPlanForQueryNode()
|
||||
|
||||
if (query_node.isGroupByWithTotals())
|
||||
{
|
||||
addTotalsHavingStep(query_plan, expression_analysis_result, query_analysis_result, planner_context, query_node, result_actions_to_execute);
|
||||
addTotalsHavingStep(query_plan, expression_analysis_result, query_analysis_result, planner_context, query_node, useful_sets);
|
||||
having_executed = true;
|
||||
}
|
||||
|
||||
addCubeOrRollupStepIfNeeded(query_plan, aggregation_analysis_result, query_analysis_result, planner_context, select_query_info, query_node);
|
||||
|
||||
if (!having_executed && expression_analysis_result.hasHaving())
|
||||
addFilterStep(query_plan, expression_analysis_result.getHaving(), "HAVING", result_actions_to_execute);
|
||||
addFilterStep(query_plan, expression_analysis_result.getHaving(), "HAVING", useful_sets);
|
||||
}
|
||||
|
||||
if (query_processing_info.isFromAggregationState())
|
||||
@ -1676,18 +1647,18 @@ void Planner::buildPlanForQueryNode()
|
||||
{
|
||||
if (expression_analysis_result.hasWindow())
|
||||
{
|
||||
const auto & window_analysis_result = expression_analysis_result.getWindow();
|
||||
auto & window_analysis_result = expression_analysis_result.getWindow();
|
||||
if (expression_analysis_result.hasAggregation())
|
||||
addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before window functions", result_actions_to_execute);
|
||||
addExpressionStep(query_plan, window_analysis_result.before_window_actions, "Before window functions", useful_sets);
|
||||
|
||||
addWindowSteps(query_plan, planner_context, window_analysis_result);
|
||||
}
|
||||
|
||||
if (expression_analysis_result.hasQualify())
|
||||
addFilterStep(query_plan, expression_analysis_result.getQualify(), "QUALIFY", result_actions_to_execute);
|
||||
addFilterStep(query_plan, expression_analysis_result.getQualify(), "QUALIFY", useful_sets);
|
||||
|
||||
const auto & projection_analysis_result = expression_analysis_result.getProjection();
|
||||
addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", result_actions_to_execute);
|
||||
auto & projection_analysis_result = expression_analysis_result.getProjection();
|
||||
addExpressionStep(query_plan, projection_analysis_result.projection_actions, "Projection", useful_sets);
|
||||
|
||||
if (query_node.isDistinct())
|
||||
{
|
||||
@ -1702,8 +1673,8 @@ void Planner::buildPlanForQueryNode()
|
||||
|
||||
if (expression_analysis_result.hasSort())
|
||||
{
|
||||
const auto & sort_analysis_result = expression_analysis_result.getSort();
|
||||
addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", result_actions_to_execute);
|
||||
auto & sort_analysis_result = expression_analysis_result.getSort();
|
||||
addExpressionStep(query_plan, sort_analysis_result.before_order_by_actions, "Before ORDER BY", useful_sets);
|
||||
}
|
||||
}
|
||||
else
|
||||
@ -1755,8 +1726,8 @@ void Planner::buildPlanForQueryNode()
|
||||
|
||||
if (!query_processing_info.isFromAggregationState() && expression_analysis_result.hasLimitBy())
|
||||
{
|
||||
const auto & limit_by_analysis_result = expression_analysis_result.getLimitBy();
|
||||
addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", result_actions_to_execute);
|
||||
auto & limit_by_analysis_result = expression_analysis_result.getLimitBy();
|
||||
addExpressionStep(query_plan, limit_by_analysis_result.before_limit_by_actions, "Before LIMIT BY", useful_sets);
|
||||
addLimitByStep(query_plan, limit_by_analysis_result, query_node);
|
||||
}
|
||||
|
||||
@ -1787,8 +1758,8 @@ void Planner::buildPlanForQueryNode()
|
||||
/// Project names is not done on shards, because initiator will not find columns in blocks
|
||||
if (!query_processing_info.isToAggregationState())
|
||||
{
|
||||
const auto & projection_analysis_result = expression_analysis_result.getProjection();
|
||||
addExpressionStep(query_plan, projection_analysis_result.project_names_actions, "Project names", result_actions_to_execute);
|
||||
auto & projection_analysis_result = expression_analysis_result.getProjection();
|
||||
addExpressionStep(query_plan, projection_analysis_result.project_names_actions, "Project names", useful_sets);
|
||||
}
|
||||
|
||||
// For additional_result_filter setting
|
||||
@ -1796,7 +1767,7 @@ void Planner::buildPlanForQueryNode()
|
||||
}
|
||||
|
||||
if (!select_query_options.only_analyze)
|
||||
addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute);
|
||||
addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, useful_sets);
|
||||
|
||||
query_node_to_plan_step_mapping[&query_node] = query_plan.getRootNode();
|
||||
}
|
||||
|
@ -759,15 +759,15 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
|
||||
lambda_arguments_names_and_types.emplace_back(lambda_argument_name, std::move(lambda_argument_type));
|
||||
}
|
||||
|
||||
auto lambda_actions_dag = std::make_shared<ActionsDAG>();
|
||||
actions_stack.emplace_back(*lambda_actions_dag, node);
|
||||
ActionsDAG lambda_actions_dag;
|
||||
actions_stack.emplace_back(lambda_actions_dag, node);
|
||||
|
||||
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->removeUnusedActions(Names(1, 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));
|
||||
|
||||
auto expression_actions_settings = ExpressionActionsSettings::fromContext(planner_context->getQueryContext(), CompileExpressions::yes);
|
||||
auto lambda_actions = std::make_shared<ExpressionActions>(lambda_actions_dag, expression_actions_settings);
|
||||
auto lambda_actions = std::make_shared<ExpressionActions>(std::move(lambda_actions_dag), expression_actions_settings);
|
||||
|
||||
Names captured_column_names;
|
||||
ActionsDAG::NodeRawConstPtrs lambda_children;
|
||||
@ -881,14 +881,14 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
|
||||
const auto & function_node = node->as<FunctionNode &>();
|
||||
auto function_node_name = action_node_name_helper.calculateActionNodeName(node);
|
||||
|
||||
auto index_hint_actions_dag = std::make_shared<ActionsDAG>();
|
||||
auto & index_hint_actions_dag_outputs = index_hint_actions_dag->getOutputs();
|
||||
ActionsDAG index_hint_actions_dag;
|
||||
auto & index_hint_actions_dag_outputs = index_hint_actions_dag.getOutputs();
|
||||
std::unordered_set<std::string_view> index_hint_actions_dag_output_node_names;
|
||||
PlannerActionsVisitor actions_visitor(planner_context);
|
||||
|
||||
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)
|
||||
{
|
||||
|
@ -25,7 +25,7 @@ class TableNode;
|
||||
|
||||
struct FiltersForTableExpression
|
||||
{
|
||||
ActionsDAGPtr filter_actions;
|
||||
std::optional<ActionsDAG> filter_actions;
|
||||
PrewhereInfoPtr prewhere_info;
|
||||
};
|
||||
|
||||
|
@ -64,7 +64,7 @@ public:
|
||||
: projection_analysis_result(std::move(projection_analysis_result_))
|
||||
{}
|
||||
|
||||
const ProjectionAnalysisResult & getProjection() const
|
||||
ProjectionAnalysisResult & getProjection()
|
||||
{
|
||||
return projection_analysis_result;
|
||||
}
|
||||
@ -74,7 +74,7 @@ public:
|
||||
return where_analysis_result.filter_actions != nullptr;
|
||||
}
|
||||
|
||||
const FilterAnalysisResult & getWhere() const
|
||||
FilterAnalysisResult & getWhere()
|
||||
{
|
||||
return where_analysis_result;
|
||||
}
|
||||
@ -89,7 +89,7 @@ public:
|
||||
return !aggregation_analysis_result.aggregation_keys.empty() || !aggregation_analysis_result.aggregate_descriptions.empty();
|
||||
}
|
||||
|
||||
const AggregationAnalysisResult & getAggregation() const
|
||||
AggregationAnalysisResult & getAggregation()
|
||||
{
|
||||
return aggregation_analysis_result;
|
||||
}
|
||||
@ -104,7 +104,7 @@ public:
|
||||
return having_analysis_result.filter_actions != nullptr;
|
||||
}
|
||||
|
||||
const FilterAnalysisResult & getHaving() const
|
||||
FilterAnalysisResult & getHaving()
|
||||
{
|
||||
return having_analysis_result;
|
||||
}
|
||||
@ -119,7 +119,7 @@ public:
|
||||
return !window_analysis_result.window_descriptions.empty();
|
||||
}
|
||||
|
||||
const WindowAnalysisResult & getWindow() const
|
||||
WindowAnalysisResult & getWindow()
|
||||
{
|
||||
return window_analysis_result;
|
||||
}
|
||||
@ -134,7 +134,7 @@ public:
|
||||
return qualify_analysis_result.filter_actions != nullptr;
|
||||
}
|
||||
|
||||
const FilterAnalysisResult & getQualify() const
|
||||
FilterAnalysisResult & getQualify()
|
||||
{
|
||||
return qualify_analysis_result;
|
||||
}
|
||||
@ -149,7 +149,7 @@ public:
|
||||
return sort_analysis_result.before_order_by_actions != nullptr;
|
||||
}
|
||||
|
||||
const SortAnalysisResult & getSort() const
|
||||
SortAnalysisResult & getSort()
|
||||
{
|
||||
return sort_analysis_result;
|
||||
}
|
||||
@ -164,7 +164,7 @@ public:
|
||||
return limit_by_analysis_result.before_limit_by_actions != nullptr;
|
||||
}
|
||||
|
||||
const LimitByAnalysisResult & getLimitBy() const
|
||||
LimitByAnalysisResult & getLimitBy()
|
||||
{
|
||||
return limit_by_analysis_result;
|
||||
}
|
||||
|
@ -437,7 +437,7 @@ void updatePrewhereOutputsIfNeeded(SelectQueryInfo & table_expression_query_info
|
||||
|
||||
std::unordered_set<const ActionsDAG::Node *> required_output_nodes;
|
||||
|
||||
for (const auto * input : prewhere_actions->getInputs())
|
||||
for (const auto * input : prewhere_actions.getInputs())
|
||||
{
|
||||
if (required_columns.contains(input->result_name))
|
||||
required_output_nodes.insert(input);
|
||||
@ -446,7 +446,7 @@ void updatePrewhereOutputsIfNeeded(SelectQueryInfo & table_expression_query_info
|
||||
if (required_output_nodes.empty())
|
||||
return;
|
||||
|
||||
auto & prewhere_outputs = prewhere_actions->getOutputs();
|
||||
auto & prewhere_outputs = prewhere_actions.getOutputs();
|
||||
for (const auto & output : prewhere_outputs)
|
||||
{
|
||||
auto required_output_node_it = required_output_nodes.find(output);
|
||||
@ -459,7 +459,7 @@ void updatePrewhereOutputsIfNeeded(SelectQueryInfo & table_expression_query_info
|
||||
prewhere_outputs.insert(prewhere_outputs.end(), required_output_nodes.begin(), required_output_nodes.end());
|
||||
}
|
||||
|
||||
FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage,
|
||||
std::optional<FilterDAGInfo> buildRowPolicyFilterIfNeeded(const StoragePtr & storage,
|
||||
SelectQueryInfo & table_expression_query_info,
|
||||
PlannerContextPtr & planner_context,
|
||||
std::set<std::string> & used_row_policies)
|
||||
@ -480,7 +480,7 @@ FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage,
|
||||
return buildFilterInfo(row_policy_filter->expression, table_expression_query_info.table_expression, planner_context);
|
||||
}
|
||||
|
||||
FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage,
|
||||
std::optional<FilterDAGInfo> buildCustomKeyFilterIfNeeded(const StoragePtr & storage,
|
||||
SelectQueryInfo & table_expression_query_info,
|
||||
PlannerContextPtr & planner_context)
|
||||
{
|
||||
@ -514,7 +514,7 @@ FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage,
|
||||
}
|
||||
|
||||
/// Apply filters from additional_table_filters setting
|
||||
FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage,
|
||||
std::optional<FilterDAGInfo> buildAdditionalFiltersIfNeeded(const StoragePtr & storage,
|
||||
const String & table_expression_alias,
|
||||
SelectQueryInfo & table_expression_query_info,
|
||||
PlannerContextPtr & planner_context)
|
||||
@ -590,21 +590,21 @@ UInt64 mainQueryNodeBlockSizeByLimit(const SelectQueryInfo & select_query_info)
|
||||
}
|
||||
|
||||
std::unique_ptr<ExpressionStep> createComputeAliasColumnsStep(
|
||||
const std::unordered_map<std::string, ActionsDAGPtr> & alias_column_expressions, const DataStream & current_data_stream)
|
||||
std::unordered_map<std::string, ActionsDAG> & alias_column_expressions, const DataStream & current_data_stream)
|
||||
{
|
||||
ActionsDAGPtr merged_alias_columns_actions_dag = std::make_shared<ActionsDAG>(current_data_stream.header.getColumnsWithTypeAndName());
|
||||
ActionsDAG::NodeRawConstPtrs action_dag_outputs = merged_alias_columns_actions_dag->getInputs();
|
||||
ActionsDAG merged_alias_columns_actions_dag(current_data_stream.header.getColumnsWithTypeAndName());
|
||||
ActionsDAG::NodeRawConstPtrs action_dag_outputs = merged_alias_columns_actions_dag.getInputs();
|
||||
|
||||
for (const auto & [column_name, alias_column_actions_dag] : alias_column_expressions)
|
||||
for (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());
|
||||
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)
|
||||
merged_alias_columns_actions_dag->addOrReplaceInOutputs(*output_node);
|
||||
merged_alias_columns_actions_dag->removeUnusedActions(false);
|
||||
merged_alias_columns_actions_dag.addOrReplaceInOutputs(*output_node);
|
||||
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));
|
||||
alias_column_step->setStepDescription("Compute alias columns");
|
||||
@ -647,7 +647,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
|
||||
auto table_expression_query_info = select_query_info;
|
||||
table_expression_query_info.table_expression = table_expression;
|
||||
table_expression_query_info.filter_actions_dag = table_expression_data.getFilterActions();
|
||||
if (const auto & filter_actions = table_expression_data.getFilterActions())
|
||||
table_expression_query_info.filter_actions_dag = std::make_shared<const ActionsDAG>(filter_actions->clone());
|
||||
table_expression_query_info.current_table_chosen_for_reading_with_parallel_replicas
|
||||
= table_node == planner_context->getGlobalPlannerContext()->parallel_replicas_table;
|
||||
|
||||
@ -778,7 +779,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
if (prewhere_actions)
|
||||
{
|
||||
prewhere_info = std::make_shared<PrewhereInfo>();
|
||||
prewhere_info->prewhere_actions = prewhere_actions;
|
||||
prewhere_info->prewhere_actions = prewhere_actions->clone();
|
||||
prewhere_info->prewhere_column_name = prewhere_actions->getOutputs().at(0)->result_name;
|
||||
prewhere_info->remove_prewhere_column = true;
|
||||
prewhere_info->need_filter = true;
|
||||
@ -789,11 +790,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
const auto & columns_names = table_expression_data.getColumnNames();
|
||||
|
||||
std::vector<std::pair<FilterDAGInfo, std::string>> where_filters;
|
||||
const auto add_filter = [&](const FilterDAGInfo & filter_info, std::string description)
|
||||
const auto add_filter = [&](FilterDAGInfo & filter_info, std::string description)
|
||||
{
|
||||
if (!filter_info.actions)
|
||||
return;
|
||||
|
||||
bool is_final = table_expression_query_info.table_expression_modifiers
|
||||
&& table_expression_query_info.table_expression_modifiers->hasFinal();
|
||||
bool optimize_move_to_prewhere
|
||||
@ -803,46 +801,45 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
if (storage->canMoveConditionsToPrewhere() && optimize_move_to_prewhere && (!supported_prewhere_columns || supported_prewhere_columns->contains(filter_info.column_name)))
|
||||
{
|
||||
if (!prewhere_info)
|
||||
prewhere_info = std::make_shared<PrewhereInfo>();
|
||||
|
||||
if (!prewhere_info->prewhere_actions)
|
||||
{
|
||||
prewhere_info->prewhere_actions = filter_info.actions;
|
||||
prewhere_info = std::make_shared<PrewhereInfo>();
|
||||
prewhere_info->prewhere_actions = std::move(filter_info.actions);
|
||||
prewhere_info->prewhere_column_name = filter_info.column_name;
|
||||
prewhere_info->remove_prewhere_column = filter_info.do_remove_column;
|
||||
prewhere_info->need_filter = true;
|
||||
}
|
||||
else if (!prewhere_info->row_level_filter)
|
||||
{
|
||||
prewhere_info->row_level_filter = 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->need_filter = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
where_filters.emplace_back(filter_info, std::move(description));
|
||||
where_filters.emplace_back(std::move(filter_info), std::move(description));
|
||||
}
|
||||
|
||||
}
|
||||
else
|
||||
{
|
||||
where_filters.emplace_back(filter_info, std::move(description));
|
||||
where_filters.emplace_back(std::move(filter_info), std::move(description));
|
||||
}
|
||||
};
|
||||
|
||||
auto row_policy_filter_info
|
||||
= buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context, used_row_policies);
|
||||
add_filter(row_policy_filter_info, "Row-level security filter");
|
||||
if (row_policy_filter_info.actions)
|
||||
table_expression_data.setRowLevelFilterActions(row_policy_filter_info.actions);
|
||||
if (row_policy_filter_info)
|
||||
{
|
||||
table_expression_data.setRowLevelFilterActions(row_policy_filter_info->actions.clone());
|
||||
add_filter(*row_policy_filter_info, "Row-level security filter");
|
||||
}
|
||||
|
||||
if (query_context->canUseParallelReplicasCustomKey())
|
||||
{
|
||||
if (settings.parallel_replicas_count > 1)
|
||||
{
|
||||
auto parallel_replicas_custom_key_filter_info
|
||||
= buildCustomKeyFilterIfNeeded(storage, table_expression_query_info, planner_context);
|
||||
add_filter(parallel_replicas_custom_key_filter_info, "Parallel replicas custom key filter");
|
||||
if (auto parallel_replicas_custom_key_filter_info= buildCustomKeyFilterIfNeeded(storage, table_expression_query_info, planner_context))
|
||||
add_filter(*parallel_replicas_custom_key_filter_info, "Parallel replicas custom key filter");
|
||||
}
|
||||
else if (auto * distributed = typeid_cast<StorageDistributed *>(storage.get());
|
||||
distributed && query_context->canUseParallelReplicasCustomKeyForCluster(*distributed->getCluster()))
|
||||
@ -857,9 +854,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
}
|
||||
|
||||
const auto & table_expression_alias = table_expression->getOriginalAlias();
|
||||
auto additional_filters_info
|
||||
= buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context);
|
||||
add_filter(additional_filters_info, "additional filter");
|
||||
if (auto additional_filters_info = buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context))
|
||||
add_filter(*additional_filters_info, "additional filter");
|
||||
|
||||
from_stage = storage->getQueryProcessingStage(
|
||||
query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info);
|
||||
@ -997,22 +993,20 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
}
|
||||
}
|
||||
|
||||
const auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions();
|
||||
auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions();
|
||||
if (!alias_column_expressions.empty() && query_plan.isInitialized() && from_stage == QueryProcessingStage::FetchColumns)
|
||||
{
|
||||
auto alias_column_step = createComputeAliasColumnsStep(alias_column_expressions, query_plan.getCurrentDataStream());
|
||||
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() &&
|
||||
from_stage == QueryProcessingStage::FetchColumns &&
|
||||
filter_info.actions)
|
||||
from_stage == QueryProcessingStage::FetchColumns)
|
||||
{
|
||||
auto filter_step = std::make_unique<FilterStep>(query_plan.getCurrentDataStream(),
|
||||
filter_info.actions,
|
||||
std::move(filter_info.actions),
|
||||
filter_info.column_name,
|
||||
filter_info.do_remove_column);
|
||||
filter_step->setStepDescription(description);
|
||||
@ -1088,7 +1082,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
query_plan = std::move(subquery_planner).extractQueryPlan();
|
||||
}
|
||||
|
||||
const auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions();
|
||||
auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions();
|
||||
if (!alias_column_expressions.empty() && query_plan.isInitialized() && from_stage == QueryProcessingStage::FetchColumns)
|
||||
{
|
||||
auto alias_column_step = createComputeAliasColumnsStep(alias_column_expressions, query_plan.getCurrentDataStream());
|
||||
@ -1103,21 +1097,21 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
|
||||
if (from_stage == QueryProcessingStage::FetchColumns)
|
||||
{
|
||||
auto rename_actions_dag = std::make_shared<ActionsDAG>(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName());
|
||||
ActionsDAG rename_actions_dag(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName());
|
||||
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);
|
||||
if (!column_identifier)
|
||||
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(), 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");
|
||||
query_plan.addStep(std::move(rename_step));
|
||||
}
|
||||
@ -1157,9 +1151,9 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
|
||||
void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextPtr & planner_context, const FunctionOverloadResolverPtr & to_nullable_function)
|
||||
{
|
||||
auto cast_actions_dag = std::make_shared<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))
|
||||
{
|
||||
@ -1168,11 +1162,11 @@ void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextP
|
||||
type_to_check = type_to_check_low_cardinality->getDictionaryType();
|
||||
|
||||
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));
|
||||
cast_join_columns_step->setStepDescription("Cast JOIN columns to Nullable");
|
||||
plan_to_add_cast.addStep(std::move(cast_join_columns_step));
|
||||
@ -1218,14 +1212,16 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
|
||||
join_table_expression,
|
||||
planner_context);
|
||||
|
||||
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);
|
||||
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(), std::move(join_clauses_and_actions.left_join_expressions_actions));
|
||||
left_join_expressions_actions_step->setStepDescription("JOIN actions");
|
||||
appendSetsFromActionsDAG(left_join_expressions_actions_step->getExpression(), left_join_tree_query_plan.useful_sets);
|
||||
left_plan.addStep(std::move(left_join_expressions_actions_step));
|
||||
|
||||
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);
|
||||
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(), std::move(join_clauses_and_actions.right_join_expressions_actions));
|
||||
right_join_expressions_actions_step->setStepDescription("JOIN actions");
|
||||
appendSetsFromActionsDAG(right_join_expressions_actions_step->getExpression(), right_join_tree_query_plan.useful_sets);
|
||||
right_plan.addStep(std::move(right_join_expressions_actions_step));
|
||||
}
|
||||
|
||||
@ -1263,19 +1259,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 cast_actions_dag = std::make_shared<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);
|
||||
if (it == plan_column_name_to_cast_type.end())
|
||||
continue;
|
||||
|
||||
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
|
||||
= std::make_unique<ExpressionStep>(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag));
|
||||
cast_join_columns_step->setStepDescription("Cast JOIN USING columns");
|
||||
@ -1418,8 +1414,10 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
|
||||
{
|
||||
ExpressionActionsPtr & mixed_join_expression = table_join->getMixedJoinExpression();
|
||||
mixed_join_expression = std::make_shared<ExpressionActions>(
|
||||
join_clauses_and_actions.mixed_join_expressions_actions,
|
||||
std::move(*join_clauses_and_actions.mixed_join_expressions_actions),
|
||||
ExpressionActionsSettings::fromContext(planner_context->getQueryContext()));
|
||||
|
||||
appendSetsFromActionsDAG(mixed_join_expression->getActionsDAG(), left_join_tree_query_plan.useful_sets);
|
||||
}
|
||||
}
|
||||
else if (join_node.isUsingJoinExpression())
|
||||
@ -1465,7 +1463,8 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
|
||||
|
||||
auto result_plan = QueryPlan();
|
||||
|
||||
if (join_algorithm->isFilled())
|
||||
bool is_filled_join = join_algorithm->isFilled();
|
||||
if (is_filled_join)
|
||||
{
|
||||
auto filled_join_step = std::make_unique<FilledJoinStep>(
|
||||
left_plan.getCurrentDataStream(),
|
||||
@ -1574,12 +1573,12 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
|
||||
result_plan.unitePlans(std::move(join_step), {std::move(plans)});
|
||||
}
|
||||
|
||||
auto drop_unused_columns_after_join_actions_dag = std::make_shared<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;
|
||||
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;
|
||||
|
||||
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();
|
||||
|
||||
for (size_t i = 0; i < drop_unused_columns_after_join_actions_dag_outputs_size; ++i)
|
||||
@ -1618,15 +1617,10 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
|
||||
for (const auto & right_join_tree_query_plan_row_policy : right_join_tree_query_plan.used_row_policies)
|
||||
left_join_tree_query_plan.used_row_policies.insert(right_join_tree_query_plan_row_policy);
|
||||
|
||||
/// Collect all required actions dags in `left_join_tree_query_plan.actions_dags`
|
||||
for (auto && action_dag : right_join_tree_query_plan.actions_dags)
|
||||
left_join_tree_query_plan.actions_dags.emplace_back(action_dag);
|
||||
if (join_clauses_and_actions.left_join_expressions_actions)
|
||||
left_join_tree_query_plan.actions_dags.emplace_back(std::move(join_clauses_and_actions.left_join_expressions_actions));
|
||||
if (join_clauses_and_actions.right_join_expressions_actions)
|
||||
left_join_tree_query_plan.actions_dags.emplace_back(std::move(join_clauses_and_actions.right_join_expressions_actions));
|
||||
if (join_clauses_and_actions.mixed_join_expressions_actions)
|
||||
left_join_tree_query_plan.actions_dags.push_back(join_clauses_and_actions.mixed_join_expressions_actions);
|
||||
/// Collect all required actions sets in `left_join_tree_query_plan.useful_sets`
|
||||
if (!is_filled_join)
|
||||
for (const auto & useful_set : right_join_tree_query_plan.useful_sets)
|
||||
left_join_tree_query_plan.useful_sets.insert(useful_set);
|
||||
|
||||
auto mapping = std::move(left_join_tree_query_plan.query_node_to_plan_step_mapping);
|
||||
auto & r_mapping = right_join_tree_query_plan.query_node_to_plan_step_mapping;
|
||||
@ -1636,7 +1630,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
|
||||
.query_plan = std::move(result_plan),
|
||||
.from_stage = QueryProcessingStage::FetchColumns,
|
||||
.used_row_policies = std::move(left_join_tree_query_plan.used_row_policies),
|
||||
.actions_dags = std::move(left_join_tree_query_plan.actions_dags),
|
||||
.useful_sets = std::move(left_join_tree_query_plan.useful_sets),
|
||||
.query_node_to_plan_step_mapping = std::move(mapping),
|
||||
};
|
||||
}
|
||||
@ -1656,7 +1650,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_
|
||||
auto plan = std::move(join_tree_query_plan.query_plan);
|
||||
auto plan_output_columns = plan.getCurrentDataStream().header.getColumnsWithTypeAndName();
|
||||
|
||||
ActionsDAGPtr array_join_action_dag = std::make_shared<ActionsDAG>(plan_output_columns);
|
||||
ActionsDAG array_join_action_dag(plan_output_columns);
|
||||
PlannerActionsVisitor actions_visitor(planner_context);
|
||||
std::unordered_set<std::string> array_join_expressions_output_nodes;
|
||||
|
||||
@ -1667,29 +1661,28 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_
|
||||
array_join_column_names.insert(array_join_column_identifier);
|
||||
|
||||
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)
|
||||
{
|
||||
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);
|
||||
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_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);
|
||||
|
||||
join_tree_query_plan.actions_dags.push_back(array_join_action_dag);
|
||||
|
||||
auto array_join_actions = std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), 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");
|
||||
appendSetsFromActionsDAG(array_join_actions->getExpression(), join_tree_query_plan.useful_sets);
|
||||
plan.addStep(std::move(array_join_actions));
|
||||
|
||||
auto drop_unused_columns_before_array_join_actions_dag = std::make_shared<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;
|
||||
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();
|
||||
|
||||
for (size_t i = 0; i < drop_unused_columns_before_array_join_actions_dag_outputs_size; ++i)
|
||||
@ -1723,7 +1716,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_
|
||||
.query_plan = std::move(plan),
|
||||
.from_stage = QueryProcessingStage::FetchColumns,
|
||||
.used_row_policies = std::move(join_tree_query_plan.used_row_policies),
|
||||
.actions_dags = std::move(join_tree_query_plan.actions_dags),
|
||||
.useful_sets = std::move(join_tree_query_plan.useful_sets),
|
||||
.query_node_to_plan_step_mapping = std::move(join_tree_query_plan.query_node_to_plan_step_mapping),
|
||||
};
|
||||
}
|
||||
|
@ -11,12 +11,14 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using UsefulSets = std::unordered_set<FutureSetPtr>;
|
||||
|
||||
struct JoinTreeQueryPlan
|
||||
{
|
||||
QueryPlan query_plan;
|
||||
QueryProcessingStage::Enum from_stage;
|
||||
std::set<std::string> used_row_policies{};
|
||||
std::vector<ActionsDAGPtr> actions_dags{};
|
||||
UsefulSets useful_sets{};
|
||||
std::unordered_map<const QueryNode *, const QueryPlan::Node *> query_node_to_plan_step_mapping{};
|
||||
};
|
||||
|
||||
|
@ -180,13 +180,13 @@ std::set<JoinTableSide> extractJoinTableSidesFromExpression(//const ActionsDAG::
|
||||
}
|
||||
|
||||
const ActionsDAG::Node * appendExpression(
|
||||
ActionsDAGPtr & dag,
|
||||
ActionsDAG & dag,
|
||||
const QueryTreeNodePtr & expression,
|
||||
const PlannerContextPtr & planner_context,
|
||||
const JoinNode & join_node)
|
||||
{
|
||||
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)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"JOIN {} ON clause contains multiple expressions",
|
||||
@ -196,9 +196,9 @@ const ActionsDAG::Node * appendExpression(
|
||||
}
|
||||
|
||||
void buildJoinClause(
|
||||
ActionsDAGPtr & left_dag,
|
||||
ActionsDAGPtr & right_dag,
|
||||
ActionsDAGPtr & mixed_dag,
|
||||
ActionsDAG & left_dag,
|
||||
ActionsDAG & right_dag,
|
||||
ActionsDAG & mixed_dag,
|
||||
const PlannerContextPtr & planner_context,
|
||||
const QueryTreeNodePtr & join_expression,
|
||||
const TableExpressionSet & left_table_expressions,
|
||||
@ -379,8 +379,8 @@ JoinClausesAndActions buildJoinClausesAndActions(
|
||||
const JoinNode & join_node,
|
||||
const PlannerContextPtr & planner_context)
|
||||
{
|
||||
ActionsDAGPtr left_join_actions = std::make_shared<ActionsDAG>(left_table_expression_columns);
|
||||
ActionsDAGPtr right_join_actions = std::make_shared<ActionsDAG>(right_table_expression_columns);
|
||||
ActionsDAG left_join_actions(left_table_expression_columns);
|
||||
ActionsDAG right_join_actions(right_table_expression_columns);
|
||||
ColumnsWithTypeAndName mixed_table_expression_columns;
|
||||
for (const auto & left_column : left_table_expression_columns)
|
||||
{
|
||||
@ -390,7 +390,7 @@ JoinClausesAndActions buildJoinClausesAndActions(
|
||||
{
|
||||
mixed_table_expression_columns.push_back(right_column);
|
||||
}
|
||||
ActionsDAGPtr mixed_join_actions = std::make_shared<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.
|
||||
* If we do not ignore it, this function will be replaced by underlying constant.
|
||||
@ -501,12 +501,12 @@ JoinClausesAndActions buildJoinClausesAndActions(
|
||||
{
|
||||
const ActionsDAG::Node * dag_filter_condition_node = nullptr;
|
||||
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
|
||||
dag_filter_condition_node = left_filter_condition_nodes[0];
|
||||
|
||||
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);
|
||||
}
|
||||
@ -517,12 +517,12 @@ JoinClausesAndActions buildJoinClausesAndActions(
|
||||
const ActionsDAG::Node * dag_filter_condition_node = nullptr;
|
||||
|
||||
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
|
||||
dag_filter_condition_node = right_filter_condition_nodes[0];
|
||||
|
||||
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);
|
||||
}
|
||||
@ -559,10 +559,10 @@ JoinClausesAndActions buildJoinClausesAndActions(
|
||||
}
|
||||
|
||||
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))
|
||||
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())
|
||||
@ -579,24 +579,24 @@ JoinClausesAndActions buildJoinClausesAndActions(
|
||||
* SELECT * FROM t1 JOIN t2 ON tuple(t1.a) == tuple(t2.b)
|
||||
*/
|
||||
auto wrap_nullsafe_function = FunctionFactory::instance().get("tuple", planner_context->getQueryContext());
|
||||
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}, {});
|
||||
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}, {});
|
||||
}
|
||||
|
||||
left_join_actions->addOrReplaceInOutputs(*left_key_node);
|
||||
right_join_actions->addOrReplaceInOutputs(*right_key_node);
|
||||
left_join_actions.addOrReplaceInOutputs(*left_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::Right, right_key_node->result_name);
|
||||
}
|
||||
}
|
||||
|
||||
result.left_join_expressions_actions = left_join_actions->clone();
|
||||
result.left_join_expressions_actions = left_join_actions.clone();
|
||||
result.left_join_tmp_expression_actions = std::move(left_join_actions);
|
||||
result.left_join_expressions_actions->removeUnusedActions(join_left_actions_names);
|
||||
result.right_join_expressions_actions = right_join_actions->clone();
|
||||
result.left_join_expressions_actions.removeUnusedActions(join_left_actions_names);
|
||||
result.right_join_expressions_actions = right_join_actions.clone();
|
||||
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)
|
||||
{
|
||||
@ -604,24 +604,24 @@ JoinClausesAndActions buildJoinClausesAndActions(
|
||||
/// 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)
|
||||
{
|
||||
auto mixed_join_expressions_actions = std::make_shared<ActionsDAG>(mixed_table_expression_columns);
|
||||
ActionsDAG mixed_join_expressions_actions(mixed_table_expression_columns);
|
||||
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)
|
||||
throw Exception(
|
||||
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};
|
||||
mixed_join_expressions_actions->removeUnusedActions(required_names);
|
||||
result.mixed_join_expressions_actions = mixed_join_expressions_actions;
|
||||
mixed_join_expressions_actions.removeUnusedActions(required_names);
|
||||
result.mixed_join_expressions_actions = std::move(mixed_join_expressions_actions);
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto & join_clause = result.join_clauses.front();
|
||||
const auto & mixed_filter_condition_nodes = join_clause.getMixedFilterConditionNodes();
|
||||
auto mixed_join_expressions_actions = ActionsDAG::buildFilterActionsDAG(mixed_filter_condition_nodes, {}, true);
|
||||
result.mixed_join_expressions_actions = mixed_join_expressions_actions;
|
||||
result.mixed_join_expressions_actions = std::move(mixed_join_expressions_actions);
|
||||
}
|
||||
auto outputs = result.mixed_join_expressions_actions->getOutputs();
|
||||
if (outputs.size() != 1)
|
||||
|
@ -182,15 +182,15 @@ struct JoinClausesAndActions
|
||||
/// Join clauses. Actions dag nodes point into join_expression_actions.
|
||||
JoinClauses join_clauses;
|
||||
/// Whole JOIN ON section expressions
|
||||
ActionsDAGPtr left_join_tmp_expression_actions;
|
||||
ActionsDAGPtr right_join_tmp_expression_actions;
|
||||
ActionsDAG left_join_tmp_expression_actions;
|
||||
ActionsDAG right_join_tmp_expression_actions;
|
||||
/// Left join expressions actions
|
||||
ActionsDAGPtr left_join_expressions_actions;
|
||||
ActionsDAG left_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.
|
||||
/// 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.
|
||||
|
@ -73,7 +73,7 @@ public:
|
||||
}
|
||||
|
||||
/// Add alias column
|
||||
void addAliasColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier, ActionsDAGPtr actions_dag, bool is_selected_column = true)
|
||||
void addAliasColumn(const NameAndTypePair & column, const ColumnIdentifier & column_identifier, ActionsDAG actions_dag, bool is_selected_column = true)
|
||||
{
|
||||
alias_column_expressions.emplace(column.name, std::move(actions_dag));
|
||||
addColumnImpl(column, column_identifier, is_selected_column);
|
||||
@ -94,7 +94,7 @@ public:
|
||||
}
|
||||
|
||||
/// Get ALIAS columns names mapped to expressions
|
||||
const std::unordered_map<std::string, ActionsDAGPtr> & getAliasColumnExpressions() const
|
||||
std::unordered_map<std::string, ActionsDAG> & getAliasColumnExpressions()
|
||||
{
|
||||
return alias_column_expressions;
|
||||
}
|
||||
@ -211,32 +211,32 @@ public:
|
||||
is_merge_tree = is_merge_tree_value;
|
||||
}
|
||||
|
||||
const ActionsDAGPtr & getPrewhereFilterActions() const
|
||||
const std::optional<ActionsDAG> & getPrewhereFilterActions() const
|
||||
{
|
||||
return prewhere_filter_actions;
|
||||
}
|
||||
|
||||
void setRowLevelFilterActions(ActionsDAGPtr row_level_filter_actions_value)
|
||||
void setRowLevelFilterActions(ActionsDAG row_level_filter_actions_value)
|
||||
{
|
||||
row_level_filter_actions = std::move(row_level_filter_actions_value);
|
||||
}
|
||||
|
||||
const ActionsDAGPtr & getRowLevelFilterActions() const
|
||||
const std::optional<ActionsDAG> & getRowLevelFilterActions() const
|
||||
{
|
||||
return row_level_filter_actions;
|
||||
}
|
||||
|
||||
void setPrewhereFilterActions(ActionsDAGPtr prewhere_filter_actions_value)
|
||||
void setPrewhereFilterActions(ActionsDAG prewhere_filter_actions_value)
|
||||
{
|
||||
prewhere_filter_actions = std::move(prewhere_filter_actions_value);
|
||||
}
|
||||
|
||||
const ActionsDAGPtr & getFilterActions() const
|
||||
const std::optional<ActionsDAG> & getFilterActions() const
|
||||
{
|
||||
return filter_actions;
|
||||
}
|
||||
|
||||
void setFilterActions(ActionsDAGPtr filter_actions_value)
|
||||
void setFilterActions(ActionsDAG filter_actions_value)
|
||||
{
|
||||
filter_actions = std::move(filter_actions_value);
|
||||
}
|
||||
@ -277,7 +277,7 @@ private:
|
||||
NameSet selected_column_names_set;
|
||||
|
||||
/// Expression to calculate ALIAS columns
|
||||
std::unordered_map<std::string, ActionsDAGPtr> alias_column_expressions;
|
||||
std::unordered_map<std::string, ActionsDAG> alias_column_expressions;
|
||||
|
||||
/// Valid for table, table function, array join, query, union nodes
|
||||
ColumnNameToColumn column_name_to_column;
|
||||
@ -289,16 +289,16 @@ private:
|
||||
ColumnIdentifierToColumnName column_identifier_to_column_name;
|
||||
|
||||
/// Valid for table, table function
|
||||
ActionsDAGPtr filter_actions;
|
||||
std::optional<ActionsDAG> filter_actions;
|
||||
|
||||
/// Valid for table, table function
|
||||
PrewhereInfoPtr prewhere_info;
|
||||
|
||||
/// Valid for table, table function
|
||||
ActionsDAGPtr prewhere_filter_actions;
|
||||
std::optional<ActionsDAG> prewhere_filter_actions;
|
||||
|
||||
/// Valid for table, table function
|
||||
ActionsDAGPtr row_level_filter_actions;
|
||||
std::optional<ActionsDAG> row_level_filter_actions;
|
||||
|
||||
/// Is storage remote
|
||||
bool is_remote = false;
|
||||
|
@ -11,10 +11,12 @@
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
|
||||
#include <Columns/getLeastSuperColumn.h>
|
||||
#include <Columns/ColumnSet.h>
|
||||
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/indexHint.h>
|
||||
|
||||
#include <Storages/StorageDummy.h>
|
||||
|
||||
@ -442,22 +444,22 @@ FilterDAGInfo buildFilterInfo(QueryTreeNodePtr filter_query_tree,
|
||||
collectSourceColumns(filter_query_tree, planner_context, false /*keep_alias_columns*/);
|
||||
collectSets(filter_query_tree, *planner_context);
|
||||
|
||||
auto filter_actions_dag = std::make_shared<ActionsDAG>();
|
||||
ActionsDAG filter_actions_dag;
|
||||
|
||||
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)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Filter actions must return single output node. Actual {}",
|
||||
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);
|
||||
|
||||
std::string filter_node_name = filter_actions_outputs[0]->result_name;
|
||||
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))
|
||||
filter_actions_outputs.push_back(filter_input_node);
|
||||
|
||||
@ -477,4 +479,32 @@ ASTPtr parseAdditionalResultFilter(const Settings & settings)
|
||||
return additional_result_filter_ast;
|
||||
}
|
||||
|
||||
void appendSetsFromActionsDAG(const ActionsDAG & dag, UsefulSets & useful_sets)
|
||||
{
|
||||
for (const auto & node : dag.getNodes())
|
||||
{
|
||||
if (node.column)
|
||||
{
|
||||
const IColumn * column = node.column.get();
|
||||
if (const auto * column_const = typeid_cast<const ColumnConst *>(column))
|
||||
column = &column_const->getDataColumn();
|
||||
|
||||
if (const auto * column_set = typeid_cast<const ColumnSet *>(column))
|
||||
useful_sets.insert(column_set->getData());
|
||||
}
|
||||
|
||||
if (node.type == ActionsDAG::ActionType::FUNCTION && node.function_base->getName() == "indexHint")
|
||||
{
|
||||
ActionsDAG::NodeRawConstPtrs children;
|
||||
if (const auto * adaptor = typeid_cast<const FunctionToFunctionBaseAdaptor *>(node.function_base.get()))
|
||||
{
|
||||
if (const auto * index_hint = typeid_cast<const FunctionIndexHint *>(adaptor->getFunction().get()))
|
||||
{
|
||||
appendSetsFromActionsDAG(index_hint->getActions(), useful_sets);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -88,4 +88,7 @@ FilterDAGInfo buildFilterInfo(QueryTreeNodePtr filter_query_tree,
|
||||
|
||||
ASTPtr parseAdditionalResultFilter(const Settings & settings);
|
||||
|
||||
using UsefulSets = std::unordered_set<FutureSetPtr>;
|
||||
void appendSetsFromActionsDAG(const ActionsDAG & dag, UsefulSets & useful_sets);
|
||||
|
||||
}
|
||||
|
@ -303,15 +303,15 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B
|
||||
const auto & header = ports[set_counter]->getHeader();
|
||||
|
||||
/// Here we create a DAG which fills missing keys and adds `__grouping_set` column
|
||||
auto dag = std::make_shared<ActionsDAG>(header.getColumnsWithTypeAndName());
|
||||
ActionsDAG dag(header.getColumnsWithTypeAndName());
|
||||
ActionsDAG::NodeRawConstPtrs outputs;
|
||||
outputs.reserve(output_header.columns() + 1);
|
||||
|
||||
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"});
|
||||
|
||||
grouping_node = &dag->materializeNode(*grouping_node);
|
||||
grouping_node = &dag.materializeNode(*grouping_node);
|
||||
outputs.push_back(grouping_node);
|
||||
|
||||
const auto & missing_columns = grouping_sets_params[set_counter].missing_keys;
|
||||
@ -332,22 +332,22 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B
|
||||
column_with_default->finalize();
|
||||
|
||||
auto column = ColumnConst::create(std::move(column_with_default), 0);
|
||||
const auto * node = &dag->addColumn({ColumnPtr(std::move(column)), col.type, col.name});
|
||||
node = &dag->materializeNode(*node);
|
||||
const auto * node = &dag.addColumn({ColumnPtr(std::move(column)), col.type, col.name});
|
||||
node = &dag.materializeNode(*node);
|
||||
outputs.push_back(node);
|
||||
}
|
||||
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())
|
||||
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
|
||||
outputs.push_back(column_node);
|
||||
}
|
||||
}
|
||||
|
||||
dag->getOutputs().swap(outputs);
|
||||
auto expression = std::make_shared<ExpressionActions>(dag, settings.getActionsSettings());
|
||||
dag.getOutputs().swap(outputs);
|
||||
auto expression = std::make_shared<ExpressionActions>(std::move(dag), settings.getActionsSettings());
|
||||
auto transform = std::make_shared<ExpressionTransform>(header, expression);
|
||||
|
||||
connect(*ports[set_counter], transform->getInputPort());
|
||||
|
@ -36,30 +36,30 @@ 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)
|
||||
{
|
||||
auto dag = std::make_shared<ActionsDAG>(header.getColumnsWithTypeAndName());
|
||||
auto & outputs = dag->getOutputs();
|
||||
ActionsDAG dag(header.getColumnsWithTypeAndName());
|
||||
auto & outputs = dag.getOutputs();
|
||||
|
||||
if (use_nulls)
|
||||
{
|
||||
auto to_nullable = FunctionFactory::instance().get("toNullable", nullptr);
|
||||
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())
|
||||
{
|
||||
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);
|
||||
const auto * grouping_node = &dag->addColumn(
|
||||
const auto * grouping_node = &dag.addColumn(
|
||||
{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);
|
||||
|
||||
auto expression = std::make_shared<ExpressionActions>(dag, settings.getActionsSettings());
|
||||
auto expression = std::make_shared<ExpressionActions>(std::move(dag), settings.getActionsSettings());
|
||||
return std::make_shared<ExpressionTransform>(header, expression);
|
||||
}
|
||||
|
||||
|
@ -33,7 +33,7 @@ void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missi
|
||||
};
|
||||
|
||||
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));
|
||||
}
|
||||
|
||||
|
@ -10,33 +10,33 @@
|
||||
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
|
||||
{
|
||||
{
|
||||
.returns_single_stream = false,
|
||||
.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(
|
||||
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))
|
||||
, actions_dag(actions_dag_)
|
||||
, actions_dag(std::move(actions_dag_))
|
||||
{
|
||||
}
|
||||
|
||||
void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings)
|
||||
{
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag, settings.getActionsSettings());
|
||||
auto expression = std::make_shared<ExpressionActions>(std::move(actions_dag), settings.getActionsSettings());
|
||||
|
||||
pipeline.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
@ -49,7 +49,7 @@ void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const Bu
|
||||
pipeline.getHeader().getColumnsWithTypeAndName(),
|
||||
output_stream->header.getColumnsWithTypeAndName(),
|
||||
ActionsDAG::MatchColumnsMode::Name);
|
||||
auto convert_actions = std::make_shared<ExpressionActions>(convert_actions_dag, settings.getActionsSettings());
|
||||
auto convert_actions = std::make_shared<ExpressionActions>(std::move(convert_actions_dag), settings.getActionsSettings());
|
||||
|
||||
pipeline.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
@ -61,20 +61,20 @@ void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const Bu
|
||||
void ExpressionStep::describeActions(FormatSettings & settings) const
|
||||
{
|
||||
String prefix(settings.offset, settings.indent_char);
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag);
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag.clone());
|
||||
expression->describeActions(settings.out, prefix);
|
||||
}
|
||||
|
||||
void ExpressionStep::describeActions(JSONBuilder::JSONMap & map) const
|
||||
{
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag);
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag.clone());
|
||||
map.add("Expression", expression->toTree());
|
||||
}
|
||||
|
||||
void ExpressionStep::updateOutputStream()
|
||||
{
|
||||
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)
|
||||
return;
|
||||
|
@ -1,12 +1,10 @@
|
||||
#pragma once
|
||||
#include <Processors/QueryPlan/ITransformingStep.h>
|
||||
#include <Interpreters/ActionsDAG.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ActionsDAG;
|
||||
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
|
||||
|
||||
class ExpressionTransform;
|
||||
class JoiningTransform;
|
||||
|
||||
@ -15,21 +13,22 @@ class ExpressionStep : public ITransformingStep
|
||||
{
|
||||
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"; }
|
||||
|
||||
void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) 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;
|
||||
|
||||
private:
|
||||
void updateOutputStream() override;
|
||||
|
||||
ActionsDAGPtr actions_dag;
|
||||
ActionsDAG actions_dag;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -9,9 +9,9 @@
|
||||
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)
|
||||
{
|
||||
preserves_sorting &= std::find_if(
|
||||
@ -35,28 +35,27 @@ static ITransformingStep::Traits getTraits(const ActionsDAGPtr & expression, con
|
||||
|
||||
FilterStep::FilterStep(
|
||||
const DataStream & input_stream_,
|
||||
const ActionsDAGPtr & actions_dag_,
|
||||
ActionsDAG actions_dag_,
|
||||
String filter_column_name_,
|
||||
bool remove_filter_column_)
|
||||
: ITransformingStep(
|
||||
input_stream_,
|
||||
FilterTransform::transformHeader(
|
||||
input_stream_.header,
|
||||
actions_dag_.get(),
|
||||
&actions_dag_,
|
||||
filter_column_name_,
|
||||
remove_filter_column_),
|
||||
getTraits(actions_dag_, input_stream_.header, input_stream_.sort_description, remove_filter_column_, filter_column_name_))
|
||||
, actions_dag(actions_dag_)
|
||||
, actions_dag(std::move(actions_dag_))
|
||||
, filter_column_name(std::move(filter_column_name_))
|
||||
, remove_filter_column(remove_filter_column_)
|
||||
{
|
||||
actions_dag = actions_dag->clone();
|
||||
actions_dag->removeAliasesForFilter(filter_column_name);
|
||||
actions_dag.removeAliasesForFilter(filter_column_name);
|
||||
}
|
||||
|
||||
void FilterStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings)
|
||||
{
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag, settings.getActionsSettings());
|
||||
auto expression = std::make_shared<ExpressionActions>(std::move(actions_dag), settings.getActionsSettings());
|
||||
|
||||
pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type)
|
||||
{
|
||||
@ -70,7 +69,7 @@ void FilterStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQ
|
||||
pipeline.getHeader().getColumnsWithTypeAndName(),
|
||||
output_stream->header.getColumnsWithTypeAndName(),
|
||||
ActionsDAG::MatchColumnsMode::Name);
|
||||
auto convert_actions = std::make_shared<ExpressionActions>(convert_actions_dag, settings.getActionsSettings());
|
||||
auto convert_actions = std::make_shared<ExpressionActions>(std::move(convert_actions_dag), settings.getActionsSettings());
|
||||
|
||||
pipeline.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
@ -88,7 +87,7 @@ void FilterStep::describeActions(FormatSettings & settings) const
|
||||
settings.out << " (removed)";
|
||||
settings.out << '\n';
|
||||
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag);
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag.clone());
|
||||
expression->describeActions(settings.out, prefix);
|
||||
}
|
||||
|
||||
@ -97,7 +96,7 @@ void FilterStep::describeActions(JSONBuilder::JSONMap & map) const
|
||||
map.add("Filter Column", filter_column_name);
|
||||
map.add("Removes Filter", remove_filter_column);
|
||||
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag);
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag.clone());
|
||||
map.add("Expression", expression->toTree());
|
||||
}
|
||||
|
||||
@ -105,7 +104,7 @@ void FilterStep::updateOutputStream()
|
||||
{
|
||||
output_stream = createOutputStream(
|
||||
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());
|
||||
|
||||
if (!getDataStreamTraits().preserves_sorting)
|
||||
|
@ -1,19 +1,17 @@
|
||||
#pragma once
|
||||
#include <Processors/QueryPlan/ITransformingStep.h>
|
||||
#include <Interpreters/ActionsDAG.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ActionsDAG;
|
||||
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
|
||||
|
||||
/// Implements WHERE, HAVING operations. See FilterTransform.
|
||||
class FilterStep : public ITransformingStep
|
||||
{
|
||||
public:
|
||||
FilterStep(
|
||||
const DataStream & input_stream_,
|
||||
const ActionsDAGPtr & actions_dag_,
|
||||
ActionsDAG actions_dag_,
|
||||
String filter_column_name_,
|
||||
bool remove_filter_column_);
|
||||
|
||||
@ -23,15 +21,15 @@ public:
|
||||
void describeActions(JSONBuilder::JSONMap & map) const override;
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
const ActionsDAGPtr & getExpression() const { return actions_dag; }
|
||||
ActionsDAGPtr & getExpression() { return actions_dag; }
|
||||
const ActionsDAG & getExpression() const { return actions_dag; }
|
||||
ActionsDAG & getExpression() { return actions_dag; }
|
||||
const String & getFilterColumnName() const { return filter_column_name; }
|
||||
bool removesFilterColumn() const { return remove_filter_column; }
|
||||
|
||||
private:
|
||||
void updateOutputStream() override;
|
||||
|
||||
ActionsDAGPtr actions_dag;
|
||||
ActionsDAG actions_dag;
|
||||
String filter_column_name;
|
||||
bool remove_filter_column;
|
||||
};
|
||||
|
@ -48,10 +48,10 @@ size_t tryConvertOuterJoinToInnerJoin(QueryPlan::Node * parent_node, QueryPlan::
|
||||
bool right_stream_safe = true;
|
||||
|
||||
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)
|
||||
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)
|
||||
return 0;
|
||||
|
@ -10,27 +10,27 @@
|
||||
namespace DB::QueryPlanOptimizations
|
||||
{
|
||||
/// build actions DAG from stack of steps
|
||||
static ActionsDAGPtr buildActionsForPlanPath(std::vector<ActionsDAGPtr> & dag_stack)
|
||||
static std::optional<ActionsDAG> buildActionsForPlanPath(std::vector<const ActionsDAG *> & dag_stack)
|
||||
{
|
||||
if (dag_stack.empty())
|
||||
return nullptr;
|
||||
return {};
|
||||
|
||||
ActionsDAGPtr path_actions = dag_stack.back()->clone();
|
||||
ActionsDAG path_actions = dag_stack.back()->clone();
|
||||
dag_stack.pop_back();
|
||||
while (!dag_stack.empty())
|
||||
{
|
||||
ActionsDAGPtr clone = dag_stack.back()->clone();
|
||||
ActionsDAG clone = dag_stack.back()->clone();
|
||||
dag_stack.pop_back();
|
||||
path_actions->mergeInplace(std::move(*clone));
|
||||
path_actions.mergeInplace(std::move(clone));
|
||||
}
|
||||
return path_actions;
|
||||
}
|
||||
|
||||
static std::set<std::string>
|
||||
getOriginalDistinctColumns(const ColumnsWithTypeAndName & distinct_columns, std::vector<ActionsDAGPtr> & dag_stack)
|
||||
getOriginalDistinctColumns(const ColumnsWithTypeAndName & distinct_columns, std::vector<const ActionsDAG *> & dag_stack)
|
||||
{
|
||||
auto actions = buildActionsForPlanPath(dag_stack);
|
||||
FindOriginalNodeForOutputName original_node_finder(actions);
|
||||
FindOriginalNodeForOutputName original_node_finder(*actions);
|
||||
std::set<std::string> original_distinct_columns;
|
||||
for (const auto & column : distinct_columns)
|
||||
{
|
||||
@ -65,7 +65,7 @@ size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node)
|
||||
/// (3) gather actions DAG to find original names for columns in distinct step later
|
||||
std::vector<ITransformingStep *> steps_to_update;
|
||||
QueryPlan::Node * node = parent_node;
|
||||
std::vector<ActionsDAGPtr> dag_stack;
|
||||
std::vector<const ActionsDAG *> dag_stack;
|
||||
while (!node->children.empty())
|
||||
{
|
||||
auto * step = dynamic_cast<ITransformingStep *>(node->step.get());
|
||||
@ -79,9 +79,9 @@ size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node)
|
||||
steps_to_update.push_back(step);
|
||||
|
||||
if (const auto * const expr = typeid_cast<const ExpressionStep *>(step); expr)
|
||||
dag_stack.push_back(expr->getExpression());
|
||||
dag_stack.push_back(&expr->getExpression());
|
||||
else if (const auto * const filter = typeid_cast<const FilterStep *>(step); filter)
|
||||
dag_stack.push_back(filter->getExpression());
|
||||
dag_stack.push_back(&filter->getExpression());
|
||||
|
||||
node = node->children.front();
|
||||
}
|
||||
|
@ -101,7 +101,7 @@ static NameSet findIdentifiersOfNode(const ActionsDAG::Node * node)
|
||||
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();
|
||||
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 * filter = assert_cast<FilterStep *>(parent.get());
|
||||
const auto & expression = filter->getExpression();
|
||||
auto & expression = filter->getExpression();
|
||||
const auto & filter_column_name = filter->getFilterColumnName();
|
||||
bool removes_filter = filter->removesFilterColumn();
|
||||
|
||||
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
|
||||
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)
|
||||
{
|
||||
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 * 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_node = expression->tryFindInOutputs(filter_column_name);
|
||||
const auto * filter_node = expression.tryFindInOutputs(filter_column_name);
|
||||
if (update_parent_filter && !filter_node && !filter->removesFilterColumn())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"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.
|
||||
/// Expression/Filter -> Child -> Something
|
||||
@ -147,10 +147,10 @@ addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes,
|
||||
/// Expression/Filter -> Child -> Filter -> Something
|
||||
|
||||
/// 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.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()))
|
||||
{
|
||||
@ -176,7 +176,7 @@ addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes,
|
||||
{
|
||||
/// This means that all predicates of filter were pushed down.
|
||||
/// 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
|
||||
{
|
||||
@ -192,7 +192,7 @@ tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, con
|
||||
bool can_remove_filter = true, size_t child_idx = 0)
|
||||
{
|
||||
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;
|
||||
}
|
||||
|
||||
@ -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 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(),
|
||||
left_stream_available_columns_to_push_down,
|
||||
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)
|
||||
{
|
||||
const auto & result_name = join_filter_push_down_actions.left_stream_filter_to_push_down->getOutputs()[0]->result_name;
|
||||
updated_steps += addNewFilterStepOrThrow(parent_node,
|
||||
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,
|
||||
0 /*child_idx*/,
|
||||
false /*update_parent_filter*/);
|
||||
LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"),
|
||||
"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);
|
||||
}
|
||||
|
||||
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,
|
||||
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,
|
||||
1 /*child_idx*/,
|
||||
false /*update_parent_filter*/);
|
||||
LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"),
|
||||
"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);
|
||||
}
|
||||
|
||||
if (updated_steps > 0)
|
||||
{
|
||||
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())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"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.
|
||||
@ -391,7 +393,7 @@ static size_t tryPushDownOverJoinStep(QueryPlan::Node * parent_node, QueryPlan::
|
||||
{
|
||||
/// This means that all predicates of filter were pushed down.
|
||||
/// 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
|
||||
{
|
||||
@ -416,7 +418,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes
|
||||
if (!filter)
|
||||
return 0;
|
||||
|
||||
if (filter->getExpression()->hasStatefulFunctions())
|
||||
if (filter->getExpression().hasStatefulFunctions())
|
||||
return 0;
|
||||
|
||||
if (auto * aggregating = typeid_cast<AggregatingStep *>(child.get()))
|
||||
@ -430,7 +432,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes
|
||||
return 0;
|
||||
|
||||
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);
|
||||
|
||||
@ -597,7 +599,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes
|
||||
|
||||
filter_node.step = std::make_unique<FilterStep>(
|
||||
filter_node.children.front()->step->getOutputStream(),
|
||||
filter->getExpression()->clone(),
|
||||
filter->getExpression().clone(),
|
||||
filter->getFilterColumnName(),
|
||||
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()))
|
||||
{
|
||||
FilterDAGInfo info{filter->getExpression(), filter->getFilterColumnName(), filter->removesFilterColumn()};
|
||||
FilterDAGInfo info{filter->getExpression().clone(), filter->getFilterColumnName(), filter->removesFilterColumn()};
|
||||
read_from_merge->addFilter(std::move(info));
|
||||
std::swap(*parent_node, *child_node);
|
||||
return 1;
|
||||
|
@ -28,10 +28,10 @@ size_t tryLiftUpArrayJoin(QueryPlan::Node * parent_node, QueryPlan::Nodes & node
|
||||
const auto & expression = expression_step ? expression_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.
|
||||
if (split_actions.first->trivial())
|
||||
if (split_actions.first.trivial())
|
||||
return 0;
|
||||
|
||||
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());
|
||||
|
||||
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
|
||||
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());
|
||||
|
||||
parent->setStepDescription(description + " [split]");
|
||||
|
@ -66,13 +66,13 @@ size_t tryExecuteFunctionsAfterSorting(QueryPlan::Node * parent_node, QueryPlan:
|
||||
NameSet sort_columns;
|
||||
for (const auto & col : sorting_step->getSortDescription())
|
||||
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.
|
||||
if (unneeded_for_sorting->trivial())
|
||||
if (unneeded_for_sorting.trivial())
|
||||
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;
|
||||
|
||||
// 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.children.front()->step->getOutputStream(),
|
||||
expression->getExpression()->clone());
|
||||
expression->getExpression().clone());
|
||||
}
|
||||
|
||||
/// - Expression - Something
|
||||
|
@ -37,18 +37,18 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &)
|
||||
|
||||
if (parent_expr && child_expr)
|
||||
{
|
||||
const auto & child_actions = child_expr->getExpression();
|
||||
const auto & parent_actions = parent_expr->getExpression();
|
||||
auto & child_actions = child_expr->getExpression();
|
||||
auto & parent_actions = parent_expr->getExpression();
|
||||
|
||||
/// We cannot combine actions with arrayJoin and stateful function because we not always can reorder them.
|
||||
/// Example: select rowNumberInBlock() from (select arrayJoin([1, 2]))
|
||||
/// 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;
|
||||
|
||||
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() + ")");
|
||||
|
||||
parent_node->step = std::move(expr);
|
||||
@ -57,16 +57,16 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &)
|
||||
}
|
||||
else if (parent_filter && child_expr)
|
||||
{
|
||||
const auto & child_actions = child_expr->getExpression();
|
||||
const auto & parent_actions = parent_filter->getExpression();
|
||||
auto & child_actions = child_expr->getExpression();
|
||||
auto & parent_actions = parent_filter->getExpression();
|
||||
|
||||
if (child_actions->hasArrayJoin() && parent_actions->hasStatefulFunctions())
|
||||
if (child_actions.hasArrayJoin() && parent_actions.hasStatefulFunctions())
|
||||
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(),
|
||||
merged,
|
||||
std::move(merged),
|
||||
parent_filter->getFilterColumnName(),
|
||||
parent_filter->removesFilterColumn());
|
||||
filter->setStepDescription("(" + parent_filter->getStepDescription() + " + " + child_expr->getStepDescription() + ")");
|
||||
@ -93,32 +93,31 @@ size_t tryMergeFilters(QueryPlan::Node * parent_node, QueryPlan::Nodes &)
|
||||
|
||||
if (parent_filter && child_filter)
|
||||
{
|
||||
const auto & child_actions = child_filter->getExpression();
|
||||
const auto & parent_actions = parent_filter->getExpression();
|
||||
auto & child_actions = child_filter->getExpression();
|
||||
auto & parent_actions = parent_filter->getExpression();
|
||||
|
||||
if (child_actions->hasArrayJoin())
|
||||
if (child_actions.hasArrayJoin())
|
||||
return 0;
|
||||
|
||||
auto actions = child_actions->clone();
|
||||
const auto & child_filter_node = actions->findInOutputs(child_filter->getFilterColumnName());
|
||||
const auto & child_filter_node = child_actions.findInOutputs(child_filter->getFilterColumnName());
|
||||
if (child_filter->removesFilterColumn())
|
||||
removeFromOutputs(*actions, child_filter_node);
|
||||
removeFromOutputs(child_actions, child_filter_node);
|
||||
|
||||
actions->mergeInplace(std::move(*parent_actions->clone()));
|
||||
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())
|
||||
removeFromOutputs(*actions, parent_filter_node);
|
||||
removeFromOutputs(child_actions, parent_filter_node);
|
||||
|
||||
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}, {});
|
||||
auto & outputs = actions->getOutputs();
|
||||
const auto & condition = child_actions.addFunction(func_builder_and, {&child_filter_node, &parent_filter_node}, {});
|
||||
auto & outputs = child_actions.getOutputs();
|
||||
outputs.insert(outputs.begin(), &condition);
|
||||
|
||||
actions->removeUnusedActions(false);
|
||||
child_actions.removeUnusedActions(false);
|
||||
|
||||
auto filter = std::make_unique<FilterStep>(child_filter->getInputStreams().front(),
|
||||
actions,
|
||||
std::move(child_actions),
|
||||
condition.result_name,
|
||||
true);
|
||||
filter->setStepDescription("(" + parent_filter->getStepDescription() + " + " + child_filter->getStepDescription() + ")");
|
||||
|
@ -56,12 +56,12 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &)
|
||||
return;
|
||||
|
||||
const auto & storage_prewhere_info = source_step_with_filter->getPrewhereInfo();
|
||||
if (storage_prewhere_info && storage_prewhere_info->prewhere_actions)
|
||||
if (storage_prewhere_info)
|
||||
return;
|
||||
|
||||
/// TODO: We can also check for UnionStep, such as StorageBuffer and local distributed plans.
|
||||
QueryPlan::Node * filter_node = (stack.rbegin() + 1)->node;
|
||||
const auto * filter_step = typeid_cast<FilterStep *>(filter_node->step.get());
|
||||
auto * filter_step = typeid_cast<FilterStep *>(filter_node->step.get());
|
||||
if (!filter_step)
|
||||
return;
|
||||
|
||||
@ -85,10 +85,11 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &)
|
||||
|
||||
Names queried_columns = source_step_with_filter->requiredSourceColumns();
|
||||
|
||||
const auto & source_filter_actions_dag = source_step_with_filter->getFilterActionsDAG();
|
||||
MergeTreeWhereOptimizer where_optimizer{
|
||||
std::move(column_compressed_sizes),
|
||||
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,
|
||||
storage.supportedPrewhereColumns(),
|
||||
getLogger("QueryPlanOptimizePrewhere")};
|
||||
@ -110,20 +111,20 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &)
|
||||
prewhere_info->need_filter = true;
|
||||
prewhere_info->remove_prewhere_column = optimize_result.fully_moved_to_prewhere && filter_step->removesFilterColumn();
|
||||
|
||||
auto filter_expression = filter_step->getExpression();
|
||||
auto filter_expression = std::move(filter_step->getExpression());
|
||||
const auto & filter_column_name = filter_step->getFilterColumnName();
|
||||
|
||||
if (prewhere_info->remove_prewhere_column)
|
||||
{
|
||||
removeFromOutput(*filter_expression, filter_column_name);
|
||||
auto & outputs = filter_expression->getOutputs();
|
||||
removeFromOutput(filter_expression, filter_column_name);
|
||||
auto & outputs = filter_expression.getOutputs();
|
||||
size_t size = outputs.size();
|
||||
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);
|
||||
}
|
||||
|
||||
auto split_result = filter_step->getExpression()->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.
|
||||
/// Splited actions may have inputs which are needed only for PREWHERE.
|
||||
@ -139,15 +140,15 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &)
|
||||
/// So, here we restore removed inputs for PREWHERE actions
|
||||
{
|
||||
std::unordered_set<const ActionsDAG::Node *> first_outputs(
|
||||
split_result.first->getOutputs().begin(), split_result.first->getOutputs().end());
|
||||
for (const auto * input : split_result.first->getInputs())
|
||||
split_result.first.getOutputs().begin(), split_result.first.getOutputs().end());
|
||||
for (const auto * input : split_result.first.getInputs())
|
||||
{
|
||||
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.
|
||||
/// 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -164,16 +165,16 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &)
|
||||
{
|
||||
prewhere_info->prewhere_column_name = conditions.front()->result_name;
|
||||
if (prewhere_info->remove_prewhere_column)
|
||||
prewhere_info->prewhere_actions->getOutputs().push_back(conditions.front());
|
||||
prewhere_info->prewhere_actions.getOutputs().push_back(conditions.front());
|
||||
}
|
||||
else
|
||||
{
|
||||
prewhere_info->remove_prewhere_column = true;
|
||||
|
||||
FunctionOverloadResolverPtr func_builder_and = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>());
|
||||
const auto * node = &prewhere_info->prewhere_actions->addFunction(func_builder_and, std::move(conditions), {});
|
||||
const auto * node = &prewhere_info->prewhere_actions.addFunction(func_builder_and, std::move(conditions), {});
|
||||
prewhere_info->prewhere_column_name = node->result_name;
|
||||
prewhere_info->prewhere_actions->getOutputs().push_back(node);
|
||||
prewhere_info->prewhere_actions.getOutputs().push_back(node);
|
||||
}
|
||||
|
||||
source_step_with_filter->updatePrewhereInfo(prewhere_info);
|
||||
|
@ -18,16 +18,16 @@ void optimizePrimaryKeyConditionAndLimit(const Stack & stack)
|
||||
const auto & storage_prewhere_info = source_step_with_filter->getPrewhereInfo();
|
||||
if (storage_prewhere_info)
|
||||
{
|
||||
source_step_with_filter->addFilter(storage_prewhere_info->prewhere_actions, storage_prewhere_info->prewhere_column_name);
|
||||
source_step_with_filter->addFilter(storage_prewhere_info->prewhere_actions.clone(), storage_prewhere_info->prewhere_column_name);
|
||||
if (storage_prewhere_info->row_level_filter)
|
||||
source_step_with_filter->addFilter(storage_prewhere_info->row_level_filter, storage_prewhere_info->row_level_column_name);
|
||||
source_step_with_filter->addFilter(storage_prewhere_info->row_level_filter->clone(), storage_prewhere_info->row_level_column_name);
|
||||
}
|
||||
|
||||
for (auto iter = stack.rbegin() + 1; iter != stack.rend(); ++iter)
|
||||
{
|
||||
if (auto * filter_step = typeid_cast<FilterStep *>(iter->node->step.get()))
|
||||
{
|
||||
source_step_with_filter->addFilter(filter_step->getExpression(), filter_step->getFilterColumnName());
|
||||
source_step_with_filter->addFilter(filter_step->getExpression().clone(), filter_step->getFilterColumnName());
|
||||
}
|
||||
else if (auto * limit_step = typeid_cast<LimitStep *>(iter->node->step.get()))
|
||||
{
|
||||
|
@ -171,17 +171,17 @@ static void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filt
|
||||
}
|
||||
}
|
||||
|
||||
static void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression)
|
||||
static void appendExpression(std::optional<ActionsDAG> & dag, const ActionsDAG & expression)
|
||||
{
|
||||
if (dag)
|
||||
dag->mergeInplace(std::move(*expression->clone()));
|
||||
dag->mergeInplace(expression.clone());
|
||||
else
|
||||
dag = expression->clone();
|
||||
dag = expression.clone();
|
||||
}
|
||||
|
||||
/// This function builds a common DAG which is a merge of DAGs from Filter and Expression steps chain.
|
||||
/// Additionally, build a set of fixed columns.
|
||||
void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns & fixed_columns, size_t & limit)
|
||||
void buildSortingDAG(QueryPlan::Node & node, std::optional<ActionsDAG> & dag, FixedColumns & fixed_columns, size_t & limit)
|
||||
{
|
||||
IQueryPlanStep * step = node.step.get();
|
||||
if (auto * reading = typeid_cast<ReadFromMergeTree *>(step))
|
||||
@ -191,13 +191,11 @@ void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns &
|
||||
/// Should ignore limit if there is filtering.
|
||||
limit = 0;
|
||||
|
||||
if (prewhere_info->prewhere_actions)
|
||||
{
|
||||
//std::cerr << "====== Adding prewhere " << std::endl;
|
||||
appendExpression(dag, prewhere_info->prewhere_actions);
|
||||
if (const auto * filter_expression = dag->tryFindInOutputs(prewhere_info->prewhere_column_name))
|
||||
appendFixedColumnsFromFilterExpression(*filter_expression, fixed_columns);
|
||||
}
|
||||
//std::cerr << "====== Adding prewhere " << std::endl;
|
||||
appendExpression(dag, prewhere_info->prewhere_actions);
|
||||
if (const auto * filter_expression = dag->tryFindInOutputs(prewhere_info->prewhere_column_name))
|
||||
appendFixedColumnsFromFilterExpression(*filter_expression, fixed_columns);
|
||||
|
||||
}
|
||||
return;
|
||||
}
|
||||
@ -212,7 +210,7 @@ void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns &
|
||||
const auto & actions = expression->getExpression();
|
||||
|
||||
/// Should ignore limit because arrayJoin() can reduce the number of rows in case of empty array.
|
||||
if (actions->hasArrayJoin())
|
||||
if (actions.hasArrayJoin())
|
||||
limit = 0;
|
||||
|
||||
appendExpression(dag, actions);
|
||||
@ -330,7 +328,7 @@ void enreachFixedColumns(const ActionsDAG & dag, FixedColumns & fixed_columns)
|
||||
|
||||
InputOrderInfoPtr buildInputOrderInfo(
|
||||
const FixedColumns & fixed_columns,
|
||||
const ActionsDAGPtr & dag,
|
||||
const std::optional<ActionsDAG> & dag,
|
||||
const SortDescription & description,
|
||||
const KeyDescription & sorting_key,
|
||||
size_t limit)
|
||||
@ -507,7 +505,7 @@ struct AggregationInputOrder
|
||||
|
||||
AggregationInputOrder buildInputOrderInfo(
|
||||
const FixedColumns & fixed_columns,
|
||||
const ActionsDAGPtr & dag,
|
||||
const std::optional<ActionsDAG> & dag,
|
||||
const Names & group_by_keys,
|
||||
const ActionsDAG & sorting_key_dag,
|
||||
const Names & sorting_key_columns)
|
||||
@ -693,7 +691,7 @@ AggregationInputOrder buildInputOrderInfo(
|
||||
InputOrderInfoPtr buildInputOrderInfo(
|
||||
const ReadFromMergeTree * reading,
|
||||
const FixedColumns & fixed_columns,
|
||||
const ActionsDAGPtr & dag,
|
||||
const std::optional<ActionsDAG> & dag,
|
||||
const SortDescription & description,
|
||||
size_t limit)
|
||||
{
|
||||
@ -709,7 +707,7 @@ InputOrderInfoPtr buildInputOrderInfo(
|
||||
InputOrderInfoPtr buildInputOrderInfo(
|
||||
ReadFromMerge * merge,
|
||||
const FixedColumns & fixed_columns,
|
||||
const ActionsDAGPtr & dag,
|
||||
const std::optional<ActionsDAG> & dag,
|
||||
const SortDescription & description,
|
||||
size_t limit)
|
||||
{
|
||||
@ -745,7 +743,7 @@ InputOrderInfoPtr buildInputOrderInfo(
|
||||
AggregationInputOrder buildInputOrderInfo(
|
||||
ReadFromMergeTree * reading,
|
||||
const FixedColumns & fixed_columns,
|
||||
const ActionsDAGPtr & dag,
|
||||
const std::optional<ActionsDAG> & dag,
|
||||
const Names & group_by_keys)
|
||||
{
|
||||
const auto & sorting_key = reading->getStorageMetadata()->getSortingKey();
|
||||
@ -760,7 +758,7 @@ AggregationInputOrder buildInputOrderInfo(
|
||||
AggregationInputOrder buildInputOrderInfo(
|
||||
ReadFromMerge * merge,
|
||||
const FixedColumns & fixed_columns,
|
||||
const ActionsDAGPtr & dag,
|
||||
const std::optional<ActionsDAG> & dag,
|
||||
const Names & group_by_keys)
|
||||
{
|
||||
const auto & tables = merge->getSelectedTables();
|
||||
@ -801,7 +799,7 @@ InputOrderInfoPtr buildInputOrderInfo(SortingStep & sorting, QueryPlan::Node & n
|
||||
const auto & description = sorting.getSortDescription();
|
||||
size_t limit = sorting.getLimit();
|
||||
|
||||
ActionsDAGPtr dag;
|
||||
std::optional<ActionsDAG> dag;
|
||||
FixedColumns fixed_columns;
|
||||
buildSortingDAG(node, dag, fixed_columns, limit);
|
||||
|
||||
@ -855,7 +853,7 @@ AggregationInputOrder buildInputOrderInfo(AggregatingStep & aggregating, QueryPl
|
||||
const auto & keys = aggregating.getParams().keys;
|
||||
size_t limit = 0;
|
||||
|
||||
ActionsDAGPtr dag;
|
||||
std::optional<ActionsDAG> dag;
|
||||
FixedColumns fixed_columns;
|
||||
buildSortingDAG(node, dag, fixed_columns, limit);
|
||||
|
||||
@ -1076,13 +1074,13 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node,
|
||||
for (const auto & actions_dag : window_desc.partition_by_actions)
|
||||
{
|
||||
order_by_elements_actions.emplace_back(
|
||||
std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(context, CompileExpressions::yes)));
|
||||
std::make_shared<ExpressionActions>(actions_dag->clone(), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes)));
|
||||
}
|
||||
|
||||
for (const auto & actions_dag : window_desc.order_by_actions)
|
||||
{
|
||||
order_by_elements_actions.emplace_back(
|
||||
std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(context, CompileExpressions::yes)));
|
||||
std::make_shared<ExpressionActions>(actions_dag->clone(), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes)));
|
||||
}
|
||||
|
||||
auto order_optimizer = std::make_shared<ReadInOrderOptimizer>(
|
||||
|
@ -43,7 +43,7 @@ static DAGIndex buildDAGIndex(const ActionsDAG & dag)
|
||||
/// Required analysis info from aggregate projection.
|
||||
struct AggregateProjectionInfo
|
||||
{
|
||||
ActionsDAGPtr before_aggregation;
|
||||
std::optional<ActionsDAG> before_aggregation;
|
||||
Names keys;
|
||||
AggregateDescriptions aggregates;
|
||||
|
||||
@ -274,7 +274,7 @@ static void appendAggregateFunctions(
|
||||
}
|
||||
}
|
||||
|
||||
ActionsDAGPtr analyzeAggregateProjection(
|
||||
std::optional<ActionsDAG> analyzeAggregateProjection(
|
||||
const AggregateProjectionInfo & info,
|
||||
const QueryDAG & query,
|
||||
const DAGIndex & query_index,
|
||||
@ -394,7 +394,7 @@ ActionsDAGPtr analyzeAggregateProjection(
|
||||
// LOG_TRACE(getLogger("optimizeUseProjections"), "Folding actions by projection");
|
||||
|
||||
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;
|
||||
}
|
||||
|
||||
@ -406,7 +406,7 @@ struct AggregateProjectionCandidate : public ProjectionCandidate
|
||||
|
||||
/// Actions which need to be applied to columns from projection
|
||||
/// in order to get all the columns required for aggregation.
|
||||
ActionsDAGPtr dag;
|
||||
ActionsDAG dag;
|
||||
};
|
||||
|
||||
struct MinMaxProjectionCandidate
|
||||
@ -481,13 +481,13 @@ AggregateProjectionCandidates getAggregateProjectionCandidates(
|
||||
if (auto proj_dag = analyzeAggregateProjection(info, dag, query_index, keys, aggregates))
|
||||
{
|
||||
// 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());
|
||||
auto block = reading.getMergeTreeData().getMinMaxCountProjectionBlock(
|
||||
metadata,
|
||||
candidate.dag->getRequiredColumnsNames(),
|
||||
(dag.filter_node ? dag.dag : nullptr),
|
||||
candidate.dag.getRequiredColumnsNames(),
|
||||
(dag.filter_node ? &*dag.dag : nullptr),
|
||||
parts,
|
||||
max_added_blocks.get(),
|
||||
context);
|
||||
@ -538,7 +538,7 @@ AggregateProjectionCandidates getAggregateProjectionCandidates(
|
||||
if (auto proj_dag = analyzeAggregateProjection(info, dag, query_index, keys, aggregates))
|
||||
{
|
||||
// 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;
|
||||
candidates.real.emplace_back(std::move(candidate));
|
||||
}
|
||||
@ -666,7 +666,7 @@ std::optional<String> optimizeUseAggregateProjections(QueryPlan::Node & node, Qu
|
||||
/// Selecting best candidate.
|
||||
for (auto & candidate : candidates.real)
|
||||
{
|
||||
auto required_column_names = candidate.dag->getRequiredColumnsNames();
|
||||
auto required_column_names = candidate.dag.getRequiredColumnsNames();
|
||||
|
||||
bool analyzed = analyzeProjectionCandidate(
|
||||
candidate,
|
||||
@ -677,7 +677,7 @@ std::optional<String> optimizeUseAggregateProjections(QueryPlan::Node & node, Qu
|
||||
query_info,
|
||||
context,
|
||||
max_added_blocks,
|
||||
candidate.dag);
|
||||
&candidate.dag);
|
||||
|
||||
if (!analyzed)
|
||||
continue;
|
||||
@ -767,7 +767,7 @@ std::optional<String> optimizeUseAggregateProjections(QueryPlan::Node & node, Qu
|
||||
projection_reading = reader.readFromParts(
|
||||
/* parts = */ {},
|
||||
/* alter_conversions = */ {},
|
||||
best_candidate->dag->getRequiredColumnsNames(),
|
||||
best_candidate->dag.getRequiredColumnsNames(),
|
||||
proj_snapshot,
|
||||
projection_query_info,
|
||||
context,
|
||||
@ -779,7 +779,7 @@ std::optional<String> optimizeUseAggregateProjections(QueryPlan::Node & node, Qu
|
||||
|
||||
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)));
|
||||
projection_reading = std::make_unique<ReadFromPreparedSource>(std::move(pipe));
|
||||
}
|
||||
@ -810,17 +810,19 @@ std::optional<String> optimizeUseAggregateProjections(QueryPlan::Node & node, Qu
|
||||
if (best_candidate)
|
||||
{
|
||||
aggregate_projection_node = &nodes.emplace_back();
|
||||
|
||||
if (candidates.has_filter)
|
||||
{
|
||||
const auto & result_name = best_candidate->dag.getOutputs().front()->result_name;
|
||||
aggregate_projection_node->step = std::make_unique<FilterStep>(
|
||||
projection_reading_node.step->getOutputStream(),
|
||||
best_candidate->dag,
|
||||
best_candidate->dag->getOutputs().front()->result_name,
|
||||
std::move(best_candidate->dag),
|
||||
result_name,
|
||||
true);
|
||||
}
|
||||
else
|
||||
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);
|
||||
}
|
||||
|
@ -25,7 +25,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.
|
||||
/// This may happen e.g. if we have PREWHERE.
|
||||
@ -33,7 +33,7 @@ static ActionsDAGPtr makeMaterializingDAG(const Block & proj_header, const Block
|
||||
size_t num_columns = main_header.columns();
|
||||
/// This is a error; will have block structure mismatch later.
|
||||
if (proj_header.columns() != num_columns)
|
||||
return nullptr;
|
||||
return {};
|
||||
|
||||
std::vector<size_t> const_positions;
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
@ -47,17 +47,17 @@ static ActionsDAGPtr makeMaterializingDAG(const Block & proj_header, const Block
|
||||
}
|
||||
|
||||
if (const_positions.empty())
|
||||
return nullptr;
|
||||
return {};
|
||||
|
||||
ActionsDAGPtr dag = std::make_unique<ActionsDAG>();
|
||||
auto & outputs = dag->getOutputs();
|
||||
ActionsDAG dag;
|
||||
auto & outputs = dag.getOutputs();
|
||||
for (const auto & col : proj_header.getColumnsWithTypeAndName())
|
||||
outputs.push_back(&dag->addInput(col));
|
||||
outputs.push_back(&dag.addInput(col));
|
||||
|
||||
for (auto pos : const_positions)
|
||||
{
|
||||
auto & output = outputs[pos];
|
||||
output = &dag->materializeNode(*output);
|
||||
output = &dag.materializeNode(*output);
|
||||
}
|
||||
|
||||
return dag;
|
||||
@ -174,7 +174,7 @@ std::optional<String> optimizeUseNormalProjections(Stack & stack, QueryPlan::Nod
|
||||
query_info,
|
||||
context,
|
||||
max_added_blocks,
|
||||
query.filter_node ? query.dag : nullptr);
|
||||
query.filter_node ? &*query.dag : nullptr);
|
||||
|
||||
if (!analyzed)
|
||||
continue;
|
||||
@ -244,14 +244,14 @@ std::optional<String> optimizeUseNormalProjections(Stack & stack, QueryPlan::Nod
|
||||
{
|
||||
expr_or_filter_node.step = std::make_unique<FilterStep>(
|
||||
projection_reading_node.step->getOutputStream(),
|
||||
query.dag,
|
||||
std::move(*query.dag),
|
||||
query.filter_node->result_name,
|
||||
true);
|
||||
}
|
||||
else
|
||||
expr_or_filter_node.step = std::make_unique<ExpressionStep>(
|
||||
projection_reading_node.step->getOutputStream(),
|
||||
query.dag);
|
||||
std::move(*query.dag));
|
||||
|
||||
expr_or_filter_node.children.push_back(&projection_reading_node);
|
||||
next_node = &expr_or_filter_node;
|
||||
@ -269,7 +269,7 @@ std::optional<String> optimizeUseNormalProjections(Stack & stack, QueryPlan::Nod
|
||||
|
||||
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();
|
||||
auto & expr_node = nodes.emplace_back();
|
||||
expr_node.step = std::move(converting);
|
||||
|
@ -65,12 +65,12 @@ std::shared_ptr<PartitionIdToMaxBlock> getMaxAddedBlocks(ReadFromMergeTree * rea
|
||||
return {};
|
||||
}
|
||||
|
||||
void QueryDAG::appendExpression(const ActionsDAGPtr & expression)
|
||||
void QueryDAG::appendExpression(const ActionsDAG & expression)
|
||||
{
|
||||
if (dag)
|
||||
dag->mergeInplace(std::move(*expression->clone()));
|
||||
dag->mergeInplace(expression.clone());
|
||||
else
|
||||
dag = expression->clone();
|
||||
dag = expression.clone();
|
||||
}
|
||||
|
||||
const ActionsDAG::Node * findInOutputs(ActionsDAG & dag, const std::string & name, bool remove)
|
||||
@ -121,22 +121,19 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs &
|
||||
{
|
||||
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))
|
||||
filter_nodes.push_back(filter_expression);
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
if (prewhere_info->prewhere_actions)
|
||||
{
|
||||
appendExpression(prewhere_info->prewhere_actions);
|
||||
if (const auto * filter_expression
|
||||
= findInOutputs(*dag, prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column))
|
||||
filter_nodes.push_back(filter_expression);
|
||||
else
|
||||
return false;
|
||||
}
|
||||
appendExpression(prewhere_info->prewhere_actions);
|
||||
if (const auto * filter_expression
|
||||
= findInOutputs(*dag, prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column))
|
||||
filter_nodes.push_back(filter_expression);
|
||||
else
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
@ -150,7 +147,7 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs &
|
||||
if (auto * expression = typeid_cast<ExpressionStep *>(step))
|
||||
{
|
||||
const auto & actions = expression->getExpression();
|
||||
if (actions->hasArrayJoin())
|
||||
if (actions.hasArrayJoin())
|
||||
return false;
|
||||
|
||||
appendExpression(actions);
|
||||
@ -160,7 +157,7 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs &
|
||||
if (auto * filter = typeid_cast<FilterStep *>(step))
|
||||
{
|
||||
const auto & actions = filter->getExpression();
|
||||
if (actions->hasArrayJoin())
|
||||
if (actions.hasArrayJoin())
|
||||
return false;
|
||||
|
||||
appendExpression(actions);
|
||||
@ -214,7 +211,7 @@ bool analyzeProjectionCandidate(
|
||||
const SelectQueryInfo & query_info,
|
||||
const ContextPtr & context,
|
||||
const std::shared_ptr<PartitionIdToMaxBlock> & max_added_blocks,
|
||||
const ActionsDAGPtr & dag)
|
||||
const ActionsDAG * dag)
|
||||
{
|
||||
MergeTreeData::DataPartsVector projection_parts;
|
||||
MergeTreeData::DataPartsVector normal_parts;
|
||||
@ -239,7 +236,8 @@ bool analyzeProjectionCandidate(
|
||||
|
||||
auto projection_query_info = query_info;
|
||||
projection_query_info.prewhere_info = nullptr;
|
||||
projection_query_info.filter_actions_dag = dag;
|
||||
if (dag)
|
||||
projection_query_info.filter_actions_dag = std::make_unique<ActionsDAG>(dag->clone());
|
||||
|
||||
auto projection_result_ptr = reader.estimateNumMarksToRead(
|
||||
std::move(projection_parts),
|
||||
|
@ -25,14 +25,14 @@ std::shared_ptr<PartitionIdToMaxBlock> getMaxAddedBlocks(ReadFromMergeTree * rea
|
||||
/// Additionally, for all the Filter steps, we collect filter conditions into filter_nodes.
|
||||
struct QueryDAG
|
||||
{
|
||||
ActionsDAGPtr dag;
|
||||
std::optional<ActionsDAG> dag;
|
||||
const ActionsDAG::Node * filter_node = nullptr;
|
||||
|
||||
bool build(QueryPlan::Node & node);
|
||||
|
||||
private:
|
||||
bool buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs & filter_nodes);
|
||||
void appendExpression(const ActionsDAGPtr & expression);
|
||||
void appendExpression(const ActionsDAG & expression);
|
||||
};
|
||||
|
||||
struct ProjectionCandidate
|
||||
@ -60,6 +60,6 @@ bool analyzeProjectionCandidate(
|
||||
const SelectQueryInfo & query_info,
|
||||
const ContextPtr & context,
|
||||
const std::shared_ptr<PartitionIdToMaxBlock> & max_added_blocks,
|
||||
const ActionsDAGPtr & dag);
|
||||
const ActionsDAG * dag);
|
||||
|
||||
}
|
||||
|
@ -43,10 +43,10 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
void logActionsDAG(const String & prefix, const ActionsDAGPtr & actions)
|
||||
void logActionsDAG(const String & prefix, const ActionsDAG & actions)
|
||||
{
|
||||
if constexpr (debug_logging_enabled)
|
||||
LOG_DEBUG(getLogger("redundantDistinct"), "{} :\n{}", prefix, actions->dumpDAG());
|
||||
LOG_DEBUG(getLogger("redundantDistinct"), "{} :\n{}", prefix, actions.dumpDAG());
|
||||
}
|
||||
|
||||
using DistinctColumns = std::set<std::string_view>;
|
||||
@ -65,25 +65,25 @@ namespace
|
||||
}
|
||||
|
||||
/// build actions DAG from stack of steps
|
||||
ActionsDAGPtr buildActionsForPlanPath(std::vector<ActionsDAGPtr> & dag_stack)
|
||||
std::optional<ActionsDAG> buildActionsForPlanPath(std::vector<const ActionsDAG *> & dag_stack)
|
||||
{
|
||||
if (dag_stack.empty())
|
||||
return nullptr;
|
||||
return {};
|
||||
|
||||
ActionsDAGPtr path_actions = dag_stack.back()->clone();
|
||||
ActionsDAG path_actions = dag_stack.back()->clone();
|
||||
dag_stack.pop_back();
|
||||
while (!dag_stack.empty())
|
||||
{
|
||||
ActionsDAGPtr clone = dag_stack.back()->clone();
|
||||
ActionsDAG clone = dag_stack.back()->clone();
|
||||
logActionsDAG("DAG to merge", clone);
|
||||
dag_stack.pop_back();
|
||||
path_actions->mergeInplace(std::move(*clone));
|
||||
path_actions.mergeInplace(std::move(clone));
|
||||
}
|
||||
return path_actions;
|
||||
}
|
||||
|
||||
bool compareAggregationKeysWithDistinctColumns(
|
||||
const Names & aggregation_keys, const DistinctColumns & distinct_columns, std::vector<std::vector<ActionsDAGPtr>> actions_chain)
|
||||
const Names & aggregation_keys, const DistinctColumns & distinct_columns, std::vector<std::vector<const ActionsDAG *>> actions_chain)
|
||||
{
|
||||
logDebug("aggregation_keys", aggregation_keys);
|
||||
logDebug("aggregation_keys size", aggregation_keys.size());
|
||||
@ -93,7 +93,8 @@ namespace
|
||||
std::set<String> source_columns;
|
||||
for (auto & actions : actions_chain)
|
||||
{
|
||||
FindOriginalNodeForOutputName original_node_finder(buildActionsForPlanPath(actions));
|
||||
auto tmp_actions = buildActionsForPlanPath(actions);
|
||||
FindOriginalNodeForOutputName original_node_finder(*tmp_actions);
|
||||
for (const auto & column : current_columns)
|
||||
{
|
||||
logDebug("distinct column name", column);
|
||||
@ -131,10 +132,10 @@ namespace
|
||||
return true;
|
||||
|
||||
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)
|
||||
return !filter->getExpression()->hasArrayJoin();
|
||||
return !filter->getExpression().hasArrayJoin();
|
||||
|
||||
if (typeid_cast<const LimitStep *>(step) || typeid_cast<const LimitByStep *>(step) || typeid_cast<const SortingStep *>(step)
|
||||
|| typeid_cast<const WindowStep *>(step))
|
||||
@ -152,8 +153,8 @@ namespace
|
||||
const DistinctStep * distinct_step = typeid_cast<DistinctStep *>(distinct_node->step.get());
|
||||
chassert(distinct_step);
|
||||
|
||||
std::vector<ActionsDAGPtr> dag_stack;
|
||||
std::vector<std::vector<ActionsDAGPtr>> actions_chain;
|
||||
std::vector<const ActionsDAG *> dag_stack;
|
||||
std::vector<std::vector<const ActionsDAG *>> actions_chain;
|
||||
const DistinctStep * inner_distinct_step = nullptr;
|
||||
const IQueryPlanStep * aggregation_before_distinct = nullptr;
|
||||
const QueryPlan::Node * node = distinct_node;
|
||||
@ -182,9 +183,9 @@ namespace
|
||||
}
|
||||
|
||||
if (const auto * const expr = typeid_cast<const ExpressionStep *>(current_step); expr)
|
||||
dag_stack.push_back(expr->getExpression());
|
||||
dag_stack.push_back(&expr->getExpression());
|
||||
else if (const auto * const filter = typeid_cast<const FilterStep *>(current_step); filter)
|
||||
dag_stack.push_back(filter->getExpression());
|
||||
dag_stack.push_back(&filter->getExpression());
|
||||
|
||||
node = node->children.front();
|
||||
if (inner_distinct_step = typeid_cast<DistinctStep *>(node->step.get()); inner_distinct_step)
|
||||
@ -222,7 +223,7 @@ namespace
|
||||
chassert(distinct_step);
|
||||
const auto distinct_columns = getDistinctColumns(distinct_step);
|
||||
|
||||
std::vector<ActionsDAGPtr> dag_stack;
|
||||
std::vector<const ActionsDAG *> dag_stack;
|
||||
const DistinctStep * inner_distinct_step = nullptr;
|
||||
const QueryPlan::Node * node = distinct_node;
|
||||
while (!node->children.empty())
|
||||
@ -235,9 +236,9 @@ namespace
|
||||
}
|
||||
|
||||
if (const auto * const expr = typeid_cast<const ExpressionStep *>(current_step); expr)
|
||||
dag_stack.push_back(expr->getExpression());
|
||||
dag_stack.push_back(&expr->getExpression());
|
||||
else if (const auto * const filter = typeid_cast<const FilterStep *>(current_step); filter)
|
||||
dag_stack.push_back(filter->getExpression());
|
||||
dag_stack.push_back(&filter->getExpression());
|
||||
|
||||
node = node->children.front();
|
||||
inner_distinct_step = typeid_cast<DistinctStep *>(node->step.get());
|
||||
@ -259,11 +260,11 @@ namespace
|
||||
if (distinct_columns.size() != inner_distinct_columns.size())
|
||||
return false;
|
||||
|
||||
ActionsDAGPtr path_actions;
|
||||
ActionsDAG path_actions;
|
||||
if (!dag_stack.empty())
|
||||
{
|
||||
/// build actions DAG to find original column names
|
||||
path_actions = buildActionsForPlanPath(dag_stack);
|
||||
path_actions = std::move(*buildActionsForPlanPath(dag_stack));
|
||||
logActionsDAG("distinct pass: merged DAG", path_actions);
|
||||
|
||||
/// compare columns of two DISTINCTs
|
||||
|
@ -213,12 +213,12 @@ private:
|
||||
logStep("checking for stateful function", node);
|
||||
if (const auto * expr = typeid_cast<const ExpressionStep *>(step); expr)
|
||||
{
|
||||
if (expr->getExpression()->hasStatefulFunctions())
|
||||
if (expr->getExpression().hasStatefulFunctions())
|
||||
return false;
|
||||
}
|
||||
else if (const auto * filter = typeid_cast<const FilterStep *>(step); filter)
|
||||
{
|
||||
if (filter->getExpression()->hasStatefulFunctions())
|
||||
if (filter->getExpression().hasStatefulFunctions())
|
||||
return false;
|
||||
}
|
||||
else
|
||||
|
@ -17,13 +17,13 @@ size_t trySplitFilter(QueryPlan::Node * node, QueryPlan::Nodes & nodes)
|
||||
const std::string & filter_column_name = filter_step->getFilterColumnName();
|
||||
|
||||
/// Do not split if there are function like runningDifference.
|
||||
if (expr->hasStatefulFunctions())
|
||||
if (expr.hasStatefulFunctions())
|
||||
return 0;
|
||||
|
||||
bool filter_name_clashs_with_input = false;
|
||||
if (filter_step->removesFilterColumn())
|
||||
{
|
||||
for (const auto * input : expr->getInputs())
|
||||
for (const auto * input : expr.getInputs())
|
||||
{
|
||||
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;
|
||||
|
||||
bool remove_filter = false;
|
||||
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();
|
||||
|
||||
@ -53,11 +53,11 @@ size_t trySplitFilter(QueryPlan::Node * node, QueryPlan::Nodes & nodes)
|
||||
{
|
||||
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)
|
||||
{
|
||||
filter_output = &split.first->addAlias(*filter_output, split_filter_name);
|
||||
filter_output = &split.first.addAlias(*filter_output, split_filter_name);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
@ -74,11 +74,11 @@ void removeInjectiveFunctionsFromResultsRecursively(const ActionsDAG::Node * nod
|
||||
/// Our objective is to replace injective function nodes in `actions` results with its children
|
||||
/// until only the irreducible subset of nodes remains. Against these set of nodes we will match partition key expression
|
||||
/// to determine if it maps all rows with the same value of group by key to the same partition.
|
||||
NodeSet removeInjectiveFunctionsFromResultsRecursively(const ActionsDAGPtr & actions)
|
||||
NodeSet removeInjectiveFunctionsFromResultsRecursively(const ActionsDAG & actions)
|
||||
{
|
||||
NodeSet irreducible;
|
||||
NodeSet visited;
|
||||
for (const auto & node : actions->getOutputs())
|
||||
for (const auto & node : actions.getOutputs())
|
||||
removeInjectiveFunctionsFromResultsRecursively(node, irreducible, visited);
|
||||
return irreducible;
|
||||
}
|
||||
@ -146,19 +146,19 @@ bool allOutputsDependsOnlyOnAllowedNodes(
|
||||
/// 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'.
|
||||
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())
|
||||
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;
|
||||
|
||||
/// 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);
|
||||
|
||||
const auto & gb_key_required_columns = group_by_key_actions->getRequiredColumnsNames();
|
||||
const auto & gb_key_required_columns = group_by_key_actions.getRequiredColumnsNames();
|
||||
|
||||
const auto & partition_actions = reading.getStorageMetadata()->getPartitionKey().expression->getActionsDAG();
|
||||
|
||||
@ -169,7 +169,7 @@ bool isPartitionKeySuitsGroupByKey(
|
||||
|
||||
const auto irreducibe_nodes = removeInjectiveFunctionsFromResultsRecursively(group_by_key_actions);
|
||||
|
||||
const auto matches = matchTrees(group_by_key_actions->getOutputs(), partition_actions);
|
||||
const auto matches = matchTrees(group_by_key_actions.getOutputs(), partition_actions);
|
||||
|
||||
return allOutputsDependsOnlyOnAllowedNodes(partition_actions, irreducibe_nodes, matches);
|
||||
}
|
||||
|
@ -943,7 +943,7 @@ SplitPartsWithRangesByPrimaryKeyResult splitPartsWithRangesByPrimaryKey(
|
||||
|
||||
auto syntax_result = TreeRewriter(context).analyze(filter_function, primary_key.expression->getRequiredColumnsWithTypes());
|
||||
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));
|
||||
auto description = fmt::format(
|
||||
"filter values in ({}, {}]", i ? ::toString(borders[i - 1]) : "-inf", i < borders.size() ? ::toString(borders[i]) : "+inf");
|
||||
|
@ -109,8 +109,7 @@ bool restorePrewhereInputs(PrewhereInfo & info, const NameSet & inputs)
|
||||
if (info.row_level_filter)
|
||||
added = added || restoreDAGInputs(*info.row_level_filter, inputs);
|
||||
|
||||
if (info.prewhere_actions)
|
||||
added = added || restoreDAGInputs(*info.prewhere_actions, inputs);
|
||||
added = added || restoreDAGInputs(info.prewhere_actions, inputs);
|
||||
|
||||
return added;
|
||||
}
|
||||
@ -175,7 +174,6 @@ static void updateSortDescriptionForOutputStream(
|
||||
Block original_header = output_stream.header.cloneEmpty();
|
||||
if (prewhere_info)
|
||||
{
|
||||
if (prewhere_info->prewhere_actions)
|
||||
{
|
||||
FindOriginalNodeForOutputName original_column_finder(prewhere_info->prewhere_actions);
|
||||
for (auto & column : original_header)
|
||||
@ -188,7 +186,7 @@ static void updateSortDescriptionForOutputStream(
|
||||
|
||||
if (prewhere_info->row_level_filter)
|
||||
{
|
||||
FindOriginalNodeForOutputName original_column_finder(prewhere_info->row_level_filter);
|
||||
FindOriginalNodeForOutputName original_column_finder(*prewhere_info->row_level_filter);
|
||||
for (auto & column : original_header)
|
||||
{
|
||||
const auto * original_node = original_column_finder.find(column.name);
|
||||
@ -802,7 +800,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_
|
||||
info.use_uncompressed_cache);
|
||||
};
|
||||
|
||||
auto sorting_expr = std::make_shared<ExpressionActions>(metadata_for_reading->getSortingKey().expression->getActionsDAG().clone());
|
||||
auto sorting_expr = metadata_for_reading->getSortingKey().expression;
|
||||
|
||||
SplitPartsWithRangesByPrimaryKeyResult split_ranges_result = splitPartsWithRangesByPrimaryKey(
|
||||
metadata_for_reading->getPrimaryKey(),
|
||||
@ -834,10 +832,10 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_
|
||||
pipes[0].getHeader().getColumnsWithTypeAndName(),
|
||||
pipes[1].getHeader().getColumnsWithTypeAndName(),
|
||||
ActionsDAG::MatchColumnsMode::Name);
|
||||
auto converting_expr = std::make_shared<ExpressionActions>(std::move(conversion_action));
|
||||
pipes[0].addSimpleTransform(
|
||||
[conversion_action](const Block & header)
|
||||
[converting_expr](const Block & header)
|
||||
{
|
||||
auto converting_expr = std::make_shared<ExpressionActions>(conversion_action);
|
||||
return std::make_shared<ExpressionTransform>(header, converting_expr);
|
||||
});
|
||||
return Pipe::unitePipes(std::move(pipes));
|
||||
@ -851,16 +849,16 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_
|
||||
info.use_uncompressed_cache);
|
||||
}
|
||||
|
||||
static ActionsDAGPtr createProjection(const Block & header)
|
||||
static ActionsDAG createProjection(const Block & header)
|
||||
{
|
||||
return std::make_shared<ActionsDAG>(header.getNamesAndTypesList());
|
||||
return ActionsDAG(header.getNamesAndTypesList());
|
||||
}
|
||||
|
||||
Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder(
|
||||
RangesInDataParts && parts_with_ranges,
|
||||
size_t num_streams,
|
||||
const Names & column_names,
|
||||
ActionsDAGPtr & out_projection,
|
||||
std::optional<ActionsDAG> & out_projection,
|
||||
const InputOrderInfoPtr & input_order_info)
|
||||
{
|
||||
const auto & settings = context->getSettingsRef();
|
||||
@ -1052,7 +1050,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder(
|
||||
for (size_t j = 0; j < prefix_size; ++j)
|
||||
sort_description.emplace_back(sorting_columns[j], input_order_info->direction);
|
||||
|
||||
auto sorting_key_expr = std::make_shared<ExpressionActions>(sorting_key_prefix_expr);
|
||||
auto sorting_key_expr = std::make_shared<ExpressionActions>(std::move(sorting_key_prefix_expr));
|
||||
|
||||
auto merge_streams = [&](Pipe & pipe)
|
||||
{
|
||||
@ -1176,7 +1174,7 @@ bool ReadFromMergeTree::doNotMergePartsAcrossPartitionsFinal() const
|
||||
}
|
||||
|
||||
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 & data_settings = data.getSettings();
|
||||
@ -1217,7 +1215,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
|
||||
/// we will store lonely parts with level > 0 to use parallel select on them.
|
||||
RangesInDataParts non_intersecting_parts_by_primary_key;
|
||||
|
||||
auto sorting_expr = std::make_shared<ExpressionActions>(metadata_for_reading->getSortingKey().expression->getActionsDAG().clone());
|
||||
auto sorting_expr = metadata_for_reading->getSortingKey().expression;
|
||||
|
||||
if (prewhere_info)
|
||||
{
|
||||
@ -1338,7 +1336,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
|
||||
|
||||
if (!merging_pipes.empty() && !no_merging_pipes.empty())
|
||||
{
|
||||
out_projection = nullptr; /// We do projection here
|
||||
out_projection = {}; /// We do projection here
|
||||
Pipes pipes;
|
||||
pipes.resize(2);
|
||||
pipes[0] = Pipe::unitePipes(std::move(merging_pipes));
|
||||
@ -1347,10 +1345,10 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
|
||||
pipes[0].getHeader().getColumnsWithTypeAndName(),
|
||||
pipes[1].getHeader().getColumnsWithTypeAndName(),
|
||||
ActionsDAG::MatchColumnsMode::Name);
|
||||
auto converting_expr = std::make_shared<ExpressionActions>(std::move(conversion_action));
|
||||
pipes[0].addSimpleTransform(
|
||||
[conversion_action](const Block & header)
|
||||
[converting_expr](const Block & header)
|
||||
{
|
||||
auto converting_expr = std::make_shared<ExpressionActions>(conversion_action);
|
||||
return std::make_shared<ExpressionTransform>(header, converting_expr);
|
||||
});
|
||||
return Pipe::unitePipes(std::move(pipes));
|
||||
@ -1384,7 +1382,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
|
||||
|
||||
static void buildIndexes(
|
||||
std::optional<ReadFromMergeTree::Indexes> & indexes,
|
||||
ActionsDAGPtr filter_actions_dag,
|
||||
const ActionsDAG * filter_actions_dag,
|
||||
const MergeTreeData & data,
|
||||
const MergeTreeData::DataPartsVector & parts,
|
||||
const ContextPtr & context,
|
||||
@ -1524,11 +1522,12 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes)
|
||||
/// (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
|
||||
/// parallel replicas and unused shards into optimization, similar to projection analysis.
|
||||
query_info.filter_actions_dag = filter_actions_dag;
|
||||
if (filter_actions_dag)
|
||||
query_info.filter_actions_dag = std::make_shared<const ActionsDAG>(filter_actions_dag->clone());
|
||||
|
||||
buildIndexes(
|
||||
indexes,
|
||||
filter_actions_dag,
|
||||
query_info.filter_actions_dag.get(),
|
||||
data,
|
||||
prepared_parts,
|
||||
context,
|
||||
@ -1570,7 +1569,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
|
||||
const Names & primary_key_column_names = primary_key.column_names;
|
||||
|
||||
if (!indexes)
|
||||
buildIndexes(indexes, query_info_.filter_actions_dag, data, parts, context_, query_info_, metadata_snapshot);
|
||||
buildIndexes(indexes, query_info_.filter_actions_dag.get(), data, parts, context_, query_info_, metadata_snapshot);
|
||||
|
||||
if (indexes->part_values && indexes->part_values->empty())
|
||||
return std::make_shared<AnalysisResult>(std::move(result));
|
||||
@ -1838,7 +1837,7 @@ bool ReadFromMergeTree::isQueryWithSampling() const
|
||||
}
|
||||
|
||||
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();
|
||||
Names column_names_to_read = result.column_names_to_read;
|
||||
@ -1899,7 +1898,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);
|
||||
|
||||
@ -1988,7 +1987,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons
|
||||
/// Projection, that needed to drop columns, which have appeared by execution
|
||||
/// of some extra expressions, and to allow execute the same expressions later.
|
||||
/// 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
|
||||
? groupStreamsByPartition(result, result_projection)
|
||||
@ -2005,7 +2004,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons
|
||||
|
||||
if (result.sampling.use_sampling)
|
||||
{
|
||||
auto sampling_actions = std::make_shared<ExpressionActions>(result.sampling.filter_expression);
|
||||
auto sampling_actions = std::make_shared<ExpressionActions>(result.sampling.filter_expression->clone());
|
||||
pipe.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
return std::make_shared<FilterTransform>(
|
||||
@ -2018,12 +2017,12 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons
|
||||
|
||||
Block cur_header = pipe.getHeader();
|
||||
|
||||
auto append_actions = [&result_projection](ActionsDAGPtr actions)
|
||||
auto append_actions = [&result_projection](ActionsDAG actions)
|
||||
{
|
||||
if (!result_projection)
|
||||
result_projection = std::move(actions);
|
||||
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)
|
||||
@ -2043,7 +2042,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons
|
||||
|
||||
if (result_projection)
|
||||
{
|
||||
auto projection_actions = std::make_shared<ExpressionActions>(result_projection);
|
||||
auto projection_actions = std::make_shared<ExpressionActions>(std::move(*result_projection));
|
||||
pipe.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
return std::make_shared<ExpressionTransform>(header, projection_actions);
|
||||
@ -2060,7 +2059,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons
|
||||
ActionsDAG::MatchColumnsMode::Name,
|
||||
true);
|
||||
|
||||
auto converting_dag_expr = std::make_shared<ExpressionActions>(convert_actions_dag);
|
||||
auto converting_dag_expr = std::make_shared<ExpressionActions>(std::move(convert_actions_dag));
|
||||
|
||||
pipe.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
@ -2130,7 +2129,6 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const
|
||||
prefix.push_back(format_settings.indent_char);
|
||||
prefix.push_back(format_settings.indent_char);
|
||||
|
||||
if (prewhere_info->prewhere_actions)
|
||||
{
|
||||
format_settings.out << prefix << "Prewhere filter" << '\n';
|
||||
format_settings.out << prefix << "Prewhere filter column: " << prewhere_info->prewhere_column_name;
|
||||
@ -2138,7 +2136,7 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const
|
||||
format_settings.out << " (removed)";
|
||||
format_settings.out << '\n';
|
||||
|
||||
auto expression = std::make_shared<ExpressionActions>(prewhere_info->prewhere_actions);
|
||||
auto expression = std::make_shared<ExpressionActions>(prewhere_info->prewhere_actions.clone());
|
||||
expression->describeActions(format_settings.out, prefix);
|
||||
}
|
||||
|
||||
@ -2147,7 +2145,7 @@ void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const
|
||||
format_settings.out << prefix << "Row level filter" << '\n';
|
||||
format_settings.out << prefix << "Row level filter column: " << prewhere_info->row_level_column_name << '\n';
|
||||
|
||||
auto expression = std::make_shared<ExpressionActions>(prewhere_info->row_level_filter);
|
||||
auto expression = std::make_shared<ExpressionActions>(prewhere_info->row_level_filter->clone());
|
||||
expression->describeActions(format_settings.out, prefix);
|
||||
}
|
||||
}
|
||||
@ -2168,12 +2166,11 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const
|
||||
std::unique_ptr<JSONBuilder::JSONMap> prewhere_info_map = std::make_unique<JSONBuilder::JSONMap>();
|
||||
prewhere_info_map->add("Need filter", prewhere_info->need_filter);
|
||||
|
||||
if (prewhere_info->prewhere_actions)
|
||||
{
|
||||
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 remove filter column", prewhere_info->remove_prewhere_column);
|
||||
auto expression = std::make_shared<ExpressionActions>(prewhere_info->prewhere_actions);
|
||||
auto expression = std::make_shared<ExpressionActions>(prewhere_info->prewhere_actions.clone());
|
||||
prewhere_filter_map->add("Prewhere filter expression", expression->toTree());
|
||||
|
||||
prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map));
|
||||
@ -2183,7 +2180,7 @@ void ReadFromMergeTree::describeActions(JSONBuilder::JSONMap & map) const
|
||||
{
|
||||
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);
|
||||
auto expression = std::make_shared<ExpressionActions>(prewhere_info->row_level_filter);
|
||||
auto expression = std::make_shared<ExpressionActions>(prewhere_info->row_level_filter->clone());
|
||||
row_level_filter_map->add("Row level filter expression", expression->toTree());
|
||||
|
||||
prewhere_info_map->add("Row level filter", std::move(row_level_filter_map));
|
||||
|
@ -23,7 +23,7 @@ struct MergeTreeDataSelectSamplingData
|
||||
bool read_nothing = false;
|
||||
Float64 used_sample_factor = 1.0;
|
||||
std::shared_ptr<ASTFunction> filter_function;
|
||||
ActionsDAGPtr filter_expression;
|
||||
std::shared_ptr<const ActionsDAG> filter_expression;
|
||||
};
|
||||
|
||||
struct UsefulSkipIndexes
|
||||
@ -243,9 +243,9 @@ private:
|
||||
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 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);
|
||||
|
||||
@ -253,13 +253,13 @@ private:
|
||||
RangesInDataParts && parts_with_ranges,
|
||||
size_t num_streams,
|
||||
const Names & column_names,
|
||||
ActionsDAGPtr & out_projection,
|
||||
std::optional<ActionsDAG> & out_projection,
|
||||
const InputOrderInfoPtr & input_order_info);
|
||||
|
||||
bool doNotMergePartsAcrossPartitionsFinal() const;
|
||||
|
||||
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;
|
||||
|
||||
|
@ -459,7 +459,7 @@ Pipe ReadFromSystemNumbersStep::makePipe()
|
||||
chassert(numbers_storage.step != UInt64{0});
|
||||
|
||||
/// Build rpn of query filters
|
||||
KeyCondition condition(filter_actions_dag, context, column_names, key_expression);
|
||||
KeyCondition condition(filter_actions_dag ? &*filter_actions_dag : nullptr, context, column_names, key_expression);
|
||||
|
||||
if (condition.extractPlainRanges(ranges))
|
||||
{
|
||||
|
@ -34,9 +34,8 @@ Block SourceStepWithFilter::applyPrewhereActions(Block block, const PrewhereInfo
|
||||
block.erase(prewhere_info->row_level_column_name);
|
||||
}
|
||||
|
||||
if (prewhere_info->prewhere_actions)
|
||||
{
|
||||
block = prewhere_info->prewhere_actions->updateHeader(block);
|
||||
block = prewhere_info->prewhere_actions.updateHeader(block);
|
||||
|
||||
auto & prewhere_column = block.getByName(prewhere_info->prewhere_column_name);
|
||||
if (!prewhere_column.type->canBeUsedInBooleanContext())
|
||||
@ -102,7 +101,6 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con
|
||||
prefix.push_back(format_settings.indent_char);
|
||||
prefix.push_back(format_settings.indent_char);
|
||||
|
||||
if (prewhere_info->prewhere_actions)
|
||||
{
|
||||
format_settings.out << prefix << "Prewhere filter" << '\n';
|
||||
format_settings.out << prefix << "Prewhere filter column: " << prewhere_info->prewhere_column_name;
|
||||
@ -110,7 +108,7 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con
|
||||
format_settings.out << " (removed)";
|
||||
format_settings.out << '\n';
|
||||
|
||||
auto expression = std::make_shared<ExpressionActions>(prewhere_info->prewhere_actions);
|
||||
auto expression = std::make_shared<ExpressionActions>(prewhere_info->prewhere_actions.clone());
|
||||
expression->describeActions(format_settings.out, prefix);
|
||||
}
|
||||
|
||||
@ -119,7 +117,7 @@ void SourceStepWithFilter::describeActions(FormatSettings & format_settings) con
|
||||
format_settings.out << prefix << "Row level filter" << '\n';
|
||||
format_settings.out << prefix << "Row level filter column: " << prewhere_info->row_level_column_name << '\n';
|
||||
|
||||
auto expression = std::make_shared<ExpressionActions>(prewhere_info->row_level_filter);
|
||||
auto expression = std::make_shared<ExpressionActions>(prewhere_info->row_level_filter->clone());
|
||||
expression->describeActions(format_settings.out, prefix);
|
||||
}
|
||||
}
|
||||
@ -132,12 +130,11 @@ void SourceStepWithFilter::describeActions(JSONBuilder::JSONMap & map) const
|
||||
std::unique_ptr<JSONBuilder::JSONMap> prewhere_info_map = std::make_unique<JSONBuilder::JSONMap>();
|
||||
prewhere_info_map->add("Need filter", prewhere_info->need_filter);
|
||||
|
||||
if (prewhere_info->prewhere_actions)
|
||||
{
|
||||
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 remove filter column", prewhere_info->remove_prewhere_column);
|
||||
auto expression = std::make_shared<ExpressionActions>(prewhere_info->prewhere_actions);
|
||||
auto expression = std::make_shared<ExpressionActions>(prewhere_info->prewhere_actions.clone());
|
||||
prewhere_filter_map->add("Prewhere filter expression", expression->toTree());
|
||||
|
||||
prewhere_info_map->add("Prewhere filter", std::move(prewhere_filter_map));
|
||||
@ -147,7 +144,7 @@ void SourceStepWithFilter::describeActions(JSONBuilder::JSONMap & map) const
|
||||
{
|
||||
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);
|
||||
auto expression = std::make_shared<ExpressionActions>(prewhere_info->row_level_filter);
|
||||
auto expression = std::make_shared<ExpressionActions>(prewhere_info->row_level_filter->clone());
|
||||
row_level_filter_map->add("Row level filter expression", expression->toTree());
|
||||
|
||||
prewhere_info_map->add("Row level filter", std::move(row_level_filter_map));
|
||||
|
@ -33,7 +33,8 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
const ActionsDAGPtr & getFilterActionsDAG() const { return filter_actions_dag; }
|
||||
const std::optional<ActionsDAG> & getFilterActionsDAG() const { return filter_actions_dag; }
|
||||
std::optional<ActionsDAG> detachFilterActionsDAG() { return std::move(filter_actions_dag); }
|
||||
|
||||
const SelectQueryInfo & getQueryInfo() const { return query_info; }
|
||||
const PrewhereInfoPtr & getPrewhereInfo() const { return prewhere_info; }
|
||||
@ -44,9 +45,9 @@ public:
|
||||
|
||||
const Names & requiredSourceColumns() const { return required_source_columns; }
|
||||
|
||||
void addFilter(ActionsDAGPtr filter_dag, std::string column_name)
|
||||
void addFilter(ActionsDAG filter_dag, std::string column_name)
|
||||
{
|
||||
filter_nodes.nodes.push_back(&filter_dag->findInOutputs(column_name));
|
||||
filter_nodes.nodes.push_back(&filter_dag.findInOutputs(column_name));
|
||||
filter_dags.push_back(std::move(filter_dag));
|
||||
}
|
||||
|
||||
@ -59,7 +60,7 @@ public:
|
||||
void applyFilters()
|
||||
{
|
||||
applyFilters(std::move(filter_nodes));
|
||||
filter_dags = {};
|
||||
filter_dags.clear();
|
||||
}
|
||||
|
||||
virtual void applyFilters(ActionDAGNodes added_filter_nodes);
|
||||
@ -80,12 +81,12 @@ protected:
|
||||
ContextPtr context;
|
||||
std::optional<size_t> limit;
|
||||
|
||||
ActionsDAGPtr filter_actions_dag;
|
||||
std::optional<ActionsDAG> filter_actions_dag;
|
||||
|
||||
private:
|
||||
/// Will be cleared after applyFilters() is called.
|
||||
ActionDAGNodes filter_nodes;
|
||||
std::vector<ActionsDAGPtr> filter_dags;
|
||||
std::vector<ActionsDAG> filter_dags;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -29,7 +29,7 @@ TotalsHavingStep::TotalsHavingStep(
|
||||
const DataStream & input_stream_,
|
||||
const AggregateDescriptions & aggregates_,
|
||||
bool overflow_row_,
|
||||
const ActionsDAGPtr & actions_dag_,
|
||||
std::optional<ActionsDAG> actions_dag_,
|
||||
const std::string & filter_column_,
|
||||
bool remove_filter_,
|
||||
TotalsMode totals_mode_,
|
||||
@ -39,7 +39,7 @@ TotalsHavingStep::TotalsHavingStep(
|
||||
input_stream_,
|
||||
TotalsHavingTransform::transformHeader(
|
||||
input_stream_.header,
|
||||
actions_dag_.get(),
|
||||
actions_dag_ ? &*actions_dag_ : nullptr,
|
||||
filter_column_,
|
||||
remove_filter_,
|
||||
final_,
|
||||
@ -47,7 +47,7 @@ TotalsHavingStep::TotalsHavingStep(
|
||||
getTraits(!filter_column_.empty()))
|
||||
, aggregates(aggregates_)
|
||||
, overflow_row(overflow_row_)
|
||||
, actions_dag(actions_dag_)
|
||||
, actions_dag(std::move(actions_dag_))
|
||||
, filter_column_name(filter_column_)
|
||||
, remove_filter(remove_filter_)
|
||||
, totals_mode(totals_mode_)
|
||||
@ -58,7 +58,7 @@ TotalsHavingStep::TotalsHavingStep(
|
||||
|
||||
void TotalsHavingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings)
|
||||
{
|
||||
auto expression_actions = actions_dag ? std::make_shared<ExpressionActions>(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>(
|
||||
pipeline.getHeader(),
|
||||
@ -101,13 +101,16 @@ void TotalsHavingStep::describeActions(FormatSettings & settings) const
|
||||
if (actions_dag)
|
||||
{
|
||||
bool first = true;
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag);
|
||||
for (const auto & action : expression->getActions())
|
||||
if (actions_dag)
|
||||
{
|
||||
settings.out << prefix << (first ? "Actions: "
|
||||
: " ");
|
||||
first = false;
|
||||
settings.out << action.toString() << '\n';
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag->clone());
|
||||
for (const auto & action : expression->getActions())
|
||||
{
|
||||
settings.out << prefix << (first ? "Actions: "
|
||||
: " ");
|
||||
first = false;
|
||||
settings.out << action.toString() << '\n';
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -118,8 +121,11 @@ void TotalsHavingStep::describeActions(JSONBuilder::JSONMap & map) const
|
||||
if (actions_dag)
|
||||
{
|
||||
map.add("Filter column", filter_column_name);
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag);
|
||||
map.add("Expression", expression->toTree());
|
||||
if (actions_dag)
|
||||
{
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag->clone());
|
||||
map.add("Expression", expression->toTree());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -129,7 +135,7 @@ void TotalsHavingStep::updateOutputStream()
|
||||
input_streams.front(),
|
||||
TotalsHavingTransform::transformHeader(
|
||||
input_streams.front().header,
|
||||
actions_dag.get(),
|
||||
getActions(),
|
||||
filter_column_name,
|
||||
remove_filter,
|
||||
final,
|
||||
|
@ -1,13 +1,11 @@
|
||||
#pragma once
|
||||
#include <Processors/QueryPlan/ITransformingStep.h>
|
||||
#include <Processors/Transforms/finalizeChunk.h>
|
||||
#include <Interpreters/ActionsDAG.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ActionsDAG;
|
||||
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
|
||||
|
||||
enum class TotalsMode : uint8_t;
|
||||
|
||||
/// Execute HAVING and calculate totals. See TotalsHavingTransform.
|
||||
@ -18,7 +16,7 @@ public:
|
||||
const DataStream & input_stream_,
|
||||
const AggregateDescriptions & aggregates_,
|
||||
bool overflow_row_,
|
||||
const ActionsDAGPtr & actions_dag_,
|
||||
std::optional<ActionsDAG> actions_dag_,
|
||||
const std::string & filter_column_,
|
||||
bool remove_filter_,
|
||||
TotalsMode totals_mode_,
|
||||
@ -32,7 +30,7 @@ public:
|
||||
void describeActions(JSONBuilder::JSONMap & map) 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:
|
||||
void updateOutputStream() override;
|
||||
@ -40,7 +38,7 @@ private:
|
||||
const AggregateDescriptions aggregates;
|
||||
|
||||
bool overflow_row;
|
||||
ActionsDAGPtr actions_dag;
|
||||
std::optional<ActionsDAG> actions_dag;
|
||||
String filter_column_name;
|
||||
bool remove_filter;
|
||||
TotalsMode totals_mode;
|
||||
|
@ -6,9 +6,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ActionsDAG;
|
||||
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
|
||||
|
||||
class WindowTransform;
|
||||
|
||||
class WindowStep : public ITransformingStep
|
||||
|
@ -16,13 +16,13 @@ protected:
|
||||
/// Represents pushed down filters in source
|
||||
std::shared_ptr<const KeyCondition> key_condition;
|
||||
|
||||
void setKeyConditionImpl(const ActionsDAGPtr & 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>(
|
||||
filter_actions_dag,
|
||||
filter_actions_dag ? &*filter_actions_dag : nullptr,
|
||||
context,
|
||||
keys.getNames(),
|
||||
std::make_shared<ExpressionActions>(std::make_shared<ActionsDAG>(keys.getColumnsWithTypeAndName())));
|
||||
std::make_shared<ExpressionActions>(ActionsDAG(keys.getColumnsWithTypeAndName())));
|
||||
}
|
||||
|
||||
public:
|
||||
@ -33,6 +33,6 @@ public:
|
||||
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.
|
||||
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);
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -203,7 +203,7 @@ FillingTransform::FillingTransform(
|
||||
, use_with_fill_by_sorting_prefix(use_with_fill_by_sorting_prefix_)
|
||||
{
|
||||
if (interpolate_description)
|
||||
interpolate_actions = std::make_shared<ExpressionActions>(interpolate_description->actions);
|
||||
interpolate_actions = std::make_shared<ExpressionActions>(interpolate_description->actions.clone());
|
||||
|
||||
std::vector<bool> is_fill_column(header_.columns());
|
||||
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_types = partition_name_types.getTypes();
|
||||
partition_minmax_idx_expr = std::make_shared<ExpressionActions>(
|
||||
std::make_shared<ActionsDAG>(partition_name_types), ExpressionActionsSettings::fromContext(getContext()));
|
||||
ActionsDAG(partition_name_types), ExpressionActionsSettings::fromContext(getContext()));
|
||||
}
|
||||
|
||||
NamesAndTypesList all_name_types = metadata_snapshot->getColumns().getAllPhysical();
|
||||
@ -526,7 +526,7 @@ void StorageHive::initMinMaxIndexExpression()
|
||||
hivefile_name_types.push_back(column);
|
||||
}
|
||||
hivefile_minmax_idx_expr = std::make_shared<ExpressionActions>(
|
||||
std::make_shared<ActionsDAG>(hivefile_name_types), ExpressionActionsSettings::fromContext(getContext()));
|
||||
ActionsDAG(hivefile_name_types), ExpressionActionsSettings::fromContext(getContext()));
|
||||
}
|
||||
|
||||
ASTPtr StorageHive::extractKeyExpressionList(const ASTPtr & node)
|
||||
@ -583,7 +583,7 @@ static HiveFilePtr createHiveFile(
|
||||
|
||||
HiveFiles StorageHive::collectHiveFilesFromPartition(
|
||||
const Apache::Hadoop::Hive::Partition & partition,
|
||||
const ActionsDAGPtr & filter_actions_dag,
|
||||
const ActionsDAG * filter_actions_dag,
|
||||
const HiveTableMetadataPtr & hive_table_metadata,
|
||||
const HDFSFSPtr & fs,
|
||||
const ContextPtr & context_,
|
||||
@ -681,7 +681,7 @@ StorageHive::listDirectory(const String & path, const HiveTableMetadataPtr & hiv
|
||||
HiveFilePtr StorageHive::getHiveFileIfNeeded(
|
||||
const FileInfo & file_info,
|
||||
const FieldVector & fields,
|
||||
const ActionsDAGPtr & filter_actions_dag,
|
||||
const ActionsDAG * filter_actions_dag,
|
||||
const HiveTableMetadataPtr & hive_table_metadata,
|
||||
const ContextPtr & context_,
|
||||
PruneLevel prune_level) const
|
||||
@ -828,7 +828,7 @@ void ReadFromHive::createFiles()
|
||||
if (hive_files)
|
||||
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());
|
||||
}
|
||||
|
||||
@ -950,7 +950,7 @@ void ReadFromHive::initializePipeline(QueryPipelineBuilder & pipeline, const Bui
|
||||
|
||||
HiveFiles StorageHive::collectHiveFiles(
|
||||
size_t max_threads,
|
||||
const ActionsDAGPtr & filter_actions_dag,
|
||||
const ActionsDAG * filter_actions_dag,
|
||||
const HiveTableMetadataPtr & hive_table_metadata,
|
||||
const HDFSFSPtr & fs,
|
||||
const ContextPtr & context_,
|
||||
@ -1023,12 +1023,12 @@ SinkToStoragePtr StorageHive::write(const ASTPtr & /*query*/, const StorageMetad
|
||||
std::optional<UInt64> StorageHive::totalRows(const Settings & settings) const
|
||||
{
|
||||
/// 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
|
||||
@ -1043,7 +1043,7 @@ void StorageHive::checkAlterIsPossible(const AlterCommands & commands, ContextPt
|
||||
}
|
||||
|
||||
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
|
||||
if (!supportsSubsetOfColumns())
|
||||
|
@ -57,7 +57,7 @@ public:
|
||||
bool supportsSubsetOfColumns() const;
|
||||
|
||||
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;
|
||||
|
||||
protected:
|
||||
@ -90,7 +90,7 @@ private:
|
||||
|
||||
HiveFiles collectHiveFiles(
|
||||
size_t max_threads,
|
||||
const ActionsDAGPtr & filter_actions_dag,
|
||||
const ActionsDAG * filter_actions_dag,
|
||||
const HiveTableMetadataPtr & hive_table_metadata,
|
||||
const HDFSFSPtr & fs,
|
||||
const ContextPtr & context_,
|
||||
@ -98,7 +98,7 @@ private:
|
||||
|
||||
HiveFiles collectHiveFilesFromPartition(
|
||||
const Apache::Hadoop::Hive::Partition & partition,
|
||||
const ActionsDAGPtr & filter_actions_dag,
|
||||
const ActionsDAG * filter_actions_dag,
|
||||
const HiveTableMetadataPtr & hive_table_metadata,
|
||||
const HDFSFSPtr & fs,
|
||||
const ContextPtr & context_,
|
||||
@ -107,7 +107,7 @@ private:
|
||||
HiveFilePtr getHiveFileIfNeeded(
|
||||
const FileInfo & file_info,
|
||||
const FieldVector & fields,
|
||||
const ActionsDAGPtr & filter_actions_dag,
|
||||
const ActionsDAG * filter_actions_dag,
|
||||
const HiveTableMetadataPtr & hive_table_metadata,
|
||||
const ContextPtr & context_,
|
||||
PruneLevel prune_level = PruneLevel::Max) const;
|
||||
@ -115,7 +115,7 @@ private:
|
||||
void lazyInitialize();
|
||||
|
||||
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;
|
||||
|
||||
|
@ -237,7 +237,7 @@ StorageID IStorage::getStorageID() const
|
||||
return storage_id;
|
||||
}
|
||||
|
||||
ConditionSelectivityEstimator IStorage::getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAGPtr &, ContextPtr) const
|
||||
ConditionSelectivityEstimator IStorage::getConditionSelectivityEstimatorByPredicate(const StorageSnapshotPtr &, const ActionsDAG *, ContextPtr) const
|
||||
{
|
||||
return {};
|
||||
}
|
||||
@ -325,9 +325,8 @@ std::string PrewhereInfo::dump() const
|
||||
ss << "row_level_filter " << row_level_filter->dumpDAG() << "\n";
|
||||
}
|
||||
|
||||
if (prewhere_actions)
|
||||
{
|
||||
ss << "prewhere_actions " << prewhere_actions->dumpDAG() << "\n";
|
||||
ss << "prewhere_actions " << prewhere_actions.dumpDAG() << "\n";
|
||||
}
|
||||
|
||||
ss << "remove_prewhere_column " << remove_prewhere_column
|
||||
@ -341,10 +340,8 @@ std::string FilterDAGInfo::dump() const
|
||||
WriteBufferFromOwnString ss;
|
||||
ss << "FilterDAGInfo for column '" << column_name <<"', do_remove_column "
|
||||
<< do_remove_column << "\n";
|
||||
if (actions)
|
||||
{
|
||||
ss << "actions " << actions->dumpDAG() << "\n";
|
||||
}
|
||||
|
||||
ss << "actions " << actions.dumpDAG() << "\n";
|
||||
|
||||
return ss.str();
|
||||
}
|
||||
|
@ -135,7 +135,7 @@ public:
|
||||
/// Returns true if the storage supports queries with the PREWHERE section.
|
||||
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.
|
||||
/// This is needed for engines whose aggregates data from multiple tables, like Merge.
|
||||
@ -684,7 +684,7 @@ public:
|
||||
virtual std::optional<UInt64> totalRows(const Settings &) const { return {}; }
|
||||
|
||||
/// 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:
|
||||
/// - memory (approximated, resident)
|
||||
|
@ -231,7 +231,7 @@ bool traverseDAGFilter(
|
||||
}
|
||||
|
||||
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)
|
||||
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);
|
||||
|
||||
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>
|
||||
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;
|
||||
result.expression_list_ast = std::make_shared<ASTExpressionList>();
|
||||
result.expression = std::make_shared<ExpressionActions>(std::make_shared<ActionsDAG>(), ExpressionActionsSettings{});
|
||||
result.expression = std::make_shared<ExpressionActions>(ActionsDAG(), ExpressionActionsSettings{});
|
||||
return result;
|
||||
}
|
||||
|
||||
|
@ -163,8 +163,8 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns
|
||||
if (dag)
|
||||
{
|
||||
dag->addMaterializingOutputActions();
|
||||
auto actions = std::make_shared<
|
||||
ExpressionActions>(std::move(dag),
|
||||
auto actions = std::make_shared<ExpressionActions>(
|
||||
std::move(*dag),
|
||||
ExpressionActionsSettings::fromSettings(data_part_info_for_read->getContext()->getSettingsRef()));
|
||||
actions->execute(additional_columns);
|
||||
}
|
||||
|
@ -629,7 +629,7 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown(
|
||||
if (const auto * index_hint = typeid_cast<const FunctionIndexHint *>(adaptor->getFunction().get()))
|
||||
{
|
||||
const auto & index_hint_dag = index_hint->getActions();
|
||||
children = index_hint_dag->getOutputs();
|
||||
children = index_hint_dag.getOutputs();
|
||||
|
||||
for (auto & arg : children)
|
||||
arg = &cloneASTWithInversionPushDown(*arg, inverted_dag, to_inverted, context, need_inversion);
|
||||
@ -696,22 +696,22 @@ const std::unordered_map<String, KeyCondition::SpaceFillingCurveType> KeyConditi
|
||||
{"hilbertEncode", SpaceFillingCurveType::Hilbert}
|
||||
};
|
||||
|
||||
ActionsDAGPtr KeyCondition::cloneASTWithInversionPushDown(ActionsDAG::NodeRawConstPtrs nodes, const ContextPtr & context)
|
||||
ActionsDAG KeyCondition::cloneASTWithInversionPushDown(ActionsDAG::NodeRawConstPtrs nodes, const ContextPtr & context)
|
||||
{
|
||||
auto res = std::make_shared<ActionsDAG>();
|
||||
ActionsDAG res;
|
||||
|
||||
std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> to_inverted;
|
||||
|
||||
for (auto & node : nodes)
|
||||
node = &DB::cloneASTWithInversionPushDown(*node, *res, to_inverted, context, false);
|
||||
node = &DB::cloneASTWithInversionPushDown(*node, res, to_inverted, context, false);
|
||||
|
||||
if (nodes.size() > 1)
|
||||
{
|
||||
auto function_builder = FunctionFactory::instance().get("and", context);
|
||||
nodes = {&res->addFunction(function_builder, std::move(nodes), "")};
|
||||
nodes = {&res.addFunction(function_builder, std::move(nodes), "")};
|
||||
}
|
||||
|
||||
res->getOutputs().swap(nodes);
|
||||
res.getOutputs().swap(nodes);
|
||||
|
||||
return res;
|
||||
}
|
||||
@ -730,7 +730,7 @@ Block KeyCondition::getBlockWithConstants(
|
||||
if (syntax_analyzer_result)
|
||||
{
|
||||
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)
|
||||
result.insert(ColumnWithTypeAndName{action_node->column, action_node->result_type, action_node->result_name});
|
||||
@ -785,7 +785,7 @@ void KeyCondition::getAllSpaceFillingCurves()
|
||||
}
|
||||
|
||||
KeyCondition::KeyCondition(
|
||||
ActionsDAGPtr filter_dag,
|
||||
const ActionsDAG * filter_dag,
|
||||
ContextPtr context,
|
||||
const Names & key_column_names,
|
||||
const ExpressionActionsPtr & key_expr_,
|
||||
@ -826,9 +826,9 @@ KeyCondition::KeyCondition(
|
||||
* are pushed down and applied (when possible) to leaf nodes.
|
||||
*/
|
||||
auto inverted_dag = cloneASTWithInversionPushDown({filter_dag->getOutputs().at(0)}, context);
|
||||
assert(inverted_dag->getOutputs().size() == 1);
|
||||
assert(inverted_dag.getOutputs().size() == 1);
|
||||
|
||||
const auto * inverted_dag_filter_node = inverted_dag->getOutputs()[0];
|
||||
const auto * inverted_dag_filter_node = inverted_dag.getOutputs()[0];
|
||||
|
||||
RPNBuilder<RPNElement> builder(inverted_dag_filter_node, context, [&](const RPNBuilderTreeNode & node, RPNElement & out)
|
||||
{
|
||||
|
@ -42,7 +42,7 @@ class KeyCondition
|
||||
public:
|
||||
/// Construct key condition from ActionsDAG nodes
|
||||
KeyCondition(
|
||||
ActionsDAGPtr filter_dag,
|
||||
const ActionsDAG * filter_dag,
|
||||
ContextPtr context,
|
||||
const Names & key_column_names,
|
||||
const ExpressionActionsPtr & key_expr,
|
||||
@ -134,7 +134,7 @@ public:
|
||||
DataTypePtr current_type,
|
||||
bool single_point = false);
|
||||
|
||||
static ActionsDAGPtr cloneASTWithInversionPushDown(ActionsDAG::NodeRawConstPtrs nodes, const ContextPtr & context);
|
||||
static ActionsDAG cloneASTWithInversionPushDown(ActionsDAG::NodeRawConstPtrs nodes, const ContextPtr & context);
|
||||
|
||||
bool matchesExactContinuousRange() const;
|
||||
|
||||
|
@ -474,7 +474,7 @@ StoragePolicyPtr MergeTreeData::getStoragePolicy() const
|
||||
}
|
||||
|
||||
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->getSettingsRef().allow_statistics_optimize)
|
||||
return {};
|
||||
@ -750,7 +750,7 @@ ExpressionActionsPtr MergeTreeData::getMinMaxExpr(const KeyDescription & partiti
|
||||
if (!partition_key.column_names.empty())
|
||||
partition_key_columns = partition_key.expression->getRequiredColumnsWithTypes();
|
||||
|
||||
return std::make_shared<ExpressionActions>(std::make_shared<ActionsDAG>(partition_key_columns), settings);
|
||||
return std::make_shared<ExpressionActions>(ActionsDAG(partition_key_columns), settings);
|
||||
}
|
||||
|
||||
Names MergeTreeData::getMinMaxColumnsNames(const KeyDescription & partition_key)
|
||||
@ -1138,7 +1138,7 @@ Block MergeTreeData::getBlockWithVirtualsForFilter(
|
||||
|
||||
|
||||
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())
|
||||
return 0;
|
||||
@ -1146,7 +1146,7 @@ std::optional<UInt64> MergeTreeData::totalRowsByPartitionPredicateImpl(
|
||||
auto metadata_snapshot = getInMemoryMetadataPtr();
|
||||
auto virtual_columns_block = getBlockWithVirtualsForFilter(metadata_snapshot, {parts[0]});
|
||||
|
||||
auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), nullptr, /*allow_non_deterministic_functions=*/ false);
|
||||
auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag.getOutputs().at(0), nullptr, /*allow_non_deterministic_functions=*/ false);
|
||||
if (!filter_dag)
|
||||
return {};
|
||||
|
||||
@ -1156,7 +1156,7 @@ std::optional<UInt64> MergeTreeData::totalRowsByPartitionPredicateImpl(
|
||||
if (!virtual_columns_block.has(input->result_name))
|
||||
valid = false;
|
||||
|
||||
PartitionPruner partition_pruner(metadata_snapshot, filter_dag, local_context, true /* strict */);
|
||||
PartitionPruner partition_pruner(metadata_snapshot, &*filter_dag, local_context, true /* strict */);
|
||||
if (partition_pruner.isUseless() && !valid)
|
||||
return {};
|
||||
|
||||
@ -1164,7 +1164,7 @@ std::optional<UInt64> MergeTreeData::totalRowsByPartitionPredicateImpl(
|
||||
if (valid)
|
||||
{
|
||||
virtual_columns_block = getBlockWithVirtualsForFilter(metadata_snapshot, parts);
|
||||
VirtualColumnUtils::filterBlockWithDAG(filter_dag, virtual_columns_block, local_context);
|
||||
VirtualColumnUtils::filterBlockWithExpression(VirtualColumnUtils::buildFilterExpression(std::move(*filter_dag), local_context), virtual_columns_block);
|
||||
part_values = VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_part");
|
||||
if (part_values.empty())
|
||||
return 0;
|
||||
@ -4405,25 +4405,25 @@ bool MergeTreeData::tryRemovePartImmediately(DataPartPtr && part)
|
||||
|
||||
size_t MergeTreeData::getTotalActiveSizeInBytes() const
|
||||
{
|
||||
return total_active_size_bytes.load(std::memory_order_acquire);
|
||||
return total_active_size_bytes.load();
|
||||
}
|
||||
|
||||
|
||||
size_t MergeTreeData::getTotalActiveSizeInRows() const
|
||||
{
|
||||
return total_active_size_rows.load(std::memory_order_acquire);
|
||||
return total_active_size_rows.load();
|
||||
}
|
||||
|
||||
|
||||
size_t MergeTreeData::getActivePartsCount() const
|
||||
{
|
||||
return total_active_size_parts.load(std::memory_order_acquire);
|
||||
return total_active_size_parts.load();
|
||||
}
|
||||
|
||||
|
||||
size_t MergeTreeData::getOutdatedPartsCount() const
|
||||
{
|
||||
return total_outdated_parts_count.load(std::memory_order_relaxed);
|
||||
return total_outdated_parts_count.load();
|
||||
}
|
||||
|
||||
size_t MergeTreeData::getNumberOfOutdatedPartsWithExpiredRemovalTime() const
|
||||
@ -6839,7 +6839,7 @@ using PartitionIdToMaxBlock = std::unordered_map<String, Int64>;
|
||||
Block MergeTreeData::getMinMaxCountProjectionBlock(
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const Names & required_columns,
|
||||
const ActionsDAGPtr & filter_dag,
|
||||
const ActionsDAG * filter_dag,
|
||||
const DataPartsVector & parts,
|
||||
const PartitionIdToMaxBlock * max_block_numbers_to_read,
|
||||
ContextPtr query_context) const
|
||||
@ -7078,7 +7078,7 @@ ActionDAGNodes MergeTreeData::getFiltersForPrimaryKeyAnalysis(const InterpreterS
|
||||
|
||||
ActionDAGNodes filter_nodes;
|
||||
if (auto additional_filter_info = select.getAdditionalQueryInfo())
|
||||
filter_nodes.nodes.push_back(&additional_filter_info->actions->findInOutputs(additional_filter_info->column_name));
|
||||
filter_nodes.nodes.push_back(&additional_filter_info->actions.findInOutputs(additional_filter_info->column_name));
|
||||
|
||||
if (before_where)
|
||||
filter_nodes.nodes.push_back(&before_where->dag.findInOutputs(where_column_name));
|
||||
@ -8184,16 +8184,16 @@ void MergeTreeData::removePartContributionToDataVolume(const DataPartPtr & part)
|
||||
|
||||
void MergeTreeData::increaseDataVolume(ssize_t bytes, ssize_t rows, ssize_t parts)
|
||||
{
|
||||
total_active_size_bytes.fetch_add(bytes, std::memory_order_acq_rel);
|
||||
total_active_size_rows.fetch_add(rows, std::memory_order_acq_rel);
|
||||
total_active_size_parts.fetch_add(parts, std::memory_order_acq_rel);
|
||||
total_active_size_bytes.fetch_add(bytes);
|
||||
total_active_size_rows.fetch_add(rows);
|
||||
total_active_size_parts.fetch_add(parts);
|
||||
}
|
||||
|
||||
void MergeTreeData::setDataVolume(size_t bytes, size_t rows, size_t parts)
|
||||
{
|
||||
total_active_size_bytes.store(bytes, std::memory_order_release);
|
||||
total_active_size_rows.store(rows, std::memory_order_release);
|
||||
total_active_size_parts.store(parts, std::memory_order_release);
|
||||
total_active_size_bytes.store(bytes);
|
||||
total_active_size_rows.store(rows);
|
||||
total_active_size_parts.store(parts);
|
||||
}
|
||||
|
||||
bool MergeTreeData::insertQueryIdOrThrow(const String & query_id, size_t max_queries) const
|
||||
|
@ -403,7 +403,7 @@ public:
|
||||
Block getMinMaxCountProjectionBlock(
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const Names & required_columns,
|
||||
const ActionsDAGPtr & filter_dag,
|
||||
const ActionsDAG * filter_dag,
|
||||
const DataPartsVector & parts,
|
||||
const PartitionIdToMaxBlock * max_block_numbers_to_read,
|
||||
ContextPtr query_context) const;
|
||||
@ -426,7 +426,7 @@ public:
|
||||
|
||||
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;
|
||||
|
||||
@ -1227,7 +1227,7 @@ protected:
|
||||
boost::iterator_range<DataPartIteratorByStateAndInfo> range, const ColumnsDescription & storage_columns);
|
||||
|
||||
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)
|
||||
{
|
||||
|
@ -430,7 +430,7 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling(
|
||||
|
||||
ASTPtr query = sampling.filter_function;
|
||||
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));
|
||||
}
|
||||
}
|
||||
|
||||
@ -444,7 +444,7 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling(
|
||||
}
|
||||
|
||||
void MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset(
|
||||
std::optional<KeyCondition> & part_offset_condition, const ActionsDAGPtr & filter_dag, ContextPtr context)
|
||||
std::optional<KeyCondition> & part_offset_condition, const ActionsDAG * filter_dag, ContextPtr context)
|
||||
{
|
||||
if (!filter_dag)
|
||||
return;
|
||||
@ -465,10 +465,10 @@ void MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset(
|
||||
return;
|
||||
|
||||
part_offset_condition.emplace(KeyCondition{
|
||||
dag,
|
||||
&*dag,
|
||||
context,
|
||||
sample.getNames(),
|
||||
std::make_shared<ExpressionActions>(std::make_shared<ActionsDAG>(sample.getColumnsWithTypeAndName()), ExpressionActionsSettings{}),
|
||||
std::make_shared<ExpressionActions>(ActionsDAG(sample.getColumnsWithTypeAndName()), ExpressionActionsSettings{}),
|
||||
{}});
|
||||
}
|
||||
|
||||
@ -476,7 +476,7 @@ std::optional<std::unordered_set<String>> MergeTreeDataSelectExecutor::filterPar
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MergeTreeData & data,
|
||||
const MergeTreeData::DataPartsVector & parts,
|
||||
const ActionsDAGPtr & filter_dag,
|
||||
const ActionsDAG * filter_dag,
|
||||
ContextPtr context)
|
||||
{
|
||||
if (!filter_dag)
|
||||
@ -488,7 +488,7 @@ std::optional<std::unordered_set<String>> MergeTreeDataSelectExecutor::filterPar
|
||||
return {};
|
||||
|
||||
auto virtual_columns_block = data.getBlockWithVirtualsForFilter(metadata_snapshot, parts);
|
||||
VirtualColumnUtils::filterBlockWithDAG(dag, virtual_columns_block, context);
|
||||
VirtualColumnUtils::filterBlockWithExpression(VirtualColumnUtils::buildFilterExpression(std::move(*dag), context), virtual_columns_block);
|
||||
return VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_part");
|
||||
}
|
||||
|
||||
|
@ -161,7 +161,7 @@ public:
|
||||
|
||||
/// If possible, construct optional key condition from predicates containing _part_offset column.
|
||||
static void buildKeyConditionFromPartOffset(
|
||||
std::optional<KeyCondition> & part_offset_condition, const ActionsDAGPtr & filter_dag, ContextPtr context);
|
||||
std::optional<KeyCondition> & part_offset_condition, const ActionsDAG * filter_dag, ContextPtr context);
|
||||
|
||||
/// If possible, filter using expression on virtual columns.
|
||||
/// Example: SELECT count() FROM table WHERE _part = 'part_name'
|
||||
@ -170,7 +170,7 @@ public:
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MergeTreeData & data,
|
||||
const MergeTreeData::DataPartsVector & parts,
|
||||
const ActionsDAGPtr & filter_dag,
|
||||
const ActionsDAG * filter_dag,
|
||||
ContextPtr context);
|
||||
|
||||
/// Filter parts using minmax index and partition key.
|
||||
|
@ -333,7 +333,7 @@ MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const Selec
|
||||
return std::make_shared<MergeTreeIndexConditionAnnoy>(index, query, distance_function, context);
|
||||
};
|
||||
|
||||
MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const ActionsDAGPtr &, ContextPtr) const
|
||||
MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const ActionsDAG *, ContextPtr) const
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeTreeIndexAnnoy cannot be created with ActionsDAG");
|
||||
}
|
||||
|
@ -99,7 +99,7 @@ public:
|
||||
MergeTreeIndexGranulePtr createIndexGranule() const override;
|
||||
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
|
||||
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const;
|
||||
MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAGPtr &, ContextPtr) const override;
|
||||
MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAG *, ContextPtr) const override;
|
||||
bool isVectorSearch() const override { return true; }
|
||||
|
||||
private:
|
||||
|
@ -201,7 +201,7 @@ bool maybeTrueOnBloomFilter(const IColumn * hash_column, const BloomFilterPtr &
|
||||
}
|
||||
|
||||
MergeTreeIndexConditionBloomFilter::MergeTreeIndexConditionBloomFilter(
|
||||
const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_)
|
||||
const ActionsDAG * filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_)
|
||||
: WithContext(context_), header(header_), hash_functions(hash_functions_)
|
||||
{
|
||||
if (!filter_actions_dag)
|
||||
@ -897,7 +897,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexBloomFilter::createIndexAggregator(con
|
||||
return std::make_shared<MergeTreeIndexAggregatorBloomFilter>(bits_per_row, hash_functions, index.column_names);
|
||||
}
|
||||
|
||||
MergeTreeIndexConditionPtr MergeTreeIndexBloomFilter::createIndexCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const
|
||||
MergeTreeIndexConditionPtr MergeTreeIndexBloomFilter::createIndexCondition(const ActionsDAG * filter_actions_dag, ContextPtr context) const
|
||||
{
|
||||
return std::make_shared<MergeTreeIndexConditionBloomFilter>(filter_actions_dag, context, index.sample_block, hash_functions);
|
||||
}
|
||||
|
@ -69,7 +69,7 @@ public:
|
||||
std::vector<std::pair<size_t, ColumnPtr>> predicate;
|
||||
};
|
||||
|
||||
MergeTreeIndexConditionBloomFilter(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_);
|
||||
MergeTreeIndexConditionBloomFilter(const ActionsDAG * filter_actions_dag, ContextPtr context_, const Block & header_, size_t hash_functions_);
|
||||
|
||||
bool alwaysUnknownOrTrue() const override;
|
||||
|
||||
@ -142,7 +142,7 @@ public:
|
||||
|
||||
MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
|
||||
|
||||
MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context) const override;
|
||||
MergeTreeIndexConditionPtr createIndexCondition(const ActionsDAG * filter_actions_dag, ContextPtr context) const override;
|
||||
|
||||
private:
|
||||
size_t bits_per_row;
|
||||
|
@ -138,7 +138,7 @@ void MergeTreeIndexAggregatorBloomFilterText::update(const Block & block, size_t
|
||||
}
|
||||
|
||||
MergeTreeConditionBloomFilterText::MergeTreeConditionBloomFilterText(
|
||||
const ActionsDAGPtr & filter_actions_dag,
|
||||
const ActionsDAG * filter_actions_dag,
|
||||
ContextPtr context,
|
||||
const Block & index_sample_block,
|
||||
const BloomFilterParameters & params_,
|
||||
@ -733,7 +733,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexBloomFilterText::createIndexAggregator
|
||||
}
|
||||
|
||||
MergeTreeIndexConditionPtr MergeTreeIndexBloomFilterText::createIndexCondition(
|
||||
const ActionsDAGPtr & filter_dag, ContextPtr context) const
|
||||
const ActionsDAG * filter_dag, ContextPtr context) const
|
||||
{
|
||||
return std::make_shared<MergeTreeConditionBloomFilterText>(filter_dag, context, index.sample_block, params, token_extractor.get());
|
||||
}
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user