From 5154814bf941eb5b9b3b6262136d6f6fc27294fd Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Fri, 18 Jan 2019 19:30:35 +0300 Subject: [PATCH] try to get rid of IAST::range #4058 --- dbms/src/Interpreters/ActionsVisitor.cpp | 15 +++++----- dbms/src/Interpreters/ActionsVisitor.h | 10 ++----- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 4 +-- .../Interpreters/InterpreterCreateQuery.cpp | 14 ++++----- .../LogicalExpressionsOptimizer.cpp | 8 ----- dbms/src/Interpreters/PreparedSets.h | 20 +++++++++++++ dbms/src/Interpreters/QueryNormalizer.h | 1 + dbms/src/Interpreters/executeQuery.cpp | 14 +++++---- dbms/src/Interpreters/getClusterName.cpp | 14 +++++++-- dbms/src/Parsers/ASTIdentifier.h | 1 - dbms/src/Parsers/ExpressionElementParsers.cpp | 6 ---- dbms/src/Parsers/ExpressionListParsers.cpp | 30 ------------------- dbms/src/Parsers/IAST.h | 8 +---- dbms/src/Parsers/IParserBase.cpp | 2 -- dbms/src/Storages/MergeTree/KeyCondition.cpp | 21 +++++++------ .../MergeTree/MergeTreeWhereOptimizer.cpp | 2 +- .../ReplicatedMergeTreeMutationEntry.h | 1 + dbms/src/Storages/SelectQueryInfo.h | 12 +------- 18 files changed, 75 insertions(+), 108 deletions(-) create mode 100644 dbms/src/Interpreters/PreparedSets.h diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index 8d679a4cb58..b53d166ddb4 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -131,7 +131,7 @@ void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool SetPtr set = std::make_shared(size_limits, create_ordered_set); set->createFromAST(set_element_types, elements_ast, context); - prepared_sets[right_arg->range] = std::move(set); + prepared_sets[right_arg->getTreeHash()] = std::move(set); } static String getUniqueName(const Block & block, const String & prefix) @@ -380,12 +380,12 @@ void ActionsVisitor::visit(const ASTPtr & ast) /// Select the name in the next cycle. argument_names.emplace_back(); } - else if (prepared_sets.count(child->range) && functionIsInOrGlobalInOperator(node->name) && arg == 1) + else if (functionIsInOrGlobalInOperator(node->name) && arg == 1 && prepared_sets.count(child->getTreeHash())) { ColumnWithTypeAndName column; column.type = std::make_shared(); - const SetPtr & set = prepared_sets[child->range]; + const SetPtr & set = prepared_sets[child->getTreeHash()]; /// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name, /// so that sets with the same literal representation do not fuse together (they can have different types). @@ -530,9 +530,10 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc */ const IAST & args = *node->arguments; const ASTPtr & arg = args.children.at(1); + IAST::Hash tree_hash = arg->getTreeHash(); /// Already converted. - if (prepared_sets.count(arg->range)) + if (prepared_sets.count(tree_hash)) return; /// If the subquery or table name for SELECT. @@ -552,7 +553,7 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc if (storage_set) { - prepared_sets[arg->range] = storage_set->getSet(); + prepared_sets[tree_hash] = storage_set->getSet(); return; } } @@ -566,7 +567,7 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc /// If you already created a Set with the same subquery / table. if (subquery_for_set.set) { - prepared_sets[arg->range] = subquery_for_set.set; + prepared_sets[tree_hash] = subquery_for_set.set; return; } @@ -612,7 +613,7 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc } subquery_for_set.set = set; - prepared_sets[arg->range] = set; + prepared_sets[tree_hash] = set; } else { diff --git a/dbms/src/Interpreters/ActionsVisitor.h b/dbms/src/Interpreters/ActionsVisitor.h index 805b0ec02c4..d4a6d388fbc 100644 --- a/dbms/src/Interpreters/ActionsVisitor.h +++ b/dbms/src/Interpreters/ActionsVisitor.h @@ -1,6 +1,7 @@ #pragma once -#include +#include +#include #include @@ -10,13 +11,6 @@ namespace DB class Context; class ASTFunction; - -class Set; -using SetPtr = std::shared_ptr; -/// Will compare sets by their position in query string. It's possible because IAST::clone() doesn't chane IAST::range. -/// It should be taken into account when we want to change AST part which contains sets. -using PreparedSets = std::unordered_map; - class Join; using JoinPtr = std::shared_ptr; diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 8c1e13815ec..6d39a1303a4 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -279,7 +279,7 @@ void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_ return; } - prepared_sets[subquery_or_table_name->range] = std::move(set); + prepared_sets[subquery_or_table_name->getTreeHash()] = std::move(set); } @@ -308,7 +308,7 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block & { const ASTPtr & arg = args.children.at(1); - if (!prepared_sets.count(arg->range)) /// Not already prepared. + if (!prepared_sets.count(arg->getTreeHash())) /// Not already prepared. { if (typeid_cast(arg.get()) || isIdentifier(arg)) { diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 429c257b8d5..53bf85d2099 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -339,11 +339,10 @@ ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns) const auto column_declaration = std::make_shared(); column_declaration->name = column.name; - StringPtr type_name = std::make_shared(column.type->getName()); - auto pos = type_name->data(); - const auto end = pos + type_name->size(); - ParserIdentifierWithOptionalParameters storage_p; + String type_name = column.type->getName(); + auto pos = type_name.data(); + const auto end = pos + type_name.size(); column_declaration->type = parseQuery(storage_p, pos, end, "data type", 0); columns_list->children.emplace_back(column_declaration); } @@ -362,11 +361,10 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) column_declaration->name = column.name; - StringPtr type_name = std::make_shared(column.type->getName()); - auto type_name_pos = type_name->data(); - const auto type_name_end = type_name_pos + type_name->size(); - ParserIdentifierWithOptionalParameters storage_p; + String type_name = column.type->getName(); + auto type_name_pos = type_name.data(); + const auto type_name_end = type_name_pos + type_name.size(); column_declaration->type = parseQuery(storage_p, type_name_pos, type_name_end, "data type", 0); const auto defaults_it = columns.defaults.find(column.name); diff --git a/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp b/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp index 5226c96dce6..d7f04ff2a25 100644 --- a/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp @@ -228,17 +228,10 @@ void LogicalExpressionsOptimizer::addInExpression(const DisjunctiveEqualityChain /// Construct a list of literals `x1, ..., xN` from the string `expr = x1 OR ... OR expr = xN` ASTPtr value_list = std::make_shared(); - const char * min_range_first = nullptr; - const char * max_range_second = nullptr; for (const auto function : equality_functions) { const auto & operands = getFunctionOperands(function); value_list->children.push_back(operands[1]); - /// Get range min/max from all literals x1...xN, which will be used as tuple_functions' range - if (min_range_first == nullptr || min_range_first > operands[1]->range.first) - min_range_first = operands[1]->range.first; - if (max_range_second == nullptr || max_range_second < operands[1]->range.second) - max_range_second = operands[1]->range.second; } /// Sort the literals so that they are specified in the same order in the IN expression. @@ -260,7 +253,6 @@ void LogicalExpressionsOptimizer::addInExpression(const DisjunctiveEqualityChain auto tuple_function = std::make_shared(); tuple_function->name = "tuple"; - tuple_function->range = StringRange(min_range_first, max_range_second); tuple_function->arguments = value_list; tuple_function->children.push_back(tuple_function->arguments); diff --git a/dbms/src/Interpreters/PreparedSets.h b/dbms/src/Interpreters/PreparedSets.h new file mode 100644 index 00000000000..4276e62a239 --- /dev/null +++ b/dbms/src/Interpreters/PreparedSets.h @@ -0,0 +1,20 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +struct ASTHalfHash +{ + UInt64 operator()(const IAST::Hash & ast_hash) const { return ast_hash.first; } +}; + +class Set; +using SetPtr = std::shared_ptr; + +using PreparedSets = std::unordered_map; + +} diff --git a/dbms/src/Interpreters/QueryNormalizer.h b/dbms/src/Interpreters/QueryNormalizer.h index 3e55e0253e6..57f4645569a 100644 --- a/dbms/src/Interpreters/QueryNormalizer.h +++ b/dbms/src/Interpreters/QueryNormalizer.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 3911e437fa6..728225fdb80 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -150,7 +150,7 @@ static std::tuple executeQueryImpl( ParserQuery parser(end, settings.enable_debug_queries); ASTPtr ast; - size_t query_size; + const char * query_end; /// Don't limit the size of internal queries. size_t max_query_size = 0; @@ -162,10 +162,11 @@ static std::tuple executeQueryImpl( /// TODO Parser should fail early when max_query_size limit is reached. ast = parseQuery(parser, begin, end, "", max_query_size); - /// Copy query into string. It will be written to log and presented in processlist. If an INSERT query, string will not include data to insertion. - if (!(begin <= ast->range.first && ast->range.second <= end)) - throw Exception("Unexpected behavior: AST chars range is not inside source range", ErrorCodes::LOGICAL_ERROR); - query_size = ast->range.second - begin; + const auto * insert_query = dynamic_cast(ast.get()); + if (insert_query && insert_query->data) + query_end = insert_query->data; + else + query_end = end; } catch (...) { @@ -180,7 +181,8 @@ static std::tuple executeQueryImpl( throw; } - String query(begin, query_size); + /// Copy query into string. It will be written to log and presented in processlist. If an INSERT query, string will not include data to insertion. + String query(begin, query_end); BlockIO res; try diff --git a/dbms/src/Interpreters/getClusterName.cpp b/dbms/src/Interpreters/getClusterName.cpp index 4312da3e1a6..bc32e3dbea7 100644 --- a/dbms/src/Interpreters/getClusterName.cpp +++ b/dbms/src/Interpreters/getClusterName.cpp @@ -24,12 +24,22 @@ std::string getClusterName(const IAST & node) if (const ASTLiteral * ast_lit = typeid_cast(&node)) return ast_lit->value.safeGet(); + /// A hack to support hyphens in cluster names. if (const ASTFunction * ast_func = typeid_cast(&node)) { - if (!ast_func->range.first || !ast_func->range.second) + if (ast_func->name != "minus" || !ast_func->arguments || ast_func->arguments->children.size() < 2) throw Exception("Illegal expression instead of cluster name.", ErrorCodes::BAD_ARGUMENTS); - return String(ast_func->range.first, ast_func->range.second); + String name; + for (const auto & arg : ast_func->arguments->children) + { + if (name.empty()) + name += getClusterName(*arg); + else + name += "-" + getClusterName(*arg); + } + + return name; } throw Exception("Illegal expression instead of cluster name.", ErrorCodes::BAD_ARGUMENTS); diff --git a/dbms/src/Parsers/ASTIdentifier.h b/dbms/src/Parsers/ASTIdentifier.h index 4805a1f4fd5..a2b7f3ea555 100644 --- a/dbms/src/Parsers/ASTIdentifier.h +++ b/dbms/src/Parsers/ASTIdentifier.h @@ -19,7 +19,6 @@ public: : name(name_) , special(false) { - range = StringRange(name.data(), name.data() + name.size()); } /** Get the text that identifies this element. */ diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 9ba848dbe1a..a1fab382c36 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -672,8 +672,6 @@ bool ParserRightExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expec bool ParserExtractExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - auto begin = pos; - if (!ParserKeyword("EXTRACT").ignore(pos, expected)) return false; @@ -734,14 +732,10 @@ bool ParserExtractExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp auto function = std::make_shared(); auto exp_list = std::make_shared(); - function->range.first = begin->begin; - function->range.second = pos->begin; function->name = function_name; //"toYear"; function->arguments = exp_list; function->children.push_back(exp_list); exp_list->children.push_back(expr); - exp_list->range.first = begin->begin; - exp_list->range.second = pos->begin; node = function; return true; diff --git a/dbms/src/Parsers/ExpressionListParsers.cpp b/dbms/src/Parsers/ExpressionListParsers.cpp index 9f17b3935f5..c732ce4f38e 100644 --- a/dbms/src/Parsers/ExpressionListParsers.cpp +++ b/dbms/src/Parsers/ExpressionListParsers.cpp @@ -138,7 +138,6 @@ static bool parseOperator(IParser::Pos & pos, const char * op, Expected & expect bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { bool first = true; - Pos begin = pos; while (1) { @@ -174,16 +173,12 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node return false; /// the first argument of the function is the previous element, the second is the next one - function->range.first = begin->begin; - function->range.second = pos->begin; function->name = it[1]; function->arguments = exp_list; function->children.push_back(exp_list); exp_list->children.push_back(node); exp_list->children.push_back(elem); - exp_list->range.first = begin->begin; - exp_list->range.second = pos->begin; /** special exception for the access operator to the element of the array `x[y]`, which * contains the infix part '[' and the suffix ''] '(specified as' [') @@ -243,8 +238,6 @@ bool ParserBetweenExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp ASTPtr left; ASTPtr right; - Pos begin = pos; - if (!elem_parser.parse(pos, subject, expected)) return false; @@ -279,14 +272,10 @@ bool ParserBetweenExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp args_le->children.emplace_back(subject); args_le->children.emplace_back(right); - f_ge->range.first = begin->begin; - f_ge->range.second = pos->begin; f_ge->name = "greaterOrEquals"; f_ge->arguments = args_ge; f_ge->children.emplace_back(f_ge->arguments); - f_le->range.first = begin->begin; - f_le->range.second = pos->begin; f_le->name = "lessOrEquals"; f_le->arguments = args_le; f_le->children.emplace_back(f_le->arguments); @@ -294,8 +283,6 @@ bool ParserBetweenExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp args_and->children.emplace_back(f_ge); args_and->children.emplace_back(f_le); - f_and->range.first = begin->begin; - f_and->range.second = pos->begin; f_and->name = "and"; f_and->arguments = args_and; f_and->children.emplace_back(f_and->arguments); @@ -315,8 +302,6 @@ bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expect ASTPtr elem_then; ASTPtr elem_else; - Pos begin = pos; - if (!elem_parser.parse(pos, elem_cond, expected)) return false; @@ -339,8 +324,6 @@ bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expect /// function arguments auto exp_list = std::make_shared(); - function->range.first = begin->begin; - function->range.second = pos->begin; function->name = "if"; function->arguments = exp_list; function->children.push_back(exp_list); @@ -348,8 +331,6 @@ bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expect exp_list->children.push_back(elem_cond); exp_list->children.push_back(elem_then); exp_list->children.push_back(elem_else); - exp_list->range.first = begin->begin; - exp_list->range.second = pos->begin; node = function; } @@ -423,7 +404,6 @@ bool ParserLambdaExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expe bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { /// try to find any of the valid operators - Pos begin = pos; const char ** it; for (it = operators; *it; it += 2) { @@ -471,15 +451,11 @@ bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Ex /// function arguments auto exp_list = std::make_shared(); - function->range.first = begin->begin; - function->range.second = pos->begin; function->name = it[1]; function->arguments = exp_list; function->children.push_back(exp_list); exp_list->children.push_back(elem); - exp_list->range.first = begin->begin; - exp_list->range.second = pos->begin; node = function; } @@ -595,8 +571,6 @@ bool ParserNullityChecking::parseImpl(Pos & pos, ASTPtr & node, Expected & expec bool ParserIntervalOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - Pos begin = pos; - /// If no INTERVAL keyword, go to nested parser. if (!ParserKeyword("INTERVAL").ignore(pos, expected)) return next_parser.parse(pos, node, expected); @@ -620,15 +594,11 @@ bool ParserIntervalOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expec auto exp_list = std::make_shared(); /// the first argument of the function is the previous element, the second is the next one - function->range.first = begin->begin; - function->range.second = pos->begin; function->name = function_name; function->arguments = exp_list; function->children.push_back(exp_list); exp_list->children.push_back(expr); - exp_list->range.first = begin->begin; - exp_list->range.second = pos->begin; node = function; return true; diff --git a/dbms/src/Parsers/IAST.h b/dbms/src/Parsers/IAST.h index a597d40080f..3c4b9dec007 100644 --- a/dbms/src/Parsers/IAST.h +++ b/dbms/src/Parsers/IAST.h @@ -7,7 +7,6 @@ #include #include -#include #include @@ -54,7 +53,6 @@ class IAST : public std::enable_shared_from_this { public: ASTs children; - StringRange range; SemanticPtr semantic; @@ -209,11 +207,7 @@ public: virtual void formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const { - throw Exception("Unknown element in AST: " + getID() - + ((range.first && (range.second > range.first)) - ? " '" + std::string(range.first, range.second - range.first) + "'" - : ""), - ErrorCodes::UNKNOWN_ELEMENT_IN_AST); + throw Exception("Unknown element in AST: " + getID(), ErrorCodes::UNKNOWN_ELEMENT_IN_AST); } void cloneChildren(); diff --git a/dbms/src/Parsers/IParserBase.cpp b/dbms/src/Parsers/IParserBase.cpp index dffb1d7597d..ddbac8e92ee 100644 --- a/dbms/src/Parsers/IParserBase.cpp +++ b/dbms/src/Parsers/IParserBase.cpp @@ -22,8 +22,6 @@ bool IParserBase::parse(Pos & pos, ASTPtr & node, Expected & expected) node = nullptr; pos = begin; } - else if (node) - node->range = StringRange(begin, pos); return res; } diff --git a/dbms/src/Storages/MergeTree/KeyCondition.cpp b/dbms/src/Storages/MergeTree/KeyCondition.cpp index d386b865231..87d9f8def3d 100644 --- a/dbms/src/Storages/MergeTree/KeyCondition.cpp +++ b/dbms/src/Storages/MergeTree/KeyCondition.cpp @@ -635,18 +635,18 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo DataTypePtr key_expr_type; /// Type of expression containing key column size_t key_arg_pos; /// Position of argument with key column (non-const argument) - size_t key_column_num; /// Number of a key column (inside key_column_names array) + size_t key_column_num = -1; /// Number of a key column (inside key_column_names array) MonotonicFunctionsChain chain; bool is_set_const = false; bool is_constant_transformed = false; - if (prepared_sets.count(args[1]->range) - && tryPrepareSetIndex(args[0], context, out, prepared_sets[args[1]->range], key_column_num)) - { - key_arg_pos = 0; - is_set_const = true; - } - else if (getConstant(args[1], block_with_constants, const_value, const_type) + if (prepared_sets.count(args[1]->getTreeHash()) + && tryPrepareSetIndex(args[0], context, out, prepared_sets[args[1]->getTreeHash()], key_column_num)) + { + key_arg_pos = 0; + is_set_const = true; + } + else if (getConstant(args[1], block_with_constants, const_value, const_type) && isKeyPossiblyWrappedByMonotonicFunctions(args[0], context, key_column_num, key_expr_type, chain)) { key_arg_pos = 0; @@ -671,6 +671,9 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo else return false; + if (key_column_num == static_cast(-1)) + throw Exception("`key_column_num` wasn't initialized. It is a bug.", ErrorCodes::LOGICAL_ERROR); + std::string func_name = func->name; /// Transformed constant must weaken the condition, for example "x > 5" must weaken to "round(x) >= 5" @@ -1015,7 +1018,7 @@ bool KeyCondition::mayBeTrueInParallelogram(const std::vector & parallelo { auto in_func = typeid_cast(element.in_function.get()); const ASTs & args = typeid_cast(*in_func->arguments).children; - PreparedSets::const_iterator it = prepared_sets.find(args[1]->range); + PreparedSets::const_iterator it = prepared_sets.find(args[1]->getTreeHash()); if (in_func && it != prepared_sets.end()) { rpn_stack.emplace_back(element.set_index->mayBeTrueInRange(parallelogram, data_types)); diff --git a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 850c696b266..3eb9576b1c0 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -334,7 +334,7 @@ bool MergeTreeWhereOptimizer::isPrimaryKeyAtom(const IAST * const ast) const if ((primary_key_columns.count(first_arg_name) && isConstant(args[1])) || (primary_key_columns.count(second_arg_name) && isConstant(args[0])) || (primary_key_columns.count(first_arg_name) - && (prepared_sets.count(args[1]->range) || typeid_cast(args[1].get())))) + && (typeid_cast(args[1].get()) || prepared_sets.count(args[1]->getTreeHash())))) return true; } diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h index 68aab6fa021..8f253df3f10 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB diff --git a/dbms/src/Storages/SelectQueryInfo.h b/dbms/src/Storages/SelectQueryInfo.h index 1aed0a98f8f..e68078a6712 100644 --- a/dbms/src/Storages/SelectQueryInfo.h +++ b/dbms/src/Storages/SelectQueryInfo.h @@ -1,24 +1,14 @@ #pragma once +#include #include -#include -#include namespace DB { -class IAST; -using ASTPtr = std::shared_ptr; - class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; -class Set; -using SetPtr = std::shared_ptr; - -/// Information about calculated sets in right hand side of IN. -using PreparedSets = std::unordered_map; - struct PrewhereInfo { /// Actions which are executed in order to alias columns are used for prewhere actions.