mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge
This commit is contained in:
parent
7ef5c508d9
commit
277fab46b9
@ -42,31 +42,9 @@ namespace DB
|
|||||||
const auto ipv4_bytes_length = 4;
|
const auto ipv4_bytes_length = 4;
|
||||||
const auto ipv6_bytes_length = 16;
|
const auto ipv6_bytes_length = 16;
|
||||||
|
|
||||||
class FunctionIPv6NumToString : public IFunction
|
class IPv6Format
|
||||||
{
|
{
|
||||||
public:
|
private:
|
||||||
static constexpr auto name = "IPv6NumToString";
|
|
||||||
static IFunction * create(const Context & context) { return new FunctionIPv6NumToString; }
|
|
||||||
|
|
||||||
String getName() const { return name; }
|
|
||||||
|
|
||||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
|
||||||
{
|
|
||||||
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() != ipv6_bytes_length)
|
|
||||||
throw Exception("Illegal type " + arguments[0]->getName() +
|
|
||||||
" of argument of function " + getName() +
|
|
||||||
", expected FixedString(" + toString(ipv6_bytes_length) + ")",
|
|
||||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
||||||
|
|
||||||
return new DataTypeString;
|
|
||||||
}
|
|
||||||
|
|
||||||
/// integer logarithm, return ceil(log(value, base)) (the smallest integer greater or equal than log(value, base)
|
/// integer logarithm, return ceil(log(value, base)) (the smallest integer greater or equal than log(value, base)
|
||||||
static constexpr uint32_t int_log(const uint32_t value, const uint32_t base, const bool carry = false)
|
static constexpr uint32_t int_log(const uint32_t value, const uint32_t base, const bool carry = false)
|
||||||
{
|
{
|
||||||
@ -99,23 +77,25 @@ public:
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// print IPv4 address as %u.%u.%u.%u
|
/// print IPv4 address as %u.%u.%u.%u
|
||||||
static void ipv4_format(const unsigned char * src, char *& dst)
|
static void ipv4_format(const unsigned char * src, char *& dst, UInt8 zeroed_tail_bytes_count)
|
||||||
{
|
{
|
||||||
constexpr auto size = sizeof(UInt32);
|
const auto limit = ipv4_bytes_length - zeroed_tail_bytes_count;
|
||||||
|
|
||||||
for (const auto i : ext::range(0, size))
|
for (const auto i : ext::range(0, ipv4_bytes_length))
|
||||||
{
|
{
|
||||||
print_integer<10, UInt8>(dst, src[i]);
|
UInt8 byte = (i < limit) ? src[i] : 0;
|
||||||
|
print_integer<10, UInt8>(dst, byte);
|
||||||
|
|
||||||
if (i != size - 1)
|
if (i != ipv4_bytes_length - 1)
|
||||||
*dst++ = '.';
|
*dst++ = '.';
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public:
|
||||||
/** rewritten inet_ntop6 from http://svn.apache.org/repos/asf/apr/apr/trunk/network_io/unix/inet_pton.c
|
/** 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,
|
* 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 ipv6_format(const unsigned char * src, char *& dst)
|
static const void apply(const unsigned char * src, char *& dst, UInt8 zeroed_tail_bytes_count = 0)
|
||||||
{
|
{
|
||||||
struct { int base, len; } best{-1}, cur{-1};
|
struct { int base, len; } best{-1}, cur{-1};
|
||||||
std::array<uint16_t, ipv6_bytes_length / sizeof(uint16_t)> words{};
|
std::array<uint16_t, ipv6_bytes_length / sizeof(uint16_t)> words{};
|
||||||
@ -123,7 +103,7 @@ public:
|
|||||||
/** Preprocess:
|
/** Preprocess:
|
||||||
* Copy the input (bytewise) array into a wordwise array.
|
* Copy the input (bytewise) array into a wordwise array.
|
||||||
* Find the longest run of 0x00's in src[] for :: shorthanding. */
|
* Find the longest run of 0x00's in src[] for :: shorthanding. */
|
||||||
for (const auto i : ext::range(0, ipv6_bytes_length))
|
for (const auto i : ext::range(0, ipv6_bytes_length - zeroed_tail_bytes_count))
|
||||||
words[i / 2] |= src[i] << ((1 - (i % 2)) << 3);
|
words[i / 2] |= src[i] << ((1 - (i % 2)) << 3);
|
||||||
|
|
||||||
for (const auto i : ext::range(0, words.size()))
|
for (const auto i : ext::range(0, words.size()))
|
||||||
@ -172,7 +152,7 @@ public:
|
|||||||
/// Is this address an encapsulated IPv4?
|
/// Is this address an encapsulated IPv4?
|
||||||
if (i == 6 && best.base == 0 && (best.len == 6 || (best.len == 5 && words[5] == 0xffffu)))
|
if (i == 6 && best.base == 0 && (best.len == 6 || (best.len == 5 && words[5] == 0xffffu)))
|
||||||
{
|
{
|
||||||
ipv4_format(src + 12, dst);
|
ipv4_format(src + 12, dst, std::min(zeroed_tail_bytes_count, static_cast<UInt8>(ipv4_bytes_length)));
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -185,6 +165,32 @@ public:
|
|||||||
|
|
||||||
*dst++ = '\0';
|
*dst++ = '\0';
|
||||||
}
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
class FunctionIPv6NumToString : public IFunction
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = "IPv6NumToString";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionIPv6NumToString; }
|
||||||
|
|
||||||
|
String getName() const { return name; }
|
||||||
|
|
||||||
|
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||||
|
{
|
||||||
|
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() != ipv6_bytes_length)
|
||||||
|
throw Exception("Illegal type " + arguments[0]->getName() +
|
||||||
|
" of argument of function " + getName() +
|
||||||
|
", expected FixedString(" + toString(ipv6_bytes_length) + ")",
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
return new DataTypeString;
|
||||||
|
}
|
||||||
|
|
||||||
void execute(Block & block, const ColumnNumbers & arguments, const size_t result)
|
void execute(Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||||
{
|
{
|
||||||
@ -216,7 +222,7 @@ public:
|
|||||||
|
|
||||||
for (size_t offset = 0, i = 0; offset < vec_in.size(); offset += ipv6_bytes_length, ++i)
|
for (size_t offset = 0, i = 0; offset < vec_in.size(); offset += ipv6_bytes_length, ++i)
|
||||||
{
|
{
|
||||||
ipv6_format(&vec_in[offset], pos);
|
IPv6Format::apply(&vec_in[offset], pos);
|
||||||
offsets_res[i] = pos - begin;
|
offsets_res[i] = pos - begin;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -236,7 +242,7 @@ public:
|
|||||||
|
|
||||||
char buf[INET6_ADDRSTRLEN];
|
char buf[INET6_ADDRSTRLEN];
|
||||||
char * dst = buf;
|
char * dst = buf;
|
||||||
ipv6_format(reinterpret_cast<const unsigned char *>(data_in.data()), dst);
|
IPv6Format::apply(reinterpret_cast<const unsigned char *>(data_in.data()), dst);
|
||||||
|
|
||||||
block.getByPosition(result).column = new ColumnConstString{col_in->size(), buf};
|
block.getByPosition(result).column = new ColumnConstString{col_in->size(), buf};
|
||||||
}
|
}
|
||||||
@ -247,6 +253,172 @@ public:
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
class FunctionCutIPv6 : public IFunction
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = "cutIPv6";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionCutIPv6; }
|
||||||
|
|
||||||
|
String getName() const { return name; }
|
||||||
|
|
||||||
|
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||||
|
{
|
||||||
|
if (arguments.size() != 3)
|
||||||
|
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||||
|
+ toString(arguments.size()) + ", should be 3.",
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
const auto ptr = typeid_cast<const DataTypeFixedString *>(arguments[0].get());
|
||||||
|
if (!ptr || ptr->getN() != ipv6_bytes_length)
|
||||||
|
throw Exception("Illegal type " + arguments[0]->getName() +
|
||||||
|
" of argument 1 of function " + getName() +
|
||||||
|
", expected FixedString(" + toString(ipv6_bytes_length) + ")",
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
if (!typeid_cast<const DataTypeUInt8 *>(arguments[1].get()))
|
||||||
|
throw Exception("Illegal type " + arguments[1]->getName() +
|
||||||
|
" of argument 2 of function " + getName(),
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
if (!typeid_cast<const DataTypeUInt8 *>(arguments[2].get()))
|
||||||
|
throw Exception("Illegal type " + arguments[2]->getName() +
|
||||||
|
" of argument 3 of function " + getName(),
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
return new DataTypeString;
|
||||||
|
}
|
||||||
|
|
||||||
|
void execute(Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||||
|
{
|
||||||
|
const auto & col_name_type = block.getByPosition(arguments[0]);
|
||||||
|
const ColumnPtr & column = col_name_type.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;
|
||||||
|
const auto & col_ipv4_zeroed_tail_bytes_type = block.getByPosition(arguments[2]);
|
||||||
|
const auto & col_ipv4_zeroed_tail_bytes = col_ipv4_zeroed_tail_bytes_type.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() +
|
||||||
|
" of column " + col_in->getName() +
|
||||||
|
" argument of function " + getName() +
|
||||||
|
", expected FixedString(" + toString(ipv6_bytes_length) + ")",
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
const auto ipv6_zeroed_tail_bytes = typeid_cast<const ColumnConst<UInt8> *>(col_ipv6_zeroed_tail_bytes.get());
|
||||||
|
if (!ipv6_zeroed_tail_bytes)
|
||||||
|
throw Exception("Illegal type " + col_ipv6_zeroed_tail_bytes_type.type->getName() +
|
||||||
|
" of argument 2 of function " + getName(),
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
UInt8 ipv6_zeroed_tail_bytes_count = ipv6_zeroed_tail_bytes->getData();
|
||||||
|
if (ipv6_zeroed_tail_bytes_count > ipv6_bytes_length)
|
||||||
|
throw Exception("Illegal value for argument 2 " + col_ipv6_zeroed_tail_bytes_type.type->getName() +
|
||||||
|
" of function " + getName(),
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
const auto ipv4_zeroed_tail_bytes = typeid_cast<const ColumnConst<UInt8> *>(col_ipv4_zeroed_tail_bytes.get());
|
||||||
|
if (!ipv4_zeroed_tail_bytes)
|
||||||
|
throw Exception("Illegal type " + col_ipv4_zeroed_tail_bytes_type.type->getName() +
|
||||||
|
" of argument 3 of function " + getName(),
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
UInt8 ipv4_zeroed_tail_bytes_count = ipv4_zeroed_tail_bytes->getData();
|
||||||
|
if (ipv4_zeroed_tail_bytes_count > ipv6_bytes_length)
|
||||||
|
throw Exception("Illegal value for argument 3 " + col_ipv4_zeroed_tail_bytes_type.type->getName() +
|
||||||
|
" of function " + getName(),
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
const auto size = col_in->size();
|
||||||
|
const auto & vec_in = col_in->getChars();
|
||||||
|
|
||||||
|
auto col_res = new 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 * INET6_ADDRSTRLEN);
|
||||||
|
offsets_res.resize(size);
|
||||||
|
|
||||||
|
auto begin = reinterpret_cast<char *>(&vec_res[0]);
|
||||||
|
auto pos = begin;
|
||||||
|
|
||||||
|
for (size_t offset = 0, i = 0; offset < vec_in.size(); offset += ipv6_bytes_length, ++i)
|
||||||
|
{
|
||||||
|
const auto address = &vec_in[offset];
|
||||||
|
UInt8 zeroed_tail_bytes_count = isIPv4Mapped(address) ? ipv4_zeroed_tail_bytes_count : ipv6_zeroed_tail_bytes_count;
|
||||||
|
cutAddress(address, pos, zeroed_tail_bytes_count);
|
||||||
|
offsets_res[i] = pos - begin;
|
||||||
|
}
|
||||||
|
|
||||||
|
vec_res.resize(pos - begin);
|
||||||
|
}
|
||||||
|
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() != ipv6_bytes_length)
|
||||||
|
throw Exception("Illegal type " + col_name_type.type->getName() +
|
||||||
|
" of column " + col_in->getName() +
|
||||||
|
" argument of function " + getName() +
|
||||||
|
", expected FixedString(" + toString(ipv6_bytes_length) + ")",
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
const auto ipv6_zeroed_tail_bytes = typeid_cast<const ColumnConst<UInt8> *>(col_ipv6_zeroed_tail_bytes.get());
|
||||||
|
if (!ipv6_zeroed_tail_bytes)
|
||||||
|
throw Exception("Illegal type " + col_ipv6_zeroed_tail_bytes_type.type->getName() +
|
||||||
|
" of argument 2 of function " + getName(),
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
UInt8 ipv6_zeroed_tail_bytes_count = ipv6_zeroed_tail_bytes->getData();
|
||||||
|
if (ipv6_zeroed_tail_bytes_count > ipv6_bytes_length)
|
||||||
|
throw Exception("Illegal value for argument 2 " + col_ipv6_zeroed_tail_bytes_type.type->getName() +
|
||||||
|
" of function " + getName(),
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
const auto ipv4_zeroed_tail_bytes = typeid_cast<const ColumnConst<UInt8> *>(col_ipv4_zeroed_tail_bytes.get());
|
||||||
|
if (!ipv4_zeroed_tail_bytes)
|
||||||
|
throw Exception("Illegal type " + col_ipv4_zeroed_tail_bytes_type.type->getName() +
|
||||||
|
" of argument 3 of function " + getName(),
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
UInt8 ipv4_zeroed_tail_bytes_count = ipv4_zeroed_tail_bytes->getData();
|
||||||
|
if (ipv4_zeroed_tail_bytes_count > ipv6_bytes_length)
|
||||||
|
throw Exception("Illegal value for argument 3 " + col_ipv6_zeroed_tail_bytes_type.type->getName() +
|
||||||
|
" of function " + getName(),
|
||||||
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
const auto & data_in = col_in->getData();
|
||||||
|
|
||||||
|
char buf[INET6_ADDRSTRLEN];
|
||||||
|
char * dst = buf;
|
||||||
|
|
||||||
|
const auto address = reinterpret_cast<const unsigned char *>(data_in.data());
|
||||||
|
UInt8 zeroed_tail_bytes_count = isIPv4Mapped(address) ? ipv4_zeroed_tail_bytes_count : ipv6_zeroed_tail_bytes_count;
|
||||||
|
cutAddress(address, dst, zeroed_tail_bytes_count);
|
||||||
|
|
||||||
|
block.getByPosition(result).column = new ColumnConstString{col_in->size(), buf};
|
||||||
|
}
|
||||||
|
else
|
||||||
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
|
+ " of argument of function " + getName(),
|
||||||
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
bool isIPv4Mapped(const unsigned char * address) const
|
||||||
|
{
|
||||||
|
return (*reinterpret_cast<const UInt64 *>(&address[0]) == 0) &&
|
||||||
|
((*reinterpret_cast<const UInt64 *>(&address[8]) & 0x00000000FFFFFFFFull) == 0x00000000FFFF0000ull);
|
||||||
|
}
|
||||||
|
|
||||||
|
void cutAddress(const unsigned char * address, char *& dst, UInt8 zeroed_tail_bytes_count)
|
||||||
|
{
|
||||||
|
IPv6Format::apply(address, dst, zeroed_tail_bytes_count);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
class FunctionIPv6StringToNum : public IFunction
|
class FunctionIPv6StringToNum : public IFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
@ -767,6 +939,69 @@ public:
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
class FunctionIPv4ToIPv6 : public IFunction
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = "IPv4ToIPv6";
|
||||||
|
static IFunction * create(const Context & context) { return new FunctionIPv4ToIPv6; }
|
||||||
|
|
||||||
|
String getName() const { return name; }
|
||||||
|
|
||||||
|
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||||
|
{
|
||||||
|
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);
|
||||||
|
|
||||||
|
if (typeid_cast<const DataTypeUInt32 *>(arguments[0].get()) == nullptr)
|
||||||
|
throw Exception("Illegal type " + arguments[0]->getName() +
|
||||||
|
" of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
return new DataTypeFixedString{16};
|
||||||
|
}
|
||||||
|
|
||||||
|
void execute(Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||||
|
{
|
||||||
|
const auto & col_name_type = block.getByPosition(arguments[0]);
|
||||||
|
const ColumnPtr & column = col_name_type.column;
|
||||||
|
|
||||||
|
if (const auto col_in = typeid_cast<const ColumnVector<UInt32> *>(column.get()))
|
||||||
|
{
|
||||||
|
const auto col_res = new ColumnFixedString{ipv6_bytes_length};
|
||||||
|
block.getByPosition(result).column = col_res;
|
||||||
|
|
||||||
|
auto & vec_res = col_res->getChars();
|
||||||
|
vec_res.resize(col_in->size() * ipv6_bytes_length);
|
||||||
|
|
||||||
|
const auto & vec_in = col_in->getData();
|
||||||
|
|
||||||
|
for (size_t out_offset = 0, i = 0; out_offset < vec_res.size(); out_offset += ipv6_bytes_length, ++i)
|
||||||
|
mapIPv4ToIPv6(vec_in[i], &vec_res[out_offset]);
|
||||||
|
}
|
||||||
|
else if (const auto col_in = typeid_cast<const ColumnConst<UInt32> *>(column.get()))
|
||||||
|
{
|
||||||
|
std::string buf;
|
||||||
|
buf.resize(ipv6_bytes_length);
|
||||||
|
mapIPv4ToIPv6(col_in->getData(), reinterpret_cast<unsigned char *>(&buf[0]));
|
||||||
|
|
||||||
|
ColumnConstString * col_res = new ColumnConstString(ipv6_bytes_length, buf,
|
||||||
|
new DataTypeFixedString{ipv6_bytes_length});
|
||||||
|
block.getByPosition(result).column = col_res;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
|
+ " of argument of function " + getName(),
|
||||||
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
void mapIPv4ToIPv6(UInt32 in, unsigned char * buf) const
|
||||||
|
{
|
||||||
|
*reinterpret_cast<UInt64 *>(&buf[0]) = 0;
|
||||||
|
*reinterpret_cast<UInt64 *>(&buf[8]) = 0x00000000FFFF0000ull | (static_cast<UInt64>(ntohl(in)) << 32);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
class FunctionHex : public IFunction
|
class FunctionHex : public IFunction
|
||||||
{
|
{
|
||||||
|
@ -8,10 +8,12 @@ void registerFunctionsCoding(FunctionFactory & factory)
|
|||||||
{
|
{
|
||||||
factory.registerFunction<FunctionToStringCutToZero>();
|
factory.registerFunction<FunctionToStringCutToZero>();
|
||||||
factory.registerFunction<FunctionIPv6NumToString>();
|
factory.registerFunction<FunctionIPv6NumToString>();
|
||||||
|
factory.registerFunction<FunctionCutIPv6>();
|
||||||
factory.registerFunction<FunctionIPv6StringToNum>();
|
factory.registerFunction<FunctionIPv6StringToNum>();
|
||||||
factory.registerFunction<FunctionIPv4NumToString>();
|
factory.registerFunction<FunctionIPv4NumToString>();
|
||||||
factory.registerFunction<FunctionIPv4StringToNum>();
|
factory.registerFunction<FunctionIPv4StringToNum>();
|
||||||
factory.registerFunction<FunctionIPv4NumToStringClassC>();
|
factory.registerFunction<FunctionIPv4NumToStringClassC>();
|
||||||
|
factory.registerFunction<FunctionIPv4ToIPv6>();
|
||||||
factory.registerFunction<FunctionHex>();
|
factory.registerFunction<FunctionHex>();
|
||||||
factory.registerFunction<FunctionUnhex>();
|
factory.registerFunction<FunctionUnhex>();
|
||||||
factory.registerFunction<FunctionBitmaskToArray>();
|
factory.registerFunction<FunctionBitmaskToArray>();
|
||||||
|
@ -16,3 +16,79 @@
|
|||||||
1
|
1
|
||||||
1
|
1
|
||||||
1
|
1
|
||||||
|
00000000000000000000FFFF4D583737
|
||||||
|
00000000000000000000FFFF4D583737
|
||||||
|
00000000000000000000FFFF7F000001
|
||||||
|
00000000000000000000FFFFC1FC110A
|
||||||
|
2001:db8:ac10:fe01:feed:babe:cafe:f00d
|
||||||
|
2001:db8:ac10:fe01:feed:babe:cafe:f000
|
||||||
|
2001:db8:ac10:fe01:feed:babe:cafe:0
|
||||||
|
2001:db8:ac10:fe01:feed:babe:ca00:0
|
||||||
|
2001:db8:ac10:fe01:feed:babe::
|
||||||
|
2001:db8:ac10:fe01:feed:ba00::
|
||||||
|
2001:db8:ac10:fe01:feed::
|
||||||
|
2001:db8:ac10:fe01:fe00::
|
||||||
|
2001:db8:ac10:fe01::
|
||||||
|
2001:db8:ac10:fe00::
|
||||||
|
2001:db8:ac10::
|
||||||
|
2001:db8:ac00::
|
||||||
|
2001:db8::
|
||||||
|
2001:d00::
|
||||||
|
2001::
|
||||||
|
2000::
|
||||||
|
::
|
||||||
|
2001:db8:ac10:fe01:feed:babe:cafe:f00d
|
||||||
|
2001:db8:ac10:fe01:feed:babe:cafe:f00d
|
||||||
|
2001:db8:ac10:fe01:feed:babe:cafe:f00d
|
||||||
|
2001:db8:ac10:fe01:feed:babe:cafe:f00d
|
||||||
|
2001:db8:ac10:fe01:feed:babe:cafe:f00d
|
||||||
|
2001:db8:ac10:fe01:feed:babe:cafe:f00d
|
||||||
|
2001:db8:ac10:fe01:feed:babe:cafe:f00d
|
||||||
|
2001:db8:ac10:fe01:feed:babe:cafe:f00d
|
||||||
|
2001:db8:ac10:fe01:feed:babe:cafe:f00d
|
||||||
|
2001:db8:ac10:fe01:feed:babe:cafe:f00d
|
||||||
|
2001:db8:ac10:fe01:feed:babe:cafe:f00d
|
||||||
|
2001:db8:ac10:fe01:feed:babe:cafe:f00d
|
||||||
|
2001:db8:ac10:fe01:feed:babe:cafe:f00d
|
||||||
|
2001:db8:ac10:fe01:feed:babe:cafe:f00d
|
||||||
|
2001:db8:ac10:fe01:feed:babe:cafe:f00d
|
||||||
|
2001:db8:ac10:fe01:feed:babe:cafe:f00d
|
||||||
|
::ffff:193.252.17.10
|
||||||
|
::ffff:193.252.17.10
|
||||||
|
::ffff:193.252.17.10
|
||||||
|
::ffff:193.252.17.10
|
||||||
|
::ffff:193.252.17.10
|
||||||
|
::ffff:193.252.17.10
|
||||||
|
::ffff:193.252.17.10
|
||||||
|
::ffff:193.252.17.10
|
||||||
|
::ffff:193.252.17.10
|
||||||
|
::ffff:193.252.17.10
|
||||||
|
::ffff:193.252.17.10
|
||||||
|
::ffff:193.252.17.10
|
||||||
|
::ffff:193.252.17.10
|
||||||
|
::ffff:193.252.17.10
|
||||||
|
::ffff:193.252.17.10
|
||||||
|
::ffff:193.252.17.10
|
||||||
|
::ffff:193.252.17.10
|
||||||
|
::ffff:193.252.17.0
|
||||||
|
::ffff:193.252.0.0
|
||||||
|
::ffff:193.0.0.0
|
||||||
|
::ffff:0.0.0.0
|
||||||
|
::ff00:0:0
|
||||||
|
::
|
||||||
|
::
|
||||||
|
::
|
||||||
|
::
|
||||||
|
::
|
||||||
|
::
|
||||||
|
::
|
||||||
|
::
|
||||||
|
::
|
||||||
|
::
|
||||||
|
::
|
||||||
|
2001:db8:ac10:fe01:abad:babe:fa00:0
|
||||||
|
2001:db8:ac10:fe01:dead:c0de:ca00:0
|
||||||
|
2001:db8:ac10:fe01:feed:babe:ca00:0
|
||||||
|
::ffff:77.0.0.0
|
||||||
|
::ffff:127.0.0.0
|
||||||
|
::ffff:193.0.0.0
|
||||||
|
@ -17,3 +17,109 @@ select IPv6StringToNum('') == toFixedString(materialize(''), 16);
|
|||||||
select IPv6StringToNum(materialize('')) == toFixedString(materialize(''), 16);
|
select IPv6StringToNum(materialize('')) == toFixedString(materialize(''), 16);
|
||||||
select IPv6StringToNum('not an ip string') == toFixedString(materialize(''), 16);
|
select IPv6StringToNum('not an ip string') == toFixedString(materialize(''), 16);
|
||||||
select IPv6StringToNum(materialize('not an ip string')) == toFixedString(materialize(''), 16);
|
select IPv6StringToNum(materialize('not an ip string')) == toFixedString(materialize(''), 16);
|
||||||
|
|
||||||
|
/* IPv4ToIPv6 */
|
||||||
|
|
||||||
|
SELECT hex(IPv4ToIPv6(1297626935));
|
||||||
|
|
||||||
|
/* Тест с таблицей */
|
||||||
|
|
||||||
|
DROP TABLE IF EXISTS test.addresses;
|
||||||
|
CREATE TABLE test.addresses(addr UInt32) ENGINE = Memory;
|
||||||
|
INSERT INTO test.addresses(addr) VALUES (1297626935), (2130706433), (3254522122);
|
||||||
|
SELECT hex(IPv4ToIPv6(addr)) FROM test.addresses ORDER BY addr ASC;
|
||||||
|
|
||||||
|
/* cutIPv6 */
|
||||||
|
|
||||||
|
/* Реальный IPv6-адрес */
|
||||||
|
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 0);
|
||||||
|
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 1, 0);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 2, 0);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 3, 0);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 4, 0);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 5, 0);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 6, 0);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 7, 0);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 8, 0);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 9, 0);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 10, 0);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 11, 0);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 12, 0);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 13, 0);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 14, 0);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 15, 0);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 16, 0);
|
||||||
|
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 1);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 2);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 3);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 4);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 5);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 6);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 7);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 8);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 9);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 10);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 11);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 12);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 13);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 14);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 15);
|
||||||
|
SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 16);
|
||||||
|
|
||||||
|
/* IPv4-mapped IPv6-адрес */
|
||||||
|
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 0);
|
||||||
|
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 1, 0);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 2, 0);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 3, 0);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 4, 0);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 5, 0);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 6, 0);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 7, 0);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 8, 0);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 9, 0);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 10, 0);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 11, 0);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 12, 0);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 13, 0);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 14, 0);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 15, 0);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 16, 0);
|
||||||
|
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 1);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 2);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 3);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 4);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 5);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 6);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 7);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 8);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 9);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 10);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 11);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 12);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 13);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 14);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 15);
|
||||||
|
SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 16);
|
||||||
|
|
||||||
|
/* Тест с таблицами */
|
||||||
|
|
||||||
|
/* Реальные IPv6-адреса */
|
||||||
|
|
||||||
|
DROP TABLE IF EXISTS test.addresses;
|
||||||
|
CREATE TABLE test.addresses(addr String) ENGINE = Memory;
|
||||||
|
INSERT INTO test.addresses(addr) VALUES ('20010DB8AC10FE01FEEDBABECAFEF00D'), ('20010DB8AC10FE01DEADC0DECAFED00D'), ('20010DB8AC10FE01ABADBABEFACEB00C');
|
||||||
|
SELECT cutIPv6(toFixedString(unhex(addr), 16), 3, 0) FROM test.addresses ORDER BY addr ASC;
|
||||||
|
|
||||||
|
/* IPv4-mapped IPv6-адреса */
|
||||||
|
|
||||||
|
DROP TABLE IF EXISTS test.addresses;
|
||||||
|
CREATE TABLE test.addresses(addr String) ENGINE = Memory;
|
||||||
|
INSERT INTO test.addresses(addr) VALUES ('00000000000000000000FFFFC1FC110A'), ('00000000000000000000FFFF4D583737'), ('00000000000000000000FFFF7F000001');
|
||||||
|
SELECT cutIPv6(toFixedString(unhex(addr), 16), 0, 3) FROM test.addresses ORDER BY addr ASC;
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user