From 51e2c58a539a31a6268f2d4f50fd91701caaa915 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Wed, 21 Jun 2023 20:15:12 -0700 Subject: [PATCH] Implement endianness-independent SipHash and MergeTree checksum serialization --- programs/obfuscator/Obfuscator.cpp | 13 +- .../AggregateFunctionUniq.h | 3 +- src/AggregateFunctions/UniqVariadicHash.h | 8 +- src/Analyzer/IQueryTreeNode.cpp | 5 +- src/Client/QueryFuzzer.cpp | 3 +- src/Columns/ColumnUnique.h | 3 +- src/Common/SipHash.h | 112 +++++++++--------- src/Common/TransformEndianness.hpp | 3 +- src/Common/examples/hashes_test.cpp | 3 +- src/Common/getHashOfLoadedBinary.cpp | 3 +- src/Common/randomSeed.cpp | 2 +- src/DataTypes/Serializations/PathInData.cpp | 4 +- src/Functions/PolygonUtils.h | 4 +- src/Functions/array/arrayDistinct.cpp | 3 +- src/Functions/array/arrayEnumerateRanked.h | 6 +- src/IO/MMappedFileCache.h | 5 +- src/IO/UncompressedCache.h | 5 +- src/Interpreters/AggregationCommon.h | 6 +- src/Interpreters/AsynchronousInsertQueue.cpp | 4 +- src/Parsers/IAST.cpp | 4 +- .../Impl/ConstantExpressionTemplate.cpp | 3 +- .../Transforms/LimitByTransform.cpp | 5 +- src/Storages/LiveView/LiveViewSink.h | 7 +- src/Storages/LiveView/StorageLiveView.cpp | 3 +- src/Storages/MarkCache.h | 6 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 10 +- src/Storages/MergeTree/MarkRange.cpp | 12 +- .../MergeTree/MergeTreeDataPartChecksum.cpp | 15 +-- .../MergeTree/MergeTreeDataPartCompact.cpp | 2 +- .../MergeTree/MergeTreeDataPartInMemory.cpp | 2 +- .../MergeTree/MergeTreeDataPartWide.cpp | 6 +- .../MergeTreeDataPartWriterCompact.cpp | 12 +- .../MergeTreeDataPartWriterOnDisk.cpp | 6 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 12 +- .../MergeTree/MergeTreeMarksLoader.cpp | 12 +- src/Storages/MergeTree/MergeTreePartition.cpp | 10 +- .../ReplicatedMergeTreePartHeader.cpp | 4 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 9 +- 38 files changed, 138 insertions(+), 197 deletions(-) diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 3042ae2bb57..43b31843afe 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -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(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(&src), sizeof(UUID)); /// Saving version and variant from an old UUID - hash.get128(reinterpret_cast(&dst)); + // hash.get128Impl(reinterpret_cast(&dst)); dst.items[1] = (dst.items[1] & 0x1fffffffffffffffull) | (src.items[1] & 0xe000000000000000ull); dst.items[0] = (dst.items[0] & 0xffffffffffff0fffull) | (src.items[0] & 0x000000000000f000ull); diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index de68e9076a0..8310f02767c 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -298,10 +298,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(key); } diff --git a/src/AggregateFunctions/UniqVariadicHash.h b/src/AggregateFunctions/UniqVariadicHash.h index 94f54a7a059..840380e7f0f 100644 --- a/src/AggregateFunctions/UniqVariadicHash.h +++ b/src/AggregateFunctions/UniqVariadicHash.h @@ -107,9 +107,7 @@ struct UniqVariadicHash ++column; } - UInt128 key; - hash.get128(key); - return key; + return hash.get128(); } }; @@ -131,9 +129,7 @@ struct UniqVariadicHash ++column; } - UInt128 key; - hash.get128(key); - return key; + return hash.get128(); } }; diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index f1056975f7f..7e5d778f1c1 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -229,10 +229,7 @@ IQueryTreeNode::Hash IQueryTreeNode::getTreeHash() const } } - Hash result; - hash_state.get128(result); - - return result; + return getSipHash128AsLoHi(hash_state); } QueryTreeNodePtr IQueryTreeNode::clone() const diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index 5ce95c82528..f20c869e119 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -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 = getSipHash128AsLoHi(sip_hash); /// Save only tables with unique definition. if (created_tables_hashes.insert(hash).second) diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index 377255d80c7..fd0144cba12 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -670,8 +670,9 @@ UInt128 ColumnUnique::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); } diff --git a/src/Common/SipHash.h b/src/Common/SipHash.h index cdec00d4bcc..e9d45c7201f 100644 --- a/src/Common/SipHash.h +++ b/src/Common/SipHash.h @@ -13,6 +13,8 @@ * (~ 700 MB/sec, 15 million strings per second) */ +#include "TransformEndianness.hpp" + #include #include #include @@ -22,14 +24,10 @@ #include #include - -namespace DB -{ -namespace ErrorCodes +namespace DB::ErrorCodes { extern const int LOGICAL_ERROR; } -} #define SIPROUND \ do \ @@ -90,6 +88,20 @@ private: SIPROUND; } + /// @brief Retrieves the result in some form with the endianness of the platform taken into account. + /// @warning This can only be done once! + void get128Impl(char * out) + { + finalize(); +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + unalignedStore(out + 8, v0 ^ v1); + unalignedStore(out, v2 ^ v3); +#else + unalignedStore(out, v0 ^ v1); + unalignedStore(out + 8, v2 ^ v3); +#endif + } + public: /// Arguments - seed. SipHash(UInt64 key0 = 0, UInt64 key1 = 0, bool is_reference_128_ = false) /// NOLINT @@ -161,60 +173,26 @@ public: } } - template + template ALWAYS_INLINE void update(const T & x) { if constexpr (std::endian::native == std::endian::big) { - T rev_x = x; - char *start = reinterpret_cast(&rev_x); - char *end = start + sizeof(T); - std::reverse(start, end); - update(reinterpret_cast(&rev_x), sizeof(rev_x)); /// NOLINT + auto transformed_x = x; + if constexpr (!std::is_same_v) + transformed_x = Transform()(x); + else + DB::transformEndianness(transformed_x); + + update(reinterpret_cast(&transformed_x), sizeof(transformed_x)); /// NOLINT } else update(reinterpret_cast(&x), sizeof(x)); /// NOLINT } - 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()); - } - - /// 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(out + 8, v0 ^ v1); - unalignedStore(out, v2 ^ v3); -#else - unalignedStore(out, v0 ^ v1); - unalignedStore(out + 8, v2 ^ v3); -#endif - } - - template - ALWAYS_INLINE void get128(T & lo, T & hi) - { - static_assert(sizeof(T) == 8); - finalize(); - lo = v0 ^ v1; - hi = v2 ^ v3; - } - - template - ALWAYS_INLINE void get128(T & dst) - { - static_assert(sizeof(T) == 16); - get128(reinterpret_cast(&dst)); - } + 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)); } UInt64 get64() { @@ -222,10 +200,23 @@ public: return v0 ^ v1 ^ v2 ^ v3; } + template + requires (sizeof(T) == 8) + ALWAYS_INLINE void get128(T & lo, T & hi) + { + finalize(); + lo = v0 ^ v1; + hi = v2 ^ v3; + } + 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 +238,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 +254,18 @@ public: #include -inline void sipHash128(const char * data, const size_t size, char * out) +inline std::array getSipHash128AsArray(SipHash & sip_hash) { - SipHash hash; - hash.update(data, size); - hash.get128(out); + std::array arr; + *reinterpret_cast(arr.data()) = sip_hash.get128(); + return arr; +} + +inline std::pair getSipHash128AsLoHi(SipHash & sip_hash) +{ + std::pair lo_hi; + sip_hash.get128(lo_hi.first, lo_hi.second); + return lo_hi; } 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 0a9055dde15..ccfec78ab64 100644 --- a/src/Common/TransformEndianness.hpp +++ b/src/Common/TransformEndianness.hpp @@ -2,6 +2,7 @@ #include #include +#include #include @@ -46,7 +47,7 @@ inline void transformEndianness(T & value) } template -requires std::is_scoped_enum_v +requires std::is_enum_v || std::is_scoped_enum_v inline void transformEndianness(T & x) { using UnderlyingType = std::underlying_type_t; diff --git a/src/Common/examples/hashes_test.cpp b/src/Common/examples/hashes_test.cpp index eccf7c9b2e6..99479e79302 100644 --- a/src/Common/examples/hashes_test.cpp +++ b/src/Common/examples/hashes_test.cpp @@ -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(); diff --git a/src/Common/getHashOfLoadedBinary.cpp b/src/Common/getHashOfLoadedBinary.cpp index b81300b8536..6487bcd4f1c 100644 --- a/src/Common/getHashOfLoadedBinary.cpp +++ b/src/Common/getHashOfLoadedBinary.cpp @@ -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); } diff --git a/src/Common/randomSeed.cpp b/src/Common/randomSeed.cpp index 9f0ffd8a6c7..e1aa56fa811 100644 --- a/src/Common/randomSeed.cpp +++ b/src/Common/randomSeed.cpp @@ -39,7 +39,7 @@ DB::UInt64 randomSeed() #if defined(__linux__) struct utsname sysinfo; if (uname(&sysinfo) == 0) - hash.update(sysinfo); + hash.update(sysinfo); #endif return hash.get64(); diff --git a/src/DataTypes/Serializations/PathInData.cpp b/src/DataTypes/Serializations/PathInData.cpp index 2a5f508650f..cf78d7cbb14 100644 --- a/src/DataTypes/Serializations/PathInData.cpp +++ b/src/DataTypes/Serializations/PathInData.cpp @@ -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) diff --git a/src/Functions/PolygonUtils.h b/src/Functions/PolygonUtils.h index 9aae42cce41..9c28e349413 100644 --- a/src/Functions/PolygonUtils.h +++ b/src/Functions/PolygonUtils.h @@ -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(); } } diff --git a/src/Functions/array/arrayDistinct.cpp b/src/Functions/array/arrayDistinct.cpp index 527624794ea..ea331d6bdad 100644 --- a/src/Functions/array/arrayDistinct.cpp +++ b/src/Functions/array/arrayDistinct.cpp @@ -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)) { diff --git a/src/Functions/array/arrayEnumerateRanked.h b/src/Functions/array/arrayEnumerateRanked.h index 0733f1e2d43..3fc31e46f5e 100644 --- a/src/Functions/array/arrayEnumerateRanked.h +++ b/src/Functions/array/arrayEnumerateRanked.h @@ -134,18 +134,14 @@ private: /// Hash a set of keys into a UInt128 value. static inline UInt128 ALWAYS_INLINE hash128depths(const std::vector & 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(); } diff --git a/src/IO/MMappedFileCache.h b/src/IO/MMappedFileCache.h index 0a8a80d15d0..bb30829ed69 100644 --- a/src/IO/MMappedFileCache.h +++ b/src/IO/MMappedFileCache.h @@ -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 diff --git a/src/IO/UncompressedCache.h b/src/IO/UncompressedCache.h index 2e654b27ed7..b115a18014f 100644 --- a/src/IO/UncompressedCache.h +++ b/src/IO/UncompressedCache.h @@ -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 diff --git a/src/Interpreters/AggregationCommon.h b/src/Interpreters/AggregationCommon.h index 7ba9011f18b..77ca2c49b31 100644 --- a/src/Interpreters/AggregationCommon.h +++ b/src/Interpreters/AggregationCommon.h @@ -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. diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 00347663fbd..d174c626dd6 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -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 diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 0138372ce89..7c85b63ceff 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -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 getSipHash128AsLoHi(hash_state); } diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 5d438d47de6..2d5264fa4e3 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -369,8 +369,7 @@ 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 = getSipHash128AsLoHi(hash_state); size_t res = 0; boost::hash_combine(res, res128.first); boost::hash_combine(res, res128.second); diff --git a/src/Processors/Transforms/LimitByTransform.cpp b/src/Processors/Transforms/LimitByTransform.cpp index cb2804007bd..5e6d7dc816a 100644 --- a/src/Processors/Transforms/LimitByTransform.cpp +++ b/src/Processors/Transforms/LimitByTransform.cpp @@ -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::max() - group_offset || count < group_length + group_offset)) diff --git a/src/Storages/LiveView/LiveViewSink.h b/src/Storages/LiveView/LiveViewSink.h index e163400f2af..792133ced64 100644 --- a/src/Storages/LiveView/LiveViewSink.h +++ b/src/Storages/LiveView/LiveViewSink.h @@ -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); diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 173bb128c4a..5719529533e 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -681,7 +681,6 @@ QueryPipelineBuilder StorageLiveView::completeQuery(Pipes pipes) bool StorageLiveView::getNewBlocks(const std::lock_guard & lock) { SipHash hash; - UInt128 key; BlocksPtr new_blocks = std::make_shared(); BlocksMetadataPtr new_blocks_metadata = std::make_shared(); @@ -713,7 +712,7 @@ bool StorageLiveView::getNewBlocks(const std::lock_guard & 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 diff --git a/src/Storages/MarkCache.h b/src/Storages/MarkCache.h index 0f4af57fc8e..495cebb080f 100644 --- a/src/Storages/MarkCache.h +++ b/src/Storages/MarkCache.h @@ -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 diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 7050a98a4bc..91069d30c03 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -2052,14 +2052,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 diff --git a/src/Storages/MergeTree/MarkRange.cpp b/src/Storages/MergeTree/MarkRange.cpp index c6c197919f4..bd8546f04cc 100644 --- a/src/Storages/MergeTree/MarkRange.cpp +++ b/src/Storages/MergeTree/MarkRange.cpp @@ -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); } } diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index 6628cd68eaf..d09cae7870f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -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(&ret)); - return ret; + return getSipHash128AsLoHi(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 = getSipHash128AsLoHi(hash_of_all_files_state); + hash_of_uncompressed_files = getSipHash128AsLoHi(hash_of_uncompressed_files_state); + uncompressed_hash_of_compressed_files = getSipHash128AsLoHi(uncompressed_hash_of_compressed_files_state); } String MinimalisticDataPartChecksums::getSerializedString(const MergeTreeDataPartChecksums & full_checksums, bool minimalistic) diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 9c47608e364..70e5ee05d2e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -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); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 468747a6c36..a8d38a1bff8 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -167,7 +167,7 @@ IMergeTreeDataPart::Checksum MergeTreeDataPartInMemory::calculateBlockChecksum() column.column->updateHashFast(hash); checksum.uncompressed_size = block.bytes(); - hash.get128(checksum.uncompressed_hash); + checksum.uncompressed_hash = getSipHash128AsLoHi(hash); return checksum; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 2d886e2058b..846442fd3e0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -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); } } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 5e1da21da5b..9cd297c4e80 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -228,8 +228,8 @@ void MergeTreeDataPartWriterCompact::writeDataBlock(const Block & block, const G }; - writeIntBinary(plain_hashing.count(), marks_out); - writeIntBinary(static_cast(0), marks_out); + writeBinaryLittleEndian(plain_hashing.count(), marks_out); + writeBinaryLittleEndian(static_cast(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(0), marks_out); + writeBinaryLittleEndian(plain_hashing.count(), marks_out); + writeBinaryLittleEndian(static_cast(0), marks_out); } - writeIntBinary(static_cast(0), marks_out); + writeBinaryLittleEndian(static_cast(0), marks_out); } for (const auto & [_, stream] : streams_by_codec) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index f57ffa5ee14..79b72d4ae39 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -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; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index f9fe6f2c8ab..bcf340e0f55 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -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; diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp index 5c722eec380..c9b22c8a03e 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp @@ -160,7 +160,7 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl() size_t granularity; reader->readStrict( reinterpret_cast(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(plain_marks); ProfileEvents::increment(ProfileEvents::LoadedMarksCount, marks_count * columns_in_mark); diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index bce33438229..63efd137b0b 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -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 diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartHeader.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartHeader.cpp index 88f4a3ec66f..24d907dbad6 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartHeader.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartHeader.cpp @@ -12,9 +12,7 @@ static std::array getSipHash(const String & str) { SipHash hash; hash.update(str.data(), str.size()); - std::array result; - hash.get128(result.data()); - return result; + return getSipHash128AsArray(hash); } ReplicatedMergeTreePartHeader ReplicatedMergeTreePartHeader::fromColumnsAndChecksumsZNodes( diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 393d418c083..1d0acd782b3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -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));