Merge pull request #55837 from ClibMouse/feature/big-endian-ip-aggregate-function-states

Correct aggregate functions ser/deserialization to be endianness-independent.
This commit is contained in:
Alexey Milovidov 2023-11-02 14:50:26 +01:00 committed by GitHub
commit b0bd052f8d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 126 additions and 64 deletions

View File

@ -45,6 +45,10 @@ else ()
target_compile_definitions(common PUBLIC WITH_COVERAGE=0)
endif ()
if (TARGET ch_contrib::crc32_s390x)
target_link_libraries(common PUBLIC ch_contrib::crc32_s390x)
endif()
target_include_directories(common PUBLIC .. "${CMAKE_CURRENT_BINARY_DIR}/..")
target_link_libraries (common

View File

@ -35,6 +35,10 @@
#pragma clang diagnostic ignored "-Wreserved-identifier"
#endif
#if defined(__s390x__)
#include <base/crc32c_s390x.h>
#define CRC_INT s390x_crc32c
#endif
/**
* The std::string_view-like container to avoid creating strings to find substrings in the hash table.
@ -264,8 +268,8 @@ inline size_t hashLessThan8(const char * data, size_t size)
if (size >= 4)
{
UInt64 a = unalignedLoad<uint32_t>(data);
return hashLen16(size + (a << 3), unalignedLoad<uint32_t>(data + size - 4));
UInt64 a = unalignedLoadLittleEndian<uint32_t>(data);
return hashLen16(size + (a << 3), unalignedLoadLittleEndian<uint32_t>(data + size - 4));
}
if (size > 0)
@ -285,8 +289,8 @@ inline size_t hashLessThan16(const char * data, size_t size)
{
if (size > 8)
{
UInt64 a = unalignedLoad<UInt64>(data);
UInt64 b = unalignedLoad<UInt64>(data + size - 8);
UInt64 a = unalignedLoadLittleEndian<UInt64>(data);
UInt64 b = unalignedLoadLittleEndian<UInt64>(data + size - 8);
return hashLen16(a, rotateByAtLeast1(b + size, static_cast<UInt8>(size))) ^ b;
}
@ -315,13 +319,13 @@ struct CRC32Hash
do
{
UInt64 word = unalignedLoad<UInt64>(pos);
UInt64 word = unalignedLoadLittleEndian<UInt64>(pos);
res = static_cast<unsigned>(CRC_INT(res, word));
pos += 8;
} while (pos + 8 < end);
UInt64 word = unalignedLoad<UInt64>(end - 8); /// I'm not sure if this is normal.
UInt64 word = unalignedLoadLittleEndian<UInt64>(end - 8); /// I'm not sure if this is normal.
res = static_cast<unsigned>(CRC_INT(res, word));
return res;

26
base/base/crc32c_s390x.h Normal file
View File

@ -0,0 +1,26 @@
#pragma once
#include <crc32-s390x.h>
inline uint32_t s390x_crc32c_u8(uint32_t crc, uint8_t v)
{
return crc32c_le_vx(crc, reinterpret_cast<unsigned char *>(&v), sizeof(v));
}
inline uint32_t s390x_crc32c_u16(uint32_t crc, uint16_t v)
{
v = __builtin_bswap16(v);
return crc32c_le_vx(crc, reinterpret_cast<unsigned char *>(&v), sizeof(v));
}
inline uint32_t s390x_crc32c_u32(uint32_t crc, uint32_t v)
{
v = __builtin_bswap32(v);
return crc32c_le_vx(crc, reinterpret_cast<unsigned char *>(&v), sizeof(v));
}
inline uint64_t s390x_crc32c(uint64_t crc, uint64_t v)
{
v = __builtin_bswap64(v);
return crc32c_le_vx(static_cast<uint32_t>(crc), reinterpret_cast<unsigned char *>(&v), sizeof(uint64_t));
}

View File

@ -95,7 +95,7 @@ public:
size_t size = set.size();
writeVarUInt(size, buf);
for (const auto & elem : set)
writeIntBinary(elem, buf);
writeBinaryLittleEndian(elem.key, buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override

View File

@ -72,14 +72,14 @@ public:
{
writeBinary(has(), buf);
if (has())
writeBinary(value, buf);
writeBinaryLittleEndian(value, buf);
}
void read(ReadBuffer & buf, const ISerialization & /*serialization*/, Arena *)
{
readBinary(has_value, buf);
if (has())
readBinary(value, buf);
readBinaryLittleEndian(value, buf);
}
@ -1275,13 +1275,13 @@ struct AggregateFunctionAnyHeavyData : Data
void write(WriteBuffer & buf, const ISerialization & serialization) const
{
Data::write(buf, serialization);
writeBinary(counter, buf);
writeBinaryLittleEndian(counter, buf);
}
void read(ReadBuffer & buf, const ISerialization & serialization, Arena * arena)
{
Data::read(buf, serialization, arena);
readBinary(counter, buf);
readBinaryLittleEndian(counter, buf);
}
static const char * name() { return "anyHeavy"; }

