Merge pull request #50276 from ClibMouse/feature/mergetree-checksum-big-endian-support

MergeTree & SipHash checksum big-endian support
This commit is contained in:
Robert Schulze 2023-08-17 13:27:18 +02:00 committed by GitHub
commit ec7daed52f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
62 changed files with 178 additions and 247 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

@ -365,17 +365,14 @@ static void transformFixedString(const UInt8 * src, UInt8 * dst, size_t size, UI
hash.update(seed);
hash.update(i);
const auto checksum = getSipHash128AsArray(hash);
if (size >= 16)
{
char * hash_dst = reinterpret_cast<char *>(std::min(pos, end - 16));
hash.get128(hash_dst);
auto * hash_dst = std::min(pos, end - 16);
memcpy(hash_dst, checksum.data(), checksum.size());
}
else
{
char value[16];
hash.get128(value);
memcpy(dst, value, end - dst);
}
memcpy(dst, checksum.data(), end - dst);
pos += 16;
++i;
@ -401,7 +398,7 @@ static void transformUUID(const UUID & src_uuid, UUID & dst_uuid, UInt64 seed)
hash.update(reinterpret_cast<const char *>(&src), sizeof(UUID));
/// Saving version and variant from an old UUID
hash.get128(reinterpret_cast<char *>(&dst));
dst = hash.get128();
dst.items[1] = (dst.items[1] & 0x1fffffffffffffffull) | (src.items[1] & 0xe000000000000000ull);
dst.items[0] = (dst.items[0] & 0xffffffffffff0fffull) | (src.items[0] & 0x000000000000f000ull);

View File

@ -315,10 +315,9 @@ struct Adder
{
StringRef value = column.getDataAt(row_num);
UInt128 key;
SipHash hash;
hash.update(value.data, value.size);
hash.get128(key);
const auto key = hash.get128();
data.set.template insert<const UInt128 &, use_single_level_hash_table>(key);
}

View File

@ -107,9 +107,7 @@ struct UniqVariadicHash<true, false>
++column;
}
UInt128 key;
hash.get128(key);
return key;
return hash.get128();
}
};
@ -131,9 +129,7 @@ struct UniqVariadicHash<true, true>
++column;
}
UInt128 key;
hash.get128(key);
return key;
return hash.get128();
}
};

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

@ -229,10 +229,7 @@ IQueryTreeNode::Hash IQueryTreeNode::getTreeHash() const
}
}
Hash result;
hash_state.get128(result);
return result;
return getSipHash128AsPair(hash_state);
}
QueryTreeNodePtr IQueryTreeNode::clone() const

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

@ -2033,7 +2033,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);
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

@ -521,8 +521,7 @@ void QueryFuzzer::fuzzCreateQuery(ASTCreateQuery & create)
if (create.storage)
create.storage->updateTreeHash(sip_hash);
IAST::Hash hash;
sip_hash.get128(hash);
const auto hash = getSipHash128AsPair(sip_hash);
/// Save only tables with unique definition.
if (created_tables_hashes.insert(hash).second)

View File

@ -670,8 +670,9 @@ UInt128 ColumnUnique<ColumnType>::IncrementalHash::getHash(const ColumnType & co
for (size_t i = 0; i < column_size; ++i)
column.updateHashWithValue(i, sip_hash);
hash = sip_hash.get128();
std::lock_guard lock(mutex);
sip_hash.get128(hash);
cur_hash = hash;
num_added_rows.store(column_size);
}

View File

