diff --git a/contrib/cityhash102/include/city.h b/contrib/cityhash102/include/city.h index 87363d16444..c98eb7e3585 100644 --- a/contrib/cityhash102/include/city.h +++ b/contrib/cityhash102/include/city.h @@ -73,8 +73,8 @@ struct uint128 uint128() = default; uint128(uint64 low64_, uint64 high64_) : low64(low64_), high64(high64_) {} - friend bool operator ==(const uint128 & x, const uint128 & y) { return (x.low64 == y.low64) && (x.high64 == y.high64); } - friend bool operator !=(const uint128 & x, const uint128 & y) { return !(x == y); } + + friend auto operator<=>(const uint128 &, const uint128 &) = default; }; inline uint64 Uint128Low64(const uint128 & x) { return x.low64; } diff --git a/src/Analyzer/HashUtils.h b/src/Analyzer/HashUtils.h index 2203e7d5203..3727ea1ea14 100644 --- a/src/Analyzer/HashUtils.h +++ b/src/Analyzer/HashUtils.h @@ -20,7 +20,7 @@ struct QueryTreeNodeWithHash {} QueryTreeNodePtrType node = nullptr; - std::pair hash; + CityHash_v1_0_2::uint128 hash; }; template @@ -55,6 +55,6 @@ struct std::hash> { size_t operator()(const DB::QueryTreeNodeWithHash & node_with_hash) const { - return node_with_hash.hash.first; + return node_with_hash.hash.low64; } }; diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index 763963b734a..3f6816696b4 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -106,7 +106,7 @@ public: */ bool isEqual(const IQueryTreeNode & rhs, CompareOptions compare_options = { .compare_aliases = true }) const; - using Hash = std::pair; + using Hash = CityHash_v1_0_2::uint128; using HashState = SipHash; /** Get tree hash identifying current tree diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index abf02547ccd..2d997bb7eff 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2032,7 +2032,7 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden auto & nearest_query_scope_query_node = nearest_query_scope->scope_node->as(); auto & mutable_context = nearest_query_scope_query_node.getMutableContext(); - auto scalar_query_hash_string = std::to_string(node_with_hash.hash.first) + '_' + std::to_string(node_with_hash.hash.second); + const auto scalar_query_hash_string = DB::toString(node_with_hash.hash); if (mutable_context->hasQueryContext()) mutable_context->getQueryContext()->addScalar(scalar_query_hash_string, scalar_block); diff --git a/src/Common/SipHash.h b/src/Common/SipHash.h index c77fe7f2bcf..00fe07648a5 100644 --- a/src/Common/SipHash.h +++ b/src/Common/SipHash.h @@ -24,6 +24,8 @@ #include #include +#include + namespace DB::ErrorCodes { extern const int LOGICAL_ERROR; @@ -247,11 +249,11 @@ inline std::array getSipHash128AsArray(SipHash & sip_hash) return arr; } -inline std::pair getSipHash128AsLoHi(SipHash & sip_hash) +inline CityHash_v1_0_2::uint128 getSipHash128AsLoHi(SipHash & sip_hash) { - std::pair lo_hi; - sip_hash.get128(lo_hi.first, lo_hi.second); - return lo_hi; + CityHash_v1_0_2::uint128 result; + sip_hash.get128(result.low64, result.high64); + return result; } inline UInt128 sipHash128Keyed(UInt64 key0, UInt64 key1, const char * data, const size_t size) diff --git a/src/Common/TransformEndianness.hpp b/src/Common/TransformEndianness.hpp index ccfec78ab64..c7bea2ef15a 100644 --- a/src/Common/TransformEndianness.hpp +++ b/src/Common/TransformEndianness.hpp @@ -4,6 +4,8 @@ #include #include +#include + #include namespace DB @@ -66,4 +68,11 @@ inline void transformEndianness(StrongTypedef & x) { transformEndianness(x.toUnderType()); } + +template +inline void transformEndianness(CityHash_v1_0_2::uint128 & x) +{ + transformEndianness(x.low64); + transformEndianness(x.high64); +} } diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index aa4c9b17e48..a9d1c108061 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -1161,6 +1161,15 @@ inline String toString(const T & x) return buf.str(); } +inline String toString(const CityHash_v1_0_2::uint128 & hash) +{ + WriteBufferFromOwnString buf; + writeText(hash.low64, buf); + writeChar('_', buf); + writeText(hash.high64, buf); + return buf.str(); +} + template inline String toStringWithFinalSeparator(const std::vector & x, const String & final_sep) { diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index 70543227b58..4fd90dad371 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -118,7 +118,7 @@ private: { size_t operator() (const IAST::Hash & hash) const { - return hash.first; + return hash.low64; } }; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a9055bbb1b9..f179e38b0f8 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1505,7 +1505,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const } } auto hash = table_expression->getTreeHash(); - String key = toString(hash.first) + '_' + toString(hash.second); + auto key = toString(hash); StoragePtr & res = table_function_results[key]; if (!res) { @@ -1656,7 +1656,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const auto new_hash = table_expression->getTreeHash(); if (hash != new_hash) { - key = toString(new_hash.first) + '_' + toString(new_hash.second); + key = toString(new_hash); table_function_results[key] = res; } } @@ -1665,8 +1665,8 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const TableFunctionPtr & table_function_ptr) { - auto hash = table_expression->getTreeHash(); - String key = toString(hash.first) + '_' + toString(hash.second); + const auto hash = table_expression->getTreeHash(); + const auto key = toString(hash); StoragePtr & res = table_function_results[key]; if (!res) diff --git a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index 3d821b60e81..ded99df9f74 100644 --- a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -98,7 +98,7 @@ static auto getQueryInterpreter(const ASTSubquery & subquery, ExecuteScalarSubqu void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr & ast, Data & data) { auto hash = subquery.getTreeHash(); - auto scalar_query_hash_str = toString(hash.first) + "_" + toString(hash.second); + const auto scalar_query_hash_str = toString(hash); std::unique_ptr interpreter = nullptr; bool hit = false; diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 5b633fee9b6..08378c3158b 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -115,7 +115,7 @@ public: if (alias.empty()) { auto hash = subquery_or_table_name->getTreeHash(); - external_table_name = fmt::format("_data_{}_{}", hash.first, hash.second); + external_table_name = fmt::format("_data_{}", toString(hash)); } else external_table_name = alias; diff --git a/src/Interpreters/MonotonicityCheckVisitor.h b/src/Interpreters/MonotonicityCheckVisitor.h index c95f5209760..c1838fa105c 100644 --- a/src/Interpreters/MonotonicityCheckVisitor.h +++ b/src/Interpreters/MonotonicityCheckVisitor.h @@ -39,8 +39,8 @@ public: bool canOptimize(const ASTFunction & ast_function) const { /// if GROUP BY contains the same function ORDER BY shouldn't be optimized - auto hash = ast_function.getTreeHash(); - String key = toString(hash.first) + '_' + toString(hash.second); + const auto hash = ast_function.getTreeHash(); + const auto key = toString(hash); if (group_by_function_hashes.count(key)) return false; diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 67822ecf440..c2c01b912f9 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -207,7 +207,7 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) String PreparedSets::toString(const PreparedSets::Hash & key, const DataTypes & types) { WriteBufferFromOwnString buf; - buf << "__set_" << key.first << "_" << key.second; + buf << "__set_" << DB::toString(key); if (!types.empty()) { buf << "("; diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index cb240f5260a..b953b8470e1 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -127,10 +127,10 @@ class PreparedSets { public: - using Hash = std::pair; + using Hash = CityHash_v1_0_2::uint128; struct Hashing { - UInt64 operator()(const Hash & key) const { return key.first ^ key.second; } + UInt64 operator()(const Hash & key) const { return key.low64 ^ key.high64; } }; using SetsFromTuple = std::unordered_map>, Hashing>; diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index fd4d2c9d846..1a7b5cd73ec 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -451,8 +451,8 @@ void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, Context { for (auto & elem : set->children) { - auto hash = elem->getTreeHash(); - String key = toString(hash.first) + '_' + toString(hash.second); + const auto hash = elem->getTreeHash(); + const auto key = toString(hash); group_by_hashes.insert(key); } } @@ -461,8 +461,8 @@ void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, Context { for (auto & elem : group_by->children) { - auto hash = elem->getTreeHash(); - String key = toString(hash.first) + '_' + toString(hash.second); + const auto hash = elem->getTreeHash(); + const auto key = toString(hash); group_by_hashes.insert(key); } } diff --git a/src/Parsers/ASTSubquery.cpp b/src/Parsers/ASTSubquery.cpp index a3408f12330..92adad666ed 100644 --- a/src/Parsers/ASTSubquery.cpp +++ b/src/Parsers/ASTSubquery.cpp @@ -19,11 +19,9 @@ void ASTSubquery::appendColumnNameImpl(WriteBuffer & ostr) const } else { - Hash hash = getTreeHash(); + const auto hash = getTreeHash(); writeCString("__subquery_", ostr); - writeText(hash.first, ostr); - ostr.write('_'); - writeText(hash.second, ostr); + writeString(toString(hash), ostr); } } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 7a8ab36518d..d0464c7b950 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -78,7 +78,7 @@ public: /** Get hash code, identifying this element and its subtree. */ - using Hash = std::pair; + using Hash = CityHash_v1_0_2::uint128; Hash getTreeHash() const; void updateTreeHash(SipHash & hash_state) const; virtual void updateTreeHashImpl(SipHash & hash_state) const; diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 7575828e64d..b39716c7712 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -165,7 +165,7 @@ public: case QueryTreeNodeType::LAMBDA: { auto lambda_hash = node->getTreeHash(); - result = "__lambda_" + toString(lambda_hash.first) + '_' + toString(lambda_hash.second); + result = "__lambda_" + toString(lambda_hash); break; } default: diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index 3c75d4fbea8..f0e6e44ad8c 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -114,8 +114,8 @@ const ColumnIdentifier * PlannerContext::getColumnNodeIdentifierOrNull(const Que PlannerContext::SetKey PlannerContext::createSetKey(const QueryTreeNodePtr & set_source_node) { - auto set_source_hash = set_source_node->getTreeHash(); - return "__set_" + toString(set_source_hash.first) + '_' + toString(set_source_hash.second); + const auto set_source_hash = set_source_node->getTreeHash(); + return "__set_" + toString(set_source_hash); } } diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 2d5264fa4e3..c388c370848 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -371,8 +371,8 @@ size_t ConstantExpressionTemplate::TemplateStructure::getTemplateHash(const ASTP const auto res128 = getSipHash128AsLoHi(hash_state); size_t res = 0; - boost::hash_combine(res, res128.first); - boost::hash_combine(res, res128.second); + boost::hash_combine(res, res128.low64); + boost::hash_combine(res, res128.high64); return res; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9c5e45aa488..fafa35d34c1 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -187,8 +187,8 @@ static void checkSuspiciousIndices(const ASTFunction * index_function) std::unordered_set unique_index_expression_hashes; for (const auto & child : index_function->arguments->children) { - IAST::Hash hash = child->getTreeHash(); - UInt64 first_half_of_hash = hash.first; + const IAST::Hash hash = child->getTreeHash(); + const auto & first_half_of_hash = hash.low64; if (!unique_index_expression_hashes.emplace(first_half_of_hash).second) throw Exception(ErrorCodes::BAD_ARGUMENTS, diff --git a/src/Storages/System/StorageSystemQueryCache.cpp b/src/Storages/System/StorageSystemQueryCache.cpp index 117fb4e8a5c..b6ab2c54407 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().first); + res_columns[6]->insert(key.ast->getTreeHash().low64); } } diff --git a/src/Storages/buildQueryTreeForShard.cpp b/src/Storages/buildQueryTreeForShard.cpp index 1ee7d747fcc..18ec8e48c3e 100644 --- a/src/Storages/buildQueryTreeForShard.cpp +++ b/src/Storages/buildQueryTreeForShard.cpp @@ -232,8 +232,8 @@ TableNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, ContextMutablePtr & mutable_context, size_t subquery_depth) { - auto subquery_hash = subquery_node->getTreeHash(); - String temporary_table_name = fmt::format("_data_{}_{}", subquery_hash.first, subquery_hash.second); + const auto subquery_hash = subquery_node->getTreeHash(); + const auto temporary_table_name = fmt::format("_data_{}", toString(subquery_hash)); const auto & external_tables = mutable_context->getExternalTables(); auto external_table_it = external_tables.find(temporary_table_name);