View File

@ -35,8 +35,8 @@ struct AggregateFunctionMapCombinatorData
using SearchType = KeyType;
std::unordered_map<KeyType, AggregateDataPtr> merged_maps;
static void writeKey(KeyType key, WriteBuffer & buf) { writeBinary(key, buf); }
static void readKey(KeyType & key, ReadBuffer & buf) { readBinary(key, buf); }
static void writeKey(KeyType key, WriteBuffer & buf) { writeBinaryLittleEndian(key, buf); }
static void readKey(KeyType & key, ReadBuffer & buf) { readBinaryLittleEndian(key, buf); }
};
template <>

View File

@ -330,10 +330,6 @@ if (TARGET ch_contrib::cpuid)
target_link_libraries(clickhouse_common_io PRIVATE ch_contrib::cpuid)
endif()
if (TARGET ch_contrib::crc32_s390x)
target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::crc32_s390x)
endif()
if (TARGET ch_contrib::crc32-vpmsum)
target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::crc32-vpmsum)
endif()

View File

@ -54,30 +54,7 @@ inline DB::UInt64 intHash64(DB::UInt64 x)
#endif
#if defined(__s390x__) && __BYTE_ORDER__==__ORDER_BIG_ENDIAN__
#include <crc32-s390x.h>
inline uint32_t s390x_crc32_u8(uint32_t crc, uint8_t v)
{
return crc32c_le_vx(crc, reinterpret_cast<unsigned char *>(&v), sizeof(v));
}
inline uint32_t s390x_crc32_u16(uint32_t crc, uint16_t v)
{
v = std::byteswap(v);
return crc32c_le_vx(crc, reinterpret_cast<unsigned char *>(&v), sizeof(v));
}
inline uint32_t s390x_crc32_u32(uint32_t crc, uint32_t v)
{
v = std::byteswap(v);
return crc32c_le_vx(crc, reinterpret_cast<unsigned char *>(&v), sizeof(v));
}
inline uint64_t s390x_crc32(uint64_t crc, uint64_t v)
{
v = std::byteswap(v);
return crc32c_le_vx(static_cast<uint32_t>(crc), reinterpret_cast<unsigned char *>(&v), sizeof(uint64_t));
}
#include <base/crc32c_s390x.h>
#endif
/// NOTE: Intel intrinsic can be confusing.
@ -92,7 +69,7 @@ inline DB::UInt64 intHashCRC32(DB::UInt64 x)
#elif (defined(__PPC64__) || defined(__powerpc64__)) && __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
return crc32_ppc(-1U, reinterpret_cast<const unsigned char *>(&x), sizeof(x));
#elif defined(__s390x__) && __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
return s390x_crc32(-1U, x);
return s390x_crc32c(-1U, x);
#else
/// On other platforms we do not have CRC32. NOTE This can be confusing.
/// NOTE: consider using intHash32()
@ -108,7 +85,7 @@ inline DB::UInt64 intHashCRC32(DB::UInt64 x, DB::UInt64 updated_value)
#elif (defined(__PPC64__) || defined(__powerpc64__)) && __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
return crc32_ppc(updated_value, reinterpret_cast<const unsigned char *>(&x), sizeof(x));
#elif defined(__s390x__) && __BYTE_ORDER__==__ORDER_BIG_ENDIAN__
return s390x_crc32(updated_value, x);
return s390x_crc32c(updated_value, x);
#else
/// On other platforms we do not have CRC32. NOTE This can be confusing.
return intHash64(x) ^ updated_value;
@ -403,8 +380,8 @@ struct UInt128HashCRC32
size_t operator()(UInt128 x) const
{
UInt64 crc = -1ULL;
crc = s390x_crc32(crc, x.items[UInt128::_impl::little(0)]);
crc = s390x_crc32(crc, x.items[UInt128::_impl::little(1)]);
crc = s390x_crc32c(crc, x.items[UInt128::_impl::little(0)]);
crc = s390x_crc32c(crc, x.items[UInt128::_impl::little(1)]);
return crc;
}
};
@ -472,10 +449,10 @@ struct UInt256HashCRC32
size_t operator()(UInt256 x) const
{
UInt64 crc = -1ULL;
crc = s390x_crc32(crc, x.items[UInt256::_impl::little(0)]);
crc = s390x_crc32(crc, x.items[UInt256::_impl::little(1)]);
crc = s390x_crc32(crc, x.items[UInt256::_impl::little(2)]);
crc = s390x_crc32(crc, x.items[UInt256::_impl::little(3)]);
crc = s390x_crc32c(crc, x.items[UInt256::_impl::little(0)]);
crc = s390x_crc32c(crc, x.items[UInt256::_impl::little(1)]);
crc = s390x_crc32c(crc, x.items[UInt256::_impl::little(2)]);
crc = s390x_crc32c(crc, x.items[UInt256::_impl::little(3)]);
return crc;
}
};
@ -557,7 +534,10 @@ struct IntHash32
else if constexpr (sizeof(T) <= sizeof(UInt64))
{
DB::UInt64 out {0};
if constexpr (std::endian::native == std::endian::little)
std::memcpy(&out, &key, sizeof(T));
else
std::memcpy(reinterpret_cast<char*>(&out) + sizeof(DB::UInt64) - sizeof(T), &key, sizeof(T));
return intHash32<salt>(out);
}

