From 6e8583a028b01fd71b4be648f6a915e57ff141e1 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Fri, 2 Apr 2021 19:31:33 +0800 Subject: [PATCH] Add array size check for Nested Type subcolumns when Update --- src/Functions/registerFunctions.cpp | 2 + src/Functions/validateNestedArraySizes.cpp | 113 ++++++++++++++++++ src/Interpreters/MutationsInterpreter.cpp | 60 +++++++++- ...date_nested_type_subcolumn_check.reference | 16 +++ ...788_update_nested_type_subcolumn_check.sql | 29 +++++ 5 files changed, 219 insertions(+), 1 deletion(-) create mode 100644 src/Functions/validateNestedArraySizes.cpp create mode 100644 tests/queries/0_stateless/01788_update_nested_type_subcolumn_check.reference create mode 100644 tests/queries/0_stateless/01788_update_nested_type_subcolumn_check.sql diff --git a/src/Functions/registerFunctions.cpp b/src/Functions/registerFunctions.cpp index d827cc40a86..6db0f9183b2 100644 --- a/src/Functions/registerFunctions.cpp +++ b/src/Functions/registerFunctions.cpp @@ -45,6 +45,7 @@ void registerFunctionsUnixTimestamp64(FunctionFactory & factory); void registerFunctionBitHammingDistance(FunctionFactory & factory); void registerFunctionTupleHammingDistance(FunctionFactory & factory); void registerFunctionsStringHash(FunctionFactory & factory); +void registerFunctionValidateNestedArraySizes(FunctionFactory & factory); #if !defined(ARCADIA_BUILD) void registerFunctionBayesAB(FunctionFactory &); #endif @@ -103,6 +104,7 @@ void registerFunctions() registerFunctionBitHammingDistance(factory); registerFunctionTupleHammingDistance(factory); registerFunctionsStringHash(factory); + registerFunctionValidateNestedArraySizes(factory); #if !defined(ARCADIA_BUILD) registerFunctionBayesAB(factory); diff --git a/src/Functions/validateNestedArraySizes.cpp b/src/Functions/validateNestedArraySizes.cpp new file mode 100644 index 00000000000..796dc0a957e --- /dev/null +++ b/src/Functions/validateNestedArraySizes.cpp @@ -0,0 +1,113 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_COLUMN; + extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; +} + +class FunctionValidateNestedArraySizes : public IFunction +{ +public: + static constexpr auto name = "validateNestedArraySizes"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + 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( + "Function " + getName() + " needs one argument; passed " + toString(arguments.size()) + ".", + 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(); +} + +ColumnPtr FunctionValidateNestedArraySizes::executeImpl( + const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const +{ + const ColumnUInt8 * condition_column = typeid_cast(arguments[0].column.get()); + + size_t args_num = arguments.size(); + + for (size_t i = 0; i < input_rows_count; ++i) + { + if (!condition_column->getData()[i]) + 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) + { + ColumnWithTypeAndName current_arg = arguments[args_idx]; + const ColumnArray * current_column = nullptr; + if (const auto *const_array = checkAndGetColumnConst(current_arg.column.get())) + { + current_column = checkAndGetColumn(&const_array->getDataColumn()); + length = current_column->getOffsets()[0]; + } + else + { + current_column = typeid_cast(current_arg.column.get()); + auto & offsets = current_column->getOffsets(); + length = offsets[i] - offsets[i - 1]; + } + + if (args_idx == 1) + { + first_length = length; + } + else if (first_length != length) + { + throw Exception( + "Elements '" + arguments[1].column->getName() + "' and '" + arguments[i].column->getName() + + +"' of Nested data structure (Array columns) have different array sizes.", + ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); + } + } + } + + auto res = ColumnUInt8::create(input_rows_count); + auto & vec_res = res->getData(); + for (size_t row_num = 0; row_num < input_rows_count; ++row_num) + vec_res[row_num] = 1; + + return res; +} + +void registerFunctionValidateNestedArraySizes(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 3573d48b837..7a804c464d3 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -26,6 +26,7 @@ #include #include #include +#include namespace DB @@ -349,6 +350,34 @@ static void validateUpdateColumns( } } +std::pair> getFullNestedSubColumnUpdatedExpr( + const String & column, + NamesAndTypesList all_columns, + std::unordered_map column_to_update_expression) +{ + std::vector res; + auto source_name = Nested::splitName(column).first; + + /// Check this nested subcolumn + for (const auto & it : all_columns) + { + auto split = Nested::splitName(it.name); + if (split.first == source_name && !split.second.empty()) + { + if (column_to_update_expression.find(it.name) == column_to_update_expression.end()) + { + /// Update partial nested subcolumns + return std::make_pair(false, res); + } + else + { + res.push_back(column_to_update_expression[it.name]); + } + } + } + + return std::make_pair(true, res); +} ASTPtr MutationsInterpreter::prepare(bool dry_run) { @@ -441,9 +470,38 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) auto type_literal = std::make_shared(columns_desc.getPhysical(column).type->getName()); const auto & update_expr = kv.second; + + ASTPtr condition = getPartitionAndPredicateExpressionForMutationCommand(command); + + /// And new check validateNestedArraySizes for Nested subcolumns + if (!Nested::splitName(column).second.empty()) + { + std::shared_ptr function = nullptr; + + auto nested_update_exprs = getFullNestedSubColumnUpdatedExpr(column, all_columns, command.column_to_update_expression); + if (nested_update_exprs.first == false) + { + function = makeASTFunction("validateNestedArraySizes", + condition, + update_expr->clone(), + std::make_shared(column)); + } + else + { + function = std::make_shared(); + function->name = "validateNestedArraySizes"; + function->arguments = std::make_shared(); + function->children.push_back(function->arguments); + function->arguments->children.push_back(condition); + for (auto it : nested_update_exprs.second) + function->arguments->children.push_back(it->clone()); + } + condition = makeASTFunction("and", condition, function); + } + auto updated_column = makeASTFunction("CAST", makeASTFunction("if", - getPartitionAndPredicateExpressionForMutationCommand(command), + condition, makeASTFunction("CAST", update_expr->clone(), type_literal), diff --git a/tests/queries/0_stateless/01788_update_nested_type_subcolumn_check.reference b/tests/queries/0_stateless/01788_update_nested_type_subcolumn_check.reference new file mode 100644 index 00000000000..3fc3d65c0e0 --- /dev/null +++ b/tests/queries/0_stateless/01788_update_nested_type_subcolumn_check.reference @@ -0,0 +1,16 @@ +1 [100,200] ['aa','bb'] [1,2] +0 [0,1] ['aa','bb'] [0,0] +1 [100,200] ['aa','bb'] [1,2] +2 [100,200,300] ['a','b','c'] [10,20,30] +3 [3,4] ['aa','bb'] [3,6] +4 [4,5] ['aa','bb'] [4,8] +0 [0,1] ['aa','bb'] [0,0] +1 [100,200] ['aa','bb'] [1,2] +2 [100,200,300] ['a','b','c'] [100,200,300] +3 [3,4] ['aa','bb'] [3,6] +4 [4,5] ['aa','bb'] [4,8] +0 [0,1] ['aa','bb'] [0,0] +1 [100,200] ['aa','bb'] [1,2] +2 [100,200,300] ['a','b','c'] [100,200,300] +3 [68,72] ['aa','bb'] [68,72] +4 [4,5] ['aa','bb'] [4,8] diff --git a/tests/queries/0_stateless/01788_update_nested_type_subcolumn_check.sql b/tests/queries/0_stateless/01788_update_nested_type_subcolumn_check.sql new file mode 100644 index 00000000000..29e702b62f9 --- /dev/null +++ b/tests/queries/0_stateless/01788_update_nested_type_subcolumn_check.sql @@ -0,0 +1,29 @@ +DROP TABLE IF EXISTS test_wide_nested; + +CREATE TABLE test_wide_nested +( + `id` Int, + `info.id` Array(Int), + `info.name` Array(String), + `info.age` Array(Int) +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO test_wide_nested SELECT number, [number,number+1] , ['aa','bb'],[number, number*2] FROM numbers(5); + +alter table test_wide_nested update `info.id` =[100,200] where id = 1 settings mutations_sync=1; +select * from test_wide_nested where id = 1; +alter table test_wide_nested update `info.id` =[100,200,300], `info.age`=[10,20,30], `info.name`=['a','b','c'] where id = 2 settings mutations_sync=1; +select * from test_wide_nested; +alter table test_wide_nested update `info.id` =[100,200], `info.age`=[10,20,30], `info.name`=['a','b','c'] where id = 0 settings mutations_sync=1; -- { serverError 341 } +kill mutation where table = 'test_wide_nested' and database = currentDatabase(); +alter table test_wide_nested update `info.id` =[100,200,300], `info.age`=[10,20,30] where id = 1 settings mutations_sync=1; -- { serverError 341 } +kill mutation where table = 'test_wide_nested' and database = currentDatabase(); +alter table test_wide_nested update `info.id` =[100,200,300], `info.age`=`info.id`, `info.name`=['a','b','c'] where id = 2 settings mutations_sync=1; +select * from test_wide_nested; +alter table test_wide_nested update `info.id` =[100,200], `info.age`=[68,72] where id = 3 settings mutations_sync=1; +alter table test_wide_nested update `info.id` = `info.age` where id = 3 settings mutations_sync=1; +select * from test_wide_nested; +DROP TABLE test_wide_nested;