minor fixes

This commit is contained in:
Anton Popov 2021-04-16 17:19:22 +03:00
parent b14798af2a
commit 0a0c510aa2
5 changed files with 91 additions and 51 deletions

View File

@ -73,16 +73,16 @@ ColumnPtr FunctionValidateNestedArraySizes::executeImpl(
size_t length = 0; size_t length = 0;
for (size_t args_idx = 1; args_idx < args_num; ++args_idx) for (size_t args_idx = 1; args_idx < args_num; ++args_idx)
{ {
ColumnWithTypeAndName current_arg = arguments[args_idx]; const auto & current_arg = arguments[args_idx];
const ColumnArray * current_column = nullptr; const ColumnArray * current_column = nullptr;
if (const auto *const_array = checkAndGetColumnConst<ColumnArray>(current_arg.column.get())) if (const auto * const_array = checkAndGetColumnConst<ColumnArray>(current_arg.column.get()))
{ {
current_column = checkAndGetColumn<ColumnArray>(&const_array->getDataColumn()); current_column = checkAndGetColumn<ColumnArray>(&const_array->getDataColumn());
length = current_column->getOffsets()[0]; length = current_column->getOffsets()[0];
} }
else else
{ {
current_column = typeid_cast<const ColumnArray *>(current_arg.column.get()); current_column = checkAndGetColumn<ColumnArray>(current_arg.column.get());
const auto & offsets = current_column->getOffsets(); const auto & offsets = current_column->getOffsets();
length = offsets[i] - offsets[i - 1]; length = offsets[i] - offsets[i - 1];
} }
@ -102,12 +102,7 @@ ColumnPtr FunctionValidateNestedArraySizes::executeImpl(
} }
} }
auto res = ColumnUInt8::create(input_rows_count); return ColumnUInt8::create(input_rows_count, 1);
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) void registerFunctionValidateNestedArraySizes(FunctionFactory & factory)

View File

