Merge pull request #53556 from ClibMouse/feature/big-endian-UUID-correction

Change Big Endian-UUID to work the same as Little Endian-UUID
This commit is contained in:
Robert Schulze 2023-08-22 11:57:44 +02:00 committed by GitHub
commit acaa2eee58
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
32 changed files with 276 additions and 119 deletions

View File

@ -390,7 +390,10 @@ static void transformFixedString(const UInt8 * src, UInt8 * dst, size_t size, UI
static void transformUUID(const UUID & src_uuid, UUID & dst_uuid, UInt64 seed) static void transformUUID(const UUID & src_uuid, UUID & dst_uuid, UInt64 seed)
{ {
const UInt128 & src = src_uuid.toUnderType(); auto src_copy = src_uuid;
transformEndianness<std::endian::little, std::endian::native>(src_copy);
const UInt128 & src = src_copy.toUnderType();
UInt128 & dst = dst_uuid.toUnderType(); UInt128 & dst = dst_uuid.toUnderType();
SipHash hash; SipHash hash;
@ -400,8 +403,9 @@ static void transformUUID(const UUID & src_uuid, UUID & dst_uuid, UInt64 seed)
/// Saving version and variant from an old UUID /// Saving version and variant from an old UUID
dst = hash.get128(); dst = hash.get128();
dst.items[1] = (dst.items[1] & 0x1fffffffffffffffull) | (src.items[1] & 0xe000000000000000ull); const UInt64 trace[2] = {0x000000000000f000ull, 0xe000000000000000ull};
dst.items[0] = (dst.items[0] & 0xffffffffffff0fffull) | (src.items[0] & 0x000000000000f000ull); UUIDHelpers::getLowBytes(dst_uuid) = (UUIDHelpers::getLowBytes(dst_uuid) & (0xffffffffffffffffull - trace[1])) | (UUIDHelpers::getLowBytes(src_uuid) & trace[1]);
UUIDHelpers::getHighBytes(dst_uuid) = (UUIDHelpers::getHighBytes(dst_uuid) & (0xffffffffffffffffull - trace[0])) | (UUIDHelpers::getHighBytes(src_uuid) & trace[0]);
} }
class FixedStringModel : public IModel class FixedStringModel : public IModel

View File

@ -11,6 +11,7 @@
#include <Common/Config/ConfigReloader.h> #include <Common/Config/ConfigReloader.h>
#include <Common/StringUtils/StringUtils.h> #include <Common/StringUtils/StringUtils.h>
#include <Common/quoteString.h> #include <Common/quoteString.h>
#include <Common/TransformEndianness.hpp>
#include <Core/Settings.h> #include <Core/Settings.h>
#include <Interpreters/executeQuery.h> #include <Interpreters/executeQuery.h>
#include <Parsers/Access/ASTGrantQuery.h> #include <Parsers/Access/ASTGrantQuery.h>
@ -49,6 +50,7 @@ namespace
md5.update(type_storage_chars, strlen(type_storage_chars)); md5.update(type_storage_chars, strlen(type_storage_chars));
UUID result; UUID result;
memcpy(&result, md5.digest().data(), md5.digestLength()); memcpy(&result, md5.digest().data(), md5.digestLength());
transformEndianness<std::endian::native, std::endian::little>(result);
return result; return result;
} }

View File

@ -100,6 +100,17 @@ void AggregateFunctionBoundingRatioData::deserialize(ReadBuffer & buf)
} }
} }
inline void writeBinary(const AggregateFunctionBoundingRatioData::Point & p, WriteBuffer & buf)
{
writePODBinary(p, buf);
}
inline void readBinary(AggregateFunctionBoundingRatioData::Point & p, ReadBuffer & buf)
{
readPODBinary(p, buf);
}
class AggregateFunctionBoundingRatio final : public IAggregateFunctionDataHelper<AggregateFunctionBoundingRatioData, AggregateFunctionBoundingRatio> class AggregateFunctionBoundingRatio final : public IAggregateFunctionDataHelper<AggregateFunctionBoundingRatioData, AggregateFunctionBoundingRatio>
{ {
private: private:

View File

@ -783,6 +783,16 @@ public:
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
result[i] = std::numeric_limits<float>::quiet_NaN(); result[i] = std::numeric_limits<float>::quiet_NaN();
} }
friend void writeBinary(const Kind & x, WriteBuffer & buf)
{
writePODBinary(x, buf);
}
friend void readBinary(Kind & x, ReadBuffer & buf)
{
readPODBinary(x, buf);
}
}; };
#undef SMALL_THRESHOLD #undef SMALL_THRESHOLD

View File

@ -276,3 +276,12 @@ private:
return NanLikeValueConstructor<ResultType, std::is_floating_point_v<ResultType>>::getValue(); return NanLikeValueConstructor<ResultType, std::is_floating_point_v<ResultType>>::getValue();
} }
}; };
namespace DB
{
template <typename T>
void readBinary(std::pair<T, UInt32> & x, ReadBuffer & buf)
{
readPODBinary(x, buf);
}
}

View File

