mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
dbms: add function IPv4NumToStringClassC [#METR-14266]
This commit is contained in:
parent
7da67fd139
commit
d98d920d05
@ -463,8 +463,6 @@ public:
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionIPv4NumToString : public IFunction
|
||||
{
|
||||
public:
|
||||
@ -661,6 +659,115 @@ public:
|
||||
};
|
||||
|
||||
|
||||
class FunctionIPv4NumToStringClassC : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "IPv4NumToStringClassC";
|
||||
static IFunction * create(const Context & context) { return new FunctionIPv4NumToStringClassC; }
|
||||
|
||||
/// Получить имя функции.
|
||||
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]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName() + ", expected UInt32",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return new DataTypeString;
|
||||
}
|
||||
|
||||
static void formatIP(UInt32 ip, char *& out)
|
||||
{
|
||||
char * begin = out;
|
||||
|
||||
for (auto i = 0; i < 3; ++i)
|
||||
*(out++) = 'x';
|
||||
|
||||
/// Запишем все задом наперед.
|
||||
for (size_t offset = 8; offset <= 24; offset += 8)
|
||||
{
|
||||
if (offset > 0)
|
||||
*(out++) = '.';
|
||||
|
||||
/// Достаем очередной байт.
|
||||
UInt32 value = (ip >> offset) & static_cast<UInt32>(255);
|
||||
|
||||
/// Быстрее, чем sprintf.
|
||||
if (value == 0)
|
||||
{
|
||||
*(out++) = '0';
|
||||
}
|
||||
else
|
||||
{
|
||||
while (value > 0)
|
||||
{
|
||||
*(out++) = '0' + value % 10;
|
||||
value /= 10;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// И развернем.
|
||||
std::reverse(begin, out);
|
||||
|
||||
*(out++) = '\0';
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnPtr column = block.getByPosition(arguments[0]).column;
|
||||
|
||||
if (const ColumnVector<UInt32> * col = typeid_cast<const ColumnVector<UInt32> *>(&*column))
|
||||
{
|
||||
const ColumnVector<UInt32>::Container_t & vec_in = col->getData();
|
||||
|
||||
ColumnString * 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(vec_in.size() * INET_ADDRSTRLEN); /// самое длинное значение: 255.255.255.255\0
|
||||
offsets_res.resize(vec_in.size());
|
||||
char * begin = reinterpret_cast<char *>(&vec_res[0]);
|
||||
char * pos = begin;
|
||||
|
||||
for (size_t i = 0; i < vec_in.size(); ++i)
|
||||
{
|
||||
formatIP(vec_in[i], pos);
|
||||
offsets_res[i] = pos - begin;
|
||||
}
|
||||
|
||||
vec_res.resize(pos - begin);
|
||||
}
|
||||
else if (const ColumnConst<UInt32> * col = typeid_cast<const ColumnConst<UInt32> *>(&*column))
|
||||
{
|
||||
char buf[16];
|
||||
char * pos = buf;
|
||||
formatIP(col->getData(), pos);
|
||||
|
||||
ColumnConstString * col_res = new ColumnConstString(col->size(), buf);
|
||||
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);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionHex : public IFunction
|
||||
{
|
||||
public:
|
||||
|
@ -11,6 +11,7 @@ void registerFunctionsCoding(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionIPv6StringToNum>();
|
||||
factory.registerFunction<FunctionIPv4NumToString>();
|
||||
factory.registerFunction<FunctionIPv4StringToNum>();
|
||||
factory.registerFunction<FunctionIPv4NumToStringClassC>();
|
||||
factory.registerFunction<FunctionHex>();
|
||||
factory.registerFunction<FunctionUnhex>();
|
||||
factory.registerFunction<FunctionBitmaskToArray>();
|
||||
|
@ -0,0 +1,4 @@
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
@ -0,0 +1,4 @@
|
||||
select IPv4NumToStringClassC(toUInt32(0)) = '0.0.0.xxx';
|
||||
select IPv4NumToStringClassC(0x7f000001) = '127.0.0.xxx';
|
||||
select sum(IPv4NumToStringClassC(materialize(toUInt32(0))) = '0.0.0.xxx') = count() array join range(1024) as n;
|
||||
select sum(IPv4NumToStringClassC(materialize(0x7f000001)) = '127.0.0.xxx') = count() array join range(1024) as n;
|
Loading…
Reference in New Issue
Block a user