@ -13,6 +13,8 @@
* (~ 700 MB/sec, 15 million strings per second)
*/
#include "TransformEndianness.hpp"
#include <bit>
#include <string>
#include <type_traits>
@ -22,14 +24,12 @@
#include <base/unaligned.h>
#include <Common/Exception.h>
#include <city.h>
namespace DB
{
namespace ErrorCodes
namespace DB::ErrorCodes
{
extern const int LOGICAL_ERROR;
}
}
#define SIPROUND \
do \
@ -161,71 +161,50 @@ public:
}
}
template <typename T>
template <typename Transform = void, typename T>
ALWAYS_INLINE void update(const T & x)
{
if constexpr (std::endian::native == std::endian::big)
{
T rev_x = x;
char *start = reinterpret_cast<char *>(&rev_x);
char *end = start + sizeof(T);
std::reverse(start, end);
update(reinterpret_cast<const char *>(&rev_x), sizeof(rev_x)); /// NOLINT
auto transformed_x = x;
if constexpr (!std::is_same_v<Transform, void>)
transformed_x = Transform()(x);
else
DB::transformEndianness<std::endian::little>(transformed_x);
update(reinterpret_cast<const char *>(&transformed_x), sizeof(transformed_x)); /// NOLINT
}
else
update(reinterpret_cast<const char *>(&x), sizeof(x)); /// NOLINT
}
ALWAYS_INLINE void update(const std::string & x)
{
update(x.data(), x.length());
}
ALWAYS_INLINE void update(const std::string & x) { update(x.data(), x.length()); }
ALWAYS_INLINE void update(const std::string_view x) { update(x.data(), x.size()); }
ALWAYS_INLINE void update(const char * s) { update(std::string_view(s)); }
ALWAYS_INLINE void update(const std::string_view x)
{
update(x.data(), x.size());
}
/// Get the result in some form. This can only be done once!
ALWAYS_INLINE void get128(char * out)
{
finalize();
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
unalignedStore<UInt64>(out + 8, v0 ^ v1);
unalignedStore<UInt64>(out, v2 ^ v3);
#else
unalignedStore<UInt64>(out, v0 ^ v1);
unalignedStore<UInt64>(out + 8, v2 ^ v3);
#endif
}
template <typename T>
ALWAYS_INLINE void get128(T & lo, T & hi)
{
static_assert(sizeof(T) == 8);
finalize();
lo = v0 ^ v1;
hi = v2 ^ v3;
}
template <typename T>
ALWAYS_INLINE void get128(T & dst)
{
static_assert(sizeof(T) == 16);
get128(reinterpret_cast<char *>(&dst));
}
UInt64 get64()
ALWAYS_INLINE UInt64 get64()
{
finalize();
return v0 ^ v1 ^ v2 ^ v3;
}
UInt128 get128()
template <typename T>
requires (sizeof(T) == 8)
ALWAYS_INLINE void get128(T & lo, T & hi)
{
finalize();
lo = v0 ^ v1;
hi = v2 ^ v3;
}
ALWAYS_INLINE UInt128 get128()
{
UInt128 res;
get128(res);
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
get128(res.items[1], res.items[0]);
#else
get128(res.items[0], res.items[1]);
#endif
return res;
}
@ -247,9 +226,7 @@ public:
{
lo = std::byteswap(lo);
hi = std::byteswap(hi);
auto tmp = hi;
hi = lo;
lo = tmp;
std::swap(lo, hi);
}
UInt128 res = hi;
@ -265,11 +242,18 @@ public:
#include <cstddef>
inline void sipHash128(const char * data, const size_t size, char * out)
inline std::array<char, 16> getSipHash128AsArray(SipHash & sip_hash)
{
SipHash hash;
hash.update(data, size);
hash.get128(out);
std::array<char, 16> arr;
*reinterpret_cast<UInt128*>(arr.data()) = sip_hash.get128();
return arr;
}
inline CityHash_v1_0_2::uint128 getSipHash128AsPair(SipHash & sip_hash)
{
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)
@ -309,7 +293,7 @@ inline UInt64 sipHash64(const char * data, const size_t size)
}
template <typename T>
UInt64 sipHash64(const T & x)
inline UInt64 sipHash64(const T & x)
{
SipHash hash;
hash.update(x);

View File

@ -2,6 +2,7 @@
#include <base/Decimal_fwd.h>
#include <base/extended_types.h>
#include <base/strong_typedef.h>
#include <city.h>
@ -48,7 +49,7 @@ inline void transformEndianness(T & value)
}
template <std::endian ToEndian, std::endian FromEndian = std::endian::native, typename T>
requires std::is_scoped_enum_v<T>
requires std::is_enum_v<T> || std::is_scoped_enum_v<T>
inline void transformEndianness(T & x)
{
using UnderlyingType = std::underlying_type_t<T>;

View File

@ -94,7 +94,8 @@ int main(int, char **)
{
SipHash hash;
hash.update(strings[i].data(), strings[i].size());
hash.get128(&hashes[i * 16]);
const auto hashed_value = getSipHash128AsArray(hash);
memcpy(&hashes[i * 16], hashed_value.data(), hashed_value.size());
}
watch.stop();

View File

@ -37,8 +37,7 @@ SipHash getHashOfLoadedBinary()
std::string getHashOfLoadedBinaryHex()
{
SipHash hash = getHashOfLoadedBinary();
UInt128 checksum;
hash.get128(checksum);
const auto checksum = hash.get128();
return getHexUIntUppercase(checksum);
}

View File

@ -39,7 +39,7 @@ DB::UInt64 randomSeed()
#if defined(__linux__)
struct utsname sysinfo;
if (uname(&sysinfo) == 0)
hash.update(sysinfo);
hash.update<std::identity>(sysinfo);
#endif
return hash.get64();

View File

@ -133,8 +133,7 @@ void compileSortDescriptionIfNeeded(SortDescription & description, const DataTyp
SipHash sort_description_dump_hash;
sort_description_dump_hash.update(description_dump);
UInt128 sort_description_hash_key;
sort_description_dump_hash.get128(sort_description_hash_key);
const auto sort_description_hash_key = sort_description_dump_hash.get128();
{
std::lock_guard lock(mutex);

View File

@ -65,9 +65,7 @@ UInt128 PathInData::getPartsHash(const Parts::const_iterator & begin, const Part
hash.update(part_it->anonymous_array_level);
}
UInt128 res;
hash.get128(res);
return res;
return hash.get128();
}
void PathInData::buildPath(const Parts & other_parts)

View File

@ -635,9 +635,7 @@ UInt128 sipHash128(Polygon && polygon)
for (auto & inner : inners)
hash_ring(inner);
UInt128 res;
hash.get128(res);
return res;
return hash.get128();
}
}

