Adapt changes around SipHash

This commit is contained in:
ltrk2 2023-07-05 17:35:44 -07:00
parent 6b96a3943d
commit ba4072f049
23 changed files with 59 additions and 41 deletions

View File

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

View File

@ -20,7 +20,7 @@ struct QueryTreeNodeWithHash
{}
QueryTreeNodePtrType node = nullptr;
std::pair<UInt64, UInt64> hash;
CityHash_v1_0_2::uint128 hash;
};
template <typename T>
@ -55,6 +55,6 @@ struct std::hash<DB::QueryTreeNodeWithHash<T>>
{
size_t operator()(const DB::QueryTreeNodeWithHash<T> & node_with_hash) const
{
return node_with_hash.hash.first;
return node_with_hash.hash.low64;
}
};

View File

@ -106,7 +106,7 @@ public:
*/
bool isEqual(const IQueryTreeNode & rhs, CompareOptions compare_options = { .compare_aliases = true }) const;
using Hash = std::pair<UInt64, UInt64>;
using Hash = CityHash_v1_0_2::uint128;
using HashState = SipHash;
/** Get tree hash identifying current tree

View File

@ -2032,7 +2032,7 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden
auto & nearest_query_scope_query_node = nearest_query_scope->scope_node->as<QueryNode &>();
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);

View File

@ -24,6 +24,8 @@
#include <base/unaligned.h>
#include <Common/Exception.h>
#include <city.h>
namespace DB::ErrorCodes
{
extern const int LOGICAL_ERROR;
@ -247,11 +249,11 @@ inline std::array<char, 16> getSipHash128AsArray(SipHash & sip_hash)
return arr;
}
inline std::pair<UInt64, UInt64> getSipHash128AsLoHi(SipHash & sip_hash)
inline CityHash_v1_0_2::uint128 getSipHash128AsLoHi(SipHash & sip_hash)
{
std::pair<UInt64, UInt64> 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)

View File

@ -4,6 +4,8 @@
#include <base/extended_types.h>
#include <base/strong_typedef.h>
#include <city.h>
#include <utility>
namespace DB
@ -66,4 +68,11 @@ inline void transformEndianness(StrongTypedef<T, Tag> & x)
{
transformEndianness<endian>(x.toUnderType());
}
template <std::endian endian>
inline void transformEndianness(CityHash_v1_0_2::uint128 & x)
{
transformEndianness<endian>(x.low64);
transformEndianness<endian>(x.high64);
}
}

View File

@ -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 <typename T>
inline String toStringWithFinalSeparator(const std::vector<T> & x, const String & final_sep)
{

View File

@ -118,7 +118,7 @@ private:
{
size_t operator() (const IAST::Hash & hash) const
{
return hash.first;
return hash.low64;
}
};

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -127,10 +127,10 @@ class PreparedSets
{
public:
using Hash = std::pair<UInt64, UInt64>;
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<Hash, std::vector<std::shared_ptr<FutureSetFromTuple>>, Hashing>;

View File

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

View File

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

View File

@ -78,7 +78,7 @@ public:
/** Get hash code, identifying this element and its subtree.
*/
using Hash = std::pair<UInt64, UInt64>;
using Hash = CityHash_v1_0_2::uint128;
Hash getTreeHash() const;
void updateTreeHash(SipHash & hash_state) const;
virtual void updateTreeHashImpl(SipHash & hash_state) const;

View File

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

View File

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

View File

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

View File

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

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().first);
res_columns[6]->insert(key.ast->getTreeHash().low64);
}
}

View File

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