mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Refactor ActionsDAG
This commit is contained in:
parent
15921fbfcb
commit
9a39459888
@ -14,7 +14,7 @@ AddingDefaultBlockOutputStream::AddingDefaultBlockOutputStream(
|
||||
: output(output_), header(header_)
|
||||
{
|
||||
auto dag = addMissingDefaults(header_, output->getHeader().getNamesAndTypesList(), columns_, context_);
|
||||
adding_defaults_actions = std::make_shared<ExpressionActions>(std::move(dag), context_);
|
||||
adding_defaults_actions = std::make_shared<ExpressionActions>(std::move(dag), ExpressionActionsSettings::fromContext(context_));
|
||||
}
|
||||
|
||||
void AddingDefaultBlockOutputStream::write(const Block & block)
|
||||
|
@ -174,7 +174,7 @@ Block AddingDefaultsBlockInputStream::readImpl()
|
||||
auto dag = evaluateMissingDefaults(evaluate_block, header.getNamesAndTypesList(), columns, context, false);
|
||||
if (dag)
|
||||
{
|
||||
auto actions = std::make_shared<ExpressionActions>(std::move(dag), context);
|
||||
auto actions = std::make_shared<ExpressionActions>(std::move(dag), ExpressionActionsSettings::fromContext(context));
|
||||
actions->execute(evaluate_block);
|
||||
}
|
||||
|
||||
|
@ -32,7 +32,7 @@ namespace ErrorCodes
|
||||
ActionsDAG::ActionsDAG(const NamesAndTypesList & inputs_)
|
||||
{
|
||||
for (const auto & input : inputs_)
|
||||
addInput(input.name, input.type);
|
||||
index.push_back(&addInput(input.name, input.type));
|
||||
}
|
||||
|
||||
ActionsDAG::ActionsDAG(const ColumnsWithTypeAndName & inputs_)
|
||||
@ -49,10 +49,10 @@ ActionsDAG::ActionsDAG(const ColumnsWithTypeAndName & inputs_)
|
||||
/// without any respect to header structure. So, it is a way to drop materialized column and use
|
||||
/// constant value from header.
|
||||
/// We cannot remove such input right now cause inputs positions are important in some cases.
|
||||
addColumn(input);
|
||||
index.push_back(&addColumn(input));
|
||||
}
|
||||
else
|
||||
addInput(input.name, input.type);
|
||||
index.push_back(&addInput(input.name, input.type));
|
||||
}
|
||||
}
|
||||
|
||||
@ -321,7 +321,7 @@ void ActionsDAG::removeUnusedActions(bool allow_remove_inputs)
|
||||
std::unordered_set<const Node *> visited_nodes;
|
||||
std::stack<Node *> stack;
|
||||
|
||||
for (auto * node : index)
|
||||
for (const auto * node : index)
|
||||
{
|
||||
visited_nodes.insert(node);
|
||||
stack.push(const_cast<Node *>(node));
|
||||
@ -374,22 +374,20 @@ void ActionsDAG::removeUnusedActions(bool allow_remove_inputs)
|
||||
|
||||
void ActionsDAG::addAliases(const NamesWithAliases & aliases, bool project)
|
||||
{
|
||||
std::unordered_map<std::string_view, std::list<const Node *>> names_map;
|
||||
std::unordered_map<std::string_view, const Node *> names_map;
|
||||
for (const auto * node : index)
|
||||
names_map[node->result_name].push_back(node);
|
||||
names_map.emplace(node->result_name, node);
|
||||
|
||||
NodeRawConstPtrs required_nodes;
|
||||
|
||||
for (const auto & item : aliases)
|
||||
{
|
||||
auto & nodes_list = names_map[item.first];
|
||||
if (nodes_list.empty())
|
||||
auto it = names_map.find(item.first);
|
||||
if (it == names_map.end())
|
||||
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER,
|
||||
"Unknown column: {}, there are only columns {}", item.first, dumpNames());
|
||||
|
||||
const auto * child = nodes_list.front();
|
||||
nodes_list.pop_front();
|
||||
required_nodes.push_back(child);
|
||||
required_nodes.push_back(it->second);
|
||||
}
|
||||
|
||||
if (project)
|
||||
@ -535,79 +533,97 @@ void ActionsDAG::compileExpressions(size_t min_count_to_compile_expression)
|
||||
removeUnusedActions();
|
||||
}
|
||||
|
||||
void ActionsDAG::transformHeader(Block & block)
|
||||
{
|
||||
auto inputs_mapping = buildNameToNodeMapping(inputs);
|
||||
auto inputs_pos = getInputsPositions(block, inputs_mapping);
|
||||
// void ActionsDAG::transformHeader(Block & block)
|
||||
// {
|
||||
// NameToNodeMap names_mapping;
|
||||
// std::unordered_map<const Node *, size_t> nodes_mapping;
|
||||
// for (size_t i = 0, size = inputs.size(); i < size; ++i)
|
||||
// {
|
||||
// const auto * input = inputs[i];
|
||||
// names_mapping[input->result_name].emplace_back(i);
|
||||
// nodes_mapping[input] = i;
|
||||
// }
|
||||
|
||||
ColumnsWithTypeAndName result;
|
||||
result.reserve(index.size());
|
||||
for (const auto * node : result)
|
||||
{
|
||||
|
||||
}
|
||||
}
|
||||
// auto inputs_mapping = buildNameToNodeMapping(inputs);
|
||||
// auto inputs_pos = getInputsPositions(block, inputs_mapping);
|
||||
|
||||
ActionsDAG::NameToNodeMap ActionsDAG::buildNameToNodeMapping(const NodeRawConstPtrs & nodes)
|
||||
{
|
||||
NameToNodeMap map;
|
||||
for (size_t i = 0, size = nodes.size(); i < size; ++i)
|
||||
{
|
||||
const auto * node = nodes[i];
|
||||
map[node->result_name].emplace_back(i);
|
||||
}
|
||||
// ColumnsWithTypeAndName result;
|
||||
// result.reserve(index.size());
|
||||
// for (const auto * node : result)
|
||||
// {
|
||||
// if (node->type = NodeType::INPUT)
|
||||
// {
|
||||
// ssize_t pos = inputs_pos[nodes_mapping[node]];
|
||||
// if (pos >= 0)
|
||||
// result.push_back(block.getByPosition(pos));
|
||||
// }
|
||||
// else if (node->column)
|
||||
// result.push_back({node->column, node->result_type, node->result_name});
|
||||
// else
|
||||
// result.push_back({node->result_type->, node->result_type, node->result_name});
|
||||
// }
|
||||
// }
|
||||
|
||||
return map;
|
||||
}
|
||||
// ActionsDAG::NameToNodeMap ActionsDAG::buildNameToNodeMapping(const NodeRawConstPtrs & nodes)
|
||||
// {
|
||||
// NameToNodeMap map;
|
||||
// for (size_t i = 0, size = nodes.size(); i < size; ++i)
|
||||
// {
|
||||
// const auto * node = nodes[i];
|
||||
// map[node->result_name].emplace_back(i);
|
||||
// }
|
||||
|
||||
static std::vector<ssize_t> ActionsDAG::getInputsPositions(const Block & block, const NameToNodeMap & inputs_mapping)
|
||||
{
|
||||
std::vector<ssize_t> inputs_pos(inputs.size(), -1);
|
||||
// return map;
|
||||
// }
|
||||
|
||||
for (size_t pos = 0; pos < block.columns(); ++pos)
|
||||
{
|
||||
const auto & col = block.getByPosition(pos);
|
||||
auto it = inputs_mapping.find(col.name);
|
||||
if (it != inputs_mapping.end())
|
||||
{
|
||||
for (auto input_pos : it->second)
|
||||
{
|
||||
if (inputs_pos[input_pos] < 0)
|
||||
{
|
||||
inputs_pos[input_pos] = pos;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
// static std::vector<ssize_t> ActionsDAG::getInputsPositions(const Block & block, const NameToNodeMap & inputs_mapping)
|
||||
// {
|
||||
// std::vector<ssize_t> inputs_pos(inputs.size(), -1);
|
||||
|
||||
return inputs_pos;
|
||||
}
|
||||
// for (size_t pos = 0; pos < block.columns(); ++pos)
|
||||
// {
|
||||
// const auto & col = block.getByPosition(pos);
|
||||
// auto it = inputs_mapping.find(col.name);
|
||||
// if (it != inputs_mapping.end())
|
||||
// {
|
||||
// for (auto input_pos : it->second)
|
||||
// {
|
||||
// if (inputs_pos[input_pos] < 0)
|
||||
// {
|
||||
// inputs_pos[input_pos] = pos;
|
||||
// break;
|
||||
// }
|
||||
// }
|
||||
// }
|
||||
// }
|
||||
|
||||
void ActionsDAG::transformBlock(Block & block, std::vector<ssize_t> inputs_pos, ColumnsWithTypeAndName result_columns)
|
||||
{
|
||||
if (project_input))
|
||||
{
|
||||
block.clear();
|
||||
}
|
||||
else
|
||||
{
|
||||
std::sort(inputs_pos.rbegin(), inputs_pos.rend());
|
||||
for (auto input : execution_context.inputs_pos)
|
||||
if (input >= 0)
|
||||
block.erase(input);
|
||||
}
|
||||
// return inputs_pos;
|
||||
// }
|
||||
|
||||
Block res;
|
||||
// void ActionsDAG::transformBlock(Block & block, std::vector<ssize_t> inputs_pos, ColumnsWithTypeAndName result_columns)
|
||||
// {
|
||||
// if (project_input))
|
||||
// {
|
||||
// block.clear();
|
||||
// }
|
||||
// else
|
||||
// {
|
||||
// std::sort(inputs_pos.rbegin(), inputs_pos.rend());
|
||||
// for (auto input : execution_context.inputs_pos)
|
||||
// if (input >= 0)
|
||||
// block.erase(input);
|
||||
// }
|
||||
|
||||
for (auto & col : result_columns)
|
||||
res.insert(std::move(col));
|
||||
// Block res;
|
||||
|
||||
for (const auto & item : block)
|
||||
res.insert(std::move(item));
|
||||
// for (auto & col : result_columns)
|
||||
// res.insert(std::move(col));
|
||||
|
||||
block.swap(res);
|
||||
}
|
||||
// for (const auto & item : block)
|
||||
// res.insert(std::move(item));
|
||||
|
||||
// block.swap(res);
|
||||
// }
|
||||
|
||||
std::string ActionsDAG::dumpDAG() const
|
||||
{
|
||||
|
@ -236,15 +236,15 @@ public:
|
||||
bool hasStatefulFunctions() const;
|
||||
bool trivial() const; /// If actions has no functions or array join.
|
||||
|
||||
void transformHeader(Block & block);
|
||||
// void transformHeader(Block & block);
|
||||
|
||||
/// This map helps to find input position by it's name.
|
||||
/// Key is a view to input::result_name.
|
||||
/// Result is a list because it is allowed for inputs to have same names.
|
||||
using NameToNodeMap = std::unordered_map<std::string_view, std::list<size_t>>;
|
||||
static NameToNodeMap buildNameToNodeMapping(const NodeRawConstPtrs & nodes);
|
||||
static std::vector<ssize_t> getInputsPositions(const Block & block, const NameToNodeMap & inputs_mapping);
|
||||
void transformHeader(Block & block, ColumnsWithTypeAndName result_columns) const;
|
||||
// /// This map helps to find input position by it's name.
|
||||
// /// Key is a view to input::result_name.
|
||||
// /// Result is a list because it is allowed for inputs to have same names.
|
||||
// using NameToNodeMap = std::unordered_map<std::string_view, std::list<size_t>>;
|
||||
// static NameToNodeMap buildNameToNodeMapping(const NodeRawConstPtrs & nodes);
|
||||
// static std::vector<ssize_t> getInputsPositions(const Block & block, const NameToNodeMap & inputs_mapping);
|
||||
// void transformBlock(Block & block, std::vector<ssize_t> inputs_pos, ColumnsWithTypeAndName result_columns) const;
|
||||
|
||||
void compileExpressions(size_t min_count_to_compile_expression);
|
||||
|
||||
|
@ -399,7 +399,11 @@ class ScopeStack::Index
|
||||
ActionsDAG::NodeRawConstPtrs & index;
|
||||
|
||||
public:
|
||||
explicit Index(ActionsDAG::NodeRawConstPtrs & index_) : index(index_) {}
|
||||
explicit Index(ActionsDAG::NodeRawConstPtrs & index_) : index(index_)
|
||||
{
|
||||
for (const auto * node : index)
|
||||
map.emplace(node->result_name, node);
|
||||
}
|
||||
|
||||
void addNode(const ActionsDAG::Node * node)
|
||||
{
|
||||
@ -465,18 +469,15 @@ ScopeStack::ScopeStack(ActionsDAGPtr actions_dag, const Context & context_)
|
||||
level.index = std::make_unique<ScopeStack::Index>(level.actions_dag->getIndex());
|
||||
|
||||
for (const auto & node : level.actions_dag->getIndex())
|
||||
{
|
||||
level.index->addNode(node);
|
||||
|
||||
if (node->type == ActionsDAG::ActionType::INPUT)
|
||||
level.inputs.emplace(node->result_name);
|
||||
}
|
||||
}
|
||||
|
||||
void ScopeStack::pushLevel(const NamesAndTypesList & input_columns)
|
||||
{
|
||||
auto & level = stack.emplace_back();
|
||||
level.actions_dag = std::make_shared<ActionsDAG>();
|
||||
level.index = std::make_unique<ScopeStack::Index>(level.actions_dag->getIndex());
|
||||
const auto & prev = stack[stack.size() - 2];
|
||||
|
||||
for (const auto & input_column : input_columns)
|
||||
@ -1009,7 +1010,9 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
||||
String result_name = lambda->arguments->children.at(1)->getColumnName();
|
||||
lambda_dag->removeUnusedActions(Names(1, result_name));
|
||||
|
||||
auto lambda_actions = std::make_shared<ExpressionActions>(lambda_dag, data.context);
|
||||
auto lambda_actions = std::make_shared<ExpressionActions>(
|
||||
lambda_dag,
|
||||
ExpressionActionsSettings::fromContext(data.context));
|
||||
|
||||
DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type;
|
||||
|
||||
|
@ -188,7 +188,7 @@ void AsynchronousMetrics::update()
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
{
|
||||
if (auto compiled_expression_cache = global_context.getCompiledExpressionCache())
|
||||
if (auto * compiled_expression_cache = CompiledExpressionCacheFactory::instance().tryGetCache())
|
||||
new_values["CompiledExpressionCacheCount"] = compiled_expression_cache->count();
|
||||
}
|
||||
#endif
|
||||
|
@ -15,6 +15,8 @@
|
||||
#include <Processors/Sources/DelayedSource.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Processors/QueryPlan/ExpressionStep.h>
|
||||
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
||||
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
@ -93,7 +95,7 @@ std::unique_ptr<QueryPlan> createLocalPlan(
|
||||
ActionsDAG::MatchColumnsMode::Name,
|
||||
true);
|
||||
|
||||
auto converting = std::make_unique<ExpressionStep>(query_plan->getCurrentDataStream(), convert_actions_dag, context);
|
||||
auto converting = std::make_unique<ExpressionStep>(query_plan->getCurrentDataStream(), convert_actions_dag);
|
||||
converting->setStepDescription("Convert block structure for query from local replica");
|
||||
query_plan->addStep(std::move(converting));
|
||||
|
||||
@ -284,7 +286,9 @@ void SelectStreamFactory::createForShard(
|
||||
if (try_results.empty() || local_delay < max_remote_delay)
|
||||
{
|
||||
auto plan = createLocalPlan(modified_query_ast, header, context, stage);
|
||||
return QueryPipeline::getPipe(std::move(*plan->buildQueryPipeline(QueryPlanOptimizationSettings(context.getSettingsRef()))));
|
||||
return QueryPipeline::getPipe(std::move(*plan->buildQueryPipeline(
|
||||
QueryPlanOptimizationSettings::fromContext(*context_ptr),
|
||||
BuildQueryPipelineSettings::fromContext(*context_ptr))));
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -157,7 +157,7 @@ void executeQuery(
|
||||
input_streams.emplace_back(plan->getCurrentDataStream());
|
||||
|
||||
auto header = input_streams.front().header;
|
||||
auto union_step = std::make_unique<UnionStep>(std::move(input_streams), header, context);
|
||||
auto union_step = std::make_unique<UnionStep>(std::move(input_streams), header);
|
||||
query_plan.unitePlans(std::move(union_step), std::move(plans));
|
||||
}
|
||||
|
||||
|
@ -44,7 +44,7 @@ namespace ErrorCodes
|
||||
|
||||
ExpressionActions::~ExpressionActions() = default;
|
||||
|
||||
ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_, const Settings & settings)
|
||||
ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings)
|
||||
{
|
||||
actions_dag = actions_dag_->clone();
|
||||
|
||||
@ -142,7 +142,7 @@ void ExpressionActions::linearizeActions()
|
||||
|
||||
ExpressionActions::Arguments arguments;
|
||||
arguments.reserve(cur.node->children.size());
|
||||
for (auto * child : cur.node->children)
|
||||
for (const auto * child : cur.node->children)
|
||||
{
|
||||
auto & arg = data[reverse_index[child]];
|
||||
|
||||
@ -207,9 +207,8 @@ void ExpressionActions::linearizeActions()
|
||||
auto pos = required_columns.size();
|
||||
actions[cur.position].arguments.front().pos = pos;
|
||||
required_columns.push_back({input->result_name, input->result_type});
|
||||
input_positions[input->result_name].emplace_back(pos);
|
||||
}
|
||||
|
||||
input_positions = actions_dag->buildNameToNodeMapping(inputs);
|
||||
}
|
||||
|
||||
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Core/Block.h>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <Interpreters/ActionsDAG.h>
|
||||
#include <Interpreters/ExpressionActionsSettings.h>
|
||||
|
||||
#include <variant>
|
||||
|
||||
@ -29,7 +30,6 @@ using ArrayJoinActionPtr = std::shared_ptr<ArrayJoinAction>;
|
||||
class ExpressionActions;
|
||||
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
||||
|
||||
class Context;
|
||||
|
||||
/// Sequence of actions on the block.
|
||||
/// Is used to calculate expressions.
|
||||
@ -81,18 +81,9 @@ private:
|
||||
size_t max_temporary_non_const_columns = 0;
|
||||
|
||||
public:
|
||||
struct Settings
|
||||
{
|
||||
bool compile_expressions = false;
|
||||
size_t min_count_to_compile_expression = 0;
|
||||
|
||||
size_t max_temporary_columns = 0;
|
||||
size_t max_temporary_non_const_columns = 0;
|
||||
}
|
||||
|
||||
ExpressionActions() = delete;
|
||||
~ExpressionActions();
|
||||
explicit ExpressionActions(ActionsDAGPtr actions_dag_, const Context & context);
|
||||
explicit ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings);
|
||||
ExpressionActions(const ExpressionActions &) = default;
|
||||
ExpressionActions & operator=(const ExpressionActions &) = default;
|
||||
|
||||
|
24
src/Interpreters/ExpressionActionsSettings.cpp
Normal file
24
src/Interpreters/ExpressionActionsSettings.cpp
Normal file
@ -0,0 +1,24 @@
|
||||
#include <Interpreters/ExpressionActionsSettings.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ExpressionActionsSettings ExpressionActionsSettings::fromSettings(const Settings & from)
|
||||
{
|
||||
ExpressionActionsSettings settings;
|
||||
settings.compile_expressions = from.compile_expressions;
|
||||
settings.min_count_to_compile_expression = from.min_count_to_compile_expression;
|
||||
settings.max_temporary_columns = from.max_temporary_columns;
|
||||
settings.max_temporary_non_const_columns = from.max_temporary_non_const_columns;
|
||||
|
||||
return settings;
|
||||
}
|
||||
|
||||
ExpressionActionsSettings ExpressionActionsSettings::fromContext(const Context & from)
|
||||
{
|
||||
return fromSettings(from.getSettingsRef());
|
||||
}
|
||||
|
||||
}
|
23
src/Interpreters/ExpressionActionsSettings.h
Normal file
23
src/Interpreters/ExpressionActionsSettings.h
Normal file
@ -0,0 +1,23 @@
|
||||
#pragma once
|
||||
|
||||
#include <cstddef>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct Settings;
|
||||
class Context;
|
||||
|
||||
struct ExpressionActionsSettings
|
||||
{
|
||||
bool compile_expressions = false;
|
||||
size_t min_count_to_compile_expression = 0;
|
||||
|
||||
size_t max_temporary_columns = 0;
|
||||
size_t max_temporary_non_const_columns = 0;
|
||||
|
||||
static ExpressionActionsSettings fromSettings(const Settings & from);
|
||||
static ExpressionActionsSettings fromContext(const Context & from);
|
||||
};
|
||||
|
||||
}
|
@ -910,7 +910,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere(
|
||||
auto tmp_actions_dag = std::make_shared<ActionsDAG>(sourceColumns());
|
||||
getRootActions(select_query->prewhere(), only_types, tmp_actions_dag);
|
||||
tmp_actions_dag->removeUnusedActions({prewhere_column_name});
|
||||
auto tmp_actions = std::make_shared<ExpressionActions>(tmp_actions_dag, context);
|
||||
auto tmp_actions = std::make_shared<ExpressionActions>(tmp_actions_dag, ExpressionActionsSettings::fromContext(context));
|
||||
auto required_columns = tmp_actions->getRequiredColumns();
|
||||
NameSet required_source_columns(required_columns.begin(), required_columns.end());
|
||||
|
||||
@ -1040,7 +1040,7 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain
|
||||
{
|
||||
auto actions_dag = std::make_shared<ActionsDAG>(columns_after_join);
|
||||
getRootActions(child, only_types, actions_dag);
|
||||
group_by_elements_actions.emplace_back(std::make_shared<ExpressionActions>(actions_dag, context));
|
||||
group_by_elements_actions.emplace_back(std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(context)));
|
||||
}
|
||||
}
|
||||
|
||||
@ -1198,7 +1198,8 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai
|
||||
{
|
||||
auto actions_dag = std::make_shared<ActionsDAG>(columns_after_join);
|
||||
getRootActions(child, only_types, actions_dag);
|
||||
order_by_elements_actions.emplace_back(std::make_shared<ExpressionActions>(actions_dag, context));
|
||||
order_by_elements_actions.emplace_back(
|
||||
std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(context)));
|
||||
}
|
||||
}
|
||||
|
||||
@ -1339,18 +1340,26 @@ ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool project_r
|
||||
|
||||
if (!(add_aliases && project_result))
|
||||
{
|
||||
NameSet name_set(result_names.begin(), result_names.end());
|
||||
/// We will not delete the original columns.
|
||||
for (const auto & column_name_type : sourceColumns())
|
||||
result_names.push_back(column_name_type.name);
|
||||
{
|
||||
if (name_set.count(column_name_type.name) == 0)
|
||||
{
|
||||
result_names.push_back(column_name_type.name);
|
||||
name_set.insert(column_name_type.name);
|
||||
}
|
||||
}
|
||||
|
||||
actions_dag->removeUnusedActions(result_names);
|
||||
}
|
||||
|
||||
actions_dag->removeUnusedActions(result_names);
|
||||
return actions_dag;
|
||||
}
|
||||
|
||||
ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool project_result)
|
||||
{
|
||||
return std::make_shared<ExpressionActions>(getActionsDAG(add_aliases, project_result), context);
|
||||
return std::make_shared<ExpressionActions>(getActionsDAG(add_aliases, project_result), ExpressionActionsSettings::fromContext(context));
|
||||
}
|
||||
|
||||
|
||||
@ -1359,7 +1368,7 @@ ExpressionActionsPtr ExpressionAnalyzer::getConstActions()
|
||||
auto actions = std::make_shared<ActionsDAG>(NamesAndTypesList());
|
||||
|
||||
getRootActions(query, true, actions, true);
|
||||
return std::make_shared<ExpressionActions>(actions, context);
|
||||
return std::make_shared<ExpressionActions>(actions, ExpressionActionsSettings::fromContext(context));
|
||||
}
|
||||
|
||||
ActionsDAGPtr SelectQueryExpressionAnalyzer::simpleSelectActions()
|
||||
@ -1450,7 +1459,9 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
|
||||
Block before_prewhere_sample = source_header;
|
||||
if (sanitizeBlock(before_prewhere_sample))
|
||||
{
|
||||
ExpressionActions(prewhere_info->prewhere_actions, context).execute(before_prewhere_sample);
|
||||
ExpressionActions(
|
||||
prewhere_info->prewhere_actions,
|
||||
ExpressionActionsSettings::fromSettings(context.getSettingsRef())).execute(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)
|
||||
@ -1483,7 +1494,9 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
|
||||
before_where_sample = source_header;
|
||||
if (sanitizeBlock(before_where_sample))
|
||||
{
|
||||
ExpressionActions(before_where, context).execute(before_where_sample);
|
||||
ExpressionActions(
|
||||
before_where,
|
||||
ExpressionActionsSettings::fromSettings(context.getSettingsRef())).execute(before_where_sample);
|
||||
auto & column_elem = before_where_sample.getByName(query.where()->getColumnName());
|
||||
/// If the filter column is a constant, record it.
|
||||
if (column_elem.column)
|
||||
|
@ -900,7 +900,7 @@ void ActionsDAG::compileFunctions(size_t min_count_to_compile_expression)
|
||||
}
|
||||
}
|
||||
|
||||
static CompiledExpressionCacheFactory & CompiledExpressionCacheFactory::getInstance()
|
||||
CompiledExpressionCacheFactory & CompiledExpressionCacheFactory::instance()
|
||||
{
|
||||
static CompiledExpressionCacheFactory factory;
|
||||
return factory;
|
||||
@ -914,7 +914,7 @@ void CompiledExpressionCacheFactory::init(size_t cache_size)
|
||||
cache = std::make_unique<CompiledExpressionCache>(cache_size);
|
||||
}
|
||||
|
||||
void CompiledExpressionCache * CompiledExpressionCacheFactory::tryGetCache()
|
||||
CompiledExpressionCache * CompiledExpressionCacheFactory::tryGetCache()
|
||||
{
|
||||
return cache.get();
|
||||
}
|
||||
|
@ -104,10 +104,9 @@ class CompiledExpressionCacheFactory
|
||||
{
|
||||
private:
|
||||
std::unique_ptr<CompiledExpressionCache> cache;
|
||||
size_t cache_size;
|
||||
|
||||
public:
|
||||
static CompiledExpressionCacheFactory & getInstance();
|
||||
static CompiledExpressionCacheFactory & instance();
|
||||
|
||||
void init(size_t cache_size);
|
||||
CompiledExpressionCache * tryGetCache();
|
||||
|
@ -14,6 +14,8 @@
|
||||
|
||||
#include <Storages/StorageView.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
||||
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
||||
#include <Processors/printPipeline.h>
|
||||
|
||||
namespace DB
|
||||
@ -251,7 +253,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl()
|
||||
interpreter.buildQueryPlan(plan);
|
||||
|
||||
if (settings.optimize)
|
||||
plan.optimize(QueryPlanOptimizationSettings(context.getSettingsRef()));
|
||||
plan.optimize(QueryPlanOptimizationSettings::fromContext(context));
|
||||
|
||||
plan.explainPlan(buf, settings.query_plan_options);
|
||||
}
|
||||
@ -265,7 +267,9 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl()
|
||||
|
||||
InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), context, SelectQueryOptions());
|
||||
interpreter.buildQueryPlan(plan);
|
||||
auto pipeline = plan.buildQueryPipeline(QueryPlanOptimizationSettings(context.getSettingsRef()));
|
||||
auto pipeline = plan.buildQueryPipeline(
|
||||
QueryPlanOptimizationSettings::fromContext(context),
|
||||
BuildQueryPipelineSettings::fromContext(context));
|
||||
|
||||
if (settings.graph)
|
||||
{
|
||||
|
@ -250,7 +250,7 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
}
|
||||
}
|
||||
|
||||
res.pipeline = QueryPipeline::unitePipelines(std::move(pipelines), {}, context);
|
||||
res.pipeline = QueryPipeline::unitePipelines(std::move(pipelines), {}, ExpressionActionsSettings::fromContext(context));
|
||||
}
|
||||
}
|
||||
|
||||
@ -378,7 +378,7 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
res.pipeline.getHeader().getColumnsWithTypeAndName(),
|
||||
header.getColumnsWithTypeAndName(),
|
||||
ActionsDAG::MatchColumnsMode::Position);
|
||||
auto actions = std::make_shared<ExpressionActions>(actions_dag, context);
|
||||
auto actions = std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(context));
|
||||
|
||||
res.pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr
|
||||
{
|
||||
|
@ -62,6 +62,7 @@
|
||||
#include <Processors/QueryPlan/SettingQuotaAndLimitsStep.h>
|
||||
#include <Processors/QueryPlan/TotalsHavingStep.h>
|
||||
#include <Processors/QueryPlan/WindowStep.h>
|
||||
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
||||
#include <Processors/Sources/NullSource.h>
|
||||
#include <Processors/Sources/SourceFromInputStream.h>
|
||||
#include <Processors/Transforms/AggregatingTransform.h>
|
||||
@ -536,7 +537,7 @@ void InterpreterSelectQuery::buildQueryPlan(QueryPlan & query_plan)
|
||||
ActionsDAG::MatchColumnsMode::Name,
|
||||
true);
|
||||
|
||||
auto converting = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), convert_actions_dag, *context);
|
||||
auto converting = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), convert_actions_dag);
|
||||
query_plan.addStep(std::move(converting));
|
||||
}
|
||||
}
|
||||
@ -548,7 +549,9 @@ BlockIO InterpreterSelectQuery::execute()
|
||||
|
||||
buildQueryPlan(query_plan);
|
||||
|
||||
res.pipeline = std::move(*query_plan.buildQueryPipeline(QueryPlanOptimizationSettings(context->getSettingsRef())));
|
||||
res.pipeline = std::move(*query_plan.buildQueryPipeline(
|
||||
QueryPlanOptimizationSettings::fromContext(*context),
|
||||
BuildQueryPipelineSettings::fromContext(*context)));
|
||||
return res;
|
||||
}
|
||||
|
||||
@ -593,7 +596,9 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
|
||||
|
||||
if (analysis_result.prewhere_info)
|
||||
{
|
||||
ExpressionActions(analysis_result.prewhere_info->prewhere_actions, *context).execute(header);
|
||||
ExpressionActions(
|
||||
analysis_result.prewhere_info->prewhere_actions,
|
||||
ExpressionActionsSettings::fromContext(*context)).execute(header);
|
||||
if (analysis_result.prewhere_info->remove_prewhere_column)
|
||||
header.erase(analysis_result.prewhere_info->prewhere_column_name);
|
||||
}
|
||||
@ -849,8 +854,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
|
||||
query_plan.getCurrentDataStream(),
|
||||
expressions.prewhere_info->prewhere_actions,
|
||||
expressions.prewhere_info->prewhere_column_name,
|
||||
expressions.prewhere_info->remove_prewhere_column,
|
||||
*context);
|
||||
expressions.prewhere_info->remove_prewhere_column);
|
||||
|
||||
prewhere_step->setStepDescription("PREWHERE");
|
||||
query_plan.addStep(std::move(prewhere_step));
|
||||
@ -988,8 +992,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
|
||||
{
|
||||
QueryPlanStepPtr array_join_step = std::make_unique<ArrayJoinStep>(
|
||||
query_plan.getCurrentDataStream(),
|
||||
expressions.array_join,
|
||||
*context);
|
||||
expressions.array_join);
|
||||
|
||||
array_join_step->setStepDescription("ARRAY JOIN");
|
||||
query_plan.addStep(std::move(array_join_step));
|
||||
@ -1592,17 +1595,18 @@ void InterpreterSelectQuery::executeFetchColumns(
|
||||
|
||||
query_info.syntax_analyzer_result = syntax_analyzer_result;
|
||||
query_info.sets = query_analyzer->getPreparedSets();
|
||||
auto actions_settings = ExpressionActionsSettings::fromContext(*context);
|
||||
|
||||
if (prewhere_info)
|
||||
{
|
||||
query_info.prewhere_info = std::make_shared<PrewhereInfo>(
|
||||
std::make_shared<ExpressionActions>(prewhere_info->prewhere_actions, *context),
|
||||
std::make_shared<ExpressionActions>(prewhere_info->prewhere_actions, actions_settings),
|
||||
prewhere_info->prewhere_column_name);
|
||||
|
||||
if (prewhere_info->alias_actions)
|
||||
query_info.prewhere_info->alias_actions = std::make_shared<ExpressionActions>(prewhere_info->alias_actions, *context);
|
||||
query_info.prewhere_info->alias_actions = std::make_shared<ExpressionActions>(prewhere_info->alias_actions, actions_settings);
|
||||
if (prewhere_info->remove_columns_actions)
|
||||
query_info.prewhere_info->remove_columns_actions = std::make_shared<ExpressionActions>(prewhere_info->remove_columns_actions, *context);
|
||||
query_info.prewhere_info->remove_columns_actions = std::make_shared<ExpressionActions>(prewhere_info->remove_columns_actions, actions_settings);
|
||||
|
||||
query_info.prewhere_info->remove_prewhere_column = prewhere_info->remove_prewhere_column;
|
||||
query_info.prewhere_info->need_filter = prewhere_info->need_filter;
|
||||
@ -1700,8 +1704,7 @@ void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsD
|
||||
query_plan.getCurrentDataStream(),
|
||||
expression,
|
||||
getSelectQuery().where()->getColumnName(),
|
||||
remove_filter,
|
||||
*context);
|
||||
remove_filter);
|
||||
|
||||
where_step->setStepDescription("WHERE");
|
||||
query_plan.addStep(std::move(where_step));
|
||||
@ -1710,7 +1713,7 @@ void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsD
|
||||
|
||||
void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info)
|
||||
{
|
||||
auto expression_before_aggregation = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), expression, *context);
|
||||
auto expression_before_aggregation = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), expression);
|
||||
expression_before_aggregation->setStepDescription("Before GROUP BY");
|
||||
query_plan.addStep(std::move(expression_before_aggregation));
|
||||
|
||||
@ -1809,7 +1812,7 @@ void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const Actions
|
||||
{
|
||||
auto having_step = std::make_unique<FilterStep>(
|
||||
query_plan.getCurrentDataStream(),
|
||||
expression, getSelectQuery().having()->getColumnName(), false, *context);
|
||||
expression, getSelectQuery().having()->getColumnName(), false);
|
||||
|
||||
having_step->setStepDescription("HAVING");
|
||||
query_plan.addStep(std::move(having_step));
|
||||
@ -1822,7 +1825,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving(QueryPlan & query_plan, bool
|
||||
|
||||
auto totals_having_step = std::make_unique<TotalsHavingStep>(
|
||||
query_plan.getCurrentDataStream(),
|
||||
overflow_row, expression, *context,
|
||||
overflow_row, expression,
|
||||
has_having ? getSelectQuery().having()->getColumnName() : "",
|
||||
settings.totals_mode, settings.totals_auto_threshold, final);
|
||||
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include <Processors/QueryPlan/UnionStep.h>
|
||||
#include <Processors/QueryPlan/LimitStep.h>
|
||||
#include <Processors/QueryPlan/OffsetStep.h>
|
||||
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
@ -371,7 +372,7 @@ void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan)
|
||||
}
|
||||
|
||||
auto max_threads = context->getSettingsRef().max_threads;
|
||||
auto union_step = std::make_unique<UnionStep>(std::move(data_streams), result_header, *context, max_threads);
|
||||
auto union_step = std::make_unique<UnionStep>(std::move(data_streams), result_header, max_threads);
|
||||
|
||||
query_plan.unitePlans(std::move(union_step), std::move(plans));
|
||||
|
||||
@ -413,7 +414,9 @@ BlockIO InterpreterSelectWithUnionQuery::execute()
|
||||
QueryPlan query_plan;
|
||||
buildQueryPlan(query_plan);
|
||||
|
||||
auto pipeline = query_plan.buildQueryPipeline(QueryPlanOptimizationSettings(context->getSettingsRef()));
|
||||
auto pipeline = query_plan.buildQueryPipeline(
|
||||
QueryPlanOptimizationSettings::fromContext(*context),
|
||||
BuildQueryPipelineSettings::fromContext(*context));
|
||||
|
||||
res.pipeline = std::move(*pipeline);
|
||||
res.pipeline.addInterpreterContext(context);
|
||||
|
@ -24,6 +24,7 @@
|
||||
#include <Interpreters/MetricLog.h>
|
||||
#include <Interpreters/AsynchronousMetricLog.h>
|
||||
#include <Interpreters/OpenTelemetrySpanLog.h>
|
||||
#include <Interpreters/ExpressionJIT.h>
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <Access/AllowedClientHosts.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
@ -270,7 +271,8 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
case Type::DROP_COMPILED_EXPRESSION_CACHE:
|
||||
context.checkAccess(AccessType::SYSTEM_DROP_COMPILED_EXPRESSION_CACHE);
|
||||
system_context.dropCompiledExpressionCache();
|
||||
if (auto * cache = CompiledExpressionCacheFactory::instance().tryGetCache())
|
||||
cache->reset();
|
||||
break;
|
||||
#endif
|
||||
case Type::RELOAD_DICTIONARY:
|
||||
|
@ -748,12 +748,12 @@ QueryPipelinePtr MutationsInterpreter::addStreamsForLaterStages(const std::vecto
|
||||
if (i < stage.filter_column_names.size())
|
||||
{
|
||||
/// Execute DELETEs.
|
||||
plan.addStep(std::make_unique<FilterStep>(plan.getCurrentDataStream(), step->actions(), stage.filter_column_names[i], false, context));
|
||||
plan.addStep(std::make_unique<FilterStep>(plan.getCurrentDataStream(), step->actions(), stage.filter_column_names[i], false));
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Execute UPDATE or final projection.
|
||||
plan.addStep(std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), step->actions(), context));
|
||||
plan.addStep(std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), step->actions()));
|
||||
}
|
||||
}
|
||||
|
||||
@ -767,7 +767,10 @@ QueryPipelinePtr MutationsInterpreter::addStreamsForLaterStages(const std::vecto
|
||||
}
|
||||
}
|
||||
|
||||
auto pipeline = plan.buildQueryPipeline(QueryPlanOptimizationSettings(context.getSettingsRef()));
|
||||
auto pipeline = plan.buildQueryPipeline(
|
||||
QueryPlanOptimizationSettings::fromContext(context),
|
||||
BuildQueryPipelineSettings::fromContext(context));
|
||||
|
||||
pipeline->addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
return std::make_shared<MaterializingTransform>(header);
|
||||
|
@ -122,7 +122,7 @@ void performRequiredConversions(Block & block, const NamesAndTypesList & require
|
||||
|
||||
if (auto dag = createExpressions(block, conversion_expr_list, true, required_columns, context))
|
||||
{
|
||||
auto expression = std::make_shared<ExpressionActions>(std::move(dag), context);
|
||||
auto expression = std::make_shared<ExpressionActions>(std::move(dag), ExpressionActionsSettings::fromContext(context));
|
||||
expression->execute(block);
|
||||
}
|
||||
}
|
||||
|
@ -50,6 +50,7 @@ SRCS(
|
||||
EmbeddedDictionaries.cpp
|
||||
ExecuteScalarSubqueriesVisitor.cpp
|
||||
ExpressionActions.cpp
|
||||
ExpressionActionsSettings.cpp
|
||||
ExpressionAnalyzer.cpp
|
||||
ExternalDictionariesLoader.cpp
|
||||
ExternalLoader.cpp
|
||||
|
@ -212,7 +212,7 @@ void QueryPipeline::setOutputFormat(ProcessorPtr output)
|
||||
QueryPipeline QueryPipeline::unitePipelines(
|
||||
std::vector<std::unique_ptr<QueryPipeline>> pipelines,
|
||||
const Block & common_header,
|
||||
const ActionsDAG::Settings & settings,
|
||||
const ExpressionActionsSettings & settings,
|
||||
size_t max_threads_limit,
|
||||
Processors * collected_processors)
|
||||
{
|
||||
|
@ -28,6 +28,8 @@ using SubqueriesForSets = std::unordered_map<String, SubqueryForSet>;
|
||||
|
||||
struct SizeLimits;
|
||||
|
||||
struct ExpressionActionsSettings;
|
||||
|
||||
class QueryPipeline
|
||||
{
|
||||
public:
|
||||
@ -89,7 +91,7 @@ public:
|
||||
static QueryPipeline unitePipelines(
|
||||
std::vector<std::unique_ptr<QueryPipeline>> pipelines,
|
||||
const Block & common_header,
|
||||
const Context & context,
|
||||
const ExpressionActionsSettings & settings,
|
||||
size_t max_threads_limit = 0,
|
||||
Processors * collected_processors = nullptr);
|
||||
|
||||
|
@ -28,7 +28,7 @@ AddingDelayedSourceStep::AddingDelayedSourceStep(
|
||||
{
|
||||
}
|
||||
|
||||
void AddingDelayedSourceStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void AddingDelayedSourceStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
source->setQueryPlanStep(this);
|
||||
pipeline.addDelayedStream(source);
|
||||
|
@ -19,7 +19,7 @@ public:
|
||||
|
||||
String getName() const override { return "AddingDelayedSource"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
private:
|
||||
ProcessorPtr source;
|
||||
|
@ -45,7 +45,7 @@ AggregatingStep::AggregatingStep(
|
||||
{
|
||||
}
|
||||
|
||||
void AggregatingStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void AggregatingStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
QueryPipelineProcessorsCollector collector(pipeline, this);
|
||||
|
||||
|
@ -27,7 +27,7 @@ public:
|
||||
|
||||
String getName() const override { return "Aggregating"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
void describeActions(FormatSettings &) const override;
|
||||
void describePipeline(FormatSettings & settings) const override;
|
||||
|
@ -46,7 +46,7 @@ void ArrayJoinStep::updateInputStream(DataStream input_stream, Block result_head
|
||||
res_header = std::move(result_header);
|
||||
}
|
||||
|
||||
void ArrayJoinStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void ArrayJoinStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings)
|
||||
{
|
||||
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type)
|
||||
{
|
||||
@ -60,7 +60,7 @@ void ArrayJoinStep::transformPipeline(QueryPipeline & pipeline)
|
||||
pipeline.getHeader().getColumnsWithTypeAndName(),
|
||||
res_header.getColumnsWithTypeAndName(),
|
||||
ActionsDAG::MatchColumnsMode::Name);
|
||||
auto actions = std::make_shared<ExpressionActions>(actions_dag);
|
||||
auto actions = std::make_shared<ExpressionActions>(actions_dag, settings.getActionsSettings());
|
||||
|
||||
pipeline.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
|
@ -7,15 +7,13 @@ namespace DB
|
||||
class ArrayJoinAction;
|
||||
using ArrayJoinActionPtr = std::shared_ptr<ArrayJoinAction>;
|
||||
|
||||
class Context;
|
||||
|
||||
class ArrayJoinStep : public ITransformingStep
|
||||
{
|
||||
public:
|
||||
explicit ArrayJoinStep(const DataStream & input_stream_, ArrayJoinActionPtr array_join_, const Context & context_);
|
||||
explicit ArrayJoinStep(const DataStream & input_stream_, ArrayJoinActionPtr array_join_);
|
||||
String getName() const override { return "ArrayJoin"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings) override;
|
||||
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
@ -26,7 +24,6 @@ public:
|
||||
private:
|
||||
ArrayJoinActionPtr array_join;
|
||||
Block res_header;
|
||||
const Context & context;
|
||||
};
|
||||
|
||||
}
|
||||
|
21
src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp
Normal file
21
src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp
Normal file
@ -0,0 +1,21 @@
|
||||
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
BuildQueryPipelineSettings BuildQueryPipelineSettings::fromSettings(const Settings & from)
|
||||
{
|
||||
BuildQueryPipelineSettings settings;
|
||||
settings.actions_settings = ExpressionActionsSettings::fromSettings(from);
|
||||
return settings;
|
||||
}
|
||||
|
||||
BuildQueryPipelineSettings BuildQueryPipelineSettings::fromContext(const Context & from)
|
||||
{
|
||||
return fromSettings(from.getSettingsRef());
|
||||
}
|
||||
|
||||
}
|
21
src/Processors/QueryPlan/BuildQueryPipelineSettings.h
Normal file
21
src/Processors/QueryPlan/BuildQueryPipelineSettings.h
Normal file
@ -0,0 +1,21 @@
|
||||
#pragma once
|
||||
#include <cstddef>
|
||||
#include <Interpreters/ExpressionActionsSettings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct Settings;
|
||||
class Context;
|
||||
|
||||
struct BuildQueryPipelineSettings
|
||||
{
|
||||
ExpressionActionsSettings actions_settings;
|
||||
|
||||
const ExpressionActionsSettings & getActionsSettings() const { return actions_settings; }
|
||||
|
||||
static BuildQueryPipelineSettings fromSettings(const Settings & from);
|
||||
static BuildQueryPipelineSettings fromContext(const Context & from);
|
||||
};
|
||||
|
||||
}
|
@ -2,6 +2,7 @@
|
||||
#include <Processors/QueryPipeline.h>
|
||||
#include <Processors/Transforms/CreatingSetsTransform.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -42,7 +43,7 @@ CreatingSetStep::CreatingSetStep(
|
||||
{
|
||||
}
|
||||
|
||||
void CreatingSetStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void CreatingSetStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
pipeline.addCreatingSetsTransform(getOutputStream().header, std::move(subquery_for_set), network_transfer_limits, context);
|
||||
}
|
||||
@ -60,8 +61,7 @@ void CreatingSetStep::describeActions(FormatSettings & settings) const
|
||||
settings.out << description << '\n';
|
||||
}
|
||||
|
||||
CreatingSetsStep::CreatingSetsStep(DataStreams input_streams_, const Context & context_)
|
||||
: context(context_)
|
||||
CreatingSetsStep::CreatingSetsStep(DataStreams input_streams_)
|
||||
{
|
||||
if (input_streams_.empty())
|
||||
throw Exception("CreatingSetsStep cannot be created with no inputs", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -73,7 +73,7 @@ CreatingSetsStep::CreatingSetsStep(DataStreams input_streams_, const Context & c
|
||||
assertBlocksHaveEqualStructure(output_stream->header, input_streams[i].header, "CreatingSets");
|
||||
}
|
||||
|
||||
QueryPipelinePtr CreatingSetsStep::updatePipeline(QueryPipelines pipelines)
|
||||
QueryPipelinePtr CreatingSetsStep::updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & settings)
|
||||
{
|
||||
if (pipelines.empty())
|
||||
throw Exception("CreatingSetsStep cannot be created with no inputs", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -89,7 +89,7 @@ QueryPipelinePtr CreatingSetsStep::updatePipeline(QueryPipelines pipelines)
|
||||
if (pipelines.size() > 1)
|
||||
{
|
||||
QueryPipelineProcessorsCollector collector(delayed_pipeline, this);
|
||||
delayed_pipeline = QueryPipeline::unitePipelines(std::move(pipelines), output_stream->header, context);
|
||||
delayed_pipeline = QueryPipeline::unitePipelines(std::move(pipelines), output_stream->header, settings.getActionsSettings());
|
||||
processors = collector.detachProcessors();
|
||||
}
|
||||
else
|
||||
@ -147,7 +147,7 @@ void addCreatingSetsStep(
|
||||
return;
|
||||
}
|
||||
|
||||
auto creating_sets = std::make_unique<CreatingSetsStep>(std::move(input_streams), context);
|
||||
auto creating_sets = std::make_unique<CreatingSetsStep>(std::move(input_streams));
|
||||
creating_sets->setStepDescription("Create sets before main query execution");
|
||||
query_plan.unitePlans(std::move(creating_sets), std::move(plans));
|
||||
}
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
|
||||
String getName() const override { return "CreatingSet"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
@ -34,17 +34,16 @@ private:
|
||||
class CreatingSetsStep : public IQueryPlanStep
|
||||
{
|
||||
public:
|
||||
explicit CreatingSetsStep(DataStreams input_streams_, const Context & context_);
|
||||
explicit CreatingSetsStep(DataStreams input_streams_);
|
||||
|
||||
String getName() const override { return "CreatingSets"; }
|
||||
|
||||
QueryPipelinePtr updatePipeline(QueryPipelines pipelines) override;
|
||||
QueryPipelinePtr updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & settings) override;
|
||||
|
||||
void describePipeline(FormatSettings & settings) const override;
|
||||
|
||||
private:
|
||||
Processors processors;
|
||||
const Context & context;
|
||||
};
|
||||
|
||||
void addCreatingSetsStep(
|
||||
|
@ -30,7 +30,7 @@ CubeStep::CubeStep(const DataStream & input_stream_, AggregatingTransformParamsP
|
||||
output_stream->distinct_columns.insert(params->params.src_header.getByPosition(key).name);
|
||||
}
|
||||
|
||||
void CubeStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void CubeStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
pipeline.resize(1);
|
||||
|
||||
|
@ -17,7 +17,7 @@ public:
|
||||
|
||||
String getName() const override { return "Cube"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
const Aggregator::Params & getParams() const;
|
||||
private:
|
||||
|
@ -62,7 +62,7 @@ DistinctStep::DistinctStep(
|
||||
}
|
||||
}
|
||||
|
||||
void DistinctStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void DistinctStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
if (checkColumnsAlreadyDistinct(columns, input_streams.front().distinct_columns))
|
||||
return;
|
||||
|
@ -18,7 +18,7 @@ public:
|
||||
|
||||
String getName() const override { return "Distinct"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
|
@ -40,13 +40,12 @@ static ITransformingStep::Traits getJoinTraits()
|
||||
};
|
||||
}
|
||||
|
||||
ExpressionStep::ExpressionStep(const DataStream & input_stream_, ActionsDAGPtr actions_dag_, const Context & context_)
|
||||
ExpressionStep::ExpressionStep(const DataStream & input_stream_, ActionsDAGPtr actions_dag_)
|
||||
: ITransformingStep(
|
||||
input_stream_,
|
||||
Transform::transformHeader(input_stream_.header, std::make_shared<ExpressionActions>(actions_dag_, context_)),
|
||||
Transform::transformHeader(input_stream_.header, std::make_shared<ExpressionActions>(actions_dag_, ExpressionActionsSettings{})),
|
||||
getTraits(actions_dag_))
|
||||
, actions_dag(std::move(actions_dag_))
|
||||
, context(context_)
|
||||
{
|
||||
/// Some columns may be removed by expression.
|
||||
updateDistinctColumns(output_stream->header, output_stream->distinct_columns);
|
||||
@ -55,7 +54,8 @@ ExpressionStep::ExpressionStep(const DataStream & input_stream_, ActionsDAGPtr a
|
||||
void ExpressionStep::updateInputStream(DataStream input_stream, bool keep_header)
|
||||
{
|
||||
Block out_header = keep_header ? std::move(output_stream->header)
|
||||
: Transform::transformHeader(input_stream.header, std::make_shared<ExpressionActions>(actions_dag, context));
|
||||
: Transform::transformHeader(input_stream.header,
|
||||
std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings{}));
|
||||
output_stream = createOutputStream(
|
||||
input_stream,
|
||||
std::move(out_header),
|
||||
@ -65,9 +65,9 @@ void ExpressionStep::updateInputStream(DataStream input_stream, bool keep_header
|
||||
input_streams.emplace_back(std::move(input_stream));
|
||||
}
|
||||
|
||||
void ExpressionStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void ExpressionStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings)
|
||||
{
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag, context);
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag, settings.getActionsSettings());
|
||||
pipeline.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
return std::make_shared<Transform>(header, expression);
|
||||
@ -79,7 +79,7 @@ void ExpressionStep::transformPipeline(QueryPipeline & pipeline)
|
||||
pipeline.getHeader().getColumnsWithTypeAndName(),
|
||||
output_stream->header.getColumnsWithTypeAndName(),
|
||||
ActionsDAG::MatchColumnsMode::Name);
|
||||
auto convert_actions = std::make_shared<ExpressionActions>(convert_actions_dag, context);
|
||||
auto convert_actions = std::make_shared<ExpressionActions>(convert_actions_dag, settings.getActionsSettings());
|
||||
|
||||
pipeline.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
@ -93,7 +93,7 @@ void ExpressionStep::describeActions(FormatSettings & settings) const
|
||||
String prefix(settings.offset, ' ');
|
||||
bool first = true;
|
||||
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag, context);
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings{});
|
||||
for (const auto & action : expression->getActions())
|
||||
{
|
||||
settings.out << prefix << (first ? "Actions: "
|
||||
@ -117,7 +117,7 @@ JoinStep::JoinStep(const DataStream & input_stream_, JoinPtr join_)
|
||||
{
|
||||
}
|
||||
|
||||
void JoinStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void JoinStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
/// In case joined subquery has totals, and we don't, add default chunk to totals.
|
||||
bool add_default_totals = false;
|
||||
|
@ -19,21 +19,19 @@ class ExpressionStep : public ITransformingStep
|
||||
public:
|
||||
using Transform = ExpressionTransform;
|
||||
|
||||
explicit ExpressionStep(const DataStream & input_stream_, ActionsDAGPtr actions_dag_, const Context & context_);
|
||||
explicit ExpressionStep(const DataStream & input_stream_, ActionsDAGPtr actions_dag_);
|
||||
String getName() const override { return "Expression"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings) override;
|
||||
|
||||
void updateInputStream(DataStream input_stream, bool keep_header);
|
||||
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
const ActionsDAGPtr & getExpression() const { return actions_dag; }
|
||||
const Context & getContext() const { return context; }
|
||||
|
||||
private:
|
||||
ActionsDAGPtr actions_dag;
|
||||
const Context & context;
|
||||
};
|
||||
|
||||
/// TODO: add separate step for join.
|
||||
@ -45,7 +43,7 @@ public:
|
||||
explicit JoinStep(const DataStream & input_stream_, JoinPtr join_);
|
||||
String getName() const override { return "Join"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
private:
|
||||
JoinPtr join;
|
||||
|
@ -25,7 +25,7 @@ ExtremesStep::ExtremesStep(const DataStream & input_stream_)
|
||||
{
|
||||
}
|
||||
|
||||
void ExtremesStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void ExtremesStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
pipeline.addExtremesTransform();
|
||||
}
|
||||
|
@ -11,7 +11,7 @@ public:
|
||||
|
||||
String getName() const override { return "Extremes"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -35,7 +35,7 @@ FillingStep::FillingStep(const DataStream & input_stream_, SortDescription sort_
|
||||
throw Exception("FillingStep expects single input", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
void FillingStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void FillingStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
pipeline.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
|
@ -13,7 +13,7 @@ public:
|
||||
|
||||
String getName() const override { return "Filling"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
|
@ -31,7 +31,11 @@ FilterStep::FilterStep(
|
||||
bool remove_filter_column_)
|
||||
: ITransformingStep(
|
||||
input_stream_,
|
||||
FilterTransform::transformHeader(input_stream_.header, std::make_shared<ExpressionActions>(actions_dag_, {}), filter_column_name_, remove_filter_column_),
|
||||
FilterTransform::transformHeader(
|
||||
input_stream_.header,
|
||||
std::make_shared<ExpressionActions>(actions_dag_, ExpressionActionsSettings{}),
|
||||
filter_column_name_,
|
||||
remove_filter_column_),
|
||||
getTraits(actions_dag_))
|
||||
, actions_dag(std::move(actions_dag_))
|
||||
, filter_column_name(std::move(filter_column_name_))
|
||||
@ -45,7 +49,11 @@ void FilterStep::updateInputStream(DataStream input_stream, bool keep_header)
|
||||
{
|
||||
Block out_header = std::move(output_stream->header);
|
||||
if (keep_header)
|
||||
out_header = FilterTransform::transformHeader(input_stream.header, std::make_shared<ExpressionActions>(actions_dag, {}), filter_column_name, remove_filter_column);
|
||||
out_header = FilterTransform::transformHeader(
|
||||
input_stream.header,
|
||||
std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings{}),
|
||||
filter_column_name,
|
||||
remove_filter_column);
|
||||
|
||||
output_stream = createOutputStream(
|
||||
input_stream,
|
||||
@ -56,9 +64,9 @@ void FilterStep::updateInputStream(DataStream input_stream, bool keep_header)
|
||||
input_streams.emplace_back(std::move(input_stream));
|
||||
}
|
||||
|
||||
void FilterStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void FilterStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings)
|
||||
{
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag);
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag, settings.getActionsSettings());
|
||||
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type)
|
||||
{
|
||||
bool on_totals = stream_type == QueryPipeline::StreamType::Totals;
|
||||
@ -71,7 +79,7 @@ void FilterStep::transformPipeline(QueryPipeline & pipeline)
|
||||
pipeline.getHeader().getColumnsWithTypeAndName(),
|
||||
output_stream->header.getColumnsWithTypeAndName(),
|
||||
ActionsDAG::MatchColumnsMode::Name);
|
||||
auto convert_actions = std::make_shared<ExpressionActions>(convert_actions_dag);
|
||||
auto convert_actions = std::make_shared<ExpressionActions>(convert_actions_dag, settings.getActionsSettings());
|
||||
|
||||
pipeline.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
@ -90,7 +98,7 @@ void FilterStep::describeActions(FormatSettings & settings) const
|
||||
settings.out << '\n';
|
||||
|
||||
bool first = true;
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag, {});
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings{});
|
||||
for (const auto & action : expression->getActions())
|
||||
{
|
||||
settings.out << prefix << (first ? "Actions: "
|
||||
|
@ -15,11 +15,10 @@ public:
|
||||
const DataStream & input_stream_,
|
||||
ActionsDAGPtr actions_dag_,
|
||||
String filter_column_name_,
|
||||
bool remove_filter_column_,
|
||||
const Context & context_);
|
||||
bool remove_filter_column_);
|
||||
|
||||
String getName() const override { return "Filter"; }
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings) override;
|
||||
|
||||
void updateInputStream(DataStream input_stream, bool keep_header);
|
||||
|
||||
@ -28,13 +27,11 @@ public:
|
||||
const ActionsDAGPtr & getExpression() const { return actions_dag; }
|
||||
const String & getFilterColumnName() const { return filter_column_name; }
|
||||
bool removesFilterColumn() const { return remove_filter_column; }
|
||||
const Context & getContext() const { return context; }
|
||||
|
||||
private:
|
||||
ActionsDAGPtr actions_dag;
|
||||
String filter_column_name;
|
||||
bool remove_filter_column;
|
||||
const Context & context;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -51,7 +51,7 @@ void FinishSortingStep::updateLimit(size_t limit_)
|
||||
}
|
||||
}
|
||||
|
||||
void FinishSortingStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void FinishSortingStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
bool need_finish_sorting = (prefix_description.size() < result_description.size());
|
||||
if (pipeline.getNumStreams() > 1)
|
||||
|
@ -18,7 +18,7 @@ public:
|
||||
|
||||
String getName() const override { return "FinishSorting"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
#include <Core/Block.h>
|
||||
#include <Core/SortDescription.h>
|
||||
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -75,7 +76,7 @@ public:
|
||||
/// * header from each pipeline is the same as header from corresponding input_streams
|
||||
/// Result pipeline must contain any number of streams with compatible output header is hasOutputStream(),
|
||||
/// or pipeline should be completed otherwise.
|
||||
virtual QueryPipelinePtr updatePipeline(QueryPipelines pipelines) = 0;
|
||||
virtual QueryPipelinePtr updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & settings) = 0;
|
||||
|
||||
const DataStreams & getInputStreams() const { return input_streams; }
|
||||
|
||||
|
@ -9,11 +9,11 @@ ISourceStep::ISourceStep(DataStream output_stream_)
|
||||
output_stream = std::move(output_stream_);
|
||||
}
|
||||
|
||||
QueryPipelinePtr ISourceStep::updatePipeline(QueryPipelines)
|
||||
QueryPipelinePtr ISourceStep::updatePipeline(QueryPipelines, const BuildQueryPipelineSettings & settings)
|
||||
{
|
||||
auto pipeline = std::make_unique<QueryPipeline>();
|
||||
QueryPipelineProcessorsCollector collector(*pipeline, this);
|
||||
initializePipeline(*pipeline);
|
||||
initializePipeline(*pipeline, settings);
|
||||
auto added_processors = collector.detachProcessors();
|
||||
processors.insert(processors.end(), added_processors.begin(), added_processors.end());
|
||||
return pipeline;
|
||||
|
@ -10,9 +10,9 @@ class ISourceStep : public IQueryPlanStep
|
||||
public:
|
||||
explicit ISourceStep(DataStream output_stream_);
|
||||
|
||||
QueryPipelinePtr updatePipeline(QueryPipelines pipelines) override;
|
||||
QueryPipelinePtr updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & settings) override;
|
||||
|
||||
virtual void initializePipeline(QueryPipeline & pipeline) = 0;
|
||||
virtual void initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings) = 0;
|
||||
|
||||
void describePipeline(FormatSettings & settings) const override;
|
||||
|
||||
|
@ -36,16 +36,16 @@ DataStream ITransformingStep::createOutputStream(
|
||||
}
|
||||
|
||||
|
||||
QueryPipelinePtr ITransformingStep::updatePipeline(QueryPipelines pipelines)
|
||||
QueryPipelinePtr ITransformingStep::updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & settings)
|
||||
{
|
||||
if (collect_processors)
|
||||
{
|
||||
QueryPipelineProcessorsCollector collector(*pipelines.front(), this);
|
||||
transformPipeline(*pipelines.front());
|
||||
transformPipeline(*pipelines.front(), settings);
|
||||
processors = collector.detachProcessors();
|
||||
}
|
||||
else
|
||||
transformPipeline(*pipelines.front());
|
||||
transformPipeline(*pipelines.front(), settings);
|
||||
|
||||
return std::move(pipelines.front());
|
||||
}
|
||||
|
@ -48,9 +48,9 @@ public:
|
||||
|
||||
ITransformingStep(DataStream input_stream, Block output_header, Traits traits, bool collect_processors_ = true);
|
||||
|
||||
QueryPipelinePtr updatePipeline(QueryPipelines pipelines) override;
|
||||
QueryPipelinePtr updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & settings) override;
|
||||
|
||||
virtual void transformPipeline(QueryPipeline & pipeline) = 0;
|
||||
virtual void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings) = 0;
|
||||
|
||||
const TransformTraits & getTransformTraits() const { return transform_traits; }
|
||||
const DataStreamTraits & getDataStreamTraits() const { return data_stream_traits; }
|
||||
|
@ -33,7 +33,7 @@ LimitByStep::LimitByStep(
|
||||
}
|
||||
|
||||
|
||||
void LimitByStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void LimitByStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
pipeline.resize(1);
|
||||
|
||||
|
@ -14,7 +14,7 @@ public:
|
||||
|
||||
String getName() const override { return "LimitBy"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
|
@ -42,7 +42,7 @@ void LimitStep::updateInputStream(DataStream input_stream)
|
||||
output_stream = createOutputStream(input_streams.front(), output_stream->header, getDataStreamTraits());
|
||||
}
|
||||
|
||||
void LimitStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void LimitStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
auto transform = std::make_shared<LimitTransform>(
|
||||
pipeline.getHeader(), limit, offset, pipeline.getNumStreams(), always_read_till_end, with_ties, description);
|
||||
|
@ -18,7 +18,7 @@ public:
|
||||
|
||||
String getName() const override { return "Limit"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
|
@ -56,7 +56,7 @@ void MergeSortingStep::updateLimit(size_t limit_)
|
||||
}
|
||||
}
|
||||
|
||||
void MergeSortingStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void MergeSortingStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
|
||||
{
|
||||
|
@ -24,7 +24,7 @@ public:
|
||||
|
||||
String getName() const override { return "MergeSorting"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
|
@ -40,7 +40,7 @@ MergingAggregatedStep::MergingAggregatedStep(
|
||||
output_stream->distinct_columns.insert(params->params.intermediate_header.getByPosition(key).name);
|
||||
}
|
||||
|
||||
void MergingAggregatedStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void MergingAggregatedStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
if (!memory_efficient_aggregation)
|
||||
{
|
||||
|
@ -21,7 +21,7 @@ public:
|
||||
|
||||
String getName() const override { return "MergingAggregated"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
|
@ -53,7 +53,7 @@ MergingFinal::MergingFinal(
|
||||
// output_stream->sort_mode = DataStream::SortMode::Stream;
|
||||
}
|
||||
|
||||
void MergingFinal::transformPipeline(QueryPipeline & pipeline)
|
||||
void MergingFinal::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
const auto & header = pipeline.getHeader();
|
||||
size_t num_outputs = pipeline.getNumStreams();
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
|
||||
String getName() const override { return "MergingFinal"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
|
@ -46,7 +46,7 @@ void MergingSortedStep::updateLimit(size_t limit_)
|
||||
}
|
||||
}
|
||||
|
||||
void MergingSortedStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void MergingSortedStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
/// If there are several streams, then we merge them into one
|
||||
if (pipeline.getNumStreams() > 1)
|
||||
|
@ -19,7 +19,7 @@ public:
|
||||
|
||||
String getName() const override { return "MergingSorted"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
|
@ -28,7 +28,7 @@ OffsetStep::OffsetStep(const DataStream & input_stream_, size_t offset_)
|
||||
{
|
||||
}
|
||||
|
||||
void OffsetStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void OffsetStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
auto transform = std::make_shared<OffsetTransform>(
|
||||
pipeline.getHeader(), offset, pipeline.getNumStreams());
|
||||
|
@ -13,7 +13,7 @@ public:
|
||||
|
||||
String getName() const override { return "Offset"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
|
@ -1,12 +1,20 @@
|
||||
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
QueryPlanOptimizationSettings::QueryPlanOptimizationSettings(const Settings & settings)
|
||||
QueryPlanOptimizationSettings QueryPlanOptimizationSettings::fromSettings(const Settings & from)
|
||||
{
|
||||
max_optimizations_to_apply = settings.query_plan_max_optimizations_to_apply;
|
||||
QueryPlanOptimizationSettings settings;
|
||||
settings.max_optimizations_to_apply = from.query_plan_max_optimizations_to_apply;
|
||||
return settings;
|
||||
}
|
||||
|
||||
QueryPlanOptimizationSettings QueryPlanOptimizationSettings::fromContext(const Context & from)
|
||||
{
|
||||
return fromSettings(from.getSettingsRef());
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -6,15 +6,16 @@ namespace DB
|
||||
{
|
||||
|
||||
struct Settings;
|
||||
class Context;
|
||||
|
||||
struct QueryPlanOptimizationSettings
|
||||
{
|
||||
QueryPlanOptimizationSettings() = delete;
|
||||
explicit QueryPlanOptimizationSettings(const Settings & settings);
|
||||
|
||||
/// If not zero, throw if too many optimizations were applied to query plan.
|
||||
/// It helps to avoid infinite optimization loop.
|
||||
size_t max_optimizations_to_apply = 0;
|
||||
|
||||
static QueryPlanOptimizationSettings fromSettings(const Settings & from);
|
||||
static QueryPlanOptimizationSettings fromContext(const Context & from);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -37,7 +37,6 @@ static size_t tryAddNewFilterStep(
|
||||
auto & child = child_node->step;
|
||||
|
||||
auto * filter = static_cast<FilterStep *>(parent.get());
|
||||
const auto & context = filter->getContext();
|
||||
const auto & expression = filter->getExpression();
|
||||
const auto & filter_column_name = filter->getFilterColumnName();
|
||||
bool removes_filter = filter->removesFilterColumn();
|
||||
@ -70,7 +69,7 @@ static size_t tryAddNewFilterStep(
|
||||
if (!found_filter_column || filter_is_constant)
|
||||
/// 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, context);
|
||||
parent = std::make_unique<ExpressionStep>(child->getOutputStream(), expression);
|
||||
|
||||
/// Add new Filter step before Aggregating.
|
||||
/// Expression/Filter -> Aggregating -> Something
|
||||
@ -83,7 +82,7 @@ static size_t tryAddNewFilterStep(
|
||||
auto split_filter_column_name = (*split_filter->getIndex().rbegin())->result_name;
|
||||
node.step = std::make_unique<FilterStep>(
|
||||
node.children.at(0)->step->getOutputStream(),
|
||||
std::move(split_filter), std::move(split_filter_column_name), true, context);
|
||||
std::move(split_filter), std::move(split_filter_column_name), true);
|
||||
|
||||
return 3;
|
||||
}
|
||||
|
@ -27,8 +27,6 @@ size_t tryLiftUpArrayJoin(QueryPlan::Node * parent_node, QueryPlan::Nodes & node
|
||||
const auto & array_join = array_join_step->arrayJoin();
|
||||
const auto & expression = expression_step ? expression_step->getExpression()
|
||||
: filter_step->getExpression();
|
||||
const auto & context = expression_step ? expression_step->getContext()
|
||||
: filter_step->getContext();
|
||||
|
||||
auto split_actions = expression->splitActionsBeforeArrayJoin(array_join->columns);
|
||||
|
||||
@ -49,14 +47,12 @@ size_t tryLiftUpArrayJoin(QueryPlan::Node * parent_node, QueryPlan::Nodes & node
|
||||
|
||||
if (expression_step)
|
||||
child = std::make_unique<ExpressionStep>(child_node->children.at(0)->step->getOutputStream(),
|
||||
std::move(split_actions.first),
|
||||
context);
|
||||
std::move(split_actions.first));
|
||||
else
|
||||
child = std::make_unique<FilterStep>(child_node->children.at(0)->step->getOutputStream(),
|
||||
std::move(split_actions.first),
|
||||
filter_step->getFilterColumnName(),
|
||||
filter_step->removesFilterColumn(),
|
||||
context);
|
||||
filter_step->removesFilterColumn());
|
||||
|
||||
child->setStepDescription(std::move(description));
|
||||
|
||||
@ -72,17 +68,15 @@ size_t tryLiftUpArrayJoin(QueryPlan::Node * parent_node, QueryPlan::Nodes & node
|
||||
/// Expression/Filter -> ArrayJoin -> node -> Something
|
||||
|
||||
node.step = std::make_unique<ExpressionStep>(node.children.at(0)->step->getOutputStream(),
|
||||
std::move(split_actions.first),
|
||||
context);
|
||||
std::move(split_actions.first));
|
||||
node.step->setStepDescription(description);
|
||||
array_join_step->updateInputStream(node.step->getOutputStream(), {});
|
||||
|
||||
if (expression_step)
|
||||
parent = std::make_unique<ExpressionStep>(array_join_step->getOutputStream(), split_actions.second, context);
|
||||
parent = std::make_unique<ExpressionStep>(array_join_step->getOutputStream(), split_actions.second);
|
||||
else
|
||||
parent = std::make_unique<FilterStep>(array_join_step->getOutputStream(), split_actions.second,
|
||||
filter_step->getFilterColumnName(), filter_step->removesFilterColumn(),
|
||||
context);
|
||||
filter_step->getFilterColumnName(), filter_step->removesFilterColumn());
|
||||
|
||||
parent->setStepDescription(description + " [split]");
|
||||
return 3;
|
||||
|
@ -22,7 +22,6 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &)
|
||||
|
||||
if (parent_expr && child_expr)
|
||||
{
|
||||
const auto & context = child_expr->getContext();
|
||||
const auto & child_actions = child_expr->getExpression();
|
||||
const auto & parent_actions = parent_expr->getExpression();
|
||||
|
||||
@ -34,7 +33,7 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &)
|
||||
|
||||
auto merged = ActionsDAG::merge(std::move(*child_actions), std::move(*parent_actions));
|
||||
|
||||
auto expr = std::make_unique<ExpressionStep>(child_expr->getInputStreams().front(), merged, context);
|
||||
auto expr = std::make_unique<ExpressionStep>(child_expr->getInputStreams().front(), merged);
|
||||
expr->setStepDescription("(" + parent_expr->getStepDescription() + " + " + child_expr->getStepDescription() + ")");
|
||||
|
||||
parent_node->step = std::move(expr);
|
||||
@ -43,7 +42,6 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &)
|
||||
}
|
||||
else if (parent_filter && child_expr)
|
||||
{
|
||||
const auto & context = child_expr->getContext();
|
||||
const auto & child_actions = child_expr->getExpression();
|
||||
const auto & parent_actions = parent_filter->getExpression();
|
||||
|
||||
@ -55,8 +53,7 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &)
|
||||
auto filter = std::make_unique<FilterStep>(child_expr->getInputStreams().front(),
|
||||
merged,
|
||||
parent_filter->getFilterColumnName(),
|
||||
parent_filter->removesFilterColumn(),
|
||||
context);
|
||||
parent_filter->removesFilterColumn());
|
||||
filter->setStepDescription("(" + parent_filter->getStepDescription() + " + " + child_expr->getStepDescription() + ")");
|
||||
|
||||
parent_node->step = std::move(filter);
|
||||
|
@ -14,7 +14,6 @@ size_t trySplitFilter(QueryPlan::Node * node, QueryPlan::Nodes & nodes)
|
||||
return 0;
|
||||
|
||||
const auto & expr = filter_step->getExpression();
|
||||
const auto & context = filter_step->getContext();
|
||||
|
||||
/// Do not split if there are function like runningDifference.
|
||||
if (expr->hasStatefulFunctions())
|
||||
@ -39,10 +38,9 @@ size_t trySplitFilter(QueryPlan::Node * node, QueryPlan::Nodes & nodes)
|
||||
filter_node.children.at(0)->step->getOutputStream(),
|
||||
std::move(split.first),
|
||||
filter_step->getFilterColumnName(),
|
||||
remove_filter,
|
||||
context);
|
||||
remove_filter);
|
||||
|
||||
node->step = std::make_unique<ExpressionStep>(filter_node.step->getOutputStream(), std::move(split.second), context);
|
||||
node->step = std::make_unique<ExpressionStep>(filter_node.step->getOutputStream(), std::move(split.second));
|
||||
|
||||
filter_node.step->setStepDescription("(" + description + ")[split]");
|
||||
node->step->setStepDescription(description);
|
||||
|
@ -46,7 +46,7 @@ void PartialSortingStep::updateLimit(size_t limit_)
|
||||
}
|
||||
}
|
||||
|
||||
void PartialSortingStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void PartialSortingStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
|
||||
{
|
||||
|
@ -18,7 +18,7 @@ public:
|
||||
|
||||
String getName() const override { return "PartialSorting"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
|
@ -7,6 +7,8 @@
|
||||
#include <Interpreters/ArrayJoinAction.h>
|
||||
#include <stack>
|
||||
#include <Processors/QueryPlan/Optimizations/Optimizations.h>
|
||||
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
||||
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -130,7 +132,9 @@ void QueryPlan::addStep(QueryPlanStepPtr step)
|
||||
" input expected", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
QueryPipelinePtr QueryPlan::buildQueryPipeline(const QueryPlanOptimizationSettings & optimization_settings)
|
||||
QueryPipelinePtr QueryPlan::buildQueryPipeline(
|
||||
const QueryPlanOptimizationSettings & optimization_settings,
|
||||
const BuildQueryPipelineSettings & build_pipeline_settings)
|
||||
{
|
||||
checkInitialized();
|
||||
optimize(optimization_settings);
|
||||
@ -160,7 +164,7 @@ QueryPipelinePtr QueryPlan::buildQueryPipeline(const QueryPlanOptimizationSettin
|
||||
if (next_child == frame.node->children.size())
|
||||
{
|
||||
bool limit_max_threads = frame.pipelines.empty();
|
||||
last_pipeline = frame.node->step->updatePipeline(std::move(frame.pipelines));
|
||||
last_pipeline = frame.node->step->updatePipeline(std::move(frame.pipelines), build_pipeline_settings);
|
||||
|
||||
if (limit_max_threads && max_threads)
|
||||
last_pipeline->limitMaxThreads(max_threads);
|
||||
@ -177,7 +181,9 @@ QueryPipelinePtr QueryPlan::buildQueryPipeline(const QueryPlanOptimizationSettin
|
||||
return last_pipeline;
|
||||
}
|
||||
|
||||
Pipe QueryPlan::convertToPipe(const QueryPlanOptimizationSettings & optimization_settings)
|
||||
Pipe QueryPlan::convertToPipe(
|
||||
const QueryPlanOptimizationSettings & optimization_settings,
|
||||
const BuildQueryPipelineSettings & build_pipeline_settings)
|
||||
{
|
||||
if (!isInitialized())
|
||||
return {};
|
||||
@ -185,7 +191,7 @@ Pipe QueryPlan::convertToPipe(const QueryPlanOptimizationSettings & optimization
|
||||
if (isCompleted())
|
||||
throw Exception("Cannot convert completed QueryPlan to Pipe", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return QueryPipeline::getPipe(std::move(*buildQueryPipeline(optimization_settings)));
|
||||
return QueryPipeline::getPipe(std::move(*buildQueryPipeline(optimization_settings, build_pipeline_settings)));
|
||||
}
|
||||
|
||||
void QueryPlan::addInterpreterContext(std::shared_ptr<Context> context)
|
||||
|
@ -5,7 +5,6 @@
|
||||
#include <set>
|
||||
|
||||
#include <Core/Names.h>
|
||||
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -26,6 +25,9 @@ using QueryPlanPtr = std::unique_ptr<QueryPlan>;
|
||||
|
||||
class Pipe;
|
||||
|
||||
struct QueryPlanOptimizationSettings;
|
||||
struct BuildQueryPipelineSettings;
|
||||
|
||||
/// A tree of query steps.
|
||||
/// The goal of QueryPlan is to build QueryPipeline.
|
||||
/// QueryPlan let delay pipeline creation which is helpful for pipeline-level optimizations.
|
||||
@ -46,10 +48,14 @@ public:
|
||||
|
||||
void optimize(const QueryPlanOptimizationSettings & optimization_settings);
|
||||
|
||||
QueryPipelinePtr buildQueryPipeline(const QueryPlanOptimizationSettings & optimization_settings);
|
||||
QueryPipelinePtr buildQueryPipeline(
|
||||
const QueryPlanOptimizationSettings & optimization_settings,
|
||||
const BuildQueryPipelineSettings & build_pipeline_settings);
|
||||
|
||||
/// If initialized, build pipeline and convert to pipe. Otherwise, return empty pipe.
|
||||
Pipe convertToPipe(const QueryPlanOptimizationSettings & optimization_settings);
|
||||
Pipe convertToPipe(
|
||||
const QueryPlanOptimizationSettings & optimization_settings,
|
||||
const BuildQueryPipelineSettings & build_pipeline_settings);
|
||||
|
||||
struct ExplainPlanOptions
|
||||
{
|
||||
|
@ -11,7 +11,7 @@ ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_, std::shared_ptr<Conte
|
||||
{
|
||||
}
|
||||
|
||||
void ReadFromPreparedSource::initializePipeline(QueryPipeline & pipeline)
|
||||
void ReadFromPreparedSource::initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
for (const auto & processor : pipe.getProcessors())
|
||||
processors.emplace_back(processor);
|
||||
|
@ -13,7 +13,7 @@ public:
|
||||
|
||||
String getName() const override { return "ReadFromPreparedSource"; }
|
||||
|
||||
void initializePipeline(QueryPipeline & pipeline) override;
|
||||
void initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
private:
|
||||
Pipe pipe;
|
||||
|
@ -10,7 +10,7 @@ ReadNothingStep::ReadNothingStep(Block output_header)
|
||||
{
|
||||
}
|
||||
|
||||
void ReadNothingStep::initializePipeline(QueryPipeline & pipeline)
|
||||
void ReadNothingStep::initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
pipeline.init(Pipe(std::make_shared<NullSource>(getOutputStream().header)));
|
||||
}
|
||||
|
@ -12,7 +12,7 @@ public:
|
||||
|
||||
String getName() const override { return "ReadNothing"; }
|
||||
|
||||
void initializePipeline(QueryPipeline & pipeline) override;
|
||||
void initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -26,7 +26,7 @@ ReverseRowsStep::ReverseRowsStep(const DataStream & input_stream_)
|
||||
{
|
||||
}
|
||||
|
||||
void ReverseRowsStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void ReverseRowsStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
pipeline.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
|
@ -8,11 +8,11 @@ namespace DB
|
||||
class ReverseRowsStep : public ITransformingStep
|
||||
{
|
||||
public:
|
||||
ReverseRowsStep(const DataStream & input_stream_);
|
||||
explicit ReverseRowsStep(const DataStream & input_stream_);
|
||||
|
||||
String getName() const override { return "ReverseRows"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -30,7 +30,7 @@ RollupStep::RollupStep(const DataStream & input_stream_, AggregatingTransformPar
|
||||
output_stream->distinct_columns.insert(params->params.src_header.getByPosition(key).name);
|
||||
}
|
||||
|
||||
void RollupStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void RollupStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
pipeline.resize(1);
|
||||
|
||||
|
@ -16,7 +16,7 @@ public:
|
||||
|
||||
String getName() const override { return "Rollup"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
private:
|
||||
AggregatingTransformParamsPtr params;
|
||||
|
@ -39,7 +39,7 @@ SettingQuotaAndLimitsStep::SettingQuotaAndLimitsStep(
|
||||
{
|
||||
}
|
||||
|
||||
void SettingQuotaAndLimitsStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void SettingQuotaAndLimitsStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
/// Table lock is stored inside pipeline here.
|
||||
pipeline.setLimits(limits);
|
||||
|
@ -30,7 +30,7 @@ public:
|
||||
|
||||
String getName() const override { return "SettingQuotaAndLimits"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
private:
|
||||
std::shared_ptr<Context> context;
|
||||
|
@ -28,7 +28,6 @@ TotalsHavingStep::TotalsHavingStep(
|
||||
const DataStream & input_stream_,
|
||||
bool overflow_row_,
|
||||
const ActionsDAGPtr & actions_dag_,
|
||||
const Context & context_,
|
||||
const std::string & filter_column_,
|
||||
TotalsMode totals_mode_,
|
||||
double auto_include_threshold_,
|
||||
@ -37,7 +36,7 @@ TotalsHavingStep::TotalsHavingStep(
|
||||
input_stream_,
|
||||
TotalsHavingTransform::transformHeader(
|
||||
input_stream_.header,
|
||||
(actions_dag_ ? std::make_shared<ExpressionActions>(actions_dag_, context_) : nullptr),
|
||||
(actions_dag_ ? std::make_shared<ExpressionActions>(actions_dag_, ExpressionActionsSettings{}) : nullptr),
|
||||
final_),
|
||||
getTraits(!filter_column_.empty()))
|
||||
, overflow_row(overflow_row_)
|
||||
@ -46,14 +45,14 @@ TotalsHavingStep::TotalsHavingStep(
|
||||
, totals_mode(totals_mode_)
|
||||
, auto_include_threshold(auto_include_threshold_)
|
||||
, final(final_)
|
||||
, context(context_)
|
||||
{
|
||||
}
|
||||
|
||||
void TotalsHavingStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void TotalsHavingStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings)
|
||||
{
|
||||
auto totals_having = std::make_shared<TotalsHavingTransform>(
|
||||
pipeline.getHeader(), overflow_row, (actions_dag ? std::make_shared<ExpressionActions>(actions_dag, context) : nullptr),
|
||||
pipeline.getHeader(), overflow_row,
|
||||
(actions_dag ? std::make_shared<ExpressionActions>(actions_dag, settings.getActionsSettings()) : nullptr),
|
||||
filter_column_name, totals_mode, auto_include_threshold, final);
|
||||
|
||||
pipeline.addTotalsHavingTransform(std::move(totals_having));
|
||||
@ -85,7 +84,7 @@ void TotalsHavingStep::describeActions(FormatSettings & settings) const
|
||||
if (actions_dag)
|
||||
{
|
||||
bool first = true;
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag, context);
|
||||
auto expression = std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings{});
|
||||
for (const auto & action : expression->getActions())
|
||||
{
|
||||
settings.out << prefix << (first ? "Actions: "
|
||||
|
@ -17,7 +17,6 @@ public:
|
||||
const DataStream & input_stream_,
|
||||
bool overflow_row_,
|
||||
const ActionsDAGPtr & actions_dag_,
|
||||
const Context & context_,
|
||||
const std::string & filter_column_,
|
||||
TotalsMode totals_mode_,
|
||||
double auto_include_threshold_,
|
||||
@ -25,7 +24,7 @@ public:
|
||||
|
||||
String getName() const override { return "TotalsHaving"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings) override;
|
||||
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
@ -38,7 +37,6 @@ private:
|
||||
TotalsMode totals_mode;
|
||||
double auto_include_threshold;
|
||||
bool final;
|
||||
const Context & context;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,7 +1,7 @@
|
||||
#include <Processors/QueryPlan/UnionStep.h>
|
||||
#include <Processors/QueryPipeline.h>
|
||||
#include <Processors/Sources/NullSource.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -18,7 +18,7 @@ UnionStep::UnionStep(DataStreams input_streams_, Block result_header, size_t max
|
||||
output_stream = DataStream{.header = header};
|
||||
}
|
||||
|
||||
QueryPipelinePtr UnionStep::updatePipeline(QueryPipelines pipelines)
|
||||
QueryPipelinePtr UnionStep::updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & settings)
|
||||
{
|
||||
auto pipeline = std::make_unique<QueryPipeline>();
|
||||
QueryPipelineProcessorsCollector collector(*pipeline, this);
|
||||
@ -30,7 +30,7 @@ QueryPipelinePtr UnionStep::updatePipeline(QueryPipelines pipelines)
|
||||
return pipeline;
|
||||
}
|
||||
|
||||
*pipeline = QueryPipeline::unitePipelines(std::move(pipelines), output_stream->header, context, max_threads);
|
||||
*pipeline = QueryPipeline::unitePipelines(std::move(pipelines), output_stream->header, settings.getActionsSettings(), max_threads);
|
||||
|
||||
processors = collector.detachProcessors();
|
||||
return pipeline;
|
||||
|
@ -9,11 +9,11 @@ class UnionStep : public IQueryPlanStep
|
||||
{
|
||||
public:
|
||||
/// max_threads is used to limit the number of threads for result pipeline.
|
||||
UnionStep(DataStreams input_streams_, Block result_header, const Context & context_, size_t max_threads_ = 0);
|
||||
UnionStep(DataStreams input_streams_, Block result_header, size_t max_threads_ = 0);
|
||||
|
||||
String getName() const override { return "Union"; }
|
||||
|
||||
QueryPipelinePtr updatePipeline(QueryPipelines pipelines) override;
|
||||
QueryPipelinePtr updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & settings) override;
|
||||
|
||||
void describePipeline(FormatSettings & settings) const override;
|
||||
|
||||
@ -21,7 +21,6 @@ private:
|
||||
Block header;
|
||||
size_t max_threads;
|
||||
Processors processors;
|
||||
const Context & context;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -62,7 +62,7 @@ WindowStep::WindowStep(const DataStream & input_stream_,
|
||||
|
||||
}
|
||||
|
||||
void WindowStep::transformPipeline(QueryPipeline & pipeline)
|
||||
void WindowStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
pipeline.addSimpleTransform([&](const Block & /*header*/)
|
||||
{
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
|
||||
String getName() const override { return "Window"; }
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
|
@ -95,6 +95,7 @@ SRCS(
|
||||
QueryPlan/AddingDelayedSourceStep.cpp
|
||||
QueryPlan/AggregatingStep.cpp
|
||||
QueryPlan/ArrayJoinStep.cpp
|
||||
QueryPlan/BuildQueryPipelineSettings.cpp
|
||||
QueryPlan/CreatingSetsStep.cpp
|
||||
QueryPlan/CubeStep.cpp
|
||||
QueryPlan/DistinctStep.cpp
|
||||
|
@ -190,7 +190,9 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns
|
||||
additional_columns, columns, metadata_snapshot->getColumns(), storage.global_context);
|
||||
if (dag)
|
||||
{
|
||||
auto actions = std::make_shared<ExpressionActions>(std::move(dag), storage.global_context);
|
||||
auto actions = std::make_shared<
|
||||
ExpressionActions>(std::move(dag),
|
||||
ExpressionActionsSettings::fromSettings(storage.global_context.getSettingsRef()));
|
||||
actions->execute(additional_columns);
|
||||
}
|
||||
|
||||
|
@ -442,7 +442,9 @@ void MergeTreeData::checkPartitionKeyAndInitMinMax(const KeyDescription & new_pa
|
||||
|
||||
/// Add all columns used in the partition key to the min-max index.
|
||||
const NamesAndTypesList & minmax_idx_columns_with_types = new_partition_key.expression->getRequiredColumnsWithTypes();
|
||||
minmax_idx_expr = std::make_shared<ExpressionActions>(std::make_shared<ActionsDAG>(minmax_idx_columns_with_types), global_context);
|
||||
minmax_idx_expr = std::make_shared<ExpressionActions>(
|
||||
std::make_shared<ActionsDAG>(minmax_idx_columns_with_types),
|
||||
ExpressionActionsSettings::fromSettings(global_context.getSettingsRef()));
|
||||
for (const NameAndTypePair & column : minmax_idx_columns_with_types)
|
||||
{
|
||||
minmax_idx_columns.emplace_back(column.name);
|
||||
|
@ -860,8 +860,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts(
|
||||
plan->getCurrentDataStream(),
|
||||
filter_expression,
|
||||
filter_function->getColumnName(),
|
||||
false,
|
||||
context);
|
||||
false);
|
||||
|
||||
sampling_step->setStepDescription("Sampling");
|
||||
plan->addStep(std::move(sampling_step));
|
||||
@ -869,7 +868,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts(
|
||||
|
||||
if (result_projection)
|
||||
{
|
||||
auto projection_step = std::make_unique<ExpressionStep>(plan->getCurrentDataStream(), result_projection, context);
|
||||
auto projection_step = std::make_unique<ExpressionStep>(plan->getCurrentDataStream(), result_projection);
|
||||
projection_step->setStepDescription("Remove unused columns after reading from storage");
|
||||
plan->addStep(std::move(projection_step));
|
||||
}
|
||||
@ -884,7 +883,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts(
|
||||
|
||||
auto adding_column_action = ActionsDAG::makeAddingColumnActions(std::move(column));
|
||||
|
||||
auto adding_column = std::make_unique<ExpressionStep>(plan->getCurrentDataStream(), std::move(adding_column_action), context);
|
||||
auto adding_column = std::make_unique<ExpressionStep>(plan->getCurrentDataStream(), std::move(adding_column_action));
|
||||
adding_column->setStepDescription("Add _sample_factor column");
|
||||
plan->addStep(std::move(adding_column));
|
||||
}
|
||||
@ -1329,7 +1328,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder(
|
||||
|
||||
auto expression_step = std::make_unique<ExpressionStep>(
|
||||
plan->getCurrentDataStream(),
|
||||
sorting_key_prefix_expr, context);
|
||||
sorting_key_prefix_expr);
|
||||
|
||||
expression_step->setStepDescription("Calculate sorting key prefix");
|
||||
plan->addStep(std::move(expression_step));
|
||||
@ -1352,7 +1351,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder(
|
||||
input_streams.emplace_back(plan->getCurrentDataStream());
|
||||
|
||||
const auto & common_header = plans.front()->getCurrentDataStream().header;
|
||||
auto union_step = std::make_unique<UnionStep>(std::move(input_streams), common_header, context);
|
||||
auto union_step = std::make_unique<UnionStep>(std::move(input_streams), common_header);
|
||||
|
||||
auto plan = std::make_unique<QueryPlan>();
|
||||
plan->unitePlans(std::move(union_step), std::move(plans));
|
||||
@ -1610,7 +1609,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal(
|
||||
for (const auto & partition_plan : partition_plans)
|
||||
input_streams.push_back(partition_plan->getCurrentDataStream());
|
||||
|
||||
auto union_step = std::make_unique<UnionStep>(std::move(input_streams), result_header, context);
|
||||
auto union_step = std::make_unique<UnionStep>(std::move(input_streams), result_header);
|
||||
union_step->setStepDescription("Unite sources after FINAL");
|
||||
QueryPlanPtr plan = std::make_unique<QueryPlan>();
|
||||
plan->unitePlans(std::move(union_step), std::move(partition_plans));
|
||||
|
@ -4,6 +4,8 @@
|
||||
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
||||
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
||||
#include <Processors/QueryPipeline.h>
|
||||
#include <Core/Defines.h>
|
||||
|
||||
@ -33,7 +35,7 @@ public:
|
||||
std::move(*MergeTreeDataSelectExecutor(part->storage)
|
||||
.readFromParts({part}, column_names, metadata_snapshot, query_info, context, max_block_size, num_streams));
|
||||
|
||||
return query_plan.convertToPipe(QueryPlanOptimizationSettings(context.getSettingsRef()));
|
||||
return query_plan.convertToPipe(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context));
|
||||
}
|
||||
|
||||
|
||||
|
@ -29,7 +29,8 @@
|
||||
#include <Processors/QueryPlan/SettingQuotaAndLimitsStep.h>
|
||||
#include <Processors/QueryPlan/ReadFromPreparedSource.h>
|
||||
#include <Processors/QueryPlan/UnionStep.h>
|
||||
|
||||
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
||||
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
@ -166,7 +167,9 @@ Pipe StorageBuffer::read(
|
||||
{
|
||||
QueryPlan plan;
|
||||
read(plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams);
|
||||
return plan.convertToPipe(QueryPlanOptimizationSettings(context.getSettingsRef()));
|
||||
return plan.convertToPipe(
|
||||
QueryPlanOptimizationSettings::fromContext(context),
|
||||
BuildQueryPipelineSettings::fromContext(context));
|
||||
}
|
||||
|
||||
void StorageBuffer::read(
|
||||
@ -254,8 +257,7 @@ void StorageBuffer::read(
|
||||
|
||||
auto adding_missed = std::make_unique<ExpressionStep>(
|
||||
query_plan.getCurrentDataStream(),
|
||||
std::move(actions),
|
||||
context);
|
||||
std::move(actions));
|
||||
|
||||
adding_missed->setStepDescription("Add columns missing in destination table");
|
||||
query_plan.addStep(std::move(adding_missed));
|
||||
@ -265,7 +267,7 @@ void StorageBuffer::read(
|
||||
header.getColumnsWithTypeAndName(),
|
||||
ActionsDAG::MatchColumnsMode::Name);
|
||||
|
||||
auto converting = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), actions_dag, context);
|
||||
auto converting = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), actions_dag);
|
||||
|
||||
converting->setStepDescription("Convert destination table columns to Buffer table structure");
|
||||
query_plan.addStep(std::move(converting));
|
||||
@ -372,7 +374,7 @@ void StorageBuffer::read(
|
||||
plans.emplace_back(std::make_unique<QueryPlan>(std::move(buffers_plan)));
|
||||
query_plan = QueryPlan();
|
||||
|
||||
auto union_step = std::make_unique<UnionStep>(std::move(input_streams), result_header, context);
|
||||
auto union_step = std::make_unique<UnionStep>(std::move(input_streams), result_header);
|
||||
union_step->setStepDescription("Unite sources from Buffer table");
|
||||
query_plan.unitePlans(std::move(union_step), std::move(plans));
|
||||
}
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user