ClickHouse/dbms/src/Functions/IFunction.h

417 lines
18 KiB
C++
Raw Normal View History

2011-08-09 15:57:33 +00:00
#pragma once
#include <memory>
2011-08-09 19:19:00 +00:00
#include <Core/Names.h>
2017-06-13 02:06:53 +00:00
#include <Core/Field.h>
#include <Core/Block.h>
#include <Core/ColumnNumbers.h>
#include <DataTypes/IDataType.h>
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
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
2017-06-13 02:06:53 +00:00
extern const int NOT_IMPLEMENTED;
2018-02-02 08:33:36 +00:00
extern const int LOGICAL_ERROR;
}
2018-02-08 16:59:04 +00:00
/// The simplest executable object.
/// Motivation:
2018-02-09 19:15:40 +00:00
/// * Prepare something heavy once before main execution loop instead of doing it for each block.
2018-02-09 19:20:18 +00:00
/// * Provide const interface for IFunctionBase (later).
2018-02-02 08:33:36 +00:00
class IPreparedFunction
2011-08-09 15:57:33 +00:00
{
public:
2018-02-02 08:33:36 +00:00
virtual ~IPreparedFunction() = default;
/// Get the main function name.
virtual String getName() const = 0;
virtual void execute(Block & block, const ColumnNumbers & arguments, size_t result) = 0;
};
using PreparedFunctionPtr = std::shared_ptr<IPreparedFunction>;
class PreparedFunctionImpl : public IPreparedFunction
{
public:
void execute(Block & block, const ColumnNumbers & arguments, size_t result) final;
protected:
virtual void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) = 0;
/** 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 block,
* 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; }
/** 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; }
/** Some arguments could remain constant during this implementation.
*/
2018-02-02 08:33:36 +00:00
virtual ColumnNumbers getArgumentsThatAreAlwaysConstant() const { return {}; }
private:
bool defaultImplementationForNulls(Block & block, const ColumnNumbers & args, size_t result);
bool defaultImplementationForConstantArguments(Block & block, const ColumnNumbers & args, size_t result);
};
using ValuePlaceholders = std::vector<llvm::Value *>;
2018-02-08 16:59:04 +00:00
/// Function with known arguments and return type.
2018-02-02 08:33:36 +00:00
class IFunctionBase
{
public:
virtual ~IFunctionBase() = default;
2017-05-28 14:32:59 +00:00
/// Get the main function name.
virtual String getName() const = 0;
2018-02-02 08:33:36 +00:00
virtual const DataTypes & getArgumentTypes() const = 0;
virtual const DataTypePtr & getReturnType() const = 0;
2018-02-02 08:33:36 +00:00
/// Do preparations and return executable.
/// sample_block should contain data types of arguments and values of constants, if relevant.
virtual PreparedFunctionPtr prepare(const Block & sample_block) const = 0;
2018-02-02 08:33:36 +00:00
/// TODO: make const
virtual void execute(Block & block, const ColumnNumbers & arguments, size_t result)
{
return prepare(block)->execute(block, arguments, result);
}
virtual bool isCompilable() const { return false; }
/** Produce LLVM IR code that operates on *scalar* values. JIT-compilation is only supported for native
* data types, i.e. numbers. This method will never be called if there is a non-number argument or
* a non-number result type. Also, for any compilable function default behavior on NULL values is assumed,
* i.e. the result is NULL if and only if any argument is NULL.
*
* 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.
*/
virtual llvm::Value * compile(llvm::IRBuilderBase & /*builder*/, const ValuePlaceholders & /*values*/) const
{
throw Exception(getName() + " is not JIT-compilable", ErrorCodes::NOT_IMPLEMENTED);
}
/** 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; }
/** 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);
* multiply(const, x) where x is an integer and constant is not divisable 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.
*
* sample_block should contain data types of arguments and values of constants, if relevant.
*/
2017-12-01 19:34:51 +00:00
virtual bool isInjective(const Block & /*sample_block*/) { 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.
*/
virtual bool isDeterministic() { return true; }
virtual bool isDeterministicInScopeOfQuery() { 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; }
2018-02-02 08:33:36 +00:00
/// The property of monotonicity for a certain range.
struct Monotonicity
{
2018-02-02 08:33:36 +00:00
bool is_monotonic = false; /// Is the function monotonous (nondecreasing or nonincreasing).
bool is_positive = true; /// true if the function is nondecreasing, false, if notincreasing. If is_monotonic = false, then it does not matter.
bool is_always_monotonic = false; /// Is true if function is monotonic on the whole input range I
Monotonicity(bool is_monotonic_ = false, bool is_positive_ = true, bool is_always_monotonic_ = false)
2018-02-02 08:33:36 +00:00
: is_monotonic(is_monotonic_), is_positive(is_positive_), is_always_monotonic(is_always_monotonic_) {}
};
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
{
2018-02-02 08:33:36 +00:00
throw Exception("Function " + getName() + " has no information about its monotonicity.", ErrorCodes::NOT_IMPLEMENTED);
}
2018-02-02 08:33:36 +00:00
};
using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
2018-02-09 19:20:18 +00:00
/// Creates IFunctionBase from argument types list.
2018-02-02 08:33:36 +00:00
class IFunctionBuilder
{
public:
virtual ~IFunctionBuilder() = default;
/// Get the main function name.
virtual String getName() const = 0;
/// Override and return true if function could take different number of arguments.
virtual bool isVariadic() const { return false; }
/// For non-variadic functions, return number of arguments; otherwise return zero (that should be ignored).
virtual size_t getNumberOfArguments() const = 0;
/// Throw if number of arguments is incorrect. Default implementation will check only in non-variadic case.
virtual void checkNumberOfArguments(size_t number_of_arguments) const = 0;
2018-02-09 19:20:18 +00:00
/// Check arguments and return IFunctionBase.
2018-02-02 08:33:36 +00:00
virtual FunctionBasePtr build(const ColumnsWithTypeAndName & arguments) const = 0;
/// 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.
virtual void getLambdaArgumentTypes(DataTypes & arguments) const = 0;
2018-02-02 08:33:36 +00:00
};
2018-02-02 08:33:36 +00:00
using FunctionBuilderPtr = std::shared_ptr<IFunctionBuilder>;
2018-02-02 08:33:36 +00:00
class FunctionBuilderImpl : public IFunctionBuilder
{
public:
FunctionBasePtr build(const ColumnsWithTypeAndName & arguments) const final
{
2018-02-02 08:33:36 +00:00
return buildImpl(arguments, getReturnType(arguments));
}
2018-02-02 08:33:36 +00:00
/// Default implementation. Will check only in non-variadic case.
void checkNumberOfArguments(size_t number_of_arguments) const override;
2018-02-02 08:33:36 +00:00
DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments) const;
void getLambdaArgumentTypes(DataTypes & arguments) const override
{
checkNumberOfArguments(arguments.size());
getLambdaArgumentTypesImpl(arguments);
}
2018-02-02 08:33:36 +00:00
protected:
/// 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
{
2018-02-02 08:33:36 +00:00
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 DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const
{
2018-02-02 08:33:36 +00:00
throw Exception("getReturnType is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
2018-02-02 08:33:36 +00:00
/** If useDefaultImplementationForNulls() is true, than change arguments for getReturnType() and buildImpl():
* if some of arguments are Nullable(Nothing) then don't call getReturnType(), call buildImpl() 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 buildImpl
*
* Otherwise build returns buildImpl(arguments, getReturnType(arguments));
2017-07-24 01:00:31 +00:00
*/
virtual bool useDefaultImplementationForNulls() const { return true; }
2018-02-02 08:33:36 +00:00
virtual FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const = 0;
virtual void getLambdaArgumentTypesImpl(DataTypes & /*arguments*/) const
{
throw Exception("Function " + getName() + " can't have lambda-expressions as arguments", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
2018-02-02 08:33:36 +00:00
};
2018-02-08 16:59:04 +00:00
/// Previous function interface.
2018-02-02 08:33:36 +00:00
class IFunction : public std::enable_shared_from_this<IFunction>,
public FunctionBuilderImpl, public IFunctionBase, public PreparedFunctionImpl
{
public:
String getName() const override = 0;
/// TODO: make const
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override = 0;
2018-02-02 08:33:36 +00:00
/// Override this functions to change default implementation behavior. See details in IMyFunction.
bool useDefaultImplementationForNulls() const override { return true; }
bool useDefaultImplementationForConstants() const override { return false; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; }
using PreparedFunctionImpl::execute;
using FunctionBuilderImpl::getReturnTypeImpl;
using FunctionBuilderImpl::getLambdaArgumentTypesImpl;
using FunctionBuilderImpl::getReturnType;
virtual bool isCompilable(const DataTypes & /*types*/) const { return false; }
PreparedFunctionPtr prepare(const Block & /*sample_block*/) const final
2018-02-02 08:33:36 +00:00
{
throw Exception("prepare is not implemented for IFunction", ErrorCodes::NOT_IMPLEMENTED);
}
virtual llvm::Value * compile(llvm::IRBuilderBase & /*builder*/, const DataTypes & /*types*/, const ValuePlaceholders & /*values*/) const
{
throw Exception(getName() + " is not JIT-compilable", ErrorCodes::NOT_IMPLEMENTED);
}
2018-02-02 08:33:36 +00:00
const DataTypes & getArgumentTypes() const final
{
2018-02-02 08:33:36 +00:00
throw Exception("getArgumentTypes is not implemented for IFunction", ErrorCodes::NOT_IMPLEMENTED);
}
const DataTypePtr & getReturnType() const final
{
throw Exception("getReturnType is not implemented for IFunction", ErrorCodes::NOT_IMPLEMENTED);
}
2018-02-02 08:33:36 +00:00
protected:
FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & /*arguments*/, const DataTypePtr & /*return_type*/) const final
2018-02-02 08:33:36 +00:00
{
throw Exception("buildImpl is not implemented for IFunction", ErrorCodes::NOT_IMPLEMENTED);
}
};
2018-02-08 16:59:04 +00:00
/// Wrappers over IFunction.
2018-02-02 08:33:36 +00:00
class DefaultExecutable final : public PreparedFunctionImpl
{
public:
explicit DefaultExecutable(std::shared_ptr<IFunction> function) : function(std::move(function)) {}
String getName() const override { return function->getName(); }
2018-02-02 08:33:36 +00:00
protected:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) final
{
return function->executeImpl(block, arguments, result);
}
bool useDefaultImplementationForNulls() const final { return function->useDefaultImplementationForNulls(); }
bool useDefaultImplementationForConstants() const final { return function->useDefaultImplementationForConstants(); }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return function->getArgumentsThatAreAlwaysConstant(); }
private:
std::shared_ptr<IFunction> function;
2011-08-09 15:57:33 +00:00
};
2018-02-02 08:33:36 +00:00
class DefaultFunction final : public IFunctionBase
{
public:
DefaultFunction(std::shared_ptr<IFunction> function, DataTypes arguments, DataTypePtr return_type)
2018-02-02 08:33:36 +00:00
: 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; }
bool isCompilable() const override { return function->isCompilable(arguments); }
llvm::Value * compile(llvm::IRBuilderBase & builder, const ValuePlaceholders & values) const override { return function->compile(builder, arguments, values); }
PreparedFunctionPtr prepare(const Block & /*sample_block*/) const override { return std::make_shared<DefaultExecutable>(function); }
2018-02-02 08:33:36 +00:00
bool isSuitableForConstantFolding() const override { return function->isSuitableForConstantFolding(); }
bool isInjective(const Block & sample_block) override { return function->isInjective(sample_block); }
bool isDeterministic() override { return function->isDeterministic(); }
2018-02-02 08:33:36 +00:00
bool isDeterministicInScopeOfQuery() override { return function->isDeterministicInScopeOfQuery(); }
bool hasInformationAboutMonotonicity() const override { return function->hasInformationAboutMonotonicity(); }
IFunctionBase::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 DefaultFunctionBuilder : public FunctionBuilderImpl
{
public:
explicit DefaultFunctionBuilder(std::shared_ptr<IFunction> function) : function(std::move(function)) {}
void checkNumberOfArguments(size_t number_of_arguments) const override
{
return function->checkNumberOfArguments(number_of_arguments);
}
String getName() const override { return function->getName(); };
bool isVariadic() const override { return function->isVariadic(); }
size_t getNumberOfArguments() const override { return function->getNumberOfArguments(); }
protected:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { return function->getReturnTypeImpl(arguments); }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { return function->getReturnTypeImpl(arguments); }
bool useDefaultImplementationForNulls() const override { return function->useDefaultImplementationForNulls(); }
FunctionBasePtr buildImpl(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_shared<DefaultFunction>(function, data_types, return_type);
}
void getLambdaArgumentTypesImpl(DataTypes & arguments) const override { return function->getLambdaArgumentTypesImpl(arguments); }
2018-02-02 08:33:36 +00:00
private:
std::shared_ptr<IFunction> function;
};
2011-08-09 15:57:33 +00:00
using FunctionPtr = std::shared_ptr<IFunction>;
2011-08-09 19:19:00 +00:00
2011-08-09 15:57:33 +00:00
}