From 6e8583a028b01fd71b4be648f6a915e57ff141e1 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Fri, 2 Apr 2021 19:31:33 +0800 Subject: [PATCH 01/58] 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; From 8c60f95551423f84721c3d4edd666dad41cbcbd1 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Wed, 7 Apr 2021 10:39:01 +0800 Subject: [PATCH 02/58] Fix error --- src/Functions/validateNestedArraySizes.cpp | 9 ++++----- src/Interpreters/MutationsInterpreter.cpp | 5 +++-- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Functions/validateNestedArraySizes.cpp b/src/Functions/validateNestedArraySizes.cpp index 796dc0a957e..8e27f3933a4 100644 --- a/src/Functions/validateNestedArraySizes.cpp +++ b/src/Functions/validateNestedArraySizes.cpp @@ -12,7 +12,6 @@ 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; } @@ -58,7 +57,7 @@ ColumnPtr FunctionValidateNestedArraySizes::executeImpl( 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]) @@ -82,7 +81,7 @@ ColumnPtr FunctionValidateNestedArraySizes::executeImpl( auto & offsets = current_column->getOffsets(); length = offsets[i] - offsets[i - 1]; } - + if (args_idx == 1) { first_length = length; @@ -96,13 +95,13 @@ ColumnPtr FunctionValidateNestedArraySizes::executeImpl( } } } - + 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; + return res; } void registerFunctionValidateNestedArraySizes(FunctionFactory & factory) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 7a804c464d3..55497d04a2d 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -485,8 +485,9 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) condition, update_expr->clone(), std::make_shared(column)); + condition = makeASTFunction("and", condition, function); } - else + else if (nested_update_exprs.second.size() > 1) { function = std::make_shared(); function->name = "validateNestedArraySizes"; @@ -495,8 +496,8 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) 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); } - condition = makeASTFunction("and", condition, function); } auto updated_column = makeASTFunction("CAST", From 260a77118f9b714e822a09a129e077d100138fec Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 3 Mar 2021 23:40:43 +0300 Subject: [PATCH 03/58] Apply idle_connnection_timeout/poll_interval after each query Since right now it is impossible to change those settings, only via user profile. --- src/Server/TCPHandler.cpp | 23 +++++++++++++++++------ 1 file changed, 17 insertions(+), 6 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index d1a0ea61066..1488b6cc97f 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -155,6 +155,8 @@ void TCPHandler::runImpl() } Settings connection_settings = connection_context->getSettings(); + UInt64 idle_connection_timeout = connection_settings.idle_connection_timeout; + UInt64 poll_interval = connection_settings.poll_interval; sendHello(); @@ -165,10 +167,10 @@ void TCPHandler::runImpl() /// We are waiting for a packet from the client. Thus, every `poll_interval` seconds check whether we need to shut down. { Stopwatch idle_time; - while (!server.isCancelled() && !static_cast(*in).poll( - std::min(connection_settings.poll_interval, connection_settings.idle_connection_timeout) * 1000000)) + UInt64 timeout_ms = std::min(poll_interval, idle_connection_timeout) * 1000000; + while (!server.isCancelled() && !static_cast(*in).poll(timeout_ms)) { - if (idle_time.elapsedSeconds() > connection_settings.idle_connection_timeout) + if (idle_time.elapsedSeconds() > idle_connection_timeout) { LOG_TRACE(log, "Closing idle connection"); return; @@ -209,6 +211,15 @@ void TCPHandler::runImpl() if (!receivePacket()) continue; + /** If Query received, then settings in query_context has been updated + * So, update some other connection settings, for flexibility. + */ + { + const Settings & settings = query_context->getSettingsRef(); + idle_connection_timeout = settings.idle_connection_timeout; + poll_interval = settings.poll_interval; + } + /** If part_uuids got received in previous packet, trying to read again. */ if (state.empty() && state.part_uuids && !receivePacket()) @@ -271,10 +282,10 @@ void TCPHandler::runImpl() if (context != query_context) throw Exception("Unexpected context in InputBlocksReader", ErrorCodes::LOGICAL_ERROR); - size_t poll_interval; + size_t poll_interval_ms; int receive_timeout; - std::tie(poll_interval, receive_timeout) = getReadTimeouts(connection_settings); - if (!readDataNext(poll_interval, receive_timeout)) + std::tie(poll_interval_ms, receive_timeout) = getReadTimeouts(connection_settings); + if (!readDataNext(poll_interval_ms, receive_timeout)) { state.block_in.reset(); state.maybe_compressed_in.reset(); From 4f36ac7f64a03d30fd53323f445f43b1012717ee Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Tue, 13 Apr 2021 17:31:49 +0800 Subject: [PATCH 04/58] Fix error --- src/Functions/validateNestedArraySizes.cpp | 18 ++++++---- src/Interpreters/MutationsInterpreter.cpp | 34 +++++++++---------- ...date_nested_type_subcolumn_check.reference | 4 +++ ...788_update_nested_type_subcolumn_check.sql | 5 +-- 4 files changed, 36 insertions(+), 25 deletions(-) diff --git a/src/Functions/validateNestedArraySizes.cpp b/src/Functions/validateNestedArraySizes.cpp index 8e27f3933a4..6b651ec933e 100644 --- a/src/Functions/validateNestedArraySizes.cpp +++ b/src/Functions/validateNestedArraySizes.cpp @@ -15,11 +15,16 @@ namespace ErrorCodes extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; } +/** 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. + */ class FunctionValidateNestedArraySizes : public IFunction { public: static constexpr auto name = "validateNestedArraySizes"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } bool isVariadic() const override { return true; } @@ -35,7 +40,7 @@ DataTypePtr FunctionValidateNestedArraySizes::getReturnTypeImpl(const DataTypes if (num_args < 3) throw Exception( - "Function " + getName() + " needs one argument; passed " + toString(arguments.size()) + ".", + "Function " + getName() + " needs more than two arguments; passed " + toString(arguments.size()) + ".", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); if (!WhichDataType(arguments[0]).isUInt8()) @@ -78,7 +83,7 @@ ColumnPtr FunctionValidateNestedArraySizes::executeImpl( else { current_column = typeid_cast(current_arg.column.get()); - auto & offsets = current_column->getOffsets(); + const auto & offsets = current_column->getOffsets(); length = offsets[i] - offsets[i - 1]; } @@ -89,9 +94,10 @@ ColumnPtr FunctionValidateNestedArraySizes::executeImpl( 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); + 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); } } } diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 879738805ff..cd00b811ab2 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -352,29 +352,29 @@ static void validateUpdateColumns( std::pair> getFullNestedSubColumnUpdatedExpr( const String & column, - NamesAndTypesList all_columns, - std::unordered_map column_to_update_expression) + 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) + /// Check this nested subcolumn + for (const auto & it : all_columns) + { + auto split = Nested::splitName(it.name); + if (split.first == source_name && !split.second.empty()) { - 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()) { - 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]); - } + /// 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); } @@ -427,7 +427,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) auto dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns); /// First, break a sequence of commands into stages. - for (const auto & command : commands) + for (auto & command : commands) { if (command.type == MutationCommand::DELETE) { @@ -479,7 +479,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) std::shared_ptr function = nullptr; auto nested_update_exprs = getFullNestedSubColumnUpdatedExpr(column, all_columns, command.column_to_update_expression); - if (nested_update_exprs.first == false) + if (!nested_update_exprs.first) { function = makeASTFunction("validateNestedArraySizes", condition, 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 index 3fc3d65c0e0..ffa2b2a735f 100644 --- a/tests/queries/0_stateless/01788_update_nested_type_subcolumn_check.reference +++ b/tests/queries/0_stateless/01788_update_nested_type_subcolumn_check.reference @@ -4,6 +4,10 @@ 2 [100,200,300] ['a','b','c'] [10,20,30] 3 [3,4] ['aa','bb'] [3,6] 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] 1 [100,200] ['aa','bb'] [1,2] 2 [100,200,300] ['a','b','c'] [100,200,300] 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 index 29e702b62f9..45345da2ff1 100644 --- a/tests/queries/0_stateless/01788_update_nested_type_subcolumn_check.sql +++ b/tests/queries/0_stateless/01788_update_nested_type_subcolumn_check.sql @@ -1,3 +1,4 @@ +USE default; DROP TABLE IF EXISTS test_wide_nested; CREATE TABLE test_wide_nested @@ -18,9 +19,9 @@ 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(); +kill mutation where table = 'test_wide_nested'; 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(); +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; alter table test_wide_nested update `info.id` =[100,200], `info.age`=[68,72] where id = 3 settings mutations_sync=1; From 61e2ffc0dde6e74bac9f3c8b54c9efecc9cf6b38 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Wed, 14 Apr 2021 16:53:06 +0800 Subject: [PATCH 05/58] Fix build error --- src/Interpreters/MutationsInterpreter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index cd00b811ab2..d403352019c 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -494,7 +494,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) function->arguments = std::make_shared(); function->children.push_back(function->arguments); function->arguments->children.push_back(condition); - for (auto it : nested_update_exprs.second) + for (const auto & it : nested_update_exprs.second) function->arguments->children.push_back(it->clone()); condition = makeASTFunction("and", condition, function); } From ce6cf1630b840f14a4f5cd9d445863fb76429eed Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Mon, 12 Apr 2021 11:55:54 +0300 Subject: [PATCH 06/58] Added tests. --- .../{s3_mock => s3_mocks}/mock_s3.py | 4 +- .../s3_mocks/unstable_server.py | 66 +++++++++++++++++++ tests/integration/test_storage_s3/test.py | 52 ++++++++++----- 3 files changed, 103 insertions(+), 19 deletions(-) rename tests/integration/test_storage_s3/{s3_mock => s3_mocks}/mock_s3.py (89%) create mode 100644 tests/integration/test_storage_s3/s3_mocks/unstable_server.py diff --git a/tests/integration/test_storage_s3/s3_mock/mock_s3.py b/tests/integration/test_storage_s3/s3_mocks/mock_s3.py similarity index 89% rename from tests/integration/test_storage_s3/s3_mock/mock_s3.py rename to tests/integration/test_storage_s3/s3_mocks/mock_s3.py index 088cc883e57..3e876689175 100644 --- a/tests/integration/test_storage_s3/s3_mock/mock_s3.py +++ b/tests/integration/test_storage_s3/s3_mocks/mock_s3.py @@ -1,3 +1,5 @@ +import sys + from bottle import abort, route, run, request, response @@ -21,4 +23,4 @@ def ping(): return 'OK' -run(host='0.0.0.0', port=8080) +run(host='0.0.0.0', port=int(sys.argv[1])) diff --git a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py new file mode 100644 index 00000000000..2d9c5aef2af --- /dev/null +++ b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py @@ -0,0 +1,66 @@ +import http.server +import random +import re +import socket +import struct +import sys + + +class RequestHandler(http.server.BaseHTTPRequestHandler): + def do_HEAD(self): + if self.path == "/root/test.csv": + self.line = f"{random.randint(10000, 99999)},{random.randint(10000, 99999)},{random.randint(10000, 99999)}\n".encode() + self.from_bytes = 0 + self.end_bytes = len(self.line)*500000 + self.size = self.end_bytes + self.stop_at = 1000000 + self.new_at = len(self.line)*10 + + if "Range" in self.headers: + cr = self.headers["Range"] + parts = re.split("[ -/=]+", cr) + assert parts[0] == "bytes" + self.from_bytes = int(parts[1]) + if parts[2]: + self.end_bytes = int(parts[2])+1 + self.send_response(206) + self.send_header("Content-Range", f"bytes {self.from_bytes}-{self.end_bytes-1}/{self.size}") + else: + self.send_response(200) + + self.send_header("Accept-Ranges", "bytes") + self.send_header("Content-Type", "text/plain") + self.send_header("Content-Length", f"{self.end_bytes-self.from_bytes}") + self.end_headers() + + elif self.path == "/": + self.send_response(200) + self.send_header("Content-Type", "text/plain") + self.end_headers() + + else: + self.send_response(404) + self.send_header("Content-Type", "text/plain") + self.end_headers() + + + def do_GET(self): + self.do_HEAD() + if self.path == "/root/test.csv": + for c, i in enumerate(range(self.from_bytes, self.end_bytes)): + self.wfile.write(self.line[i % len(self.line):i % len(self.line) + 1]) + if (c + 1) % self.new_at == 0: + self.line = f"{random.randint(10000, 99999)},{random.randint(10000, 99999)},{random.randint(10000, 99999)}\n".encode() + if (c + 1) % self.stop_at == 0: + #self.wfile._sock.setsockopt(socket.SOL_SOCKET, socket.SO_LINGER, struct.pack("ii", 0, 0)) + #self.wfile._sock.shutdown(socket.SHUT_RDWR) + #self.wfile._sock.close() + print('Dropping connection') + break + + elif self.path == "/": + self.wfile.write(b"OK") + + +httpd = http.server.HTTPServer(('0.0.0.0', int(sys.argv[1])), RequestHandler) +httpd.serve_forever() diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 9e91aae66b3..5a307148ef8 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -96,7 +96,7 @@ def cluster(): prepare_s3_bucket(cluster) logging.info("S3 bucket created") - run_s3_mock(cluster) + run_s3_mocks(cluster) yield cluster finally: @@ -384,26 +384,33 @@ def test_s3_glob_scheherazade(cluster): assert run_query(instance, query).splitlines() == ["1001\t1001\t1001\t1001"] -def run_s3_mock(cluster): - logging.info("Starting s3 mock") - container_id = cluster.get_container_id('resolver') - current_dir = os.path.dirname(__file__) - cluster.copy_file_to_container(container_id, os.path.join(current_dir, "s3_mock", "mock_s3.py"), "mock_s3.py") - cluster.exec_in_container(container_id, ["python", "mock_s3.py"], detach=True) +def run_s3_mocks(cluster): + logging.info("Starting s3 mocks") + mocks = ( + ("mock_s3.py", "resolver", "8080"), + ("unstable_server.py", "resolver", "8081"), + ) + for mock_filename, container, port in mocks: + container_id = cluster.get_container_id(container) + current_dir = os.path.dirname(__file__) + cluster.copy_file_to_container(container_id, os.path.join(current_dir, "s3_mocks", mock_filename), mock_filename) + cluster.exec_in_container(container_id, ["python", mock_filename, port], detach=True) - # Wait for S3 mock start - for attempt in range(10): - ping_response = cluster.exec_in_container(cluster.get_container_id('resolver'), - ["curl", "-s", "http://resolver:8080/"], nothrow=True) - if ping_response != 'OK': - if attempt == 9: - assert ping_response == 'OK', 'Expected "OK", but got "{}"'.format(ping_response) + # Wait for S3 mocks to start + for mock_filename, container, port in mocks: + for attempt in range(10): + ping_response = cluster.exec_in_container(cluster.get_container_id(container), + ["curl", "-s", f"http://{container}:{port}/"], nothrow=True) + print(f"http://{container}:{port}/", ping_response) + if ping_response != 'OK': + if attempt == 9: + assert ping_response == 'OK', 'Expected "OK", but got "{}"'.format(ping_response) + else: + time.sleep(1) else: - time.sleep(1) - else: - break + break - logging.info("S3 mock started") + logging.info("S3 mocks started") def replace_config(old, new): @@ -523,6 +530,15 @@ def test_storage_s3_get_gzip(cluster, extension, method): run_query(instance, f"DROP TABLE {name}") +def test_storage_s3_get_unstable(cluster): + bucket = cluster.minio_bucket + instance = cluster.instances["dummy"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + get_query = f"SELECT count() FROM s3('http://resolver:8081/{cluster.minio_bucket}/test.csv', 'CSV', '{table_format}')" + result = run_query(instance, get_query) + assert result.splitlines() == ["500000"] + + def test_storage_s3_put_uncompressed(cluster): bucket = cluster.minio_bucket instance = cluster.instances["dummy"] From a63db53576610fb4f49a11ad145aae2faca79edb Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Tue, 13 Apr 2021 00:38:45 +0300 Subject: [PATCH 07/58] Improved tests. --- .../s3_mocks/unstable_server.py | 50 ++++++++++++++++--- tests/integration/test_storage_s3/test.py | 7 ++- 2 files changed, 46 insertions(+), 11 deletions(-) diff --git a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py index 2d9c5aef2af..770eb4d7ca6 100644 --- a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py +++ b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py @@ -6,15 +6,47 @@ import struct import sys +def gen_n_digit_number(n): + assert 0 < n < 19 + return random.randint(10**(n-1), 10**n-1) + +def gen_line(): + # Create fixed length CSV line. + columns = 4 + total_digits = digits_to_go = (columns-1)*19//2 + columns_to_go = columns-1 + + row = [] + def add_number(): + nonlocal digits_to_go + nonlocal columns_to_go + min_digits = max(1, digits_to_go - (columns_to_go-1) * 18) + max_digits = min(18, digits_to_go - (columns_to_go-1)) + digits = random.randint(min_digits, max_digits) + row.append(gen_n_digit_number(digits)) + columns_to_go -= 1 + digits_to_go -= digits + + for i in range(columns // 2): + add_number() + row.append(1) + for i in range(columns - 1 - columns // 2): + add_number() + + line = ",".join(map(str, row)) + "\n" + assert total_digits + 1 == len(line) - columns + return line.encode() + + +line = gen_line() + class RequestHandler(http.server.BaseHTTPRequestHandler): def do_HEAD(self): if self.path == "/root/test.csv": - self.line = f"{random.randint(10000, 99999)},{random.randint(10000, 99999)},{random.randint(10000, 99999)}\n".encode() self.from_bytes = 0 - self.end_bytes = len(self.line)*500000 + self.end_bytes = len(line)*500000 self.size = self.end_bytes - self.stop_at = 1000000 - self.new_at = len(self.line)*10 + self.stop_at = random.randint(900000, 1100000) if "Range" in self.headers: cr = self.headers["Range"] @@ -45,12 +77,16 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): def do_GET(self): + global line + self.do_HEAD() if self.path == "/root/test.csv": + lines = 0 for c, i in enumerate(range(self.from_bytes, self.end_bytes)): - self.wfile.write(self.line[i % len(self.line):i % len(self.line) + 1]) - if (c + 1) % self.new_at == 0: - self.line = f"{random.randint(10000, 99999)},{random.randint(10000, 99999)},{random.randint(10000, 99999)}\n".encode() + j = i % len(line) + self.wfile.write(line[j:j+1]) + if line[j:j+1] == b'\n': + line = gen_line() if (c + 1) % self.stop_at == 0: #self.wfile._sock.setsockopt(socket.SOL_SOCKET, socket.SO_LINGER, struct.pack("ii", 0, 0)) #self.wfile._sock.shutdown(socket.SHUT_RDWR) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 5a307148ef8..c239dc68810 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -401,7 +401,6 @@ def run_s3_mocks(cluster): for attempt in range(10): ping_response = cluster.exec_in_container(cluster.get_container_id(container), ["curl", "-s", f"http://{container}:{port}/"], nothrow=True) - print(f"http://{container}:{port}/", ping_response) if ping_response != 'OK': if attempt == 9: assert ping_response == 'OK', 'Expected "OK", but got "{}"'.format(ping_response) @@ -533,10 +532,10 @@ def test_storage_s3_get_gzip(cluster, extension, method): def test_storage_s3_get_unstable(cluster): bucket = cluster.minio_bucket instance = cluster.instances["dummy"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" - get_query = f"SELECT count() FROM s3('http://resolver:8081/{cluster.minio_bucket}/test.csv', 'CSV', '{table_format}')" + table_format = "column1 Int64, column2 Int64, column3 Int64, column4 Int64" + get_query = f"SELECT count(), sum(column3) FROM s3('http://resolver:8081/{cluster.minio_bucket}/test.csv', 'CSV', '{table_format}') FORMAT CSV" result = run_query(instance, get_query) - assert result.splitlines() == ["500000"] + assert result.splitlines() == ["500000,500000"] def test_storage_s3_put_uncompressed(cluster): From 5b8a3831e6492054dc4ffbe1ea2c1ce90255b3aa Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Tue, 13 Apr 2021 22:11:41 +0300 Subject: [PATCH 08/58] Better tests. --- tests/integration/test_storage_s3/s3_mocks/unstable_server.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py index 770eb4d7ca6..4ef03f08e0f 100644 --- a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py +++ b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py @@ -39,6 +39,7 @@ def gen_line(): line = gen_line() +random.seed('Unstable server/1.0') class RequestHandler(http.server.BaseHTTPRequestHandler): def do_HEAD(self): @@ -46,7 +47,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self.from_bytes = 0 self.end_bytes = len(line)*500000 self.size = self.end_bytes - self.stop_at = random.randint(900000, 1100000) + self.stop_at = random.randint(900000, 1200000) # Block size is 1024**2. if "Range" in self.headers: cr = self.headers["Range"] From edd3931d4e17562cc06b76488125b212fae3ef65 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Tue, 13 Apr 2021 23:19:17 +0300 Subject: [PATCH 09/58] Fixed tests. --- .../s3_mocks/unstable_server.py | 35 ++++++------------- 1 file changed, 11 insertions(+), 24 deletions(-) diff --git a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py index 4ef03f08e0f..4a27845ff9f 100644 --- a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py +++ b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py @@ -10,22 +10,14 @@ def gen_n_digit_number(n): assert 0 < n < 19 return random.randint(10**(n-1), 10**n-1) + def gen_line(): - # Create fixed length CSV line. columns = 4 - total_digits = digits_to_go = (columns-1)*19//2 - columns_to_go = columns-1 row = [] def add_number(): - nonlocal digits_to_go - nonlocal columns_to_go - min_digits = max(1, digits_to_go - (columns_to_go-1) * 18) - max_digits = min(18, digits_to_go - (columns_to_go-1)) - digits = random.randint(min_digits, max_digits) + digits = random.randint(1, 18) row.append(gen_n_digit_number(digits)) - columns_to_go -= 1 - digits_to_go -= digits for i in range(columns // 2): add_number() @@ -34,20 +26,21 @@ def gen_line(): add_number() line = ",".join(map(str, row)) + "\n" - assert total_digits + 1 == len(line) - columns return line.encode() -line = gen_line() -random.seed('Unstable server/1.0') +random.seed("Unstable server/1.0") +lines = b"".join((gen_line() for _ in range(500000))) + class RequestHandler(http.server.BaseHTTPRequestHandler): def do_HEAD(self): if self.path == "/root/test.csv": self.from_bytes = 0 - self.end_bytes = len(line)*500000 + self.end_bytes = len(lines) self.size = self.end_bytes - self.stop_at = random.randint(900000, 1200000) # Block size is 1024**2. + self.send_block_size = 256 + self.stop_at = random.randint(900000, 1200000) // self.send_block_size # Block size is 1024**2. if "Range" in self.headers: cr = self.headers["Range"] @@ -78,16 +71,10 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): def do_GET(self): - global line - self.do_HEAD() if self.path == "/root/test.csv": - lines = 0 - for c, i in enumerate(range(self.from_bytes, self.end_bytes)): - j = i % len(line) - self.wfile.write(line[j:j+1]) - if line[j:j+1] == b'\n': - line = gen_line() + for c, i in enumerate(range(self.from_bytes, self.end_bytes, self.send_block_size)): + self.wfile.write(lines[i:min(i+self.send_block_size, self.end_bytes)]) if (c + 1) % self.stop_at == 0: #self.wfile._sock.setsockopt(socket.SOL_SOCKET, socket.SO_LINGER, struct.pack("ii", 0, 0)) #self.wfile._sock.shutdown(socket.SHUT_RDWR) @@ -99,5 +86,5 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self.wfile.write(b"OK") -httpd = http.server.HTTPServer(('0.0.0.0', int(sys.argv[1])), RequestHandler) +httpd = http.server.HTTPServer(("0.0.0.0", int(sys.argv[1])), RequestHandler) httpd.serve_forever() From 10bf56e540d9ab4ed34dc805ff4988ec06007e4e Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Tue, 13 Apr 2021 01:25:19 +0300 Subject: [PATCH 10/58] Attempt to add retries to S3 reads. --- src/Core/Settings.h | 1 + src/Disks/S3/DiskS3.cpp | 19 +++++++---- src/Disks/S3/DiskS3.h | 2 ++ src/Disks/S3/registerDiskS3.cpp | 1 + src/IO/ReadBufferFromS3.cpp | 45 ++++++++++++++++++++------ src/IO/ReadBufferFromS3.h | 3 +- src/Storages/StorageS3.cpp | 7 ++++ src/Storages/StorageS3.h | 2 ++ src/TableFunctions/TableFunctionS3.cpp | 2 ++ 9 files changed, 65 insertions(+), 17 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d31073ae932..bb17ffc9167 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -70,6 +70,7 @@ class IColumn; M(UInt64, connections_with_failover_max_tries, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, "The maximum number of attempts to connect to replicas.", 0) \ M(UInt64, s3_min_upload_part_size, 512*1024*1024, "The minimum size of part to upload during multipart upload to S3.", 0) \ M(UInt64, s3_max_single_part_upload_size, 64*1024*1024, "The maximum size of object to upload using singlepart upload to S3.", 0) \ + M(Int64, s3_max_single_read_retries, 16, "The maximum number of retries during single S3 read.", 0) \ M(UInt64, s3_max_redirects, 10, "Max number of S3 redirects hops allowed.", 0) \ M(UInt64, s3_max_connections, 1024, "The maximum number of connections per server.", 0) \ M(Bool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \ diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 1de4ab843ac..215f86e1bc7 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -249,8 +249,12 @@ class ReadIndirectBufferFromS3 final : public ReadBufferFromFileBase { public: ReadIndirectBufferFromS3( - std::shared_ptr client_ptr_, const String & bucket_, DiskS3::Metadata metadata_, size_t buf_size_) - : client_ptr(std::move(client_ptr_)), bucket(bucket_), metadata(std::move(metadata_)), buf_size(buf_size_) + std::shared_ptr client_ptr_, const String & bucket_, DiskS3::Metadata metadata_, Int64 s3_max_single_read_retries_, size_t buf_size_) + : client_ptr(std::move(client_ptr_)) + , bucket(bucket_) + , metadata(std::move(metadata_)) + , s3_max_single_read_retries(s3_max_single_read_retries_) + , buf_size(buf_size_) { } @@ -306,7 +310,7 @@ private: const auto & [path, size] = metadata.s3_objects[i]; if (size > offset) { - auto buf = std::make_unique(client_ptr, bucket, metadata.s3_root_path + path, buf_size); + auto buf = std::make_unique(client_ptr, bucket, metadata.s3_root_path + path, s3_max_single_read_retries, buf_size); buf->seek(offset, SEEK_SET); return buf; } @@ -335,7 +339,7 @@ private: ++current_buf_idx; const auto & path = metadata.s3_objects[current_buf_idx].first; - current_buf = std::make_unique(client_ptr, bucket, metadata.s3_root_path + path, buf_size); + current_buf = std::make_unique(client_ptr, bucket, metadata.s3_root_path + path, s3_max_single_read_retries, buf_size); current_buf->next(); working_buffer = current_buf->buffer(); absolute_position += working_buffer.size(); @@ -346,6 +350,7 @@ private: std::shared_ptr client_ptr; const String & bucket; DiskS3::Metadata metadata; + Int64 s3_max_single_read_retries; size_t buf_size; size_t absolute_position = 0; @@ -559,6 +564,7 @@ DiskS3::DiskS3( String bucket_, String s3_root_path_, String metadata_path_, + Int64 s3_max_single_read_retries_, size_t min_upload_part_size_, size_t max_single_part_upload_size_, size_t min_bytes_for_seek_, @@ -572,6 +578,7 @@ DiskS3::DiskS3( , bucket(std::move(bucket_)) , s3_root_path(std::move(s3_root_path_)) , metadata_path(std::move(metadata_path_)) + , s3_max_single_read_retries(s3_max_single_read_retries_) , min_upload_part_size(min_upload_part_size_) , max_single_part_upload_size(max_single_part_upload_size_) , min_bytes_for_seek(min_bytes_for_seek_) @@ -678,7 +685,7 @@ std::unique_ptr DiskS3::readFile(const String & path, si LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Read from file by path: {}. Existing S3 objects: {}", backQuote(metadata_path + path), metadata.s3_objects.size()); - auto reader = std::make_unique(client, bucket, metadata, buf_size); + auto reader = std::make_unique(client, bucket, metadata, s3_max_single_read_retries, buf_size); return std::make_unique(std::move(reader), min_bytes_for_seek); } @@ -982,7 +989,7 @@ int DiskS3::readSchemaVersion(const String & source_bucket, const String & sourc if (!checkObjectExists(source_bucket, source_path + SCHEMA_VERSION_OBJECT)) return version; - ReadBufferFromS3 buffer (client, source_bucket, source_path + SCHEMA_VERSION_OBJECT); + ReadBufferFromS3 buffer(client, source_bucket, source_path + SCHEMA_VERSION_OBJECT, s3_max_single_read_retries); readIntText(version, buffer); return version; diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 87aab71fc44..b6581c6e5b7 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -40,6 +40,7 @@ public: String bucket_, String s3_root_path_, String metadata_path_, + Int64 s3_max_single_read_retries_, size_t min_upload_part_size_, size_t max_single_part_upload_size_, size_t min_bytes_for_seek_, @@ -180,6 +181,7 @@ private: const String bucket; const String s3_root_path; String metadata_path; + Int64 s3_max_single_read_retries; size_t min_upload_part_size; size_t max_single_part_upload_size; size_t min_bytes_for_seek; diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index a15b6bcf822..2a9dbda930d 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -152,6 +152,7 @@ void registerDiskS3(DiskFactory & factory) uri.bucket, uri.key, metadata_path, + context->getSettingsRef().s3_max_single_read_retries, context->getSettingsRef().s3_min_upload_part_size, context->getSettingsRef().s3_max_single_part_upload_size, config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index fd07a7f309a..fd3b46a50f9 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -16,6 +16,7 @@ namespace ProfileEvents { extern const Event S3ReadMicroseconds; extern const Event S3ReadBytes; + extern const Event S3ReadRequestsErrors; } namespace DB @@ -29,26 +30,50 @@ namespace ErrorCodes ReadBufferFromS3::ReadBufferFromS3( - std::shared_ptr client_ptr_, const String & bucket_, const String & key_, size_t buffer_size_) - : SeekableReadBuffer(nullptr, 0), client_ptr(std::move(client_ptr_)), bucket(bucket_), key(key_), buffer_size(buffer_size_) + std::shared_ptr client_ptr_, const String & bucket_, const String & key_, Int64 s3_max_single_read_retries_, size_t buffer_size_) + : SeekableReadBuffer(nullptr, 0) + , client_ptr(std::move(client_ptr_)) + , bucket(bucket_) + , key(key_) + , s3_max_single_read_retries(s3_max_single_read_retries_) + , buffer_size(buffer_size_) { } bool ReadBufferFromS3::nextImpl() { - if (!initialized) - { + if (!impl) impl = initialize(); - initialized = true; - } Stopwatch watch; - auto res = impl->next(); + bool next_result = false; + + for (Int64 attempt = s3_max_single_read_retries; s3_max_single_read_retries < 0 || attempt >= 0; --attempt) + { + if (!impl) + impl = initialize(); + + try + { + next_result = impl->next(); + break; + } + catch (const Exception & e) + { + ProfileEvents::increment(ProfileEvents::S3ReadRequestsErrors, 1); + + impl.reset(); + offset = getPosition(); + + if (!attempt) + throw; + } + } + watch.stop(); ProfileEvents::increment(ProfileEvents::S3ReadMicroseconds, watch.elapsedMicroseconds()); - - if (!res) + if (!next_result) return false; internal_buffer = impl->buffer(); @@ -60,7 +85,7 @@ bool ReadBufferFromS3::nextImpl() off_t ReadBufferFromS3::seek(off_t offset_, int whence) { - if (initialized) + if (impl) throw Exception("Seek is allowed only before first read attempt from the buffer.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE); if (whence != SEEK_SET) diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index 829b73d0af6..f8fe2407a91 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -27,8 +27,8 @@ private: std::shared_ptr client_ptr; String bucket; String key; + Int64 s3_max_single_read_retries; size_t buffer_size; - bool initialized = false; off_t offset = 0; Aws::S3::Model::GetObjectResult read_result; std::unique_ptr impl; @@ -40,6 +40,7 @@ public: std::shared_ptr client_ptr_, const String & bucket_, const String & key_, + Int64 s3_max_single_read_retries_, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE); bool nextImpl() override; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index a5cbd004d55..84d6b7a3b4b 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -166,6 +166,7 @@ StorageS3Source::StorageS3Source( ContextPtr context_, const ColumnsDescription & columns_, UInt64 max_block_size_, + Int64 s3_max_single_read_retries_, const String compression_hint_, const std::shared_ptr & client_, const String & bucket_, @@ -177,6 +178,7 @@ StorageS3Source::StorageS3Source( , format(format_) , columns_desc(columns_) , max_block_size(max_block_size_) + , s3_max_single_read_retries(s3_max_single_read_retries_) , compression_hint(compression_hint_) , client(client_) , sample_block(sample_block_) @@ -312,6 +314,7 @@ StorageS3::StorageS3( const String & secret_access_key_, const StorageID & table_id_, const String & format_name_, + Int64 s3_max_single_read_retries_, UInt64 min_upload_part_size_, UInt64 max_single_part_upload_size_, UInt64 max_connections_, @@ -323,6 +326,7 @@ StorageS3::StorageS3( : IStorage(table_id_) , client_auth{uri_, access_key_id_, secret_access_key_, max_connections_, {}, {}} /// Client and settings will be updated later , format_name(format_name_) + , s3_max_single_read_retries(s3_max_single_read_retries_) , min_upload_part_size(min_upload_part_size_) , max_single_part_upload_size(max_single_part_upload_size_) , compression_method(compression_method_) @@ -389,6 +393,7 @@ Pipe StorageS3::read( local_context, metadata_snapshot->getColumns(), max_block_size, + s3_max_single_read_retries, compression_method, client_auth.client, client_auth.uri.bucket, @@ -476,6 +481,7 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) UInt64 min_upload_part_size = args.getLocalContext()->getSettingsRef().s3_min_upload_part_size; UInt64 max_single_part_upload_size = args.getLocalContext()->getSettingsRef().s3_max_single_part_upload_size; UInt64 max_connections = args.getLocalContext()->getSettingsRef().s3_max_connections; + Int64 s3_max_single_read_retries = args.getLocalContext()->getSettingsRef().s3_max_single_read_retries; String compression_method; String format_name; @@ -496,6 +502,7 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) secret_access_key, args.table_id, format_name, + s3_max_single_read_retries, min_upload_part_size, max_single_part_upload_size, max_connections, diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 1e1d76fa6e3..506f8eac076 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -100,6 +100,7 @@ public: const String & secret_access_key, const StorageID & table_id_, const String & format_name_, + Int64 s3_max_single_read_retries_, UInt64 min_upload_part_size_, UInt64 max_single_part_upload_size_, UInt64 max_connections_, @@ -145,6 +146,7 @@ private: ClientAuthentificaiton client_auth; String format_name; + Int64 s3_max_single_read_retries; size_t min_upload_part_size; size_t max_single_part_upload_size; String compression_method; diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 2da597f49ff..fbff23a23d0 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -86,6 +86,7 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, Context UInt64 min_upload_part_size = context->getSettingsRef().s3_min_upload_part_size; UInt64 max_single_part_upload_size = context->getSettingsRef().s3_max_single_part_upload_size; UInt64 max_connections = context->getSettingsRef().s3_max_connections; + Int64 s3_max_single_read_retries = context->getSettingsRef().s3_max_single_read_retries; StoragePtr storage = StorageS3::create( s3_uri, @@ -93,6 +94,7 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, Context secret_access_key, StorageID(getDatabaseName(), table_name), format, + s3_max_single_read_retries, min_upload_part_size, max_single_part_upload_size, max_connections, From cf903d526c3731b2aa7dd732bf5ceb0056982fa5 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Tue, 13 Apr 2021 12:09:03 +0300 Subject: [PATCH 11/58] Better logging. --- src/IO/ReadBufferFromS3.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index fd3b46a50f9..f77a9dfb2be 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -63,6 +63,9 @@ bool ReadBufferFromS3::nextImpl() { ProfileEvents::increment(ProfileEvents::S3ReadRequestsErrors, 1); + LOG_INFO(log, "Caught exception while reading S3 object. Bucket: {}, Key: {}, Offset: {}, Remaining attempts: {}, Message: {}", + bucket, key, getPosition(), attempt, e.message()); + impl.reset(); offset = getPosition(); From 817c5cec255df4e509c84f0ab59d752d6790a8b8 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Tue, 13 Apr 2021 22:11:58 +0300 Subject: [PATCH 12/58] Correct S3 read retries. --- src/IO/ReadBufferFromS3.cpp | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index f77a9dfb2be..07d1246b370 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -12,6 +12,7 @@ # include + namespace ProfileEvents { extern const Event S3ReadMicroseconds; @@ -40,9 +41,11 @@ ReadBufferFromS3::ReadBufferFromS3( { } - bool ReadBufferFromS3::nextImpl() { + /// Restoring valid value of `count()` during `nextImpl()`. See `ReadBuffer::next()`. + pos = working_buffer.begin(); + if (!impl) impl = initialize(); @@ -57,6 +60,10 @@ bool ReadBufferFromS3::nextImpl() try { next_result = impl->next(); + /// FIXME. 1. Poco `istream` cannot read less than buffer_size or this state is being discarded during + /// istream <-> iostream conversion. `gcount` always contains 0, + /// that's why we always have error "Cannot read from istream at offset 0". + break; } catch (const Exception & e) @@ -67,7 +74,6 @@ bool ReadBufferFromS3::nextImpl() bucket, key, getPosition(), attempt, e.message()); impl.reset(); - offset = getPosition(); if (!attempt) throw; @@ -102,7 +108,6 @@ off_t ReadBufferFromS3::seek(off_t offset_, int whence) return offset; } - off_t ReadBufferFromS3::getPosition() { return offset + count(); @@ -110,13 +115,13 @@ off_t ReadBufferFromS3::getPosition() std::unique_ptr ReadBufferFromS3::initialize() { - LOG_TRACE(log, "Read S3 object. Bucket: {}, Key: {}, Offset: {}", bucket, key, std::to_string(offset)); + LOG_TRACE(log, "Read S3 object. Bucket: {}, Key: {}, Offset: {}", bucket, key, getPosition()); Aws::S3::Model::GetObjectRequest req; req.SetBucket(bucket); req.SetKey(key); - if (offset != 0) - req.SetRange("bytes=" + std::to_string(offset) + "-"); + if (getPosition()) + req.SetRange("bytes=" + std::to_string(getPosition()) + "-"); Aws::S3::Model::GetObjectOutcome outcome = client_ptr->GetObject(req); From f59d35efd8fb38d7e856fb95d330d59a1aaa308b Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 15 Apr 2021 10:53:00 +0300 Subject: [PATCH 13/58] Rebase fix. --- src/Storages/StorageS3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 84d6b7a3b4b..c2632362fc2 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -199,7 +199,7 @@ bool StorageS3Source::initialize() file_path = bucket + "/" + current_key; read_buf = wrapReadBufferWithCompressionMethod( - std::make_unique(client, bucket, current_key), chooseCompressionMethod(current_key, compression_hint)); + std::make_unique(client, bucket, current_key, s3_max_single_read_retries), chooseCompressionMethod(current_key, compression_hint)); auto input_format = FormatFactory::instance().getInput(format, *read_buf, sample_block, getContext(), max_block_size); reader = std::make_shared(input_format); From 95866e6e8d634adcaad1f7c8ec34e426f63b232e Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 15 Apr 2021 10:57:08 +0300 Subject: [PATCH 14/58] Rebase fix. --- src/Storages/StorageS3.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 506f8eac076..6a7c09608bf 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -55,6 +55,7 @@ public: ContextPtr context_, const ColumnsDescription & columns_, UInt64 max_block_size_, + Int64 s3_max_single_read_retries_, const String compression_hint_, const std::shared_ptr & client_, const String & bucket, @@ -71,6 +72,7 @@ private: String format; ColumnsDescription columns_desc; UInt64 max_block_size; + Int64 s3_max_single_read_retries; String compression_hint; std::shared_ptr client; Block sample_block; From f3e152739b20e32a183c4060d5efff91ba355f51 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 15 Apr 2021 14:00:59 +0300 Subject: [PATCH 15/58] Rebase fix. --- src/TableFunctions/TableFunctionS3Cluster.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp index 26ef07ef97f..9d8539347a3 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ b/src/TableFunctions/TableFunctionS3Cluster.cpp @@ -112,9 +112,14 @@ StoragePtr TableFunctionS3Cluster::executeImpl( UInt64 min_upload_part_size = context->getSettingsRef().s3_min_upload_part_size; UInt64 max_single_part_upload_size = context->getSettingsRef().s3_max_single_part_upload_size; UInt64 max_connections = context->getSettingsRef().s3_max_connections; + Int64 s3_max_single_read_retries = context->getSettingsRef().s3_max_single_read_retries; storage = StorageS3::create( s3_uri, access_key_id, secret_access_key, StorageID(getDatabaseName(), table_name), - format, min_upload_part_size, max_single_part_upload_size, max_connections, + format, + s3_max_single_read_retries, + min_upload_part_size, + max_single_part_upload_size, + max_connections, getActualTableStructure(context), ConstraintsDescription{}, context, compression_method, /*distributed_processing=*/true); } From 801a5608962e504cf9b11064228125398622ca79 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sat, 17 Apr 2021 20:04:32 +0300 Subject: [PATCH 16/58] Add aliases MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Добавил синонимы к функциям. --- .../sql-reference/functions/json-functions.md | 30 ++++++++++++++----- 1 file changed, 22 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/functions/json-functions.md b/docs/en/sql-reference/functions/json-functions.md index ca6ef684faf..d545a0ae4e6 100644 --- a/docs/en/sql-reference/functions/json-functions.md +++ b/docs/en/sql-reference/functions/json-functions.md @@ -16,46 +16,60 @@ The following assumptions are made: ## visitParamHas(params, name) {#visitparamhasparams-name} -Checks whether there is a field with the ‘name’ name. +Checks whether there is a field with the `name` name. + +Alias: `simpleJSONHas`. ## visitParamExtractUInt(params, name) {#visitparamextractuintparams-name} -Parses UInt64 from the value of the field named ‘name’. If this is a string field, it tries to parse a number from the beginning of the string. If the field doesn’t exist, or it exists but doesn’t contain a number, it returns 0. +Parses UInt64 from the value of the field named `name`. If this is a string field, it tries to parse a number from the beginning of the string. If the field doesn’t exist, or it exists but doesn’t contain a number, it returns 0. + +Alias: `simpleJSONExtractUInt`. ## visitParamExtractInt(params, name) {#visitparamextractintparams-name} The same as for Int64. +Alias: `simpleJSONExtractInt`. + ## visitParamExtractFloat(params, name) {#visitparamextractfloatparams-name} The same as for Float64. +Alias: `simpleJSONExtractFloat`. + ## visitParamExtractBool(params, name) {#visitparamextractboolparams-name} Parses a true/false value. The result is UInt8. +Alias: `simpleJSONExtractBool`. + ## visitParamExtractRaw(params, name) {#visitparamextractrawparams-name} Returns the value of a field, including separators. +Alias: `simpleJSONExtractRaw`. + Examples: ``` sql -visitParamExtractRaw('{"abc":"\\n\\u0000"}', 'abc') = '"\\n\\u0000"' -visitParamExtractRaw('{"abc":{"def":[1,2,3]}}', 'abc') = '{"def":[1,2,3]}' +visitParamExtractRaw('{"abc":"\\n\\u0000"}', 'abc') = '"\\n\\u0000"'; +visitParamExtractRaw('{"abc":{"def":[1,2,3]}}', 'abc') = '{"def":[1,2,3]}'; ``` ## visitParamExtractString(params, name) {#visitparamextractstringparams-name} Parses the string in double quotes. The value is unescaped. If unescaping failed, it returns an empty string. +Alias: `simpleJSONExtractString`. + Examples: ``` sql -visitParamExtractString('{"abc":"\\n\\u0000"}', 'abc') = '\n\0' -visitParamExtractString('{"abc":"\\u263a"}', 'abc') = '☺' -visitParamExtractString('{"abc":"\\u263"}', 'abc') = '' -visitParamExtractString('{"abc":"hello}', 'abc') = '' +visitParamExtractString('{"abc":"\\n\\u0000"}', 'abc') = '\n\0'; +visitParamExtractString('{"abc":"\\u263a"}', 'abc') = '☺'; +visitParamExtractString('{"abc":"\\u263"}', 'abc') = ''; +visitParamExtractString('{"abc":"hello}', 'abc') = ''; ``` There is currently no support for code points in the format `\uXXXX\uYYYY` that are not from the basic multilingual plane (they are converted to CESU-8 instead of UTF-8). From 4e22cdbfc40eae3d161b478dd675f64f5a9b92c0 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sat, 17 Apr 2021 21:01:21 +0300 Subject: [PATCH 17/58] Update column.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Добавил информацию об удалении столбца в материализованном представлении. --- docs/en/sql-reference/statements/alter/column.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index 3ece30be5b8..23a0a5ebfbb 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -74,6 +74,8 @@ Deletes the column with the name `name`. If the `IF EXISTS` clause is specified, Deletes data from the file system. Since this deletes entire files, the query is completed almost instantly. +**NOTE**: You can't delete a column if it is used in a [Materialized View](../../../sql-reference/statements/create/view.md/#materialized). Otherwise, it returns an error. + Example: ``` sql @@ -180,7 +182,7 @@ ALTER TABLE table_name MODIFY column_name REMOVE property; ALTER TABLE table_with_ttl MODIFY COLUMN column_ttl REMOVE TTL; ``` -## See Also +**See Also** - [REMOVE TTL](ttl.md). From 200925c3186cef7608d975141748e524784d9e53 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sat, 17 Apr 2021 21:18:18 +0300 Subject: [PATCH 18/58] Update column.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поправил формулировку. --- docs/en/sql-reference/statements/alter/column.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index 23a0a5ebfbb..0568e700d3f 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -74,7 +74,7 @@ Deletes the column with the name `name`. If the `IF EXISTS` clause is specified, Deletes data from the file system. Since this deletes entire files, the query is completed almost instantly. -**NOTE**: You can't delete a column if it is used in a [Materialized View](../../../sql-reference/statements/create/view.md/#materialized). Otherwise, it returns an error. +**NOTE**: You can't delete a column if it's referenced by [Materialized View](../../../sql-reference/statements/create/view.md/#materialized). Otherwise, it returns an error. Example: From e7a044c4c2a689720e162b6c499a0e073a13e8d1 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sat, 17 Apr 2021 22:52:27 +0300 Subject: [PATCH 19/58] Fix link MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Исправил ссылку. --- docs/en/sql-reference/statements/alter/column.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index 0568e700d3f..ec65d2ffb14 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -74,7 +74,7 @@ Deletes the column with the name `name`. If the `IF EXISTS` clause is specified, Deletes data from the file system. Since this deletes entire files, the query is completed almost instantly. -**NOTE**: You can't delete a column if it's referenced by [Materialized View](../../../sql-reference/statements/create/view.md/#materialized). Otherwise, it returns an error. +**NOTE**: You can't delete a column if it's referenced by [materialized view](../../../sql-reference/statements/create/view.md#materialized). Otherwise, it returns an error. Example: From d12db2374f3e03cdfaea56c58b4b119697b3d113 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sun, 18 Apr 2021 00:42:36 +0300 Subject: [PATCH 20/58] Fix note MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поправил примечание. --- docs/en/sql-reference/statements/alter/column.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index ec65d2ffb14..884150d9b7b 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -74,7 +74,8 @@ Deletes the column with the name `name`. If the `IF EXISTS` clause is specified, Deletes data from the file system. Since this deletes entire files, the query is completed almost instantly. -**NOTE**: You can't delete a column if it's referenced by [materialized view](../../../sql-reference/statements/create/view.md#materialized). Otherwise, it returns an error. +!!! info "Note" + You can’t delete a column if it is referenced by [materialized view](../../../sql-reference/statements/create/view.md#materialized). Otherwise, it returns an error. Example: From 24319b3cdd3bdd4e3f89c66677edbb5d5e9a827d Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sun, 18 Apr 2021 01:12:33 +0300 Subject: [PATCH 21/58] Add aliases to russian text MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Выполнил перевод на русский язык. --- .../sql-reference/functions/json-functions.md | 30 ++++++++++++++----- 1 file changed, 22 insertions(+), 8 deletions(-) diff --git a/docs/ru/sql-reference/functions/json-functions.md b/docs/ru/sql-reference/functions/json-functions.md index 5d419d26981..8f6dacfed68 100644 --- a/docs/ru/sql-reference/functions/json-functions.md +++ b/docs/ru/sql-reference/functions/json-functions.md @@ -16,46 +16,60 @@ toc_title: JSON ## visitParamHas(params, name) {#visitparamhasparams-name} -Проверить наличие поля с именем name. +Проверить наличие поля с именем `name`. + +Алиас: `simpleJSONHas`. ## visitParamExtractUInt(params, name) {#visitparamextractuintparams-name} -Распарсить UInt64 из значения поля с именем name. Если поле строковое - попытаться распарсить число из начала строки. Если такого поля нет, или если оно есть, но содержит не число, то вернуть 0. +Распарсить UInt64 из значения поля с именем `name`. Если поле строковое - попытаться распарсить число из начала строки. Если такого поля нет, или если оно есть, но содержит не число, то вернуть 0. + +Алиас: `simpleJSONExtractUInt`. ## visitParamExtractInt(params, name) {#visitparamextractintparams-name} Аналогично для Int64. +Алиас: `simpleJSONExtractInt`. + ## visitParamExtractFloat(params, name) {#visitparamextractfloatparams-name} Аналогично для Float64. +Алиас: `simpleJSONExtractFloat`. + ## visitParamExtractBool(params, name) {#visitparamextractboolparams-name} Распарсить значение true/false. Результат - UInt8. +Алиас: `simpleJSONExtractBool`. + ## visitParamExtractRaw(params, name) {#visitparamextractrawparams-name} Вернуть значение поля, включая разделители. +Алиас: `simpleJSONExtractRaw`. + Примеры: ``` sql -visitParamExtractRaw('{"abc":"\\n\\u0000"}', 'abc') = '"\\n\\u0000"' -visitParamExtractRaw('{"abc":{"def":[1,2,3]}}', 'abc') = '{"def":[1,2,3]}' +visitParamExtractRaw('{"abc":"\\n\\u0000"}', 'abc') = '"\\n\\u0000"'; +visitParamExtractRaw('{"abc":{"def":[1,2,3]}}', 'abc') = '{"def":[1,2,3]}'; ``` ## visitParamExtractString(params, name) {#visitparamextractstringparams-name} Распарсить строку в двойных кавычках. У значения убирается экранирование. Если убрать экранированные символы не удалось, то возвращается пустая строка. +Алиас: `simpleJSONExtractString`. + Примеры: ``` sql -visitParamExtractString('{"abc":"\\n\\u0000"}', 'abc') = '\n\0' -visitParamExtractString('{"abc":"\\u263a"}', 'abc') = '☺' -visitParamExtractString('{"abc":"\\u263"}', 'abc') = '' -visitParamExtractString('{"abc":"hello}', 'abc') = '' +visitParamExtractString('{"abc":"\\n\\u0000"}', 'abc') = '\n\0'; +visitParamExtractString('{"abc":"\\u263a"}', 'abc') = '☺'; +visitParamExtractString('{"abc":"\\u263"}', 'abc') = ''; +visitParamExtractString('{"abc":"hello}', 'abc') = ''; ``` На данный момент, не поддерживаются записанные в формате `\uXXXX\uYYYY` кодовые точки не из basic multilingual plane (они переводятся не в UTF-8, а в CESU-8). From 8e573c41e636bdddc39263fc65f5baf3d831a25b Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sun, 18 Apr 2021 01:47:43 +0300 Subject: [PATCH 22/58] Start --- docs/en/operations/system-tables/columns.md | 8 +- docs/en/operations/system-tables/tables.md | 54 +++++----- docs/ru/operations/system-tables/columns.md | 47 ++++++++- docs/ru/operations/system-tables/tables.md | 99 ++++++++++++++----- .../sql-reference/statements/create/table.md | 2 +- 5 files changed, 157 insertions(+), 53 deletions(-) diff --git a/docs/en/operations/system-tables/columns.md b/docs/en/operations/system-tables/columns.md index 92a6315d06b..0084a391390 100644 --- a/docs/en/operations/system-tables/columns.md +++ b/docs/en/operations/system-tables/columns.md @@ -4,7 +4,9 @@ Contains information about columns in all the tables. You can use this table to get information similar to the [DESCRIBE TABLE](../../sql-reference/statements/misc.md#misc-describe-table) query, but for multiple tables at once. -The `system.columns` table contains the following columns (the column type is shown in brackets): +Columns from [temporary tables](../../sql-reference/statements/create/table.md#temporary-tables) are visible in the `system.columns` only in those session where they have been created. They are shown with the empty `database` field. + +Columns: - `database` ([String](../../sql-reference/data-types/string.md)) — Database name. - `table` ([String](../../sql-reference/data-types/string.md)) — Table name. @@ -26,7 +28,7 @@ The `system.columns` table contains the following columns (the column type is sh **Example** ```sql -:) select * from system.columns LIMIT 2 FORMAT Vertical; +select * from system.columns LIMIT 2 FORMAT Vertical; ``` ```text @@ -65,8 +67,6 @@ is_in_sorting_key: 0 is_in_primary_key: 0 is_in_sampling_key: 0 compression_codec: - -2 rows in set. Elapsed: 0.002 sec. ``` [Original article](https://clickhouse.tech/docs/en/operations/system_tables/columns) diff --git a/docs/en/operations/system-tables/tables.md b/docs/en/operations/system-tables/tables.md index 6ad1425e032..bfc3050c057 100644 --- a/docs/en/operations/system-tables/tables.md +++ b/docs/en/operations/system-tables/tables.md @@ -1,59 +1,65 @@ # system.tables {#system-tables} -Contains metadata of each table that the server knows about. Detached tables are not shown in `system.tables`. +Contains metadata of each table that the server knows about. -This table contains the following columns (the column type is shown in brackets): +Detached tables are not shown in `system.tables`. -- `database` (String) — The name of the database the table is in. +[Temporary tables](../../sql-reference/statements/create/table.md#temporary-tables) are visible in the `system.tables` only in those session where they have been created. They are shown with the empty `database` field and with the `is_temporary` flag switched on. -- `name` (String) — Table name. +Columns: -- `engine` (String) — Table engine name (without parameters). +- `database` ([String](../../sql-reference/data-types/string.md)) — The name of the database the table is in. -- `is_temporary` (UInt8) - Flag that indicates whether the table is temporary. +- `name` ([String](../../sql-reference/data-types/string.md)) — Table name. -- `data_path` (String) - Path to the table data in the file system. +- `engine` ([String](../../sql-reference/data-types/string.md)) — Table engine name (without parameters). -- `metadata_path` (String) - Path to the table metadata in the file system. +- `is_temporary` ([UInt8](../../sql-reference/data-types/int-uint.md)) - Flag that indicates whether the table is temporary. -- `metadata_modification_time` (DateTime) - Time of latest modification of the table metadata. +- `data_path` ([String](../../sql-reference/data-types/string.md)) - Path to the table data in the file system. -- `dependencies_database` (Array(String)) - Database dependencies. +- `metadata_path` ([String](../../sql-reference/data-types/string.md)) - Path to the table metadata in the file system. -- `dependencies_table` (Array(String)) - Table dependencies ([MaterializedView](../../engines/table-engines/special/materializedview.md) tables based on the current table). +- `metadata_modification_time` ([DateTime](../../sql-reference/data-types/datetime.md)) - Time of latest modification of the table metadata. -- `create_table_query` (String) - The query that was used to create the table. +- `dependencies_database` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) - Database dependencies. -- `engine_full` (String) - Parameters of the table engine. +- `dependencies_table` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) - Table dependencies ([MaterializedView](../../engines/table-engines/special/materializedview.md) tables based on the current table). -- `partition_key` (String) - The partition key expression specified in the table. +- `create_table_query` ([String](../../sql-reference/data-types/string.md)) - The query that was used to create the table. -- `sorting_key` (String) - The sorting key expression specified in the table. +- `engine_full` ([String](../../sql-reference/data-types/string.md)) - Parameters of the table engine. -- `primary_key` (String) - The primary key expression specified in the table. +- `partition_key` ([String](../../sql-reference/data-types/string.md)) - The partition key expression specified in the table. -- `sampling_key` (String) - The sampling key expression specified in the table. +- `sorting_key` ([String](../../sql-reference/data-types/string.md)) - The sorting key expression specified in the table. -- `storage_policy` (String) - The storage policy: +- `primary_key` ([String](../../sql-reference/data-types/string.md)) - The primary key expression specified in the table. + +- `sampling_key` ([String](../../sql-reference/data-types/string.md)) - The sampling key expression specified in the table. + +- `storage_policy` ([String](../../sql-reference/data-types/string.md)) - The storage policy: - [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) - [Distributed](../../engines/table-engines/special/distributed.md#distributed) -- `total_rows` (Nullable(UInt64)) - Total number of rows, if it is possible to quickly determine exact number of rows in the table, otherwise `Null` (including underying `Buffer` table). +- `total_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - Total number of rows, if it is possible to quickly determine exact number of rows in the table, otherwise `Null` (including underying `Buffer` table). -- `total_bytes` (Nullable(UInt64)) - Total number of bytes, if it is possible to quickly determine exact number of bytes for the table on storage, otherwise `Null` (**does not** includes any underlying storage). +- `total_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - Total number of bytes, if it is possible to quickly determine exact number of bytes for the table on storage, otherwise `Null` (**does not** includes any underlying storage). - If the table stores data on disk, returns used space on disk (i.e. compressed). - If the table stores data in memory, returns approximated number of used bytes in memory. -- `lifetime_rows` (Nullable(UInt64)) - Total number of rows INSERTed since server start (only for `Buffer` tables). +- `lifetime_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - Total number of rows INSERTed since server start (only for `Buffer` tables). -- `lifetime_bytes` (Nullable(UInt64)) - Total number of bytes INSERTed since server start (only for `Buffer` tables). +- `lifetime_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - Total number of bytes INSERTed since server start (only for `Buffer` tables). The `system.tables` table is used in `SHOW TABLES` query implementation. +**Example** + ```sql -:) SELECT * FROM system.tables LIMIT 2 FORMAT Vertical; +SELECT * FROM system.tables LIMIT 2 FORMAT Vertical; ``` ```text @@ -100,8 +106,6 @@ sampling_key: storage_policy: total_rows: ᴺᵁᴸᴸ total_bytes: ᴺᵁᴸᴸ - -2 rows in set. Elapsed: 0.004 sec. ``` [Original article](https://clickhouse.tech/docs/en/operations/system_tables/tables) diff --git a/docs/ru/operations/system-tables/columns.md b/docs/ru/operations/system-tables/columns.md index af4cff85439..71a21860234 100644 --- a/docs/ru/operations/system-tables/columns.md +++ b/docs/ru/operations/system-tables/columns.md @@ -4,7 +4,9 @@ С помощью этой таблицы можно получить информацию аналогично запросу [DESCRIBE TABLE](../../sql-reference/statements/misc.md#misc-describe-table), но для многих таблиц сразу. -Таблица `system.columns` содержит столбцы (тип столбца указан в скобках): +Колонки [временных таблиц](../../sql-reference/statements/create/table.md#temporary-tables) содержатся в `system.columns` только в тех сессиях, в которых эти таблицы были созданы. Поле `database` у таких колонок пустое. + +Cтолбцы: - `database` ([String](../../sql-reference/data-types/string.md)) — имя базы данных. - `table` ([String](../../sql-reference/data-types/string.md)) — имя таблицы. @@ -23,3 +25,46 @@ - `is_in_sampling_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — флаг, показывающий включение столбца в ключ выборки. - `compression_codec` ([String](../../sql-reference/data-types/string.md)) — имя кодека сжатия. +**Пример** + +```sql +select * from system.columns LIMIT 2 FORMAT Vertical; +``` + +```text +Row 1: +────── +database: system +table: aggregate_function_combinators +name: name +type: String +default_kind: +default_expression: +data_compressed_bytes: 0 +data_uncompressed_bytes: 0 +marks_bytes: 0 +comment: +is_in_partition_key: 0 +is_in_sorting_key: 0 +is_in_primary_key: 0 +is_in_sampling_key: 0 +compression_codec: + +Row 2: +────── +database: system +table: aggregate_function_combinators +name: is_internal +type: UInt8 +default_kind: +default_expression: +data_compressed_bytes: 0 +data_uncompressed_bytes: 0 +marks_bytes: 0 +comment: +is_in_partition_key: 0 +is_in_sorting_key: 0 +is_in_primary_key: 0 +is_in_sampling_key: 0 +compression_codec: +``` \ No newline at end of file diff --git a/docs/ru/operations/system-tables/tables.md b/docs/ru/operations/system-tables/tables.md index 42e55b1f6b7..fb034119d41 100644 --- a/docs/ru/operations/system-tables/tables.md +++ b/docs/ru/operations/system-tables/tables.md @@ -1,39 +1,94 @@ # system.tables {#system-tables} -Содержит метаданные каждой таблицы, о которой знает сервер. Отсоединённые таблицы не отображаются в `system.tables`. +Содержит метаданные каждой таблицы, о которой знает сервер. -Эта таблица содержит следующие столбцы (тип столбца показан в скобках): +Отсоединённые таблицы не отображаются в `system.tables`. -- `database String` — имя базы данных, в которой находится таблица. -- `name` (String) — имя таблицы. -- `engine` (String) — движок таблицы (без параметров). -- `is_temporary` (UInt8) — флаг, указывающий на то, временная это таблица или нет. -- `data_path` (String) — путь к данным таблицы в файловой системе. -- `metadata_path` (String) — путь к табличным метаданным в файловой системе. -- `metadata_modification_time` (DateTime) — время последней модификации табличных метаданных. -- `dependencies_database` (Array(String)) — зависимости базы данных. -- `dependencies_table` (Array(String)) — табличные зависимости (таблицы [MaterializedView](../../engines/table-engines/special/materializedview.md), созданные на базе текущей таблицы). -- `create_table_query` (String) — запрос, которым создавалась таблица. -- `engine_full` (String) — параметры табличного движка. -- `partition_key` (String) — ключ партиционирования таблицы. -- `sorting_key` (String) — ключ сортировки таблицы. -- `primary_key` (String) - первичный ключ таблицы. -- `sampling_key` (String) — ключ сэмплирования таблицы. -- `storage_policy` (String) - политика хранения данных: +Информация о [временных таблицах](../../sql-reference/statements/create/table.md#temporary-tables) содержится в `system.tables` только в тех сессиях, в которых эти таблицы были созданы. Поле `database` у таких таблиц пустое, а флаг `is_temporary` включен. + +Столбцы: + +- `database` ([String](../../sql-reference/data-types/string.md)) — имя базы данных, в которой находится таблица. +- `name` ([String](../../sql-reference/data-types/string.md)) — имя таблицы. +- `engine` ([String](../../sql-reference/data-types/string.md)) — движок таблицы (без параметров). +- `is_temporary` ([UInt8](../../sql-reference/data-types/int-uint.md)) — флаг, указывающий на то, временная это таблица или нет. +- `data_path` ([String](../../sql-reference/data-types/string.md)) — путь к данным таблицы в файловой системе. +- `metadata_path` ([String](../../sql-reference/data-types/string.md)) — путь к табличным метаданным в файловой системе. +- `metadata_modification_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — время последней модификации табличных метаданных. +- `dependencies_database` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — зависимости базы данных. +- `dependencies_table` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — табличные зависимости (таблицы [MaterializedView](../../engines/table-engines/special/materializedview.md), созданные на базе текущей таблицы). +- `create_table_query` ([String](../../sql-reference/data-types/string.md)) — запрос, которым создавалась таблица. +- `engine_full` ([String](../../sql-reference/data-types/string.md)) — параметры табличного движка. +- `partition_key` ([String](../../sql-reference/data-types/string.md)) — ключ партиционирования таблицы. +- `sorting_key` ([String](../../sql-reference/data-types/string.md)) — ключ сортировки таблицы. +- `primary_key` ([String](../../sql-reference/data-types/string.md)) - первичный ключ таблицы. +- `sampling_key` ([String](../../sql-reference/data-types/string.md)) — ключ сэмплирования таблицы. +- `storage_policy` ([String](../../sql-reference/data-types/string.md)) - политика хранения данных: - [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) - [Distributed](../../engines/table-engines/special/distributed.md#distributed) -- `total_rows` (Nullable(UInt64)) - общее количество строк, если есть возможность быстро определить точное количество строк в таблице, в противном случае `Null` (включая базовую таблицу `Buffer`). +- `total_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - общее количество строк, если есть возможность быстро определить точное количество строк в таблице, в противном случае `Null` (включая базовую таблицу `Buffer`). -- `total_bytes` (Nullable(UInt64)) - общее количество байт, если можно быстро определить точное количество байт для таблицы на накопителе, в противном случае `Null` (**не включает** в себя никакого базового хранилища). +- `total_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - общее количество байт, если можно быстро определить точное количество байт для таблицы на накопителе, в противном случае `Null` (**не включает** в себя никакого базового хранилища). - Если таблица хранит данные на диске, возвращает используемое пространство на диске (т. е. сжатое). - Если таблица хранит данные в памяти, возвращает приблизительное количество используемых байт в памяти. -- `lifetime_rows` (Nullable(UInt64)) - общее количество строк, добавленных оператором `INSERT` с момента запуска сервера (только для таблиц `Buffer`). +- `lifetime_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - общее количество строк, добавленных оператором `INSERT` с момента запуска сервера (только для таблиц `Buffer`). -- `lifetime_bytes` (Nullable(UInt64)) - общее количество байт, добавленных оператором `INSERT` с момента запуска сервера (только для таблиц `Buffer`). +- `lifetime_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - общее количество байт, добавленных оператором `INSERT` с момента запуска сервера (только для таблиц `Buffer`). Таблица `system.tables` используется при выполнении запроса `SHOW TABLES`. +**Пример** + +```sql +SELECT * FROM system.tables LIMIT 2 FORMAT Vertical; +``` + +```text +Row 1: +────── +database: system +name: aggregate_function_combinators +uuid: 00000000-0000-0000-0000-000000000000 +engine: SystemAggregateFunctionCombinators +is_temporary: 0 +data_paths: [] +metadata_path: /var/lib/clickhouse/metadata/system/aggregate_function_combinators.sql +metadata_modification_time: 1970-01-01 03:00:00 +dependencies_database: [] +dependencies_table: [] +create_table_query: +engine_full: +partition_key: +sorting_key: +primary_key: +sampling_key: +storage_policy: +total_rows: ᴺᵁᴸᴸ +total_bytes: ᴺᵁᴸᴸ + +Row 2: +────── +database: system +name: asynchronous_metrics +uuid: 00000000-0000-0000-0000-000000000000 +engine: SystemAsynchronousMetrics +is_temporary: 0 +data_paths: [] +metadata_path: /var/lib/clickhouse/metadata/system/asynchronous_metrics.sql +metadata_modification_time: 1970-01-01 03:00:00 +dependencies_database: [] +dependencies_table: [] +create_table_query: +engine_full: +partition_key: +sorting_key: +primary_key: +sampling_key: +storage_policy: +total_rows: ᴺᵁᴸᴸ +total_bytes: ᴺᵁᴸᴸ +``` \ No newline at end of file diff --git a/docs/ru/sql-reference/statements/create/table.md b/docs/ru/sql-reference/statements/create/table.md index b998435bcd8..8768049596e 100644 --- a/docs/ru/sql-reference/statements/create/table.md +++ b/docs/ru/sql-reference/statements/create/table.md @@ -230,7 +230,7 @@ CREATE TABLE codec_example ) ENGINE = MergeTree() ``` -## Временные таблицы {#vremennye-tablitsy} +## Временные таблицы {#temporary-tables} ClickHouse поддерживает временные таблицы со следующими характеристиками: From 310632d602323b59e5824d5bcc7427d0388cf122 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sun, 18 Apr 2021 02:05:32 +0300 Subject: [PATCH 23/58] One more link added --- docs/en/operations/system-tables/tables.md | 2 +- docs/ru/operations/system-tables/tables.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/system-tables/tables.md b/docs/en/operations/system-tables/tables.md index bfc3050c057..5f515b63fbf 100644 --- a/docs/en/operations/system-tables/tables.md +++ b/docs/en/operations/system-tables/tables.md @@ -2,7 +2,7 @@ Contains metadata of each table that the server knows about. -Detached tables are not shown in `system.tables`. +[Detached](../../sql-reference/statements/detach.md) tables are not shown in `system.tables`. [Temporary tables](../../sql-reference/statements/create/table.md#temporary-tables) are visible in the `system.tables` only in those session where they have been created. They are shown with the empty `database` field and with the `is_temporary` flag switched on. diff --git a/docs/ru/operations/system-tables/tables.md b/docs/ru/operations/system-tables/tables.md index fb034119d41..a69e7214b96 100644 --- a/docs/ru/operations/system-tables/tables.md +++ b/docs/ru/operations/system-tables/tables.md @@ -2,7 +2,7 @@ Содержит метаданные каждой таблицы, о которой знает сервер. -Отсоединённые таблицы не отображаются в `system.tables`. +Отсоединённые таблицы ([DETACH](../../sql-reference/statements/detach.md)) не отображаются в `system.tables`. Информация о [временных таблицах](../../sql-reference/statements/create/table.md#temporary-tables) содержится в `system.tables` только в тех сессиях, в которых эти таблицы были созданы. Поле `database` у таких таблиц пустое, а флаг `is_temporary` включен. From f4f5e5c09e562433a91d170e202fe430e91bd60f Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sun, 18 Apr 2021 02:28:47 +0300 Subject: [PATCH 24/58] Start --- docs/en/guides/apply-catboost-model.md | 3 +++ docs/ru/guides/apply-catboost-model.md | 4 +++- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/docs/en/guides/apply-catboost-model.md b/docs/en/guides/apply-catboost-model.md index f614b121714..7c2c8a575ec 100644 --- a/docs/en/guides/apply-catboost-model.md +++ b/docs/en/guides/apply-catboost-model.md @@ -159,6 +159,9 @@ The fastest way to evaluate a CatBoost model is compile `libcatboostmodel./home/catboost/models/*_model.xml ``` +!!! note "Note" + You can change path to the CatBoost model configuration later without restarting server. + ## 4. Run the Model Inference from SQL {#run-model-inference} For test model run the ClickHouse client `$ clickhouse client`. diff --git a/docs/ru/guides/apply-catboost-model.md b/docs/ru/guides/apply-catboost-model.md index 11964c57fc7..d8c6f41f339 100644 --- a/docs/ru/guides/apply-catboost-model.md +++ b/docs/ru/guides/apply-catboost-model.md @@ -158,7 +158,9 @@ FROM amazon_train /home/catboost/data/libcatboostmodel.so /home/catboost/models/*_model.xml ``` - +!!! note "Примечание" + Вы можете позднее изменить путь к CatBoost и конфигурации модели без перезагрузки сервера. + ## 4. Запустите вывод модели из SQL {#run-model-inference} Для тестирования модели запустите клиент ClickHouse `$ clickhouse client`. From 3f156b83c983eb55fe3c6f2bd4990afdacceaaee Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 18 Apr 2021 09:07:09 +0300 Subject: [PATCH 25/58] Add some comments in OptimizeShardingKeyRewriteInVisitor --- .../OptimizeShardingKeyRewriteInVisitor.cpp | 24 ++++++++++++------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp index 4d1c0526910..a031b8dc962 100644 --- a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp +++ b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp @@ -13,7 +13,7 @@ using namespace DB; Field executeFunctionOnField( const Field & field, const std::string & name, - const ExpressionActionsPtr & expr, + const ExpressionActionsPtr & sharding_expr, const std::string & sharding_key_column_name) { DataTypePtr type = applyVisitor(FieldToDataType{}, field); @@ -25,17 +25,23 @@ Field executeFunctionOnField( Block block{column}; size_t num_rows = 1; - expr->execute(block, num_rows); + sharding_expr->execute(block, num_rows); ColumnWithTypeAndName & ret = block.getByName(sharding_key_column_name); return (*ret.column)[0]; } -/// Return true if shard may contain such value (or it is unknown), otherwise false. +/// @param sharding_column_value - one of values from IN +/// @param sharding_column_name - name of that column +/// @param sharding_expr - expression of sharding_key for the Distributed() table +/// @param sharding_key_column_name - name of the column for sharding_expr +/// @param shard_info - info for the current shard (to compare shard_num with calculated) +/// @param slots - weight -> shard mapping +/// @return true if shard may contain such value (or it is unknown), otherwise false. bool shardContains( const Field & sharding_column_value, const std::string & sharding_column_name, - const ExpressionActionsPtr & expr, + const ExpressionActionsPtr & sharding_expr, const std::string & sharding_key_column_name, const Cluster::ShardInfo & shard_info, const Cluster::SlotToShard & slots) @@ -45,7 +51,7 @@ bool shardContains( if (sharding_column_value.isNull()) return false; - Field sharding_value = executeFunctionOnField(sharding_column_value, sharding_column_name, expr, sharding_key_column_name); + Field sharding_value = executeFunctionOnField(sharding_column_value, sharding_column_name, sharding_expr, sharding_key_column_name); UInt64 value = sharding_value.get(); const auto shard_num = slots[value % slots.size()] + 1; return shard_info.shard_num == shard_num; @@ -78,10 +84,10 @@ void OptimizeShardingKeyRewriteInMatcher::visit(ASTFunction & function, Data & d if (!identifier) return; - const auto & expr = data.sharding_key_expr; + const auto & sharding_expr = data.sharding_key_expr; const auto & sharding_key_column_name = data.sharding_key_column_name; - if (!expr->getRequiredColumnsWithTypes().contains(identifier->name())) + if (!sharding_expr->getRequiredColumnsWithTypes().contains(identifier->name())) return; /// NOTE: that we should not take care about empty tuple, @@ -93,7 +99,7 @@ void OptimizeShardingKeyRewriteInMatcher::visit(ASTFunction & function, Data & d std::erase_if(tuple_elements->children, [&](auto & child) { auto * literal = child->template as(); - return literal && !shardContains(literal->value, identifier->name(), expr, sharding_key_column_name, data.shard_info, data.slots); + return literal && !shardContains(literal->value, identifier->name(), sharding_expr, sharding_key_column_name, data.shard_info, data.slots); }); } else if (auto * tuple_literal = right->as(); @@ -102,7 +108,7 @@ void OptimizeShardingKeyRewriteInMatcher::visit(ASTFunction & function, Data & d auto & tuple = tuple_literal->value.get(); std::erase_if(tuple, [&](auto & child) { - return !shardContains(child, identifier->name(), expr, sharding_key_column_name, data.shard_info, data.slots); + return !shardContains(child, identifier->name(), sharding_expr, sharding_key_column_name, data.shard_info, data.slots); }); } } From 154da6c1db9236a00d494e9e530dd2ca8242b796 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sun, 18 Apr 2021 13:21:54 +0300 Subject: [PATCH 26/58] Translate to Russian MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Перевел на русский язык. --- docs/en/sql-reference/statements/alter/column.md | 2 +- docs/ru/sql-reference/statements/alter/column.md | 5 ++++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index 884150d9b7b..d661bd4cd59 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -74,7 +74,7 @@ Deletes the column with the name `name`. If the `IF EXISTS` clause is specified, Deletes data from the file system. Since this deletes entire files, the query is completed almost instantly. -!!! info "Note" +!!! warning "Warning" You can’t delete a column if it is referenced by [materialized view](../../../sql-reference/statements/create/view.md#materialized). Otherwise, it returns an error. Example: diff --git a/docs/ru/sql-reference/statements/alter/column.md b/docs/ru/sql-reference/statements/alter/column.md index 87fc1c78cd0..158ab2e7385 100644 --- a/docs/ru/sql-reference/statements/alter/column.md +++ b/docs/ru/sql-reference/statements/alter/column.md @@ -63,6 +63,9 @@ DROP COLUMN [IF EXISTS] name Запрос удаляет данные из файловой системы. Так как это представляет собой удаление целых файлов, запрос выполняется почти мгновенно. +!!! warning "Предупреждение" + Вы не можете удалить столбец, используемый в [материализованном представлениии](../../../sql-reference/statements/create/view.md#materialized). В противном случае будет ошибка. + Пример: ``` sql @@ -155,7 +158,7 @@ ALTER TABLE table_name MODIFY column_name REMOVE property; ALTER TABLE table_with_ttl MODIFY COLUMN column_ttl REMOVE TTL; ``` -## Смотрите также +**Смотрите также** - [REMOVE TTL](ttl.md). From d844b7bb2feecb13edd376655c64e1ec7c844d5b Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sun, 18 Apr 2021 18:23:01 +0300 Subject: [PATCH 27/58] Update json-functions.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Внес небольшие поправки в формулировки. --- docs/ru/sql-reference/functions/json-functions.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/ru/sql-reference/functions/json-functions.md b/docs/ru/sql-reference/functions/json-functions.md index 8f6dacfed68..1043aca7615 100644 --- a/docs/ru/sql-reference/functions/json-functions.md +++ b/docs/ru/sql-reference/functions/json-functions.md @@ -16,13 +16,13 @@ toc_title: JSON ## visitParamHas(params, name) {#visitparamhasparams-name} -Проверить наличие поля с именем `name`. +Проверяет наличие поля с именем `name`. Алиас: `simpleJSONHas`. ## visitParamExtractUInt(params, name) {#visitparamextractuintparams-name} -Распарсить UInt64 из значения поля с именем `name`. Если поле строковое - попытаться распарсить число из начала строки. Если такого поля нет, или если оно есть, но содержит не число, то вернуть 0. +Разбирает UInt64 из значения поля с именем `name`. Если поле строковое, пытается разобрать число из начала строки. Если такого поля нет, или если оно есть, но содержит не число, то вернуть 0. Алиас: `simpleJSONExtractUInt`. @@ -40,13 +40,13 @@ toc_title: JSON ## visitParamExtractBool(params, name) {#visitparamextractboolparams-name} -Распарсить значение true/false. Результат - UInt8. +Разбирает значение true/false. Результат — UInt8. Алиас: `simpleJSONExtractBool`. ## visitParamExtractRaw(params, name) {#visitparamextractrawparams-name} -Вернуть значение поля, включая разделители. +Возвращает значение поля, включая разделители. Алиас: `simpleJSONExtractRaw`. @@ -59,7 +59,7 @@ visitParamExtractRaw('{"abc":{"def":[1,2,3]}}', 'abc') = '{"def":[1,2,3]}'; ## visitParamExtractString(params, name) {#visitparamextractstringparams-name} -Распарсить строку в двойных кавычках. У значения убирается экранирование. Если убрать экранированные символы не удалось, то возвращается пустая строка. +Разбирает строку в двойных кавычках. У значения убирается экранирование. Если убрать экранированные символы не удалось, то возвращается пустая строка. Алиас: `simpleJSONExtractString`. @@ -72,9 +72,9 @@ visitParamExtractString('{"abc":"\\u263"}', 'abc') = ''; visitParamExtractString('{"abc":"hello}', 'abc') = ''; ``` -На данный момент, не поддерживаются записанные в формате `\uXXXX\uYYYY` кодовые точки не из basic multilingual plane (они переводятся не в UTF-8, а в CESU-8). +На данный момент не поддерживаются записанные в формате `\uXXXX\uYYYY` кодовые точки не из basic multilingual plane (они переводятся не в UTF-8, а в CESU-8). -Следующие функции используют [simdjson](https://github.com/lemire/simdjson) который разработан под более сложные требования для разбора JSON. Упомянутое выше предположение 2 по-прежнему применимо. +Следующие функции используют [simdjson](https://github.com/lemire/simdjson), который разработан под более сложные требования для разбора JSON. Упомянутое выше предположение 2 по-прежнему применимо. ## isValidJSON(json) {#isvalidjsonjson} From bc5d953ef664e95c6be22988b77e4fe1de22ffb8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 18 Apr 2021 09:43:13 +0300 Subject: [PATCH 28/58] Report an error if jemalloc.background_thread was requested --- programs/server/CMakeLists.txt | 1 + programs/server/Server.cpp | 28 ++++++++++++++++++++++++++++ 2 files changed, 29 insertions(+) diff --git a/programs/server/CMakeLists.txt b/programs/server/CMakeLists.txt index 697851b294b..3a04228942b 100644 --- a/programs/server/CMakeLists.txt +++ b/programs/server/CMakeLists.txt @@ -19,6 +19,7 @@ set (CLICKHOUSE_SERVER_LINK clickhouse_storages_system clickhouse_table_functions string_utils + jemalloc ${LINK_RESOURCE_LIB} diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index e3b4316079c..e874122250c 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -101,6 +101,10 @@ # include #endif +#if USE_JEMALLOC +# include +#endif + namespace CurrentMetrics { extern const Metric Revision; @@ -109,11 +113,35 @@ namespace CurrentMetrics extern const Metric MaxDDLEntryID; } +#if USE_JEMALLOC +static bool jemallocOptionEnabled(const char *name) +{ + bool value; + size_t size = sizeof(value); + + if (mallctl(name, reinterpret_cast(&value), &size, /* newp= */ nullptr, /* newlen= */ 0)) + throw Poco::SystemException("mallctl() failed"); + + return value; +} +#else +static bool jemallocOptionEnabled(const char *) { return 0; } +#endif + int mainEntryClickHouseServer(int argc, char ** argv) { DB::Server app; + if (jemallocOptionEnabled("opt.background_thread")) + { + LOG_ERROR(&app.logger(), + "jemalloc.background_thread was requested, " + "however ClickHouse uses percpu_arena and background_thread most likely will not give any benefits, " + "and also background_thread is not compatible with ClickHouse watchdog " + "(that can be disabled with CLICKHOUSE_WATCHDOG_ENABLE=0)"); + } + /// Do not fork separate process from watchdog if we attached to terminal. /// Otherwise it breaks gdb usage. /// Can be overridden by environment variable (cannot use server config at this moment). From 2561a67fd82a9d5e8d7ad22cd01ef9b39a49fe15 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 18 Apr 2021 12:17:02 +0300 Subject: [PATCH 29/58] Replace !__clang__ with !defined(__clang) to fix gcc builds $ gg 'if !__clang__' | cut -d: -f1 | sort -u | xargs sed -i 's/#if !__clang__/#if !defined(__clang__)/g' --- base/common/DateLUTImpl.h | 4 ++-- src/AggregateFunctions/AggregateFunctionUniqUpTo.h | 4 ++-- src/AggregateFunctions/ReservoirSamplerDeterministic.h | 4 ++-- src/Columns/ColumnString.h | 4 ++-- src/Common/Allocator.h | 4 ++-- src/Common/UInt128.h | 8 ++++---- src/Common/tests/compact_array.cpp | 4 ++-- src/Common/tests/parallel_aggregation.cpp | 4 ++-- src/Common/tests/parallel_aggregation2.cpp | 8 ++++---- src/Core/Field.h | 8 ++++---- src/Core/Types.h | 4 ++-- src/Disks/tests/gtest_disk.cpp | 2 +- src/Functions/GatherUtils/Sources.h | 4 ++-- src/Functions/PolygonUtils.h | 6 +++--- src/IO/parseDateTimeBestEffort.cpp | 4 ++-- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 ++-- src/Storages/tests/gtest_storage_log.cpp | 2 +- 17 files changed, 39 insertions(+), 39 deletions(-) diff --git a/base/common/DateLUTImpl.h b/base/common/DateLUTImpl.h index 363f281584e..9e60181e802 100644 --- a/base/common/DateLUTImpl.h +++ b/base/common/DateLUTImpl.h @@ -25,7 +25,7 @@ #if defined(__PPC__) -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic ignored "-Wmaybe-uninitialized" #endif #endif @@ -1266,7 +1266,7 @@ public: }; #if defined(__PPC__) -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic pop #endif #endif diff --git a/src/AggregateFunctions/AggregateFunctionUniqUpTo.h b/src/AggregateFunctions/AggregateFunctionUniqUpTo.h index 88f26ab312e..853e1cb6447 100644 --- a/src/AggregateFunctions/AggregateFunctionUniqUpTo.h +++ b/src/AggregateFunctions/AggregateFunctionUniqUpTo.h @@ -17,7 +17,7 @@ #include -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Warray-bounds" #endif @@ -280,7 +280,7 @@ public: } -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic pop #endif diff --git a/src/AggregateFunctions/ReservoirSamplerDeterministic.h b/src/AggregateFunctions/ReservoirSamplerDeterministic.h index 5cf97ae0f85..9c62160b964 100644 --- a/src/AggregateFunctions/ReservoirSamplerDeterministic.h +++ b/src/AggregateFunctions/ReservoirSamplerDeterministic.h @@ -163,7 +163,7 @@ public: sorted = false; } -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wclass-memaccess" #endif @@ -191,7 +191,7 @@ public: } } -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic pop #endif diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index 1792491c60d..0814ebaa826 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -111,7 +111,7 @@ public: } /// Suppress gcc 7.3.1 warning: '*((void*)& +8)' may be used uninitialized in this function -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wmaybe-uninitialized" #endif @@ -128,7 +128,7 @@ public: offsets.push_back(new_size); } -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic pop #endif diff --git a/src/Common/Allocator.h b/src/Common/Allocator.h index e3c6ddf9ff4..ebfd654d558 100644 --- a/src/Common/Allocator.h +++ b/src/Common/Allocator.h @@ -277,7 +277,7 @@ private: * GCC 4.9 mistakenly assumes that we can call `free` from a pointer to the stack. * In fact, the combination of conditions inside AllocatorWithStackMemory does not allow this. */ -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wfree-nonheap-object" #endif @@ -359,6 +359,6 @@ extern template class Allocator; extern template class Allocator; extern template class Allocator; -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic pop #endif diff --git a/src/Common/UInt128.h b/src/Common/UInt128.h index 06fddee8dc9..be96f409673 100644 --- a/src/Common/UInt128.h +++ b/src/Common/UInt128.h @@ -19,7 +19,7 @@ namespace DB struct UInt128 { /// Suppress gcc7 warnings: 'prev_key.DB::UInt128::low' may be used uninitialized in this function -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wmaybe-uninitialized" #endif @@ -92,7 +92,7 @@ struct UInt128 return static_cast(low); } -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic pop #endif @@ -150,7 +150,7 @@ struct DummyUInt256 { /// Suppress gcc7 warnings: 'prev_key.DB::UInt256::a' may be used uninitialized in this function -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wmaybe-uninitialized" #endif @@ -179,7 +179,7 @@ struct DummyUInt256 bool operator== (const UInt64 rhs) const { return a == rhs && b == 0 && c == 0 && d == 0; } bool operator!= (const UInt64 rhs) const { return !operator==(rhs); } -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic pop #endif diff --git a/src/Common/tests/compact_array.cpp b/src/Common/tests/compact_array.cpp index a63859ac712..af6257e1963 100644 --- a/src/Common/tests/compact_array.cpp +++ b/src/Common/tests/compact_array.cpp @@ -1,5 +1,5 @@ /// Bug in GCC: https://gcc.gnu.org/bugzilla/show_bug.cgi?id=59124 -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Warray-bounds" #endif @@ -263,6 +263,6 @@ int main() return 0; } -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic pop #endif diff --git a/src/Common/tests/parallel_aggregation.cpp b/src/Common/tests/parallel_aggregation.cpp index e39be163619..045a385671b 100644 --- a/src/Common/tests/parallel_aggregation.cpp +++ b/src/Common/tests/parallel_aggregation.cpp @@ -69,7 +69,7 @@ static void aggregate1(Map & map, Source::const_iterator begin, Source::const_it ++map[*it]; } -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wmaybe-uninitialized" #endif @@ -122,7 +122,7 @@ static void aggregate22(MapTwoLevel & map, Source::const_iterator begin, Source: } } -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic pop #endif diff --git a/src/Common/tests/parallel_aggregation2.cpp b/src/Common/tests/parallel_aggregation2.cpp index 1f8e598b122..e2ad36232de 100644 --- a/src/Common/tests/parallel_aggregation2.cpp +++ b/src/Common/tests/parallel_aggregation2.cpp @@ -62,7 +62,7 @@ struct AggregateIndependent } }; -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wmaybe-uninitialized" #endif @@ -115,7 +115,7 @@ struct AggregateIndependentWithSequentialKeysOptimization } }; -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic pop #endif @@ -265,7 +265,7 @@ struct Creator void operator()(Value &) const {} }; -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wmaybe-uninitialized" #endif @@ -275,7 +275,7 @@ struct Updater void operator()(Value & x) const { ++x; } }; -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic pop #endif diff --git a/src/Core/Field.h b/src/Core/Field.h index 77549854982..5c4c2e165ad 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -96,7 +96,7 @@ template bool decimalEqual(T x, T y, UInt32 x_scale, UInt32 y_scale template bool decimalLess(T x, T y, UInt32 x_scale, UInt32 y_scale); template bool decimalLessOrEqual(T x, T y, UInt32 x_scale, UInt32 y_scale); -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wmaybe-uninitialized" #endif @@ -159,7 +159,7 @@ private: T dec; UInt32 scale; }; -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic pop #endif @@ -563,7 +563,7 @@ public: { case Types::Null: return f(field.template get()); // gcc 8.2.1 -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wmaybe-uninitialized" #endif @@ -583,7 +583,7 @@ public: case Types::Int128: return f(field.template get()); case Types::UInt256: return f(field.template get()); case Types::Int256: return f(field.template get()); -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic pop #endif } diff --git a/src/Core/Types.h b/src/Core/Types.h index a463455cbe1..b9ecda4a46d 100644 --- a/src/Core/Types.h +++ b/src/Core/Types.h @@ -15,7 +15,7 @@ namespace DB struct Null {}; /// Ignore strange gcc warning https://gcc.gnu.org/bugzilla/show_bug.cgi?id=55776 -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wshadow" #endif @@ -59,7 +59,7 @@ enum class TypeIndex LowCardinality, Map, }; -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic pop #endif diff --git a/src/Disks/tests/gtest_disk.cpp b/src/Disks/tests/gtest_disk.cpp index 525b5e6ce38..3b9dca63002 100644 --- a/src/Disks/tests/gtest_disk.cpp +++ b/src/Disks/tests/gtest_disk.cpp @@ -4,7 +4,7 @@ #include #include "gtest_disk.h" -#if !__clang__ +#if !defined(__clang__) # pragma GCC diagnostic push # pragma GCC diagnostic ignored "-Wsuggest-override" #endif diff --git a/src/Functions/GatherUtils/Sources.h b/src/Functions/GatherUtils/Sources.h index e3eb5f6df75..4dbaff9f567 100644 --- a/src/Functions/GatherUtils/Sources.h +++ b/src/Functions/GatherUtils/Sources.h @@ -140,7 +140,7 @@ struct NumericArraySource : public ArraySourceImpl> /// The methods can be virtual or not depending on the template parameter. See IStringSource. -#if !__clang__ +#if !defined(__clang__) # pragma GCC diagnostic push # pragma GCC diagnostic ignored "-Wsuggest-override" #elif __clang_major__ >= 11 @@ -233,7 +233,7 @@ struct ConstSource : public Base } }; -#if !__clang__ || __clang_major__ >= 11 +#if !defined(__clang__) || __clang_major__ >= 11 # pragma GCC diagnostic pop #endif diff --git a/src/Functions/PolygonUtils.h b/src/Functions/PolygonUtils.h index c8e96f0b63a..27ee1d2a99f 100644 --- a/src/Functions/PolygonUtils.h +++ b/src/Functions/PolygonUtils.h @@ -12,7 +12,7 @@ /// Warning in boost::geometry during template strategy substitution. #pragma GCC diagnostic push -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic ignored "-Wmaybe-uninitialized" #endif @@ -285,7 +285,7 @@ void PointInPolygonWithGrid::calcGridAttributes( const Point & max_corner = box.max_corner(); #pragma GCC diagnostic push -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic ignored "-Wmaybe-uninitialized" #endif @@ -322,7 +322,7 @@ void PointInPolygonWithGrid::buildGrid() for (size_t row = 0; row < grid_size; ++row) { #pragma GCC diagnostic push -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic ignored "-Wmaybe-uninitialized" #endif CoordinateType y_min = min_corner.y() + row * cell_height; diff --git a/src/IO/parseDateTimeBestEffort.cpp b/src/IO/parseDateTimeBestEffort.cpp index 26745a8f138..a7a9eaaa4dd 100644 --- a/src/IO/parseDateTimeBestEffort.cpp +++ b/src/IO/parseDateTimeBestEffort.cpp @@ -46,7 +46,7 @@ inline size_t readAlpha(char * res, size_t max_chars, ReadBuffer & in) } #if defined(__PPC__) -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic ignored "-Wmaybe-uninitialized" #endif #endif @@ -634,7 +634,7 @@ ReturnType parseDateTime64BestEffortImpl(DateTime64 & res, UInt32 scale, ReadBuf } #if defined(__PPC__) -#if !__clang__ +#if !defined(__clang__) #pragma GCC diagnostic pop #endif #endif diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 71a99aa2a87..36032f9208f 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1107,13 +1107,13 @@ void IMergeTreeDataPart::remove(bool keep_s3) const { /// Remove each expected file in directory, then remove directory itself. - #if !__clang__ + #if !defined(__clang__) # pragma GCC diagnostic push # pragma GCC diagnostic ignored "-Wunused-variable" #endif for (const auto & [file, _] : checksums.files) volume->getDisk()->removeSharedFile(to + "/" + file, keep_s3); - #if !__clang__ + #if !defined(__clang__) # pragma GCC diagnostic pop #endif diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index de2139a4a5a..41c1b6ac75a 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -19,7 +19,7 @@ #include #include -#if !__clang__ +#if !defined(__clang__) # pragma GCC diagnostic push # pragma GCC diagnostic ignored "-Wsuggest-override" #endif From 2fd78b7eac97cee33b5b8abcdb88231d6e27082a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 18 Apr 2021 10:01:05 +0300 Subject: [PATCH 30/58] Add -Wundef for gcc --- cmake/warnings.cmake | 1 + 1 file changed, 1 insertion(+) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index a398c59e981..a85fe8963c7 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -171,6 +171,7 @@ elseif (COMPILER_GCC) add_cxx_compile_options(-Wtrampolines) # Obvious add_cxx_compile_options(-Wunused) + add_cxx_compile_options(-Wundef) # Warn if vector operation is not implemented via SIMD capabilities of the architecture add_cxx_compile_options(-Wvector-operation-performance) # XXX: libstdc++ has some of these for 3way compare From d476a4a9bdacec244a8119f84aea7004d87ab1fa Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 17 Apr 2021 16:25:57 +0300 Subject: [PATCH 31/58] Add type conversion for optimize_skip_unused_shards_rewrite_in MSan reports [1]: 2021.04.17 15:20:06.665152 [ 56 ] {2336bf92-0269-4acd-8b3f-f09623223d18} executeQuery: (from [::1]:44744, using production parser) SELECT * FROM dist_01757 WHERE dummy IN ('255', 0) FORMAT Null ... 0 0x305af885 in (anonymous namespace)::shardContains() obj-x86_64-linux-gnu/../src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp:50:28 ... Uninitialized value was created by an allocation of 'sharding_value' in the stack frame of function '_ZN12_GLOBAL__N_113shardContainsERKN2DB5FieldERKNSt3__112basic_stringIcNS4_11char_traitsIcEENS4_9allocatorIcEEEERKNS4_10shared_ptrINS0_17ExpressionActionsEEESC_RKNS0_7Cluster9ShardInfoERKNS4_6vectorImNS8_ImEEEE' 0 0x305ae260 in (anonymous namespace)::shardContains() obj-x86_64-linux-gnu/../src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp:42 [1]: https://clickhouse-test-reports.s3.yandex.net/23212/d02eb3f6b7a8dc07bbed5605e1afcf7634436b3a/fuzzer_msan/report.html#fail1 P.S. fuzzers are great! --- .../OptimizeShardingKeyRewriteInVisitor.cpp | 8 ++++++++ ...optimize_skip_unused_shards_rewrite_in.reference | 3 +++ ...01756_optimize_skip_unused_shards_rewrite_in.sql | 13 ++++++++++++- 3 files changed, 23 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp index a031b8dc962..399def00006 100644 --- a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp +++ b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include namespace @@ -52,6 +53,13 @@ bool shardContains( return false; Field sharding_value = executeFunctionOnField(sharding_column_value, sharding_column_name, sharding_expr, sharding_key_column_name); + /// The value from IN can be non-numeric, + /// but in this case it should be convertible to numeric type, let's try. + sharding_value = convertFieldToType(sharding_value, DataTypeUInt64()); + /// In case of conversion is not possible (NULL), shard cannot contain the value anyway. + if (sharding_value.isNull()) + return false; + UInt64 value = sharding_value.get(); const auto shard_num = slots[value % slots.size()] + 1; return shard_info.shard_num == shard_num; diff --git a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference index 494e9ca3237..a1bfcf043da 100644 --- a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference +++ b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.reference @@ -17,6 +17,9 @@ others 0 0 0 +different types -- prohibited +different types -- conversion +0 optimize_skip_unused_shards_limit 0 0 diff --git a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.sql b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.sql index 59e2ad75fcc..dc481ccca72 100644 --- a/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.sql +++ b/tests/queries/0_stateless/01756_optimize_skip_unused_shards_rewrite_in.sql @@ -5,6 +5,7 @@ drop table if exists dist_01756; drop table if exists dist_01756_str; +drop table if exists dist_01756_column; drop table if exists data_01756_str; -- SELECT @@ -90,8 +91,10 @@ select * from dist_01756 where dummy in (0); -- { serverError 507 } -- optimize_skip_unused_shards does not support non-constants select * from dist_01756 where dummy in (select * from system.one); -- { serverError 507 } select * from dist_01756 where dummy in (toUInt8(0)); -- { serverError 507 } --- wrong type +-- wrong type (tuple) select * from dist_01756 where dummy in ('0'); -- { serverError 507 } +-- intHash64 does not accept string +select * from dist_01756 where dummy in ('0', '2'); -- { serverError 43 } -- NOT IN does not supported select * from dist_01756 where dummy not in (0, 2); -- { serverError 507 } @@ -110,6 +113,7 @@ select (2 IN (2,)), * from dist_01756 where dummy in (0, 2) format Null; select (dummy IN (toUInt8(2),)), * from dist_01756 where dummy in (0, 2) format Null; -- different type +select 'different types -- prohibited'; create table data_01756_str (key String) engine=Memory(); create table dist_01756_str as data_01756_str engine=Distributed(test_cluster_two_shards, currentDatabase(), data_01756_str, cityHash64(key)); select * from dist_01756_str where key in ('0', '2'); @@ -117,6 +121,12 @@ select * from dist_01756_str where key in ('0', Null); -- { serverError 507 } select * from dist_01756_str where key in (0, 2); -- { serverError 53 } select * from dist_01756_str where key in (0, Null); -- { serverError 53 } +-- different type #2 +select 'different types -- conversion'; +create table dist_01756_column as system.one engine=Distributed(test_cluster_two_shards, system, one, dummy); +select * from dist_01756_column where dummy in (0, '255'); +select * from dist_01756_column where dummy in (0, '255foo'); -- { serverError 53 } + -- optimize_skip_unused_shards_limit select 'optimize_skip_unused_shards_limit'; select * from dist_01756 where dummy in (0, 2) settings optimize_skip_unused_shards_limit=1; -- { serverError 507 } @@ -124,4 +134,5 @@ select * from dist_01756 where dummy in (0, 2) settings optimize_skip_unused_sha drop table dist_01756; drop table dist_01756_str; +drop table dist_01756_column; drop table data_01756_str; From 3030db9043b298ea491de4652e5115460a7b8039 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Apr 2021 01:28:03 +0300 Subject: [PATCH 32/58] Fix integration tests for Hedged requests --- tests/integration/test_hedged_requests/configs/users.xml | 2 ++ tests/integration/test_hedged_requests/configs/users1.xml | 2 ++ .../integration/test_hedged_requests_parallel/configs/users.xml | 2 ++ .../test_hedged_requests_parallel/configs/users1.xml | 2 ++ 4 files changed, 8 insertions(+) diff --git a/tests/integration/test_hedged_requests/configs/users.xml b/tests/integration/test_hedged_requests/configs/users.xml index a3ab176b811..ac42155a18a 100644 --- a/tests/integration/test_hedged_requests/configs/users.xml +++ b/tests/integration/test_hedged_requests/configs/users.xml @@ -5,6 +5,8 @@ in_order 100 2000 + 1 + 1 diff --git a/tests/integration/test_hedged_requests/configs/users1.xml b/tests/integration/test_hedged_requests/configs/users1.xml index 2a54396feca..bc68c2c579d 100644 --- a/tests/integration/test_hedged_requests/configs/users1.xml +++ b/tests/integration/test_hedged_requests/configs/users1.xml @@ -2,6 +2,8 @@ + 1 + 1 diff --git a/tests/integration/test_hedged_requests_parallel/configs/users.xml b/tests/integration/test_hedged_requests_parallel/configs/users.xml index 3f3578903b4..9600c0c7124 100644 --- a/tests/integration/test_hedged_requests_parallel/configs/users.xml +++ b/tests/integration/test_hedged_requests_parallel/configs/users.xml @@ -6,6 +6,8 @@ 2 100 2000 + 1 + 1 diff --git a/tests/integration/test_hedged_requests_parallel/configs/users1.xml b/tests/integration/test_hedged_requests_parallel/configs/users1.xml index 2a54396feca..bc68c2c579d 100644 --- a/tests/integration/test_hedged_requests_parallel/configs/users1.xml +++ b/tests/integration/test_hedged_requests_parallel/configs/users1.xml @@ -2,6 +2,8 @@ + 1 + 1 From 472c131420096f3a5c92d3ea90e3030ed5f25f2b Mon Sep 17 00:00:00 2001 From: TCeason Date: Sat, 10 Apr 2021 09:37:56 +0800 Subject: [PATCH 33/58] Add MySQL read history data bytes judgment --- src/Core/Settings.h | 2 + .../MySQL/DatabaseConnectionMySQL.cpp | 9 +- src/Databases/MySQL/DatabaseConnectionMySQL.h | 2 +- .../MySQL/FetchTablesColumnsList.cpp | 7 +- src/Databases/MySQL/FetchTablesColumnsList.h | 3 +- src/Databases/MySQL/MaterializeMetadata.cpp | 35 +++++--- src/Databases/MySQL/MaterializeMetadata.h | 4 +- .../MySQL/MaterializeMySQLSyncThread.cpp | 36 ++++---- src/Dictionaries/MySQLDictionarySource.cpp | 47 +++++----- src/Dictionaries/MySQLDictionarySource.h | 8 +- src/Formats/MySQLBlockInputStream.cpp | 89 ++++++++++++------- src/Formats/MySQLBlockInputStream.h | 33 ++++--- src/Storages/StorageMySQL.cpp | 5 +- src/TableFunctions/TableFunctionMySQL.cpp | 2 +- .../configs/users_disable_bytes_settings.xml | 21 +++++ .../configs/users_disable_rows_settings.xml | 21 +++++ .../materialize_with_ddl.py | 16 ++++ .../test_materialize_mysql_database/test.py | 12 ++- 18 files changed, 235 insertions(+), 117 deletions(-) create mode 100644 tests/integration/test_materialize_mysql_database/configs/users_disable_bytes_settings.xml create mode 100644 tests/integration/test_materialize_mysql_database/configs/users_disable_rows_settings.xml diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ff58f5d4e5f..a115d986e5c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -445,6 +445,8 @@ class IColumn; M(Bool, database_replicated_always_detach_permanently, false, "Execute DETACH TABLE as DETACH TABLE PERMANENTLY if database engine is Replicated", 0) \ M(DistributedDDLOutputMode, distributed_ddl_output_mode, DistributedDDLOutputMode::THROW, "Format of distributed DDL query result", 0) \ M(UInt64, distributed_ddl_entry_format_version, 1, "Version of DDL entry to write into ZooKeeper", 0) \ + M(UInt64, max_read_mysql_rows, DEFAULT_BLOCK_SIZE, "Limit maximum rows when MaterializeMySQL flush history data. 0 -> Disable it.", 0) \ + M(UInt64, max_read_mysql_bytes, (DEFAULT_INSERT_BLOCK_SIZE * 256), "Limit maximum bytes when MaterializeMySQL flush history data. 0 -> Disable it.", 0) \ \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp index 9615aa85e05..4e63bed9c6d 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp @@ -198,7 +198,7 @@ ASTPtr DatabaseConnectionMySQL::getCreateDatabaseQuery() const void DatabaseConnectionMySQL::fetchTablesIntoLocalCache(ContextPtr local_context) const { - const auto & tables_with_modification_time = fetchTablesWithModificationTime(); + const auto & tables_with_modification_time = fetchTablesWithModificationTime(local_context); destroyLocalCacheExtraTables(tables_with_modification_time); fetchLatestTablesStructureIntoCache(tables_with_modification_time, local_context); @@ -252,7 +252,7 @@ void DatabaseConnectionMySQL::fetchLatestTablesStructureIntoCache( } } -std::map DatabaseConnectionMySQL::fetchTablesWithModificationTime() const +std::map DatabaseConnectionMySQL::fetchTablesWithModificationTime(ContextPtr local_context) const { Block tables_status_sample_block { @@ -268,7 +268,8 @@ std::map DatabaseConnectionMySQL::fetchTablesWithModificationTim " WHERE TABLE_SCHEMA = " << quote << database_name_in_mysql; std::map tables_with_modification_time; - MySQLBlockInputStream result(mysql_pool.get(), query.str(), tables_status_sample_block, DEFAULT_BLOCK_SIZE); + StreamSettings mysql_input_stream_settings(local_context->getSettingsRef()); + MySQLBlockInputStream result(mysql_pool.get(), query.str(), tables_status_sample_block, mysql_input_stream_settings); while (Block block = result.read()) { @@ -292,7 +293,7 @@ DatabaseConnectionMySQL::fetchTablesColumnsList(const std::vector & tabl mysql_pool, database_name_in_mysql, tables_name, - settings.external_table_functions_use_nulls, + settings, database_settings->mysql_datatypes_support_level); } diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.h b/src/Databases/MySQL/DatabaseConnectionMySQL.h index a626924d6dd..7e81003e9a9 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.h +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.h @@ -108,7 +108,7 @@ private: void fetchTablesIntoLocalCache(ContextPtr context) const; - std::map fetchTablesWithModificationTime() const; + std::map fetchTablesWithModificationTime(ContextPtr local_context) const; std::map fetchTablesColumnsList(const std::vector & tables_name, ContextPtr context) const; diff --git a/src/Databases/MySQL/FetchTablesColumnsList.cpp b/src/Databases/MySQL/FetchTablesColumnsList.cpp index 6344553ba69..07e0e75dd9b 100644 --- a/src/Databases/MySQL/FetchTablesColumnsList.cpp +++ b/src/Databases/MySQL/FetchTablesColumnsList.cpp @@ -44,7 +44,7 @@ std::map fetchTablesColumnsList( mysqlxx::PoolWithFailover & pool, const String & database_name, const std::vector & tables_name, - bool external_table_functions_use_nulls, + const Settings & settings, MultiEnum type_support) { std::map tables_and_columns; @@ -78,7 +78,8 @@ std::map fetchTablesColumnsList( " WHERE TABLE_SCHEMA = " << quote << database_name << " AND TABLE_NAME IN " << toQueryStringWithQuote(tables_name) << " ORDER BY ORDINAL_POSITION"; - MySQLBlockInputStream result(pool.get(), query.str(), tables_columns_sample_block, DEFAULT_BLOCK_SIZE); + StreamSettings mysql_input_stream_settings(settings); + MySQLBlockInputStream result(pool.get(), query.str(), tables_columns_sample_block, mysql_input_stream_settings); while (Block block = result.read()) { const auto & table_name_col = *block.getByPosition(0).column; @@ -99,7 +100,7 @@ std::map fetchTablesColumnsList( convertMySQLDataType( type_support, column_type_col[i].safeGet(), - external_table_functions_use_nulls && is_nullable_col[i].safeGet(), + settings.external_table_functions_use_nulls && is_nullable_col[i].safeGet(), is_unsigned_col[i].safeGet(), char_max_length_col[i].safeGet(), precision_col[i].safeGet(), diff --git a/src/Databases/MySQL/FetchTablesColumnsList.h b/src/Databases/MySQL/FetchTablesColumnsList.h index d609f4896e7..55f18e0115f 100644 --- a/src/Databases/MySQL/FetchTablesColumnsList.h +++ b/src/Databases/MySQL/FetchTablesColumnsList.h @@ -12,6 +12,7 @@ #include #include +#include namespace DB { @@ -20,7 +21,7 @@ std::map fetchTablesColumnsList( mysqlxx::PoolWithFailover & pool, const String & database_name, const std::vector & tables_name, - bool external_table_functions_use_nulls, + const Settings & settings, MultiEnum type_support); } diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index f5e648903ed..c389ab5a1b0 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -24,7 +24,8 @@ namespace ErrorCodes } static std::unordered_map fetchTablesCreateQuery( - const mysqlxx::PoolWithFailover::Entry & connection, const String & database_name, const std::vector & fetch_tables) + const mysqlxx::PoolWithFailover::Entry & connection, const String & database_name, + const std::vector & fetch_tables, const Settings & global_settings) { std::unordered_map tables_create_query; for (const auto & fetch_table_name : fetch_tables) @@ -34,9 +35,10 @@ static std::unordered_map fetchTablesCreateQuery( {std::make_shared(), "Create Table"}, }; + StreamSettings mysql_input_stream_settings(global_settings, false, true); MySQLBlockInputStream show_create_table( connection, "SHOW CREATE TABLE " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(fetch_table_name), - show_create_table_header, DEFAULT_BLOCK_SIZE, false, true); + show_create_table_header, mysql_input_stream_settings); Block create_query_block = show_create_table.read(); if (!create_query_block || create_query_block.rows() != 1) @@ -49,13 +51,14 @@ static std::unordered_map fetchTablesCreateQuery( } -static std::vector fetchTablesInDB(const mysqlxx::PoolWithFailover::Entry & connection, const std::string & database) +static std::vector fetchTablesInDB(const mysqlxx::PoolWithFailover::Entry & connection, const std::string & database, const Settings & global_settings) { Block header{{std::make_shared(), "table_name"}}; String query = "SELECT TABLE_NAME AS table_name FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_TYPE != 'VIEW' AND TABLE_SCHEMA = " + quoteString(database); std::vector tables_in_db; - MySQLBlockInputStream input(connection, query, header, DEFAULT_BLOCK_SIZE); + StreamSettings mysql_input_stream_settings(global_settings); + MySQLBlockInputStream input(connection, query, header, mysql_input_stream_settings); while (Block block = input.read()) { @@ -77,7 +80,8 @@ void MaterializeMetadata::fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & c {std::make_shared(), "Executed_Gtid_Set"}, }; - MySQLBlockInputStream input(connection, "SHOW MASTER STATUS;", header, DEFAULT_BLOCK_SIZE, false, true); + StreamSettings mysql_input_stream_settings(settings, false, true); + MySQLBlockInputStream input(connection, "SHOW MASTER STATUS;", header, mysql_input_stream_settings); Block master_status = input.read(); if (!master_status || master_status.rows() != 1) @@ -99,7 +103,8 @@ void MaterializeMetadata::fetchMasterVariablesValue(const mysqlxx::PoolWithFailo }; const String & fetch_query = "SHOW VARIABLES WHERE Variable_name = 'binlog_checksum'"; - MySQLBlockInputStream variables_input(connection, fetch_query, variables_header, DEFAULT_BLOCK_SIZE, false, true); + StreamSettings mysql_input_stream_settings(settings, false, true); + MySQLBlockInputStream variables_input(connection, fetch_query, variables_header, mysql_input_stream_settings); while (Block variables_block = variables_input.read()) { @@ -114,7 +119,7 @@ void MaterializeMetadata::fetchMasterVariablesValue(const mysqlxx::PoolWithFailo } } -static bool checkSyncUserPrivImpl(const mysqlxx::PoolWithFailover::Entry & connection, WriteBuffer & out) +static bool checkSyncUserPrivImpl(const mysqlxx::PoolWithFailover::Entry & connection, const Settings & global_settings, WriteBuffer & out) { Block sync_user_privs_header { @@ -122,7 +127,8 @@ static bool checkSyncUserPrivImpl(const mysqlxx::PoolWithFailover::Entry & conne }; String grants_query, sub_privs; - MySQLBlockInputStream input(connection, "SHOW GRANTS FOR CURRENT_USER();", sync_user_privs_header, DEFAULT_BLOCK_SIZE); + StreamSettings mysql_input_stream_settings(global_settings); + MySQLBlockInputStream input(connection, "SHOW GRANTS FOR CURRENT_USER();", sync_user_privs_header, mysql_input_stream_settings); while (Block block = input.read()) { for (size_t index = 0; index < block.rows(); ++index) @@ -146,11 +152,11 @@ static bool checkSyncUserPrivImpl(const mysqlxx::PoolWithFailover::Entry & conne return false; } -static void checkSyncUserPriv(const mysqlxx::PoolWithFailover::Entry & connection) +static void checkSyncUserPriv(const mysqlxx::PoolWithFailover::Entry & connection, const Settings & global_settings) { WriteBufferFromOwnString out; - if (!checkSyncUserPrivImpl(connection, out)) + if (!checkSyncUserPrivImpl(connection, global_settings, out)) throw Exception("MySQL SYNC USER ACCESS ERR: mysql sync user needs " "at least GLOBAL PRIVILEGES:'RELOAD, REPLICATION SLAVE, REPLICATION CLIENT' " "and SELECT PRIVILEGE on MySQL Database." @@ -167,7 +173,8 @@ bool MaterializeMetadata::checkBinlogFileExists(const mysqlxx::PoolWithFailover: {std::make_shared(), "File_size"} }; - MySQLBlockInputStream input(connection, "SHOW MASTER LOGS", logs_header, DEFAULT_BLOCK_SIZE, false, true); + StreamSettings mysql_input_stream_settings(settings, false, true); + MySQLBlockInputStream input(connection, "SHOW MASTER LOGS", logs_header, mysql_input_stream_settings); while (Block block = input.read()) { @@ -222,7 +229,7 @@ void MaterializeMetadata::transaction(const MySQLReplication::Position & positio commitMetadata(std::move(fun), persistent_tmp_path, persistent_path); } -MaterializeMetadata::MaterializeMetadata(const String & path_) : persistent_path(path_) +MaterializeMetadata::MaterializeMetadata(const String & path_, const Settings & settings_) : persistent_path(path_), settings(settings_) { if (Poco::File(persistent_path).exists()) { @@ -244,7 +251,7 @@ void MaterializeMetadata::startReplication( mysqlxx::PoolWithFailover::Entry & connection, const String & database, bool & opened_transaction, std::unordered_map & need_dumping_tables) { - checkSyncUserPriv(connection); + checkSyncUserPriv(connection, settings); if (checkBinlogFileExists(connection)) return; @@ -263,7 +270,7 @@ void MaterializeMetadata::startReplication( connection->query("START TRANSACTION /*!40100 WITH CONSISTENT SNAPSHOT */;").execute(); opened_transaction = true; - need_dumping_tables = fetchTablesCreateQuery(connection, database, fetchTablesInDB(connection, database)); + need_dumping_tables = fetchTablesCreateQuery(connection, database, fetchTablesInDB(connection, database, settings), settings); connection->query("UNLOCK TABLES;").execute(); } catch (...) diff --git a/src/Databases/MySQL/MaterializeMetadata.h b/src/Databases/MySQL/MaterializeMetadata.h index 3a82d1349ba..079786c261c 100644 --- a/src/Databases/MySQL/MaterializeMetadata.h +++ b/src/Databases/MySQL/MaterializeMetadata.h @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB { @@ -25,6 +26,7 @@ namespace DB struct MaterializeMetadata { const String persistent_path; + const Settings settings; String binlog_file; UInt64 binlog_position; @@ -50,7 +52,7 @@ struct MaterializeMetadata bool & opened_transaction, std::unordered_map & need_dumping_tables); - MaterializeMetadata(const String & path_); + MaterializeMetadata(const String & path_, const Settings & settings_); }; } diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index b8e135eef32..82161ac5c8d 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -90,7 +90,7 @@ MaterializeMySQLSyncThread::~MaterializeMySQLSyncThread() } } -static void checkMySQLVariables(const mysqlxx::Pool::Entry & connection) +static void checkMySQLVariables(const mysqlxx::Pool::Entry & connection, const Settings & settings) { Block variables_header{ {std::make_shared(), "Variable_name"}, @@ -104,19 +104,19 @@ static void checkMySQLVariables(const mysqlxx::Pool::Entry & connection) "OR (Variable_name = 'default_authentication_plugin' AND upper(Value) = 'MYSQL_NATIVE_PASSWORD') " "OR (Variable_name = 'log_bin_use_v1_row_events' AND upper(Value) = 'OFF');"; - MySQLBlockInputStream variables_input(connection, check_query, variables_header, DEFAULT_BLOCK_SIZE, false, true); + StreamSettings mysql_input_stream_settings(settings, false, true); + MySQLBlockInputStream variables_input(connection, check_query, variables_header, mysql_input_stream_settings); - Block variables_block = variables_input.read(); - if (!variables_block || variables_block.rows() != 5) + std::unordered_map variables_error_message{ + {"log_bin", "log_bin = 'ON'"}, + {"binlog_format", "binlog_format='ROW'"}, + {"binlog_row_image", "binlog_row_image='FULL'"}, + {"default_authentication_plugin", "default_authentication_plugin='mysql_native_password'"}, + {"log_bin_use_v1_row_events", "log_bin_use_v1_row_events='OFF'"} + }; + + while (Block variables_block = variables_input.read()) { - std::unordered_map variables_error_message{ - {"log_bin", "log_bin = 'ON'"}, - {"binlog_format", "binlog_format='ROW'"}, - {"binlog_row_image", "binlog_row_image='FULL'"}, - {"default_authentication_plugin", "default_authentication_plugin='mysql_native_password'"}, - {"log_bin_use_v1_row_events", "log_bin_use_v1_row_events='OFF'"} - }; - ColumnPtr variable_name_column = variables_block.getByName("Variable_name").column; for (size_t index = 0; index < variables_block.rows(); ++index) @@ -126,7 +126,10 @@ static void checkMySQLVariables(const mysqlxx::Pool::Entry & connection) if (error_message_it != variables_error_message.end()) variables_error_message.erase(error_message_it); } + } + if (!variables_error_message.empty()) + { bool first = true; WriteBufferFromOwnString error_message; error_message << "Illegal MySQL variables, the MaterializeMySQL engine requires "; @@ -167,7 +170,7 @@ void MaterializeMySQLSyncThread::synchronization() try { MaterializeMetadata metadata( - DatabaseCatalog::instance().getDatabase(database_name)->getMetadataPath() + "/.metadata"); + DatabaseCatalog::instance().getDatabase(database_name)->getMetadataPath() + "/.metadata", getContext()->getSettingsRef()); bool need_reconnect = true; Stopwatch watch; @@ -240,7 +243,7 @@ void MaterializeMySQLSyncThread::assertMySQLAvailable() { try { - checkMySQLVariables(pool.get()); + checkMySQLVariables(pool.get(), getContext()->getSettingsRef()); } catch (const mysqlxx::ConnectionFailed & e) { @@ -326,9 +329,10 @@ static inline void dumpDataForTables( tryToExecuteQuery(query_prefix + " " + iterator->second, query_context, database_name, comment); /// create table. auto out = std::make_shared(getTableOutput(database_name, table_name, query_context)); + StreamSettings mysql_input_stream_settings(context->getSettingsRef()); MySQLBlockInputStream input( connection, "SELECT * FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name), - out->getHeader(), DEFAULT_BLOCK_SIZE); + out->getHeader(), mysql_input_stream_settings); Stopwatch watch; copyData(input, *out, is_cancelled); @@ -375,7 +379,7 @@ bool MaterializeMySQLSyncThread::prepareSynchronized(MaterializeMetadata & metad opened_transaction = false; - checkMySQLVariables(connection); + checkMySQLVariables(connection, getContext()->getSettingsRef()); std::unordered_map need_dumping_tables; metadata.startReplication(connection, mysql_database_name, opened_transaction, need_dumping_tables); diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index a78da1c7e74..676863ae588 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -4,9 +4,15 @@ #include "DictionarySourceFactory.h" #include "DictionaryStructure.h" #include "registerDictionaries.h" +#include +#include namespace DB { + +[[maybe_unused]] +static const size_t default_num_tries_on_connection_loss = 3; + namespace ErrorCodes { extern const int SUPPORT_IS_DISABLED; @@ -14,20 +20,20 @@ namespace ErrorCodes void registerDictionarySourceMysql(DictionarySourceFactory & factory) { - auto create_table_source = [=](const DictionaryStructure & dict_struct, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - Block & sample_block, - ContextPtr /* context */, + auto create_table_source = [=]([[maybe_unused]] const DictionaryStructure & dict_struct, + [[maybe_unused]] const Poco::Util::AbstractConfiguration & config, + [[maybe_unused]] const std::string & config_prefix, + [[maybe_unused]] Block & sample_block, + [[maybe_unused]] ContextPtr context, const std::string & /* default_database */, bool /* check_config */) -> DictionarySourcePtr { #if USE_MYSQL - return std::make_unique(dict_struct, config, config_prefix + ".mysql", sample_block); + StreamSettings mysql_input_stream_settings(context->getSettingsRef() + , config.getBool(config_prefix + ".mysql.close_connection", false) || config.getBool(config_prefix + ".mysql.share_connection", false) + , false + , config.getBool(config_prefix + ".mysql.fail_on_connection_loss", false) ? 1 : default_num_tries_on_connection_loss); + return std::make_unique(dict_struct, config, config_prefix + ".mysql", sample_block, mysql_input_stream_settings); #else - (void)dict_struct; - (void)config; - (void)config_prefix; - (void)sample_block; throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Dictionary source of type `mysql` is disabled because ClickHouse was built without mysql support."); #endif @@ -45,22 +51,21 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory) # include # include # include -# include # include "readInvalidateQuery.h" # include # include +# include namespace DB { -static const UInt64 max_block_size = 8192; -static const size_t default_num_tries_on_connection_loss = 3; MySQLDictionarySource::MySQLDictionarySource( const DictionaryStructure & dict_struct_, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - const Block & sample_block_) + const Block & sample_block_, + const StreamSettings & settings_) : log(&Poco::Logger::get("MySQLDictionarySource")) , update_time{std::chrono::system_clock::from_time_t(0)} , dict_struct{dict_struct_} @@ -74,10 +79,7 @@ MySQLDictionarySource::MySQLDictionarySource( , query_builder{dict_struct, db, "", table, where, IdentifierQuotingStyle::Backticks} , load_all_query{query_builder.composeLoadAllQuery()} , invalidate_query{config.getString(config_prefix + ".invalidate_query", "")} - , close_connection( - config.getBool(config_prefix + ".close_connection", false) || config.getBool(config_prefix + ".share_connection", false)) - , max_tries_for_mysql_block_input_stream( - config.getBool(config_prefix + ".fail_on_connection_loss", false) ? 1 : default_num_tries_on_connection_loss) + , settings(settings_) { } @@ -98,8 +100,7 @@ MySQLDictionarySource::MySQLDictionarySource(const MySQLDictionarySource & other , last_modification{other.last_modification} , invalidate_query{other.invalidate_query} , invalidate_query_response{other.invalidate_query_response} - , close_connection{other.close_connection} - , max_tries_for_mysql_block_input_stream{other.max_tries_for_mysql_block_input_stream} + , settings(other.settings) { } @@ -122,7 +123,7 @@ std::string MySQLDictionarySource::getUpdateFieldAndDate() BlockInputStreamPtr MySQLDictionarySource::loadFromQuery(const String & query) { return std::make_shared( - pool, query, sample_block, max_block_size, close_connection, false, max_tries_for_mysql_block_input_stream); + pool, query, sample_block, settings); } BlockInputStreamPtr MySQLDictionarySource::loadAll() @@ -245,7 +246,7 @@ LocalDateTime MySQLDictionarySource::getLastModification(mysqlxx::Pool::Entry & ++fetched_rows; } - if (close_connection && allow_connection_closure) + if (settings.auto_close && allow_connection_closure) { connection.disconnect(); } @@ -269,7 +270,7 @@ std::string MySQLDictionarySource::doInvalidateQuery(const std::string & request Block invalidate_sample_block; ColumnPtr column(ColumnString::create()); invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared(), "Sample Block")); - MySQLBlockInputStream block_input_stream(pool->get(), request, invalidate_sample_block, 1, close_connection); + MySQLBlockInputStream block_input_stream(pool->get(), request, invalidate_sample_block, settings); return readInvalidateQuery(block_input_stream); } diff --git a/src/Dictionaries/MySQLDictionarySource.h b/src/Dictionaries/MySQLDictionarySource.h index dcd911146aa..ef1d81b862f 100644 --- a/src/Dictionaries/MySQLDictionarySource.h +++ b/src/Dictionaries/MySQLDictionarySource.h @@ -12,7 +12,7 @@ # include "DictionaryStructure.h" # include "ExternalQueryBuilder.h" # include "IDictionarySource.h" - +# include namespace Poco { @@ -35,7 +35,8 @@ public: const DictionaryStructure & dict_struct_, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - const Block & sample_block_); + const Block & sample_block_, + const StreamSettings & settings_); /// copy-constructor is provided in order to support cloneability MySQLDictionarySource(const MySQLDictionarySource & other); @@ -87,8 +88,7 @@ private: LocalDateTime last_modification; std::string invalidate_query; mutable std::string invalidate_query_response; - const bool close_connection; - const size_t max_tries_for_mysql_block_input_stream; + const StreamSettings settings; }; } diff --git a/src/Formats/MySQLBlockInputStream.cpp b/src/Formats/MySQLBlockInputStream.cpp index e2620c71fbb..38261cbc329 100644 --- a/src/Formats/MySQLBlockInputStream.cpp +++ b/src/Formats/MySQLBlockInputStream.cpp @@ -30,6 +30,29 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } +StreamSettings::StreamSettings(const Settings & settings) +{ + max_read_mysql_rows = settings.max_read_mysql_rows; + max_read_bytes_size = settings.max_read_mysql_bytes; +} + +StreamSettings::StreamSettings(const Settings & settings, bool auto_close_, bool fetch_by_name_) + : auto_close(auto_close_) + , fetch_by_name(fetch_by_name_) +{ + max_read_mysql_rows = settings.max_read_mysql_rows; + max_read_bytes_size = settings.max_read_mysql_bytes; +} + +StreamSettings::StreamSettings(const Settings & settings, bool auto_close_, bool fetch_by_name_, size_t max_retry_) + : auto_close(auto_close_) + , fetch_by_name(fetch_by_name_) + , default_num_tries_on_connection_loss(max_retry_) +{ + max_read_mysql_rows = settings.max_read_mysql_rows; + max_read_bytes_size = settings.max_read_mysql_bytes; +} + MySQLBlockInputStream::Connection::Connection( const mysqlxx::PoolWithFailover::Entry & entry_, const std::string & query_str) @@ -44,29 +67,19 @@ MySQLBlockInputStream::MySQLBlockInputStream( const mysqlxx::PoolWithFailover::Entry & entry, const std::string & query_str, const Block & sample_block, - const UInt64 max_block_size_, - const bool auto_close_, - const bool fetch_by_name_) + const StreamSettings & settings_) : log(&Poco::Logger::get("MySQLBlockInputStream")) , connection{std::make_unique(entry, query_str)} - , max_block_size{max_block_size_} - , auto_close{auto_close_} - , fetch_by_name(fetch_by_name_) + , settings{std::make_unique(settings_)} { description.init(sample_block); initPositionMappingFromQueryResultStructure(); } /// For descendant MySQLWithFailoverBlockInputStream -MySQLBlockInputStream::MySQLBlockInputStream( - const Block & sample_block_, - UInt64 max_block_size_, - bool auto_close_, - bool fetch_by_name_) + MySQLBlockInputStream::MySQLBlockInputStream(const Block &sample_block_, const StreamSettings & settings_) : log(&Poco::Logger::get("MySQLBlockInputStream")) - , max_block_size(max_block_size_) - , auto_close(auto_close_) - , fetch_by_name(fetch_by_name_) + , settings(std::make_unique(settings_)) { description.init(sample_block_); } @@ -76,14 +89,10 @@ MySQLWithFailoverBlockInputStream::MySQLWithFailoverBlockInputStream( mysqlxx::PoolWithFailoverPtr pool_, const std::string & query_str_, const Block & sample_block_, - const UInt64 max_block_size_, - const bool auto_close_, - const bool fetch_by_name_, - const size_t max_tries_) - : MySQLBlockInputStream(sample_block_, max_block_size_, auto_close_, fetch_by_name_) - , pool(pool_) - , query_str(query_str_) - , max_tries(max_tries_) + const StreamSettings & settings_) +: MySQLBlockInputStream(sample_block_, settings_) +, pool(pool_) +, query_str(query_str_) { } @@ -101,12 +110,12 @@ void MySQLWithFailoverBlockInputStream::readPrefix() } catch (const mysqlxx::ConnectionLost & ecl) /// There are two retriable failures: CR_SERVER_GONE_ERROR, CR_SERVER_LOST { - LOG_WARNING(log, "Failed connection ({}/{}). Trying to reconnect... (Info: {})", count_connect_attempts, max_tries, ecl.displayText()); + LOG_WARNING(log, "Failed connection ({}/{}). Trying to reconnect... (Info: {})", count_connect_attempts, settings->default_num_tries_on_connection_loss, ecl.displayText()); } - if (++count_connect_attempts > max_tries) + if (++count_connect_attempts > settings->default_num_tries_on_connection_loss) { - LOG_ERROR(log, "Failed to create connection to MySQL. ({}/{})", count_connect_attempts, max_tries); + LOG_ERROR(log, "Failed to create connection to MySQL. ({}/{})", count_connect_attempts, settings->default_num_tries_on_connection_loss); throw; } } @@ -118,45 +127,57 @@ namespace { using ValueType = ExternalResultDescription::ValueType; - void insertValue(const IDataType & data_type, IColumn & column, const ValueType type, const mysqlxx::Value & value) + void insertValue(const IDataType & data_type, IColumn & column, const ValueType type, const mysqlxx::Value & value, size_t & read_bytes_size) { switch (type) { case ValueType::vtUInt8: assert_cast(column).insertValue(value.getUInt()); + read_bytes_size += 1; break; case ValueType::vtUInt16: assert_cast(column).insertValue(value.getUInt()); + read_bytes_size += 2; break; case ValueType::vtUInt32: assert_cast(column).insertValue(value.getUInt()); + read_bytes_size += 4; break; case ValueType::vtUInt64: assert_cast(column).insertValue(value.getUInt()); + read_bytes_size += 8; break; case ValueType::vtInt8: assert_cast(column).insertValue(value.getInt()); + read_bytes_size += 1; break; case ValueType::vtInt16: assert_cast(column).insertValue(value.getInt()); + read_bytes_size += 2; break; case ValueType::vtInt32: assert_cast(column).insertValue(value.getInt()); + read_bytes_size += 4; break; case ValueType::vtInt64: assert_cast(column).insertValue(value.getInt()); + read_bytes_size += 8; break; case ValueType::vtFloat32: assert_cast(column).insertValue(value.getDouble()); + read_bytes_size += 4; break; case ValueType::vtFloat64: assert_cast(column).insertValue(value.getDouble()); + read_bytes_size += 8; break; case ValueType::vtString: assert_cast(column).insertData(value.data(), value.size()); + read_bytes_size += assert_cast(column).byteSize(); break; case ValueType::vtDate: assert_cast(column).insertValue(UInt16(value.getDate().getDayNum())); + read_bytes_size += 2; break; case ValueType::vtDateTime: { @@ -166,10 +187,12 @@ namespace if (time < 0) time = 0; assert_cast(column).insertValue(time); + read_bytes_size += 4; break; } case ValueType::vtUUID: assert_cast(column).insert(parse(value.data(), value.size())); + read_bytes_size += assert_cast(column).byteSize(); break; case ValueType::vtDateTime64:[[fallthrough]]; case ValueType::vtDecimal32: [[fallthrough]]; @@ -179,10 +202,12 @@ namespace { ReadBuffer buffer(const_cast(value.data()), value.size(), 0); data_type.getDefaultSerialization()->deserializeWholeText(column, buffer, FormatSettings{}); + read_bytes_size += column.sizeOfValueIfFixed(); break; } case ValueType::vtFixedString: assert_cast(column).insertData(value.data(), value.size()); + read_bytes_size += column.sizeOfValueIfFixed(); break; default: throw Exception("Unsupported value type", ErrorCodes::NOT_IMPLEMENTED); @@ -198,7 +223,7 @@ Block MySQLBlockInputStream::readImpl() auto row = connection->result.fetch(); if (!row) { - if (auto_close) + if (settings->auto_close) connection->entry.disconnect(); return {}; @@ -209,6 +234,8 @@ Block MySQLBlockInputStream::readImpl() columns[i] = description.sample_block.getByPosition(i).column->cloneEmpty(); size_t num_rows = 0; + size_t read_bytes_size = 0; + while (row) { for (size_t index = 0; index < position_mapping.size(); ++index) @@ -224,12 +251,12 @@ Block MySQLBlockInputStream::readImpl() { ColumnNullable & column_nullable = assert_cast(*columns[index]); const auto & data_type = assert_cast(*sample.type); - insertValue(*data_type.getNestedType(), column_nullable.getNestedColumn(), description.types[index].first, value); + insertValue(*data_type.getNestedType(), column_nullable.getNestedColumn(), description.types[index].first, value, read_bytes_size); column_nullable.getNullMapData().emplace_back(false); } else { - insertValue(*sample.type, *columns[index], description.types[index].first, value); + insertValue(*sample.type, *columns[index], description.types[index].first, value, read_bytes_size); } } else @@ -245,7 +272,7 @@ Block MySQLBlockInputStream::readImpl() } ++num_rows; - if (num_rows == max_block_size) + if (num_rows == settings->max_read_mysql_rows || (settings->max_read_bytes_size && read_bytes_size >= settings->max_read_bytes_size)) break; row = connection->result.fetch(); @@ -257,7 +284,7 @@ void MySQLBlockInputStream::initPositionMappingFromQueryResultStructure() { position_mapping.resize(description.sample_block.columns()); - if (!fetch_by_name) + if (!settings->fetch_by_name) { if (description.sample_block.columns() != connection->result.getNumFields()) throw Exception{"mysqlxx::UseQueryResult contains " + toString(connection->result.getNumFields()) + " columns while " diff --git a/src/Formats/MySQLBlockInputStream.h b/src/Formats/MySQLBlockInputStream.h index 86380c1597a..f4c06629955 100644 --- a/src/Formats/MySQLBlockInputStream.h +++ b/src/Formats/MySQLBlockInputStream.h @@ -6,11 +6,25 @@ #include #include #include - +#include namespace DB { +struct StreamSettings +{ + size_t max_read_mysql_rows; + size_t max_read_bytes_size; + bool auto_close = false; + bool fetch_by_name = false; + size_t default_num_tries_on_connection_loss = 5; + + StreamSettings(const Settings & settings); + StreamSettings(const Settings & settings, bool auto_close_, bool fetch_by_name_); + StreamSettings(const Settings & settings, bool auto_close_, bool fetch_by_name_, size_t max_retry_); + +}; + /// Allows processing results of a MySQL query as a sequence of Blocks, simplifies chaining class MySQLBlockInputStream : public IBlockInputStream { @@ -19,16 +33,14 @@ public: const mysqlxx::PoolWithFailover::Entry & entry, const std::string & query_str, const Block & sample_block, - const UInt64 max_block_size_, - const bool auto_close_ = false, - const bool fetch_by_name_ = false); + const StreamSettings & settings_); String getName() const override { return "MySQL"; } Block getHeader() const override { return description.sample_block.cloneEmpty(); } protected: - MySQLBlockInputStream(const Block & sample_block_, UInt64 max_block_size_, bool auto_close_, bool fetch_by_name_); + MySQLBlockInputStream(const Block & sample_block_, const struct StreamSettings & settings); Block readImpl() override; void initPositionMappingFromQueryResultStructure(); @@ -44,9 +56,7 @@ protected: Poco::Logger * log; std::unique_ptr connection; - const UInt64 max_block_size; - const bool auto_close; - const bool fetch_by_name; + const std::unique_ptr settings; std::vector position_mapping; ExternalResultDescription description; }; @@ -57,23 +67,18 @@ protected: class MySQLWithFailoverBlockInputStream final : public MySQLBlockInputStream { public: - static constexpr inline auto MAX_TRIES_MYSQL_CONNECT = 5; MySQLWithFailoverBlockInputStream( mysqlxx::PoolWithFailoverPtr pool_, const std::string & query_str_, const Block & sample_block_, - const UInt64 max_block_size_, - const bool auto_close_ = false, - const bool fetch_by_name_ = false, - const size_t max_tries_ = MAX_TRIES_MYSQL_CONNECT); + const StreamSettings & settings_); private: void readPrefix() override; mysqlxx::PoolWithFailoverPtr pool; std::string query_str; - size_t max_tries; }; } diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index c635b0e939c..67fbcfc781e 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -71,7 +71,7 @@ Pipe StorageMySQL::read( SelectQueryInfo & query_info_, ContextPtr context_, QueryProcessingStage::Enum /*processed_stage*/, - size_t max_block_size_, + size_t /*max_read_mysql_rows*/, unsigned) { metadata_snapshot->check(column_names_, getVirtuals(), getStorageID()); @@ -95,8 +95,9 @@ Pipe StorageMySQL::read( sample_block.insert({ column_data.type, column_data.name }); } + StreamSettings mysql_input_stream_settings(context_->getSettingsRef(), true, false); return Pipe(std::make_shared( - std::make_shared(pool, query, sample_block, max_block_size_, /* auto_close = */ true))); + std::make_shared(pool, query, sample_block, mysql_input_stream_settings))); } diff --git a/src/TableFunctions/TableFunctionMySQL.cpp b/src/TableFunctions/TableFunctionMySQL.cpp index c7dda873a00..e2eb2511409 100644 --- a/src/TableFunctions/TableFunctionMySQL.cpp +++ b/src/TableFunctions/TableFunctionMySQL.cpp @@ -79,7 +79,7 @@ void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, ContextPtr ColumnsDescription TableFunctionMySQL::getActualTableStructure(ContextPtr context) const { const auto & settings = context->getSettingsRef(); - const auto tables_and_columns = fetchTablesColumnsList(*pool, remote_database_name, {remote_table_name}, settings.external_table_functions_use_nulls, settings.mysql_datatypes_support_level); + const auto tables_and_columns = fetchTablesColumnsList(*pool, remote_database_name, {remote_table_name}, settings, settings.mysql_datatypes_support_level); const auto columns = tables_and_columns.find(remote_table_name); if (columns == tables_and_columns.end()) diff --git a/tests/integration/test_materialize_mysql_database/configs/users_disable_bytes_settings.xml b/tests/integration/test_materialize_mysql_database/configs/users_disable_bytes_settings.xml new file mode 100644 index 00000000000..cb5a38a57c9 --- /dev/null +++ b/tests/integration/test_materialize_mysql_database/configs/users_disable_bytes_settings.xml @@ -0,0 +1,21 @@ + + + + + 1 + Atomic + 1 + 0 + + + + + + + + ::/0 + + default + + + diff --git a/tests/integration/test_materialize_mysql_database/configs/users_disable_rows_settings.xml b/tests/integration/test_materialize_mysql_database/configs/users_disable_rows_settings.xml new file mode 100644 index 00000000000..a245b8e978b --- /dev/null +++ b/tests/integration/test_materialize_mysql_database/configs/users_disable_rows_settings.xml @@ -0,0 +1,21 @@ + + + + + 1 + Atomic + 0 + 1 + + + + + + + + ::/0 + + default + + + diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 38574a81d0a..c80ee5833ef 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -842,3 +842,19 @@ def system_tables_test(clickhouse_node, mysql_node, service_name): mysql_node.query("CREATE TABLE system_tables_test.test (id int NOT NULL PRIMARY KEY) ENGINE=InnoDB") clickhouse_node.query("CREATE DATABASE system_tables_test ENGINE = MaterializeMySQL('{}:3306', 'system_tables_test', 'root', 'clickhouse')".format(service_name)) check_query(clickhouse_node, "SELECT partition_key, sorting_key, primary_key FROM system.tables WHERE database = 'system_tables_test' AND name = 'test'", "intDiv(id, 4294967)\tid\tid\n") + +def mysql_settings_test(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") + mysql_node.query("CREATE DATABASE test_database") + mysql_node.query("CREATE TABLE test_database.a (id INT(11) NOT NULL PRIMARY KEY, value VARCHAR(255))") + mysql_node.query("INSERT INTO test_database.a VALUES(1, 'foo')") + mysql_node.query("INSERT INTO test_database.a VALUES(2, 'bar')") + + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format(service_name)) + check_query(clickhouse_node, "SELECT COUNT() FROM test_database.a FORMAT TSV", "2\n") + + assert clickhouse_node.query("SELECT COUNT(DISTINCT blockNumber()) FROM test_database.a FORMAT TSV") == "2\n" + + clickhouse_node.query("DROP DATABASE test_database") + mysql_node.query("DROP DATABASE test_database") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 3c41c0a2177..0215dcf9b42 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -16,7 +16,8 @@ cluster = ClickHouseCluster(__file__) node_db_ordinary = cluster.add_instance('node1', user_configs=["configs/users.xml"], with_mysql=False, stay_alive=True) node_db_atomic = cluster.add_instance('node2', user_configs=["configs/users_db_atomic.xml"], with_mysql=False, stay_alive=True) - +node_disable_bytes_settings = cluster.add_instance('node3', user_configs=["configs/users_disable_bytes_settings.xml"], with_mysql=False, stay_alive=True) +node_disable_rows_settings = cluster.add_instance('node4', user_configs=["configs/users_disable_rows_settings.xml"], with_mysql=False, stay_alive=True) @pytest.fixture(scope="module") def started_cluster(): @@ -289,5 +290,12 @@ def test_multi_table_update(started_cluster, started_mysql_8_0, started_mysql_5_ @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) -def test_system_tables_table(started_cluster, started_mysql_8_0, clickhouse_node): +def test_system_tables_table(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): + materialize_with_ddl.system_tables_test(clickhouse_node, started_mysql_5_7, "mysql1") materialize_with_ddl.system_tables_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + + +@pytest.mark.parametrize(('clickhouse_node'), [node_disable_bytes_settings, node_disable_rows_settings]) +def test_mysql_settings(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): + materialize_with_ddl.mysql_settings_test(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.mysql_settings_test(clickhouse_node, started_mysql_8_0, "mysql8_0") From 87aa9044409b32f5a21ba03dd731fc7a3a51e01d Mon Sep 17 00:00:00 2001 From: TCeason Date: Tue, 13 Apr 2021 09:46:33 +0800 Subject: [PATCH 34/58] Modify according to review opinion --- src/Core/Settings.h | 4 ++-- src/Formats/MySQLBlockInputStream.cpp | 20 +++----------------- src/Formats/MySQLBlockInputStream.h | 10 ++++------ 3 files changed, 9 insertions(+), 25 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a115d986e5c..6cd3e75c03d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -445,8 +445,8 @@ class IColumn; M(Bool, database_replicated_always_detach_permanently, false, "Execute DETACH TABLE as DETACH TABLE PERMANENTLY if database engine is Replicated", 0) \ M(DistributedDDLOutputMode, distributed_ddl_output_mode, DistributedDDLOutputMode::THROW, "Format of distributed DDL query result", 0) \ M(UInt64, distributed_ddl_entry_format_version, 1, "Version of DDL entry to write into ZooKeeper", 0) \ - M(UInt64, max_read_mysql_rows, DEFAULT_BLOCK_SIZE, "Limit maximum rows when MaterializeMySQL flush history data. 0 -> Disable it.", 0) \ - M(UInt64, max_read_mysql_bytes, (DEFAULT_INSERT_BLOCK_SIZE * 256), "Limit maximum bytes when MaterializeMySQL flush history data. 0 -> Disable it.", 0) \ + M(UInt64, max_read_mysql_rows, DEFAULT_BLOCK_SIZE, "Limit maximum number of rows when MaterializeMySQL should flush history data. If equal to 0, this setting is disabled", 0) \ + M(UInt64, max_read_mysql_bytes, (DEFAULT_INSERT_BLOCK_SIZE * 256), "Limit maximum number of bytes when MaterializeMySQL should flush history data. If equal to 0, this setting is disabled", 0) \ \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ diff --git a/src/Formats/MySQLBlockInputStream.cpp b/src/Formats/MySQLBlockInputStream.cpp index 38261cbc329..10ebf89e949 100644 --- a/src/Formats/MySQLBlockInputStream.cpp +++ b/src/Formats/MySQLBlockInputStream.cpp @@ -30,27 +30,13 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -StreamSettings::StreamSettings(const Settings & settings) -{ - max_read_mysql_rows = settings.max_read_mysql_rows; - max_read_bytes_size = settings.max_read_mysql_bytes; -} - -StreamSettings::StreamSettings(const Settings & settings, bool auto_close_, bool fetch_by_name_) - : auto_close(auto_close_) - , fetch_by_name(fetch_by_name_) -{ - max_read_mysql_rows = settings.max_read_mysql_rows; - max_read_bytes_size = settings.max_read_mysql_bytes; -} - StreamSettings::StreamSettings(const Settings & settings, bool auto_close_, bool fetch_by_name_, size_t max_retry_) - : auto_close(auto_close_) + : max_read_mysql_rows(settings.max_read_mysql_rows) + , max_read_bytes_size(settings.max_read_mysql_bytes) + , auto_close(auto_close_) , fetch_by_name(fetch_by_name_) , default_num_tries_on_connection_loss(max_retry_) { - max_read_mysql_rows = settings.max_read_mysql_rows; - max_read_bytes_size = settings.max_read_mysql_bytes; } MySQLBlockInputStream::Connection::Connection( diff --git a/src/Formats/MySQLBlockInputStream.h b/src/Formats/MySQLBlockInputStream.h index f4c06629955..d6a16379874 100644 --- a/src/Formats/MySQLBlockInputStream.h +++ b/src/Formats/MySQLBlockInputStream.h @@ -15,13 +15,11 @@ struct StreamSettings { size_t max_read_mysql_rows; size_t max_read_bytes_size; - bool auto_close = false; - bool fetch_by_name = false; - size_t default_num_tries_on_connection_loss = 5; + bool auto_close; + bool fetch_by_name; + size_t default_num_tries_on_connection_loss; - StreamSettings(const Settings & settings); - StreamSettings(const Settings & settings, bool auto_close_, bool fetch_by_name_); - StreamSettings(const Settings & settings, bool auto_close_, bool fetch_by_name_, size_t max_retry_); + StreamSettings(const Settings & settings, bool auto_close_ = false, bool fetch_by_name_ = false, size_t max_retry_ = 5); }; From 63403c709ca836857c2ccaa07e26b926c8afa261 Mon Sep 17 00:00:00 2001 From: TCeason Date: Wed, 14 Apr 2021 13:41:15 +0800 Subject: [PATCH 35/58] modify settings name to external_xxx and rewrite Storage MySQL max_block_size --- src/Core/Settings.h | 4 ++-- src/Formats/MySQLBlockInputStream.cpp | 6 +++--- src/Formats/MySQLBlockInputStream.h | 4 ++-- src/Storages/StorageMySQL.cpp | 5 ++++- .../configs/users_disable_bytes_settings.xml | 4 ++-- .../configs/users_disable_rows_settings.xml | 4 ++-- .../test_storage_mysql/configs/users.xml | 18 +++++++++++++++++ tests/integration/test_storage_mysql/test.py | 20 +++++++++++++++++++ 8 files changed, 53 insertions(+), 12 deletions(-) create mode 100644 tests/integration/test_storage_mysql/configs/users.xml diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6cd3e75c03d..be3a7ce739b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -445,8 +445,8 @@ class IColumn; M(Bool, database_replicated_always_detach_permanently, false, "Execute DETACH TABLE as DETACH TABLE PERMANENTLY if database engine is Replicated", 0) \ M(DistributedDDLOutputMode, distributed_ddl_output_mode, DistributedDDLOutputMode::THROW, "Format of distributed DDL query result", 0) \ M(UInt64, distributed_ddl_entry_format_version, 1, "Version of DDL entry to write into ZooKeeper", 0) \ - M(UInt64, max_read_mysql_rows, DEFAULT_BLOCK_SIZE, "Limit maximum number of rows when MaterializeMySQL should flush history data. If equal to 0, this setting is disabled", 0) \ - M(UInt64, max_read_mysql_bytes, (DEFAULT_INSERT_BLOCK_SIZE * 256), "Limit maximum number of bytes when MaterializeMySQL should flush history data. If equal to 0, this setting is disabled", 0) \ + M(UInt64, external_storage_max_read_rows, 0, "Limit maximum number of rows when MaterializeMySQL should flush history data. If equal to 0, this setting is disabled", 0) \ + M(UInt64, external_storage_max_read_bytes, (DEFAULT_INSERT_BLOCK_SIZE * 256), "Limit maximum number of bytes when MaterializeMySQL should flush history data. If equal to 0, this setting is disabled", 0) \ \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ diff --git a/src/Formats/MySQLBlockInputStream.cpp b/src/Formats/MySQLBlockInputStream.cpp index 10ebf89e949..445b159dc4b 100644 --- a/src/Formats/MySQLBlockInputStream.cpp +++ b/src/Formats/MySQLBlockInputStream.cpp @@ -31,8 +31,8 @@ namespace ErrorCodes } StreamSettings::StreamSettings(const Settings & settings, bool auto_close_, bool fetch_by_name_, size_t max_retry_) - : max_read_mysql_rows(settings.max_read_mysql_rows) - , max_read_bytes_size(settings.max_read_mysql_bytes) + : max_read_mysql_row_nums(settings.external_storage_max_read_rows) + , max_read_mysql_bytes_size(settings.external_storage_max_read_bytes) , auto_close(auto_close_) , fetch_by_name(fetch_by_name_) , default_num_tries_on_connection_loss(max_retry_) @@ -258,7 +258,7 @@ Block MySQLBlockInputStream::readImpl() } ++num_rows; - if (num_rows == settings->max_read_mysql_rows || (settings->max_read_bytes_size && read_bytes_size >= settings->max_read_bytes_size)) + if (num_rows == settings->max_read_mysql_row_nums || (settings->max_read_mysql_bytes_size && read_bytes_size >= settings->max_read_mysql_bytes_size)) break; row = connection->result.fetch(); diff --git a/src/Formats/MySQLBlockInputStream.h b/src/Formats/MySQLBlockInputStream.h index d6a16379874..4190addc71d 100644 --- a/src/Formats/MySQLBlockInputStream.h +++ b/src/Formats/MySQLBlockInputStream.h @@ -13,8 +13,8 @@ namespace DB struct StreamSettings { - size_t max_read_mysql_rows; - size_t max_read_bytes_size; + size_t max_read_mysql_row_nums; + size_t max_read_mysql_bytes_size; bool auto_close; bool fetch_by_name; size_t default_num_tries_on_connection_loss; diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 67fbcfc781e..984682ac0a4 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -71,7 +71,7 @@ Pipe StorageMySQL::read( SelectQueryInfo & query_info_, ContextPtr context_, QueryProcessingStage::Enum /*processed_stage*/, - size_t /*max_read_mysql_rows*/, + size_t max_block_size, unsigned) { metadata_snapshot->check(column_names_, getVirtuals(), getStorageID()); @@ -95,6 +95,9 @@ Pipe StorageMySQL::read( sample_block.insert({ column_data.type, column_data.name }); } + if (!context_->getSettingsRef().external_storage_max_read_rows) + context_->setSetting("external_storage_max_read_rows", max_block_size); + StreamSettings mysql_input_stream_settings(context_->getSettingsRef(), true, false); return Pipe(std::make_shared( std::make_shared(pool, query, sample_block, mysql_input_stream_settings))); diff --git a/tests/integration/test_materialize_mysql_database/configs/users_disable_bytes_settings.xml b/tests/integration/test_materialize_mysql_database/configs/users_disable_bytes_settings.xml index cb5a38a57c9..4516cb80c17 100644 --- a/tests/integration/test_materialize_mysql_database/configs/users_disable_bytes_settings.xml +++ b/tests/integration/test_materialize_mysql_database/configs/users_disable_bytes_settings.xml @@ -4,8 +4,8 @@ 1 Atomic - 1 - 0 + 1 + 0 diff --git a/tests/integration/test_materialize_mysql_database/configs/users_disable_rows_settings.xml b/tests/integration/test_materialize_mysql_database/configs/users_disable_rows_settings.xml index a245b8e978b..dea20eb9e12 100644 --- a/tests/integration/test_materialize_mysql_database/configs/users_disable_rows_settings.xml +++ b/tests/integration/test_materialize_mysql_database/configs/users_disable_rows_settings.xml @@ -4,8 +4,8 @@ 1 Atomic - 0 - 1 + 0 + 1 diff --git a/tests/integration/test_storage_mysql/configs/users.xml b/tests/integration/test_storage_mysql/configs/users.xml new file mode 100644 index 00000000000..27c4d46984e --- /dev/null +++ b/tests/integration/test_storage_mysql/configs/users.xml @@ -0,0 +1,18 @@ + + + + + 2 + + + + + + + + ::/0 + + default + + + diff --git a/tests/integration/test_storage_mysql/test.py b/tests/integration/test_storage_mysql/test.py index 4520b3f3837..9c3abd799af 100644 --- a/tests/integration/test_storage_mysql/test.py +++ b/tests/integration/test_storage_mysql/test.py @@ -9,6 +9,7 @@ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'], with_mysql=True) node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'], with_mysql_cluster=True) +node3 = cluster.add_instance('node3', main_configs=['configs/remote_servers.xml'], user_configs=['configs/users.xml'], with_mysql=True) create_table_sql_template = """ CREATE TABLE `clickhouse`.`{}` ( @@ -260,6 +261,25 @@ def test_mysql_distributed(started_cluster): assert(result == 'host2\nhost4\n' or result == 'host3\nhost4\n') +def test_external_settings(started_cluster): + table_name = 'test_external_settings' + conn = get_mysql_conn() + create_mysql_table(conn, table_name) + + node3.query(''' +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse'); +'''.format(table_name, table_name)) + node3.query( + "INSERT INTO {}(id, name, money) select number, concat('name_', toString(number)), 3 from numbers(100) ".format( + table_name)) + assert node3.query("SELECT count() FROM {}".format(table_name)).rstrip() == '100' + assert node3.query("SELECT sum(money) FROM {}".format(table_name)).rstrip() == '300' + node3.query("select value from system.settings where name = 'max_block_size' FORMAT TSV") == "2\n" + node3.query("select value from system.settings where name = 'external_storage_max_read_rows' FORMAT TSV") == "0\n" + assert node3.query("SELECT COUNT(DISTINCT blockNumber()) FROM {} FORMAT TSV".format(table_name)) == '50\n' + conn.close() + + if __name__ == '__main__': with contextmanager(started_cluster)() as cluster: for name, instance in list(cluster.instances.items()): From b4bf53dfc9a6668b723ad45f2d14472ee6ed80a3 Mon Sep 17 00:00:00 2001 From: TCeason Date: Fri, 16 Apr 2021 09:42:55 +0800 Subject: [PATCH 36/58] add some comment and modify a parameter type --- src/Core/Settings.h | 4 ++-- src/Formats/MySQLBlockInputStream.cpp | 2 +- src/Formats/MySQLBlockInputStream.h | 3 ++- src/Storages/StorageMySQL.cpp | 4 +--- 4 files changed, 6 insertions(+), 7 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index be3a7ce739b..2b0815291b5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -445,8 +445,8 @@ class IColumn; M(Bool, database_replicated_always_detach_permanently, false, "Execute DETACH TABLE as DETACH TABLE PERMANENTLY if database engine is Replicated", 0) \ M(DistributedDDLOutputMode, distributed_ddl_output_mode, DistributedDDLOutputMode::THROW, "Format of distributed DDL query result", 0) \ M(UInt64, distributed_ddl_entry_format_version, 1, "Version of DDL entry to write into ZooKeeper", 0) \ - M(UInt64, external_storage_max_read_rows, 0, "Limit maximum number of rows when MaterializeMySQL should flush history data. If equal to 0, this setting is disabled", 0) \ - M(UInt64, external_storage_max_read_bytes, (DEFAULT_INSERT_BLOCK_SIZE * 256), "Limit maximum number of bytes when MaterializeMySQL should flush history data. If equal to 0, this setting is disabled", 0) \ + M(UInt64, external_storage_max_read_rows, 0, "Limit maximum number of rows when table with external engine should flush history data. Now supported only for MySQL table engine, database engine, dictionary and MaterializeMySQL. If equal to 0, this setting is disabled", 0) \ + M(UInt64, external_storage_max_read_bytes, 0, "Limit maximum number of bytes when table with external engine should flush history data. Now supported only for MySQL table engine, database engine, dictionary and MaterializeMySQL. If equal to 0, this setting is disabled", 0) \ \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ diff --git a/src/Formats/MySQLBlockInputStream.cpp b/src/Formats/MySQLBlockInputStream.cpp index 445b159dc4b..be0cb31f22d 100644 --- a/src/Formats/MySQLBlockInputStream.cpp +++ b/src/Formats/MySQLBlockInputStream.cpp @@ -31,7 +31,7 @@ namespace ErrorCodes } StreamSettings::StreamSettings(const Settings & settings, bool auto_close_, bool fetch_by_name_, size_t max_retry_) - : max_read_mysql_row_nums(settings.external_storage_max_read_rows) + : max_read_mysql_row_nums((settings.external_storage_max_read_rows) ? settings.external_storage_max_read_rows : settings.max_block_size) , max_read_mysql_bytes_size(settings.external_storage_max_read_bytes) , auto_close(auto_close_) , fetch_by_name(fetch_by_name_) diff --git a/src/Formats/MySQLBlockInputStream.h b/src/Formats/MySQLBlockInputStream.h index 4190addc71d..12deb9c3146 100644 --- a/src/Formats/MySQLBlockInputStream.h +++ b/src/Formats/MySQLBlockInputStream.h @@ -13,6 +13,7 @@ namespace DB struct StreamSettings { + /// Check if setting is enabled, otherwise use common `max_block_size` setting. size_t max_read_mysql_row_nums; size_t max_read_mysql_bytes_size; bool auto_close; @@ -38,7 +39,7 @@ public: Block getHeader() const override { return description.sample_block.cloneEmpty(); } protected: - MySQLBlockInputStream(const Block & sample_block_, const struct StreamSettings & settings); + MySQLBlockInputStream(const Block & sample_block_, const StreamSettings & settings); Block readImpl() override; void initPositionMappingFromQueryResultStructure(); diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 984682ac0a4..bc07679a0de 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -71,7 +71,7 @@ Pipe StorageMySQL::read( SelectQueryInfo & query_info_, ContextPtr context_, QueryProcessingStage::Enum /*processed_stage*/, - size_t max_block_size, + size_t /*max_block_size*/, unsigned) { metadata_snapshot->check(column_names_, getVirtuals(), getStorageID()); @@ -95,8 +95,6 @@ Pipe StorageMySQL::read( sample_block.insert({ column_data.type, column_data.name }); } - if (!context_->getSettingsRef().external_storage_max_read_rows) - context_->setSetting("external_storage_max_read_rows", max_block_size); StreamSettings mysql_input_stream_settings(context_->getSettingsRef(), true, false); return Pipe(std::make_shared( From 9d4295f261a3bc554f8e4a123e1a05e839914088 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Mon, 19 Apr 2021 11:02:36 +0300 Subject: [PATCH 37/58] Removed infinite retries. --- src/Core/Settings.h | 2 +- src/Disks/S3/DiskS3.cpp | 6 +++--- src/Disks/S3/DiskS3.h | 4 ++-- src/IO/ReadBufferFromS3.cpp | 4 ++-- src/IO/ReadBufferFromS3.h | 4 ++-- src/Storages/StorageS3.cpp | 6 +++--- src/Storages/StorageS3.h | 8 ++++---- src/TableFunctions/TableFunctionS3.cpp | 2 +- src/TableFunctions/TableFunctionS3Cluster.cpp | 2 +- 9 files changed, 19 insertions(+), 19 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index bb17ffc9167..0eebee4e3ae 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -70,7 +70,7 @@ class IColumn; M(UInt64, connections_with_failover_max_tries, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, "The maximum number of attempts to connect to replicas.", 0) \ M(UInt64, s3_min_upload_part_size, 512*1024*1024, "The minimum size of part to upload during multipart upload to S3.", 0) \ M(UInt64, s3_max_single_part_upload_size, 64*1024*1024, "The maximum size of object to upload using singlepart upload to S3.", 0) \ - M(Int64, s3_max_single_read_retries, 16, "The maximum number of retries during single S3 read.", 0) \ + M(UInt64, s3_max_single_read_retries, 4, "The maximum number of retries during single S3 read.", 0) \ M(UInt64, s3_max_redirects, 10, "Max number of S3 redirects hops allowed.", 0) \ M(UInt64, s3_max_connections, 1024, "The maximum number of connections per server.", 0) \ M(Bool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \ diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 215f86e1bc7..90c69b7f3b3 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -249,7 +249,7 @@ class ReadIndirectBufferFromS3 final : public ReadBufferFromFileBase { public: ReadIndirectBufferFromS3( - std::shared_ptr client_ptr_, const String & bucket_, DiskS3::Metadata metadata_, Int64 s3_max_single_read_retries_, size_t buf_size_) + std::shared_ptr client_ptr_, const String & bucket_, DiskS3::Metadata metadata_, UInt64 s3_max_single_read_retries_, size_t buf_size_) : client_ptr(std::move(client_ptr_)) , bucket(bucket_) , metadata(std::move(metadata_)) @@ -350,7 +350,7 @@ private: std::shared_ptr client_ptr; const String & bucket; DiskS3::Metadata metadata; - Int64 s3_max_single_read_retries; + UInt64 s3_max_single_read_retries; size_t buf_size; size_t absolute_position = 0; @@ -564,7 +564,7 @@ DiskS3::DiskS3( String bucket_, String s3_root_path_, String metadata_path_, - Int64 s3_max_single_read_retries_, + UInt64 s3_max_single_read_retries_, size_t min_upload_part_size_, size_t max_single_part_upload_size_, size_t min_bytes_for_seek_, diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index b6581c6e5b7..72548addfdf 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -40,7 +40,7 @@ public: String bucket_, String s3_root_path_, String metadata_path_, - Int64 s3_max_single_read_retries_, + UInt64 s3_max_single_read_retries_, size_t min_upload_part_size_, size_t max_single_part_upload_size_, size_t min_bytes_for_seek_, @@ -181,7 +181,7 @@ private: const String bucket; const String s3_root_path; String metadata_path; - Int64 s3_max_single_read_retries; + UInt64 s3_max_single_read_retries; size_t min_upload_part_size; size_t max_single_part_upload_size; size_t min_bytes_for_seek; diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 07d1246b370..1e27b0284b7 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -31,7 +31,7 @@ namespace ErrorCodes ReadBufferFromS3::ReadBufferFromS3( - std::shared_ptr client_ptr_, const String & bucket_, const String & key_, Int64 s3_max_single_read_retries_, size_t buffer_size_) + std::shared_ptr client_ptr_, const String & bucket_, const String & key_, UInt64 s3_max_single_read_retries_, size_t buffer_size_) : SeekableReadBuffer(nullptr, 0) , client_ptr(std::move(client_ptr_)) , bucket(bucket_) @@ -52,7 +52,7 @@ bool ReadBufferFromS3::nextImpl() Stopwatch watch; bool next_result = false; - for (Int64 attempt = s3_max_single_read_retries; s3_max_single_read_retries < 0 || attempt >= 0; --attempt) + for (Int64 attempt = static_cast(s3_max_single_read_retries); attempt >= 0; --attempt) { if (!impl) impl = initialize(); diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index f8fe2407a91..1f4124d909f 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -27,7 +27,7 @@ private: std::shared_ptr client_ptr; String bucket; String key; - Int64 s3_max_single_read_retries; + UInt64 s3_max_single_read_retries; size_t buffer_size; off_t offset = 0; Aws::S3::Model::GetObjectResult read_result; @@ -40,7 +40,7 @@ public: std::shared_ptr client_ptr_, const String & bucket_, const String & key_, - Int64 s3_max_single_read_retries_, + UInt64 s3_max_single_read_retries_, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE); bool nextImpl() override; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index c2632362fc2..0bda8fac3ed 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -166,7 +166,7 @@ StorageS3Source::StorageS3Source( ContextPtr context_, const ColumnsDescription & columns_, UInt64 max_block_size_, - Int64 s3_max_single_read_retries_, + UInt64 s3_max_single_read_retries_, const String compression_hint_, const std::shared_ptr & client_, const String & bucket_, @@ -314,7 +314,7 @@ StorageS3::StorageS3( const String & secret_access_key_, const StorageID & table_id_, const String & format_name_, - Int64 s3_max_single_read_retries_, + UInt64 s3_max_single_read_retries_, UInt64 min_upload_part_size_, UInt64 max_single_part_upload_size_, UInt64 max_connections_, @@ -478,10 +478,10 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) secret_access_key = engine_args[2]->as().value.safeGet(); } + UInt64 s3_max_single_read_retries = args.getLocalContext()->getSettingsRef().s3_max_single_read_retries; UInt64 min_upload_part_size = args.getLocalContext()->getSettingsRef().s3_min_upload_part_size; UInt64 max_single_part_upload_size = args.getLocalContext()->getSettingsRef().s3_max_single_part_upload_size; UInt64 max_connections = args.getLocalContext()->getSettingsRef().s3_max_connections; - Int64 s3_max_single_read_retries = args.getLocalContext()->getSettingsRef().s3_max_single_read_retries; String compression_method; String format_name; diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 6a7c09608bf..b068f82cfb1 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -55,7 +55,7 @@ public: ContextPtr context_, const ColumnsDescription & columns_, UInt64 max_block_size_, - Int64 s3_max_single_read_retries_, + UInt64 s3_max_single_read_retries_, const String compression_hint_, const std::shared_ptr & client_, const String & bucket, @@ -72,7 +72,7 @@ private: String format; ColumnsDescription columns_desc; UInt64 max_block_size; - Int64 s3_max_single_read_retries; + UInt64 s3_max_single_read_retries; String compression_hint; std::shared_ptr client; Block sample_block; @@ -102,7 +102,7 @@ public: const String & secret_access_key, const StorageID & table_id_, const String & format_name_, - Int64 s3_max_single_read_retries_, + UInt64 s3_max_single_read_retries_, UInt64 min_upload_part_size_, UInt64 max_single_part_upload_size_, UInt64 max_connections_, @@ -148,7 +148,7 @@ private: ClientAuthentificaiton client_auth; String format_name; - Int64 s3_max_single_read_retries; + UInt64 s3_max_single_read_retries; size_t min_upload_part_size; size_t max_single_part_upload_size; String compression_method; diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index fbff23a23d0..973899d2101 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -83,10 +83,10 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, Context { Poco::URI uri (filename); S3::URI s3_uri (uri); + UInt64 s3_max_single_read_retries = context->getSettingsRef().s3_max_single_read_retries; UInt64 min_upload_part_size = context->getSettingsRef().s3_min_upload_part_size; UInt64 max_single_part_upload_size = context->getSettingsRef().s3_max_single_part_upload_size; UInt64 max_connections = context->getSettingsRef().s3_max_connections; - Int64 s3_max_single_read_retries = context->getSettingsRef().s3_max_single_read_retries; StoragePtr storage = StorageS3::create( s3_uri, diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp index 9d8539347a3..16f48c70608 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ b/src/TableFunctions/TableFunctionS3Cluster.cpp @@ -109,10 +109,10 @@ StoragePtr TableFunctionS3Cluster::executeImpl( Poco::URI uri (filename); S3::URI s3_uri (uri); /// Actually this parameters are not used + UInt64 s3_max_single_read_retries = context->getSettingsRef().s3_max_single_read_retries; UInt64 min_upload_part_size = context->getSettingsRef().s3_min_upload_part_size; UInt64 max_single_part_upload_size = context->getSettingsRef().s3_max_single_part_upload_size; UInt64 max_connections = context->getSettingsRef().s3_max_connections; - Int64 s3_max_single_read_retries = context->getSettingsRef().s3_max_single_read_retries; storage = StorageS3::create( s3_uri, access_key_id, secret_access_key, StorageID(getDatabaseName(), table_name), format, From 8f1b5073867175abfab5c4ee66a5cc9e2ebce393 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Mon, 19 Apr 2021 11:46:42 +0300 Subject: [PATCH 38/58] Fix test_secure_socket and remove redundant settings in hedged tests --- tests/integration/test_hedged_requests/configs/users1.xml | 2 -- .../test_hedged_requests_parallel/configs/users1.xml | 2 -- tests/integration/test_secure_socket/test.py | 4 ++-- 3 files changed, 2 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_hedged_requests/configs/users1.xml b/tests/integration/test_hedged_requests/configs/users1.xml index bc68c2c579d..2a54396feca 100644 --- a/tests/integration/test_hedged_requests/configs/users1.xml +++ b/tests/integration/test_hedged_requests/configs/users1.xml @@ -2,8 +2,6 @@ - 1 - 1 diff --git a/tests/integration/test_hedged_requests_parallel/configs/users1.xml b/tests/integration/test_hedged_requests_parallel/configs/users1.xml index bc68c2c579d..2a54396feca 100644 --- a/tests/integration/test_hedged_requests_parallel/configs/users1.xml +++ b/tests/integration/test_hedged_requests_parallel/configs/users1.xml @@ -2,8 +2,6 @@ - 1 - 1 diff --git a/tests/integration/test_secure_socket/test.py b/tests/integration/test_secure_socket/test.py index 0ca6e6a6e6b..65c789f9d02 100644 --- a/tests/integration/test_secure_socket/test.py +++ b/tests/integration/test_secure_socket/test.py @@ -64,7 +64,7 @@ def test(started_cluster): assert end - start < 10 start = time.time() - error = NODES['node1'].query_and_get_error('SELECT * FROM distributed_table settings receive_timeout=5, send_timeout=5, use_hedged_requests=0;') + error = NODES['node1'].query_and_get_error('SELECT * FROM distributed_table settings receive_timeout=5, send_timeout=5, use_hedged_requests=0, async_socket_for_remote=1;') end = time.time() assert end - start < 10 @@ -73,7 +73,7 @@ def test(started_cluster): assert error.find('DB::ReadBufferFromPocoSocket::nextImpl()') == -1 start = time.time() - error = NODES['node1'].query_and_get_error('SELECT * FROM distributed_table settings receive_timeout=5, send_timeout=5;') + error = NODES['node1'].query_and_get_error('SELECT * FROM distributed_table settings receive_timeout=5, send_timeout=5, use_hedged_requests=1, async_socket_for_remote=1;') end = time.time() assert end - start < 10 From 7f6bf6513f6e8f53d4729d5955893a99a35ed027 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 19 Apr 2021 11:46:28 +0200 Subject: [PATCH 39/58] Link keeper-bench to clickhouse_common_zookeeper --- utils/keeper-bench/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/keeper-bench/CMakeLists.txt b/utils/keeper-bench/CMakeLists.txt index 7efd2c24f82..2f12194d1b7 100644 --- a/utils/keeper-bench/CMakeLists.txt +++ b/utils/keeper-bench/CMakeLists.txt @@ -1,2 +1,2 @@ add_executable(keeper-bench Generator.cpp Runner.cpp Stats.cpp main.cpp) -target_link_libraries(keeper-bench PRIVATE dbms) +target_link_libraries(keeper-bench PRIVATE clickhouse_common_zookeeper) From 648b941922be781479117864d5f00f0a9b0d4c79 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 16 Apr 2021 17:56:08 +0300 Subject: [PATCH 40/58] done --- src/Server/TCPHandler.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index c6cd74f6c6a..53f66a74631 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -985,8 +985,6 @@ bool TCPHandler::receivePacket() switch (packet_type) { - case Protocol::Client::ReadTaskResponse: - throw Exception("ReadTaskResponse must be received only after requesting in callback", ErrorCodes::LOGICAL_ERROR); case Protocol::Client::IgnoredPartUUIDs: /// Part uuids packet if any comes before query. receiveIgnoredPartUUIDs(); From d9d94752d11f127a999f60b3c2570f4315afedd1 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 19 Apr 2021 17:05:28 +0300 Subject: [PATCH 41/58] fix broken perf test --- tests/performance/agg_functions_min_max_any.xml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/performance/agg_functions_min_max_any.xml b/tests/performance/agg_functions_min_max_any.xml index 79c9e2c6976..6ca9e3eb65a 100644 --- a/tests/performance/agg_functions_min_max_any.xml +++ b/tests/performance/agg_functions_min_max_any.xml @@ -6,7 +6,9 @@ group_scale - 1000000 + + 1000000 + From 0cf58d1fb76cf0b6acea67ea50ccd069743548d4 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Mon, 19 Apr 2021 19:34:33 +0300 Subject: [PATCH 42/58] Update docs/ru/sql-reference/functions/json-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/json-functions.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/functions/json-functions.md b/docs/ru/sql-reference/functions/json-functions.md index 1043aca7615..25b4c337ba9 100644 --- a/docs/ru/sql-reference/functions/json-functions.md +++ b/docs/ru/sql-reference/functions/json-functions.md @@ -22,7 +22,7 @@ toc_title: JSON ## visitParamExtractUInt(params, name) {#visitparamextractuintparams-name} -Разбирает UInt64 из значения поля с именем `name`. Если поле строковое, пытается разобрать число из начала строки. Если такого поля нет, или если оно есть, но содержит не число, то вернуть 0. +Пытается выделить число типа UInt64 из значения поля с именем `name`. Если поле строковое, пытается выделить число из начала строки. Если такого поля нет, или если оно есть, но содержит не число, то возвращает 0. Алиас: `simpleJSONExtractUInt`. @@ -306,4 +306,3 @@ SELECT JSONExtractKeysAndValuesRaw('{"a": [-100, 200.0], "b":{"c": {"d": "hello" │ [('d','"hello"'),('f','"world"')] │ └───────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` - From 5fdb08fa900eaa979bb00da82130169ff38fade7 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Mon, 19 Apr 2021 19:34:53 +0300 Subject: [PATCH 43/58] Update docs/ru/sql-reference/functions/json-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/json-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/json-functions.md b/docs/ru/sql-reference/functions/json-functions.md index 25b4c337ba9..916f59ab4b2 100644 --- a/docs/ru/sql-reference/functions/json-functions.md +++ b/docs/ru/sql-reference/functions/json-functions.md @@ -40,7 +40,7 @@ toc_title: JSON ## visitParamExtractBool(params, name) {#visitparamextractboolparams-name} -Разбирает значение true/false. Результат — UInt8. +Пытается выделить значение true/false. Результат — UInt8. Алиас: `simpleJSONExtractBool`. From 7af622941876cc1fb51c53b0d8380003b8c5b0e2 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Mon, 19 Apr 2021 19:35:20 +0300 Subject: [PATCH 44/58] Update docs/ru/sql-reference/functions/json-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/json-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/json-functions.md b/docs/ru/sql-reference/functions/json-functions.md index 916f59ab4b2..4de487c03ad 100644 --- a/docs/ru/sql-reference/functions/json-functions.md +++ b/docs/ru/sql-reference/functions/json-functions.md @@ -74,7 +74,7 @@ visitParamExtractString('{"abc":"hello}', 'abc') = ''; На данный момент не поддерживаются записанные в формате `\uXXXX\uYYYY` кодовые точки не из basic multilingual plane (они переводятся не в UTF-8, а в CESU-8). -Следующие функции используют [simdjson](https://github.com/lemire/simdjson), который разработан под более сложные требования для разбора JSON. Упомянутое выше предположение 2 по-прежнему применимо. +Следующие функции используют [simdjson](https://github.com/lemire/simdjson), который разработан под более сложные требования для разбора JSON. Упомянутое выше допущение 2 по-прежнему применимо. ## isValidJSON(json) {#isvalidjsonjson} From 6f5144a2e760dd69a5826b1d7e8f3758424efae1 Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Mon, 19 Apr 2021 20:01:58 +0300 Subject: [PATCH 45/58] Update docs/ru/guides/apply-catboost-model.md --- docs/ru/guides/apply-catboost-model.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/guides/apply-catboost-model.md b/docs/ru/guides/apply-catboost-model.md index d8c6f41f339..db2be63692f 100644 --- a/docs/ru/guides/apply-catboost-model.md +++ b/docs/ru/guides/apply-catboost-model.md @@ -159,7 +159,7 @@ FROM amazon_train /home/catboost/models/*_model.xml ``` !!! note "Примечание" - Вы можете позднее изменить путь к CatBoost и конфигурации модели без перезагрузки сервера. + Вы можете позднее изменить путь к конфигурации модели CatBoost без перезагрузки сервера. ## 4. Запустите вывод модели из SQL {#run-model-inference} From 6abe37b13e48d260f437026217131dabd6c1f8bd Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Mon, 19 Apr 2021 20:04:51 +0300 Subject: [PATCH 46/58] Apply suggestions from code review Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/en/operations/system-tables/columns.md | 2 +- docs/en/operations/system-tables/tables.md | 4 ++-- docs/ru/operations/system-tables/columns.md | 4 ++-- docs/ru/operations/system-tables/tables.md | 8 ++++---- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/docs/en/operations/system-tables/columns.md b/docs/en/operations/system-tables/columns.md index 0084a391390..9160dca9a1a 100644 --- a/docs/en/operations/system-tables/columns.md +++ b/docs/en/operations/system-tables/columns.md @@ -28,7 +28,7 @@ Columns: **Example** ```sql -select * from system.columns LIMIT 2 FORMAT Vertical; +SELECT * FROM system.columns LIMIT 2 FORMAT Vertical; ``` ```text diff --git a/docs/en/operations/system-tables/tables.md b/docs/en/operations/system-tables/tables.md index 5f515b63fbf..ccc9ab94f8b 100644 --- a/docs/en/operations/system-tables/tables.md +++ b/docs/en/operations/system-tables/tables.md @@ -43,9 +43,9 @@ Columns: - [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) - [Distributed](../../engines/table-engines/special/distributed.md#distributed) -- `total_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - Total number of rows, if it is possible to quickly determine exact number of rows in the table, otherwise `Null` (including underying `Buffer` table). +- `total_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - Total number of rows, if it is possible to quickly determine exact number of rows in the table, otherwise `NULL` (including underying `Buffer` table). -- `total_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - Total number of bytes, if it is possible to quickly determine exact number of bytes for the table on storage, otherwise `Null` (**does not** includes any underlying storage). +- `total_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - Total number of bytes, if it is possible to quickly determine exact number of bytes for the table on storage, otherwise `NULL` (does not includes any underlying storage). - If the table stores data on disk, returns used space on disk (i.e. compressed). - If the table stores data in memory, returns approximated number of used bytes in memory. diff --git a/docs/ru/operations/system-tables/columns.md b/docs/ru/operations/system-tables/columns.md index 71a21860234..b8a0aef2299 100644 --- a/docs/ru/operations/system-tables/columns.md +++ b/docs/ru/operations/system-tables/columns.md @@ -28,7 +28,7 @@ Cтолбцы: **Пример** ```sql -select * from system.columns LIMIT 2 FORMAT Vertical; +SELECT * FROM system.columns LIMIT 2 FORMAT Vertical; ``` ```text @@ -67,4 +67,4 @@ is_in_sorting_key: 0 is_in_primary_key: 0 is_in_sampling_key: 0 compression_codec: -``` \ No newline at end of file +``` diff --git a/docs/ru/operations/system-tables/tables.md b/docs/ru/operations/system-tables/tables.md index a69e7214b96..11bb6a9eda2 100644 --- a/docs/ru/operations/system-tables/tables.md +++ b/docs/ru/operations/system-tables/tables.md @@ -17,7 +17,7 @@ - `metadata_modification_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — время последней модификации табличных метаданных. - `dependencies_database` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — зависимости базы данных. - `dependencies_table` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — табличные зависимости (таблицы [MaterializedView](../../engines/table-engines/special/materializedview.md), созданные на базе текущей таблицы). -- `create_table_query` ([String](../../sql-reference/data-types/string.md)) — запрос, которым создавалась таблица. +- `create_table_query` ([String](../../sql-reference/data-types/string.md)) — запрос, при помощи которого создавалась таблица. - `engine_full` ([String](../../sql-reference/data-types/string.md)) — параметры табличного движка. - `partition_key` ([String](../../sql-reference/data-types/string.md)) — ключ партиционирования таблицы. - `sorting_key` ([String](../../sql-reference/data-types/string.md)) — ключ сортировки таблицы. @@ -28,9 +28,9 @@ - [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) - [Distributed](../../engines/table-engines/special/distributed.md#distributed) -- `total_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - общее количество строк, если есть возможность быстро определить точное количество строк в таблице, в противном случае `Null` (включая базовую таблицу `Buffer`). +- `total_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - общее количество строк, если есть возможность быстро определить точное количество строк в таблице, в противном случае `NULL` (включая базовую таблицу `Buffer`). -- `total_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - общее количество байт, если можно быстро определить точное количество байт для таблицы на накопителе, в противном случае `Null` (**не включает** в себя никакого базового хранилища). +- `total_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - общее количество байт, если можно быстро определить точное количество байт для таблицы на накопителе, в противном случае `NULL` (не включает в себя никакого базового хранилища). - Если таблица хранит данные на диске, возвращает используемое пространство на диске (т. е. сжатое). - Если таблица хранит данные в памяти, возвращает приблизительное количество используемых байт в памяти. @@ -91,4 +91,4 @@ sampling_key: storage_policy: total_rows: ᴺᵁᴸᴸ total_bytes: ᴺᵁᴸᴸ -``` \ No newline at end of file +``` From f7d3f7722246a7909071d352e8abc2adb1db3852 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Apr 2021 22:36:45 +0300 Subject: [PATCH 47/58] Fix bad test 01602_max_distributed_connections --- .../01602_max_distributed_connections.sh | 32 ++++++++++++++----- 1 file changed, 24 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01602_max_distributed_connections.sh b/tests/queries/0_stateless/01602_max_distributed_connections.sh index 93c6071c091..9dceec936b9 100755 --- a/tests/queries/0_stateless/01602_max_distributed_connections.sh +++ b/tests/queries/0_stateless/01602_max_distributed_connections.sh @@ -4,13 +4,29 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -common_opts=( - "--format=Null" +# We check that even if max_threads is small, the setting max_distributed_connections +# will allow to process queries on multiple shards concurrently. - "--max_threads=1" - "--max_distributed_connections=3" -) +# We do sleep 0.15 seconds on ten machines. +# If concurrency is one (bad) the query will take at least 1.5 seconds and the following loops are guaranteed to be infinite. +# If concurrency is 10 (good), the query may take less than 1 second with non-zero probability +# and the following loops will finish with probability 1 assuming independent random variables. -# NOTE: the test use higher timeout to avoid flakiness. -timeout 9s ${CLICKHOUSE_CLIENT} "$@" "${common_opts[@]}" -q "select sleep(3) from remote('127.{1,2,3,4,5}', system.one)" --prefer_localhost_replica=0 -timeout 9s ${CLICKHOUSE_CLIENT} "$@" "${common_opts[@]}" -q "select sleep(3) from remote('127.{1,2,3,4,5}', system.one)" --prefer_localhost_replica=1 +while true; do + timeout 1 ${CLICKHOUSE_CLIENT} --max_threads 1 --max_distributed_connections 10 --query " + SELECT sleep(0.15) FROM remote('127.{1..10}', system.one) FORMAT Null" --prefer_localhost_replica=0 && break +done + +while true; do + timeout 1 ${CLICKHOUSE_CLIENT} --max_threads 1 --max_distributed_connections 10 --query " + SELECT sleep(0.15) FROM remote('127.{1..10}', system.one) FORMAT Null" --prefer_localhost_replica=1 && break +done + +# If max_distributed_connections is low and async_socket_for_remote is disabled, +# the concurrency of distributed queries will be also low. + +timeout 1 ${CLICKHOUSE_CLIENT} --max_threads 1 --max_distributed_connections 1 --async_socket_for_remote 0 --query " + SELECT sleep(0.15) FROM remote('127.{1..10}', system.one) FORMAT Null" --prefer_localhost_replica=0 && echo 'Fail' + +timeout 1 ${CLICKHOUSE_CLIENT} --max_threads 1 --max_distributed_connections 1 --async_socket_for_remote 0 --query " + SELECT sleep(0.15) FROM remote('127.{1..10}', system.one) FORMAT Null" --prefer_localhost_replica=1 && echo 'Fail' From 0784efa813f069810424468425853e2ac6daba37 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Apr 2021 22:42:39 +0300 Subject: [PATCH 48/58] Better test --- .../0_stateless/01602_max_distributed_connections.reference | 1 + tests/queries/0_stateless/01602_max_distributed_connections.sh | 2 ++ 2 files changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/01602_max_distributed_connections.reference b/tests/queries/0_stateless/01602_max_distributed_connections.reference index e69de29bb2d..7326d960397 100644 --- a/tests/queries/0_stateless/01602_max_distributed_connections.reference +++ b/tests/queries/0_stateless/01602_max_distributed_connections.reference @@ -0,0 +1 @@ +Ok diff --git a/tests/queries/0_stateless/01602_max_distributed_connections.sh b/tests/queries/0_stateless/01602_max_distributed_connections.sh index 9dceec936b9..e13b6ed211a 100755 --- a/tests/queries/0_stateless/01602_max_distributed_connections.sh +++ b/tests/queries/0_stateless/01602_max_distributed_connections.sh @@ -30,3 +30,5 @@ timeout 1 ${CLICKHOUSE_CLIENT} --max_threads 1 --max_distributed_connections 1 - timeout 1 ${CLICKHOUSE_CLIENT} --max_threads 1 --max_distributed_connections 1 --async_socket_for_remote 0 --query " SELECT sleep(0.15) FROM remote('127.{1..10}', system.one) FORMAT Null" --prefer_localhost_replica=1 && echo 'Fail' + +echo 'Ok' From 4ad4dbbc0959f27a31537d2b568905ec72a0b88f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Apr 2021 23:26:11 +0300 Subject: [PATCH 49/58] Regenerate Arcadia --- src/AggregateFunctions/ya.make | 1 + 1 file changed, 1 insertion(+) diff --git a/src/AggregateFunctions/ya.make b/src/AggregateFunctions/ya.make index 3a8f0ad9fba..64605aee659 100644 --- a/src/AggregateFunctions/ya.make +++ b/src/AggregateFunctions/ya.make @@ -50,6 +50,7 @@ SRCS( AggregateFunctionStatisticsSimple.cpp AggregateFunctionStudentTTest.cpp AggregateFunctionSum.cpp + AggregateFunctionSumCount.cpp AggregateFunctionSumMap.cpp AggregateFunctionTopK.cpp AggregateFunctionUniq.cpp From 1dbbe2d07577ceedf45de51930aba703df610451 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 19 Apr 2021 18:06:54 -0300 Subject: [PATCH 50/58] add check that p.pruning works --- .../0_stateless/01783_merge_engine_join_key_condition.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01783_merge_engine_join_key_condition.sql b/tests/queries/0_stateless/01783_merge_engine_join_key_condition.sql index 115ee42fe11..372c1bd3572 100644 --- a/tests/queries/0_stateless/01783_merge_engine_join_key_condition.sql +++ b/tests/queries/0_stateless/01783_merge_engine_join_key_condition.sql @@ -10,7 +10,7 @@ CREATE TABLE foo_merge as foo ENGINE=Merge(currentDatabase(), '^foo'); CREATE TABLE t2 (Id Int32, Val Int32, X Int32) Engine=Memory; INSERT INTO t2 values (4, 3, 4); -SET force_primary_key = 1; +SET force_primary_key = 1, force_index_by_date=1; SELECT * FROM foo_merge WHERE Val = 3 AND Id = 3; SELECT count(), X FROM foo_merge JOIN t2 USING Val WHERE Val = 3 AND Id = 3 AND t2.X == 4 GROUP BY X; From a584d78f19045cc00710b4c6c147baa102d17c27 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 20 Apr 2021 01:45:38 +0300 Subject: [PATCH 51/58] Auto version update to [21.5.1.6601] [54450] --- cmake/autogenerated_versions.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index 9d74179902d..fc687b4b19c 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -3,7 +3,7 @@ SET(VERSION_REVISION 54450) SET(VERSION_MAJOR 21) SET(VERSION_MINOR 5) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH 3827789b3d8fd2021952e57e5110343d26daa1a1) -SET(VERSION_DESCRIBE v21.5.1.1-prestable) -SET(VERSION_STRING 21.5.1.1) +SET(VERSION_GITHASH 96fced4c3cf432fb0b401d2ab01f0c56e5f74a96) +SET(VERSION_DESCRIBE v21.5.1.6601-prestable) +SET(VERSION_STRING 21.5.1.6601) # end of autochange From db82e9e3d50fec9d9c93fa5bdc4ff67cf226b1be Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 20 Apr 2021 01:48:20 +0300 Subject: [PATCH 52/58] Auto version update to [21.6.1.1] [54451] --- cmake/autogenerated_versions.txt | 8 ++++---- debian/changelog | 4 ++-- docker/client/Dockerfile | 2 +- docker/server/Dockerfile | 2 +- docker/test/Dockerfile | 2 +- .../System/StorageSystemContributors.generated.cpp | 13 +++++++++++++ 6 files changed, 22 insertions(+), 9 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index fc687b4b19c..51f4b974161 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -1,9 +1,9 @@ # This strings autochanged from release_lib.sh: -SET(VERSION_REVISION 54450) +SET(VERSION_REVISION 54451) SET(VERSION_MAJOR 21) -SET(VERSION_MINOR 5) +SET(VERSION_MINOR 6) SET(VERSION_PATCH 1) SET(VERSION_GITHASH 96fced4c3cf432fb0b401d2ab01f0c56e5f74a96) -SET(VERSION_DESCRIBE v21.5.1.6601-prestable) -SET(VERSION_STRING 21.5.1.6601) +SET(VERSION_DESCRIBE v21.6.1.1-prestable) +SET(VERSION_STRING 21.6.1.1) # end of autochange diff --git a/debian/changelog b/debian/changelog index be77dfdefe9..8b6626416a9 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (21.5.1.1) unstable; urgency=low +clickhouse (21.6.1.1) unstable; urgency=low * Modified source code - -- clickhouse-release Fri, 02 Apr 2021 18:34:26 +0300 + -- clickhouse-release Tue, 20 Apr 2021 01:48:16 +0300 diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index 2efba9735ae..569025dec1c 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" -ARG version=21.5.1.* +ARG version=21.6.1.* RUN apt-get update \ && apt-get install --yes --no-install-recommends \ diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index 05ca29f22d4..48c978366c6 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:20.04 ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" -ARG version=21.5.1.* +ARG version=21.6.1.* ARG gosu_ver=1.10 # set non-empty deb_location_url url to create a docker image diff --git a/docker/test/Dockerfile b/docker/test/Dockerfile index 976c46ebe27..0e4646386ce 100644 --- a/docker/test/Dockerfile +++ b/docker/test/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" -ARG version=21.5.1.* +ARG version=21.6.1.* RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index 46ead225102..b8741e6951c 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -17,6 +17,7 @@ const char * auto_contributors[] { "Aleksei Semiglazov", "Aleksey", "Aleksey Akulovich", + "Alex", "Alex Bocharov", "Alex Karo", "Alex Krash", @@ -144,6 +145,7 @@ const char * auto_contributors[] { "Chao Wang", "Chen Yufei", "Chienlung Cheung", + "Christian", "Ciprian Hacman", "Clement Rodriguez", "Clément Rodriguez", @@ -175,6 +177,7 @@ const char * auto_contributors[] { "Dmitry Belyavtsev", "Dmitry Bilunov", "Dmitry Galuza", + "Dmitry Krylov", "Dmitry Luhtionov", "Dmitry Moskowski", "Dmitry Muzyka", @@ -185,6 +188,7 @@ const char * auto_contributors[] { "Dongdong Yang", "DoomzD", "Dr. Strange Looker", + "Egor O'Sten", "Ekaterina", "Eldar Zaitov", "Elena Baskakova", @@ -286,6 +290,7 @@ const char * auto_contributors[] { "Jochen Schalanda", "John", "John Hummel", + "John Skopis", "Jonatas Freitas", "Kang Liu", "Karl Pietrzak", @@ -395,6 +400,7 @@ const char * auto_contributors[] { "NeZeD [Mac Pro]", "Neeke Gao", "Neng Liu", + "Nickolay Yastrebov", "Nico Mandery", "Nico Piderman", "Nicolae Vartolomei", @@ -472,6 +478,7 @@ const char * auto_contributors[] { "Sami Kerola", "Samuel Chou", "Saulius Valatka", + "Serg Kulakov", "Serge Rider", "Sergei Bocharov", "Sergei Semin", @@ -606,6 +613,7 @@ const char * auto_contributors[] { "abyss7", "achimbab", "achulkov2", + "adevyatova", "ageraab", "akazz", "akonyaev", @@ -631,6 +639,7 @@ const char * auto_contributors[] { "artpaul", "asiana21", "avasiliev", + "avogar", "avsharapov", "awesomeleo", "benamazing", @@ -647,6 +656,8 @@ const char * auto_contributors[] { "centos7", "champtar", "chang.chen", + "changvvb", + "chasingegg", "chengy8934", "chenqi", "chenxing-xc", @@ -769,6 +780,7 @@ const char * auto_contributors[] { "maxim-babenko", "maxkuzn", "maxulan", + "mehanizm", "melin", "memo", "meo", @@ -831,6 +843,7 @@ const char * auto_contributors[] { "shangshujie", "shedx", "simon-says", + "songenjie", "spff", "spongedc", "spyros87", From 8ef84eeaa1af55d766b89d6f8ba6879abf2678ee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 20 Apr 2021 03:16:39 +0300 Subject: [PATCH 53/58] Add test for #7815 --- .../01823_array_low_cardinality_KuliginStepan.reference | 1 + .../01823_array_low_cardinality_KuliginStepan.sql | 7 +++++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/01823_array_low_cardinality_KuliginStepan.reference create mode 100644 tests/queries/0_stateless/01823_array_low_cardinality_KuliginStepan.sql diff --git a/tests/queries/0_stateless/01823_array_low_cardinality_KuliginStepan.reference b/tests/queries/0_stateless/01823_array_low_cardinality_KuliginStepan.reference new file mode 100644 index 00000000000..2439021d2e0 --- /dev/null +++ b/tests/queries/0_stateless/01823_array_low_cardinality_KuliginStepan.reference @@ -0,0 +1 @@ +[['a'],['b','c']] diff --git a/tests/queries/0_stateless/01823_array_low_cardinality_KuliginStepan.sql b/tests/queries/0_stateless/01823_array_low_cardinality_KuliginStepan.sql new file mode 100644 index 00000000000..528a3b464b3 --- /dev/null +++ b/tests/queries/0_stateless/01823_array_low_cardinality_KuliginStepan.sql @@ -0,0 +1,7 @@ +create temporary table test ( + arr Array(Array(LowCardinality(String))) +); + +insert into test(arr) values ([['a'], ['b', 'c']]); + +select arrayFilter(x -> 1, arr) from test; From 3381d0dce9c5f3b51a781962ebbf1205ed12123e Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 20 Apr 2021 03:20:36 +0300 Subject: [PATCH 54/58] Update 01602_max_distributed_connections.sh --- .../01602_max_distributed_connections.sh | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/01602_max_distributed_connections.sh b/tests/queries/0_stateless/01602_max_distributed_connections.sh index e13b6ed211a..772acb39344 100755 --- a/tests/queries/0_stateless/01602_max_distributed_connections.sh +++ b/tests/queries/0_stateless/01602_max_distributed_connections.sh @@ -7,19 +7,19 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # We check that even if max_threads is small, the setting max_distributed_connections # will allow to process queries on multiple shards concurrently. -# We do sleep 0.15 seconds on ten machines. -# If concurrency is one (bad) the query will take at least 1.5 seconds and the following loops are guaranteed to be infinite. -# If concurrency is 10 (good), the query may take less than 1 second with non-zero probability +# We do sleep 1.5 seconds on ten machines. +# If concurrency is one (bad) the query will take at least 15 seconds and the following loops are guaranteed to be infinite. +# If concurrency is 10 (good), the query may take less than 10 second with non-zero probability # and the following loops will finish with probability 1 assuming independent random variables. while true; do - timeout 1 ${CLICKHOUSE_CLIENT} --max_threads 1 --max_distributed_connections 10 --query " - SELECT sleep(0.15) FROM remote('127.{1..10}', system.one) FORMAT Null" --prefer_localhost_replica=0 && break + timeout 10 ${CLICKHOUSE_CLIENT} --max_threads 1 --max_distributed_connections 10 --query " + SELECT sleep(1.5) FROM remote('127.{1..10}', system.one) FORMAT Null" --prefer_localhost_replica=0 && break done while true; do - timeout 1 ${CLICKHOUSE_CLIENT} --max_threads 1 --max_distributed_connections 10 --query " - SELECT sleep(0.15) FROM remote('127.{1..10}', system.one) FORMAT Null" --prefer_localhost_replica=1 && break + timeout 10 ${CLICKHOUSE_CLIENT} --max_threads 1 --max_distributed_connections 10 --query " + SELECT sleep(1.5) FROM remote('127.{1..10}', system.one) FORMAT Null" --prefer_localhost_replica=1 && break done # If max_distributed_connections is low and async_socket_for_remote is disabled, From 62899436db3c9872f2b28b262539fcdb178cd508 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 20 Apr 2021 03:34:21 +0300 Subject: [PATCH 55/58] Revert "add uniqThetaSketch" --- contrib/datasketches-cpp | 2 +- docker/test/fasttest/run.sh | 2 - .../aggregate-functions/reference/uniq.md | 1 - .../reference/uniqcombined.md | 1 - .../reference/uniqexact.md | 1 - .../reference/uniqhll12.md | 1 - .../reference/uniqthetasketch.md | 39 --- .../AggregateFunctionUniq.cpp | 6 - .../AggregateFunctionUniq.h | 20 -- src/AggregateFunctions/ThetaSketchData.h | 119 --------- src/Common/config.h.in | 1 - .../RemoveInjectiveFunctionsVisitor.cpp | 3 +- .../01798_uniq_theta_sketch.reference | 230 ------------------ .../0_stateless/01798_uniq_theta_sketch.sql | 211 ---------------- .../01799_long_uniq_theta_sketch.reference | 219 ----------------- .../01799_long_uniq_theta_sketch.sql | 35 --- .../queries/0_stateless/arcadia_skip_list.txt | 2 - 17 files changed, 2 insertions(+), 891 deletions(-) delete mode 100644 docs/en/sql-reference/aggregate-functions/reference/uniqthetasketch.md delete mode 100644 src/AggregateFunctions/ThetaSketchData.h delete mode 100644 tests/queries/0_stateless/01798_uniq_theta_sketch.reference delete mode 100644 tests/queries/0_stateless/01798_uniq_theta_sketch.sql delete mode 100644 tests/queries/0_stateless/01799_long_uniq_theta_sketch.reference delete mode 100644 tests/queries/0_stateless/01799_long_uniq_theta_sketch.sql diff --git a/contrib/datasketches-cpp b/contrib/datasketches-cpp index 45885c0c8c0..f915d35b2de 160000 --- a/contrib/datasketches-cpp +++ b/contrib/datasketches-cpp @@ -1 +1 @@ -Subproject commit 45885c0c8c0807bb9480886d60ca7042000a4c43 +Subproject commit f915d35b2de676683493c86c585141a1e1c83334 diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 971cf3ead76..8d292e34eb3 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -312,8 +312,6 @@ function run_tests 01533_collate_in_nullable 01542_collate_in_array 01543_collate_in_tuple - 01798_uniq_theta_sketch - 01799_long_uniq_theta_sketch _orc_ arrow avro diff --git a/docs/en/sql-reference/aggregate-functions/reference/uniq.md b/docs/en/sql-reference/aggregate-functions/reference/uniq.md index f060b85c976..7ba2cdc6cb8 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/uniq.md +++ b/docs/en/sql-reference/aggregate-functions/reference/uniq.md @@ -38,4 +38,3 @@ We recommend using this function in almost all scenarios. - [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md#agg_function-uniqcombined64) - [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniqhll12.md#agg_function-uniqhll12) - [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact) -- [uniqThetaSketch](../../../sql-reference/aggregate-functions/reference/uniqthetasketch.md#agg_function-uniqthetasketch) diff --git a/docs/en/sql-reference/aggregate-functions/reference/uniqcombined.md b/docs/en/sql-reference/aggregate-functions/reference/uniqcombined.md index 7bd392ef870..4434686ae61 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/uniqcombined.md +++ b/docs/en/sql-reference/aggregate-functions/reference/uniqcombined.md @@ -49,4 +49,3 @@ Compared to the [uniq](../../../sql-reference/aggregate-functions/reference/uniq - [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md#agg_function-uniqcombined64) - [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniqhll12.md#agg_function-uniqhll12) - [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact) -- [uniqThetaSketch](../../../sql-reference/aggregate-functions/reference/uniqthetasketch.md#agg_function-uniqthetasketch) diff --git a/docs/en/sql-reference/aggregate-functions/reference/uniqexact.md b/docs/en/sql-reference/aggregate-functions/reference/uniqexact.md index d758c179d7a..eee675016ee 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/uniqexact.md +++ b/docs/en/sql-reference/aggregate-functions/reference/uniqexact.md @@ -23,4 +23,3 @@ The function takes a variable number of parameters. Parameters can be `Tuple`, ` - [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) - [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniqcombined) - [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniqhll12) -- [uniqThetaSketch](../../../sql-reference/aggregate-functions/reference/uniqthetasketch.md#agg_function-uniqthetasketch) diff --git a/docs/en/sql-reference/aggregate-functions/reference/uniqhll12.md b/docs/en/sql-reference/aggregate-functions/reference/uniqhll12.md index b65a0151e18..4983220ed7f 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/uniqhll12.md +++ b/docs/en/sql-reference/aggregate-functions/reference/uniqhll12.md @@ -37,4 +37,3 @@ We don’t recommend using this function. In most cases, use the [uniq](../../.. - [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) - [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md#agg_function-uniqcombined) - [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact) -- [uniqThetaSketch](../../../sql-reference/aggregate-functions/reference/uniqthetasketch.md#agg_function-uniqthetasketch) diff --git a/docs/en/sql-reference/aggregate-functions/reference/uniqthetasketch.md b/docs/en/sql-reference/aggregate-functions/reference/uniqthetasketch.md deleted file mode 100644 index dd744a34190..00000000000 --- a/docs/en/sql-reference/aggregate-functions/reference/uniqthetasketch.md +++ /dev/null @@ -1,39 +0,0 @@ ---- -toc_priority: 195 ---- - -# uniqThetaSketch {#agg_function-uniqthetasketch} - -Calculates the approximate number of different argument values, using the [Theta Sketch Framework](https://datasketches.apache.org/docs/Theta/ThetaSketchFramework.html). - -``` sql -uniqThetaSketch(x[, ...]) -``` - -**Arguments** - -The function takes a variable number of parameters. Parameters can be `Tuple`, `Array`, `Date`, `DateTime`, `String`, or numeric types. - -**Returned value** - -- A [UInt64](../../../sql-reference/data-types/int-uint.md)-type number. - -**Implementation details** - -Function: - -- Calculates a hash for all parameters in the aggregate, then uses it in calculations. - -- Uses the [KMV](https://datasketches.apache.org/docs/Theta/InverseEstimate.html) algorithm to approximate the number of different argument values. - - 4096(2^12) 64-bit sketch are used. The size of the state is about 41 KB. - -- The relative error is 3.125% (95% confidence), see the [relative error table](https://datasketches.apache.org/docs/Theta/ThetaErrorTable.html) for detail. - -**See Also** - -- [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) -- [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md#agg_function-uniqcombined) -- [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md#agg_function-uniqcombined64) -- [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniqhll12.md#agg_function-uniqhll12) -- [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact) diff --git a/src/AggregateFunctions/AggregateFunctionUniq.cpp b/src/AggregateFunctions/AggregateFunctionUniq.cpp index 7637e076d5b..32fdb188529 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.cpp +++ b/src/AggregateFunctions/AggregateFunctionUniq.cpp @@ -132,12 +132,6 @@ void registerAggregateFunctionsUniq(AggregateFunctionFactory & factory) factory.registerFunction("uniqExact", {createAggregateFunctionUniq>, properties}); - -#if USE_DATASKETCHES - factory.registerFunction("uniqThetaSketch", - {createAggregateFunctionUniq, properties}); -#endif - } } diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index 191fc873bb6..253af6e2895 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -22,7 +22,6 @@ #include #include -#include #include @@ -125,19 +124,6 @@ struct AggregateFunctionUniqExactData }; -/// uniqThetaSketch -#if USE_DATASKETCHES - -struct AggregateFunctionUniqThetaSketchData -{ - using Set = ThetaSketchData; - Set set; - - static String getName() { return "uniqThetaSketch"; } -}; - -#endif - namespace detail { @@ -203,12 +189,6 @@ struct OneAdder data.set.insert(key); } } -#if USE_DATASKETCHES - else if constexpr (std::is_same_v) - { - data.set.insertOriginal(column.getDataAt(row_num)); - } -#endif } }; diff --git a/src/AggregateFunctions/ThetaSketchData.h b/src/AggregateFunctions/ThetaSketchData.h deleted file mode 100644 index b2e32db6c75..00000000000 --- a/src/AggregateFunctions/ThetaSketchData.h +++ /dev/null @@ -1,119 +0,0 @@ -#pragma once - -#if !defined(ARCADIA_BUILD) -# include -#endif - -#if USE_DATASKETCHES - -#include -#include -#include // Y_IGNORE -#include // Y_IGNORE - - -namespace DB -{ - - -template -class ThetaSketchData : private boost::noncopyable -{ -private: - std::unique_ptr sk_update; - std::unique_ptr sk_union; - - inline datasketches::update_theta_sketch * getSkUpdate() - { - if (!sk_update) - sk_update = std::make_unique(datasketches::update_theta_sketch::builder().build()); - return sk_update.get(); - } - - inline datasketches::theta_union * getSkUnion() - { - if (!sk_union) - sk_union = std::make_unique(datasketches::theta_union::builder().build()); - return sk_union.get(); - } - -public: - using value_type = Key; - - ThetaSketchData() = default; - ~ThetaSketchData() = default; - - /// Insert original value without hash, as `datasketches::update_theta_sketch.update` will do the hash internal. - void insertOriginal(const StringRef & value) - { - getSkUpdate()->update(value.data, value.size); - } - - /// Note that `datasketches::update_theta_sketch.update` will do the hash again. - void insert(Key value) - { - getSkUpdate()->update(value); - } - - UInt64 size() const - { - if (sk_union) - return static_cast(sk_union->get_result().get_estimate()); - else if (sk_update) - return static_cast(sk_update->get_estimate()); - else - return 0; - } - - void merge(const ThetaSketchData & rhs) - { - datasketches::theta_union * u = getSkUnion(); - - if (sk_update) - { - u->update(*sk_update); - sk_update.reset(nullptr); - } - - if (rhs.sk_update) - u->update(*rhs.sk_update); - else if (rhs.sk_union) - u->update(rhs.sk_union->get_result()); - } - - /// You can only call for an empty object. - void read(DB::ReadBuffer & in) - { - datasketches::compact_theta_sketch::vector_bytes bytes; - readVectorBinary(bytes, in); - if (!bytes.empty()) - { - auto sk = datasketches::compact_theta_sketch::deserialize(bytes.data(), bytes.size()); - getSkUnion()->update(sk); - } - } - - void write(DB::WriteBuffer & out) const - { - if (sk_update) - { - auto bytes = sk_update->compact().serialize(); - writeVectorBinary(bytes, out); - } - else if (sk_union) - { - auto bytes = sk_union->get_result().serialize(); - writeVectorBinary(bytes, out); - } - else - { - datasketches::compact_theta_sketch::vector_bytes bytes; - writeVectorBinary(bytes, out); - } - } -}; - - -} - -#endif diff --git a/src/Common/config.h.in b/src/Common/config.h.in index 28a21ea7764..ee2bfe3df53 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -15,4 +15,3 @@ #cmakedefine01 USE_GRPC #cmakedefine01 USE_STATS #cmakedefine01 CLICKHOUSE_SPLIT_BINARY -#cmakedefine01 USE_DATASKETCHES diff --git a/src/Interpreters/RemoveInjectiveFunctionsVisitor.cpp b/src/Interpreters/RemoveInjectiveFunctionsVisitor.cpp index 1dca2db859b..f46e80a6370 100644 --- a/src/Interpreters/RemoveInjectiveFunctionsVisitor.cpp +++ b/src/Interpreters/RemoveInjectiveFunctionsVisitor.cpp @@ -12,8 +12,7 @@ namespace DB static bool isUniq(const ASTFunction & func) { return func.name == "uniq" || func.name == "uniqExact" || func.name == "uniqHLL12" - || func.name == "uniqCombined" || func.name == "uniqCombined64" - || func.name == "uniqThetaSketch"; + || func.name == "uniqCombined" || func.name == "uniqCombined64"; } /// Remove injective functions of one argument: replace with a child diff --git a/tests/queries/0_stateless/01798_uniq_theta_sketch.reference b/tests/queries/0_stateless/01798_uniq_theta_sketch.reference deleted file mode 100644 index 214d581f4b2..00000000000 --- a/tests/queries/0_stateless/01798_uniq_theta_sketch.reference +++ /dev/null @@ -1,230 +0,0 @@ -uniqThetaSketch many agrs -10 10 100 100 1000 1000 -17 10 10 100 100 610 610 766 -52 10 10 100 100 608 608 766 -5 10 10 100 100 608 608 765 -9 10 10 100 100 608 608 765 -13 10 10 100 100 607 607 765 -46 10 10 100 100 607 607 765 -48 10 10 100 100 609 609 765 -50 10 10 100 100 608 608 765 -54 10 10 100 100 609 609 765 -56 10 10 100 100 608 608 765 -uniqThetaSketch distinct -123 -143 -uniqThetaSketch arrays -2 -3 -3 -uniqThetaSketch complex types -3 -3 -3 -3 -3 -3 -3 -3 -3 -uniqThetaSketch decimals -(0,0,0) -(101,101,101) -uniqThetaSketch remove injective -SELECT uniqThetaSketch(x) -FROM -( - SELECT number % 2 AS x - FROM numbers(10) -) -SELECT uniqThetaSketch(x + y) -FROM -( - SELECT - number % 2 AS x, - number % 3 AS y - FROM numbers(10) -) -SELECT uniqThetaSketch(x) -FROM -( - SELECT number % 2 AS x - FROM numbers(10) -) -SELECT uniqThetaSketch(x) -FROM -( - SELECT number % 2 AS x - FROM numbers(10) -) -SELECT uniqThetaSketch(x) -FROM -( - SELECT number % 2 AS x - FROM numbers(10) -) -SELECT uniqThetaSketch(x) -FROM -( - SELECT number % 2 AS x - FROM numbers(10) -) -SELECT uniqThetaSketch(x) -FROM -( - SELECT number % 2 AS x - FROM numbers(10) -) -SELECT uniqThetaSketch(x + y) -FROM -( - SELECT - number % 2 AS x, - number % 3 AS y - FROM numbers(10) -) -SELECT uniqThetaSketch(-x) -FROM -( - SELECT number % 2 AS x - FROM numbers(10) -) -SELECT uniqThetaSketch(bitNot(x)) -FROM -( - SELECT number % 2 AS x - FROM numbers(10) -) -SELECT uniqThetaSketch(bitNot(-x)) -FROM -( - SELECT number % 2 AS x - FROM numbers(10) -) -SELECT uniqThetaSketch(-bitNot(-x)) -FROM -( - SELECT number % 2 AS x - FROM numbers(10) -) -1000 1000 -2014-06-01 1000 1000 -1000 1000 -2014-06-01 1000 1000 -2014-06-01 0 0 7 7 -2014-06-01 0 1 7 7 -2014-06-01 0 2 7 7 -2014-06-01 0 3 7 7 -2014-06-01 0 4 7 7 -2014-06-01 0 5 7 7 -2014-06-01 0 6 7 7 -2014-06-01 0 7 7 7 -2014-06-01 0 8 7 7 -2014-06-01 0 9 7 7 -2014-06-01 1 10 7 7 -2014-06-01 1 11 7 7 -2014-06-01 1 12 7 7 -2014-06-01 1 13 7 7 -2014-06-01 1 14 7 7 -2014-06-01 1 15 7 7 -2014-06-01 1 16 7 7 -2014-06-01 1 17 7 7 -2014-06-01 1 18 7 7 -2014-06-01 1 19 7 7 -2014-06-01 2 20 7 7 -2014-06-01 2 21 7 7 -2014-06-01 2 22 7 7 -2014-06-01 2 23 7 7 -2014-06-01 2 24 7 7 -2014-06-01 2 25 7 7 -2014-06-01 2 26 7 7 -2014-06-01 2 27 7 7 -2014-06-01 2 28 7 7 -2014-06-01 2 29 7 7 -2014-06-01 3 30 7 7 -2014-06-01 3 31 7 7 -2014-06-01 3 32 7 7 -2014-06-01 3 33 7 7 -2014-06-01 3 34 7 7 -2014-06-01 3 35 7 7 -2014-06-01 3 36 7 7 -2014-06-01 3 37 7 7 -2014-06-01 3 38 7 7 -2014-06-01 3 39 7 7 -2014-06-01 4 40 7 7 -2014-06-01 4 41 7 7 -2014-06-01 4 42 7 7 -2014-06-01 4 43 7 7 -2014-06-01 4 44 7 7 -2014-06-01 4 45 7 7 -2014-06-01 4 46 7 7 -2014-06-01 4 47 7 7 -2014-06-01 4 48 7 7 -2014-06-01 4 49 7 7 -2014-06-01 5 50 7 7 -2014-06-01 5 51 7 7 -2014-06-01 5 52 7 7 -2014-06-01 5 53 7 7 -2014-06-01 5 54 7 7 -2014-06-01 5 55 7 7 -2014-06-01 5 56 7 7 -2014-06-01 5 57 7 7 -2014-06-01 5 58 7 7 -2014-06-01 5 59 7 7 -2014-06-01 6 60 7 7 -2014-06-01 6 61 7 7 -2014-06-01 6 62 7 7 -2014-06-01 6 63 7 7 -2014-06-01 6 64 7 7 -2014-06-01 6 65 7 7 -2014-06-01 6 66 7 7 -2014-06-01 6 67 7 7 -2014-06-01 6 68 7 7 -2014-06-01 6 69 7 7 -2014-06-01 7 70 7 7 -2014-06-01 7 71 7 7 -2014-06-01 7 72 7 7 -2014-06-01 7 73 7 7 -2014-06-01 7 74 7 7 -2014-06-01 7 75 7 7 -2014-06-01 7 76 7 7 -2014-06-01 7 77 7 7 -2014-06-01 7 78 7 7 -2014-06-01 7 79 7 7 -2014-06-01 8 80 7 7 -2014-06-01 8 81 7 7 -2014-06-01 8 82 7 7 -2014-06-01 8 83 7 7 -2014-06-01 8 84 7 7 -2014-06-01 8 85 7 7 -2014-06-01 8 86 7 7 -2014-06-01 8 87 7 7 -2014-06-01 8 88 7 7 -2014-06-01 8 89 7 7 -2014-06-01 9 90 7 7 -2014-06-01 9 91 7 7 -2014-06-01 9 92 7 7 -2014-06-01 9 93 7 7 -2014-06-01 9 94 7 7 -2014-06-01 9 95 7 7 -2014-06-01 9 96 7 7 -2014-06-01 9 97 7 7 -2014-06-01 9 98 7 7 -2014-06-01 9 99 7 7 -2014-06-01 0 7 7 -2014-06-01 1 7 7 -2014-06-01 2 7 7 -2014-06-01 3 7 7 -2014-06-01 4 7 7 -2014-06-01 5 7 7 -2014-06-01 6 7 7 -2014-06-01 7 7 7 -2014-06-01 8 7 7 -2014-06-01 9 7 7 -2014-06-01 7 7 -0 333333 53 53 -1 333333 53 53 -2 333333 53 53 -0 333333 53 53 -1 333333 53 53 -2 333333 53 53 diff --git a/tests/queries/0_stateless/01798_uniq_theta_sketch.sql b/tests/queries/0_stateless/01798_uniq_theta_sketch.sql deleted file mode 100644 index 0093eb4c345..00000000000 --- a/tests/queries/0_stateless/01798_uniq_theta_sketch.sql +++ /dev/null @@ -1,211 +0,0 @@ -SELECT 'uniqThetaSketch many agrs'; - -SELECT - uniqThetaSketch(x), uniqThetaSketch((x)), uniqThetaSketch(x, y), uniqThetaSketch((x, y)), uniqThetaSketch(x, y, z), uniqThetaSketch((x, y, z)) -FROM -( - SELECT - number % 10 AS x, - intDiv(number, 10) % 10 AS y, - toString(intDiv(number, 100) % 10) AS z - FROM system.numbers LIMIT 1000 -); - - -SELECT k, - uniqThetaSketch(x), uniqThetaSketch((x)), uniqThetaSketch(x, y), uniqThetaSketch((x, y)), uniqThetaSketch(x, y, z), uniqThetaSketch((x, y, z)), - count() AS c -FROM -( - SELECT - (number + 0x8ffcbd8257219a26) * 0x66bb3430c06d2353 % 131 AS k, - number % 10 AS x, - intDiv(number, 10) % 10 AS y, - toString(intDiv(number, 100) % 10) AS z - FROM system.numbers LIMIT 100000 -) -GROUP BY k -ORDER BY c DESC, k ASC -LIMIT 10; - - -SELECT 'uniqThetaSketch distinct'; - -SET count_distinct_implementation = 'uniqThetaSketch'; -SELECT count(DISTINCT x) FROM (SELECT number % 123 AS x FROM system.numbers LIMIT 1000); -SELECT count(DISTINCT x, y) FROM (SELECT number % 11 AS x, number % 13 AS y FROM system.numbers LIMIT 1000); - - -SELECT 'uniqThetaSketch arrays'; - -SELECT uniqThetaSketchArray([0, 1, 1], [0, 1, 1], [0, 1, 1]); -SELECT uniqThetaSketchArray([0, 1, 1], [0, 1, 1], [0, 1, 0]); -SELECT uniqThetaSketch(x) FROM (SELECT arrayJoin([[1, 2], [1, 2], [1, 2, 3], []]) AS x); - - -SELECT 'uniqThetaSketch complex types'; - -SELECT uniqThetaSketch(x) FROM (SELECT arrayJoin([[], ['a'], ['a', 'b'], []]) AS x); -SELECT uniqThetaSketch(x) FROM (SELECT arrayJoin([[[]], [['a', 'b']], [['a'], ['b']], [['a', 'b']]]) AS x); -SELECT uniqThetaSketch(x, x) FROM (SELECT arrayJoin([[], ['a'], ['a', 'b'], []]) AS x); -SELECT uniqThetaSketch(x, arrayMap(elem -> [elem, elem], x)) FROM (SELECT arrayJoin([[], ['a'], ['a', 'b'], []]) AS x); -SELECT uniqThetaSketch(x, toString(x)) FROM (SELECT arrayJoin([[], ['a'], ['a', 'b'], []]) AS x); -SELECT uniqThetaSketch((x, x)) FROM (SELECT arrayJoin([[], ['a'], ['a', 'b'], []]) AS x); -SELECT uniqThetaSketch((x, arrayMap(elem -> [elem, elem], x))) FROM (SELECT arrayJoin([[], ['a'], ['a', 'b'], []]) AS x); -SELECT uniqThetaSketch((x, toString(x))) FROM (SELECT arrayJoin([[], ['a'], ['a', 'b'], []]) AS x); -SELECT uniqThetaSketch(x) FROM (SELECT arrayJoin([[], ['a'], ['a', NULL, 'b'], []]) AS x); - - -SELECT 'uniqThetaSketch decimals'; - -DROP TABLE IF EXISTS decimal; -CREATE TABLE decimal -( - a Decimal32(4), - b Decimal64(8), - c Decimal128(8) -) ENGINE = Memory; - -SELECT (uniqThetaSketch(a), uniqThetaSketch(b), uniqThetaSketch(c)) -FROM (SELECT * FROM decimal ORDER BY a); - -INSERT INTO decimal (a, b, c) -SELECT toDecimal32(number - 50, 4), toDecimal64(number - 50, 8) / 3, toDecimal128(number - 50, 8) / 5 -FROM system.numbers LIMIT 101; - -SELECT (uniqThetaSketch(a), uniqThetaSketch(b), uniqThetaSketch(c)) -FROM (SELECT * FROM decimal ORDER BY a); - -DROP TABLE decimal; - - -SELECT 'uniqThetaSketch remove injective'; - -set optimize_injective_functions_inside_uniq = 1; - -EXPLAIN SYNTAX select uniqThetaSketch(x) from (select number % 2 as x from numbers(10)); -EXPLAIN SYNTAX select uniqThetaSketch(x + y) from (select number % 2 as x, number % 3 y from numbers(10)); -EXPLAIN SYNTAX select uniqThetaSketch(-x) from (select number % 2 as x from numbers(10)); -EXPLAIN SYNTAX select uniqThetaSketch(bitNot(x)) from (select number % 2 as x from numbers(10)); -EXPLAIN SYNTAX select uniqThetaSketch(bitNot(-x)) from (select number % 2 as x from numbers(10)); -EXPLAIN SYNTAX select uniqThetaSketch(-bitNot(-x)) from (select number % 2 as x from numbers(10)); - -set optimize_injective_functions_inside_uniq = 0; - -EXPLAIN SYNTAX select uniqThetaSketch(x) from (select number % 2 as x from numbers(10)); -EXPLAIN SYNTAX select uniqThetaSketch(x + y) from (select number % 2 as x, number % 3 y from numbers(10)); -EXPLAIN SYNTAX select uniqThetaSketch(-x) from (select number % 2 as x from numbers(10)); -EXPLAIN SYNTAX select uniqThetaSketch(bitNot(x)) from (select number % 2 as x from numbers(10)); -EXPLAIN SYNTAX select uniqThetaSketch(bitNot(-x)) from (select number % 2 as x from numbers(10)); -EXPLAIN SYNTAX select uniqThetaSketch(-bitNot(-x)) from (select number % 2 as x from numbers(10)); - - -DROP TABLE IF EXISTS stored_aggregates; - --- simple -CREATE TABLE stored_aggregates -( - d Date, - Uniq AggregateFunction(uniq, UInt64), - UniqThetaSketch AggregateFunction(uniqThetaSketch, UInt64) -) -ENGINE = AggregatingMergeTree(d, d, 8192); - -INSERT INTO stored_aggregates -SELECT - toDate('2014-06-01') AS d, - uniqState(number) AS Uniq, - uniqThetaSketchState(number) AS UniqThetaSketch -FROM -( - SELECT * FROM system.numbers LIMIT 1000 -); - -SELECT uniqMerge(Uniq), uniqThetaSketchMerge(UniqThetaSketch) FROM stored_aggregates; - -SELECT d, uniqMerge(Uniq), uniqThetaSketchMerge(UniqThetaSketch) FROM stored_aggregates GROUP BY d ORDER BY d; - -OPTIMIZE TABLE stored_aggregates; - -SELECT uniqMerge(Uniq), uniqThetaSketchMerge(UniqThetaSketch) FROM stored_aggregates; - -SELECT d, uniqMerge(Uniq), uniqThetaSketchMerge(UniqThetaSketch) FROM stored_aggregates GROUP BY d ORDER BY d; - -DROP TABLE stored_aggregates; - --- complex -CREATE TABLE stored_aggregates -( - d Date, - k1 UInt64, - k2 String, - Uniq AggregateFunction(uniq, UInt64), - UniqThetaSketch AggregateFunction(uniqThetaSketch, UInt64) -) -ENGINE = AggregatingMergeTree(d, (d, k1, k2), 8192); - -INSERT INTO stored_aggregates -SELECT - toDate('2014-06-01') AS d, - intDiv(number, 100) AS k1, - toString(intDiv(number, 10)) AS k2, - uniqState(toUInt64(number % 7)) AS Uniq, - uniqThetaSketchState(toUInt64(number % 7)) AS UniqThetaSketch -FROM -( - SELECT * FROM system.numbers LIMIT 1000 -) -GROUP BY d, k1, k2 -ORDER BY d, k1, k2; - -SELECT d, k1, k2, - uniqMerge(Uniq), uniqThetaSketchMerge(UniqThetaSketch) -FROM stored_aggregates -GROUP BY d, k1, k2 -ORDER BY d, k1, k2; - -SELECT d, k1, - uniqMerge(Uniq), uniqThetaSketchMerge(UniqThetaSketch) -FROM stored_aggregates -GROUP BY d, k1 -ORDER BY d, k1; - -SELECT d, - uniqMerge(Uniq), uniqThetaSketchMerge(UniqThetaSketch) -FROM stored_aggregates -GROUP BY d -ORDER BY d; - -DROP TABLE stored_aggregates; - ----- sum + uniq with more data -drop table if exists summing_merge_tree_null; -drop table if exists summing_merge_tree_aggregate_function; -create table summing_merge_tree_null ( - d materialized today(), - k UInt64, - c UInt64, - u UInt64 -) engine=Null; - -create materialized view summing_merge_tree_aggregate_function ( - d Date, - k UInt64, - c UInt64, - un AggregateFunction(uniq, UInt64), - ut AggregateFunction(uniqThetaSketch, UInt64) -) engine=SummingMergeTree(d, k, 8192) -as select d, k, sum(c) as c, uniqState(u) as un, uniqThetaSketchState(u) as ut -from summing_merge_tree_null -group by d, k; - --- prime number 53 to avoid resonanse between %3 and %53 -insert into summing_merge_tree_null select number % 3, 1, number % 53 from numbers(999999); - -select k, sum(c), uniqMerge(un), uniqThetaSketchMerge(ut) from summing_merge_tree_aggregate_function group by k order by k; -optimize table summing_merge_tree_aggregate_function; -select k, sum(c), uniqMerge(un), uniqThetaSketchMerge(ut) from summing_merge_tree_aggregate_function group by k order by k; - -drop table summing_merge_tree_aggregate_function; -drop table summing_merge_tree_null; - diff --git a/tests/queries/0_stateless/01799_long_uniq_theta_sketch.reference b/tests/queries/0_stateless/01799_long_uniq_theta_sketch.reference deleted file mode 100644 index c3f96cab493..00000000000 --- a/tests/queries/0_stateless/01799_long_uniq_theta_sketch.reference +++ /dev/null @@ -1,219 +0,0 @@ -uniqThetaSketch -1 1 -3 1 -6 1 -7 1 -9 1 -11 1 -14 1 -17 1 -19 1 -20 2 -26 1 -31 1 -35 1 -36 1 -0 162 -1 162 -3 162 -6 162 -7 163 -9 163 -10 81 -11 163 -13 162 -14 162 -17 162 -19 162 -20 162 -21 162 -22 162 -26 162 -31 162 -35 162 -36 162 -0 55018 -1 54020 -3 53774 -6 53947 -7 53839 -9 54408 -10 26876 -11 54985 -13 53479 -14 53516 -17 53331 -19 53680 -20 54211 -21 53054 -22 54690 -26 53716 -31 54139 -35 52331 -36 53766 -uniqThetaSketch round(float) -0.125 1 -0.5 1 -0.05 1 -0.143 1 -0.056 1 -0.048 2 -0.083 1 -0.25 1 -0.1 1 -0.028 1 -0.027 1 -0.031 1 -0.067 1 -0.037 1 -0.045 162 -0.125 163 -0.5 162 -0.05 162 -0.143 162 -0.091 81 -0.056 162 -0.048 162 -0.083 163 -0.25 162 -1 162 -0.1 163 -0.028 162 -0.027 162 -0.031 162 -0.067 162 -0.043 162 -0.037 162 -0.071 162 -0.045 53054 -0.125 53839 -0.5 54020 -0.05 53680 -0.143 53947 -0.091 26876 -0.056 53331 -0.048 54211 -0.083 54985 -0.25 53774 -1 55018 -0.1 54408 -0.028 52331 -0.027 53766 -0.031 54139 -0.067 53516 -0.043 54690 -0.037 53716 -0.071 53479 -uniqThetaSketch round(toFloat32()) -0.5 1 -0.05 1 -0.25 1 -0.048 2 -0.083 1 -0.125 1 -0.031 1 -0.143 1 -0.028 1 -0.067 1 -0.027 1 -0.056 1 -0.037 1 -0.1 1 -0.5 162 -0.05 162 -0.25 162 -0.048 162 -0.091 81 -0.043 162 -0.071 162 -0.083 163 -0.125 163 -0.031 162 -0.143 162 -0.028 162 -0.067 162 -0.045 162 -0.027 162 -0.056 162 -0.037 162 -0.1 163 -1 162 -0.5 54020 -0.05 53680 -0.25 53774 -0.048 54211 -0.091 26876 -0.043 54690 -0.071 53479 -0.083 54985 -0.125 53839 -0.031 54139 -0.143 53947 -0.028 52331 -0.067 53516 -0.045 53054 -0.027 53766 -0.056 53331 -0.037 53716 -0.1 54408 -1 55018 -uniqThetaSketch IPv4NumToString -1 1 -3 1 -6 1 -7 1 -9 1 -11 1 -14 1 -17 1 -19 1 -20 2 -26 1 -31 1 -35 1 -36 1 -0 162 -1 162 -3 162 -6 162 -7 163 -9 163 -10 81 -11 163 -13 162 -14 162 -17 162 -19 162 -20 162 -21 162 -22 162 -26 162 -31 162 -35 162 -36 162 -0 54929 -1 53802 -3 54706 -6 54700 -7 53592 -9 54036 -10 27392 -11 53768 -13 54566 -14 53104 -17 54243 -19 55003 -20 53398 -21 53831 -22 54603 -26 54607 -31 54012 -35 54826 -36 54910 -uniqThetaSketch remote() -1 -uniqThetaSketch precise -10000000 -10021957 -10021969 -10094819 diff --git a/tests/queries/0_stateless/01799_long_uniq_theta_sketch.sql b/tests/queries/0_stateless/01799_long_uniq_theta_sketch.sql deleted file mode 100644 index 572c26634bb..00000000000 --- a/tests/queries/0_stateless/01799_long_uniq_theta_sketch.sql +++ /dev/null @@ -1,35 +0,0 @@ -SELECT 'uniqThetaSketch'; - -SELECT Y, uniqThetaSketch(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; -SELECT Y, uniqThetaSketch(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; -SELECT Y, uniqThetaSketch(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; - -SELECT 'uniqThetaSketch round(float)'; - -SELECT Y, uniqThetaSketch(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; -SELECT Y, uniqThetaSketch(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; -SELECT Y, uniqThetaSketch(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; - -SELECT 'uniqThetaSketch round(toFloat32())'; - -SELECT Y, uniqThetaSketch(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; -SELECT Y, uniqThetaSketch(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; -SELECT Y, uniqThetaSketch(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; - -SELECT 'uniqThetaSketch IPv4NumToString'; - -SELECT Y, uniqThetaSketch(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; -SELECT Y, uniqThetaSketch(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; -SELECT Y, uniqThetaSketch(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; - -SELECT 'uniqThetaSketch remote()'; - -SELECT uniqThetaSketch(dummy) FROM remote('127.0.0.{2,3}', system.one); - - -SELECT 'uniqThetaSketch precise'; -SELECT uniqExact(number) FROM numbers(1e7); -SELECT uniqCombined(number) FROM numbers(1e7); -SELECT uniqCombined64(number) FROM numbers(1e7); -SELECT uniqThetaSketch(number) FROM numbers(1e7); - diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 14d6dd09ddc..f435c00a989 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -228,8 +228,6 @@ 01780_clickhouse_dictionary_source_loop 01790_dist_INSERT_block_structure_mismatch_types_and_names 01791_dist_INSERT_block_structure_mismatch -01798_uniq_theta_sketch -01799_long_uniq_theta_sketch 01801_distinct_group_by_shard 01804_dictionary_decimal256_type 01801_s3_distributed From e60af08ce3b2cfffcf64453119caca109436d1f9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 20 Apr 2021 04:29:19 +0300 Subject: [PATCH 56/58] Fix flaky test 01666_merge_tree_max_query_limit.sh --- tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh index 5bb93371483..c5fbb35a9cd 100755 --- a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh +++ b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh @@ -15,13 +15,13 @@ drop table if exists simple; create table simple (i int, j int) engine = MergeTree order by i settings index_granularity = 1, max_concurrent_queries = 1, min_marks_to_honor_max_concurrent_queries = 2; -insert into simple select number, number + 100 from numbers(1000); +insert into simple select number, number + 100 from numbers(5000); " query_id="long_running_query-$CLICKHOUSE_DATABASE" echo "Spin up a long running query" -${CLICKHOUSE_CLIENT} --query "select sleepEachRow(0.01) from simple settings max_block_size = 1 format Null" --query_id "$query_id" > /dev/null 2>&1 & +${CLICKHOUSE_CLIENT} --query "select sleepEachRow(0.1) from simple settings max_block_size = 1 format Null" --query_id "$query_id" > /dev/null 2>&1 & wait_for_query_to_start "$query_id" # query which reads marks >= min_marks_to_honor_max_concurrent_queries is throttled From 0a0c510aa21066888cef1294828d443c5ccdf5d2 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 16 Apr 2021 17:19:22 +0300 Subject: [PATCH 57/58] minor fixes --- src/Functions/validateNestedArraySizes.cpp | 13 ++-- src/Interpreters/MutationsInterpreter.cpp | 50 ++++++++------- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 6 +- ...date_nested_type_subcolumn_check.reference | 9 +-- ...788_update_nested_type_subcolumn_check.sql | 64 +++++++++++++++---- 5 files changed, 91 insertions(+), 51 deletions(-) diff --git a/src/Functions/validateNestedArraySizes.cpp b/src/Functions/validateNestedArraySizes.cpp index 6b651ec933e..b589b1e2e5c 100644 --- a/src/Functions/validateNestedArraySizes.cpp +++ b/src/Functions/validateNestedArraySizes.cpp @@ -73,16 +73,16 @@ ColumnPtr FunctionValidateNestedArraySizes::executeImpl( size_t length = 0; 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; - if (const auto *const_array = checkAndGetColumnConst(current_arg.column.get())) + 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()); + current_column = checkAndGetColumn(current_arg.column.get()); const auto & offsets = current_column->getOffsets(); length = offsets[i] - offsets[i - 1]; } @@ -102,12 +102,7 @@ ColumnPtr FunctionValidateNestedArraySizes::executeImpl( } } - 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; + return ColumnUInt8::create(input_rows_count, 1); } void registerFunctionValidateNestedArraySizes(FunctionFactory & factory) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index d403352019c..1315f9efa05 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -350,33 +350,34 @@ static void validateUpdateColumns( } } -std::pair> getFullNestedSubColumnUpdatedExpr( - const String & column, - NamesAndTypesList & all_columns, - std::unordered_map & column_to_update_expression) +/// Returns ASTs of updated nested subcolumns, if all of subcolumns were updated. +/// They are used to validate sizes of nested arrays. +/// If some of subcolumns were updated and some weren't, +/// 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> getExpressionsOfUpdatedNestedSubcolumns( + const String & column_name, + const NamesAndTypesList & all_columns, + const std::unordered_map & column_to_update_expression) { std::vector res; - auto source_name = Nested::splitName(column).first; + auto source_name = Nested::splitName(column_name).first; /// Check this nested subcolumn - for (const auto & it : all_columns) + for (const auto & column : all_columns) { - auto split = Nested::splitName(it.name); - if (split.first == source_name && !split.second.empty()) + auto split = Nested::splitName(column.name); + if (isArray(column.type) && 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]); - } + auto it = column_to_update_expression.find(column.name); + if (it == column_to_update_expression.end()) + return {}; + + res.push_back(it->second); } } - return std::make_pair(true, res); + return res; } 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'. - auto type_literal = std::make_shared(columns_desc.getPhysical(column).type->getName()); + const auto & type = columns_desc.getPhysical(column).type; + auto type_literal = std::make_shared(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()) + if (isArray(type) && !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) + auto nested_update_exprs = getExpressionsOfUpdatedNestedSubcolumns(column, all_columns, command.column_to_update_expression); + if (!nested_update_exprs) { function = makeASTFunction("validateNestedArraySizes", condition, @@ -487,14 +489,14 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) std::make_shared(column)); condition = makeASTFunction("and", condition, function); } - else if (nested_update_exprs.second.size() > 1) + else if (nested_update_exprs->size() > 1) { 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 (const auto & it : nested_update_exprs.second) + for (const auto & it : *nested_update_exprs) function->arguments->children.push_back(it->clone()); condition = makeASTFunction("and", condition, function); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index a1691fe7931..57e8cca46cd 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -3,6 +3,7 @@ #include #include #include +#include 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()); auto disk = data_part->volume->getDisk(); - String mrk_path = fullPath(disk, part_path + name + marks_file_extension); - String bin_path = fullPath(disk, part_path + name + DATA_FILE_EXTENSION); + String escaped_name = escapeForFileName(name); + 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::CompressedReadBufferFromFile bin_in(bin_path, 0, 0, 0, nullptr); bool must_be_last = false; 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 index ffa2b2a735f..c6f75cab8b7 100644 --- a/tests/queries/0_stateless/01788_update_nested_type_subcolumn_check.reference +++ b/tests/queries/0_stateless/01788_update_nested_type_subcolumn_check.reference @@ -4,10 +4,6 @@ 2 [100,200,300] ['a','b','c'] [10,20,30] 3 [3,4] ['aa','bb'] [3,6] 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] 1 [100,200] ['aa','bb'] [1,2] 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] 3 [68,72] ['aa','bb'] [68,72] 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 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 index 45345da2ff1..8e850b70c24 100644 --- a/tests/queries/0_stateless/01788_update_nested_type_subcolumn_check.sql +++ b/tests/queries/0_stateless/01788_update_nested_type_subcolumn_check.sql @@ -1,4 +1,3 @@ -USE default; DROP TABLE IF EXISTS test_wide_nested; CREATE TABLE test_wide_nested @@ -12,19 +11,60 @@ 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); +set mutations_sync = 1; -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; +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; +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; -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`=[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; + +alter table test_wide_nested update `info.id` = [100,200,300], `info.age` = `info.id`, `info.name` = ['a','b','c'] where id = 2; 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; + +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; + +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; From a25e6d96a504085f67f161470d4c349aa52930c8 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 20 Apr 2021 15:10:34 +0300 Subject: [PATCH 58/58] Update quantiletdigest.md --- .../aggregate-functions/reference/quantiletdigest.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/quantiletdigest.md b/docs/en/sql-reference/aggregate-functions/reference/quantiletdigest.md index dcc665a68af..dd0d59978d1 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/quantiletdigest.md +++ b/docs/en/sql-reference/aggregate-functions/reference/quantiletdigest.md @@ -6,7 +6,7 @@ toc_priority: 207 Computes an approximate [quantile](https://en.wikipedia.org/wiki/Quantile) of a numeric data sequence using the [t-digest](https://github.com/tdunning/t-digest/blob/master/docs/t-digest-paper/histo.pdf) algorithm. -The maximum error is 1%. Memory consumption is `log(n)`, where `n` is a number of values. The result depends on the order of running the query, and is nondeterministic. +Memory consumption is `log(n)`, where `n` is a number of values. The result depends on the order of running the query, and is nondeterministic. The performance of the function is lower than performance of [quantile](../../../sql-reference/aggregate-functions/reference/quantile.md#quantile) or [quantileTiming](../../../sql-reference/aggregate-functions/reference/quantiletiming.md#quantiletiming). In terms of the ratio of State size to precision, this function is much better than `quantile`.