Remove commented code.

This commit is contained in:
Nikolai Kochetov 2021-03-11 20:03:39 +03:00
parent 78b7137877
commit a5a8a70e6f
6 changed files with 71 additions and 374 deletions

View File

@ -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.

View File

@ -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);
};

View File

@ -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());

View File

@ -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);

View File

@ -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;

View File

@ -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();