This commit is contained in:
Alexey Arno 2015-09-02 16:05:17 +03:00
parent 7ef5c508d9
commit 277fab46b9
4 changed files with 453 additions and 34 deletions

View File

@ -42,31 +42,9 @@ namespace DB
const auto ipv4_bytes_length = 4;
const auto ipv6_bytes_length = 16;
class FunctionIPv6NumToString : public IFunction
class IPv6Format
{
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;
}
private:
/// 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)
{
@ -99,23 +77,25 @@ public:
}
/// 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++ = '.';
}
}
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 */
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};
std::array<uint16_t, ipv6_bytes_length / sizeof(uint16_t)> words{};
@ -123,7 +103,7 @@ public:
/** Preprocess:
* Copy the input (bytewise) array into a wordwise array.
* 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);
for (const auto i : ext::range(0, words.size()))
@ -172,7 +152,7 @@ public:
/// Is this address an encapsulated IPv4?
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;
}
@ -185,6 +165,32 @@ public:
*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)
{
@ -216,7 +222,7 @@ public:
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;
}
@ -236,7 +242,7 @@ public:
char buf[INET6_ADDRSTRLEN];
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};
}
@ -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
{
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
{

View File

@ -8,10 +8,12 @@ void registerFunctionsCoding(FunctionFactory & factory)
{
factory.registerFunction<FunctionToStringCutToZero>();
factory.registerFunction<FunctionIPv6NumToString>();
factory.registerFunction<FunctionCutIPv6>();
factory.registerFunction<FunctionIPv6StringToNum>();
factory.registerFunction<FunctionIPv4NumToString>();
factory.registerFunction<FunctionIPv4StringToNum>();
factory.registerFunction<FunctionIPv4NumToStringClassC>();
factory.registerFunction<FunctionIPv4ToIPv6>();
factory.registerFunction<FunctionHex>();
factory.registerFunction<FunctionUnhex>();
factory.registerFunction<FunctionBitmaskToArray>();

View File

@ -16,3 +16,79 @@
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

View File

@ -17,3 +17,109 @@ select IPv6StringToNum('') == toFixedString(materialize(''), 16);
select IPv6StringToNum(materialize('')) == toFixedString(materialize(''), 16);
select IPv6StringToNum('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;