mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Added UUID functions [#METR-23792].
This commit is contained in:
parent
599308aef0
commit
d77b4f12fb
@ -41,6 +41,8 @@ namespace DB
|
||||
|
||||
const auto ipv4_bytes_length = 4;
|
||||
const auto ipv6_bytes_length = 16;
|
||||
const auto uuid_bytes_length = 16;
|
||||
const auto uuid_text_length = 36;
|
||||
|
||||
class IPv6Format
|
||||
{
|
||||
@ -94,7 +96,8 @@ private:
|
||||
public:
|
||||
/** rewritten inet_ntop6 from http://svn.apache.org/repos/asf/apr/apr/trunk/network_io/unix/inet_pton.c
|
||||
* performs significantly faster than the reference implementation due to the absence of sprintf calls,
|
||||
* bounds checking, unnecessary string copying and length calculation */
|
||||
* bounds checking, unnecessary string copying and length calculation
|
||||
*/
|
||||
static const void apply(const unsigned char * src, char *& dst, UInt8 zeroed_tail_bytes_count = 0)
|
||||
{
|
||||
struct { int base, len; } best{-1}, cur{-1};
|
||||
@ -167,6 +170,7 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionIPv6NumToString : public IFunction
|
||||
{
|
||||
public:
|
||||
@ -194,13 +198,13 @@ public:
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
|
||||
{
|
||||
const auto & col_name_type = block.getByPosition(arguments[0]);
|
||||
const ColumnPtr & column = col_name_type.column;
|
||||
const auto & col_type_name = block.getByPosition(arguments[0]);
|
||||
const ColumnPtr & column = col_type_name.column;
|
||||
|
||||
if (const auto col_in = typeid_cast<const ColumnFixedString *>(column.get()))
|
||||
{
|
||||
if (col_in->getN() != ipv6_bytes_length)
|
||||
throw Exception("Illegal type " + col_name_type.type->getName() +
|
||||
throw Exception("Illegal type " + col_type_name.type->getName() +
|
||||
" of column " + col_in->getName() +
|
||||
" argument of function " + getName() +
|
||||
", expected FixedString(" + toString(ipv6_bytes_length) + ")",
|
||||
@ -232,7 +236,7 @@ public:
|
||||
{
|
||||
const auto data_type_fixed_string = typeid_cast<const DataTypeFixedString *>(col_in->getDataType().get());
|
||||
if (!data_type_fixed_string || data_type_fixed_string->getN() != ipv6_bytes_length)
|
||||
throw Exception("Illegal type " + col_name_type.type->getName() +
|
||||
throw Exception("Illegal type " + col_type_name.type->getName() +
|
||||
" of column " + col_in->getName() +
|
||||
" argument of function " + getName() +
|
||||
", expected FixedString(" + toString(ipv6_bytes_length) + ")",
|
||||
@ -253,6 +257,7 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionCutIPv6 : public IFunction
|
||||
{
|
||||
public:
|
||||
@ -290,8 +295,8 @@ public:
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
|
||||
{
|
||||
const auto & col_name_type = block.getByPosition(arguments[0]);
|
||||
const ColumnPtr & column = col_name_type.column;
|
||||
const auto & col_type_name = block.getByPosition(arguments[0]);
|
||||
const ColumnPtr & column = col_type_name.column;
|
||||
|
||||
const auto & col_ipv6_zeroed_tail_bytes_type = block.getByPosition(arguments[1]);
|
||||
const auto & col_ipv6_zeroed_tail_bytes = col_ipv6_zeroed_tail_bytes_type.column;
|
||||
@ -301,7 +306,7 @@ public:
|
||||
if (const auto col_in = typeid_cast<const ColumnFixedString *>(column.get()))
|
||||
{
|
||||
if (col_in->getN() != ipv6_bytes_length)
|
||||
throw Exception("Illegal type " + col_name_type.type->getName() +
|
||||
throw Exception("Illegal type " + col_type_name.type->getName() +
|
||||
" of column " + col_in->getName() +
|
||||
" argument of function " + getName() +
|
||||
", expected FixedString(" + toString(ipv6_bytes_length) + ")",
|
||||
@ -359,7 +364,7 @@ public:
|
||||
{
|
||||
const auto data_type_fixed_string = typeid_cast<const DataTypeFixedString *>(col_in->getDataType().get());
|
||||
if (!data_type_fixed_string || data_type_fixed_string->getN() != ipv6_bytes_length)
|
||||
throw Exception("Illegal type " + col_name_type.type->getName() +
|
||||
throw Exception("Illegal type " + col_type_name.type->getName() +
|
||||
" of column " + col_in->getName() +
|
||||
" argument of function " + getName() +
|
||||
", expected FixedString(" + toString(ipv6_bytes_length) + ")",
|
||||
@ -419,6 +424,7 @@ private:
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionIPv6StringToNum : public IFunction
|
||||
{
|
||||
public:
|
||||
@ -596,7 +602,7 @@ public:
|
||||
{
|
||||
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
|
||||
|
||||
if (const auto col_in = typeid_cast<const ColumnString *>(&*column))
|
||||
if (const auto col_in = typeid_cast<const ColumnString *>(column.get()))
|
||||
{
|
||||
const auto col_res = std::make_shared<ColumnFixedString>(ipv6_bytes_length);
|
||||
block.getByPosition(result).column = col_res;
|
||||
@ -616,7 +622,7 @@ public:
|
||||
src_offset = offsets_src[i];
|
||||
}
|
||||
}
|
||||
else if (const auto col_in = typeid_cast<const ColumnConstString *>(&*column))
|
||||
else if (const auto col_in = typeid_cast<const ColumnConstString *>(column.get()))
|
||||
{
|
||||
String out(ipv6_bytes_length, 0);
|
||||
ipv6_scan(col_in->getData().data(), reinterpret_cast<unsigned char *>(&out[0]));
|
||||
@ -632,6 +638,8 @@ public:
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionIPv4NumToString : public IFunction
|
||||
{
|
||||
public:
|
||||
@ -696,9 +704,9 @@ public:
|
||||
/// Выполнить функцию над блоком.
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
const ColumnPtr column = block.getByPosition(arguments[0]).column;
|
||||
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
|
||||
|
||||
if (const ColumnUInt32 * col = typeid_cast<const ColumnUInt32 *>(&*column))
|
||||
if (const ColumnUInt32 * col = typeid_cast<const ColumnUInt32 *>(column.get()))
|
||||
{
|
||||
const ColumnUInt32::Container_t & vec_in = col->getData();
|
||||
|
||||
@ -721,7 +729,7 @@ public:
|
||||
|
||||
vec_res.resize(pos - begin);
|
||||
}
|
||||
else if (const ColumnConst<UInt32> * col = typeid_cast<const ColumnConst<UInt32> *>(&*column))
|
||||
else if (const ColumnConst<UInt32> * col = typeid_cast<const ColumnConst<UInt32> *>(column.get()))
|
||||
{
|
||||
char buf[16];
|
||||
char * pos = buf;
|
||||
@ -737,6 +745,7 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionIPv4StringToNum : public IFunction
|
||||
{
|
||||
public:
|
||||
@ -790,9 +799,9 @@ public:
|
||||
/// Выполнить функцию над блоком.
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
const ColumnPtr column = block.getByPosition(arguments[0]).column;
|
||||
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
|
||||
|
||||
if (const ColumnString * col = typeid_cast<const ColumnString *>(&*column))
|
||||
if (const ColumnString * col = typeid_cast<const ColumnString *>(column.get()))
|
||||
{
|
||||
auto col_res = std::make_shared<ColumnUInt32>();
|
||||
block.getByPosition(result).column = col_res;
|
||||
@ -810,7 +819,7 @@ public:
|
||||
prev_offset = offsets_src[i];
|
||||
}
|
||||
}
|
||||
else if (const ColumnConstString * col = typeid_cast<const ColumnConstString *>(&*column))
|
||||
else if (const ColumnConstString * col = typeid_cast<const ColumnConstString *>(column.get()))
|
||||
{
|
||||
auto col_res = std::make_shared<ColumnConst<UInt32>>(col->size(), parseIPv4(col->getData().c_str()));
|
||||
block.getByPosition(result).column = col_res;
|
||||
@ -890,9 +899,9 @@ public:
|
||||
/// Выполнить функцию над блоком.
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
const ColumnPtr column = block.getByPosition(arguments[0]).column;
|
||||
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
|
||||
|
||||
if (const ColumnUInt32 * col = typeid_cast<const ColumnUInt32 *>(&*column))
|
||||
if (const ColumnUInt32 * col = typeid_cast<const ColumnUInt32 *>(column.get()))
|
||||
{
|
||||
const ColumnUInt32::Container_t & vec_in = col->getData();
|
||||
|
||||
@ -915,7 +924,7 @@ public:
|
||||
|
||||
vec_res.resize(pos - begin);
|
||||
}
|
||||
else if (const ColumnConst<UInt32> * col = typeid_cast<const ColumnConst<UInt32> *>(&*column))
|
||||
else if (const ColumnConst<UInt32> * col = typeid_cast<const ColumnConst<UInt32> *>(column.get()))
|
||||
{
|
||||
char buf[16];
|
||||
char * pos = buf;
|
||||
@ -931,6 +940,7 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionIPv4ToIPv6 : public IFunction
|
||||
{
|
||||
public:
|
||||
@ -955,8 +965,8 @@ public:
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
|
||||
{
|
||||
const auto & col_name_type = block.getByPosition(arguments[0]);
|
||||
const ColumnPtr & column = col_name_type.column;
|
||||
const auto & col_type_name = block.getByPosition(arguments[0]);
|
||||
const ColumnPtr & column = col_type_name.column;
|
||||
|
||||
if (const auto col_in = typeid_cast<const ColumnUInt32 *>(column.get()))
|
||||
{
|
||||
@ -996,6 +1006,268 @@ private:
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionUUIDNumToString : public IFunction
|
||||
{
|
||||
private:
|
||||
static void formatHex(const UInt8 * src, UInt8 * dst, size_t num_bytes)
|
||||
{
|
||||
constexpr auto hex = "0123456789abcdef";
|
||||
size_t src_pos = 0;
|
||||
size_t dst_pos = 0;
|
||||
for (; src_pos < num_bytes; ++src_pos)
|
||||
{
|
||||
dst[dst_pos++] = hex[src[src_pos] / 16];
|
||||
dst[dst_pos++] = hex[src[src_pos] % 16];
|
||||
}
|
||||
}
|
||||
|
||||
static void formatUUID(const UInt8 * src16, UInt8 * dst36)
|
||||
{
|
||||
formatHex(&src16[0], &dst36[0], 4);
|
||||
dst36[8] = '-';
|
||||
formatHex(&src16[4], &dst36[9], 2);
|
||||
dst36[13] = '-';
|
||||
formatHex(&src16[6], &dst36[14], 2);
|
||||
dst36[18] = '-';
|
||||
formatHex(&src16[8], &dst36[19], 2);
|
||||
dst36[23] = '-';
|
||||
formatHex(&src16[10], &dst36[24], 6);
|
||||
}
|
||||
|
||||
public:
|
||||
static constexpr auto name = "UUIDNumToString";
|
||||
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionUUIDNumToString>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const auto ptr = typeid_cast<const DataTypeFixedString *>(arguments[0].get());
|
||||
if (!ptr || ptr->getN() != uuid_bytes_length)
|
||||
throw Exception("Illegal type " + arguments[0]->getName() +
|
||||
" of argument of function " + getName() +
|
||||
", expected FixedString(" + toString(uuid_bytes_length) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
const ColumnWithTypeAndName & col_type_name = block.getByPosition(arguments[0]);
|
||||
const ColumnPtr & column = col_type_name.column;
|
||||
|
||||
if (const auto col_in = typeid_cast<const ColumnFixedString *>(column.get()))
|
||||
{
|
||||
if (col_in->getN() != uuid_bytes_length)
|
||||
throw Exception("Illegal type " + col_type_name.type->getName() +
|
||||
" of column " + col_in->getName() +
|
||||
" argument of function " + getName() +
|
||||
", expected FixedString(" + toString(uuid_bytes_length) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto size = col_in->size();
|
||||
const auto & vec_in = col_in->getChars();
|
||||
|
||||
auto col_res = std::make_shared<ColumnString>();
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
ColumnString::Chars_t & vec_res = col_res->getChars();
|
||||
ColumnString::Offsets_t & offsets_res = col_res->getOffsets();
|
||||
vec_res.resize(size * (uuid_text_length + 1));
|
||||
offsets_res.resize(size);
|
||||
|
||||
size_t src_offset = 0;
|
||||
size_t dst_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
formatUUID(&vec_in[src_offset], &vec_res[dst_offset]);
|
||||
src_offset += uuid_bytes_length;
|
||||
dst_offset += uuid_text_length;
|
||||
vec_res[dst_offset] = 0;
|
||||
++dst_offset;
|
||||
offsets_res[i] = dst_offset;
|
||||
}
|
||||
}
|
||||
else if (const auto col_in = typeid_cast<const ColumnConst<String> *>(column.get()))
|
||||
{
|
||||
const auto data_type_fixed_string = typeid_cast<const DataTypeFixedString *>(col_in->getDataType().get());
|
||||
if (!data_type_fixed_string || data_type_fixed_string->getN() != uuid_bytes_length)
|
||||
throw Exception("Illegal type " + col_type_name.type->getName() +
|
||||
" of column " + col_in->getName() +
|
||||
" argument of function " + getName() +
|
||||
", expected FixedString(" + toString(uuid_bytes_length) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto & data_in = col_in->getData();
|
||||
|
||||
char buf[uuid_text_length];
|
||||
formatUUID(reinterpret_cast<const UInt8 *>(data_in.data()), reinterpret_cast<UInt8 *>(buf));
|
||||
|
||||
block.getByPosition(result).column = std::make_shared<ColumnConstString>(col_in->size(), String(buf, uuid_text_length));
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionUUIDStringToNum : public IFunction
|
||||
{
|
||||
private:
|
||||
static void parseHex(const UInt8 * src, UInt8 * dst, size_t num_bytes)
|
||||
{
|
||||
size_t src_pos = 0;
|
||||
size_t dst_pos = 0;
|
||||
for (; dst_pos < num_bytes; ++dst_pos)
|
||||
{
|
||||
dst[dst_pos] = unhex(src[src_pos]) * 16 + unhex(src[src_pos + 1]);
|
||||
src_pos += 2;
|
||||
}
|
||||
}
|
||||
|
||||
static void parseUUID(const UInt8 * src36, UInt8 * dst16)
|
||||
{
|
||||
/// If string is not like UUID - implementation specific behaviour.
|
||||
|
||||
parseHex(&src36[0], &dst16[0], 4);
|
||||
parseHex(&src36[9], &dst16[4], 2);
|
||||
parseHex(&src36[14], &dst16[6], 2);
|
||||
parseHex(&src36[19], &dst16[8], 2);
|
||||
parseHex(&src36[24], &dst16[10], 6);
|
||||
}
|
||||
|
||||
public:
|
||||
static constexpr auto name = "UUIDStringToNum";
|
||||
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionUUIDStringToNum>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
/// String or FixedString(36)
|
||||
if (!typeid_cast<const DataTypeString *>(arguments[0].get()))
|
||||
{
|
||||
const auto ptr = typeid_cast<const DataTypeFixedString *>(arguments[0].get());
|
||||
if (!ptr || ptr->getN() != uuid_text_length)
|
||||
throw Exception("Illegal type " + arguments[0]->getName() +
|
||||
" of argument of function " + getName() +
|
||||
", expected FixedString(" + toString(uuid_text_length) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeFixedString>(uuid_bytes_length);
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
const ColumnWithTypeAndName & col_type_name = block.getByPosition(arguments[0]);
|
||||
const ColumnPtr & column = col_type_name.column;
|
||||
|
||||
if (const auto col_in = typeid_cast<const ColumnString *>(column.get()))
|
||||
{
|
||||
const auto & vec_in = col_in->getChars();
|
||||
const auto & offsets_in = col_in->getOffsets();
|
||||
const size_t size = offsets_in.size();
|
||||
|
||||
auto col_res = std::make_shared<ColumnFixedString>(uuid_bytes_length);
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
ColumnString::Chars_t & vec_res = col_res->getChars();
|
||||
vec_res.resize(size * uuid_bytes_length);
|
||||
|
||||
size_t src_offset = 0;
|
||||
size_t dst_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
/// If string has incorrect length - then return zero UUID.
|
||||
/// If string has correct length but contains something not like UUID - implementation specific behaviour.
|
||||
|
||||
size_t string_size = offsets_in[i] - src_offset;
|
||||
if (string_size == uuid_text_length + 1)
|
||||
parseUUID(&vec_in[src_offset], &vec_res[dst_offset]);
|
||||
else
|
||||
memset(&vec_res[dst_offset], 0, uuid_bytes_length);
|
||||
|
||||
dst_offset += uuid_bytes_length;
|
||||
src_offset += string_size;
|
||||
}
|
||||
}
|
||||
else if (const auto col_in = typeid_cast<const ColumnFixedString *>(column.get()))
|
||||
{
|
||||
if (col_in->getN() != uuid_text_length)
|
||||
throw Exception("Illegal type " + col_type_name.type->getName() +
|
||||
" of column " + col_in->getName() +
|
||||
" argument of function " + getName() +
|
||||
", expected FixedString(" + toString(uuid_text_length) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto size = col_in->size();
|
||||
const auto & vec_in = col_in->getChars();
|
||||
|
||||
auto col_res = std::make_shared<ColumnFixedString>(uuid_bytes_length);
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
ColumnString::Chars_t & vec_res = col_res->getChars();
|
||||
vec_res.resize(size * uuid_bytes_length);
|
||||
|
||||
size_t src_offset = 0;
|
||||
size_t dst_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
parseUUID(&vec_in[src_offset], &vec_res[dst_offset]);
|
||||
src_offset += uuid_text_length;
|
||||
dst_offset += uuid_bytes_length;
|
||||
}
|
||||
}
|
||||
else if (const auto col_in = typeid_cast<const ColumnConst<String> *>(column.get()))
|
||||
{
|
||||
const auto & data_in = col_in->getData();
|
||||
|
||||
String res;
|
||||
|
||||
if (data_in.size() == uuid_text_length)
|
||||
{
|
||||
char buf[uuid_bytes_length];
|
||||
parseUUID(reinterpret_cast<const UInt8 *>(data_in.data()), reinterpret_cast<UInt8 *>(buf));
|
||||
res.assign(buf, uuid_bytes_length);
|
||||
}
|
||||
else
|
||||
res.resize(uuid_bytes_length, '\0');
|
||||
|
||||
block.getByPosition(result).column = std::make_shared<ColumnConstString>(
|
||||
col_in->size(), res, std::make_shared<DataTypeFixedString>(uuid_bytes_length));
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionHex : public IFunction
|
||||
{
|
||||
public:
|
||||
@ -1298,9 +1570,9 @@ public:
|
||||
/// Выполнить функцию над блоком.
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
||||
{
|
||||
const ColumnPtr column = block.getByPosition(arguments[0]).column;
|
||||
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
|
||||
|
||||
if (const ColumnString * col = typeid_cast<const ColumnString *>(&*column))
|
||||
if (const ColumnString * col = typeid_cast<const ColumnString *>(column.get()))
|
||||
{
|
||||
std::shared_ptr<ColumnString> col_res = std::make_shared<ColumnString>();
|
||||
block.getByPosition(result).column = col_res;
|
||||
@ -1332,7 +1604,7 @@ public:
|
||||
|
||||
out_vec.resize(pos - begin);
|
||||
}
|
||||
else if(const ColumnConstString * col = typeid_cast<const ColumnConstString *>(&*column))
|
||||
else if(const ColumnConstString * col = typeid_cast<const ColumnConstString *>(column.get()))
|
||||
{
|
||||
const std::string & src = col->getData();
|
||||
std::string res(src.size(), '\0');
|
||||
|
@ -14,6 +14,8 @@ void registerFunctionsCoding(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionIPv4StringToNum>();
|
||||
factory.registerFunction<FunctionIPv4NumToStringClassC>();
|
||||
factory.registerFunction<FunctionIPv4ToIPv6>();
|
||||
factory.registerFunction<FunctionUUIDNumToString>();
|
||||
factory.registerFunction<FunctionUUIDStringToNum>();
|
||||
factory.registerFunction<FunctionHex>();
|
||||
factory.registerFunction<FunctionUnhex>();
|
||||
factory.registerFunction<FunctionBitmaskToArray>();
|
||||
|
6
dbms/tests/queries/0_stateless/00396_uuid.reference
Normal file
6
dbms/tests/queries/0_stateless/00396_uuid.reference
Normal file
@ -0,0 +1,6 @@
|
||||
01234567-89ab-cdef-0123-456789abcdef 1 1
|
||||
01234567-89ab-cdef-0123-456789abcdef 1 1
|
||||
0123456789ABCDEF0123456789ABCDEF
|
||||
0123456789ABCDEF0123456789ABCDEF
|
||||
01234567-89ab-cdef-0123-456789abcdef 01234567-89ab-cdef-0123-456789abcdef 01234567-89ab-cdef-0123-456789abcdef
|
||||
01234567-89ab-cdef-0123-456789abcdef 01234567-89ab-cdef-0123-456789abcdef 01234567-89ab-cdef-0123-456789abcdef
|
6
dbms/tests/queries/0_stateless/00396_uuid.sql
Normal file
6
dbms/tests/queries/0_stateless/00396_uuid.sql
Normal file
@ -0,0 +1,6 @@
|
||||
SELECT UUIDNumToString(toFixedString(unhex('0123456789ABCDEF0123456789ABCDEF' AS hex) AS bytes, 16) AS uuid_binary) AS uuid_string, hex(UUIDStringToNum(uuid_string)) = hex AS test1, UUIDStringToNum(uuid_string) = bytes AS test2;
|
||||
SELECT UUIDNumToString(toFixedString(unhex(materialize('0123456789ABCDEF0123456789ABCDEF') AS hex) AS bytes, 16) AS uuid_binary) AS uuid_string, hex(UUIDStringToNum(uuid_string)) = hex AS test1, UUIDStringToNum(uuid_string) = bytes AS test2;
|
||||
SELECT hex(UUIDStringToNum('01234567-89ab-cdef-0123-456789abcdef'));
|
||||
SELECT hex(UUIDStringToNum(materialize('01234567-89ab-cdef-0123-456789abcdef')));
|
||||
SELECT '01234567-89ab-cdef-0123-456789abcdef' AS str, UUIDNumToString(UUIDStringToNum(str)), UUIDNumToString(UUIDStringToNum(toFixedString(str, 36)));
|
||||
SELECT materialize('01234567-89ab-cdef-0123-456789abcdef') AS str, UUIDNumToString(UUIDStringToNum(str)), UUIDNumToString(UUIDStringToNum(toFixedString(str, 36)));
|
Loading…
Reference in New Issue
Block a user