From 1db8e773710c3d2058912988a5c0ada2d1cf3115 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Nov 2020 17:54:59 +0300 Subject: [PATCH] Add comments. Update ActionsDAG::Index --- src/Functions/FunctionsMiscellaneous.h | 2 +- src/Interpreters/ActionsVisitor.cpp | 4 +- src/Interpreters/ExpressionActions.cpp | 78 +++++++++---------- src/Interpreters/ExpressionActions.h | 50 ++++++++---- src/Interpreters/ExpressionAnalyzer.cpp | 4 +- src/Interpreters/ExpressionJIT.cpp | 4 +- .../Transforms/TotalsHavingTransform.cpp | 2 +- src/Storages/ColumnsDescription.cpp | 2 +- src/Storages/MergeTree/KeyCondition.cpp | 8 +- src/Storages/MergeTree/MergeTreeData.cpp | 4 +- src/Storages/ReadInOrderOptimizer.cpp | 2 +- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/TTLDescription.cpp | 2 +- 13 files changed, 90 insertions(+), 74 deletions(-) diff --git a/src/Functions/FunctionsMiscellaneous.h b/src/Functions/FunctionsMiscellaneous.h index 096f640b553..064ddcc3f31 100644 --- a/src/Functions/FunctionsMiscellaneous.h +++ b/src/Functions/FunctionsMiscellaneous.h @@ -201,7 +201,7 @@ public: { /// Check that expression does not contain unusual actions that will break columnss structure. for (const auto & action : expression_actions->getActions()) - if (action.node->type == ActionsDAG::Type::ARRAY_JOIN) + if (action.node->type == ActionsDAG::ActionType::ARRAY_JOIN) throw Exception("Expression with arrayJoin or other unusual action cannot be captured", ErrorCodes::BAD_ARGUMENTS); std::unordered_map arguments_map; diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index b24cee9ff71..b47667efdbd 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -414,7 +414,7 @@ ScopeStack::ScopeStack(ActionsDAGPtr actions, const Context & context_) level.actions = std::move(actions); for (const auto & node : level.actions->getIndex()) - if (node->type == ActionsDAG::Type::INPUT) + if (node->type == ActionsDAG::ActionType::INPUT) level.inputs.emplace(node->result_name); } @@ -451,7 +451,7 @@ size_t ScopeStack::getColumnLevel(const std::string & name) const auto & index = stack[i].actions->getIndex(); auto it = index.find(name); - if (it != index.end() && (*it)->type != ActionsDAG::Type::INPUT) + if (it != index.end() && (*it)->type != ActionsDAG::ActionType::INPUT) return i; } diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 5e389b9b5bd..82d50d63b0f 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -66,16 +66,16 @@ std::string ExpressionActions::Action::toString() const std::stringstream out; switch (node->type) { - case ActionsDAG::Type::COLUMN: + case ActionsDAG::ActionType::COLUMN: out << "COLUMN " << (node->column ? node->column->getName() : "(no column)"); break; - case ActionsDAG::Type::ALIAS: + case ActionsDAG::ActionType::ALIAS: out << "ALIAS " << node->children.front()->result_name << " " << arguments.front(); break; - case ActionsDAG::Type::FUNCTION: + case ActionsDAG::ActionType::FUNCTION: out << "FUNCTION " << (node->is_function_compiled ? "[compiled] " : "") << (node->function_base ? node->function_base->getName() : "(no function)") << "("; for (size_t i = 0; i < node->children.size(); ++i) @@ -87,11 +87,11 @@ std::string ExpressionActions::Action::toString() const out << ")"; break; - case ActionsDAG::Type::ARRAY_JOIN: + case ActionsDAG::ActionType::ARRAY_JOIN: out << "ARRAY JOIN " << node->children.front()->result_name << " " << arguments.front(); break; - case ActionsDAG::Type::INPUT: + case ActionsDAG::ActionType::INPUT: out << "INPUT " << arguments.front(); break; } @@ -201,7 +201,7 @@ void ExpressionActions::executeAction(const Action & action, ExecutionContext & switch (action.node->type) { - case ActionsDAG::Type::FUNCTION: + case ActionsDAG::ActionType::FUNCTION: { auto & res_column = columns[action.result_position]; if (res_column.type || res_column.column) @@ -227,7 +227,7 @@ void ExpressionActions::executeAction(const Action & action, ExecutionContext & break; } - case ActionsDAG::Type::ARRAY_JOIN: + case ActionsDAG::ActionType::ARRAY_JOIN: { size_t array_join_key_pos = action.arguments.front().pos; auto array_join_key = columns[array_join_key_pos]; @@ -260,7 +260,7 @@ void ExpressionActions::executeAction(const Action & action, ExecutionContext & break; } - case ActionsDAG::Type::COLUMN: + case ActionsDAG::ActionType::COLUMN: { auto & res_column = columns[action.result_position]; res_column.column = action.node->column->cloneResized(num_rows); @@ -269,7 +269,7 @@ void ExpressionActions::executeAction(const Action & action, ExecutionContext & break; } - case ActionsDAG::Type::ALIAS: + case ActionsDAG::ActionType::ALIAS: { const auto & arg = action.arguments.front(); if (action.result_position != arg.pos) @@ -286,7 +286,7 @@ void ExpressionActions::executeAction(const Action & action, ExecutionContext & break; } - case ActionsDAG::Type::INPUT: + case ActionsDAG::ActionType::INPUT: { auto pos = execution_context.inputs_pos[action.arguments.front().pos]; if (pos < 0) @@ -315,7 +315,7 @@ Names ExpressionActions::getRequiredColumns() const bool ExpressionActions::hasArrayJoin() const { for (const auto & action : actions) - if (action.node->type == ActionsDAG::Type::ARRAY_JOIN) + if (action.node->type == ActionsDAG::ActionType::ARRAY_JOIN) return true; return false; @@ -388,7 +388,7 @@ std::string ExpressionActions::dumpActions() const bool ActionsDAG::hasArrayJoin() const { for (const auto & node : nodes) - if (node.type == Type::ARRAY_JOIN) + if (node.type == ActionType::ARRAY_JOIN) return true; return false; @@ -397,7 +397,7 @@ bool ActionsDAG::hasArrayJoin() const bool ActionsDAG::empty() const { for (const auto & node : nodes) - if (node.type != Type::INPUT) + if (node.type != ActionType::INPUT) return false; return true; @@ -468,7 +468,7 @@ ActionsDAGPtr ActionsDAG::splitActionsBeforeArrayJoin(const NameSet & array_join /// Make a copy part. if (cur.next_child_to_visit == cur.node->children.size()) { - if (cur.node->type == Type::INPUT && array_joined_columns.count(cur.node->result_name)) + if (cur.node->type == ActionType::INPUT && array_joined_columns.count(cur.node->result_name)) cur_data.depend_on_array_join = true; cur_data.visited = true; @@ -487,7 +487,7 @@ ActionsDAGPtr ActionsDAG::splitActionsBeforeArrayJoin(const NameSet & array_join /// If children is not created, int may be from split part. if (!child_data.to_this) { - if (child->type == Type::COLUMN) /// Just create new node for COLUMN action. + if (child->type == ActionType::COLUMN) /// Just create new node for COLUMN action. { child_data.to_this = &this_nodes.emplace_back(*child); } @@ -495,13 +495,13 @@ ActionsDAGPtr ActionsDAG::splitActionsBeforeArrayJoin(const NameSet & array_join { /// Node from split part is added as new input. Node input_node; - input_node.type = Type::INPUT; + input_node.type = ActionType::INPUT; input_node.result_type = child->result_type; input_node.result_name = child->result_name; // getUniqueNameForIndex(index, child->result_name); child_data.to_this = &this_nodes.emplace_back(std::move(input_node)); /// This node is needed for current action, so put it to index also. - split_index[child_data.to_split->result_name] = child_data.to_split; + split_index.replace(child_data.to_split); } } @@ -522,11 +522,11 @@ ActionsDAGPtr ActionsDAG::splitActionsBeforeArrayJoin(const NameSet & array_join if (cur_data.used_in_result) { - split_index[copy.result_name] = © + split_index.replace(©); /// If this node is needed in result, add it as input. Node input_node; - input_node.type = Type::INPUT; + input_node.type = ActionType::INPUT; input_node.result_type = node.result_type; input_node.result_name = node.result_name; cur_data.to_this = &this_nodes.emplace_back(std::move(input_node)); @@ -567,7 +567,7 @@ bool ExpressionActions::checkColumnIsAlwaysFalse(const String & column_name) con for (auto it = actions.rbegin(); it != actions.rend(); ++it) { const auto & action = *it; - if (action.node->type == ActionsDAG::Type::FUNCTION && action.node->function_base) + if (action.node->type == ActionsDAG::ActionType::FUNCTION && action.node->function_base) { if (action.node->result_name == column_name && action.node->children.size() > 1) { @@ -585,7 +585,7 @@ bool ExpressionActions::checkColumnIsAlwaysFalse(const String & column_name) con { for (const auto & action : actions) { - if (action.node->type == ActionsDAG::Type::COLUMN && action.node->result_name == set_to_check) + if (action.node->type == ActionsDAG::ActionType::COLUMN && action.node->result_name == set_to_check) { // Constant ColumnSet cannot be empty, so we only need to check non-constant ones. if (const auto * column_set = checkAndGetColumn(action.node->column.get())) @@ -793,7 +793,7 @@ ActionsDAG::Node & ActionsDAG::addNode(Node node, bool can_replace) auto & res = nodes.emplace_back(std::move(node)); - index[res.result_name] = &res; + index.replace(&res); return res; } @@ -809,7 +809,7 @@ ActionsDAG::Node & ActionsDAG::getNode(const std::string & name) const ActionsDAG::Node & ActionsDAG::addInput(std::string name, DataTypePtr type) { Node node; - node.type = Type::INPUT; + node.type = ActionType::INPUT; node.result_type = std::move(type); node.result_name = std::move(name); @@ -819,7 +819,7 @@ const ActionsDAG::Node & ActionsDAG::addInput(std::string name, DataTypePtr type const ActionsDAG::Node & ActionsDAG::addInput(ColumnWithTypeAndName column) { Node node; - node.type = Type::INPUT; + node.type = ActionType::INPUT; node.result_type = std::move(column.type); node.result_name = std::move(column.name); node.column = std::move(column.column); @@ -833,7 +833,7 @@ const ActionsDAG::Node & ActionsDAG::addColumn(ColumnWithTypeAndName column) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add column {} because it is nullptr", column.name); Node node; - node.type = Type::COLUMN; + node.type = ActionType::COLUMN; node.result_type = std::move(column.type); node.result_name = std::move(column.name); node.column = std::move(column.column); @@ -846,7 +846,7 @@ const ActionsDAG::Node & ActionsDAG::addAlias(const std::string & name, std::str auto & child = getNode(name); Node node; - node.type = Type::ALIAS; + node.type = ActionType::ALIAS; node.result_type = child.result_type; node.result_name = std::move(alias); node.column = child.column; @@ -865,7 +865,7 @@ const ActionsDAG::Node & ActionsDAG::addArrayJoin(const std::string & source_nam throw Exception("ARRAY JOIN requires array argument", ErrorCodes::TYPE_MISMATCH); Node node; - node.type = Type::ARRAY_JOIN; + node.type = ActionType::ARRAY_JOIN; node.result_type = array_type->getNestedType(); node.result_name = std::move(result_name); node.children.emplace_back(&child); @@ -894,7 +894,7 @@ const ActionsDAG::Node & ActionsDAG::addFunction( size_t num_arguments = argument_names.size(); Node node; - node.type = Type::FUNCTION; + node.type = ActionType::FUNCTION; node.function_builder = function; node.children.reserve(num_arguments); @@ -977,7 +977,7 @@ NamesAndTypesList ActionsDAG::getRequiredColumns() const { NamesAndTypesList result; for (const auto & node : nodes) - if (node.type == Type::INPUT) + if (node.type == ActionType::INPUT) result.push_back({node.result_name, node.result_type}); return result; @@ -1077,7 +1077,7 @@ void ActionsDAG::removeUnusedActions() if (!node->children.empty() && node->column && isColumnConst(*node->column) && node->allow_constant_folding) { /// Constant folding. - node->type = ActionsDAG::Type::COLUMN; + node->type = ActionsDAG::ActionType::COLUMN; node->children.clear(); } @@ -1114,7 +1114,7 @@ void ActionsDAG::addAliases(const NamesWithAliases & aliases, std::vectorresult_type; node.result_name = std::move(item.second); node.column = child->column; @@ -1160,7 +1160,7 @@ bool ActionsDAG::tryRestoreColumn(const std::string & column_name) auto & node = *it; if (node.result_name == column_name) { - index[node.result_name] = &node; + index.replace(&node); return true; } } @@ -1308,7 +1308,7 @@ ExpressionActionsPtr ActionsDAG::linearizeActions() const arguments.emplace_back(argument); } - if (node->type == Type::INPUT) + if (node->type == ActionType::INPUT) { /// Argument for input is special. It contains the position from required columns. ExpressionActions::Argument argument; @@ -1328,7 +1328,7 @@ ExpressionActionsPtr ActionsDAG::linearizeActions() const if (parent_data.num_created_children == parent->children.size()) { - auto & push_stack = parent->type == Type::ARRAY_JOIN ? ready_array_joins : ready_nodes; + auto & push_stack = parent->type == ActionType::ARRAY_JOIN ? ready_array_joins : ready_nodes; push_stack.push(parent); } } @@ -1398,23 +1398,23 @@ std::string ActionsDAG::dumpDAG() const out << map[&node] << " : "; switch (node.type) { - case ActionsDAG::Type::COLUMN: + case ActionsDAG::ActionType::COLUMN: out << "COLUMN "; break; - case ActionsDAG::Type::ALIAS: + case ActionsDAG::ActionType::ALIAS: out << "ALIAS "; break; - case ActionsDAG::Type::FUNCTION: + case ActionsDAG::ActionType::FUNCTION: out << "FUNCTION "; break; - case ActionsDAG::Type::ARRAY_JOIN: + case ActionsDAG::ActionType::ARRAY_JOIN: out << "ARRAY JOIN "; break; - case ActionsDAG::Type::INPUT: + case ActionsDAG::ActionType::INPUT: out << "INPUT "; break; } diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index 51a3fc9e384..bdc35c8e0f2 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -61,11 +61,20 @@ using ExpressionActionsPtr = std::shared_ptr; class ActionsDAG; using ActionsDAGPtr = std::shared_ptr; +/// Directed acyclic graph of expressions. +/// This is an intermediate representation of actions which is usually built from expression list AST. +/// Node of DAG describe calculation of a single column with known type, name, and constant value (if applicable). +/// +/// DAG representation is useful in case we need to know explicit dependencies between actions. +/// It is helpful when it is needed to optimize actions, remove unused expressions, compile subexpressions, +/// split or merge parts of graph, calculate expressions on partial input. +/// +/// Built DAG is used by ExpressionActions, which calculates expressions on block. class ActionsDAG { public: - enum class Type + enum class ActionType { /// Column which must be in input. INPUT, @@ -82,7 +91,7 @@ public: { std::vector children; - Type type; + ActionType type; std::string result_name; DataTypePtr result_type; @@ -108,24 +117,11 @@ public: { private: std::list 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::iterator> map; public: - Node *& operator[](std::string_view key) - { - auto res = map.emplace(key, list.end()); - if (res.second) - res.first->second = list.emplace(list.end(), nullptr); - - return *res.first->second; - } - - void swap(Index & other) - { - list.swap(other.list); - map.swap(other.map); - } - auto size() const { return list.size(); } bool contains(std::string_view key) const { return map.count(key) != 0; } @@ -146,6 +142,20 @@ public: /// 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); } + + /// 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(Node * node) { auto it = map.find(node->result_name); @@ -155,6 +165,12 @@ public: list.erase(it->second); map.erase(it); } + + void swap(Index & other) + { + list.swap(other.list); + map.swap(other.map); + } }; using Nodes = std::list; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 46d15705ece..60975b75196 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -77,7 +77,7 @@ bool allowEarlyConstantFolding(const ActionsDAG & actions, const Settings & sett for (const auto & node : actions.getNodes()) { - if (node.type == ActionsDAG::Type::FUNCTION && node.function_base) + if (node.type == ActionsDAG::ActionType::FUNCTION && node.function_base) { auto name = node.function_base->getName(); if (name == "ignore") @@ -1309,7 +1309,7 @@ void ExpressionAnalysisResult::checkActions() const { if (actions) for (const auto & node : actions->getNodes()) - if (node.type == ActionsDAG::Type::ARRAY_JOIN) + if (node.type == ActionsDAG::ActionType::ARRAY_JOIN) throw Exception("PREWHERE cannot contain ARRAY JOIN action", ErrorCodes::ILLEGAL_PREWHERE); }; diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index d36b08bcd74..05aebbd1f40 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -593,7 +593,7 @@ static bool isCompilableConstant(const ActionsDAG::Node & node) static bool isCompilableFunction(const ActionsDAG::Node & node) { - return node.type == ActionsDAG::Type::FUNCTION && isCompilable(*node.function_base); + return node.type == ActionsDAG::ActionType::FUNCTION && isCompilable(*node.function_base); } static LLVMFunction::CompileDAG getCompilableDAG( @@ -884,7 +884,7 @@ void ActionsDAG::compileFunctions() for (const auto * child : new_children) arguments.emplace_back(child->column, child->result_type, child->result_name); - frame.node->type = ActionsDAG::Type::FUNCTION; + frame.node->type = ActionsDAG::ActionType::FUNCTION; frame.node->function_base = fn; frame.node->function = fn->prepare(arguments); frame.node->children.swap(new_children); diff --git a/src/Processors/Transforms/TotalsHavingTransform.cpp b/src/Processors/Transforms/TotalsHavingTransform.cpp index 109ead164a5..c81c389b6fa 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.cpp +++ b/src/Processors/Transforms/TotalsHavingTransform.cpp @@ -164,7 +164,7 @@ void TotalsHavingTransform::transform(Chunk & chunk) for (const auto & action : expression->getActions()) { - if (action.node->type == ActionsDAG::Type::ARRAY_JOIN) + if (action.node->type == ActionsDAG::ActionType::ARRAY_JOIN) throw Exception("Having clause cannot contain arrayJoin", ErrorCodes::ILLEGAL_COLUMN); } diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index c55d608110a..dbe0b0247f6 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -502,7 +502,7 @@ Block validateColumnsDefaultsAndGetSampleBlock(ASTPtr default_expr_list, const N auto syntax_analyzer_result = TreeRewriter(context).analyze(default_expr_list, all_columns); const auto actions = ExpressionAnalyzer(default_expr_list, syntax_analyzer_result, context).getActions(true); for (const auto & action : actions->getActions()) - if (action.node->type == ActionsDAG::Type::ARRAY_JOIN) + if (action.node->type == ActionsDAG::ActionType::ARRAY_JOIN) throw Exception("Unsupported default value that requires ARRAY JOIN action", ErrorCodes::THERE_IS_NO_DEFAULT_VALUE); return actions->getSampleBlock(); diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 6771d99c100..80b65f73d99 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -616,7 +616,7 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( * which while not strictly monotonic, are monotonic everywhere on the input range. */ const auto & children = action.node->children; - if (action.node->type == ActionsDAG::Type::FUNCTION + if (action.node->type == ActionsDAG::ActionType::FUNCTION && children.size() == 1 && children[0]->result_name == expr_name) { @@ -683,7 +683,7 @@ bool KeyCondition::canConstantBeWrappedByFunctions( chain.push(cur_node); - if (cur_node->type == ActionsDAG::Type::FUNCTION && cur_node->children.size() <= 2) + if (cur_node->type == ActionsDAG::ActionType::FUNCTION && cur_node->children.size() <= 2) { if (!cur_node->function_base->isDeterministic()) is_valid_chain = false; @@ -705,7 +705,7 @@ bool KeyCondition::canConstantBeWrappedByFunctions( cur_node = next_node; } - else if (cur_node->type == ActionsDAG::Type::ALIAS) + else if (cur_node->type == ActionsDAG::ActionType::ALIAS) cur_node = cur_node->children.front(); else is_valid_chain = false; @@ -725,7 +725,7 @@ bool KeyCondition::canConstantBeWrappedByFunctions( const auto * func = chain.top(); chain.pop(); - if (func->type != ActionsDAG::Type::FUNCTION) + if (func->type != ActionsDAG::ActionType::FUNCTION) continue; if (func->children.size() == 1) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 43dfdfe7beb..4bfe97f2dc3 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -259,10 +259,10 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam { for (const auto & action : expr.getActions()) { - if (action.node->type == ActionsDAG::Type::ARRAY_JOIN) + if (action.node->type == ActionsDAG::ActionType::ARRAY_JOIN) throw Exception(key_name + " key cannot contain array joins", ErrorCodes::ILLEGAL_COLUMN); - if (action.node->type == ActionsDAG::Type::FUNCTION) + if (action.node->type == ActionsDAG::ActionType::FUNCTION) { IFunctionBase & func = *action.node->function_base; if (!func.isDeterministic()) diff --git a/src/Storages/ReadInOrderOptimizer.cpp b/src/Storages/ReadInOrderOptimizer.cpp index 6761e0b967a..3613fbff5cf 100644 --- a/src/Storages/ReadInOrderOptimizer.cpp +++ b/src/Storages/ReadInOrderOptimizer.cpp @@ -57,7 +57,7 @@ InputOrderInfoPtr ReadInOrderOptimizer::getInputOrder(const StorageMetadataPtr & bool found_function = false; for (const auto & action : elements_actions[i]->getActions()) { - if (action.node->type != ActionsDAG::Type::FUNCTION) + if (action.node->type != ActionsDAG::ActionType::FUNCTION) continue; if (found_function) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 4a98bc4673f..977df904dca 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -201,7 +201,7 @@ bool isExpressionActionsDeterministics(const ExpressionActionsPtr & actions) { for (const auto & action : actions->getActions()) { - if (action.node->type != ActionsDAG::Type::FUNCTION) + if (action.node->type != ActionsDAG::ActionType::FUNCTION) continue; if (!action.node->function_base->isDeterministic()) return false; diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 50a74ddf7f0..b1c6a033d8b 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -54,7 +54,7 @@ void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const Strin { for (const auto & action : ttl_expression->getActions()) { - if (action.node->type == ActionsDAG::Type::FUNCTION) + if (action.node->type == ActionsDAG::ActionType::FUNCTION) { IFunctionBase & func = *action.node->function_base; if (!func.isDeterministic())