ClickHouse/src/Functions/IFunction.h

535 lines
26 KiB
C++
Raw Normal View History

2011-08-09 15:57:33 +00:00
#pragma once
#include <Core/ColumnNumbers.h>
#include <Core/ColumnsWithTypeAndName.h>
#include <Core/Names.h>
2022-11-28 15:02:59 +00:00
#include <Core/IResolvedFunction.h>
#include <Common/Exception.h>
#include <DataTypes/IDataType.h>
2011-08-09 15:57:33 +00:00
#include "config.h"
#include <memory>
/// This file contains user interface for functions.
2011-08-09 15:57:33 +00:00
namespace llvm
{
class LLVMContext;
class Value;
class IRBuilderBase;
}
2011-08-09 15:57:33 +00:00
namespace DB
{
namespace ErrorCodes
{
2017-06-13 02:06:53 +00:00
extern const int NOT_IMPLEMENTED;
2021-05-15 17:33:15 +00:00
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
2019-09-27 13:44:33 +00:00
class Field;
2018-02-08 16:59:04 +00:00
/// The simplest executable object.
/// Motivation:
2020-10-14 14:04:50 +00:00
/// * Prepare something heavy once before main execution loop instead of doing it for each columns.
2018-02-09 19:20:18 +00:00
/// * Provide const interface for IFunctionBase (later).
/// * Create one executable function per thread to use caches without synchronization (later).
2019-12-08 21:06:37 +00:00
class IExecutableFunction
2011-08-09 15:57:33 +00:00
{
public:
2019-12-08 21:06:37 +00:00
virtual ~IExecutableFunction() = default;
2018-02-02 08:33:36 +00:00
/// Get the main function name.
virtual String getName() const = 0;
2021-05-15 17:33:15 +00:00
ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const;
2019-12-09 14:41:55 +00:00
2021-05-15 17:33:15 +00:00
protected:
virtual ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const = 0;
virtual ColumnPtr executeDryRunImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const
{
return executeImpl(arguments, result_type, input_rows_count);
}
/** Default implementation in presence of Nullable arguments or NULL constants as arguments is the following:
* if some of arguments are NULL constants then return NULL constant,
* if some of arguments are Nullable, then execute function as usual for columns,
* where Nullable columns are substituted with nested columns (they have arbitrary values in rows corresponding to NULL value)
* and wrap result in Nullable column where NULLs are in all rows where any of arguments are NULL.
*/
virtual bool useDefaultImplementationForNulls() const { return true; }
/** Default implementation in presence of arguments with type Nothing is the following:
* If some of arguments have type Nothing then default implementation is to return constant column with type Nothing
*/
virtual bool useDefaultImplementationForNothing() const { return true; }
2021-05-15 17:33:15 +00:00
/** If the function have non-zero number of arguments,
* and if all arguments are constant, that we could automatically provide default implementation:
* arguments are converted to ordinary columns with single value, then function is executed as usual,
* and then the result is converted to constant column.
*/
virtual bool useDefaultImplementationForConstants() const { return false; }
/** If function arguments has single low cardinality column and all other arguments are constants, call function on nested column.
* Otherwise, convert all low cardinality columns to ordinary columns.
* Returns ColumnLowCardinality if at least one argument is ColumnLowCardinality.
*/
virtual bool useDefaultImplementationForLowCardinalityColumns() const { return true; }
/** If function arguments has single sparse column and all other arguments are constants, call function on nested column.
* Otherwise, convert all sparse columns to ordinary columns.
* If default value doesn't change after function execution, returns sparse column as a result.
2021-09-20 13:09:20 +00:00
* Otherwise, result column is converted to full.
*/
virtual bool useDefaultImplementationForSparseColumns() const { return true; }
2021-05-15 17:33:15 +00:00
/** Some arguments could remain constant during this implementation.
*/
virtual ColumnNumbers getArgumentsThatAreAlwaysConstant() const { return {}; }
/** True if function can be called on default arguments (include Nullable's) and won't throw.
* Counterexample: modulo(0, 0)
*/
virtual bool canBeExecutedOnDefaultArguments() const { return true; }
private:
ColumnPtr defaultImplementationForConstantArguments(
const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const;
ColumnPtr defaultImplementationForNulls(
const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const;
ColumnPtr defaultImplementationForNothing(
const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count) const;
2021-05-15 17:33:15 +00:00
ColumnPtr executeWithoutLowCardinalityColumns(
const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const;
2019-12-09 14:41:55 +00:00
ColumnPtr executeWithoutSparseColumns(
const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const;
2018-02-02 08:33:36 +00:00
};
2019-12-08 21:06:37 +00:00
using ExecutableFunctionPtr = std::shared_ptr<IExecutableFunction>;
2018-02-02 08:33:36 +00:00
2021-05-03 22:34:40 +00:00
using Values = std::vector<llvm::Value *>;
2021-05-15 17:33:15 +00:00
/** Function with known arguments and return type (when the specific overload was chosen).
* It is also the point where all function-specific properties are known.
*/
2022-11-28 15:02:59 +00:00
class IFunctionBase : public IResolvedFunction
2018-02-02 08:33:36 +00:00
{
public:
2021-05-15 17:33:15 +00:00
2022-11-28 15:02:59 +00:00
~IFunctionBase() override = default;
virtual ColumnPtr execute( /// NOLINT
const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run = false) const
2021-05-15 17:33:15 +00:00
{
return prepare(arguments)->execute(arguments, result_type, input_rows_count, dry_run);
}
2017-05-28 14:32:59 +00:00
/// Get the main function name.
virtual String getName() const = 0;
2022-11-28 15:02:59 +00:00
const Array & getParameters() const final
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "IFunctionBase doesn't support getParameters method");
}
2018-02-02 08:33:36 +00:00
/// Do preparations and return executable.
2020-10-14 14:04:50 +00:00
/// sample_columns should contain data types of arguments and values of constants, if relevant.
2020-10-15 16:52:25 +00:00
virtual ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName & arguments) const = 0;
#if USE_EMBEDDED_COMPILER
virtual bool isCompilable() const { return false; }
/** Produce LLVM IR code that operates on scalar values. See `toNativeType` in DataTypes/Native.h
* for supported value types and how they map to LLVM types.
*
* NOTE: the builder is actually guaranteed to be exactly `llvm::IRBuilder<>`, so you may safely
* downcast it to that type. This method is specified with `IRBuilderBase` because forward-declaring
* templates with default arguments is impossible and including LLVM in such a generic header
* as this one is a major pain.
*/
2021-05-03 22:34:40 +00:00
virtual llvm::Value * compile(llvm::IRBuilderBase & /*builder*/, Values /*values*/) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{} is not JIT-compilable", getName());
}
#endif
virtual bool isStateful() const { return false; }
/** Should we evaluate this function while constant folding, if arguments are constants?
* Usually this is true. Notable counterexample is function 'sleep'.
* If we will call it during query analysis, we will sleep extra amount of time.
*/
virtual bool isSuitableForConstantFolding() const { return true; }
2022-09-05 01:50:24 +00:00
/** If function isSuitableForConstantFolding then, this method will be called during query analysis
2021-05-23 22:06:38 +00:00
* if some arguments are constants. For example logical functions (AndFunction, OrFunction) can
* return they result based on some constant arguments.
* Arguments are passed without modifications, useDefaultImplementationForNulls, useDefaultImplementationForNothing,
* useDefaultImplementationForConstants, useDefaultImplementationForLowCardinality are not applied.
*/
virtual ColumnPtr getConstantResultForNonConstArguments(
const ColumnsWithTypeAndName & /* arguments */, const DataTypePtr & /* result_type */) const { return nullptr; }
/** Function is called "injective" if it returns different result for different values of arguments.
* Example: hex, negate, tuple...
*
* Function could be injective with some arguments fixed to some constant values.
* Examples:
* plus(const, x);
2019-01-22 19:56:53 +00:00
* multiply(const, x) where x is an integer and constant is not divisible by two;
* concat(x, 'const');
* concat(x, 'const', y) where const contain at least one non-numeric character;
* concat with FixedString
* dictGet... functions takes name of dictionary as its argument,
* and some dictionaries could be explicitly defined as injective.
*
* It could be used, for example, to remove useless function applications from GROUP BY.
*
* Sometimes, function is not really injective, but considered as injective, for purpose of query optimization.
* For example, toString function is not injective for Float64 data type,
* as it returns 'nan' for many different representation of NaNs.
* But we assume, that it is injective. This could be documented as implementation-specific behaviour.
*
2020-10-14 14:04:50 +00:00
* sample_columns should contain data types of arguments and values of constants, if relevant.
* NOTE: to check is function injective with any arguments, you can pass
2020-10-14 14:04:50 +00:00
* empty columns as sample_columns (since most of the time function will
* ignore it anyway, and creating arguments just for checking is
* function injective or not is overkill).
*/
2020-10-14 14:04:50 +00:00
virtual bool isInjective(const ColumnsWithTypeAndName & /*sample_columns*/) const { return false; }
/** Function is called "deterministic", if it returns same result for same values of arguments.
* Most of functions are deterministic. Notable counterexample is rand().
* Sometimes, functions are "deterministic" in scope of single query
* (even for distributed query), but not deterministic it general.
* Example: now(). Another example: functions that work with periodically updated dictionaries.
*/
2021-05-15 17:33:15 +00:00
virtual bool isDeterministic() const { return true; }
2021-05-15 17:33:15 +00:00
virtual bool isDeterministicInScopeOfQuery() const { return true; }
2018-02-02 08:33:36 +00:00
/** Lets you know if the function is monotonic in a range of values.
* This is used to work with the index in a sorted chunk of data.
* And allows to use the index not only when it is written, for example `date >= const`, but also, for example, `toMonth(date) >= 11`.
* All this is considered only for functions of one argument.
*/
virtual bool hasInformationAboutMonotonicity() const { return false; }
struct ShortCircuitSettings
{
/// Should we enable lazy execution for the first argument of short-circuit function?
/// Example: if(cond, then, else), we don't need to execute cond lazily.
bool enable_lazy_execution_for_first_argument;
/// Should we enable lazy execution for functions, that are common descendants of
/// different short-circuit function arguments?
/// Example 1: if (cond, expr1(..., expr, ...), expr2(..., expr, ...)), we don't need
/// to execute expr lazily, because it's used in both branches.
/// Example 2: and(expr1, expr2(..., expr, ...), expr3(..., expr, ...)), here we
/// should enable lazy execution for expr, because it must be filtered by expr1.
bool enable_lazy_execution_for_common_descendants_of_arguments;
/// Should we enable lazy execution without checking isSuitableForShortCircuitArgumentsExecution?
/// Example: toTypeName(expr), even if expr contains functions that are not suitable for
/// lazy execution (because of their simplicity), we shouldn't execute them at all.
bool force_enable_lazy_execution;
};
2021-05-18 13:05:55 +00:00
/** Function is called "short-circuit" if it's arguments can be evaluated lazily
* (examples: and, or, if, multiIf). If function is short circuit, it should be
* able to work with lazy executed arguments,
2021-05-18 13:05:55 +00:00
* this method will be called before function execution.
* If function is short circuit, it must define all fields in settings for
* appropriate preparations. Number of arguments is provided because some settings might depend on it.
* Example: multiIf(cond, else, then) and multiIf(cond1, else1, cond2, else2, ...), the first
* version can enable enable_lazy_execution_for_common_descendants_of_arguments setting, the second - not.
2021-05-18 13:05:55 +00:00
*/
2021-06-22 16:21:23 +00:00
virtual bool isShortCircuit(ShortCircuitSettings & /*settings*/, size_t /*number_of_arguments*/) const { return false; }
2021-05-18 13:55:11 +00:00
/** Should we evaluate this function lazily in short-circuit function arguments?
* If function can throw an exception or it's computationally heavy, then
* it's suitable, otherwise it's not (due to the overhead of lazy execution).
* Suitability may depend on function arguments.
*/
2021-06-22 16:21:23 +00:00
virtual bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const = 0;
2018-02-02 08:33:36 +00:00
/// The property of monotonicity for a certain range.
struct Monotonicity
{
2022-09-22 15:49:27 +00:00
bool is_monotonic = false; /// Is the function monotonous (non-decreasing or non-increasing).
bool is_positive = true; /// true if the function is non-decreasing, false if non-increasing. If is_monotonic = false, then it does not matter.
2018-02-02 08:33:36 +00:00
bool is_always_monotonic = false; /// Is true if function is monotonic on the whole input range I
2022-09-22 15:49:27 +00:00
bool is_strict = false; /// true if the function is strictly decreasing or increasing.
2018-02-02 08:33:36 +00:00
};
2018-02-02 08:33:36 +00:00
/** Get information about monotonicity on a range of values. Call only if hasInformationAboutMonotonicity.
* NULL can be passed as one of the arguments. This means that the corresponding range is unlimited on the left or on the right.
*/
2018-02-02 08:33:36 +00:00
virtual Monotonicity getMonotonicityForRange(const IDataType & /*type*/, const Field & /*left*/, const Field & /*right*/) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its monotonicity", getName());
}
2021-05-15 17:33:15 +00:00
2018-02-02 08:33:36 +00:00
};
2022-11-30 18:48:09 +00:00
using FunctionBasePtr = std::shared_ptr<const IFunctionBase>;
2018-02-02 08:33:36 +00:00
2019-12-08 21:06:37 +00:00
2021-05-15 17:33:15 +00:00
/** Creates IFunctionBase from argument types list (chooses one function overload).
*/
2019-12-08 21:06:37 +00:00
class IFunctionOverloadResolver
2018-02-02 08:33:36 +00:00
{
public:
2019-12-08 21:06:37 +00:00
virtual ~IFunctionOverloadResolver() = default;
2018-02-02 08:33:36 +00:00
virtual FunctionBasePtr build(const ColumnsWithTypeAndName & arguments) const;
2021-05-15 17:33:15 +00:00
void getLambdaArgumentTypes(DataTypes & arguments) const;
void checkNumberOfArguments(size_t number_of_arguments) const;
2018-02-02 08:33:36 +00:00
/// Get the main function name.
virtual String getName() const = 0;
2021-05-15 17:33:15 +00:00
/// For non-variadic functions, return number of arguments; otherwise return zero (that should be ignored).
virtual size_t getNumberOfArguments() const = 0;
/// TODO: This method should not be duplicated here and in IFunctionBase
/// See the comment for the same method in IFunctionBase
2021-05-15 17:33:15 +00:00
virtual bool isDeterministic() const { return true; }
virtual bool isDeterministicInScopeOfQuery() const { return true; }
virtual bool isInjective(const ColumnsWithTypeAndName &) const { return false; }
/// Override and return true if function needs to depend on the state of the data.
2021-05-15 17:33:15 +00:00
virtual bool isStateful() const { return false; }
2018-02-02 08:33:36 +00:00
/// Override and return true if function could take different number of arguments.
2021-05-15 17:33:15 +00:00
virtual bool isVariadic() const { return false; }
2018-02-02 08:33:36 +00:00
/// For non-variadic functions, return number of arguments; otherwise return zero (that should be ignored).
/// For higher-order functions (functions, that have lambda expression as at least one argument).
/// You pass data types with empty DataTypeFunction for lambda arguments.
/// This function will replace it with DataTypeFunction containing actual types.
2021-05-15 17:33:15 +00:00
virtual void getLambdaArgumentTypesImpl(DataTypes & arguments [[maybe_unused]]) const
{
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {} can't have lambda-expressions as arguments", getName());
2021-05-15 17:33:15 +00:00
}
2019-10-02 17:51:00 +00:00
/// Returns indexes of arguments, that must be ColumnConst
2021-05-15 17:33:15 +00:00
virtual ColumnNumbers getArgumentsThatAreAlwaysConstant() const { return {}; }
2019-10-02 17:51:00 +00:00
/// Returns indexes if arguments, that can be Nullable without making result of function Nullable
/// (for functions like isNull(x))
2021-05-15 17:33:15 +00:00
virtual ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t number_of_arguments [[maybe_unused]]) const { return {}; }
2021-05-15 17:33:15 +00:00
protected:
2011-08-09 15:57:33 +00:00
virtual FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & /* arguments */, const DataTypePtr & /* result_type */) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "buildImpl is not implemented for {}", getName());
}
2011-08-09 19:19:00 +00:00
2021-05-15 17:33:15 +00:00
virtual DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getReturnType is not implemented for {}", getName());
2021-05-15 17:33:15 +00:00
}
/// This function will be called in default implementation. You can overload it or the previous one.
virtual DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const
{
DataTypes data_types(arguments.size());
for (size_t i = 0; i < arguments.size(); ++i)
data_types[i] = arguments[i].type;
return getReturnTypeImpl(data_types);
}
2021-05-24 20:27:32 +00:00
/** If useDefaultImplementationForNulls() is true, then change arguments for getReturnType() and build():
2021-05-15 17:33:15 +00:00
* if some of arguments are Nullable(Nothing) then don't call getReturnType(), call build() with return_type = Nullable(Nothing),
* if some of arguments are Nullable, then:
* - Nullable types are substituted with nested types for getReturnType() function
* - wrap getReturnType() result in Nullable type and pass to build
*
* Otherwise build returns build(arguments, getReturnType(arguments));
*/
virtual bool useDefaultImplementationForNulls() const { return true; }
/** If useDefaultImplementationForNothing() is true, then change arguments for getReturnType() and build():
* if some of arguments are Nothing then don't call getReturnType(), call build() with return_type = Nothing,
* Otherwise build returns build(arguments, getReturnType(arguments));
*/
virtual bool useDefaultImplementationForNothing() const { return true; }
/** If useDefaultImplementationForLowCardinalityColumns() is true, then change arguments for getReturnType() and build().
2021-05-15 17:33:15 +00:00
* If function arguments has low cardinality types, convert them to ordinary types.
* getReturnType returns ColumnLowCardinality if at least one argument type is ColumnLowCardinality.
*/
virtual bool useDefaultImplementationForLowCardinalityColumns() const { return true; }
/** If function arguments has single sparse column and all other arguments are constants, call function on nested column.
* Otherwise, convert all sparse columns to ordinary columns.
* If default value doesn't change after function execution, returns sparse column as a result.
2021-09-20 13:09:20 +00:00
* Otherwise, result column is converted to full.
*/
virtual bool useDefaultImplementationForSparseColumns() const { return true; }
/// If it isn't, will convert all ColumnLowCardinality arguments to full columns.
2021-05-15 17:33:15 +00:00
virtual bool canBeExecutedOnLowCardinalityDictionary() const { return true; }
private:
DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments) const;
2021-05-15 17:33:15 +00:00
DataTypePtr getReturnTypeWithoutLowCardinality(const ColumnsWithTypeAndName & arguments) const;
2018-02-02 08:33:36 +00:00
};
2019-12-08 21:06:37 +00:00
using FunctionOverloadResolverPtr = std::shared_ptr<IFunctionOverloadResolver>;
2011-08-09 15:57:33 +00:00
2021-05-17 07:30:42 +00:00
/// Old function interface. Check documentation in IFunction.h.
2022-09-05 01:50:24 +00:00
/// If client do not need stateful properties it can implement this interface.
2021-05-17 07:30:42 +00:00
class IFunction
{
public:
2011-08-09 19:19:00 +00:00
2021-05-17 07:30:42 +00:00
virtual ~IFunction() = default;
virtual String getName() const = 0;
virtual ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const = 0;
virtual ColumnPtr executeImplDryRun(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const
{
return executeImpl(arguments, result_type, input_rows_count);
}
/** Default implementation in presence of Nullable arguments or NULL constants as arguments is the following:
* if some of arguments are NULL constants then return NULL constant,
* if some of arguments are Nullable, then execute function as usual for columns,
* where Nullable columns are substituted with nested columns (they have arbitrary values in rows corresponding to NULL value)
* and wrap result in Nullable column where NULLs are in all rows where any of arguments are NULL.
*/
virtual bool useDefaultImplementationForNulls() const { return true; }
/** Default implementation in presence of arguments with type Nothing is the following:
* If some of arguments have type Nothing then default implementation is to return constant column with type Nothing
*/
virtual bool useDefaultImplementationForNothing() const { return true; }
2021-05-17 07:30:42 +00:00
/** If the function have non-zero number of arguments,
* and if all arguments are constant, that we could automatically provide default implementation:
* arguments are converted to ordinary columns with single value, then function is executed as usual,
* and then the result is converted to constant column.
*/
virtual bool useDefaultImplementationForConstants() const { return false; }
2021-07-10 08:29:54 +00:00
/** Some arguments could remain constant during this implementation.
*/
virtual ColumnNumbers getArgumentsThatAreAlwaysConstant() const { return {}; }
2021-05-17 07:30:42 +00:00
/** If function arguments has single low cardinality column and all other arguments are constants, call function on nested column.
* Otherwise, convert all low cardinality columns to ordinary columns.
* Returns ColumnLowCardinality if at least one argument is ColumnLowCardinality.
*/
virtual bool useDefaultImplementationForLowCardinalityColumns() const { return true; }
/** If function arguments has single sparse column and all other arguments are constants, call function on nested column.
* Otherwise, convert all sparse columns to ordinary columns.
* If default value doesn't change after function execution, returns sparse column as a result.
2021-09-20 13:09:20 +00:00
* Otherwise, result column is converted to full.
*/
virtual bool useDefaultImplementationForSparseColumns() const { return true; }
2021-05-17 07:30:42 +00:00
/// If it isn't, will convert all ColumnLowCardinality arguments to full columns.
virtual bool canBeExecutedOnLowCardinalityDictionary() const { return true; }
/** True if function can be called on default arguments (include Nullable's) and won't throw.
* Counterexample: modulo(0, 0)
*/
virtual bool canBeExecutedOnDefaultArguments() const { return true; }
/// Properties from IFunctionBase (see IFunction.h)
virtual bool isSuitableForConstantFolding() const { return true; }
2021-05-24 11:25:02 +00:00
virtual ColumnPtr getConstantResultForNonConstArguments(const ColumnsWithTypeAndName & /*arguments*/, const DataTypePtr & /*result_type*/) const { return nullptr; }
2021-05-17 07:30:42 +00:00
virtual bool isInjective(const ColumnsWithTypeAndName & /*sample_columns*/) const { return false; }
virtual bool isDeterministic() const { return true; }
virtual bool isDeterministicInScopeOfQuery() const { return true; }
virtual bool isStateful() const { return false; }
using ShortCircuitSettings = IFunctionBase::ShortCircuitSettings;
2021-06-22 16:21:23 +00:00
virtual bool isShortCircuit(ShortCircuitSettings & /*settings*/, size_t /*number_of_arguments*/) const { return false; }
virtual bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const = 0;
2021-05-17 09:49:35 +00:00
2021-05-17 07:30:42 +00:00
virtual bool hasInformationAboutMonotonicity() const { return false; }
using Monotonicity = IFunctionBase::Monotonicity;
virtual Monotonicity getMonotonicityForRange(const IDataType & /*type*/, const Field & /*left*/, const Field & /*right*/) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its monotonicity", getName());
2021-05-17 07:30:42 +00:00
}
/// For non-variadic functions, return number of arguments; otherwise return zero (that should be ignored).
virtual size_t getNumberOfArguments() const = 0;
virtual DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getReturnType is not implemented for {}", getName());
2021-05-17 07:30:42 +00:00
}
/// Get the result type by argument type. If the function does not apply to these arguments, throw an exception.
virtual DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const
{
DataTypes data_types(arguments.size());
for (size_t i = 0; i < arguments.size(); ++i)
data_types[i] = arguments[i].type;
return getReturnTypeImpl(data_types);
}
virtual bool isVariadic() const { return false; }
virtual void getLambdaArgumentTypes(DataTypes & /*arguments*/) const
{
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {} can't have lambda-expressions as arguments", getName());
2021-05-17 07:30:42 +00:00
}
virtual ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const { return {}; }
#if USE_EMBEDDED_COMPILER
bool isCompilable(const DataTypes & arguments) const;
llvm::Value * compile(llvm::IRBuilderBase &, const DataTypes & arguments, Values values) const;
#endif
protected:
#if USE_EMBEDDED_COMPILER
virtual bool isCompilableImpl(const DataTypes &) const { return false; }
virtual llvm::Value * compileImpl(llvm::IRBuilderBase &, const DataTypes &, Values) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{} is not JIT-compilable", getName());
2021-05-17 07:30:42 +00:00
}
#endif
};
using FunctionPtr = std::shared_ptr<IFunction>;
2011-08-09 15:57:33 +00:00
}