From 50cd59a72b77ab75fb71228e25fa2a683e4ab83b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 13 Nov 2023 09:55:58 +0000 Subject: [PATCH] Remove default value for argument 'ignore_aliases' from IAST::getTreeHash() --- src/Interpreters/ActionsVisitor.cpp | 4 ++-- src/Interpreters/Aggregator.cpp | 6 +++--- src/Interpreters/ComparisonGraph.cpp | 4 ++-- src/Interpreters/ComparisonGraph.h | 2 +- src/Interpreters/Context.cpp | 6 +++--- src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp | 2 +- src/Interpreters/GlobalSubqueriesVisitor.h | 4 ++-- src/Interpreters/LogicalExpressionsOptimizer.cpp | 2 +- src/Interpreters/MonotonicityCheckVisitor.h | 2 +- src/Interpreters/PredicateRewriteVisitor.cpp | 4 ++-- src/Interpreters/QueryAliasesVisitor.cpp | 2 +- src/Interpreters/RewriteUniqToCountVisitor.cpp | 2 +- src/Interpreters/TreeCNFConverter.h | 6 +++--- src/Interpreters/TreeOptimizer.cpp | 4 ++-- src/Interpreters/WhereConstraintsOptimizer.cpp | 2 +- src/Parsers/ASTSetQuery.cpp | 2 +- src/Parsers/ASTSubquery.cpp | 2 +- src/Parsers/ASTWithAlias.cpp | 2 +- src/Parsers/IAST.h | 2 +- src/Storages/ConstraintsDescription.cpp | 4 ++-- src/Storages/KVStorageUtils.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/RPNBuilder.cpp | 6 +++--- src/Storages/System/StorageSystemQueryCache.cpp | 2 +- 24 files changed, 38 insertions(+), 38 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 3c4ed435f91..ab226d999c3 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -413,7 +413,7 @@ FutureSetPtr makeExplicitSet( auto set_element_keys = Set::getElementTypes(set_element_types, context->getSettingsRef().transform_null_in); - auto set_key = right_arg->getTreeHash(); + auto set_key = right_arg->getTreeHash(/*ignore_aliases=*/ true); if (auto set = prepared_sets.findTuple(set_key, set_element_keys)) return set; /// Already prepared. @@ -1391,7 +1391,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool set_key = query_tree->getTreeHash(); } else - set_key = right_in_operand->getTreeHash(); + set_key = right_in_operand->getTreeHash(/*ignore_aliases=*/ true); if (auto set = data.prepared_sets->findSubquery(set_key)) return set; diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 6329b55208e..129c02f032b 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -164,11 +164,11 @@ public: return 0; SipHash hash; - hash.update(select.tables()->getTreeHash()); + hash.update(select.tables()->getTreeHash(/*ignore_aliases=*/ true)); if (const auto where = select.where()) - hash.update(where->getTreeHash()); + hash.update(where->getTreeHash(/*ignore_aliases=*/ true)); if (const auto group_by = select.groupBy()) - hash.update(group_by->getTreeHash()); + hash.update(group_by->getTreeHash(/*ignore_aliases=*/ true)); return hash.get64(); } diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index 53e63903c43..4eacbae7a30 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -215,7 +215,7 @@ ComparisonGraph::ComparisonGraph(const NodeContainer & atomic_formulas, Co [node](const Node & constraint_node) { if constexpr (with_ast) - return constraint_node->getTreeHash() == node->getTreeHash() + return constraint_node->getTreeHash(/*ignore_aliases=*/ true) == node->getTreeHash(/*ignore_aliases=*/ true) && constraint_node->getColumnName() == node->getColumnName(); else return constraint_node->isEqual(*node); @@ -474,7 +474,7 @@ std::optional ComparisonGraph::getComponentId(const Node & node) c [node](const Node & constraint_node) { if constexpr (with_ast) - return constraint_node->getTreeHash() == node->getTreeHash() + return constraint_node->getTreeHash(/*ignore_aliases=*/ true) == node->getTreeHash(/*ignore_aliases=*/ true) && constraint_node->getColumnName() == node->getColumnName(); else return constraint_node->getTreeHash() == node->getTreeHash(); diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index 4fd90dad371..200d4af38fc 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -125,7 +125,7 @@ private: static auto getHash(const Node & node) { if constexpr (with_ast) - return node->getTreeHash(); + return node->getTreeHash(/*ignore_aliases=*/ true); else return QueryTreeNodePtrWithHash{node}; } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index ac552a3969c..75cc5f8366c 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1644,7 +1644,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const return res; } } - auto hash = table_expression->getTreeHash(); + auto hash = table_expression->getTreeHash(/*ignore_aliases=*/ true); auto key = toString(hash); StoragePtr & res = table_function_results[key]; if (!res) @@ -1804,7 +1804,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const /// /// remote('127.1', system.one) -> remote('127.1', 'system.one'), /// - auto new_hash = table_expression->getTreeHash(); + auto new_hash = table_expression->getTreeHash(/*ignore_aliases=*/ true); if (hash != new_hash) { key = toString(new_hash); @@ -1816,7 +1816,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const TableFunctionPtr & table_function_ptr) { - const auto hash = table_expression->getTreeHash(); + const auto hash = table_expression->getTreeHash(/*ignore_aliases=*/ true); const auto key = toString(hash); StoragePtr & res = table_function_results[key]; diff --git a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index 07a928be493..77a022e066b 100644 --- a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -109,7 +109,7 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr String subquery_alias = subquery.alias; bool prefer_alias_to_column_name = subquery.prefer_alias_to_column_name; - auto hash = subquery.getTreeHash(); + auto hash = subquery.getTreeHash(/*ignore_aliases=*/ true); const auto scalar_query_hash_str = toString(hash); std::unique_ptr interpreter = nullptr; diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 08378c3158b..384b562c80c 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -114,7 +114,7 @@ public: String external_table_name; if (alias.empty()) { - auto hash = subquery_or_table_name->getTreeHash(); + auto hash = subquery_or_table_name->getTreeHash(/*ignore_aliases=*/ true); external_table_name = fmt::format("_data_{}", toString(hash)); } else @@ -164,7 +164,7 @@ public: StoragePtr external_storage = external_storage_holder->getTable(); external_tables.emplace(external_table_name, external_storage_holder); - auto set_key = database_and_table_name->getTreeHash(); + auto set_key = database_and_table_name->getTreeHash(/*ignore_aliases=*/ true); if (!prepared_sets->findSubquery(set_key)) { diff --git a/src/Interpreters/LogicalExpressionsOptimizer.cpp b/src/Interpreters/LogicalExpressionsOptimizer.cpp index 78297c0ef5c..3689e6ac2db 100644 --- a/src/Interpreters/LogicalExpressionsOptimizer.cpp +++ b/src/Interpreters/LogicalExpressionsOptimizer.cpp @@ -139,7 +139,7 @@ void LogicalExpressionsOptimizer::collectDisjunctiveEqualityChains() const auto * literal = equals_expression_list->children[1]->as(); if (literal && literal->alias.empty()) { - auto expr_lhs = equals_expression_list->children[0]->getTreeHash(); + auto expr_lhs = equals_expression_list->children[0]->getTreeHash(/*ignore_aliases=*/ true); OrWithExpression or_with_expression{function, expr_lhs, function->tryGetAlias()}; disjunctive_equality_chains_map[or_with_expression].functions.push_back(equals); found_chain = true; diff --git a/src/Interpreters/MonotonicityCheckVisitor.h b/src/Interpreters/MonotonicityCheckVisitor.h index c1838fa105c..cc386825024 100644 --- a/src/Interpreters/MonotonicityCheckVisitor.h +++ b/src/Interpreters/MonotonicityCheckVisitor.h @@ -39,7 +39,7 @@ public: bool canOptimize(const ASTFunction & ast_function) const { /// if GROUP BY contains the same function ORDER BY shouldn't be optimized - const auto hash = ast_function.getTreeHash(); + const auto hash = ast_function.getTreeHash(/*ignore_aliases=*/ true); const auto key = toString(hash); if (group_by_function_hashes.count(key)) return false; diff --git a/src/Interpreters/PredicateRewriteVisitor.cpp b/src/Interpreters/PredicateRewriteVisitor.cpp index 40de887da27..dc0125e538a 100644 --- a/src/Interpreters/PredicateRewriteVisitor.cpp +++ b/src/Interpreters/PredicateRewriteVisitor.cpp @@ -159,7 +159,7 @@ static void getConjunctionHashesFrom(const ASTPtr & ast, std::set & /// Clone not to modify `ast` ASTPtr pred_copy = pred->clone(); useAliasInsteadOfIdentifier(pred_copy); - hashes.emplace(pred_copy->getTreeHash()); + hashes.emplace(pred_copy->getTreeHash(/*ignore_aliases=*/ true)); } } @@ -188,7 +188,7 @@ bool PredicateRewriteVisitorData::rewriteSubquery(ASTSelectQuery & subquery, con ASTPtr optimize_predicate = predicate->clone(); cleanAliasAndCollectIdentifiers(optimize_predicate, identifiers); - auto predicate_hash = optimize_predicate->getTreeHash(); + auto predicate_hash = optimize_predicate->getTreeHash(/*ignore_aliases=*/ true); if (hashes.contains(predicate_hash)) continue; diff --git a/src/Interpreters/QueryAliasesVisitor.cpp b/src/Interpreters/QueryAliasesVisitor.cpp index 345a27bc3ab..8f96044abec 100644 --- a/src/Interpreters/QueryAliasesVisitor.cpp +++ b/src/Interpreters/QueryAliasesVisitor.cpp @@ -121,7 +121,7 @@ void QueryAliasesMatcher::visitOther(const ASTPtr & ast, Data & data) String alias = ast->tryGetAlias(); if (!alias.empty()) { - if (aliases.contains(alias) && ast->getTreeHash() != aliases[alias]->getTreeHash()) + if (aliases.contains(alias) && ast->getTreeHash(/*ignore_aliases=*/ true) != aliases[alias]->getTreeHash(/*ignore_aliases=*/ true)) throw Exception(wrongAliasMessage(ast, aliases[alias], alias), ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS); aliases[alias] = ast; diff --git a/src/Interpreters/RewriteUniqToCountVisitor.cpp b/src/Interpreters/RewriteUniqToCountVisitor.cpp index 539271aa997..ddec6fe063e 100644 --- a/src/Interpreters/RewriteUniqToCountVisitor.cpp +++ b/src/Interpreters/RewriteUniqToCountVisitor.cpp @@ -27,7 +27,7 @@ bool matchFnUniq(String name) bool expressionEquals(const ASTPtr & lhs, const ASTPtr & rhs, const Aliases & alias) { - if (lhs->getTreeHash() == rhs->getTreeHash()) + if (lhs->getTreeHash(/*ignore_aliases=*/ true) == rhs->getTreeHash(/*ignore_aliases=*/ true)) { return true; } diff --git a/src/Interpreters/TreeCNFConverter.h b/src/Interpreters/TreeCNFConverter.h index 7f2fee4e6fd..3cc8b05d6c6 100644 --- a/src/Interpreters/TreeCNFConverter.h +++ b/src/Interpreters/TreeCNFConverter.h @@ -20,15 +20,15 @@ public: /// for set bool operator<(const AtomicFormula & rhs) const { - return ast->getTreeHash() == rhs.ast->getTreeHash() + return ast->getTreeHash(/*ignore_aliases=*/ true) == rhs.ast->getTreeHash(/*ignore_aliases=*/ true) ? negative < rhs.negative - : ast->getTreeHash() < rhs.ast->getTreeHash(); + : ast->getTreeHash(/*ignore_aliases=*/ true) < rhs.ast->getTreeHash(/*ignore_aliases=*/ true); } bool operator==(const AtomicFormula & rhs) const { return negative == rhs.negative && - ast->getTreeHash() == rhs.ast->getTreeHash() && + ast->getTreeHash(/*ignore_aliases=*/ true) == rhs.ast->getTreeHash(/*ignore_aliases=*/ true) && ast->getColumnName() == rhs.ast->getColumnName(); } }; diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 9612a4d6c3f..49d337267ae 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -409,7 +409,7 @@ void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, Context { for (auto & elem : set->children) { - const auto hash = elem->getTreeHash(); + const auto hash = elem->getTreeHash(/*ignore_aliases=*/ true); const auto key = toString(hash); group_by_hashes.insert(key); } @@ -419,7 +419,7 @@ void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, Context { for (auto & elem : group_by->children) { - const auto hash = elem->getTreeHash(); + const auto hash = elem->getTreeHash(/*ignore_aliases=*/ true); const auto key = toString(hash); group_by_hashes.insert(key); } diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index 91c19fa264e..5a0102f1ee7 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -37,7 +37,7 @@ enum class MatchState MatchState match(CNFQuery::AtomicFormula a, CNFQuery::AtomicFormula b) { bool match_means_ok = (a.negative == b.negative); - if (a.ast->getTreeHash() == b.ast->getTreeHash()) + if (a.ast->getTreeHash(/*ignore_aliases=*/ true) == b.ast->getTreeHash(/*ignore_aliases=*/ true)) return match_means_ok ? MatchState::FULL_MATCH : MatchState::NOT_MATCH; return MatchState::NONE; diff --git a/src/Parsers/ASTSetQuery.cpp b/src/Parsers/ASTSetQuery.cpp index e2c60e8369d..cd266cc930f 100644 --- a/src/Parsers/ASTSetQuery.cpp +++ b/src/Parsers/ASTSetQuery.cpp @@ -66,7 +66,7 @@ void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState &, Forma void ASTSetQuery::appendColumnName(WriteBuffer & ostr) const { - Hash hash = getTreeHash(); + Hash hash = getTreeHash(/*ignore_aliases=*/ true); writeCString("__settings_", ostr); writeText(hash.low64, ostr); diff --git a/src/Parsers/ASTSubquery.cpp b/src/Parsers/ASTSubquery.cpp index 75dfccd6e13..844520b2f64 100644 --- a/src/Parsers/ASTSubquery.cpp +++ b/src/Parsers/ASTSubquery.cpp @@ -19,7 +19,7 @@ void ASTSubquery::appendColumnNameImpl(WriteBuffer & ostr) const } else { - const auto hash = getTreeHash(); + const auto hash = getTreeHash(/*ignore_aliases=*/ true); writeCString("__subquery_", ostr); writeString(toString(hash), ostr); } diff --git a/src/Parsers/ASTWithAlias.cpp b/src/Parsers/ASTWithAlias.cpp index 5d1122ae4d8..6f64e33d33d 100644 --- a/src/Parsers/ASTWithAlias.cpp +++ b/src/Parsers/ASTWithAlias.cpp @@ -19,7 +19,7 @@ void ASTWithAlias::formatImpl(const FormatSettings & settings, FormatState & sta { /// If we have previously output this node elsewhere in the query, now it is enough to output only the alias. /// This is needed because the query can become extraordinary large after substitution of aliases. - if (!alias.empty() && !state.printed_asts_with_alias.emplace(frame.current_select, alias, getTreeHash()).second) + if (!alias.empty() && !state.printed_asts_with_alias.emplace(frame.current_select, alias, getTreeHash(/*ignore_aliases=*/ true)).second) { settings.ostr << (settings.hilite ? IAST::hilite_identifier : ""); settings.writeIdentifier(alias); diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 9afd59caa05..c147437797d 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -82,7 +82,7 @@ public: * useful for common subexpression elimination. Set 'ignore_aliases = false' if you don't want that behavior. */ using Hash = CityHash_v1_0_2::uint128; - Hash getTreeHash(bool ignore_aliases = true) const; + Hash getTreeHash(bool ignore_aliases) const; void updateTreeHash(SipHash & hash_state, bool ignore_aliases = true) const; virtual void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const; diff --git a/src/Storages/ConstraintsDescription.cpp b/src/Storages/ConstraintsDescription.cpp index 249ed8be428..219c3fd0c97 100644 --- a/src/Storages/ConstraintsDescription.cpp +++ b/src/Storages/ConstraintsDescription.cpp @@ -167,7 +167,7 @@ const ASTs & ConstraintsDescription::getConstraints() const std::optional ConstraintsDescription::getAtomIds(const ASTPtr & ast) const { - const auto hash = ast->getTreeHash(); + const auto hash = ast->getTreeHash(/*ignore_aliases=*/ true); auto it = ast_to_atom_ids.find(hash); if (it != ast_to_atom_ids.end()) return it->second; @@ -321,7 +321,7 @@ void ConstraintsDescription::update() ast_to_atom_ids.clear(); for (size_t i = 0; i < cnf_constraints.size(); ++i) for (size_t j = 0; j < cnf_constraints[i].size(); ++j) - ast_to_atom_ids[cnf_constraints[i][j].ast->getTreeHash()].push_back({i, j}); + ast_to_atom_ids[cnf_constraints[i][j].ast->getTreeHash(/*ignore_aliases=*/ true)].push_back({i, j}); graph = buildGraph(); } diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index 1884671a41d..3031fc6bf9d 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -72,7 +72,7 @@ bool traverseASTFilter( return false; value = args.children.at(1); - PreparedSets::Hash set_key = value->getTreeHash(); + PreparedSets::Hash set_key = value->getTreeHash(/*ignore_aliases=*/ true); FutureSetPtr future_set; if ((value->as() || value->as())) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1c0f9208fef..32c6974f1d0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -199,7 +199,7 @@ static void checkSuspiciousIndices(const ASTFunction * index_function) std::unordered_set unique_index_expression_hashes; for (const auto & child : index_function->arguments->children) { - const IAST::Hash hash = child->getTreeHash(); + const IAST::Hash hash = child->getTreeHash(/*ignore_aliases=*/ true); const auto & first_half_of_hash = hash.low64; if (!unique_index_expression_hashes.emplace(first_half_of_hash).second) diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index f2dcb77ee3b..3ffef1de718 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -314,7 +314,7 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet() const if (ast_node && prepared_sets) { - auto key = ast_node->getTreeHash(); + auto key = ast_node->getTreeHash(/*ignore_aliases=*/ true); const auto & sets = prepared_sets->getSetsFromTuple(); auto it = sets.find(key); if (it != sets.end() && !it->second.empty()) @@ -338,9 +338,9 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet(const DataTypes & data_types) if (prepared_sets && ast_node) { if (ast_node->as() || ast_node->as()) - return prepared_sets->findSubquery(ast_node->getTreeHash()); + return prepared_sets->findSubquery(ast_node->getTreeHash(/*ignore_aliases=*/ true)); - return prepared_sets->findTuple(ast_node->getTreeHash(), data_types); + return prepared_sets->findTuple(ast_node->getTreeHash(/*ignore_aliases=*/ true), data_types); } else if (dag_node) { diff --git a/src/Storages/System/StorageSystemQueryCache.cpp b/src/Storages/System/StorageSystemQueryCache.cpp index 03757101ddf..8538820cf41 100644 --- a/src/Storages/System/StorageSystemQueryCache.cpp +++ b/src/Storages/System/StorageSystemQueryCache.cpp @@ -50,7 +50,7 @@ void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr res_columns[3]->insert(key.is_shared); res_columns[4]->insert(key.is_compressed); res_columns[5]->insert(std::chrono::system_clock::to_time_t(key.expires_at)); - res_columns[6]->insert(key.ast->getTreeHash().low64); + res_columns[6]->insert(key.ast->getTreeHash(/*ignore_aliases=*/ false).low64); /// query cache considers aliases (issue #56258) } }