From 92777691b59d02654c09ab685124ead47852e5b9 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 17 May 2022 10:16:40 +0000 Subject: [PATCH 001/110] fix check for non-deterministic functions in TTL expressions --- .../MergeTree/registerStorageMergeTree.cpp | 4 +-- src/Storages/TTLDescription.cpp | 26 ++++++------------- src/Storages/TTLDescription.h | 7 +++-- .../0_stateless/00933_ttl_simple.reference | 8 +++--- .../queries/0_stateless/00933_ttl_simple.sql | 10 ++++--- .../02296_ttl_non_deterministic.reference | 0 .../02296_ttl_non_deterministic.sql | 15 +++++++++++ 7 files changed, 38 insertions(+), 32 deletions(-) create mode 100644 tests/queries/0_stateless/02296_ttl_non_deterministic.reference create mode 100644 tests/queries/0_stateless/02296_ttl_non_deterministic.sql diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 6d8c3b313d4..3219f26aeaa 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -570,7 +570,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (args.storage_def->ttl_table) { metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST( - args.storage_def->ttl_table->ptr(), metadata.columns, args.getContext(), metadata.primary_key); + args.storage_def->ttl_table->ptr(), metadata.columns, args.getContext(), metadata.primary_key, args.attach); } if (args.query.columns_list && args.query.columns_list->indices) @@ -593,7 +593,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) auto column_ttl_asts = columns.getColumnTTLs(); for (const auto & [name, ast] : column_ttl_asts) { - auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, columns, args.getContext(), metadata.primary_key); + auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, columns, args.getContext(), metadata.primary_key, args.attach); metadata.column_ttls_by_name[name] = new_ttl_entry; } diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index b745da13484..1479545e103 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -57,22 +57,9 @@ namespace void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const String & result_column_name) { - for (const auto & action : ttl_expression->getActions()) - { - if (action.node->type == ActionsDAG::ActionType::FUNCTION) - { - IFunctionBase & func = *action.node->function_base; - if (!func.isDeterministic()) - throw Exception( - "TTL expression cannot contain non-deterministic functions, " - "but contains function " - + func.getName(), - ErrorCodes::BAD_ARGUMENTS); - } - } + ttl_expression->getActionsDAG().assertDeterministic(); const auto & result_column = ttl_expression->getSampleBlock().getByName(result_column_name); - if (!typeid_cast(result_column.type.get()) && !typeid_cast(result_column.type.get())) { @@ -164,7 +151,8 @@ TTLDescription TTLDescription::getTTLFromAST( const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context, - const KeyDescription & primary_key) + const KeyDescription & primary_key, + bool is_attach) { TTLDescription result; const auto * ttl_element = definition_ast->as(); @@ -295,7 +283,8 @@ TTLDescription TTLDescription::getTTLFromAST( } } - checkTTLExpression(result.expression, result.result_column); + if (!is_attach) + checkTTLExpression(result.expression, result.result_column); return result; } @@ -333,7 +322,8 @@ TTLTableDescription TTLTableDescription::getTTLForTableFromAST( const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context, - const KeyDescription & primary_key) + const KeyDescription & primary_key, + bool is_attach) { TTLTableDescription result; if (!definition_ast) @@ -344,7 +334,7 @@ TTLTableDescription TTLTableDescription::getTTLForTableFromAST( bool have_unconditional_delete_ttl = false; for (const auto & ttl_element_ptr : definition_ast->children) { - auto ttl = TTLDescription::getTTLFromAST(ttl_element_ptr, columns, context, primary_key); + auto ttl = TTLDescription::getTTLFromAST(ttl_element_ptr, columns, context, primary_key, is_attach); if (ttl.mode == TTLMode::DELETE) { if (!ttl.where_expression) diff --git a/src/Storages/TTLDescription.h b/src/Storages/TTLDescription.h index 8f60eb604b5..91ef7b44d37 100644 --- a/src/Storages/TTLDescription.h +++ b/src/Storages/TTLDescription.h @@ -82,9 +82,8 @@ struct TTLDescription /// Codec name which will be used to recompress data ASTPtr recompression_codec; - /// Parse TTL structure from definition. Able to parse both column and table - /// TTLs. - static TTLDescription getTTLFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context, const KeyDescription & primary_key); + /// Parse TTL structure from definition. Able to parse both column and table TTLs. + static TTLDescription getTTLFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context, const KeyDescription & primary_key, bool is_attach = false); TTLDescription() = default; TTLDescription(const TTLDescription & other); @@ -121,7 +120,7 @@ struct TTLTableDescription TTLTableDescription & operator=(const TTLTableDescription & other); static TTLTableDescription getTTLForTableFromAST( - const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context, const KeyDescription & primary_key); + const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context, const KeyDescription & primary_key, bool is_attach = false); /// Parse description from string static TTLTableDescription parse(const String & str, const ColumnsDescription & columns, ContextPtr context, const KeyDescription & primary_key); diff --git a/tests/queries/0_stateless/00933_ttl_simple.reference b/tests/queries/0_stateless/00933_ttl_simple.reference index e3982814eab..72f5134e235 100644 --- a/tests/queries/0_stateless/00933_ttl_simple.reference +++ b/tests/queries/0_stateless/00933_ttl_simple.reference @@ -6,11 +6,11 @@ 2000-10-10 00:00:00 0 2100-10-10 00:00:00 3 2100-10-10 2 -CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL now() - 1000\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL CAST(\'2000-10-10 00:00:00\', \'DateTime\')\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 1 0 -CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL now() + 1000\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL CAST(\'2100-10-10 00:00:00\', \'DateTime\')\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 1 1 -CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL today() - 1\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL CAST(\'2000-10-10\', \'Date\')\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 1 0 -CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL today() + 1\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 +CREATE TABLE default.ttl_00933_1\n(\n `b` Int32,\n `a` Int32 TTL CAST(\'2100-10-10\', \'Date\')\n)\nENGINE = MergeTree\nPARTITION BY tuple()\nORDER BY tuple()\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192 1 1 diff --git a/tests/queries/0_stateless/00933_ttl_simple.sql b/tests/queries/0_stateless/00933_ttl_simple.sql index 2bf686822d5..1cd41c0a113 100644 --- a/tests/queries/0_stateless/00933_ttl_simple.sql +++ b/tests/queries/0_stateless/00933_ttl_simple.sql @@ -53,7 +53,9 @@ select * from ttl_00933_1 order by d; -- const DateTime TTL positive drop table if exists ttl_00933_1; -create table ttl_00933_1 (b Int, a Int ttl now()-1000) engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0; +create table ttl_00933_1 (b Int, a Int ttl '2000-10-10 00:00:00'::DateTime) +engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0; + show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); optimize table ttl_00933_1 final; @@ -61,7 +63,7 @@ select * from ttl_00933_1; -- const DateTime TTL negative drop table if exists ttl_00933_1; -create table ttl_00933_1 (b Int, a Int ttl now()+1000) engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0; +create table ttl_00933_1 (b Int, a Int ttl '2100-10-10 00:00:00'::DateTime) engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0; show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); optimize table ttl_00933_1 final; @@ -69,7 +71,7 @@ select * from ttl_00933_1; -- const Date TTL positive drop table if exists ttl_00933_1; -create table ttl_00933_1 (b Int, a Int ttl today()-1) engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0; +create table ttl_00933_1 (b Int, a Int ttl '2000-10-10'::Date) engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0; show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); optimize table ttl_00933_1 final; @@ -77,7 +79,7 @@ select * from ttl_00933_1; -- const Date TTL negative drop table if exists ttl_00933_1; -create table ttl_00933_1 (b Int, a Int ttl today()+1) engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0; +create table ttl_00933_1 (b Int, a Int ttl '2100-10-10'::Date) engine = MergeTree order by tuple() partition by tuple() settings min_bytes_for_wide_part = 0; show create table ttl_00933_1; insert into ttl_00933_1 values (1, 1); optimize table ttl_00933_1 final; diff --git a/tests/queries/0_stateless/02296_ttl_non_deterministic.reference b/tests/queries/0_stateless/02296_ttl_non_deterministic.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02296_ttl_non_deterministic.sql b/tests/queries/0_stateless/02296_ttl_non_deterministic.sql new file mode 100644 index 00000000000..b5667aafcab --- /dev/null +++ b/tests/queries/0_stateless/02296_ttl_non_deterministic.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS t_ttl_non_deterministic; + +CREATE TABLE t_ttl_non_deterministic(A Int64) +ENGINE = MergeTree ORDER BY A TTL now() + toIntervalMonth(1); -- {serverError BAD_ARGUMENTS} + +CREATE TABLE t_ttl_non_deterministic(A Int64) ENGINE = MergeTree ORDER BY A; +ALTER TABLE t_ttl_non_deterministic MODIFY TTL now() + toIntervalMonth(1); -- {serverError BAD_ARGUMENTS} +DROP TABLE t_ttl_non_deterministic; + +CREATE TABLE t_ttl_non_deterministic(A Int64, B Int64 TTL now() + toIntervalMonth(1)) +ENGINE = MergeTree ORDER BY A; -- {serverError BAD_ARGUMENTS} + +CREATE TABLE t_ttl_non_deterministic(A Int64, B Int64) ENGINE = MergeTree ORDER BY A; +ALTER TABLE t_ttl_non_deterministic MODIFY COLUMN B Int64 TTL now() + toIntervalMonth(1); -- {serverError BAD_ARGUMENTS} +DROP TABLE t_ttl_non_deterministic; From ef08d24347ddb220d790469613c107d89aafd2b4 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 18 May 2022 12:10:52 +0000 Subject: [PATCH 002/110] retrun back some checks for TTL expressions --- src/Storages/TTLDescription.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 1479545e103..0ea9e378913 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -55,9 +55,11 @@ TTLAggregateDescription & TTLAggregateDescription::operator=(const TTLAggregateD namespace { -void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const String & result_column_name) +void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const String & result_column_name, bool is_attach) { - ttl_expression->getActionsDAG().assertDeterministic(); + /// Do not apply this check in ATTACH queries for compatibility reasons. + if (!is_attach) + ttl_expression->getActionsDAG().assertDeterministic(); const auto & result_column = ttl_expression->getSampleBlock().getByName(result_column_name); if (!typeid_cast(result_column.type.get()) @@ -283,8 +285,7 @@ TTLDescription TTLDescription::getTTLFromAST( } } - if (!is_attach) - checkTTLExpression(result.expression, result.result_column); + checkTTLExpression(result.expression, result.result_column, is_attach); return result; } From a0e9154959895536ea5e9161f031b41d74ec3b83 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Jun 2023 22:04:59 +0200 Subject: [PATCH 003/110] Disable compile_expressions --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c14bd420c5c..1e7beb07e3d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -141,7 +141,7 @@ class IColumn; M(Bool, allow_suspicious_low_cardinality_types, false, "In CREATE TABLE statement allows specifying LowCardinality modifier for types of small fixed size (8 or less). Enabling this may increase merge times and memory consumption.", 0) \ M(Bool, allow_suspicious_fixed_string_types, false, "In CREATE TABLE statement allows creating columns of type FixedString(n) with n > 256. FixedString with length >= 256 is suspicious and most likely indicates misusage", 0) \ M(Bool, allow_suspicious_indices, false, "Reject primary/secondary indexes and sorting keys with identical expressions", 0) \ - M(Bool, compile_expressions, true, "Compile some scalar functions and operators to native code.", 0) \ + M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \ M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \ M(Bool, compile_aggregate_expressions, true, "Compile aggregate functions to native code. This feature has a bug and should not be used.", 0) \ M(UInt64, min_count_to_compile_aggregate_expression, 3, "The number of identical aggregate expressions before they are JIT-compiled", 0) \ From a1b29dc1386cfbf446e8326c4e186458c2fabec4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Jun 2023 23:42:55 +0200 Subject: [PATCH 004/110] Add tests --- .../02789_jit_cannot_convert_column.reference | 0 .../02789_jit_cannot_convert_column.sql | 11 +++++++ .../02790_jit_wrong_result.reference | 3 ++ .../0_stateless/02790_jit_wrong_result.sql | 23 ++++++++++++++ .../0_stateless/02791_llvm_16_bug.reference | 0 .../queries/0_stateless/02791_llvm_16_bug.sql | 31 +++++++++++++++++++ 6 files changed, 68 insertions(+) create mode 100644 tests/queries/0_stateless/02789_jit_cannot_convert_column.reference create mode 100644 tests/queries/0_stateless/02789_jit_cannot_convert_column.sql create mode 100644 tests/queries/0_stateless/02790_jit_wrong_result.reference create mode 100644 tests/queries/0_stateless/02790_jit_wrong_result.sql create mode 100644 tests/queries/0_stateless/02791_llvm_16_bug.reference create mode 100644 tests/queries/0_stateless/02791_llvm_16_bug.sql diff --git a/tests/queries/0_stateless/02789_jit_cannot_convert_column.reference b/tests/queries/0_stateless/02789_jit_cannot_convert_column.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02789_jit_cannot_convert_column.sql b/tests/queries/0_stateless/02789_jit_cannot_convert_column.sql new file mode 100644 index 00000000000..f5e694a38be --- /dev/null +++ b/tests/queries/0_stateless/02789_jit_cannot_convert_column.sql @@ -0,0 +1,11 @@ +SELECT + sum(c), + toInt32((h - null::Nullable(DateTime)) / 3600) + 1 AS a +FROM +( + SELECT count() AS c, h + FROM ( SELECT now() AS h ) + WHERE toInt32((h - null::Nullable(DateTime)) / 3600) + 1 = 1 + GROUP BY h +) +GROUP BY a settings min_count_to_compile_expression = 0; diff --git a/tests/queries/0_stateless/02790_jit_wrong_result.reference b/tests/queries/0_stateless/02790_jit_wrong_result.reference new file mode 100644 index 00000000000..75a179923dc --- /dev/null +++ b/tests/queries/0_stateless/02790_jit_wrong_result.reference @@ -0,0 +1,3 @@ +-943999939 +1259570390 +0 diff --git a/tests/queries/0_stateless/02790_jit_wrong_result.sql b/tests/queries/0_stateless/02790_jit_wrong_result.sql new file mode 100644 index 00000000000..2bcf89715d1 --- /dev/null +++ b/tests/queries/0_stateless/02790_jit_wrong_result.sql @@ -0,0 +1,23 @@ +DROP TABLE IF EXISTS t2; +CREATE TABLE t2 (c0 Int32, c1 Int32, c2 String) ENGINE = Log() ; +INSERT INTO t2(c1, c0) VALUES (1697596429, 1259570390); +INSERT INTO t2(c1, c2) VALUES (-871444251, 's,'); +INSERT INTO t2(c0, c2, c1) VALUES (-943999939, '', 1756486294); + +SELECT MIN(t2.c0) +FROM t2 +GROUP BY log(-(t2.c0 / (t2.c0 - t2.c0))) +HAVING NOT (NOT (-(NOT MIN(t2.c0)))) +UNION ALL +SELECT MIN(t2.c0) +FROM t2 +GROUP BY log(-(t2.c0 / (t2.c0 - t2.c0))) +HAVING NOT (NOT (NOT (-(NOT MIN(t2.c0))))) +UNION ALL +SELECT MIN(t2.c0) +FROM t2 +GROUP BY log(-(t2.c0 / (t2.c0 - t2.c0))) +HAVING (NOT (NOT (-(NOT MIN(t2.c0))))) IS NULL +SETTINGS aggregate_functions_null_for_empty = 1, enable_optimize_predicate_expression = 0, min_count_to_compile_expression = 0; + +DROP TABLE t2; diff --git a/tests/queries/0_stateless/02791_llvm_16_bug.reference b/tests/queries/0_stateless/02791_llvm_16_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02791_llvm_16_bug.sql b/tests/queries/0_stateless/02791_llvm_16_bug.sql new file mode 100644 index 00000000000..59cb961757a --- /dev/null +++ b/tests/queries/0_stateless/02791_llvm_16_bug.sql @@ -0,0 +1,31 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t3; + +create table t1 (pkey UInt32, c8 UInt32, c9 String, c10 Float32, c11 String, primary key(c8)) engine = ReplacingMergeTree; +create table t3 (vkey UInt32, pkey UInt32, c15 UInt32) engine = Log; + +SET min_count_to_compile_expression = 0; + +with cte_4 as ( + select + ref_10.c11 as c_2_c2350_1, + ref_9.c9 as c_2_c2351_2 + from + t1 as ref_9 + right outer join t1 as ref_10 + on (ref_9.c11 = ref_10.c9) + inner join t3 as ref_11 + on (ref_10.c8 = ref_11.vkey) + where ((ref_10.pkey + ref_11.pkey) between ref_11.vkey and (case when (-30.87 >= ref_9.c10) then ref_11.c15 else ref_11.pkey end))) +select + ref_13.c_2_c2350_1 as c_2_c2357_3 + from + cte_4 as ref_13 + where (ref_13.c_2_c2351_2) in ( + select + ref_14.c_2_c2351_2 as c_5_c2352_0 + from + cte_4 as ref_14); + +DROP TABLE t1; +DROP TABLE t3; From cb7e695ba3ad94726e2716dee0d78ec3c8ff31da Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Jun 2023 14:53:16 +0200 Subject: [PATCH 005/110] Update test --- tests/queries/0_stateless/02790_jit_wrong_result.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02790_jit_wrong_result.sql b/tests/queries/0_stateless/02790_jit_wrong_result.sql index 2bcf89715d1..eb373986882 100644 --- a/tests/queries/0_stateless/02790_jit_wrong_result.sql +++ b/tests/queries/0_stateless/02790_jit_wrong_result.sql @@ -18,6 +18,7 @@ SELECT MIN(t2.c0) FROM t2 GROUP BY log(-(t2.c0 / (t2.c0 - t2.c0))) HAVING (NOT (NOT (-(NOT MIN(t2.c0))))) IS NULL +ORDER BY 1 SETTINGS aggregate_functions_null_for_empty = 1, enable_optimize_predicate_expression = 0, min_count_to_compile_expression = 0; DROP TABLE t2; From ca2e0fc0fea6d763e0b4bdf83f1c89768e45b99f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 17 Jun 2023 15:38:30 +0200 Subject: [PATCH 006/110] Update test --- tests/queries/0_stateless/02790_jit_wrong_result.reference | 2 +- tests/queries/0_stateless/02790_jit_wrong_result.sql | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02790_jit_wrong_result.reference b/tests/queries/0_stateless/02790_jit_wrong_result.reference index 75a179923dc..fe3b98956d3 100644 --- a/tests/queries/0_stateless/02790_jit_wrong_result.reference +++ b/tests/queries/0_stateless/02790_jit_wrong_result.reference @@ -1,3 +1,3 @@ -943999939 -1259570390 0 +1259570390 diff --git a/tests/queries/0_stateless/02790_jit_wrong_result.sql b/tests/queries/0_stateless/02790_jit_wrong_result.sql index eb373986882..4105fb6786a 100644 --- a/tests/queries/0_stateless/02790_jit_wrong_result.sql +++ b/tests/queries/0_stateless/02790_jit_wrong_result.sql @@ -4,6 +4,8 @@ INSERT INTO t2(c1, c0) VALUES (1697596429, 1259570390); INSERT INTO t2(c1, c2) VALUES (-871444251, 's,'); INSERT INTO t2(c0, c2, c1) VALUES (-943999939, '', 1756486294); +SELECT * FROM +( SELECT MIN(t2.c0) FROM t2 GROUP BY log(-(t2.c0 / (t2.c0 - t2.c0))) @@ -18,6 +20,7 @@ SELECT MIN(t2.c0) FROM t2 GROUP BY log(-(t2.c0 / (t2.c0 - t2.c0))) HAVING (NOT (NOT (-(NOT MIN(t2.c0))))) IS NULL +) ORDER BY 1 SETTINGS aggregate_functions_null_for_empty = 1, enable_optimize_predicate_expression = 0, min_count_to_compile_expression = 0; From 17c83ab1727367344b6760206f5f5874b13a4fdb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Jun 2023 03:37:36 +0200 Subject: [PATCH 007/110] Do not randomize compile_expressions --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 9242ca8a0b0..fb9ac4f34c6 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -589,7 +589,7 @@ class SettingsRandomizer: "filesystem_prefetch_step_bytes": lambda: random.choice( [0, "100Mi"] ), # 0 means 'auto' - "compile_expressions": lambda: random.randint(0, 1), + # "compile_expressions": lambda: random.randint(0, 1), - this setting has a bug "compile_aggregate_expressions": lambda: random.randint(0, 1), "compile_sort_description": lambda: random.randint(0, 1), "merge_tree_coarse_index_granularity": lambda: random.randint(2, 32), From db82e94e68c48dd01a2e91be597cbedc7b56a188 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 20 Jun 2023 20:17:01 +0200 Subject: [PATCH 008/110] Enable JIT again --- src/Core/Settings.h | 2 +- tests/clickhouse-test | 2 +- .../0_stateless/02791_llvm_16_bug.reference | 0 .../queries/0_stateless/02791_llvm_16_bug.sql | 31 ------------------- 4 files changed, 2 insertions(+), 33 deletions(-) delete mode 100644 tests/queries/0_stateless/02791_llvm_16_bug.reference delete mode 100644 tests/queries/0_stateless/02791_llvm_16_bug.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1e7beb07e3d..c14bd420c5c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -141,7 +141,7 @@ class IColumn; M(Bool, allow_suspicious_low_cardinality_types, false, "In CREATE TABLE statement allows specifying LowCardinality modifier for types of small fixed size (8 or less). Enabling this may increase merge times and memory consumption.", 0) \ M(Bool, allow_suspicious_fixed_string_types, false, "In CREATE TABLE statement allows creating columns of type FixedString(n) with n > 256. FixedString with length >= 256 is suspicious and most likely indicates misusage", 0) \ M(Bool, allow_suspicious_indices, false, "Reject primary/secondary indexes and sorting keys with identical expressions", 0) \ - M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \ + M(Bool, compile_expressions, true, "Compile some scalar functions and operators to native code.", 0) \ M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \ M(Bool, compile_aggregate_expressions, true, "Compile aggregate functions to native code. This feature has a bug and should not be used.", 0) \ M(UInt64, min_count_to_compile_aggregate_expression, 3, "The number of identical aggregate expressions before they are JIT-compiled", 0) \ diff --git a/tests/clickhouse-test b/tests/clickhouse-test index fb9ac4f34c6..9242ca8a0b0 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -589,7 +589,7 @@ class SettingsRandomizer: "filesystem_prefetch_step_bytes": lambda: random.choice( [0, "100Mi"] ), # 0 means 'auto' - # "compile_expressions": lambda: random.randint(0, 1), - this setting has a bug + "compile_expressions": lambda: random.randint(0, 1), "compile_aggregate_expressions": lambda: random.randint(0, 1), "compile_sort_description": lambda: random.randint(0, 1), "merge_tree_coarse_index_granularity": lambda: random.randint(2, 32), diff --git a/tests/queries/0_stateless/02791_llvm_16_bug.reference b/tests/queries/0_stateless/02791_llvm_16_bug.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02791_llvm_16_bug.sql b/tests/queries/0_stateless/02791_llvm_16_bug.sql deleted file mode 100644 index 59cb961757a..00000000000 --- a/tests/queries/0_stateless/02791_llvm_16_bug.sql +++ /dev/null @@ -1,31 +0,0 @@ -DROP TABLE IF EXISTS t1; -DROP TABLE IF EXISTS t3; - -create table t1 (pkey UInt32, c8 UInt32, c9 String, c10 Float32, c11 String, primary key(c8)) engine = ReplacingMergeTree; -create table t3 (vkey UInt32, pkey UInt32, c15 UInt32) engine = Log; - -SET min_count_to_compile_expression = 0; - -with cte_4 as ( - select - ref_10.c11 as c_2_c2350_1, - ref_9.c9 as c_2_c2351_2 - from - t1 as ref_9 - right outer join t1 as ref_10 - on (ref_9.c11 = ref_10.c9) - inner join t3 as ref_11 - on (ref_10.c8 = ref_11.vkey) - where ((ref_10.pkey + ref_11.pkey) between ref_11.vkey and (case when (-30.87 >= ref_9.c10) then ref_11.c15 else ref_11.pkey end))) -select - ref_13.c_2_c2350_1 as c_2_c2357_3 - from - cte_4 as ref_13 - where (ref_13.c_2_c2351_2) in ( - select - ref_14.c_2_c2351_2 as c_5_c2352_0 - from - cte_4 as ref_14); - -DROP TABLE t1; -DROP TABLE t3; From e62ef756cf76b84ed367e9418a5b7241c422ce32 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 23 Jun 2023 14:47:58 +0200 Subject: [PATCH 009/110] Enable Azure on macOS --- contrib/CMakeLists.txt | 4 ++-- contrib/azure | 2 +- contrib/curl-cmake/CMakeLists.txt | 7 +++++++ 3 files changed, 10 insertions(+), 3 deletions(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 2af468970f1..0de8ff172a5 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -144,9 +144,9 @@ add_contrib (libuv-cmake libuv) add_contrib (liburing-cmake liburing) add_contrib (amqpcpp-cmake AMQP-CPP) # requires: libuv add_contrib (cassandra-cmake cassandra) # requires: libuv +add_contrib (curl-cmake curl) +add_contrib (azure-cmake azure) # requires: curl if (NOT OS_DARWIN) - add_contrib (curl-cmake curl) - add_contrib (azure-cmake azure) # requires: curl add_contrib (sentry-native-cmake sentry-native) # requires: curl endif() add_contrib (fmtlib-cmake fmtlib) diff --git a/contrib/azure b/contrib/azure index 096049bf24f..352ff0a61cb 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit 096049bf24fffafcaccc132b9367694532716731 +Subproject commit 352ff0a61cb319ac1cc38c4058443ddf70147530 diff --git a/contrib/curl-cmake/CMakeLists.txt b/contrib/curl-cmake/CMakeLists.txt index 70d9c2816dc..45ae461a69f 100644 --- a/contrib/curl-cmake/CMakeLists.txt +++ b/contrib/curl-cmake/CMakeLists.txt @@ -166,6 +166,13 @@ target_compile_definitions (_curl PRIVATE libcurl_EXPORTS OS="${CMAKE_SYSTEM_NAME}" ) + +if (OS_DARWIN) + # don't add this and you'll get the most weird compile errors + target_compile_definitions(_curl PRIVATE USE_ARES) + target_link_libraries (_curl PRIVATE ch_contrib::c-ares) +endif() + target_include_directories (_curl SYSTEM PUBLIC "${LIBRARY_DIR}/include" "${LIBRARY_DIR}/lib" From e2936b8721d33a3440395b58dd3fddbbd760f62a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 5 Jul 2023 20:50:58 +0200 Subject: [PATCH 010/110] Fix an error --- src/Storages/TTLDescription.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 62e69360272..e0b365a29cd 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -60,6 +60,10 @@ void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const Strin /// Do not apply this check in ATTACH queries for compatibility reasons. if (!is_attach) { + if (ttl_expression->getRequiredColumns().empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "TTL expression {} does not depend on any of the columns of the table", result_column_name); + for (const auto & action : ttl_expression->getActions()) { if (action.node->type == ActionsDAG::ActionType::FUNCTION) From a16340663d858613dd7e4ee2795c6964ba2e7768 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 6 Jul 2023 03:21:39 +0300 Subject: [PATCH 011/110] Update CMakeLists.txt --- contrib/curl-cmake/CMakeLists.txt | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/contrib/curl-cmake/CMakeLists.txt b/contrib/curl-cmake/CMakeLists.txt index 45ae461a69f..237d280e5b8 100644 --- a/contrib/curl-cmake/CMakeLists.txt +++ b/contrib/curl-cmake/CMakeLists.txt @@ -163,23 +163,18 @@ target_compile_definitions (_curl PRIVATE HAVE_CONFIG_H BUILDING_LIBCURL CURL_HIDDEN_SYMBOLS + USE_ARES libcurl_EXPORTS OS="${CMAKE_SYSTEM_NAME}" ) -if (OS_DARWIN) - # don't add this and you'll get the most weird compile errors - target_compile_definitions(_curl PRIVATE USE_ARES) - target_link_libraries (_curl PRIVATE ch_contrib::c-ares) -endif() - target_include_directories (_curl SYSTEM PUBLIC "${LIBRARY_DIR}/include" "${LIBRARY_DIR}/lib" . # curl_config.h ) -target_link_libraries (_curl PRIVATE OpenSSL::SSL) +target_link_libraries (_curl PRIVATE OpenSSL::SSL ch_contrib::c-ares) # The library is large - avoid bloat (XXX: is it?) if (OMIT_HEAVY_DEBUG_SYMBOLS) From 899105bef4bb71d97eb7d44e6aec9d950d9f0cc4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 6 Jul 2023 02:41:05 +0200 Subject: [PATCH 012/110] Fix build --- contrib/curl-cmake/CMakeLists.txt | 3 +-- contrib/curl-cmake/curl_config.h | 1 + 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/curl-cmake/CMakeLists.txt b/contrib/curl-cmake/CMakeLists.txt index 237d280e5b8..8c5ef15fa79 100644 --- a/contrib/curl-cmake/CMakeLists.txt +++ b/contrib/curl-cmake/CMakeLists.txt @@ -90,7 +90,7 @@ set (SRCS "${LIBRARY_DIR}/lib/curl_gethostname.c" "${LIBRARY_DIR}/lib/gopher.c" "${LIBRARY_DIR}/lib/http_proxy.c" - "${LIBRARY_DIR}/lib/asyn-thread.c" + "${LIBRARY_DIR}/lib/asyn-ares.c" "${LIBRARY_DIR}/lib/curl_gssapi.c" "${LIBRARY_DIR}/lib/http_ntlm.c" "${LIBRARY_DIR}/lib/curl_ntlm_wb.c" @@ -163,7 +163,6 @@ target_compile_definitions (_curl PRIVATE HAVE_CONFIG_H BUILDING_LIBCURL CURL_HIDDEN_SYMBOLS - USE_ARES libcurl_EXPORTS OS="${CMAKE_SYSTEM_NAME}" ) diff --git a/contrib/curl-cmake/curl_config.h b/contrib/curl-cmake/curl_config.h index f56ba3eccd5..a38aa60fe6d 100644 --- a/contrib/curl-cmake/curl_config.h +++ b/contrib/curl-cmake/curl_config.h @@ -50,3 +50,4 @@ #define ENABLE_IPV6 #define USE_OPENSSL #define USE_THREADS_POSIX +#define USE_ARES From 89656723fbd332e240dd40d630fa8a87cf22f0a2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Nov 2023 16:02:33 +0100 Subject: [PATCH 013/110] Fix tests --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 2 ++ src/Storages/TTLDescription.cpp | 8 ++++---- src/Storages/TTLDescription.h | 6 ++++-- tests/queries/0_stateless/00933_ttl_simple.sql | 3 ++- tests/queries/0_stateless/01070_alter_with_ttl.sql | 2 ++ tests/queries/0_stateless/01070_materialize_ttl.sql | 2 ++ tests/queries/0_stateless/01070_modify_ttl.sql | 2 ++ .../queries/0_stateless/01070_modify_ttl_recalc_only.sql | 2 ++ 9 files changed, 21 insertions(+), 7 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3a1ceb132bd..ed4d33a88bf 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -152,6 +152,7 @@ class IColumn; M(Bool, allow_suspicious_low_cardinality_types, false, "In CREATE TABLE statement allows specifying LowCardinality modifier for types of small fixed size (8 or less). Enabling this may increase merge times and memory consumption.", 0) \ M(Bool, allow_suspicious_fixed_string_types, false, "In CREATE TABLE statement allows creating columns of type FixedString(n) with n > 256. FixedString with length >= 256 is suspicious and most likely indicates misusage", 0) \ M(Bool, allow_suspicious_indices, false, "Reject primary/secondary indexes and sorting keys with identical expressions", 0) \ + M(Bool, allow_suspicious_ttl_expressions, false, "Reject TTL expressions that don't depend on any of table's columns. It indicates a user error most of the time.", 0) \ M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \ M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \ M(Bool, compile_aggregate_expressions, true, "Compile aggregate functions to native code.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 38039839e1e..e10b847a658 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -7,6 +7,7 @@ #include #include + namespace DB { @@ -80,6 +81,7 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { + {"23.11", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}}}, {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index e0b365a29cd..15ed3f0684b 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -55,10 +55,10 @@ TTLAggregateDescription & TTLAggregateDescription::operator=(const TTLAggregateD namespace { -void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const String & result_column_name, bool is_attach) +void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const String & result_column_name, bool allow_suspicious) { - /// Do not apply this check in ATTACH queries for compatibility reasons. - if (!is_attach) + /// Do not apply this check in ATTACH queries for compatibility reasons and if explicitly allowed. + if (!allow_suspicious) { if (ttl_expression->getRequiredColumns().empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, @@ -297,7 +297,7 @@ TTLDescription TTLDescription::getTTLFromAST( } } - checkTTLExpression(result.expression, result.result_column, is_attach); + checkTTLExpression(result.expression, result.result_column, is_attach || context->getSettingsRef().allow_suspicious_ttl_expressions); return result; } diff --git a/src/Storages/TTLDescription.h b/src/Storages/TTLDescription.h index 91ef7b44d37..aab5b43e53e 100644 --- a/src/Storages/TTLDescription.h +++ b/src/Storages/TTLDescription.h @@ -1,4 +1,5 @@ #pragma once + #include #include #include @@ -7,6 +8,7 @@ #include #include + namespace DB { @@ -83,7 +85,7 @@ struct TTLDescription ASTPtr recompression_codec; /// Parse TTL structure from definition. Able to parse both column and table TTLs. - static TTLDescription getTTLFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context, const KeyDescription & primary_key, bool is_attach = false); + static TTLDescription getTTLFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context, const KeyDescription & primary_key, bool is_attach); TTLDescription() = default; TTLDescription(const TTLDescription & other); @@ -120,7 +122,7 @@ struct TTLTableDescription TTLTableDescription & operator=(const TTLTableDescription & other); static TTLTableDescription getTTLForTableFromAST( - const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context, const KeyDescription & primary_key, bool is_attach = false); + const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr context, const KeyDescription & primary_key, bool is_attach); /// Parse description from string static TTLTableDescription parse(const String & str, const ColumnsDescription & columns, ContextPtr context, const KeyDescription & primary_key); diff --git a/tests/queries/0_stateless/00933_ttl_simple.sql b/tests/queries/0_stateless/00933_ttl_simple.sql index 3c29c915631..c1df338a0ff 100644 --- a/tests/queries/0_stateless/00933_ttl_simple.sql +++ b/tests/queries/0_stateless/00933_ttl_simple.sql @@ -8,7 +8,8 @@ -- ┌───────────────now()─┬─toDate(toTimeZone(now(), 'America/Mazatlan'))─┬────today()─┐ -- │ 2023-07-24 06:24:06 │ 2023-07-23 │ 2023-07-24 │ -- └─────────────────────┴───────────────────────────────────────────────┴────────────┘ -set session_timezone = ''; +SET session_timezone = ''; +SET allow_suspicious_ttl_expressions = 1; drop table if exists ttl_00933_1; diff --git a/tests/queries/0_stateless/01070_alter_with_ttl.sql b/tests/queries/0_stateless/01070_alter_with_ttl.sql index 3adc3ccd6ae..44d422cbe6d 100644 --- a/tests/queries/0_stateless/01070_alter_with_ttl.sql +++ b/tests/queries/0_stateless/01070_alter_with_ttl.sql @@ -1,5 +1,7 @@ drop table if exists alter_ttl; +SET allow_suspicious_ttl_expressions = 1; + create table alter_ttl(i Int) engine = MergeTree order by i ttl toDate('2020-05-05'); alter table alter_ttl add column s String; alter table alter_ttl modify column s String ttl toDate('2020-01-01'); diff --git a/tests/queries/0_stateless/01070_materialize_ttl.sql b/tests/queries/0_stateless/01070_materialize_ttl.sql index b6a03f2ca10..b322b67882c 100644 --- a/tests/queries/0_stateless/01070_materialize_ttl.sql +++ b/tests/queries/0_stateless/01070_materialize_ttl.sql @@ -1,5 +1,7 @@ -- Tags: no-parallel +SET allow_suspicious_ttl_expressions = 1; + drop table if exists ttl; create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d); diff --git a/tests/queries/0_stateless/01070_modify_ttl.sql b/tests/queries/0_stateless/01070_modify_ttl.sql index 0716ccd7043..4ffd59fc8a9 100644 --- a/tests/queries/0_stateless/01070_modify_ttl.sql +++ b/tests/queries/0_stateless/01070_modify_ttl.sql @@ -1,5 +1,7 @@ -- Tags: no-parallel +SET allow_suspicious_ttl_expressions = 1; + drop table if exists ttl; create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d); diff --git a/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql b/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql index 7ac70d41871..2700cc03ff5 100644 --- a/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql +++ b/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql @@ -5,6 +5,8 @@ set mutations_sync = 2; -- system.parts has server default, timezone cannot be randomized set session_timezone = ''; +SET allow_suspicious_ttl_expressions = 1; + drop table if exists ttl; create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) From 39ea0c426a4da3c3bb6986a855d588540f7c5ce3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Nov 2023 16:05:15 +0100 Subject: [PATCH 014/110] Fix build --- src/Storages/AlterCommands.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index f5293c52bb0..1a5d3c4a229 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -633,7 +633,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) } else if (type == MODIFY_TTL) { - metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST(ttl, metadata.columns, context, metadata.primary_key); + metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST(ttl, metadata.columns, context, metadata.primary_key, context->getSettingsRef().allow_suspicious_ttl_expressions); } else if (type == REMOVE_TTL) { @@ -1025,13 +1025,13 @@ void AlterCommands::apply(StorageInMemoryMetadata & metadata, ContextPtr context metadata_copy.column_ttls_by_name.clear(); for (const auto & [name, ast] : column_ttl_asts) { - auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, metadata_copy.columns, context, metadata_copy.primary_key); + auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, metadata_copy.columns, context, metadata_copy.primary_key, context->getSettingsRef().allow_suspicious_ttl_expressions); metadata_copy.column_ttls_by_name[name] = new_ttl_entry; } if (metadata_copy.table_ttl.definition_ast != nullptr) metadata_copy.table_ttl = TTLTableDescription::getTTLForTableFromAST( - metadata_copy.table_ttl.definition_ast, metadata_copy.columns, context, metadata_copy.primary_key); + metadata_copy.table_ttl.definition_ast, metadata_copy.columns, context, metadata_copy.primary_key, context->getSettingsRef().allow_suspicious_ttl_expressions); metadata = std::move(metadata_copy); } From 6bbf4498089b70c6abdf1ae970fe7238cdd3ba73 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Nov 2023 16:06:35 +0100 Subject: [PATCH 015/110] Fix build --- src/Storages/TTLDescription.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 15ed3f0684b..41a222525bf 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -389,7 +389,7 @@ TTLTableDescription TTLTableDescription::parse(const String & str, const Columns ASTPtr ast = parseQuery(parser, str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); FunctionNameNormalizer().visit(ast.get()); - return getTTLForTableFromAST(ast, columns, context, primary_key); + return getTTLForTableFromAST(ast, columns, context, primary_key, context->getSettingsRef().allow_suspicious_ttl_expressions); } } From dc0d6e432cf09e92ef6de05f9b4eea7b4f44d650 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Nov 2023 16:11:54 +0100 Subject: [PATCH 016/110] Fix build --- src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp index 41ff93e28f7..40cf64327d3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp @@ -409,7 +409,7 @@ StorageInMemoryMetadata ReplicatedMergeTreeTableMetadata::Diff::getNewMetadata(c ParserTTLExpressionList parser; auto ttl_for_table_ast = parseQuery(parser, new_ttl_table, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); new_metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST( - ttl_for_table_ast, new_metadata.columns, context, new_metadata.primary_key); + ttl_for_table_ast, new_metadata.columns, context, new_metadata.primary_key, context->getSettingsRef().allow_suspicious_ttl_expressions); } else /// TTL was removed { @@ -422,7 +422,7 @@ StorageInMemoryMetadata ReplicatedMergeTreeTableMetadata::Diff::getNewMetadata(c new_metadata.column_ttls_by_name.clear(); for (const auto & [name, ast] : new_metadata.columns.getColumnTTLs()) { - auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, new_metadata.columns, context, new_metadata.primary_key); + auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, new_metadata.columns, context, new_metadata.primary_key, context->getSettingsRef().allow_suspicious_ttl_expressions); new_metadata.column_ttls_by_name[name] = new_ttl_entry; } @@ -454,7 +454,7 @@ StorageInMemoryMetadata ReplicatedMergeTreeTableMetadata::Diff::getNewMetadata(c if (!ttl_table_changed && new_metadata.table_ttl.definition_ast != nullptr) new_metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST( - new_metadata.table_ttl.definition_ast, new_metadata.columns, context, new_metadata.primary_key); + new_metadata.table_ttl.definition_ast, new_metadata.columns, context, new_metadata.primary_key, context->getSettingsRef().allow_suspicious_ttl_expressions); if (!projections_changed) { From c5fabcc4727d6137f0f54d58d91a38aba9e03d8d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Nov 2023 17:50:50 +0100 Subject: [PATCH 017/110] Fix tests --- src/Storages/MergeTree/registerStorageMergeTree.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index e02e8695135..9a5af77d57c 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -581,10 +581,12 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (args.storage_def->sample_by) metadata.sampling_key = KeyDescription::getKeyFromAST(args.storage_def->sample_by->ptr(), metadata.columns, context); + bool allow_suspicious_ttl = args.attach || args.getLocalContext()->getSettingsRef().allow_suspicious_ttl_expressions; + if (args.storage_def->ttl_table) { metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST( - args.storage_def->ttl_table->ptr(), metadata.columns, context, metadata.primary_key, args.attach); + args.storage_def->ttl_table->ptr(), metadata.columns, context, metadata.primary_key, allow_suspicious_ttl); } if (args.query.columns_list && args.query.columns_list->indices) @@ -602,7 +604,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) auto column_ttl_asts = columns.getColumnTTLs(); for (const auto & [name, ast] : column_ttl_asts) { - auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, columns, context, metadata.primary_key, args.attach); + auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, columns, context, metadata.primary_key, allow_suspicious_ttl); metadata.column_ttls_by_name[name] = new_ttl_entry; } From 81a50703c3e73f64660790a6686450121366e5b4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Nov 2023 23:39:40 +0100 Subject: [PATCH 018/110] Fix tests --- src/Storages/MergeTree/MergeTreeData.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index bfd1693bfb2..13503dbf444 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2884,7 +2884,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context if (!settings.allow_non_metadata_alters) { - auto mutation_commands = commands.getMutationCommands(new_metadata, settings.materialize_ttl_after_modify, getContext()); + auto mutation_commands = commands.getMutationCommands(new_metadata, settings.materialize_ttl_after_modify, local_context); if (!mutation_commands.empty()) throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, @@ -2893,7 +2893,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context queryToString(mutation_commands.ast())); } - commands.apply(new_metadata, getContext()); + commands.apply(new_metadata, local_context); if (commands.hasInvertedIndex(new_metadata) && !settings.allow_experimental_inverted_index) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, @@ -3133,7 +3133,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context if (!columns_to_check_conversion.empty()) { auto old_header = old_metadata.getSampleBlock(); - performRequiredConversions(old_header, columns_to_check_conversion, getContext()); + performRequiredConversions(old_header, columns_to_check_conversion, local_context); } if (old_metadata.hasSettingsChanges()) @@ -3165,7 +3165,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context } if (setting_name == "storage_policy") - checkStoragePolicy(getContext()->getStoragePolicy(new_value.safeGet())); + checkStoragePolicy(local_context->getStoragePolicy(new_value.safeGet())); } /// Check if it is safe to reset the settings From 86d14a07e127adfdd2bab53d420dce65a36521a0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 24 Nov 2023 02:12:08 +0100 Subject: [PATCH 019/110] Fix test --- tests/queries/0_stateless/01070_mutations_with_dependencies.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01070_mutations_with_dependencies.sql b/tests/queries/0_stateless/01070_mutations_with_dependencies.sql index 566bb16b10c..813ebf3f5a7 100644 --- a/tests/queries/0_stateless/01070_mutations_with_dependencies.sql +++ b/tests/queries/0_stateless/01070_mutations_with_dependencies.sql @@ -34,6 +34,8 @@ select count() from ttl where s = 'b'; drop table ttl; -- check only that it doesn't throw exceptions. +SET allow_suspicious_ttl_expressions = 1; + create table ttl (i Int, s String) engine = MergeTree order by i ttl toDate('2000-01-01') TO DISK 'default'; alter table ttl materialize ttl; drop table ttl; From 411660c267a5c378069d209067a225801bd9b3fa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 24 Nov 2023 17:35:56 +0100 Subject: [PATCH 020/110] Fix integration tests --- tests/integration/test_multiple_disks/test.py | 2 +- tests/integration/test_ttl_move/test.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index 30669feb6b3..2484ef1bdf1 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -1834,7 +1834,7 @@ def _insert_merge_execute( ORDER BY tuple() PARTITION BY tuple() TTL now()-1 TO VOLUME 'external' - SETTINGS storage_policy='{policy}' + SETTINGS storage_policy='{policy}', allow_suspicious_ttl_expressions = 1 """.format( name=name, policy=policy ) diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index c1c076277bb..3faa20506ed 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -297,7 +297,7 @@ def test_moves_work_after_storage_policy_change(started_cluster, name, engine): # Second expression is preferred because d1 > now()-3600. node1.query( - """ALTER TABLE {name} MODIFY TTL now()-3600 TO DISK 'jbod1', d1 TO DISK 'external'""".format( + """ALTER TABLE {name} MODIFY TTL now()-3600 TO DISK 'jbod1', d1 TO DISK 'external' SETTINGS allow_suspicious_ttl_expressions = 1""".format( name=name ) ) From 1790626406ae707c551fccce6ceb166d81d360bc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 1 Oct 2022 16:32:04 +0200 Subject: [PATCH 021/110] Check what will happen if we build ClickHouse with Musl --- PreLoad.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/PreLoad.cmake b/PreLoad.cmake index 4879e721ae3..223e7aedd42 100644 --- a/PreLoad.cmake +++ b/PreLoad.cmake @@ -86,7 +86,7 @@ if (OS MATCHES "Linux" # - compile musl with debug and -fasynchronous-unwind-tables # # But none of this changes anything so far. - set (CMAKE_TOOLCHAIN_FILE "cmake/linux/toolchain-x86_64.cmake" CACHE INTERNAL "") + set (CMAKE_TOOLCHAIN_FILE "cmake/linux/toolchain-x86_64-musl.cmake" CACHE INTERNAL "") elseif (ARCH MATCHES "^(aarch64.*|AARCH64.*|arm64.*|ARM64.*)") set (CMAKE_TOOLCHAIN_FILE "cmake/linux/toolchain-aarch64.cmake" CACHE INTERNAL "") elseif (ARCH MATCHES "^(ppc64le.*|PPC64LE.*)") From 043e9239d69ea54768850d292b4f3fe02efab896 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Nov 2023 18:59:54 +0100 Subject: [PATCH 022/110] Update Azure and Sentry --- contrib/azure | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/azure b/contrib/azure index 096049bf24f..352ff0a61cb 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit 096049bf24fffafcaccc132b9367694532716731 +Subproject commit 352ff0a61cb319ac1cc38c4058443ddf70147530 From 6effb1311f31b73d8f3f60560f5efd218b8bc71c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Nov 2023 19:01:41 +0100 Subject: [PATCH 023/110] Compatibility with Musl in HDFS and RocksDB --- contrib/libhdfs3-cmake/CMakeLists.txt | 5 +++++ contrib/rocksdb-cmake/CMakeLists.txt | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/contrib/libhdfs3-cmake/CMakeLists.txt b/contrib/libhdfs3-cmake/CMakeLists.txt index 4278575fd7f..8cd951af746 100644 --- a/contrib/libhdfs3-cmake/CMakeLists.txt +++ b/contrib/libhdfs3-cmake/CMakeLists.txt @@ -26,6 +26,11 @@ ADD_DEFINITIONS(-D__STDC_FORMAT_MACROS) ADD_DEFINITIONS(-D_GNU_SOURCE) ADD_DEFINITIONS(-D_GLIBCXX_USE_NANOSLEEP) ADD_DEFINITIONS(-DHAVE_NANOSLEEP) + +if (USE_MUSL) + ADD_DEFINITIONS(-DSTRERROR_R_RETURN_INT) +endif () + set(HAVE_STEADY_CLOCK 1) set(HAVE_NESTED_EXCEPTION 1) SET(HAVE_BOOST_CHRONO 0) diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index 7d7666dff87..c4220ba90ac 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -117,7 +117,7 @@ endif() add_definitions(-DROCKSDB_PLATFORM_POSIX -DROCKSDB_LIB_IO_POSIX) -if (OS_LINUX OR OS_FREEBSD) +if ((OS_LINUX OR OS_FREEBSD) AND NOT USE_MUSL) add_definitions(-DROCKSDB_PTHREAD_ADAPTIVE_MUTEX) endif() From 6d4af13f911e2d4e68e8429d68fa84dc5874a85d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Nov 2023 19:13:12 +0100 Subject: [PATCH 024/110] Update HDFS --- contrib/libhdfs3 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libhdfs3 b/contrib/libhdfs3 index bdcb91354b1..b9598e60167 160000 --- a/contrib/libhdfs3 +++ b/contrib/libhdfs3 @@ -1 +1 @@ -Subproject commit bdcb91354b1c05b21e73043a112a6f1e3b013497 +Subproject commit b9598e6016720a7c088bfe85ce1fa0410f9d2103 From 9621c519e14745104b04f267ab4b24301f0df11c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Nov 2023 19:20:47 +0100 Subject: [PATCH 025/110] Clarifications --- contrib/unixodbc-cmake/CMakeLists.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/unixodbc-cmake/CMakeLists.txt b/contrib/unixodbc-cmake/CMakeLists.txt index 3317654cd67..6fbe8c14ebb 100644 --- a/contrib/unixodbc-cmake/CMakeLists.txt +++ b/contrib/unixodbc-cmake/CMakeLists.txt @@ -1,7 +1,7 @@ option (ENABLE_ODBC "Enable ODBC library" ${ENABLE_LIBRARIES}) -if (NOT OS_LINUX) +if (NOT OS_LINUX OR USE_MUSL) if (ENABLE_ODBC) - message(STATUS "ODBC is only supported on Linux") + message(STATUS "ODBC is only supported on Linux with dynamic linking") endif() set (ENABLE_ODBC OFF CACHE INTERNAL "") endif () From b2d8103d19b5af20a6d1b2f10019d551790ba2cc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Nov 2023 23:25:39 +0100 Subject: [PATCH 026/110] Add special build with Musl --- .github/workflows/master.yml | 6 ++++++ .github/workflows/pull_request.yml | 5 +++++ docker/packager/packager | 9 +++++++++ tests/ci/ci_config.py | 8 ++++++++ 4 files changed, 28 insertions(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index e662a5b6f98..9ffb58a696a 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -205,6 +205,12 @@ jobs: with: build_name: binary_amd64_compat checkout_depth: 0 + BuilderBinAmd64Musl: + needs: [ DockerHubPush ] + uses: ./.github/workflows/reusable_build.yml + with: + build_name: binary_amd64_musl + checkout_depth: 0 BuilderBinAarch64V80Compat: needs: [DockerHubPush] uses: ./.github/workflows/reusable_build.yml diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index f8f052d9226..1620b00c33f 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -242,6 +242,11 @@ jobs: uses: ./.github/workflows/reusable_build.yml with: build_name: binary_amd64_compat + BuilderBinAmd64Musl: + needs: [ FastTest, StyleCheck ] + uses: ./.github/workflows/reusable_build.yml + with: + build_name: binary_amd64_musl BuilderBinAarch64V80Compat: needs: [FastTest, StyleCheck] uses: ./.github/workflows/reusable_build.yml diff --git a/docker/packager/packager b/docker/packager/packager index b5bcbada1da..ff77892b8b3 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -145,6 +145,7 @@ def parse_env_variables( RISCV_SUFFIX = "-riscv64" S390X_SUFFIX = "-s390x" AMD64_COMPAT_SUFFIX = "-amd64-compat" + AMD64_MUSL_SUFFIX = "-amd64-musl" result = [] result.append("OUTPUT_DIR=/output") @@ -163,6 +164,7 @@ def parse_env_variables( is_cross_s390x = compiler.endswith(S390X_SUFFIX) is_cross_freebsd = compiler.endswith(FREEBSD_SUFFIX) is_amd64_compat = compiler.endswith(AMD64_COMPAT_SUFFIX) + is_amd64_musl = compiler.endswith(AMD64_MUSL_SUFFIX) if is_cross_darwin: cc = compiler[: -len(DARWIN_SUFFIX)] @@ -232,6 +234,12 @@ def parse_env_variables( cc = compiler[: -len(AMD64_COMPAT_SUFFIX)] result.append("DEB_ARCH=amd64") cmake_flags.append("-DNO_SSE3_OR_HIGHER=1") + elif is_amd64_musl: + cc = compiler[: -len(AMD64_MUSL_SUFFIX)] + result.append("DEB_ARCH=amd64") + cmake_flags.append( + "-DCMAKE_TOOLCHAIN_FILE=/build/cmake/freebsd/toolchain-x86_64-musl.cmake" + ) else: cc = compiler result.append("DEB_ARCH=amd64") @@ -396,6 +404,7 @@ def parse_args() -> argparse.Namespace: "clang-17-riscv64", "clang-17-s390x", "clang-17-amd64-compat", + "clang-17-amd64-musl", "clang-17-freebsd", ), default="clang-17", diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index d1d2021753e..aa3aa5654aa 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -208,6 +208,13 @@ CI_CONFIG = CiConfig( static_binary_name="amd64compat", comment="SSE2-only build", ), + "binary_amd64_musl": BuildConfig( + name="binary_amd64_musl", + compiler="clang-17-amd64-musl", + package_type="binary", + static_binary_name="amd64musl", + comment="Build with Musl", + ), "binary_riscv64": BuildConfig( name="binary_riscv64", compiler="clang-17-riscv64", @@ -249,6 +256,7 @@ CI_CONFIG = CiConfig( "binary_riscv64", "binary_s390x", "binary_amd64_compat", + "binary_amd64_musl", ], }, test_configs={ From 51fecff769b29413c801b9fbb5dafbedc0bf865c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Nov 2023 02:26:59 +0100 Subject: [PATCH 027/110] Fix style --- .github/workflows/master.yml | 2 +- .github/workflows/pull_request.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 9ffb58a696a..05654926fd7 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -206,7 +206,7 @@ jobs: build_name: binary_amd64_compat checkout_depth: 0 BuilderBinAmd64Musl: - needs: [ DockerHubPush ] + needs: [DockerHubPush] uses: ./.github/workflows/reusable_build.yml with: build_name: binary_amd64_musl diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 1620b00c33f..c089e9c6d94 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -243,7 +243,7 @@ jobs: with: build_name: binary_amd64_compat BuilderBinAmd64Musl: - needs: [ FastTest, StyleCheck ] + needs: [FastTest, StyleCheck] uses: ./.github/workflows/reusable_build.yml with: build_name: binary_amd64_musl From 8ec4a90cb443d27a333e78938733f9218913890f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Nov 2023 17:55:13 +0100 Subject: [PATCH 028/110] Fix error --- docker/packager/packager | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/packager b/docker/packager/packager index ff77892b8b3..ade36a55591 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -238,7 +238,7 @@ def parse_env_variables( cc = compiler[: -len(AMD64_MUSL_SUFFIX)] result.append("DEB_ARCH=amd64") cmake_flags.append( - "-DCMAKE_TOOLCHAIN_FILE=/build/cmake/freebsd/toolchain-x86_64-musl.cmake" + "-DCMAKE_TOOLCHAIN_FILE=/build/cmake/linux/toolchain-x86_64-musl.cmake" ) else: cc = compiler From 347efc669e5cef46a95bb5dc6195a00c6bb9c104 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 27 Nov 2023 18:28:10 +0100 Subject: [PATCH 029/110] support memory soft limit for keeper --- src/Coordination/CoordinationSettings.h | 1 + src/Coordination/KeeperDispatcher.cpp | 50 ++++++++++++++++ .../test_keeper_memory_soft_limit/__init__.py | 0 .../configs/enable_keeper.xml | 29 ++++++++++ .../test_keeper_memory_soft_limit/test.py | 58 +++++++++++++++++++ 5 files changed, 138 insertions(+) create mode 100644 tests/integration/test_keeper_memory_soft_limit/__init__.py create mode 100644 tests/integration/test_keeper_memory_soft_limit/configs/enable_keeper.xml create mode 100644 tests/integration/test_keeper_memory_soft_limit/test.py diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index a58f2b04797..13ef304b353 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -43,6 +43,7 @@ struct Settings; M(UInt64, max_requests_batch_bytes_size, 100*1024, "Max size in bytes of batch of requests that can be sent to RAFT", 0) \ M(UInt64, max_flush_batch_size, 1000, "Max size of batch of requests that can be flushed together", 0) \ M(UInt64, max_requests_quick_batch_size, 100, "Max size of batch of requests to try to get before proceeding with RAFT. Keeper will not wait for requests but take only requests that are already in queue" , 0) \ + M(UInt64, max_memory_usage_soft_limit, 0, "Soft limit in bytes of keeper memory usage", 0) \ M(Bool, quorum_reads, false, "Execute read requests as writes through whole RAFT consesus with similar speed", 0) \ M(Bool, force_sync, true, "Call fsync on each change in RAFT changelog", 0) \ M(Bool, compress_logs, false, "Write compressed coordination logs in ZSTD format", 0) \ diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index f69a9c11e97..26348707e19 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -51,6 +51,48 @@ namespace ErrorCodes extern const int SYSTEM_ERROR; } +namespace +{ + +bool checkIfRequestIncreaseMem(const Coordination::ZooKeeperRequestPtr & request) +{ + if (request->getOpNum() == Coordination::OpNum::Create || request->getOpNum() == Coordination::OpNum::CreateIfNotExists) + { + return false; + } + else if (request->getOpNum() == Coordination::OpNum::Multi) + { + Coordination::ZooKeeperMultiRequest & multi_req = dynamic_cast(*request); + Int64 memory_delta = 0; + for (const auto & sub_req : multi_req.requests) + { + auto sub_zk_request = std::dynamic_pointer_cast(sub_req); + switch (sub_zk_request->getOpNum()) + { + case Coordination::OpNum::Create: + case Coordination::OpNum::CreateIfNotExists: + { + Coordination::ZooKeeperCreateRequest & create_req = dynamic_cast(*sub_zk_request); + memory_delta += create_req.bytesSize(); + break; + } + case Coordination::OpNum::Remove: + { + Coordination::ZooKeeperRemoveRequest & remove_req = dynamic_cast(*sub_zk_request); + memory_delta -= remove_req.bytesSize(); + break; + } + default: + break; + } + } + return memory_delta > 0; + } + + return true; +} + +} KeeperDispatcher::KeeperDispatcher() : responses_queue(std::numeric_limits::max()) @@ -93,6 +135,13 @@ void KeeperDispatcher::requestThread() if (shutdown_called) break; + Int64 mem_soft_limit = configuration_and_settings->coordination_settings->max_memory_usage_soft_limit; + if (configuration_and_settings->standalone_keeper && mem_soft_limit > 0 && total_memory_tracker.get() >= mem_soft_limit && !checkIfRequestIncreaseMem(request.request)) + { + addErrorResponses({request}, Coordination::Error::ZCONNECTIONLOSS); + continue; + } + KeeperStorage::RequestsForSessions current_batch; size_t current_batch_bytes_size = 0; @@ -349,6 +398,7 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ if (keeper_context->shutdown_called) return false; + /// Put close requests without timeouts if (request->getOpNum() == Coordination::OpNum::Close) { diff --git a/tests/integration/test_keeper_memory_soft_limit/__init__.py b/tests/integration/test_keeper_memory_soft_limit/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_memory_soft_limit/configs/enable_keeper.xml b/tests/integration/test_keeper_memory_soft_limit/configs/enable_keeper.xml new file mode 100644 index 00000000000..a3217b34501 --- /dev/null +++ b/tests/integration/test_keeper_memory_soft_limit/configs/enable_keeper.xml @@ -0,0 +1,29 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 10 + 5 + 5000 + 10000 + trace + + + 0 + 0 + 0 + + + + + 1 + localhost + 9234 + + + + diff --git a/tests/integration/test_keeper_memory_soft_limit/test.py b/tests/integration/test_keeper_memory_soft_limit/test.py new file mode 100644 index 00000000000..29275c794a1 --- /dev/null +++ b/tests/integration/test_keeper_memory_soft_limit/test.py @@ -0,0 +1,58 @@ +#!/usr/bin/env python3 +import random +import string +import pytest +from helpers.cluster import ClickHouseCluster +from helpers import keeper_utils +from kazoo.client import KazooClient, KazooState +from kazoo.exceptions import ConnectionLoss + +cluster = ClickHouseCluster(__file__) + +# clickhouse itself will use external zookeeper +node = cluster.add_instance( + "node", + main_configs=["configs/enable_keeper.xml"], + stay_alive=True, + with_zookeeper=True, +) + +def random_string(length): + return "".join(random.choices(string.ascii_lowercase + string.digits, k=length)) + +def get_connection_zk(nodename, timeout=30.0): + _fake_zk_instance = KazooClient( + hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout + ) + _fake_zk_instance.start() + return _fake_zk_instance + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + +def test_soft_limit_create(started_cluster): + keeper_utils.wait_until_connected(started_cluster, node) + try: + node_zk = get_connection_zk("node") + loop_time = 1000000 + + for i in range(loop_time): + node_zk.create("/test_soft_limit/node_" + str(i), random_string(100)) + except ConnectionLoss: + txn = node_zk.transaction() + for i in range(10): + txn.delete("/test_soft_limit/node_" + str(i)) + + txn.create("/test_soft_limit/node_1000001" + str(i), "abcde") + txn.commit() + return + + raise Exception("all records are inserted but no error occurs") From 8e1b61cc131d2936e8d93883e315692c32c4d4fc Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 27 Nov 2023 22:03:05 +0100 Subject: [PATCH 030/110] fix style --- tests/integration/test_keeper_memory_soft_limit/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_keeper_memory_soft_limit/test.py b/tests/integration/test_keeper_memory_soft_limit/test.py index 29275c794a1..cfd255f9128 100644 --- a/tests/integration/test_keeper_memory_soft_limit/test.py +++ b/tests/integration/test_keeper_memory_soft_limit/test.py @@ -17,9 +17,11 @@ node = cluster.add_instance( with_zookeeper=True, ) + def random_string(length): return "".join(random.choices(string.ascii_lowercase + string.digits, k=length)) + def get_connection_zk(nodename, timeout=30.0): _fake_zk_instance = KazooClient( hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout @@ -38,6 +40,7 @@ def started_cluster(): finally: cluster.shutdown() + def test_soft_limit_create(started_cluster): keeper_utils.wait_until_connected(started_cluster, node) try: From 3aa4a6ae2c7ae50d96a56e9624fa46fa1c233cc7 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 28 Nov 2023 10:47:22 +0100 Subject: [PATCH 031/110] check integration test --- src/Coordination/KeeperDispatcher.cpp | 1 + tests/integration/test_keeper_memory_soft_limit/test.py | 7 +++++-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 26348707e19..908aa900982 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -138,6 +138,7 @@ void KeeperDispatcher::requestThread() Int64 mem_soft_limit = configuration_and_settings->coordination_settings->max_memory_usage_soft_limit; if (configuration_and_settings->standalone_keeper && mem_soft_limit > 0 && total_memory_tracker.get() >= mem_soft_limit && !checkIfRequestIncreaseMem(request.request)) { + LOG_TRACE(log, "Processing requests refused because of mem_soft_limit {}, request type is {}", mem_soft_limit, request.request->getOpNum()); addErrorResponses({request}, Coordination::Error::ZCONNECTIONLOSS); continue; } diff --git a/tests/integration/test_keeper_memory_soft_limit/test.py b/tests/integration/test_keeper_memory_soft_limit/test.py index cfd255f9128..4114b513864 100644 --- a/tests/integration/test_keeper_memory_soft_limit/test.py +++ b/tests/integration/test_keeper_memory_soft_limit/test.py @@ -46,15 +46,18 @@ def test_soft_limit_create(started_cluster): try: node_zk = get_connection_zk("node") loop_time = 1000000 + node_zk.create("/test_soft_limit", b"abc") for i in range(loop_time): - node_zk.create("/test_soft_limit/node_" + str(i), random_string(100)) + node_zk.create( + "/test_soft_limit/node_" + str(i), random_string(100).encode() + ) except ConnectionLoss: txn = node_zk.transaction() for i in range(10): txn.delete("/test_soft_limit/node_" + str(i)) - txn.create("/test_soft_limit/node_1000001" + str(i), "abcde") + txn.create("/test_soft_limit/node_1000001" + str(i), b"abcde") txn.commit() return From e4da317f056d31715358d5b806f4b00d5fe04b83 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 1 Dec 2023 18:07:45 +0100 Subject: [PATCH 032/110] fix test --- src/Coordination/KeeperDispatcher.cpp | 8 +-- tests/integration/helpers/cluster.py | 5 +- .../configs/enable_keeper.xml | 30 ----------- .../configs/keeper_config1.xml | 49 ++++++++++++++++++ .../configs/keeper_config2.xml | 50 +++++++++++++++++++ .../configs/keeper_config3.xml | 46 +++++++++++++++++ .../test_keeper_memory_soft_limit/test.py | 11 ++-- 7 files changed, 158 insertions(+), 41 deletions(-) delete mode 100644 tests/integration/test_keeper_memory_soft_limit/configs/enable_keeper.xml create mode 100644 tests/integration/test_keeper_memory_soft_limit/configs/keeper_config1.xml create mode 100644 tests/integration/test_keeper_memory_soft_limit/configs/keeper_config2.xml create mode 100644 tests/integration/test_keeper_memory_soft_limit/configs/keeper_config3.xml diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 908aa900982..60b6a521d64 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -58,7 +58,7 @@ bool checkIfRequestIncreaseMem(const Coordination::ZooKeeperRequestPtr & request { if (request->getOpNum() == Coordination::OpNum::Create || request->getOpNum() == Coordination::OpNum::CreateIfNotExists) { - return false; + return true; } else if (request->getOpNum() == Coordination::OpNum::Multi) { @@ -89,7 +89,7 @@ bool checkIfRequestIncreaseMem(const Coordination::ZooKeeperRequestPtr & request return memory_delta > 0; } - return true; + return false; } } @@ -136,9 +136,9 @@ void KeeperDispatcher::requestThread() break; Int64 mem_soft_limit = configuration_and_settings->coordination_settings->max_memory_usage_soft_limit; - if (configuration_and_settings->standalone_keeper && mem_soft_limit > 0 && total_memory_tracker.get() >= mem_soft_limit && !checkIfRequestIncreaseMem(request.request)) + if (configuration_and_settings->standalone_keeper && mem_soft_limit > 0 && total_memory_tracker.get() >= mem_soft_limit && checkIfRequestIncreaseMem(request.request)) { - LOG_TRACE(log, "Processing requests refused because of mem_soft_limit {}, request type is {}", mem_soft_limit, request.request->getOpNum()); + LOG_TRACE(log, "Processing requests refused because of mem_soft_limit {}, the total used memory is {}, request type is {}", mem_soft_limit, total_memory_tracker.get(), request.request->getOpNum()); addErrorResponses({request}, Coordination::Error::ZCONNECTIONLOSS); continue; } diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 5e4bb32cf94..22173850c92 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -390,6 +390,7 @@ class ClickHouseCluster: odbc_bridge_bin_path=None, library_bridge_bin_path=None, zookeeper_config_path=None, + keeper_config_dir=None, custom_dockerd_host=None, zookeeper_keyfile=None, zookeeper_certfile=None, @@ -426,6 +427,8 @@ class ClickHouseCluster: else p.join(HELPERS_DIR, "zookeeper_config.xml") ) + self.keeper_config_dir = p.join(self.base_dir, keeper_config_dir) if keeper_config_dir is not None else HELPERS_DIR + project_name = ( pwd.getpwuid(os.getuid()).pw_name + p.basename(self.base_dir) + self.name ) @@ -2725,7 +2728,7 @@ class ClickHouseCluster: if self.use_keeper: # TODO: remove hardcoded paths from here for i in range(1, 4): shutil.copy( - os.path.join(HELPERS_DIR, f"keeper_config{i}.xml"), + os.path.join(self.keeper_config_dir, f"keeper_config{i}.xml"), os.path.join( self.keeper_instance_dir_prefix + f"{i}", "config" ), diff --git a/tests/integration/test_keeper_memory_soft_limit/configs/enable_keeper.xml b/tests/integration/test_keeper_memory_soft_limit/configs/enable_keeper.xml deleted file mode 100644 index c289ea23d64..00000000000 --- a/tests/integration/test_keeper_memory_soft_limit/configs/enable_keeper.xml +++ /dev/null @@ -1,30 +0,0 @@ - - - 9181 - 1 - /var/lib/clickhouse/coordination/log - /var/lib/clickhouse/coordination/snapshots - - - 10 - 5 - 5000 - 10000 - trace - 10000000 - - - 0 - 0 - 0 - - - - - 1 - localhost - 9234 - - - - diff --git a/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config1.xml b/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config1.xml new file mode 100644 index 00000000000..fe45d09d915 --- /dev/null +++ b/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config1.xml @@ -0,0 +1,49 @@ + + true + :: + 0.0.0.0 + + + trace + /var/log/clickhouse-keeper/clickhouse-keeper.log + /var/log/clickhouse-keeper/clickhouse-keeper.err.log + + + + 2181 + + az-zoo1 + + 1 + + + 10000 + 15000 + trace + false + 2000 + 4000 + 200000000 + + 1 + + + + + 1 + zoo1 + 9444 + + + 2 + zoo2 + 9444 + + + 3 + zoo3 + 9444 + + + + diff --git a/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config2.xml b/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config2.xml new file mode 100644 index 00000000000..f7f6a5718b5 --- /dev/null +++ b/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config2.xml @@ -0,0 +1,50 @@ + + true + :: + 0.0.0.0 + + + trace + /var/log/clickhouse-keeper/clickhouse-keeper.log + /var/log/clickhouse-keeper/clickhouse-keeper.err.log + + + + 2181 + 2 + + az-zoo2 + 1 + + + + 10000 + 15000 + trace + false + 2000 + 4000 + 20000000 + + 1 + + + + + 1 + zoo1 + 9444 + + + 2 + zoo2 + 9444 + + + 3 + zoo3 + 9444 + + + + diff --git a/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config3.xml b/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config3.xml new file mode 100644 index 00000000000..82345aebc46 --- /dev/null +++ b/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config3.xml @@ -0,0 +1,46 @@ + + true + :: + 0.0.0.0 + + + trace + /var/log/clickhouse-keeper/clickhouse-keeper.log + /var/log/clickhouse-keeper/clickhouse-keeper.err.log + + + + 2181 + 3 + + + 10000 + 15000 + trace + false + 2000 + 4000 + 20000000 + + 1 + + + + + 1 + zoo1 + 9444 + + + 2 + zoo2 + 9444 + + + 3 + zoo3 + 9444 + + + + diff --git a/tests/integration/test_keeper_memory_soft_limit/test.py b/tests/integration/test_keeper_memory_soft_limit/test.py index a90342fb069..d6f3d013a7b 100644 --- a/tests/integration/test_keeper_memory_soft_limit/test.py +++ b/tests/integration/test_keeper_memory_soft_limit/test.py @@ -7,12 +7,11 @@ from helpers import keeper_utils from kazoo.client import KazooClient, KazooState from kazoo.exceptions import ConnectionLoss -cluster = ClickHouseCluster(__file__) +cluster = ClickHouseCluster(__file__, keeper_config_dir="configs/") # clickhouse itself will use external zookeeper node = cluster.add_instance( "node", - main_configs=["configs/enable_keeper.xml"], stay_alive=True, with_zookeeper=True, ) @@ -24,7 +23,7 @@ def random_string(length): def get_connection_zk(nodename, timeout=30.0): _fake_zk_instance = KazooClient( - hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout + hosts=cluster.get_instance_ip(nodename) + ":2181", timeout=timeout ) _fake_zk_instance.start() return _fake_zk_instance @@ -42,10 +41,10 @@ def started_cluster(): def test_soft_limit_create(started_cluster): - keeper_utils.wait_until_connected(started_cluster, node) + started_cluster.wait_zookeeper_to_start() try: - node_zk = get_connection_zk("node") - loop_time = 10000 + node_zk = get_connection_zk("zoo1") + loop_time = 100000 node_zk.create("/test_soft_limit", b"abc") for i in range(loop_time): From fc0f080cb459528d3bf4721b1078aac11f778916 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 1 Dec 2023 20:39:38 +0100 Subject: [PATCH 033/110] fix black --- tests/integration/helpers/cluster.py | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 22173850c92..c0b145b047a 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -427,7 +427,11 @@ class ClickHouseCluster: else p.join(HELPERS_DIR, "zookeeper_config.xml") ) - self.keeper_config_dir = p.join(self.base_dir, keeper_config_dir) if keeper_config_dir is not None else HELPERS_DIR + self.keeper_config_dir = ( + p.join(self.base_dir, keeper_config_dir) + if keeper_config_dir + else HELPERS_DIR + ) project_name = ( pwd.getpwuid(os.getuid()).pw_name + p.basename(self.base_dir) + self.name @@ -2728,7 +2732,9 @@ class ClickHouseCluster: if self.use_keeper: # TODO: remove hardcoded paths from here for i in range(1, 4): shutil.copy( - os.path.join(self.keeper_config_dir, f"keeper_config{i}.xml"), + os.path.join( + self.keeper_config_dir, f"keeper_config{i}.xml" + ), os.path.join( self.keeper_instance_dir_prefix + f"{i}", "config" ), From 9a68461cdb1dde1fdd9f22e6b8958f497590d25d Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 3 Dec 2023 12:43:16 +0100 Subject: [PATCH 034/110] Update src/Coordination/KeeperDispatcher.cpp --- src/Coordination/KeeperDispatcher.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 60b6a521d64..711b9939451 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -399,7 +399,6 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ if (keeper_context->shutdown_called) return false; - /// Put close requests without timeouts if (request->getOpNum() == Coordination::OpNum::Close) { From acc53431c7de1caa1e2bdf542fdc9fa9bd804957 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 3 Dec 2023 12:43:20 +0100 Subject: [PATCH 035/110] Update src/Coordination/KeeperDispatcher.cpp --- src/Coordination/KeeperDispatcher.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 711b9939451..544f4e033bb 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -138,7 +138,7 @@ void KeeperDispatcher::requestThread() Int64 mem_soft_limit = configuration_and_settings->coordination_settings->max_memory_usage_soft_limit; if (configuration_and_settings->standalone_keeper && mem_soft_limit > 0 && total_memory_tracker.get() >= mem_soft_limit && checkIfRequestIncreaseMem(request.request)) { - LOG_TRACE(log, "Processing requests refused because of mem_soft_limit {}, the total used memory is {}, request type is {}", mem_soft_limit, total_memory_tracker.get(), request.request->getOpNum()); + LOG_TRACE(log, "Processing requests refused because of max_memory_usage_soft_limit {}, the total used memory is {}, request type is {}", mem_soft_limit, total_memory_tracker.get(), request.request->getOpNum()); addErrorResponses({request}, Coordination::Error::ZCONNECTIONLOSS); continue; } From fefbf397c005bcd6553c123b7c3ed5f054f12559 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 3 Dec 2023 13:44:35 +0100 Subject: [PATCH 036/110] Add more tests for RWLock. --- src/Common/tests/gtest_rw_lock.cpp | 230 +++++++++++++++++++++++++++++ 1 file changed, 230 insertions(+) diff --git a/src/Common/tests/gtest_rw_lock.cpp b/src/Common/tests/gtest_rw_lock.cpp index 5ea50f70d4e..7de3ced2d0d 100644 --- a/src/Common/tests/gtest_rw_lock.cpp +++ b/src/Common/tests/gtest_rw_lock.cpp @@ -24,6 +24,39 @@ namespace DB } +namespace +{ + class Events + { + public: + Events() : start_time(std::chrono::steady_clock::now()) {} + + void add(String && event) + { + String timepoint = std::to_string(std::chrono::duration_cast(std::chrono::steady_clock::now() - start_time).count()); + if (timepoint.length() < 5) + timepoint.insert(0, 5 - timepoint.length(), ' '); + std::lock_guard lock{mutex}; + std::cout << timepoint << " : " << event << std::endl; + events.emplace_back(std::move(event)); + } + + void check(const Strings & expected_events) + { + std::lock_guard lock{mutex}; + EXPECT_EQ(events.size(), expected_events.size()); + for (size_t i = 0; i != events.size(); ++i) + EXPECT_EQ(events[i], (i < expected_events.size() ? expected_events[i] : "")); + } + + private: + const std::chrono::time_point start_time; + Strings events TSA_GUARDED_BY(mutex); + mutable std::mutex mutex; + }; +} + + TEST(Common, RWLock1) { /// Tests with threads require this, because otherwise @@ -287,3 +320,200 @@ TEST(Common, RWLockNotUpgradeableWithNoQuery) read_thread.join(); } + + +TEST(Common, RWLockWriteLockTimeoutDuringRead) +{ + static auto rw_lock = RWLockImpl::create(); + Events events; + + std::thread ra_thread([&] () + { + events.add("Locking ra"); + auto ra = rw_lock->getLock(RWLockImpl::Read, "ra"); + events.add(ra ? "Locked ra" : "Failed to lock ra"); + EXPECT_NE(ra, nullptr); + + std::this_thread::sleep_for(std::chrono::duration(400)); + + events.add("Unlocking ra"); + ra.reset(); + events.add("Unlocked ra"); + }); + + std::thread wc_thread([&] () + { + std::this_thread::sleep_for(std::chrono::duration(100)); + events.add("Locking wc"); + auto wc = rw_lock->getLock(RWLockImpl::Write, "wc", std::chrono::milliseconds(200)); + events.add(wc ? "Locked wc" : "Failed to lock wc"); + EXPECT_EQ(wc, nullptr); + }); + + ra_thread.join(); + wc_thread.join(); + + { + events.add("Locking wd"); + auto wd = rw_lock->getLock(RWLockImpl::Write, "wd", std::chrono::milliseconds(1000)); + events.add(wd ? "Locked wd" : "Failed to lock wd"); + EXPECT_NE(wd, nullptr); + events.add("Unlocking wd"); + wd.reset(); + events.add("Unlocked wd"); + } + + events.check( + {"Locking ra", + "Locked ra", + "Locking wc", + "Failed to lock wc", + "Unlocking ra", + "Unlocked ra", + "Locking wd", + "Locked wd", + "Unlocking wd", + "Unlocked wd"}); +} + + +TEST(Common, RWLockWriteLockTimeoutDuringTwoReads) +{ + static auto rw_lock = RWLockImpl::create(); + Events events; + + std::thread ra_thread([&] () + { + events.add("Locking ra"); + auto ra = rw_lock->getLock(RWLockImpl::Read, "ra"); + events.add(ra ? "Locked ra" : "Failed to lock ra"); + EXPECT_NE(ra, nullptr); + + std::this_thread::sleep_for(std::chrono::duration(400)); + + events.add("Unlocking ra"); + ra.reset(); + events.add("Unlocked ra"); + }); + + std::thread rb_thread([&] () + { + std::this_thread::sleep_for(std::chrono::duration(200)); + events.add("Locking rb"); + + auto rb = rw_lock->getLock(RWLockImpl::Read, "rb"); + events.add(rb ? "Locked rb" : "Failed to lock rb"); + EXPECT_NE(rb, nullptr); + + std::this_thread::sleep_for(std::chrono::duration(200)); + events.add("Unlocking rb"); + rb.reset(); + events.add("Unlocked rb"); + }); + + std::thread wc_thread([&] () + { + std::this_thread::sleep_for(std::chrono::duration(100)); + events.add("Locking wc"); + auto wc = rw_lock->getLock(RWLockImpl::Write, "wc", std::chrono::milliseconds(200)); + events.add(wc ? "Locked wc" : "Failed to lock wc"); + EXPECT_EQ(wc, nullptr); + }); + + ra_thread.join(); + rb_thread.join(); + wc_thread.join(); + + { + events.add("Locking wd"); + auto wd = rw_lock->getLock(RWLockImpl::Write, "wd", std::chrono::milliseconds(1000)); + events.add(wd ? "Locked wd" : "Failed to lock wd"); + EXPECT_NE(wd, nullptr); + events.add("Unlocking wd"); + wd.reset(); + events.add("Unlocked wd"); + } + + events.check( + {"Locking ra", + "Locked ra", + "Locking wc", + "Locking rb", + "Failed to lock wc", + "Locked rb", + "Unlocking ra", + "Unlocked ra", + "Unlocking rb", + "Unlocked rb", + "Locking wd", + "Locked wd", + "Unlocking wd", + "Unlocked wd"}); +} + + +TEST(Common, RWLockWriteLockTimeoutDuringWriteWithWaitingRead) +{ + static auto rw_lock = RWLockImpl::create(); + Events events; + + std::thread wa_thread([&] () + { + events.add("Locking wa"); + auto wa = rw_lock->getLock(RWLockImpl::Write, "wa"); + events.add(wa ? "Locked wa" : "Failed to lock wa"); + EXPECT_NE(wa, nullptr); + + std::this_thread::sleep_for(std::chrono::duration(500)); + + events.add("Unlocking wa"); + wa.reset(); + events.add("Unlocked wa"); + }); + + std::thread wb_thread([&] () + { + std::this_thread::sleep_for(std::chrono::duration(100)); + events.add("Locking wb"); + auto wc = rw_lock->getLock(RWLockImpl::Write, "wc", std::chrono::milliseconds(200)); + events.add(wc ? "Locked wb" : "Failed to lock wb"); + EXPECT_EQ(wc, nullptr); + }); + + std::thread rc_thread([&] () + { + std::this_thread::sleep_for(std::chrono::duration(200)); + events.add("Locking rc"); + auto rc = rw_lock->getLock(RWLockImpl::Read, "rc", std::chrono::milliseconds(200)); + events.add(rc ? "Locked rc" : "Failed to lock rc"); + EXPECT_EQ(rc, nullptr); + }); + + wa_thread.join(); + wb_thread.join(); + rc_thread.join(); + + { + events.add("Locking wd"); + auto wd = rw_lock->getLock(RWLockImpl::Write, "wd", std::chrono::milliseconds(1000)); + events.add(wd ? "Locked wd" : "Failed to lock wd"); + EXPECT_NE(wd, nullptr); + events.add("Unlocking wd"); + wd.reset(); + events.add("Unlocked wd"); + } + + events.check( + {"Locking wa", + "Locked wa", + "Locking wb", + "Locking rc", + "Failed to lock wb", + "Failed to lock rc", + "Unlocking wa", + "Unlocked wa", + "Locking wd", + "Locked wd", + "Unlocking wd", + "Unlocked wd"}); +} From 44d6605b8a42db29d477d46c2fccd67046d6b4f8 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 3 Dec 2023 13:46:52 +0100 Subject: [PATCH 037/110] Fix RWLock inconsistency after write lock timeout. --- src/Common/RWLock.cpp | 120 +++++++++++++++++++++++++++++++----------- src/Common/RWLock.h | 23 +++++--- 2 files changed, 105 insertions(+), 38 deletions(-) diff --git a/src/Common/RWLock.cpp b/src/Common/RWLock.cpp index 2d0fcfa3e74..5a13bb83f29 100644 --- a/src/Common/RWLock.cpp +++ b/src/Common/RWLock.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include namespace ProfileEvents @@ -155,12 +157,19 @@ RWLockImpl::getLock(RWLockImpl::Type type, const String & query_id, const std::c if (type == Type::Write) { + /// Always add a group for a writer (writes are never performed simultaneously). writers_queue.emplace_back(type); /// SM1: may throw (nothing to roll back) } - else if (readers_queue.empty() || - (rdlock_owner == readers_queue.begin() && readers_queue.size() == 1 && !writers_queue.empty())) + else { - readers_queue.emplace_back(type); /// SM1: may throw (nothing to roll back) + /// We don't always add a group to readers_queue here because multiple readers can use the same group. + /// We can reuse the last group if we're in write phase now, or if the last group didn't get ownership yet, + /// or even if it got ownership but there are no writers waiting in writers_queue. + bool can_use_last_group = !readers_queue.empty() && + ((rdlock_owner == readers_queue.end()) || !rdlock_owner->ownership || writers_queue.empty()); + + if (!can_use_last_group) + readers_queue.emplace_back(type); /// SM1: may throw (nothing to roll back) } GroupsContainer::iterator it_group = (type == Type::Write) ? std::prev(writers_queue.end()) : std::prev(readers_queue.end()); @@ -169,11 +178,12 @@ RWLockImpl::getLock(RWLockImpl::Type type, const String & query_id, const std::c if (rdlock_owner == readers_queue.end() && wrlock_owner == writers_queue.end()) { (type == Read ? rdlock_owner : wrlock_owner) = it_group; /// SM2: nothrow + grantOwnership(it_group); } else { /// Wait until our group becomes the lock owner - const auto predicate = [&] () { return it_group == (type == Read ? rdlock_owner : wrlock_owner); }; + const auto predicate = [&] () { return it_group->ownership; }; if (lock_deadline_tp == std::chrono::time_point::max()) { @@ -193,10 +203,12 @@ RWLockImpl::getLock(RWLockImpl::Type type, const String & query_id, const std::c /// Rollback(SM1): nothrow if (it_group->requests == 0) { - /// When WRITE lock fails, we need to notify next read that is waiting, - /// to avoid handing request, hence next=true. - dropOwnerGroupAndPassOwnership(it_group, /* next= */ true); + ((type == Read) ? readers_queue : writers_queue).erase(it_group); } + /// While we were waiting for this write lock (which has just failed) more readers could start waiting, + /// we need to wake up them now. + if ((rdlock_owner != readers_queue.end()) && writers_queue.empty()) + grantOwnershipToAllReaders(); return nullptr; } } @@ -216,7 +228,7 @@ RWLockImpl::getLock(RWLockImpl::Type type, const String & query_id, const std::c /// Methods std::list<>::emplace_back() and std::unordered_map<>::emplace() provide strong exception safety /// We only need to roll back the changes to these objects: owner_queries and the readers/writers queue if (it_group->requests == 0) - dropOwnerGroupAndPassOwnership(it_group, /* next= */ false); /// Rollback(SM1): nothrow + dropOwnerGroupAndPassOwnership(it_group); /// Rollback(SM1): nothrow throw; } @@ -246,8 +258,6 @@ void RWLockImpl::unlock(GroupsContainer::iterator group_it, const String & query /// All of these are Undefined behavior and nothing we can do! if (rdlock_owner == readers_queue.end() && wrlock_owner == writers_queue.end()) return; - if (rdlock_owner != readers_queue.end() && group_it != rdlock_owner) - return; if (wrlock_owner != writers_queue.end() && group_it != wrlock_owner) return; @@ -264,12 +274,26 @@ void RWLockImpl::unlock(GroupsContainer::iterator group_it, const String & query /// If we are the last remaining referrer, remove this QNode and notify the next one if (--group_it->requests == 0) /// SM: nothrow - dropOwnerGroupAndPassOwnership(group_it, /* next= */ false); + dropOwnerGroupAndPassOwnership(group_it); } -void RWLockImpl::dropOwnerGroupAndPassOwnership(GroupsContainer::iterator group_it, bool next) noexcept +void RWLockImpl::dropOwnerGroupAndPassOwnership(GroupsContainer::iterator group_it) noexcept { + /// All readers with ownership must finish before switching to write phase. + /// Such readers has iterators from `readers_queue.begin()` to `rdlock_owner`, so if `rdlock_owner` is equal to `readers_queue.begin()` + /// that means there is only one reader with ownership left in the readers_queue and we can proceed to generic procedure. + if ((group_it->type == Read) && (rdlock_owner != readers_queue.begin()) && (rdlock_owner != readers_queue.end())) + { + if (rdlock_owner == group_it) + --rdlock_owner; + readers_queue.erase(group_it); + /// If there are no writers waiting in writers_queue then we can wake up other readers. + if (writers_queue.empty()) + grantOwnershipToAllReaders(); + return; + } + rdlock_owner = readers_queue.end(); wrlock_owner = writers_queue.end(); @@ -278,42 +302,78 @@ void RWLockImpl::dropOwnerGroupAndPassOwnership(GroupsContainer::iterator group_ readers_queue.erase(group_it); /// Prepare next phase if (!writers_queue.empty()) - { wrlock_owner = writers_queue.begin(); - } else - { rdlock_owner = readers_queue.begin(); - } } else { writers_queue.erase(group_it); /// Prepare next phase if (!readers_queue.empty()) - { - if (next && readers_queue.size() > 1) - { - rdlock_owner = std::next(readers_queue.begin()); - } - else - { - rdlock_owner = readers_queue.begin(); - } - } + rdlock_owner = readers_queue.begin(); else - { wrlock_owner = writers_queue.begin(); - } } if (rdlock_owner != readers_queue.end()) { - rdlock_owner->cv.notify_all(); + grantOwnershipToAllReaders(); } else if (wrlock_owner != writers_queue.end()) { - wrlock_owner->cv.notify_one(); + grantOwnership(wrlock_owner); } } + + +void RWLockImpl::grantOwnership(GroupsContainer::iterator group_it) noexcept +{ + if (!group_it->ownership) + { + group_it->ownership = true; + group_it->cv.notify_all(); + } +} + + +void RWLockImpl::grantOwnershipToAllReaders() noexcept +{ + if (rdlock_owner != readers_queue.end()) + { + for (;;) + { + grantOwnership(rdlock_owner); + if (std::next(rdlock_owner) == readers_queue.end()) + break; + ++rdlock_owner; + } + } +} + + +std::unordered_map RWLockImpl::getOwnerQueryIds() const +{ + std::lock_guard lock{internal_state_mtx}; + return owner_queries; +} + + +String RWLockImpl::getOwnerQueryIdsDescription() const +{ + auto map = getOwnerQueryIds(); + WriteBufferFromOwnString out; + bool need_comma = false; + for (const auto & [query_id, num_owners] : map) + { + if (need_comma) + out << ", "; + out << query_id; + if (num_owners != 1) + out << " (" << num_owners << ")"; + need_comma = true; + } + return out.str(); +} + } diff --git a/src/Common/RWLock.h b/src/Common/RWLock.h index 156e4297d81..1eb6964a991 100644 --- a/src/Common/RWLock.h +++ b/src/Common/RWLock.h @@ -62,35 +62,42 @@ public: inline static const String NO_QUERY = String(); inline static const auto default_locking_timeout_ms = std::chrono::milliseconds(120000); + /// Returns all query_id owning locks (both read and write) right now. + /// !! This function are for debugging and logging purposes only, DO NOT use them for synchronization! + std::unordered_map getOwnerQueryIds() const; + String getOwnerQueryIdsDescription() const; + private: /// Group of locking requests that should be granted simultaneously /// i.e. one or several readers or a single writer struct Group { const Type type; - size_t requests; + size_t requests = 0; + bool ownership = false; /// whether this group got ownership? (that means `cv` is notified and the locking requests should stop waiting) std::condition_variable cv; /// all locking requests of the group wait on this condvar - explicit Group(Type type_) : type{type_}, requests{0} {} + explicit Group(Type type_) : type{type_} {} }; using GroupsContainer = std::list; - using OwnerQueryIds = std::unordered_map; + using OwnerQueryIds = std::unordered_map; mutable std::mutex internal_state_mtx; GroupsContainer readers_queue; GroupsContainer writers_queue; - GroupsContainer::iterator rdlock_owner{readers_queue.end()}; /// equals to readers_queue.begin() in read phase - /// or readers_queue.end() otherwise + GroupsContainer::iterator rdlock_owner{readers_queue.end()}; /// last group with ownership in readers_queue in read phase + /// or readers_queue.end() in writer phase GroupsContainer::iterator wrlock_owner{writers_queue.end()}; /// equals to writers_queue.begin() in write phase - /// or writers_queue.end() otherwise + /// or writers_queue.end() in read phase OwnerQueryIds owner_queries; RWLockImpl() = default; void unlock(GroupsContainer::iterator group_it, const String & query_id) noexcept; - /// @param next - notify next after begin, used on writer lock failures - void dropOwnerGroupAndPassOwnership(GroupsContainer::iterator group_it, bool next) noexcept; + void dropOwnerGroupAndPassOwnership(GroupsContainer::iterator group_it) noexcept; + void grantOwnership(GroupsContainer::iterator group_it) noexcept; + void grantOwnershipToAllReaders() noexcept; }; } From 0457aba4a96243609cd2595c561c49941f24cba3 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 3 Dec 2023 14:04:54 +0100 Subject: [PATCH 038/110] Show owner query ids in the message for the DEADLOCK_AVOIDED error. --- src/Storages/IStorage.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index af1c032bc56..88603d56ebb 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -41,8 +41,8 @@ RWLockImpl::LockHolder IStorage::tryLockTimed( { const String type_str = type == RWLockImpl::Type::Read ? "READ" : "WRITE"; throw Exception(ErrorCodes::DEADLOCK_AVOIDED, - "{} locking attempt on \"{}\" has timed out! ({}ms) Possible deadlock avoided. Client should retry", - type_str, getStorageID(), acquire_timeout.count()); + "{} locking attempt on \"{}\" has timed out! ({}ms) Possible deadlock avoided. Client should retry. Owner query ids: {}", + type_str, getStorageID(), acquire_timeout.count(), rwlock->getOwnerQueryIdsDescription()); } return lock_holder; } From 366095affcc01792caeccb2b94317ad2698225e8 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 3 Dec 2023 19:59:26 +0100 Subject: [PATCH 039/110] Fix test test_create_or_drop_tables_during_backup. --- .../test_concurrency.py | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py index ab37846db9a..65f05d186ce 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py @@ -216,7 +216,7 @@ def test_create_or_drop_tables_during_backup(db_engine, table_engine): node = nodes[randint(0, num_nodes - 1)] # "DROP TABLE IF EXISTS" still can throw some errors (e.g. "WRITE locking attempt on node0 has timed out!") # So we use query_and_get_answer_with_error() to ignore any errors. - # `lock_acquire_timeout` is also reduced because we don't wait our test to wait too long. + # `lock_acquire_timeout` is reduced because we don't wait our test to wait too long. node.query_and_get_answer_with_error( f"DROP TABLE IF EXISTS {table_name} SYNC", settings={"lock_acquire_timeout": 10}, @@ -227,15 +227,23 @@ def test_create_or_drop_tables_during_backup(db_engine, table_engine): table_name1 = f"mydb.tbl{randint(1, num_nodes)}" table_name2 = f"mydb.tbl{randint(1, num_nodes)}" node = nodes[randint(0, num_nodes - 1)] + # `lock_acquire_timeout` is reduced because we don't wait our test to wait too long. node.query_and_get_answer_with_error( - f"RENAME TABLE {table_name1} TO {table_name2}" + f"RENAME TABLE {table_name1} TO {table_name2}", + settings={"lock_acquire_timeout": 10}, ) def truncate_tables(): while time.time() < end_time: table_name = f"mydb.tbl{randint(1, num_nodes)}" node = nodes[randint(0, num_nodes - 1)] - node.query(f"TRUNCATE TABLE IF EXISTS {table_name} SYNC") + # "TRUNCATE TABLE IF EXISTS" still can throw some errors (e.g. "WRITE locking attempt on node0 has timed out!") + # So we use query_and_get_answer_with_error() to ignore any errors. + # `lock_acquire_timeout` is reduced because we don't wait our test to wait too long. + node.query_and_get_answer_with_error( + f"TRUNCATE TABLE IF EXISTS {table_name} SYNC", + settings={"lock_acquire_timeout": 10}, + ) def make_backups(): ids = [] From 73ff9847554277234393663a99b1308ee4815998 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 4 Dec 2023 09:33:34 +0000 Subject: [PATCH 040/110] Remove select() usage --- .../glibc-compatibility/glibc-compatibility.c | 1 - base/poco/Foundation/CMakeLists.txt | 1 - .../include/Poco/DirectoryWatcher.h | 228 ------- base/poco/Foundation/src/DirectoryWatcher.cpp | 602 ------------------ src/Client/LineReader.cpp | 9 +- src/IO/ReadBufferFromFileDescriptor.cpp | 18 - src/IO/ReadBufferFromFileDescriptor.h | 4 - 7 files changed, 3 insertions(+), 860 deletions(-) delete mode 100644 base/poco/Foundation/include/Poco/DirectoryWatcher.h delete mode 100644 base/poco/Foundation/src/DirectoryWatcher.cpp diff --git a/base/glibc-compatibility/glibc-compatibility.c b/base/glibc-compatibility/glibc-compatibility.c index 49bb81a58be..738cda47877 100644 --- a/base/glibc-compatibility/glibc-compatibility.c +++ b/base/glibc-compatibility/glibc-compatibility.c @@ -30,7 +30,6 @@ int __gai_sigqueue(int sig, const union sigval val, pid_t caller_pid) } -#include #include #include diff --git a/base/poco/Foundation/CMakeLists.txt b/base/poco/Foundation/CMakeLists.txt index d0dde8a51a5..dfb41a33fb1 100644 --- a/base/poco/Foundation/CMakeLists.txt +++ b/base/poco/Foundation/CMakeLists.txt @@ -55,7 +55,6 @@ set (SRCS src/DigestStream.cpp src/DirectoryIterator.cpp src/DirectoryIteratorStrategy.cpp - src/DirectoryWatcher.cpp src/Environment.cpp src/Error.cpp src/ErrorHandler.cpp diff --git a/base/poco/Foundation/include/Poco/DirectoryWatcher.h b/base/poco/Foundation/include/Poco/DirectoryWatcher.h deleted file mode 100644 index 00964a5512a..00000000000 --- a/base/poco/Foundation/include/Poco/DirectoryWatcher.h +++ /dev/null @@ -1,228 +0,0 @@ -// -// DirectoryWatcher.h -// -// Library: Foundation -// Package: Filesystem -// Module: DirectoryWatcher -// -// Definition of the DirectoryWatcher class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef Foundation_DirectoryWatcher_INCLUDED -#define Foundation_DirectoryWatcher_INCLUDED - - -#include "Poco/Foundation.h" - - -#ifndef POCO_NO_INOTIFY - - -# include "Poco/AtomicCounter.h" -# include "Poco/BasicEvent.h" -# include "Poco/File.h" -# include "Poco/Runnable.h" -# include "Poco/Thread.h" - - -namespace Poco -{ - - -class DirectoryWatcherStrategy; - - -class Foundation_API DirectoryWatcher : protected Runnable -/// This class is used to get notifications about changes -/// to the filesystem, more specifically, to a specific -/// directory. Changes to a directory are reported via -/// events. -/// -/// A thread will be created that watches the specified -/// directory for changes. Events are reported in the context -/// of this thread. -/// -/// Note that changes to files in subdirectories of the watched -/// directory are not reported. Separate DirectoryWatcher objects -/// must be created for these directories if they should be watched. -/// -/// Changes to file attributes are not reported. -/// -/// On Windows, this class is implemented using FindFirstChangeNotification()/FindNextChangeNotification(). -/// On Linux, this class is implemented using inotify. -/// On FreeBSD and Darwin (Mac OS X, iOS), this class uses kevent/kqueue. -/// On all other platforms, the watched directory is periodically scanned -/// for changes. This can negatively affect performance if done too often. -/// Therefore, the interval in which scans are done can be specified in -/// the constructor. Note that periodic scanning will also be done on FreeBSD -/// and Darwin if events for changes to files (DW_ITEM_MODIFIED) are enabled. -/// -/// DW_ITEM_MOVED_FROM and DW_ITEM_MOVED_TO events will only be reported -/// on Linux. On other platforms, a file rename or move operation -/// will be reported via a DW_ITEM_REMOVED and a DW_ITEM_ADDED event. -/// The order of these two events is not defined. -/// -/// An event mask can be specified to enable only certain events. -{ -public: - enum DirectoryEventType - { - DW_ITEM_ADDED = 1, - /// A new item has been created and added to the directory. - - DW_ITEM_REMOVED = 2, - /// An item has been removed from the directory. - - DW_ITEM_MODIFIED = 4, - /// An item has been modified. - - DW_ITEM_MOVED_FROM = 8, - /// An item has been renamed or moved. This event delivers the old name. - - DW_ITEM_MOVED_TO = 16 - /// An item has been renamed or moved. This event delivers the new name. - }; - - enum DirectoryEventMask - { - DW_FILTER_ENABLE_ALL = 31, - /// Enables all event types. - - DW_FILTER_DISABLE_ALL = 0 - /// Disables all event types. - }; - - enum - { - DW_DEFAULT_SCAN_INTERVAL = 5 /// Default scan interval for platforms that don't provide a native notification mechanism. - }; - - struct DirectoryEvent - { - DirectoryEvent(const File & f, DirectoryEventType ev) : item(f), event(ev) { } - - const File & item; /// The directory or file that has been changed. - DirectoryEventType event; /// The kind of event. - }; - - BasicEvent itemAdded; - /// Fired when a file or directory has been created or added to the directory. - - BasicEvent itemRemoved; - /// Fired when a file or directory has been removed from the directory. - - BasicEvent itemModified; - /// Fired when a file or directory has been modified. - - BasicEvent itemMovedFrom; - /// Fired when a file or directory has been renamed. This event delivers the old name. - - BasicEvent itemMovedTo; - /// Fired when a file or directory has been moved. This event delivers the new name. - - BasicEvent scanError; - /// Fired when an error occurs while scanning for changes. - - DirectoryWatcher(const std::string & path, int eventMask = DW_FILTER_ENABLE_ALL, int scanInterval = DW_DEFAULT_SCAN_INTERVAL); - /// Creates a DirectoryWatcher for the directory given in path. - /// To enable only specific events, an eventMask can be specified by - /// OR-ing the desired event IDs (e.g., DW_ITEM_ADDED | DW_ITEM_MODIFIED). - /// On platforms where no native filesystem notifications are available, - /// scanInterval specifies the interval in seconds between scans - /// of the directory. - - DirectoryWatcher(const File & directory, int eventMask = DW_FILTER_ENABLE_ALL, int scanInterval = DW_DEFAULT_SCAN_INTERVAL); - /// Creates a DirectoryWatcher for the specified directory - /// To enable only specific events, an eventMask can be specified by - /// OR-ing the desired event IDs (e.g., DW_ITEM_ADDED | DW_ITEM_MODIFIED). - /// On platforms where no native filesystem notifications are available, - /// scanInterval specifies the interval in seconds between scans - /// of the directory. - - ~DirectoryWatcher(); - /// Destroys the DirectoryWatcher. - - void suspendEvents(); - /// Suspends sending of events. Can be called multiple times, but every - /// call to suspendEvent() must be matched by a call to resumeEvents(). - - void resumeEvents(); - /// Resumes events, after they have been suspended with a call to suspendEvents(). - - bool eventsSuspended() const; - /// Returns true iff events are suspended. - - int eventMask() const; - /// Returns the value of the eventMask passed to the constructor. - - int scanInterval() const; - /// Returns the scan interval in seconds. - - const File & directory() const; - /// Returns the directory being watched. - - bool supportsMoveEvents() const; - /// Returns true iff the platform supports DW_ITEM_MOVED_FROM/itemMovedFrom and - /// DW_ITEM_MOVED_TO/itemMovedTo events. - -protected: - void init(); - void stop(); - void run(); - -private: - DirectoryWatcher(); - DirectoryWatcher(const DirectoryWatcher &); - DirectoryWatcher & operator=(const DirectoryWatcher &); - - Thread _thread; - File _directory; - int _eventMask; - AtomicCounter _eventsSuspended; - int _scanInterval; - DirectoryWatcherStrategy * _pStrategy; -}; - - -// -// inlines -// - - -inline bool DirectoryWatcher::eventsSuspended() const -{ - return _eventsSuspended.value() > 0; -} - - -inline int DirectoryWatcher::eventMask() const -{ - return _eventMask; -} - - -inline int DirectoryWatcher::scanInterval() const -{ - return _scanInterval; -} - - -inline const File & DirectoryWatcher::directory() const -{ - return _directory; -} - - -} // namespace Poco - - -#endif // POCO_NO_INOTIFY - - -#endif // Foundation_DirectoryWatcher_INCLUDED diff --git a/base/poco/Foundation/src/DirectoryWatcher.cpp b/base/poco/Foundation/src/DirectoryWatcher.cpp deleted file mode 100644 index b559da65e09..00000000000 --- a/base/poco/Foundation/src/DirectoryWatcher.cpp +++ /dev/null @@ -1,602 +0,0 @@ -// -// DirectoryWatcher.cpp -// -// Library: Foundation -// Package: Filesystem -// Module: DirectoryWatcher -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/DirectoryWatcher.h" - - -#ifndef POCO_NO_INOTIFY - - -#include "Poco/Path.h" -#include "Poco/Glob.h" -#include "Poco/DirectoryIterator.h" -#include "Poco/Event.h" -#include "Poco/Exception.h" -#include "Poco/Buffer.h" -#if POCO_OS == POCO_OS_LINUX || POCO_OS == POCO_OS_ANDROID - #include - #include - #include -#elif POCO_OS == POCO_OS_MAC_OS_X || POCO_OS == POCO_OS_FREE_BSD - #include - #include - #include - #include - #include - #if (POCO_OS == POCO_OS_FREE_BSD) && !defined(O_EVTONLY) - #define O_EVTONLY 0x8000 - #endif -#endif -#include -#include -#include - -namespace Poco { - - -class DirectoryWatcherStrategy -{ -public: - DirectoryWatcherStrategy(DirectoryWatcher& owner): - _owner(owner) - { - } - - virtual ~DirectoryWatcherStrategy() - { - } - - DirectoryWatcher& owner() - { - return _owner; - } - - virtual void run() = 0; - virtual void stop() = 0; - virtual bool supportsMoveEvents() const = 0; - -protected: - struct ItemInfo - { - ItemInfo(): - size(0) - { - } - - ItemInfo(const ItemInfo& other): - path(other.path), - size(other.size), - lastModified(other.lastModified) - { - } - - explicit ItemInfo(const File& f): - path(f.path()), - size(f.isFile() ? f.getSize() : 0), - lastModified(f.getLastModified()) - { - } - - std::string path; - File::FileSize size; - Timestamp lastModified; - }; - typedef std::map ItemInfoMap; - - void scan(ItemInfoMap& entries) - { - DirectoryIterator it(owner().directory()); - DirectoryIterator end; - while (it != end) - { - entries[it.path().getFileName()] = ItemInfo(*it); - ++it; - } - } - - void compare(ItemInfoMap& oldEntries, ItemInfoMap& newEntries) - { - for (ItemInfoMap::iterator itn = newEntries.begin(); itn != newEntries.end(); ++itn) - { - ItemInfoMap::iterator ito = oldEntries.find(itn->first); - if (ito != oldEntries.end()) - { - if ((owner().eventMask() & DirectoryWatcher::DW_ITEM_MODIFIED) && !owner().eventsSuspended()) - { - if (itn->second.size != ito->second.size || itn->second.lastModified != ito->second.lastModified) - { - Poco::File f(itn->second.path); - DirectoryWatcher::DirectoryEvent ev(f, DirectoryWatcher::DW_ITEM_MODIFIED); - owner().itemModified(&owner(), ev); - } - } - oldEntries.erase(ito); - } - else if ((owner().eventMask() & DirectoryWatcher::DW_ITEM_ADDED) && !owner().eventsSuspended()) - { - Poco::File f(itn->second.path); - DirectoryWatcher::DirectoryEvent ev(f, DirectoryWatcher::DW_ITEM_ADDED); - owner().itemAdded(&owner(), ev); - } - } - if ((owner().eventMask() & DirectoryWatcher::DW_ITEM_REMOVED) && !owner().eventsSuspended()) - { - for (ItemInfoMap::iterator it = oldEntries.begin(); it != oldEntries.end(); ++it) - { - Poco::File f(it->second.path); - DirectoryWatcher::DirectoryEvent ev(f, DirectoryWatcher::DW_ITEM_REMOVED); - owner().itemRemoved(&owner(), ev); - } - } - } - -private: - DirectoryWatcherStrategy(); - DirectoryWatcherStrategy(const DirectoryWatcherStrategy&); - DirectoryWatcherStrategy& operator = (const DirectoryWatcherStrategy&); - - DirectoryWatcher& _owner; -}; - - -#if POCO_OS == POCO_OS_WINDOWS_NT - - -class WindowsDirectoryWatcherStrategy: public DirectoryWatcherStrategy -{ -public: - WindowsDirectoryWatcherStrategy(DirectoryWatcher& owner): - DirectoryWatcherStrategy(owner) - { - _hStopped = CreateEventW(NULL, FALSE, FALSE, NULL); - if (!_hStopped) - throw SystemException("cannot create event"); - } - - ~WindowsDirectoryWatcherStrategy() - { - CloseHandle(_hStopped); - } - - void run() - { - ItemInfoMap entries; - scan(entries); - - DWORD filter = FILE_NOTIFY_CHANGE_FILE_NAME | FILE_NOTIFY_CHANGE_DIR_NAME; - if (owner().eventMask() & DirectoryWatcher::DW_ITEM_MODIFIED) - filter |= FILE_NOTIFY_CHANGE_SIZE | FILE_NOTIFY_CHANGE_LAST_WRITE; - - std::string path(owner().directory().path()); - HANDLE hChange = FindFirstChangeNotificationA(path.c_str(), FALSE, filter); - - if (hChange == INVALID_HANDLE_VALUE) - { - try - { - FileImpl::handleLastErrorImpl(path); - } - catch (Poco::Exception& exc) - { - owner().scanError(&owner(), exc); - } - return; - } - - bool stopped = false; - while (!stopped) - { - try - { - HANDLE h[2]; - h[0] = _hStopped; - h[1] = hChange; - switch (WaitForMultipleObjects(2, h, FALSE, INFINITE)) - { - case WAIT_OBJECT_0: - stopped = true; - break; - case WAIT_OBJECT_0 + 1: - { - ItemInfoMap newEntries; - scan(newEntries); - compare(entries, newEntries); - std::swap(entries, newEntries); - if (FindNextChangeNotification(hChange) == FALSE) - { - FileImpl::handleLastErrorImpl(path); - } - } - break; - default: - throw SystemException("failed to wait for directory changes"); - } - } - catch (Poco::Exception& exc) - { - owner().scanError(&owner(), exc); - } - } - FindCloseChangeNotification(hChange); - } - - void stop() - { - SetEvent(_hStopped); - } - - bool supportsMoveEvents() const - { - return false; - } - -private: - HANDLE _hStopped; -}; - - -#elif POCO_OS == POCO_OS_LINUX || POCO_OS == POCO_OS_ANDROID - - -class LinuxDirectoryWatcherStrategy: public DirectoryWatcherStrategy -{ -public: - LinuxDirectoryWatcherStrategy(DirectoryWatcher& owner): - DirectoryWatcherStrategy(owner), - _fd(-1), - _stopped(false) - { - _fd = inotify_init(); - if (_fd == -1) throw Poco::IOException("cannot initialize inotify", errno); - } - - ~LinuxDirectoryWatcherStrategy() - { - close(_fd); - } - - void run() - { - int mask = 0; - if (owner().eventMask() & DirectoryWatcher::DW_ITEM_ADDED) - mask |= IN_CREATE; - if (owner().eventMask() & DirectoryWatcher::DW_ITEM_REMOVED) - mask |= IN_DELETE; - if (owner().eventMask() & DirectoryWatcher::DW_ITEM_MODIFIED) - mask |= IN_MODIFY; - if (owner().eventMask() & DirectoryWatcher::DW_ITEM_MOVED_FROM) - mask |= IN_MOVED_FROM; - if (owner().eventMask() & DirectoryWatcher::DW_ITEM_MOVED_TO) - mask |= IN_MOVED_TO; - int wd = inotify_add_watch(_fd, owner().directory().path().c_str(), mask); - if (wd == -1) - { - try - { - FileImpl::handleLastErrorImpl(owner().directory().path()); - } - catch (Poco::Exception& exc) - { - owner().scanError(&owner(), exc); - } - } - - Poco::Buffer buffer(4096); - while (!_stopped.load(std::memory_order_relaxed)) - { - fd_set fds; - FD_ZERO(&fds); - FD_SET(_fd, &fds); - - struct timeval tv; - tv.tv_sec = 0; - tv.tv_usec = 200000; - - if (select(_fd + 1, &fds, NULL, NULL, &tv) == 1) - { - int n = read(_fd, buffer.begin(), buffer.size()); - int i = 0; - if (n > 0) - { - while (n > 0) - { - struct inotify_event* event = reinterpret_cast(buffer.begin() + i); - - if (event->len > 0) - { - if (!owner().eventsSuspended()) - { - Poco::Path p(owner().directory().path()); - p.makeDirectory(); - p.setFileName(event->name); - Poco::File f(p.toString()); - - if ((event->mask & IN_CREATE) && (owner().eventMask() & DirectoryWatcher::DW_ITEM_ADDED)) - { - DirectoryWatcher::DirectoryEvent ev(f, DirectoryWatcher::DW_ITEM_ADDED); - owner().itemAdded(&owner(), ev); - } - if ((event->mask & IN_DELETE) && (owner().eventMask() & DirectoryWatcher::DW_ITEM_REMOVED)) - { - DirectoryWatcher::DirectoryEvent ev(f, DirectoryWatcher::DW_ITEM_REMOVED); - owner().itemRemoved(&owner(), ev); - } - if ((event->mask & IN_MODIFY) && (owner().eventMask() & DirectoryWatcher::DW_ITEM_MODIFIED)) - { - DirectoryWatcher::DirectoryEvent ev(f, DirectoryWatcher::DW_ITEM_MODIFIED); - owner().itemModified(&owner(), ev); - } - if ((event->mask & IN_MOVED_FROM) && (owner().eventMask() & DirectoryWatcher::DW_ITEM_MOVED_FROM)) - { - DirectoryWatcher::DirectoryEvent ev(f, DirectoryWatcher::DW_ITEM_MOVED_FROM); - owner().itemMovedFrom(&owner(), ev); - } - if ((event->mask & IN_MOVED_TO) && (owner().eventMask() & DirectoryWatcher::DW_ITEM_MOVED_TO)) - { - DirectoryWatcher::DirectoryEvent ev(f, DirectoryWatcher::DW_ITEM_MOVED_TO); - owner().itemMovedTo(&owner(), ev); - } - } - } - - i += sizeof(inotify_event) + event->len; - n -= sizeof(inotify_event) + event->len; - } - } - } - } - } - - void stop() - { - _stopped.store(true, std::memory_order_relaxed); - } - - bool supportsMoveEvents() const - { - return true; - } - -private: - int _fd; - std::atomic _stopped; -}; - - -#elif POCO_OS == POCO_OS_MAC_OS_X || POCO_OS == POCO_OS_FREE_BSD - - -class BSDDirectoryWatcherStrategy: public DirectoryWatcherStrategy -{ -public: - BSDDirectoryWatcherStrategy(DirectoryWatcher& owner): - DirectoryWatcherStrategy(owner), - _queueFD(-1), - _dirFD(-1), - _stopped(false) - { - _dirFD = open(owner.directory().path().c_str(), O_EVTONLY); - if (_dirFD < 0) throw Poco::FileNotFoundException(owner.directory().path()); - _queueFD = kqueue(); - if (_queueFD < 0) - { - close(_dirFD); - throw Poco::SystemException("Cannot create kqueue", errno); - } - } - - ~BSDDirectoryWatcherStrategy() - { - close(_dirFD); - close(_queueFD); - } - - void run() - { - Poco::Timestamp lastScan; - ItemInfoMap entries; - scan(entries); - - while (!_stopped.load(std::memory_order_relaxed)) - { - struct timespec timeout; - timeout.tv_sec = 0; - timeout.tv_nsec = 200000000; - unsigned eventFilter = NOTE_WRITE; - struct kevent event; - struct kevent eventData; - EV_SET(&event, _dirFD, EVFILT_VNODE, EV_ADD | EV_CLEAR, eventFilter, 0, 0); - int nEvents = kevent(_queueFD, &event, 1, &eventData, 1, &timeout); - if (nEvents < 0 || eventData.flags == EV_ERROR) - { - try - { - FileImpl::handleLastErrorImpl(owner().directory().path()); - } - catch (Poco::Exception& exc) - { - owner().scanError(&owner(), exc); - } - } - else if (nEvents > 0 || ((owner().eventMask() & DirectoryWatcher::DW_ITEM_MODIFIED) && lastScan.isElapsed(owner().scanInterval()*1000000))) - { - ItemInfoMap newEntries; - scan(newEntries); - compare(entries, newEntries); - std::swap(entries, newEntries); - lastScan.update(); - } - } - } - - void stop() - { - _stopped.store(true, std::memory_order_relaxed); - } - - bool supportsMoveEvents() const - { - return false; - } - -private: - int _queueFD; - int _dirFD; - std::atomic _stopped; -}; - - -#else - - -class PollingDirectoryWatcherStrategy: public DirectoryWatcherStrategy -{ -public: - PollingDirectoryWatcherStrategy(DirectoryWatcher& owner): - DirectoryWatcherStrategy(owner) - { - } - - ~PollingDirectoryWatcherStrategy() - { - } - - void run() - { - ItemInfoMap entries; - scan(entries); - while (!_stopped.tryWait(1000*owner().scanInterval())) - { - try - { - ItemInfoMap newEntries; - scan(newEntries); - compare(entries, newEntries); - std::swap(entries, newEntries); - } - catch (Poco::Exception& exc) - { - owner().scanError(&owner(), exc); - } - } - } - - void stop() - { - _stopped.set(); - } - - bool supportsMoveEvents() const - { - return false; - } - -private: - Poco::Event _stopped; -}; - - -#endif - - -DirectoryWatcher::DirectoryWatcher(const std::string& path, int eventMask, int scanInterval): - _directory(path), - _eventMask(eventMask), - _scanInterval(scanInterval) -{ - init(); -} - - -DirectoryWatcher::DirectoryWatcher(const Poco::File& directory, int eventMask, int scanInterval): - _directory(directory), - _eventMask(eventMask), - _scanInterval(scanInterval) -{ - init(); -} - - -DirectoryWatcher::~DirectoryWatcher() -{ - try - { - stop(); - delete _pStrategy; - } - catch (...) - { - poco_unexpected(); - } -} - - -void DirectoryWatcher::suspendEvents() -{ - poco_assert (_eventsSuspended > 0); - - _eventsSuspended--; -} - - -void DirectoryWatcher::resumeEvents() -{ - _eventsSuspended++; -} - - -void DirectoryWatcher::init() -{ - if (!_directory.exists()) - throw Poco::FileNotFoundException(_directory.path()); - - if (!_directory.isDirectory()) - throw Poco::InvalidArgumentException("not a directory", _directory.path()); - -#if POCO_OS == POCO_OS_WINDOWS_NT - _pStrategy = new WindowsDirectoryWatcherStrategy(*this); -#elif POCO_OS == POCO_OS_LINUX || POCO_OS == POCO_OS_ANDROID - _pStrategy = new LinuxDirectoryWatcherStrategy(*this); -#elif POCO_OS == POCO_OS_MAC_OS_X || POCO_OS == POCO_OS_FREE_BSD - _pStrategy = new BSDDirectoryWatcherStrategy(*this); -#else - _pStrategy = new PollingDirectoryWatcherStrategy(*this); -#endif - _thread.start(*this); -} - - -void DirectoryWatcher::run() -{ - _pStrategy->run(); -} - - -void DirectoryWatcher::stop() -{ - _pStrategy->stop(); - _thread.join(); -} - - -bool DirectoryWatcher::supportsMoveEvents() const -{ - return _pStrategy->supportsMoveEvents(); -} - - -} // namespace Poco - - -#endif // POCO_NO_INOTIFY diff --git a/src/Client/LineReader.cpp b/src/Client/LineReader.cpp index 2ec90240fd1..dde6a7ba3a7 100644 --- a/src/Client/LineReader.cpp +++ b/src/Client/LineReader.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include @@ -27,11 +27,8 @@ void trim(String & s) /// Allows delaying the start of query execution until the entirety of query is inserted. bool hasInputData() { - timeval timeout = {0, 0}; - fd_set fds{}; - FD_ZERO(&fds); - FD_SET(STDIN_FILENO, &fds); - return select(1, &fds, nullptr, nullptr, &timeout) == 1; + pollfd fd{STDIN_FILENO, POLLIN, 0}; + return poll(&fd, 1, 0) == 1; } struct NoCaseCompare diff --git a/src/IO/ReadBufferFromFileDescriptor.cpp b/src/IO/ReadBufferFromFileDescriptor.cpp index 6c0c1681a4c..6c4c028f642 100644 --- a/src/IO/ReadBufferFromFileDescriptor.cpp +++ b/src/IO/ReadBufferFromFileDescriptor.cpp @@ -249,24 +249,6 @@ void ReadBufferFromFileDescriptor::rewind() file_offset_of_buffer_end = 0; } - -/// Assuming file descriptor supports 'select', check that we have data to read or wait until timeout. -bool ReadBufferFromFileDescriptor::poll(size_t timeout_microseconds) const -{ - fd_set fds; - FD_ZERO(&fds); - FD_SET(fd, &fds); - timeval timeout = { time_t(timeout_microseconds / 1000000), suseconds_t(timeout_microseconds % 1000000) }; - - int res = select(1, &fds, nullptr, nullptr, &timeout); - - if (-1 == res) - throwFromErrno("Cannot select", ErrorCodes::CANNOT_SELECT); - - return res > 0; -} - - size_t ReadBufferFromFileDescriptor::getFileSize() { return getSizeFromFileDescriptor(fd, getFileName()); diff --git a/src/IO/ReadBufferFromFileDescriptor.h b/src/IO/ReadBufferFromFileDescriptor.h index 64340770cf2..4762998c67b 100644 --- a/src/IO/ReadBufferFromFileDescriptor.h +++ b/src/IO/ReadBufferFromFileDescriptor.h @@ -75,10 +75,6 @@ public: size_t readBigAt(char * to, size_t n, size_t offset, const std::function &) override; bool supportsReadAt() override { return use_pread; } - -private: - /// Assuming file descriptor supports 'select', check that we have data to read or wait until timeout. - bool poll(size_t timeout_microseconds) const; }; From 434c2113d207b4507cac7f7cac88dbf553cc3474 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 4 Dec 2023 10:51:17 +0000 Subject: [PATCH 041/110] Fix style --- src/IO/ReadBufferFromFileDescriptor.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/IO/ReadBufferFromFileDescriptor.cpp b/src/IO/ReadBufferFromFileDescriptor.cpp index 6c4c028f642..5a67dc7528c 100644 --- a/src/IO/ReadBufferFromFileDescriptor.cpp +++ b/src/IO/ReadBufferFromFileDescriptor.cpp @@ -39,7 +39,6 @@ namespace ErrorCodes extern const int CANNOT_READ_FROM_FILE_DESCRIPTOR; extern const int ARGUMENT_OUT_OF_BOUND; extern const int CANNOT_SEEK_THROUGH_FILE; - extern const int CANNOT_SELECT; extern const int CANNOT_ADVISE; } From 638d0102f8a51d34bb86f5f6676447655c10317a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 4 Dec 2023 16:01:32 +0100 Subject: [PATCH 042/110] Fix error_count in case of distributed_directory_monitor_max_sleep_time_ms>5min In this case the error counter will be decremented everytime. Signed-off-by: Azat Khuzhin --- .../DistributedAsyncInsertDirectoryQueue.cpp | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index 8479412ad64..16cf6ee2003 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -218,6 +218,15 @@ void DistributedAsyncInsertDirectoryQueue::run() /// No errors while processing existing files. /// Let's see maybe there are more files to process. do_sleep = false; + + const auto now = std::chrono::system_clock::now(); + if (now - last_decrease_time > decrease_error_count_period) + { + std::lock_guard status_lock(status_mutex); + + status.error_count /= 2; + last_decrease_time = now; + } } catch (...) { @@ -236,15 +245,6 @@ void DistributedAsyncInsertDirectoryQueue::run() else LOG_TEST(LogFrequencyLimiter(log, 30), "Skipping send data over distributed table."); - const auto now = std::chrono::system_clock::now(); - if (now - last_decrease_time > decrease_error_count_period) - { - std::lock_guard status_lock(status_mutex); - - status.error_count /= 2; - last_decrease_time = now; - } - if (do_sleep) break; } From 604cec475a4277f16d2326e9ec394f84d579ca04 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 4 Dec 2023 16:10:05 +0100 Subject: [PATCH 043/110] Fix possible distributed sends stuck due to "No such file or directory" In case of restoring from current_batch.txt it is possible that the some file from the batch will not be exist, and the fix submitted in #49884 was not complete, since it will fail later in markAsSend() (due to it tries to obtain file size there): 2023.12.04 05:43:12.676658 [ 5006 ] {} dist.DirectoryMonitor.work4: std::exception. Code: 1001, type: std::__1::__fs::filesystem::filesystem_error, e.what() = filesystem error: in file_size: No such file or directory ["/work4/clickhouse/data/dist/shard8_all_replicas//150426396.bin"], Stack trace (when copying this message, always include the lines below): 0. ./.build/./contrib/llvm-project/libcxx/include/exception:134: std::runtime_error::runtime_error(String const&) @ 0x00000000177e83f4 in /usr/lib/debug/usr/bin/clickhouse.debug 1. ./.build/./contrib/llvm-project/libcxx/include/string:1499: std::system_error::system_error(std::error_code, String const&) @ 0x00000000177f0fd5 in /usr/lib/debug/usr/bin/clickhouse.debug 2. ./.build/./contrib/llvm-project/libcxx/include/__filesystem/filesystem_error.h:42: std::__fs::filesystem::filesystem_error::filesystem_error[abi:v15000](String const&, std::__fs::filesystem::path const&, std::error_code) @ 0x000000000b844ca1 in /usr/lib/debug/usr/bin/clickhouse.debug 3. ./.build/./contrib/llvm-project/libcxx/include/__filesystem/filesystem_error.h:90: void std::__fs::filesystem::__throw_filesystem_error[abi:v15000](String&, std::__fs::filesystem::path const&, std::error_code const&) @ 0x000000001778f953 in /usr/lib/debug/usr/bin/clickhouse.debug 4. ./.build/./contrib/llvm-project/libcxx/src/filesystem/filesystem_common.h:0: std::__fs::filesystem::detail::(anonymous namespace)::ErrorHandler::report(std::error_code const&) const @ 0x0000000017793ef7 in /usr/lib/debug/usr/bin/clickhouse.debug 5. ./.build/./contrib/llvm-project/libcxx/src/filesystem/operations.cpp:0: std::__fs::filesystem::__file_size(std::__fs::filesystem::path const&, std::error_code*) @ 0x0000000017793e26 in /usr/lib/debug/usr/bin/clickhouse.debug 6. ./.build/./src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp:707: DB::DistributedAsyncInsertDirectoryQueue::markAsSend(String const&) @ 0x0000000011cd92c5 in /usr/lib/debug/usr/bin/clickhouse.debug 7. ./.build/./contrib/llvm-project/libcxx/include/__iterator/wrap_iter.h:100: DB::DistributedAsyncInsertBatch::send() @ 0x0000000011cdd81c in /usr/lib/debug/usr/bin/clickhouse.debug 8. ./.build/./src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp:0: DB::DistributedAsyncInsertDirectoryQueue::processFilesWithBatching() @ 0x0000000011cd5054 in /usr/lib/debug/usr/bin/clickhouse.debug 9. ./.build/./src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp:417: DB::DistributedAsyncInsertDirectoryQueue::processFiles() @ 0x0000000011cd3440 in /usr/lib/debug/usr/bin/clickhouse.debug 10. ./.build/./src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp:0: DB::DistributedAsyncInsertDirectoryQueue::run() @ 0x0000000011cd3878 in /usr/lib/debug/usr/bin/clickhouse.debug 11. ./.build/./contrib/llvm-project/libcxx/include/__functional/function.h:0: DB::BackgroundSchedulePoolTaskInfo::execute() @ 0x00000000103dbc34 in /usr/lib/debug/usr/bin/clickhouse.debug 12. ./.build/./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:701: DB::BackgroundSchedulePool::threadFunction() @ 0x00000000103de1b6 in /usr/lib/debug/usr/bin/clickhouse.debug 13. ./.build/./src/Core/BackgroundSchedulePool.cpp:0: void std::__function::__policy_invoker::__call_impl::ThreadFromGlobalPoolImpl, StrongTypedef, char const*)::$_0>(DB::BackgroundSchedulePool::BackgroundSchedulePool(unsigned long, StrongTypedef, StrongTypedef, char const*)::$_0&&)::'lambda'(), void ()>>(std::__function::__policy_storage const*) @ 0x00000000103de7d1 in /usr/lib/debug/usr/bin/clickhouse.debug 14. ./.build/./base/base/../base/wide_integer_impl.h:809: ThreadPoolImpl::worker(std::__list_iterator) @ 0x000000000b8c5502 in /usr/lib/debug/usr/bin/clickhouse.debug 15. ./.build/./contrib/llvm-project/libcxx/include/__memory/unique_ptr.h:302: void* std::__thread_proxy[abi:v15000]>, void ThreadPoolImpl::scheduleImpl(std::function, Priority, std::optional, bool)::'lambda0'()>>(void*) @ 0x000000000b8c936e in /usr/lib/debug/usr/bin/clickhouse.debug 16. ? @ 0x00007f1be8b30fd4 in ? 17. ? @ 0x00007f1be8bb15bc in ? And instead of ignoring errors, DistributedAsyncInsertBatch::valid() had been added, that should be called when the files had been read from the current_batch.txt, if it is not valid (some files from the batch did not exist), then there is no sense in trying to send the same batch, so just this file will be ignored, and files will be processed in a regular order. Signed-off-by: Azat Khuzhin --- .../DistributedAsyncInsertBatch.cpp | 25 +++++++++++++------ .../Distributed/DistributedAsyncInsertBatch.h | 7 ++++++ .../DistributedAsyncInsertDirectoryQueue.cpp | 11 +++++++- 3 files changed, 34 insertions(+), 9 deletions(-) diff --git a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp index b82cf1d7ffc..97268cf1389 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -163,6 +164,22 @@ void DistributedAsyncInsertBatch::deserialize() readText(in); } +bool DistributedAsyncInsertBatch::valid() +{ + chassert(!files.empty()); + + bool res = true; + for (const auto & file : files) + { + if (!fs::exists(file)) + { + LOG_WARNING(parent.log, "File {} does not exists, likely due abnormal shutdown", file); + res = false; + } + } + return res; +} + void DistributedAsyncInsertBatch::writeText(WriteBuffer & out) { for (const auto & file : files) @@ -201,14 +218,6 @@ void DistributedAsyncInsertBatch::sendBatch() { for (const auto & file : files) { - /// In case of recovery it is possible that some of files will be - /// missing, if server had been restarted abnormally - if (recovered && !fs::exists(file)) - { - LOG_WARNING(parent.log, "File {} does not exists, likely due abnormal shutdown", file); - continue; - } - ReadBufferFromFile in(file); const auto & distributed_header = DistributedAsyncInsertHeader::read(in, parent.log); diff --git a/src/Storages/Distributed/DistributedAsyncInsertBatch.h b/src/Storages/Distributed/DistributedAsyncInsertBatch.h index 867a0de89fa..db96634d6f1 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertBatch.h +++ b/src/Storages/Distributed/DistributedAsyncInsertBatch.h @@ -18,9 +18,16 @@ public: bool isEnoughSize() const; void send(); + /// Write batch to current_batch.txt void serialize(); + + /// Read batch from current_batch.txt void deserialize(); + /// Does all required files exists? + /// (The only way variant when it is valid is during restoring batch from disk). + bool valid(); + size_t total_rows = 0; size_t total_bytes = 0; std::vector files; diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index 16cf6ee2003..9412becf484 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -559,7 +559,16 @@ void DistributedAsyncInsertDirectoryQueue::processFilesWithBatching() DistributedAsyncInsertBatch batch(*this); batch.deserialize(); - batch.send(); + + /// In case of recovery it is possible that some of files will be + /// missing, if server had been restarted abnormally + /// (between unlink(*.bin) and unlink(current_batch.txt)). + /// + /// But current_batch_file_path should be removed anyway, since if some + /// file was missing, then the batch is not complete and there is no + /// point in trying to pretend that it will not break deduplication. + if (batch.valid()) + batch.send(); auto dir_sync_guard = getDirectorySyncGuard(relative_path); fs::remove(current_batch_file_path); From 7986fe619a63eeb58656e3605f25cae3ab5f8789 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 4 Dec 2023 16:45:32 +0100 Subject: [PATCH 044/110] Introduce DistributedAsyncInsertionFailures - event for async INSERT failures Useful for alerts Signed-off-by: Azat Khuzhin --- src/Common/ProfileEvents.cpp | 1 + .../DistributedAsyncInsertDirectoryQueue.cpp | 10 +++++++++- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 92b39163878..1831d09cc30 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -238,6 +238,7 @@ M(DictCacheLockReadNs, "Number of nanoseconds spend in waiting for read lock to lookup the data for the dictionaries of 'cache' types.") \ \ M(DistributedSyncInsertionTimeoutExceeded, "A timeout has exceeded while waiting for shards during synchronous insertion into a Distributed table (with 'distributed_foreground_insert' = 1)") \ + M(DistributedAsyncInsertionFailures, "Number of failures for asynchronous insertion into a Distributed table (with 'distributed_foreground_insert' = 0)") \ M(DataAfterMergeDiffersFromReplica, R"( Number of times data after merge is not byte-identical to the data on another replicas. There could be several reasons: 1. Using newer version of compression library after server update. diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index 9412becf484..84f3b6767c8 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -17,13 +17,14 @@ #include #include #include -#include +#include #include #include #include #include #include #include +#include #include #include #include @@ -39,6 +40,11 @@ namespace CurrentMetrics extern const Metric BrokenDistributedBytesToInsert; } +namespace ProfileEvents +{ + extern const Event DistributedAsyncInsertionFailures; +} + namespace fs = std::filesystem; namespace DB @@ -419,6 +425,8 @@ try } catch (...) { + ProfileEvents::increment(ProfileEvents::DistributedAsyncInsertionFailures); + std::lock_guard status_lock(status_mutex); ++status.error_count; From 70f436483c295eea0ab4ebed481da8fd80fb50d7 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 5 Dec 2023 00:40:15 +0100 Subject: [PATCH 045/110] address part of comments --- src/Coordination/KeeperDispatcher.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 544f4e033bb..1299e9c9f20 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -56,7 +56,9 @@ namespace bool checkIfRequestIncreaseMem(const Coordination::ZooKeeperRequestPtr & request) { - if (request->getOpNum() == Coordination::OpNum::Create || request->getOpNum() == Coordination::OpNum::CreateIfNotExists) + if (request->getOpNum() == Coordination::OpNum::Create + || request->getOpNum() == Coordination::OpNum::CreateIfNotExists + || request->getOpNum() == Coordination::OpNum::Set) { return true; } @@ -76,6 +78,12 @@ bool checkIfRequestIncreaseMem(const Coordination::ZooKeeperRequestPtr & request memory_delta += create_req.bytesSize(); break; } + case Coordination::OpNum::Set: + { + Coordination::ZooKeeperSetRequest & set_req = dynamic_cast(*sub_zk_request); + memory_delta += set_req.bytesSize(); + break; + } case Coordination::OpNum::Remove: { Coordination::ZooKeeperRemoveRequest & remove_req = dynamic_cast(*sub_zk_request); From 60ba1abc176169c67a7709ae9c688bd49cf79d1c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 5 Dec 2023 12:58:29 +0000 Subject: [PATCH 046/110] Fix build --- src/Client/LineReader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/LineReader.cpp b/src/Client/LineReader.cpp index dde6a7ba3a7..b3559657ced 100644 --- a/src/Client/LineReader.cpp +++ b/src/Client/LineReader.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include From ed618d3be1e7b171a1e19b79fd493460418b9954 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 6 Dec 2023 04:24:43 +0000 Subject: [PATCH 047/110] Fix crash in clickhouse-local --- src/Interpreters/Context.cpp | 2 +- src/Interpreters/Context.h | 2 +- src/Storages/System/StorageSystemServerSettings.cpp | 13 ++++++++----- .../02933_local_system_setting.reference | 0 .../0_stateless/02933_local_system_setting.sh | 7 +++++++ 5 files changed, 17 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/02933_local_system_setting.reference create mode 100755 tests/queries/0_stateless/02933_local_system_setting.sh diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 0a8a8f1f529..a4dff302420 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4830,7 +4830,7 @@ void Context::initializeBackgroundExecutorsIfNeeded() shared->are_background_executors_initialized = true; } -bool Context::areBackgroundExecutorsInitialized() +bool Context::areBackgroundExecutorsInitialized() const { SharedLockGuard lock(shared->background_executors_mutex); return shared->are_background_executors_initialized; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 8c169dd664f..b6bec5ba5d0 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1208,7 +1208,7 @@ public: /// Background executors related methods void initializeBackgroundExecutorsIfNeeded(); - bool areBackgroundExecutorsInitialized(); + bool areBackgroundExecutorsInitialized() const; MergeMutateBackgroundExecutorPtr getMergeMutateExecutor() const; OrdinaryBackgroundExecutorPtr getMovesExecutor() const; diff --git a/src/Storages/System/StorageSystemServerSettings.cpp b/src/Storages/System/StorageSystemServerSettings.cpp index 3a3acabc5a3..439e3972bc7 100644 --- a/src/Storages/System/StorageSystemServerSettings.cpp +++ b/src/Storages/System/StorageSystemServerSettings.cpp @@ -46,17 +46,20 @@ void StorageSystemServerSettings::fillData(MutableColumns & res_columns, Context {"max_concurrent_insert_queries", std::to_string(context->getProcessList().getMaxInsertQueriesAmount())}, {"max_concurrent_select_queries", std::to_string(context->getProcessList().getMaxSelectQueriesAmount())}, - {"background_pool_size", std::to_string(context->getMergeMutateExecutor()->getMaxThreads())}, - {"background_move_pool_size", std::to_string(context->getMovesExecutor()->getMaxThreads())}, - {"background_fetches_pool_size", std::to_string(context->getFetchesExecutor()->getMaxThreads())}, - {"background_common_pool_size", std::to_string(context->getCommonExecutor()->getMaxThreads())}, - {"background_buffer_flush_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundBufferFlushSchedulePoolSize))}, {"background_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundSchedulePoolSize))}, {"background_message_broker_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundMessageBrokerSchedulePoolSize))}, {"background_distributed_schedule_pool_size", std::to_string(CurrentMetrics::get(CurrentMetrics::BackgroundDistributedSchedulePoolSize))} }; + if (context->areBackgroundExecutorsInitialized()) + { + updated.insert({"background_pool_size", std::to_string(context->getMergeMutateExecutor()->getMaxThreads())}); + updated.insert({"background_move_pool_size", std::to_string(context->getMovesExecutor()->getMaxThreads())}); + updated.insert({"background_fetches_pool_size", std::to_string(context->getFetchesExecutor()->getMaxThreads())}); + updated.insert({"background_common_pool_size", std::to_string(context->getCommonExecutor()->getMaxThreads())}); + } + const auto & config = context->getConfigRef(); ServerSettings settings; settings.loadSettingsFromConfig(config); diff --git a/tests/queries/0_stateless/02933_local_system_setting.reference b/tests/queries/0_stateless/02933_local_system_setting.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02933_local_system_setting.sh b/tests/queries/0_stateless/02933_local_system_setting.sh new file mode 100755 index 00000000000..c6d19f2445f --- /dev/null +++ b/tests/queries/0_stateless/02933_local_system_setting.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "select * from system.server_settings format Null;" From 6321576ff8dc32cd97508f989e169f65527a3d13 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 6 Dec 2023 20:23:09 +0100 Subject: [PATCH 048/110] Add a test --- tests/integration/helpers/postgres_utility.py | 14 ++++----- .../test.py | 29 +++++++++++++++++++ 2 files changed, 36 insertions(+), 7 deletions(-) diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index 9b431317b4a..8ad8d0a6851 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -189,11 +189,11 @@ class PostgresManager: database_name = self.database_or_default(database_name) self.drop_postgres_db(database_name) self.created_postgres_db_list.add(database_name) - self.cursor.execute(f"CREATE DATABASE {database_name}") + self.cursor.execute(f"CREATE DATABASE \"{database_name}\"") def drop_postgres_db(self, database_name=""): database_name = self.database_or_default(database_name) - self.cursor.execute(f"DROP DATABASE IF EXISTS {database_name} WITH (FORCE)") + self.cursor.execute(f"DROP DATABASE IF EXISTS \"{database_name}\" WITH (FORCE)") if database_name in self.created_postgres_db_list: self.created_postgres_db_list.remove(database_name) @@ -212,19 +212,19 @@ class PostgresManager: if len(schema_name) == 0: self.instance.query( f""" - CREATE DATABASE {database_name} + CREATE DATABASE \"{database_name}\" ENGINE = PostgreSQL('{self.ip}:{self.port}', '{postgres_database}', 'postgres', 'mysecretpassword')""" ) else: self.instance.query( f""" - CREATE DATABASE {database_name} + CREATE DATABASE \"{database_name}\" ENGINE = PostgreSQL('{self.ip}:{self.port}', '{postgres_database}', 'postgres', 'mysecretpassword', '{schema_name}')""" ) def drop_clickhouse_postgres_db(self, database_name=""): database_name = self.database_or_default(database_name) - self.instance.query(f"DROP DATABASE IF EXISTS {database_name}") + self.instance.query(f"DROP DATABASE IF EXISTS \"{database_name}\"") if database_name in self.created_ch_postgres_db_list: self.created_ch_postgres_db_list.remove(database_name) @@ -362,7 +362,7 @@ def check_tables_are_synchronized( result_query = f"select * from {table_path} order by {order_by};" expected = instance.query( - f"select * from {postgres_database}.{table_name} order by {order_by};" + f"select * from `{postgres_database}`.`{table_name}` order by {order_by};" ) result = instance.query(result_query) @@ -376,7 +376,7 @@ def check_tables_are_synchronized( if result != expected: count = int(instance.query(f"select count() from {table_path}")) expected_count = int( - instance.query(f"select count() from {postgres_database}.{table_name}") + instance.query(f"select count() from `{postgres_database}`.`{table_name}`") ) print(f"Having {count}, expected {expected_count}") assert result == expected diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index e8053730c44..f94e6d481e3 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -58,6 +58,7 @@ instance2 = cluster.add_instance( pg_manager = PostgresManager() pg_manager2 = PostgresManager() pg_manager_instance2 = PostgresManager() +pg_manager3 = PostgresManager() @pytest.fixture(scope="module") @@ -80,6 +81,12 @@ def started_cluster(): pg_manager2.init( instance2, cluster.postgres_ip, cluster.postgres_port, "postgres_database2" ) + pg_manager3.init( + instance, + cluster.postgres_ip, + cluster.postgres_port, + default_database="postgres-postgres", + ) yield cluster @@ -810,6 +817,28 @@ def test_replica_consumer(started_cluster): pg_manager_instance2.clear() +def test_symbols_in_publication_name(started_cluster): + table = "test_symbols_in_publication_name" + + pg_manager3.create_postgres_table(table) + instance.query( + f"INSERT INTO `{pg_manager3.get_default_database()}`.`{table}` SELECT number, number from numbers(0, 50)" + ) + + pg_manager3.create_materialized_db( + ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + settings=[ + f"materialized_postgresql_tables_list = '{table}'", + "materialized_postgresql_backoff_min_ms = 100", + "materialized_postgresql_backoff_max_ms = 100", + ], + ) + check_tables_are_synchronized( + instance, table, postgres_database=pg_manager3.get_default_database() + ) + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From 4dab2515dce00603bfd0b478dfb2686a950d3b5e Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 6 Dec 2023 19:46:41 +0000 Subject: [PATCH 049/110] Automatic style fix --- tests/integration/helpers/postgres_utility.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index 8ad8d0a6851..f67d57a8fbb 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -189,11 +189,11 @@ class PostgresManager: database_name = self.database_or_default(database_name) self.drop_postgres_db(database_name) self.created_postgres_db_list.add(database_name) - self.cursor.execute(f"CREATE DATABASE \"{database_name}\"") + self.cursor.execute(f'CREATE DATABASE "{database_name}"') def drop_postgres_db(self, database_name=""): database_name = self.database_or_default(database_name) - self.cursor.execute(f"DROP DATABASE IF EXISTS \"{database_name}\" WITH (FORCE)") + self.cursor.execute(f'DROP DATABASE IF EXISTS "{database_name}" WITH (FORCE)') if database_name in self.created_postgres_db_list: self.created_postgres_db_list.remove(database_name) @@ -224,7 +224,7 @@ class PostgresManager: def drop_clickhouse_postgres_db(self, database_name=""): database_name = self.database_or_default(database_name) - self.instance.query(f"DROP DATABASE IF EXISTS \"{database_name}\"") + self.instance.query(f'DROP DATABASE IF EXISTS "{database_name}"') if database_name in self.created_ch_postgres_db_list: self.created_ch_postgres_db_list.remove(database_name) From 71277892cf453b1fb37093182d7c91254a21596b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 7 Dec 2023 15:55:23 +0000 Subject: [PATCH 050/110] init --- src/Functions/date_trunc.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/date_trunc.cpp b/src/Functions/date_trunc.cpp index 414512fc4f8..c3903fef137 100644 --- a/src/Functions/date_trunc.cpp +++ b/src/Functions/date_trunc.cpp @@ -47,7 +47,7 @@ public: throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be constant string: " "name of datepart", getName()); - datepart_param = datepart_column->getValue(); + datepart_param = Poco::toLower(datepart_column->getValue()); if (datepart_param.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "First argument (name of datepart) for function {} cannot be empty", getName()); From 27bcd14181610ef882c0c11514b4a08779a1083a Mon Sep 17 00:00:00 2001 From: Bharat Nallan Chakravarthy Date: Thu, 7 Dec 2023 08:43:00 -0800 Subject: [PATCH 051/110] add SHA512_256 function --- src/Functions/FunctionsHashing.h | 15 +++++++++++++++ src/Functions/FunctionsHashingSSL.cpp | 1 + 2 files changed, 16 insertions(+) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 345b9a11e0d..3c43cab81e8 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -318,6 +318,20 @@ struct SHA512Impl SHA512_Final(out_char_data, &ctx); } }; + +struct SHA512Impl256 +{ + static constexpr auto name = "SHA512_256"; + enum { length = SHA512_256_DIGEST_LENGTH }; + + static void apply(const char * begin, const size_t size, unsigned char * out_char_data) + { + SHA512_CTX ctx; + SHA512_256_Init(&ctx); + SHA512_256_Update(&ctx, reinterpret_cast(begin), size); + SHA512_256_Final(out_char_data, &ctx); + } +}; #endif struct SipHash64Impl @@ -1801,6 +1815,7 @@ using FunctionSHA224 = FunctionStringHashFixedString; using FunctionSHA256 = FunctionStringHashFixedString; using FunctionSHA384 = FunctionStringHashFixedString; using FunctionSHA512 = FunctionStringHashFixedString; +using FunctionSHA512_256 = FunctionStringHashFixedString; #endif using FunctionSipHash128 = FunctionAnyHash; using FunctionSipHash128Keyed = FunctionAnyHash; diff --git a/src/Functions/FunctionsHashingSSL.cpp b/src/Functions/FunctionsHashingSSL.cpp index 2eb0ed88f28..7fe7530d9da 100644 --- a/src/Functions/FunctionsHashingSSL.cpp +++ b/src/Functions/FunctionsHashingSSL.cpp @@ -22,6 +22,7 @@ REGISTER_FUNCTION(HashingSSL) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); } } From 6b6c72d991eb9b86579ee75700578475828a3421 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 7 Dec 2023 16:43:11 +0000 Subject: [PATCH 052/110] add tests --- .../02935_date_trunc_case_unsensitiveness.reference | 9 +++++++++ .../02935_date_trunc_case_unsensitiveness.sql | 9 +++++++++ 2 files changed, 18 insertions(+) create mode 100644 tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.reference create mode 100644 tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql diff --git a/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.reference b/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.reference new file mode 100644 index 00000000000..c8e3d37f08b --- /dev/null +++ b/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.reference @@ -0,0 +1,9 @@ +2022-03-01 00:00:00 +2022-03-01 +2022-02-28 +2022-03-01 00:00:00 +2022-03-01 +2022-02-28 +2022-03-01 00:00:00 +2022-03-01 +2022-02-28 diff --git a/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql b/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql new file mode 100644 index 00000000000..ecf6877d477 --- /dev/null +++ b/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql @@ -0,0 +1,9 @@ +SELECT dateTrunc('DAY', toDateTime('2022-03-01 12:55:55')); +SELECT dateTrunc('MONTH', toDateTime64('2022-03-01 12:55:55', 2)); +SELECT dateTrunc('WEEK', toDate('2022-03-01')); +SELECT dateTrunc('Day', toDateTime('2022-03-01 12:55:55')); +SELECT dateTrunc('Month', toDateTime64('2022-03-01 12:55:55', 2)); +SELECT dateTrunc('Week', toDate('2022-03-01')); +SELECT dateTrunc('day', toDateTime('2022-03-01 12:55:55')); +SELECT dateTrunc('month', toDateTime64('2022-03-01 12:55:55', 2)); +SELECT dateTrunc('week', toDate('2022-03-01')); From 23ef9e477063d52927b40b0b8b1b0c9fac986041 Mon Sep 17 00:00:00 2001 From: Bharat Nallan Chakravarthy Date: Thu, 7 Dec 2023 08:57:57 -0800 Subject: [PATCH 053/110] add tests --- .../02012_sha512_fixedstring.reference | 22 +++++++++++++++++++ .../0_stateless/02012_sha512_fixedstring.sql | 4 ++++ ...2041_openssl_hash_functions_test.reference | 1 + .../02041_openssl_hash_functions_test.sql | 1 + ...2184_hash_functions_and_ip_types.reference | 1 + .../02184_hash_functions_and_ip_types.sql | 1 + 6 files changed, 30 insertions(+) diff --git a/tests/queries/0_stateless/02012_sha512_fixedstring.reference b/tests/queries/0_stateless/02012_sha512_fixedstring.reference index dfc2f87087c..93303a23fe1 100644 --- a/tests/queries/0_stateless/02012_sha512_fixedstring.reference +++ b/tests/queries/0_stateless/02012_sha512_fixedstring.reference @@ -1,5 +1,7 @@ CF83E1357EEFB8BDF1542850D66D8007D620E4050B5715DC83F4A921D36CE9CE47D0D13C5D85F2B0FF8318D2877EEC2F63B931BD47417A81A538327AF927DA3E DDAF35A193617ABACC417349AE20413112E6FA4E89A97EA20A9EEEE64B55D39A2192992A274FC1A836BA3C23A3FEEBBD454D4423643CE80E2A9AC94FA54CA49F +C672B8D1EF56ED28AB87C3622C5114069BDD3AD7B8F9737498D0C01ECEF0967A +53048E2681941EF99B2E29B76B4C7DABE4C2D0C634FC6D46E0E2F13107E7AF23 5809F3ECB4AA006F71AF562D4381F2BF64EA0931FD530E939740D0C38F6EEB2A71FA0113A21C170569D8319B8C4DE8A1C1A5ABA1A1C5B23A886B06712D373B9E 6FDB5E5BCCBD093ECC48DD262A99E6B867D6F48E1DAE014D26428365E7529B0022F000CBF852BEA38F43A2034E8FE7555AC41B9EA9E27FE72F4E968926998EA8 8018978D8AAE19322205E4CACFA045CDF7A0C4A5773A93FD24331064AFC5726F324B76802AA6FC30DFC412A6E5C3EEF4693AE4E2D0A1EA24A2D3EC46439B7923 @@ -20,3 +22,23 @@ E7A061D9B066E2CA44CF959A76FC04D8B02998CB9D46A60C19E015EA9389F3F9595CBBC4CC46E131 3438D75650E1EDB8A11EF4F63A5DFF239A70B28B6A14F13FCFDD14D02BE8BD00E84DF956C159CFDC85D6E44DB62D00E9206F40453FFD9CC97C38449527D33FF6 DB229C3A53B0340E94EFDA4D03B54F161313699757CAC312F377B731AE6C62010E0C0010E78F73E6D6B0BB438F644D176244B2614897799F9FA3F85DA980C218 FDD9FD54050D95855B8E3A34F3A54E309E1CA87CD44A8506EB10051D1CA650DB64ABD0BE4F4F44E45F630C22CA270FA7694AC2261DF2EFD766B8CED53F285A27 +99B13DC7BAF988EC34949E59E636884214C0155DA3BE69605F9DBEE0F9BB26FD +C93FBA93074FDEF8301B66D8DB835BFA2D50616EA108EC32CF617A7A40AF9746 +03CBA6DD6F5410B8E3E18D5271CAA786B2E67D8BE8281C893886B677C6A6F28D +BB612013EC3039A5943037FE9D92D976F85E8B69A3729019D674581DF101181A +40AC3D10E1820DBB473DC04CD32CCC14294F7BE5B3298C3BCC88F7F6AD7FCEB0 +659A9B47C2EC80125CB790FFCB1E234F2D33A93F6CF86DEF501305977A03BC75 +C03E2022A819F90B11FE1686C2827F67138B4DE7677C14D08972055336958263 +801A5F66EE0CD5EB1C214A9E61C3C4558A6795262E5A70D22AA392087F5C019D +AAD6E69EA25EECBC0A26CB60F9E2EBA878637A7CAA0CDF325D29C2516CE9D1A6 +B0B7F2E5E00E1F44578540A0B34138133654CA4803C245834D8C720BC46E6ADD +6475E517346B8C888418E1610E28969D10583CF431D05916A7DEE49633FE6279 +43A8BFDFBA86848376580714584F699E9775B7B31DDB7C4BBF0530D97564F5F9 +3B10D6BC94B31128583716432192F1B60D7BC23282A099F343CD5BB79323CD48 +B23EC2FC5B9D07E1853D50372600C8AEBA77F571B364D2D7D05E7B47ABA3E679 +F0DAA6078297F475B7E3B3968A2227A645DAA2ABA7426700532DCE1A04B67DB0 +4084B1624E1D9F5F80EE65C216316269E14A42CDF8607A651DFC602319CF9C66 +B63FBA87B2DCB490EE45B39F660A9DEFBD5383774B1C81D7A8CF9DE4C3FAA190 +BC70EB3AFD6D8252C92BB655920FFDCCC8239E99476697F1CDD4F847533671AC +6388E69B35B7A80469CD4C4EB49568C3143FEE6143112009120088A8607CC676 +7CE6CDE5DDE81925E714717F01771325298381B1934B021590965B30D23EC1C9 diff --git a/tests/queries/0_stateless/02012_sha512_fixedstring.sql b/tests/queries/0_stateless/02012_sha512_fixedstring.sql index cd014a58e65..ca9520350db 100644 --- a/tests/queries/0_stateless/02012_sha512_fixedstring.sql +++ b/tests/queries/0_stateless/02012_sha512_fixedstring.sql @@ -4,6 +4,9 @@ SELECT hex(SHA512('')); SELECT hex(SHA512('abc')); +SELECT hex(SHA512_256('')); +SELECT hex(SHA512_256('abc')); + DROP TABLE IF EXISTS defaults; CREATE TABLE defaults ( @@ -13,5 +16,6 @@ CREATE TABLE defaults INSERT INTO defaults SELECT s FROM generateRandom('s FixedString(20)', 1, 1, 1) LIMIT 20; SELECT hex(SHA512(s)) FROM defaults; +SELECT hex(SHA512_256(s)) FROM defaults; DROP TABLE defaults; diff --git a/tests/queries/0_stateless/02041_openssl_hash_functions_test.reference b/tests/queries/0_stateless/02041_openssl_hash_functions_test.reference index e5298179e6f..948e7913c5f 100644 --- a/tests/queries/0_stateless/02041_openssl_hash_functions_test.reference +++ b/tests/queries/0_stateless/02041_openssl_hash_functions_test.reference @@ -6,3 +6,4 @@ A94A8FE5CCB19BA61C4C0873D391E987982FBBD3 9F86D081884C7D659A2FEAA0C55AD015A3BF4F1B2B0B822CD15D6C15B0F00A08 768412320F7B0AA5812FCE428DC4706B3CAE50E02A64CAA16A782249BFE8EFC4B7EF1CCB126255D196047DFEDF17A0A9 EE26B0DD4AF7E749AA1A8EE3C10AE9923F618980772E473F8819A5D4940E0DB27AC185F8A0E1D5F84F88BC887FD67B143732C304CC5FA9AD8E6F57F50028A8FF +3D37FE58435E0D87323DEE4A2C1B339EF954DE63716EE79F5747F94D974F913F diff --git a/tests/queries/0_stateless/02041_openssl_hash_functions_test.sql b/tests/queries/0_stateless/02041_openssl_hash_functions_test.sql index 71940b24af4..7c712cf35fb 100644 --- a/tests/queries/0_stateless/02041_openssl_hash_functions_test.sql +++ b/tests/queries/0_stateless/02041_openssl_hash_functions_test.sql @@ -8,3 +8,4 @@ SELECT hex(SHA224('test')); SELECT hex(SHA256('test')); SELECT hex(SHA384('test')); SELECT hex(SHA512('test')); +SELECT hex(SHA512_256('test')); diff --git a/tests/queries/0_stateless/02184_hash_functions_and_ip_types.reference b/tests/queries/0_stateless/02184_hash_functions_and_ip_types.reference index b305806cd08..aad21567fca 100644 --- a/tests/queries/0_stateless/02184_hash_functions_and_ip_types.reference +++ b/tests/queries/0_stateless/02184_hash_functions_and_ip_types.reference @@ -34,6 +34,7 @@ hex(SHA1(ipv6)): A6D5DCE882AC44804382DE4639E6001612E1C8B5 hex(SHA224(ipv6)): F6995FD7BED2BCA21F68DAC6BBABE742DC1BA177BA8594CEF1715C52 hex(SHA256(ipv6)): F75497BAD6F7747BD6B150B6F69BA2DEE354F1C2A34B7BEA6183973B78640250 hex(SHA512(ipv6)): 0C2893CCBF44BC19CCF339AEED5B68CBFD5A2EF38263A48FE21C3379BA4438E7FF7A02F59D7542442C6E6ED538E6D13D65D3573DADB381651D3D8A5DEA232EAC +hex(SHA512_256(ipv6)): 1A2248FEB5A9D2D8D6C2482F132CFC19448B59DE75358E1F7ECAF444004F85A3 farmFingerprint64(ipv6): 6643158734288374888 javaHash(ipv6): 684606770 xxh3(ipv6): 4051340969481364358 diff --git a/tests/queries/0_stateless/02184_hash_functions_and_ip_types.sql b/tests/queries/0_stateless/02184_hash_functions_and_ip_types.sql index d96574ef4fe..c90c8b90c9e 100644 --- a/tests/queries/0_stateless/02184_hash_functions_and_ip_types.sql +++ b/tests/queries/0_stateless/02184_hash_functions_and_ip_types.sql @@ -38,6 +38,7 @@ SELECT hex(SHA224(ipv6)), hex(SHA256(ipv6)), hex(SHA512(ipv6)), + hex(SHA512_256(ipv6)), farmFingerprint64(ipv6), javaHash(ipv6), xxh3(ipv6), From 2c1f57ba407f35fba4f902f020a720d716c74208 Mon Sep 17 00:00:00 2001 From: Bharat Nallan Chakravarthy Date: Thu, 7 Dec 2023 08:58:11 -0800 Subject: [PATCH 054/110] add docs --- docs/en/sql-reference/functions/hash-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 7276437ec82..1f1875839f9 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -319,9 +319,9 @@ This is a relatively fast non-cryptographic hash function of average quality for Calculates a 64-bit hash code from any type of integer. It works faster than intHash32. Average quality. -## SHA1, SHA224, SHA256, SHA512 +## SHA1, SHA224, SHA256, SHA512, SHA512_256 -Calculates SHA-1, SHA-224, SHA-256, SHA-512 hash from a string and returns the resulting set of bytes as [FixedString](/docs/en/sql-reference/data-types/fixedstring.md). +Calculates SHA-1, SHA-224, SHA-256, SHA-512, SHA-512-256 hash from a string and returns the resulting set of bytes as [FixedString](/docs/en/sql-reference/data-types/fixedstring.md). **Syntax** From 5de99eceec87771c2441bc2042bf134feb10c048 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Dec 2023 21:38:59 +0100 Subject: [PATCH 055/110] Fix "logs and exception messages formatting", part 1 --- src/Common/mysqlxx/Pool.cpp | 73 ++++++++++++++++++++++--------- src/Common/mysqlxx/mysqlxx/Pool.h | 24 ++-------- 2 files changed, 55 insertions(+), 42 deletions(-) diff --git a/src/Common/mysqlxx/Pool.cpp b/src/Common/mysqlxx/Pool.cpp index 43df0aa6708..cc5b18214c8 100644 --- a/src/Common/mysqlxx/Pool.cpp +++ b/src/Common/mysqlxx/Pool.cpp @@ -52,8 +52,7 @@ void Pool::Entry::decrementRefCount() Pool::Pool(const Poco::Util::AbstractConfiguration & cfg, const std::string & config_name, unsigned default_connections_, unsigned max_connections_, const char * parent_config_name_) - : logger(Poco::Logger::get("mysqlxx::Pool")) - , default_connections(default_connections_) + : default_connections(default_connections_) , max_connections(max_connections_) { server = cfg.getString(config_name + ".host"); @@ -127,6 +126,38 @@ Pool::Pool(const Poco::Util::AbstractConfiguration & cfg, const std::string & co } +Pool::Pool( + const std::string & db_, + const std::string & server_, + const std::string & user_, + const std::string & password_, + unsigned port_, + const std::string & socket_, + unsigned connect_timeout_, + unsigned rw_timeout_, + unsigned default_connections_, + unsigned max_connections_, + unsigned enable_local_infile_, + bool opt_reconnect_) + : default_connections(default_connections_) + , max_connections(max_connections_) + , db(db_) + , server(server_) + , user(user_) + , password(password_) + , port(port_) + , socket(socket_) + , connect_timeout(connect_timeout_) + , rw_timeout(rw_timeout_) + , enable_local_infile(enable_local_infile_) + , opt_reconnect(opt_reconnect_) +{ + LOG_DEBUG(log, + "Created MySQL Pool with settings: connect_timeout={}, read_write_timeout={}, default_connections_number={}, max_connections_number={}", + connect_timeout, rw_timeout, default_connections, max_connections); +} + + Pool::~Pool() { std::lock_guard lock(mutex); @@ -148,29 +179,29 @@ Pool::Entry Pool::get(uint64_t wait_timeout) initialize(); for (;;) { - logger.trace("(%s): Iterating through existing MySQL connections", getDescription()); + LOG_TRACE(log, "{}: Iterating through existing MySQL connections", getDescription()); for (auto & connection : connections) { if (connection->ref_count == 0) { - logger.test("Found free connection in pool, returning it to the caller"); + LOG_TEST(log, "Found free connection in pool, returning it to the caller"); return Entry(connection, this); } } - logger.trace("(%s): Trying to allocate a new connection.", getDescription()); + LOG_TRACE(log, "{}: Trying to allocate a new connection.", getDescription()); if (connections.size() < static_cast(max_connections)) { Connection * conn = allocConnection(); if (conn) return Entry(conn, this); - logger.trace("(%s): Unable to create a new connection: Allocation failed.", getDescription()); + LOG_TRACE(log, "{}: Unable to create a new connection: Allocation failed.", getDescription()); } else { - logger.trace("(%s): Unable to create a new connection: Max number of connections has been reached.", getDescription()); + LOG_TRACE(log, "{}: Unable to create a new connection: Max number of connections has been reached.", getDescription()); } if (!wait_timeout) @@ -180,7 +211,7 @@ Pool::Entry Pool::get(uint64_t wait_timeout) throw Poco::Exception("mysqlxx::Pool is full (connection_wait_timeout is exceeded)"); lock.unlock(); - logger.trace("(%s): Sleeping for %d seconds.", getDescription(), MYSQLXX_POOL_SLEEP_ON_CONNECT_FAIL); + LOG_TRACE(log, "{}: Sleeping for {} seconds.", getDescription(), MYSQLXX_POOL_SLEEP_ON_CONNECT_FAIL); sleepForSeconds(MYSQLXX_POOL_SLEEP_ON_CONNECT_FAIL); lock.lock(); } @@ -206,7 +237,7 @@ Pool::Entry Pool::tryGet() return res; } - logger.debug("(%s): Idle connection to MySQL server cannot be recovered, dropping it.", getDescription()); + LOG_DEBUG(log, "{}: Idle connection to MySQL server cannot be recovered, dropping it.", getDescription()); /// This one is disconnected, cannot be reestablished and so needs to be disposed of. connection_it = connections.erase(connection_it); @@ -229,7 +260,7 @@ Pool::Entry Pool::tryGet() void Pool::removeConnection(Connection* connection) { - logger.trace("(%s): Removing connection.", getDescription()); + LOG_TRACE(log, "{}: Removing connection.", getDescription()); std::lock_guard lock(mutex); if (connection) @@ -260,8 +291,8 @@ void Pool::Entry::forceConnected() const else sleepForSeconds(MYSQLXX_POOL_SLEEP_ON_CONNECT_FAIL); - pool->logger.debug( - "Creating a new MySQL connection to %s with settings: connect_timeout=%u, read_write_timeout=%u", + LOG_DEBUG(pool->log, + "Creating a new MySQL connection to {} with settings: connect_timeout={}, read_write_timeout={}", pool->description, pool->connect_timeout, pool->rw_timeout); data->conn.connect( @@ -287,21 +318,21 @@ bool Pool::Entry::tryForceConnected() const auto * const mysql_driver = data->conn.getDriver(); const auto prev_connection_id = mysql_thread_id(mysql_driver); - pool->logger.trace("Entry(connection %lu): sending PING to check if it is alive.", prev_connection_id); + LOG_TRACE(pool->log, "Entry(connection {}): sending PING to check if it is alive.", prev_connection_id); if (data->conn.ping()) /// Attempts to reestablish lost connection { const auto current_connection_id = mysql_thread_id(mysql_driver); if (prev_connection_id != current_connection_id) { - pool->logger.debug("Entry(connection %lu): Reconnected to MySQL server. Connection id changed: %lu -> %lu", - current_connection_id, prev_connection_id, current_connection_id); + LOG_DEBUG(pool->log, "Entry(connection {}): Reconnected to MySQL server. Connection id changed: {} -> {}", + current_connection_id, prev_connection_id, current_connection_id); } - pool->logger.trace("Entry(connection %lu): PING ok.", current_connection_id); + LOG_TRACE(pool->log, "Entry(connection {}): PING ok.", current_connection_id); return true; } - pool->logger.trace("Entry(connection %lu): PING failed.", prev_connection_id); + LOG_TRACE(pool->log, "Entry(connection {}): PING failed.", prev_connection_id); return false; } @@ -326,10 +357,10 @@ Pool::Connection * Pool::allocConnection(bool dont_throw_if_failed_first_time) try { - logger.debug("Connecting to %s", description); + LOG_DEBUG(log, "Connecting to {}", description); - logger.debug( - "Creating a new MySQL connection to %s with settings: connect_timeout=%u, read_write_timeout=%u", + LOG_DEBUG(log, + "Creating a new MySQL connection to {} with settings: connect_timeout={}, read_write_timeout={}", description, connect_timeout, rw_timeout); conn_ptr->conn.connect( @@ -349,7 +380,7 @@ Pool::Connection * Pool::allocConnection(bool dont_throw_if_failed_first_time) } catch (mysqlxx::ConnectionFailed & e) { - logger.error(e.what()); + LOG_ERROR(log, "Failed to connect to MySQL ({}): {}", description, e.what()); if ((!was_successful && !dont_throw_if_failed_first_time) || e.errnum() == ER_ACCESS_DENIED_ERROR diff --git a/src/Common/mysqlxx/mysqlxx/Pool.h b/src/Common/mysqlxx/mysqlxx/Pool.h index 52d116e39ce..bb4d0cefbdc 100644 --- a/src/Common/mysqlxx/mysqlxx/Pool.h +++ b/src/Common/mysqlxx/mysqlxx/Pool.h @@ -169,28 +169,10 @@ public: unsigned default_connections_ = MYSQLXX_POOL_DEFAULT_START_CONNECTIONS, unsigned max_connections_ = MYSQLXX_POOL_DEFAULT_MAX_CONNECTIONS, unsigned enable_local_infile_ = MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE, - bool opt_reconnect_ = MYSQLXX_DEFAULT_MYSQL_OPT_RECONNECT) - : logger(Poco::Logger::get("mysqlxx::Pool")) - , default_connections(default_connections_) - , max_connections(max_connections_) - , db(db_) - , server(server_) - , user(user_) - , password(password_) - , port(port_) - , socket(socket_) - , connect_timeout(connect_timeout_) - , rw_timeout(rw_timeout_) - , enable_local_infile(enable_local_infile_) - , opt_reconnect(opt_reconnect_) - { - logger.debug( - "Created MySQL Pool with settings: connect_timeout=%u, read_write_timeout=%u, default_connections_number=%u, max_connections_number=%u", - connect_timeout, rw_timeout, default_connections, max_connections); - } + bool opt_reconnect_ = MYSQLXX_DEFAULT_MYSQL_OPT_RECONNECT); Pool(const Pool & other) - : logger(other.logger), default_connections{other.default_connections}, + : default_connections{other.default_connections}, max_connections{other.max_connections}, db{other.db}, server{other.server}, user{other.user}, password{other.password}, @@ -220,7 +202,7 @@ public: void removeConnection(Connection * connection); protected: - Poco::Logger & logger; + Poco::Logger * log = &Poco::Logger::get("mysqlxx::Pool"); /// Number of MySQL connections which are created at launch. unsigned default_connections; From ba1404136ccf4ec0f4478b5de69baf377e17fefa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Dec 2023 21:42:57 +0100 Subject: [PATCH 056/110] Fix "logs and exception messages formatting", part 2 --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 115e5ac7ba3..5407fbeecb3 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2241,7 +2241,7 @@ def reportLogStats(args): GROUP BY message_format_string ORDER BY count() DESC LIMIT 20 FORMAT TSVWithNamesAndTypes """ value = clickhouse_execute(args, query).decode(errors="replace") - print("\nTop messages that does not match its format string:\n") + print("\nTop messages that don't match its format string:\n") print(value) print("\n") From 6ed8ea96affe969b69a28814dc9cb721a946339f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Dec 2023 21:43:47 +0100 Subject: [PATCH 057/110] Fix "logs and exception messages formatting", part 2 --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 5407fbeecb3..6178626bcf2 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2241,7 +2241,7 @@ def reportLogStats(args): GROUP BY message_format_string ORDER BY count() DESC LIMIT 20 FORMAT TSVWithNamesAndTypes """ value = clickhouse_execute(args, query).decode(errors="replace") - print("\nTop messages that don't match its format string:\n") + print("\nTop messages not matching their format strings:\n") print(value) print("\n") From 81e0d081a20353a4d327a88742eb3f83527ea659 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Dec 2023 21:48:43 +0100 Subject: [PATCH 058/110] Fix "logs and exception messages formatting", part 2 --- tests/clickhouse-test | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 6178626bcf2..3675bc496b9 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2269,11 +2269,11 @@ def reportLogStats(args): 'Attempt to read after eof', 'String size is too big ({}), maximum: {}' ) AS known_short_messages SELECT count() AS c, message_format_string, substr(any(message), 1, 120), - min(if(length(regexpExtract(message, '(.*)\\([A-Z0-9_]+\\)')) as pref > 0, pref, length(message)) - 26 AS length_without_exception_boilerplate) AS min_length_without_exception_boilerplate + min(if(notEmpty(regexpExtract(message, '(.*)\\([A-Z0-9_]+\\)') as prefix), prefix, length(message)) - 26 AS length_without_exception_boilerplate) AS min_length_without_exception_boilerplate FROM system.text_log WHERE (now() - toIntervalMinute(240)) < event_time AND (length(message_format_string) < 16 - OR (message ilike '%DB::Exception%' AND length_without_exception_boilerplate < 30)) + OR (message ILIKE '%DB::Exception%' AND length_without_exception_boilerplate < 30)) AND message_format_string NOT IN known_short_messages GROUP BY message_format_string ORDER BY c DESC LIMIT 50 FORMAT TSVWithNamesAndTypes """ From 0ee13fde16a6608dbfb0aed780664427be8199f0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Dec 2023 22:01:53 +0100 Subject: [PATCH 059/110] Fix "logs and exception messages formatting", part 3 --- src/Functions/FunctionsAES.cpp | 7 ++++--- src/Functions/FunctionsAES.h | 21 ++++++++++----------- src/Functions/aes_decrypt_mysql.cpp | 7 ++++--- src/Functions/aes_encrypt_mysql.cpp | 6 +++--- src/Functions/decrypt.cpp | 6 +++--- src/Functions/encrypt.cpp | 6 +++--- src/Functions/tryDecrypt.cpp | 11 ++++++----- src/IO/FileEncryptionCommon.cpp | 24 ++++++++++++------------ 8 files changed, 45 insertions(+), 43 deletions(-) diff --git a/src/Functions/FunctionsAES.cpp b/src/Functions/FunctionsAES.cpp index 2911d115e35..d1fcd192e6f 100644 --- a/src/Functions/FunctionsAES.cpp +++ b/src/Functions/FunctionsAES.cpp @@ -15,14 +15,13 @@ namespace ErrorCodes { extern const int OPENSSL_ERROR; } -} + namespace OpenSSLDetails { void onError(std::string error_message) { - error_message += ". OpenSSL error code: " + std::to_string(ERR_get_error()); - throw DB::Exception::createDeprecated(error_message, DB::ErrorCodes::OPENSSL_ERROR); + throw Exception(ErrorCodes::OPENSSL_ERROR, "{}. OpenSSL error code: {}", error_message, ERR_get_error()); } StringRef foldEncryptionKeyInMySQLCompatitableMode(size_t cipher_key_size, StringRef key, std::array & folded_key) @@ -48,4 +47,6 @@ const EVP_CIPHER * getCipherByName(StringRef cipher_name) } +} + #endif diff --git a/src/Functions/FunctionsAES.h b/src/Functions/FunctionsAES.h index c748be5b9b8..4792c997f51 100644 --- a/src/Functions/FunctionsAES.h +++ b/src/Functions/FunctionsAES.h @@ -25,13 +25,14 @@ #include + namespace DB { namespace ErrorCodes { extern const int BAD_ARGUMENTS; } -} + namespace OpenSSLDetails { @@ -60,7 +61,7 @@ struct KeyHolder inline StringRef setKey(size_t cipher_key_size, StringRef key) const { if (key.size != cipher_key_size) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid key size: {} expected {}", key.size, cipher_key_size); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid key size: {} expected {}", key.size, cipher_key_size); return key; } @@ -72,7 +73,7 @@ struct KeyHolder inline StringRef setKey(size_t cipher_key_size, StringRef key) { if (key.size < cipher_key_size) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid key size: {} expected {}", key.size, cipher_key_size); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid key size: {} expected {}", key.size, cipher_key_size); // MySQL does something fancy with the keys that are too long, // ruining compatibility with OpenSSL and not improving security. @@ -118,7 +119,7 @@ inline void validateCipherMode(const EVP_CIPHER * evp_cipher) } } - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unsupported cipher mode"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported cipher mode"); } template @@ -127,13 +128,11 @@ inline void validateIV(StringRef iv_value, const size_t cipher_iv_size) // In MySQL mode we don't care if IV is longer than expected, only if shorter. if ((mode == CipherMode::MySQLCompatibility && iv_value.size != 0 && iv_value.size < cipher_iv_size) || (mode == CipherMode::OpenSSLCompatibility && iv_value.size != 0 && iv_value.size != cipher_iv_size)) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid IV size: {} expected {}", iv_value.size, cipher_iv_size); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid IV size: {} expected {}", iv_value.size, cipher_iv_size); } } -namespace DB -{ template class FunctionEncrypt : public IFunction { @@ -313,12 +312,12 @@ private: // in GCM mode IV can be of arbitrary size (>0), IV is optional for other modes. if (mode == CipherMode::RFC5116_AEAD_AES_GCM && iv_value.size == 0) { - throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid IV size {} != expected size {}", iv_value.size, iv_size); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid IV size {} != expected size {}", iv_value.size, iv_size); } if (mode != CipherMode::RFC5116_AEAD_AES_GCM && key_value.size != key_size) { - throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid key size {} != expected size {}", key_value.size, key_size); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid key size {} != expected size {}", key_value.size, key_size); } } @@ -608,12 +607,12 @@ private: // in GCM mode IV can be of arbitrary size (>0), for other modes IV is optional. if (mode == CipherMode::RFC5116_AEAD_AES_GCM && iv_value.size == 0) { - throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid IV size {} != expected size {}", iv_value.size, iv_size); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid IV size {} != expected size {}", iv_value.size, iv_size); } if (key_value.size != key_size) { - throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid key size {} != expected size {}", key_value.size, key_size); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid key size {} != expected size {}", key_value.size, key_size); } } diff --git a/src/Functions/aes_decrypt_mysql.cpp b/src/Functions/aes_decrypt_mysql.cpp index bb8ef3809d1..8043fa422ad 100644 --- a/src/Functions/aes_decrypt_mysql.cpp +++ b/src/Functions/aes_decrypt_mysql.cpp @@ -5,6 +5,10 @@ #include #include + +namespace DB +{ + namespace { @@ -17,9 +21,6 @@ struct DecryptMySQLModeImpl } -namespace DB -{ - REGISTER_FUNCTION(AESDecryptMysql) { factory.registerFunction>(); diff --git a/src/Functions/aes_encrypt_mysql.cpp b/src/Functions/aes_encrypt_mysql.cpp index 0dcb4108770..fb120151c25 100644 --- a/src/Functions/aes_encrypt_mysql.cpp +++ b/src/Functions/aes_encrypt_mysql.cpp @@ -5,6 +5,9 @@ #include #include +namespace DB +{ + namespace { @@ -16,9 +19,6 @@ struct EncryptMySQLModeImpl } -namespace DB -{ - REGISTER_FUNCTION(AESEncryptMysql) { factory.registerFunction>(); diff --git a/src/Functions/decrypt.cpp b/src/Functions/decrypt.cpp index 1a2d5e2b621..6ad58b959e9 100644 --- a/src/Functions/decrypt.cpp +++ b/src/Functions/decrypt.cpp @@ -5,6 +5,9 @@ #include #include +namespace DB +{ + namespace { @@ -17,9 +20,6 @@ struct DecryptImpl } -namespace DB -{ - REGISTER_FUNCTION(Decrypt) { factory.registerFunction>(); diff --git a/src/Functions/encrypt.cpp b/src/Functions/encrypt.cpp index 38feafbea19..d76148f2176 100644 --- a/src/Functions/encrypt.cpp +++ b/src/Functions/encrypt.cpp @@ -5,6 +5,9 @@ #include #include +namespace DB +{ + namespace { @@ -16,9 +19,6 @@ struct EncryptImpl } -namespace DB -{ - REGISTER_FUNCTION(Encrypt) { factory.registerFunction>(); diff --git a/src/Functions/tryDecrypt.cpp b/src/Functions/tryDecrypt.cpp index eccccb1e4a4..90d10103dd1 100644 --- a/src/Functions/tryDecrypt.cpp +++ b/src/Functions/tryDecrypt.cpp @@ -3,8 +3,12 @@ #if USE_SSL -# include -# include +#include +#include + + +namespace DB +{ namespace { @@ -18,9 +22,6 @@ struct TryDecryptImpl } -namespace DB -{ - REGISTER_FUNCTION(TryDecrypt) { factory.registerFunction>(FunctionDocumentation{ diff --git a/src/IO/FileEncryptionCommon.cpp b/src/IO/FileEncryptionCommon.cpp index 6f4db4975f7..c3348ef6068 100644 --- a/src/IO/FileEncryptionCommon.cpp +++ b/src/IO/FileEncryptionCommon.cpp @@ -97,7 +97,7 @@ namespace uint8_t * ciphertext = reinterpret_cast(out.position()); int ciphertext_size = 0; if (!EVP_EncryptUpdate(evp_ctx, ciphertext, &ciphertext_size, &in[in_size], static_cast(part_size))) - throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Failed to encrypt"); + throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Failed to encrypt: {}", ERR_get_error()); in_size += part_size; if (ciphertext_size) @@ -120,7 +120,7 @@ namespace uint8_t ciphertext[kBlockSize]; int ciphertext_size = 0; if (!EVP_EncryptUpdate(evp_ctx, ciphertext, &ciphertext_size, padded_data, safe_cast(padded_data_size))) - throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Failed to encrypt"); + throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Failed to encrypt: {}", ERR_get_error()); if (!ciphertext_size) return 0; @@ -140,7 +140,7 @@ namespace int ciphertext_size = 0; if (!EVP_EncryptFinal_ex(evp_ctx, ciphertext, &ciphertext_size)) - throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Failed to finalize encrypting"); + throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Failed to finalize encrypting: {}", ERR_get_error()); if (ciphertext_size) out.write(reinterpret_cast(ciphertext), ciphertext_size); return ciphertext_size; @@ -152,7 +152,7 @@ namespace uint8_t * plaintext = reinterpret_cast(out); int plaintext_size = 0; if (!EVP_DecryptUpdate(evp_ctx, plaintext, &plaintext_size, in, safe_cast(size))) - throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Failed to decrypt"); + throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Failed to decrypt: {}", ERR_get_error()); return plaintext_size; } @@ -165,7 +165,7 @@ namespace uint8_t plaintext[kBlockSize]; int plaintext_size = 0; if (!EVP_DecryptUpdate(evp_ctx, plaintext, &plaintext_size, padded_data, safe_cast(padded_data_size))) - throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Failed to decrypt"); + throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Failed to decrypt: {}", ERR_get_error()); if (!plaintext_size) return 0; @@ -184,7 +184,7 @@ namespace uint8_t plaintext[kBlockSize]; int plaintext_size = 0; if (!EVP_DecryptFinal_ex(evp_ctx, plaintext, &plaintext_size)) - throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Failed to finalize decrypting"); + throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Failed to finalize decrypting: {}", ERR_get_error()); if (plaintext_size) memcpy(out, plaintext, plaintext_size); return plaintext_size; @@ -291,11 +291,11 @@ void Encryptor::encrypt(const char * data, size_t size, WriteBuffer & out) auto * evp_ctx = evp_ctx_ptr.get(); if (!EVP_EncryptInit_ex(evp_ctx, evp_cipher, nullptr, nullptr, nullptr)) - throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Failed to initialize encryption context with cipher"); + throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Failed to initialize encryption context with cipher: {}", ERR_get_error()); if (!EVP_EncryptInit_ex(evp_ctx, nullptr, nullptr, reinterpret_cast(key.c_str()), reinterpret_cast(current_iv.c_str()))) - throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Failed to set key and IV for encryption"); + throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Failed to set key and IV for encryption: {}", ERR_get_error()); size_t in_size = 0; size_t out_size = 0; @@ -320,7 +320,7 @@ void Encryptor::encrypt(const char * data, size_t size, WriteBuffer & out) out_size += encryptFinal(evp_ctx, out); if (out_size != in_size) - throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Only part of the data was encrypted"); + throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Only part of the data was encrypted: {} out of {} bytes", out_size, in_size); offset += in_size; } @@ -335,11 +335,11 @@ void Encryptor::decrypt(const char * data, size_t size, char * out) auto * evp_ctx = evp_ctx_ptr.get(); if (!EVP_DecryptInit_ex(evp_ctx, evp_cipher, nullptr, nullptr, nullptr)) - throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Failed to initialize decryption context with cipher"); + throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Failed to initialize decryption context with cipher: {}", ERR_get_error()); if (!EVP_DecryptInit_ex(evp_ctx, nullptr, nullptr, reinterpret_cast(key.c_str()), reinterpret_cast(current_iv.c_str()))) - throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Failed to set key and IV for decryption"); + throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Failed to set key and IV for decryption: {}", ERR_get_error()); size_t in_size = 0; size_t out_size = 0; @@ -364,7 +364,7 @@ void Encryptor::decrypt(const char * data, size_t size, char * out) out_size += decryptFinal(evp_ctx, &out[out_size]); if (out_size != in_size) - throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Only part of the data was decrypted"); + throw Exception(ErrorCodes::DATA_ENCRYPTION_ERROR, "Only part of the data was decrypted: {} out of {} bytes", out_size, in_size); offset += in_size; } From 5dcdbf499b8fdb0d9ae2e6bfdf7ddad37c91314a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Dec 2023 22:04:48 +0100 Subject: [PATCH 060/110] Fix "logs and exception messages formatting", part 4 --- tests/clickhouse-test | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 115e5ac7ba3..6e1c44de529 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2205,7 +2205,7 @@ def reportLogStats(args): GROUP BY message_format_string ORDER BY count DESC LIMIT 100 - FORMAT TSVWithNamesAndTypes + FORMAT PrettySpaceNoEscapes """ value = clickhouse_execute(args, query).decode(errors="replace") print("\nTop patterns of log messages:\n") @@ -2225,7 +2225,7 @@ def reportLogStats(args): GROUP BY pattern ORDER BY count DESC LIMIT 30 - FORMAT TSVWithNamesAndTypes + FORMAT PrettySpaceNoEscapes """ value = clickhouse_execute(args, query).decode(errors="replace") print("\nTop messages without format string (fmt::runtime):\n") @@ -2238,7 +2238,7 @@ def reportLogStats(args): WHERE (now() - toIntervalMinute(240)) < event_time AND (message NOT LIKE (replaceRegexpAll(message_format_string, '{[:.0-9dfx]*}', '%') AS s)) AND (message NOT LIKE concat('%Exception: ', s, '%')) - GROUP BY message_format_string ORDER BY count() DESC LIMIT 20 FORMAT TSVWithNamesAndTypes + GROUP BY message_format_string ORDER BY count() DESC LIMIT 20 FORMAT PrettySpaceNoEscapes """ value = clickhouse_execute(args, query).decode(errors="replace") print("\nTop messages that does not match its format string:\n") @@ -2275,7 +2275,7 @@ def reportLogStats(args): AND (length(message_format_string) < 16 OR (message ilike '%DB::Exception%' AND length_without_exception_boilerplate < 30)) AND message_format_string NOT IN known_short_messages - GROUP BY message_format_string ORDER BY c DESC LIMIT 50 FORMAT TSVWithNamesAndTypes + GROUP BY message_format_string ORDER BY c DESC LIMIT 50 FORMAT PrettySpaceNoEscapes """ value = clickhouse_execute(args, query).decode(errors="replace") print("\nTop short messages:\n") From 43ddf349b611f0c84b365efef3508733f716a65c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Dec 2023 22:10:33 +0100 Subject: [PATCH 061/110] Remove bad test --- .../01600_parts_states_metrics_long.reference | 4 -- .../01600_parts_states_metrics_long.sh | 40 ------------------- 2 files changed, 44 deletions(-) delete mode 100644 tests/queries/0_stateless/01600_parts_states_metrics_long.reference delete mode 100755 tests/queries/0_stateless/01600_parts_states_metrics_long.sh diff --git a/tests/queries/0_stateless/01600_parts_states_metrics_long.reference b/tests/queries/0_stateless/01600_parts_states_metrics_long.reference deleted file mode 100644 index 98fb6a68656..00000000000 --- a/tests/queries/0_stateless/01600_parts_states_metrics_long.reference +++ /dev/null @@ -1,4 +0,0 @@ -1 -1 -1 -1 diff --git a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh b/tests/queries/0_stateless/01600_parts_states_metrics_long.sh deleted file mode 100755 index 89ce84f6dbc..00000000000 --- a/tests/queries/0_stateless/01600_parts_states_metrics_long.sh +++ /dev/null @@ -1,40 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -# NOTE: database = $CLICKHOUSE_DATABASE is unwanted -verify_sql="SELECT - (SELECT sumIf(value, metric = 'PartsActive'), sumIf(value, metric = 'PartsOutdated') FROM system.metrics) - = (SELECT sum(active), sum(NOT active) FROM - (SELECT active FROM system.parts UNION ALL SELECT active FROM system.projection_parts))" - -# The query is not atomic - it can compare states between system.parts and system.metrics from different points in time. -# So, there is inherent race condition. But it should get expected result eventually. -# In case of test failure, this code will do infinite loop and timeout. -verify() -{ - while true - do - result=$( $CLICKHOUSE_CLIENT -m --query="$verify_sql" ) - [ "$result" = "1" ] && break - sleep 0.1 - done - echo 1 -} - -$CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=1 --query="DROP TABLE IF EXISTS test_table" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test_table(data Date) ENGINE = MergeTree PARTITION BY toYear(data) ORDER BY data;" - -$CLICKHOUSE_CLIENT --query="INSERT INTO test_table VALUES ('1992-01-01')" -verify - -$CLICKHOUSE_CLIENT --query="INSERT INTO test_table VALUES ('1992-01-02')" -verify - -$CLICKHOUSE_CLIENT --query="OPTIMIZE TABLE test_table FINAL" -verify - -$CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=1 --query="DROP TABLE test_table" -verify From 84f51793456be4eb9abede08c1a3759780de9ff5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Dec 2023 22:16:50 +0100 Subject: [PATCH 062/110] Remove bad test --- .../01051_system_stack_trace.reference | 18 -------------- .../0_stateless/01051_system_stack_trace.sql | 24 ------------------- 2 files changed, 42 deletions(-) delete mode 100644 tests/queries/0_stateless/01051_system_stack_trace.reference delete mode 100644 tests/queries/0_stateless/01051_system_stack_trace.sql diff --git a/tests/queries/0_stateless/01051_system_stack_trace.reference b/tests/queries/0_stateless/01051_system_stack_trace.reference deleted file mode 100644 index 29f6990e6b4..00000000000 --- a/tests/queries/0_stateless/01051_system_stack_trace.reference +++ /dev/null @@ -1,18 +0,0 @@ --- { echo } -SELECT count() > 0 FROM system.stack_trace WHERE query_id != '' AND thread_name = 'TCPHandler'; -1 --- opimization for not reading /proc/self/task/{}/comm and avoid sending signal -SELECT countIf(thread_id > 0) > 0 FROM system.stack_trace; -1 --- optimization for trace -SELECT length(trace) > 0 FROM system.stack_trace WHERE length(trace) > 0 LIMIT 1; -1 --- optimization for query_id -SELECT length(query_id) > 0 FROM system.stack_trace WHERE query_id != '' AND thread_name = 'TCPHandler' LIMIT 1; -1 --- optimization for thread_name -SELECT length(thread_name) > 0 FROM system.stack_trace WHERE thread_name != '' LIMIT 1; -1 --- enough rows (optimizations works "correctly") -SELECT count() > 100 FROM system.stack_trace; -1 diff --git a/tests/queries/0_stateless/01051_system_stack_trace.sql b/tests/queries/0_stateless/01051_system_stack_trace.sql deleted file mode 100644 index 93cc5763f19..00000000000 --- a/tests/queries/0_stateless/01051_system_stack_trace.sql +++ /dev/null @@ -1,24 +0,0 @@ --- Tags: no-parallel --- Tag no-parallel: to decrease failure probability of collecting stack traces - --- Process one thread at a time -SET max_block_size = 1; - --- It is OK to have bigger timeout here since: --- a) this test is marked as no-parallel --- b) there is a filter by thread_name, so it will send signals only to the threads with the name TCPHandler --- c) max_block_size is 1 -SET storage_system_stack_trace_pipe_read_timeout_ms = 5000; - --- { echo } -SELECT count() > 0 FROM system.stack_trace WHERE query_id != '' AND thread_name = 'TCPHandler'; --- opimization for not reading /proc/self/task/{}/comm and avoid sending signal -SELECT countIf(thread_id > 0) > 0 FROM system.stack_trace; --- optimization for trace -SELECT length(trace) > 0 FROM system.stack_trace WHERE length(trace) > 0 LIMIT 1; --- optimization for query_id -SELECT length(query_id) > 0 FROM system.stack_trace WHERE query_id != '' AND thread_name = 'TCPHandler' LIMIT 1; --- optimization for thread_name -SELECT length(thread_name) > 0 FROM system.stack_trace WHERE thread_name != '' LIMIT 1; --- enough rows (optimizations works "correctly") -SELECT count() > 100 FROM system.stack_trace; From b4cac92e28badd9de1d07c29b7d937bfd2a20a3b Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 7 Dec 2023 17:52:30 +0100 Subject: [PATCH 063/110] Add some comments. --- src/Common/RWLock.cpp | 9 +++++++++ src/Common/tests/gtest_rw_lock.cpp | 2 +- .../test_backup_restore_on_cluster/test_concurrency.py | 3 ++- 3 files changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Common/RWLock.cpp b/src/Common/RWLock.cpp index 5a13bb83f29..a3ba9523f6c 100644 --- a/src/Common/RWLock.cpp +++ b/src/Common/RWLock.cpp @@ -177,6 +177,7 @@ RWLockImpl::getLock(RWLockImpl::Type type, const String & query_id, const std::c /// Lock is free to acquire if (rdlock_owner == readers_queue.end() && wrlock_owner == writers_queue.end()) { + /// Set `rdlock_owner` or `wrlock_owner` and make it owner. (type == Read ? rdlock_owner : wrlock_owner) = it_group; /// SM2: nothrow grantOwnership(it_group); } @@ -341,13 +342,21 @@ void RWLockImpl::grantOwnershipToAllReaders() noexcept { if (rdlock_owner != readers_queue.end()) { + size_t num_new_owners = 0; + for (;;) { + if (!rdlock_owner->ownership) + ++num_new_owners; grantOwnership(rdlock_owner); if (std::next(rdlock_owner) == readers_queue.end()) break; ++rdlock_owner; } + + /// There couldn't be more than one reader group which is not an owner. + /// (Because we add a new reader group only if the last reader group is already an owner - see the `can_use_last_group` variable.) + chassert(num_new_owners <= 1); } } diff --git a/src/Common/tests/gtest_rw_lock.cpp b/src/Common/tests/gtest_rw_lock.cpp index 7de3ced2d0d..16ba01d02c6 100644 --- a/src/Common/tests/gtest_rw_lock.cpp +++ b/src/Common/tests/gtest_rw_lock.cpp @@ -37,7 +37,7 @@ namespace if (timepoint.length() < 5) timepoint.insert(0, 5 - timepoint.length(), ' '); std::lock_guard lock{mutex}; - std::cout << timepoint << " : " << event << std::endl; + //std::cout << timepoint << " : " << event << std::endl; events.emplace_back(std::move(event)); } diff --git a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py index 65f05d186ce..c08f3c9c242 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py @@ -237,7 +237,8 @@ def test_create_or_drop_tables_during_backup(db_engine, table_engine): while time.time() < end_time: table_name = f"mydb.tbl{randint(1, num_nodes)}" node = nodes[randint(0, num_nodes - 1)] - # "TRUNCATE TABLE IF EXISTS" still can throw some errors (e.g. "WRITE locking attempt on node0 has timed out!") + # "TRUNCATE TABLE IF EXISTS" still can throw some errors + # (e.g. "WRITE locking attempt on node0 has timed out!" if the table engine is "Log"). # So we use query_and_get_answer_with_error() to ignore any errors. # `lock_acquire_timeout` is reduced because we don't wait our test to wait too long. node.query_and_get_answer_with_error( From 2c86d7db711aa1b2156f44585867149fcd004ce1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Dec 2023 22:20:39 +0100 Subject: [PATCH 064/110] Remove bad test (3) --- ...02896_memory_accounting_for_user.reference | 0 .../02896_memory_accounting_for_user.sh | 46 ------------------- 2 files changed, 46 deletions(-) delete mode 100644 tests/queries/0_stateless/02896_memory_accounting_for_user.reference delete mode 100755 tests/queries/0_stateless/02896_memory_accounting_for_user.sh diff --git a/tests/queries/0_stateless/02896_memory_accounting_for_user.reference b/tests/queries/0_stateless/02896_memory_accounting_for_user.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02896_memory_accounting_for_user.sh b/tests/queries/0_stateless/02896_memory_accounting_for_user.sh deleted file mode 100755 index f3016671420..00000000000 --- a/tests/queries/0_stateless/02896_memory_accounting_for_user.sh +++ /dev/null @@ -1,46 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-parallel, long, no-random-settings - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - - -total_iterations=16 -parallelism=32 - -$CLICKHOUSE_CLIENT --query='DROP TABLE IF EXISTS test_inserts' -$CLICKHOUSE_CLIENT --query='CREATE TABLE test_inserts ENGINE=Null AS system.numbers' - -run_query() { - ( $CLICKHOUSE_CLIENT --query='SELECT * FROM numbers_mt(1000000) FORMAT CSV' | $CLICKHOUSE_CLIENT --max_threads 8 --max_memory_usage_for_user 1073741824 -q 'INSERT INTO test_inserts FORMAT CSV' 2>/dev/null ) -} - -for ((i = 1; i <= total_iterations; i++)); do - for ((j = 1; j <= parallelism; j++)); do - run_query & pids+=($!) - done - - EXIT_CODE=0 - new_pids=() - for pid in "${pids[@]:0:parallelism}"; do - CODE=0 - wait "${pid}" || CODE=$? - run_query & new_pids+=($!) - if [[ "${CODE}" != "0" ]]; then - EXIT_CODE=1; - fi - done - for pid in "${pids[@]:parallelism}"; do - CODE=0 - wait "${pid}" || CODE=$? - if [[ "${CODE}" != "0" ]]; then - EXIT_CODE=1; - fi - done - pids=("${new_pids[@]}") - - if [[ $EXIT_CODE -ne 0 ]]; then - exit $EXIT_CODE - fi -done From 57c79ecd6ada35145a3c9a5163a5c7585763fa36 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Dec 2023 22:25:29 +0100 Subject: [PATCH 065/110] Remove bad test (4) --- ...avoiding_when_reading_from_cache.reference | 1 - ...o_seek_avoiding_when_reading_from_cache.sh | 36 ------------------- 2 files changed, 37 deletions(-) delete mode 100644 tests/queries/1_stateful/00180_no_seek_avoiding_when_reading_from_cache.reference delete mode 100755 tests/queries/1_stateful/00180_no_seek_avoiding_when_reading_from_cache.sh diff --git a/tests/queries/1_stateful/00180_no_seek_avoiding_when_reading_from_cache.reference b/tests/queries/1_stateful/00180_no_seek_avoiding_when_reading_from_cache.reference deleted file mode 100644 index d05b1f927f4..00000000000 --- a/tests/queries/1_stateful/00180_no_seek_avoiding_when_reading_from_cache.reference +++ /dev/null @@ -1 +0,0 @@ -0 0 diff --git a/tests/queries/1_stateful/00180_no_seek_avoiding_when_reading_from_cache.sh b/tests/queries/1_stateful/00180_no_seek_avoiding_when_reading_from_cache.sh deleted file mode 100755 index 2e1b807c496..00000000000 --- a/tests/queries/1_stateful/00180_no_seek_avoiding_when_reading_from_cache.sh +++ /dev/null @@ -1,36 +0,0 @@ -#!/usr/bin/env bash - -# Tags: no-parallel, no-random-settings, long - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - - -# Test assumes that the whole table is residing in the cache, but `hits_s3` has only 128Mi of cache. -# So we need to create a smaller table. -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS hits_s3_sampled" -$CLICKHOUSE_CLIENT -q "CREATE TABLE hits_s3_sampled AS test.hits_s3" -$CLICKHOUSE_CLIENT -q "INSERT INTO hits_s3_sampled SELECT * FROM test.hits_s3 SAMPLE 0.01" -$CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE hits_s3_sampled FINAL" - -$CLICKHOUSE_CLIENT -q "SYSTEM DROP FILESYSTEM CACHE" - -# Warm up the cache -$CLICKHOUSE_CLIENT -q "SELECT * FROM hits_s3_sampled WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null" -$CLICKHOUSE_CLIENT -q "SELECT * FROM hits_s3_sampled WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null" - -query_id=02906_read_from_cache_$RANDOM -$CLICKHOUSE_CLIENT --query_id ${query_id} -q "SELECT * FROM hits_s3_sampled WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10 FORMAT Null" - -$CLICKHOUSE_CLIENT -nq " - SYSTEM FLUSH LOGS; - - -- AsynchronousReaderIgnoredBytes = 0: no seek-avoiding happened - -- CachedReadBufferReadFromSourceBytes = 0: sanity check to ensure we read only from cache - SELECT ProfileEvents['AsynchronousReaderIgnoredBytes'], ProfileEvents['CachedReadBufferReadFromSourceBytes'] - FROM system.query_log - WHERE query_id = '$query_id' AND type = 'QueryFinish' AND event_date >= yesterday() AND current_database = currentDatabase() -" - -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS hits_s3_sampled" From 01135211e9134b2a4ce02b6209affa3bcc64a81e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Dec 2023 03:12:54 +0100 Subject: [PATCH 066/110] Fix test --- tests/integration/test_multiple_disks/test.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index 2484ef1bdf1..0a887cf151d 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -1834,10 +1834,11 @@ def _insert_merge_execute( ORDER BY tuple() PARTITION BY tuple() TTL now()-1 TO VOLUME 'external' - SETTINGS storage_policy='{policy}', allow_suspicious_ttl_expressions = 1 + SETTINGS storage_policy='{policy}' """.format( name=name, policy=policy - ) + ), + settings={"allow_suspicious_ttl_expressions": 1} ) for i in range(parts): From 64e64797ae4331ce06013020d201b512c0522686 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Dec 2023 03:15:01 +0100 Subject: [PATCH 067/110] Update version --- src/Core/SettingsChangesHistory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index f09afc09567..54c28fb9f92 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -81,7 +81,7 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { - {"23.11", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}}}, + {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}}}, {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, From d810bea8e302646b793c71986df9c2635ead54cc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Dec 2023 03:17:13 +0100 Subject: [PATCH 068/110] Fix test --- tests/integration/test_ttl_move/test.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 071392d90fb..d50d45570a1 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -300,9 +300,10 @@ def test_moves_work_after_storage_policy_change(started_cluster, name, engine): # Second expression is preferred because d1 > now()-3600. node1.query( - """ALTER TABLE {name} MODIFY TTL now()-3600 TO DISK 'jbod1', d1 TO DISK 'external' SETTINGS allow_suspicious_ttl_expressions = 1""".format( + """ALTER TABLE {name} MODIFY TTL now()-3600 TO DISK 'jbod1', d1 TO DISK 'external'""".format( name=name - ) + ), + settings={"allow_suspicious_ttl_expressions": 1} ) wait_expire_1 = 12 From 567d47935e193df61ccfe8b1ffee2f57a2c73e7a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Dec 2023 03:20:12 +0100 Subject: [PATCH 069/110] Revert "Check what will happen if we build ClickHouse with Musl" This reverts commit 1790626406ae707c551fccce6ceb166d81d360bc. --- PreLoad.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/PreLoad.cmake b/PreLoad.cmake index 223e7aedd42..4879e721ae3 100644 --- a/PreLoad.cmake +++ b/PreLoad.cmake @@ -86,7 +86,7 @@ if (OS MATCHES "Linux" # - compile musl with debug and -fasynchronous-unwind-tables # # But none of this changes anything so far. - set (CMAKE_TOOLCHAIN_FILE "cmake/linux/toolchain-x86_64-musl.cmake" CACHE INTERNAL "") + set (CMAKE_TOOLCHAIN_FILE "cmake/linux/toolchain-x86_64.cmake" CACHE INTERNAL "") elseif (ARCH MATCHES "^(aarch64.*|AARCH64.*|arm64.*|ARM64.*)") set (CMAKE_TOOLCHAIN_FILE "cmake/linux/toolchain-aarch64.cmake" CACHE INTERNAL "") elseif (ARCH MATCHES "^(ppc64le.*|PPC64LE.*)") From 845960cc3146abe5a608f286f5043a8e60f2988f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Dec 2023 03:28:21 +0100 Subject: [PATCH 070/110] Leave only one test for now --- .../02790_jit_wrong_result.reference | 3 --- .../0_stateless/02790_jit_wrong_result.sql | 27 ------------------- 2 files changed, 30 deletions(-) delete mode 100644 tests/queries/0_stateless/02790_jit_wrong_result.reference delete mode 100644 tests/queries/0_stateless/02790_jit_wrong_result.sql diff --git a/tests/queries/0_stateless/02790_jit_wrong_result.reference b/tests/queries/0_stateless/02790_jit_wrong_result.reference deleted file mode 100644 index fe3b98956d3..00000000000 --- a/tests/queries/0_stateless/02790_jit_wrong_result.reference +++ /dev/null @@ -1,3 +0,0 @@ --943999939 -0 -1259570390 diff --git a/tests/queries/0_stateless/02790_jit_wrong_result.sql b/tests/queries/0_stateless/02790_jit_wrong_result.sql deleted file mode 100644 index 4105fb6786a..00000000000 --- a/tests/queries/0_stateless/02790_jit_wrong_result.sql +++ /dev/null @@ -1,27 +0,0 @@ -DROP TABLE IF EXISTS t2; -CREATE TABLE t2 (c0 Int32, c1 Int32, c2 String) ENGINE = Log() ; -INSERT INTO t2(c1, c0) VALUES (1697596429, 1259570390); -INSERT INTO t2(c1, c2) VALUES (-871444251, 's,'); -INSERT INTO t2(c0, c2, c1) VALUES (-943999939, '', 1756486294); - -SELECT * FROM -( -SELECT MIN(t2.c0) -FROM t2 -GROUP BY log(-(t2.c0 / (t2.c0 - t2.c0))) -HAVING NOT (NOT (-(NOT MIN(t2.c0)))) -UNION ALL -SELECT MIN(t2.c0) -FROM t2 -GROUP BY log(-(t2.c0 / (t2.c0 - t2.c0))) -HAVING NOT (NOT (NOT (-(NOT MIN(t2.c0))))) -UNION ALL -SELECT MIN(t2.c0) -FROM t2 -GROUP BY log(-(t2.c0 / (t2.c0 - t2.c0))) -HAVING (NOT (NOT (-(NOT MIN(t2.c0))))) IS NULL -) -ORDER BY 1 -SETTINGS aggregate_functions_null_for_empty = 1, enable_optimize_predicate_expression = 0, min_count_to_compile_expression = 0; - -DROP TABLE t2; From 129e0349b7bea363b7d290c8b81d75ea4c57ca92 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 8 Dec 2023 02:33:04 +0000 Subject: [PATCH 071/110] Automatic style fix --- tests/integration/test_multiple_disks/test.py | 2 +- tests/integration/test_ttl_move/test.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index 0a887cf151d..fdd81284b2a 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -1838,7 +1838,7 @@ def _insert_merge_execute( """.format( name=name, policy=policy ), - settings={"allow_suspicious_ttl_expressions": 1} + settings={"allow_suspicious_ttl_expressions": 1}, ) for i in range(parts): diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index d50d45570a1..1df9bc6d3c9 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -303,7 +303,7 @@ def test_moves_work_after_storage_policy_change(started_cluster, name, engine): """ALTER TABLE {name} MODIFY TTL now()-3600 TO DISK 'jbod1', d1 TO DISK 'external'""".format( name=name ), - settings={"allow_suspicious_ttl_expressions": 1} + settings={"allow_suspicious_ttl_expressions": 1}, ) wait_expire_1 = 12 From 7bc7159a3db9f7e6c042dbbcee70067d5a3d5339 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Dec 2023 03:35:55 +0100 Subject: [PATCH 072/110] Update submodule --- contrib/azure | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/azure b/contrib/azure index 352ff0a61cb..d94ae337c8f 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit 352ff0a61cb319ac1cc38c4058443ddf70147530 +Subproject commit d94ae337c8ffbf74e99d412ac55e38f2190490f5 From 7598ae1c041457c4040c17d5d2ed63f0f1bdde7e Mon Sep 17 00:00:00 2001 From: Bharat Nallan Chakravarthy Date: Thu, 7 Dec 2023 23:53:08 -0800 Subject: [PATCH 073/110] fix test --- .../02415_all_new_functions_must_be_documented.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 7bb0b965fbc..a08434bfd7b 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -51,6 +51,7 @@ LpNormalize MACNumToString MACStringToNum MACStringToOUI +SHA512_256 URLHash URLHierarchy URLPathHierarchy From 16cbcf700ecf8b12e78b74b5a4cea0228ff7590b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 7 Dec 2023 22:52:27 +0100 Subject: [PATCH 074/110] Prohibit parallel read from system.stack_trace Before rewriting system.stack_trace to handle max_block_size (in #54946) parallel reading from system.stack_trace was prohibited, because this could lead to hang of system.stack_trace table. But that rewrite broke this guarantee, so let's fix it to avoid possible hung. Signed-off-by: Azat Khuzhin --- src/Storages/System/StorageSystemStackTrace.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/System/StorageSystemStackTrace.cpp b/src/Storages/System/StorageSystemStackTrace.cpp index 477b784952e..b286a14353e 100644 --- a/src/Storages/System/StorageSystemStackTrace.cpp +++ b/src/Storages/System/StorageSystemStackTrace.cpp @@ -57,12 +57,13 @@ std::atomic signal_latch = false; /// Only need for thread sanitizer. /** Notes: * Only one query from the table can be processed at the moment of time. - * This is ensured by the mutex in fillData function. + * This is ensured by the mutex in StorageSystemStackTraceSource. * We obtain information about threads by sending signal and receiving info from the signal handler. * Information is passed via global variables and pipe is used for signaling. * Actually we can send all information via pipe, but we read from it with timeout just in case, - * so it's convenient to use is only for signaling. + * so it's convenient to use it only for signaling. */ +std::mutex mutex; StackTrace stack_trace{NoCapture{}}; @@ -229,6 +230,8 @@ public: , pipe_read_timeout_ms(static_cast(context->getSettingsRef().storage_system_stack_trace_pipe_read_timeout_ms.totalMilliseconds())) , log(log_) , proc_it("/proc/self/task") + /// It shouldn't be possible to do concurrent reads from this table. + , lock(mutex) { /// Create a mask of what columns are needed in the result. NameSet names_set(column_names.begin(), column_names.end()); @@ -241,9 +244,6 @@ public: protected: Chunk generate() override { - /// It shouldn't be possible to do concurrent reads from this table. - std::lock_guard lock(mutex); - MutableColumns res_columns = header.cloneEmptyColumns(); ColumnPtr thread_ids; @@ -357,7 +357,7 @@ private: size_t signals_sent = 0; size_t signals_sent_ms = 0; - std::mutex mutex; + std::unique_lock lock; ColumnPtr getFilteredThreadIds() { From f41d3be4b2e673f318b2c40a57e10b5a15454537 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 8 Dec 2023 11:28:14 +0100 Subject: [PATCH 075/110] Fix bad merge --- .../test_postgresql_replica_database_engine_2/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index eb5982da302..df72a2f705c 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -941,6 +941,7 @@ def test_symbols_in_publication_name(started_cluster): ) check_tables_are_synchronized( instance, table, postgres_database=pg_manager3.get_default_database() + ) if __name__ == "__main__": From b1043b7ad3ec102261ee40ed2348dd656406698e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 8 Dec 2023 12:11:00 +0100 Subject: [PATCH 076/110] Increase log verbosity of some messages in system.stack_trace This can help to debug 01051_system_stack_trace test failures. Signed-off-by: Azat Khuzhin --- src/Storages/System/StorageSystemStackTrace.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/System/StorageSystemStackTrace.cpp b/src/Storages/System/StorageSystemStackTrace.cpp index b286a14353e..a860930087c 100644 --- a/src/Storages/System/StorageSystemStackTrace.cpp +++ b/src/Storages/System/StorageSystemStackTrace.cpp @@ -190,7 +190,7 @@ ThreadIdToName getFilteredThreadNames(ASTPtr query, ContextPtr context, const Pa tid_to_name[tid] = thread_name; all_thread_names->insert(thread_name); } - LOG_TEST(log, "Read {} thread names for {} threads, took {} ms", tid_to_name.size(), thread_ids.size(), watch.elapsedMilliseconds()); + LOG_TRACE(log, "Read {} thread names for {} threads, took {} ms", tid_to_name.size(), thread_ids.size(), watch.elapsedMilliseconds()); Block block { ColumnWithTypeAndName(std::move(all_thread_names), std::make_shared(), "thread_name") }; VirtualColumnUtils::filterBlockWithQuery(query, block, context); @@ -250,7 +250,7 @@ protected: { Stopwatch watch; thread_ids = getFilteredThreadIds(); - LOG_TEST(log, "Read {} threads, took {} ms", thread_ids->size(), watch.elapsedMilliseconds()); + LOG_TRACE(log, "Read {} threads, took {} ms", thread_ids->size(), watch.elapsedMilliseconds()); } if (thread_ids->empty()) return Chunk(); @@ -332,7 +332,7 @@ protected: ++sequence_num; } } - LOG_TEST(log, "Send signal to {} threads (total), took {} ms", signals_sent, signals_sent_ms); + LOG_TRACE(log, "Send signal to {} threads (total), took {} ms", signals_sent, signals_sent_ms); UInt64 num_rows = res_columns.at(0)->size(); Chunk chunk(std::move(res_columns), num_rows); From f3fada0873bbc5894c8c3b77c28591d30aa8c480 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 8 Dec 2023 10:51:53 +0000 Subject: [PATCH 077/110] Analyzer: Fix JOIN ON true with join_use_nulls --- src/Planner/PlannerJoinTree.cpp | 28 +++++++++---------- .../0_stateless/02000_join_on_const.reference | 4 +++ .../0_stateless/02000_join_on_const.sql | 25 +++++++++++++++++ 3 files changed, 43 insertions(+), 14 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 95ea260e727..abcf971b832 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -988,18 +988,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ if (join_strictness == JoinStrictness::All || join_strictness == JoinStrictness::Semi || join_strictness == JoinStrictness::Anti) join_constant = tryExtractConstantFromJoinNode(join_table_expression); - if (join_constant) - { - /** If there is JOIN with always true constant, we transform it to cross. - * If there is JOIN with always false constant, we do not process JOIN keys. - * It is expected by join algorithm to handle such case. - * - * Example: SELECT * FROM test_table AS t1 INNER JOIN test_table AS t2 ON 1; - */ - if (*join_constant) - join_kind = JoinKind::Cross; - } - else if (join_node.isOnJoinExpression()) + if (!join_constant && join_node.isOnJoinExpression()) { join_clauses_and_actions = buildJoinClausesAndActions(left_plan_output_columns, right_plan_output_columns, @@ -1079,7 +1068,6 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); - bool join_use_nulls = settings.join_use_nulls; auto to_nullable_function = FunctionFactory::instance().get("toNullable", query_context); auto join_cast_plan_columns_to_nullable = [&](QueryPlan & plan_to_add_cast) @@ -1105,7 +1093,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ plan_to_add_cast.addStep(std::move(cast_join_columns_step)); }; - if (join_use_nulls) + if (settings.join_use_nulls) { if (isFull(join_kind)) { @@ -1124,6 +1112,18 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ auto table_join = std::make_shared(settings, query_context->getGlobalTemporaryVolume()); table_join->getTableJoin() = join_node.toASTTableJoin()->as(); + + if (join_constant) + { + /** If there is JOIN with always true constant, we transform it to cross. + * If there is JOIN with always false constant, we do not process JOIN keys. + * It is expected by join algorithm to handle such case. + * + * Example: SELECT * FROM test_table AS t1 INNER JOIN test_table AS t2 ON 1; + */ + if (*join_constant) + join_kind = JoinKind::Cross; + } table_join->getTableJoin().kind = join_kind; if (join_kind == JoinKind::Comma) diff --git a/tests/queries/0_stateless/02000_join_on_const.reference b/tests/queries/0_stateless/02000_join_on_const.reference index e9d1c685fdd..848ecedf9e3 100644 --- a/tests/queries/0_stateless/02000_join_on_const.reference +++ b/tests/queries/0_stateless/02000_join_on_const.reference @@ -65,3 +65,7 @@ SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 SELECT * FROM (SELECT 1 as a) as t1 SEMI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; 1 ('',0) +2 +4 2 Nullable(UInt64) UInt8 +4 2 UInt64 Nullable(UInt8) +4 2 Nullable(UInt64) Nullable(UInt8) diff --git a/tests/queries/0_stateless/02000_join_on_const.sql b/tests/queries/0_stateless/02000_join_on_const.sql index 3205c084672..a68e75443d8 100644 --- a/tests/queries/0_stateless/02000_join_on_const.sql +++ b/tests/queries/0_stateless/02000_join_on_const.sql @@ -90,6 +90,31 @@ SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 -- { echoOff } +SELECT a + 1 +FROM (SELECT 1 as x) as t1 +LEFT JOIN ( SELECT 1 AS a ) AS t2 +ON TRUE +SETTINGS allow_experimental_analyzer=1, join_use_nulls=1; + +SELECT a + 1, x + 1, toTypeName(a), toTypeName(x) +FROM (SELECT 1 as x) as t1 +LEFT JOIN ( SELECT sum(number) as a from numbers(3) GROUP BY NULL) AS t2 +ON TRUE +SETTINGS allow_experimental_analyzer=1, join_use_nulls=1; + +SELECT a + 1, x + 1, toTypeName(a), toTypeName(x) +FROM (SELECT 1 as x) as t1 +RIGHT JOIN ( SELECT sum(number) as a from numbers(3) GROUP BY NULL) AS t2 +ON TRUE +SETTINGS allow_experimental_analyzer=1, join_use_nulls=1; + +SELECT a + 1, x + 1, toTypeName(a), toTypeName(x) +FROM (SELECT 1 as x) as t1 +FULL JOIN ( SELECT sum(number) as a from numbers(3) GROUP BY NULL) AS t2 +ON TRUE +SETTINGS allow_experimental_analyzer=1, join_use_nulls=1; + + DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; From 75441d9812078eada13a275a3a148ee4b28facb9 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 8 Dec 2023 14:44:30 -0400 Subject: [PATCH 078/110] test for #33308 --- ...p_content_type_with_http_headers_progress.reference | 3 +++ ...935_http_content_type_with_http_headers_progress.sh | 10 ++++++++++ 2 files changed, 13 insertions(+) create mode 100644 tests/queries/0_stateless/02935_http_content_type_with_http_headers_progress.reference create mode 100755 tests/queries/0_stateless/02935_http_content_type_with_http_headers_progress.sh diff --git a/tests/queries/0_stateless/02935_http_content_type_with_http_headers_progress.reference b/tests/queries/0_stateless/02935_http_content_type_with_http_headers_progress.reference new file mode 100644 index 00000000000..2f48f1c5c00 --- /dev/null +++ b/tests/queries/0_stateless/02935_http_content_type_with_http_headers_progress.reference @@ -0,0 +1,3 @@ + Content-Type: text/csv; charset=UTF-8; header=absent + Content-Type: application/json; charset=UTF-8 + Content-Type: text/tab-separated-values; charset=UTF-8 diff --git a/tests/queries/0_stateless/02935_http_content_type_with_http_headers_progress.sh b/tests/queries/0_stateless/02935_http_content_type_with_http_headers_progress.sh new file mode 100755 index 00000000000..570acc40ea9 --- /dev/null +++ b/tests/queries/0_stateless/02935_http_content_type_with_http_headers_progress.sh @@ -0,0 +1,10 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +(seq 1 200| xargs -n1 -P0 -Ixxx curl -v "${CLICKHOUSE_URL}/?http_headers_progress_interval_ms=1&send_progress_in_http_headers=true&query=select+sleepEachRow(0.01)from+numbers(10)+FORMAT+CSV" 2>&1|grep -Eo " Content-Type:.*$")|sort -u +(seq 1 200| xargs -n1 -P0 -Ixxx curl -v "${CLICKHOUSE_URL}/?http_headers_progress_interval_ms=1&send_progress_in_http_headers=true&query=select+sleepEachRow(0.01)from+numbers(10)+FORMAT+JSON" 2>&1|grep -Eo " Content-Type:.*$")|sort -u +(seq 1 200| xargs -n1 -P0 -Ixxx curl -v "${CLICKHOUSE_URL}/?http_headers_progress_interval_ms=1&send_progress_in_http_headers=true&query=select+sleepEachRow(0.01)from+numbers(10)+FORMAT+TSV" 2>&1|grep -Eo " Content-Type:.*$")|sort -u + From d9520eefb5c02a186930a050888ff6603c8edc4d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Dec 2023 22:53:09 +0300 Subject: [PATCH 079/110] Revert "Revert "Update Sentry"" --- contrib/sentry-native | 2 +- contrib/sentry-native-cmake/CMakeLists.txt | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/contrib/sentry-native b/contrib/sentry-native index ae10fb8c224..d1f0883e020 160000 --- a/contrib/sentry-native +++ b/contrib/sentry-native @@ -1 +1 @@ -Subproject commit ae10fb8c224c3f41571446e1ed7fd57b9e5e366b +Subproject commit d1f0883e020f5c1b182c2b36ea26ea9c6fa64e39 diff --git a/contrib/sentry-native-cmake/CMakeLists.txt b/contrib/sentry-native-cmake/CMakeLists.txt index 377f955f856..6364e75db28 100644 --- a/contrib/sentry-native-cmake/CMakeLists.txt +++ b/contrib/sentry-native-cmake/CMakeLists.txt @@ -13,6 +13,7 @@ set (SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/sentry-native") set (SRCS ${SRC_DIR}/vendor/mpack.c + ${SRC_DIR}/vendor/stb_sprintf.c ${SRC_DIR}/src/sentry_alloc.c ${SRC_DIR}/src/sentry_backend.c ${SRC_DIR}/src/sentry_core.c @@ -21,6 +22,7 @@ set (SRCS ${SRC_DIR}/src/sentry_json.c ${SRC_DIR}/src/sentry_logger.c ${SRC_DIR}/src/sentry_options.c + ${SRC_DIR}/src/sentry_os.c ${SRC_DIR}/src/sentry_random.c ${SRC_DIR}/src/sentry_ratelimiter.c ${SRC_DIR}/src/sentry_scope.c @@ -29,6 +31,7 @@ set (SRCS ${SRC_DIR}/src/sentry_string.c ${SRC_DIR}/src/sentry_sync.c ${SRC_DIR}/src/sentry_transport.c + ${SRC_DIR}/src/sentry_tracing.c ${SRC_DIR}/src/sentry_utils.c ${SRC_DIR}/src/sentry_uuid.c ${SRC_DIR}/src/sentry_value.c From ce995989cac8d8777c3625a2333aa7e5d12c163d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Dec 2023 20:55:11 +0100 Subject: [PATCH 080/110] Fix test --- tests/queries/0_stateless/02864_statistic_exception.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02864_statistic_exception.sql b/tests/queries/0_stateless/02864_statistic_exception.sql index c37f6b1ce06..092fa9bda85 100644 --- a/tests/queries/0_stateless/02864_statistic_exception.sql +++ b/tests/queries/0_stateless/02864_statistic_exception.sql @@ -47,7 +47,7 @@ ALTER TABLE t1 MATERIALIZE STATISTIC b TYPE tdigest; -- { serverError ILLEGAL_ST ALTER TABLE t1 ADD STATISTIC a TYPE tdigest; ALTER TABLE t1 ADD STATISTIC b TYPE tdigest; -ALTER TABLE t1 MODIFY COLUMN a Float64 TTL now() + INTERVAL 1 MONTH; +ALTER TABLE t1 MODIFY COLUMN a Float64 TTL toDateTime(b) + INTERVAL 1 MONTH; ALTER TABLE t1 MODIFY COLUMN a Int64; -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN } DROP TABLE t1; From 73123e0ab2a5cac69fa730608e92cca9d0efe914 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 8 Dec 2023 16:28:57 -0400 Subject: [PATCH 081/110] test for #33308 --- ..._type_with_http_headers_progress.reference | 25 +++++++++++++++++-- ...content_type_with_http_headers_progress.sh | 9 ++++--- 2 files changed, 29 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02935_http_content_type_with_http_headers_progress.reference b/tests/queries/0_stateless/02935_http_content_type_with_http_headers_progress.reference index 2f48f1c5c00..d847d4e22ef 100644 --- a/tests/queries/0_stateless/02935_http_content_type_with_http_headers_progress.reference +++ b/tests/queries/0_stateless/02935_http_content_type_with_http_headers_progress.reference @@ -1,3 +1,24 @@ - Content-Type: text/csv; charset=UTF-8; header=absent - Content-Type: application/json; charset=UTF-8 +TSV Content-Type: text/tab-separated-values; charset=UTF-8 +TabSeparatedWithNamesAndTypes + Content-Type: text/tab-separated-values; charset=UTF-8 +CSV + Content-Type: text/csv; charset=UTF-8; header=absent +CSVWithNames + Content-Type: text/csv; charset=UTF-8; header=present +Null + Content-Type: text/plain; charset=UTF-8 +Native + Content-Type: application/octet-stream +RowBinary + Content-Type: application/octet-stream +JSONStrings + Content-Type: application/json; charset=UTF-8 +JSON + Content-Type: application/json; charset=UTF-8 +JSONEachRow + Content-Type: application/x-ndjson; charset=UTF-8 +Values + Content-Type: text/plain; charset=UTF-8 +Vertical + Content-Type: text/plain; charset=UTF-8 diff --git a/tests/queries/0_stateless/02935_http_content_type_with_http_headers_progress.sh b/tests/queries/0_stateless/02935_http_content_type_with_http_headers_progress.sh index 570acc40ea9..cd705650bad 100755 --- a/tests/queries/0_stateless/02935_http_content_type_with_http_headers_progress.sh +++ b/tests/queries/0_stateless/02935_http_content_type_with_http_headers_progress.sh @@ -4,7 +4,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -(seq 1 200| xargs -n1 -P0 -Ixxx curl -v "${CLICKHOUSE_URL}/?http_headers_progress_interval_ms=1&send_progress_in_http_headers=true&query=select+sleepEachRow(0.01)from+numbers(10)+FORMAT+CSV" 2>&1|grep -Eo " Content-Type:.*$")|sort -u -(seq 1 200| xargs -n1 -P0 -Ixxx curl -v "${CLICKHOUSE_URL}/?http_headers_progress_interval_ms=1&send_progress_in_http_headers=true&query=select+sleepEachRow(0.01)from+numbers(10)+FORMAT+JSON" 2>&1|grep -Eo " Content-Type:.*$")|sort -u -(seq 1 200| xargs -n1 -P0 -Ixxx curl -v "${CLICKHOUSE_URL}/?http_headers_progress_interval_ms=1&send_progress_in_http_headers=true&query=select+sleepEachRow(0.01)from+numbers(10)+FORMAT+TSV" 2>&1|grep -Eo " Content-Type:.*$")|sort -u +for frmt in TSV TabSeparatedWithNamesAndTypes CSV CSVWithNames Null Native RowBinary JSONStrings JSON JSONEachRow Values Vertical +do + echo $frmt + url="${CLICKHOUSE_URL}/?http_headers_progress_interval_ms=1&send_progress_in_http_headers=true&query=select+sleepEachRow(0.01)from+numbers(10)+FORMAT+${frmt}" + (seq 1 200| xargs -n1 -P0 -Ixxx curl -Ss -v -o /dev/null ${url} 2>&1|grep -Eo " Content-Type:.*$")|strings|sort -u +done From bb084466c4af93fb6ddd8ba5223ef1ddf89c0c71 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Dec 2023 23:51:12 +0300 Subject: [PATCH 082/110] Update build-cross-osx.md --- docs/en/development/build-cross-osx.md | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/docs/en/development/build-cross-osx.md b/docs/en/development/build-cross-osx.md index a04d676e92d..eddf24448c1 100644 --- a/docs/en/development/build-cross-osx.md +++ b/docs/en/development/build-cross-osx.md @@ -28,18 +28,20 @@ sudo apt-get install clang-17 Let’s remember the path where we install `cctools` as ${CCTOOLS} ``` bash +mkdir ~/cctools export CCTOOLS=$(cd ~/cctools && pwd) -mkdir ${CCTOOLS} cd ${CCTOOLS} -git clone --depth=1 https://github.com/tpoechtrager/apple-libtapi.git +git clone https://github.com/tpoechtrager/apple-libtapi.git cd apple-libtapi +git checkout 15dfc2a8c9a2a89d06ff227560a69f5265b692f9 INSTALLPREFIX=${CCTOOLS} ./build.sh ./install.sh cd .. -git clone --depth=1 https://github.com/tpoechtrager/cctools-port.git +git clone https://github.com/tpoechtrager/cctools-port.git cd cctools-port/cctools +git checkout 2a3e1c2a6ff54a30f898b70cfb9ba1692a55fad7 ./configure --prefix=$(readlink -f ${CCTOOLS}) --with-libtapi=$(readlink -f ${CCTOOLS}) --target=x86_64-apple-darwin make install ``` From 57b740f84b512ab0b91db5ea243ce058e7f67dfe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Dec 2023 22:14:24 +0100 Subject: [PATCH 083/110] Disable iconv in libxml2 --- .../libxml2-cmake/linux_x86_64/include/libxml/xmlversion.h | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/contrib/libxml2-cmake/linux_x86_64/include/libxml/xmlversion.h b/contrib/libxml2-cmake/linux_x86_64/include/libxml/xmlversion.h index 9eabfaa50c8..c2faeb47cb1 100644 --- a/contrib/libxml2-cmake/linux_x86_64/include/libxml/xmlversion.h +++ b/contrib/libxml2-cmake/linux_x86_64/include/libxml/xmlversion.h @@ -270,7 +270,7 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); * * Whether iconv support is available */ -#if 1 +#if 0 #define LIBXML_ICONV_ENABLED #endif @@ -499,5 +499,3 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); } #endif /* __cplusplus */ #endif - - From 89cce448140d8af703569b5fd3bcb2bd2aa860e0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Dec 2023 23:20:20 +0100 Subject: [PATCH 084/110] Cut useless code --- contrib/sentry-native | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/sentry-native b/contrib/sentry-native index d1f0883e020..bc359f86cbf 160000 --- a/contrib/sentry-native +++ b/contrib/sentry-native @@ -1 +1 @@ -Subproject commit d1f0883e020f5c1b182c2b36ea26ea9c6fa64e39 +Subproject commit bc359f86cbf0f73f6fd4b6bfb4ede0c1f8c9400f From 64ae04095406c93abd5974dfd63e1144249fab9e Mon Sep 17 00:00:00 2001 From: Bharat Nallan Chakravarthy Date: Fri, 8 Dec 2023 07:37:38 -0800 Subject: [PATCH 085/110] fix test --- src/Functions/FunctionsHashingSSL.cpp | 17 ++++++++++++++++- ...l_new_functions_must_be_documented.reference | 1 - 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsHashingSSL.cpp b/src/Functions/FunctionsHashingSSL.cpp index 7fe7530d9da..b716a11f9c3 100644 --- a/src/Functions/FunctionsHashingSSL.cpp +++ b/src/Functions/FunctionsHashingSSL.cpp @@ -22,7 +22,22 @@ REGISTER_FUNCTION(HashingSSL) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(FunctionDocumentation{ + .description = R"(Calculates the SHA512_256 hash of the given string.)", + .syntax = "SELECT SHA512_256(s);", + .arguments = {{"s", "The input [String](../../sql-reference/data-types/string.md)."}}, + .returned_value + = "The SHA512_256 hash of the given input string returned as a [FixedString](../../sql-reference/data-types/fixedstring.md).", + .examples + = {{"", + "SELECT HEX(SHA512_256('abc'));", + R"( +┌─hex(SHA512_256('abc'))───────────────────────────────────────────┐ +│ 53048E2681941EF99B2E29B76B4C7DABE4C2D0C634FC6D46E0E2F13107E7AF23 │ +└──────────────────────────────────────────────────────────────────┘ + )" + }} + }); } } diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 91e1c3544b0..379eea4dbbb 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -51,7 +51,6 @@ LpNormalize MACNumToString MACStringToNum MACStringToOUI -SHA512_256 URLHash URLHierarchy URLPathHierarchy From 0d1bdff3476e2e4406b327ccd7d0bee3bffa8939 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Dec 2023 16:57:42 +0100 Subject: [PATCH 086/110] Deprecate LIVE VIEW --- .../sql-reference/statements/create/view.md | 147 +----------------- 1 file changed, 3 insertions(+), 144 deletions(-) diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 2a8d6788889..56828745048 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -90,152 +90,11 @@ Views look the same as normal tables. For example, they are listed in the result To delete a view, use [DROP VIEW](../../../sql-reference/statements/drop.md#drop-view). Although `DROP TABLE` works for VIEWs as well. -## Live View [Experimental] +## Live View [Deprecated] -:::note -This is an experimental feature that may change in backwards-incompatible ways in the future releases. Enable usage of live views and `WATCH` query using [allow_experimental_live_view](../../../operations/settings/settings.md#allow-experimental-live-view) setting. Input the command `set allow_experimental_live_view = 1`. -::: +This feature is deprecated and will be removed in the future. -```sql -CREATE LIVE VIEW [IF NOT EXISTS] [db.]table_name [WITH REFRESH [value_in_sec]] AS SELECT ... -``` - -Live views store result of the corresponding [SELECT](../../../sql-reference/statements/select/index.md) query and are updated any time the result of the query changes. Query result as well as partial result needed to combine with new data are stored in memory providing increased performance for repeated queries. Live views can provide push notifications when query result changes using the [WATCH](../../../sql-reference/statements/watch.md) query. - -Live views are triggered by insert into the innermost table specified in the query. - -Live views work similarly to how a query in a distributed table works. But instead of combining partial results from different servers they combine partial result from current data with partial result from the new data. When a live view query includes a subquery then the cached partial result is only stored for the innermost subquery. - -:::info -- [Table function](../../../sql-reference/table-functions/index.md) is not supported as the innermost table. -- Tables that do not have inserts such as a [dictionary](../../../sql-reference/dictionaries/index.md), [system table](../../../operations/system-tables/index.md), a [normal view](#normal), or a [materialized view](#materialized) will not trigger a live view. -- Only queries where one can combine partial result from the old data plus partial result from the new data will work. Live view will not work for queries that require the complete data set to compute the final result or aggregations where the state of the aggregation must be preserved. -- Does not work with replicated or distributed tables where inserts are performed on different nodes. -- Can't be triggered by multiple tables. - -See [WITH REFRESH](#live-view-with-refresh) to force periodic updates of a live view that in some cases can be used as a workaround. -::: - -### Monitoring Live View Changes - -You can monitor changes in the `LIVE VIEW` query result using [WATCH](../../../sql-reference/statements/watch.md) query. - -```sql -WATCH [db.]live_view -``` - -**Example:** - -```sql -CREATE TABLE mt (x Int8) Engine = MergeTree ORDER BY x; -CREATE LIVE VIEW lv AS SELECT sum(x) FROM mt; -``` -Watch a live view while doing a parallel insert into the source table. - -```sql -WATCH lv; -``` - -```bash -┌─sum(x)─┬─_version─┐ -│ 1 │ 1 │ -└────────┴──────────┘ -┌─sum(x)─┬─_version─┐ -│ 3 │ 2 │ -└────────┴──────────┘ -┌─sum(x)─┬─_version─┐ -│ 6 │ 3 │ -└────────┴──────────┘ -``` - -```sql -INSERT INTO mt VALUES (1); -INSERT INTO mt VALUES (2); -INSERT INTO mt VALUES (3); -``` - -Or add [EVENTS](../../../sql-reference/statements/watch.md#events-clause) clause to just get change events. - -```sql -WATCH [db.]live_view EVENTS; -``` - -**Example:** - -```sql -WATCH lv EVENTS; -``` - -```bash -┌─version─┐ -│ 1 │ -└─────────┘ -┌─version─┐ -│ 2 │ -└─────────┘ -┌─version─┐ -│ 3 │ -└─────────┘ -``` - -You can execute [SELECT](../../../sql-reference/statements/select/index.md) query on a live view in the same way as for any regular view or a table. If the query result is cached it will return the result immediately without running the stored query on the underlying tables. - -```sql -SELECT * FROM [db.]live_view WHERE ... -``` - -### Force Live View Refresh - -You can force live view refresh using the `ALTER LIVE VIEW [db.]table_name REFRESH` statement. - -### WITH REFRESH Clause - -When a live view is created with a `WITH REFRESH` clause then it will be automatically refreshed after the specified number of seconds elapse since the last refresh or trigger. - -```sql -CREATE LIVE VIEW [db.]table_name WITH REFRESH [value_in_sec] AS SELECT ... -``` - -If the refresh value is not specified then the value specified by the [periodic_live_view_refresh](../../../operations/settings/settings.md#periodic-live-view-refresh) setting is used. - -**Example:** - -```sql -CREATE LIVE VIEW lv WITH REFRESH 5 AS SELECT now(); -WATCH lv -``` - -```bash -┌───────────────now()─┬─_version─┐ -│ 2021-02-21 08:47:05 │ 1 │ -└─────────────────────┴──────────┘ -┌───────────────now()─┬─_version─┐ -│ 2021-02-21 08:47:10 │ 2 │ -└─────────────────────┴──────────┘ -┌───────────────now()─┬─_version─┐ -│ 2021-02-21 08:47:15 │ 3 │ -└─────────────────────┴──────────┘ -``` - -```sql -WATCH lv -``` - -``` -Code: 60. DB::Exception: Received from localhost:9000. DB::Exception: Table default.lv does not exist.. -``` - -### Live View Usage - -Most common uses of live view tables include: - -- Providing push notifications for query result changes to avoid polling. -- Caching results of most frequent queries to provide immediate query results. -- Watching for table changes and triggering a follow-up select queries. -- Watching metrics from system tables using periodic refresh. - -**See Also** -- [ALTER LIVE VIEW](../alter/view.md#alter-live-view) +For your convenience, the old documentation is located [here](https://pastila.nl/?00f32652/fdf07272a7b54bda7e13b919264e449f.md) ## Window View [Experimental] From 55d5a3affa2189ae1957680428706cd3d0874785 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 28 Jul 2023 09:47:09 +0200 Subject: [PATCH 087/110] Correctly wait background threads There are some places that make it possible to trigger use-after-free from threads because some global variable had been destroyed before, for example some logger. Signed-off-by: Azat Khuzhin --- programs/keeper/Keeper.cpp | 7 +++++++ programs/server/Server.cpp | 8 ++++++++ src/Common/ThreadPool.cpp | 7 +++++++ src/Common/ThreadPool.h | 3 +++ 4 files changed, 25 insertions(+) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index e04e669abae..1acf7e39b04 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -328,6 +328,13 @@ try config().getUInt("max_thread_pool_free_size", 1000), config().getUInt("thread_pool_queue_size", 10000) ); + /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). + SCOPE_EXIT({ + Stopwatch watch; + LOG_INFO(log, "Waiting for background threads"); + GlobalThreadPool::instance().shutdown(); + LOG_INFO(log, "Background threads finished in {} ms", watch.elapsedMilliseconds()); + }); static ServerErrorHandler error_handler; Poco::ErrorHandler::set(&error_handler); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index de0cfb9b9fa..ed9cb2fafa9 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -697,6 +697,14 @@ try server_settings.max_thread_pool_size, server_settings.max_thread_pool_free_size, server_settings.thread_pool_queue_size); + /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). + SCOPE_EXIT({ + Stopwatch watch; + LOG_INFO(log, "Waiting for background threads"); + GlobalThreadPool::instance().shutdown(); + LOG_INFO(log, "Background threads finished in {} ms", watch.elapsedMilliseconds()); + }); + #if USE_AZURE_BLOB_STORAGE /// It makes sense to deinitialize libxml after joining of all threads diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 8cba13373b9..565affb0c65 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -500,3 +500,10 @@ GlobalThreadPool & GlobalThreadPool::instance() return *the_instance; } +void GlobalThreadPool::shutdown() +{ + if (the_instance) + { + the_instance->finalize(); + } +} diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index c8eefedd838..3117509ab8f 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -109,6 +109,8 @@ public: void addOnDestroyCallback(OnDestroyCallback && callback); private: + friend class GlobalThreadPool; + mutable std::mutex mutex; std::condition_variable job_finished; std::condition_variable new_job_or_shutdown; @@ -205,6 +207,7 @@ class GlobalThreadPool : public FreeThreadPool, private boost::noncopyable public: static void initialize(size_t max_threads = 10000, size_t max_free_threads = 1000, size_t queue_size = 10000); static GlobalThreadPool & instance(); + static void shutdown(); }; From b7edde3621aa49c6a8c34c4a9b85e1fdba6c212d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 3 Aug 2023 21:28:37 +0200 Subject: [PATCH 088/110] Shutdown embedded keeper even if it was not started successfully Since even when it does not, there are still few threads left that should be joined (request, response, snapshot, snapshot s3): There are some problems with keeper and invalid config, that got stuck in: libc.so.6`___lldb_unnamed_symbol3457 + 231 libc.so.6`pthread_cond_timedwait + 571 clickhouse`std::__1::__libcpp_condvar_timedwait[abi:v15000](__cv=0x00007f475c41a708, __m=0x00007f475c41a6b0, __ts=0x00007f45a99f1440) at __threading_support:341:10 clickhouse`std::__1::condition_variable::__do_timed_wait(this=0x00007f475c41a708, lk=0x00007f45a99f1738, tp=time_point > > @ 0x00007f45a99f1488) at condition_variable.cpp:76:14 clickhouse`std::__1::cv_status std::__1::condition_variable::wait_for >(this=0x00007f475c41a708, __lk=0x00007f45a99f1738, __d=0x00007f45a99f1590) at __mutex_base:457:9 clickhouse`void std::__1::condition_variable::__do_timed_wait(this=0x00007f475c41a708, __lk=0x00007f45a99f1738, __tp=time_point > > @ 0x00007f45a99f15a8) at __mutex_base:512:5 clickhouse`std::__1::cv_status std::__1::condition_variable::wait_until > >(this=0x00007f475c41a708, __lk=0x00007f45a99f1738, __t=0x00007f45a99f1698) at __mutex_base:415:5 clickhouse`bool std::__1::condition_variable::wait_until >, bool ConcurrentBoundedQueue::popImpl(DB::KeeperStorage::RequestForSession&, std::__1::optional)::'lambda'()>(this=0x00007f475c41a708, __lk=0x00007f45a99f1738, __t=0x00007f45a99f1698, __pred=(unnamed class) @ 0x00007f45a99f1650)::'lambda'()) at __mutex_base:427:13 clickhouse`bool std::__1::condition_variable::wait_for[abi:v15000], bool ConcurrentBoundedQueue::popImpl(DB::KeeperStorage::RequestForSession&, std::__1::optional)::'lambda'()>(this=0x00007f475c41a708, __lk=0x00007f45a99f1738, __d=0x00007f45a99f1720, __pred=(unnamed class) @ 0x00007f45a99f16b8)::'lambda'()) at __mutex_base:471:12 clickhouse`bool ConcurrentBoundedQueue::popImpl(this=0x00007f475c41a680, x=0x00007f45a99f1ba0, timeout_milliseconds= Has Value=true ) at ConcurrentBoundedQueue.h:75:50 clickhouse`ConcurrentBoundedQueue::tryPop(this=0x00007f475c41a680, x=0x00007f45a99f1ba0, milliseconds=5000) at ConcurrentBoundedQueue.h:161:16 clickhouse`DB::KeeperDispatcher::requestThread(this=0x00007f475c8fac18) at KeeperDispatcher.cpp:88:33 clickhouse`DB::KeeperDispatcher::initialize(this=0x00007f45a99f2020)::$_0::operator()() const at KeeperDispatcher.cpp:349:52 clickhouse`decltype(__f=0x00007f45a99f2020)::$_0&>()()) std::__1::__invoke[abi:v15000] const&)::$_0&>(DB::KeeperDispatcher::initialize(Poco::Util::AbstractConfiguration const&, bool, bool, std::__1::shared_ptr const&)::$_0&) at invoke.h:394:23 clickhouse`decltype(__f=0x00007f45a99f2020, __t=size=0, (null)=__tuple_indices<> @ 0x00007f45a99f1d28) std::__1::__apply_tuple_impl[abi:v15000] const&)::$_0&, std::__1::tuple<>&>(DB::KeeperDispatcher::initialize(Poco::Util::AbstractConfiguration const&, bool, bool, std::__1::shared_ptr const&)::$_0&, std::__1::tuple<>&, std::__1::__tuple_indices<>) at tuple:1789:1 clickhouse`decltype(__f=0x00007f45a99f2020, __t=size=0) std::__1::apply[abi:v15000] const&)::$_0&, std::__1::tuple<>&>(DB::KeeperDispatcher::initialize(Poco::Util::AbstractConfiguration const&, bool, bool, std::__1::shared_ptr const&)::$_0&, std::__1::tuple<>&) at tuple:1798:1 clickhouse`ThreadFromGlobalPoolImpl::ThreadFromGlobalPoolImpl(DB::KeeperDispatcher::initialize(Poco::Util::AbstractConfiguration const&, bool, bool, std::__1::shared_ptr const&)::$_0&&)::'lambda'()::operator()() at ThreadPool.h:245:13 clickhouse`decltype(__f=0x00007f475c9703c0)::$_0>()()) std::__1::__invoke[abi:v15000]::ThreadFromGlobalPoolImpl const&)::$_0>(DB::KeeperDispatcher::initialize(Poco::Util::AbstractConfiguration const&, bool, bool, std::__1::shared_ptr const&)::$_0&&)::'lambda'()&>(DB::KeeperDispatcher::initialize(Poco::Util::AbstractConfiguration const&, bool, bool, std::__1::shared_ptr const&)::$_0&&) at invoke.h:394:23 clickhouse`void std::__1::__invoke_void_return_wrapper::__call::ThreadFromGlobalPoolImpl(DB::KeeperDispatcher::initialize(Poco::Util::AbstractConfiguration const&, bool, bool, std::__1::shared_ptr const&)::$_0&&)::'lambda'()&>(ThreadFromGlobalPoolImpl::ThreadFromGlobalPoolImpl const&)::$_0>(DB::KeeperDispatcher::initialize(Poco::Util::AbstractConfiguration const&, bool, bool, std::__1::shared_ptr const&)::$_0&&)::'lambda'()&) at invoke.h:479:9 clickhouse`std::__1::__function::__default_alloc_func::ThreadFromGlobalPoolImpl const&)::$_0>(DB::KeeperDispatcher::initialize(Poco::Util::AbstractConfiguration const&, bool, bool, std::__1::shared_ptr const&)::$_0&&)::'lambda'(), void ()>::operator(this=0x00007f475c9703c0)[abi:v15000]() at function.h:235:12 clickhouse`void std::__1::__function::__policy_invoker::__call_impl::ThreadFromGlobalPoolImpl(DB::KeeperDispatcher::initialize(Poco::Util::AbstractConfiguration const&, bool, bool, std::__1::shared_ptr const&)::$_0&&)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) at function.h:716:16 clickhouse`std::__1::__function::__policy_func::operator(this=0x00007f45a99f22d8)[abi:v15000]() const at function.h:848:16 clickhouse`std::__1::function::operator(this=0x00007f45a99f22d8)() const at function.h:1187:12 clickhouse`ThreadPoolImpl::worker(this=0x00007f475c8229c0, thread_it=std::__1::list >::iterator @ 0x00007f45a99f23a8) at ThreadPool.cpp:427:13 clickhouse`void ThreadPoolImpl::scheduleImpl(this=0x00007f45cea0c8a8)>, Priority, std::__1::optional, bool)::'lambda0'()::operator()() const at ThreadPool.cpp:180:73 clickhouse`decltype(__f=0x00007f45cea0c8a8)()) std::__1::__invoke[abi:v15000]::scheduleImpl(std::__1::function, Priority, std::__1::optional, bool)::'lambda0'()>(void&&) at invoke.h:394:23 clickhouse`void std::__1::__thread_execute[abi:v15000] >, void ThreadPoolImpl::scheduleImpl(std::__1::function, Priority, std::__1::optional, bool)::'lambda0'()>(__t=size=2, (null)=__tuple_indices<> @ 0x00007f45a99f2408)>, Priority, std::__1::optional, bool)::'lambda0'()>&, std::__1::__tuple_indices<>) at thread:284:5 clickhouse`void* std::__1::__thread_proxy[abi:v15000] >, void ThreadPoolImpl::scheduleImpl(std::__1::function, Priority, std::__1::optional, bool)::'lambda0'()> >(__vp=0x00007f45cea0c8a0) at thread:295:5 libc.so.6`___lldb_unnamed_symbol3481 + 755 libc.so.6`___lldb_unnamed_symbol3865 + 11 This should fix the problem with test_keeper_incorrect_config/test.py::test_invalid_configs test. CI: https://s3.amazonaws.com/clickhouse-test-reports/52717/72b1052f7c2d453308262924e767ab8dc2206933/integration_tests__asan__[1_6].html Signed-off-by: Azat Khuzhin --- programs/server/Server.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index ed9cb2fafa9..bb16670f0b2 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1645,10 +1645,10 @@ try LOG_INFO(log, "Closed connections to servers for tables. But {} remain. Probably some tables of other users cannot finish their connections after context shutdown.", current_connections); else LOG_INFO(log, "Closed connections to servers for tables."); - - global_context->shutdownKeeperDispatcher(); } + global_context->shutdownKeeperDispatcher(); + /// Wait server pool to avoid use-after-free of destroyed context in the handlers server_pool.joinAll(); From b6e3e5691cadbe1a60a6202ebdecc9da18b0823b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Dec 2023 18:58:35 +0100 Subject: [PATCH 089/110] Fix Musl build --- cmake/target.cmake | 19 +++++-------------- contrib/llvm-project | 2 +- 2 files changed, 6 insertions(+), 15 deletions(-) diff --git a/cmake/target.cmake b/cmake/target.cmake index 887f79bf24e..c627fe73c94 100644 --- a/cmake/target.cmake +++ b/cmake/target.cmake @@ -73,19 +73,10 @@ if (CMAKE_CROSSCOMPILING) message (FATAL_ERROR "Trying to cross-compile to unsupported system: ${CMAKE_SYSTEM_NAME}!") endif () - if (USE_MUSL) - # use of undeclared identifier 'PTHREAD_RECURSIVE_MUTEX_INITIALIZER_NP' - set (ENABLE_SENTRY OFF CACHE INTERNAL "") - set (ENABLE_ODBC OFF CACHE INTERNAL "") - set (ENABLE_GRPC OFF CACHE INTERNAL "") - set (ENABLE_HDFS OFF CACHE INTERNAL "") - set (ENABLE_EMBEDDED_COMPILER OFF CACHE INTERNAL "") - # use of drand48_data - set (ENABLE_AZURE_BLOB_STORAGE OFF CACHE INTERNAL "") - endif () - - # Don't know why but CXX_STANDARD doesn't work for cross-compilation - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++20") - message (STATUS "Cross-compiling for target: ${CMAKE_CXX_COMPILE_TARGET}") endif () + +if (USE_MUSL) + # Does not work for unknown reason + set (ENABLE_RUST OFF CACHE INTERNAL "") +endif () diff --git a/contrib/llvm-project b/contrib/llvm-project index e7b8befca85..1834e42289c 160000 --- a/contrib/llvm-project +++ b/contrib/llvm-project @@ -1 +1 @@ -Subproject commit e7b8befca85c8b847614432dba250c22d35fbae0 +Subproject commit 1834e42289c58402c804a87be4d489892b88f3ec From f0df110b15cce688ad09514a6915e061d89d8e99 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Dec 2023 19:30:53 +0100 Subject: [PATCH 090/110] Enable some libraries --- cmake/target.cmake | 2 -- 1 file changed, 2 deletions(-) diff --git a/cmake/target.cmake b/cmake/target.cmake index c627fe73c94..1680715d15f 100644 --- a/cmake/target.cmake +++ b/cmake/target.cmake @@ -42,10 +42,8 @@ if (CMAKE_CROSSCOMPILING) if (ARCH_AARCH64) # FIXME: broken dependencies set (ENABLE_GRPC OFF CACHE INTERNAL "") - set (ENABLE_SENTRY OFF CACHE INTERNAL "") elseif (ARCH_PPC64LE) set (ENABLE_GRPC OFF CACHE INTERNAL "") - set (ENABLE_SENTRY OFF CACHE INTERNAL "") elseif (ARCH_RISCV64) # RISC-V support is preliminary set (GLIBC_COMPATIBILITY OFF CACHE INTERNAL "") From 179a0a2e8f3789b23578b73565af8c26a1578cdc Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 9 Dec 2023 13:50:54 +0100 Subject: [PATCH 091/110] Fixed condition and add more chasserts for consistency checks. --- src/Common/RWLock.cpp | 28 +++++++++++++++++++++------- 1 file changed, 21 insertions(+), 7 deletions(-) diff --git a/src/Common/RWLock.cpp b/src/Common/RWLock.cpp index a3ba9523f6c..c8a5c692494 100644 --- a/src/Common/RWLock.cpp +++ b/src/Common/RWLock.cpp @@ -163,14 +163,14 @@ RWLockImpl::getLock(RWLockImpl::Type type, const String & query_id, const std::c else { /// We don't always add a group to readers_queue here because multiple readers can use the same group. - /// We can reuse the last group if we're in write phase now, or if the last group didn't get ownership yet, + /// We can reuse the last group if the last group didn't get ownership yet, /// or even if it got ownership but there are no writers waiting in writers_queue. - bool can_use_last_group = !readers_queue.empty() && - ((rdlock_owner == readers_queue.end()) || !rdlock_owner->ownership || writers_queue.empty()); + bool can_use_last_group = !readers_queue.empty() && (!readers_queue.back().ownership || writers_queue.empty()); if (!can_use_last_group) readers_queue.emplace_back(type); /// SM1: may throw (nothing to roll back) } + GroupsContainer::iterator it_group = (type == Type::Write) ? std::prev(writers_queue.end()) : std::prev(readers_queue.end()); @@ -215,6 +215,9 @@ RWLockImpl::getLock(RWLockImpl::Type type, const String & query_id, const std::c } } + /// Our group must be an owner here. + chassert(it_group->ownership); + if (request_has_query_id) { try @@ -250,17 +253,28 @@ RWLockImpl::getLock(RWLockImpl::Type type, const String & query_id, const std::c * it is guaranteed that all three steps have been executed successfully and the resulting state is consistent. * With the mutex locked the order of steps to restore the lock's state can be arbitrary * - * We do not employ try-catch: if something bad happens, there is nothing we can do =( + * We do not employ try-catch: if something bad happens and chassert() is disabled, there is nothing we can do + * (we can't throw an exception here because RWLockImpl::unlock() is called from the destructor ~LockHolderImpl). */ void RWLockImpl::unlock(GroupsContainer::iterator group_it, const String & query_id) noexcept { std::lock_guard state_lock(internal_state_mtx); - /// All of these are Undefined behavior and nothing we can do! - if (rdlock_owner == readers_queue.end() && wrlock_owner == writers_queue.end()) + /// Our group must be an owner here. + if (!group_it->ownership) + { + chassert(false && "RWLockImpl::unlock() is called for a non-owner group"); return; - if (wrlock_owner != writers_queue.end() && group_it != wrlock_owner) + } + + /// Check consistency. + if ((group_it->type == Read) + ? !(rdlock_owner != readers_queue.end() && wrlock_owner == writers_queue.end()) + : !(wrlock_owner != writers_queue.end() && rdlock_owner == readers_queue.end() && group_it == wrlock_owner)) + { + chassert(false && "RWLockImpl::unlock() found the rwlock inconsistent"); return; + } /// If query_id is not empty it must be listed in parent->owner_queries if (query_id != NO_QUERY) From f3def573c62e36b1fa40691584ab0de7ecfa13ff Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Dec 2023 21:54:09 +0100 Subject: [PATCH 092/110] Switch Fast Test to Musl --- docker/test/fasttest/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index d3695ba2613..5af05034415 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -16,7 +16,7 @@ export LLVM_VERSION=${LLVM_VERSION:-17} # it being undefined. Also read it as array so that we can pass an empty list # of additional variable to cmake properly, and it doesn't generate an extra # empty parameter. -# Read it as CMAKE_FLAGS to not lose exported FASTTEST_CMAKE_FLAGS on subsequential launch +# Read it as CMAKE_FLAGS to not lose exported FASTTEST_CMAKE_FLAGS on subsequent launch read -ra CMAKE_FLAGS <<< "${FASTTEST_CMAKE_FLAGS:-}" # Run only matching tests. @@ -197,7 +197,7 @@ function run_cmake ( cd "$FASTTEST_BUILD" - cmake "$FASTTEST_SOURCE" -DCMAKE_CXX_COMPILER="clang++-${LLVM_VERSION}" -DCMAKE_C_COMPILER="clang-${LLVM_VERSION}" "${CMAKE_LIBS_CONFIG[@]}" "${CMAKE_FLAGS[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/cmake_log.txt" + cmake "$FASTTEST_SOURCE" -DCMAKE_CXX_COMPILER="clang++-${LLVM_VERSION}" -DCMAKE_C_COMPILER="clang-${LLVM_VERSION}" -DCMAKE_TOOLCHAIN_FILE="${FASTTEST_SOURCE}/cmake/linux/toolchain-x86_64-musl.cmake" "${CMAKE_LIBS_CONFIG[@]}" "${CMAKE_FLAGS[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/cmake_log.txt" ) } From 472a31e5818d5e8b0e215913f323ff6955ec1f80 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 9 Dec 2023 21:27:40 +0100 Subject: [PATCH 093/110] Fix SIGSEGV for aggregation of sparse columns with any() RESPECT NULL Add proper check into any() RESPECT NULL Fixes: https://github.com/ClickHouse/ClickHouse/issues/57709 Fixes: https://github.com/ClickHouse/ClickHouse/pull/57189 (cc @Algunenano) CI: https://s3.amazonaws.com/clickhouse-test-reports/52717/7ec9c6a7f7490b5075407410dc193a2534216e15/fuzzer_astfuzzerdebug/report.html Signed-off-by: Azat Khuzhin --- src/AggregateFunctions/AggregateFunctionAny.cpp | 2 +- .../02941_any_RESPECT_NULL_sparse_column.reference | 1 + .../0_stateless/02941_any_RESPECT_NULL_sparse_column.sql | 5 +++++ 3 files changed, 7 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02941_any_RESPECT_NULL_sparse_column.reference create mode 100644 tests/queries/0_stateless/02941_any_RESPECT_NULL_sparse_column.sql diff --git a/src/AggregateFunctions/AggregateFunctionAny.cpp b/src/AggregateFunctions/AggregateFunctionAny.cpp index 15681eca817..a6010ff07c3 100644 --- a/src/AggregateFunctions/AggregateFunctionAny.cpp +++ b/src/AggregateFunctions/AggregateFunctionAny.cpp @@ -110,7 +110,7 @@ public: } } } - else + else if (row_begin < row_end) { size_t pos = First ? row_begin : row_end - 1; add(place, columns, pos, arena); diff --git a/tests/queries/0_stateless/02941_any_RESPECT_NULL_sparse_column.reference b/tests/queries/0_stateless/02941_any_RESPECT_NULL_sparse_column.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/02941_any_RESPECT_NULL_sparse_column.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/02941_any_RESPECT_NULL_sparse_column.sql b/tests/queries/0_stateless/02941_any_RESPECT_NULL_sparse_column.sql new file mode 100644 index 00000000000..df86b740c21 --- /dev/null +++ b/tests/queries/0_stateless/02941_any_RESPECT_NULL_sparse_column.sql @@ -0,0 +1,5 @@ +-- regression for the case when aggregate function will be called with from==to for sparse column +DROP TABLE IF EXISTS data_sparse_column; +CREATE TABLE data_sparse_column (`key` Int64, `value` Int32) ENGINE = MergeTree ORDER BY key; +INSERT INTO data_sparse_column VALUES (1, 0); +SELECT any(value) RESPECT NULLS FROM data_sparse_column; From 7af1c0885daaf1e41470c5fdd92abfc7b6b2befc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 9 Dec 2023 22:47:25 +0100 Subject: [PATCH 094/110] Do not call add*() from addBatchSparseSinglePlace() if there are no rows Signed-off-by: Azat Khuzhin --- src/AggregateFunctions/IAggregateFunction.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index 2a62b76adbb..a8254baac3a 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -549,8 +549,10 @@ public: auto to = std::lower_bound(offsets.begin(), offsets.end(), row_end) - offsets.begin() + 1; size_t num_defaults = (row_end - row_begin) - (to - from); - static_cast(this)->addBatchSinglePlace(from, to, place, &values, arena, -1); - static_cast(this)->addManyDefaults(place, &values, num_defaults, arena); + if (from < to) + static_cast(this)->addBatchSinglePlace(from, to, place, &values, arena, -1); + if (num_defaults > 0) + static_cast(this)->addManyDefaults(place, &values, num_defaults, arena); } void addBatchSinglePlaceNotNull( /// NOLINT From 6ccbc2ea75c39234dff748cfb147bd44d65d3e75 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 3 Aug 2023 21:37:37 +0200 Subject: [PATCH 095/110] Move io_uring reader into the Context from static to make it's thread joinable v2: fix for standalone keeper build CI: https://s3.amazonaws.com/clickhouse-test-reports/52717/72b1052f7c2d453308262924e767ab8dc2206933/stateless_tests__debug__[4_5].html Signed-off-by: Azat Khuzhin --- src/Coordination/Standalone/Context.cpp | 17 +++++++++++++++++ src/Coordination/Standalone/Context.h | 6 ++++++ src/Disks/IO/createReadBufferFromFileBase.cpp | 10 +++++++--- src/Interpreters/Context.cpp | 17 +++++++++++++++++ src/Interpreters/Context.h | 4 ++++ 5 files changed, 51 insertions(+), 3 deletions(-) diff --git a/src/Coordination/Standalone/Context.cpp b/src/Coordination/Standalone/Context.cpp index dba4a8934b9..97a034b22a4 100644 --- a/src/Coordination/Standalone/Context.cpp +++ b/src/Coordination/Standalone/Context.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -62,6 +63,11 @@ struct ContextSharedPart : boost::noncopyable mutable std::unique_ptr asynchronous_local_fs_reader; mutable std::unique_ptr synchronous_local_fs_reader; +#if USE_LIBURING + mutable OnceFlag io_uring_reader_initialized; + mutable std::unique_ptr io_uring_reader; +#endif + mutable OnceFlag threadpool_writer_initialized; mutable std::unique_ptr threadpool_writer; @@ -225,6 +231,17 @@ IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) co } } +#if USE_LIBURING +IOUringReader & Context::getIOURingReader() const +{ + callOnce(shared->io_uring_reader_initialized, [&] { + shared->io_uring_reader = std::make_unique(512); + }); + + return *shared->io_uring_reader; +} +#endif + std::shared_ptr Context::getFilesystemCacheLog() const { return nullptr; diff --git a/src/Coordination/Standalone/Context.h b/src/Coordination/Standalone/Context.h index 7555618233d..5aecf850d7c 100644 --- a/src/Coordination/Standalone/Context.h +++ b/src/Coordination/Standalone/Context.h @@ -20,6 +20,8 @@ #include +#include "config.h" + namespace DB { @@ -28,6 +30,7 @@ class Macros; class FilesystemCacheLog; class FilesystemReadPrefetchesLog; class BlobStorageLog; +class IOUringReader; /// A small class which owns ContextShared. /// We don't use something like unique_ptr directly to allow ContextShared type to be incomplete. @@ -127,6 +130,9 @@ public: ApplicationType getApplicationType() const { return ApplicationType::KEEPER; } IAsynchronousReader & getThreadPoolReader(FilesystemReaderType type) const; +#if USE_LIBURING + IOUringReader & getIOURingReader() const; +#endif std::shared_ptr getAsyncReadCounters() const; ThreadPool & getThreadPoolWriter() const; diff --git a/src/Disks/IO/createReadBufferFromFileBase.cpp b/src/Disks/IO/createReadBufferFromFileBase.cpp index 80dbc8df988..236dd43e9ee 100644 --- a/src/Disks/IO/createReadBufferFromFileBase.cpp +++ b/src/Disks/IO/createReadBufferFromFileBase.cpp @@ -101,12 +101,16 @@ std::unique_ptr createReadBufferFromFileBase( else if (settings.local_fs_method == LocalFSReadMethod::io_uring) { #if USE_LIBURING - static std::shared_ptr reader = std::make_shared(512); - if (!reader->isSupported()) + auto global_context = Context::getGlobalContextInstance(); + if (!global_context) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot obtain io_uring reader (global context not initialized)"); + + auto & reader = global_context->getIOURingReader(); + if (!reader.isSupported()) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "io_uring is not supported by this system"); res = std::make_unique( - *reader, + reader, settings.priority, filename, buffer_size, diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index f976dea1ca5..aa4ef4155a6 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -35,6 +35,7 @@ #include #include #include +#include #include #include #include @@ -309,6 +310,11 @@ struct ContextSharedPart : boost::noncopyable mutable OnceFlag threadpool_writer_initialized; mutable std::unique_ptr threadpool_writer; +#if USE_LIBURING + mutable OnceFlag io_uring_reader_initialized; + mutable std::unique_ptr io_uring_reader; +#endif + mutable ThrottlerPtr replicated_fetches_throttler; /// A server-wide throttler for replicated fetches mutable ThrottlerPtr replicated_sends_throttler; /// A server-wide throttler for replicated sends @@ -4861,6 +4867,17 @@ IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) co } } +#if USE_LIBURING +IOUringReader & Context::getIOURingReader() const +{ + callOnce(shared->io_uring_reader_initialized, [&] { + shared->io_uring_reader = std::make_unique(512); + }); + + return *shared->io_uring_reader; +} +#endif + ThreadPool & Context::getThreadPoolWriter() const { callOnce(shared->threadpool_writer_initialized, [&] { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 0336e82a011..c798837a6f5 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -109,6 +109,7 @@ class AsynchronousInsertLog; class BackupLog; class BlobStorageLog; class IAsynchronousReader; +class IOUringReader; struct MergeTreeSettings; struct InitialAllRangesAnnouncement; struct ParallelReadRequest; @@ -1210,6 +1211,9 @@ public: OrdinaryBackgroundExecutorPtr getCommonExecutor() const; IAsynchronousReader & getThreadPoolReader(FilesystemReaderType type) const; +#if USE_LIBURING + IOUringReader & getIOURingReader() const; +#endif std::shared_ptr getAsyncReadCounters() const; From cc9afe6f6eed0826341819801adadc0b9e4f99b3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 6 Oct 2023 11:15:52 +0200 Subject: [PATCH 096/110] Replace --no-system-tables with loading virtual tables of system database lazily First of all the problem is that that simple 'SELECT 1' cannot be runned without system.one, which makes --no-system-tables almost useless: $ ./clickhouse-debug local --no-system-tables -q "select 1" Code: 81. DB::Exception: Database system does not exist. (UNKNOWN_DATABASE) Secondly there are just too much flags, and this one (--no-system-tables) is too damn specific. This patch should improve startup time of the clickhouse-local almost 3x in debug builds. Signed-off-by: Azat Khuzhin --- .../operations/utilities/clickhouse-local.md | 1 - .../operations/utilities/clickhouse-local.md | 1 - .../operations/utilities/clickhouse-local.md | 1 - programs/local/LocalServer.cpp | 5 +- src/Databases/DatabasesCommon.cpp | 51 ++++++++- src/Databases/DatabasesCommon.h | 12 +++ src/Storages/System/attachSystemTables.cpp | 102 +++++++++--------- src/Storages/System/attachSystemTablesImpl.h | 31 ++++++ 8 files changed, 145 insertions(+), 59 deletions(-) diff --git a/docs/en/operations/utilities/clickhouse-local.md b/docs/en/operations/utilities/clickhouse-local.md index c863282efc1..1dac2d25ea5 100644 --- a/docs/en/operations/utilities/clickhouse-local.md +++ b/docs/en/operations/utilities/clickhouse-local.md @@ -216,7 +216,6 @@ Arguments: - `--logger.level` — Log level. - `--ignore-error` — do not stop processing if a query failed. - `-c`, `--config-file` — path to configuration file in same format as for ClickHouse server, by default the configuration empty. -- `--no-system-tables` — do not attach system tables. - `--help` — arguments references for `clickhouse-local`. - `-V`, `--version` — print version information and exit. diff --git a/docs/ru/operations/utilities/clickhouse-local.md b/docs/ru/operations/utilities/clickhouse-local.md index 6f0394a183d..92712a6f6b2 100644 --- a/docs/ru/operations/utilities/clickhouse-local.md +++ b/docs/ru/operations/utilities/clickhouse-local.md @@ -45,7 +45,6 @@ $ clickhouse-local --structure "table_structure" --input-format "format_of_incom - `--logger.level` — уровень логирования. - `--ignore-error` — не прекращать обработку если запрос выдал ошибку. - `-c`, `--config-file` — путь к файлу конфигурации. По умолчанию `clickhouse-local` запускается с пустой конфигурацией. Конфигурационный файл имеет тот же формат, что и для сервера ClickHouse, и в нём можно использовать все конфигурационные параметры сервера. Обычно подключение конфигурации не требуется; если требуется установить отдельный параметр, то это можно сделать ключом с именем параметра. -- `--no-system-tables` — запуск без использования системных таблиц. - `--help` — вывод справочной информации о `clickhouse-local`. - `-V`, `--version` — вывод текущей версии и выход. diff --git a/docs/zh/operations/utilities/clickhouse-local.md b/docs/zh/operations/utilities/clickhouse-local.md index 7428ae06a6e..e8c9503626b 100644 --- a/docs/zh/operations/utilities/clickhouse-local.md +++ b/docs/zh/operations/utilities/clickhouse-local.md @@ -45,7 +45,6 @@ clickhouse-local --structure "table_structure" --input-format "format_of_incomin - `--logger.level` — 日志级别。 - `--ignore-error` — 当查询失败时,不停止处理。 - `-c`, `--config-file` — 与ClickHouse服务器格式相同配置文件的路径,默认情况下配置为空。 -- `--no-system-tables` — 不附加系统表。 - `--help` — `clickhouse-local`使用帮助信息。 - `-V`, `--version` — 打印版本信息并退出。 diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index fbb64ea1135..41553f02c52 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -761,7 +761,7 @@ void LocalServer::processConfig() LOG_DEBUG(log, "Loaded metadata."); } - else if (!config().has("no-system-tables")) + else { attachSystemTablesLocal(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE)); attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA)); @@ -842,7 +842,6 @@ void LocalServer::addOptions(OptionsDescription & options_description) ("logger.log", po::value(), "Log file name") ("logger.level", po::value(), "Log level") - ("no-system-tables", "do not attach system tables (better startup time)") ("path", po::value(), "Storage path") ("only-system-tables", "attach only system tables from specified path") ("top_level_domains_path", po::value(), "Path to lists with custom TLDs") @@ -871,8 +870,6 @@ void LocalServer::processOptions(const OptionsDescription &, const CommandLineOp config().setString("table-file", options["file"].as()); if (options.count("structure")) config().setString("table-structure", options["structure"].as()); - if (options.count("no-system-tables")) - config().setBool("no-system-tables", true); if (options.count("only-system-tables")) config().setBool("only-system-tables", true); if (options.count("database")) diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 7bc445c5b5d..b86cbeaf3e9 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -193,7 +194,7 @@ DatabaseWithOwnTablesBase::DatabaseWithOwnTablesBase(const String & name_, const bool DatabaseWithOwnTablesBase::isTableExist(const String & table_name, ContextPtr) const { std::lock_guard lock(mutex); - return tables.find(table_name) != tables.end(); + return tables.find(table_name) != tables.end() || lazy_tables.find(table_name) != lazy_tables.end(); } StoragePtr DatabaseWithOwnTablesBase::tryGetTable(const String & table_name, ContextPtr) const @@ -205,6 +206,9 @@ StoragePtr DatabaseWithOwnTablesBase::tryGetTable(const String & table_name, Con DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(ContextPtr, const FilterByNameFunction & filter_by_table_name) const { std::lock_guard lock(mutex); + + loadLazyTables(); + if (!filter_by_table_name) return std::make_unique(tables, database_name); @@ -256,6 +260,12 @@ void DatabaseWithOwnTablesBase::attachTable(ContextPtr /* context_ */, const Str attachTableUnlocked(table_name, table); } +void DatabaseWithOwnTablesBase::registerLazyTable(ContextPtr, const String & table_name, LazyTableCreator table_creator, const String &) +{ + std::lock_guard lock(mutex); + registerLazyTableUnlocked(table_name, std::move(table_creator)); +} + void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, const StoragePtr & table) { auto table_id = table->getStorageID(); @@ -281,6 +291,12 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, c table->is_detached = false; } +void DatabaseWithOwnTablesBase::registerLazyTableUnlocked(const String & table_name, LazyTableCreator table_creator) +{ + if (!lazy_tables.emplace(table_name, std::move(table_creator)).second) + throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {} already registered.", table_name); +} + void DatabaseWithOwnTablesBase::shutdown() { /// You can not hold a lock during shutdown. @@ -381,10 +397,43 @@ void DatabaseWithOwnTablesBase::createTableRestoredFromBackup(const ASTPtr & cre StoragePtr DatabaseWithOwnTablesBase::tryGetTableNoWait(const String & table_name) const { std::lock_guard lock(mutex); + auto it = tables.find(table_name); if (it != tables.end()) return it->second; + + const auto lazy_it = lazy_tables.find(table_name); + if (lazy_it != lazy_tables.end()) + { + LOG_DEBUG(log, "Attaching lazy table {}", backQuoteIfNeed(table_name)); + auto storage = lazy_it->second(); + lazy_tables.erase(lazy_it); + /// FIXME: it should call attachTable(), but need to reorder locking bits for this + (const_cast(this))->attachTableUnlocked(table_name, storage); + + it = tables.find(table_name); + if (it != tables.end()) + return it->second; + } + return {}; } +void DatabaseWithOwnTablesBase::loadLazyTables() const +{ + while (!lazy_tables.empty()) + { + auto lazy_it = lazy_tables.begin(); + + const auto table_name = lazy_it->first; + LOG_DEBUG(log, "Attaching lazy table {}", backQuoteIfNeed(table_name)); + auto storage = lazy_it->second(); + + lazy_tables.erase(lazy_it); + + /// FIXME: it should call attachTable(), but need to reorder locking bits for this + (const_cast(this))->attachTableUnlocked(table_name, storage); + } +} + } diff --git a/src/Databases/DatabasesCommon.h b/src/Databases/DatabasesCommon.h index fc67596d3de..9de03b90544 100644 --- a/src/Databases/DatabasesCommon.h +++ b/src/Databases/DatabasesCommon.h @@ -24,6 +24,9 @@ class Context; class DatabaseWithOwnTablesBase : public IDatabase, protected WithContext { public: + using LazyTableCreator = std::function; + using LazyTables = std::map; + bool isTableExist(const String & table_name, ContextPtr context) const override; StoragePtr tryGetTable(const String & table_name, ContextPtr context) const override; @@ -32,6 +35,10 @@ public: void attachTable(ContextPtr context, const String & table_name, const StoragePtr & table, const String & relative_table_path) override; + /// Register tables lazily (attach will be done only when the table will be used). + /// This is needed to improve startup time of clickhouse-local. + virtual void registerLazyTable(ContextPtr context, const String & table_name, LazyTableCreator table_creator, const String & relative_table_path = {}); + StoragePtr detachTable(ContextPtr context, const String & table_name) override; DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override; @@ -45,14 +52,19 @@ public: protected: Tables tables TSA_GUARDED_BY(mutex); + /// Tables that are attached lazily + mutable LazyTables lazy_tables TSA_GUARDED_BY(mutex); Poco::Logger * log; DatabaseWithOwnTablesBase(const String & name_, const String & logger, ContextPtr context); void attachTableUnlocked(const String & table_name, const StoragePtr & table) TSA_REQUIRES(mutex); + void registerLazyTableUnlocked(const String & table_name, LazyTableCreator table_creator) TSA_REQUIRES(mutex); StoragePtr detachTableUnlocked(const String & table_name) TSA_REQUIRES(mutex); StoragePtr getTableUnlocked(const String & table_name) const TSA_REQUIRES(mutex); StoragePtr tryGetTableNoWait(const String & table_name) const; + + void loadLazyTables() const TSA_REQUIRES(mutex); }; } diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index ca49a546b94..e9bed14dc6a 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -110,64 +110,64 @@ namespace DB void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database) { - attach(context, system_database, "one"); - attach(context, system_database, "numbers", false); - attach(context, system_database, "numbers_mt", true); - attach(context, system_database, "zeros", false); - attach(context, system_database, "zeros_mt", true); - attach(context, system_database, "databases"); - attach(context, system_database, "tables"); - attach(context, system_database, "columns"); - attach(context, system_database, "functions"); - attach(context, system_database, "events"); - attach(context, system_database, "settings"); - attach(context, system_database, "server_settings"); - attach(context, system_database, "settings_changes"); - attach>(context, system_database, "merge_tree_settings"); - attach>(context, system_database, "replicated_merge_tree_settings"); - attach(context, system_database, "build_options"); - attach(context, system_database, "formats"); - attach(context, system_database, "table_functions"); - attach(context, system_database, "aggregate_function_combinators"); - attach(context, system_database, "data_type_families"); - attach(context, system_database, "collations"); - attach(context, system_database, "table_engines"); - attach(context, system_database, "contributors"); - attach(context, system_database, "users"); - attach(context, system_database, "roles"); - attach(context, system_database, "grants"); - attach(context, system_database, "role_grants"); - attach(context, system_database, "current_roles"); - attach(context, system_database, "enabled_roles"); - attach(context, system_database, "settings_profiles"); - attach(context, system_database, "settings_profile_elements"); - attach(context, system_database, "row_policies"); - attach(context, system_database, "quotas"); - attach(context, system_database, "quota_limits"); - attach(context, system_database, "quota_usage"); - attach(context, system_database, "quotas_usage"); - attach(context, system_database, "user_directories"); - attach(context, system_database, "privileges"); - attach(context, system_database, "errors"); - attach(context, system_database, "warnings"); - attach(context, system_database, "data_skipping_indices"); - attach(context, system_database, "licenses"); - attach(context, system_database, "time_zones"); - attach(context, system_database, "backups"); - attach(context, system_database, "schema_inference_cache"); - attach(context, system_database, "dropped_tables"); - attach(context, system_database, "scheduler"); + attachLazy(context, system_database, "one"); + attachLazy(context, system_database, "numbers", false); + attachLazy(context, system_database, "numbers_mt", true); + attachLazy(context, system_database, "zeros", false); + attachLazy(context, system_database, "zeros_mt", true); + attachLazy(context, system_database, "databases"); + attachLazy(context, system_database, "tables"); + attachLazy(context, system_database, "columns"); + attachLazy(context, system_database, "functions"); + attachLazy(context, system_database, "events"); + attachLazy(context, system_database, "settings"); + attachLazy(context, system_database, "server_settings"); + attachLazy(context, system_database, "settings_changes"); + attachLazy>(context, system_database, "merge_tree_settings"); + attachLazy>(context, system_database, "replicated_merge_tree_settings"); + attachLazy(context, system_database, "build_options"); + attachLazy(context, system_database, "formats"); + attachLazy(context, system_database, "table_functions"); + attachLazy(context, system_database, "aggregate_function_combinators"); + attachLazy(context, system_database, "data_type_families"); + attachLazy(context, system_database, "collations"); + attachLazy(context, system_database, "table_engines"); + attachLazy(context, system_database, "contributors"); + attachLazy(context, system_database, "users"); + attachLazy(context, system_database, "roles"); + attachLazy(context, system_database, "grants"); + attachLazy(context, system_database, "role_grants"); + attachLazy(context, system_database, "current_roles"); + attachLazy(context, system_database, "enabled_roles"); + attachLazy(context, system_database, "settings_profiles"); + attachLazy(context, system_database, "settings_profile_elements"); + attachLazy(context, system_database, "row_policies"); + attachLazy(context, system_database, "quotas"); + attachLazy(context, system_database, "quota_limits"); + attachLazy(context, system_database, "quota_usage"); + attachLazy(context, system_database, "quotas_usage"); + attachLazy(context, system_database, "user_directories"); + attachLazy(context, system_database, "privileges"); + attachLazy(context, system_database, "errors"); + attachLazy(context, system_database, "warnings"); + attachLazy(context, system_database, "data_skipping_indices"); + attachLazy(context, system_database, "licenses"); + attachLazy(context, system_database, "time_zones"); + attachLazy(context, system_database, "backups"); + attachLazy(context, system_database, "schema_inference_cache"); + attachLazy(context, system_database, "dropped_tables"); + attachLazy(context, system_database, "scheduler"); #if defined(__ELF__) && !defined(OS_FREEBSD) - attach(context, system_database, "symbols"); + attachLazy(context, system_database, "symbols"); #endif #if USE_RDKAFKA - attach(context, system_database, "kafka_consumers"); + attachLazy(context, system_database, "kafka_consumers"); #endif #ifdef OS_LINUX - attach(context, system_database, "stack_trace"); + attachLazy(context, system_database, "stack_trace"); #endif #if USE_ROCKSDB - attach(context, system_database, "rocksdb"); + attachLazy(context, system_database, "rocksdb"); #endif } diff --git a/src/Storages/System/attachSystemTablesImpl.h b/src/Storages/System/attachSystemTablesImpl.h index a1fae985d92..b97bb86537e 100644 --- a/src/Storages/System/attachSystemTablesImpl.h +++ b/src/Storages/System/attachSystemTablesImpl.h @@ -1,11 +1,42 @@ #pragma once #include +#include #include namespace DB { +template +void attachLazy(ContextPtr context, IDatabase & system_database, const String & table_name, StorageArgs && ... args) +{ + assert(system_database.getDatabaseName() == DatabaseCatalog::SYSTEM_DATABASE); + /// TODO: move lazy interface into IDatabase + auto & db = dynamic_cast(system_database); + if (system_database.getUUID() == UUIDHelpers::Nil) + { + /// Attach to Ordinary database. + auto table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name); + db.registerLazyTable(context, table_name, [table_id, ... captured_args = std::forward(args)] mutable + { + return std::make_shared(table_id, std::forward(captured_args)...); + }); + } + else + { + /// Attach to Atomic database. + /// NOTE: UUIDs are not persistent, but it's ok since no data are stored on disk for these storages + /// and path is actually not used + auto table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name, UUIDHelpers::generateV4()); + DatabaseCatalog::instance().addUUIDMapping(table_id.uuid); + String path = "store/" + DatabaseCatalog::getPathForUUID(table_id.uuid); + db.registerLazyTable(context, table_name, [table_id, ... captured_args = std::forward(args)] mutable + { + return std::make_shared(table_id, std::forward(captured_args)...); + }, path); + } +} + template void attach(ContextPtr context, IDatabase & system_database, const String & table_name, StorageArgs && ... args) { From f75484a0b1d89a356b1079d9af2abe860670fb76 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 9 Dec 2023 20:29:47 +0100 Subject: [PATCH 097/110] Make IDatabase::attachTableUnlocked() virtual and use it for overrides (mostly) --- src/Databases/DatabaseAtomic.cpp | 7 +++---- src/Databases/DatabaseAtomic.h | 3 ++- src/Databases/DatabaseLazy.cpp | 3 +-- src/Databases/DatabaseLazy.h | 5 +++-- src/Databases/DatabaseMemory.cpp | 8 +++---- src/Databases/DatabasesCommon.cpp | 21 +++++++------------ src/Databases/DatabasesCommon.h | 6 ++---- src/Databases/IDatabase.cpp | 10 +++++++++ src/Databases/IDatabase.h | 17 ++++++++++----- .../MySQL/DatabaseMaterializedMySQL.cpp | 4 ++-- .../MySQL/DatabaseMaterializedMySQL.h | 4 ++-- src/Databases/MySQL/DatabaseMySQL.cpp | 4 +--- src/Databases/MySQL/DatabaseMySQL.h | 4 ++-- .../DatabaseMaterializedPostgreSQL.cpp | 4 ++-- .../DatabaseMaterializedPostgreSQL.h | 4 ++-- .../PostgreSQL/DatabasePostgreSQL.cpp | 4 +--- src/Databases/PostgreSQL/DatabasePostgreSQL.h | 3 ++- 17 files changed, 59 insertions(+), 52 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 1daa6351c23..9fd288ed9e5 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -89,15 +89,14 @@ void DatabaseAtomic::drop(ContextPtr) fs::remove_all(getMetadataPath()); } -void DatabaseAtomic::attachTable(ContextPtr /* context_ */, const String & name, const StoragePtr & table, const String & relative_table_path) +void DatabaseAtomic::attachTableUnlocked(ContextPtr local_context, const String & name, const StoragePtr & table, const String & relative_table_path) { assert(relative_table_path != data_path && !relative_table_path.empty()); DetachedTables not_in_use; - std::lock_guard lock(mutex); not_in_use = cleanupDetachedTables(); auto table_id = table->getStorageID(); assertDetachedTableNotInUse(table_id.uuid); - DatabaseOrdinary::attachTableUnlocked(name, table); + DatabaseOrdinary::attachTableUnlocked(local_context, name, table, relative_table_path); table_name_to_path.emplace(std::make_pair(name, relative_table_path)); } @@ -325,7 +324,7 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora /// It throws if `table_metadata_path` already exists (it's possible if table was detached) renameNoReplace(table_metadata_tmp_path, table_metadata_path); /// Commit point (a sort of) - attachTableUnlocked(query.getTable(), table); /// Should never throw + attachTableUnlocked(query_context, query.getTable(), table, /*relative_table_path=*/ {}); /// Should never throw table_name_to_path.emplace(query.getTable(), table_data_path); } catch (...) diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index 83cb51be1ff..1eb6dd1cbbd 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -38,7 +38,6 @@ public: void dropTable(ContextPtr context, const String & table_name, bool sync) override; void dropTableImpl(ContextPtr context, const String & table_name, bool sync); - void attachTable(ContextPtr context, const String & name, const StoragePtr & table, const String & relative_table_path) override; StoragePtr detachTable(ContextPtr context, const String & name) override; String getTableDataPath(const String & table_name) const override; @@ -66,6 +65,8 @@ public: void setDetachedTableNotInUseForce(const UUID & uuid) override; protected: + void attachTableUnlocked(ContextPtr local_context, const String & name, const StoragePtr & table, const String & relative_table_path) TSA_REQUIRES(mutex) override; + void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path, const String & statement, ContextPtr query_context) override; void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, const String & table_metadata_tmp_path, const String & table_metadata_path, ContextPtr query_context) override; diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index 896ae99656f..ec9e1e5ee57 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -161,10 +161,9 @@ bool DatabaseLazy::empty() const return tables_cache.empty(); } -void DatabaseLazy::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & table, const String &) +void DatabaseLazy::attachTableUnlocked(ContextPtr /* context_ */, const String & table_name, const StoragePtr & table, const String &) { LOG_DEBUG(log, "Attach table {}.", backQuote(table_name)); - std::lock_guard lock(mutex); time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); auto [it, inserted] = tables_cache.emplace(std::piecewise_construct, diff --git a/src/Databases/DatabaseLazy.h b/src/Databases/DatabaseLazy.h index 2b1b119754d..370c49557de 100644 --- a/src/Databases/DatabaseLazy.h +++ b/src/Databases/DatabaseLazy.h @@ -64,14 +64,15 @@ public: DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override; - void attachTable(ContextPtr context, const String & table_name, const StoragePtr & table, const String & relative_table_path) override; - StoragePtr detachTable(ContextPtr context, const String & table_name) override; void shutdown() override; ~DatabaseLazy() override; +protected: + void attachTableUnlocked(ContextPtr context, const String & table_name, const StoragePtr & table, const String & relative_table_path) TSA_REQUIRES(mutex) override; + private: struct CacheExpirationQueueElement { diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index 2a7a2ad8ccc..7be9d5e2cb7 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -33,13 +33,13 @@ DatabaseMemory::DatabaseMemory(const String & name_, ContextPtr context_) } void DatabaseMemory::createTable( - ContextPtr /*context*/, + ContextPtr local_context, const String & table_name, const StoragePtr & table, const ASTPtr & query) { std::lock_guard lock{mutex}; - attachTableUnlocked(table_name, table); + attachTableUnlocked(local_context, table_name, table, /*relative_table_path=*/ {}); /// Clean the query from temporary flags. ASTPtr query_to_store = query; @@ -56,7 +56,7 @@ void DatabaseMemory::createTable( } void DatabaseMemory::dropTable( - ContextPtr /*context*/, + ContextPtr local_context, const String & table_name, bool /*sync*/) { @@ -83,7 +83,7 @@ void DatabaseMemory::dropTable( catch (...) { std::lock_guard lock{mutex}; - attachTableUnlocked(table_name, table); + attachTableUnlocked(local_context, table_name, table, /*relative_table_path=*/ {}); throw; } diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index b86cbeaf3e9..b2bb6fbd5c7 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -254,19 +254,13 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n return res; } -void DatabaseWithOwnTablesBase::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & table, const String &) -{ - std::lock_guard lock(mutex); - attachTableUnlocked(table_name, table); -} - void DatabaseWithOwnTablesBase::registerLazyTable(ContextPtr, const String & table_name, LazyTableCreator table_creator, const String &) { std::lock_guard lock(mutex); registerLazyTableUnlocked(table_name, std::move(table_creator)); } -void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, const StoragePtr & table) +void DatabaseWithOwnTablesBase::attachTableUnlocked(ContextPtr, const String & name, const StoragePtr & table, const String &) { auto table_id = table->getStorageID(); if (table_id.database_name != database_name) @@ -279,7 +273,7 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, c DatabaseCatalog::instance().addUUIDMapping(table_id.uuid, shared_from_this(), table); } - if (!tables.emplace(table_name, table).second) + if (!tables.emplace(name, table).second) { if (table_id.hasUUID()) DatabaseCatalog::instance().removeUUIDMapping(table_id.uuid); @@ -408,8 +402,7 @@ StoragePtr DatabaseWithOwnTablesBase::tryGetTableNoWait(const String & table_nam LOG_DEBUG(log, "Attaching lazy table {}", backQuoteIfNeed(table_name)); auto storage = lazy_it->second(); lazy_tables.erase(lazy_it); - /// FIXME: it should call attachTable(), but need to reorder locking bits for this - (const_cast(this))->attachTableUnlocked(table_name, storage); + (const_cast(this))->attachTableUnlocked(Context::getGlobalContextInstance(), table_name, storage, /*relative_table_path=*/ {}); it = tables.find(table_name); if (it != tables.end()) @@ -421,6 +414,10 @@ StoragePtr DatabaseWithOwnTablesBase::tryGetTableNoWait(const String & table_nam void DatabaseWithOwnTablesBase::loadLazyTables() const { + if (lazy_tables.empty()) + return; + + ContextPtr global_context = Context::getGlobalContextInstance(); while (!lazy_tables.empty()) { auto lazy_it = lazy_tables.begin(); @@ -430,9 +427,7 @@ void DatabaseWithOwnTablesBase::loadLazyTables() const auto storage = lazy_it->second(); lazy_tables.erase(lazy_it); - - /// FIXME: it should call attachTable(), but need to reorder locking bits for this - (const_cast(this))->attachTableUnlocked(table_name, storage); + (const_cast(this))->attachTableUnlocked(global_context, table_name, storage, /*relative_table_path=*/ {}); } } diff --git a/src/Databases/DatabasesCommon.h b/src/Databases/DatabasesCommon.h index 9de03b90544..94591d499aa 100644 --- a/src/Databases/DatabasesCommon.h +++ b/src/Databases/DatabasesCommon.h @@ -33,11 +33,9 @@ public: bool empty() const override; - void attachTable(ContextPtr context, const String & table_name, const StoragePtr & table, const String & relative_table_path) override; - /// Register tables lazily (attach will be done only when the table will be used). /// This is needed to improve startup time of clickhouse-local. - virtual void registerLazyTable(ContextPtr context, const String & table_name, LazyTableCreator table_creator, const String & relative_table_path = {}); + virtual void registerLazyTable(ContextPtr context, const String & table_name, LazyTableCreator table_creator, const String & relative_table_path = {}); /// NOLINT StoragePtr detachTable(ContextPtr context, const String & table_name) override; @@ -58,7 +56,7 @@ protected: DatabaseWithOwnTablesBase(const String & name_, const String & logger, ContextPtr context); - void attachTableUnlocked(const String & table_name, const StoragePtr & table) TSA_REQUIRES(mutex); + void attachTableUnlocked(ContextPtr context, const String & name, const StoragePtr & table, const String & relative_table_path) TSA_REQUIRES(mutex) override; void registerLazyTableUnlocked(const String & table_name, LazyTableCreator table_creator) TSA_REQUIRES(mutex); StoragePtr detachTableUnlocked(const String & table_name) TSA_REQUIRES(mutex); StoragePtr getTableUnlocked(const String & table_name) const TSA_REQUIRES(mutex); diff --git a/src/Databases/IDatabase.cpp b/src/Databases/IDatabase.cpp index 09640d2f86e..1aa35ef9a96 100644 --- a/src/Databases/IDatabase.cpp +++ b/src/Databases/IDatabase.cpp @@ -46,4 +46,14 @@ void IDatabase::createTableRestoredFromBackup(const ASTPtr & create_table_query, backQuoteIfNeed(create_table_query->as().getTable())); } +/// Add a table to the database, but do not add it to the metadata. The database may not support this method. +/// +/// Note: ATTACH TABLE statement actually uses createTable method. +void IDatabase::attachTable(ContextPtr context, const String & name, const StoragePtr & table, const String & relative_table_path) /// NOLINT +{ + std::lock_guard lock(mutex); + attachTableUnlocked(context, name, table, relative_table_path); +} + + } diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index e5afa0eb944..84af4f55e30 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -269,11 +269,13 @@ public: /// Add a table to the database, but do not add it to the metadata. The database may not support this method. /// - /// Note: ATTACH TABLE statement actually uses createTable method. - virtual void attachTable(ContextPtr /* context */, const String & /*name*/, const StoragePtr & /*table*/, [[maybe_unused]] const String & relative_table_path = {}) /// NOLINT - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "There is no ATTACH TABLE query for Database{}", getEngineName()); - } + /// @param relative_table_path - only for Atomic engine + /// + /// Note: + /// - ATTACH TABLE statement actually uses createTable method. + /// - Instead of overriding this method you should override attachTableUnlocked() + /// (This method is only for DatabasesOverlay to override) + virtual void attachTable(ContextPtr context, const String & name, const StoragePtr & table, const String & relative_table_path = {}); /// NOLINT /// Forget about the table without deleting it, and return it. The database may not support this method. virtual StoragePtr detachTable(ContextPtr /* context */, const String & /*name*/) @@ -430,6 +432,11 @@ protected: return nullptr; } + virtual void attachTableUnlocked(ContextPtr /*context*/, const String & /*name*/, const StoragePtr & /*table*/, const String & /*relative_table_path*/ = {}) TSA_REQUIRES(mutex) /// NOLINT + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "There is no ATTACH TABLE query for Database{}", getEngineName()); + } + mutable std::mutex mutex; String database_name TSA_GUARDED_BY(mutex); String comment TSA_GUARDED_BY(mutex); diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index a31e74cc7ae..965a3ac9965 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -101,10 +101,10 @@ void DatabaseMaterializedMySQL::dropTable(ContextPtr context_, const String & na DatabaseAtomic::dropTable(context_, name, sync); } -void DatabaseMaterializedMySQL::attachTable(ContextPtr context_, const String & name, const StoragePtr & table, const String & relative_table_path) +void DatabaseMaterializedMySQL::attachTableUnlocked(ContextPtr context_, const String & name, const StoragePtr & table, const String & relative_table_path) { checkIsInternalQuery(context_, "ATTACH TABLE"); - DatabaseAtomic::attachTable(context_, name, table, relative_table_path); + DatabaseAtomic::attachTableUnlocked(context_, name, table, relative_table_path); } StoragePtr DatabaseMaterializedMySQL::detachTable(ContextPtr context_, const String & name) diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.h b/src/Databases/MySQL/DatabaseMaterializedMySQL.h index 895498723fd..0744b79257f 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.h @@ -48,6 +48,8 @@ protected: LoadTaskPtr startup_mysql_database_task; + void attachTableUnlocked(ContextPtr context_, const String & name, const StoragePtr & table, const String & relative_table_path) TSA_REQUIRES(mutex) override; + public: String getEngineName() const override { return "MaterializedMySQL"; } @@ -58,8 +60,6 @@ public: void dropTable(ContextPtr context_, const String & name, bool sync) override; - void attachTable(ContextPtr context_, const String & name, const StoragePtr & table, const String & relative_table_path) override; - StoragePtr detachTable(ContextPtr context_, const String & name) override; void renameTable(ContextPtr context_, const String & name, IDatabase & to_database, const String & to_name, bool exchange, bool dictionary) override; diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 7d2ed7a9662..42bceaa0e08 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -361,10 +361,8 @@ void DatabaseMySQL::cleanOutdatedTables() } } -void DatabaseMySQL::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & storage, const String &) +void DatabaseMySQL::attachTableUnlocked(ContextPtr /* context_ */, const String & table_name, const StoragePtr & storage, const String &) { - std::lock_guard lock{mutex}; - if (!local_tables_cache.contains(table_name)) throw Exception(ErrorCodes::UNKNOWN_TABLE, "Cannot attach table {}.{} because it does not exist.", backQuoteIfNeed(database_name), backQuoteIfNeed(table_name)); diff --git a/src/Databases/MySQL/DatabaseMySQL.h b/src/Databases/MySQL/DatabaseMySQL.h index e5b1f434d2f..33bf26059c8 100644 --- a/src/Databases/MySQL/DatabaseMySQL.h +++ b/src/Databases/MySQL/DatabaseMySQL.h @@ -84,9 +84,9 @@ public: void dropTable(ContextPtr context, const String & table_name, bool sync) override; - void attachTable(ContextPtr context, const String & table_name, const StoragePtr & storage, const String & relative_table_path) override; - protected: + void attachTableUnlocked(ContextPtr context, const String & table_name, const StoragePtr & storage, const String & relative_table_path) TSA_REQUIRES(mutex) override; + ASTPtr getCreateTableQueryImpl(const String & name, ContextPtr context, bool throw_on_error) const override; private: diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 78be0611631..52914d54478 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -322,7 +322,7 @@ void DatabaseMaterializedPostgreSQL::createTable(ContextPtr local_context, const } -void DatabaseMaterializedPostgreSQL::attachTable(ContextPtr context_, const String & table_name, const StoragePtr & table, const String & relative_table_path) +void DatabaseMaterializedPostgreSQL::attachTableUnlocked(ContextPtr context_, const String & table_name, const StoragePtr & table, const String & relative_table_path) { /// If there is query context then we need to attach materialized storage. /// If there is no query context then we need to attach internal storage from atomic database. @@ -362,7 +362,7 @@ void DatabaseMaterializedPostgreSQL::attachTable(ContextPtr context_, const Stri } else { - DatabaseAtomic::attachTable(context_, table_name, table, relative_table_path); + DatabaseAtomic::attachTableUnlocked(context_, table_name, table, relative_table_path); } } diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h index 8feb5a014e1..52d7224cbdf 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h @@ -50,8 +50,6 @@ public: void createTable(ContextPtr context, const String & table_name, const StoragePtr & table, const ASTPtr & query) override; - void attachTable(ContextPtr context, const String & table_name, const StoragePtr & table, const String & relative_table_path) override; - void detachTablePermanently(ContextPtr context, const String & table_name) override; StoragePtr detachTable(ContextPtr context, const String & table_name) override; @@ -71,6 +69,8 @@ public: String getPostgreSQLDatabaseName() const { return remote_database_name; } protected: + void attachTableUnlocked(ContextPtr context, const String & table_name, const StoragePtr & table, const String & relative_table_path) TSA_REQUIRES(mutex) override; + ASTPtr getCreateTableQueryImpl(const String & table_name, ContextPtr local_context, bool throw_on_error) const override; private: diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 24f04c16029..4f0f85bc787 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -216,10 +216,8 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr } -void DatabasePostgreSQL::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & storage, const String &) +void DatabasePostgreSQL::attachTableUnlocked(ContextPtr /* context_ */, const String & table_name, const StoragePtr & storage, const String &) { - std::lock_guard lock{mutex}; - if (!checkPostgresTable(table_name)) throw Exception(ErrorCodes::UNKNOWN_TABLE, "Cannot attach PostgreSQL table {} because it does not exist in PostgreSQL (database: {})", diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.h b/src/Databases/PostgreSQL/DatabasePostgreSQL.h index d731e06649b..30d0070b2d7 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.h @@ -54,13 +54,14 @@ public: void createTable(ContextPtr, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) override; void dropTable(ContextPtr, const String & table_name, bool sync) override; - void attachTable(ContextPtr context, const String & table_name, const StoragePtr & storage, const String & relative_table_path) override; StoragePtr detachTable(ContextPtr context, const String & table_name) override; void drop(ContextPtr /*context*/) override; void shutdown() override; protected: + void attachTableUnlocked(ContextPtr context, const String & table_name, const StoragePtr & storage, const String & relative_table_path) TSA_REQUIRES(mutex) override; + ASTPtr getCreateTableQueryImpl(const String & table_name, ContextPtr context, bool throw_on_error) const override; private: From c14721bb03966548e831c39d9f822462fd88be81 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 9 Dec 2023 22:49:19 +0100 Subject: [PATCH 098/110] Move lazy table initialization interface into IDatabase Signed-off-by: Azat Khuzhin --- src/Databases/DatabasesCommon.cpp | 21 ++++++++------------ src/Databases/DatabasesCommon.h | 9 +-------- src/Databases/IDatabase.cpp | 6 ++++++ src/Databases/IDatabase.h | 14 ++++++++++++- src/Storages/System/attachSystemTablesImpl.h | 6 ++---- 5 files changed, 30 insertions(+), 26 deletions(-) diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index b2bb6fbd5c7..512a7550050 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -254,12 +254,6 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n return res; } -void DatabaseWithOwnTablesBase::registerLazyTable(ContextPtr, const String & table_name, LazyTableCreator table_creator, const String &) -{ - std::lock_guard lock(mutex); - registerLazyTableUnlocked(table_name, std::move(table_creator)); -} - void DatabaseWithOwnTablesBase::attachTableUnlocked(ContextPtr, const String & name, const StoragePtr & table, const String &) { auto table_id = table->getStorageID(); @@ -285,9 +279,9 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(ContextPtr, const String & n table->is_detached = false; } -void DatabaseWithOwnTablesBase::registerLazyTableUnlocked(const String & table_name, LazyTableCreator table_creator) +void DatabaseWithOwnTablesBase::registerLazyTableUnlocked(const String & table_name, LazyTableCreator table_creator, const String & relative_table_path) { - if (!lazy_tables.emplace(table_name, std::move(table_creator)).second) + if (!lazy_tables.emplace(table_name, std::make_pair(relative_table_path, std::move(table_creator))).second) throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {} already registered.", table_name); } @@ -400,9 +394,10 @@ StoragePtr DatabaseWithOwnTablesBase::tryGetTableNoWait(const String & table_nam if (lazy_it != lazy_tables.end()) { LOG_DEBUG(log, "Attaching lazy table {}", backQuoteIfNeed(table_name)); - auto storage = lazy_it->second(); + auto relative_table_path = lazy_it->second.first; + auto storage = lazy_it->second.second(); lazy_tables.erase(lazy_it); - (const_cast(this))->attachTableUnlocked(Context::getGlobalContextInstance(), table_name, storage, /*relative_table_path=*/ {}); + (const_cast(this))->attachTableUnlocked(Context::getGlobalContextInstance(), table_name, storage, relative_table_path); it = tables.find(table_name); if (it != tables.end()) @@ -424,10 +419,10 @@ void DatabaseWithOwnTablesBase::loadLazyTables() const const auto table_name = lazy_it->first; LOG_DEBUG(log, "Attaching lazy table {}", backQuoteIfNeed(table_name)); - auto storage = lazy_it->second(); - + auto relative_table_path = lazy_it->second.first; + auto storage = lazy_it->second.second(); lazy_tables.erase(lazy_it); - (const_cast(this))->attachTableUnlocked(global_context, table_name, storage, /*relative_table_path=*/ {}); + (const_cast(this))->attachTableUnlocked(global_context, table_name, storage, relative_table_path); } } diff --git a/src/Databases/DatabasesCommon.h b/src/Databases/DatabasesCommon.h index 94591d499aa..71704b95110 100644 --- a/src/Databases/DatabasesCommon.h +++ b/src/Databases/DatabasesCommon.h @@ -24,19 +24,12 @@ class Context; class DatabaseWithOwnTablesBase : public IDatabase, protected WithContext { public: - using LazyTableCreator = std::function; - using LazyTables = std::map; - bool isTableExist(const String & table_name, ContextPtr context) const override; StoragePtr tryGetTable(const String & table_name, ContextPtr context) const override; bool empty() const override; - /// Register tables lazily (attach will be done only when the table will be used). - /// This is needed to improve startup time of clickhouse-local. - virtual void registerLazyTable(ContextPtr context, const String & table_name, LazyTableCreator table_creator, const String & relative_table_path = {}); /// NOLINT - StoragePtr detachTable(ContextPtr context, const String & table_name) override; DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override; @@ -57,7 +50,7 @@ protected: DatabaseWithOwnTablesBase(const String & name_, const String & logger, ContextPtr context); void attachTableUnlocked(ContextPtr context, const String & name, const StoragePtr & table, const String & relative_table_path) TSA_REQUIRES(mutex) override; - void registerLazyTableUnlocked(const String & table_name, LazyTableCreator table_creator) TSA_REQUIRES(mutex); + void registerLazyTableUnlocked(const String & table_name, LazyTableCreator table_creator, const String & relative_table_path) TSA_REQUIRES(mutex) override; StoragePtr detachTableUnlocked(const String & table_name) TSA_REQUIRES(mutex); StoragePtr getTableUnlocked(const String & table_name) const TSA_REQUIRES(mutex); StoragePtr tryGetTableNoWait(const String & table_name) const; diff --git a/src/Databases/IDatabase.cpp b/src/Databases/IDatabase.cpp index 1aa35ef9a96..ffd789dd531 100644 --- a/src/Databases/IDatabase.cpp +++ b/src/Databases/IDatabase.cpp @@ -55,5 +55,11 @@ void IDatabase::attachTable(ContextPtr context, const String & name, const Stora attachTableUnlocked(context, name, table, relative_table_path); } +void IDatabase::registerLazyTable(ContextPtr, const String & table_name, LazyTableCreator table_creator, const String & relative_table_path) /// NOLINT +{ + std::lock_guard lock(mutex); + registerLazyTableUnlocked(table_name, std::move(table_creator), relative_table_path); +} + } diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 84af4f55e30..58e4c7e364b 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -125,7 +125,6 @@ public: using DatabaseTablesIteratorPtr = std::unique_ptr; - /** Database engine. * It is responsible for: * - initialization of set of known tables and dictionaries; @@ -138,6 +137,10 @@ using DatabaseTablesIteratorPtr = std::unique_ptr; class IDatabase : public std::enable_shared_from_this { public: + using LazyTableCreator = std::function; + /// Map{table_name, Pair{relative_table_path, LazyTableCreator}} + using LazyTables = std::map>; + IDatabase() = delete; explicit IDatabase(String database_name_) : database_name(std::move(database_name_)) {} @@ -277,6 +280,10 @@ public: /// (This method is only for DatabasesOverlay to override) virtual void attachTable(ContextPtr context, const String & name, const StoragePtr & table, const String & relative_table_path = {}); /// NOLINT + /// Register tables lazily (attach will be done only when the table will be used) instead of attaching it. + /// This is needed to improve startup time of clickhouse-local. + virtual void registerLazyTable(ContextPtr context, const String & table_name, LazyTableCreator table_creator, const String & relative_table_path = {}); + /// Forget about the table without deleting it, and return it. The database may not support this method. virtual StoragePtr detachTable(ContextPtr /* context */, const String & /*name*/) { @@ -437,6 +444,11 @@ protected: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "There is no ATTACH TABLE query for Database{}", getEngineName()); } + virtual void registerLazyTableUnlocked(const String & /* table_name */, LazyTableCreator /* table_creator */, const String & /* relative_table_path */) TSA_REQUIRES(mutex) /// NOLINT + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "There lazy table initialization support for Database{}", getEngineName()); + } + mutable std::mutex mutex; String database_name TSA_GUARDED_BY(mutex); String comment TSA_GUARDED_BY(mutex); diff --git a/src/Storages/System/attachSystemTablesImpl.h b/src/Storages/System/attachSystemTablesImpl.h index b97bb86537e..2d32c1b3ea4 100644 --- a/src/Storages/System/attachSystemTablesImpl.h +++ b/src/Storages/System/attachSystemTablesImpl.h @@ -11,13 +11,11 @@ template void attachLazy(ContextPtr context, IDatabase & system_database, const String & table_name, StorageArgs && ... args) { assert(system_database.getDatabaseName() == DatabaseCatalog::SYSTEM_DATABASE); - /// TODO: move lazy interface into IDatabase - auto & db = dynamic_cast(system_database); if (system_database.getUUID() == UUIDHelpers::Nil) { /// Attach to Ordinary database. auto table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name); - db.registerLazyTable(context, table_name, [table_id, ... captured_args = std::forward(args)] mutable + system_database.registerLazyTable(context, table_name, [table_id, ... captured_args = std::forward(args)] mutable { return std::make_shared(table_id, std::forward(captured_args)...); }); @@ -30,7 +28,7 @@ void attachLazy(ContextPtr context, IDatabase & system_database, const String & auto table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name, UUIDHelpers::generateV4()); DatabaseCatalog::instance().addUUIDMapping(table_id.uuid); String path = "store/" + DatabaseCatalog::getPathForUUID(table_id.uuid); - db.registerLazyTable(context, table_name, [table_id, ... captured_args = std::forward(args)] mutable + system_database.registerLazyTable(context, table_name, [table_id, ... captured_args = std::forward(args)] mutable { return std::make_shared(table_id, std::forward(captured_args)...); }, path); From 4f9d90761ba6657ec3741d635798073a3656d54b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 9 Dec 2023 23:01:47 +0100 Subject: [PATCH 099/110] Attach tables lazily only for clickhouse-local Signed-off-by: Azat Khuzhin --- programs/local/LocalServer.cpp | 4 +- src/Storages/System/attachSystemTables.cpp | 107 ++++++++++--------- src/Storages/System/attachSystemTables.h | 6 +- src/Storages/System/attachSystemTablesImpl.h | 13 ++- 4 files changed, 73 insertions(+), 57 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 41553f02c52..8e526812957 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -744,7 +744,7 @@ void LocalServer::processConfig() LOG_DEBUG(log, "Loading metadata from {}", path); auto startup_system_tasks = loadMetadataSystem(global_context); - attachSystemTablesLocal(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE)); + attachSystemTablesLocal(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE)); attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA)); attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE)); waitLoad(TablesLoaderForegroundPoolId, startup_system_tasks); @@ -763,7 +763,7 @@ void LocalServer::processConfig() } else { - attachSystemTablesLocal(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE)); + attachSystemTablesLocal(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE)); attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA)); attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE)); } diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index e9bed14dc6a..2d7e728bbba 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -108,72 +108,75 @@ namespace DB { +template void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database) { - attachLazy(context, system_database, "one"); - attachLazy(context, system_database, "numbers", false); - attachLazy(context, system_database, "numbers_mt", true); - attachLazy(context, system_database, "zeros", false); - attachLazy(context, system_database, "zeros_mt", true); - attachLazy(context, system_database, "databases"); - attachLazy(context, system_database, "tables"); - attachLazy(context, system_database, "columns"); - attachLazy(context, system_database, "functions"); - attachLazy(context, system_database, "events"); - attachLazy(context, system_database, "settings"); - attachLazy(context, system_database, "server_settings"); - attachLazy(context, system_database, "settings_changes"); - attachLazy>(context, system_database, "merge_tree_settings"); - attachLazy>(context, system_database, "replicated_merge_tree_settings"); - attachLazy(context, system_database, "build_options"); - attachLazy(context, system_database, "formats"); - attachLazy(context, system_database, "table_functions"); - attachLazy(context, system_database, "aggregate_function_combinators"); - attachLazy(context, system_database, "data_type_families"); - attachLazy(context, system_database, "collations"); - attachLazy(context, system_database, "table_engines"); - attachLazy(context, system_database, "contributors"); - attachLazy(context, system_database, "users"); - attachLazy(context, system_database, "roles"); - attachLazy(context, system_database, "grants"); - attachLazy(context, system_database, "role_grants"); - attachLazy(context, system_database, "current_roles"); - attachLazy(context, system_database, "enabled_roles"); - attachLazy(context, system_database, "settings_profiles"); - attachLazy(context, system_database, "settings_profile_elements"); - attachLazy(context, system_database, "row_policies"); - attachLazy(context, system_database, "quotas"); - attachLazy(context, system_database, "quota_limits"); - attachLazy(context, system_database, "quota_usage"); - attachLazy(context, system_database, "quotas_usage"); - attachLazy(context, system_database, "user_directories"); - attachLazy(context, system_database, "privileges"); - attachLazy(context, system_database, "errors"); - attachLazy(context, system_database, "warnings"); - attachLazy(context, system_database, "data_skipping_indices"); - attachLazy(context, system_database, "licenses"); - attachLazy(context, system_database, "time_zones"); - attachLazy(context, system_database, "backups"); - attachLazy(context, system_database, "schema_inference_cache"); - attachLazy(context, system_database, "dropped_tables"); - attachLazy(context, system_database, "scheduler"); + attachLazyOrNot(context, system_database, "one"); + attachLazyOrNot(context, system_database, "numbers", false); + attachLazyOrNot(context, system_database, "numbers_mt", true); + attachLazyOrNot(context, system_database, "zeros", false); + attachLazyOrNot(context, system_database, "zeros_mt", true); + attachLazyOrNot(context, system_database, "databases"); + attachLazyOrNot(context, system_database, "tables"); + attachLazyOrNot(context, system_database, "columns"); + attachLazyOrNot(context, system_database, "functions"); + attachLazyOrNot(context, system_database, "events"); + attachLazyOrNot(context, system_database, "settings"); + attachLazyOrNot(context, system_database, "server_settings"); + attachLazyOrNot(context, system_database, "settings_changes"); + attachLazyOrNot>(context, system_database, "merge_tree_settings"); + attachLazyOrNot>(context, system_database, "replicated_merge_tree_settings"); + attachLazyOrNot(context, system_database, "build_options"); + attachLazyOrNot(context, system_database, "formats"); + attachLazyOrNot(context, system_database, "table_functions"); + attachLazyOrNot(context, system_database, "aggregate_function_combinators"); + attachLazyOrNot(context, system_database, "data_type_families"); + attachLazyOrNot(context, system_database, "collations"); + attachLazyOrNot(context, system_database, "table_engines"); + attachLazyOrNot(context, system_database, "contributors"); + attachLazyOrNot(context, system_database, "users"); + attachLazyOrNot(context, system_database, "roles"); + attachLazyOrNot(context, system_database, "grants"); + attachLazyOrNot(context, system_database, "role_grants"); + attachLazyOrNot(context, system_database, "current_roles"); + attachLazyOrNot(context, system_database, "enabled_roles"); + attachLazyOrNot(context, system_database, "settings_profiles"); + attachLazyOrNot(context, system_database, "settings_profile_elements"); + attachLazyOrNot(context, system_database, "row_policies"); + attachLazyOrNot(context, system_database, "quotas"); + attachLazyOrNot(context, system_database, "quota_limits"); + attachLazyOrNot(context, system_database, "quota_usage"); + attachLazyOrNot(context, system_database, "quotas_usage"); + attachLazyOrNot(context, system_database, "user_directories"); + attachLazyOrNot(context, system_database, "privileges"); + attachLazyOrNot(context, system_database, "errors"); + attachLazyOrNot(context, system_database, "warnings"); + attachLazyOrNot(context, system_database, "data_skipping_indices"); + attachLazyOrNot(context, system_database, "licenses"); + attachLazyOrNot(context, system_database, "time_zones"); + attachLazyOrNot(context, system_database, "backups"); + attachLazyOrNot(context, system_database, "schema_inference_cache"); + attachLazyOrNot(context, system_database, "dropped_tables"); + attachLazyOrNot(context, system_database, "scheduler"); #if defined(__ELF__) && !defined(OS_FREEBSD) - attachLazy(context, system_database, "symbols"); + attachLazyOrNot(context, system_database, "symbols"); #endif #if USE_RDKAFKA - attachLazy(context, system_database, "kafka_consumers"); + attachLazyOrNot(context, system_database, "kafka_consumers"); #endif #ifdef OS_LINUX - attachLazy(context, system_database, "stack_trace"); + attachLazyOrNot(context, system_database, "stack_trace"); #endif #if USE_ROCKSDB - attachLazy(context, system_database, "rocksdb"); + attachLazyOrNot(context, system_database, "rocksdb"); #endif } +template void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database); +template void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database); void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, bool has_zookeeper) { - attachSystemTablesLocal(context, system_database); + attachSystemTablesLocal(context, system_database); attach(context, system_database, "parts"); attach(context, system_database, "projection_parts"); diff --git a/src/Storages/System/attachSystemTables.h b/src/Storages/System/attachSystemTables.h index 4c1a79f84dd..d09189ca92f 100644 --- a/src/Storages/System/attachSystemTables.h +++ b/src/Storages/System/attachSystemTables.h @@ -9,8 +9,12 @@ namespace DB class AsynchronousMetrics; class IDatabase; -void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, bool has_zookeeper); +template void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database); +void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, bool has_zookeeper); void attachSystemTablesAsync(ContextPtr context, IDatabase & system_database, AsynchronousMetrics & async_metrics); +extern template void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database); +extern template void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database); + } diff --git a/src/Storages/System/attachSystemTablesImpl.h b/src/Storages/System/attachSystemTablesImpl.h index 2d32c1b3ea4..45fd7957513 100644 --- a/src/Storages/System/attachSystemTablesImpl.h +++ b/src/Storages/System/attachSystemTablesImpl.h @@ -10,7 +10,6 @@ namespace DB template void attachLazy(ContextPtr context, IDatabase & system_database, const String & table_name, StorageArgs && ... args) { - assert(system_database.getDatabaseName() == DatabaseCatalog::SYSTEM_DATABASE); if (system_database.getUUID() == UUIDHelpers::Nil) { /// Attach to Ordinary database. @@ -38,7 +37,6 @@ void attachLazy(ContextPtr context, IDatabase & system_database, const String & template void attach(ContextPtr context, IDatabase & system_database, const String & table_name, StorageArgs && ... args) { - assert(system_database.getDatabaseName() == DatabaseCatalog::SYSTEM_DATABASE); if (system_database.getUUID() == UUIDHelpers::Nil) { /// Attach to Ordinary database. @@ -57,4 +55,15 @@ void attach(ContextPtr context, IDatabase & system_database, const String & tabl } } +template +void attachLazyOrNot(ContextPtr context, IDatabase & system_database, const String & table_name, StorageArgs && ... args) +{ + assert(system_database.getDatabaseName() == DatabaseCatalog::SYSTEM_DATABASE); + + if constexpr (lazy) + attachLazy(context, system_database, table_name, std::forward(args)...); + else + attach(context, system_database, table_name, std::forward(args)...); +} + } From 100b36e8a912b090e79a3181058149afc1ddf47a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Dec 2023 01:32:42 +0100 Subject: [PATCH 100/110] Do not check in replication --- src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp | 6 +++--- src/Storages/StorageReplicatedMergeTree.cpp | 1 - 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp index 40cf64327d3..f4e4756279f 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp @@ -409,7 +409,7 @@ StorageInMemoryMetadata ReplicatedMergeTreeTableMetadata::Diff::getNewMetadata(c ParserTTLExpressionList parser; auto ttl_for_table_ast = parseQuery(parser, new_ttl_table, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); new_metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST( - ttl_for_table_ast, new_metadata.columns, context, new_metadata.primary_key, context->getSettingsRef().allow_suspicious_ttl_expressions); + ttl_for_table_ast, new_metadata.columns, context, new_metadata.primary_key, true /* allow_suspicious; because it is replication */); } else /// TTL was removed { @@ -422,7 +422,7 @@ StorageInMemoryMetadata ReplicatedMergeTreeTableMetadata::Diff::getNewMetadata(c new_metadata.column_ttls_by_name.clear(); for (const auto & [name, ast] : new_metadata.columns.getColumnTTLs()) { - auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, new_metadata.columns, context, new_metadata.primary_key, context->getSettingsRef().allow_suspicious_ttl_expressions); + auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, new_metadata.columns, context, new_metadata.primary_key, true /* allow_suspicious; because it is replication */); new_metadata.column_ttls_by_name[name] = new_ttl_entry; } @@ -454,7 +454,7 @@ StorageInMemoryMetadata ReplicatedMergeTreeTableMetadata::Diff::getNewMetadata(c if (!ttl_table_changed && new_metadata.table_ttl.definition_ast != nullptr) new_metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST( - new_metadata.table_ttl.definition_ast, new_metadata.columns, context, new_metadata.primary_key, context->getSettingsRef().allow_suspicious_ttl_expressions); + new_metadata.table_ttl.definition_ast, new_metadata.columns, context, new_metadata.primary_key, true /* allow_suspicious; because it is replication */); if (!projections_changed) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ac771cdd0ae..4fb21705534 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1379,7 +1379,6 @@ void StorageReplicatedMergeTree::setTableStructure(const StorageID & table_id, c checkTTLExpressions(new_metadata, old_metadata); setProperties(new_metadata, old_metadata); - DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata); } From b2572fceda315b7dee276bf63ca401960806f09d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Dec 2023 01:42:43 +0100 Subject: [PATCH 101/110] Add a test --- .../02296_ttl_non_deterministic.sql | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/tests/queries/0_stateless/02296_ttl_non_deterministic.sql b/tests/queries/0_stateless/02296_ttl_non_deterministic.sql index b5667aafcab..14d8979a624 100644 --- a/tests/queries/0_stateless/02296_ttl_non_deterministic.sql +++ b/tests/queries/0_stateless/02296_ttl_non_deterministic.sql @@ -1,15 +1,34 @@ +-- Tags: replica + DROP TABLE IF EXISTS t_ttl_non_deterministic; CREATE TABLE t_ttl_non_deterministic(A Int64) ENGINE = MergeTree ORDER BY A TTL now() + toIntervalMonth(1); -- {serverError BAD_ARGUMENTS} +CREATE TABLE t_ttl_non_deterministic(A Int64) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/ttl1', '1') ORDER BY A TTL now() + toIntervalMonth(1); -- {serverError BAD_ARGUMENTS} + + CREATE TABLE t_ttl_non_deterministic(A Int64) ENGINE = MergeTree ORDER BY A; ALTER TABLE t_ttl_non_deterministic MODIFY TTL now() + toIntervalMonth(1); -- {serverError BAD_ARGUMENTS} DROP TABLE t_ttl_non_deterministic; +CREATE TABLE t_ttl_non_deterministic(A Int64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/ttl2', '1') ORDER BY A; +ALTER TABLE t_ttl_non_deterministic MODIFY TTL now() + toIntervalMonth(1); -- {serverError BAD_ARGUMENTS} +DROP TABLE t_ttl_non_deterministic; + + CREATE TABLE t_ttl_non_deterministic(A Int64, B Int64 TTL now() + toIntervalMonth(1)) ENGINE = MergeTree ORDER BY A; -- {serverError BAD_ARGUMENTS} +CREATE TABLE t_ttl_non_deterministic(A Int64, B Int64 TTL now() + toIntervalMonth(1)) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/ttl3', '1') ORDER BY A; -- {serverError BAD_ARGUMENTS} + + CREATE TABLE t_ttl_non_deterministic(A Int64, B Int64) ENGINE = MergeTree ORDER BY A; ALTER TABLE t_ttl_non_deterministic MODIFY COLUMN B Int64 TTL now() + toIntervalMonth(1); -- {serverError BAD_ARGUMENTS} DROP TABLE t_ttl_non_deterministic; + +CREATE TABLE t_ttl_non_deterministic(A Int64, B Int64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/ttl4', '1') ORDER BY A; +ALTER TABLE t_ttl_non_deterministic MODIFY COLUMN B Int64 TTL now() + toIntervalMonth(1); -- {serverError BAD_ARGUMENTS} +DROP TABLE t_ttl_non_deterministic; From bfe2ffbce9fcc4650f6350aa1f26adc4067362f0 Mon Sep 17 00:00:00 2001 From: Bharat Nallan Chakravarthy Date: Sat, 9 Dec 2023 16:41:54 -0800 Subject: [PATCH 102/110] use the EVP interface to satisfy s390x builds --- src/Functions/FunctionsHashing.h | 20 +++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 3c43cab81e8..9468bc259e3 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -28,6 +28,11 @@ # include # include # include +#if USE_BORINGSSL +# include +#else +# include +#endif #endif #include @@ -322,14 +327,19 @@ struct SHA512Impl struct SHA512Impl256 { static constexpr auto name = "SHA512_256"; - enum { length = SHA512_256_DIGEST_LENGTH }; + enum { length = 32 }; static void apply(const char * begin, const size_t size, unsigned char * out_char_data) { - SHA512_CTX ctx; - SHA512_256_Init(&ctx); - SHA512_256_Update(&ctx, reinterpret_cast(begin), size); - SHA512_256_Final(out_char_data, &ctx); + /// Here, we use the EVP interface that is common to both BoringSSL and OpenSSL. Though BoringSSL is the default + /// SSL library that we use, for S390X architecture only OpenSSL is supported. But the SHA512-256, SHA512_256_Init, + /// SHA512_256_Update, SHA512_256_Final methods to calculate hash (similar to the other SHA functions) aren't available + /// in the current version of OpenSSL that we use which necessitates the use of the EVP interface. + auto md_ctx = EVP_MD_CTX_create(); + EVP_DigestInit_ex(md_ctx, EVP_sha512_256(), nullptr /*engine*/); + EVP_DigestUpdate(md_ctx, begin, size); + EVP_DigestFinal_ex(md_ctx, out_char_data, nullptr /*size*/); + EVP_MD_CTX_destroy(md_ctx); } }; #endif From fcdead023c4350233ef1e0f7f82a71653ed62229 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Dec 2023 07:57:38 +0100 Subject: [PATCH 103/110] Proper server termination in case of intermediate failures during initialization For instance on CI it fails because KeeperDispatch::initialize() failed, but there is already proper KeeperDispatch::shutdown() in dtor, the problem is that you first need to shutdown global context, that will terminate all thread pools, and only after you can join global thread pool. CI: https://s3.amazonaws.com/clickhouse-test-reports/52717/6ccbc2ea75c39234dff748cfb147bd44d65d3e75/integration_tests__asan__[3_4].html Signed-off-by: Azat Khuzhin --- programs/server/Server.cpp | 116 ++++++++++++++++++------------------- 1 file changed, 58 insertions(+), 58 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index bb16670f0b2..bc361ba31f7 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -657,6 +657,11 @@ try CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); + Poco::ThreadPool server_pool(3, server_settings.max_connections); + std::mutex servers_lock; + std::vector servers; + std::vector servers_to_start_before_tables; + /** Context contains all that query execution is dependent: * settings, available functions, data types, aggregate functions, databases, ... */ @@ -705,6 +710,59 @@ try LOG_INFO(log, "Background threads finished in {} ms", watch.elapsedMilliseconds()); }); + /// NOTE: global context should be destroyed *before* GlobalThreadPool::shutdown() + SCOPE_EXIT({ + /** Ask to cancel background jobs all table engines, + * and also query_log. + * It is important to do early, not in destructor of Context, because + * table engines could use Context on destroy. + */ + LOG_INFO(log, "Shutting down storages."); + + global_context->shutdown(); + + LOG_DEBUG(log, "Shut down storages."); + + if (!servers_to_start_before_tables.empty()) + { + LOG_DEBUG(log, "Waiting for current connections to servers for tables to finish."); + size_t current_connections = 0; + { + std::lock_guard lock(servers_lock); + for (auto & server : servers_to_start_before_tables) + { + server.stop(); + current_connections += server.currentConnections(); + } + } + + if (current_connections) + LOG_INFO(log, "Closed all listening sockets. Waiting for {} outstanding connections.", current_connections); + else + LOG_INFO(log, "Closed all listening sockets."); + + if (current_connections > 0) + current_connections = waitServersToFinish(servers_to_start_before_tables, servers_lock, server_settings.shutdown_wait_unfinished); + + if (current_connections) + LOG_INFO(log, "Closed connections to servers for tables. But {} remain. Probably some tables of other users cannot finish their connections after context shutdown.", current_connections); + else + LOG_INFO(log, "Closed connections to servers for tables."); + } + + global_context->shutdownKeeperDispatcher(); + + /// Wait server pool to avoid use-after-free of destroyed context in the handlers + server_pool.joinAll(); + + /** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available. + * At this moment, no one could own shared part of Context. + */ + global_context.reset(); + shared_context.reset(); + LOG_DEBUG(log, "Destroyed global context."); + }); + #if USE_AZURE_BLOB_STORAGE /// It makes sense to deinitialize libxml after joining of all threads @@ -763,10 +821,6 @@ try } } - Poco::ThreadPool server_pool(3, server_settings.max_connections); - std::mutex servers_lock; - std::vector servers; - std::vector servers_to_start_before_tables; /// This object will periodically calculate some metrics. ServerAsynchronousMetrics async_metrics( global_context, @@ -1606,60 +1660,6 @@ try /// try set up encryption. There are some errors in config, error will be printed and server wouldn't start. CompressionCodecEncrypted::Configuration::instance().load(config(), "encryption_codecs"); - SCOPE_EXIT({ - async_metrics.stop(); - - /** Ask to cancel background jobs all table engines, - * and also query_log. - * It is important to do early, not in destructor of Context, because - * table engines could use Context on destroy. - */ - LOG_INFO(log, "Shutting down storages."); - - global_context->shutdown(); - - LOG_DEBUG(log, "Shut down storages."); - - if (!servers_to_start_before_tables.empty()) - { - LOG_DEBUG(log, "Waiting for current connections to servers for tables to finish."); - size_t current_connections = 0; - { - std::lock_guard lock(servers_lock); - for (auto & server : servers_to_start_before_tables) - { - server.stop(); - current_connections += server.currentConnections(); - } - } - - if (current_connections) - LOG_INFO(log, "Closed all listening sockets. Waiting for {} outstanding connections.", current_connections); - else - LOG_INFO(log, "Closed all listening sockets."); - - if (current_connections > 0) - current_connections = waitServersToFinish(servers_to_start_before_tables, servers_lock, server_settings.shutdown_wait_unfinished); - - if (current_connections) - LOG_INFO(log, "Closed connections to servers for tables. But {} remain. Probably some tables of other users cannot finish their connections after context shutdown.", current_connections); - else - LOG_INFO(log, "Closed connections to servers for tables."); - } - - global_context->shutdownKeeperDispatcher(); - - /// Wait server pool to avoid use-after-free of destroyed context in the handlers - server_pool.joinAll(); - - /** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available. - * At this moment, no one could own shared part of Context. - */ - global_context.reset(); - shared_context.reset(); - LOG_DEBUG(log, "Destroyed global context."); - }); - /// DNSCacheUpdater uses BackgroundSchedulePool which lives in shared context /// and thus this object must be created after the SCOPE_EXIT object where shared /// context is destroyed. From ece8c86ebc02908e36786ca32d68f9ce0ddcb35d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Dec 2023 08:27:30 +0100 Subject: [PATCH 104/110] Fix attaching table in Atomic engine after attachTableUnlocked() interface --- src/Databases/DatabaseAtomic.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 9fd288ed9e5..1d78e4aff67 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -324,7 +324,7 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora /// It throws if `table_metadata_path` already exists (it's possible if table was detached) renameNoReplace(table_metadata_tmp_path, table_metadata_path); /// Commit point (a sort of) - attachTableUnlocked(query_context, query.getTable(), table, /*relative_table_path=*/ {}); /// Should never throw + DatabaseWithOwnTablesBase::attachTableUnlocked(query_context, query.getTable(), table, /*relative_table_path=*/ {}); /// Should never throw table_name_to_path.emplace(query.getTable(), table_data_path); } catch (...) From ce9fd2c57a678a7a684ec88889faac36469ae3f6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Dec 2023 08:33:07 +0100 Subject: [PATCH 105/110] Fix test_dictionaries_update_and_reload/test.py::test_reload_while_loading flakiness On CI this test fails due to start_time == prev_start_time: start_time, duration = get_loading_start_time("slow"), get_loading_duration("slow") > assert start_time > prev_start_time E assert time.struct_time(tm_year=2023, tm_mon=12, tm_mday=9, tm_hour=23, tm_min=42, tm_sec=2, tm_wday=5, tm_yday=343, tm_isdst=-1) > time.struct_time(tm_year=2023, tm_mon=12, tm_mday=9, tm_hour=23, tm_min=42, tm_sec=2, tm_wday=5, tm_yday=343, tm_isdst=-1) The reason I guess is that there is sleep(0.5), while the loading_start_time is DateTime not DateTime64, so you cannot distinguish values if the difference is less then one second CI: https://s3.amazonaws.com/clickhouse-test-reports/57710/7af1c0885daaf1e41470c5fdd92abfc7b6b2befc/integration_tests__asan__[3_4].html Signed-off-by: Azat Khuzhin --- .../test_dictionaries_update_and_reload/test.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_dictionaries_update_and_reload/test.py b/tests/integration/test_dictionaries_update_and_reload/test.py index 99d08f1b6ea..3d96d0b8dd4 100644 --- a/tests/integration/test_dictionaries_update_and_reload/test.py +++ b/tests/integration/test_dictionaries_update_and_reload/test.py @@ -92,16 +92,16 @@ def test_reload_while_loading(started_cluster): assert get_status("slow") == "NOT_LOADED" assert get_loading_duration("slow") == 0 - # It's not possible to get a value from the dictionary within 0.5 second, so the following query fails by timeout. + # It's not possible to get a value from the dictionary within 1 second, so the following query fails by timeout. with pytest.raises(QueryTimeoutExceedException): - query("SELECT dictGetInt32('slow', 'a', toUInt64(5))", timeout=0.5) + query("SELECT dictGetInt32('slow', 'a', toUInt64(5))", timeout=1) # The dictionary is now loading. assert get_status("slow") == "LOADING" start_time, duration = get_loading_start_time("slow"), get_loading_duration("slow") assert duration > 0 - time.sleep(0.5) # Still loading. + time.sleep(1) # Still loading. assert get_status("slow") == "LOADING" prev_start_time, prev_duration = start_time, duration start_time, duration = get_loading_start_time("slow"), get_loading_duration("slow") @@ -110,14 +110,14 @@ def test_reload_while_loading(started_cluster): # SYSTEM RELOAD DICTIONARY should restart loading. with pytest.raises(QueryTimeoutExceedException): - query("SYSTEM RELOAD DICTIONARY 'slow'", timeout=0.5) + query("SYSTEM RELOAD DICTIONARY 'slow'", timeout=1) assert get_status("slow") == "LOADING" prev_start_time, prev_duration = start_time, duration start_time, duration = get_loading_start_time("slow"), get_loading_duration("slow") assert start_time > prev_start_time assert duration < prev_duration - time.sleep(0.5) # Still loading. + time.sleep(1) # Still loading. assert get_status("slow") == "LOADING" prev_start_time, prev_duration = start_time, duration start_time, duration = get_loading_start_time("slow"), get_loading_duration("slow") @@ -128,7 +128,7 @@ def test_reload_while_loading(started_cluster): replace_in_file_in_container( "/etc/clickhouse-server/dictionaries/slow.xml", "sleep 100", "sleep 0" ) - time.sleep(5) # Configuration files are reloaded once in 5 seconds. + query("SYSTEM RELOAD CONFIG") # This time loading should finish quickly. assert get_status("slow") == "LOADED" From 8f5a60eb5ab3422b6b9e6242c45a14f4f220a419 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Dec 2023 15:37:10 +0100 Subject: [PATCH 106/110] More descriptive comment for destroying context prio global thread pool Signed-off-by: Azat Khuzhin --- programs/server/Server.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index bc361ba31f7..8076d108083 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -711,6 +711,7 @@ try }); /// NOTE: global context should be destroyed *before* GlobalThreadPool::shutdown() + /// Otherwise GlobalThreadPool::shutdown() will hang, since Context holds some threads. SCOPE_EXIT({ /** Ask to cancel background jobs all table engines, * and also query_log. From a6cf2297ef07ca0b065d944c9e30a3986285331d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Dec 2023 15:37:36 +0100 Subject: [PATCH 107/110] Fix Context use-after-free in ServerAsynchronousMetrics TSan found [1]: WARNING: ThreadSanitizer: data race on vptr (ctor/dtor vs virtual call) (pid=598) Write of size 8 at 0x7ffcf9ad9cb0 by main thread: 0 DB::AsynchronousMetrics::~AsynchronousMetrics() build_docker/./src/Common/AsynchronousMetrics.cpp:299:1 (clickhouse+0xf38ed6e) (BuildId: 31dcc7d77a0b2aaf9de7aca070b5f6ed6ac3dcbf) 1 DB::ServerAsynchronousMetrics::~ServerAsynchronousMetrics() build_docker/./src/Interpreters/ServerAsynchronousMetrics.h:10:7 (clickhouse+0xf2b2220) (BuildId: 31dcc7d77a0b2aaf9de7aca070b5f6ed6ac3dcbf) 2 DB::Server::main(std::__1::vector, std::__1::allocator>, std::__1::allocator, std::__1::allocator>>> const&) build_docker/./programs/server/Server.cpp:1995:1 (clickhouse+0xf2b2220) 3 Poco::Util::Application::run() build_docker/./base/poco/Util/src/Application.cpp:315:8 (clickhouse+0x1d6c535e) (BuildId: 31dcc7d77a0b2aaf9de7aca070b5f6ed6ac3dcbf) 4 DB::Server::run() build_docker/./programs/server/Server.cpp:397:25 (clickhouse+0xf29d341) (BuildId: 31dcc7d77a0b2aaf9de7aca070b5f6ed6ac3dcbf) 5 Poco::Util::ServerApplication::run(int, char**) build_docker/./base/poco/Util/src/ServerApplication.cpp:131:9 (clickhouse+0x1d6e50b4) (BuildId: 31dcc7d77a0b2aaf9de7aca070b5f6ed6ac3dcbf) 6 mainEntryClickHouseServer(int, char**) build_docker/./programs/server/Server.cpp:203:20 (clickhouse+0xf29a4a3) (BuildId: 31dcc7d77a0b2aaf9de7aca070b5f6ed6ac3dcbf) 7 main build_docker/./programs/main.cpp:505:12 (clickhouse+0x72d72a0) (BuildId: 31dcc7d77a0b2aaf9de7aca070b5f6ed6ac3dcbf) Previous read of size 8 at 0x7ffcf9ad9cb0 by thread T727: 0 DB::AsynchronousMetrics::update(std::__1::chrono::time_point>>) build_docker/./src/Common/AsynchronousMetrics.cpp:1559:5 (clickhouse+0xf38dec5) (BuildId: 31dcc7d77a0b2aaf9de7aca070b5f6ed6ac3dcbf) 1 DB::AsynchronousMetrics::run() build_docker/./src/Common/AsynchronousMetrics.cpp:354:13 (clickhouse+0xf38f996) (BuildId: 31dcc7d77a0b2aaf9de7aca070b5f6ed6ac3dcbf) 2 DB::AsynchronousMetrics::start()::$_0::operator()() const build_docker/./src/Common/AsynchronousMetrics.cpp:273:62 (clickhouse+0xf3921ca) (BuildId: 31dcc7d77a0b2aaf9de7aca070b5f6ed6ac3dcbf) 3 decltype(std::declval()()) std::__1::__invoke[abi:v15000](DB::AsynchronousMetrics::start()::$_0&) build_docker/./contrib/llvm-project/libcxx/include/__functional/invoke.h:394:23 (clickhouse+0xf3921ca) 4 decltype(auto) std::__1::__apply_tuple_impl[abi:v15000]&>(DB::AsynchronousMetrics::start()::$_0&, std::__1::tuple<>&, std::__1::__tuple_indices<>) build_docker/./contrib/llvm-project/libcxx/include/tuple:1789:1 (clickhouse+0xf3921ca) 5 decltype(auto) std::__1::apply[abi:v15000]&>(DB::AsynchronousMetrics::start()::$_0&, std::__1::tuple<>&) build_docker/./contrib/llvm-project/libcxx/include/tuple:1798:1 (clickhouse+0xf3921ca) 6 ThreadFromGlobalPoolImpl::ThreadFromGlobalPoolImpl(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'()::operator()() build_docker/./src/Common/ThreadPool.h:253:13 (clickhouse+0xf3921ca) 7 decltype(std::declval()()) std::__1::__invoke[abi:v15000]::ThreadFromGlobalPoolImpl(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'()&>(DB::AsynchronousMetrics::start()::$_0&&) build_docker/./contrib/llvm-project/libcxx/include/__functional/invoke.h:394:23 (clickhouse+0xf3921ca) 8 void std::__1::__invoke_void_return_wrapper::__call::ThreadFromGlobalPoolImpl(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'()&>(ThreadFromGlobalPoolImpl::ThreadFromGlobalPoolImpl(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'()&) build_docker/./contrib/llvm-project/libcxx/include/__functional/invoke.h:479:9 (clickhouse+0xf3921ca) 9 std::__1::__function::__default_alloc_func::ThreadFromGlobalPoolImpl(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'(), void ()>::operator()[abi:v15000]() build_docker/./contrib/llvm-project/libcxx/include/__functional/function.h:235:12 (clickhouse+0xf3921ca) 10 void std::__1::__function::__policy_invoker::__call_impl::ThreadFromGlobalPoolImpl(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'(), void ()>>(std::__1::__function::__policy_storage const*) build_docker/./contrib/llvm-project/libcxx/include/__functional/function.h:716:16 (clickhouse+0xf3921ca) 11 std::__1::__function::__policy_func::operator()[abi:v15000]() const build_docker/./contrib/llvm-project/libcxx/include/__functional/function.h:848:16 (clickhouse+0xf305b2e) (BuildId: 31dcc7d77a0b2aaf9de7aca070b5f6ed6ac3dcbf) 12 std::__1::function::operator()() const build_docker/./contrib/llvm-project/libcxx/include/__functional/function.h:1187:12 (clickhouse+0xf305b2e) 13 ThreadPoolImpl::worker(std::__1::__list_iterator) build_docker/./src/Common/ThreadPool.cpp:421:13 (clickhouse+0xf305b2e) 14 void ThreadPoolImpl::scheduleImpl(std::__1::function, Priority, std::__1::optional, bool)::'lambda0'()::operator()() const build_docker/./src/Common/ThreadPool.cpp:183:73 (clickhouse+0xf30c6d1) (BuildId: 31dcc7d77a0b2aaf9de7aca070b5f6ed6ac3dcbf) 15 decltype(std::declval()()) std::__1::__invoke[abi:v15000]::scheduleImpl(std::__1::function, Priority, std::__1::optional, bool)::'lambda0'()>(void&&) build_docker/./contrib/llvm-project/libcxx/include/__functional/invoke.h:394:23 (clickhouse+0xf30c6d1) 16 void std::__1::__thread_execute[abi:v15000]>, void ThreadPoolImpl::scheduleImpl(std::__1::function, Priority, std::__1::optional, bool)::'lambda0'()>(std::__1::tuple::scheduleImpl(std::__1::function, Priority, std::__1::optional, bool)::'lambda0'()>&, std::__1::__tuple_indices<>) build_docker/./contrib/llvm-project/libcxx/include/thread:284:5 (clickhouse+0xf30c6d1) 17 void* std::__1::__thread_proxy[abi:v15000]>, void ThreadPoolImpl::scheduleImpl(std::__1::function, Priority, std::__1::optional, bool)::'lambda0'()>>(void*) build_docker/./contrib/llvm-project/libcxx/include/thread:295:5 (clickhouse+0xf30c6d1) SUMMARY: ThreadSanitizer: data race on vptr (ctor/dtor vs virtual call) build_docker/./src/Common/AsynchronousMetrics.cpp:299:1 in DB::AsynchronousMetrics::~AsynchronousMetrics() Or MSan [2]: ==573==WARNING: MemorySanitizer: use-of-uninitialized-value 0 0x55b57079112d in std::__1::weak_ptr::lock() const build_docker/./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:1645:20 1 0x55b57079112d in DB::WithContextImpl>::getContext() const build_docker/./src/Interpreters/Context_fwd.h:41:28 2 0x55b57079112d in DB::ServerAsynchronousMetrics::updateImpl(std::__1::unordered_map, std::__1::allocator>, DB::AsynchronousMetricValue, std::__1::hash, std::__1::allocator>>, std::__1::equal_to, std::__1::allocator>>, std::__1::allocator, std::__1::allocator> const, DB::AsynchronousMetricValue>>>&, std::__1::chrono::time_point>>, std::__1::chrono::time_point>>) build_docker/./src/Interpreters/ServerAsynchronousMetrics.cpp:260:63 3 0x55b559540309 in DB::AsynchronousMetrics::update(std::__1::chrono::time_point>>) build_docker/./src/Common/AsynchronousMetrics.cpp:1559:5 4 0x55b55954258c in DB::AsynchronousMetrics::run() build_docker/./src/Common/AsynchronousMetrics.cpp:354:13 5 0x55b559549111 in DB::AsynchronousMetrics::start()::$_0::operator()() const build_docker/./src/Common/AsynchronousMetrics.cpp:273:62 6 0x55b559549111 in decltype(std::declval()()) std::__1::__invoke[abi:v15000](DB::AsynchronousMetrics::start()::$_0&) build_docker/./contrib/llvm-project/libcxx/include/__functional/invoke.h:394:23 7 0x55b559549111 in decltype(auto) std::__1::__apply_tuple_impl[abi:v15000]&>(DB::AsynchronousMetrics::start()::$_0&, std::__1::tuple<>&, std::__1::__tuple_indices<>) build_docker/./contrib/llvm-project/libcxx/include/tuple:1789:1 8 0x55b559549111 in decltype(auto) std::__1::apply[abi:v15000]&>(DB::AsynchronousMetrics::start()::$_0&, std::__1::tuple<>&) build_docker/./contrib/llvm-project/libcxx/include/tuple:1798:1 9 0x55b559549111 in ThreadFromGlobalPoolImpl::ThreadFromGlobalPoolImpl(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'()::operator()() build_docker/./src/Common/ThreadPool.h:253:13 10 0x55b559549111 in decltype(std::declval()()) std::__1::__invoke[abi:v15000]::ThreadFromGlobalPoolImpl(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'()&>(DB::AsynchronousMetrics::start()::$_0&&) build_docker/./contrib/llvm-project/libcxx/include/__functional/invoke.h:394:23 11 0x55b559549111 in void std::__1::__invoke_void_return_wrapper::__call::ThreadFromGlobalPoolImpl(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'()&>(ThreadFromGlobalPoolImpl::ThreadFromGlobalPoolImpl(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'()&) build_docker/./contrib/llvm-project/libcxx/include/__functional/invoke.h:479:9 12 0x55b559549111 in std::__1::__function::__default_alloc_func::ThreadFromGlobalPoolImpl(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'(), void ()>::operator()[abi:v15000]() build_docker/./contrib/llvm-project/libcxx/include/__functional/function.h:235:12 13 0x55b559549111 in void std::__1::__function::__policy_invoker::__call_impl::ThreadFromGlobalPoolImpl(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'(), void ()>>(std::__1::__function::__policy_storage const*) build_docker/./contrib/llvm-project/libcxx/include/__functional/function.h:716:16 14 0x55b5593eb38a in std::__1::__function::__policy_func::operator()[abi:v15000]() const build_docker/./contrib/llvm-project/libcxx/include/__functional/function.h:848:16 15 0x55b5593eb38a in std::__1::function::operator()() const build_docker/./contrib/llvm-project/libcxx/include/__functional/function.h:1187:12 16 0x55b5593eb38a in ThreadPoolImpl::worker(std::__1::__list_iterator) build_docker/./src/Common/ThreadPool.cpp:421:13 17 0x55b5593f9a0a in void ThreadPoolImpl::scheduleImpl(std::__1::function, Priority, std::__1::optional, bool)::'lambda0'()::operator()() const build_docker/./src/Common/ThreadPool.cpp:183:73 18 0x55b5593f9a0a in decltype(std::declval()()) std::__1::__invoke[abi:v15000]::scheduleImpl(std::__1::function, Priority, std::__1::optional, bool)::'lambda0'()>(void&&) build_docker/./contrib/llvm-project/libcxx/include/__functional/invoke.h:394:23 19 0x55b5593f9a0a in void std::__1::__thread_execute[abi:v15000]>, void ThreadPoolImpl::scheduleImpl(std::__1::function, Priority, std::__1::optional, bool)::'lambda0'()>(std::__1::tuple::scheduleImpl(std::__1::function, Priority, std::__1::optional, bool)::'lambda0'()>&, std::__1::__tuple_indices<>) build_docker/./contrib/llvm-project/libcxx/include/thread:284:5 20 0x55b5593f9a0a in void* std::__1::__thread_proxy[abi:v15000]>, void ThreadPoolImpl::scheduleImpl(std::__1::function, Priority, std::__1::optional, bool)::'lambda0'()>>(void*) build_docker/./contrib/llvm-project/libcxx/include/thread:295:5 21 0x7f7ff3899ac2 (/lib/x86_64-linux-gnu/libc.so.6+0x94ac2) (BuildId: a43bfc8428df6623cd498c9c0caeb91aec9be4f9) 22 0x7f7ff392ba3f (/lib/x86_64-linux-gnu/libc.so.6+0x126a3f) (BuildId: a43bfc8428df6623cd498c9c0caeb91aec9be4f9) Member fields were destroyed 0 0x55b541a72c9d in __sanitizer_dtor_callback_fields (/usr/bin/clickhouse+0x7c6dc9d) (BuildId: 57941f7730deefd0a2028f1d9f3e173472a4aa76) 1 0x55b559304b83 in std::__1::weak_ptr::~weak_ptr() build_docker/./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:1397:26 2 0x55b559304b83 in std::__1::weak_ptr::~weak_ptr() build_docker/./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:1553:1 3 0x55b559304b83 in DB::WithContextImpl>::~WithContextImpl() build_docker/./src/Interpreters/Context_fwd.h:30:8 4 0x55b559304b83 in DB::ServerAsynchronousMetrics::~ServerAsynchronousMetrics() build_docker/./src/Interpreters/ServerAsynchronousMetrics.h:10:7 5 0x55b559304b83 in DB::Server::main(std::__1::vector, std::__1::allocator>, std::__1::allocator, std::__1::allocator>>> const&) build_docker/./programs/server/Server.cpp:1995:1 6 0x55b5789f579e in Poco::Util::Application::run() build_docker/./base/poco/Util/src/Application.cpp:315:8 7 0x55b5592d762a in DB::Server::run() build_docker/./programs/server/Server.cpp:397:25 8 0x55b578a3efdf in Poco::Util::ServerApplication::run(int, char**) build_docker/./base/poco/Util/src/ServerApplication.cpp:131:9 9 0x55b5592d0489 in mainEntryClickHouseServer(int, char**) build_docker/./programs/server/Server.cpp:203:20 10 0x55b541acd4ab in main build_docker/./programs/main.cpp:505:12 11 0x7f7ff382ed8f (/lib/x86_64-linux-gnu/libc.so.6+0x29d8f) (BuildId: a43bfc8428df6623cd498c9c0caeb91aec9be4f9) SUMMARY: MemorySanitizer: use-of-uninitialized-value build_docker/./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:1645:20 in std::__1::weak_ptr::lock() const [1]: https://s3.amazonaws.com/clickhouse-test-reports/52717/fcdead023c4350233ef1e0f7f82a71653ed62229/stress_test__tsan_.html [2]: https://s3.amazonaws.com/clickhouse-test-reports/52717/fcdead023c4350233ef1e0f7f82a71653ed62229/stress_test__msan_.html Signed-off-by: Azat Khuzhin --- src/Interpreters/ServerAsynchronousMetrics.cpp | 4 ++-- src/Interpreters/ServerAsynchronousMetrics.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 84d31bae13f..2499eb5ec17 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -54,8 +54,8 @@ ServerAsynchronousMetrics::ServerAsynchronousMetrics( int update_period_seconds, int heavy_metrics_update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_) - : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_) - , WithContext(global_context_) + : WithContext(global_context_) + , AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_) , heavy_metric_update_period(heavy_metrics_update_period_seconds) { /// sanity check diff --git a/src/Interpreters/ServerAsynchronousMetrics.h b/src/Interpreters/ServerAsynchronousMetrics.h index 8243699a111..ce7411a1de0 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.h +++ b/src/Interpreters/ServerAsynchronousMetrics.h @@ -7,7 +7,7 @@ namespace DB { -class ServerAsynchronousMetrics : public AsynchronousMetrics, WithContext +class ServerAsynchronousMetrics : WithContext, public AsynchronousMetrics { public: ServerAsynchronousMetrics( From 2707e38c693e1eeb53e2c8c14ba87fa3514c756f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Dec 2023 16:00:19 +0100 Subject: [PATCH 108/110] Fix dead-lock in DatabaseMaterializedPostgreSQL after interface changes Deadlock for ATTACH: /src/ch/clickhouse/src/Common/ThreadFuzzer.cpp:0::pthread_mutex_lock /src/ch/clickhouse/contrib/llvm-project/libcxx/src/mutex.cpp:39::std::__1::mutex::lock() /src/ch/clickhouse/src/Databases/DatabaseOrdinary.cpp:240::DB::DatabaseOrdinary::waitTableStarted(std::__1::basic_string, std::__1::allocator> const&) const /src/ch/clickhouse/src/Databases/DatabasesCommon.cpp:203::DB::DatabaseWithOwnTablesBase::tryGetTable(std::__1::basic_string, std::__1::allocator> const&, std::__1::shared_ptr) const /src/ch/clickhouse/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp:335::DB::DatabaseMaterializedPostgreSQL::attachTableUnlocked(std::__1::shared_ptr, std::__1::basic_string, std::__1::allocator> const&, std::__1::shared_ptr const&, std::__1::basic_string, std::__1::allocator> const&) /src/ch/clickhouse/contrib/llvm-project/libcxx/include/__mutex_base:100::DB::IDatabase::attachTable(std::__1::shared_ptr, std::__1::basic_string, std::__1::allocator> const&, std::__1::shared_ptr const&, std::__1::basic_string, std::__1::allocator> const&) /src/ch/clickhouse/contrib/llvm-project/libcxx/include/string:1499::DB::DatabaseMaterializedPostgreSQL::createTable(std::__1::shared_ptr, std::__1::basic_string, std::__1::allocator> const&, std::__1::shared_ptr const&, std::__1::shared_ptr const&) /src/ch/clickhouse/contrib/llvm-project/libcxx/include/string:1499::DB::InterpreterCreateQuery::doCreateTable(DB::ASTCreateQuery&, DB::InterpreterCreateQuery::TableProperties const&, std::__1::unique_ptr>&) /src/ch/clickhouse/src/Interpreters/InterpreterCreateQuery.cpp:0::DB::InterpreterCreateQuery::createTable(DB::ASTCreateQuery&) /src/ch/clickhouse/src/Interpreters/InterpreterCreateQuery.cpp:1751::DB::InterpreterCreateQuery::execute() /src/ch/clickhouse/src/Interpreters/executeQuery.cpp:0::DB::executeQueryImpl(char const*, char const*, std::__1::shared_ptr, DB::QueryFlags, DB::QueryProcessingStage::Enum, DB::ReadBuffer*) /src/ch/clickhouse/src/Interpreters/executeQuery.cpp:1287::DB::executeQuery(std::__1::basic_string, std::__1::allocator> const&, std::__1::shared_ptr, DB::QueryFlags, DB::QueryProcessingStage::Enum) /src/ch/clickhouse/contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:612::DB::TCPHandler::runImpl() /src/ch/clickhouse/src/Server/TCPHandler.cpp:2294::DB::TCPHandler::run() /src/ch/clickhouse/base/poco/Net/src/TCPServerConnection.cpp:57::Poco::Net::TCPServerConnection::start() /src/ch/clickhouse/contrib/llvm-project/libcxx/include/__memory/unique_ptr.h:48::Poco::Net::TCPServerDispatcher::run() /src/ch/clickhouse/base/poco/Foundation/src/ThreadPool.cpp:202::Poco::PooledThread::run() /src/ch/clickhouse/base/poco/Foundation/include/Poco/SharedPtr.h:139::Poco::ThreadImpl::runnableEntry(void*) CI: https://s3.amazonaws.com/clickhouse-test-reports/55271/ece8c86ebc02908e36786ca32d68f9ce0ddcb35d/integration_tests__tsan__[1_6].html Signed-off-by: Azat Khuzhin --- src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp | 4 ++-- src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 52914d54478..78be0611631 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -322,7 +322,7 @@ void DatabaseMaterializedPostgreSQL::createTable(ContextPtr local_context, const } -void DatabaseMaterializedPostgreSQL::attachTableUnlocked(ContextPtr context_, const String & table_name, const StoragePtr & table, const String & relative_table_path) +void DatabaseMaterializedPostgreSQL::attachTable(ContextPtr context_, const String & table_name, const StoragePtr & table, const String & relative_table_path) { /// If there is query context then we need to attach materialized storage. /// If there is no query context then we need to attach internal storage from atomic database. @@ -362,7 +362,7 @@ void DatabaseMaterializedPostgreSQL::attachTableUnlocked(ContextPtr context_, co } else { - DatabaseAtomic::attachTableUnlocked(context_, table_name, table, relative_table_path); + DatabaseAtomic::attachTable(context_, table_name, table, relative_table_path); } } diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h index 52d7224cbdf..8feb5a014e1 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h @@ -50,6 +50,8 @@ public: void createTable(ContextPtr context, const String & table_name, const StoragePtr & table, const ASTPtr & query) override; + void attachTable(ContextPtr context, const String & table_name, const StoragePtr & table, const String & relative_table_path) override; + void detachTablePermanently(ContextPtr context, const String & table_name) override; StoragePtr detachTable(ContextPtr context, const String & table_name) override; @@ -69,8 +71,6 @@ public: String getPostgreSQLDatabaseName() const { return remote_database_name; } protected: - void attachTableUnlocked(ContextPtr context, const String & table_name, const StoragePtr & table, const String & relative_table_path) TSA_REQUIRES(mutex) override; - ASTPtr getCreateTableQueryImpl(const String & table_name, ContextPtr local_context, bool throw_on_error) const override; private: From 450e8da7ce60acfca52c8fb5ae1801c492ad2cc7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Dec 2023 18:17:41 +0100 Subject: [PATCH 109/110] Fix data race on vptr (ctor/dtor vs virtual call) in AsynchronousMetrics TSan report [1]: Exception: Sanitizer assert found for instance ================== WARNING: ThreadSanitizer: data race on vptr (ctor/dtor vs virtual call) (pid=1) Write of size 8 at 0x7ffce68f5680 by main thread: 0 DB::AsynchronousMetrics::~AsynchronousMetrics() build_docker/./src/Common/AsynchronousMetrics.cpp:299:1 (clickhouse+0xf38fa6e) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4) 1 DB::ServerAsynchronousMetrics::~ServerAsynchronousMetrics() build_docker/./src/Interpreters/ServerAsynchronousMetrics.h:10:7 (clickhouse+0xf2b2f0f) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4) 2 DB::Server::main(std::__1::vector, std::__1::allocator>, std::__1::allocator, std::__1::allocator>>> const&) build_docker/./programs/server/Server.cpp:1996:1 (clickhouse+0xf2b2f0f) 3 Poco::Util::Application::run() build_docker/./base/poco/Util/src/Application.cpp:315:8 (clickhouse+0x1d6c95fe) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4) 4 DB::Server::run() build_docker/./programs/server/Server.cpp:397:25 (clickhouse+0xf29e041) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4) 5 Poco::Util::ServerApplication::run(int, char**) build_docker/./base/poco/Util/src/ServerApplication.cpp:131:9 (clickhouse+0x1d6e9354) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4) 6 mainEntryClickHouseServer(int, char**) build_docker/./programs/server/Server.cpp:203:20 (clickhouse+0xf29b1a3) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4) 7 main build_docker/./programs/main.cpp:505:12 (clickhouse+0x72d7fa0) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4) Previous read of size 8 at 0x7ffce68f5680 by thread T649: 0 DB::AsynchronousMetrics::update(std::__1::chrono::time_point>>) build_docker/./src/Common/AsynchronousMetrics.cpp:1559:5 (clickhouse+0xf38ebc5) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4) 1 DB::AsynchronousMetrics::run() build_docker/./src/Common/AsynchronousMetrics.cpp:354:13 (clickhouse+0xf390696) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4) 2 DB::AsynchronousMetrics::start()::$_0::operator()() const build_docker/./src/Common/AsynchronousMetrics.cpp:273:62 (clickhouse+0xf392eca) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4) 3 decltype(std::declval()()) std::__1::__invoke[abi:v15000](DB::AsynchronousMetrics::start()::$_0&) build_docker/./contrib/llvm-project/libcxx/include/__functional/invoke.h:394:23 (clickhouse+0xf392eca) 4 decltype(auto) std::__1::__apply_tuple_impl[abi:v15000]&>(DB::AsynchronousMetrics::start()::$_0&, std::__1::tuple<>&, std::__1::__tuple_indices<>) build_docker/./contrib/llvm-project/libcxx/include/tuple:1789:1 (clickhouse+0xf392eca) 5 decltype(auto) std::__1::apply[abi:v15000]&>(DB::AsynchronousMetrics::start()::$_0&, std::__1::tuple<>&) build_docker/./contrib/llvm-project/libcxx/include/tuple:1798:1 (clickhouse+0xf392eca) 6 ThreadFromGlobalPoolImpl::ThreadFromGlobalPoolImpl(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'()::operator()() build_docker/./src/Common/ThreadPool.h:253:13 (clickhouse+0xf392eca) 7 decltype(std::declval()()) std::__1::__invoke[abi:v15000]::ThreadFromGlobalPoolImpl(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'()&>(DB::AsynchronousMetrics::start()::$_0&&) build_docker/./contrib/llvm-project/libcxx/include/__functional/invoke.h:394:23 (clickhouse+0xf392eca) 8 void std::__1::__invoke_void_return_wrapper::__call::ThreadFromGlobalPoolImpl(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'()&>(ThreadFromGlobalPoolImpl::ThreadFromGlobalPoolImpl(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'()&) build_docker/./contrib/llvm-project/libcxx/include/__functional/invoke.h:479:9 (clickhouse+0xf392eca) 9 std::__1::__function::__default_alloc_func::ThreadFromGlobalPoolImpl(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'(), void ()>::operator()[abi:v15000]() build_docker/./contrib/llvm-project/libcxx/include/__functional/function.h:235:12 (clickhouse+0xf392eca) 10 void std::__1::__function::__policy_invoker::__call_impl::ThreadFromGlobalPoolImpl(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'(), void ()>>(std::__1::__function::__policy_storage const*) build_docker/./contrib/llvm-project/libcxx/include/__functional/function.h:716:16 (clickhouse+0xf392eca) 11 std::__1::__function::__policy_func::operator()[abi:v15000]() const build_docker/./contrib/llvm-project/libcxx/include/__functional/function.h:848:16 (clickhouse+0xf30682e) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4) 12 std::__1::function::operator()() const build_docker/./contrib/llvm-project/libcxx/include/__functional/function.h:1187:12 (clickhouse+0xf30682e) 13 ThreadPoolImpl::worker(std::__1::__list_iterator) build_docker/./src/Common/ThreadPool.cpp:421:13 (clickhouse+0xf30682e) 14 void ThreadPoolImpl::scheduleImpl(std::__1::function, Priority, std::__1::optional, bool)::'lambda0'()::operator()() const build_docker/./src/Common/ThreadPool.cpp:183:73 (clickhouse+0xf30d3d1) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4) 15 decltype(std::declval()()) std::__1::__invoke[abi:v15000]::scheduleImpl(std::__1::function, Priority, std::__1::optional, bool)::'lambda0'()>(void&&) build_docker/./contrib/llvm-project/libcxx/include/__functional/invoke.h:394:23 (clickhouse+0xf30d3d1) 16 void std::__1::__thread_execute[abi:v15000]>, void ThreadPoolImpl::scheduleImpl(std::__1::function, Priority, std::__1::optional, bool)::'lambda0'()>(std::__1::tuple::scheduleImpl(std::__1::function, Priority, std::__1::optional, bool)::'lambda0'()>&, std::__1::__tuple_indices<>) build_docker/./contrib/llvm-project/libcxx/include/thread:284:5 (clickhouse+0xf30d3d1) 17 void* std::__1::__thread_proxy[abi:v15000]>, void ThreadPoolImpl::scheduleImpl(std::__1::function, Priority, std::__1::optional, bool)::'lambda0'()>>(void*) build_docker/./contrib/llvm-project/libcxx/include/thread:295:5 (clickhouse+0xf30d3d1) SUMMARY: ThreadSanitizer: data race on vptr (ctor/dtor vs virtual call) build_docker/./src/Common/AsynchronousMetrics.cpp:299:1 in DB::AsynchronousMetrics::~AsynchronousMetrics() And the problem is real, see [2]. [1]: https://s3.amazonaws.com/clickhouse-test-reports/52717/a6cf2297ef07ca0b065d944c9e30a3986285331d/integration_tests__tsan__[2_6]/integration_run_parallel0_0.log [2]: https://github.com/google/sanitizers/wiki/ThreadSanitizerPopularDataRaces#data-race-on-vptr Signed-off-by: Azat Khuzhin --- src/Coordination/KeeperAsynchronousMetrics.cpp | 6 ++++++ src/Coordination/KeeperAsynchronousMetrics.h | 1 + src/Interpreters/ServerAsynchronousMetrics.cpp | 6 ++++++ src/Interpreters/ServerAsynchronousMetrics.h | 2 ++ 4 files changed, 15 insertions(+) diff --git a/src/Coordination/KeeperAsynchronousMetrics.cpp b/src/Coordination/KeeperAsynchronousMetrics.cpp index 890079e98f7..4471012e917 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.cpp +++ b/src/Coordination/KeeperAsynchronousMetrics.cpp @@ -113,6 +113,12 @@ KeeperAsynchronousMetrics::KeeperAsynchronousMetrics( { } +KeeperAsynchronousMetrics::~KeeperAsynchronousMetrics() +{ + /// NOTE: stop() from base class is not enough, since this leads to leak on vptr + stop(); +} + void KeeperAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values, TimePoint /*update_time*/, TimePoint /*current_time*/) { #if USE_NURAFT diff --git a/src/Coordination/KeeperAsynchronousMetrics.h b/src/Coordination/KeeperAsynchronousMetrics.h index 14092c11c15..457a7112507 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.h +++ b/src/Coordination/KeeperAsynchronousMetrics.h @@ -14,6 +14,7 @@ class KeeperAsynchronousMetrics : public AsynchronousMetrics public: KeeperAsynchronousMetrics( ContextPtr context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_); + ~KeeperAsynchronousMetrics() override; private: ContextPtr context; diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 2499eb5ec17..8cf7dc39d97 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -63,6 +63,12 @@ ServerAsynchronousMetrics::ServerAsynchronousMetrics( throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Setting asynchronous_metrics_update_period_s and asynchronous_heavy_metrics_update_period_s must not be zero"); } +ServerAsynchronousMetrics::~ServerAsynchronousMetrics() +{ + /// NOTE: stop() from base class is not enough, since this leads to leak on vptr + stop(); +} + void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) { if (auto mark_cache = getContext()->getMarkCache()) diff --git a/src/Interpreters/ServerAsynchronousMetrics.h b/src/Interpreters/ServerAsynchronousMetrics.h index ce7411a1de0..a579d12de2c 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.h +++ b/src/Interpreters/ServerAsynchronousMetrics.h @@ -15,6 +15,8 @@ public: int update_period_seconds, int heavy_metrics_update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_); + ~ServerAsynchronousMetrics() override; + private: void updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) override; void logImpl(AsynchronousMetricValues & new_values) override; From 644ef7b63f7a96a7675d178f1e329e0895824c4f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Dec 2023 23:51:59 +0300 Subject: [PATCH 110/110] Revert "Fix RWLock inconsistency after write lock timeout" --- src/Common/RWLock.cpp | 151 +++--------- src/Common/RWLock.h | 23 +- src/Common/tests/gtest_rw_lock.cpp | 230 ------------------ src/Storages/IStorage.cpp | 4 +- .../test_concurrency.py | 15 +- 5 files changed, 47 insertions(+), 376 deletions(-) diff --git a/src/Common/RWLock.cpp b/src/Common/RWLock.cpp index c8a5c692494..2d0fcfa3e74 100644 --- a/src/Common/RWLock.cpp +++ b/src/Common/RWLock.cpp @@ -3,8 +3,6 @@ #include #include #include -#include -#include namespace ProfileEvents @@ -157,34 +155,25 @@ RWLockImpl::getLock(RWLockImpl::Type type, const String & query_id, const std::c if (type == Type::Write) { - /// Always add a group for a writer (writes are never performed simultaneously). writers_queue.emplace_back(type); /// SM1: may throw (nothing to roll back) } - else + else if (readers_queue.empty() || + (rdlock_owner == readers_queue.begin() && readers_queue.size() == 1 && !writers_queue.empty())) { - /// We don't always add a group to readers_queue here because multiple readers can use the same group. - /// We can reuse the last group if the last group didn't get ownership yet, - /// or even if it got ownership but there are no writers waiting in writers_queue. - bool can_use_last_group = !readers_queue.empty() && (!readers_queue.back().ownership || writers_queue.empty()); - - if (!can_use_last_group) - readers_queue.emplace_back(type); /// SM1: may throw (nothing to roll back) + readers_queue.emplace_back(type); /// SM1: may throw (nothing to roll back) } - GroupsContainer::iterator it_group = (type == Type::Write) ? std::prev(writers_queue.end()) : std::prev(readers_queue.end()); /// Lock is free to acquire if (rdlock_owner == readers_queue.end() && wrlock_owner == writers_queue.end()) { - /// Set `rdlock_owner` or `wrlock_owner` and make it owner. (type == Read ? rdlock_owner : wrlock_owner) = it_group; /// SM2: nothrow - grantOwnership(it_group); } else { /// Wait until our group becomes the lock owner - const auto predicate = [&] () { return it_group->ownership; }; + const auto predicate = [&] () { return it_group == (type == Read ? rdlock_owner : wrlock_owner); }; if (lock_deadline_tp == std::chrono::time_point::max()) { @@ -204,20 +193,15 @@ RWLockImpl::getLock(RWLockImpl::Type type, const String & query_id, const std::c /// Rollback(SM1): nothrow if (it_group->requests == 0) { - ((type == Read) ? readers_queue : writers_queue).erase(it_group); + /// When WRITE lock fails, we need to notify next read that is waiting, + /// to avoid handing request, hence next=true. + dropOwnerGroupAndPassOwnership(it_group, /* next= */ true); } - /// While we were waiting for this write lock (which has just failed) more readers could start waiting, - /// we need to wake up them now. - if ((rdlock_owner != readers_queue.end()) && writers_queue.empty()) - grantOwnershipToAllReaders(); return nullptr; } } } - /// Our group must be an owner here. - chassert(it_group->ownership); - if (request_has_query_id) { try @@ -232,7 +216,7 @@ RWLockImpl::getLock(RWLockImpl::Type type, const String & query_id, const std::c /// Methods std::list<>::emplace_back() and std::unordered_map<>::emplace() provide strong exception safety /// We only need to roll back the changes to these objects: owner_queries and the readers/writers queue if (it_group->requests == 0) - dropOwnerGroupAndPassOwnership(it_group); /// Rollback(SM1): nothrow + dropOwnerGroupAndPassOwnership(it_group, /* next= */ false); /// Rollback(SM1): nothrow throw; } @@ -253,28 +237,19 @@ RWLockImpl::getLock(RWLockImpl::Type type, const String & query_id, const std::c * it is guaranteed that all three steps have been executed successfully and the resulting state is consistent. * With the mutex locked the order of steps to restore the lock's state can be arbitrary * - * We do not employ try-catch: if something bad happens and chassert() is disabled, there is nothing we can do - * (we can't throw an exception here because RWLockImpl::unlock() is called from the destructor ~LockHolderImpl). + * We do not employ try-catch: if something bad happens, there is nothing we can do =( */ void RWLockImpl::unlock(GroupsContainer::iterator group_it, const String & query_id) noexcept { std::lock_guard state_lock(internal_state_mtx); - /// Our group must be an owner here. - if (!group_it->ownership) - { - chassert(false && "RWLockImpl::unlock() is called for a non-owner group"); + /// All of these are Undefined behavior and nothing we can do! + if (rdlock_owner == readers_queue.end() && wrlock_owner == writers_queue.end()) return; - } - - /// Check consistency. - if ((group_it->type == Read) - ? !(rdlock_owner != readers_queue.end() && wrlock_owner == writers_queue.end()) - : !(wrlock_owner != writers_queue.end() && rdlock_owner == readers_queue.end() && group_it == wrlock_owner)) - { - chassert(false && "RWLockImpl::unlock() found the rwlock inconsistent"); + if (rdlock_owner != readers_queue.end() && group_it != rdlock_owner) + return; + if (wrlock_owner != writers_queue.end() && group_it != wrlock_owner) return; - } /// If query_id is not empty it must be listed in parent->owner_queries if (query_id != NO_QUERY) @@ -289,26 +264,12 @@ void RWLockImpl::unlock(GroupsContainer::iterator group_it, const String & query /// If we are the last remaining referrer, remove this QNode and notify the next one if (--group_it->requests == 0) /// SM: nothrow - dropOwnerGroupAndPassOwnership(group_it); + dropOwnerGroupAndPassOwnership(group_it, /* next= */ false); } -void RWLockImpl::dropOwnerGroupAndPassOwnership(GroupsContainer::iterator group_it) noexcept +void RWLockImpl::dropOwnerGroupAndPassOwnership(GroupsContainer::iterator group_it, bool next) noexcept { - /// All readers with ownership must finish before switching to write phase. - /// Such readers has iterators from `readers_queue.begin()` to `rdlock_owner`, so if `rdlock_owner` is equal to `readers_queue.begin()` - /// that means there is only one reader with ownership left in the readers_queue and we can proceed to generic procedure. - if ((group_it->type == Read) && (rdlock_owner != readers_queue.begin()) && (rdlock_owner != readers_queue.end())) - { - if (rdlock_owner == group_it) - --rdlock_owner; - readers_queue.erase(group_it); - /// If there are no writers waiting in writers_queue then we can wake up other readers. - if (writers_queue.empty()) - grantOwnershipToAllReaders(); - return; - } - rdlock_owner = readers_queue.end(); wrlock_owner = writers_queue.end(); @@ -317,86 +278,42 @@ void RWLockImpl::dropOwnerGroupAndPassOwnership(GroupsContainer::iterator group_ readers_queue.erase(group_it); /// Prepare next phase if (!writers_queue.empty()) + { wrlock_owner = writers_queue.begin(); + } else + { rdlock_owner = readers_queue.begin(); + } } else { writers_queue.erase(group_it); /// Prepare next phase if (!readers_queue.empty()) - rdlock_owner = readers_queue.begin(); + { + if (next && readers_queue.size() > 1) + { + rdlock_owner = std::next(readers_queue.begin()); + } + else + { + rdlock_owner = readers_queue.begin(); + } + } else + { wrlock_owner = writers_queue.begin(); + } } if (rdlock_owner != readers_queue.end()) { - grantOwnershipToAllReaders(); + rdlock_owner->cv.notify_all(); } else if (wrlock_owner != writers_queue.end()) { - grantOwnership(wrlock_owner); + wrlock_owner->cv.notify_one(); } } - - -void RWLockImpl::grantOwnership(GroupsContainer::iterator group_it) noexcept -{ - if (!group_it->ownership) - { - group_it->ownership = true; - group_it->cv.notify_all(); - } -} - - -void RWLockImpl::grantOwnershipToAllReaders() noexcept -{ - if (rdlock_owner != readers_queue.end()) - { - size_t num_new_owners = 0; - - for (;;) - { - if (!rdlock_owner->ownership) - ++num_new_owners; - grantOwnership(rdlock_owner); - if (std::next(rdlock_owner) == readers_queue.end()) - break; - ++rdlock_owner; - } - - /// There couldn't be more than one reader group which is not an owner. - /// (Because we add a new reader group only if the last reader group is already an owner - see the `can_use_last_group` variable.) - chassert(num_new_owners <= 1); - } -} - - -std::unordered_map RWLockImpl::getOwnerQueryIds() const -{ - std::lock_guard lock{internal_state_mtx}; - return owner_queries; -} - - -String RWLockImpl::getOwnerQueryIdsDescription() const -{ - auto map = getOwnerQueryIds(); - WriteBufferFromOwnString out; - bool need_comma = false; - for (const auto & [query_id, num_owners] : map) - { - if (need_comma) - out << ", "; - out << query_id; - if (num_owners != 1) - out << " (" << num_owners << ")"; - need_comma = true; - } - return out.str(); -} - } diff --git a/src/Common/RWLock.h b/src/Common/RWLock.h index 1eb6964a991..156e4297d81 100644 --- a/src/Common/RWLock.h +++ b/src/Common/RWLock.h @@ -62,42 +62,35 @@ public: inline static const String NO_QUERY = String(); inline static const auto default_locking_timeout_ms = std::chrono::milliseconds(120000); - /// Returns all query_id owning locks (both read and write) right now. - /// !! This function are for debugging and logging purposes only, DO NOT use them for synchronization! - std::unordered_map getOwnerQueryIds() const; - String getOwnerQueryIdsDescription() const; - private: /// Group of locking requests that should be granted simultaneously /// i.e. one or several readers or a single writer struct Group { const Type type; - size_t requests = 0; + size_t requests; - bool ownership = false; /// whether this group got ownership? (that means `cv` is notified and the locking requests should stop waiting) std::condition_variable cv; /// all locking requests of the group wait on this condvar - explicit Group(Type type_) : type{type_} {} + explicit Group(Type type_) : type{type_}, requests{0} {} }; using GroupsContainer = std::list; - using OwnerQueryIds = std::unordered_map; + using OwnerQueryIds = std::unordered_map; mutable std::mutex internal_state_mtx; GroupsContainer readers_queue; GroupsContainer writers_queue; - GroupsContainer::iterator rdlock_owner{readers_queue.end()}; /// last group with ownership in readers_queue in read phase - /// or readers_queue.end() in writer phase + GroupsContainer::iterator rdlock_owner{readers_queue.end()}; /// equals to readers_queue.begin() in read phase + /// or readers_queue.end() otherwise GroupsContainer::iterator wrlock_owner{writers_queue.end()}; /// equals to writers_queue.begin() in write phase - /// or writers_queue.end() in read phase + /// or writers_queue.end() otherwise OwnerQueryIds owner_queries; RWLockImpl() = default; void unlock(GroupsContainer::iterator group_it, const String & query_id) noexcept; - void dropOwnerGroupAndPassOwnership(GroupsContainer::iterator group_it) noexcept; - void grantOwnership(GroupsContainer::iterator group_it) noexcept; - void grantOwnershipToAllReaders() noexcept; + /// @param next - notify next after begin, used on writer lock failures + void dropOwnerGroupAndPassOwnership(GroupsContainer::iterator group_it, bool next) noexcept; }; } diff --git a/src/Common/tests/gtest_rw_lock.cpp b/src/Common/tests/gtest_rw_lock.cpp index 16ba01d02c6..5ea50f70d4e 100644 --- a/src/Common/tests/gtest_rw_lock.cpp +++ b/src/Common/tests/gtest_rw_lock.cpp @@ -24,39 +24,6 @@ namespace DB } -namespace -{ - class Events - { - public: - Events() : start_time(std::chrono::steady_clock::now()) {} - - void add(String && event) - { - String timepoint = std::to_string(std::chrono::duration_cast(std::chrono::steady_clock::now() - start_time).count()); - if (timepoint.length() < 5) - timepoint.insert(0, 5 - timepoint.length(), ' '); - std::lock_guard lock{mutex}; - //std::cout << timepoint << " : " << event << std::endl; - events.emplace_back(std::move(event)); - } - - void check(const Strings & expected_events) - { - std::lock_guard lock{mutex}; - EXPECT_EQ(events.size(), expected_events.size()); - for (size_t i = 0; i != events.size(); ++i) - EXPECT_EQ(events[i], (i < expected_events.size() ? expected_events[i] : "")); - } - - private: - const std::chrono::time_point start_time; - Strings events TSA_GUARDED_BY(mutex); - mutable std::mutex mutex; - }; -} - - TEST(Common, RWLock1) { /// Tests with threads require this, because otherwise @@ -320,200 +287,3 @@ TEST(Common, RWLockNotUpgradeableWithNoQuery) read_thread.join(); } - - -TEST(Common, RWLockWriteLockTimeoutDuringRead) -{ - static auto rw_lock = RWLockImpl::create(); - Events events; - - std::thread ra_thread([&] () - { - events.add("Locking ra"); - auto ra = rw_lock->getLock(RWLockImpl::Read, "ra"); - events.add(ra ? "Locked ra" : "Failed to lock ra"); - EXPECT_NE(ra, nullptr); - - std::this_thread::sleep_for(std::chrono::duration(400)); - - events.add("Unlocking ra"); - ra.reset(); - events.add("Unlocked ra"); - }); - - std::thread wc_thread([&] () - { - std::this_thread::sleep_for(std::chrono::duration(100)); - events.add("Locking wc"); - auto wc = rw_lock->getLock(RWLockImpl::Write, "wc", std::chrono::milliseconds(200)); - events.add(wc ? "Locked wc" : "Failed to lock wc"); - EXPECT_EQ(wc, nullptr); - }); - - ra_thread.join(); - wc_thread.join(); - - { - events.add("Locking wd"); - auto wd = rw_lock->getLock(RWLockImpl::Write, "wd", std::chrono::milliseconds(1000)); - events.add(wd ? "Locked wd" : "Failed to lock wd"); - EXPECT_NE(wd, nullptr); - events.add("Unlocking wd"); - wd.reset(); - events.add("Unlocked wd"); - } - - events.check( - {"Locking ra", - "Locked ra", - "Locking wc", - "Failed to lock wc", - "Unlocking ra", - "Unlocked ra", - "Locking wd", - "Locked wd", - "Unlocking wd", - "Unlocked wd"}); -} - - -TEST(Common, RWLockWriteLockTimeoutDuringTwoReads) -{ - static auto rw_lock = RWLockImpl::create(); - Events events; - - std::thread ra_thread([&] () - { - events.add("Locking ra"); - auto ra = rw_lock->getLock(RWLockImpl::Read, "ra"); - events.add(ra ? "Locked ra" : "Failed to lock ra"); - EXPECT_NE(ra, nullptr); - - std::this_thread::sleep_for(std::chrono::duration(400)); - - events.add("Unlocking ra"); - ra.reset(); - events.add("Unlocked ra"); - }); - - std::thread rb_thread([&] () - { - std::this_thread::sleep_for(std::chrono::duration(200)); - events.add("Locking rb"); - - auto rb = rw_lock->getLock(RWLockImpl::Read, "rb"); - events.add(rb ? "Locked rb" : "Failed to lock rb"); - EXPECT_NE(rb, nullptr); - - std::this_thread::sleep_for(std::chrono::duration(200)); - events.add("Unlocking rb"); - rb.reset(); - events.add("Unlocked rb"); - }); - - std::thread wc_thread([&] () - { - std::this_thread::sleep_for(std::chrono::duration(100)); - events.add("Locking wc"); - auto wc = rw_lock->getLock(RWLockImpl::Write, "wc", std::chrono::milliseconds(200)); - events.add(wc ? "Locked wc" : "Failed to lock wc"); - EXPECT_EQ(wc, nullptr); - }); - - ra_thread.join(); - rb_thread.join(); - wc_thread.join(); - - { - events.add("Locking wd"); - auto wd = rw_lock->getLock(RWLockImpl::Write, "wd", std::chrono::milliseconds(1000)); - events.add(wd ? "Locked wd" : "Failed to lock wd"); - EXPECT_NE(wd, nullptr); - events.add("Unlocking wd"); - wd.reset(); - events.add("Unlocked wd"); - } - - events.check( - {"Locking ra", - "Locked ra", - "Locking wc", - "Locking rb", - "Failed to lock wc", - "Locked rb", - "Unlocking ra", - "Unlocked ra", - "Unlocking rb", - "Unlocked rb", - "Locking wd", - "Locked wd", - "Unlocking wd", - "Unlocked wd"}); -} - - -TEST(Common, RWLockWriteLockTimeoutDuringWriteWithWaitingRead) -{ - static auto rw_lock = RWLockImpl::create(); - Events events; - - std::thread wa_thread([&] () - { - events.add("Locking wa"); - auto wa = rw_lock->getLock(RWLockImpl::Write, "wa"); - events.add(wa ? "Locked wa" : "Failed to lock wa"); - EXPECT_NE(wa, nullptr); - - std::this_thread::sleep_for(std::chrono::duration(500)); - - events.add("Unlocking wa"); - wa.reset(); - events.add("Unlocked wa"); - }); - - std::thread wb_thread([&] () - { - std::this_thread::sleep_for(std::chrono::duration(100)); - events.add("Locking wb"); - auto wc = rw_lock->getLock(RWLockImpl::Write, "wc", std::chrono::milliseconds(200)); - events.add(wc ? "Locked wb" : "Failed to lock wb"); - EXPECT_EQ(wc, nullptr); - }); - - std::thread rc_thread([&] () - { - std::this_thread::sleep_for(std::chrono::duration(200)); - events.add("Locking rc"); - auto rc = rw_lock->getLock(RWLockImpl::Read, "rc", std::chrono::milliseconds(200)); - events.add(rc ? "Locked rc" : "Failed to lock rc"); - EXPECT_EQ(rc, nullptr); - }); - - wa_thread.join(); - wb_thread.join(); - rc_thread.join(); - - { - events.add("Locking wd"); - auto wd = rw_lock->getLock(RWLockImpl::Write, "wd", std::chrono::milliseconds(1000)); - events.add(wd ? "Locked wd" : "Failed to lock wd"); - EXPECT_NE(wd, nullptr); - events.add("Unlocking wd"); - wd.reset(); - events.add("Unlocked wd"); - } - - events.check( - {"Locking wa", - "Locked wa", - "Locking wb", - "Locking rc", - "Failed to lock wb", - "Failed to lock rc", - "Unlocking wa", - "Unlocked wa", - "Locking wd", - "Locked wd", - "Unlocking wd", - "Unlocked wd"}); -} diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 88603d56ebb..af1c032bc56 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -41,8 +41,8 @@ RWLockImpl::LockHolder IStorage::tryLockTimed( { const String type_str = type == RWLockImpl::Type::Read ? "READ" : "WRITE"; throw Exception(ErrorCodes::DEADLOCK_AVOIDED, - "{} locking attempt on \"{}\" has timed out! ({}ms) Possible deadlock avoided. Client should retry. Owner query ids: {}", - type_str, getStorageID(), acquire_timeout.count(), rwlock->getOwnerQueryIdsDescription()); + "{} locking attempt on \"{}\" has timed out! ({}ms) Possible deadlock avoided. Client should retry", + type_str, getStorageID(), acquire_timeout.count()); } return lock_holder; } diff --git a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py index c08f3c9c242..ab37846db9a 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_concurrency.py @@ -216,7 +216,7 @@ def test_create_or_drop_tables_during_backup(db_engine, table_engine): node = nodes[randint(0, num_nodes - 1)] # "DROP TABLE IF EXISTS" still can throw some errors (e.g. "WRITE locking attempt on node0 has timed out!") # So we use query_and_get_answer_with_error() to ignore any errors. - # `lock_acquire_timeout` is reduced because we don't wait our test to wait too long. + # `lock_acquire_timeout` is also reduced because we don't wait our test to wait too long. node.query_and_get_answer_with_error( f"DROP TABLE IF EXISTS {table_name} SYNC", settings={"lock_acquire_timeout": 10}, @@ -227,24 +227,15 @@ def test_create_or_drop_tables_during_backup(db_engine, table_engine): table_name1 = f"mydb.tbl{randint(1, num_nodes)}" table_name2 = f"mydb.tbl{randint(1, num_nodes)}" node = nodes[randint(0, num_nodes - 1)] - # `lock_acquire_timeout` is reduced because we don't wait our test to wait too long. node.query_and_get_answer_with_error( - f"RENAME TABLE {table_name1} TO {table_name2}", - settings={"lock_acquire_timeout": 10}, + f"RENAME TABLE {table_name1} TO {table_name2}" ) def truncate_tables(): while time.time() < end_time: table_name = f"mydb.tbl{randint(1, num_nodes)}" node = nodes[randint(0, num_nodes - 1)] - # "TRUNCATE TABLE IF EXISTS" still can throw some errors - # (e.g. "WRITE locking attempt on node0 has timed out!" if the table engine is "Log"). - # So we use query_and_get_answer_with_error() to ignore any errors. - # `lock_acquire_timeout` is reduced because we don't wait our test to wait too long. - node.query_and_get_answer_with_error( - f"TRUNCATE TABLE IF EXISTS {table_name} SYNC", - settings={"lock_acquire_timeout": 10}, - ) + node.query(f"TRUNCATE TABLE IF EXISTS {table_name} SYNC") def make_backups(): ids = []