@ -350,33 +350,34 @@ static void validateUpdateColumns(
} }
} }
std::pair<bool, std::vector<ASTPtr>> getFullNestedSubColumnUpdatedExpr( /// Returns ASTs of updated nested subcolumns, if all of subcolumns were updated.
const String & column, /// They are used to validate sizes of nested arrays.
NamesAndTypesList & all_columns, /// If some of subcolumns were updated and some weren't,
std::unordered_map<String, ASTPtr> & column_to_update_expression) /// it makes sense to validate only updated columns with their old versions,
/// because their sizes couldn't change, since sizes of all nested subcolumns must be consistent.
static std::optional<std::vector<ASTPtr>> getExpressionsOfUpdatedNestedSubcolumns(
const String & column_name,
const NamesAndTypesList & all_columns,
const std::unordered_map<String, ASTPtr> & column_to_update_expression)
{ {
std::vector<ASTPtr> res; std::vector<ASTPtr> res;
auto source_name = Nested::splitName(column).first; auto source_name = Nested::splitName(column_name).first;
/// Check this nested subcolumn /// Check this nested subcolumn
for (const auto & it : all_columns) for (const auto & column : all_columns)
{ {
auto split = Nested::splitName(it.name); auto split = Nested::splitName(column.name);
if (split.first == source_name && !split.second.empty()) if (isArray(column.type) && split.first == source_name && !split.second.empty())
{ {
if (column_to_update_expression.find(it.name) == column_to_update_expression.end()) auto it = column_to_update_expression.find(column.name);
{ if (it == column_to_update_expression.end())
/// Update partial nested subcolumns return {};
return std::make_pair(false, res);
} res.push_back(it->second);
else
{
res.push_back(column_to_update_expression[it.name]);
}
} }
} }
return std::make_pair(true, res); return res;
} }
ASTPtr MutationsInterpreter::prepare(bool dry_run) ASTPtr MutationsInterpreter::prepare(bool dry_run)
@ -467,19 +468,20 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
/// ///
/// Outer CAST is added just in case if we don't trust the returning type of 'if'. /// Outer CAST is added just in case if we don't trust the returning type of 'if'.
auto type_literal = std::make_shared<ASTLiteral>(columns_desc.getPhysical(column).type->getName()); const auto & type = columns_desc.getPhysical(column).type;
auto type_literal = std::make_shared<ASTLiteral>(type->getName());
const auto & update_expr = kv.second; const auto & update_expr = kv.second;
ASTPtr condition = getPartitionAndPredicateExpressionForMutationCommand(command); ASTPtr condition = getPartitionAndPredicateExpressionForMutationCommand(command);
/// And new check validateNestedArraySizes for Nested subcolumns /// And new check validateNestedArraySizes for Nested subcolumns
if (!Nested::splitName(column).second.empty()) if (isArray(type) && !Nested::splitName(column).second.empty())
{ {
std::shared_ptr<ASTFunction> function = nullptr; std::shared_ptr<ASTFunction> function = nullptr;
auto nested_update_exprs = getFullNestedSubColumnUpdatedExpr(column, all_columns, command.column_to_update_expression); auto nested_update_exprs = getExpressionsOfUpdatedNestedSubcolumns(column, all_columns, command.column_to_update_expression);
if (!nested_update_exprs.first) if (!nested_update_exprs)
{ {
function = makeASTFunction("validateNestedArraySizes", function = makeASTFunction("validateNestedArraySizes",
condition, condition,
@ -487,14 +489,14 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
std::make_shared<ASTIdentifier>(column)); std::make_shared<ASTIdentifier>(column));
condition = makeASTFunction("and", condition, function); condition = makeASTFunction("and", condition, function);
} }
else if (nested_update_exprs.second.size() > 1) else if (nested_update_exprs->size() > 1)
{ {
function = std::make_shared<ASTFunction>(); function = std::make_shared<ASTFunction>();
function->name = "validateNestedArraySizes"; function->name = "validateNestedArraySizes";
function->arguments = std::make_shared<ASTExpressionList>(); function->arguments = std::make_shared<ASTExpressionList>();
function->children.push_back(function->arguments); function->children.push_back(function->arguments);
function->arguments->children.push_back(condition); function->arguments->children.push_back(condition);
for (const auto & it : nested_update_exprs.second) for (const auto & it : *nested_update_exprs)
function->arguments->children.push_back(it->clone()); function->arguments->children.push_back(it->clone());
condition = makeASTFunction("and", condition, function); condition = makeASTFunction("and", condition, function);
} }

View File

@ -3,6 +3,7 @@
#include <Compression/CompressionFactory.h> #include <Compression/CompressionFactory.h>
#include <Compression/CompressedReadBufferFromFile.h> #include <Compression/CompressedReadBufferFromFile.h>
#include <DataTypes/Serializations/ISerialization.h> #include <DataTypes/Serializations/ISerialization.h>
#include <Common/escapeForFileName.h>
namespace DB namespace DB
{ {
@ -393,8 +394,9 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const String & name,
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot validate column of non fixed type {}", type.getName()); throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot validate column of non fixed type {}", type.getName());
auto disk = data_part->volume->getDisk(); auto disk = data_part->volume->getDisk();
String mrk_path = fullPath(disk, part_path + name + marks_file_extension); String escaped_name = escapeForFileName(name);
String bin_path = fullPath(disk, part_path + name + DATA_FILE_EXTENSION); String mrk_path = fullPath(disk, part_path + escaped_name + marks_file_extension);
String bin_path = fullPath(disk, part_path + escaped_name + DATA_FILE_EXTENSION);
DB::ReadBufferFromFile mrk_in(mrk_path); DB::ReadBufferFromFile mrk_in(mrk_path);
DB::CompressedReadBufferFromFile bin_in(bin_path, 0, 0, 0, nullptr); DB::CompressedReadBufferFromFile bin_in(bin_path, 0, 0, 0, nullptr);
bool must_be_last = false; bool must_be_last = false;

View File

@ -4,10 +4,6 @@
2 [100,200,300] ['a','b','c'] [10,20,30] 2 [100,200,300] ['a','b','c'] [10,20,30]
3 [3,4] ['aa','bb'] [3,6] 3 [3,4] ['aa','bb'] [3,6]
4 [4,5] ['aa','bb'] [4,8] 4 [4,5] ['aa','bb'] [4,8]
waiting default test_wide_nested mutation_2.txt UPDATE `info.id` = [100, 200] WHERE id = 1
waiting default test_wide_nested mutation_3.txt UPDATE `info.id` = [100, 200, 300], `info.age` = [10, 20, 30], `info.name` = [\'a\', \'b\', \'c\'] WHERE id = 2
waiting default test_wide_nested mutation_4.txt UPDATE `info.id` = [100, 200], `info.age` = [10, 20, 30], `info.name` = [\'a\', \'b\', \'c\'] WHERE id = 0
waiting default test_wide_nested mutation_5.txt UPDATE `info.id` = [100, 200, 300], `info.age` = [10, 20, 30] WHERE id = 1
0 [0,1] ['aa','bb'] [0,0] 0 [0,1] ['aa','bb'] [0,0]
1 [100,200] ['aa','bb'] [1,2] 1 [100,200] ['aa','bb'] [1,2]
2 [100,200,300] ['a','b','c'] [100,200,300] 2 [100,200,300] ['a','b','c'] [100,200,300]
@ -18,3 +14,8 @@ waiting default test_wide_nested mutation_5.txt UPDATE `info.id` = [100, 200, 30
2 [100,200,300] ['a','b','c'] [100,200,300] 2 [100,200,300] ['a','b','c'] [100,200,300]
3 [68,72] ['aa','bb'] [68,72] 3 [68,72] ['aa','bb'] [68,72]
4 [4,5] ['aa','bb'] [4,8] 4 [4,5] ['aa','bb'] [4,8]
0 0 aa 0
1 1 bb 2
2 2 aa 4
3 3 aa 6
4 4 aa 8

View File

@ -1,4 +1,3 @@
USE default;
DROP TABLE IF EXISTS test_wide_nested; DROP TABLE IF EXISTS test_wide_nested;
CREATE TABLE test_wide_nested CREATE TABLE test_wide_nested
@ -12,19 +11,60 @@ ENGINE = MergeTree
ORDER BY tuple() ORDER BY tuple()
SETTINGS min_bytes_for_wide_part = 0; 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); set mutations_sync = 1;
alter table test_wide_nested update `info.id` =[100,200] where id = 1 settings mutations_sync=1; INSERT INTO test_wide_nested SELECT number, [number,number + 1], ['aa','bb'], [number,number * 2] FROM numbers(5);
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; alter table test_wide_nested update `info.id` = [100,200] where id = 1;
select * from test_wide_nested where id = 1 order by id;
alter table test_wide_nested update `info.id` = [100,200,300], `info.age` = [10,20,30], `info.name` = ['a','b','c'] where id = 2;
select * from test_wide_nested; 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'; alter table test_wide_nested update `info.id` = [100,200,300], `info.age` = `info.id`, `info.name` = ['a','b','c'] where id = 2;
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';
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; 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; alter table test_wide_nested update `info.id` = [100,200], `info.age`=[68,72] where id = 3;
alter table test_wide_nested update `info.id` = `info.age` where id = 3;
select * from test_wide_nested; 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; -- { serverError 341 }
-- Recreate table, because KILL MUTATION is not suitable for parallel tests execution.
DROP TABLE test_wide_nested; DROP TABLE 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,300], `info.age` = [10,20,30] where id = 1; -- { serverError 341 }
DROP TABLE test_wide_nested;
DROP TABLE IF EXISTS test_wide_not_nested;
CREATE TABLE test_wide_not_nested
(
`id` Int,
`info.id` Int,
`info.name` String,
`info.age` Int
)
ENGINE = MergeTree
ORDER BY tuple()
SETTINGS min_bytes_for_wide_part = 0;
INSERT INTO test_wide_not_nested SELECT number, number, 'aa', number * 2 FROM numbers(5);
ALTER TABLE test_wide_not_nested UPDATE `info.name` = 'bb' WHERE id = 1;
SELECT * FROM test_wide_not_nested ORDER BY id;
DROP TABLE test_wide_not_nested;