mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Compilable
This commit is contained in:
parent
f6a2209169
commit
37d13d4bce
11
dbms/src/Functions/DynamicTarget/CMakeLists.txt
Normal file
11
dbms/src/Functions/DynamicTarget/CMakeLists.txt
Normal file
@ -0,0 +1,11 @@
|
||||
include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake)
|
||||
add_headers_and_sources(clickhouse_functions_dynamic_target .)
|
||||
add_library(clickhouse_functions_dynamic_target
|
||||
${clickhouse_functions_dynamic_target_sources}
|
||||
${clickhouse_functions_dynamic_target_headers})
|
||||
target_link_libraries(clickhouse_functions_dynamic_target PRIVATE dbms)
|
||||
|
||||
if (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" OR CMAKE_BUILD_TYPE_UC STREQUAL "MINSIZEREL")
|
||||
# Won't generate debug info for files with heavy template instantiation to achieve faster linking and lower size.
|
||||
target_compile_options(clickhouse_functions_dynamic_target PRIVATE "-g0")
|
||||
endif ()
|
263
dbms/src/Functions/DynamicTarget/DynamicFunctionAdaptors.h
Normal file
263
dbms/src/Functions/DynamicTarget/DynamicFunctionAdaptors.h
Normal file
@ -0,0 +1,263 @@
|
||||
#pragma once
|
||||
#include <Functions/IFunctionImpl.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Adaptors are implement user interfaces from IFunction.h via developer interfaces from IFunctionImpl.h
|
||||
/// Typically, you don't need to change this classes.
|
||||
|
||||
class ExecutableFunctionAdaptor final : public IExecutableFunction
|
||||
{
|
||||
public:
|
||||
explicit ExecutableFunctionAdaptor(ExecutableFunctionImplPtr impl_) : impl(std::move(impl_)) {}
|
||||
|
||||
String getName() const final { return impl->getName(); }
|
||||
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count, bool dry_run) final;
|
||||
|
||||
void createLowCardinalityResultCache(size_t cache_size) override;
|
||||
|
||||
private:
|
||||
ExecutableFunctionImplPtr impl;
|
||||
|
||||
/// Cache is created by function createLowCardinalityResultCache()
|
||||
ExecutableFunctionLowCardinalityResultCachePtr low_cardinality_result_cache;
|
||||
|
||||
bool defaultImplementationForConstantArguments(
|
||||
Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count, bool dry_run);
|
||||
|
||||
bool defaultImplementationForNulls(
|
||||
Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count, bool dry_run);
|
||||
|
||||
void executeWithoutLowCardinalityColumns(
|
||||
Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count, bool dry_run);
|
||||
};
|
||||
|
||||
class FunctionBaseAdaptor final : public IFunctionBase
|
||||
{
|
||||
public:
|
||||
explicit FunctionBaseAdaptor(FunctionBaseImplPtr impl_) : impl(std::move(impl_)) {}
|
||||
|
||||
String getName() const final { return impl->getName(); }
|
||||
|
||||
const DataTypes & getArgumentTypes() const final { return impl->getArgumentTypes(); }
|
||||
const DataTypePtr & getReturnType() const final { return impl->getReturnType(); }
|
||||
|
||||
ExecutableFunctionPtr prepare(const Block & sample_block, const ColumnNumbers & arguments, size_t result) const final
|
||||
{
|
||||
return std::make_shared<ExecutableFunctionAdaptor>(impl->prepare(sample_block, arguments, result));
|
||||
}
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
|
||||
bool isCompilable() const final { return impl->isCompilable(); }
|
||||
|
||||
llvm::Value * compile(llvm::IRBuilderBase & builder, ValuePlaceholders values) const override
|
||||
{
|
||||
return impl->compile(builder, std::move(values));
|
||||
}
|
||||
|
||||
#endif
|
||||
|
||||
bool isStateful() const final { return impl->isStateful(); }
|
||||
bool isSuitableForConstantFolding() const final { return impl->isSuitableForConstantFolding(); }
|
||||
|
||||
ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block & block, const ColumnNumbers & arguments) const final
|
||||
{
|
||||
return impl->getResultIfAlwaysReturnsConstantAndHasArguments(block, arguments);
|
||||
}
|
||||
|
||||
bool isInjective(const Block & sample_block) final { return impl->isInjective(sample_block); }
|
||||
bool isDeterministic() const final { return impl->isDeterministic(); }
|
||||
bool isDeterministicInScopeOfQuery() const final { return impl->isDeterministicInScopeOfQuery(); }
|
||||
bool hasInformationAboutMonotonicity() const final { return impl->hasInformationAboutMonotonicity(); }
|
||||
|
||||
Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const final
|
||||
{
|
||||
return impl->getMonotonicityForRange(type, left, right);
|
||||
}
|
||||
|
||||
const IFunctionBaseImpl * getImpl() const { return impl.get(); }
|
||||
|
||||
private:
|
||||
FunctionBaseImplPtr impl;
|
||||
};
|
||||
|
||||
|
||||
class FunctionOverloadResolverAdaptor final : public IFunctionOverloadResolver
|
||||
{
|
||||
public:
|
||||
explicit FunctionOverloadResolverAdaptor(FunctionOverloadResolverImplPtr impl_) : impl(std::move(impl_)) {}
|
||||
|
||||
String getName() const final { return impl->getName(); }
|
||||
|
||||
bool isDeterministic() const final { return impl->isDeterministic(); }
|
||||
|
||||
bool isDeterministicInScopeOfQuery() const final { return impl->isDeterministicInScopeOfQuery(); }
|
||||
|
||||
bool isStateful() const final { return impl->isStateful(); }
|
||||
|
||||
bool isVariadic() const final { return impl->isVariadic(); }
|
||||
|
||||
size_t getNumberOfArguments() const final { return impl->getNumberOfArguments(); }
|
||||
|
||||
void checkNumberOfArguments(size_t number_of_arguments) const final;
|
||||
|
||||
FunctionBaseImplPtr buildImpl(const ColumnsWithTypeAndName & arguments) const
|
||||
{
|
||||
return impl->build(arguments, getReturnType(arguments));
|
||||
}
|
||||
|
||||
FunctionBasePtr build(const ColumnsWithTypeAndName & arguments) const final
|
||||
{
|
||||
return std::make_shared<FunctionBaseAdaptor>(buildImpl(arguments));
|
||||
}
|
||||
|
||||
void getLambdaArgumentTypes(DataTypes & arguments) const final
|
||||
{
|
||||
checkNumberOfArguments(arguments.size());
|
||||
impl->getLambdaArgumentTypes(arguments);
|
||||
}
|
||||
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return impl->getArgumentsThatAreAlwaysConstant(); }
|
||||
|
||||
ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t number_of_arguments) const final
|
||||
{
|
||||
return impl->getArgumentsThatDontImplyNullableReturnType(number_of_arguments);
|
||||
}
|
||||
|
||||
private:
|
||||
FunctionOverloadResolverImplPtr impl;
|
||||
|
||||
DataTypePtr getReturnTypeWithoutLowCardinality(const ColumnsWithTypeAndName & arguments) const;
|
||||
DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments) const;
|
||||
};
|
||||
|
||||
|
||||
/// Following classes are implement IExecutableFunctionImpl, IFunctionBaseImpl and IFunctionOverloadResolverImpl via IFunction.
|
||||
|
||||
class DefaultExecutable final : public IExecutableFunctionImpl
|
||||
{
|
||||
public:
|
||||
explicit DefaultExecutable(std::shared_ptr<IFunction> function_) : function(std::move(function_)) {}
|
||||
|
||||
String getName() const override { return function->getName(); }
|
||||
|
||||
protected:
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) final
|
||||
{
|
||||
return function->executeImpl(block, arguments, result, input_rows_count);
|
||||
}
|
||||
void executeDryRun(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) final
|
||||
{
|
||||
return function->executeImplDryRun(block, arguments, result, input_rows_count);
|
||||
}
|
||||
bool useDefaultImplementationForNulls() const final { return function->useDefaultImplementationForNulls(); }
|
||||
bool useDefaultImplementationForConstants() const final { return function->useDefaultImplementationForConstants(); }
|
||||
bool useDefaultImplementationForLowCardinalityColumns() const final { return function->useDefaultImplementationForLowCardinalityColumns(); }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return function->getArgumentsThatAreAlwaysConstant(); }
|
||||
bool canBeExecutedOnDefaultArguments() const override { return function->canBeExecutedOnDefaultArguments(); }
|
||||
|
||||
private:
|
||||
std::shared_ptr<IFunction> function;
|
||||
};
|
||||
|
||||
class DefaultFunction final : public IFunctionBaseImpl
|
||||
{
|
||||
public:
|
||||
DefaultFunction(std::shared_ptr<IFunction> function_, DataTypes arguments_, DataTypePtr return_type_)
|
||||
: function(std::move(function_)), arguments(std::move(arguments_)), return_type(std::move(return_type_)) {}
|
||||
|
||||
String getName() const override { return function->getName(); }
|
||||
|
||||
const DataTypes & getArgumentTypes() const override { return arguments; }
|
||||
const DataTypePtr & getReturnType() const override { return return_type; }
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
|
||||
bool isCompilable() const override { return function->isCompilable(arguments); }
|
||||
|
||||
llvm::Value * compile(llvm::IRBuilderBase & builder, ValuePlaceholders values) const override { return function->compile(builder, arguments, std::move(values)); }
|
||||
|
||||
#endif
|
||||
|
||||
ExecutableFunctionImplPtr prepare(const Block & /*sample_block*/, const ColumnNumbers & /*arguments*/, size_t /*result*/) const override
|
||||
{
|
||||
return std::make_unique<DefaultExecutable>(function);
|
||||
}
|
||||
|
||||
bool isSuitableForConstantFolding() const override { return function->isSuitableForConstantFolding(); }
|
||||
ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block & block, const ColumnNumbers & arguments_) const override
|
||||
{
|
||||
return function->getResultIfAlwaysReturnsConstantAndHasArguments(block, arguments_);
|
||||
}
|
||||
|
||||
bool isStateful() const override { return function->isStateful(); }
|
||||
|
||||
bool isInjective(const Block & sample_block) override { return function->isInjective(sample_block); }
|
||||
|
||||
bool isDeterministic() const override { return function->isDeterministic(); }
|
||||
|
||||
bool isDeterministicInScopeOfQuery() const override { return function->isDeterministicInScopeOfQuery(); }
|
||||
|
||||
bool hasInformationAboutMonotonicity() const override { return function->hasInformationAboutMonotonicity(); }
|
||||
|
||||
using Monotonicity = IFunctionBase::Monotonicity;
|
||||
Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override
|
||||
{
|
||||
return function->getMonotonicityForRange(type, left, right);
|
||||
}
|
||||
private:
|
||||
std::shared_ptr<IFunction> function;
|
||||
DataTypes arguments;
|
||||
DataTypePtr return_type;
|
||||
};
|
||||
|
||||
class DefaultOverloadResolver : public IFunctionOverloadResolverImpl
|
||||
{
|
||||
public:
|
||||
explicit DefaultOverloadResolver(std::shared_ptr<IFunction> function_) : function(std::move(function_)) {}
|
||||
|
||||
void checkNumberOfArgumentsIfVariadic(size_t number_of_arguments) const override
|
||||
{
|
||||
return function->checkNumberOfArgumentsIfVariadic(number_of_arguments);
|
||||
}
|
||||
|
||||
bool isDeterministic() const override { return function->isDeterministic(); }
|
||||
bool isDeterministicInScopeOfQuery() const override { return function->isDeterministicInScopeOfQuery(); }
|
||||
|
||||
String getName() const override { return function->getName(); }
|
||||
bool isStateful() const override { return function->isStateful(); }
|
||||
bool isVariadic() const override { return function->isVariadic(); }
|
||||
size_t getNumberOfArguments() const override { return function->getNumberOfArguments(); }
|
||||
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return function->getArgumentsThatAreAlwaysConstant(); }
|
||||
ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t number_of_arguments) const override
|
||||
{
|
||||
return function->getArgumentsThatDontImplyNullableReturnType(number_of_arguments);
|
||||
}
|
||||
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const override { return function->getReturnTypeImpl(arguments); }
|
||||
DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments) const override { return function->getReturnTypeImpl(arguments); }
|
||||
|
||||
bool useDefaultImplementationForNulls() const override { return function->useDefaultImplementationForNulls(); }
|
||||
bool useDefaultImplementationForLowCardinalityColumns() const override { return function->useDefaultImplementationForLowCardinalityColumns(); }
|
||||
bool canBeExecutedOnLowCardinalityDictionary() const override { return function->canBeExecutedOnLowCardinalityDictionary(); }
|
||||
|
||||
FunctionBaseImplPtr build(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override
|
||||
{
|
||||
DataTypes data_types(arguments.size());
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
data_types[i] = arguments[i].type;
|
||||
return std::make_unique<DefaultFunction>(function, data_types, return_type);
|
||||
}
|
||||
|
||||
void getLambdaArgumentTypes(DataTypes & arguments) const override { function->getLambdaArgumentTypes(arguments); }
|
||||
|
||||
private:
|
||||
std::shared_ptr<IFunction> function;
|
||||
};
|
||||
|
||||
|
||||
}
|
39
dbms/src/Functions/DynamicTarget/Selector.h
Normal file
39
dbms/src/Functions/DynamicTarget/Selector.h
Normal file
@ -0,0 +1,39 @@
|
||||
#pragma once
|
||||
|
||||
#include "Target.h"
|
||||
|
||||
namespace DB::DynamicTarget
|
||||
{
|
||||
|
||||
class PerformanceStatistic
|
||||
{};
|
||||
|
||||
template <typename... Params>
|
||||
class SelectorExecutor
|
||||
{
|
||||
public:
|
||||
using Executor = std::function<void(Params...)>;
|
||||
// Should register all executors before
|
||||
void registerExecutor(std::optional<TargetArch> arch, Executor executor)
|
||||
{
|
||||
if (!arch || IsArchSupported(*arch)) {
|
||||
executors_.emplace_back(std::move(executor));
|
||||
}
|
||||
}
|
||||
|
||||
void execute(Params... params)
|
||||
{
|
||||
if (executors_.empty()) {
|
||||
throw "There are no realizations for this arch Arch";
|
||||
}
|
||||
int impl = 0;
|
||||
// TODO: choose implementation.
|
||||
executors_[impl](params...);
|
||||
}
|
||||
|
||||
private:
|
||||
std::vector<Executor> executors_;
|
||||
PerformanceStatistic statistic_;
|
||||
};
|
||||
|
||||
} // namespace DB::DynamicTarget
|
12
dbms/src/Functions/DynamicTarget/Target.cpp
Normal file
12
dbms/src/Functions/DynamicTarget/Target.cpp
Normal file
@ -0,0 +1,12 @@
|
||||
#include "Target.h"
|
||||
|
||||
namespace DB::DynamicTarget
|
||||
{
|
||||
|
||||
bool IsArchSupported(TargetArch arch)
|
||||
{
|
||||
// TODO(dakovalkov): use cpuid
|
||||
return arch != TargetArch::AVX512;
|
||||
}
|
||||
|
||||
} // namespace DB::DynamicTarget
|
@ -43,40 +43,45 @@ enum class TargetArch : int {
|
||||
# error "Only CLANG and GCC compilers are supported"
|
||||
#endif
|
||||
|
||||
#define DECLARE_DEFAULT_CODE (...) \
|
||||
#define DECLARE_DEFAULT_CODE(...) \
|
||||
namespace TargetSpecific::Default { \
|
||||
using namespace DB::DynamicTarget::TargetSpecific::Default; \
|
||||
__VA_ARGS__ \
|
||||
}
|
||||
|
||||
#define DECLARE_SSE4_SPECIFIC_CODE (...) \
|
||||
#define DECLARE_SSE4_SPECIFIC_CODE(...) \
|
||||
BEGIN_SSE4_SPECIFIC_CODE \
|
||||
namespace TargetSpecific::SSE4 { \
|
||||
using namespace DB::DynamicTarget::TargetSpecific::SSE4; \
|
||||
__VA_ARGS__ \
|
||||
} \
|
||||
END_TARGET_SPECIFIC_CODE
|
||||
|
||||
#define DECLARE_AVX_SPECIFIC_CODE (...) \
|
||||
#define DECLARE_AVX_SPECIFIC_CODE(...) \
|
||||
BEGIN_AVX_SPECIFIC_CODE \
|
||||
namespace TargetSpecific::AVX { \
|
||||
using namespace DB::DynamicTarget::TargetSpecific::AVX; \
|
||||
__VA_ARGS__ \
|
||||
} \
|
||||
END_TARGET_SPECIFIC_CODE
|
||||
|
||||
#define DECLARE_AVX2_SPECIFIC_CODE (...) \
|
||||
#define DECLARE_AVX2_SPECIFIC_CODE(...) \
|
||||
BEGIN_AVX2_SPECIFIC_CODE \
|
||||
namespace TargetSpecific::AVX2 { \
|
||||
using namespace DB::DynamicTarget::TargetSpecific::AVX2; \
|
||||
__VA_ARGS__ \
|
||||
} \
|
||||
END_TARGET_SPECIFIC_CODE
|
||||
|
||||
#define DECLARE_AVX512_SPECIFIC_CODE (...) \
|
||||
#define DECLARE_AVX512_SPECIFIC_CODE(...) \
|
||||
BEGIN_AVX512_SPECIFIC_CODE \
|
||||
namespace TargetSpecific::AVX512 { \
|
||||
using namespace DB::DynamicTarget::TargetSpecific::AVX512; \
|
||||
__VA_ARGS__ \
|
||||
} \
|
||||
END_TARGET_SPECIFIC_CODE
|
||||
|
||||
#define DYNAMIC_CODE (...) \
|
||||
#define DECLARE_MULTITARGET_CODE(...) \
|
||||
DECLARE_DEFAULT_CODE (__VA_ARGS__) \
|
||||
DECLARE_SSE4_SPECIFIC_CODE (__VA_ARGS__) \
|
||||
DECLARE_AVX_SPECIFIC_CODE (__VA_ARGS__) \
|
||||
@ -103,4 +108,6 @@ DECLARE_AVX512_SPECIFIC_CODE(
|
||||
constexpr auto BuildArch = TargetArch::AVX512;
|
||||
) // DECLARE_AVX512_SPECIFIC_CODE
|
||||
|
||||
bool IsArchSupported(TargetArch arch);
|
||||
|
||||
} // namespace DB::DynamicTarget
|
@ -120,7 +120,7 @@ void CompressionCodecFactory::registerCompressionCodecWithType(
|
||||
|
||||
if (byte_code)
|
||||
if (!family_code_with_codec.emplace(*byte_code, creator).second)
|
||||
throw Exception("CompressionCodecFactory: the codec family name '" + family_name + "' is not unique", ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception("CompressionCodecFactory: the codec family code '" + std::to_string(*byte_code) + "' is not unique", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
void CompressionCodecFactory::registerCompressionCodec(const String & family_name, std::optional<uint8_t> byte_code, Creator creator)
|
||||
|
@ -91,3 +91,6 @@ target_link_libraries(clickhouse_functions PRIVATE clickhouse_functions_url)
|
||||
|
||||
add_subdirectory(array)
|
||||
target_link_libraries(clickhouse_functions PRIVATE clickhouse_functions_array)
|
||||
|
||||
add_subdirectory(DynamicTarget)
|
||||
target_link_libraries(clickhouse_functions PRIVATE clickhouse_functions_dynamic_target)
|
@ -6,6 +6,8 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
|
||||
#include <Functions/DynamicTarget/Target.h>
|
||||
#include <Functions/DynamicTarget/Selector.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -27,43 +29,14 @@ struct NameEndsWith
|
||||
static constexpr auto name = "endsWith";
|
||||
};
|
||||
|
||||
using DynamicTarget::TargetArch;
|
||||
|
||||
DECLARE_MULTITARGET_CODE(
|
||||
|
||||
template <typename Name>
|
||||
class FunctionStartsEndsWith : public IFunction
|
||||
{
|
||||
class FunctionStartsEndsWithImpl {
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionStartsEndsWith>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 2;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isStringOrFixedString(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!isStringOrFixedString(arguments[1]))
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
static void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
|
||||
{
|
||||
const IColumn * haystack_column = block.getByPosition(arguments[0]).column.get();
|
||||
const IColumn * needle_column = block.getByPosition(arguments[1]).column.get();
|
||||
@ -82,14 +55,14 @@ public:
|
||||
else if (const ColumnConst * haystack_const_fixed = checkAndGetColumnConst<ColumnFixedString>(haystack_column))
|
||||
dispatch<ConstSource<FixedStringSource>>(ConstSource<FixedStringSource>(*haystack_const_fixed), needle_column, vec_res);
|
||||
else
|
||||
throw Exception("Illegal combination of columns as arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
throw Exception("Illegal combination of columns as arguments of function " "getName()", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename HaystackSource>
|
||||
void dispatch(HaystackSource haystack_source, const IColumn * needle_column, PaddedPODArray<UInt8> & res_data) const
|
||||
static void dispatch(HaystackSource haystack_source, const IColumn * needle_column, PaddedPODArray<UInt8> & res_data)
|
||||
{
|
||||
if (const ColumnString * needle = checkAndGetColumn<ColumnString>(needle_column))
|
||||
execute<HaystackSource, StringSource>(haystack_source, StringSource(*needle), res_data);
|
||||
@ -100,7 +73,7 @@ private:
|
||||
else if (const ColumnConst * needle_const_fixed = checkAndGetColumnConst<ColumnFixedString>(needle_column))
|
||||
execute<HaystackSource, ConstSource<FixedStringSource>>(haystack_source, ConstSource<FixedStringSource>(*needle_const_fixed), res_data);
|
||||
else
|
||||
throw Exception("Illegal combination of columns as arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
throw Exception("Illegal combination of columns as arguments of function " "getName()", ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
template <typename HaystackSource, typename NeedleSource>
|
||||
@ -136,4 +109,58 @@ private:
|
||||
}
|
||||
};
|
||||
|
||||
) // DECLARE_MULTITARGET_CODE
|
||||
|
||||
template <typename Name>
|
||||
class FunctionStartsEndsWith : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionStartsEndsWith>();
|
||||
}
|
||||
|
||||
FunctionStartsEndsWith() {
|
||||
executor_.registerExecutor(std::nullopt, TargetSpecific::Default::FunctionStartsEndsWithImpl<Name>::executeImpl);
|
||||
executor_.registerExecutor(TargetArch::SSE4, TargetSpecific::SSE4::FunctionStartsEndsWithImpl<Name>::executeImpl);
|
||||
executor_.registerExecutor(TargetArch::AVX, TargetSpecific::AVX::FunctionStartsEndsWithImpl<Name>::executeImpl);
|
||||
executor_.registerExecutor(TargetArch::AVX2, TargetSpecific::AVX2::FunctionStartsEndsWithImpl<Name>::executeImpl);
|
||||
executor_.registerExecutor(TargetArch::AVX512, TargetSpecific::AVX512::FunctionStartsEndsWithImpl<Name>::executeImpl);
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 2;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isStringOrFixedString(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!isStringOrFixedString(arguments[1]))
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
executor_.execute(block, arguments, result, input_rows_count);
|
||||
}
|
||||
private:
|
||||
DynamicTarget::SelectorExecutor<Block &, const ColumnNumbers &, size_t, size_t> executor_;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -194,7 +194,7 @@ using FunctionOverloadResolverImplPtr = std::unique_ptr<IFunctionOverloadResolve
|
||||
|
||||
|
||||
/// Previous function interface.
|
||||
class IFunction : public std::enable_shared_from_this<IFunction>
|
||||
class IFunction
|
||||
{
|
||||
public:
|
||||
virtual ~IFunction() = default;
|
||||
|
Loading…
Reference in New Issue
Block a user