View File

@ -75,22 +75,22 @@ struct StringHashTableHash
size_t ALWAYS_INLINE operator()(StringKey8 key) const
{
size_t res = -1ULL;
res = s390x_crc32(res, key);
res = s390x_crc32c(res, key);
return res;
}
size_t ALWAYS_INLINE operator()(StringKey16 key) const
{
size_t res = -1ULL;
res = s390x_crc32(res, key.items[UInt128::_impl::little(0)]);
res = s390x_crc32(res, key.items[UInt128::_impl::little(1)]);
res = s390x_crc32c(res, key.items[UInt128::_impl::little(0)]);
res = s390x_crc32c(res, key.items[UInt128::_impl::little(1)]);
return res;
}
size_t ALWAYS_INLINE operator()(StringKey24 key) const
{
size_t res = -1ULL;
res = s390x_crc32(res, key.a);
res = s390x_crc32(res, key.b);
res = s390x_crc32(res, key.c);
res = s390x_crc32c(res, key.a);
res = s390x_crc32c(res, key.b);
res = s390x_crc32c(res, key.c);
return res;
}
#else

View File

@ -4,6 +4,7 @@
#include <Common/HyperLogLogBiasEstimator.h>
#include <Common/CompactArray.h>
#include <Common/HashTable/Hash.h>
#include <Common/TransformEndianness.hpp>
#include <IO/ReadBuffer.h>
#include <IO/WriteBuffer.h>
@ -330,7 +331,26 @@ public:
void read(DB::ReadBuffer & in)
{
if constexpr (std::endian::native == std::endian::little)
in.readStrict(reinterpret_cast<char *>(this), sizeof(*this));
else
{
in.readStrict(reinterpret_cast<char *>(&rank_store), sizeof(RankStore));
constexpr size_t denom_size = sizeof(DenominatorCalculatorType);
std::array<char, denom_size> denominator_copy;
in.readStrict(denominator_copy.begin(), denom_size);
for (size_t i = 0; i < denominator_copy.size(); i += (sizeof(UInt32) / sizeof(char)))
{
UInt32 * cur = reinterpret_cast<UInt32 *>(&denominator_copy[i]);
DB::transformEndianness<std::endian::native, std::endian::little>(*cur);
}
memcpy(reinterpret_cast<char *>(&denominator), denominator_copy.begin(), denom_size);
in.readStrict(reinterpret_cast<char *>(&zeros), sizeof(ZerosCounterType));
DB::transformEndianness<std::endian::native, std::endian::little>(zeros);
}
}
void readAndMerge(DB::ReadBuffer & in)
@ -352,7 +372,27 @@ public:
void write(DB::WriteBuffer & out) const
{
if constexpr (std::endian::native == std::endian::little)
out.write(reinterpret_cast<const char *>(this), sizeof(*this));
else
{
out.write(reinterpret_cast<const char *>(&rank_store), sizeof(RankStore));
constexpr size_t denom_size = sizeof(DenominatorCalculatorType);
std::array<char, denom_size> denominator_copy;
memcpy(denominator_copy.begin(), reinterpret_cast<const char *>(&denominator), denom_size);
for (size_t i = 0; i < denominator_copy.size(); i += (sizeof(UInt32) / sizeof(char)))
{
UInt32 * cur = reinterpret_cast<UInt32 *>(&denominator_copy[i]);
DB::transformEndianness<std::endian::little, std::endian::native>(*cur);
}
out.write(denominator_copy.begin(), denom_size);
auto zeros_copy = zeros;
DB::transformEndianness<std::endian::little, std::endian::native>(zeros_copy);
out.write(reinterpret_cast<const char *>(&zeros_copy), sizeof(ZerosCounterType));
}
}
/// Read and write in text mode is suboptimal (but compatible with OLAPServer and Metrage).