View File

@ -268,10 +268,9 @@ void FunctionArrayDistinct::executeHashed(
if (nullable_col && (*src_null_map)[j])
continue;
UInt128 hash;
SipHash hash_function;
src_data.updateHashWithValue(j, hash_function);
hash_function.get128(hash);
const auto hash = hash_function.get128();
if (!set.find(hash))
{

View File

@ -133,18 +133,14 @@ private:
/// Hash a set of keys into a UInt128 value.
static inline UInt128 ALWAYS_INLINE hash128depths(const std::vector<size_t> & indices, const ColumnRawPtrs & key_columns)
{
UInt128 key;
SipHash hash;
for (size_t j = 0, keys_size = key_columns.size(); j < keys_size; ++j)
{
// Debug: const auto & field = (*key_columns[j])[indices[j]]; DUMP(j, indices[j], field);
key_columns[j]->updateHashWithValue(indices[j], hash);
}
hash.get128(key);
return key;
return hash.get128();
}

View File

@ -33,15 +33,12 @@ public:
/// Calculate key from path to file and offset.
static UInt128 hash(const String & path_to_file, size_t offset, ssize_t length = -1)
{
UInt128 key;
SipHash hash;
hash.update(path_to_file.data(), path_to_file.size() + 1);
hash.update(offset);
hash.update(length);
hash.get128(key);
return key;
return hash.get128();
}
template <typename LoadFunc>

View File

@ -51,14 +51,11 @@ public:
/// Calculate key from path to file and offset.
static UInt128 hash(const String & path_to_file, size_t offset)
{
UInt128 key;
SipHash hash;
hash.update(path_to_file.data(), path_to_file.size() + 1);
hash.update(offset);
hash.get128(key);
return key;
return hash.get128();
}
template <typename LoadFunc>

View File

@ -1170,6 +1170,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

@ -253,15 +253,11 @@ static inline T ALWAYS_INLINE packFixed(
static inline UInt128 ALWAYS_INLINE hash128( /// NOLINT
size_t i, size_t keys_size, const ColumnRawPtrs & key_columns)
{
UInt128 key;
SipHash hash;
for (size_t j = 0; j < keys_size; ++j)
key_columns[j]->updateHashWithValue(i, hash);
hash.get128(key);
return key;
return hash.get128();
}
/** Serialize keys into a continuous chunk of memory.

View File

@ -694,8 +694,7 @@ void Aggregator::compileAggregateFunctionsIfNeeded()
SipHash aggregate_functions_description_hash;
aggregate_functions_description_hash.update(functions_description);
UInt128 aggregate_functions_description_hash_key;
aggregate_functions_description_hash.get128(aggregate_functions_description_hash_key);
const auto aggregate_functions_description_hash_key = aggregate_functions_description_hash.get128();
{
std::lock_guard<std::mutex> lock(mutex);

View File

@ -105,9 +105,7 @@ UInt128 AsynchronousInsertQueue::InsertQuery::calculateHash() const
applyVisitor(FieldVisitorHash(siphash), setting.getValue());
}
UInt128 res;
siphash.get128(res);
return res;
return siphash.get128();
}
bool AsynchronousInsertQueue::InsertQuery::operator==(const InsertQuery & other) const

View File

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

View File

@ -1561,7 +1561,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)
{
@ -1712,7 +1712,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;
}
}
@ -1721,8 +1721,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

@ -160,9 +160,7 @@ UInt128 CompileDAG::hash() const
}
}
UInt128 result;
hash.get128(result);
return result;
return hash.get128();
}
}

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

@ -409,8 +409,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);
}
}
@ -419,8 +419,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

@ -69,9 +69,9 @@ void ASTSetQuery::appendColumnName(WriteBuffer & ostr) const
Hash hash = getTreeHash();
writeCString("__settings_", ostr);
writeText(hash.first, ostr);
writeText(hash.low64, ostr);
ostr.write('_');
writeText(hash.second, ostr);
writeText(hash.high64, ostr);
}
}

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

@ -118,9 +118,7 @@ IAST::Hash IAST::getTreeHash() const
{
SipHash hash_state;
updateTreeHash(hash_state);
IAST::Hash res;
hash_state.get128(res);
return res;
return getSipHash128AsPair(hash_state);
}

View File

@ -79,7 +79,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

@ -166,7 +166,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

@ -115,8 +115,7 @@ const ColumnIdentifier * PlannerContext::getColumnNodeIdentifierOrNull(const Que
PlannerContext::SetKey PlannerContext::createSetKey(const DataTypePtr & left_operand_type, const QueryTreeNodePtr & set_source_node)
{
auto set_source_hash = set_source_node->getTreeHash();
const auto set_source_hash = set_source_node->getTreeHash();
if (set_source_node->as<ConstantNode>())
{
/* We need to hash the type of the left operand because we can build different sets for different types.
@ -127,11 +126,11 @@ PlannerContext::SetKey PlannerContext::createSetKey(const DataTypePtr & left_ope
* - `{1, 2.5} :: Set(Decimal(9, 1))` for a
* - `{1} :: Set(Decimal(9, 0))` for b (2.5 omitted because bercause it's not representable as Decimal(9, 0)).
*/
return "__set_" + left_operand_type->getName() + '_' + toString(set_source_hash.first) + '_' + toString(set_source_hash.second);
return "__set_" + left_operand_type->getName() + '_' + toString(set_source_hash);
}
/// For other cases we will cast left operand to the type of the set source, so no difference in types.
return "__set_" + toString(set_source_hash.first) + '_' + toString(set_source_hash.second);
return "__set_" + toString(set_source_hash);
}
}

