2017-07-21 06:35:58 +00:00
|
|
|
#include <Functions/FunctionHelpers.h>
|
2019-12-09 13:12:54 +00:00
|
|
|
#include <Functions/IFunctionImpl.h>
|
2017-07-21 06:35:58 +00:00
|
|
|
#include <Columns/ColumnTuple.h>
|
|
|
|
#include <Columns/ColumnString.h>
|
|
|
|
#include <Columns/ColumnFixedString.h>
|
2017-07-24 01:00:31 +00:00
|
|
|
#include <Columns/ColumnNullable.h>
|
2019-10-17 10:53:41 +00:00
|
|
|
#include <Columns/ColumnLowCardinality.h>
|
2019-08-21 02:28:04 +00:00
|
|
|
#include <Common/assert_cast.h>
|
2017-07-24 01:00:31 +00:00
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
2019-10-17 10:53:41 +00:00
|
|
|
#include <DataTypes/DataTypeLowCardinality.h>
|
2017-09-26 15:25:24 +00:00
|
|
|
#include <IO/WriteHelpers.h>
|
2017-07-21 06:35:58 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2018-04-29 22:43:02 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int ILLEGAL_COLUMN;
|
2019-04-09 19:19:30 +00:00
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
2019-11-04 05:29:54 +00:00
|
|
|
extern const int SIZES_OF_ARRAYS_DOESNT_MATCH;
|
2019-12-23 14:54:06 +00:00
|
|
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
2018-04-29 22:43:02 +00:00
|
|
|
}
|
|
|
|
|
2017-07-21 06:35:58 +00:00
|
|
|
const ColumnConst * checkAndGetColumnConstStringOrFixedString(const IColumn * column)
|
|
|
|
{
|
2019-06-27 19:28:52 +00:00
|
|
|
if (!isColumnConst(*column))
|
2017-07-21 06:35:58 +00:00
|
|
|
return {};
|
|
|
|
|
2019-08-21 02:28:04 +00:00
|
|
|
const ColumnConst * res = assert_cast<const ColumnConst *>(column);
|
2017-07-21 06:35:58 +00:00
|
|
|
|
|
|
|
if (checkColumn<ColumnString>(&res->getDataColumn())
|
|
|
|
|| checkColumn<ColumnFixedString>(&res->getDataColumn()))
|
|
|
|
return res;
|
|
|
|
|
|
|
|
return {};
|
|
|
|
}
|
|
|
|
|
2017-07-24 01:00:31 +00:00
|
|
|
|
2017-12-09 12:23:09 +00:00
|
|
|
Columns convertConstTupleToConstantElements(const ColumnConst & column)
|
2017-07-21 06:35:58 +00:00
|
|
|
{
|
2019-08-21 02:28:04 +00:00
|
|
|
const ColumnTuple & src_tuple = assert_cast<const ColumnTuple &>(column.getDataColumn());
|
2019-03-25 01:43:54 +00:00
|
|
|
const auto & src_tuple_columns = src_tuple.getColumns();
|
2017-12-08 00:50:25 +00:00
|
|
|
size_t tuple_size = src_tuple_columns.size();
|
|
|
|
size_t rows = column.size();
|
2017-07-21 06:35:58 +00:00
|
|
|
|
2017-12-09 12:23:09 +00:00
|
|
|
Columns res(tuple_size);
|
2017-12-08 00:50:25 +00:00
|
|
|
for (size_t i = 0; i < tuple_size; ++i)
|
2017-12-14 01:43:19 +00:00
|
|
|
res[i] = ColumnConst::create(src_tuple_columns[i], rows);
|
2017-07-21 06:35:58 +00:00
|
|
|
|
2017-12-09 12:23:09 +00:00
|
|
|
return res;
|
2017-07-21 06:35:58 +00:00
|
|
|
}
|
|
|
|
|
2017-07-24 01:00:31 +00:00
|
|
|
|
2018-01-19 02:15:10 +00:00
|
|
|
static Block createBlockWithNestedColumnsImpl(const Block & block, const std::unordered_set<size_t> & args)
|
2017-07-24 01:00:31 +00:00
|
|
|
{
|
|
|
|
Block res;
|
2017-12-09 13:12:54 +00:00
|
|
|
size_t columns = block.columns();
|
2017-07-24 01:00:31 +00:00
|
|
|
|
2017-12-09 12:23:09 +00:00
|
|
|
for (size_t i = 0; i < columns; ++i)
|
2017-07-24 01:00:31 +00:00
|
|
|
{
|
|
|
|
const auto & col = block.getByPosition(i);
|
|
|
|
|
2018-01-19 02:15:10 +00:00
|
|
|
if (args.count(i) && col.type->isNullable())
|
2017-07-24 01:00:31 +00:00
|
|
|
{
|
2018-01-19 02:15:10 +00:00
|
|
|
const DataTypePtr & nested_type = static_cast<const DataTypeNullable &>(*col.type).getNestedType();
|
2017-07-24 01:00:31 +00:00
|
|
|
|
2018-01-19 02:15:10 +00:00
|
|
|
if (!col.column)
|
2017-07-24 01:00:31 +00:00
|
|
|
{
|
2018-01-19 02:15:10 +00:00
|
|
|
res.insert({nullptr, nested_type, col.name});
|
2017-07-24 01:00:31 +00:00
|
|
|
}
|
2019-06-27 18:50:20 +00:00
|
|
|
else if (auto * nullable = checkAndGetColumn<ColumnNullable>(*col.column))
|
2018-01-19 02:15:10 +00:00
|
|
|
{
|
2019-06-26 17:20:33 +00:00
|
|
|
const auto & nested_col = nullable->getNestedColumnPtr();
|
2019-06-27 18:50:20 +00:00
|
|
|
res.insert({nested_col, nested_type, col.name});
|
|
|
|
}
|
|
|
|
else if (auto * const_column = checkAndGetColumn<ColumnConst>(*col.column))
|
|
|
|
{
|
|
|
|
const auto & nested_col = checkAndGetColumn<ColumnNullable>(const_column->getDataColumn())->getNestedColumnPtr();
|
|
|
|
res.insert({ ColumnConst::create(nested_col, col.column->size()), nested_type, col.name});
|
2018-01-19 02:15:10 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
throw Exception("Illegal column for DataTypeNullable", ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
res.insert(col);
|
2017-07-24 01:00:31 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-01-19 02:15:10 +00:00
|
|
|
Block createBlockWithNestedColumns(const Block & block, const ColumnNumbers & args)
|
|
|
|
{
|
|
|
|
std::unordered_set<size_t> args_set(args.begin(), args.end());
|
|
|
|
return createBlockWithNestedColumnsImpl(block, args_set);
|
|
|
|
}
|
|
|
|
|
|
|
|
Block createBlockWithNestedColumns(const Block & block, const ColumnNumbers & args, size_t result)
|
2017-07-24 01:00:31 +00:00
|
|
|
{
|
2018-01-19 02:15:10 +00:00
|
|
|
std::unordered_set<size_t> args_set(args.begin(), args.end());
|
|
|
|
args_set.insert(result);
|
|
|
|
return createBlockWithNestedColumnsImpl(block, args_set);
|
2017-07-24 01:00:31 +00:00
|
|
|
}
|
|
|
|
|
2019-04-09 19:19:30 +00:00
|
|
|
void validateArgumentType(const IFunction & func, const DataTypes & arguments,
|
2019-10-24 10:43:47 +00:00
|
|
|
size_t argument_index, bool (* validator_func)(const IDataType &),
|
|
|
|
const char * expected_type_description)
|
2019-04-09 19:19:30 +00:00
|
|
|
{
|
|
|
|
if (arguments.size() <= argument_index)
|
|
|
|
throw Exception("Incorrect number of arguments of function " + func.getName(),
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
const auto & argument = arguments[argument_index];
|
2020-03-08 21:40:00 +00:00
|
|
|
if (!validator_func(*argument))
|
2019-04-09 19:19:30 +00:00
|
|
|
throw Exception("Illegal type " + argument->getName() +
|
|
|
|
" of " + std::to_string(argument_index) +
|
|
|
|
" argument of function " + func.getName() +
|
|
|
|
" expected " + expected_type_description,
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
}
|
|
|
|
|
2019-10-14 10:15:10 +00:00
|
|
|
namespace
|
|
|
|
{
|
2019-10-24 10:43:47 +00:00
|
|
|
void validateArgumentsImpl(const IFunction & func,
|
|
|
|
const ColumnsWithTypeAndName & arguments,
|
|
|
|
size_t argument_offset,
|
2019-12-25 14:50:13 +00:00
|
|
|
const FunctionArgumentDescriptors & descriptors)
|
2019-10-14 10:15:10 +00:00
|
|
|
{
|
2019-12-25 14:50:13 +00:00
|
|
|
for (size_t i = 0; i < descriptors.size(); ++i)
|
2019-10-14 10:15:10 +00:00
|
|
|
{
|
|
|
|
const auto argument_index = i + argument_offset;
|
|
|
|
if (argument_index >= arguments.size())
|
|
|
|
{
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
|
|
|
const auto & arg = arguments[i + argument_offset];
|
2019-12-25 14:50:13 +00:00
|
|
|
const auto descriptor = descriptors[i];
|
2020-03-23 02:12:31 +00:00
|
|
|
if (int error_code = descriptor.isValid(arg.type, arg.column); error_code != 0)
|
2019-12-23 14:54:06 +00:00
|
|
|
throw Exception("Illegal type of argument #" + std::to_string(i)
|
2019-12-25 14:58:10 +00:00
|
|
|
+ (descriptor.argument_name ? " '" + std::string(descriptor.argument_name) + "'" : String{})
|
2019-12-23 14:54:06 +00:00
|
|
|
+ " of function " + func.getName()
|
2019-12-25 14:50:13 +00:00
|
|
|
+ (descriptor.expected_type_description ? String(", expected ") + descriptor.expected_type_description : String{})
|
2019-12-25 14:58:10 +00:00
|
|
|
+ (arg.type ? ", got " + arg.type->getName() : String{}),
|
2020-03-23 02:12:31 +00:00
|
|
|
error_code);
|
2019-10-14 10:15:10 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|
|
|
|
|
2019-12-25 14:50:13 +00:00
|
|
|
int FunctionArgumentDescriptor::isValid(const DataTypePtr & data_type, const ColumnPtr & column) const
|
2019-12-23 14:54:06 +00:00
|
|
|
{
|
2020-03-08 21:40:00 +00:00
|
|
|
if (type_validator_func && (data_type == nullptr || !type_validator_func(*data_type)))
|
2019-12-25 14:50:13 +00:00
|
|
|
return ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT;
|
2019-12-23 14:54:06 +00:00
|
|
|
|
2020-03-08 21:40:00 +00:00
|
|
|
if (column_validator_func && (column == nullptr || !column_validator_func(*column)))
|
2019-12-25 14:50:13 +00:00
|
|
|
return ErrorCodes::ILLEGAL_COLUMN;
|
2019-12-23 14:54:06 +00:00
|
|
|
|
2019-12-25 14:50:13 +00:00
|
|
|
return 0;
|
2019-12-23 14:54:06 +00:00
|
|
|
}
|
|
|
|
|
2019-10-24 10:43:47 +00:00
|
|
|
void validateFunctionArgumentTypes(const IFunction & func,
|
|
|
|
const ColumnsWithTypeAndName & arguments,
|
2019-12-23 14:54:06 +00:00
|
|
|
const FunctionArgumentDescriptors & mandatory_args,
|
|
|
|
const FunctionArgumentDescriptors & optional_args)
|
2019-10-14 10:15:10 +00:00
|
|
|
{
|
2019-12-23 14:54:06 +00:00
|
|
|
if (arguments.size() < mandatory_args.size() || arguments.size() > mandatory_args.size() + optional_args.size())
|
2019-10-24 10:43:47 +00:00
|
|
|
{
|
2020-03-23 02:12:31 +00:00
|
|
|
auto join_argument_types = [](const auto & args, const String sep = ", ")
|
2019-10-24 10:43:47 +00:00
|
|
|
{
|
|
|
|
String result;
|
|
|
|
for (const auto & a : args)
|
|
|
|
{
|
|
|
|
using A = std::decay_t<decltype(a)>;
|
2019-12-23 14:54:06 +00:00
|
|
|
if constexpr (std::is_same_v<A, FunctionArgumentDescriptor>)
|
|
|
|
{
|
|
|
|
if (a.argument_name)
|
|
|
|
result += "'" + std::string(a.argument_name) + "' : ";
|
2019-12-25 21:07:53 +00:00
|
|
|
if (a.expected_type_description)
|
|
|
|
result += a.expected_type_description;
|
2019-12-23 14:54:06 +00:00
|
|
|
}
|
2019-10-24 10:43:47 +00:00
|
|
|
else if constexpr (std::is_same_v<A, ColumnWithTypeAndName>)
|
|
|
|
result += a.type->getName();
|
|
|
|
|
|
|
|
result += sep;
|
|
|
|
}
|
|
|
|
|
2020-03-09 02:55:28 +00:00
|
|
|
if (!args.empty())
|
2019-10-24 10:43:47 +00:00
|
|
|
result.erase(result.end() - sep.length(), result.end());
|
|
|
|
|
|
|
|
return result;
|
|
|
|
};
|
|
|
|
|
2019-12-23 14:54:06 +00:00
|
|
|
throw Exception("Incorrect number of arguments for function " + func.getName()
|
|
|
|
+ " provided " + std::to_string(arguments.size())
|
2020-03-23 02:12:31 +00:00
|
|
|
+ (!arguments.empty() ? " (" + join_argument_types(arguments) + ")" : String{})
|
2019-12-23 14:54:06 +00:00
|
|
|
+ ", expected " + std::to_string(mandatory_args.size())
|
2020-03-08 21:40:00 +00:00
|
|
|
+ (!optional_args.empty() ? " to " + std::to_string(mandatory_args.size() + optional_args.size()) : "")
|
2020-03-23 02:12:31 +00:00
|
|
|
+ " (" + join_argument_types(mandatory_args)
|
|
|
|
+ (!optional_args.empty() ? ", [" + join_argument_types(optional_args) + "]" : "")
|
2019-12-23 14:54:06 +00:00
|
|
|
+ ")",
|
2019-10-14 10:15:10 +00:00
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
2019-10-24 10:43:47 +00:00
|
|
|
}
|
2019-10-14 10:15:10 +00:00
|
|
|
|
|
|
|
validateArgumentsImpl(func, arguments, 0, mandatory_args);
|
2020-03-08 21:40:00 +00:00
|
|
|
if (!optional_args.empty())
|
2019-10-14 10:15:10 +00:00
|
|
|
validateArgumentsImpl(func, arguments, mandatory_args.size(), optional_args);
|
|
|
|
}
|
|
|
|
|
2019-11-04 05:29:54 +00:00
|
|
|
std::pair<std::vector<const IColumn *>, const ColumnArray::Offset *>
|
|
|
|
checkAndGetNestedArrayOffset(const IColumn ** columns, size_t num_arguments)
|
|
|
|
{
|
|
|
|
assert(num_arguments > 0);
|
|
|
|
std::vector<const IColumn *> nested_columns(num_arguments);
|
|
|
|
const ColumnArray::Offsets * offsets = nullptr;
|
|
|
|
for (size_t i = 0; i < num_arguments; ++i)
|
|
|
|
{
|
|
|
|
const ColumnArray::Offsets * offsets_i = nullptr;
|
|
|
|
if (const ColumnArray * arr = checkAndGetColumn<const ColumnArray>(columns[i]))
|
|
|
|
{
|
|
|
|
nested_columns[i] = &arr->getData();
|
|
|
|
offsets_i = &arr->getOffsets();
|
|
|
|
}
|
|
|
|
else
|
|
|
|
throw Exception("Illegal column " + columns[i]->getName() + " as argument of function", ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
if (i == 0)
|
|
|
|
offsets = offsets_i;
|
|
|
|
else if (*offsets_i != *offsets)
|
|
|
|
throw Exception("Lengths of all arrays passed to aggregate function must be equal.", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH);
|
|
|
|
}
|
|
|
|
return {nested_columns, offsets->data()};
|
|
|
|
}
|
|
|
|
|
2017-07-21 06:35:58 +00:00
|
|
|
}
|