Added UUID functions [#METR-23792].

This commit is contained in:
Alexey Milovidov 2016-12-11 13:57:34 +03:00
parent 599308aef0
commit d77b4f12fb
4 changed files with 311 additions and 25 deletions

View File

@ -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');

View File

@ -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>();

View 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

View 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)));