View File

@ -402,11 +402,10 @@ size_t ConstantExpressionTemplate::TemplateStructure::getTemplateHash(const ASTP
/// Allows distinguish expression in the last column in Values format
hash_state.update(salt);
IAST::Hash res128;
hash_state.get128(res128);
const auto res128 = getSipHash128AsPair(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

@ -33,14 +33,11 @@ void LimitByTransform::transform(Chunk & chunk)
for (UInt64 row = 0; row < num_rows; ++row)
{
UInt128 key{};
SipHash hash;
for (auto position : key_positions)
columns[position]->updateHashWithValue(row, hash);
hash.get128(key);
const auto key = hash.get128();
auto count = keys_counts[key]++;
if (count >= group_offset
&& (group_length > std::numeric_limits<UInt64>::max() - group_offset || count < group_length + group_offset))

View File

@ -32,11 +32,8 @@ public:
void onFinish() override
{
UInt128 key;
String key_str;
new_hash->get128(key);
key_str = getHexUIntLowercase(key);
const auto key = new_hash->get128();
const auto key_str = getHexUIntLowercase(key);
std::lock_guard lock(storage.mutex);

View File

@ -681,7 +681,6 @@ QueryPipelineBuilder StorageLiveView::completeQuery(Pipes pipes)
bool StorageLiveView::getNewBlocks(const std::lock_guard<std::mutex> & lock)
{
SipHash hash;
UInt128 key;
BlocksPtr new_blocks = std::make_shared<Blocks>();
BlocksMetadataPtr new_blocks_metadata = std::make_shared<BlocksMetadata>();
@ -713,7 +712,7 @@ bool StorageLiveView::getNewBlocks(const std::lock_guard<std::mutex> & lock)
new_blocks->push_back(block);
}
hash.get128(key);
const auto key = hash.get128();
/// Update blocks only if hash keys do not match
/// NOTE: hash could be different for the same result

View File

@ -50,13 +50,9 @@ public:
/// Calculate key from path to file and offset.
static UInt128 hash(const String & path_to_file)
{
UInt128 key;
SipHash hash;
hash.update(path_to_file.data(), path_to_file.size() + 1);
hash.get128(key);
return key;
return hash.get128();
}
template <typename LoadFunc>

View File

@ -2060,14 +2060,8 @@ String IMergeTreeDataPart::getZeroLevelPartBlockID(std::string_view token) const
hash.update(token.data(), token.size());
}
union
{
char bytes[16];
UInt64 words[2];
} hash_value;
hash.get128(hash_value.bytes);
return info.partition_id + "_" + toString(hash_value.words[0]) + "_" + toString(hash_value.words[1]);
const auto hash_value = hash.get128();
return info.partition_id + "_" + toString(hash_value.items[0]) + "_" + toString(hash_value.items[1]);
}
IMergeTreeDataPart::uint128 IMergeTreeDataPart::getActualChecksumByFile(const String & file_name) const

View File

@ -83,12 +83,12 @@ size_t MarkRanges::getNumberOfMarks() const
void MarkRanges::serialize(WriteBuffer & out) const
{
writeIntBinary(this->size(), out);
writeBinaryLittleEndian(this->size(), out);
for (const auto & [begin, end] : *this)
{
writeIntBinary(begin, out);
writeIntBinary(end, out);
writeBinaryLittleEndian(begin, out);
writeBinaryLittleEndian(end, out);
}
}
@ -100,13 +100,13 @@ String MarkRanges::describe() const
void MarkRanges::deserialize(ReadBuffer & in)
{
size_t size = 0;
readIntBinary(size, in);
readBinaryLittleEndian(size, in);
this->resize(size);
for (size_t i = 0; i < size; ++i)
{
readIntBinary((*this)[i].begin, in);
readIntBinary((*this)[i].end, in);
readBinaryLittleEndian((*this)[i].begin, in);
readBinaryLittleEndian((*this)[i].end, in);
}
}

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

@ -323,9 +323,7 @@ MergeTreeDataPartChecksums::Checksum::uint128 MergeTreeDataPartChecksums::getTot
hash_of_all_files.update(checksum.file_hash);
}
MergeTreeDataPartChecksums::Checksum::uint128 ret;
hash_of_all_files.get128(reinterpret_cast<char *>(&ret));
return ret;
return getSipHash128AsPair(hash_of_all_files);
}
void MinimalisticDataPartChecksums::serialize(WriteBuffer & to) const
@ -415,14 +413,9 @@ void MinimalisticDataPartChecksums::computeTotalChecksums(const MergeTreeDataPar
}
}
auto get_hash = [] (SipHash & hash, uint128 & data)
{
hash.get128(data);
};
get_hash(hash_of_all_files_state, hash_of_all_files);
get_hash(hash_of_uncompressed_files_state, hash_of_uncompressed_files);
get_hash(uncompressed_hash_of_compressed_files_state, uncompressed_hash_of_compressed_files);
hash_of_all_files = getSipHash128AsPair(hash_of_all_files_state);
hash_of_uncompressed_files = getSipHash128AsPair(hash_of_uncompressed_files_state);
uncompressed_hash_of_compressed_files = getSipHash128AsPair(uncompressed_hash_of_compressed_files_state);
}
String MinimalisticDataPartChecksums::getSerializedString(const MergeTreeDataPartChecksums & full_checksums, bool minimalistic)

