mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #56665 from rschu1ze/refactor-ast-hash-calculation
Small refactoring of AST hash calculation (follow-up to #56545)
This commit is contained in:
commit
2931bbfe62
@ -517,9 +517,9 @@ void QueryFuzzer::fuzzCreateQuery(ASTCreateQuery & create)
|
||||
SipHash sip_hash;
|
||||
sip_hash.update(original_name);
|
||||
if (create.columns_list)
|
||||
create.columns_list->updateTreeHash(sip_hash);
|
||||
create.columns_list->updateTreeHash(sip_hash, /*ignore_aliases=*/ true);
|
||||
if (create.storage)
|
||||
create.storage->updateTreeHash(sip_hash);
|
||||
create.storage->updateTreeHash(sip_hash, /*ignore_aliases=*/ true);
|
||||
|
||||
const auto hash = getSipHash128AsPair(sip_hash);
|
||||
|
||||
|
@ -377,7 +377,7 @@ uint8_t CompressionCodecDeflateQpl::getMethodByte() const
|
||||
|
||||
void CompressionCodecDeflateQpl::updateHash(SipHash & hash) const
|
||||
{
|
||||
getCodecDesc()->updateTreeHash(hash);
|
||||
getCodecDesc()->updateTreeHash(hash, /*ignore_aliases=*/ true);
|
||||
}
|
||||
|
||||
UInt32 CompressionCodecDeflateQpl::getMaxCompressedDataSize(UInt32 uncompressed_size) const
|
||||
|
@ -56,7 +56,7 @@ uint8_t CompressionCodecDelta::getMethodByte() const
|
||||
|
||||
void CompressionCodecDelta::updateHash(SipHash & hash) const
|
||||
{
|
||||
getCodecDesc()->updateTreeHash(hash);
|
||||
getCodecDesc()->updateTreeHash(hash, /*ignore_aliases=*/ true);
|
||||
}
|
||||
|
||||
namespace
|
||||
|
@ -466,7 +466,7 @@ uint8_t CompressionCodecDoubleDelta::getMethodByte() const
|
||||
|
||||
void CompressionCodecDoubleDelta::updateHash(SipHash & hash) const
|
||||
{
|
||||
getCodecDesc()->updateTreeHash(hash);
|
||||
getCodecDesc()->updateTreeHash(hash, /*ignore_aliases=*/ true);
|
||||
hash.update(data_bytes_size);
|
||||
}
|
||||
|
||||
|
@ -576,7 +576,7 @@ uint8_t CompressionCodecEncrypted::getMethodByte() const
|
||||
|
||||
void CompressionCodecEncrypted::updateHash(SipHash & hash) const
|
||||
{
|
||||
getCodecDesc()->updateTreeHash(hash);
|
||||
getCodecDesc()->updateTreeHash(hash, /*ignore_aliases=*/ true);
|
||||
}
|
||||
|
||||
UInt32 CompressionCodecEncrypted::getMaxCompressedDataSize(UInt32 uncompressed_size) const
|
||||
|
@ -66,7 +66,7 @@ uint8_t CompressionCodecFPC::getMethodByte() const
|
||||
|
||||
void CompressionCodecFPC::updateHash(SipHash & hash) const
|
||||
{
|
||||
getCodecDesc()->updateTreeHash(hash);
|
||||
getCodecDesc()->updateTreeHash(hash, /*ignore_aliases=*/ true);
|
||||
}
|
||||
|
||||
CompressionCodecFPC::CompressionCodecFPC(UInt8 float_size, UInt8 compression_level)
|
||||
|
@ -64,7 +64,7 @@ uint8_t CompressionCodecGCD::getMethodByte() const
|
||||
|
||||
void CompressionCodecGCD::updateHash(SipHash & hash) const
|
||||
{
|
||||
getCodecDesc()->updateTreeHash(hash);
|
||||
getCodecDesc()->updateTreeHash(hash, /*ignore_aliases=*/ true);
|
||||
}
|
||||
|
||||
namespace
|
||||
|
@ -364,7 +364,7 @@ uint8_t CompressionCodecGorilla::getMethodByte() const
|
||||
|
||||
void CompressionCodecGorilla::updateHash(SipHash & hash) const
|
||||
{
|
||||
getCodecDesc()->updateTreeHash(hash);
|
||||
getCodecDesc()->updateTreeHash(hash, /*ignore_aliases=*/ true);
|
||||
hash.update(data_bytes_size);
|
||||
}
|
||||
|
||||
|
@ -78,7 +78,7 @@ uint8_t CompressionCodecLZ4::getMethodByte() const
|
||||
|
||||
void CompressionCodecLZ4::updateHash(SipHash & hash) const
|
||||
{
|
||||
getCodecDesc()->updateTreeHash(hash);
|
||||
getCodecDesc()->updateTreeHash(hash, /*ignore_aliases=*/ true);
|
||||
}
|
||||
|
||||
UInt32 CompressionCodecLZ4::getMaxCompressedDataSize(UInt32 uncompressed_size) const
|
||||
|
@ -18,7 +18,7 @@ uint8_t CompressionCodecNone::getMethodByte() const
|
||||
|
||||
void CompressionCodecNone::updateHash(SipHash & hash) const
|
||||
{
|
||||
getCodecDesc()->updateTreeHash(hash);
|
||||
getCodecDesc()->updateTreeHash(hash, /*ignore_aliases=*/ true);
|
||||
}
|
||||
|
||||
UInt32 CompressionCodecNone::doCompressData(const char * source, UInt32 source_size, char * dest) const
|
||||
|
@ -723,7 +723,7 @@ CompressionCodecT64::CompressionCodecT64(std::optional<TypeIndex> type_idx_, Var
|
||||
|
||||
void CompressionCodecT64::updateHash(SipHash & hash) const
|
||||
{
|
||||
getCodecDesc()->updateTreeHash(hash);
|
||||
getCodecDesc()->updateTreeHash(hash, /*ignore_aliases=*/ true);
|
||||
hash.update(type_idx.value_or(TypeIndex::Nothing));
|
||||
hash.update(variant);
|
||||
}
|
||||
|
@ -60,7 +60,7 @@ uint8_t CompressionCodecZSTD::getMethodByte() const
|
||||
|
||||
void CompressionCodecZSTD::updateHash(SipHash & hash) const
|
||||
{
|
||||
getCodecDesc()->updateTreeHash(hash);
|
||||
getCodecDesc()->updateTreeHash(hash, /*ignore_aliases=*/ true);
|
||||
}
|
||||
|
||||
UInt32 CompressionCodecZSTD::getMaxCompressedDataSize(UInt32 uncompressed_size) const
|
||||
|
@ -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;
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
|
@ -85,7 +85,7 @@ AsynchronousInsertQueue::InsertQuery::InsertQuery(
|
||||
SipHash siphash;
|
||||
|
||||
siphash.update(data_kind);
|
||||
query->updateTreeHash(siphash);
|
||||
query->updateTreeHash(siphash, /*ignore_aliases=*/ true);
|
||||
|
||||
if (user_id)
|
||||
{
|
||||
|
@ -148,7 +148,7 @@ QueryCache::Key::Key(ASTPtr ast_, const String & user_name_)
|
||||
}
|
||||
|
||||
/// Hashing of ASTs must consider aliases (issue #56258)
|
||||
constexpr bool ignore_aliases = false;
|
||||
static constexpr bool ignore_aliases = false;
|
||||
|
||||
bool QueryCache::Key::operator==(const Key & other) const
|
||||
{
|
||||
|
@ -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();
|
||||
|
@ -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};
|
||||
}
|
||||
|
@ -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];
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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))
|
||||
{
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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();
|
||||
}
|
||||
};
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -82,8 +82,8 @@ 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;
|
||||
void updateTreeHash(SipHash & hash_state, bool ignore_aliases = true) const;
|
||||
Hash getTreeHash(bool ignore_aliases) const;
|
||||
void updateTreeHash(SipHash & hash_state, bool ignore_aliases) const;
|
||||
virtual void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const;
|
||||
|
||||
void dumpTree(WriteBuffer & ostr, size_t indent = 0) const;
|
||||
|
@ -393,7 +393,7 @@ size_t ConstantExpressionTemplate::TemplateStructure::getTemplateHash(const ASTP
|
||||
SipHash hash_state;
|
||||
hash_state.update(result_column_type->getName());
|
||||
|
||||
expression->updateTreeHash(hash_state);
|
||||
expression->updateTreeHash(hash_state, /*ignore_aliases=*/ true);
|
||||
|
||||
for (const auto & info : replaced_literals)
|
||||
hash_state.update(info.type->getName());
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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>()))
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user