2011-08-09 15:57:33 +00:00
|
|
|
#pragma once
|
|
|
|
|
2016-05-28 15:42:22 +00:00
|
|
|
#include <memory>
|
2011-08-09 19:19:00 +00:00
|
|
|
|
2019-06-24 11:17:15 +00:00
|
|
|
#include "config_core.h"
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/Names.h>
|
2017-06-13 02:06:53 +00:00
|
|
|
#include <Core/Field.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/Block.h>
|
|
|
|
#include <Core/ColumnNumbers.h>
|
|
|
|
#include <DataTypes/IDataType.h>
|
2011-08-09 15:57:33 +00:00
|
|
|
|
|
|
|
|
Add a JIT interface for row-wise default-nullable functions.
Not actually implemented, though. It does print out some jit-compiled stuff,
but that's about it. For example, this query:
select number from system.numbers where something(cast(number as Float64)) == 4
results in this on server's stderr:
define double @"something(CAST(number, 'Float64'))"(void**, i8*, void*) {
"something(CAST(number, 'Float64'))":
ret double 1.234500e+04
}
(and an exception, because that's what the non-jitted method does.)
As one may notice, this function neither reads the input (first argument;
tuple of arrays) nor writes the output (third argument; array), instead
returning some general nonsense.
In addition, `#if USE_EMBEDDED_COMPILER` doesn't work for some reason,
including LLVM headers requires -Wno-unused-parameter, this probably only
works on LLVM 5.0 due to rampant API instability, and I'm definitely
no expert on CMake. In short, there's still a long way to go.
2018-04-23 22:29:39 +00:00
|
|
|
namespace llvm
|
|
|
|
{
|
|
|
|
class LLVMContext;
|
|
|
|
class Value;
|
|
|
|
class IRBuilderBase;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2011-08-09 15:57:33 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
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;
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
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;
|
|
|
|
|
2018-12-02 11:00:23 +00:00
|
|
|
virtual void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count, bool dry_run) = 0;
|
2018-02-02 08:33:36 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
using PreparedFunctionPtr = std::shared_ptr<IPreparedFunction>;
|
|
|
|
|
2018-09-21 14:15:21 +00:00
|
|
|
/// Cache for functions result if it was executed on low cardinality column.
|
|
|
|
class PreparedFunctionLowCardinalityResultCache;
|
|
|
|
using PreparedFunctionLowCardinalityResultCachePtr = std::shared_ptr<PreparedFunctionLowCardinalityResultCache>;
|
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
class PreparedFunctionImpl : public IPreparedFunction
|
|
|
|
{
|
|
|
|
public:
|
2018-12-02 11:00:23 +00:00
|
|
|
void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count, bool dry_run = false) final;
|
2018-02-02 08:33:36 +00:00
|
|
|
|
2018-09-21 14:15:21 +00:00
|
|
|
/// Create cache which will be used to store result of function executed on LowCardinality column.
|
|
|
|
/// Only for default LowCardinality implementation.
|
2018-09-26 15:02:56 +00:00
|
|
|
/// Cannot be called concurrently for the same object.
|
2018-09-21 14:15:21 +00:00
|
|
|
void createLowCardinalityResultCache(size_t cache_size);
|
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
protected:
|
2018-04-24 07:16:39 +00:00
|
|
|
virtual void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) = 0;
|
2018-12-02 11:00:23 +00:00
|
|
|
virtual void executeImplDryRun(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
|
|
|
|
{
|
|
|
|
executeImpl(block, arguments, result, input_rows_count);
|
|
|
|
}
|
2018-02-02 08:33:36 +00:00
|
|
|
|
|
|
|
/** 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; }
|
|
|
|
|
2018-09-27 15:55:22 +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.
|
2018-04-17 17:47:27 +00:00
|
|
|
*/
|
2018-09-27 15:55:22 +00:00
|
|
|
virtual bool useDefaultImplementationForLowCardinalityColumns() const { return true; }
|
2018-04-17 17:47:27 +00:00
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
/** Some arguments could remain constant during this implementation.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2018-02-02 08:33:36 +00:00
|
|
|
virtual ColumnNumbers getArgumentsThatAreAlwaysConstant() const { return {}; }
|
|
|
|
|
2018-07-19 19:12:48 +00:00
|
|
|
/** 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; }
|
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
private:
|
2018-04-24 07:16:39 +00:00
|
|
|
bool defaultImplementationForNulls(Block & block, const ColumnNumbers & args, size_t result,
|
2018-12-02 11:00:23 +00:00
|
|
|
size_t input_rows_count, bool dry_run);
|
2018-04-24 07:16:39 +00:00
|
|
|
bool defaultImplementationForConstantArguments(Block & block, const ColumnNumbers & args, size_t result,
|
2018-12-02 11:00:23 +00:00
|
|
|
size_t input_rows_count, bool dry_run);
|
2018-09-27 15:55:22 +00:00
|
|
|
void executeWithoutLowCardinalityColumns(Block & block, const ColumnNumbers & arguments, size_t result,
|
2018-12-02 11:00:23 +00:00
|
|
|
size_t input_rows_count, bool dry_run);
|
2018-09-21 14:15:21 +00:00
|
|
|
|
|
|
|
/// Cache is created by function createLowCardinalityResultCache()
|
|
|
|
PreparedFunctionLowCardinalityResultCachePtr low_cardinality_result_cache;
|
2018-02-02 08:33:36 +00:00
|
|
|
};
|
|
|
|
|
2018-04-24 18:10:22 +00:00
|
|
|
using ValuePlaceholders = std::vector<std::function<llvm::Value * ()>>;
|
Add a JIT interface for row-wise default-nullable functions.
Not actually implemented, though. It does print out some jit-compiled stuff,
but that's about it. For example, this query:
select number from system.numbers where something(cast(number as Float64)) == 4
results in this on server's stderr:
define double @"something(CAST(number, 'Float64'))"(void**, i8*, void*) {
"something(CAST(number, 'Float64'))":
ret double 1.234500e+04
}
(and an exception, because that's what the non-jitted method does.)
As one may notice, this function neither reads the input (first argument;
tuple of arrays) nor writes the output (third argument; array), instead
returning some general nonsense.
In addition, `#if USE_EMBEDDED_COMPILER` doesn't work for some reason,
including LLVM headers requires -Wno-unused-parameter, this probably only
works on LLVM 5.0 due to rampant API instability, and I'm definitely
no expert on CMake. In short, there's still a long way to go.
2018-04-23 22:29:39 +00:00
|
|
|
|
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-04-01 07:20:54 +00:00
|
|
|
|
2017-05-28 14:32:59 +00:00
|
|
|
/// Get the main function name.
|
2017-04-01 07:20:54 +00:00
|
|
|
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;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
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.
|
2018-09-21 14:15:21 +00:00
|
|
|
virtual PreparedFunctionPtr prepare(const Block & sample_block, const ColumnNumbers & arguments, size_t result) const = 0;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
/// TODO: make const
|
2018-12-02 11:00:23 +00:00
|
|
|
virtual void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count, bool dry_run = false)
|
2018-02-02 08:33:36 +00:00
|
|
|
{
|
2018-12-02 11:00:23 +00:00
|
|
|
return prepare(block, arguments, result)->execute(block, arguments, result, input_rows_count, dry_run);
|
2018-02-02 08:33:36 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-29 10:47:03 +00:00
|
|
|
#if USE_EMBEDDED_COMPILER
|
|
|
|
|
2018-04-24 10:25:18 +00:00
|
|
|
virtual bool isCompilable() const { return false; }
|
|
|
|
|
2018-04-28 15:11:23 +00:00
|
|
|
/** 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.
|
Add a JIT interface for row-wise default-nullable functions.
Not actually implemented, though. It does print out some jit-compiled stuff,
but that's about it. For example, this query:
select number from system.numbers where something(cast(number as Float64)) == 4
results in this on server's stderr:
define double @"something(CAST(number, 'Float64'))"(void**, i8*, void*) {
"something(CAST(number, 'Float64'))":
ret double 1.234500e+04
}
(and an exception, because that's what the non-jitted method does.)
As one may notice, this function neither reads the input (first argument;
tuple of arrays) nor writes the output (third argument; array), instead
returning some general nonsense.
In addition, `#if USE_EMBEDDED_COMPILER` doesn't work for some reason,
including LLVM headers requires -Wno-unused-parameter, this probably only
works on LLVM 5.0 due to rampant API instability, and I'm definitely
no expert on CMake. In short, there's still a long way to go.
2018-04-23 22:29:39 +00:00
|
|
|
*
|
|
|
|
* 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.
|
|
|
|
*/
|
2018-04-27 21:30:38 +00:00
|
|
|
virtual llvm::Value * compile(llvm::IRBuilderBase & /*builder*/, ValuePlaceholders /*values*/) const
|
Add a JIT interface for row-wise default-nullable functions.
Not actually implemented, though. It does print out some jit-compiled stuff,
but that's about it. For example, this query:
select number from system.numbers where something(cast(number as Float64)) == 4
results in this on server's stderr:
define double @"something(CAST(number, 'Float64'))"(void**, i8*, void*) {
"something(CAST(number, 'Float64'))":
ret double 1.234500e+04
}
(and an exception, because that's what the non-jitted method does.)
As one may notice, this function neither reads the input (first argument;
tuple of arrays) nor writes the output (third argument; array), instead
returning some general nonsense.
In addition, `#if USE_EMBEDDED_COMPILER` doesn't work for some reason,
including LLVM headers requires -Wno-unused-parameter, this probably only
works on LLVM 5.0 due to rampant API instability, and I'm definitely
no expert on CMake. In short, there's still a long way to go.
2018-04-23 22:29:39 +00:00
|
|
|
{
|
2018-04-24 10:25:18 +00:00
|
|
|
throw Exception(getName() + " is not JIT-compilable", ErrorCodes::NOT_IMPLEMENTED);
|
Add a JIT interface for row-wise default-nullable functions.
Not actually implemented, though. It does print out some jit-compiled stuff,
but that's about it. For example, this query:
select number from system.numbers where something(cast(number as Float64)) == 4
results in this on server's stderr:
define double @"something(CAST(number, 'Float64'))"(void**, i8*, void*) {
"something(CAST(number, 'Float64'))":
ret double 1.234500e+04
}
(and an exception, because that's what the non-jitted method does.)
As one may notice, this function neither reads the input (first argument;
tuple of arrays) nor writes the output (third argument; array), instead
returning some general nonsense.
In addition, `#if USE_EMBEDDED_COMPILER` doesn't work for some reason,
including LLVM headers requires -Wno-unused-parameter, this probably only
works on LLVM 5.0 due to rampant API instability, and I'm definitely
no expert on CMake. In short, there's still a long way to go.
2018-04-23 22:29:39 +00:00
|
|
|
}
|
|
|
|
|
2018-04-29 10:47:03 +00:00
|
|
|
#endif
|
|
|
|
|
2019-01-30 02:47:26 +00:00
|
|
|
virtual bool isStateful() const { return false; }
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/** 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; }
|
|
|
|
|
2019-08-15 19:31:43 +00:00
|
|
|
virtual bool alwaysReturnsConstant() const { return false; }
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/** 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;
|
2017-04-01 07:20:54 +00:00
|
|
|
* 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; }
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/** 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.
|
|
|
|
*/
|
2018-02-21 17:05:21 +00:00
|
|
|
|
2018-08-08 11:26:18 +00:00
|
|
|
virtual bool isDeterministic() const { return true; }
|
2018-02-21 17:05:21 +00:00
|
|
|
|
2018-08-08 11:26:18 +00:00
|
|
|
virtual bool isDeterministicInScopeOfQuery() const { return true; }
|
2017-04-01 07:20:54 +00:00
|
|
|
|
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; }
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
/// The property of monotonicity for a certain range.
|
|
|
|
struct Monotonicity
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
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
|
|
|
|
|
2018-02-06 19:34:53 +00:00
|
|
|
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_) {}
|
|
|
|
};
|
2017-04-01 07:20:54 +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.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2018-02-02 08:33:36 +00:00
|
|
|
virtual Monotonicity getMonotonicityForRange(const IDataType & /*type*/, const Field & /*left*/, const Field & /*right*/) const
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-02-02 08:33:36 +00:00
|
|
|
throw Exception("Function " + getName() + " has no information about its monotonicity.", ErrorCodes::NOT_IMPLEMENTED);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
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;
|
|
|
|
|
2019-01-30 02:47:26 +00:00
|
|
|
/// Override and return true if function needs to depend on the state of the data.
|
|
|
|
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.
|
|
|
|
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;
|
2018-02-06 19:34:53 +00:00
|
|
|
|
|
|
|
/// 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
|
|
|
};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
using FunctionBuilderPtr = std::shared_ptr<IFunctionBuilder>;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
class FunctionBuilderImpl : public IFunctionBuilder
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
FunctionBasePtr build(const ColumnsWithTypeAndName & arguments) const final
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-02-02 08:33:36 +00:00
|
|
|
return buildImpl(arguments, getReturnType(arguments));
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
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;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments) const;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-02-06 19:34:53 +00:00
|
|
|
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
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
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);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2018-02-06 19:34:53 +00:00
|
|
|
virtual DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-02-02 08:33:36 +00:00
|
|
|
throw Exception("getReturnType is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
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; }
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-17 17:47:27 +00:00
|
|
|
/** If useDefaultImplementationForNulls() is true, than change arguments for getReturnType() and buildImpl().
|
2018-09-27 15:55:22 +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.
|
2018-04-17 17:47:27 +00:00
|
|
|
*/
|
2018-09-27 15:55:22 +00:00
|
|
|
virtual bool useDefaultImplementationForLowCardinalityColumns() const { return true; }
|
2018-04-17 17:47:27 +00:00
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
/// If it isn't, will convert all ColumnLowCardinality arguments to full columns.
|
2018-08-08 18:21:37 +00:00
|
|
|
virtual bool canBeExecutedOnLowCardinalityDictionary() const { return true; }
|
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
virtual FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const = 0;
|
2018-02-06 19:34:53 +00:00
|
|
|
|
|
|
|
virtual void getLambdaArgumentTypesImpl(DataTypes & /*arguments*/) const
|
|
|
|
{
|
|
|
|
throw Exception("Function " + getName() + " can't have lambda-expressions as arguments", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
}
|
2018-04-17 17:47:27 +00:00
|
|
|
|
|
|
|
private:
|
|
|
|
|
2018-09-27 15:55:22 +00:00
|
|
|
DataTypePtr getReturnTypeWithoutLowCardinality(const ColumnsWithTypeAndName & arguments) const;
|
2018-02-02 08:33:36 +00:00
|
|
|
};
|
2017-07-23 08:40:43 +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;
|
2019-01-30 04:37:24 +00:00
|
|
|
|
|
|
|
bool isStateful() const override { return false; }
|
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
/// TODO: make const
|
2018-04-24 07:16:39 +00:00
|
|
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override = 0;
|
2017-07-23 08:40:43 +00:00
|
|
|
|
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; }
|
2018-09-27 15:55:22 +00:00
|
|
|
bool useDefaultImplementationForLowCardinalityColumns() const override { return true; }
|
2018-02-02 08:33:36 +00:00
|
|
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; }
|
2018-07-19 19:12:48 +00:00
|
|
|
bool canBeExecutedOnDefaultArguments() const override { return true; }
|
2018-08-08 18:21:37 +00:00
|
|
|
bool canBeExecutedOnLowCardinalityDictionary() const override { return isDeterministicInScopeOfQuery(); }
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-02-06 19:34:53 +00:00
|
|
|
using PreparedFunctionImpl::execute;
|
2018-12-02 11:00:23 +00:00
|
|
|
using PreparedFunctionImpl::executeImplDryRun;
|
2018-02-06 19:34:53 +00:00
|
|
|
using FunctionBuilderImpl::getReturnTypeImpl;
|
|
|
|
using FunctionBuilderImpl::getLambdaArgumentTypesImpl;
|
|
|
|
using FunctionBuilderImpl::getReturnType;
|
|
|
|
|
2018-09-21 14:15:21 +00:00
|
|
|
PreparedFunctionPtr prepare(const Block & /*sample_block*/, const ColumnNumbers & /*arguments*/, size_t /*result*/) const final
|
2018-02-02 08:33:36 +00:00
|
|
|
{
|
|
|
|
throw Exception("prepare is not implemented for IFunction", ErrorCodes::NOT_IMPLEMENTED);
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-29 10:47:03 +00:00
|
|
|
#if USE_EMBEDDED_COMPILER
|
|
|
|
|
|
|
|
bool isCompilable() const final
|
2018-02-02 08:33:36 +00:00
|
|
|
{
|
2018-04-29 10:47:03 +00:00
|
|
|
throw Exception("isCompilable without explicit types is not implemented for IFunction", ErrorCodes::NOT_IMPLEMENTED);
|
2018-02-02 08:33:36 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-27 21:30:38 +00:00
|
|
|
llvm::Value * compile(llvm::IRBuilderBase & /*builder*/, ValuePlaceholders /*values*/) const final
|
2018-04-25 17:07:19 +00:00
|
|
|
{
|
|
|
|
throw Exception("compile without explicit types is not implemented for IFunction", ErrorCodes::NOT_IMPLEMENTED);
|
|
|
|
}
|
|
|
|
|
2018-04-29 10:47:03 +00:00
|
|
|
#endif
|
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
const DataTypes & getArgumentTypes() const final
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-02-02 08:33:36 +00:00
|
|
|
throw Exception("getArgumentTypes is not implemented for IFunction", ErrorCodes::NOT_IMPLEMENTED);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
Add a JIT interface for row-wise default-nullable functions.
Not actually implemented, though. It does print out some jit-compiled stuff,
but that's about it. For example, this query:
select number from system.numbers where something(cast(number as Float64)) == 4
results in this on server's stderr:
define double @"something(CAST(number, 'Float64'))"(void**, i8*, void*) {
"something(CAST(number, 'Float64'))":
ret double 1.234500e+04
}
(and an exception, because that's what the non-jitted method does.)
As one may notice, this function neither reads the input (first argument;
tuple of arrays) nor writes the output (third argument; array), instead
returning some general nonsense.
In addition, `#if USE_EMBEDDED_COMPILER` doesn't work for some reason,
including LLVM headers requires -Wno-unused-parameter, this probably only
works on LLVM 5.0 due to rampant API instability, and I'm definitely
no expert on CMake. In short, there's still a long way to go.
2018-04-23 22:29:39 +00:00
|
|
|
const DataTypePtr & getReturnType() const final
|
2018-02-06 19:34:53 +00:00
|
|
|
{
|
|
|
|
throw Exception("getReturnType is not implemented for IFunction", ErrorCodes::NOT_IMPLEMENTED);
|
|
|
|
}
|
|
|
|
|
2018-04-29 10:47:03 +00:00
|
|
|
#if USE_EMBEDDED_COMPILER
|
|
|
|
|
2018-04-27 21:30:38 +00:00
|
|
|
bool isCompilable(const DataTypes & arguments) const;
|
|
|
|
|
|
|
|
llvm::Value * compile(llvm::IRBuilderBase &, const DataTypes & arguments, ValuePlaceholders values) const;
|
|
|
|
|
2018-04-29 10:47:03 +00:00
|
|
|
#endif
|
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
protected:
|
2018-04-29 10:47:03 +00:00
|
|
|
|
|
|
|
#if USE_EMBEDDED_COMPILER
|
|
|
|
|
2018-04-27 21:30:38 +00:00
|
|
|
virtual bool isCompilableImpl(const DataTypes &) const { return false; }
|
|
|
|
|
|
|
|
virtual llvm::Value * compileImpl(llvm::IRBuilderBase &, const DataTypes &, ValuePlaceholders) const
|
|
|
|
{
|
|
|
|
throw Exception(getName() + " is not JIT-compilable", ErrorCodes::NOT_IMPLEMENTED);
|
|
|
|
}
|
|
|
|
|
2018-04-29 10:47:03 +00:00
|
|
|
#endif
|
|
|
|
|
2018-02-06 19:34:53 +00:00
|
|
|
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:
|
2019-08-03 11:02:40 +00:00
|
|
|
explicit DefaultExecutable(std::shared_ptr<IFunction> function_) : function(std::move(function_)) {}
|
2018-02-02 08:33:36 +00:00
|
|
|
|
2018-02-06 19:34:53 +00:00
|
|
|
String getName() const override { return function->getName(); }
|
|
|
|
|
2018-02-02 08:33:36 +00:00
|
|
|
protected:
|
2018-04-24 07:16:39 +00:00
|
|
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) final
|
2018-02-02 08:33:36 +00:00
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
return function->executeImpl(block, arguments, result, input_rows_count);
|
2018-02-02 08:33:36 +00:00
|
|
|
}
|
2018-12-02 11:00:23 +00:00
|
|
|
void executeImplDryRun(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) final
|
|
|
|
{
|
|
|
|
return function->executeImplDryRun(block, arguments, result, input_rows_count);
|
|
|
|
}
|
2018-02-02 08:33:36 +00:00
|
|
|
bool useDefaultImplementationForNulls() const final { return function->useDefaultImplementationForNulls(); }
|
|
|
|
bool useDefaultImplementationForConstants() const final { return function->useDefaultImplementationForConstants(); }
|
2018-09-27 15:55:22 +00:00
|
|
|
bool useDefaultImplementationForLowCardinalityColumns() const final { return function->useDefaultImplementationForLowCardinalityColumns(); }
|
2018-02-02 08:33:36 +00:00
|
|
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return function->getArgumentsThatAreAlwaysConstant(); }
|
2018-07-19 19:12:48 +00:00
|
|
|
bool canBeExecutedOnDefaultArguments() const override { return function->canBeExecutedOnDefaultArguments(); }
|
2018-02-02 08:33:36 +00:00
|
|
|
|
|
|
|
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:
|
2019-08-03 11:02:40 +00:00
|
|
|
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_)) {}
|
2018-02-02 08:33:36 +00:00
|
|
|
|
|
|
|
String getName() const override { return function->getName(); }
|
|
|
|
|
|
|
|
const DataTypes & getArgumentTypes() const override { return arguments; }
|
|
|
|
const DataTypePtr & getReturnType() const override { return return_type; }
|
|
|
|
|
2018-04-29 10:47:03 +00:00
|
|
|
#if USE_EMBEDDED_COMPILER
|
|
|
|
|
2018-04-24 10:25:18 +00:00
|
|
|
bool isCompilable() const override { return function->isCompilable(arguments); }
|
|
|
|
|
2018-04-27 21:30:38 +00:00
|
|
|
llvm::Value * compile(llvm::IRBuilderBase & builder, ValuePlaceholders values) const override { return function->compile(builder, arguments, std::move(values)); }
|
Add a JIT interface for row-wise default-nullable functions.
Not actually implemented, though. It does print out some jit-compiled stuff,
but that's about it. For example, this query:
select number from system.numbers where something(cast(number as Float64)) == 4
results in this on server's stderr:
define double @"something(CAST(number, 'Float64'))"(void**, i8*, void*) {
"something(CAST(number, 'Float64'))":
ret double 1.234500e+04
}
(and an exception, because that's what the non-jitted method does.)
As one may notice, this function neither reads the input (first argument;
tuple of arrays) nor writes the output (third argument; array), instead
returning some general nonsense.
In addition, `#if USE_EMBEDDED_COMPILER` doesn't work for some reason,
including LLVM headers requires -Wno-unused-parameter, this probably only
works on LLVM 5.0 due to rampant API instability, and I'm definitely
no expert on CMake. In short, there's still a long way to go.
2018-04-23 22:29:39 +00:00
|
|
|
|
2018-04-29 10:47:03 +00:00
|
|
|
#endif
|
|
|
|
|
2018-09-21 14:15:21 +00:00
|
|
|
PreparedFunctionPtr prepare(const Block & /*sample_block*/, const ColumnNumbers & /*arguments*/, size_t /*result*/) const override
|
|
|
|
{
|
|
|
|
return std::make_shared<DefaultExecutable>(function);
|
|
|
|
}
|
2018-02-02 08:33:36 +00:00
|
|
|
|
|
|
|
bool isSuitableForConstantFolding() const override { return function->isSuitableForConstantFolding(); }
|
2019-08-15 19:31:43 +00:00
|
|
|
bool alwaysReturnsConstant() const override { return function->alwaysReturnsConstant(); }
|
2018-02-02 08:33:36 +00:00
|
|
|
|
|
|
|
bool isInjective(const Block & sample_block) override { return function->isInjective(sample_block); }
|
|
|
|
|
2018-08-08 11:26:18 +00:00
|
|
|
bool isDeterministic() const override { return function->isDeterministic(); }
|
2018-02-21 17:05:21 +00:00
|
|
|
|
2018-08-08 11:26:18 +00:00
|
|
|
bool isDeterministicInScopeOfQuery() const override { return function->isDeterministicInScopeOfQuery(); }
|
2018-02-02 08:33:36 +00:00
|
|
|
|
|
|
|
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:
|
2019-08-03 11:02:40 +00:00
|
|
|
explicit DefaultFunctionBuilder(std::shared_ptr<IFunction> function_) : function(std::move(function_)) {}
|
2018-02-02 08:33:36 +00:00
|
|
|
|
|
|
|
void checkNumberOfArguments(size_t number_of_arguments) const override
|
|
|
|
{
|
|
|
|
return function->checkNumberOfArguments(number_of_arguments);
|
|
|
|
}
|
|
|
|
|
2018-06-03 17:43:56 +00:00
|
|
|
String getName() const override { return function->getName(); }
|
2019-01-30 04:37:24 +00:00
|
|
|
bool isStateful() const override { return function->isStateful(); }
|
2018-02-02 08:33:36 +00:00
|
|
|
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(); }
|
2018-09-27 15:55:22 +00:00
|
|
|
bool useDefaultImplementationForLowCardinalityColumns() const override { return function->useDefaultImplementationForLowCardinalityColumns(); }
|
2018-08-08 18:21:37 +00:00
|
|
|
bool canBeExecutedOnLowCardinalityDictionary() const override { return function->canBeExecutedOnLowCardinalityDictionary(); }
|
2018-02-02 08:33:36 +00:00
|
|
|
|
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
2018-02-06 19:34:53 +00:00
|
|
|
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
|
|
|
|
2016-05-28 15:42:22 +00:00
|
|
|
using FunctionPtr = std::shared_ptr<IFunction>;
|
2011-08-09 19:19:00 +00:00
|
|
|
|
2018-07-11 19:51:18 +00:00
|
|
|
|
|
|
|
/** Return ColumnNullable of src, with null map as OR-ed null maps of args columns in blocks.
|
|
|
|
* Or ColumnConst(ColumnNullable) if the result is always NULL or if the result is constant and always not NULL.
|
|
|
|
*/
|
|
|
|
ColumnPtr wrapInNullable(const ColumnPtr & src, const Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count);
|
|
|
|
|
2011-08-09 15:57:33 +00:00
|
|
|
}
|