View File

@ -96,14 +96,20 @@ public:
void write(WriteBuffer & wb) const
{
if constexpr (std::is_same_v<TKey, StringRef>)
writeBinary(key, wb);
else
writeBinaryLittleEndian(key, wb);
writeVarUInt(count, wb);
writeVarUInt(error, wb);
}
void read(ReadBuffer & rb)
{
if constexpr (std::is_same_v<TKey, StringRef>)
readBinary(key, rb);
else
readBinaryLittleEndian(key, rb);
readVarUInt(count, rb);
readVarUInt(error, rb);
}

View File

@ -198,7 +198,7 @@ namespace Hashes
#ifdef __SSE4_2__
return _mm_crc32_u64(-1ULL, x);
#elif defined(__s390x__) && __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
return s390x_crc32(-1ULL, x);
return s390x_crc32c(-1ULL, x);
#else
/// On other platforms we do not have CRC32. NOTE This can be confusing.
return intHash64(x);

View File

@ -86,12 +86,18 @@ public:
void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override
{
IPv x = field.get<IPv>();
if constexpr (std::is_same_v<IPv, IPv6>)
writeBinary(x, ostr);
else
writeBinaryLittleEndian(x, ostr);
}
void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const override
{
IPv x;
readBinaryLittleEndian(x.toUnderType(), istr);
if constexpr (std::is_same_v<IPv, IPv6>)
readBinary(x, istr);
else
readBinaryLittleEndian(x, istr);
field = NearestFieldType<IPv>(x);
}
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override

View File

@ -77,7 +77,7 @@ struct NgramDistanceImpl
#elif (defined(__PPC64__) || defined(__powerpc64__)) && __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
return crc32_ppc(code_points[2], reinterpret_cast<const unsigned char *>(&combined), sizeof(combined)) & 0xFFFFu;
#elif defined(__s390x__) && __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
return s390x_crc32(code_points[2], combined) & 0xFFFFu;
return s390x_crc32c(code_points[2], combined) & 0xFFFFu;
#else
return (intHashCRC32(combined) ^ intHashCRC32(code_points[2])) & 0xFFFFu;
#endif