mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 21:24:28 +00:00
Add array size check for Nested Type subcolumns when Update
This commit is contained in:
parent
7af59dbf79
commit
6e8583a028
@ -45,6 +45,7 @@ void registerFunctionsUnixTimestamp64(FunctionFactory & factory);
|
|||||||
void registerFunctionBitHammingDistance(FunctionFactory & factory);
|
void registerFunctionBitHammingDistance(FunctionFactory & factory);
|
||||||
void registerFunctionTupleHammingDistance(FunctionFactory & factory);
|
void registerFunctionTupleHammingDistance(FunctionFactory & factory);
|
||||||
void registerFunctionsStringHash(FunctionFactory & factory);
|
void registerFunctionsStringHash(FunctionFactory & factory);
|
||||||
|
void registerFunctionValidateNestedArraySizes(FunctionFactory & factory);
|
||||||
#if !defined(ARCADIA_BUILD)
|
#if !defined(ARCADIA_BUILD)
|
||||||
void registerFunctionBayesAB(FunctionFactory &);
|
void registerFunctionBayesAB(FunctionFactory &);
|
||||||
#endif
|
#endif
|
||||||
@ -103,6 +104,7 @@ void registerFunctions()
|
|||||||
registerFunctionBitHammingDistance(factory);
|
registerFunctionBitHammingDistance(factory);
|
||||||
registerFunctionTupleHammingDistance(factory);
|
registerFunctionTupleHammingDistance(factory);
|
||||||
registerFunctionsStringHash(factory);
|
registerFunctionsStringHash(factory);
|
||||||
|
registerFunctionValidateNestedArraySizes(factory);
|
||||||
|
|
||||||
#if !defined(ARCADIA_BUILD)
|
#if !defined(ARCADIA_BUILD)
|
||||||
registerFunctionBayesAB(factory);
|
registerFunctionBayesAB(factory);
|
||||||
|
113
src/Functions/validateNestedArraySizes.cpp
Normal file
113
src/Functions/validateNestedArraySizes.cpp
Normal file
@ -0,0 +1,113 @@
|
|||||||
|
#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 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<FunctionValidateNestedArraySizes>(); }
|
||||||
|
|
||||||
|
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<DataTypeUInt8>();
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnPtr FunctionValidateNestedArraySizes::executeImpl(
|
||||||
|
const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const
|
||||||
|
{
|
||||||
|
const ColumnUInt8 * condition_column = typeid_cast<const ColumnUInt8 *>(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<ColumnArray>(current_arg.column.get()))
|
||||||
|
{
|
||||||
|
current_column = checkAndGetColumn<ColumnArray>(&const_array->getDataColumn());
|
||||||
|
length = current_column->getOffsets()[0];
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
current_column = typeid_cast<const ColumnArray *>(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<FunctionValidateNestedArraySizes>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -26,6 +26,7 @@
|
|||||||
#include <Parsers/formatAST.h>
|
#include <Parsers/formatAST.h>
|
||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
#include <Processors/QueryPlan/CreatingSetsStep.h>
|
#include <Processors/QueryPlan/CreatingSetsStep.h>
|
||||||
|
#include <DataTypes/NestedUtils.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -349,6 +350,34 @@ static void validateUpdateColumns(
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::pair<bool, std::vector<ASTPtr>> getFullNestedSubColumnUpdatedExpr(
|
||||||
|
const String & column,
|
||||||
|
NamesAndTypesList all_columns,
|
||||||
|
std::unordered_map<String, ASTPtr> column_to_update_expression)
|
||||||
|
{
|
||||||
|
std::vector<ASTPtr> 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)
|
ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||||
{
|
{
|
||||||
@ -441,9 +470,38 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
|||||||
auto type_literal = std::make_shared<ASTLiteral>(columns_desc.getPhysical(column).type->getName());
|
auto type_literal = std::make_shared<ASTLiteral>(columns_desc.getPhysical(column).type->getName());
|
||||||
|
|
||||||
const auto & update_expr = kv.second;
|
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<ASTFunction> 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<ASTIdentifier>(column));
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
function = std::make_shared<ASTFunction>();
|
||||||
|
function->name = "validateNestedArraySizes";
|
||||||
|
function->arguments = std::make_shared<ASTExpressionList>();
|
||||||
|
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",
|
auto updated_column = makeASTFunction("CAST",
|
||||||
makeASTFunction("if",
|
makeASTFunction("if",
|
||||||
getPartitionAndPredicateExpressionForMutationCommand(command),
|
condition,
|
||||||
makeASTFunction("CAST",
|
makeASTFunction("CAST",
|
||||||
update_expr->clone(),
|
update_expr->clone(),
|
||||||
type_literal),
|
type_literal),
|
||||||
|
@ -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]
|
@ -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;
|
Loading…
Reference in New Issue
Block a user