Change BE-UUID to work the same as LE-UUID. Included high and low getters to provide cleaner code when accessing undertype.

This commit is contained in:
Austin Kothig 2023-08-16 16:12:31 -04:00 committed by kothiga
parent a6684f7a37
commit 6b42975d33
No known key found for this signature in database
31 changed files with 232 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)
{
const UInt128 & src = src_uuid.toUnderType();
auto src_copy = src_uuid;
transformEndianness<std::endian::native, std::endian::little>(src_copy);
const UInt128 & src = src_copy.toUnderType();
UInt128 & dst = dst_uuid.toUnderType();
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
dst = hash.get128();
dst.items[1] = (dst.items[1] & 0x1fffffffffffffffull) | (src.items[1] & 0xe000000000000000ull);
dst.items[0] = (dst.items[0] & 0xffffffffffff0fffull) | (src.items[0] & 0x000000000000f000ull);
const UInt64 trace[2] = {0x000000000000f000ull, 0xe000000000000000ull};
UUIDHelpers::getUUIDLow(dst_uuid) = (UUIDHelpers::getUUIDLow(dst_uuid) & (0xffffffffffffffffull - trace[1])) | (UUIDHelpers::getUUIDLow(src_uuid) & trace[1]);
UUIDHelpers::getUUIDHigh(dst_uuid) = (UUIDHelpers::getUUIDHigh(dst_uuid) & (0xffffffffffffffffull - trace[0])) | (UUIDHelpers::getUUIDHigh(src_uuid) & trace[0]);
}
class FixedStringModel : public IModel

View File

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

View File

@ -783,6 +783,16 @@ public:
for (size_t i = 0; i < size; ++i)
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

View File

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

View File

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

View File

@ -174,8 +174,8 @@ String GTIDSets::toPayload() const
for (const auto & set : sets)
{
// MySQL UUID is big-endian.
writeBinaryBigEndian(set.uuid.toUnderType().items[0], buffer);
writeBinaryBigEndian(set.uuid.toUnderType().items[1], buffer);
writeBinaryBigEndian(UUIDHelpers::getUUIDHigh(set.uuid), buffer);
writeBinaryBigEndian(UUIDHelpers::getUUIDLow(set.uuid), buffer);
UInt64 intervals_size = set.intervals.size();
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);
// MySQL UUID is big-endian.
UInt64 high = 0UL;
UInt64 low = 0UL;
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;
readBinaryBigEndian(UUIDHelpers::getUUIDHigh(gtid.uuid), payload);
readBinaryBigEndian(UUIDHelpers::getUUIDLow(gtid.uuid), payload);
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)
{
payload.readStrict(to, n);
#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
char *start = to, *end = to + n;
std::reverse(start, end);
#endif
}
inline void readTimeFractionalPart(ReadBuffer & payload, UInt32 & factional, UInt16 meta)

View File

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

View File

@ -11,6 +11,40 @@ namespace UUIDHelpers
/// Generate random UUID.
UUID generateV4();
const size_t HighBytes =
#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
0;
#else
1;
#endif
const size_t LowBytes =
#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
1;
#else
0;
#endif
inline uint64_t getUUIDHigh(const UUID & uuid)
{
return uuid.toUnderType().items[HighBytes];
}
inline uint64_t & getUUIDHigh(UUID & uuid)
{
return uuid.toUnderType().items[HighBytes];
}
inline uint64_t getUUIDLow(const UUID & uuid)
{
return uuid.toUnderType().items[LowBytes];
}
inline uint64_t & getUUIDLow(UUID & uuid)
{
return uuid.toUnderType().items[LowBytes];
}
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
{
UUID x = field.get<UUID>();
writeBinary(x, ostr);
writeBinaryLittleEndian(x, ostr);
}
void SerializationUUID::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const
{
UUID x;
readBinary(x, istr);
readBinaryLittleEndian(x, istr);
field = NearestFieldType<UUID>(x);
}
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
{
UUID x;
readBinary(x, istr);
readBinaryLittleEndian(x, istr);
assert_cast<ColumnVector<UUID> &>(column).getData().push_back(x);
}

View File

@ -507,8 +507,8 @@ public:
// use executeOnUInt instead of using executeOneString
// because the latter one outputs the string in the memory order
Impl::executeOneUIntOrInt(uuid[i].toUnderType().items[0], end, false, false);
Impl::executeOneUIntOrInt(uuid[i].toUnderType().items[1], end, false, true);
Impl::executeOneUIntOrInt(UUIDHelpers::getUUIDHigh(uuid[i]), end, false, false);
Impl::executeOneUIntOrInt(UUIDHelpers::getUUIDLow(uuid[i]), end, false, true);
pos += end - begin;
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)
{
vec_to[i].items[1] = vec_from[i].toUnderType().items[0];
vec_to[i].items[0] = vec_from[i].toUnderType().items[1];
}
else
{
vec_to[i] = vec_from[i].toUnderType();
}
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[0] = vec_from[i].toUnderType().items[1];
continue;
}