View File

@ -115,7 +115,7 @@ void MergeTreeDataPartCompact::loadIndexGranularityImpl(
{
marks_reader->ignore(columns_count * sizeof(MarkInCompressedFile));
size_t granularity;
readIntBinary(granularity, *marks_reader);
readBinaryLittleEndian(granularity, *marks_reader);
index_granularity_.appendMark(granularity);
}

View File

@ -172,7 +172,7 @@ IMergeTreeDataPart::Checksum MergeTreeDataPartInMemory::calculateBlockChecksum()
column.column->updateHashFast(hash);
checksum.uncompressed_size = block.bytes();
hash.get128(checksum.uncompressed_hash);
checksum.uncompressed_hash = getSipHash128AsPair(hash);
return checksum;
}

View File

@ -130,13 +130,13 @@ void MergeTreeDataPartWide::loadIndexGranularityImpl(
MarkInCompressedFile mark;
size_t granularity;
readBinary(mark.offset_in_compressed_file, *marks_reader);
readBinary(mark.offset_in_decompressed_block, *marks_reader);
readBinaryLittleEndian(mark.offset_in_compressed_file, *marks_reader);
readBinaryLittleEndian(mark.offset_in_decompressed_block, *marks_reader);
++marks_count;
if (index_granularity_info_.mark_type.adaptive)
{
readIntBinary(granularity, *marks_reader);
readBinaryLittleEndian(granularity, *marks_reader);
index_granularity_.appendMark(granularity);
}
}

