mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Update functions interface.
This commit is contained in:
parent
388f69b109
commit
571a5fbff3
@ -51,7 +51,7 @@ Columns convertConstTupleToConstantElements(const ColumnConst & column)
|
||||
}
|
||||
|
||||
|
||||
static ColumnsWithTypeAndName createBlockWithNestedColumnsImpl(const ColumnsWithTypeAndName & columns, const std::unordered_set<size_t> & args)
|
||||
static ColumnsWithTypeAndName createBlockWithNestedColumns(const ColumnsWithTypeAndName & columns)
|
||||
{
|
||||
ColumnsWithTypeAndName res;
|
||||
size_t num_columns = columns.size();
|
||||
@ -88,20 +88,6 @@ static ColumnsWithTypeAndName createBlockWithNestedColumnsImpl(const ColumnsWith
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
ColumnsWithTypeAndName createBlockWithNestedColumns(const ColumnsWithTypeAndName & columns, const ColumnNumbers & args)
|
||||
{
|
||||
std::unordered_set<size_t> args_set(args.begin(), args.end());
|
||||
return createBlockWithNestedColumnsImpl(columns, args_set);
|
||||
}
|
||||
|
||||
ColumnsWithTypeAndName createBlockWithNestedColumns(const ColumnsWithTypeAndName & columns, const ColumnNumbers & args, size_t result)
|
||||
{
|
||||
std::unordered_set<size_t> args_set(args.begin(), args.end());
|
||||
args_set.insert(result);
|
||||
return createBlockWithNestedColumnsImpl(columns, args_set);
|
||||
}
|
||||
|
||||
void validateArgumentType(const IFunction & func, const DataTypes & arguments,
|
||||
size_t argument_index, bool (* validator_func)(const IDataType &),
|
||||
const char * expected_type_description)
|
||||
|
@ -82,13 +82,9 @@ inline std::enable_if_t<IsDecimalNumber<T>, Field> toField(const T & x, UInt32 s
|
||||
Columns convertConstTupleToConstantElements(const ColumnConst & column);
|
||||
|
||||
|
||||
/// Returns the copy of a given columns in which each column specified in
|
||||
/// the "arguments" parameter is replaced with its respective nested
|
||||
/// Returns the copy of a given columns in which each column is replaced with its respective nested
|
||||
/// column if it is nullable.
|
||||
ColumnsWithTypeAndName createBlockWithNestedColumns(const ColumnsWithTypeAndName & columns, const ColumnNumbers & args);
|
||||
|
||||
/// Similar function as above. Additionally transform the result type if needed.
|
||||
ColumnsWithTypeAndName createBlockWithNestedColumns(const ColumnsWithTypeAndName & columns, const ColumnNumbers & args, size_t result);
|
||||
ColumnsWithTypeAndName createBlockWithNestedColumns(const ColumnsWithTypeAndName & columns);
|
||||
|
||||
/// Checks argument type at specified index with predicate.
|
||||
/// throws if there is no argument at specified index or if predicate returns false.
|
||||
|
@ -14,7 +14,6 @@
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <ext/range.h>
|
||||
#include <ext/collection_cast.h>
|
||||
#include <cstdlib>
|
||||
#include <memory>
|
||||
@ -104,7 +103,7 @@ void ExecutableFunctionAdaptor::createLowCardinalityResultCache(size_t cache_siz
|
||||
}
|
||||
|
||||
|
||||
ColumnPtr wrapInNullable(const ColumnPtr & src, const ColumnsWithTypeAndName & columns, const ColumnNumbers & args, size_t result, size_t input_rows_count)
|
||||
ColumnPtr wrapInNullable(const ColumnPtr & src, const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count)
|
||||
{
|
||||
ColumnPtr result_null_map_column;
|
||||
|
||||
@ -119,16 +118,14 @@ ColumnPtr wrapInNullable(const ColumnPtr & src, const ColumnsWithTypeAndName & c
|
||||
result_null_map_column = nullable->getNullMapColumnPtr();
|
||||
}
|
||||
|
||||
for (const auto & arg : args)
|
||||
for (const auto & elem : args)
|
||||
{
|
||||
const ColumnWithTypeAndName & elem = columns[arg];
|
||||
if (!elem.type->isNullable())
|
||||
continue;
|
||||
|
||||
/// Const Nullable that are NULL.
|
||||
if (elem.column->onlyNull())
|
||||
{
|
||||
auto result_type = columns[result].type;
|
||||
assert(result_type->isNullable());
|
||||
return result_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
}
|
||||
@ -207,35 +204,36 @@ NullPresence getNullPresense(const ColumnsWithTypeAndName & args)
|
||||
return res;
|
||||
}
|
||||
|
||||
bool allArgumentsAreConstants(const ColumnsWithTypeAndName & columns, const ColumnNumbers & args)
|
||||
bool allArgumentsAreConstants(const ColumnsWithTypeAndName & args)
|
||||
{
|
||||
for (auto arg : args)
|
||||
if (!isColumnConst(*columns[arg].column))
|
||||
for (const auto & arg : args)
|
||||
if (!isColumnConst(*arg.column))
|
||||
return false;
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
bool ExecutableFunctionAdaptor::defaultImplementationForConstantArguments(
|
||||
ColumnsWithTypeAndName & columns, const ColumnNumbers & args, size_t result, size_t input_rows_count, bool dry_run)
|
||||
ColumnPtr ExecutableFunctionAdaptor::defaultImplementationForConstantArguments(
|
||||
ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run)
|
||||
{
|
||||
ColumnNumbers arguments_to_remain_constants = impl->getArgumentsThatAreAlwaysConstant();
|
||||
|
||||
/// Check that these arguments are really constant.
|
||||
for (auto arg_num : arguments_to_remain_constants)
|
||||
if (arg_num < args.size() && !isColumnConst(*columns[args[arg_num]].column))
|
||||
if (arg_num < args.size() && !isColumnConst(*args[arg_num].column))
|
||||
throw Exception("Argument at index " + toString(arg_num) + " for function " + getName() + " must be constant", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if (args.empty() || !impl->useDefaultImplementationForConstants() || !allArgumentsAreConstants(columns, args))
|
||||
return false;
|
||||
if (args.empty() || !impl->useDefaultImplementationForConstants() || !allArgumentsAreConstants(args))
|
||||
return nullptr;
|
||||
|
||||
ColumnsWithTypeAndName temporary_columns;
|
||||
bool have_converted_columns = false;
|
||||
|
||||
size_t arguments_size = args.size();
|
||||
temporary_columns.reserve(arguments_size);
|
||||
for (size_t arg_num = 0; arg_num < arguments_size; ++arg_num)
|
||||
{
|
||||
const ColumnWithTypeAndName & column = columns[args[arg_num]];
|
||||
const ColumnWithTypeAndName & column = args[arg_num];
|
||||
|
||||
if (arguments_to_remain_constants.end() != std::find(arguments_to_remain_constants.begin(), arguments_to_remain_constants.end(), arg_num))
|
||||
{
|
||||
@ -255,80 +253,66 @@ bool ExecutableFunctionAdaptor::defaultImplementationForConstantArguments(
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: the function requires more arguments",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
temporary_columns.emplace_back(columns[result]);
|
||||
ColumnPtr result_column = executeWithoutLowCardinalityColumns(temporary_columns, result_type, 1, dry_run);
|
||||
|
||||
ColumnNumbers temporary_argument_numbers(arguments_size);
|
||||
for (size_t i = 0; i < arguments_size; ++i)
|
||||
temporary_argument_numbers[i] = i;
|
||||
|
||||
executeWithoutLowCardinalityColumns(temporary_columns, temporary_argument_numbers, arguments_size, 1, dry_run);
|
||||
|
||||
ColumnPtr result_column;
|
||||
/// extremely rare case, when we have function with completely const arguments
|
||||
/// but some of them produced by non isDeterministic function
|
||||
if (temporary_columns[arguments_size].column->size() > 1)
|
||||
result_column = temporary_columns[arguments_size].column->cloneResized(1);
|
||||
else
|
||||
result_column = temporary_columns[arguments_size].column;
|
||||
if (result_column->size() > 1)
|
||||
result_column = result_column->cloneResized(1);
|
||||
|
||||
columns[result].column = ColumnConst::create(result_column, input_rows_count);
|
||||
return true;
|
||||
return ColumnConst::create(result_column, input_rows_count);
|
||||
}
|
||||
|
||||
|
||||
bool ExecutableFunctionAdaptor::defaultImplementationForNulls(
|
||||
ColumnsWithTypeAndName & columns, const ColumnNumbers & args, size_t result, size_t input_rows_count, bool dry_run)
|
||||
ColumnPtr ExecutableFunctionAdaptor::defaultImplementationForNulls(ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run)
|
||||
{
|
||||
if (args.empty() || !impl->useDefaultImplementationForNulls())
|
||||
return false;
|
||||
return nullptr;
|
||||
|
||||
NullPresence null_presence = getNullPresense(columns, args);
|
||||
NullPresence null_presence = getNullPresense(args);
|
||||
|
||||
if (null_presence.has_null_constant)
|
||||
{
|
||||
auto & result_column = columns[result].column;
|
||||
auto result_type = columns[result].type;
|
||||
// Default implementation for nulls returns null result for null arguments,
|
||||
// so the result type must be nullable.
|
||||
assert(result_type->isNullable());
|
||||
|
||||
result_column = result_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
return true;
|
||||
return result_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
}
|
||||
|
||||
if (null_presence.has_nullable)
|
||||
{
|
||||
ColumnsWithTypeAndName temporary_columns = createBlockWithNestedColumns(columns, args, result);
|
||||
executeWithoutLowCardinalityColumns(temporary_columns, args, result, input_rows_count, dry_run);
|
||||
columns[result].column = wrapInNullable(temporary_columns[result].column, columns, args, result, input_rows_count);
|
||||
return true;
|
||||
ColumnsWithTypeAndName temporary_columns = createBlockWithNestedColumns(args);
|
||||
auto temporary_result_type = removeNullable(result_type);
|
||||
|
||||
auto res = executeWithoutLowCardinalityColumns(temporary_columns, temporary_result_type, input_rows_count, dry_run);
|
||||
return wrapInNullable(res, args, result_type, input_rows_count);
|
||||
}
|
||||
|
||||
return false;
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
void ExecutableFunctionAdaptor::executeWithoutLowCardinalityColumns(
|
||||
ColumnsWithTypeAndName & columns, const ColumnNumbers & args, size_t result, size_t input_rows_count, bool dry_run)
|
||||
ColumnPtr ExecutableFunctionAdaptor::executeWithoutLowCardinalityColumns(
|
||||
ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run)
|
||||
{
|
||||
if (defaultImplementationForConstantArguments(columns, args, result, input_rows_count, dry_run))
|
||||
return;
|
||||
if (auto res = defaultImplementationForConstantArguments(args, result_type, input_rows_count, dry_run))
|
||||
return res;
|
||||
|
||||
if (defaultImplementationForNulls(columns, args, result, input_rows_count, dry_run))
|
||||
return;
|
||||
if (auto res = defaultImplementationForNulls(args, result_type, input_rows_count, dry_run))
|
||||
return res;
|
||||
|
||||
if (dry_run)
|
||||
impl->executeDryRun(columns, args, result, input_rows_count);
|
||||
return impl->executeDryRun(args, result_type, input_rows_count);
|
||||
else
|
||||
impl->execute(columns, args, result, input_rows_count);
|
||||
return impl->execute(args, result_type, input_rows_count);
|
||||
}
|
||||
|
||||
static const ColumnLowCardinality * findLowCardinalityArgument(const ColumnsWithTypeAndName & columns, const ColumnNumbers & args)
|
||||
static const ColumnLowCardinality * findLowCardinalityArgument(const ColumnsWithTypeAndName & arguments)
|
||||
{
|
||||
const ColumnLowCardinality * result_column = nullptr;
|
||||
|
||||
for (auto arg : args)
|
||||
for (const auto & column : arguments)
|
||||
{
|
||||
const ColumnWithTypeAndName & column = columns[arg];
|
||||
if (const auto * low_cardinality_column = checkAndGetColumn<ColumnLowCardinality>(column.column.get()))
|
||||
{
|
||||
if (result_column)
|
||||
@ -342,15 +326,14 @@ static const ColumnLowCardinality * findLowCardinalityArgument(const ColumnsWith
|
||||
}
|
||||
|
||||
static ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes(
|
||||
ColumnsWithTypeAndName & columns, const ColumnNumbers & args, bool can_be_executed_on_default_arguments, size_t input_rows_count)
|
||||
ColumnsWithTypeAndName & args, bool can_be_executed_on_default_arguments, size_t input_rows_count)
|
||||
{
|
||||
size_t num_rows = input_rows_count;
|
||||
ColumnPtr indexes;
|
||||
|
||||
/// Find first LowCardinality column and replace it to nested dictionary.
|
||||
for (auto arg : args)
|
||||
for (auto & column : args)
|
||||
{
|
||||
ColumnWithTypeAndName & column = columns[arg];
|
||||
if (const auto * low_cardinality_column = checkAndGetColumn<ColumnLowCardinality>(column.column.get()))
|
||||
{
|
||||
/// Single LowCardinality column is supported now.
|
||||
@ -384,9 +367,8 @@ static ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes(
|
||||
}
|
||||
|
||||
/// Change size of constants.
|
||||
for (auto arg : args)
|
||||
for (auto & column : args)
|
||||
{
|
||||
ColumnWithTypeAndName & column = columns[arg];
|
||||
if (const auto * column_const = checkAndGetColumn<ColumnConst>(column.column.get()))
|
||||
{
|
||||
column.column = column_const->removeLowCardinality()->cloneResized(num_rows);
|
||||
@ -397,12 +379,10 @@ static ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes(
|
||||
return indexes;
|
||||
}
|
||||
|
||||
static void convertLowCardinalityColumnsToFull(ColumnsWithTypeAndName & columns, const ColumnNumbers & args)
|
||||
static void convertLowCardinalityColumnsToFull(ColumnsWithTypeAndName & args)
|
||||
{
|
||||
for (auto arg : args)
|
||||
for (auto & column : args)
|
||||
{
|
||||
ColumnWithTypeAndName & column = columns[arg];
|
||||
|
||||
column.column = recursiveRemoveLowCardinality(column.column);
|
||||
column.type = recursiveRemoveLowCardinality(column.type);
|
||||
}
|
||||
@ -419,19 +399,15 @@ static ColumnsWithTypeAndName cloneWithEmptyColumns(const ColumnsWithTypeAndName
|
||||
return res;
|
||||
}
|
||||
|
||||
void ExecutableFunctionAdaptor::execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count, bool dry_run)
|
||||
ColumnPtr ExecutableFunctionAdaptor::execute(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run)
|
||||
{
|
||||
if (impl->useDefaultImplementationForLowCardinalityColumns())
|
||||
{
|
||||
auto & res = columns[result];
|
||||
ColumnsWithTypeAndName columns_without_low_cardinality = cloneWithEmptyColumns(columns);
|
||||
ColumnsWithTypeAndName columns_without_low_cardinality = arguments;
|
||||
|
||||
for (auto arg : arguments)
|
||||
columns_without_low_cardinality[arg].column = columns[arg].column;
|
||||
|
||||
if (const auto * res_low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(res.type.get()))
|
||||
if (const auto * res_low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(result_type.get()))
|
||||
{
|
||||
const auto * low_cardinality_column = findLowCardinalityArgument(columns, arguments);
|
||||
const auto * low_cardinality_column = findLowCardinalityArgument(arguments);
|
||||
bool can_be_executed_on_default_arguments = impl->canBeExecutedOnDefaultArguments();
|
||||
bool use_cache = low_cardinality_result_cache && can_be_executed_on_default_arguments
|
||||
&& low_cardinality_column && low_cardinality_column->isSharedDictionary();
|
||||
@ -446,22 +422,20 @@ void ExecutableFunctionAdaptor::execute(ColumnsWithTypeAndName & columns, const
|
||||
if (cached_values)
|
||||
{
|
||||
auto indexes = cached_values->index_mapping->index(low_cardinality_column->getIndexes(), 0);
|
||||
res.column = ColumnLowCardinality::create(cached_values->function_result, indexes, true);
|
||||
return;
|
||||
return ColumnLowCardinality::create(cached_values->function_result, indexes, true);
|
||||
}
|
||||
}
|
||||
|
||||
columns_without_low_cardinality[result].type = res_low_cardinality_type->getDictionaryType();
|
||||
const auto & dictionary_type = res_low_cardinality_type->getDictionaryType();
|
||||
ColumnPtr indexes = replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes(
|
||||
columns_without_low_cardinality, arguments, can_be_executed_on_default_arguments, input_rows_count);
|
||||
columns_without_low_cardinality, can_be_executed_on_default_arguments, input_rows_count);
|
||||
|
||||
size_t new_input_rows_count = arguments.empty()
|
||||
size_t new_input_rows_count = columns_without_low_cardinality.empty()
|
||||
? input_rows_count
|
||||
: columns_without_low_cardinality[arguments.front()].column->size();
|
||||
: columns_without_low_cardinality.front().column->size();
|
||||
|
||||
executeWithoutLowCardinalityColumns(columns_without_low_cardinality, arguments, result, new_input_rows_count, dry_run);
|
||||
|
||||
auto keys = columns_without_low_cardinality[result].column->convertToFullColumnIfConst();
|
||||
auto res = executeWithoutLowCardinalityColumns(columns_without_low_cardinality, dictionary_type, new_input_rows_count, dry_run);
|
||||
auto keys = res->convertToFullColumnIfConst();
|
||||
|
||||
auto res_mut_dictionary = DataTypeLowCardinality::createColumnUnique(*res_low_cardinality_type->getDictionaryType());
|
||||
ColumnPtr res_indexes = res_mut_dictionary->uniqueInsertRangeFrom(*keys, 0, keys->size());
|
||||
@ -481,22 +455,21 @@ void ExecutableFunctionAdaptor::execute(ColumnsWithTypeAndName & columns, const
|
||||
res_indexes = cache_values->index_mapping;
|
||||
}
|
||||
|
||||
res.column = ColumnLowCardinality::create(res_dictionary, res_indexes->index(*indexes, 0), use_cache);
|
||||
return ColumnLowCardinality::create(res_dictionary, res_indexes->index(*indexes, 0), use_cache);
|
||||
}
|
||||
else
|
||||
{
|
||||
res.column = ColumnLowCardinality::create(res_dictionary, res_indexes);
|
||||
return ColumnLowCardinality::create(res_dictionary, res_indexes);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
convertLowCardinalityColumnsToFull(columns_without_low_cardinality, arguments);
|
||||
executeWithoutLowCardinalityColumns(columns_without_low_cardinality, arguments, result, input_rows_count, dry_run);
|
||||
res.column = columns_without_low_cardinality[result].column;
|
||||
convertLowCardinalityColumnsToFull(columns_without_low_cardinality);
|
||||
return executeWithoutLowCardinalityColumns(columns_without_low_cardinality, result_type, input_rows_count, dry_run);
|
||||
}
|
||||
}
|
||||
else
|
||||
executeWithoutLowCardinalityColumns(columns, arguments, result, input_rows_count, dry_run);
|
||||
return executeWithoutLowCardinalityColumns( arguments, result_type, input_rows_count, dry_run);
|
||||
}
|
||||
|
||||
void FunctionOverloadResolverAdaptor::checkNumberOfArguments(size_t number_of_arguments) const
|
||||
@ -526,9 +499,7 @@ DataTypePtr FunctionOverloadResolverAdaptor::getReturnTypeWithoutLowCardinality(
|
||||
}
|
||||
if (null_presence.has_nullable)
|
||||
{
|
||||
Block nested_columns = createBlockWithNestedColumns(
|
||||
arguments,
|
||||
ext::collection_cast<ColumnNumbers>(ext::range(0, arguments.size())));
|
||||
Block nested_columns = createBlockWithNestedColumns(arguments);
|
||||
auto return_type = impl->getReturnType(ColumnsWithTypeAndName(nested_columns.begin(), nested_columns.end()));
|
||||
return makeNullable(return_type);
|
||||
}
|
||||
|
@ -46,7 +46,7 @@ public:
|
||||
/// Get the main function name.
|
||||
virtual String getName() const = 0;
|
||||
|
||||
virtual void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count, bool dry_run) = 0;
|
||||
virtual ColumnPtr execute(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) = 0;
|
||||
|
||||
virtual void createLowCardinalityResultCache(size_t cache_size) = 0;
|
||||
};
|
||||
@ -67,16 +67,16 @@ public:
|
||||
virtual String getName() const = 0;
|
||||
|
||||
virtual const DataTypes & getArgumentTypes() const = 0;
|
||||
virtual const DataTypePtr & getReturnType() const = 0;
|
||||
virtual const DataTypePtr & getResultType() const = 0;
|
||||
|
||||
/// Do preparations and return executable.
|
||||
/// sample_columns should contain data types of arguments and values of constants, if relevant.
|
||||
virtual ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName & sample_columns, const ColumnNumbers & arguments, size_t result) const = 0;
|
||||
virtual ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName & arguments) const = 0;
|
||||
|
||||
/// TODO: make const
|
||||
virtual void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count, bool dry_run = false)
|
||||
virtual ColumnPtr execute(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run = false)
|
||||
{
|
||||
return prepare(columns, arguments, result)->execute(columns, arguments, result, input_rows_count, dry_run);
|
||||
return prepare(arguments)->execute(arguments, result_type, input_rows_count, dry_run);
|
||||
}
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
@ -111,7 +111,7 @@ public:
|
||||
* There is no need to implement function if it has zero arguments.
|
||||
* Must return ColumnConst with single row or nullptr.
|
||||
*/
|
||||
virtual ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const ColumnsWithTypeAndName & /*columns*/, const ColumnNumbers & /*arguments*/) const { return nullptr; }
|
||||
virtual ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const ColumnsWithTypeAndName & /*columns*/) const { return nullptr; }
|
||||
|
||||
/** Function is called "injective" if it returns different result for different values of arguments.
|
||||
* Example: hex, negate, tuple...
|
||||
@ -226,9 +226,9 @@ public:
|
||||
using FunctionOverloadResolverPtr = std::shared_ptr<IFunctionOverloadResolver>;
|
||||
|
||||
|
||||
/** Return ColumnNullable of src, with null map as OR-ed null maps of args columns in columnss.
|
||||
/** Return ColumnNullable of src, with null map as OR-ed null maps of args columns.
|
||||
* 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 ColumnsWithTypeAndName & columns, const ColumnNumbers & args, size_t result, size_t input_rows_count);
|
||||
ColumnPtr wrapInNullable(const ColumnPtr & src, const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count);
|
||||
|
||||
}
|
||||
|
@ -14,7 +14,7 @@ public:
|
||||
|
||||
String getName() const final { return impl->getName(); }
|
||||
|
||||
void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count, bool dry_run) final;
|
||||
ColumnPtr execute(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) final;
|
||||
|
||||
void createLowCardinalityResultCache(size_t cache_size) override;
|
||||
|
||||
@ -24,14 +24,14 @@ private:
|
||||
/// Cache is created by function createLowCardinalityResultCache()
|
||||
ExecutableFunctionLowCardinalityResultCachePtr low_cardinality_result_cache;
|
||||
|
||||
bool defaultImplementationForConstantArguments(
|
||||
ColumnsWithTypeAndName & columns, const ColumnNumbers & args, size_t result, size_t input_rows_count, bool dry_run);
|
||||
ColumnPtr defaultImplementationForConstantArguments(
|
||||
ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run);
|
||||
|
||||
bool defaultImplementationForNulls(
|
||||
ColumnsWithTypeAndName & columns, const ColumnNumbers & args, size_t result, size_t input_rows_count, bool dry_run);
|
||||
ColumnPtr defaultImplementationForNulls(
|
||||
ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run);
|
||||
|
||||
void executeWithoutLowCardinalityColumns(
|
||||
ColumnsWithTypeAndName & columns, const ColumnNumbers & args, size_t result, size_t input_rows_count, bool dry_run);
|
||||
ColumnPtr executeWithoutLowCardinalityColumns(
|
||||
ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run);
|
||||
};
|
||||
|
||||
class FunctionBaseAdaptor final : public IFunctionBase
|
||||
@ -42,12 +42,11 @@ public:
|
||||
String getName() const final { return impl->getName(); }
|
||||
|
||||
const DataTypes & getArgumentTypes() const final { return impl->getArgumentTypes(); }
|
||||
const DataTypePtr & getReturnType() const final { return impl->getReturnType(); }
|
||||
const DataTypePtr & getResultType() const final { return impl->getResultType(); }
|
||||
|
||||
ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName & sample_columns, const ColumnNumbers & arguments, size_t result) const final
|
||||
ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName & arguments) const final
|
||||
{
|
||||
ColumnsWithTypeAndName columns(const_cast<ColumnsWithTypeAndName &>(sample_columns));
|
||||
return std::make_shared<ExecutableFunctionAdaptor>(impl->prepare(columns, arguments, result));
|
||||
return std::make_shared<ExecutableFunctionAdaptor>(impl->prepare(arguments));
|
||||
}
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
@ -148,13 +147,13 @@ public:
|
||||
String getName() const override { return function->getName(); }
|
||||
|
||||
protected:
|
||||
void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) final
|
||||
ColumnPtr execute(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) final
|
||||
{
|
||||
return function->executeImpl(columns, arguments, result, input_rows_count);
|
||||
return function->executeImpl(arguments, result_type, input_rows_count);
|
||||
}
|
||||
void executeDryRun(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) final
|
||||
ColumnPtr executeDryRun(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) final
|
||||
{
|
||||
return function->executeImplDryRun(columns, arguments, result, input_rows_count);
|
||||
return function->executeImplDryRun(arguments, result_type, input_rows_count);
|
||||
}
|
||||
bool useDefaultImplementationForNulls() const final { return function->useDefaultImplementationForNulls(); }
|
||||
bool useDefaultImplementationForConstants() const final { return function->useDefaultImplementationForConstants(); }
|
||||
@ -169,23 +168,23 @@ private:
|
||||
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_)) {}
|
||||
DefaultFunction(std::shared_ptr<IFunction> function_, DataTypes arguments_, DataTypePtr result_type_)
|
||||
: function(std::move(function_)), arguments(std::move(arguments_)), result_type(std::move(result_type_)) {}
|
||||
|
||||
String getName() const override { return function->getName(); }
|
||||
|
||||
const DataTypes & getArgumentTypes() const override { return arguments; }
|
||||
const DataTypePtr & getReturnType() const override { return return_type; }
|
||||
const DataTypePtr & getResultType() const override { return result_type; }
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
|
||||
bool isCompilable() const override { return function->isCompilable(arguments); }
|
||||
bool isCompilable() const override { return function->isCompilable(getArgumentTypes()); }
|
||||
|
||||
llvm::Value * compile(llvm::IRBuilderBase & builder, ValuePlaceholders values) const override { return function->compile(builder, arguments, std::move(values)); }
|
||||
llvm::Value * compile(llvm::IRBuilderBase & builder, ValuePlaceholders values) const override { return function->compile(builder, getArgumentTypes(), std::move(values)); }
|
||||
|
||||
#endif
|
||||
|
||||
ExecutableFunctionImplPtr prepare(const ColumnsWithTypeAndName & /*sample_columns*/, const ColumnNumbers & /*arguments*/, size_t /*result*/) const override
|
||||
ExecutableFunctionImplPtr prepare(const ColumnsWithTypeAndName & /*arguments*/) const override
|
||||
{
|
||||
return std::make_unique<DefaultExecutable>(function);
|
||||
}
|
||||
@ -214,7 +213,7 @@ public:
|
||||
private:
|
||||
std::shared_ptr<IFunction> function;
|
||||
DataTypes arguments;
|
||||
DataTypePtr return_type;
|
||||
DataTypePtr result_type;
|
||||
};
|
||||
|
||||
class DefaultOverloadResolver : public IFunctionOverloadResolverImpl
|
||||
@ -244,12 +243,12 @@ public:
|
||||
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
|
||||
FunctionBaseImplPtr build(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_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);
|
||||
return std::make_unique<DefaultFunction>(function, data_types, result_type);
|
||||
}
|
||||
|
||||
void getLambdaArgumentTypes(DataTypes & arguments) const override { function->getLambdaArgumentTypes(arguments); }
|
||||
|
@ -35,10 +35,10 @@ public:
|
||||
|
||||
virtual String getName() const = 0;
|
||||
|
||||
virtual void execute(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) = 0;
|
||||
virtual void executeDryRun(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
|
||||
virtual ColumnPtr execute(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) = 0;
|
||||
virtual ColumnPtr executeDryRun(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count)
|
||||
{
|
||||
execute(columns, arguments, result, input_rows_count);
|
||||
return execute(arguments, result_type, input_rows_count);
|
||||
}
|
||||
|
||||
/** Default implementation in presence of Nullable arguments or NULL constants as arguments is the following:
|
||||
@ -87,9 +87,9 @@ public:
|
||||
virtual String getName() const = 0;
|
||||
|
||||
virtual const DataTypes & getArgumentTypes() const = 0;
|
||||
virtual const DataTypePtr & getReturnType() const = 0;
|
||||
virtual const DataTypePtr & getResultType() const = 0;
|
||||
|
||||
virtual ExecutableFunctionImplPtr prepare(const ColumnsWithTypeAndName & sample_columns, const ColumnNumbers & arguments, size_t result) const = 0;
|
||||
virtual ExecutableFunctionImplPtr prepare(const ColumnsWithTypeAndName & arguments) const = 0;
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
|
||||
@ -130,7 +130,7 @@ public:
|
||||
|
||||
virtual String getName() const = 0;
|
||||
|
||||
virtual FunctionBaseImplPtr build(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const = 0;
|
||||
virtual FunctionBaseImplPtr build(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const = 0;
|
||||
|
||||
virtual DataTypePtr getReturnType(const DataTypes & /*arguments*/) const
|
||||
{
|
||||
@ -197,10 +197,10 @@ public:
|
||||
|
||||
virtual String getName() const = 0;
|
||||
|
||||
virtual void executeImpl(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const = 0;
|
||||
virtual void executeImplDryRun(ColumnsWithTypeAndName & columns, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
virtual ColumnPtr executeImpl(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const = 0;
|
||||
virtual ColumnPtr executeImplDryRun(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const
|
||||
{
|
||||
executeImpl(columns, arguments, result, input_rows_count);
|
||||
return executeImpl(arguments, result_type, input_rows_count);
|
||||
}
|
||||
|
||||
/** Default implementation in presence of Nullable arguments or NULL constants as arguments is the following:
|
||||
|
@ -174,11 +174,11 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings,
|
||||
bool all_const = true;
|
||||
bool all_suitable_for_constant_folding = true;
|
||||
|
||||
ColumnNumbers arguments(argument_names.size());
|
||||
ColumnsWithTypeAndName arguments(argument_names.size());
|
||||
for (size_t i = 0; i < argument_names.size(); ++i)
|
||||
{
|
||||
arguments[i] = sample_block.getPositionByName(argument_names[i]);
|
||||
ColumnPtr col = sample_block.safeGetByPosition(arguments[i]).column;
|
||||
arguments[i] = sample_block.getByName(argument_names[i]);
|
||||
ColumnPtr col = arguments[i].column;
|
||||
if (!col || !isColumnConst(*col))
|
||||
all_const = false;
|
||||
|
||||
@ -189,7 +189,7 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings,
|
||||
size_t result_position = sample_block.columns();
|
||||
sample_block.insert({nullptr, result_type, result_name});
|
||||
if (!function)
|
||||
function = function_base->prepare(sample_block.data, arguments, result_position);
|
||||
function = function_base->prepare(arguments);
|
||||
function->createLowCardinalityResultCache(settings.max_threads);
|
||||
|
||||
bool compile_expressions = false;
|
||||
@ -204,7 +204,7 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings,
|
||||
if (added_column)
|
||||
sample_block.getByPosition(result_position).column = added_column;
|
||||
else
|
||||
function->execute(sample_block.data, arguments, result_position, sample_block.rows(), true);
|
||||
sample_block.getByPosition(result_position).column = function->execute(arguments, result_type, sample_block.rows(), true);
|
||||
|
||||
/// If the result is not a constant, just in case, we will consider the result as unknown.
|
||||
ColumnWithTypeAndName & col = sample_block.safeGetByPosition(result_position);
|
||||
@ -232,7 +232,7 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings,
|
||||
auto & res = sample_block.getByPosition(result_position);
|
||||
if (!res.column && function_base->isSuitableForConstantFolding())
|
||||
{
|
||||
if (auto col = function_base->getResultIfAlwaysReturnsConstantAndHasArguments(sample_block.getColumnsWithTypeAndName(), arguments))
|
||||
if (auto col = function_base->getResultIfAlwaysReturnsConstantAndHasArguments(arguments))
|
||||
{
|
||||
res.column = std::move(col);
|
||||
names_not_for_constant_folding.insert(result_name);
|
||||
@ -345,9 +345,9 @@ void ExpressionAction::execute(Block & block, bool dry_run) const
|
||||
{
|
||||
case APPLY_FUNCTION:
|
||||
{
|
||||
ColumnNumbers arguments(argument_names.size());
|
||||
ColumnsWithTypeAndName arguments(argument_names.size());
|
||||
for (size_t i = 0; i < argument_names.size(); ++i)
|
||||
arguments[i] = block.getPositionByName(argument_names[i]);
|
||||
arguments[i] = block.getByName(argument_names[i]);
|
||||
|
||||
size_t num_columns_without_result = block.columns();
|
||||
block.insert({ nullptr, result_type, result_name});
|
||||
@ -355,7 +355,7 @@ void ExpressionAction::execute(Block & block, bool dry_run) const
|
||||
ProfileEvents::increment(ProfileEvents::FunctionExecute);
|
||||
if (is_function_compiled)
|
||||
ProfileEvents::increment(ProfileEvents::CompiledFunctionExecute);
|
||||
function->execute(block.data, arguments, num_columns_without_result, input_rows_count, dry_run);
|
||||
block.getByPosition(num_columns_without_result).column = function->execute(arguments, result_type, input_rows_count, dry_run);
|
||||
|
||||
break;
|
||||
}
|
||||
@ -595,7 +595,7 @@ void ExpressionActions::addImpl(ExpressionAction action, Names & new_names)
|
||||
if (!action.function_base)
|
||||
{
|
||||
action.function_base = action.function_builder->build(arguments);
|
||||
action.result_type = action.function_base->getReturnType();
|
||||
action.result_type = action.function_base->getResultType();
|
||||
}
|
||||
}
|
||||
|
||||
@ -1550,7 +1550,6 @@ const ActionsDAG::Node & ActionsDAG::addFunction(
|
||||
|
||||
bool all_const = true;
|
||||
ColumnsWithTypeAndName arguments(num_arguments);
|
||||
ColumnNumbers argument_numbers(num_arguments);
|
||||
|
||||
for (size_t i = 0; i < num_arguments; ++i)
|
||||
{
|
||||
@ -1566,15 +1565,11 @@ const ActionsDAG::Node & ActionsDAG::addFunction(
|
||||
all_const = false;
|
||||
|
||||
arguments[i] = std::move(argument);
|
||||
argument_numbers[i] = i;
|
||||
}
|
||||
|
||||
node.function_base = function->build(arguments);
|
||||
node.result_type = node.function_base->getReturnType();
|
||||
|
||||
Block sample_block(std::move(arguments));
|
||||
sample_block.insert({nullptr, node.result_type, node.result_name});
|
||||
node.function = node.function_base->prepare(sample_block.data, argument_numbers, num_arguments);
|
||||
node.result_type = node.function_base->getResultType();
|
||||
node.function = node.function_base->prepare(arguments);
|
||||
|
||||
bool do_compile_expressions = false;
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
@ -1585,20 +1580,20 @@ const ActionsDAG::Node & ActionsDAG::addFunction(
|
||||
/// so we don't want to unfold non deterministic functions
|
||||
if (all_const && node.function_base->isSuitableForConstantFolding() && (!do_compile_expressions || node.function_base->isDeterministic()))
|
||||
{
|
||||
node.function->execute(sample_block.data, argument_numbers, num_arguments, sample_block.rows(), true);
|
||||
size_t num_rows = arguments.empty() ? 0 : arguments.front().column->size();
|
||||
auto col = node.function->execute(arguments, node.result_type, num_rows, true);
|
||||
|
||||
/// If the result is not a constant, just in case, we will consider the result as unknown.
|
||||
ColumnWithTypeAndName & col = sample_block.safeGetByPosition(num_arguments);
|
||||
if (isColumnConst(*col.column))
|
||||
if (isColumnConst(*col))
|
||||
{
|
||||
/// All constant (literal) columns in block are added with size 1.
|
||||
/// But if there was no columns in block before executing a function, the result has size 0.
|
||||
/// Change the size to 1.
|
||||
|
||||
if (col.column->empty())
|
||||
col.column = col.column->cloneResized(1);
|
||||
if (col->empty())
|
||||
col = col->cloneResized(1);
|
||||
|
||||
node.column = std::move(col.column);
|
||||
node.column = std::move(col);
|
||||
}
|
||||
}
|
||||
|
||||
@ -1607,7 +1602,7 @@ const ActionsDAG::Node & ActionsDAG::addFunction(
|
||||
/// unnecessary materialization.
|
||||
if (!node.column && node.function_base->isSuitableForConstantFolding())
|
||||
{
|
||||
if (auto col = node.function_base->getResultIfAlwaysReturnsConstantAndHasArguments(sample_block.getColumnsWithTypeAndName(), argument_numbers))
|
||||
if (auto col = node.function_base->getResultIfAlwaysReturnsConstantAndHasArguments(arguments))
|
||||
{
|
||||
node.column = std::move(col);
|
||||
node.allow_constant_folding = false;
|
||||
|
Loading…
Reference in New Issue
Block a user