2021-04-02 11:31:33 +00:00
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
|
|
|
#include <Functions/FunctionFactory.h>
|
|
|
|
#include <Functions/FunctionHelpers.h>
|
|
|
|
#include <Columns/ColumnsNumber.h>
|
|
|
|
#include <Columns/ColumnArray.h>
|
|
|
|
#include <Core/ColumnWithTypeAndName.h>
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
|
|
|
extern const int SIZES_OF_ARRAYS_DOESNT_MATCH;
|
|
|
|
}
|
|
|
|
|
2021-04-13 09:31:49 +00:00
|
|
|
/** Function validateNestedArraySizes is used to check the consistency of Nested DataType subcolumns's offsets when Update
|
|
|
|
* Arguments: num > 2
|
|
|
|
* The first argument is the condition of WHERE in UPDATE operation, only when this is true, we need to check
|
|
|
|
* The rest arguments are the subcolumns of Nested DataType.
|
|
|
|
*/
|
2021-04-02 11:31:33 +00:00
|
|
|
class FunctionValidateNestedArraySizes : public IFunction
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
static constexpr auto name = "validateNestedArraySizes";
|
2021-04-13 09:31:49 +00:00
|
|
|
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionValidateNestedArraySizes>(); }
|
2021-04-02 11:31:33 +00:00
|
|
|
|
|
|
|
String getName() const override { return name; }
|
|
|
|
bool isVariadic() const override { return true; }
|
|
|
|
size_t getNumberOfArguments() const override { return 0; }
|
|
|
|
bool useDefaultImplementationForConstants() const override { return true; }
|
|
|
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
|
|
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override;
|
|
|
|
};
|
|
|
|
|
|
|
|
DataTypePtr FunctionValidateNestedArraySizes::getReturnTypeImpl(const DataTypes & arguments) const
|
|
|
|
{
|
|
|
|
size_t num_args = arguments.size();
|
|
|
|
|
|
|
|
if (num_args < 3)
|
|
|
|
throw Exception(
|
2021-04-13 09:31:49 +00:00
|
|
|
"Function " + getName() + " needs more than two arguments; passed " + toString(arguments.size()) + ".",
|
2021-04-02 11:31:33 +00:00
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
if (!WhichDataType(arguments[0]).isUInt8())
|
|
|
|
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + " Must be UInt.",
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
|
|
|
for (size_t i = 1; i < num_args; ++i)
|
|
|
|
if (!WhichDataType(arguments[i]).isArray())
|
|
|
|
throw Exception(
|
|
|
|
"Illegal type " + arguments[i]->getName() + " of " + toString(i) + " argument of function " + getName() + " Must be Array.",
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
|
|
|
return std::make_shared<DataTypeUInt8>();
|
|
|
|
}
|
|
|
|
|
|
|
|
ColumnPtr FunctionValidateNestedArraySizes::executeImpl(
|
|
|
|
const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const
|
|
|
|
{
|
2021-05-17 08:34:50 +00:00
|
|
|
bool is_condition_const = false;
|
|
|
|
bool condition = false;
|
2021-04-02 11:31:33 +00:00
|
|
|
const ColumnUInt8 * condition_column = typeid_cast<const ColumnUInt8 *>(arguments[0].column.get());
|
2021-05-17 08:34:50 +00:00
|
|
|
if (!condition_column)
|
|
|
|
{
|
|
|
|
if (checkAndGetColumnConst<ColumnUInt8>(arguments[0].column.get()))
|
|
|
|
{
|
|
|
|
is_condition_const = true;
|
|
|
|
condition = arguments[0].column->getBool(0);
|
|
|
|
}
|
|
|
|
}
|
2021-04-02 11:31:33 +00:00
|
|
|
|
|
|
|
size_t args_num = arguments.size();
|
2021-04-07 02:39:01 +00:00
|
|
|
|
2021-04-02 11:31:33 +00:00
|
|
|
for (size_t i = 0; i < input_rows_count; ++i)
|
|
|
|
{
|
2021-05-17 08:34:50 +00:00
|
|
|
if (is_condition_const && !condition)
|
|
|
|
continue;
|
|
|
|
else if (!is_condition_const && !condition_column->getData()[i])
|
2021-04-02 11:31:33 +00:00
|
|
|
continue;
|
|
|
|
|
|
|
|
/// The condition is true, then check the row in subcolumns in Nested Type has the same array size
|
|
|
|
size_t first_length = 0;
|
|
|
|
size_t length = 0;
|
|
|
|
for (size_t args_idx = 1; args_idx < args_num; ++args_idx)
|
|
|
|
{
|
2021-04-16 14:19:22 +00:00
|
|
|
const auto & current_arg = arguments[args_idx];
|
2021-04-02 11:31:33 +00:00
|
|
|
const ColumnArray * current_column = nullptr;
|
2021-04-16 14:19:22 +00:00
|
|
|
if (const auto * const_array = checkAndGetColumnConst<ColumnArray>(current_arg.column.get()))
|
2021-04-02 11:31:33 +00:00
|
|
|
{
|
|
|
|
current_column = checkAndGetColumn<ColumnArray>(&const_array->getDataColumn());
|
|
|
|
length = current_column->getOffsets()[0];
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2021-04-16 14:19:22 +00:00
|
|
|
current_column = checkAndGetColumn<ColumnArray>(current_arg.column.get());
|
2021-04-13 09:31:49 +00:00
|
|
|
const auto & offsets = current_column->getOffsets();
|
2021-04-02 11:31:33 +00:00
|
|
|
length = offsets[i] - offsets[i - 1];
|
|
|
|
}
|
2021-04-07 02:39:01 +00:00
|
|
|
|
2021-04-02 11:31:33 +00:00
|
|
|
if (args_idx == 1)
|
|
|
|
{
|
|
|
|
first_length = length;
|
|
|
|
}
|
|
|
|
else if (first_length != length)
|
|
|
|
{
|
|
|
|
throw Exception(
|
2021-04-13 09:31:49 +00:00
|
|
|
ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH,
|
|
|
|
"Elements '{}' and '{}' of Nested data structure (Array columns) "
|
|
|
|
"have different array sizes ({} and {} respectively) on row {}",
|
|
|
|
arguments[1].name, arguments[args_idx].name, first_length, length, i);
|
2021-04-02 11:31:33 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2021-04-07 02:39:01 +00:00
|
|
|
|
2021-04-16 14:19:22 +00:00
|
|
|
return ColumnUInt8::create(input_rows_count, 1);
|
2021-04-02 11:31:33 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void registerFunctionValidateNestedArraySizes(FunctionFactory & factory)
|
|
|
|
{
|
|
|
|
factory.registerFunction<FunctionValidateNestedArraySizes>();
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|