mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
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:
commit
acaa2eee58
@ -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
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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:
|
||||||
|
@ -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
|
||||||
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
@ -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
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
|
@ -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 *>(>id.seq_no), 8);
|
payload.readStrict(reinterpret_cast<char *>(>id.seq_no), 8);
|
||||||
|
|
||||||
|
@ -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)
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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{};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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;
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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;
|
||||||
|
@ -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)
|
||||||
|
@ -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);
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
@ -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>
|
||||||
|
@ -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();
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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()));
|
||||||
|
@ -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
|
||||||
{
|
{
|
||||||
|
@ -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:
|
||||||
|
@ -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
|
||||||
|
@ -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,
|
||||||
|
Binary file not shown.
@ -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;
|
@ -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
|
@ -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;
|
Loading…
Reference in New Issue
Block a user