2017-07-21 06:35:58 +00:00
|
|
|
#include <Functions/FunctionHelpers.h>
|
2021-05-17 07:30:42 +00:00
|
|
|
#include <Functions/IFunction.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>
|
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
|
|
|
}
|
|
|
|
|
2021-07-12 09:18:04 +00:00
|
|
|
ColumnWithTypeAndName columnGetNested(const ColumnWithTypeAndName & col)
|
2017-07-24 01:00:31 +00:00
|
|
|
{
|
2021-07-12 09:18:04 +00:00
|
|
|
if (col.type->isNullable())
|
2017-07-24 01:00:31 +00:00
|
|
|
{
|
2021-07-12 09:18:04 +00:00
|
|
|
const DataTypePtr & nested_type = static_cast<const DataTypeNullable &>(*col.type).getNestedType();
|
|
|
|
|
|
|
|
if (!col.column)
|
2017-07-24 01:00:31 +00:00
|
|
|
{
|
2021-07-12 09:18:04 +00:00
|
|
|
return ColumnWithTypeAndName{nullptr, nested_type, col.name};
|
|
|
|
}
|
|
|
|
else if (const auto * nullable = checkAndGetColumn<ColumnNullable>(*col.column))
|
|
|
|
{
|
|
|
|
const auto & nested_col = nullable->getNestedColumnPtr();
|
|
|
|
return ColumnWithTypeAndName{nested_col, nested_type, col.name};
|
|
|
|
}
|
|
|
|
else if (const auto * const_column = checkAndGetColumn<ColumnConst>(*col.column))
|
|
|
|
{
|
|
|
|
const auto * nullable_column = checkAndGetColumn<ColumnNullable>(const_column->getDataColumn());
|
2017-07-24 01:00:31 +00:00
|
|
|
|
2021-07-12 09:18:04 +00:00
|
|
|
ColumnPtr nullable_res;
|
|
|
|
if (nullable_column)
|
2017-07-24 01:00:31 +00:00
|
|
|
{
|
2021-07-12 09:18:04 +00:00
|
|
|
const auto & nested_col = nullable_column->getNestedColumnPtr();
|
|
|
|
nullable_res = ColumnConst::create(nested_col, col.column->size());
|
2017-07-24 01:00:31 +00:00
|
|
|
}
|
2021-07-12 09:18:04 +00:00
|
|
|
else
|
2019-06-27 18:50:20 +00:00
|
|
|
{
|
2021-07-12 09:18:04 +00:00
|
|
|
nullable_res = makeNullable(col.column);
|
2018-01-19 02:15:10 +00:00
|
|
|
}
|
2021-07-12 09:18:04 +00:00
|
|
|
return ColumnWithTypeAndName{ nullable_res, nested_type, col.name };
|
2018-01-19 02:15:10 +00:00
|
|
|
}
|
|
|
|
else
|
2021-07-12 09:18:04 +00:00
|
|
|
throw Exception("Illegal column for DataTypeNullable", ErrorCodes::ILLEGAL_COLUMN);
|
2017-07-24 01:00:31 +00:00
|
|
|
}
|
2021-07-12 09:18:04 +00:00
|
|
|
return col;
|
|
|
|
}
|
|
|
|
|
|
|
|
ColumnsWithTypeAndName createBlockWithNestedColumns(const ColumnsWithTypeAndName & columns)
|
|
|
|
{
|
|
|
|
ColumnsWithTypeAndName res;
|
|
|
|
for (const auto & col : columns)
|
|
|
|
res.emplace_back(columnGetNested(col));
|
2017-07-24 01:00:31 +00:00
|
|
|
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
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)
|
encrypt, aes_encrypt_mysql, decrypt, aes_decrypt_mysql functions
Functions to encrypt/decrypt any input data with OpenSSL's ciphers
with custom key, iv, and add (-gcm mode only).
_mysql versions are 100% compatitable with corresponding MySQL functions
Supported modes depend on OpenSSL version, but generally are:
aes-{128,192,56}-{ecb,cbc,cfb1,cfb8,cfb128,ofb,gcm}
Please note that in a -gcm mode a 16-byte tag is appended to the ciphertext
on encryption and is expected to be found at the end of ciphertext on decryption.
Added tests that verify compatibility with MySQL functions,
and test vectors for GCM mode from OpenSSL.
Added masking rules for aes_X funtions
Rules are installed by default to config.d/query_masking_rules.xml
2020-06-16 09:22:55 +00:00
|
|
|
throw Exception("Illegal type of argument #" + std::to_string(argument_offset + i + 1) // +1 is for human-friendly 1-based indexing
|
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()};
|
|
|
|
}
|
|
|
|
|
2021-01-21 14:42:50 +00:00
|
|
|
bool areTypesEqual(const DataTypePtr & lhs, const DataTypePtr & rhs)
|
|
|
|
{
|
|
|
|
const auto & lhs_name = lhs->getName();
|
|
|
|
const auto & rhs_name = rhs->getName();
|
|
|
|
|
|
|
|
return lhs_name == rhs_name;
|
|
|
|
}
|
|
|
|
|
2021-05-15 17:33:15 +00:00
|
|
|
ColumnPtr wrapInNullable(const ColumnPtr & src, const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count)
|
|
|
|
{
|
|
|
|
ColumnPtr result_null_map_column;
|
|
|
|
|
|
|
|
/// If result is already nullable.
|
|
|
|
ColumnPtr src_not_nullable = src;
|
|
|
|
|
|
|
|
if (src->onlyNull())
|
|
|
|
return src;
|
|
|
|
else if (const auto * nullable = checkAndGetColumn<ColumnNullable>(*src))
|
|
|
|
{
|
|
|
|
src_not_nullable = nullable->getNestedColumnPtr();
|
|
|
|
result_null_map_column = nullable->getNullMapColumnPtr();
|
|
|
|
}
|
|
|
|
|
|
|
|
for (const auto & elem : args)
|
|
|
|
{
|
|
|
|
if (!elem.type->isNullable())
|
|
|
|
continue;
|
|
|
|
|
|
|
|
/// Const Nullable that are NULL.
|
|
|
|
if (elem.column->onlyNull())
|
|
|
|
{
|
|
|
|
assert(result_type->isNullable());
|
|
|
|
return result_type->createColumnConstWithDefaultValue(input_rows_count);
|
|
|
|
}
|
|
|
|
|
|
|
|
if (isColumnConst(*elem.column))
|
|
|
|
continue;
|
|
|
|
|
|
|
|
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(*elem.column))
|
|
|
|
{
|
|
|
|
const ColumnPtr & null_map_column = nullable->getNullMapColumnPtr();
|
|
|
|
if (!result_null_map_column) //-V1051
|
|
|
|
{
|
|
|
|
result_null_map_column = null_map_column;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
MutableColumnPtr mutable_result_null_map_column = IColumn::mutate(std::move(result_null_map_column));
|
|
|
|
|
|
|
|
NullMap & result_null_map = assert_cast<ColumnUInt8 &>(*mutable_result_null_map_column).getData();
|
|
|
|
const NullMap & src_null_map = assert_cast<const ColumnUInt8 &>(*null_map_column).getData();
|
|
|
|
|
|
|
|
for (size_t i = 0, size = result_null_map.size(); i < size; ++i)
|
|
|
|
result_null_map[i] |= src_null_map[i];
|
|
|
|
|
|
|
|
result_null_map_column = std::move(mutable_result_null_map_column);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!result_null_map_column)
|
|
|
|
return makeNullable(src);
|
|
|
|
|
|
|
|
return ColumnNullable::create(src_not_nullable->convertToFullColumnIfConst(), result_null_map_column);
|
|
|
|
}
|
|
|
|
|
|
|
|
NullPresence getNullPresense(const ColumnsWithTypeAndName & args)
|
|
|
|
{
|
|
|
|
NullPresence res;
|
|
|
|
|
|
|
|
for (const auto & elem : args)
|
|
|
|
{
|
|
|
|
res.has_nullable |= elem.type->isNullable();
|
|
|
|
res.has_null_constant |= elem.type->onlyNull();
|
|
|
|
}
|
|
|
|
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
2017-07-21 06:35:58 +00:00
|
|
|
}
|