View File

@ -228,8 +228,8 @@ void MergeTreeDataPartWriterCompact::writeDataBlock(const Block & block, const G
};
writeIntBinary(plain_hashing.count(), marks_out);
writeIntBinary(static_cast<UInt64>(0), marks_out);
writeBinaryLittleEndian(plain_hashing.count(), marks_out);
writeBinaryLittleEndian(static_cast<UInt64>(0), marks_out);
writeColumnSingleGranule(
block.getByName(name_and_type->name), data_part->getSerialization(name_and_type->name),
@ -239,7 +239,7 @@ void MergeTreeDataPartWriterCompact::writeDataBlock(const Block & block, const G
prev_stream->hashing_buf.next();
}
writeIntBinary(granule.rows_to_write, marks_out);
writeBinaryLittleEndian(granule.rows_to_write, marks_out);
}
}
@ -270,10 +270,10 @@ void MergeTreeDataPartWriterCompact::fillDataChecksums(IMergeTreeDataPart::Check
{
for (size_t i = 0; i < columns_list.size(); ++i)
{
writeIntBinary(plain_hashing.count(), marks_out);
writeIntBinary(static_cast<UInt64>(0), marks_out);
writeBinaryLittleEndian(plain_hashing.count(), marks_out);
writeBinaryLittleEndian(static_cast<UInt64>(0), marks_out);
}
writeIntBinary(static_cast<UInt64>(0), marks_out);
writeBinaryLittleEndian(static_cast<UInt64>(0), marks_out);
}
for (const auto & [_, stream] : streams_by_codec)

View File

@ -313,13 +313,13 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block
if (stream.compressed_hashing.offset() >= settings.min_compress_block_size)
stream.compressed_hashing.next();
writeIntBinary(stream.plain_hashing.count(), marks_out);
writeIntBinary(stream.compressed_hashing.offset(), marks_out);
writeBinaryLittleEndian(stream.plain_hashing.count(), marks_out);
writeBinaryLittleEndian(stream.compressed_hashing.offset(), marks_out);
/// Actually this numbers is redundant, but we have to store them
/// to be compatible with the normal .mrk2 file format
if (settings.can_use_adaptive_granularity)
writeIntBinary(1UL, marks_out);
writeBinaryLittleEndian(1UL, marks_out);
}
size_t pos = granule.start_row;

View File

