Remove default value for argument 'ignore_aliases' from IAST::getTreeHash()

This commit is contained in:
Robert Schulze 2023-11-13 09:55:58 +00:00
parent 6ee95433f9
commit 50cd59a72b
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
24 changed files with 38 additions and 38 deletions

View File

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

View File

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

View File

@ -215,7 +215,7 @@ ComparisonGraph<Node>::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<size_t> ComparisonGraph<Node>::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();

View File

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

View File

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

View File

@ -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<InterpreterSelectWithUnionQuery> interpreter = nullptr;

View File

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

View File

@ -139,7 +139,7 @@ void LogicalExpressionsOptimizer::collectDisjunctiveEqualityChains()
const auto * literal = equals_expression_list->children[1]->as<ASTLiteral>();
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;

View File

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

View File

@ -159,7 +159,7 @@ static void getConjunctionHashesFrom(const ASTPtr & ast, std::set<IAST::Hash> &
/// 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;

View File

@ -121,7 +121,7 @@ void QueryAliasesMatcher<T>::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;

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -167,7 +167,7 @@ const ASTs & ConstraintsDescription::getConstraints() const
std::optional<ConstraintsDescription::AtomIds> 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();
}

View File

@ -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<ASTSubquery>() || value->as<ASTIdentifier>()))

View File

@ -199,7 +199,7 @@ static void checkSuspiciousIndices(const ASTFunction * index_function)
std::unordered_set<UInt64> 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)

View File

@ -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<ASTSubquery>() || ast_node->as<ASTTableIdentifier>())
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)
{

View File

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