@ -2,9 +2,10 @@
#include <city.h> #include <city.h>
#include <Core/Types.h> #include <Core/Types.h>
#include <Core/UUID.h>
#include <base/StringRef.h>
#include <base/types.h> #include <base/types.h>
#include <base/unaligned.h> #include <base/unaligned.h>
#include <base/StringRef.h>
#include <type_traits> #include <type_traits>
@ -406,7 +407,7 @@ struct UInt128TrivialHash
struct UUIDTrivialHash struct UUIDTrivialHash
{ {
size_t operator()(DB::UUID x) const { return x.toUnderType().items[0]; } size_t operator()(DB::UUID x) const { return DB::UUIDHelpers::getHighBytes(x); }
}; };
struct UInt256Hash struct UInt256Hash

View File

@ -5,6 +5,7 @@
#include <Common/Exception.h> #include <Common/Exception.h>
#include <base/hex.h> #include <base/hex.h>
#include <Core/Settings.h> #include <Core/Settings.h>
#include <Core/UUID.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
@ -227,8 +228,8 @@ bool TracingContext::parseTraceparentHeader(std::string_view traceparent, String
++data; ++data;
this->trace_flags = unhex2(data); this->trace_flags = unhex2(data);
this->trace_id.toUnderType().items[0] = trace_id_higher_64; UUIDHelpers::getHighBytes(this->trace_id) = trace_id_higher_64;
this->trace_id.toUnderType().items[1] = trace_id_lower_64; UUIDHelpers::getLowBytes(this->trace_id) = trace_id_lower_64;
this->span_id = span_id_64; this->span_id = span_id_64;
return true; return true;
} }
@ -239,8 +240,8 @@ String TracingContext::composeTraceparentHeader() const
// parent id. // parent id.
return fmt::format( return fmt::format(
"00-{:016x}{:016x}-{:016x}-{:02x}", "00-{:016x}{:016x}-{:016x}-{:02x}",
trace_id.toUnderType().items[0], UUIDHelpers::getHighBytes(trace_id),
trace_id.toUnderType().items[1], UUIDHelpers::getLowBytes(trace_id),
span_id, span_id,
// This cast is needed because fmt is being weird and complaining that // This cast is needed because fmt is being weird and complaining that
// "mixing character types is not allowed". // "mixing character types is not allowed".
@ -335,8 +336,8 @@ TracingContextHolder::TracingContextHolder(
while (_parent_trace_context.trace_id == UUID()) while (_parent_trace_context.trace_id == UUID())
{ {
// Make sure the random generated trace_id is not 0 which is an invalid id. // Make sure the random generated trace_id is not 0 which is an invalid id.
_parent_trace_context.trace_id.toUnderType().items[0] = thread_local_rng(); UUIDHelpers::getHighBytes(_parent_trace_context.trace_id) = thread_local_rng();
_parent_trace_context.trace_id.toUnderType().items[1] = thread_local_rng(); UUIDHelpers::getLowBytes(_parent_trace_context.trace_id) = thread_local_rng();
} }
_parent_trace_context.span_id = 0; _parent_trace_context.span_id = 0;
} }

View File

@ -174,8 +174,8 @@ String GTIDSets::toPayload() const
for (const auto & set : sets) for (const auto & set : sets)
{ {
// MySQL UUID is big-endian. // MySQL UUID is big-endian.
writeBinaryBigEndian(set.uuid.toUnderType().items[0], buffer); writeBinaryBigEndian(UUIDHelpers::getHighBytes(set.uuid), buffer);
writeBinaryBigEndian(set.uuid.toUnderType().items[1], buffer); writeBinaryBigEndian(UUIDHelpers::getLowBytes(set.uuid), buffer);
UInt64 intervals_size = set.intervals.size(); UInt64 intervals_size = set.intervals.size();
buffer.write(reinterpret_cast<const char *>(&intervals_size), 8); buffer.write(reinterpret_cast<const char *>(&intervals_size), 8);

View File

@ -940,13 +940,8 @@ namespace MySQLReplication
payload.readStrict(reinterpret_cast<char *>(&commit_flag), 1); payload.readStrict(reinterpret_cast<char *>(&commit_flag), 1);
// MySQL UUID is big-endian. // MySQL UUID is big-endian.
UInt64 high = 0UL; readBinaryBigEndian(UUIDHelpers::getHighBytes(gtid.uuid), payload);
UInt64 low = 0UL; readBinaryBigEndian(UUIDHelpers::getLowBytes(gtid.uuid), payload);
readBigEndianStrict(payload, reinterpret_cast<char *>(&low), 8);
gtid.uuid.toUnderType().items[0] = low;
readBigEndianStrict(payload, reinterpret_cast<char *>(&high), 8);
gtid.uuid.toUnderType().items[1] = high;
payload.readStrict(reinterpret_cast<char *>(&gtid.seq_no), 8); payload.readStrict(reinterpret_cast<char *>(&gtid.seq_no), 8);

View File

@ -33,8 +33,10 @@ namespace MySQLReplication
inline void readBigEndianStrict(ReadBuffer & payload, char * to, size_t n) inline void readBigEndianStrict(ReadBuffer & payload, char * to, size_t n)
{ {
payload.readStrict(to, n); payload.readStrict(to, n);
#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
char *start = to, *end = to + n; char *start = to, *end = to + n;
std::reverse(start, end); std::reverse(start, end);
#endif
} }
inline void readTimeFractionalPart(ReadBuffer & payload, UInt32 & factional, UInt16 meta) inline void readTimeFractionalPart(ReadBuffer & payload, UInt32 & factional, UInt16 meta)

View File

@ -9,10 +9,11 @@ namespace UUIDHelpers
{ {
UUID generateV4() UUID generateV4()
{ {
UInt128 res{thread_local_rng(), thread_local_rng()}; UUID uuid;
res.items[0] = (res.items[0] & 0xffffffffffff0fffull) | 0x0000000000004000ull; getHighBytes(uuid) = (thread_local_rng() & 0xffffffffffff0fffull) | 0x0000000000004000ull;
res.items[1] = (res.items[1] & 0x3fffffffffffffffull) | 0x8000000000000000ull; getLowBytes(uuid) = (thread_local_rng() & 0x3fffffffffffffffull) | 0x8000000000000000ull;
return UUID{res};
return uuid;
} }
} }

View File

@ -2,6 +2,59 @@
#include <Core/Types.h> #include <Core/Types.h>
/**
* Implementation Details
* ^^^^^^^^^^^^^^^^^^^^^^
* The underlying implementation for a UUID has it represented as a 128-bit unsigned integer. Underlying this, a wide
* integer with a 64-bit unsigned integer as its base is utilized. This wide integer can be interfaced with as an array
* to access different components of the base. For example, on a Little Endian platform, accessing at index 0 will give
* you the 8 higher bytes, and index 1 will give you the 8 lower bytes. On a Big Endian platform, this is reversed where
* index 0 will give you the 8 lower bytes, and index 1 will give you the 8 higher bytes.
*
* uuid.toUnderType().items[0]
*
* // uint64_t uint64_t
* // [xxxxxxxx] [ ]
*
* uuid.toUnderType().items[1]
*
* // uint64_t uint64_t
* // [ ] [xxxxxxxx]
*
* The way that data is stored in the underlying wide integer treats the data as two 64-bit chunks sequenced in the
* array. On a Little Endian platform, this results in the following layout
*
* // Suppose uuid contains 61f0c404-5cb3-11e7-907b-a6006ad3dba0
*
* uuid.toUnderType().items[0]
*
* // uint64_t as HEX
* // [E7 11 B3 5C 04 C4 F0 61] [A0 DB D3 6A 00 A6 7B 90]
* // ^^^^^^^^^^^^^^^^^^^^^^^
*
* uuid.toUnderType().items[1]
*
* // uint64_t as HEX
* // [E7 11 B3 5C 04 C4 F0 61] [A0 DB D3 6A 00 A6 7B 90]
* // ^^^^^^^^^^^^^^^^^^^^^^^
*
* while on a Big Endian platform this would be
*
* // Suppose uuid contains 61f0c404-5cb3-11e7-907b-a6006ad3dba0
*
* uuid.toUnderType().items[0]
*
* // uint64_t as HEX
* // [90 7B A6 00 6A D3 DB A0] [61 F0 C4 04 5C B3 11 E7]
* // ^^^^^^^^^^^^^^^^^^^^^^^
*
* uuid.toUnderType().items[1]
*
* // uint64_t as HEX
* // [90 7B A6 00 6A D3 DB A0] [61 F0 C4 04 5C B3 11 E7]
* // ^^^^^^^^^^^^^^^^^^^^^^^
*/
namespace DB namespace DB
{ {
@ -11,6 +64,29 @@ namespace UUIDHelpers
/// Generate random UUID. /// Generate random UUID.
UUID generateV4(); UUID generateV4();
constexpr size_t HighBytes = (std::endian::native == std::endian::little) ? 0 : 1;
constexpr size_t LowBytes = (std::endian::native == std::endian::little) ? 1 : 0;
inline uint64_t getHighBytes(const UUID & uuid)
{
return uuid.toUnderType().items[HighBytes];
}
inline uint64_t & getHighBytes(UUID & uuid)
{
return uuid.toUnderType().items[HighBytes];
}
inline uint64_t getLowBytes(const UUID & uuid)
{
return uuid.toUnderType().items[LowBytes];
}
inline uint64_t & getLowBytes(UUID & uuid)
{
return uuid.toUnderType().items[LowBytes];
}
const UUID Nil{}; const UUID Nil{};
} }

View File

@ -111,25 +111,25 @@ void SerializationUUID::deserializeTextCSV(IColumn & column, ReadBuffer & istr,
void SerializationUUID::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const void SerializationUUID::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const
{ {
UUID x = field.get<UUID>(); UUID x = field.get<UUID>();
writeBinary(x, ostr); writeBinaryLittleEndian(x, ostr);
} }
void SerializationUUID::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const void SerializationUUID::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const
{ {
UUID x; UUID x;
readBinary(x, istr); readBinaryLittleEndian(x, istr);
field = NearestFieldType<UUID>(x); field = NearestFieldType<UUID>(x);
} }
void SerializationUUID::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const void SerializationUUID::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
{ {
writeBinary(assert_cast<const ColumnVector<UUID> &>(column).getData()[row_num], ostr); writeBinaryLittleEndian(assert_cast<const ColumnVector<UUID> &>(column).getData()[row_num], ostr);
} }
void SerializationUUID::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const void SerializationUUID::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{ {
UUID x; UUID x;
readBinary(x, istr); readBinaryLittleEndian(x, istr);
assert_cast<ColumnVector<UUID> &>(column).getData().push_back(x); assert_cast<ColumnVector<UUID> &>(column).getData().push_back(x);
} }

View File

@ -507,8 +507,8 @@ public:
// use executeOnUInt instead of using executeOneString // use executeOnUInt instead of using executeOneString
// because the latter one outputs the string in the memory order // because the latter one outputs the string in the memory order
Impl::executeOneUIntOrInt(uuid[i].toUnderType().items[0], end, false, false); Impl::executeOneUIntOrInt(UUIDHelpers::getHighBytes(uuid[i]), end, false, false);
Impl::executeOneUIntOrInt(uuid[i].toUnderType().items[1], end, false, true); Impl::executeOneUIntOrInt(UUIDHelpers::getLowBytes(uuid[i]), end, false, true);
pos += end - begin; pos += end - begin;
out_offsets[i] = pos; out_offsets[i] = pos;

View File

@ -203,18 +203,15 @@ struct ConvertImpl
} }
} }
if constexpr (std::is_same_v<FromDataType, DataTypeUUID> && std::is_same_v<ToDataType,DataTypeUInt128>) if constexpr (std::is_same_v<FromDataType, DataTypeUUID> && std::is_same_v<ToDataType, DataTypeUInt128>)
{
static_assert(std::is_same_v<DataTypeUInt128::FieldType, DataTypeUUID::FieldType::UnderlyingType>, "UInt128 and UUID types must be same");
if constexpr (std::endian::native == std::endian::little)
{ {
static_assert(
std::is_same_v<DataTypeUInt128::FieldType, DataTypeUUID::FieldType::UnderlyingType>,
"UInt128 and UUID types must be same");
vec_to[i].items[1] = vec_from[i].toUnderType().items[0]; vec_to[i].items[1] = vec_from[i].toUnderType().items[0];
vec_to[i].items[0] = vec_from[i].toUnderType().items[1]; vec_to[i].items[0] = vec_from[i].toUnderType().items[1];
}
else
{
vec_to[i] = vec_from[i].toUnderType();
}
continue; continue;
} }