View File

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

View File

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

View File

@ -51,36 +51,25 @@ UUID parseUUID(std::span<const UInt8> src)
{
UUID uuid;
const auto * src_ptr = src.data();
auto * dst = reinterpret_cast<UInt8 *>(&uuid);
const auto size = src.size();
#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
if (size == 36)
{
#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
parseHex<4>(src_ptr, dst_it + 8);
parseHex<2>(src_ptr + 9, dst_it + 12);
parseHex<2>(src_ptr + 14, dst_it + 14);
parseHex<2>(src_ptr + 19, dst_it);
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
parseHex<4>(src_ptr, dst + 8);
parseHex<2>(src_ptr + 9, dst + 12);
parseHex<2>(src_ptr + 14, dst + 14);
parseHex<2>(src_ptr + 19, dst);
parseHex<6>(src_ptr + 24, dst + 2);
}
else if (size == 32)
{
#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
parseHex<8>(src_ptr, dst_it + 8);
parseHex<8>(src_ptr + 16, dst_it);
#else
parseHex<16>(src_ptr, dst);
#endif
parseHex<8>(src_ptr, dst + 8);
parseHex<8>(src_ptr + 16, dst);
}
else
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
}
inline void readUUIDBinary(UUID & x, ReadBuffer & buf)
{
auto & uuid = x.toUnderType();
readPODBinary(uuid.items[0], buf);
readPODBinary(uuid.items[1], buf);
}
template <typename T>
inline void readIntBinary(T & x, ReadBuffer & buf)
{
@ -1106,16 +1113,33 @@ inline void readBinary(Decimal64 & 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(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(IPv6 & x, ReadBuffer & buf) { readPODBinary(x, buf); }
inline void readBinary(UUID & x, ReadBuffer & buf)
{
readUUIDBinary(x, buf);
}
template <typename A, typename B>
inline void readBinary(std::pair<A, B> & x, ReadBuffer & buf)
{
readPODBinary(x.first, buf);
readPODBinary(x.second, 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); }
template <std::endian endian, typename T>
inline void readBinaryEndian(T & x, ReadBuffer & buf)
{
readPODBinary(x, buf);
readBinary(x, buf);
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> dst;
const auto * src_ptr = reinterpret_cast<const UInt8 *>(&uuid);
auto * dst_ptr = dst.data();
#if __BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__
const std::reverse_iterator src_it(src_ptr + 16);
formatHex(src_it + 8, dst_ptr, 4);
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);
const auto * src_ptr = reinterpret_cast<const UInt8 *>(&uuid);
const std::reverse_iterator src(src_ptr + 16);
#else
formatHex(src_ptr, dst_ptr, 4);
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);
const auto * src = reinterpret_cast<const UInt8 *>(&uuid);
#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;
}

View File

@ -88,6 +88,13 @@ inline void writePODBinary(const T & x, WriteBuffer & buf)
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>
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 LocalDate & 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 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); }
/// 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)
{
transformEndianness<endian>(x);
writePODBinary(x, buf);
writeBinary(x, buf);
}
template <typename T>

View File

@ -308,7 +308,7 @@ private:
static inline size_t getFirstLevelIdx(const UUID & uuid)
{
return uuid.toUnderType().items[0] >> (64 - bits_for_first_level);
return UUIDHelpers::getUUIDHigh(uuid) >> (64 - bits_for_first_level);
}
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());
ReadBufferFromMemory buf(value, size);
UUID uuid;
readBinaryBigEndian(uuid.toUnderType().items[0], buf);
readBinaryBigEndian(uuid.toUnderType().items[1], buf);
readBinaryBigEndian(UUIDHelpers::getUUIDHigh(uuid), buf);
readBinaryBigEndian(UUIDHelpers::getUUIDLow(uuid), buf);
assert_cast<ColumnUUID &>(column).insertValue(uuid);
}

View File

@ -270,8 +270,8 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr
{
WriteBufferFromOwnString buf;
UUID value = uuid_column.getElement(row_num);
writeBinaryBigEndian(value.toUnderType().items[0], buf);
writeBinaryBigEndian(value.toUnderType().items[1], buf);
writeBinaryBigEndian(UUIDHelpers::getUUIDHigh(value), buf);
writeBinaryBigEndian(UUIDHelpers::getUUIDLow(value), buf);
std::string_view uuid_ext = buf.stringView();
packer.pack_ext(sizeof(UUID), int8_t(MsgPackExtensionTypes::UUIDType));
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
{
#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());
#endif
}
void operator() (const IPv4 & x) const
{

View File

@ -360,7 +360,7 @@ ColumnPtr fillColumnWithRandomData(
auto column = ColumnUUID::create();
column->getData().resize(limit);
/// 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;
}
case TypeIndex::Int8:

View File

@ -4,9 +4,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../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="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 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 (
`organisation_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;