This commit is contained in:
Alexey Milovidov 2023-04-21 12:38:45 +02:00
parent 629bbbd353
commit 530b764953
24 changed files with 231 additions and 247 deletions

View File

@ -5,44 +5,6 @@
#include <bit>
inline void reverseMemcpy(void * dst, const void * src, size_t size)
{
uint8_t * uint_dst = reinterpret_cast<uint8_t *>(dst);
const uint8_t * uint_src = reinterpret_cast<const uint8_t *>(src);
uint_dst += size;
while (size)
{
--uint_dst;
*uint_dst = *uint_src;
++uint_src;
--size;
}
}
template <typename T>
inline T unalignedLoadLE(const void * address)
{
T res {};
if constexpr (std::endian::native == std::endian::little)
memcpy(&res, address, sizeof(res));
else
reverseMemcpy(&res, address, sizeof(res));
return res;
}
template <typename T>
inline void unalignedStoreLE(void * address,
const typename std::enable_if<true, T>::type & src)
{
static_assert(std::is_trivially_copyable_v<T>);
if constexpr (std::endian::native == std::endian::little)
memcpy(address, &src, sizeof(src));
else
reverseMemcpy(address, &src, sizeof(src));
}
template <typename T>
inline T unalignedLoad(const void * address)
{
@ -62,3 +24,70 @@ inline void unalignedStore(void * address,
static_assert(std::is_trivially_copyable_v<T>);
memcpy(address, &src, sizeof(src));
}
inline void reverseMemcpy(void * dst, const void * src, size_t size)
{
uint8_t * uint_dst = reinterpret_cast<uint8_t *>(dst);
const uint8_t * uint_src = reinterpret_cast<const uint8_t *>(src);
uint_dst += size;
while (size)
{
--uint_dst;
*uint_dst = *uint_src;
++uint_src;
--size;
}
}
template <std::endian endian, typename T>
inline T unalignedLoadEndian(const void * address)
{
T res {};
if constexpr (std::endian::native == endian)
memcpy(&res, address, sizeof(res));
else
reverseMemcpy(&res, address, sizeof(res));
return res;
}
template <std::endian endian, typename T>
inline void unalignedStoreEndian(void * address, T & src)
{
static_assert(std::is_trivially_copyable_v<T>);
if constexpr (std::endian::native == endian)
memcpy(address, &src, sizeof(src));
else
reverseMemcpy(address, &src, sizeof(src));
}
template <typename T>
inline T unalignedLoadLittleEndian(const void * address)
{
return unalignedLoadEndian<std::endian::little, T>(address);
}
template <typename T>
inline void unalignedStoreLittleEndian(void * address,
const typename std::enable_if<true, T>::type & src)
{
unalignedStoreEndian<std::endian::little>(address, src);
}
template <typename T>
inline T unalignedLoadBigEndian(const void * address)
{
return unalignedLoadEndian<std::endian::big, T>(address);
}
template <typename T>
inline void unalignedStoreBigEndian(void * address,
const typename std::enable_if<true, T>::type & src)
{
unalignedStoreEndian<std::endian::big>(address, src);
}

View File

@ -25,7 +25,6 @@
#include <Common/formatReadable.h>
#include <Common/DNSResolver.h>
#include <Common/CurrentThread.h>
#include <Common/escapeForFileName.h>
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Common/ThreadStatus.h>
#include <Client/Connection.h>

View File

@ -4,9 +4,11 @@
#include "TaskCluster.h"
#include <Parsers/ASTFunction.h>
#include <Common/escapeForFileName.h>
#include <boost/algorithm/string/join.hpp>
namespace DB
{
namespace ErrorCodes

View File

@ -14,9 +14,9 @@
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/ZooKeeper/Types.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/escapeForFileName.h>
#include <Common/setThreadName.h>
#include <Common/ThreadPool.h>
#include <Common/escapeForFileName.h>
#include <base/range.h>
#include <base/sleep.h>
#include <boost/range/algorithm_ext/erase.hpp>

View File

@ -5,9 +5,9 @@
#include <Access/Common/AccessEntityType.h>
#include <Backups/BackupCoordinationReplicatedAccess.h>
#include <Backups/BackupCoordinationStage.h>
#include <Common/escapeForFileName.h>
#include <Common/ZooKeeper/Common.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/escapeForFileName.h>
#include <Functions/UserDefined/UserDefinedSQLObjectType.h>
#include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h>

View File

@ -1,10 +1,11 @@
#include <Backups/BackupCoordinationRemote.h>
#include <Backups/BackupCoordinationStage.h>
#include <Backups/RestoreCoordinationRemote.h>
#include <Backups/BackupCoordinationStageSync.h>
#include <Functions/UserDefined/UserDefinedSQLObjectType.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/escapeForFileName.h>
#include "Backups/BackupCoordinationStageSync.h"
namespace DB
{

View File

@ -19,8 +19,8 @@
#include <Databases/IDatabase.h>
#include <Databases/DDLDependencyVisitor.h>
#include <Storages/IStorage.h>
#include <Common/escapeForFileName.h>
#include <Common/quoteString.h>
#include <Common/escapeForFileName.h>
#include <base/insertAtEnd.h>
#include <boost/algorithm/string/join.hpp>
#include <filesystem>

View File

@ -221,7 +221,7 @@ inline UInt32 updateWeakHash32(const DB::UInt8 * pos, size_t size, DB::UInt32 up
const auto * end = pos + size;
while (pos + 8 <= end)
{
auto word = unalignedLoadLE<UInt64>(pos);
auto word = unalignedLoadLittleEndian<UInt64>(pos);
updated_value = static_cast<UInt32>(intHashCRC32(word, updated_value));
pos += 8;
@ -233,7 +233,7 @@ inline UInt32 updateWeakHash32(const DB::UInt8 * pos, size_t size, DB::UInt32 up
/// Lets' assume the string was 'abcdefghXYZ', so it's tail is 'XYZ'.
DB::UInt8 tail_size = end - pos;
/// Load tailing 8 bytes. Word is 'defghXYZ'.
auto word = unalignedLoadLE<UInt64>(end - 8);
auto word = unalignedLoadLittleEndian<UInt64>(end - 8);
/// Prepare mask which will set other 5 bytes to 0. It is 0xFFFFFFFFFFFFFFFF << 5 = 0xFFFFFF0000000000.
/// word & mask = '\0\0\0\0\0XYZ' (bytes are reversed because of little ending)
word &= (~UInt64(0)) << DB::UInt8(8 * (8 - tail_size));

View File

@ -13,6 +13,7 @@
* (~ 700 MB/sec, 15 million strings per second)
*/
#include <bit>
#include <string>
#include <type_traits>
#include <Core/Defines.h>
@ -21,6 +22,7 @@
#include <base/unaligned.h>
#include <Common/Exception.h>
namespace DB
{
namespace ErrorCodes
@ -29,15 +31,13 @@ namespace ErrorCodes
}
}
#define ROTL(x, b) static_cast<UInt64>(((x) << (b)) | ((x) >> (64 - (b))))
#define SIPROUND \
do \
{ \
v0 += v1; v1 = ROTL(v1, 13); v1 ^= v0; v0 = ROTL(v0, 32); \
v2 += v3; v3 = ROTL(v3, 16); v3 ^= v2; \
v0 += v3; v3 = ROTL(v3, 21); v3 ^= v0; \
v2 += v1; v1 = ROTL(v1, 17); v1 ^= v2; v2 = ROTL(v2, 32); \
v0 += v1; v1 = std::rotl(v1, 13); v1 ^= v0; v0 = std::rotl(v0, 32); \
v2 += v3; v3 = std::rotl(v3, 16); v3 ^= v2; \
v0 += v3; v3 = std::rotl(v3, 21); v3 ^= v0; \
v2 += v1; v1 = std::rotl(v1, 17); v1 ^= v2; v2 = std::rotl(v2, 32); \
} while(0)
/// Define macro CURRENT_BYTES_IDX for building index used in current_bytes array
@ -136,7 +136,7 @@ public:
while (data + 8 <= end)
{
current_word = unalignedLoadLE<UInt64>(data);
current_word = unalignedLoadLittleEndian<UInt64>(data);
v3 ^= current_word;
SIPROUND;
@ -242,14 +242,16 @@ public:
SIPROUND;
SIPROUND;
auto hi = v0 ^ v1 ^ v2 ^ v3;
if constexpr (std::endian::native == std::endian::big)
{
lo = __builtin_bswap64(lo);
hi = __builtin_bswap64(hi);
lo = std::byteswap(lo);
hi = std::byteswap(hi);
auto tmp = hi;
hi = lo;
lo = tmp;
}
UInt128 res = hi;
res <<= 64;
res |= lo;

View File

@ -1,48 +1,14 @@
#include <Common/ZooKeeper/ZooKeeperIO.h>
namespace Coordination
{
void write(size_t x, WriteBuffer & out)
{
x = __builtin_bswap64(x);
writeBinary(x, out);
}
#ifdef OS_DARWIN
void write(uint64_t x, WriteBuffer & out)
{
x = __builtin_bswap64(x);
writeBinary(x, out);
}
#endif
void write(int64_t x, WriteBuffer & out)
{
x = __builtin_bswap64(x);
writeBinary(x, out);
}
void write(int32_t x, WriteBuffer & out)
{
x = __builtin_bswap32(x);
writeBinary(x, out);
}
void write(uint8_t x, WriteBuffer & out)
{
writeBinary(x, out);
}
void write(OpNum x, WriteBuffer & out)
{
write(static_cast<int32_t>(x), out);
}
void write(bool x, WriteBuffer & out)
{
writeBinary(x, out);
}
void write(const std::string & s, WriteBuffer & out)
{
write(static_cast<int32_t>(s.size()), out);
@ -76,37 +42,6 @@ void write(const Error & x, WriteBuffer & out)
write(static_cast<int32_t>(x), out);
}
#ifdef OS_DARWIN
void read(uint64_t & x, ReadBuffer & in)
{
readBinary(x, in);
x = __builtin_bswap64(x);
}
#endif
void read(size_t & x, ReadBuffer & in)
{
readBinary(x, in);
x = __builtin_bswap64(x);
}
void read(int64_t & x, ReadBuffer & in)
{
readBinary(x, in);
x = __builtin_bswap64(x);
}
void read(uint8_t & x, ReadBuffer & in)
{
readBinary(x, in);
}
void read(int32_t & x, ReadBuffer & in)
{
readBinary(x, in);
x = __builtin_bswap32(x);
}
void read(OpNum & x, ReadBuffer & in)
{
int32_t raw_op_num;
@ -114,16 +49,6 @@ void read(OpNum & x, ReadBuffer & in)
x = getOpNum(raw_op_num);
}
void read(bool & x, ReadBuffer & in)
{
readBinary(x, in);
}
void read(int8_t & x, ReadBuffer & in)
{
readBinary(x, in);
}
void read(std::string & s, ReadBuffer & in)
{
int32_t size = 0;

View File

@ -1,4 +1,5 @@
#pragma once
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <IO/Operators.h>
@ -8,23 +9,20 @@
#include <vector>
#include <array>
namespace Coordination
{
using namespace DB;
void write(size_t x, WriteBuffer & out);
template <typename T>
requires is_arithmetic_v<T>
void write(T x, WriteBuffer & out)
{
writeBinaryBigEndian(x, out);
}
/// uint64_t != size_t on darwin
#ifdef OS_DARWIN
void write(uint64_t x, WriteBuffer & out);
#endif
void write(int64_t x, WriteBuffer & out);
void write(int32_t x, WriteBuffer & out);
void write(uint8_t x, WriteBuffer & out);
void write(OpNum x, WriteBuffer & out);
void write(bool x, WriteBuffer & out);
void write(const std::string & s, WriteBuffer & out);
void write(const ACL & acl, WriteBuffer & out);
void write(const Stat & stat, WriteBuffer & out);
@ -45,16 +43,14 @@ void write(const std::vector<T> & arr, WriteBuffer & out)
write(elem, out);
}
void read(size_t & x, ReadBuffer & in);
#ifdef OS_DARWIN
void read(uint64_t & x, ReadBuffer & in);
#endif
void read(int64_t & x, ReadBuffer & in);
void read(int32_t & x, ReadBuffer & in);
void read(uint8_t & x, ReadBuffer & in);
template <typename T>
requires is_arithmetic_v<T>
void read(T & x, ReadBuffer & in)
{
readBinaryBigEndian(x, in);
}
void read(OpNum & x, ReadBuffer & in);
void read(bool & x, ReadBuffer & in);
void read(int8_t & x, ReadBuffer & in);
void read(std::string & s, ReadBuffer & in);
void read(ACL & acl, ReadBuffer & in);
void read(Stat & stat, ReadBuffer & in);

View File

@ -10,6 +10,7 @@
#include <Compression/ICompressionCodec.h>
#include <Compression/CompressionFactory.h>
#include <IO/ReadBuffer.h>
#include <IO/ReadBufferFromMemory.h>
#include <IO/BufferWithOwnMemory.h>
#include <Compression/CompressionInfo.h>
#include <IO/WriteHelpers.h>
@ -191,7 +192,11 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed,
if (!disable_checksum)
{
Checksum & checksum = *reinterpret_cast<Checksum *>(own_compressed_buffer.data());
Checksum checksum;
ReadBufferFromMemory checksum_in(own_compressed_buffer.data(), sizeof(checksum));
readBinaryLittleEndian(checksum.first, checksum_in);
readBinaryLittleEndian(checksum.second, checksum_in);
validateChecksum(compressed_buffer, size_compressed_without_checksum, checksum);
}
@ -231,7 +236,11 @@ size_t CompressedReadBufferBase::readCompressedDataBlockForAsynchronous(size_t &
if (!disable_checksum)
{
Checksum & checksum = *reinterpret_cast<Checksum *>(own_compressed_buffer.data());
Checksum checksum;
ReadBufferFromMemory checksum_in(own_compressed_buffer.data(), sizeof(checksum));
readBinaryLittleEndian(checksum.first, checksum_in);
readBinaryLittleEndian(checksum.second, checksum_in);
validateChecksum(compressed_buffer, size_compressed_without_checksum, checksum);
}
@ -319,5 +328,4 @@ CompressedReadBufferBase::CompressedReadBufferBase(ReadBuffer * in, bool allow_d
CompressedReadBufferBase::~CompressedReadBufferBase() = default; /// Proper destruction of unique_ptr of forward-declared type.
}

View File

@ -5,19 +5,15 @@
#include <base/unaligned.h>
#include <base/defines.h>
#include <IO/WriteHelpers.h>
#include <Compression/CompressionFactory.h>
#include "CompressedWriteBuffer.h"
#include <Compression/CompressedWriteBuffer.h>
namespace DB
{
namespace ErrorCodes
{
}
static constexpr auto CHECKSUM_SIZE{sizeof(CityHash_v1_0_2::uint128)};
void CompressedWriteBuffer::nextImpl()
{
if (!offset())
@ -29,21 +25,23 @@ void CompressedWriteBuffer::nextImpl()
/** During compression we need buffer with capacity >= compressed_reserve_size + CHECKSUM_SIZE.
*
* If output buffer has necessary capacity, we can compress data directly in output buffer.
* If output buffer has necessary capacity, we can compress data directly into the output buffer.
* Then we can write checksum at the output buffer begin.
*
* If output buffer does not have necessary capacity. Compress data in temporary buffer.
* Then we can write checksum and temporary buffer in output buffer.
* If output buffer does not have necessary capacity. Compress data into a temporary buffer.
* Then we can write checksum and copy the temporary buffer into the output buffer.
*/
if (out.available() >= compressed_reserve_size + CHECKSUM_SIZE)
if (out.available() >= compressed_reserve_size + sizeof(CityHash_v1_0_2::uint128))
{
char * out_checksum_ptr = out.position();
char * out_compressed_ptr = out.position() + CHECKSUM_SIZE;
char * out_compressed_ptr = out.position() + sizeof(CityHash_v1_0_2::uint128);
UInt32 compressed_size = codec->compress(working_buffer.begin(), decompressed_size, out_compressed_ptr);
CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(out_compressed_ptr, compressed_size);
memcpy(out_checksum_ptr, reinterpret_cast<const char *>(&checksum), CHECKSUM_SIZE);
out.position() += CHECKSUM_SIZE + compressed_size;
writeBinaryLittleEndian(checksum.first, out);
writeBinaryLittleEndian(checksum.second, out);
out.position() += compressed_size;
}
else
{
@ -51,7 +49,10 @@ void CompressedWriteBuffer::nextImpl()
UInt32 compressed_size = codec->compress(working_buffer.begin(), decompressed_size, compressed_buffer.data());
CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(compressed_buffer.data(), compressed_size);
out.write(reinterpret_cast<const char *>(&checksum), CHECKSUM_SIZE);
writeBinaryLittleEndian(checksum.first, out);
writeBinaryLittleEndian(checksum.second, out);
out.write(compressed_buffer.data(), compressed_size);
}
}

View File

@ -293,7 +293,7 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest)
const char * dest_start = dest;
const UInt32 items_count = source_size / sizeof(ValueType);
unalignedStoreLE<UInt32>(dest, items_count);
unalignedStoreLittleEndian<UInt32>(dest, items_count);
dest += sizeof(items_count);
ValueType prev_value{};
@ -301,8 +301,8 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest)
if (source < source_end)
{
prev_value = unalignedLoadLE<ValueType>(source);
unalignedStoreLE<ValueType>(dest, prev_value);
prev_value = unalignedLoadLittleEndian<ValueType>(source);
unalignedStoreLittleEndian<ValueType>(dest, prev_value);
source += sizeof(prev_value);
dest += sizeof(prev_value);
@ -310,10 +310,10 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest)
if (source < source_end)
{
const ValueType curr_value = unalignedLoadLE<ValueType>(source);
const ValueType curr_value = unalignedLoadLittleEndian<ValueType>(source);
prev_delta = curr_value - prev_value;
unalignedStoreLE<UnsignedDeltaType>(dest, prev_delta);
unalignedStoreLittleEndian<UnsignedDeltaType>(dest, prev_delta);
source += sizeof(curr_value);
dest += sizeof(prev_delta);
@ -325,7 +325,7 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest)
int item = 2;
for (; source < source_end; source += sizeof(ValueType), ++item)
{
const ValueType curr_value = unalignedLoadLE<ValueType>(source);
const ValueType curr_value = unalignedLoadLittleEndian<ValueType>(source);
const UnsignedDeltaType delta = curr_value - prev_value;
const UnsignedDeltaType double_delta = delta - prev_delta;
@ -369,7 +369,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest,
if (source + sizeof(UInt32) > source_end)
return;
const UInt32 items_count = unalignedLoadLE<UInt32>(source);
const UInt32 items_count = unalignedLoadLittleEndian<UInt32>(source);
source += sizeof(items_count);
ValueType prev_value{};
@ -379,10 +379,10 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest,
if (source + sizeof(ValueType) > source_end || items_count < 1)
return;
prev_value = unalignedLoadLE<ValueType>(source);
prev_value = unalignedLoadLittleEndian<ValueType>(source);
if (dest + sizeof(prev_value) > output_end)
throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress the data");
unalignedStoreLE<ValueType>(dest, prev_value);
unalignedStoreLittleEndian<ValueType>(dest, prev_value);
source += sizeof(prev_value);
dest += sizeof(prev_value);
@ -391,11 +391,11 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest,
if (source + sizeof(UnsignedDeltaType) > source_end || items_count < 2)
return;
prev_delta = unalignedLoadLE<UnsignedDeltaType>(source);
prev_delta = unalignedLoadLittleEndian<UnsignedDeltaType>(source);
prev_value = prev_value + static_cast<ValueType>(prev_delta);
if (dest + sizeof(prev_value) > output_end)
throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress the data");
unalignedStoreLE<ValueType>(dest, prev_value);
unalignedStoreLittleEndian<ValueType>(dest, prev_value);
source += sizeof(prev_delta);
dest += sizeof(prev_value);
@ -428,7 +428,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest,
const ValueType curr_value = prev_value + delta;
if (dest + sizeof(curr_value) > output_end)
throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress the data");
unalignedStoreLE<ValueType>(dest, curr_value);
unalignedStoreLittleEndian<ValueType>(dest, curr_value);
dest += sizeof(curr_value);
prev_delta = curr_value - prev_value;

View File

@ -205,7 +205,7 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest,
const UInt32 items_count = source_size / sizeof(T);
unalignedStoreLE<UInt32>(dest, items_count);
unalignedStoreLittleEndian<UInt32>(dest, items_count);
dest += sizeof(items_count);
T prev_value = 0;
@ -214,8 +214,8 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest,
if (source < source_end)
{
prev_value = unalignedLoadLE<T>(source);
unalignedStoreLE<T>(dest, prev_value);
prev_value = unalignedLoadLittleEndian<T>(source);
unalignedStoreLittleEndian<T>(dest, prev_value);
source += sizeof(prev_value);
dest += sizeof(prev_value);
@ -229,7 +229,7 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest,
while (source < source_end)
{
const T curr_value = unalignedLoadLE<T>(source);
const T curr_value = unalignedLoadLittleEndian<T>(source);
source += sizeof(curr_value);
const auto xored_data = curr_value ^ prev_value;
@ -271,7 +271,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest)
if (source + sizeof(UInt32) > source_end)
return;
const UInt32 items_count = unalignedLoadLE<UInt32>(source);
const UInt32 items_count = unalignedLoadLittleEndian<UInt32>(source);
source += sizeof(items_count);
T prev_value = 0;
@ -280,8 +280,8 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest)
if (source + sizeof(T) > source_end || items_count < 1)
return;
prev_value = unalignedLoadLE<T>(source);
unalignedStoreLE<T>(dest, prev_value);
prev_value = unalignedLoadLittleEndian<T>(source);
unalignedStoreLittleEndian<T>(dest, prev_value);
source += sizeof(prev_value);
dest += sizeof(prev_value);
@ -326,7 +326,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest)
}
// else: 0b0 prefix - use prev_value
unalignedStoreLE<T>(dest, curr_value);
unalignedStoreLittleEndian<T>(dest, curr_value);
dest += sizeof(curr_value);
prev_xored_info = curr_xored_info;

View File

@ -326,7 +326,7 @@ void load(const char * src, T * buf, UInt32 tail = 64)
/// as little-endian types on big-endian machine (s390x, etc).
for (UInt32 i = 0; i < tail; ++i)
{
buf[i] = unalignedLoadLE<T>(src + i * sizeof(T));
buf[i] = unalignedLoadLittleEndian<T>(src + i * sizeof(T));
}
}
}

View File

@ -86,8 +86,8 @@ UInt32 ICompressionCodec::compress(const char * source, UInt32 source_size, char
UInt8 header_size = getHeaderSize();
/// Write data from header_size
UInt32 compressed_bytes_written = doCompressData(source, source_size, &dest[header_size]);
unalignedStoreLE<UInt32>(&dest[1], compressed_bytes_written + header_size);
unalignedStoreLE<UInt32>(&dest[5], source_size);
unalignedStoreLittleEndian<UInt32>(&dest[1], compressed_bytes_written + header_size);
unalignedStoreLittleEndian<UInt32>(&dest[5], source_size);
return header_size + compressed_bytes_written;
}
@ -114,7 +114,7 @@ UInt32 ICompressionCodec::decompress(const char * source, UInt32 source_size, ch
UInt32 ICompressionCodec::readCompressedBlockSize(const char * source)
{
UInt32 compressed_block_size = unalignedLoadLE<UInt32>(&source[1]);
UInt32 compressed_block_size = unalignedLoadLittleEndian<UInt32>(&source[1]);
if (compressed_block_size == 0)
throw Exception(ErrorCodes::CORRUPTED_DATA, "Can't decompress data: header is corrupt with compressed block size 0");
return compressed_block_size;
@ -123,7 +123,7 @@ UInt32 ICompressionCodec::readCompressedBlockSize(const char * source)
UInt32 ICompressionCodec::readDecompressedBlockSize(const char * source)
{
UInt32 decompressed_block_size = unalignedLoadLE<UInt32>(&source[5]);
UInt32 decompressed_block_size = unalignedLoadLittleEndian<UInt32>(&source[5]);
if (decompressed_block_size == 0)
throw Exception(ErrorCodes::CORRUPTED_DATA, "Can't decompress data: header is corrupt with decompressed block size 0");
return decompressed_block_size;

View File

@ -172,7 +172,7 @@ private:
throw std::runtime_error("No more data to read");
}
current_value = unalignedLoadLE<T>(data);
current_value = unalignedLoadLittleEndian<T>(data);
data = reinterpret_cast<const char *>(data) + sizeof(T);
}
};
@ -368,7 +368,7 @@ CodecTestSequence makeSeq(Args && ... args)
char * write_pos = data.data();
for (const auto & v : vals)
{
unalignedStoreLE<T>(write_pos, v);
unalignedStoreLittleEndian<T>(write_pos, v);
write_pos += sizeof(v);
}
@ -390,7 +390,7 @@ CodecTestSequence generateSeq(Generator gen, const char* gen_name, B Begin = 0,
{
const T v = static_cast<T>(gen(i));
unalignedStoreLE<T>(write_pos, v);
unalignedStoreLittleEndian<T>(write_pos, v);
write_pos += sizeof(v);
}
@ -1297,9 +1297,9 @@ TEST(LZ4Test, DecompressMalformedInput)
DB::Memory<> memory;
memory.resize(ICompressionCodec::getHeaderSize() + uncompressed_size + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER);
unalignedStoreLE<uint8_t>(memory.data(), static_cast<uint8_t>(CompressionMethodByte::LZ4));
unalignedStoreLE<uint32_t>(&memory[1], source_size);
unalignedStoreLE<uint32_t>(&memory[5], uncompressed_size);
unalignedStoreLittleEndian<uint8_t>(memory.data(), static_cast<uint8_t>(CompressionMethodByte::LZ4));
unalignedStoreLittleEndian<uint32_t>(&memory[1], source_size);
unalignedStoreLittleEndian<uint32_t>(&memory[5], uncompressed_size);
auto codec = CompressionCodecFactory::instance().get("LZ4", {});
ASSERT_THROW(codec->decompress(source, source_size, memory.data()), Exception);

View File

@ -14,6 +14,8 @@
#include <IO/Operators.h>
#include <unistd.h>
#include <bit>
namespace DB
{
@ -34,7 +36,7 @@ int32_t IFourLetterCommand::code()
String IFourLetterCommand::toName(int32_t code)
{
int reverted_code = __builtin_bswap32(code);
int reverted_code = std::byteswap(code);
return String(reinterpret_cast<char *>(&reverted_code), 4);
}
@ -42,7 +44,7 @@ int32_t IFourLetterCommand::toCode(const String & name)
{
int32_t res = *reinterpret_cast<const int32_t *>(name.data());
/// keep consistent with Coordination::read method by changing big endian to little endian.
return __builtin_bswap32(res);
return std::byteswap(res);
}
IFourLetterCommand::~IFourLetterCommand() = default;

View File

@ -245,8 +245,8 @@ public:
private:
static bool isIPv4Mapped(const UInt8 * address)
{
return (unalignedLoadLE<UInt64>(address) == 0) &&
((unalignedLoadLE<UInt64>(address + 8) & 0x00000000FFFFFFFFull) == 0x00000000FFFF0000ull);
return (unalignedLoadLittleEndian<UInt64>(address) == 0) &&
((unalignedLoadLittleEndian<UInt64>(address + 8) & 0x00000000FFFFFFFFull) == 0x00000000FFFF0000ull);
}
static void cutAddress(const unsigned char * address, char *& dst, UInt8 zeroed_tail_bytes_count)
@ -576,10 +576,11 @@ private:
static void mapIPv4ToIPv6(UInt32 in, UInt8 * buf)
{
unalignedStore<UInt64>(buf, 0);
#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
unalignedStoreLE<UInt64>(buf + 8, 0x00000000FFFF0000ull | (static_cast<UInt64>(ntohl(in)) << 32));
unalignedStoreLittleEndian<UInt64>(buf + 8, 0x00000000FFFF0000ull | (static_cast<UInt64>(ntohl(in)) << 32));
#else
unalignedStoreLE<UInt64>(buf + 8, 0x00000000FFFF0000ull | (static_cast<UInt64>(__builtin_bswap32(ntohl(in))) << 32));
unalignedStoreLittleEndian<UInt64>(buf + 8, 0x00000000FFFF0000ull | (static_cast<UInt64>(in)) << 32));
#endif
}
};

View File

@ -2,6 +2,7 @@
#include <bit>
#include <base/types.h>
#include <base/unaligned.h>
#include <Common/BitHelpers.h>
#include <Common/Exception.h>
@ -139,7 +140,7 @@ private:
source_current += bytes_to_read;
if constexpr (std::endian::native == std::endian::little)
tmp_buffer = __builtin_bswap64(tmp_buffer);
tmp_buffer = std::byteswap(tmp_buffer);
bits_buffer |= BufferType(tmp_buffer) << ((sizeof(BufferType) - sizeof(tmp_buffer)) * 8 - bits_count);
bits_count += static_cast<UInt8>(bytes_to_read) * 8;
@ -220,12 +221,9 @@ private:
"Can not write past end of buffer. Space available {} bytes, required to write {} bytes.",
available, to_write);
}
UInt64 tmp_buffer = 0;
if constexpr (std::endian::native == std::endian::little)
tmp_buffer = __builtin_bswap64(static_cast<UInt64>(bits_buffer >> (sizeof(bits_buffer) - sizeof(UInt64)) * 8));
else
tmp_buffer = static_cast<UInt64>(bits_buffer >> (sizeof(bits_buffer) - sizeof(UInt64)) * 8);
memcpy(dest_current, &tmp_buffer, to_write);
UInt64 tmp_buffer = static_cast<UInt64>(bits_buffer >> (sizeof(bits_buffer) - sizeof(UInt64)) * 8);
unalignedStoreBigEndian<UInt64>(dest_current, tmp_buffer);
dest_current += to_write;
bits_buffer <<= to_write * 8;

View File

@ -1116,33 +1116,41 @@ inline void readBinary(Decimal256 & x, ReadBuffer & buf) { readPODBinary(x.value
inline void readBinary(LocalDate & x, ReadBuffer & buf) { readPODBinary(x, buf); }
template <typename T>
template <std::endian endian, typename T>
requires is_arithmetic_v<T> && (sizeof(T) <= 8)
inline void readBinaryBigEndian(T & x, ReadBuffer & buf) /// Assuming little endian architecture.
inline void readBinaryEndian(T & x, ReadBuffer & buf)
{
readPODBinary(x, buf);
if constexpr (std::endian::native == std::endian::little)
if constexpr (std::endian::native != endian)
std::byteswap(x);
}
template <std::endian endian, typename T>
requires is_big_int_v<T>
inline void readBinaryEndian(T & x, ReadBuffer & buf)
{
if constexpr (std::endian::native == endian)
{
if constexpr (sizeof(x) == 1)
return;
else if constexpr (sizeof(x) == 2)
x = __builtin_bswap16(x);
else if constexpr (sizeof(x) == 4)
x = __builtin_bswap32(x);
else if constexpr (sizeof(x) == 8)
x = __builtin_bswap64(x);
for (size_t i = 0; i != std::size(x.items); ++i)
readBinaryEndian<endian>(x.items[i], buf);
}
else
{
for (ssize_t i = std::size(x.items) - 1; i >= 0; --i)
readBinaryEndian<endian>(x.items[i], buf);
}
}
template <typename T>
requires is_big_int_v<T>
inline void readBinaryBigEndian(T & x, ReadBuffer & buf) /// Assuming little endian architecture.
inline void readBinaryLittleEndian(T & x, ReadBuffer & buf)
{
for (size_t i = 0; i != std::size(x.items); ++i)
{
auto & item = x.items[(std::endian::native == std::endian::little) ? std::size(x.items) - i - 1 : i];
readBinaryBigEndian(item, buf);
}
readBinaryEndian<std::endian::little>(x, buf);
}
template <typename T>
inline void readBinaryBigEndian(T & x, ReadBuffer & buf)
{
readBinaryEndian<std::endian::big>(x, buf);
}

View File

@ -1172,33 +1172,45 @@ inline void writeNullTerminatedString(const String & s, WriteBuffer & buffer)
buffer.write(s.c_str(), s.size() + 1);
}
template <typename T>
template <std::endian endian, typename T>
requires is_arithmetic_v<T> && (sizeof(T) <= 8)
inline void writeBinaryBigEndian(T x, WriteBuffer & buf) /// Assuming little endian architecture.
inline void writeBinaryEndian(T x, WriteBuffer & buf)
{
if constexpr (std::endian::native == std::endian::little)
{
if constexpr (sizeof(x) == 2)
x = __builtin_bswap16(x);
else if constexpr (sizeof(x) == 4)
x = __builtin_bswap32(x);
else if constexpr (sizeof(x) == 8)
x = __builtin_bswap64(x);
}
if constexpr (std::endian::native != endian)
std::byteswap(x);
writePODBinary(x, buf);
}
template <typename T>
template <std::endian endian, typename T>
requires is_big_int_v<T>
inline void writeBinaryBigEndian(const T & x, WriteBuffer & buf) /// Assuming little endian architecture.
inline void writeBinaryEndian(const T & x, WriteBuffer & buf)
{
for (size_t i = 0; i != std::size(x.items); ++i)
if constexpr (std::endian::native == endian)
{
const auto & item = x.items[(std::endian::native == std::endian::little) ? std::size(x.items) - i - 1 : i];
writeBinaryBigEndian(item, buf);
for (size_t i = 0; i != std::size(x.items); ++i)
writeBinaryEndian<endian>(x.items[i], buf);
}
else
{
for (ssize_t i = std::size(x.items) - 1; i >= 0; --i)
writeBinaryEndian<endian>(x.items[i], buf);
}
}
template <typename T>
inline void writeBinaryLittleEndian(T x, WriteBuffer & buf)
{
writeBinaryEndian<std::endian::little>(x, buf);
}
template <typename T>
inline void writeBinaryBigEndian(T x, WriteBuffer & buf)
{
writeBinaryEndian<std::endian::big>(x, buf);
}
struct PcgSerializer
{
static void serializePcg32(const pcg32_fast & rng, WriteBuffer & buf)

View File

@ -60,7 +60,7 @@ void fillBufferWithRandomData(char * __restrict data, size_t limit, size_t size_
/// The loop can be further optimized.
UInt64 number = rng();
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
unalignedStoreLE<UInt64>(data, number);
unalignedStoreLittleEndian<UInt64>(data, number);
#else
unalignedStore<UInt64>(data, number);
#endif