@ -275,10 +275,10 @@ void MergeTreeDataPartWriterWide::flushMarkToFile(const StreamNameAndMark & stre
Stream & stream = *column_streams[stream_with_mark.stream_name];
WriteBuffer & marks_out = stream.compress_marks ? stream.marks_compressed_hashing : stream.marks_hashing;
writeIntBinary(stream_with_mark.mark.offset_in_compressed_file, marks_out);
writeIntBinary(stream_with_mark.mark.offset_in_decompressed_block, marks_out);
writeBinaryLittleEndian(stream_with_mark.mark.offset_in_compressed_file, marks_out);
writeBinaryLittleEndian(stream_with_mark.mark.offset_in_decompressed_block, marks_out);
if (settings.can_use_adaptive_granularity)
writeIntBinary(rows_in_mark, marks_out);
writeBinaryLittleEndian(rows_in_mark, marks_out);
}
StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn(
@ -452,10 +452,10 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai
"Incorrect number of marks in memory {}, on disk (at least) {}",
index_granularity.getMarksCount(), mark_num + 1);
DB::readBinary(offset_in_compressed_file, *mrk_in);
DB::readBinary(offset_in_decompressed_block, *mrk_in);
readBinaryLittleEndian(offset_in_compressed_file, *mrk_in);
readBinaryLittleEndian(offset_in_decompressed_block, *mrk_in);
if (settings.can_use_adaptive_granularity)
DB::readBinary(index_granularity_rows, *mrk_in);
readBinaryLittleEndian(index_granularity_rows, *mrk_in);
else
index_granularity_rows = data_part->index_granularity_info.fixed_index_granularity;

View File

@ -160,7 +160,7 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl()
size_t granularity;
reader->readStrict(
reinterpret_cast<char *>(plain_marks.data() + i * columns_in_mark), columns_in_mark * sizeof(MarkInCompressedFile));
readIntBinary(granularity, *reader);
readBinaryLittleEndian(granularity, *reader);
}
if (!reader->eof())
@ -170,6 +170,16 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl()
mrk_path, marks_count, expected_uncompressed_size);
}
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
std::ranges::for_each(
plain_marks,
[](auto & plain_mark)
{
plain_mark.offset_in_compressed_file = std::byteswap(plain_mark.offset_in_compressed_file);
plain_mark.offset_in_decompressed_block = std::byteswap(plain_mark.offset_in_decompressed_block);
});
#endif
auto res = std::make_shared<MarksInCompressedFile>(plain_marks);
ProfileEvents::increment(ProfileEvents::LoadedMarksCount, marks_count * columns_in_mark);

View File

@ -265,12 +265,12 @@ String MergeTreePartition::getID(const Block & partition_key_sample) const
for (const Field & field : value)
applyVisitor(hashing_visitor, field);
char hash_data[16];
hash.get128(hash_data);
result.resize(32);
for (size_t i = 0; i < 16; ++i)
const auto hash_data = getSipHash128AsArray(hash);
const auto hash_size = hash_data.size();
result.resize(hash_size * 2);
for (size_t i = 0; i < hash_size; ++i)
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
writeHexByteLowercase(hash_data[16 - 1 - i], &result[2 * i]);
writeHexByteLowercase(hash_data[hash_size - 1 - i], &result[2 * i]);
#else
writeHexByteLowercase(hash_data[i], &result[2 * i]);
#endif

View File

@ -12,9 +12,7 @@ static std::array<char, 16> getSipHash(const String & str)
{
SipHash hash;
hash.update(str.data(), str.size());
std::array<char, 16> result;
hash.get128(result.data());
return result;
return getSipHash128AsArray(hash);
}
ReplicatedMergeTreePartHeader ReplicatedMergeTreePartHeader::fromColumnsAndChecksumsZNodes(

View File

@ -254,14 +254,9 @@ namespace
for (const auto & col : cols)
col->updateHashWithValue(j, hash);
}
union
{
char bytes[16];
UInt64 words[2];
} hash_value;
hash.get128(hash_value.bytes);
block_id_vec.push_back(partition_id + "_" + DB::toString(hash_value.words[0]) + "_" + DB::toString(hash_value.words[1]));
const auto hash_value = hash.get128();
block_id_vec.push_back(partition_id + "_" + DB::toString(hash_value.items[0]) + "_" + DB::toString(hash_value.items[1]));
}
else
block_id_vec.push_back(partition_id + "_" + std::string(token));

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