ClickHouse/dbms/src/Functions/IFunction.cpp

314 lines
9.2 KiB
C++
Raw Normal View History

#include <Functions/IFunction.h>
2017-07-24 01:00:31 +00:00
#include <Functions/FunctionHelpers.h>
#include <Columns/ColumnNullable.h>
#include <DataTypes/DataTypeNull.h>
#include <DataTypes/DataTypeNullable.h>
#include <Columns/ColumnConst.h>
#include <Interpreters/ExpressionActions.h>
#include <Common/typeid_cast.h>
#include <ext/range.h>
namespace DB
{
2017-06-13 02:06:53 +00:00
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
namespace
{
/// Suppose a function which has no special support for nullable arguments
/// has been called with arguments, one or more of them being nullable.
/// Then the method below endows the result, which is nullable, with a null
/// byte map that is determined by OR-ing the null byte maps of the nullable
/// arguments.
2017-03-29 11:33:07 +00:00
void createNullMap(Block & block, const ColumnNumbers & args, size_t result)
{
ColumnNullable & res_col = static_cast<ColumnNullable &>(*block.getByPosition(result).column);
for (const auto & arg : args)
{
const ColumnWithTypeAndName & elem = block.getByPosition(arg);
if (elem.column->isNullable())
{
const ColumnNullable & nullable_col = static_cast<const ColumnNullable &>(*elem.column);
res_col.applyNullMap(nullable_col); /// TODO excessive copy in case of single nullable argument.
}
}
}
2017-07-24 01:00:31 +00:00
struct NullPresense
2016-08-05 07:49:56 +00:00
{
2017-07-24 01:00:31 +00:00
bool has_nullable = false;
bool has_null_constant = false;
};
2017-07-24 01:00:31 +00:00
NullPresense getNullPresense(const Block & block, const ColumnNumbers & args)
{
2017-07-24 01:00:31 +00:00
NullPresense res;
for (const auto & arg : args)
{
const auto & elem = block.getByPosition(arg);
2017-07-24 01:00:31 +00:00
if (!res.has_nullable)
res.has_nullable = elem.type->isNullable();
2017-07-24 22:07:39 +00:00
if (!res.has_null_constant)
2017-07-24 01:00:31 +00:00
res.has_null_constant = elem.type->isNull();
}
2017-07-24 01:00:31 +00:00
return res;
}
2017-07-24 01:00:31 +00:00
NullPresense getNullPresense(const ColumnsWithTypeAndName & args)
2016-08-05 07:49:56 +00:00
{
2017-07-24 01:00:31 +00:00
NullPresense res;
2017-07-24 01:00:31 +00:00
for (const auto & elem : args)
{
2017-07-24 01:00:31 +00:00
if (!res.has_nullable)
res.has_nullable = elem.type->isNullable();
2017-07-24 22:07:39 +00:00
if (!res.has_null_constant)
2017-07-24 01:00:31 +00:00
res.has_null_constant = elem.type->isNull();
}
2017-07-24 01:00:31 +00:00
return res;
}
2017-07-24 01:00:31 +00:00
NullPresense getNullPresense(const DataTypes & types)
{
2017-07-24 01:00:31 +00:00
NullPresense res;
2017-07-24 01:00:31 +00:00
for (const auto & type : types)
{
2017-07-24 01:00:31 +00:00
if (!res.has_nullable)
res.has_nullable = type->isNullable();
2017-07-24 22:07:39 +00:00
if (!res.has_null_constant)
2017-07-24 01:00:31 +00:00
res.has_null_constant = type->isNull();
}
2017-07-24 01:00:31 +00:00
return res;
}
/// Turn the specified set of columns into their respective nested columns.
ColumnsWithTypeAndName toNestedColumns(const ColumnsWithTypeAndName & args)
{
ColumnsWithTypeAndName new_args;
new_args.reserve(args.size());
for (const auto & arg : args)
{
if (arg.type->isNullable())
{
auto nullable_col = static_cast<const ColumnNullable *>(arg.column.get());
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
ColumnPtr nested_col = (nullable_col) ? nullable_col->getNestedColumn() : nullptr;
auto nullable_type = static_cast<const DataTypeNullable *>(arg.type.get());
const DataTypePtr & nested_type = nullable_type->getNestedType();
new_args.emplace_back(nested_col, nested_type, arg.name);
}
else
new_args.emplace_back(arg.column, arg.type, arg.name);
}
return new_args;
}
/// Turn the specified set of data types into their respective nested data types.
DataTypes toNestedDataTypes(const DataTypes & args)
{
DataTypes new_args;
new_args.reserve(args.size());
for (const auto & arg : args)
{
if (arg->isNullable())
{
auto nullable_type = static_cast<const DataTypeNullable *>(arg.get());
const DataTypePtr & nested_type = nullable_type->getNestedType();
new_args.push_back(nested_type);
}
else
new_args.push_back(arg);
}
return new_args;
}
2016-12-29 19:38:10 +00:00
2017-07-24 01:00:31 +00:00
bool allArgumentsAreConstants(const Block & block, const ColumnNumbers & args)
2016-12-29 19:38:10 +00:00
{
2017-07-24 01:00:31 +00:00
for (auto arg : args)
if (!typeid_cast<const ColumnConst *>(block.getByPosition(arg).column.get()))
return false;
return true;
2016-12-29 19:38:10 +00:00
}
2017-07-24 01:00:31 +00:00
bool defaultImplementationForConstantArguments(
IFunction & func, Block & block, const ColumnNumbers & args, size_t result)
{
2017-07-24 01:00:31 +00:00
if (args.empty() || !func.useDefaultImplementationForConstants() || !allArgumentsAreConstants(block, args))
return false;
2016-12-29 19:38:10 +00:00
2017-07-24 01:00:31 +00:00
ColumnNumbers arguments_to_remain_constants = func.getArgumentsThatAreAlwaysConstant();
2017-07-24 01:00:31 +00:00
Block temporary_block;
2016-12-29 19:38:10 +00:00
2017-07-24 01:00:31 +00:00
size_t arguments_size = args.size();
for (size_t arg_num = 0; arg_num < arguments_size; ++arg_num)
{
const ColumnWithTypeAndName & column = block.getByPosition(args[arg_num]);
2016-12-29 19:38:10 +00:00
2017-07-24 22:41:35 +00:00
if (arguments_to_remain_constants.end() != std::find(arguments_to_remain_constants.begin(), arguments_to_remain_constants.end(), arg_num))
2017-07-24 01:00:31 +00:00
temporary_block.insert(column);
else
temporary_block.insert({ static_cast<const ColumnConst *>(column.column.get())->getDataColumnPtr(), column.type, column.name });
}
2017-07-24 01:00:31 +00:00
temporary_block.insert(block.getByPosition(result));
2017-07-24 01:00:31 +00:00
ColumnNumbers temporary_argument_numbers(arguments_size);
for (size_t i = 0; i < arguments_size; ++i)
temporary_argument_numbers[i] = i;
2016-12-29 19:38:10 +00:00
2017-07-24 01:00:31 +00:00
func.execute(temporary_block, temporary_argument_numbers, arguments_size);
2017-07-24 01:00:31 +00:00
block.getByPosition(result).column = std::make_shared<ColumnConst>(temporary_block.getByPosition(arguments_size).column, block.rows());
return true;
}
2016-12-29 19:38:10 +00:00
2017-07-24 01:00:31 +00:00
bool defaultImplementationForNulls(
IFunction & func, Block & block, const ColumnNumbers & args, size_t result)
{
2017-07-24 01:00:31 +00:00
if (args.empty() || !func.useDefaultImplementationForNulls())
return false;
2017-07-24 01:00:31 +00:00
NullPresense null_presense = getNullPresense(block, args);
2017-07-24 01:00:31 +00:00
if (null_presense.has_null_constant)
{
2017-07-24 01:00:31 +00:00
block.getByPosition(result).column = block.getByPosition(result).type->createConstColumn(block.rows(), Null());
return true;
}
2017-07-24 01:00:31 +00:00
if (null_presense.has_nullable)
{
2017-07-24 01:00:31 +00:00
Block temporary_block = createBlockWithNestedColumns(block, args, result);
func.execute(temporary_block, args, result);
2017-07-24 01:00:31 +00:00
const ColumnWithTypeAndName & source_col = temporary_block.getByPosition(result);
ColumnWithTypeAndName & dest_col = block.getByPosition(result);
/// Initialize the result column.
ColumnPtr null_map = std::make_shared<ColumnUInt8>(block.rows(), 0);
dest_col.column = std::make_shared<ColumnNullable>(source_col.column, null_map);
/// Deduce the null map of the result from the null maps of the nullable columns.
createNullMap(block, args, result);
2017-07-24 01:00:31 +00:00
return true;
}
2017-07-24 01:00:31 +00:00
return false;
}
}
2016-12-29 19:38:10 +00:00
2017-07-24 01:00:31 +00:00
void IFunction::checkNumberOfArguments(size_t number_of_arguments) const
{
2017-07-24 01:00:31 +00:00
if (isVariadic())
return;
2016-12-29 19:38:10 +00:00
2017-07-24 01:00:31 +00:00
size_t expected_number_of_arguments = getNumberOfArguments();
2017-07-24 01:00:31 +00:00
if (number_of_arguments != expected_number_of_arguments)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(number_of_arguments) + ", should be " + toString(expected_number_of_arguments),
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
2016-12-29 19:38:10 +00:00
2017-07-24 01:00:31 +00:00
DataTypePtr IFunction::getReturnType(const DataTypes & arguments) const
{
2017-07-24 01:00:31 +00:00
checkNumberOfArguments(arguments.size());
2016-12-29 19:38:10 +00:00
2017-07-24 01:00:31 +00:00
if (!arguments.empty() && useDefaultImplementationForNulls())
{
2017-07-24 01:00:31 +00:00
NullPresense null_presense = getNullPresense(arguments);
if (null_presense.has_null_constant)
return std::make_shared<DataTypeNull>();
if (null_presense.has_nullable)
return std::make_shared<DataTypeNullable>(getReturnTypeImpl(toNestedDataTypes(arguments)));
}
2017-07-24 01:00:31 +00:00
return getReturnTypeImpl(arguments);
}
2016-12-29 19:38:10 +00:00
2017-07-24 01:00:31 +00:00
void IFunction::getReturnTypeAndPrerequisites(
const ColumnsWithTypeAndName & arguments,
DataTypePtr & out_return_type,
std::vector<ExpressionAction> & out_prerequisites)
{
2017-07-24 01:00:31 +00:00
checkNumberOfArguments(arguments.size());
2016-08-05 11:31:55 +00:00
2017-07-24 01:00:31 +00:00
if (!arguments.empty() && useDefaultImplementationForNulls())
{
2017-07-24 01:00:31 +00:00
NullPresense null_presense = getNullPresense(arguments);
2017-07-24 22:07:39 +00:00
2017-07-24 01:00:31 +00:00
if (null_presense.has_null_constant)
{
2017-07-24 01:00:31 +00:00
out_return_type = std::make_shared<DataTypeNull>();
return;
}
if (null_presense.has_nullable)
{
getReturnTypeAndPrerequisitesImpl(toNestedColumns(arguments), out_return_type, out_prerequisites);
out_return_type = std::make_shared<DataTypeNullable>(out_return_type);
return;
}
}
2016-08-05 11:31:55 +00:00
2017-07-24 01:00:31 +00:00
getReturnTypeAndPrerequisitesImpl(arguments, out_return_type, out_prerequisites);
}
2016-12-29 19:38:10 +00:00
2017-07-24 01:00:31 +00:00
void IFunction::getLambdaArgumentTypes(DataTypes & arguments) const
2016-10-21 13:14:41 +00:00
{
2017-07-24 01:00:31 +00:00
checkNumberOfArguments(arguments.size());
getLambdaArgumentTypesImpl(arguments);
}
2017-07-24 01:00:31 +00:00
void IFunction::execute(Block & block, const ColumnNumbers & args, size_t result)
{
if (defaultImplementationForConstantArguments(*this, block, args, result))
return;
2017-07-24 01:00:31 +00:00
if (defaultImplementationForNulls(*this, block, args, result))
return;
2017-07-24 01:00:31 +00:00
executeImpl(block, args, result);
}
2017-07-24 01:00:31 +00:00
void IFunction::execute(Block & block, const ColumnNumbers & args, const ColumnNumbers & prerequisites, size_t result)
{
if (!prerequisites.empty())
{
executeImpl(block, args, prerequisites, result);
return;
}
2017-07-24 01:00:31 +00:00
execute(block, args, result);
2016-10-21 13:14:41 +00:00
}
}