View File

@ -60,9 +60,8 @@ public:
{ {
/// https://tools.ietf.org/html/rfc4122#section-4.4 /// https://tools.ietf.org/html/rfc4122#section-4.4
UInt128 & impl = uuid.toUnderType(); UUIDHelpers::getHighBytes(uuid) = (UUIDHelpers::getHighBytes(uuid) & 0xffffffffffff0fffull) | 0x0000000000004000ull;
impl.items[0] = (impl.items[0] & 0xffffffffffff0fffull) | 0x0000000000004000ull; UUIDHelpers::getLowBytes(uuid) = (UUIDHelpers::getLowBytes(uuid) & 0x3fffffffffffffffull) | 0x8000000000000000ull;
impl.items[1] = (impl.items[1] & 0x3fffffffffffffffull) | 0x8000000000000000ull;
} }
return col_res; return col_res;

View File

@ -1,26 +1,27 @@
#include <Functions/FunctionFactory.h> #include <Functions/FunctionFactory.h>
#include <Functions/castTypeToEither.h>
#include <Functions/FunctionHelpers.h> #include <Functions/FunctionHelpers.h>
#include <Functions/castTypeToEither.h>
#include <Core/callOnTypeIndex.h> #include <Core/callOnTypeIndex.h>
#include <DataTypes/DataTypesNumber.h> #include <Columns/ColumnConst.h>
#include <DataTypes/DataTypeString.h> #include <Columns/ColumnDecimal.h>
#include <DataTypes/DataTypeFixedString.h> #include <Columns/ColumnFixedString.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <DataTypes/DataTypeDate.h> #include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h> #include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeDateTime64.h> #include <DataTypes/DataTypeDateTime64.h>
#include <Columns/ColumnString.h> #include <DataTypes/DataTypeFactory.h>
#include <Columns/ColumnFixedString.h> #include <DataTypes/DataTypeFixedString.h>
#include <Columns/ColumnConst.h> #include <DataTypes/DataTypeString.h>
#include <Columns/ColumnVector.h> #include <DataTypes/DataTypeUUID.h>
#include <Columns/ColumnDecimal.h> #include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypesNumber.h>
#include <Common/typeid_cast.h> #include <Common/TransformEndianness.hpp>
#include <Common/memcpySmall.h> #include <Common/memcpySmall.h>
#include <Common/typeid_cast.h>
#include <base/unaligned.h> #include <base/unaligned.h>
@ -261,8 +262,10 @@ public:
memcpy(static_cast<void*>(&to[i]), static_cast<const void*>(&from[i]), copy_size); memcpy(static_cast<void*>(&to[i]), static_cast<const void*>(&from[i]), copy_size);
else else
{ {
size_t offset_to = sizeof(To) > sizeof(From) ? sizeof(To) - sizeof(From) : 0; // Handle the cases of both 128-bit representation to 256-bit and 128-bit to 64-bit or lower.
memcpy(reinterpret_cast<char*>(&to[i]) + offset_to, static_cast<const void*>(&from[i]), copy_size); const size_t offset_from = sizeof(From) > sizeof(To) ? sizeof(From) - sizeof(To) : 0;
const size_t offset_to = sizeof(To) > sizeof(From) ? sizeof(To) - sizeof(From) : 0;
memcpy(reinterpret_cast<char *>(&to[i]) + offset_to, reinterpret_cast<const char *>(&from[i]) + offset_from, copy_size);
} }
} }
@ -315,7 +318,11 @@ private:
{ {
std::string_view data = src.getDataAt(i).toView(); std::string_view data = src.getDataAt(i).toView();
#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
memcpy(&data_to[offset], data.data(), std::min(n, data.size())); memcpy(&data_to[offset], data.data(), std::min(n, data.size()));
#else
reverseMemcpy(&data_to[offset], data.data(), std::min(n, data.size()));
#endif
offset += n; offset += n;
} }
} }
@ -326,7 +333,11 @@ private:
ColumnFixedString::Chars & data_to = dst.getChars(); ColumnFixedString::Chars & data_to = dst.getChars();
data_to.resize(n * rows); data_to.resize(n * rows);
#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
memcpy(data_to.data(), src.getRawData().data(), data_to.size()); memcpy(data_to.data(), src.getRawData().data(), data_to.size());
#else
reverseMemcpy(data_to.data(), src.getRawData().data(), data_to.size());
#endif
} }
static void NO_INLINE executeToString(const IColumn & src, ColumnString & dst) static void NO_INLINE executeToString(const IColumn & src, ColumnString & dst)

