2017-04-01 09:19:00 +00:00
|
|
|
#include <Functions/IFunction.h>
|
2017-07-24 01:00:31 +00:00
|
|
|
#include <Functions/FunctionHelpers.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Columns/ColumnNullable.h>
|
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
2017-12-07 08:31:47 +00:00
|
|
|
#include <DataTypes/DataTypeNothing.h>
|
2017-07-23 08:40:43 +00:00
|
|
|
#include <Columns/ColumnConst.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/ExpressionActions.h>
|
2017-07-23 08:40:43 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
|
|
|
#include <ext/range.h>
|
2017-12-09 12:23:09 +00:00
|
|
|
#include <ext/collection_cast.h>
|
2017-07-23 08:40:43 +00:00
|
|
|
|
2016-07-06 09:47:55 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2017-06-13 02:06:53 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
|
|
|
}
|
|
|
|
|
2016-07-06 09:47:55 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
2017-12-09 12:23:09 +00:00
|
|
|
|
|
|
|
/** Return ColumnNullable of src, with null map as OR-ed null maps of args columns in blocks.
|
2017-12-09 13:12:54 +00:00
|
|
|
* Or ColumnConst(ColumnNullable) if the result is always NULL or if the result is constant and always not NULL.
|
2017-12-09 12:23:09 +00:00
|
|
|
*/
|
|
|
|
ColumnPtr wrapInNullable(const ColumnPtr & src, Block & block, const ColumnNumbers & args, size_t result)
|
2016-07-06 09:47:55 +00:00
|
|
|
{
|
2017-12-09 12:23:09 +00:00
|
|
|
ColumnPtr result_null_map_column;
|
2017-12-09 15:00:21 +00:00
|
|
|
bool shared_result_map_column = true;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
for (const auto & arg : args)
|
|
|
|
{
|
|
|
|
const ColumnWithTypeAndName & elem = block.getByPosition(arg);
|
2017-12-09 12:23:09 +00:00
|
|
|
if (!elem.type->isNullable())
|
|
|
|
continue;
|
|
|
|
|
|
|
|
if (elem.column->isColumnConst())
|
|
|
|
{
|
|
|
|
/// Const Nullable that are NULL.
|
|
|
|
if (static_cast<const ColumnNullable &>(static_cast<const ColumnConst &>(*elem.column).getDataColumn()).isNullAt(0))
|
|
|
|
{
|
|
|
|
return block.getByPosition(result).type->createConstColumn(block.rows(), Null());
|
|
|
|
}
|
|
|
|
else
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
else if (elem.column->isColumnNullable())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-12-09 12:23:09 +00:00
|
|
|
const ColumnPtr & null_map_column = static_cast<const ColumnNullable &>(*elem.column).getNullMapColumn();
|
|
|
|
if (!result_null_map_column)
|
|
|
|
result_null_map_column = null_map_column;
|
|
|
|
else
|
|
|
|
{
|
2017-12-09 15:00:21 +00:00
|
|
|
if (shared_result_map_column)
|
|
|
|
{
|
|
|
|
result_null_map_column = result_null_map_column->clone();
|
|
|
|
shared_result_map_column = false;
|
|
|
|
}
|
|
|
|
|
2017-12-09 12:23:09 +00:00
|
|
|
NullMap & result_null_map = static_cast<ColumnUInt8 &>(*result_null_map_column).getData();
|
|
|
|
const NullMap & src_null_map = static_cast<const ColumnUInt8 &>(*null_map_column).getData();
|
|
|
|
|
|
|
|
for (size_t i = 0, size = result_null_map.size(); i < size; ++i)
|
|
|
|
if (src_null_map[i])
|
|
|
|
result_null_map[i] = 1;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
2017-12-09 12:23:09 +00:00
|
|
|
|
2017-12-09 13:12:54 +00:00
|
|
|
if (src->isColumnConst() && !result_null_map_column)
|
|
|
|
{
|
|
|
|
return std::make_shared<ColumnConst>(std::make_shared<ColumnNullable>(
|
|
|
|
static_cast<const ColumnConst &>(*src).getDataColumnPtr(),
|
|
|
|
std::make_shared<ColumnUInt8>(1, 0)),
|
|
|
|
block.rows());
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
if (!result_null_map_column)
|
|
|
|
result_null_map_column = std::make_shared<ColumnUInt8>(block.rows(), 0);
|
2017-12-09 12:23:09 +00:00
|
|
|
|
2017-12-09 13:12:54 +00:00
|
|
|
if (src->isColumnConst())
|
|
|
|
return std::make_shared<ColumnNullable>(src->convertToFullColumnIfConst(), result_null_map_column);
|
|
|
|
else
|
|
|
|
return std::make_shared<ColumnNullable>(src, result_null_map_column);
|
|
|
|
}
|
2016-07-06 09:47:55 +00:00
|
|
|
}
|
|
|
|
|
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;
|
2016-08-15 11:14:29 +00:00
|
|
|
};
|
|
|
|
|
2017-07-24 01:00:31 +00:00
|
|
|
NullPresense getNullPresense(const Block & block, const ColumnNumbers & args)
|
2016-07-06 09:47:55 +00:00
|
|
|
{
|
2017-07-24 01:00:31 +00:00
|
|
|
NullPresense res;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
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-12-09 06:32:22 +00:00
|
|
|
res.has_null_constant = elem.type->onlyNull();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2017-07-24 01:00:31 +00:00
|
|
|
return res;
|
2016-07-06 09:47:55 +00:00
|
|
|
}
|
|
|
|
|
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-04-01 07:20:54 +00:00
|
|
|
|
2017-07-24 01:00:31 +00:00
|
|
|
for (const auto & elem : args)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
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-12-09 06:32:22 +00:00
|
|
|
res.has_null_constant = elem.type->onlyNull();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2017-07-24 01:00:31 +00:00
|
|
|
return res;
|
2016-07-06 09:47:55 +00:00
|
|
|
}
|
|
|
|
|
2017-07-24 01:00:31 +00:00
|
|
|
NullPresense getNullPresense(const DataTypes & types)
|
2016-07-06 09:47:55 +00:00
|
|
|
{
|
2017-07-24 01:00:31 +00:00
|
|
|
NullPresense res;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-07-24 01:00:31 +00:00
|
|
|
for (const auto & type : types)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
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-12-09 06:32:22 +00:00
|
|
|
res.has_null_constant = type->onlyNull();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2017-07-24 01:00:31 +00:00
|
|
|
return res;
|
2016-07-06 09:47:55 +00:00
|
|
|
}
|
|
|
|
|
2016-08-15 11:14:29 +00:00
|
|
|
/// Turn the specified set of data types into their respective nested data types.
|
|
|
|
DataTypes toNestedDataTypes(const DataTypes & args)
|
2016-07-06 09:47:55 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
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());
|
2017-09-08 04:58:57 +00:00
|
|
|
const DataTypePtr & nested_type = nullable_type->getNestedType();
|
2017-04-01 07:20:54 +00:00
|
|
|
new_args.push_back(nested_type);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
new_args.push_back(arg);
|
|
|
|
}
|
|
|
|
|
|
|
|
return new_args;
|
2016-07-06 09:47:55 +00:00
|
|
|
}
|
|
|
|
|
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)
|
2016-07-06 09:47:55 +00:00
|
|
|
{
|
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();
|
2016-07-06 09:47:55 +00:00
|
|
|
|
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-04-01 07:20:54 +00:00
|
|
|
}
|
2016-08-15 11:14:29 +00:00
|
|
|
|
2017-07-24 01:00:31 +00:00
|
|
|
temporary_block.insert(block.getByPosition(result));
|
2016-07-06 09:47:55 +00:00
|
|
|
|
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-04-01 07:20:54 +00:00
|
|
|
|
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-07-06 09:47:55 +00:00
|
|
|
}
|
|
|
|
|
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)
|
2016-07-06 09:47:55 +00:00
|
|
|
{
|
2017-07-24 01:00:31 +00:00
|
|
|
if (args.empty() || !func.useDefaultImplementationForNulls())
|
|
|
|
return false;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-07-24 01:00:31 +00:00
|
|
|
NullPresense null_presense = getNullPresense(block, args);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-07-24 01:00:31 +00:00
|
|
|
if (null_presense.has_null_constant)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-07-24 01:00:31 +00:00
|
|
|
block.getByPosition(result).column = block.getByPosition(result).type->createConstColumn(block.rows(), Null());
|
|
|
|
return true;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2017-07-24 01:00:31 +00:00
|
|
|
if (null_presense.has_nullable)
|
2017-07-23 08:40:43 +00:00
|
|
|
{
|
2017-07-24 01:00:31 +00:00
|
|
|
Block temporary_block = createBlockWithNestedColumns(block, args, result);
|
|
|
|
func.execute(temporary_block, args, result);
|
2017-12-09 12:23:09 +00:00
|
|
|
block.getByPosition(result).column = wrapInNullable(temporary_block.getByPosition(result).column, block, args, result);
|
2017-07-24 01:00:31 +00:00
|
|
|
return true;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2016-08-10 19:12:29 +00:00
|
|
|
|
2017-07-24 01:00:31 +00:00
|
|
|
return false;
|
2016-08-15 11:14:29 +00:00
|
|
|
}
|
2016-08-10 19:12:29 +00:00
|
|
|
|
|
|
|
}
|
|
|
|
|
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
|
2016-07-06 09:47:55 +00:00
|
|
|
{
|
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();
|
2016-08-15 11:14:29 +00:00
|
|
|
|
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-08-15 11:14:29 +00:00
|
|
|
}
|
2016-07-06 09:47:55 +00:00
|
|
|
|
2016-12-29 19:38:10 +00:00
|
|
|
|
2017-07-24 01:00:31 +00:00
|
|
|
DataTypePtr IFunction::getReturnType(const DataTypes & arguments) const
|
2016-08-15 11:14:29 +00:00
|
|
|
{
|
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-04-01 07:20:54 +00:00
|
|
|
{
|
2017-07-24 01:00:31 +00:00
|
|
|
NullPresense null_presense = getNullPresense(arguments);
|
|
|
|
if (null_presense.has_null_constant)
|
2017-12-09 13:12:54 +00:00
|
|
|
{
|
2017-12-07 08:31:47 +00:00
|
|
|
return std::make_shared<DataTypeNullable>(std::make_shared<DataTypeNothing>());
|
2017-12-09 13:12:54 +00:00
|
|
|
}
|
2017-07-24 01:00:31 +00:00
|
|
|
if (null_presense.has_nullable)
|
2017-12-09 13:12:54 +00:00
|
|
|
{
|
2017-07-24 01:00:31 +00:00
|
|
|
return std::make_shared<DataTypeNullable>(getReturnTypeImpl(toNestedDataTypes(arguments)));
|
2017-12-09 13:12:54 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2017-07-24 01:00:31 +00:00
|
|
|
return getReturnTypeImpl(arguments);
|
2016-07-06 09:47:55 +00:00
|
|
|
}
|
|
|
|
|
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)
|
2016-07-06 09:47:55 +00:00
|
|
|
{
|
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-04-01 07:20:54 +00:00
|
|
|
{
|
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-04-01 07:20:54 +00:00
|
|
|
{
|
2017-12-07 08:31:47 +00:00
|
|
|
out_return_type = std::make_shared<DataTypeNullable>(std::make_shared<DataTypeNothing>());
|
2017-07-24 01:00:31 +00:00
|
|
|
return;
|
|
|
|
}
|
|
|
|
if (null_presense.has_nullable)
|
|
|
|
{
|
2017-12-09 12:23:09 +00:00
|
|
|
Block nested_block = createBlockWithNestedColumns(Block(arguments), ext::collection_cast<ColumnNumbers>(ext::range(0, arguments.size())));
|
|
|
|
getReturnTypeAndPrerequisitesImpl(ColumnsWithTypeAndName(nested_block.begin(), nested_block.end()), out_return_type, out_prerequisites);
|
2017-07-24 01:00:31 +00:00
|
|
|
out_return_type = std::make_shared<DataTypeNullable>(out_return_type);
|
|
|
|
return;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
2016-08-05 11:31:55 +00:00
|
|
|
|
2017-07-24 01:00:31 +00:00
|
|
|
getReturnTypeAndPrerequisitesImpl(arguments, out_return_type, out_prerequisites);
|
2016-07-06 09:47:55 +00:00
|
|
|
}
|
|
|
|
|
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-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
|
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-04-01 07:20:54 +00:00
|
|
|
|
2017-07-24 01:00:31 +00:00
|
|
|
if (defaultImplementationForNulls(*this, block, args, result))
|
|
|
|
return;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-07-24 01:00:31 +00:00
|
|
|
executeImpl(block, args, result);
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
|
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-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2017-07-24 01:00:31 +00:00
|
|
|
execute(block, args, result);
|
2016-10-21 13:14:41 +00:00
|
|
|
}
|
|
|
|
|
2016-07-06 09:47:55 +00:00
|
|
|
}
|
|
|
|
|