mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-15 02:41:59 +00:00
Refactor FunctionCutURLParameter is based directly on IFunction
This commit is contained in:
parent
3cb202a63b
commit
1e10d022ce
@ -1,12 +1,81 @@
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <base/StringRef.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionsStringSearchToString.h>
|
||||
#include <base/find_symbols.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct CutURLParameterImpl
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
class FunctionCutURLParameter : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "cutURLParameter";
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionCutURLParameter>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!isString(arguments[1]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnPtr column = arguments[0].column;
|
||||
const ColumnPtr column_needle = arguments[1].column;
|
||||
|
||||
const ColumnConst * col_needle = typeid_cast<const ColumnConst *>(&*column_needle);
|
||||
if (!col_needle)
|
||||
throw Exception("Second argument of function " + getName() + " must be constant string", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
|
||||
{
|
||||
auto col_res = ColumnString::create();
|
||||
|
||||
ColumnString::Chars & vec_res = col_res->getChars();
|
||||
ColumnString::Offsets & offsets_res = col_res->getOffsets();
|
||||
vector(col->getChars(), col->getOffsets(), col_needle->getValue<String>(), vec_res, offsets_res);
|
||||
|
||||
return col_res;
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal column " + arguments[0].column->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
static void vector(const ColumnString::Chars & data,
|
||||
const ColumnString::Offsets & offsets,
|
||||
std::string pattern,
|
||||
@ -72,10 +141,8 @@ struct CutURLParameterImpl
|
||||
prev_offset = cur_offset;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
struct NameCutURLParameter { static constexpr auto name = "cutURLParameter"; };
|
||||
using FunctionCutURLParameter = FunctionsStringSearchToString<CutURLParameterImpl, NameCutURLParameter>;
|
||||
};
|
||||
|
||||
REGISTER_FUNCTION(CutURLParameter)
|
||||
{
|
||||
|
Loading…
Reference in New Issue
Block a user