mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 13:13:36 +00:00
Remove commented code.
This commit is contained in:
parent
78b7137877
commit
a5a8a70e6f
@ -58,29 +58,14 @@ ActionsDAG::ActionsDAG(const ColumnsWithTypeAndName & inputs_)
|
||||
|
||||
ActionsDAG::Node & ActionsDAG::addNode(Node node)
|
||||
{
|
||||
// auto it = index.find(node.result_name);
|
||||
// if (it != index.end() && !can_replace && add_to_index)
|
||||
// throw Exception("Column '" + node.result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN);
|
||||
|
||||
auto & res = nodes.emplace_back(std::move(node));
|
||||
|
||||
if (res.type == ActionType::INPUT)
|
||||
inputs.emplace_back(&res);
|
||||
|
||||
// if (add_to_index)
|
||||
// index.replace(&res);
|
||||
return res;
|
||||
}
|
||||
|
||||
// ActionsDAG::Node & ActionsDAG::getNode(const std::string & name)
|
||||
// {
|
||||
// auto it = index.find(name);
|
||||
// if (it == index.end())
|
||||
// throw Exception("Unknown identifier: '" + name + "'", ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||
|
||||
// return **it;
|
||||
// }
|
||||
|
||||
const ActionsDAG::Node & ActionsDAG::addInput(std::string name, DataTypePtr type)
|
||||
{
|
||||
Node node;
|
||||
@ -113,22 +98,7 @@ const ActionsDAG::Node & ActionsDAG::addColumn(ColumnWithTypeAndName column)
|
||||
node.result_name = std::move(column.name);
|
||||
node.column = std::move(column.column);
|
||||
|
||||
auto * res = &addNode(std::move(node));
|
||||
|
||||
// if (materialize)
|
||||
// {
|
||||
// auto & name = res->result_name;
|
||||
|
||||
// FunctionOverloadResolverPtr func_builder_materialize =
|
||||
// std::make_shared<FunctionOverloadResolverAdaptor>(
|
||||
// std::make_unique<DefaultOverloadResolver>(
|
||||
// std::make_shared<FunctionMaterialize>()));
|
||||
|
||||
// res = &addFunction(func_builder_materialize, {res}, {}, true, false);
|
||||
// res = &addAlias(*res, name, true);
|
||||
// }
|
||||
|
||||
return *res;
|
||||
return addNode(std::move(node));
|
||||
}
|
||||
|
||||
const ActionsDAG::Node & ActionsDAG::addAlias(const Node & child, std::string alias)
|
||||
@ -175,7 +145,7 @@ const ActionsDAG::Node & ActionsDAG::addFunction(
|
||||
|
||||
for (size_t i = 0; i < num_arguments; ++i)
|
||||
{
|
||||
auto & child = *node.children[i];
|
||||
const auto & child = *node.children[i];
|
||||
|
||||
ColumnWithTypeAndName argument;
|
||||
argument.column = child.column;
|
||||
@ -241,6 +211,36 @@ const ActionsDAG::Node & ActionsDAG::addFunction(
|
||||
return addNode(std::move(node));
|
||||
}
|
||||
|
||||
const ActionsDAG::Node & ActionsDAG::findInIndex(const std::string & name) const
|
||||
{
|
||||
if (const auto * node = tryFindInIndex(name))
|
||||
return *node;
|
||||
|
||||
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown identifier: '{}'", name);
|
||||
}
|
||||
|
||||
const ActionsDAG::Node * ActionsDAG::tryFindInIndex(const std::string & name) const
|
||||
{
|
||||
for (const auto & node : index)
|
||||
if (node->result_name == name)
|
||||
return node;
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
void ActionsDAG::addOrReplaceInIndex(const Node & node)
|
||||
{
|
||||
for (auto & index_node : index)
|
||||
{
|
||||
if (index_node->result_name == node.result_name)
|
||||
{
|
||||
index_node = &node;
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
index.push_back(&node);
|
||||
}
|
||||
|
||||
NamesAndTypesList ActionsDAG::getRequiredColumns() const
|
||||
{
|
||||
@ -592,98 +592,6 @@ void ActionsDAG::compileExpressions(size_t min_count_to_compile_expression)
|
||||
}
|
||||
#endif
|
||||
|
||||
// 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;
|
||||
// }
|
||||
|
||||
// auto inputs_mapping = buildNameToNodeMapping(inputs);
|
||||
// auto inputs_pos = getInputsPositions(block, inputs_mapping);
|
||||
|
||||
// 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});
|
||||
// }
|
||||
// }
|
||||
|
||||
// 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);
|
||||
// }
|
||||
|
||||
// return map;
|
||||
// }
|
||||
|
||||
// static std::vector<ssize_t> ActionsDAG::getInputsPositions(const Block & block, const NameToNodeMap & inputs_mapping)
|
||||
// {
|
||||
// std::vector<ssize_t> inputs_pos(inputs.size(), -1);
|
||||
|
||||
// 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;
|
||||
// }
|
||||
// }
|
||||
// }
|
||||
// }
|
||||
|
||||
// return inputs_pos;
|
||||
// }
|
||||
|
||||
// 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);
|
||||
// }
|
||||
|
||||
// Block res;
|
||||
|
||||
// for (auto & col : result_columns)
|
||||
// res.insert(std::move(col));
|
||||
|
||||
// for (const auto & item : block)
|
||||
// res.insert(std::move(item));
|
||||
|
||||
// block.swap(res);
|
||||
// }
|
||||
|
||||
std::string ActionsDAG::dumpDAG() const
|
||||
{
|
||||
std::unordered_map<const Node *, size_t> map;
|
||||
@ -940,8 +848,8 @@ ActionsDAGPtr ActionsDAG::makeAddingColumnActions(ColumnWithTypeAndName column)
|
||||
auto column_name = column.name;
|
||||
const auto * column_node = &adding_column_action->addColumn(std::move(column));
|
||||
NodeRawConstPtrs inputs = {column_node};
|
||||
auto & function_node = adding_column_action->addFunction(func_builder_materialize, std::move(inputs), {});
|
||||
auto & alias_node = adding_column_action->addAlias(function_node, std::move(column_name));
|
||||
const auto & function_node = adding_column_action->addFunction(func_builder_materialize, std::move(inputs), {});
|
||||
const auto & alias_node = adding_column_action->addAlias(function_node, std::move(column_name));
|
||||
|
||||
adding_column_action->index.push_back(&alias_node);
|
||||
return adding_column_action;
|
||||
@ -1124,7 +1032,7 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set<const Node *> split
|
||||
/// At first, visit all children.
|
||||
while (cur.next_child_to_visit < cur.node->children.size())
|
||||
{
|
||||
auto * child = cur.node->children[cur.next_child_to_visit];
|
||||
const auto * child = cur.node->children[cur.next_child_to_visit];
|
||||
auto & child_data = data[child];
|
||||
|
||||
if (!child_data.visited)
|
||||
@ -1314,17 +1222,13 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsBeforeArrayJoin(const NameSet &
|
||||
|
||||
ActionsDAG::SplitResult ActionsDAG::splitActionsForFilter(const std::string & column_name) const
|
||||
{
|
||||
auto it = index.begin();
|
||||
for (; it != index.end(); ++it)
|
||||
if ((*it)->result_name == column_name)
|
||||
break;
|
||||
|
||||
if (it == index.end())
|
||||
const auto * node = tryFindInIndex(column_name);
|
||||
if (!node)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Index for ActionsDAG does not contain filter column name {}. DAG:\n{}",
|
||||
column_name, dumpDAG());
|
||||
|
||||
std::unordered_set<const Node *> split_nodes = {*it};
|
||||
std::unordered_set<const Node *> split_nodes = {node};
|
||||
return split(split_nodes);
|
||||
}
|
||||
|
||||
@ -1549,22 +1453,12 @@ ActionsDAGPtr ActionsDAG::cloneActionsForConjunction(NodeRawConstPtrs conjunctio
|
||||
|
||||
ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, bool can_remove_filter, const Names & available_inputs, const ColumnsWithTypeAndName & all_inputs)
|
||||
{
|
||||
Node * predicate;
|
||||
|
||||
{
|
||||
auto it = index.begin();
|
||||
for (; it != index.end(); ++it)
|
||||
if ((*it)->result_name == filter_name)
|
||||
break;
|
||||
|
||||
if (it == index.end())
|
||||
Node * predicate = const_cast<Node *>(tryFindInIndex(filter_name));
|
||||
if (!predicate)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Index for ActionsDAG does not contain filter column name {}. DAG:\n{}",
|
||||
filter_name, dumpDAG());
|
||||
|
||||
predicate = const_cast<Node *>(*it);
|
||||
}
|
||||
|
||||
std::unordered_set<const Node *> allowed_nodes;
|
||||
|
||||
/// Get input nodes from available_inputs names.
|
||||
|
@ -82,94 +82,10 @@ public:
|
||||
bool allow_constant_folding = true;
|
||||
};
|
||||
|
||||
/// Index is used to:
|
||||
/// * find Node by it's result_name
|
||||
/// * specify order of columns in result
|
||||
/// It represents a set of available columns.
|
||||
/// Removing of column from index is equivalent to removing of column from final result.
|
||||
///
|
||||
/// DAG allows actions with duplicating result names. In this case index will point to last added Node.
|
||||
/// It does not cause any problems as long as execution of actions does not depend on action names anymore.
|
||||
///
|
||||
/// Index is a list of nodes + [map: name -> list::iterator].
|
||||
/// List is ordered, may contain nodes with same names, or one node several times.
|
||||
// class Index
|
||||
// {
|
||||
// private:
|
||||
// std::list<Node *> list;
|
||||
// /// Map key is a string_view to Node::result_name for node from value.
|
||||
// /// Map always point to existing node, so key always valid (nodes live longer then index).
|
||||
// std::unordered_map<std::string_view, std::list<Node *>::iterator> map;
|
||||
|
||||
// public:
|
||||
// auto size() const { return list.size(); }
|
||||
// bool contains(std::string_view key) const { return map.count(key) != 0; }
|
||||
|
||||
// std::list<Node *>::iterator begin() { return list.begin(); }
|
||||
// std::list<Node *>::iterator end() { return list.end(); }
|
||||
// std::list<Node *>::const_iterator begin() const { return list.begin(); }
|
||||
// std::list<Node *>::const_iterator end() const { return list.end(); }
|
||||
// std::list<Node *>::const_reverse_iterator rbegin() const { return list.rbegin(); }
|
||||
// std::list<Node *>::const_reverse_iterator rend() const { return list.rend(); }
|
||||
// std::list<Node *>::const_iterator find(std::string_view key) const
|
||||
// {
|
||||
// auto it = map.find(key);
|
||||
// if (it == map.end())
|
||||
// return list.end();
|
||||
|
||||
// return it->second;
|
||||
// }
|
||||
|
||||
// /// Insert method doesn't check if map already have node with the same name.
|
||||
// /// If node with the same name exists, it is removed from map, but not list.
|
||||
// /// It is expected and used for project(), when result may have several columns with the same name.
|
||||
// void insert(Node * node) { map[node->result_name] = list.emplace(list.end(), node); }
|
||||
// void prepend(Node * node) { map[node->result_name] = list.emplace(list.begin(), node); }
|
||||
|
||||
// /// If node with same name exists in index, replace it. Otherwise insert new node to index.
|
||||
// void replace(Node * node)
|
||||
// {
|
||||
// if (auto handle = map.extract(node->result_name))
|
||||
// {
|
||||
// handle.key() = node->result_name; /// Change string_view
|
||||
// *handle.mapped() = node;
|
||||
// map.insert(std::move(handle));
|
||||
// }
|
||||
// else
|
||||
// insert(node);
|
||||
// }
|
||||
|
||||
// void remove(std::list<Node *>::iterator it)
|
||||
// {
|
||||
// auto map_it = map.find((*it)->result_name);
|
||||
// if (map_it != map.end() && map_it->second == it)
|
||||
// map.erase(map_it);
|
||||
|
||||
// list.erase(it);
|
||||
// }
|
||||
|
||||
// void swap(Index & other)
|
||||
// {
|
||||
// list.swap(other.list);
|
||||
// map.swap(other.map);
|
||||
// }
|
||||
// };
|
||||
|
||||
/// NOTE: std::list is an implementation detail.
|
||||
/// It allows to add and remove new nodes inplace without reallocation.
|
||||
/// Raw pointers to nodes remain valid.
|
||||
using Nodes = std::list<Node>;
|
||||
//using Inputs = std::vector<Node *>;
|
||||
|
||||
// struct ActionsSettings
|
||||
// {
|
||||
// size_t max_temporary_columns = 0;
|
||||
// size_t max_temporary_non_const_columns = 0;
|
||||
// size_t min_count_to_compile_expression = 0;
|
||||
// bool compile_expressions = false;
|
||||
// bool project_input = false;
|
||||
// bool projected_output = false;
|
||||
// };
|
||||
|
||||
private:
|
||||
Nodes nodes;
|
||||
@ -179,10 +95,6 @@ private:
|
||||
bool project_input = false;
|
||||
bool projected_output = false;
|
||||
|
||||
// #if USE_EMBEDDED_COMPILER
|
||||
// std::shared_ptr<CompiledExpressionCache> compilation_cache;
|
||||
// #endif
|
||||
|
||||
public:
|
||||
ActionsDAG() = default;
|
||||
ActionsDAG(ActionsDAG &&) = default;
|
||||
@ -214,7 +126,16 @@ public:
|
||||
NodeRawConstPtrs children,
|
||||
std::string result_name);
|
||||
|
||||
/// Index can contain any column returned from DAG.
|
||||
/// You may manually change it if needed.
|
||||
NodeRawConstPtrs & getIndex() { return index; }
|
||||
/// Find first column by name in index. This search is linear.
|
||||
const Node & findInIndex(const std::string & name) const;
|
||||
/// Same, but return nullptr if node not found.
|
||||
const Node * tryFindInIndex(const std::string & name) const;
|
||||
/// Find node with the same name in index and replace it.
|
||||
/// If was not found, add node to index end.
|
||||
void addOrReplaceInIndex(const Node & node);
|
||||
|
||||
/// Call addAlias several times.
|
||||
void addAliases(const NamesWithAliases & aliases);
|
||||
@ -239,16 +160,6 @@ public:
|
||||
bool hasStatefulFunctions() const;
|
||||
bool trivial() const; /// If actions has no functions or array join.
|
||||
|
||||
// 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 transformBlock(Block & block, std::vector<ssize_t> inputs_pos, ColumnsWithTypeAndName result_columns) const;
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
void compileExpressions(size_t min_count_to_compile_expression);
|
||||
#endif
|
||||
@ -315,26 +226,6 @@ public:
|
||||
|
||||
private:
|
||||
Node & addNode(Node node);
|
||||
// Node & getNode(const std::string & name);
|
||||
|
||||
// Node & addAlias(Node & child, std::string alias, bool can_replace);
|
||||
// Node & addFunction(
|
||||
// const FunctionOverloadResolverPtr & function,
|
||||
// Inputs children,
|
||||
// std::string result_name,
|
||||
// bool can_replace,
|
||||
// bool add_to_index = true);
|
||||
|
||||
// ActionsDAGPtr cloneEmpty() const
|
||||
// {
|
||||
// auto actions = std::make_shared<ActionsDAG>();
|
||||
// // actions->settings = settings;
|
||||
|
||||
// // #if USE_EMBEDDED_COMPILER
|
||||
// // actions->compilation_cache = compilation_cache;
|
||||
// // #endif
|
||||
// return actions;
|
||||
// }
|
||||
|
||||
void removeUnusedActions(bool allow_remove_inputs = true);
|
||||
|
||||
@ -342,7 +233,7 @@ private:
|
||||
void compileFunctions(size_t min_count_to_compile_expression);
|
||||
#endif
|
||||
|
||||
ActionsDAGPtr cloneActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs);
|
||||
static ActionsDAGPtr cloneActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs);
|
||||
};
|
||||
|
||||
|
||||
|
@ -348,15 +348,8 @@ SetPtr makeExplicitSet(
|
||||
const ASTPtr & right_arg = args.children.at(1);
|
||||
|
||||
auto column_name = left_arg->getColumnName();
|
||||
const auto & index = actions.getIndex();
|
||||
auto it = index.begin();
|
||||
for (; it != index.end(); ++it)
|
||||
if ((*it)->result_name == column_name)
|
||||
break;
|
||||
|
||||
if (it == index.end())
|
||||
throw Exception("Unknown identifier: '" + left_arg->getColumnName() + "'", ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||
const DataTypePtr & left_arg_type = (*it)->result_type;
|
||||
const auto & dag_node = actions.findInIndex(column_name);
|
||||
const DataTypePtr & left_arg_type = dag_node.result_type;
|
||||
|
||||
DataTypes set_element_types = {left_arg_type};
|
||||
const auto * left_tuple_type = typeid_cast<const DataTypeTuple *>(left_arg_type.get());
|
||||
|
@ -235,16 +235,10 @@ void ExpressionAnalyzer::analyzeAggregation()
|
||||
getRootActionsNoMakeSet(group_asts[i], true, temp_actions, false);
|
||||
|
||||
const auto & column_name = group_asts[i]->getColumnName();
|
||||
const auto & index = temp_actions->getIndex();
|
||||
auto it = index.begin();
|
||||
for (; it != index.end(); ++it)
|
||||
if ((*it)->result_name == column_name)
|
||||
break;
|
||||
if (it == index.end())
|
||||
const auto * node = temp_actions->tryFindInIndex(column_name);
|
||||
if (!node)
|
||||
throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||
|
||||
const auto & node = *it;
|
||||
|
||||
/// Constant expressions have non-null column pointer at this stage.
|
||||
if (node->column && isColumnConst(*node->column))
|
||||
{
|
||||
@ -394,14 +388,7 @@ void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node)
|
||||
auto temp_actions = std::make_shared<ActionsDAG>(columns_after_join);
|
||||
getRootActions(left_in_operand, true, temp_actions);
|
||||
|
||||
const auto & index = temp_actions->getIndex();
|
||||
auto it = index.begin();
|
||||
auto column_name = left_in_operand->getColumnName();
|
||||
for (; it != index.end(); ++it)
|
||||
if ((*it)->result_name == column_name)
|
||||
break;
|
||||
|
||||
if (it != index.end())
|
||||
if (temp_actions->tryFindInIndex(left_in_operand->getColumnName()))
|
||||
makeExplicitSet(func, *temp_actions, true, context,
|
||||
settings.size_limits_for_set, prepared_sets);
|
||||
}
|
||||
@ -456,22 +443,18 @@ bool ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions)
|
||||
aggregate.argument_names.resize(arguments.size());
|
||||
DataTypes types(arguments.size());
|
||||
|
||||
const auto & index = actions->getIndex();
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
const std::string & name = arguments[i]->getColumnName();
|
||||
auto it = index.begin();
|
||||
for (; it != index.end(); ++it)
|
||||
if ((*it)->result_name == name)
|
||||
break;
|
||||
if (it == index.end())
|
||||
const auto * dag_node = actions->tryFindInIndex(name);
|
||||
if (!dag_node)
|
||||
{
|
||||
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER,
|
||||
"Unknown identifier '{}' in aggregate function '{}'",
|
||||
name, node->formatForErrorMessage());
|
||||
}
|
||||
|
||||
types[i] = (*it)->result_type;
|
||||
types[i] = dag_node->result_type;
|
||||
aggregate.argument_names[i] = name;
|
||||
}
|
||||
|
||||
@ -607,24 +590,19 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
|
||||
= window_function.function_node->arguments->children;
|
||||
window_function.argument_types.resize(arguments.size());
|
||||
window_function.argument_names.resize(arguments.size());
|
||||
const auto & index = actions->getIndex();
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
const std::string & name = arguments[i]->getColumnName();
|
||||
const auto * node = actions->tryFindInIndex(name);
|
||||
|
||||
auto it = index.begin();
|
||||
for (; it != index.end(); ++it)
|
||||
if ((*it)->result_name == name)
|
||||
break;
|
||||
|
||||
if (it == index.end())
|
||||
if (!node)
|
||||
{
|
||||
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER,
|
||||
"Unknown identifier '{}' in window function '{}'",
|
||||
name, window_function.function_node->formatForErrorMessage());
|
||||
}
|
||||
|
||||
window_function.argument_types[i] = (*it)->result_type;
|
||||
window_function.argument_types[i] = node->result_type;
|
||||
window_function.argument_names[i] = name;
|
||||
}
|
||||
|
||||
@ -699,16 +677,8 @@ ArrayJoinActionPtr ExpressionAnalyzer::addMultipleArrayJoinAction(ActionsDAGPtr
|
||||
/// Assign new names to columns, if needed.
|
||||
if (result_source.first != result_source.second)
|
||||
{
|
||||
auto & index = actions->getIndex();
|
||||
auto it = index.begin();
|
||||
for (; it != index.end(); ++it)
|
||||
if ((*it)->result_name == result_source.second)
|
||||
break;
|
||||
|
||||
if (it == index.end())
|
||||
throw Exception("Unknown identifier: '" + result_source.second + "'", ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||
|
||||
index.push_back(&actions->addAlias(**it, result_source.first));
|
||||
const auto & node = actions->findInIndex(result_source.second);
|
||||
actions->getIndex().push_back(&actions->addAlias(node, result_source.first));
|
||||
}
|
||||
|
||||
/// Make ARRAY JOIN (replace arrays with their insides) for the columns in these new names.
|
||||
@ -905,16 +875,8 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere(
|
||||
String prewhere_column_name = select_query->prewhere()->getColumnName();
|
||||
step.addRequiredOutput(prewhere_column_name);
|
||||
|
||||
const auto & index = step.actions()->getIndex();
|
||||
auto it = index.begin();
|
||||
for (; it != index.end(); ++it)
|
||||
if ((*it)->result_name == prewhere_column_name)
|
||||
break;
|
||||
|
||||
if (it == index.end())
|
||||
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown identifier: '{}'", prewhere_column_name);
|
||||
|
||||
auto filter_type = (*it)->result_type;
|
||||
const auto & node = step.actions()->findInIndex(prewhere_column_name);
|
||||
auto filter_type = node.result_type;
|
||||
if (!filter_type->canBeUsedInBooleanContext())
|
||||
throw Exception("Invalid type for filter in PREWHERE: " + filter_type->getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER);
|
||||
@ -1010,16 +972,8 @@ bool SelectQueryExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain,
|
||||
auto where_column_name = select_query->where()->getColumnName();
|
||||
step.addRequiredOutput(where_column_name);
|
||||
|
||||
const auto & index = step.actions()->getIndex();
|
||||
auto it = index.begin();
|
||||
for (; it != index.end(); ++it)
|
||||
if ((*it)->result_name == where_column_name)
|
||||
break;
|
||||
|
||||
if (it == index.end())
|
||||
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown identifier: '{}'", where_column_name);
|
||||
|
||||
auto filter_type = (*it)->result_type;
|
||||
const auto & node = step.actions()->findInIndex(where_column_name);
|
||||
auto filter_type = node.result_type;
|
||||
if (!filter_type->canBeUsedInBooleanContext())
|
||||
throw Exception("Invalid type for filter in WHERE: " + filter_type->getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER);
|
||||
|
@ -833,7 +833,7 @@ void ActionsDAG::compileFunctions(size_t min_count_to_compile_expression)
|
||||
|
||||
while (frame.next_child_to_visit < frame.node->children.size())
|
||||
{
|
||||
auto * child = frame.node->children[frame.next_child_to_visit];
|
||||
const auto * child = frame.node->children[frame.next_child_to_visit];
|
||||
|
||||
if (visited.count(child))
|
||||
++frame.next_child_to_visit;
|
||||
|
@ -675,57 +675,22 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector<Stage> &
|
||||
stage.analyzer->appendExpression(actions_chain, kv.second, dry_run);
|
||||
|
||||
auto & actions = actions_chain.getLastStep().actions();
|
||||
auto & index = actions->getIndex();
|
||||
|
||||
for (const auto & kv : stage.column_to_updated)
|
||||
{
|
||||
auto column_name = kv.second->getColumnName();
|
||||
auto it = index.begin();
|
||||
for (; it != index.end(); ++it)
|
||||
if ((*it)->result_name == column_name)
|
||||
break;
|
||||
|
||||
if (it == index.end())
|
||||
throw Exception("Unknown identifier: '" + column_name + "'", ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||
|
||||
const auto & alias = actions->addAlias(**it, kv.first);
|
||||
bool added = false;
|
||||
for (auto & node : index)
|
||||
{
|
||||
if (node->result_name == alias.result_name)
|
||||
{
|
||||
added = true;
|
||||
node = &alias;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (!added)
|
||||
index.push_back(&alias);
|
||||
const auto & dag_node = actions->findInIndex(column_name);
|
||||
const auto & alias = actions->addAlias(dag_node, kv.first);
|
||||
actions->addOrReplaceInIndex(alias);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/// Remove all intermediate columns.
|
||||
actions_chain.addStep();
|
||||
actions_chain.getLastStep().required_output.clear();
|
||||
ActionsDAG::NodeRawConstPtrs new_index;
|
||||
// auto & actions = actions_chain.getLastStep().actions();
|
||||
// auto & index = actions->getIndex();
|
||||
for (const auto & name : stage.output_columns)
|
||||
{
|
||||
// auto it = index.begin();
|
||||
// for (; it != index.end(); ++it)
|
||||
// if ((*it)->result_name == name)
|
||||
// break;
|
||||
|
||||
|
||||
// if (it == index.end())
|
||||
// throw Exception("Unknown identifier: '" + name + "'", ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||
|
||||
// new_index.push_back(*it);
|
||||
actions_chain.getLastStep().addRequiredOutput(name);
|
||||
}
|
||||
// index.swap(new_index);
|
||||
|
||||
actions_chain.getLastActions();
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user