Refactor punycodeEncode/Decode() functions to inerit from IFunction directly

This commit is contained in:
Robert Schulze 2024-01-03 09:18:25 +00:00
parent 2186aa8f21
commit 45613f1d14
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
2 changed files with 101 additions and 26 deletions

View File

@ -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.)",
)"
}}
});
}
}

View File

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