View File

@ -53,36 +53,25 @@ UUID parseUUID(std::span<const UInt8> src)
{ {
UUID uuid; UUID uuid;
const auto * src_ptr = src.data(); const auto * src_ptr = src.data();
auto * dst = reinterpret_cast<UInt8 *>(&uuid);
const auto size = src.size(); const auto size = src.size();
#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ #if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
const std::reverse_iterator dst_it(dst + sizeof(UUID)); const std::reverse_iterator dst(reinterpret_cast<UInt8 *>(&uuid) + sizeof(UUID));
#else
auto * dst = reinterpret_cast<UInt8 *>(&uuid);
#endif #endif
if (size == 36) if (size == 36)
{ {
#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ parseHex<4>(src_ptr, dst + 8);
parseHex<4>(src_ptr, dst_it + 8); parseHex<2>(src_ptr + 9, dst + 12);
parseHex<2>(src_ptr + 9, dst_it + 12); parseHex<2>(src_ptr + 14, dst + 14);
parseHex<2>(src_ptr + 14, dst_it + 14); parseHex<2>(src_ptr + 19, dst);
parseHex<2>(src_ptr + 19, dst_it); parseHex<6>(src_ptr + 24, dst + 2);
parseHex<6>(src_ptr + 24, dst_it + 2);
#else
parseHex<4>(src_ptr, dst);
parseHex<2>(src_ptr + 9, dst + 4);
parseHex<2>(src_ptr + 14, dst + 6);
parseHex<2>(src_ptr + 19, dst + 8);
parseHex<6>(src_ptr + 24, dst + 10);
#endif
} }
else if (size == 32) else if (size == 32)
{ {
#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ parseHex<8>(src_ptr, dst + 8);
parseHex<8>(src_ptr, dst_it + 8); parseHex<8>(src_ptr + 16, dst);
parseHex<8>(src_ptr + 16, dst_it);
#else
parseHex<16>(src_ptr, dst);
#endif
} }
else else
throw Exception(ErrorCodes::CANNOT_PARSE_UUID, "Unexpected length when trying to parse UUID ({})", size); throw Exception(ErrorCodes::CANNOT_PARSE_UUID, "Unexpected length when trying to parse UUID ({})", size);

View File

@ -116,6 +116,13 @@ inline void readPODBinary(T & x, ReadBuffer & buf)
buf.readStrict(reinterpret_cast<char *>(&x), sizeof(x)); /// NOLINT buf.readStrict(reinterpret_cast<char *>(&x), sizeof(x)); /// NOLINT
} }
inline void readUUIDBinary(UUID & x, ReadBuffer & buf)
{
auto & uuid = x.toUnderType();
readPODBinary(uuid.items[0], buf);
readPODBinary(uuid.items[1], buf);
}
template <typename T> template <typename T>
inline void readIntBinary(T & x, ReadBuffer & buf) inline void readIntBinary(T & x, ReadBuffer & buf)
{ {
@ -1106,16 +1113,26 @@ inline void readBinary(Decimal64 & x, ReadBuffer & buf) { readPODBinary(x, buf);
inline void readBinary(Decimal128 & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(Decimal128 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(Decimal256 & x, ReadBuffer & buf) { readPODBinary(x.value, buf); } inline void readBinary(Decimal256 & x, ReadBuffer & buf) { readPODBinary(x.value, buf); }
inline void readBinary(LocalDate & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(LocalDate & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(UUID & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(IPv4 & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(IPv4 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(IPv6 & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(IPv6 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(UUID & x, ReadBuffer & buf)
{
readUUIDBinary(x, buf);
}
inline void readBinary(CityHash_v1_0_2::uint128 & x, ReadBuffer & buf)
{
readPODBinary(x.low64, buf);
readPODBinary(x.high64, buf);
}
inline void readBinary(StackTrace::FramePointers & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(StackTrace::FramePointers & x, ReadBuffer & buf) { readPODBinary(x, buf); }
template <std::endian endian, typename T> template <std::endian endian, typename T>
inline void readBinaryEndian(T & x, ReadBuffer & buf) inline void readBinaryEndian(T & x, ReadBuffer & buf)
{ {
readPODBinary(x, buf); readBinary(x, buf);
transformEndianness<endian>(x); transformEndianness<endian>(x);
} }

View File

@ -23,30 +23,23 @@ void formatHex(IteratorSrc src, IteratorDst dst, size_t num_bytes)
std::array<char, 36> formatUUID(const UUID & uuid) std::array<char, 36> formatUUID(const UUID & uuid)
{ {
std::array<char, 36> dst; std::array<char, 36> dst;
const auto * src_ptr = reinterpret_cast<const UInt8 *>(&uuid);
auto * dst_ptr = dst.data(); auto * dst_ptr = dst.data();
#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__ #if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
const std::reverse_iterator src_it(src_ptr + 16); const auto * src_ptr = reinterpret_cast<const UInt8 *>(&uuid);
formatHex(src_it + 8, dst_ptr, 4); const std::reverse_iterator src(src_ptr + 16);
dst[8] = '-';
formatHex(src_it + 12, dst_ptr + 9, 2);
dst[13] = '-';
formatHex(src_it + 14, dst_ptr + 14, 2);
dst[18] = '-';
formatHex(src_it, dst_ptr + 19, 2);
dst[23] = '-';
formatHex(src_it + 2, dst_ptr + 24, 6);
#else #else
formatHex(src_ptr, dst_ptr, 4); const auto * src = reinterpret_cast<const UInt8 *>(&uuid);
dst[8] = '-';
formatHex(src_ptr + 4, dst_ptr + 9, 2);
dst[13] = '-';
formatHex(src_ptr + 6, dst_ptr + 14, 2);
dst[18] = '-';
formatHex(src_ptr + 8, dst_ptr + 19, 2);
dst[23] = '-';
formatHex(src_ptr + 10, dst_ptr + 24, 6);
#endif #endif
formatHex(src + 8, dst_ptr, 4);
dst[8] = '-';
formatHex(src + 12, dst_ptr + 9, 2);
dst[13] = '-';
formatHex(src + 14, dst_ptr + 14, 2);
dst[18] = '-';
formatHex(src, dst_ptr + 19, 2);
dst[23] = '-';
formatHex(src + 2, dst_ptr + 24, 6);
return dst; return dst;
} }

View File

@ -88,6 +88,13 @@ inline void writePODBinary(const T & x, WriteBuffer & buf)
buf.write(reinterpret_cast<const char *>(&x), sizeof(x)); /// NOLINT buf.write(reinterpret_cast<const char *>(&x), sizeof(x)); /// NOLINT
} }
inline void writeUUIDBinary(const UUID & x, WriteBuffer & buf)
{
const auto & uuid = x.toUnderType();
writePODBinary(uuid.items[0], buf);
writePODBinary(uuid.items[1], buf);
}
template <typename T> template <typename T>
inline void writeIntBinary(const T & x, WriteBuffer & buf) inline void writeIntBinary(const T & x, WriteBuffer & buf)
{ {
@ -882,10 +889,20 @@ inline void writeBinary(const Decimal128 & x, WriteBuffer & buf) { writePODBinar
inline void writeBinary(const Decimal256 & x, WriteBuffer & buf) { writePODBinary(x.value, buf); } inline void writeBinary(const Decimal256 & x, WriteBuffer & buf) { writePODBinary(x.value, buf); }
inline void writeBinary(const LocalDate & x, WriteBuffer & buf) { writePODBinary(x, buf); } inline void writeBinary(const LocalDate & x, WriteBuffer & buf) { writePODBinary(x, buf); }
inline void writeBinary(const LocalDateTime & x, WriteBuffer & buf) { writePODBinary(x, buf); } inline void writeBinary(const LocalDateTime & x, WriteBuffer & buf) { writePODBinary(x, buf); }
inline void writeBinary(const UUID & x, WriteBuffer & buf) { writePODBinary(x, buf); }
inline void writeBinary(const IPv4 & x, WriteBuffer & buf) { writePODBinary(x, buf); } inline void writeBinary(const IPv4 & x, WriteBuffer & buf) { writePODBinary(x, buf); }
inline void writeBinary(const IPv6 & x, WriteBuffer & buf) { writePODBinary(x, buf); } inline void writeBinary(const IPv6 & x, WriteBuffer & buf) { writePODBinary(x, buf); }
inline void writeBinary(const UUID & x, WriteBuffer & buf)
{
writeUUIDBinary(x, buf);
}
inline void writeBinary(const CityHash_v1_0_2::uint128 & x, WriteBuffer & buf)
{
writePODBinary(x.low64, buf);
writePODBinary(x.high64, buf);
}
inline void writeBinary(const StackTrace::FramePointers & x, WriteBuffer & buf) { writePODBinary(x, buf); } inline void writeBinary(const StackTrace::FramePointers & x, WriteBuffer & buf) { writePODBinary(x, buf); }
/// Methods for outputting the value in text form for a tab-separated format. /// Methods for outputting the value in text form for a tab-separated format.
@ -1208,7 +1225,7 @@ template <std::endian endian, typename T>
inline void writeBinaryEndian(T x, WriteBuffer & buf) inline void writeBinaryEndian(T x, WriteBuffer & buf)
{ {
transformEndianness<endian>(x); transformEndianness<endian>(x);
writePODBinary(x, buf); writeBinary(x, buf);
} }
template <typename T> template <typename T>

View File

@ -308,7 +308,7 @@ private:
static inline size_t getFirstLevelIdx(const UUID & uuid) static inline size_t getFirstLevelIdx(const UUID & uuid)
{ {
return uuid.toUnderType().items[0] >> (64 - bits_for_first_level); return UUIDHelpers::getHighBytes(uuid) >> (64 - bits_for_first_level);
} }
void dropTableDataTask(); void dropTableDataTask();

View File

@ -326,8 +326,8 @@ static void insertUUID(IColumn & column, DataTypePtr type, const char * value, s
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack UUID into column with type {}.", type->getName()); throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack UUID into column with type {}.", type->getName());
ReadBufferFromMemory buf(value, size); ReadBufferFromMemory buf(value, size);
UUID uuid; UUID uuid;
readBinaryBigEndian(uuid.toUnderType().items[0], buf); readBinaryBigEndian(UUIDHelpers::getHighBytes(uuid), buf);
readBinaryBigEndian(uuid.toUnderType().items[1], buf); readBinaryBigEndian(UUIDHelpers::getLowBytes(uuid), buf);
assert_cast<ColumnUUID &>(column).insertValue(uuid); assert_cast<ColumnUUID &>(column).insertValue(uuid);
} }

View File

@ -270,8 +270,8 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr
{ {
WriteBufferFromOwnString buf; WriteBufferFromOwnString buf;
UUID value = uuid_column.getElement(row_num); UUID value = uuid_column.getElement(row_num);
writeBinaryBigEndian(value.toUnderType().items[0], buf); writeBinaryBigEndian(UUIDHelpers::getHighBytes(value), buf);
writeBinaryBigEndian(value.toUnderType().items[1], buf); writeBinaryBigEndian(UUIDHelpers::getLowBytes(value), buf);
std::string_view uuid_ext = buf.stringView(); std::string_view uuid_ext = buf.stringView();
packer.pack_ext(sizeof(UUID), int8_t(MsgPackExtensionTypes::UUIDType)); packer.pack_ext(sizeof(UUID), int8_t(MsgPackExtensionTypes::UUIDType));
packer.pack_ext_body(uuid_ext.data(), static_cast<unsigned>(uuid_ext.size())); packer.pack_ext_body(uuid_ext.data(), static_cast<unsigned>(uuid_ext.size()));

View File

@ -84,15 +84,7 @@ namespace
} }
void operator() (const UUID & x) const void operator() (const UUID & x) const
{ {
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
auto tmp_x = x.toUnderType();
char * start = reinterpret_cast<char *>(&tmp_x);
char * end = start + sizeof(tmp_x);
std::reverse(start, end);
operator()(tmp_x);
#else
operator()(x.toUnderType()); operator()(x.toUnderType());
#endif
} }
void operator() (const IPv4 & x) const void operator() (const IPv4 & x) const
{ {

View File

@ -360,7 +360,7 @@ ColumnPtr fillColumnWithRandomData(
auto column = ColumnUUID::create(); auto column = ColumnUUID::create();
column->getData().resize(limit); column->getData().resize(limit);
/// NOTE This is slightly incorrect as random UUIDs should have fixed version 4. /// NOTE This is slightly incorrect as random UUIDs should have fixed version 4.
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit, sizeof(UUID), rng); fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit, sizeof(UUID), rng, true);
return column; return column;
} }
case TypeIndex::Int8: case TypeIndex::Int8:

View File

@ -4,9 +4,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh # shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh . "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS t_uuid"
$CLICKHOUSE_CLIENT --query="CREATE TABLE t_uuid(Id UUID) ENGINE=MergeTree ORDER BY (Id)" $CLICKHOUSE_CLIENT --query="CREATE TABLE t_uuid(Id UUID) ENGINE=MergeTree ORDER BY (Id)"
$CLICKHOUSE_CLIENT --query="INSERT INTO t_uuid VALUES ('3f5ffba3-19ff-4f3d-8861-60ae6e1fc1aa'),('4bd62524-e33c-43e5-882d-f1d96cf5561e'),('7a8b45d2-c18b-4e8c-89eb-abf5bee88931'),('45bb7333-965b-4526-870e-4f941edb025b'),('a4e72d0e-f9fa-465e-8d9d-151b9ced94df'),('cb5818ab-83b5-48a8-94b0-5177e30176d9'),('701e8006-fc9f-4496-80ba-efa6817b917b'),('e0936acf-6e8f-42aa-8f56-d1363476eece'),('239bb790-5293-40df-92ae-472294b6e178'),('508d0e80-729f-4e3b-9336-4c5c8792f6be'),('94abef70-f2d6-4f7b-ad60-3889409f1dac'),('b6f1ec08-8473-4fa2-b134-73db040b0d82'),('7e54dcae-0bb4-4c4f-a636-54a705fb8b40'),('d1d258c2-a35f-4c00-abfa-8addbcbc5471'),('7c74fbd8-bf79-46ee-adfe-96271040a4f7'),('41e3a274-eea9-41d8-a128-de5a6658fcfd'),('a72dc048-f72f-470e-b0f9-60cfad6e1157'),('40634f4f-37bf-44e4-ac7c-6f024ad19990')" $CLICKHOUSE_CLIENT --query="INSERT INTO t_uuid VALUES ('3f5ffba3-19ff-4f3d-8861-60ae6e1fc1aa'),('4bd62524-e33c-43e5-882d-f1d96cf5561e'),('7a8b45d2-c18b-4e8c-89eb-abf5bee88931'),('45bb7333-965b-4526-870e-4f941edb025b'),('a4e72d0e-f9fa-465e-8d9d-151b9ced94df'),('cb5818ab-83b5-48a8-94b0-5177e30176d9'),('701e8006-fc9f-4496-80ba-efa6817b917b'),('e0936acf-6e8f-42aa-8f56-d1363476eece'),('239bb790-5293-40df-92ae-472294b6e178'),('508d0e80-729f-4e3b-9336-4c5c8792f6be'),('94abef70-f2d6-4f7b-ad60-3889409f1dac'),('b6f1ec08-8473-4fa2-b134-73db040b0d82'),('7e54dcae-0bb4-4c4f-a636-54a705fb8b40'),('d1d258c2-a35f-4c00-abfa-8addbcbc5471'),('7c74fbd8-bf79-46ee-adfe-96271040a4f7'),('41e3a274-eea9-41d8-a128-de5a6658fcfd'),('a72dc048-f72f-470e-b0f9-60cfad6e1157'),('40634f4f-37bf-44e4-ac7c-6f024ad19990')"
$CLICKHOUSE_CLIENT --query="SELECT Id FROM t_uuid FORMAT TSV" > "${CLICKHOUSE_TMP}"/data.tsv $CLICKHOUSE_CLIENT --query="SELECT Id FROM t_uuid ORDER BY (Id) FORMAT TSV" > "${CLICKHOUSE_TMP}"/data.tsv
echo FROM RAW DATA && cat "${CLICKHOUSE_TMP}"/data.tsv echo FROM RAW DATA && cat "${CLICKHOUSE_TMP}"/data.tsv
echo TRANSFORMED TO && $CLICKHOUSE_OBFUSCATOR --structure "Id UUID" --input-format TSV --output-format TSV --seed dsrub < "${CLICKHOUSE_TMP}"/data.tsv 2>/dev/null echo TRANSFORMED TO && $CLICKHOUSE_OBFUSCATOR --structure "Id UUID" --input-format TSV --output-format TSV --seed dsrub < "${CLICKHOUSE_TMP}"/data.tsv 2>/dev/null

View File

@ -1,3 +1,5 @@
DROP TABLE IF EXISTS events;
create table events ( create table events (
`organisation_id` UUID, `organisation_id` UUID,
`session_id` UUID, `session_id` UUID,

View File

@ -0,0 +1,9 @@
DROP TABLE IF EXISTS t_uuid;
CREATE TABLE t_uuid (x UUID) ENGINE=MergeTree ORDER BY x;
INSERT INTO t_uuid VALUES ('61f0c404-5cb3-11e7-907b-a6006ad3dba0'), ('992f6910-42b2-43cd-98bc-c812fbf9b683'), ('417ddc5d-e556-4d27-95dd-a34d84e46a50');
SELECT * FROM t_uuid ORDER BY x LIMIT 1 FORMAT RowBinary;
SELECT * FROM t_uuid ORDER BY x FORMAT RowBinary;
DROP TABLE IF EXISTS t_uuid;

View File

@ -0,0 +1,5 @@
61f0c404-5cb3-11e7-907b-a6006ad3dba0
403229640000000000 6.034192082918747e163
-25 4583 1555239399 7057356139103719911 -148231516101255056243829344033567469081 192050850819683407219545263398200742375
231 4583 1555239399 7057356139103719911 192050850819683407219545263398200742375 192050850819683407219545263398200742375
00000000-5cb3-11e7-0000-000000000000

View File

@ -0,0 +1,13 @@
DROP TABLE IF EXISTS t_uuid;
CREATE TABLE t_uuid (x UUID) ENGINE=MergeTree ORDER BY x;
INSERT INTO t_uuid VALUES ('61f0c404-5cb3-11e7-907b-a6006ad3dba0');
SELECT reinterpretAsUUID(x) FROM t_uuid;
SELECT reinterpretAsFloat32(x), reinterpretAsFloat64(x) FROM t_uuid;
SELECT reinterpretAsInt8(x), reinterpretAsInt16(x), reinterpretAsInt32(x), reinterpretAsInt64(x), reinterpretAsInt128(x), reinterpretAsInt256(x) FROM t_uuid;
SELECT reinterpretAsUInt8(x), reinterpretAsUInt16(x), reinterpretAsUInt32(x), reinterpretAsUInt64(x), reinterpretAsUInt128(x), reinterpretAsUInt256(x) FROM t_uuid;
SELECT reinterpretAsUUID(reinterpretAsUInt128(reinterpretAsUInt32(reinterpretAsUInt256(x)))) FROM t_uuid;
DROP TABLE IF EXISTS t_uuid;