mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-28 10:31:57 +00:00
Refactor punycodeEncode/Decode() functions to inerit from IFunction directly
This commit is contained in:
parent
2186aa8f21
commit
45613f1d14
@ -2,8 +2,11 @@
|
||||
|
||||
#if USE_IDNA
|
||||
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionStringToString.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
|
||||
#ifdef __clang__
|
||||
# pragma clang diagnostic push
|
||||
@ -24,8 +27,57 @@ namespace ErrorCodes
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
struct PunycodeEncodeImpl
|
||||
namespace
|
||||
{
|
||||
|
||||
enum class ExceptionMode
|
||||
{
|
||||
Throw,
|
||||
Null
|
||||
};
|
||||
|
||||
template <ExceptionMode exception_mode>
|
||||
class FunctionPunycodeEncode : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = (exception_mode == ExceptionMode::Null) ? "punycodeEncodeOrNull" : "punycodeEncode";
|
||||
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionPunycodeEncode>(); }
|
||||
String getName() const override { return name; }
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
FunctionArgumentDescriptors args{
|
||||
{"str", &isString<IDataType>, nullptr, "String"},
|
||||
};
|
||||
validateFunctionArgumentTypes(*this, arguments, args);
|
||||
|
||||
auto return_type = std::make_shared<DataTypeString>();
|
||||
|
||||
if constexpr (exception_mode == ExceptionMode::Null)
|
||||
return makeNullable(return_type);
|
||||
else
|
||||
return return_type;
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnPtr column = arguments[0].column;
|
||||
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
|
||||
{
|
||||
auto col_res = ColumnString::create();
|
||||
vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets());
|
||||
return col_res;
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}",
|
||||
arguments[0].column->getName(), getName());
|
||||
}
|
||||
|
||||
private:
|
||||
static void vector(
|
||||
const ColumnString::Chars & data,
|
||||
const ColumnString::Offsets & offsets,
|
||||
@ -61,15 +113,50 @@ struct PunycodeEncodeImpl
|
||||
value_puny.clear(); /// utf32_to_punycode() appends to its output string
|
||||
}
|
||||
}
|
||||
|
||||
[[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &)
|
||||
{
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported by punycodeEncode function");
|
||||
}
|
||||
};
|
||||
|
||||
struct PunycodeDecodeImpl
|
||||
template <ExceptionMode exception_mode>
|
||||
class FunctionPunycodeDecode : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = (exception_mode == ExceptionMode::Null) ? "punycodeDecodeOrNull" : "punycodeDecode";
|
||||
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionPunycodeDecode>(); }
|
||||
String getName() const override { return name; }
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
FunctionArgumentDescriptors args{
|
||||
{"str", &isString<IDataType>, nullptr, "String"},
|
||||
};
|
||||
validateFunctionArgumentTypes(*this, arguments, args);
|
||||
|
||||
auto return_type = std::make_shared<DataTypeString>();
|
||||
|
||||
if constexpr (exception_mode == ExceptionMode::Null)
|
||||
return makeNullable(return_type);
|
||||
else
|
||||
return return_type;
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnPtr column = arguments[0].column;
|
||||
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
|
||||
{
|
||||
auto col_res = ColumnString::create();
|
||||
vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets());
|
||||
return col_res;
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}",
|
||||
arguments[0].column->getName(), getName());
|
||||
}
|
||||
|
||||
private:
|
||||
static void vector(
|
||||
const ColumnString::Chars & data,
|
||||
const ColumnString::Offsets & offsets,
|
||||
@ -106,26 +193,13 @@ struct PunycodeDecodeImpl
|
||||
value_utf8.clear();
|
||||
}
|
||||
}
|
||||
|
||||
[[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &)
|
||||
{
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported by punycodeDecode function");
|
||||
}
|
||||
};
|
||||
|
||||
struct NamePunycodeEncode
|
||||
{
|
||||
static constexpr auto name = "punycodeEncode";
|
||||
};
|
||||
|
||||
struct NamePunycodeDecode
|
||||
{
|
||||
static constexpr auto name = "punycodeDecode";
|
||||
};
|
||||
}
|
||||
|
||||
REGISTER_FUNCTION(Punycode)
|
||||
{
|
||||
factory.registerFunction<FunctionStringToString<PunycodeEncodeImpl, NamePunycodeEncode>>(FunctionDocumentation{
|
||||
factory.registerFunction<FunctionPunycodeEncode<ExceptionMode::Throw>>(FunctionDocumentation{
|
||||
.description=R"(
|
||||
Computes a Punycode representation of a string.)",
|
||||
.syntax="punycodeEncode(str)",
|
||||
@ -142,7 +216,7 @@ Computes a Punycode representation of a string.)",
|
||||
}}
|
||||
});
|
||||
|
||||
factory.registerFunction<FunctionStringToString<PunycodeDecodeImpl, NamePunycodeDecode>>(FunctionDocumentation{
|
||||
factory.registerFunction<FunctionPunycodeDecode<ExceptionMode::Throw>>(FunctionDocumentation{
|
||||
.description=R"(
|
||||
Computes a Punycode representation of a string.)",
|
||||
.syntax="punycodeDecode(str)",
|
||||
@ -158,6 +232,7 @@ Computes a Punycode representation of a string.)",
|
||||
)"
|
||||
}}
|
||||
});
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -9,8 +9,8 @@ SELECT punycodeDecode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT punycodeEncode(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT punycodeDecode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
|
||||
SELECT punycodeEncode('two', 'strings'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
|
||||
SELECT punycodeDecode(toFixedString('two', 3)); -- { serverError ILLEGAL_COLUMN }
|
||||
SELECT punycodeEncode(toFixedString('two', 3)); -- { serverError ILLEGAL_COLUMN }
|
||||
SELECT punycodeDecode(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
SELECT punycodeEncode(toFixedString('two', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
||||
|
||||
SELECT '-- Regular cases';
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user