From 92777691b59d02654c09ab685124ead47852e5b9 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 17 May 2022 10:16:40 +0000 Subject: [PATCH 001/331] 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/331] 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/331] 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/331] 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/331] 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/331] 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/331] 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/331] 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/331] 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/331] 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/331] 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/331] 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 6934e27e8b94de95b3a76d096e7c7a7006a0ea34 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 20 Oct 2023 20:46:41 +0000 Subject: [PATCH 013/331] Add union mode for schema inference to infer union schema of files with different schemas --- src/Core/Settings.h | 7 +- src/Core/SettingsChangesHistory.h | 3 + src/Core/SettingsEnums.cpp | 4 + src/Core/SettingsEnums.h | 8 + src/Formats/ReadSchemaUtils.cpp | 108 +++++-- src/Formats/ReadSchemaUtils.h | 24 +- src/Formats/SchemaInferenceUtils.cpp | 58 ++++ src/Formats/SchemaInferenceUtils.h | 14 +- .../FunctionGenerateRandomStructure.cpp | 12 +- src/Processors/Formats/ISchemaReader.cpp | 14 +- src/Processors/Formats/ISchemaReader.h | 5 +- .../Impl/JSONColumnsBlockInputFormatBase.cpp | 5 + .../Impl/JSONColumnsBlockInputFormatBase.h | 3 +- .../Impl/JSONCompactEachRowRowInputFormat.cpp | 5 + .../Impl/JSONCompactEachRowRowInputFormat.h | 1 + .../Impl/JSONEachRowRowInputFormat.cpp | 5 + .../Formats/Impl/JSONEachRowRowInputFormat.h | 1 + .../Formats/Impl/MySQLDumpRowInputFormat.cpp | 5 + .../Formats/Impl/MySQLDumpRowInputFormat.h | 1 + .../Formats/Impl/RegexpRowInputFormat.h | 1 - .../Formats/Impl/ValuesBlockInputFormat.cpp | 5 + .../Formats/Impl/ValuesBlockInputFormat.h | 1 + .../RowInputFormatWithNamesAndTypes.cpp | 5 + .../Formats/RowInputFormatWithNamesAndTypes.h | 2 + src/Storages/Cache/SchemaCache.h | 5 +- src/Storages/HDFS/StorageHDFS.cpp | 154 +++++----- src/Storages/HDFS/StorageHDFS.h | 13 - src/Storages/StorageAzureBlob.cpp | 166 +++++----- src/Storages/StorageAzureBlob.h | 15 - src/Storages/StorageFile.cpp | 285 ++++++++++-------- src/Storages/StorageFile.h | 10 - src/Storages/StorageS3.cpp | 198 ++++++------ src/Storages/StorageS3.h | 15 - src/Storages/StorageURL.cpp | 187 ++++++------ src/Storages/StorageURL.h | 15 - .../StorageSystemSchemaInferenceCache.cpp | 4 +- .../test_storage_azure_blob_storage/test.py | 57 ++++ tests/integration/test_storage_hdfs/test.py | 49 +++ tests/integration/test_storage_s3/test.py | 62 ++++ ...2900_union_schema_inference_mode.reference | 33 ++ .../02900_union_schema_inference_mode.sh | 57 ++++ 41 files changed, 1029 insertions(+), 593 deletions(-) create mode 100644 tests/queries/0_stateless/02900_union_schema_inference_mode.reference create mode 100755 tests/queries/0_stateless/02900_union_schema_inference_mode.sh diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2c45108f9a5..e5b7c5ff30b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -893,11 +893,11 @@ class IColumn; M(Bool, input_format_parquet_preserve_order, false, "Avoid reordering rows when reading from Parquet files. Usually makes it much slower.", 0) \ M(Bool, input_format_parquet_filter_push_down, true, "When reading Parquet files, skip whole row groups based on the WHERE/PREWHERE expressions and min/max statistics in the Parquet metadata.", 0) \ M(Bool, input_format_allow_seeks, true, "Allow seeks while reading in ORC/Parquet/Arrow input formats", 0) \ - M(Bool, input_format_orc_allow_missing_columns, false, "Allow missing columns while reading ORC input formats", 0) \ + M(Bool, input_format_orc_allow_missing_columns, true, "Allow missing columns while reading ORC input formats", 0) \ M(Bool, input_format_orc_use_fast_decoder, true, "Use a faster ORC decoder implementation.", 0) \ - M(Bool, input_format_parquet_allow_missing_columns, false, "Allow missing columns while reading Parquet input formats", 0) \ + M(Bool, input_format_parquet_allow_missing_columns, true, "Allow missing columns while reading Parquet input formats", 0) \ M(UInt64, input_format_parquet_local_file_min_bytes_for_seek, 8192, "Min bytes required for local read (file) to do seek, instead of read with ignore in Parquet input format", 0) \ - M(Bool, input_format_arrow_allow_missing_columns, false, "Allow missing columns while reading Arrow input formats", 0) \ + M(Bool, input_format_arrow_allow_missing_columns, true, "Allow missing columns while reading Arrow input formats", 0) \ M(Char, input_format_hive_text_fields_delimiter, '\x01', "Delimiter between fields in Hive Text File", 0) \ M(Char, input_format_hive_text_collection_items_delimiter, '\x02', "Delimiter between collection(array or map) items in Hive Text File", 0) \ M(Char, input_format_hive_text_map_keys_delimiter, '\x03', "Delimiter between a pair of map key/values in Hive Text File", 0) \ @@ -925,6 +925,7 @@ class IColumn; M(Bool, input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Arrow", 0) \ M(String, column_names_for_schema_inference, "", "The list of column names to use in schema inference for formats without column names. The format: 'column1,column2,column3,...'", 0) \ M(String, schema_inference_hints, "", "The list of column names and types to use in schema inference for formats without column names. The format: 'column_name1 column_type1, column_name2 column_type2, ...'", 0) \ + M(SchemaInferenceMode, schema_inference_mode, "default", "Mode of schema inference. 'default' - assume that all files have the same schema and schema can be inferred from any file, 'union' - files can have different schemas and the resulting schema should be the a union of schemas of all files", 0) \ M(Bool, schema_inference_make_columns_nullable, true, "If set to true, all inferred types will be Nullable in schema inference for formats without information about nullability.", 0) \ M(Bool, input_format_json_read_bools_as_numbers, true, "Allow to parse bools as numbers in JSON input formats", 0) \ M(Bool, input_format_json_try_infer_numbers_from_strings, false, "Try to infer numbers from string fields while schema inference", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 38039839e1e..caa1b28b1c5 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -80,6 +80,9 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { + {"23.10", {{"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, + {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, + {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}}, {"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/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 82e158877c5..7f1162cc3ce 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -190,4 +190,8 @@ IMPLEMENT_SETTING_ENUM(ExternalCommandStderrReaction, ErrorCodes::BAD_ARGUMENTS, {"log_last", ExternalCommandStderrReaction::LOG_LAST}, {"throw", ExternalCommandStderrReaction::THROW}}) +IMPLEMENT_SETTING_ENUM(SchemaInferenceMode, ErrorCodes::BAD_ARGUMENTS, + {{"default", SchemaInferenceMode::DEFAULT}, + {"union", SchemaInferenceMode::UNION}}) + } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 7db8c88c53d..af0c73e686a 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -242,4 +242,12 @@ DECLARE_SETTING_ENUM(S3QueueAction) DECLARE_SETTING_ENUM(ExternalCommandStderrReaction) +enum class SchemaInferenceMode +{ + DEFAULT, + UNION, +}; + +DECLARE_SETTING_ENUM(SchemaInferenceMode) + } diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index b185007eda7..f2e831dfc46 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -1,12 +1,9 @@ -#include #include #include #include -#include #include -#include #include - +#include namespace DB { @@ -55,6 +52,10 @@ ColumnsDescription readSchemaFromFormat( try { NamesAndTypesList names_and_types; + SchemaInferenceMode mode = context->getSettingsRef().schema_inference_mode; + if (mode == SchemaInferenceMode::UNION && !FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name, context, format_settings)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "UNION schema inference mode is not supported for format {}, because it doesn't support reading subset of columns", format_name); + if (FormatFactory::instance().checkIfFormatHasExternalSchemaReader(format_name)) { auto external_schema_reader = FormatFactory::instance().getExternalSchemaReader(format_name, context, format_settings); @@ -71,6 +72,11 @@ try } else if (FormatFactory::instance().checkIfFormatHasSchemaReader(format_name)) { + if (mode == SchemaInferenceMode::UNION) + retry = false; + + std::vector> schemas_for_union_mode; + std::optional cached_columns; std::string exception_messages; SchemaReaderPtr schema_reader; size_t max_rows_to_read = format_settings ? format_settings->max_rows_to_read_for_schema_inference @@ -84,7 +90,15 @@ try try { read_buffer_iterator.setPreviousReadBuffer(std::move(buf)); - buf = read_buffer_iterator.next(); + std::tie(buf, cached_columns) = read_buffer_iterator.next(); + if (cached_columns) + { + if (mode == SchemaInferenceMode::DEFAULT) + return *cached_columns; + schemas_for_union_mode.emplace_back(cached_columns->getAll(), read_buffer_iterator.getLastFileName()); + continue; + } + if (!buf) break; @@ -136,12 +150,19 @@ try auto num_rows = schema_reader->readNumberOrRows(); if (num_rows) read_buffer_iterator.setNumRowsToLastFile(*num_rows); - break; + + /// In default mode, we finish when schema is inferred successfully from any file. + if (mode == SchemaInferenceMode::DEFAULT) + break; + + if (!names_and_types.empty()) + read_buffer_iterator.setSchemaToLastFile(ColumnsDescription(names_and_types)); + schemas_for_union_mode.emplace_back(names_and_types, read_buffer_iterator.getLastFileName()); } catch (...) { auto exception_message = getCurrentExceptionMessage(false); - if (schema_reader) + if (schema_reader && mode == SchemaInferenceMode::DEFAULT) { size_t rows_read = schema_reader->getNumRowsRead(); assert(rows_read <= max_rows_to_read); @@ -190,8 +211,58 @@ try } } - if (auto cached_columns = read_buffer_iterator.getCachedColumns()) - return *cached_columns; + /// If we got all schemas from cache, schema_reader can be uninitialized. + /// But we still need some stateless methods of ISchemaReader, + /// let's initialize it with empty buffer. + EmptyReadBuffer empty; + if (!schema_reader) + schema_reader = FormatFactory::instance().getSchemaReader(format_name, empty, context, format_settings); + + if (mode == SchemaInferenceMode::UNION) + { + Names names_order; /// Try to save original columns order; + std::unordered_map names_to_types; + + + for (const auto & [schema, file_name] : schemas_for_union_mode) + { + for (const auto & [name, type] : schema) + { + auto it = names_to_types.find(name); + if (it == names_to_types.end()) + { + names_order.push_back(name); + names_to_types[name] = type; + } + else + { + /// We already have column with such name. + /// Check if types are the same. + if (!type->equals(*it->second)) + { + /// If types are not the same, try to transform them according + /// to the format to find common type. + auto new_type_copy = type; + schema_reader->transformTypesFromDifferentFilesIfNeeded(it->second, new_type_copy); + + /// If types are not the same after transform, we cannot do anything, throw an exception. + if (!it->second->equals(*new_type_copy)) + throw Exception( + ErrorCodes::TYPE_MISMATCH, + "Automatically inferred type {} for column '{}'{} differs from type inferred from previous files: {}", + type->getName(), + name, + file_name.empty() ? "" : " in file " + file_name, + it->second->getName()); + } + } + } + } + + names_and_types.clear(); + for (const auto & name : names_order) + names_and_types.emplace_back(name, names_to_types[name]); + } if (names_and_types.empty()) throw Exception( @@ -206,7 +277,7 @@ try /// It will allow to execute simple data loading with query /// "INSERT INTO table SELECT * FROM ..." const auto & insertion_table = context->getInsertionTable(); - if (!schema_reader->hasStrictOrderOfColumns() && !insertion_table.empty()) + if (schema_reader && !schema_reader->hasStrictOrderOfColumns() && !insertion_table.empty()) { auto storage = DatabaseCatalog::instance().getTable(insertion_table, context); auto metadata = storage->getInMemoryMetadataPtr(); @@ -226,13 +297,15 @@ try names_and_types.erase( std::remove_if(names_and_types.begin(), names_and_types.end(), [](const NameAndTypePair & pair) { return pair.name.empty(); }), names_and_types.end()); - return ColumnsDescription(names_and_types); + + auto columns = ColumnsDescription(names_and_types); + if (mode == SchemaInferenceMode::DEFAULT) + read_buffer_iterator.setResultingSchema(columns); + return columns; } catch (Exception & e) { - if (!buf) - throw; - auto file_name = getFileNameFromReadBuffer(*buf); + auto file_name = read_buffer_iterator.getLastFileName(); if (!file_name.empty()) e.addMessage(fmt::format("(in file/uri {})", file_name)); throw; @@ -256,9 +329,9 @@ SchemaCache::Key getKeyForSchemaCache( return getKeysForSchemaCache({source}, format, format_settings, context).front(); } -static SchemaCache::Key makeSchemaCacheKey(const String & source, const String & format, const String & additional_format_info) +static SchemaCache::Key makeSchemaCacheKey(const String & source, const String & format, const String & additional_format_info, const String & schema_inference_mode) { - return SchemaCache::Key{source, format, additional_format_info}; + return SchemaCache::Key{source, format, additional_format_info, schema_inference_mode}; } SchemaCache::Keys getKeysForSchemaCache( @@ -270,13 +343,14 @@ SchemaCache::Keys getKeysForSchemaCache( /// For example, for Protobuf format additional information is the path to the schema /// and message name. String additional_format_info = FormatFactory::instance().getAdditionalInfoForSchemaCache(format, context, format_settings); + String schema_inference_mode(magic_enum::enum_name(context->getSettingsRef().schema_inference_mode.value)); SchemaCache::Keys cache_keys; cache_keys.reserve(sources.size()); std::transform( sources.begin(), sources.end(), std::back_inserter(cache_keys), - [&](const auto & source) { return makeSchemaCacheKey(source, format, additional_format_info); }); + [&](const auto & source) { return makeSchemaCacheKey(source, format, additional_format_info, schema_inference_mode); }); return cache_keys; } diff --git a/src/Formats/ReadSchemaUtils.h b/src/Formats/ReadSchemaUtils.h index c769846acbb..aade6b28fb0 100644 --- a/src/Formats/ReadSchemaUtils.h +++ b/src/Formats/ReadSchemaUtils.h @@ -13,11 +13,23 @@ struct IReadBufferIterator virtual void setPreviousReadBuffer(std::unique_ptr /* buffer */) {} - virtual std::unique_ptr next() = 0; - - virtual std::optional getCachedColumns() { return std::nullopt; } + /// Return read buffer of the next file or cached schema. + /// In DEFAULT schema inference mode cached schema can be from any file. + /// In UNION mode cached schema can be only from current file. + /// When there is no files to process, return pair (nullptr, nullopt) + virtual std::pair, std::optional> next() = 0; virtual void setNumRowsToLastFile(size_t /*num_rows*/) {} + + /// Set schema inferred from last file. Used for UNION mode to cache schema + /// per file. + virtual void setSchemaToLastFile(const ColumnsDescription & /*columns*/) {} + /// Set resulting inferred schema. Used for DEFAULT mode to cache schema + /// for all files. + virtual void setResultingSchema(const ColumnsDescription & /*columns*/) {} + + /// Get last processed file name for better exception messages. + virtual String getLastFileName() const { return ""; } }; struct SingleReadBufferIterator : public IReadBufferIterator @@ -27,12 +39,12 @@ public: { } - std::unique_ptr next() override + std::pair, std::optional> next() override { if (done) - return nullptr; + return {nullptr, {}}; done = true; - return std::move(buf); + return {std::move(buf), {}}; } private: diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 94166aa9002..13871904c56 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -547,6 +547,54 @@ namespace } } + void mergeNamedTuples(DataTypes & data_types, TypeIndexesSet & type_indexes, const FormatSettings & settings, JSONInferenceInfo * json_info) + { + if (!type_indexes.contains(TypeIndex::Tuple)) + return; + + /// Collect all names and their types from all named tuples. + std::unordered_map names_to_types; + /// Try to save original order of element names. + Names element_names; + for (auto & type : data_types) + { + const auto * tuple_type = typeid_cast(type.get()); + if (tuple_type && tuple_type->haveExplicitNames()) + { + const auto & elements = tuple_type->getElements(); + const auto & names = tuple_type->getElementNames(); + for (size_t i = 0; i != elements.size(); ++i) + { + if (!names_to_types.contains(names[i])) + element_names.push_back(names[i]); + names_to_types[names[i]].push_back(elements[i]); + } + } + } + + /// Try to find common type for each tuple element with the same name. + DataTypes element_types; + element_types.reserve(names_to_types.size()); + for (const auto & name : element_names) + { + auto types = names_to_types[name]; + transformInferredTypesIfNeededImpl(types, settings, json_info); + /// If some element have different types in different tuples, we can't do anything + if (!checkIfTypesAreEqual(types)) + return; + element_types.push_back(types.front()); + } + + DataTypePtr result_tuple = std::make_shared(element_types, element_names); + + for (auto & type : data_types) + { + const auto * tuple_type = typeid_cast(type.get()); + if (tuple_type && tuple_type->haveExplicitNames()) + type = result_tuple; + } + } + template void transformInferredTypesIfNeededImpl(DataTypes & types, const FormatSettings & settings, JSONInferenceInfo * json_info) { @@ -604,6 +652,9 @@ namespace if (settings.json.read_objects_as_strings) transformMapsAndStringsToStrings(data_types, type_indexes); + + if (json_info && json_info->allow_merging_named_tuples) + mergeNamedTuples(data_types, type_indexes, settings, json_info); }; transformTypesRecursively(types, transform_simple_types, transform_complex_types); @@ -1180,6 +1231,13 @@ void transformInferredJSONTypesIfNeeded( second = std::move(types[1]); } +void transformInferredJSONTypesFromDifferentFilesIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings) +{ + JSONInferenceInfo json_info; + json_info.allow_merging_named_tuples = true; + transformInferredJSONTypesIfNeeded(first, second, settings, &json_info); +} + void transformFinalInferredJSONTypeIfNeededImpl(DataTypePtr & data_type, const FormatSettings & settings, JSONInferenceInfo * json_info, bool remain_nothing_types = false) { if (!data_type) diff --git a/src/Formats/SchemaInferenceUtils.h b/src/Formats/SchemaInferenceUtils.h index efeb6c9c873..b492d9b22b6 100644 --- a/src/Formats/SchemaInferenceUtils.h +++ b/src/Formats/SchemaInferenceUtils.h @@ -14,6 +14,11 @@ struct JSONInferenceInfo std::unordered_set numbers_parsed_from_json_strings; /// Indicates if currently we are inferring type for Map/Object key. bool is_object_key = false; + /// When we transform types for the same column from different files + /// we cannot use DataTypeJSONPaths for inferring named tuples from JSON objects, + /// because DataTypeJSONPaths was already finalized to named tuple. IN this case + /// we can only merge named tuples from different files together. + bool allow_merging_named_tuples = false; }; /// Try to determine datatype of the value in buffer/string. If the type cannot be inferred, return nullptr. @@ -64,9 +69,7 @@ void transformInferredTypesIfNeeded(DataTypePtr & first, DataTypePtr & second, c /// from strings in json_info while inference and use it here, so we will know that Array(Int64) contains /// integer inferred from a string. /// Example 2: -/// When we have maps with different value types, we convert all types to JSON object type. -/// For example, if we have Map(String, UInt64) (like `{"a" : 123}`) and Map(String, String) (like `{"b" : 'abc'}`) -/// we will convert both types to Object('JSON'). +/// We merge DataTypeJSONPaths types to a single DataTypeJSONPaths type with union of all JSON paths. void transformInferredJSONTypesIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings, JSONInferenceInfo * json_info); /// Make final transform for types inferred in JSON format. It does 3 types of transformation: @@ -78,6 +81,11 @@ void transformInferredJSONTypesIfNeeded(DataTypePtr & first, DataTypePtr & secon /// 3) Converts all Nothing types to String types if input_format_json_infer_incomplete_types_as_strings is enabled. void transformFinalInferredJSONTypeIfNeeded(DataTypePtr & data_type, const FormatSettings & settings, JSONInferenceInfo * json_info); +/// Transform types for the same column inferred from different files. +/// Does the same as transformInferredJSONTypesIfNeeded, but also merges named Tuples together, +/// because DataTypeJSONPaths types were finalized when we finished inference for a file. +void transformInferredJSONTypesFromDifferentFilesIfNeeded(DataTypePtr & first, DataTypePtr & second, const FormatSettings & settings); + /// Make type Nullable recursively: /// - Type -> Nullable(type) /// - Array(Type) -> Array(Nullable(Type)) diff --git a/src/Functions/FunctionGenerateRandomStructure.cpp b/src/Functions/FunctionGenerateRandomStructure.cpp index f85b2596530..8e086f075a4 100644 --- a/src/Functions/FunctionGenerateRandomStructure.cpp +++ b/src/Functions/FunctionGenerateRandomStructure.cpp @@ -34,7 +34,7 @@ namespace const size_t MAX_DECIMAL256_PRECISION = 76; const size_t MAX_DEPTH = 16; - constexpr std::array simple_types + constexpr std::array simple_types { TypeIndex::Int8, TypeIndex::UInt8, @@ -64,7 +64,7 @@ namespace TypeIndex::Enum16, TypeIndex::IPv4, TypeIndex::IPv6, - TypeIndex::UUID, +// TypeIndex::UUID, }; constexpr std::array complex_types @@ -76,7 +76,7 @@ namespace TypeIndex::Map, }; - constexpr std::array map_key_types + constexpr std::array map_key_types { TypeIndex::Int8, TypeIndex::UInt8, @@ -98,11 +98,11 @@ namespace TypeIndex::IPv4, TypeIndex::Enum8, TypeIndex::Enum16, - TypeIndex::UUID, +// TypeIndex::UUID, TypeIndex::LowCardinality, }; - constexpr std::array suspicious_lc_types + constexpr std::array suspicious_lc_types { TypeIndex::Int8, TypeIndex::UInt8, @@ -125,7 +125,7 @@ namespace TypeIndex::FixedString, TypeIndex::IPv4, TypeIndex::IPv6, - TypeIndex::UUID, +// TypeIndex::UUID, }; template diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index 15b53c2a499..26c632b83dc 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -62,6 +63,14 @@ void checkFinalInferredType( type = removeNullable(type); } +void ISchemaReader::transformTypesIfNeeded(DB::DataTypePtr & type, DB::DataTypePtr & new_type) +{ + DataTypes types = {type, new_type}; + auto least_supertype = tryGetLeastSupertype(types); + if (least_supertype) + type = new_type = least_supertype; +} + IIRowSchemaReader::IIRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_, DataTypePtr default_type_) : ISchemaReader(in_) , max_rows_to_read(format_settings_.max_rows_to_read_for_schema_inference) @@ -86,11 +95,6 @@ void IIRowSchemaReader::setContext(ContextPtr & context) } } -void IIRowSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) -{ - transformInferredTypesIfNeeded(type, new_type, format_settings); -} - IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) : IIRowSchemaReader(in_, format_settings_), column_names(splitColumnNames(format_settings.column_names_for_schema_inference)) { diff --git a/src/Processors/Formats/ISchemaReader.h b/src/Processors/Formats/ISchemaReader.h index e6402ac0249..94df71a88b4 100644 --- a/src/Processors/Formats/ISchemaReader.h +++ b/src/Processors/Formats/ISchemaReader.h @@ -39,6 +39,9 @@ public: virtual void setMaxRowsAndBytesToRead(size_t, size_t) {} virtual size_t getNumRowsRead() const { return 0; } + virtual void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type); + virtual void transformTypesFromDifferentFilesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) { transformTypesIfNeeded(type, new_type); } + virtual ~ISchemaReader() = default; protected: @@ -55,8 +58,6 @@ public: bool needContext() const override { return !hints_str.empty(); } void setContext(ContextPtr & context) override; - virtual void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type); - protected: void setMaxRowsAndBytesToRead(size_t max_rows, size_t max_bytes) override { diff --git a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.cpp b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.cpp index 26bd0847fb7..1c148f5b3d3 100644 --- a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.cpp +++ b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.cpp @@ -230,6 +230,11 @@ void JSONColumnsSchemaReaderBase::transformTypesIfNeeded(DataTypePtr & type, Dat transformInferredJSONTypesIfNeeded(type, new_type, format_settings, &inference_info); } +void JSONColumnsSchemaReaderBase::transformTypesFromDifferentFilesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) +{ + transformInferredJSONTypesFromDifferentFilesIfNeeded(type, new_type, format_settings); +} + NamesAndTypesList JSONColumnsSchemaReaderBase::readSchema() { std::unordered_map names_to_types; diff --git a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h index bb52e2aa516..53d65bb3539 100644 --- a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h +++ b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h @@ -80,7 +80,8 @@ class JSONColumnsSchemaReaderBase : public ISchemaReader public: JSONColumnsSchemaReaderBase(ReadBuffer & in_, const FormatSettings & format_settings_, std::unique_ptr reader_); - void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type); + void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override; + void transformTypesFromDifferentFilesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override; bool needContext() const override { return !hints_str.empty(); } void setContext(ContextPtr & ctx) override; diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index 99186d0eb6d..b301b9527c1 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -228,6 +228,11 @@ void JSONCompactEachRowRowSchemaReader::transformTypesIfNeeded(DataTypePtr & typ transformInferredJSONTypesIfNeeded(type, new_type, format_settings, &inference_info); } +void JSONCompactEachRowRowSchemaReader::transformTypesFromDifferentFilesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) +{ + transformInferredJSONTypesFromDifferentFilesIfNeeded(type, new_type, format_settings); +} + void JSONCompactEachRowRowSchemaReader::transformFinalTypeIfNeeded(DataTypePtr & type) { transformFinalInferredJSONTypeIfNeeded(type, format_settings, &inference_info); diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h index 2e255a55d57..463d3c53a65 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h @@ -92,6 +92,7 @@ private: std::optional readRowAndGetDataTypesImpl() override; void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override; + void transformTypesFromDifferentFilesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override; void transformFinalTypeIfNeeded(DataTypePtr & type) override; JSONCompactEachRowFormatReader reader; diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index 30b08cd1d9c..95563fd2f62 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -365,6 +365,11 @@ void JSONEachRowSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTyp transformInferredJSONTypesIfNeeded(type, new_type, format_settings, &inference_info); } +void JSONEachRowSchemaReader::transformTypesFromDifferentFilesIfNeeded(DB::DataTypePtr & type, DB::DataTypePtr & new_type) +{ + transformInferredJSONTypesFromDifferentFilesIfNeeded(type, new_type, format_settings); +} + void JSONEachRowSchemaReader::transformFinalTypeIfNeeded(DataTypePtr & type) { transformFinalInferredJSONTypeIfNeeded(type, format_settings, &inference_info); diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h index ad494d07fbc..3ff1b6d317c 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h @@ -104,6 +104,7 @@ public: private: NamesAndTypesList readRowAndGetNamesAndDataTypes(bool & eof) override; void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override; + void transformTypesFromDifferentFilesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override; void transformFinalTypeIfNeeded(DataTypePtr & type) override; bool first_row = true; diff --git a/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.cpp b/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.cpp index 6ae32aa5842..7e8b4accf4d 100644 --- a/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.cpp @@ -462,6 +462,11 @@ std::optional MySQLDumpSchemaReader::readRowAndGetDataTypes() return data_types; } +void MySQLDumpSchemaReader::transformTypesIfNeeded(DB::DataTypePtr & type, DB::DataTypePtr & new_type) +{ + transformInferredTypesIfNeeded(type, new_type, format_settings); +} + void registerInputFormatMySQLDump(FormatFactory & factory) { factory.registerInputFormat("MySQLDump", []( diff --git a/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.h b/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.h index 4148b6e79a3..18b403b5f9a 100644 --- a/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.h +++ b/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.h @@ -37,6 +37,7 @@ public: private: NamesAndTypesList readSchema() override; std::optional readRowAndGetDataTypes() override; + void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override; String table_name; }; diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.h b/src/Processors/Formats/Impl/RegexpRowInputFormat.h index e4a34f3c4f2..c3b6290281a 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.h +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.h @@ -90,7 +90,6 @@ private: void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override; - using EscapingRule = FormatSettings::EscapingRule; RegexpFieldExtractor field_extractor; PeekableReadBuffer buf; diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index b0ee2f7797a..d55ccce8879 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -701,6 +701,11 @@ std::optional ValuesSchemaReader::readRowAndGetDataTypes() return data_types; } +void ValuesSchemaReader::transformTypesIfNeeded(DB::DataTypePtr & type, DB::DataTypePtr & new_type) +{ + transformInferredTypesIfNeeded(type, new_type, format_settings); +} + void registerInputFormatValues(FormatFactory & factory) { factory.registerInputFormat("Values", []( diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h index e8c3b555994..643213695a2 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h @@ -111,6 +111,7 @@ public: private: std::optional readRowAndGetDataTypes() override; + void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override; PeekableReadBuffer buf; ParserExpression parser; diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp index a6514257dd3..ffee2bea1cc 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp @@ -561,5 +561,10 @@ std::vector FormatWithNamesAndTypesSchemaReader::readNamesFromFields(con return names; } +void FormatWithNamesAndTypesSchemaReader::transformTypesIfNeeded(DB::DataTypePtr & type, DB::DataTypePtr & new_type) +{ + transformInferredTypesIfNeeded(type, new_type, format_settings); +} + } diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h index c263b3b9666..d0457990714 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h @@ -172,6 +172,8 @@ public: NamesAndTypesList readSchema() override; + void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override; + protected: virtual std::optional readRowAndGetDataTypes() override; diff --git a/src/Storages/Cache/SchemaCache.h b/src/Storages/Cache/SchemaCache.h index 6f1ce917852..1bfc18bddab 100644 --- a/src/Storages/Cache/SchemaCache.h +++ b/src/Storages/Cache/SchemaCache.h @@ -29,10 +29,11 @@ public: String source; String format; String additional_format_info; + String schema_inference_mode; bool operator==(const Key & other) const { - return source == other.source && format == other.format && additional_format_info == other.additional_format_info; + return source == other.source && format == other.format && additional_format_info == other.additional_format_info && schema_inference_mode == other.schema_inference_mode; } }; @@ -42,7 +43,7 @@ public: { size_t operator()(const Key & key) const { - return std::hash()(key.source + key.format + key.additional_format_info); + return std::hash()(key.source + key.format + key.additional_format_info + key.schema_inference_mode); } }; diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 99c3e6e29cf..75330ed7db1 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -302,10 +302,17 @@ namespace { } - std::unique_ptr next() override + std::pair, std::optional> next() override { - StorageHDFS::PathWithInfo path_with_info; bool is_first = current_index == 0; + /// For default mode check cached columns for all paths on first iteration. + if (is_first && getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT) + { + if (auto cached_columns = tryGetColumnsFromCache(paths_with_info)) + return {nullptr, cached_columns}; + } + + StorageHDFS::PathWithInfo path_with_info; while (true) { @@ -315,26 +322,33 @@ namespace throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file, because all files are empty. " "You must specify table structure manually", format); - return nullptr; + return {nullptr, std::nullopt}; } path_with_info = paths_with_info[current_index++]; if (getContext()->getSettingsRef().hdfs_skip_empty_files && path_with_info.info && path_with_info.info->size == 0) continue; + if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION) + { + std::vector paths = {path_with_info}; + if (auto cached_columns = tryGetColumnsFromCache(paths)) + return {nullptr, cached_columns}; + } + auto compression = chooseCompressionMethod(path_with_info.path, compression_method); auto impl = std::make_unique(uri_without_path, path_with_info.path, getContext()->getGlobalContext()->getConfigRef(), getContext()->getReadSettings()); if (!getContext()->getSettingsRef().hdfs_skip_empty_files || !impl->eof()) { const Int64 zstd_window_log_max = getContext()->getSettingsRef().zstd_window_log_max; - return wrapReadBufferWithCompressionMethod(std::move(impl), compression, static_cast(zstd_window_log_max)); + return {wrapReadBufferWithCompressionMethod(std::move(impl), compression, static_cast(zstd_window_log_max)), std::nullopt}; } } } void setNumRowsToLastFile(size_t num_rows) override { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_s3) + if (!getContext()->getSettingsRef().schema_inference_use_cache_for_hdfs) return; String source = uri_without_path + paths_with_info[current_index - 1].path; @@ -342,7 +356,68 @@ namespace StorageHDFS::getSchemaCache(getContext()).addNumRows(key, num_rows); } + void setSchemaToLastFile(const ColumnsDescription & columns) override + { + if (!getContext()->getSettingsRef().schema_inference_use_cache_for_hdfs + || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::UNION) + return; + + String source = uri_without_path + paths_with_info[current_index - 1].path; + auto key = getKeyForSchemaCache(source, format, std::nullopt, getContext()); + StorageHDFS::getSchemaCache(getContext()).addColumns(key, columns); + } + + void setResultingSchema(const ColumnsDescription & columns) override + { + if (!getContext()->getSettingsRef().schema_inference_use_cache_for_hdfs + || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::DEFAULT) + return; + + Strings sources; + sources.reserve(paths_with_info.size()); + std::transform(paths_with_info.begin(), paths_with_info.end(), std::back_inserter(sources), [&](const StorageHDFS::PathWithInfo & path_with_info){ return uri_without_path + path_with_info.path; }); + auto cache_keys = getKeysForSchemaCache(sources, format, {}, getContext()); + StorageHDFS::getSchemaCache(getContext()).addManyColumns(cache_keys, columns); + } + + String getLastFileName() const override + { + if (current_index != 0) + return paths_with_info[current_index - 1].path; + + return ""; + } + private: + std::optional tryGetColumnsFromCache(const std::vector & paths_with_info_) + { + auto & schema_cache = StorageHDFS::getSchemaCache(getContext()); + for (const auto & path_with_info : paths_with_info_) + { + auto get_last_mod_time = [&]() -> std::optional + { + if (path_with_info.info) + return path_with_info.info->last_mod_time; + + auto builder = createHDFSBuilder(uri_without_path + "/", getContext()->getGlobalContext()->getConfigRef()); + auto fs = createHDFSFS(builder.get()); + HDFSFileInfoPtr hdfs_info(hdfsGetPathInfo(fs.get(), path_with_info.path.c_str())); + if (hdfs_info) + return hdfs_info->mLastMod; + + return std::nullopt; + }; + + String url = uri_without_path + path_with_info.path; + auto cache_key = getKeyForSchemaCache(url, format, {}, getContext()); + auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time); + if (columns) + return columns; + } + + return std::nullopt; + } + const std::vector & paths_with_info; const String & uri_without_path; const String & format; @@ -366,25 +441,8 @@ ColumnsDescription StorageHDFS::getTableStructureFromData( "Cannot extract table structure from {} format file, because there are no files in HDFS with provided path." " You must specify table structure manually", format); - std::optional columns_from_cache; - if (ctx->getSettingsRef().schema_inference_use_cache_for_hdfs) - columns_from_cache = tryGetColumnsFromCache(paths_with_info, uri_without_path, format, ctx); - - ColumnsDescription columns; - if (columns_from_cache) - { - columns = *columns_from_cache; - } - else - { - ReadBufferIterator read_buffer_iterator(paths_with_info, uri_without_path, format, compression_method, ctx); - columns = readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, paths_with_info.size() > 1, ctx); - } - - if (ctx->getSettingsRef().schema_inference_use_cache_for_hdfs) - addColumnsToCache(paths_with_info, uri_without_path, columns, format, ctx); - - return columns; + ReadBufferIterator read_buffer_iterator(paths_with_info, uri_without_path, format, compression_method, ctx); + return readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, paths_with_info.size() > 1, ctx); } class HDFSSource::DisclosedGlobIterator::Impl @@ -1017,54 +1075,6 @@ SchemaCache & StorageHDFS::getSchemaCache(const ContextPtr & ctx) return schema_cache; } -std::optional StorageHDFS::tryGetColumnsFromCache( - const std::vector & paths_with_info, - const String & uri_without_path, - const String & format_name, - const ContextPtr & ctx) -{ - auto & schema_cache = getSchemaCache(ctx); - for (const auto & path_with_info : paths_with_info) - { - auto get_last_mod_time = [&]() -> std::optional - { - if (path_with_info.info) - return path_with_info.info->last_mod_time; - - auto builder = createHDFSBuilder(uri_without_path + "/", ctx->getGlobalContext()->getConfigRef()); - auto fs = createHDFSFS(builder.get()); - HDFSFileInfoPtr hdfs_info(hdfsGetPathInfo(fs.get(), path_with_info.path.c_str())); - if (hdfs_info) - return hdfs_info->mLastMod; - - return std::nullopt; - }; - - String url = uri_without_path + path_with_info.path; - auto cache_key = getKeyForSchemaCache(url, format_name, {}, ctx); - auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time); - if (columns) - return columns; - } - - return std::nullopt; -} - -void StorageHDFS::addColumnsToCache( - const std::vector & paths_with_info, - const String & uri_without_path, - const ColumnsDescription & columns, - const String & format_name, - const ContextPtr & ctx) -{ - auto & schema_cache = getSchemaCache(ctx); - Strings sources; - sources.reserve(paths_with_info.size()); - std::transform(paths_with_info.begin(), paths_with_info.end(), std::back_inserter(sources), [&](const PathWithInfo & path_with_info){ return uri_without_path + path_with_info.path; }); - auto cache_keys = getKeysForSchemaCache(sources, format_name, {}, ctx); - schema_cache.addManyColumns(cache_keys, columns); -} - } #endif diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index ffbf4e93ff9..babcab3ed16 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -94,19 +94,6 @@ protected: friend class HDFSSource; private: - static std::optional tryGetColumnsFromCache( - const std::vector & paths_with_info, - const String & uri_without_path, - const String & format_name, - const ContextPtr & ctx); - - static void addColumnsToCache( - const std::vector & paths, - const String & uri_without_path, - const ColumnsDescription & columns, - const String & format_name, - const ContextPtr & ctx); - std::vector uris; String format_name; String compression_method; diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index a4a686b2691..796c732ba3f 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -1215,11 +1215,18 @@ namespace { } - std::unique_ptr next() override + std::pair, std::optional> next() override { - auto [key, metadata] = file_iterator->next(); + /// For default mode check cached columns for currently read keys on first iteration. + if (first && getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT) + { + if (auto cached_columns = tryGetColumnsFromCache(read_keys.begin(), read_keys.end())) + return {nullptr, cached_columns}; + } - if (key.empty()) + current_path_with_metadata = file_iterator->next(); + + if (current_path_with_metadata.relative_path.empty()) { if (first) throw Exception( @@ -1227,49 +1234,102 @@ namespace "Cannot extract table structure from {} format file, because there are no files with provided path " "in AzureBlobStorage. You must specify table structure manually", configuration.format); - return nullptr; + return {nullptr, std::nullopt}; } - current_path = key; + first = false; - ///AzureBlobStorage file iterator could get new keys after new iteration, check them in schema cache. - if (getContext()->getSettingsRef().schema_inference_use_cache_for_azure && read_keys.size() > prev_read_keys_size) + /// AzureBlobStorage file iterator could get new keys after new iteration, check them in schema cache if schema inference mode is default. + if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT && read_keys.size() > prev_read_keys_size) { - columns_from_cache = StorageAzureBlob::tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end(), configuration, format_settings, getContext()); + auto columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end()); prev_read_keys_size = read_keys.size(); if (columns_from_cache) - return nullptr; + return {nullptr, columns_from_cache}; + } + else if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION) + { + RelativePathsWithMetadata paths = {current_path_with_metadata}; + if (auto columns_from_cache = tryGetColumnsFromCache(paths.begin(), paths.end())) + return {nullptr, columns_from_cache}; } first = false; int zstd_window_log_max = static_cast(getContext()->getSettingsRef().zstd_window_log_max); - return wrapReadBufferWithCompressionMethod( - object_storage->readObject(StoredObject(key), getContext()->getReadSettings(), {}, metadata.size_bytes), - chooseCompressionMethod(key, configuration.compression_method), - zstd_window_log_max); + return {wrapReadBufferWithCompressionMethod( + object_storage->readObject(StoredObject(current_path_with_metadata.relative_path), getContext()->getReadSettings(), {}, current_path_with_metadata.metadata.size_bytes), + chooseCompressionMethod(current_path_with_metadata.relative_path, configuration.compression_method), + zstd_window_log_max), std::nullopt}; } - std::optional getCachedColumns() override { return columns_from_cache; } - void setNumRowsToLastFile(size_t num_rows) override { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_s3) + if (!getContext()->getSettingsRef().schema_inference_use_cache_for_azure) return; - String source = fs::path(configuration.connection_url) / configuration.container / current_path; + String source = fs::path(configuration.connection_url) / configuration.container / current_path_with_metadata.relative_path; auto key = getKeyForSchemaCache(source, configuration.format, format_settings, getContext()); StorageAzureBlob::getSchemaCache(getContext()).addNumRows(key, num_rows); } + void setSchemaToLastFile(const ColumnsDescription & columns) override + { + if (!getContext()->getSettingsRef().schema_inference_use_cache_for_azure + || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::UNION) + return; + + String source = fs::path(configuration.connection_url) / configuration.container / current_path_with_metadata.relative_path; + auto key = getKeyForSchemaCache(source, configuration.format, format_settings, getContext()); + StorageAzureBlob::getSchemaCache(getContext()).addColumns(key, columns); + } + + void setResultingSchema(const ColumnsDescription & columns) override + { + if (!getContext()->getSettingsRef().schema_inference_use_cache_for_azure + || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::UNION) + return; + + auto host_and_bucket = configuration.connection_url + '/' + configuration.container; + Strings sources; + sources.reserve(read_keys.size()); + std::transform(read_keys.begin(), read_keys.end(), std::back_inserter(sources), [&](const auto & elem){ return host_and_bucket + '/' + elem.relative_path; }); + auto cache_keys = getKeysForSchemaCache(sources, configuration.format, format_settings, getContext()); + StorageAzureBlob::getSchemaCache(getContext()).addManyColumns(cache_keys, columns); + } + + String getLastFileName() const override { return current_path_with_metadata.relative_path; } + private: + std::optional tryGetColumnsFromCache(const RelativePathsWithMetadata::const_iterator & begin, const RelativePathsWithMetadata::const_iterator & end) + { + auto & schema_cache = StorageAzureBlob::getSchemaCache(getContext()); + for (auto it = begin; it < end; ++it) + { + auto get_last_mod_time = [&] -> std::optional + { + if (it->metadata.last_modified) + return it->metadata.last_modified->epochTime(); + return std::nullopt; + }; + + auto host_and_bucket = configuration.connection_url + '/' + configuration.container; + String source = host_and_bucket + '/' + it->relative_path; + auto cache_key = getKeyForSchemaCache(source, configuration.format, format_settings, getContext()); + auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time); + if (columns) + return columns; + } + + return std::nullopt; + } + std::shared_ptr file_iterator; AzureObjectStorage * object_storage; const StorageAzureBlob::Configuration & configuration; const std::optional & format_settings; const RelativePathsWithMetadata & read_keys; - std::optional columns_from_cache; size_t prev_read_keys_size; - String current_path; + RelativePathWithMetadata current_path_with_metadata; bool first = true; }; } @@ -1299,72 +1359,8 @@ ColumnsDescription StorageAzureBlob::getTableStructureFromData( object_storage, configuration.container, configuration.blobs_paths, nullptr, NamesAndTypesList{}, ctx, &read_keys); } - std::optional columns_from_cache; - if (ctx->getSettingsRef().schema_inference_use_cache_for_azure) - columns_from_cache = tryGetColumnsFromCache(read_keys.begin(), read_keys.end(), configuration, format_settings, ctx); - - ColumnsDescription columns; - if (columns_from_cache) - { - columns = *columns_from_cache; - } - else - { - ReadBufferIterator read_buffer_iterator(file_iterator, object_storage, configuration, format_settings, read_keys, ctx); - columns = readSchemaFromFormat(configuration.format, format_settings, read_buffer_iterator, configuration.withGlobs(), ctx); - } - - if (ctx->getSettingsRef().schema_inference_use_cache_for_azure) - addColumnsToCache(read_keys, columns, configuration, format_settings, configuration.format, ctx); - - return columns; - -} - -std::optional StorageAzureBlob::tryGetColumnsFromCache( - const RelativePathsWithMetadata::const_iterator & begin, - const RelativePathsWithMetadata::const_iterator & end, - const StorageAzureBlob::Configuration & configuration, - const std::optional & format_settings, - const ContextPtr & ctx) -{ - auto & schema_cache = getSchemaCache(ctx); - for (auto it = begin; it < end; ++it) - { - auto get_last_mod_time = [&] -> std::optional - { - if (it->metadata.last_modified) - return it->metadata.last_modified->epochTime(); - return std::nullopt; - }; - - auto host_and_bucket = configuration.connection_url + '/' + configuration.container; - String source = host_and_bucket + '/' + it->relative_path; - auto cache_key = getKeyForSchemaCache(source, configuration.format, format_settings, ctx); - auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time); - if (columns) - return columns; - } - - return std::nullopt; - -} - -void StorageAzureBlob::addColumnsToCache( - const RelativePathsWithMetadata & keys, - const ColumnsDescription & columns, - const StorageAzureBlob::Configuration & configuration, - const std::optional & format_settings, - const String & format_name, - const ContextPtr & ctx) -{ - auto host_and_bucket = configuration.connection_url + '/' + configuration.container; - Strings sources; - sources.reserve(keys.size()); - std::transform(keys.begin(), keys.end(), std::back_inserter(sources), [&](const auto & elem){ return host_and_bucket + '/' + elem.relative_path; }); - auto cache_keys = getKeysForSchemaCache(sources, format_name, format_settings, ctx); - auto & schema_cache = getSchemaCache(ctx); - schema_cache.addManyColumns(cache_keys, columns); + ReadBufferIterator read_buffer_iterator(file_iterator, object_storage, configuration, format_settings, read_keys, ctx); + return readSchemaFromFormat(configuration.format, format_settings, read_buffer_iterator, configuration.withGlobs(), ctx); } SchemaCache & StorageAzureBlob::getSchemaCache(const ContextPtr & ctx) diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index b97dee0caed..0862e8cc2bf 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -124,21 +124,6 @@ public: ContextPtr ctx, bool distributed_processing = false); - static std::optional tryGetColumnsFromCache( - const RelativePathsWithMetadata::const_iterator & begin, - const RelativePathsWithMetadata::const_iterator & end, - const StorageAzureBlob::Configuration & configuration, - const std::optional & format_settings, - const ContextPtr & ctx); - - static void addColumnsToCache( - const RelativePathsWithMetadata & keys, - const ColumnsDescription & columns, - const Configuration & configuration, - const std::optional & format_settings, - const String & format_name, - const ContextPtr & ctx); - private: std::string name; Configuration configuration; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 856c1f21d27..1f4d14218aa 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -444,11 +444,19 @@ namespace { } - std::unique_ptr next() override + std::pair, std::optional> next() override { + bool is_first = current_index == 0; + /// For default mode check cached columns for all paths on first iteration. + /// If we have cached columns, next() won't be called again. + if (is_first && getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT) + { + if (auto cached_columns = tryGetColumnsFromCache(paths)) + return {nullptr, cached_columns}; + } + String path; struct stat file_stat; - bool is_first = current_index == 0; do { @@ -459,14 +467,21 @@ namespace ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file, because all files are empty. You must specify table structure manually", format); - return nullptr; + return {nullptr, std::nullopt}; } path = paths[current_index++]; file_stat = getFileStat(path, false, -1, "File"); } while (getContext()->getSettingsRef().engine_file_skip_empty_files && file_stat.st_size == 0); - return createReadBuffer(path, file_stat, false, -1, compression_method, getContext()); + /// For union mode, check cached columns only for current path, because schema can be different for different files. + if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION) + { + if (auto cached_columns = tryGetColumnsFromCache({path})) + return {nullptr, cached_columns}; + } + + return {createReadBuffer(path, file_stat, false, -1, compression_method, getContext()), std::nullopt}; } void setNumRowsToLastFile(size_t num_rows) override @@ -478,7 +493,64 @@ namespace StorageFile::getSchemaCache(getContext()).addNumRows(key, num_rows); } + void setSchemaToLastFile(const ColumnsDescription & columns) override + { + if (!getContext()->getSettingsRef().schema_inference_use_cache_for_file + || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::UNION) + return; + + /// For union mode, schema can be different for different files, so we need to + /// cache last inferred schema only for last processed file. + auto cache_key = getKeyForSchemaCache(paths[current_index - 1], format, format_settings, getContext()); + StorageFile::getSchemaCache(getContext()).addColumns(cache_key, columns); + } + + void setResultingSchema(const ColumnsDescription & columns) override + { + if (!getContext()->getSettingsRef().schema_inference_use_cache_for_file + || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::DEFAULT) + return; + + /// For default mode we cache resulting schema for all paths. + auto cache_keys = getKeysForSchemaCache(paths, format, format_settings, getContext()); + StorageFile::getSchemaCache(getContext()).addManyColumns(cache_keys, columns); + } + + String getLastFileName() const override + { + if (current_index != 0) + return paths[current_index - 1]; + return ""; + } + private: + std::optional tryGetColumnsFromCache(const Strings & paths_) + { + if (!getContext()->getSettingsRef().schema_inference_use_cache_for_file) + return std::nullopt; + + /// Check if the cache contains one of the paths. + auto & schema_cache = StorageFile::getSchemaCache(getContext()); + struct stat file_stat{}; + for (const auto & path : paths_) + { + auto get_last_mod_time = [&]() -> std::optional + { + if (0 != stat(path.c_str(), &file_stat)) + return std::nullopt; + + return file_stat.st_mtime; + }; + + auto cache_key = getKeyForSchemaCache(path, format, format_settings, getContext()); + auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time); + if (columns) + return columns; + } + + return std::nullopt; + } + const std::vector & paths; size_t current_index = 0; @@ -502,8 +574,19 @@ namespace { } - std::unique_ptr next() override + std::pair, std::optional> next() override { + /// For default mode check cached columns for all initial archive paths (maybe with globs) on first iteration. + /// If we have cached columns, next() won't be called again. + if (is_first && getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT) + { + for (const auto & archive : archive_info.paths_to_archives) + { + if (auto cached_columns = tryGetColumnsFromSchemaCache(archive, archive_info.path_in_archive)) + return {nullptr, cached_columns}; + } + } + std::unique_ptr read_buf; while (true) { @@ -515,7 +598,7 @@ namespace "Cannot extract table structure from {} format file, because all files are empty. You must specify table structure manually", format); - return nullptr; + return {nullptr, std::nullopt}; } const auto & archive = archive_info.paths_to_archives[current_archive_index]; @@ -546,11 +629,11 @@ namespace if (!read_buf) continue; - last_read_file_path = processed_files.emplace_back(fmt::format("{}::{}", archive_reader->getPath(), archive_info.path_in_archive)); - columns_from_cache = tryGetColumnsFromSchemaCache(archive, last_read_file_path); + last_read_file_path = paths_for_schema_cache.emplace_back(fmt::format("{}::{}", archive_reader->getPath(), archive_info.path_in_archive)); + is_first = false; - if (columns_from_cache) - return nullptr; + if (auto cached_columns = tryGetColumnsFromSchemaCache(archive, last_read_file_path)) + return {nullptr, cached_columns}; } else { @@ -583,11 +666,17 @@ namespace continue; } - last_read_file_path = processed_files.emplace_back(fmt::format("{}::{}", archive_reader->getPath(), *filename)); - columns_from_cache = tryGetColumnsFromSchemaCache(archive, last_read_file_path); + last_read_file_path = paths_for_schema_cache.emplace_back(fmt::format("{}::{}", archive_reader->getPath(), *filename)); + is_first = false; - if (columns_from_cache) - return nullptr; + if (auto cached_columns = tryGetColumnsFromSchemaCache(archive, last_read_file_path)) + { + /// For union mode next() will be called again even if we found cached columns, + /// so we need to remember last_read_buffer to continue iterating through files in archive. + if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION) + last_read_buffer = archive_reader->readFile(std::move(file_enumerator)); + return {nullptr, cached_columns}; + } read_buf = archive_reader->readFile(std::move(file_enumerator)); } @@ -595,18 +684,13 @@ namespace break; } - is_first = false; - return read_buf; - } - - std::optional getCachedColumns() override - { - return columns_from_cache; + return {std::move(read_buf), std::nullopt}; } void setPreviousReadBuffer(std::unique_ptr buffer) override { - last_read_buffer = std::move(buffer); + if (buffer) + last_read_buffer = std::move(buffer); } void setNumRowsToLastFile(size_t num_rows) override @@ -618,13 +702,45 @@ namespace StorageFile::getSchemaCache(getContext()).addNumRows(key, num_rows); } - std::vector processed_files; - private: + void setSchemaToLastFile(const ColumnsDescription & columns) override + { + if (!getContext()->getSettingsRef().schema_inference_use_cache_for_file + || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::UNION) + return; + /// For union mode, schema can be different for different files in archive, so we need to + /// cache last inferred schema only for last processed file. + auto & schema_cache = StorageFile::getSchemaCache(getContext()); + auto cache_key = getKeyForSchemaCache(last_read_file_path, format, format_settings, getContext()); + schema_cache.addColumns(cache_key, columns); + } + + void setResultingSchema(const ColumnsDescription & columns) override + { + if (!getContext()->getSettingsRef().schema_inference_use_cache_for_file + || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::DEFAULT) + return; + + /// For default mode we cache resulting schema for all paths. + /// Also add schema for initial paths (maybe with globes) in cache, + /// so next time we won't iterate through files (that can be expensive). + for (const auto & archive : archive_info.paths_to_archives) + paths_for_schema_cache.emplace_back(fmt::format("{}::{}", archive, archive_info.path_in_archive)); + auto & schema_cache = StorageFile::getSchemaCache(getContext()); + auto cache_keys = getKeysForSchemaCache(paths_for_schema_cache, format, format_settings, getContext()); + schema_cache.addManyColumns(cache_keys, columns); + } + + String getLastFileName() const override + { + return last_read_file_path; + } + + private: std::optional tryGetColumnsFromSchemaCache(const std::string & archive_path, const std::string & full_path) { auto context = getContext(); - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_file) + if (!context->getSettingsRef().schema_inference_use_cache_for_file) return std::nullopt; struct stat file_stat; @@ -654,44 +770,13 @@ namespace std::string last_read_file_path; - std::optional columns_from_cache; - std::unique_ptr file_enumerator; std::unique_ptr last_read_buffer; String format; const std::optional & format_settings; + std::vector paths_for_schema_cache; }; - - std::optional tryGetColumnsFromCacheForArchives( - const StorageFile::ArchiveInfo & archive_info, - std::vector & paths_for_schema_cache, - const String & format, - const std::optional & format_settings, - const ContextPtr & context) - { - struct stat file_stat{}; - std::optional columns_from_cache; - - for (const auto & archive : archive_info.paths_to_archives) - { - const auto & full_path = paths_for_schema_cache.emplace_back(fmt::format("{}::{}", archive, archive_info.path_in_archive)); - - auto & schema_cache = StorageFile::getSchemaCache(context); - auto get_last_mod_time = [&]() -> std::optional - { - if (0 != stat(archive.c_str(), &file_stat)) - return std::nullopt; - - return file_stat.st_mtime; - }; - - auto cache_key = getKeyForSchemaCache(full_path, format, format_settings, context); - columns_from_cache = schema_cache.tryGetColumns(cache_key, get_last_mod_time); - } - - return columns_from_cache; - } } ColumnsDescription StorageFile::getTableStructureFromFileDescriptor(ContextPtr context) @@ -744,48 +829,19 @@ ColumnsDescription StorageFile::getTableStructureFromFile( "Cannot extract table structure from {} format file, because there are no files with provided path. " "You must specify table structure manually", format); - ColumnsDescription columns; - std::vector archive_paths_for_schema_cache; - std::optional columns_from_cache; - - if (context->getSettingsRef().schema_inference_use_cache_for_file) + if (archive_info) { - if (archive_info) - columns_from_cache = tryGetColumnsFromCacheForArchives(*archive_info, archive_paths_for_schema_cache, format, format_settings, context); - else - columns_from_cache = tryGetColumnsFromCache(paths, format, format_settings, context); + ReadBufferFromArchiveIterator read_buffer_iterator(*archive_info, format, format_settings, context); + return readSchemaFromFormat( + format, + format_settings, + read_buffer_iterator, + /*retry=*/archive_info->paths_to_archives.size() > 1 || !archive_info->isSingleFileRead(), + context); } - if (columns_from_cache) - { - columns = std::move(*columns_from_cache); - } - else - { - if (archive_info) - { - ReadBufferFromArchiveIterator read_buffer_iterator(*archive_info, format, format_settings, context); - columns = readSchemaFromFormat( - format, - format_settings, - read_buffer_iterator, - /*retry=*/archive_info->paths_to_archives.size() > 1 || !archive_info->isSingleFileRead(), - context); - - for (auto & file : read_buffer_iterator.processed_files) - archive_paths_for_schema_cache.push_back(std::move(file)); - } - else - { - ReadBufferFromFileIterator read_buffer_iterator(paths, format, compression_method, format_settings, context); - columns = readSchemaFromFormat(format, format_settings, read_buffer_iterator, paths.size() > 1, context); - } - } - - if (context->getSettingsRef().schema_inference_use_cache_for_file) - addColumnsToCache(archive_info.has_value() ? archive_paths_for_schema_cache : paths, columns, format, format_settings, context); - - return columns; + ReadBufferFromFileIterator read_buffer_iterator(paths, format, compression_method, format_settings, context); + return readSchemaFromFormat(format, format_settings, read_buffer_iterator, paths.size() > 1, context); } bool StorageFile::supportsSubsetOfColumns(const ContextPtr & context) const @@ -1972,43 +2028,6 @@ SchemaCache & StorageFile::getSchemaCache(const ContextPtr & context) return schema_cache; } -std::optional StorageFile::tryGetColumnsFromCache( - const Strings & paths, const String & format_name, const std::optional & format_settings, ContextPtr context) -{ - /// Check if the cache contains one of the paths. - auto & schema_cache = getSchemaCache(context); - struct stat file_stat{}; - for (const auto & path : paths) - { - auto get_last_mod_time = [&]() -> std::optional - { - if (0 != stat(path.c_str(), &file_stat)) - return std::nullopt; - - return file_stat.st_mtime; - }; - - auto cache_key = getKeyForSchemaCache(path, format_name, format_settings, context); - auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time); - if (columns) - return columns; - } - - return std::nullopt; -} - -void StorageFile::addColumnsToCache( - const Strings & paths, - const ColumnsDescription & columns, - const String & format_name, - const std::optional & format_settings, - const ContextPtr & context) -{ - auto & schema_cache = getSchemaCache(context); - auto cache_keys = getKeysForSchemaCache(paths, format_name, format_settings, context); - schema_cache.addManyColumns(cache_keys, columns); -} - void StorageFile::parseFileSource(String source, String & filename, String & path_to_archive) { size_t pos = source.find("::"); diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index f1464b90ab4..0bdbd642459 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -126,16 +126,6 @@ protected: private: void setStorageMetadata(CommonArguments args); - static std::optional tryGetColumnsFromCache( - const Strings & paths, const String & format_name, const std::optional & format_settings, ContextPtr context); - - static void addColumnsToCache( - const Strings & paths, - const ColumnsDescription & columns, - const String & format_name, - const std::optional & format_settings, - const ContextPtr & context); - std::string format_name; // We use format settings from global context + CREATE query for File table // function -- in this case, format_settings is set. diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index f26d59cb559..18abd66cec9 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1499,8 +1499,15 @@ namespace { } - std::unique_ptr next() override + std::pair, std::optional> next() override { + /// For default mode check cached columns for currently read keys on first iteration. + if (first && getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT) + { + if (auto cached_columns = tryGetColumnsFromCache(read_keys.begin(), read_keys.end())) + return {nullptr, cached_columns}; + } + while (true) { current_key_with_info = (*file_iterator)(); @@ -1514,36 +1521,42 @@ namespace "in S3 or all files are empty. You must specify table structure manually", configuration.format); - return nullptr; + return {nullptr, std::nullopt}; } - /// S3 file iterator could get new keys after new iteration, check them in schema cache. - if (getContext()->getSettingsRef().schema_inference_use_cache_for_s3 && read_keys.size() > prev_read_keys_size) + /// S3 file iterator could get new keys after new iteration, check them in schema cache if schema inference mode is default. + if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT && read_keys.size() > prev_read_keys_size) { - columns_from_cache = StorageS3::tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end(), configuration, format_settings, getContext()); + auto columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end()); prev_read_keys_size = read_keys.size(); if (columns_from_cache) - return nullptr; + return {nullptr, columns_from_cache}; } if (getContext()->getSettingsRef().s3_skip_empty_files && current_key_with_info.info && current_key_with_info.info->size == 0) continue; + /// In union mode, check cached columns only for current key. + if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION) + { + StorageS3::KeysWithInfo keys = {current_key_with_info}; + if (auto columns_from_cache = tryGetColumnsFromCache(keys.begin(), keys.end())) + { + first = false; + return {nullptr, columns_from_cache}; + } + } + int zstd_window_log_max = static_cast(getContext()->getSettingsRef().zstd_window_log_max); auto impl = std::make_unique(configuration.client, configuration.url.bucket, current_key_with_info.key, configuration.url.version_id, configuration.request_settings, getContext()->getReadSettings()); if (!getContext()->getSettingsRef().s3_skip_empty_files || !impl->eof()) { first = false; - return wrapReadBufferWithCompressionMethod(std::move(impl), chooseCompressionMethod(current_key_with_info.key, configuration.compression_method), zstd_window_log_max); + return {wrapReadBufferWithCompressionMethod(std::move(impl), chooseCompressionMethod(current_key_with_info.key, configuration.compression_method), zstd_window_log_max), std::nullopt}; } } } - std::optional getCachedColumns() override - { - return columns_from_cache; - } - void setNumRowsToLastFile(size_t num_rows) override { if (!getContext()->getSettingsRef().schema_inference_use_cache_for_s3) @@ -1554,12 +1567,85 @@ namespace StorageS3::getSchemaCache(getContext()).addNumRows(key, num_rows); } + void setSchemaToLastFile(const ColumnsDescription & columns) override + { + if (!getContext()->getSettingsRef().schema_inference_use_cache_for_s3 + || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::UNION) + return; + + String source = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / configuration.url.bucket / current_key_with_info.key; + auto cache_key = getKeyForSchemaCache(source, configuration.format, format_settings, getContext()); + StorageS3::getSchemaCache(getContext()).addColumns(cache_key, columns); + } + + void setResultingSchema(const ColumnsDescription & columns) override + { + if (!getContext()->getSettingsRef().schema_inference_use_cache_for_s3 + || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::DEFAULT) + return; + + auto host_and_bucket = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / configuration.url.bucket; + Strings sources; + sources.reserve(read_keys.size()); + std::transform(read_keys.begin(), read_keys.end(), std::back_inserter(sources), [&](const auto & elem){ return host_and_bucket / elem.key; }); + auto cache_keys = getKeysForSchemaCache(sources, configuration.format, format_settings, getContext()); + StorageS3::getSchemaCache(getContext()).addManyColumns(cache_keys, columns); + } + + String getLastFileName() const override { return current_key_with_info.key; } + private: + std::optional tryGetColumnsFromCache( + const StorageS3::KeysWithInfo::const_iterator & begin, + const StorageS3::KeysWithInfo::const_iterator & end) + { + if (!getContext()->getSettingsRef().schema_inference_use_cache_for_s3) + return std::nullopt; + + auto & schema_cache = StorageS3::getSchemaCache(getContext()); + for (auto it = begin; it < end; ++it) + { + auto get_last_mod_time = [&] + { + time_t last_modification_time = 0; + if (it->info) + { + last_modification_time = it->info->last_modification_time; + } + else + { + /// Note that in case of exception in getObjectInfo returned info will be empty, + /// but schema cache will handle this case and won't return columns from cache + /// because we can't say that it's valid without last modification time. + last_modification_time = S3::getObjectInfo( + *configuration.client, + configuration.url.bucket, + it->key, + configuration.url.version_id, + configuration.request_settings, + /*with_metadata=*/ false, + /*for_disk_s3=*/ false, + /*throw_on_error= */ false).last_modification_time; + } + + return last_modification_time ? std::make_optional(last_modification_time) : std::nullopt; + }; + + String path = fs::path(configuration.url.bucket) / it->key; + String source = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / path; + auto cache_key = getKeyForSchemaCache(source, configuration.format, format_settings, getContext()); + auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time); + if (columns) + return columns; + } + + return std::nullopt; + } + std::shared_ptr file_iterator; const StorageS3Source::KeysWithInfo & read_keys; const StorageS3::Configuration & configuration; const std::optional & format_settings; - std::optional columns_from_cache; StorageS3Source::KeyWithInfo current_key_with_info; size_t prev_read_keys_size; bool first = true; @@ -1576,28 +1662,10 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl( auto file_iterator = createFileIterator(configuration, false, ctx, nullptr, {}, &read_keys); - std::optional columns_from_cache; - if (ctx->getSettingsRef().schema_inference_use_cache_for_s3) - columns_from_cache = tryGetColumnsFromCache(read_keys.begin(), read_keys.end(), configuration, format_settings, ctx); - - ColumnsDescription columns; - if (columns_from_cache) - { - columns = *columns_from_cache; - } - else - { - ReadBufferIterator read_buffer_iterator(file_iterator, read_keys, configuration, format_settings, ctx); - columns = readSchemaFromFormat(configuration.format, format_settings, read_buffer_iterator, configuration.withGlobs(), ctx); - } - - if (ctx->getSettingsRef().schema_inference_use_cache_for_s3) - addColumnsToCache(read_keys, configuration, columns, configuration.format, format_settings, ctx); - - return columns; + ReadBufferIterator read_buffer_iterator(file_iterator, read_keys, configuration, format_settings, ctx); + return readSchemaFromFormat(configuration.format, format_settings, read_buffer_iterator, configuration.withGlobs(), ctx); } - void registerStorageS3Impl(const String & name, StorageFactory & factory) { factory.registerStorage(name, [](const StorageFactory::Arguments & args) @@ -1687,70 +1755,6 @@ SchemaCache & StorageS3::getSchemaCache(const ContextPtr & ctx) return schema_cache; } -std::optional StorageS3::tryGetColumnsFromCache( - const KeysWithInfo::const_iterator & begin, - const KeysWithInfo::const_iterator & end, - const Configuration & configuration, - const std::optional & format_settings, - const ContextPtr & ctx) -{ - auto & schema_cache = getSchemaCache(ctx); - for (auto it = begin; it < end; ++it) - { - auto get_last_mod_time = [&] - { - time_t last_modification_time = 0; - if (it->info) - { - last_modification_time = it->info->last_modification_time; - } - else - { - /// Note that in case of exception in getObjectInfo returned info will be empty, - /// but schema cache will handle this case and won't return columns from cache - /// because we can't say that it's valid without last modification time. - last_modification_time = S3::getObjectInfo( - *configuration.client, - configuration.url.bucket, - it->key, - configuration.url.version_id, - configuration.request_settings, - /*with_metadata=*/ false, - /*for_disk_s3=*/ false, - /*throw_on_error= */ false).last_modification_time; - } - - return last_modification_time ? std::make_optional(last_modification_time) : std::nullopt; - }; - - String path = fs::path(configuration.url.bucket) / it->key; - String source = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / path; - auto cache_key = getKeyForSchemaCache(source, configuration.format, format_settings, ctx); - auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time); - if (columns) - return columns; - } - - return std::nullopt; -} - -void StorageS3::addColumnsToCache( - const KeysWithInfo & keys, - const Configuration & configuration, - const ColumnsDescription & columns, - const String & format_name, - const std::optional & format_settings, - const ContextPtr & ctx) -{ - auto host_and_bucket = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / configuration.url.bucket; - Strings sources; - sources.reserve(keys.size()); - std::transform(keys.begin(), keys.end(), std::back_inserter(sources), [&](const auto & elem){ return host_and_bucket / elem.key; }); - auto cache_keys = getKeysForSchemaCache(sources, format_name, format_settings, ctx); - auto & schema_cache = getSchemaCache(ctx); - schema_cache.addManyColumns(cache_keys, columns); -} - } #endif diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 088f9000ce8..b22f8275bf0 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -344,21 +344,6 @@ public: using KeysWithInfo = StorageS3Source::KeysWithInfo; - static std::optional tryGetColumnsFromCache( - const KeysWithInfo::const_iterator & begin, - const KeysWithInfo::const_iterator & end, - const Configuration & configuration, - const std::optional & format_settings, - const ContextPtr & ctx); - - static void addColumnsToCache( - const KeysWithInfo & keys, - const Configuration & configuration, - const ColumnsDescription & columns, - const String & format_name, - const std::optional & format_settings, - const ContextPtr & ctx); - bool supportsTrivialCountOptimization() const override { return true; } protected: diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 58f01312399..ffb92db9279 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -708,30 +708,53 @@ namespace const HTTPHeaderEntries & headers_, const std::optional & format_settings_, const ContextPtr & context_) - : WithContext(context_), urls_to_check(urls_to_check_), format(format_), compression_method(compression_method_), headers(headers_), format_settings(format_settings_) + : WithContext(context_), format(format_), compression_method(compression_method_), headers(headers_), format_settings(format_settings_) { - it = urls_to_check.cbegin(); + url_options_to_check.reserve(urls_to_check_.size()); + for (const auto & url : urls_to_check_) + url_options_to_check.push_back(getFailoverOptions(url, getContext()->getSettingsRef().glob_expansion_max_elements)); } - std::unique_ptr next() override + std::pair, std::optional> next() override { + bool is_first = (current_index == 0); + /// For default mode check cached columns for all urls on first iteration. + if (is_first && getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT) + { + for (const auto & options : url_options_to_check) + { + if (auto cached_columns = tryGetColumnsFromCache(options)) + return {nullptr, cached_columns}; + } + } + std::pair> uri_and_buf; do { - if (it == urls_to_check.cend()) + if (current_index == url_options_to_check.size()) { - if (first) + if (is_first) throw Exception( ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file, because all files are empty. " "You must specify table structure manually", format); - return nullptr; + return {nullptr, std::nullopt}; } + if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION) + { + if (auto cached_columns = tryGetColumnsFromCache(url_options_to_check[current_index])) + { + ++current_index; + return {nullptr, cached_columns}; + } + } + + auto first_option = url_options_to_check[current_index].cbegin(); uri_and_buf = StorageURLSource::getFirstAvailableURIAndReadBuffer( - it, - urls_to_check.cend(), + first_option, + url_options_to_check[current_index].cend(), getContext(), {}, Poco::Net::HTTPRequest::HTTP_GET, @@ -742,35 +765,87 @@ namespace false, false); - ++it; + ++current_index; } while (getContext()->getSettingsRef().engine_url_skip_empty_files && uri_and_buf.second->eof()); - first = false; - return wrapReadBufferWithCompressionMethod( + current_url_option = uri_and_buf.first.toString(); + return {wrapReadBufferWithCompressionMethod( std::move(uri_and_buf.second), compression_method, - static_cast(getContext()->getSettingsRef().zstd_window_log_max)); + static_cast(getContext()->getSettingsRef().zstd_window_log_max)), std::nullopt}; } void setNumRowsToLastFile(size_t num_rows) override { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_s3) + if (!getContext()->getSettingsRef().schema_inference_use_cache_for_url) return; - String source = *std::prev(it); - auto key = getKeyForSchemaCache(source, format, format_settings, getContext()); + auto key = getKeyForSchemaCache(current_url_option, format, format_settings, getContext()); StorageURL::getSchemaCache(getContext()).addNumRows(key, num_rows); } + void setSchemaToLastFile(const ColumnsDescription & columns) override + { + if (!getContext()->getSettingsRef().schema_inference_use_cache_for_url + || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::UNION) + return; + + auto key = getKeyForSchemaCache(current_url_option, format, format_settings, getContext()); + StorageURL::getSchemaCache(getContext()).addColumns(key, columns); + } + + void setResultingSchema(const ColumnsDescription & columns) override + { + if (!getContext()->getSettingsRef().schema_inference_use_cache_for_url + || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::DEFAULT) + return; + + for (const auto & options : url_options_to_check) + { + auto keys = getKeysForSchemaCache(options, format, format_settings, getContext()); + StorageURL::getSchemaCache(getContext()).addManyColumns(keys, columns); + } + } + + String getLastFileName() const override { return current_url_option; } + private: - const std::vector & urls_to_check; - std::vector::const_iterator it; + std::optional tryGetColumnsFromCache(const Strings & urls) + { + if (!getContext()->getSettingsRef().schema_inference_use_cache_for_url) + return std::nullopt; + + auto & schema_cache = StorageURL::getSchemaCache(getContext()); + for (const auto & url : urls) + { + auto get_last_mod_time = [&]() -> std::optional + { + auto last_mod_time = StorageURL::tryGetLastModificationTime(url, headers, credentials, getContext()); + /// Some URLs could not have Last-Modified header, in this case we cannot be sure that + /// data wasn't changed after adding it's schema to cache. Use schema from cache only if + /// special setting for this case is enabled. + if (!last_mod_time && !getContext()->getSettingsRef().schema_inference_cache_require_modification_time_for_url) + return 0; + return last_mod_time; + }; + + auto cache_key = getKeyForSchemaCache(url, format, format_settings, getContext()); + auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time); + if (columns) + return columns; + } + + return std::nullopt; + } + + std::vector> url_options_to_check; + size_t current_index = 0; + String current_url_option; const String & format; const CompressionMethod & compression_method; const HTTPHeaderEntries & headers; Poco::Net::HTTPBasicCredentials credentials; const std::optional & format_settings; - bool first = true; }; } @@ -788,39 +863,12 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData( std::vector urls_to_check; if (urlWithGlobs(uri)) - { - size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements; - auto uri_descriptions = parseRemoteDescription(uri, 0, uri.size(), ',', max_addresses, "url"); - for (const auto & description : uri_descriptions) - { - auto options = parseRemoteDescription(description, 0, description.size(), '|', max_addresses, "url"); - urls_to_check.insert(urls_to_check.end(), options.begin(), options.end()); - } - } + urls_to_check = parseRemoteDescription(uri, 0, uri.size(), ',', context->getSettingsRef().glob_expansion_max_elements, "url"); else - { urls_to_check = {uri}; - } - std::optional columns_from_cache; - if (context->getSettingsRef().schema_inference_use_cache_for_url) - columns_from_cache = tryGetColumnsFromCache(urls_to_check, headers, credentials, format, format_settings, context); - - ColumnsDescription columns; - if (columns_from_cache) - { - columns = *columns_from_cache; - } - else - { - ReadBufferIterator read_buffer_iterator(urls_to_check, format, compression_method, headers, format_settings, context); - columns = readSchemaFromFormat(format, format_settings, read_buffer_iterator, urls_to_check.size() > 1, context); - } - - if (context->getSettingsRef().schema_inference_use_cache_for_url) - addColumnsToCache(urls_to_check, columns, format, format_settings, context); - - return columns; + ReadBufferIterator read_buffer_iterator(urls_to_check, format, compression_method, headers, format_settings, context); + return readSchemaFromFormat(format, format_settings, read_buffer_iterator, urls_to_check.size() > 1, context); } bool IStorageURLBase::supportsSubsetOfColumns(const ContextPtr & context) const @@ -1033,49 +1081,6 @@ SchemaCache & IStorageURLBase::getSchemaCache(const ContextPtr & context) return schema_cache; } -std::optional IStorageURLBase::tryGetColumnsFromCache( - const Strings & urls, - const HTTPHeaderEntries & headers, - const Poco::Net::HTTPBasicCredentials & credentials, - const String & format_name, - const std::optional & format_settings, - const ContextPtr & context) -{ - auto & schema_cache = getSchemaCache(context); - for (const auto & url : urls) - { - auto get_last_mod_time = [&]() -> std::optional - { - auto last_mod_time = tryGetLastModificationTime(url, headers, credentials, context); - /// Some URLs could not have Last-Modified header, in this case we cannot be sure that - /// data wasn't changed after adding it's schema to cache. Use schema from cache only if - /// special setting for this case is enabled. - if (!last_mod_time && !context->getSettingsRef().schema_inference_cache_require_modification_time_for_url) - return 0; - return last_mod_time; - }; - - auto cache_key = getKeyForSchemaCache(url, format_name, format_settings, context); - auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time); - if (columns) - return columns; - } - - return std::nullopt; -} - -void IStorageURLBase::addColumnsToCache( - const Strings & urls, - const ColumnsDescription & columns, - const String & format_name, - const std::optional & format_settings, - const ContextPtr & context) -{ - auto & schema_cache = getSchemaCache(context); - auto cache_keys = getKeysForSchemaCache(urls, format_name, format_settings, context); - schema_cache.addManyColumns(cache_keys, columns); -} - std::optional IStorageURLBase::tryGetLastModificationTime( const String & url, const HTTPHeaderEntries & headers, diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 6c5c50af326..8257bd65f9c 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -124,21 +124,6 @@ protected: private: virtual Block getHeaderBlock(const Names & column_names, const StorageSnapshotPtr & storage_snapshot) const = 0; - - static std::optional tryGetColumnsFromCache( - const Strings & urls, - const HTTPHeaderEntries & headers, - const Poco::Net::HTTPBasicCredentials & credentials, - const String & format_name, - const std::optional & format_settings, - const ContextPtr & context); - - static void addColumnsToCache( - const Strings & urls, - const ColumnsDescription & columns, - const String & format_name, - const std::optional & format_settings, - const ContextPtr & context); }; diff --git a/src/Storages/System/StorageSystemSchemaInferenceCache.cpp b/src/Storages/System/StorageSystemSchemaInferenceCache.cpp index a19cb1442c9..8dada69ee68 100644 --- a/src/Storages/System/StorageSystemSchemaInferenceCache.cpp +++ b/src/Storages/System/StorageSystemSchemaInferenceCache.cpp @@ -40,7 +40,8 @@ NamesAndTypesList StorageSystemSchemaInferenceCache::getNamesAndTypes() {"additional_format_info", std::make_shared()}, {"registration_time", std::make_shared()}, {"schema", std::make_shared(std::make_shared())}, - {"number_of_rows", std::make_shared(std::make_shared())} + {"number_of_rows", std::make_shared(std::make_shared())}, + {"schema_inference_mode", std::make_shared(std::make_shared())}, }; } @@ -64,6 +65,7 @@ static void fillDataImpl(MutableColumns & res_columns, SchemaCache & schema_cach res_columns[6]->insert(*schema_info.num_rows); else res_columns[6]->insertDefault(); + res_columns[7]->insert(key.schema_inference_mode); } } diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 3ec0506c525..c9a9880eaa8 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -1015,3 +1015,60 @@ def test_filtering_by_file_or_path(cluster): ) assert int(result) == 1 + + +def test_union_schema_inference_mode(cluster): + node = cluster.instances["node"] + azure_query( + node, + "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference1.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') select 1 as a", + ) + + azure_query( + node, + "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference2.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') select 2 as b", + ) + + node.query("system drop schema cache for azure") + + result = azure_query( + node, + "desc azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference*.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') settings schema_inference_mode='union', describe_compact_output=1 format TSV", + ) + assert result == "a\tNullable(Int64)\nb\tNullable(Int64)\n" + + result = node.query( + "select schema_inference_mode, splitByChar('/', source)[-1] as file, schema from system.schema_inference_cache where source like '%test_union_schema_inference%' order by file format TSV" + ) + assert ( + result == "UNION\ttest_union_schema_inference1.jsonl\ta Nullable(Int64)\n" + "UNION\ttest_union_schema_inference2.jsonl\tb Nullable(Int64)\n" + ) + result = azure_query( + node, + "select * from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference*.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') order by tuple(*) settings schema_inference_mode='union', describe_compact_output=1 format TSV", + ) + assert result == "1\t\\N\n" "\\N\t2\n" + node.query(f"system drop schema cache for hdfs") + result = azure_query( + node, + "desc azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference2.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') settings schema_inference_mode='union', describe_compact_output=1 format TSV", + ) + assert result == "b\tNullable(Int64)\n" + + result = azure_query( + node, + "desc azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference*.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') settings schema_inference_mode='union', describe_compact_output=1 format TSV", + ) + assert result == "a\tNullable(Int64)\n" "b\tNullable(Int64)\n" + azure_query( + node, + "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference3.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', TSV) select 'Error'", + ) + + error = azure_query( + node, + "desc azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference*.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') settings schema_inference_mode='union', describe_compact_output=1 format TSV", + expect_error="true", + ) + assert "Cannot extract table structure" in error diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index eacb5295079..8ed1e4b6c0e 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -998,6 +998,55 @@ def test_read_subcolumns(started_cluster): ) +def test_union_schema_inference_mode(started_cluster): + node = started_cluster.instances["node1"] + + node.query( + "insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference1.jsonl') select 1 as a" + ) + + node.query( + "insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference2.jsonl') select 2 as b" + ) + + node.query("system drop schema cache for hdfs") + + result = node.query( + "desc hdfs('hdfs://hdfs1:9000/test_union_schema_inference*.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" + ) + assert result == "a\tNullable(Int64)\nb\tNullable(Int64)\n" + + result = node.query( + "select schema_inference_mode, splitByChar('/', source)[-1] as file, schema from system.schema_inference_cache where source like '%test_union_schema_inference%' order by file format TSV" + ) + assert ( + result == "UNION\ttest_union_schema_inference1.jsonl\ta Nullable(Int64)\n" + "UNION\ttest_union_schema_inference2.jsonl\tb Nullable(Int64)\n" + ) + result = node.query( + "select * from hdfs('hdfs://hdfs1:9000/test_union_schema_inference*.jsonl') order by tuple(*) settings schema_inference_mode='union', describe_compact_output=1 format TSV" + ) + assert result == "1\t\\N\n" "\\N\t2\n" + node.query(f"system drop schema cache for hdfs") + result = node.query( + "desc hdfs('hdfs://hdfs1:9000/test_union_schema_inference2.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" + ) + assert result == "b\tNullable(Int64)\n" + + result = node.query( + "desc hdfs('hdfs://hdfs1:9000/test_union_schema_inference*.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" + ) + assert result == "a\tNullable(Int64)\n" "b\tNullable(Int64)\n" + node.query( + f"insert into function hdfs('hdfs://hdfs1:9000/test_union_schema_inference3.jsonl', TSV) select 'Error'" + ) + + error = node.query_and_get_error( + "desc hdfs('hdfs://hdfs1:9000/test_union_schema_inference*.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" + ) + assert "Cannot extract table structure" in error + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index ca2ddeb78b0..2f176a95e73 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -2072,3 +2072,65 @@ def test_filtering_by_file_or_path(started_cluster): ) assert int(result) == 1 + + +def test_union_schema_inference_mode(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["s3_non_default"] + + instance.query( + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_union_schema_inference1.jsonl') select 1 as a" + ) + + instance.query( + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_union_schema_inference2.jsonl') select 2 as b" + ) + + instance.query( + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_union_schema_inference3.jsonl') select 2 as c" + ) + + instance.query( + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_union_schema_inference4.jsonl', TSV) select 'Error'" + ) + + for engine in ["s3", "url"]: + instance.query("system drop schema cache for s3") + + result = instance.query( + f"desc {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_union_schema_inference{{1,2,3}}.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" + ) + assert result == "a\tNullable(Int64)\nb\tNullable(Int64)\nc\tNullable(Int64)\n" + + result = instance.query( + "select schema_inference_mode, splitByChar('/', source)[-1] as file, schema from system.schema_inference_cache where source like '%test_union_schema_inference%' order by file format TSV" + ) + assert ( + result == "UNION\ttest_union_schema_inference1.jsonl\ta Nullable(Int64)\n" + "UNION\ttest_union_schema_inference2.jsonl\tb Nullable(Int64)\n" + "UNION\ttest_union_schema_inference3.jsonl\tc Nullable(Int64)\n" + ) + result = instance.query( + f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_union_schema_inference{{1,2,3}}.jsonl') order by tuple(*) settings schema_inference_mode='union', describe_compact_output=1 format TSV" + ) + assert result == "1\t\\N\t\\N\n" "\\N\t2\t\\N\n" "\\N\t\\N\t2\n" + + instance.query(f"system drop schema cache for {engine}") + result = instance.query( + f"desc {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_union_schema_inference2.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" + ) + assert result == "b\tNullable(Int64)\n" + + result = instance.query( + f"desc {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_union_schema_inference{{1,2,3}}.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" + ) + assert ( + result == "a\tNullable(Int64)\n" + "b\tNullable(Int64)\n" + "c\tNullable(Int64)\n" + ) + + error = instance.query_and_get_error( + f"desc {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_union_schema_inference{{1,2,3,4}}.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV" + ) + assert "Cannot extract table structure" in error diff --git a/tests/queries/0_stateless/02900_union_schema_inference_mode.reference b/tests/queries/0_stateless/02900_union_schema_inference_mode.reference new file mode 100644 index 00000000000..3850ef3935b --- /dev/null +++ b/tests/queries/0_stateless/02900_union_schema_inference_mode.reference @@ -0,0 +1,33 @@ +c Nullable(String) +b Nullable(Int64) +obj Tuple(f2 Nullable(String), f3 Nullable(Int64), f1 Nullable(Int64)) +a Nullable(Int64) +{"c":"hello","b":null,"obj":{"f2":null,"f3":null,"f1":null},"a":null} +{"c":null,"b":"2","obj":{"f2":"Some string","f3":"2","f1":null},"a":null} +{"c":null,"b":null,"obj":{"f2":"2020-01-01","f3":null,"f1":"1"},"a":"1"} +UNION data1.jsonl a Nullable(Int64), obj Tuple(f1 Nullable(Int64), f2 Nullable(Date)) +UNION data2.jsonl b Nullable(Int64), obj Tuple(f2 Nullable(String), f3 Nullable(Int64)) +UNION data3.jsonl c Nullable(String) +c Nullable(String) +c Nullable(String) +b Nullable(Int64) +obj Tuple(f2 Nullable(String), f3 Nullable(Int64), f1 Nullable(Int64)) +a Nullable(Int64) +a Nullable(Int64) +obj Tuple(f1 Nullable(Int64), f2 Nullable(String), f3 Nullable(Int64)) +b Nullable(Int64) +c Nullable(String) +{"a":"1","obj":{"f1":"1","f2":"2020-01-01","f3":null},"b":null,"c":null} +{"a":null,"obj":{"f1":null,"f2":"Some string","f3":"2"},"b":"2","c":null} +{"a":null,"obj":{"f1":null,"f2":null,"f3":null},"b":null,"c":"hello"} +UNION archive.tar::data1.jsonl a Nullable(Int64), obj Tuple(f1 Nullable(Int64), f2 Nullable(Date)) +UNION archive.tar::data2.jsonl b Nullable(Int64), obj Tuple(f2 Nullable(String), f3 Nullable(Int64)) +UNION archive.tar::data3.jsonl c Nullable(String) +c Nullable(String) +a Nullable(Int64) +obj Tuple(f1 Nullable(Int64), f2 Nullable(String), f3 Nullable(Int64)) +b Nullable(Int64) +c Nullable(String) +1 +1 +1 diff --git a/tests/queries/0_stateless/02900_union_schema_inference_mode.sh b/tests/queries/0_stateless/02900_union_schema_inference_mode.sh new file mode 100755 index 00000000000..86b2e0801f9 --- /dev/null +++ b/tests/queries/0_stateless/02900_union_schema_inference_mode.sh @@ -0,0 +1,57 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-msan, no-ubsan + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +mkdir -p $CLICKHOUSE_TEST_UNIQUE_NAME +rm -rf $CLICKHOUSE_TEST_UNIQUE_NAME/* + +echo '{"a" : 1, "obj" : {"f1" : 1, "f2" : "2020-01-01"}}' > $CLICKHOUSE_TEST_UNIQUE_NAME/data1.jsonl +echo '{"b" : 2, "obj" : {"f3" : 2, "f2" : "Some string"}}' > $CLICKHOUSE_TEST_UNIQUE_NAME/data2.jsonl +echo '{"c" : "hello"}' > $CLICKHOUSE_TEST_UNIQUE_NAME/data3.jsonl + +$CLICKHOUSE_LOCAL -nm -q " +set schema_inference_mode = 'union'; +desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/data*.jsonl'); +select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME/data*.jsonl') order by tuple(*) format JSONEachRow; +select schema_inference_mode, splitByChar('/', source)[-1] as file, schema from system.schema_inference_cache order by file; +" + +$CLICKHOUSE_LOCAL -nm -q " +set schema_inference_mode = 'union'; +desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/data3.jsonl'); +desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/data*.jsonl'); +" + +cd $CLICKHOUSE_TEST_UNIQUE_NAME/ && tar -cf archive.tar data1.jsonl data2.jsonl data3.jsonl && cd .. + +$CLICKHOUSE_LOCAL -nm -q " +set schema_inference_mode = 'union'; +desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/archive.tar :: data*.jsonl'); +select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME/archive.tar :: data*.jsonl') order by tuple(*) format JSONEachRow; +select schema_inference_mode, splitByChar('/', source)[-1] as file, schema from system.schema_inference_cache order by file; +" + +$CLICKHOUSE_LOCAL -nm -q " +set schema_inference_mode = 'union'; +desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/archive.tar :: data3.jsonl'); +desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/archive.tar :: data*.jsonl'); +" + +echo 'Error' > $CLICKHOUSE_TEST_UNIQUE_NAME/data4.jsonl +$CLICKHOUSE_LOCAL -q "desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/data*.jsonl') settings schema_inference_mode='union'" 2>&1 | grep -c -F "Cannot extract table structure" + +$CLICKHOUSE_LOCAL -nm -q " +set schema_inference_mode = 'union'; +desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/data{2,3}.jsonl'); +desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/data*.jsonl'); +" 2>&1 | grep -c -F "Cannot extract table structure" + +echo 42 > $CLICKHOUSE_TEST_UNIQUE_NAME/data1.csv +echo 42, 43 > $CLICKHOUSE_TEST_UNIQUE_NAME/data2.csv + +$CLICKHOUSE_LOCAL -q "desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/data*.csv') settings schema_inference_mode='union'" 2>&1 | grep -c -F "BAD_ARGUMENTS"; + +rm -rf $CLICKHOUSE_TEST_UNIQUE_NAME From 9ec43d8cb994fad19043855345303faa8a713d33 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 20 Oct 2023 21:02:33 +0000 Subject: [PATCH 014/331] Add basic docs, fix unrelated changes --- docs/en/operations/settings/settings.md | 6 ++++++ src/Functions/FunctionGenerateRandomStructure.cpp | 12 ++++++------ 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index ccbf54843e4..1c8c9720121 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4346,6 +4346,12 @@ Possible values: Default value: 2. +### schema_inference_mode {schema_inference_mode} + +The mode of schema inference. Possible values: `default` and `union`. + +Default value: `default`. + ## compatibility {#compatibility} The `compatibility` setting causes ClickHouse to use the default settings of a previous version of ClickHouse, where the previous version is provided as the setting. diff --git a/src/Functions/FunctionGenerateRandomStructure.cpp b/src/Functions/FunctionGenerateRandomStructure.cpp index 8e086f075a4..f85b2596530 100644 --- a/src/Functions/FunctionGenerateRandomStructure.cpp +++ b/src/Functions/FunctionGenerateRandomStructure.cpp @@ -34,7 +34,7 @@ namespace const size_t MAX_DECIMAL256_PRECISION = 76; const size_t MAX_DEPTH = 16; - constexpr std::array simple_types + constexpr std::array simple_types { TypeIndex::Int8, TypeIndex::UInt8, @@ -64,7 +64,7 @@ namespace TypeIndex::Enum16, TypeIndex::IPv4, TypeIndex::IPv6, -// TypeIndex::UUID, + TypeIndex::UUID, }; constexpr std::array complex_types @@ -76,7 +76,7 @@ namespace TypeIndex::Map, }; - constexpr std::array map_key_types + constexpr std::array map_key_types { TypeIndex::Int8, TypeIndex::UInt8, @@ -98,11 +98,11 @@ namespace TypeIndex::IPv4, TypeIndex::Enum8, TypeIndex::Enum16, -// TypeIndex::UUID, + TypeIndex::UUID, TypeIndex::LowCardinality, }; - constexpr std::array suspicious_lc_types + constexpr std::array suspicious_lc_types { TypeIndex::Int8, TypeIndex::UInt8, @@ -125,7 +125,7 @@ namespace TypeIndex::FixedString, TypeIndex::IPv4, TypeIndex::IPv6, -// TypeIndex::UUID, + TypeIndex::UUID, }; template From 544b217d91c1978760a5d5fe334a3fbe447afd76 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 20 Oct 2023 21:05:26 +0000 Subject: [PATCH 015/331] Fix style --- src/Formats/ReadSchemaUtils.cpp | 1 + .../queries/0_stateless/02900_union_schema_inference_mode.sh | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index f2e831dfc46..c5630267e3f 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -14,6 +14,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int ONLY_NULLS_WHILE_READING_SCHEMA; extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; + extern const int TYPE_MISMATCH; } static std::optional getOrderedColumnsList(const NamesAndTypesList & columns_list, const Names & columns_order_hint) diff --git a/tests/queries/0_stateless/02900_union_schema_inference_mode.sh b/tests/queries/0_stateless/02900_union_schema_inference_mode.sh index 86b2e0801f9..76a7d7e4cd3 100755 --- a/tests/queries/0_stateless/02900_union_schema_inference_mode.sh +++ b/tests/queries/0_stateless/02900_union_schema_inference_mode.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh mkdir -p $CLICKHOUSE_TEST_UNIQUE_NAME -rm -rf $CLICKHOUSE_TEST_UNIQUE_NAME/* +rm -rf ${CLICKHOUSE_TEST_UNIQUE_NAME}/* echo '{"a" : 1, "obj" : {"f1" : 1, "f2" : "2020-01-01"}}' > $CLICKHOUSE_TEST_UNIQUE_NAME/data1.jsonl echo '{"b" : 2, "obj" : {"f3" : 2, "f2" : "Some string"}}' > $CLICKHOUSE_TEST_UNIQUE_NAME/data2.jsonl @@ -54,4 +54,5 @@ echo 42, 43 > $CLICKHOUSE_TEST_UNIQUE_NAME/data2.csv $CLICKHOUSE_LOCAL -q "desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/data*.csv') settings schema_inference_mode='union'" 2>&1 | grep -c -F "BAD_ARGUMENTS"; -rm -rf $CLICKHOUSE_TEST_UNIQUE_NAME +rm -rf ${CLICKHOUSE_TEST_UNIQUE_NAME} + From b0c3fee4a98d06e1de3c8ee7e52963dbfa39c9e2 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 20 Oct 2023 21:25:01 +0000 Subject: [PATCH 016/331] Fix style --- tests/queries/0_stateless/02900_union_schema_inference_mode.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02900_union_schema_inference_mode.sh b/tests/queries/0_stateless/02900_union_schema_inference_mode.sh index 76a7d7e4cd3..0e520e27cb4 100755 --- a/tests/queries/0_stateless/02900_union_schema_inference_mode.sh +++ b/tests/queries/0_stateless/02900_union_schema_inference_mode.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh mkdir -p $CLICKHOUSE_TEST_UNIQUE_NAME -rm -rf ${CLICKHOUSE_TEST_UNIQUE_NAME}/* +rm -rf "${CLICKHOUSE_TEST_UNIQUE_NAME}/*" echo '{"a" : 1, "obj" : {"f1" : 1, "f2" : "2020-01-01"}}' > $CLICKHOUSE_TEST_UNIQUE_NAME/data1.jsonl echo '{"b" : 2, "obj" : {"f3" : 2, "f2" : "Some string"}}' > $CLICKHOUSE_TEST_UNIQUE_NAME/data2.jsonl From de90e5fbb66abc61349a42e854b54a0531829a3b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 20 Oct 2023 23:39:09 +0200 Subject: [PATCH 017/331] Try fix style --- tests/queries/0_stateless/02900_union_schema_inference_mode.sh | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/02900_union_schema_inference_mode.sh b/tests/queries/0_stateless/02900_union_schema_inference_mode.sh index 0e520e27cb4..5e53b857056 100755 --- a/tests/queries/0_stateless/02900_union_schema_inference_mode.sh +++ b/tests/queries/0_stateless/02900_union_schema_inference_mode.sh @@ -6,8 +6,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh mkdir -p $CLICKHOUSE_TEST_UNIQUE_NAME -rm -rf "${CLICKHOUSE_TEST_UNIQUE_NAME}/*" - echo '{"a" : 1, "obj" : {"f1" : 1, "f2" : "2020-01-01"}}' > $CLICKHOUSE_TEST_UNIQUE_NAME/data1.jsonl echo '{"b" : 2, "obj" : {"f3" : 2, "f2" : "Some string"}}' > $CLICKHOUSE_TEST_UNIQUE_NAME/data2.jsonl echo '{"c" : "hello"}' > $CLICKHOUSE_TEST_UNIQUE_NAME/data3.jsonl From cfa510ea0ac324fc9c279f3f4afcb621104541d9 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 23 Oct 2023 14:38:34 +0000 Subject: [PATCH 018/331] Add more documentation, fix build --- docs/en/interfaces/schema-inference.md | 99 +++++++++++++++++++++++++ docs/en/operations/settings/settings.md | 3 + src/Storages/StorageS3.cpp | 12 +-- 3 files changed, 108 insertions(+), 6 deletions(-) diff --git a/docs/en/interfaces/schema-inference.md b/docs/en/interfaces/schema-inference.md index 0aadb09730a..f361bd6cdb2 100644 --- a/docs/en/interfaces/schema-inference.md +++ b/docs/en/interfaces/schema-inference.md @@ -1846,3 +1846,102 @@ DESC format(JSONAsString, '{"x" : 42, "y" : "Hello, World!"}') SETTINGS allow_ex │ json │ Object('json') │ │ │ │ │ │ └──────┴────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` + +## Schema inference modes {#schema-inference-modes} + +Schema inference from the set of data files can work in 2 different modes: `default` and `union`. +The mode is controlled by the setting `schema_inference_mode`. + +### Default mode {#default-schema-inference-mode} + +In default mode, ClickHouse assumes that all files have the same schema and tries to infer the schema by reading files one by one until it succeeds. + +Example: + +Let's say we have 3 files `data1.jsonl`, `data2.jsonl` and `data3.jsonl` with the next content: + +`data1.jsonl`: +```json +{"field1" : 1, "field2" : null} +{"field1" : 2, "field2" : null} +{"field1" : 3, "field2" : null} +``` + +`data2.jsonl`: +```json +{"field1" : 4, "field2" : "Data4"} +{"field1" : 5, "field2" : "Data5"} +{"field1" : 6, "field2" : "Data5"} +``` + +`data3.jsonl`: +```json +{"field1" : 7, "field2" : "Data7", "field3" : [1, 2, 3]} +{"field1" : 8, "field2" : "Data8", "field3" : [4, 5, 6]} +{"field1" : 9, "field2" : "Data9", "field3" : [7, 8, 9]} +``` + +Let's try to use schema inference on these 3 files: +```sql +:) DESCRIBE file('data{1,2,3}.jsonl') SETTINGS schema_inference_mode='default' +``` + +Result: +```text +┌─name───┬─type─────────────┐ +│ field1 │ Nullable(Int64) │ +│ field2 │ Nullable(String) │ +└────────┴──────────────────┘ +``` + +As we can see, we don't have `field3` from file `data3.jsonl`. +It happens because ClickHouse first tried to infer schema from file `data1.jsonl`, failed because of only nulls for field `field2`, +and then tried to infer schema from `data2.jsonl` and succeeded, so data from file `data3.jsonl` wasn't read. + +### Union mode {#default-schema-inference-mode} + +In union mode, ClickHouse assumes that files can have different schemas, so it infer schemas of all files and then union them to the common schema. + +Let's say we have 3 files `data1.jsonl`, `data2.jsonl` and `data3.jsonl` with the next content: + +`data1.jsonl`: +```json +{"field1" : 1} +{"field1" : 2} +{"field1" : 3} +``` + +`data2.jsonl`: +```json +{"field2" : "Data4"} +{"field2" : "Data5"} +{"field2" : "Data5"} +``` + +`data3.jsonl`: +```json +{"field3" : [1, 2, 3]} +{"field3" : [4, 5, 6]} +{"field3" : [7, 8, 9]} +``` + +Let's try to use schema inference on these 3 files: +```sql +:) DESCRIBE file('data{1,2,3}.jsonl') SETTINGS schema_inference_mode='union' +``` + +Result: +```text +┌─name───┬─type───────────────────┐ +│ field1 │ Nullable(Int64) │ +│ field2 │ Nullable(String) │ +│ field3 │ Array(Nullable(Int64)) │ +└────────┴────────────────────────┘ +``` + +As we can see, we have all fields from all files. + +Note: +- As some of the files may not contain some columns from the resulting schema, union mode is supported only for formats that support reading subset of columns (like JSONEachRow, Parquet, TSVWithNames, etc) and won't work for other formats (like CSV, TSV, JSONCompactEachEow, etc). +- If ClickHouse cannot infer the schema from one of the files, the exception will be thrown. +- If you have a lot of files, reading schema from all of them can take a lot of time. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 1c8c9720121..27ac051631a 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4305,6 +4305,8 @@ Default value: `1GiB`. ## Schema Inference settings +See [schema inference](../../interfaces/schema-inference.md#schema-inference-modes) documentation for more details. + ### schema_inference_use_cache_for_file {schema_inference_use_cache_for_file} Enable schemas cache for schema inference in `file` table function. @@ -4349,6 +4351,7 @@ Default value: 2. ### schema_inference_mode {schema_inference_mode} The mode of schema inference. Possible values: `default` and `union`. +See [schema inference modes](../../interfaces/schema-inference.md#schema-inference-modes) section for more details. Default value: `default`. diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index b6df46ed589..7430ec5e65c 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1539,7 +1539,7 @@ namespace /// In union mode, check cached columns only for current key. if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION) { - StorageS3::KeysWithInfo keys = {*current_key_with_info}; + StorageS3::KeysWithInfo keys = {current_key_with_info}; if (auto columns_from_cache = tryGetColumnsFromCache(keys.begin(), keys.end())) { first = false; @@ -1587,7 +1587,7 @@ namespace auto host_and_bucket = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / configuration.url.bucket; Strings sources; sources.reserve(read_keys.size()); - std::transform(read_keys.begin(), read_keys.end(), std::back_inserter(sources), [&](const auto & elem){ return host_and_bucket / elem.key; }); + std::transform(read_keys.begin(), read_keys.end(), std::back_inserter(sources), [&](const auto & elem){ return host_and_bucket / elem->key; }); auto cache_keys = getKeysForSchemaCache(sources, configuration.format, format_settings, getContext()); StorageS3::getSchemaCache(getContext()).addManyColumns(cache_keys, columns); } @@ -1608,9 +1608,9 @@ namespace auto get_last_mod_time = [&] { time_t last_modification_time = 0; - if (it->info) + if ((*it)->info) { - last_modification_time = it->info->last_modification_time; + last_modification_time = (*it)->info->last_modification_time; } else { @@ -1620,7 +1620,7 @@ namespace last_modification_time = S3::getObjectInfo( *configuration.client, configuration.url.bucket, - it->key, + (*it)->key, configuration.url.version_id, configuration.request_settings, /*with_metadata=*/ false, @@ -1631,7 +1631,7 @@ namespace return last_modification_time ? std::make_optional(last_modification_time) : std::nullopt; }; - String path = fs::path(configuration.url.bucket) / it->key; + String path = fs::path(configuration.url.bucket) / (*it)->key; String source = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / path; auto cache_key = getKeyForSchemaCache(source, configuration.format, format_settings, getContext()); auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time); From 91632e355454569b82db0ac3576d8c5857504ee4 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 24 Oct 2023 14:30:05 +0200 Subject: [PATCH 019/331] Fix typo in docs --- docs/en/interfaces/schema-inference.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/schema-inference.md b/docs/en/interfaces/schema-inference.md index f361bd6cdb2..b2da03c5776 100644 --- a/docs/en/interfaces/schema-inference.md +++ b/docs/en/interfaces/schema-inference.md @@ -1942,6 +1942,6 @@ Result: As we can see, we have all fields from all files. Note: -- As some of the files may not contain some columns from the resulting schema, union mode is supported only for formats that support reading subset of columns (like JSONEachRow, Parquet, TSVWithNames, etc) and won't work for other formats (like CSV, TSV, JSONCompactEachEow, etc). +- As some of the files may not contain some columns from the resulting schema, union mode is supported only for formats that support reading subset of columns (like JSONEachRow, Parquet, TSVWithNames, etc) and won't work for other formats (like CSV, TSV, JSONCompactEachRow, etc). - If ClickHouse cannot infer the schema from one of the files, the exception will be thrown. - If you have a lot of files, reading schema from all of them can take a lot of time. From 64e5797fc85ed208f6542a11448a0035e3014570 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 24 Oct 2023 14:30:26 +0200 Subject: [PATCH 020/331] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 2c73c1032f9..f104c13eab7 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -860,6 +860,7 @@ TLSv TPCH TSDB TSVRaw +TSVWithNames TSVs TSan TThe From 844c1fb6888bcf620484717e8599bb67a49301ce Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 24 Oct 2023 19:35:03 +0200 Subject: [PATCH 021/331] Fix --- src/Storages/StorageS3.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 7430ec5e65c..ebfd36fd9c8 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1592,7 +1592,12 @@ namespace StorageS3::getSchemaCache(getContext()).addManyColumns(cache_keys, columns); } - String getLastFileName() const override { return current_key_with_info->key; } + String getLastFileName() const override + { + if (current_key_with_info) + return current_key_with_info->key; + return ""; + } private: std::optional tryGetColumnsFromCache( From e630868eac58c362c0b9c25a879ed065e269b782 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 27 Oct 2023 14:11:27 +0000 Subject: [PATCH 022/331] Fix tests --- src/Storages/StorageAzureBlob.cpp | 2 +- ...375_system_schema_inference_cache.reference | 1 + .../02721_parquet_field_not_found.sh | 2 +- ...02900_union_schema_inference_mode.reference | 18 +++++++++--------- .../02900_union_schema_inference_mode.sh | 18 +++++++++--------- 5 files changed, 21 insertions(+), 20 deletions(-) diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index b005f1a4445..afd3e3c4b4b 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -1285,7 +1285,7 @@ namespace void setResultingSchema(const ColumnsDescription & columns) override { if (!getContext()->getSettingsRef().schema_inference_use_cache_for_azure - || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::UNION) + || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::DEFAULT) return; auto host_and_bucket = configuration.connection_url + '/' + configuration.container; diff --git a/tests/queries/0_stateless/02375_system_schema_inference_cache.reference b/tests/queries/0_stateless/02375_system_schema_inference_cache.reference index 94bdf1b5ddb..676fb441f53 100644 --- a/tests/queries/0_stateless/02375_system_schema_inference_cache.reference +++ b/tests/queries/0_stateless/02375_system_schema_inference_cache.reference @@ -5,6 +5,7 @@ additional_format_info String registration_time DateTime schema Nullable(String) number_of_rows Nullable(UInt64) +schema_inference_mode Nullable(String) x Nullable(Int64) s Nullable(String) x Nullable(Int64) diff --git a/tests/queries/0_stateless/02721_parquet_field_not_found.sh b/tests/queries/0_stateless/02721_parquet_field_not_found.sh index 72925afec6e..8b366d0e9a2 100755 --- a/tests/queries/0_stateless/02721_parquet_field_not_found.sh +++ b/tests/queries/0_stateless/02721_parquet_field_not_found.sh @@ -5,5 +5,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL -q "select 42 as x format Parquet" | $CLICKHOUSE_LOCAL --input-format Parquet --structure "x UInt32, y UInt32" -q "select * from table" 2>&1 | grep -c "THERE_IS_NO_COLUMN" +$CLICKHOUSE_LOCAL -q "select 42 as x format Parquet" | $CLICKHOUSE_LOCAL --input-format Parquet --structure "x UInt32, y UInt32" --input_format_parquet_allow_missing_columns=0 -q "select * from table" 2>&1 | grep -c "THERE_IS_NO_COLUMN" diff --git a/tests/queries/0_stateless/02900_union_schema_inference_mode.reference b/tests/queries/0_stateless/02900_union_schema_inference_mode.reference index 3850ef3935b..864cd780ddb 100644 --- a/tests/queries/0_stateless/02900_union_schema_inference_mode.reference +++ b/tests/queries/0_stateless/02900_union_schema_inference_mode.reference @@ -1,18 +1,18 @@ -c Nullable(String) -b Nullable(Int64) -obj Tuple(f2 Nullable(String), f3 Nullable(Int64), f1 Nullable(Int64)) a Nullable(Int64) -{"c":"hello","b":null,"obj":{"f2":null,"f3":null,"f1":null},"a":null} -{"c":null,"b":"2","obj":{"f2":"Some string","f3":"2","f1":null},"a":null} -{"c":null,"b":null,"obj":{"f2":"2020-01-01","f3":null,"f1":"1"},"a":"1"} +obj Tuple(f1 Nullable(Int64), f2 Nullable(String), f3 Nullable(Int64)) +b Nullable(Int64) +c Nullable(String) +{"a":"1","obj":{"f1":"1","f2":"2020-01-01","f3":null},"b":null,"c":null} +{"a":null,"obj":{"f1":null,"f2":"Some string","f3":"2"},"b":"2","c":null} +{"a":null,"obj":{"f1":null,"f2":null,"f3":null},"b":null,"c":"hello"} UNION data1.jsonl a Nullable(Int64), obj Tuple(f1 Nullable(Int64), f2 Nullable(Date)) UNION data2.jsonl b Nullable(Int64), obj Tuple(f2 Nullable(String), f3 Nullable(Int64)) UNION data3.jsonl c Nullable(String) c Nullable(String) -c Nullable(String) -b Nullable(Int64) -obj Tuple(f2 Nullable(String), f3 Nullable(Int64), f1 Nullable(Int64)) a Nullable(Int64) +obj Tuple(f1 Nullable(Int64), f2 Nullable(String), f3 Nullable(Int64)) +b Nullable(Int64) +c Nullable(String) a Nullable(Int64) obj Tuple(f1 Nullable(Int64), f2 Nullable(String), f3 Nullable(Int64)) b Nullable(Int64) diff --git a/tests/queries/0_stateless/02900_union_schema_inference_mode.sh b/tests/queries/0_stateless/02900_union_schema_inference_mode.sh index 5e53b857056..dc0dd8ae1f4 100755 --- a/tests/queries/0_stateless/02900_union_schema_inference_mode.sh +++ b/tests/queries/0_stateless/02900_union_schema_inference_mode.sh @@ -12,45 +12,45 @@ echo '{"c" : "hello"}' > $CLICKHOUSE_TEST_UNIQUE_NAME/data3.jsonl $CLICKHOUSE_LOCAL -nm -q " set schema_inference_mode = 'union'; -desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/data*.jsonl'); -select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME/data*.jsonl') order by tuple(*) format JSONEachRow; +desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/data{1,2,3}.jsonl'); +select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME/data{1,2,3}.jsonl') order by tuple(*) format JSONEachRow; select schema_inference_mode, splitByChar('/', source)[-1] as file, schema from system.schema_inference_cache order by file; " $CLICKHOUSE_LOCAL -nm -q " set schema_inference_mode = 'union'; desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/data3.jsonl'); -desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/data*.jsonl'); +desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/data{1,2,3}.jsonl'); " cd $CLICKHOUSE_TEST_UNIQUE_NAME/ && tar -cf archive.tar data1.jsonl data2.jsonl data3.jsonl && cd .. $CLICKHOUSE_LOCAL -nm -q " set schema_inference_mode = 'union'; -desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/archive.tar :: data*.jsonl'); -select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME/archive.tar :: data*.jsonl') order by tuple(*) format JSONEachRow; +desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/archive.tar :: data{1,2,3}.jsonl'); +select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME/archive.tar :: data{1,2,3}.jsonl') order by tuple(*) format JSONEachRow; select schema_inference_mode, splitByChar('/', source)[-1] as file, schema from system.schema_inference_cache order by file; " $CLICKHOUSE_LOCAL -nm -q " set schema_inference_mode = 'union'; desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/archive.tar :: data3.jsonl'); -desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/archive.tar :: data*.jsonl'); +desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/archive.tar :: data{1,2,3}.jsonl'); " echo 'Error' > $CLICKHOUSE_TEST_UNIQUE_NAME/data4.jsonl -$CLICKHOUSE_LOCAL -q "desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/data*.jsonl') settings schema_inference_mode='union'" 2>&1 | grep -c -F "Cannot extract table structure" +$CLICKHOUSE_LOCAL -q "desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/data{1,2,3,4}.jsonl') settings schema_inference_mode='union'" 2>&1 | grep -c -F "Cannot extract table structure" $CLICKHOUSE_LOCAL -nm -q " set schema_inference_mode = 'union'; desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/data{2,3}.jsonl'); -desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/data*.jsonl'); +desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/data{1,2,3,4}.jsonl'); " 2>&1 | grep -c -F "Cannot extract table structure" echo 42 > $CLICKHOUSE_TEST_UNIQUE_NAME/data1.csv echo 42, 43 > $CLICKHOUSE_TEST_UNIQUE_NAME/data2.csv -$CLICKHOUSE_LOCAL -q "desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/data*.csv') settings schema_inference_mode='union'" 2>&1 | grep -c -F "BAD_ARGUMENTS"; +$CLICKHOUSE_LOCAL -q "desc file('$CLICKHOUSE_TEST_UNIQUE_NAME/data{1,2}.csv') settings schema_inference_mode='union'" 2>&1 | grep -c -F "BAD_ARGUMENTS"; rm -rf ${CLICKHOUSE_TEST_UNIQUE_NAME} From 9eb498b48d4238158fda98884e6e5d225467fa03 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 30 Oct 2023 10:50:05 +0000 Subject: [PATCH 023/331] Fix tests for HDFS --- src/Storages/HDFS/StorageHDFS.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 730b694ef87..e692e900a18 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -391,6 +391,9 @@ namespace private: std::optional tryGetColumnsFromCache(const std::vector & paths_with_info_) { + if (!getContext()->getSettingsRef().schema_inference_use_cache_for_hdfs) + return std::nullopt; + auto & schema_cache = StorageHDFS::getSchemaCache(getContext()); for (const auto & path_with_info : paths_with_info_) { From 7689961a8f1e12bdf6050bc90dc393c59ba3f4f1 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 30 Oct 2023 19:44:02 +0000 Subject: [PATCH 024/331] Try to fix test with azuer --- .../test_storage_azure_blob_storage/test.py | 336 ++++++++++++------ 1 file changed, 230 insertions(+), 106 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 909ff490481..a90879c3a00 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -29,7 +29,6 @@ def cluster(): with_azurite=True, ) cluster.start() - yield cluster finally: cluster.shutdown() @@ -50,6 +49,7 @@ def azure_query( "DB::Exception: Azure::Core::Http::TransportException: Connection closed before getting full response or response is less than expected", "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response", "DB::Exception: Azure::Core::Http::TransportException: Error while polling for socket ready read", + "DB::Exception: Azure::Core::Http::TransportException: Fail to get a new connection" "Azure::Core::Http::TransportException, e.what() = Connection was closed by the server while trying to read a response", "Azure::Core::Http::TransportException, e.what() = Connection closed before getting full response or response is less than expected", "Azure::Core::Http::TransportException, e.what() = Connection was closed by the server while trying to read a response", @@ -69,19 +69,29 @@ def azure_query( continue -def get_azure_file_content(filename): +def get_azure_file_content(filename, port): container_name = "cont" - connection_string = "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://127.0.0.1:10000/devstoreaccount1;" - blob_service_client = BlobServiceClient.from_connection_string(connection_string) + connection_string = ( + f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" + f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" + f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" + ) + blob_service_client = BlobServiceClient.from_connection_string( + str(connection_string) + ) container_client = blob_service_client.get_container_client(container_name) blob_client = container_client.get_blob_client(filename) download_stream = blob_client.download_blob() return download_stream.readall().decode("utf-8") -def put_azure_file_content(filename, data): +def put_azure_file_content(filename, port, data): container_name = "cont" - connection_string = "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://127.0.0.1:10000/devstoreaccount1;" + connection_string = ( + f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" + f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" + f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" + ) blob_service_client = BlobServiceClient.from_connection_string(connection_string) try: container_client = blob_service_client.create_container(container_name) @@ -94,8 +104,13 @@ def put_azure_file_content(filename, data): @pytest.fixture(autouse=True, scope="function") -def delete_all_files(): - connection_string = "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://127.0.0.1:10000/devstoreaccount1;" +def delete_all_files(cluster): + port = cluster.env_variables["AZURITE_PORT"] + connection_string = ( + f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" + f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" + f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" + ) blob_service_client = BlobServiceClient.from_connection_string(connection_string) containers = blob_service_client.list_containers() for container in containers: @@ -115,7 +130,8 @@ def test_create_table_connection_string(cluster): node = cluster.instances["node"] azure_query( node, - "CREATE TABLE test_create_table_conn_string (key UInt64, data String) Engine = AzureBlobStorage('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1/;', 'cont', 'test_create_connection_string', 'CSV')", + f"CREATE TABLE test_create_table_conn_string (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}'," + f"'cont', 'test_create_connection_string', 'CSV')", ) @@ -123,57 +139,67 @@ def test_create_table_account_string(cluster): node = cluster.instances["node"] azure_query( node, - "CREATE TABLE test_create_table_account_url (key UInt64, data String) Engine = AzureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_create_connection_string', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')", + f"CREATE TABLE test_create_table_account_url (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f"'cont', 'test_create_connection_string', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')", ) def test_simple_write_account_string(cluster): node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - "CREATE TABLE test_simple_write (key UInt64, data String) Engine = AzureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')", + f"CREATE TABLE test_simple_write (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" 'cont', 'test_simple_write.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')", ) azure_query(node, "INSERT INTO test_simple_write VALUES (1, 'a')") - print(get_azure_file_content("test_simple_write.csv")) - assert get_azure_file_content("test_simple_write.csv") == '1,"a"\n' + print(get_azure_file_content("test_simple_write.csv", port)) + assert get_azure_file_content("test_simple_write.csv", port) == '1,"a"\n' def test_simple_write_connection_string(cluster): node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - "CREATE TABLE test_simple_write_connection_string (key UInt64, data String) Engine = AzureBlobStorage('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;', 'cont', 'test_simple_write_c.csv', 'CSV')", + f"CREATE TABLE test_simple_write_connection_string (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', " + f"'cont', 'test_simple_write_c.csv', 'CSV')", ) azure_query(node, "INSERT INTO test_simple_write_connection_string VALUES (1, 'a')") - print(get_azure_file_content("test_simple_write_c.csv")) - assert get_azure_file_content("test_simple_write_c.csv") == '1,"a"\n' + print(get_azure_file_content("test_simple_write_c.csv", port)) + assert get_azure_file_content("test_simple_write_c.csv", port) == '1,"a"\n' def test_simple_write_named_collection_1(cluster): node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - "CREATE TABLE test_simple_write_named_collection_1 (key UInt64, data String) Engine = AzureBlobStorage(azure_conf1)", + f"CREATE TABLE test_simple_write_named_collection_1 (key UInt64, data String) Engine = AzureBlobStorage(azure_conf1, " + f"connection_string = '{cluster.env_variables['AZURITE_CONNECTION_STRING']}')", ) azure_query( node, "INSERT INTO test_simple_write_named_collection_1 VALUES (1, 'a')" ) - print(get_azure_file_content("test_simple_write_named.csv")) - assert get_azure_file_content("test_simple_write_named.csv") == '1,"a"\n' + print(get_azure_file_content("test_simple_write_named.csv", port)) + assert get_azure_file_content("test_simple_write_named.csv", port) == '1,"a"\n' azure_query(node, "TRUNCATE TABLE test_simple_write_named_collection_1") def test_simple_write_named_collection_2(cluster): node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - "CREATE TABLE test_simple_write_named_collection_2 (key UInt64, data String) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_simple_write_named_2.csv', format='CSV')", + f"CREATE TABLE test_simple_write_named_collection_2 (key UInt64, data String) Engine = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " + f"container='cont', blob_path='test_simple_write_named_2.csv', format='CSV')", ) azure_query( node, "INSERT INTO test_simple_write_named_collection_2 VALUES (1, 'a')" ) - print(get_azure_file_content("test_simple_write_named_2.csv")) - assert get_azure_file_content("test_simple_write_named_2.csv") == '1,"a"\n' + print(get_azure_file_content("test_simple_write_named_2.csv", port)) + assert get_azure_file_content("test_simple_write_named_2.csv", port) == '1,"a"\n' def test_partition_by(cluster): @@ -182,16 +208,19 @@ def test_partition_by(cluster): partition_by = "column3" values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)" filename = "test_{_partition_id}.csv" + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - f"CREATE TABLE test_partitioned_write ({table_format}) Engine = AzureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') PARTITION BY {partition_by}", + f"CREATE TABLE test_partitioned_write ({table_format}) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') " + f"PARTITION BY {partition_by}", ) azure_query(node, f"INSERT INTO test_partitioned_write VALUES {values}") - assert "1,2,3\n" == get_azure_file_content("test_3.csv") - assert "3,2,1\n" == get_azure_file_content("test_1.csv") - assert "78,43,45\n" == get_azure_file_content("test_45.csv") + assert "1,2,3\n" == get_azure_file_content("test_3.csv", port) + assert "3,2,1\n" == get_azure_file_content("test_1.csv", port) + assert "78,43,45\n" == get_azure_file_content("test_45.csv", port) def test_partition_by_string_column(cluster): @@ -200,15 +229,18 @@ def test_partition_by_string_column(cluster): partition_by = "col_str" values = "(1, 'foo/bar'), (3, 'йцук'), (78, '你好')" filename = "test_{_partition_id}.csv" + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - f"CREATE TABLE test_partitioned_string_write ({table_format}) Engine = AzureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') PARTITION BY {partition_by}", + f"CREATE TABLE test_partitioned_string_write ({table_format}) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') " + f"PARTITION BY {partition_by}", ) azure_query(node, f"INSERT INTO test_partitioned_string_write VALUES {values}") - assert '1,"foo/bar"\n' == get_azure_file_content("test_foo/bar.csv") - assert '3,"йцук"\n' == get_azure_file_content("test_йцук.csv") - assert '78,"你好"\n' == get_azure_file_content("test_你好.csv") + assert '1,"foo/bar"\n' == get_azure_file_content("test_foo/bar.csv", port) + assert '3,"йцук"\n' == get_azure_file_content("test_йцук.csv", port) + assert '78,"你好"\n' == get_azure_file_content("test_你好.csv", port) def test_partition_by_const_column(cluster): @@ -218,46 +250,54 @@ def test_partition_by_const_column(cluster): partition_by = "'88'" values_csv = "1,2,3\n3,2,1\n78,43,45\n" filename = "test_{_partition_id}.csv" + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - f"CREATE TABLE test_partitioned_const_write ({table_format}) Engine = AzureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV') PARTITION BY {partition_by}", + f"CREATE TABLE test_partitioned_const_write ({table_format}) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV')" + f" PARTITION BY {partition_by}", ) azure_query(node, f"INSERT INTO test_partitioned_const_write VALUES {values}") - assert values_csv == get_azure_file_content("test_88.csv") + assert values_csv == get_azure_file_content("test_88.csv", port) def test_truncate(cluster): node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - "CREATE TABLE test_truncate (key UInt64, data String) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_truncate.csv', format='CSV')", + f"CREATE TABLE test_truncate (key UInt64, data String) Engine = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path='test_truncate.csv', format='CSV')", ) azure_query(node, "INSERT INTO test_truncate VALUES (1, 'a')") - assert get_azure_file_content("test_truncate.csv") == '1,"a"\n' + assert get_azure_file_content("test_truncate.csv", port) == '1,"a"\n' azure_query(node, "TRUNCATE TABLE test_truncate") with pytest.raises(Exception): - print(get_azure_file_content("test_truncate.csv")) + print(get_azure_file_content("test_truncate.csv", port)) def test_simple_read_write(cluster): node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - "CREATE TABLE test_simple_read_write (key UInt64, data String) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_simple_read_write.csv', format='CSV')", + f"CREATE TABLE test_simple_read_write (key UInt64, data String) Engine = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path='test_simple_read_write.csv', " + f"format='CSV')", ) azure_query(node, "INSERT INTO test_simple_read_write VALUES (1, 'a')") - assert get_azure_file_content("test_simple_read_write.csv") == '1,"a"\n' + assert get_azure_file_content("test_simple_read_write.csv", port) == '1,"a"\n' print(azure_query(node, "SELECT * FROM test_simple_read_write")) assert azure_query(node, "SELECT * FROM test_simple_read_write") == "1\ta\n" def test_create_new_files_on_insert(cluster): node = cluster.instances["node"] - azure_query( node, - f"create table test_multiple_inserts(a Int32, b String) ENGINE = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_parquet', format='Parquet')", + f"create table test_multiple_inserts(a Int32, b String) ENGINE = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path='test_parquet', format='Parquet')", ) azure_query(node, "truncate table test_multiple_inserts") azure_query( @@ -281,10 +321,10 @@ def test_create_new_files_on_insert(cluster): def test_overwrite(cluster): node = cluster.instances["node"] - azure_query( node, - f"create table test_overwrite(a Int32, b String) ENGINE = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_parquet_overwrite', format='Parquet')", + f"create table test_overwrite(a Int32, b String) ENGINE = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path='test_parquet_overwrite', format='Parquet')", ) azure_query(node, "truncate table test_overwrite") @@ -308,7 +348,8 @@ def test_insert_with_path_with_globs(cluster): node = cluster.instances["node"] azure_query( node, - f"create table test_insert_globs(a Int32, b String) ENGINE = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_insert_with_globs*', format='Parquet')", + f"create table test_insert_globs(a Int32, b String) ENGINE = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path='test_insert_with_globs*', format='Parquet')", ) node.query_and_get_error( f"insert into table function test_insert_globs SELECT number, randomString(100) FROM numbers(500)" @@ -331,7 +372,8 @@ def test_put_get_with_globs(cluster): azure_query( node, - f"CREATE TABLE test_put_{i}_{j} ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSV')", + f"CREATE TABLE test_put_{i}_{j} ({table_format}) Engine = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path='{path}', format='CSV')", ) query = f"insert into test_put_{i}_{j} VALUES {values}" @@ -339,7 +381,8 @@ def test_put_get_with_globs(cluster): azure_query( node, - f"CREATE TABLE test_glob_select ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv', format='CSV')", + f"CREATE TABLE test_glob_select ({table_format}) Engine = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv', format='CSV')", ) query = "select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from test_glob_select" assert azure_query(node, query).splitlines() == [ @@ -363,7 +406,8 @@ def test_azure_glob_scheherazade(cluster): unique_num = random.randint(1, 10000) azure_query( node, - f"CREATE TABLE test_scheherazade_{i}_{unique_num} ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSV')", + f"CREATE TABLE test_scheherazade_{i}_{unique_num} ({table_format}) Engine = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path='{path}', format='CSV')", ) query = ( f"insert into test_scheherazade_{i}_{unique_num} VALUES {values}" @@ -382,7 +426,8 @@ def test_azure_glob_scheherazade(cluster): azure_query( node, - f"CREATE TABLE test_glob_select_scheherazade ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='night_*/tale.csv', format='CSV')", + f"CREATE TABLE test_glob_select_scheherazade ({table_format}) Engine = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path='night_*/tale.csv', format='CSV')", ) query = "select count(), sum(column1), sum(column2), sum(column3) from test_glob_select_scheherazade" assert azure_query(node, query).splitlines() == ["1001\t1001\t1001\t1001"] @@ -394,6 +439,7 @@ def test_azure_glob_scheherazade(cluster): ) def test_storage_azure_get_gzip(cluster, extension, method): node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] filename = f"test_get_gzip.{extension}" name = f"test_get_gzip_{extension}" data = [ @@ -420,14 +466,13 @@ def test_storage_azure_get_gzip(cluster, extension, method): compressed = gzip.GzipFile(fileobj=buf, mode="wb") compressed.write(("\n".join(data)).encode()) compressed.close() - put_azure_file_content(filename, buf.getvalue()) + put_azure_file_content(filename, port, buf.getvalue()) azure_query( node, - f"""CREATE TABLE {name} (name String, id UInt32) ENGINE = AzureBlobStorage( - azure_conf2, container='cont', blob_path ='{filename}', - format='CSV', - compression='{method}')""", + f"CREATE TABLE {name} (name String, id UInt32) ENGINE = AzureBlobStorage( azure_conf2," + f" storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path ='{filename}'," + f"format='CSV', compression='{method}')", ) assert azure_query(node, f"SELECT sum(id) FROM {name}").splitlines() == ["565"] @@ -439,7 +484,9 @@ def test_schema_inference_no_globs(cluster): table_format = "column1 UInt32, column2 String, column3 UInt32" azure_query( node, - f"CREATE TABLE test_schema_inference_src ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs.csv', format='CSVWithNames')", + f"CREATE TABLE test_schema_inference_src ({table_format}) Engine = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='test_schema_inference_no_globs.csv', format='CSVWithNames')", ) query = f"insert into test_schema_inference_src SELECT number, toString(number), number * number FROM numbers(1000)" @@ -447,7 +494,8 @@ def test_schema_inference_no_globs(cluster): azure_query( node, - f"CREATE TABLE test_select_inference Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs.csv')", + f"CREATE TABLE test_select_inference Engine = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path='test_schema_inference_no_globs.csv')", ) print(node.query("SHOW CREATE TABLE test_select_inference")) @@ -474,7 +522,9 @@ def test_schema_inference_from_globs(cluster): azure_query( node, - f"CREATE TABLE test_schema_{i}_{j} ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSVWithNames')", + f"CREATE TABLE test_schema_{i}_{j} ({table_format}) Engine = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{path}', format='CSVWithNames')", ) query = f"insert into test_schema_{i}_{j} VALUES {values}" @@ -482,7 +532,8 @@ def test_schema_inference_from_globs(cluster): azure_query( node, - f"CREATE TABLE test_glob_select_inference Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv')", + f"CREATE TABLE test_glob_select_inference Engine = AzureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv')", ) print(node.query("SHOW CREATE TABLE test_glob_select_inference")) @@ -497,36 +548,47 @@ def test_schema_inference_from_globs(cluster): def test_simple_write_account_string_table_function(cluster): node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " + f"'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', " + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64, data String')" + f" VALUES (1, 'a')", ) - print(get_azure_file_content("test_simple_write_tf.csv")) - assert get_azure_file_content("test_simple_write_tf.csv") == '1,"a"\n' + print(get_azure_file_content("test_simple_write_tf.csv", port)) + assert get_azure_file_content("test_simple_write_tf.csv", port) == '1,"a"\n' def test_simple_write_connection_string_table_function(cluster): node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - "INSERT INTO TABLE FUNCTION azureBlobStorage('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;', 'cont', 'test_simple_write_connection_tf.csv', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', " + f"'cont', 'test_simple_write_connection_tf.csv', 'CSV', 'auto', 'key UInt64, data String') VALUES (1, 'a')", + ) + print(get_azure_file_content("test_simple_write_connection_tf.csv", port)) + assert ( + get_azure_file_content("test_simple_write_connection_tf.csv", port) == '1,"a"\n' ) - print(get_azure_file_content("test_simple_write_connection_tf.csv")) - assert get_azure_file_content("test_simple_write_connection_tf.csv") == '1,"a"\n' def test_simple_write_named_collection_1_table_function(cluster): node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - "INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf1) VALUES (1, 'a')", + f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf1, " + f"connection_string = '{cluster.env_variables['AZURITE_CONNECTION_STRING']}') VALUES (1, 'a')", ) - print(get_azure_file_content("test_simple_write_named.csv")) - assert get_azure_file_content("test_simple_write_named.csv") == '1,"a"\n' + print(get_azure_file_content("test_simple_write_named.csv", port)) + assert get_azure_file_content("test_simple_write_named.csv", port) == '1,"a"\n' azure_query( node, - "CREATE TABLE drop_table (key UInt64, data String) Engine = AzureBlobStorage(azure_conf1)", + f"CREATE TABLE drop_table (key UInt64, data String) Engine = AzureBlobStorage(azure_conf1, " + f"connection_string = '{cluster.env_variables['AZURITE_CONNECTION_STRING']};')", ) azure_query( @@ -537,13 +599,14 @@ def test_simple_write_named_collection_1_table_function(cluster): def test_simple_write_named_collection_2_table_function(cluster): node = cluster.instances["node"] - + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - "INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, container='cont', blob_path='test_simple_write_named_2_tf.csv', format='CSV', structure='key UInt64, data String') VALUES (1, 'a')", + f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" container='cont', blob_path='test_simple_write_named_2_tf.csv', format='CSV', structure='key UInt64, data String') VALUES (1, 'a')", ) - print(get_azure_file_content("test_simple_write_named_2_tf.csv")) - assert get_azure_file_content("test_simple_write_named_2_tf.csv") == '1,"a"\n' + print(get_azure_file_content("test_simple_write_named_2_tf.csv", port)) + assert get_azure_file_content("test_simple_write_named_2_tf.csv", port) == '1,"a"\n' def test_put_get_with_globs_tf(cluster): @@ -562,9 +625,14 @@ def test_put_get_with_globs_tf(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", + f"INSERT INTO TABLE FUNCTION azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}'," + f" container='cont', blob_path='{path}', format='CSV', compression='auto', structure='{table_format}') VALUES {values}", ) - query = f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv', format='CSV', structure='{table_format}')" + query = ( + f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv', format='CSV', structure='{table_format}')" + ) assert azure_query(node, query).splitlines() == [ "450\t450\t900\t0.csv\t{bucket}/{max_path}".format( bucket="cont", max_path=max_path @@ -576,10 +644,18 @@ def test_schema_inference_no_globs_tf(cluster): node = cluster.instances["node"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 String, column3 UInt32" - query = f"insert into table function azureBlobStorage(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs_tf.csv', format='CSVWithNames', structure='{table_format}') SELECT number, toString(number), number * number FROM numbers(1000)" + query = ( + f"insert into table function azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " + f"container='cont', blob_path='test_schema_inference_no_globs_tf.csv', format='CSVWithNames', structure='{table_format}') " + f"SELECT number, toString(number), number * number FROM numbers(1000)" + ) azure_query(node, query) - query = "select sum(column1), sum(length(column2)), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, container='cont', blob_path='test_schema_inference_no_globs_tf.csv')" + query = ( + f"select sum(column1), sum(length(column2)), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='test_schema_inference_no_globs_tf.csv')" + ) assert azure_query(node, query).splitlines() == [ "499500\t2890\t332833500\ttest_schema_inference_no_globs_tf.csv\tcont/test_schema_inference_no_globs_tf.csv" ] @@ -600,10 +676,17 @@ def test_schema_inference_from_globs_tf(cluster): max_path = max(path, max_path) values = f"({i},{j},{i + j})" - query = f"insert into table function azureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSVWithNames', structure='{table_format}') VALUES {values}" + query = ( + f"insert into table function azureBlobStorage(azure_conf2, storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " + f"container='cont', blob_path='{path}', format='CSVWithNames', structure='{table_format}') VALUES {values}" + ) azure_query(node, query) - query = f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, container='cont', blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv')" + query = ( + f"select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from azureBlobStorage(azure_conf2, " + f"storage_account_url = '{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', container='cont', " + f"blob_path='{unique_prefix}/*_{{a,b,c,d}}/?.csv')" + ) assert azure_query(node, query).splitlines() == [ "450\t450\t900\t0.csv\t{bucket}/{max_path}".format( bucket="cont", max_path=max_path @@ -617,15 +700,18 @@ def test_partition_by_tf(cluster): partition_by = "column3" values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)" filename = "test_partition_tf_{_partition_id}.csv" + port = cluster.env_variables["AZURITE_PORT"] azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values}", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " + f"'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', " + f"'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values}", ) - assert "1,2,3\n" == get_azure_file_content("test_partition_tf_3.csv") - assert "3,2,1\n" == get_azure_file_content("test_partition_tf_1.csv") - assert "78,43,45\n" == get_azure_file_content("test_partition_tf_45.csv") + assert "1,2,3\n" == get_azure_file_content("test_partition_tf_3.csv", port) + assert "3,2,1\n" == get_azure_file_content("test_partition_tf_1.csv", port) + assert "78,43,45\n" == get_azure_file_content("test_partition_tf_45.csv", port) def test_filter_using_file(cluster): @@ -637,45 +723,64 @@ def test_filter_using_file(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values}", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', 'cont', '{filename}', " + f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', " + f"'{table_format}') PARTITION BY {partition_by} VALUES {values}", ) - query = f"select count(*) from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_partition_tf_*.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') WHERE _file='test_partition_tf_3.csv'" + query = ( + f"select count(*) from azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', 'cont', 'test_partition_tf_*.csv', " + f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', " + f"'{table_format}') WHERE _file='test_partition_tf_3.csv'" + ) assert azure_query(node, query) == "1\n" def test_read_subcolumns(cluster): node = cluster.instances["node"] + storage_account_url = cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"] azure_query( node, - "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_subcolumns.tsv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.tsv', " + f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto'," + f" 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)", ) azure_query( node, - "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_subcolumns.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.jsonl', " + f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', " + f"'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)", ) res = node.query( - f"select a.b.d, _path, a.b, _file, a.e from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_subcolumns.tsv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + f"select a.b.d, _path, a.b, _file, a.e from azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.tsv'," + f" 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto'," + f" 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) assert res == "2\tcont/test_subcolumns.tsv\t(1,2)\ttest_subcolumns.tsv\t3\n" res = node.query( - f"select a.b.d, _path, a.b, _file, a.e from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_subcolumns.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + f"select a.b.d, _path, a.b, _file, a.e from azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.jsonl'," + f" 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', " + f"'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) assert res == "2\tcont/test_subcolumns.jsonl\t(1,2)\ttest_subcolumns.jsonl\t3\n" res = node.query( - f"select x.b.d, _path, x.b, _file, x.e from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_subcolumns.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + f"select x.b.d, _path, x.b, _file, x.e from azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.jsonl'," + f" 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', " + f"'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) assert res == "0\tcont/test_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" res = node.query( - f"select x.b.d, _path, x.b, _file, x.e from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_subcolumns.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" + f"select x.b.d, _path, x.b, _file, x.e from azureBlobStorage('{storage_account_url}', 'cont', 'test_subcolumns.jsonl'," + f" 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', " + f"'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" ) assert res == "42\tcont/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" @@ -683,15 +788,18 @@ def test_read_subcolumns(cluster): def test_read_from_not_existing_container(cluster): node = cluster.instances["node"] - query = f"select * from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont_not_exists', 'test_table.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto')" + query = ( + f"select * from azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', 'cont_not_exists', 'test_table.csv', " + f"'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto')" + ) expected_err_msg = "container does not exist" assert expected_err_msg in azure_query(node, query, expect_error="true") def test_function_signatures(cluster): node = cluster.instances["node"] - connection_string = "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;" - storage_account_url = "http://azurite1:10000/devstoreaccount1" + connection_string = cluster.env_variables["AZURITE_CONNECTION_STRING"] + storage_account_url = cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"] account_name = "devstoreaccount1" account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" azure_query( @@ -745,7 +853,8 @@ def check_profile_event_for_query(instance, file, profile_event, amount): query_pattern = f"azureBlobStorage%{file}".replace("'", "\\'") res = int( instance.query( - f"select ProfileEvents['{profile_event}'] from system.query_log where query like '%{query_pattern}%' and query not like '%ProfileEvents%' and type = 'QueryFinish' order by query_start_time_microseconds desc limit 1" + f"select ProfileEvents['{profile_event}'] from system.query_log where query like '%{query_pattern}%' and query not like '%ProfileEvents%' " + f"and type = 'QueryFinish' order by query_start_time_microseconds desc limit 1" ) ) @@ -804,15 +913,16 @@ def check_cache(instance, expected_files): def test_schema_inference_cache(cluster): node = cluster.instances["node"] - connection_string = "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;" - storage_account_url = "http://azurite1:10000/devstoreaccount1" + connection_string = cluster.env_variables["AZURITE_CONNECTION_STRING"] + storage_account_url = cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"] account_name = "devstoreaccount1" account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" node.query("system drop schema cache") azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache0.jsonl', '{account_name}', '{account_key}') select * from numbers(100)", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache0.jsonl', '{account_name}', '{account_key}') " + f"select * from numbers(100)", ) time.sleep(1) @@ -826,7 +936,8 @@ def test_schema_inference_cache(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache0.jsonl', '{account_name}', '{account_key}') select * from numbers(100) settings azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache0.jsonl', '{account_name}', '{account_key}') " + f"select * from numbers(100) settings azure_truncate_on_insert=1", ) time.sleep(1) @@ -836,7 +947,8 @@ def test_schema_inference_cache(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache1.jsonl', '{account_name}', '{account_key}') select * from numbers(100) settings azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache1.jsonl', '{account_name}', '{account_key}') " + f"select * from numbers(100) settings azure_truncate_on_insert=1", ) time.sleep(1) @@ -849,7 +961,8 @@ def test_schema_inference_cache(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache2.jsonl', '{account_name}', '{account_key}') select * from numbers(100) settings azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache2.jsonl', '{account_name}', '{account_key}') " + f"select * from numbers(100) settings azure_truncate_on_insert=1", ) time.sleep(1) @@ -895,7 +1008,8 @@ def test_schema_inference_cache(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache3.jsonl', '{account_name}', '{account_key}') select * from numbers(100) settings azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache3.jsonl', '{account_name}', '{account_key}') " + f"select * from numbers(100) settings azure_truncate_on_insert=1", ) time.sleep(1) @@ -919,7 +1033,8 @@ def test_schema_inference_cache(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache0.csv', '{account_name}', '{account_key}') select * from numbers(100) settings azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache0.csv', '{account_name}', '{account_key}') " + f"select * from numbers(100) settings azure_truncate_on_insert=1", ) time.sleep(1) @@ -943,7 +1058,8 @@ def test_schema_inference_cache(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache0.csv', '{account_name}', '{account_key}') select * from numbers(200) settings azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache0.csv', '{account_name}', '{account_key}') " + f"select * from numbers(200) settings azure_truncate_on_insert=1", ) time.sleep(1) @@ -958,7 +1074,8 @@ def test_schema_inference_cache(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache1.csv', '{account_name}', '{account_key}') select * from numbers(100) settings azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache1.csv', '{account_name}', '{account_key}') " + f"select * from numbers(100) settings azure_truncate_on_insert=1", ) time.sleep(1) @@ -991,7 +1108,8 @@ def test_schema_inference_cache(cluster): azure_query( node, - f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache.parquet', '{account_name}', '{account_key}') select * from numbers(100) settings azure_truncate_on_insert=1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_cache.parquet', '{account_name}', '{account_key}') " + f"select * from numbers(100) settings azure_truncate_on_insert=1", ) time.sleep(1) @@ -1007,23 +1125,29 @@ def test_schema_inference_cache(cluster): def test_filtering_by_file_or_path(cluster): node = cluster.instances["node"] + storage_account_url = cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"] azure_query( node, - "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_filter1.tsv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 1", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}','cont', 'test_filter1.tsv', 'devstoreaccount1', " + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 1", ) azure_query( node, - "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_filter2.tsv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 2", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}','cont', 'test_filter2.tsv', 'devstoreaccount1', " + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 2", ) azure_query( node, - "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_filter3.tsv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 3", + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_filter3.tsv', 'devstoreaccount1', " + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') select 3", ) node.query( - f"select count() from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_filter*.tsv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') where _file = 'test_filter1.tsv'" + f"select count() from azureBlobStorage('{storage_account_url}', 'cont', 'test_filter*.tsv', 'devstoreaccount1', " + f"'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x UInt64') " + f"where _file = 'test_filter1.tsv'" ) node.query("SYSTEM FLUSH LOGS") From 940449dd59ea79229d9322e922372bc204aa5d28 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 2 Nov 2023 15:09:53 +0000 Subject: [PATCH 025/331] Try to fix azure test --- .../test_storage_azure_blob_storage/test.py | 118 +++++++++--------- 1 file changed, 60 insertions(+), 58 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index a90879c3a00..3b4a5bf571b 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -49,7 +49,6 @@ def azure_query( "DB::Exception: Azure::Core::Http::TransportException: Connection closed before getting full response or response is less than expected", "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response", "DB::Exception: Azure::Core::Http::TransportException: Error while polling for socket ready read", - "DB::Exception: Azure::Core::Http::TransportException: Fail to get a new connection" "Azure::Core::Http::TransportException, e.what() = Connection was closed by the server while trying to read a response", "Azure::Core::Http::TransportException, e.what() = Connection closed before getting full response or response is less than expected", "Azure::Core::Http::TransportException, e.what() = Connection was closed by the server while trying to read a response", @@ -911,6 +910,66 @@ def check_cache(instance, expected_files): ) +def test_union_schema_inference_mode(cluster): + node = cluster.instances["node"] + storage_account_url = cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"] + account_name = "devstoreaccount1" + account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference1.jsonl', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'a UInt32') VALUES (1)", + ) + + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference2.jsonl', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'b UInt32') VALUES (2)", + ) + + node.query("system drop schema cache for azure") + + result = azure_query( + node, + f"desc azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference*.jsonl', '{account_name}', '{account_key}', 'auto', 'auto', 'auto') settings schema_inference_mode='union', describe_compact_output=1 format TSV", + ) + assert result == "a\tNullable(Int64)\nb\tNullable(Int64)\n" + + result = node.query( + "select schema_inference_mode, splitByChar('/', source)[-1] as file, schema from system.schema_inference_cache where source like '%test_union_schema_inference%' order by file format TSV" + ) + assert ( + result == "UNION\ttest_union_schema_inference1.jsonl\ta Nullable(Int64)\n" + "UNION\ttest_union_schema_inference2.jsonl\tb Nullable(Int64)\n" + ) + result = azure_query( + node, + f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference*.jsonl', '{account_name}', '{account_key}', 'auto', 'auto', 'auto') settings schema_inference_mode='union' format TSV", + ) + assert result == "1\t\\N\n" "\\N\t2\n" + node.query(f"system drop schema cache for hdfs") + result = azure_query( + node, + f"desc azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference2.jsonl', '{account_name}', '{account_key}', 'auto', 'auto', 'auto') settings schema_inference_mode='union', describe_compact_output=1 format TSV", + ) + assert result == "b\tNullable(Int64)\n" + + result = azure_query( + node, + f"desc azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference*.jsonl', '{account_name}', '{account_key}', 'auto', 'auto', 'auto') settings schema_inference_mode='union', describe_compact_output=1 format TSV", + ) + assert result == "a\tNullable(Int64)\n" "b\tNullable(Int64)\n" + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference3.jsonl', '{account_name}', '{account_key}', 'CSV', 'auto', 's String') VALUES ('Error')", + ) + + error = azure_query( + node, + f"desc azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference*.jsonl', '{account_name}', '{account_key}', 'auto', 'auto', 'auto') settings schema_inference_mode='union', describe_compact_output=1 format TSV", + expect_error="true", + ) + assert "Cannot extract table structure" in error + + def test_schema_inference_cache(cluster): node = cluster.instances["node"] connection_string = cluster.env_variables["AZURITE_CONNECTION_STRING"] @@ -1157,60 +1216,3 @@ def test_filtering_by_file_or_path(cluster): ) assert int(result) == 1 - - -def test_union_schema_inference_mode(cluster): - node = cluster.instances["node"] - azure_query( - node, - "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference1.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') select 1 as a", - ) - - azure_query( - node, - "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference2.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') select 2 as b", - ) - - node.query("system drop schema cache for azure") - - result = azure_query( - node, - "desc azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference*.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') settings schema_inference_mode='union', describe_compact_output=1 format TSV", - ) - assert result == "a\tNullable(Int64)\nb\tNullable(Int64)\n" - - result = node.query( - "select schema_inference_mode, splitByChar('/', source)[-1] as file, schema from system.schema_inference_cache where source like '%test_union_schema_inference%' order by file format TSV" - ) - assert ( - result == "UNION\ttest_union_schema_inference1.jsonl\ta Nullable(Int64)\n" - "UNION\ttest_union_schema_inference2.jsonl\tb Nullable(Int64)\n" - ) - result = azure_query( - node, - "select * from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference*.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') order by tuple(*) settings schema_inference_mode='union', describe_compact_output=1 format TSV", - ) - assert result == "1\t\\N\n" "\\N\t2\n" - node.query(f"system drop schema cache for hdfs") - result = azure_query( - node, - "desc azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference2.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') settings schema_inference_mode='union', describe_compact_output=1 format TSV", - ) - assert result == "b\tNullable(Int64)\n" - - result = azure_query( - node, - "desc azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference*.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') settings schema_inference_mode='union', describe_compact_output=1 format TSV", - ) - assert result == "a\tNullable(Int64)\n" "b\tNullable(Int64)\n" - azure_query( - node, - "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference3.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', TSV) select 'Error'", - ) - - error = azure_query( - node, - "desc azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_union_schema_inference*.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') settings schema_inference_mode='union', describe_compact_output=1 format TSV", - expect_error="true", - ) - assert "Cannot extract table structure" in error From 6345d94d2bde4de879e27b29920f680f5314db65 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 3 Nov 2023 16:27:04 +0000 Subject: [PATCH 026/331] Fix test --- tests/integration/test_storage_azure_blob_storage/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 3b4a5bf571b..6c72dece5d8 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -942,7 +942,7 @@ def test_union_schema_inference_mode(cluster): ) result = azure_query( node, - f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference*.jsonl', '{account_name}', '{account_key}', 'auto', 'auto', 'auto') settings schema_inference_mode='union' format TSV", + f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_union_schema_inference*.jsonl', '{account_name}', '{account_key}', 'auto', 'auto', 'auto') order by tuple(*) settings schema_inference_mode='union' format TSV", ) assert result == "1\t\\N\n" "\\N\t2\n" node.query(f"system drop schema cache for hdfs") From 081fa9f3def0bdcc680581ba4b8e5f96ccc2d828 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 20 Nov 2023 15:53:28 +0000 Subject: [PATCH 027/331] Address comments --- src/Core/SettingsChangesHistory.h | 2 +- src/Formats/ReadSchemaUtils.cpp | 9 ++++++++- src/Formats/ReadSchemaUtils.h | 17 ++++++++++++----- src/Formats/SchemaInferenceUtils.cpp | 2 +- 4 files changed, 22 insertions(+), 8 deletions(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index caa1b28b1c5..c55c0ece15e 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -80,7 +80,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.10", {{"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, + {"23.11", {{"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}}, {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index c5630267e3f..43931be3449 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -55,7 +55,14 @@ try NamesAndTypesList names_and_types; SchemaInferenceMode mode = context->getSettingsRef().schema_inference_mode; if (mode == SchemaInferenceMode::UNION && !FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name, context, format_settings)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "UNION schema inference mode is not supported for format {}, because it doesn't support reading subset of columns", format_name); + { + String additional_message; + /// Better exception message for WithNames(AndTypes) formats. + if (format_name.ends_with("WithNames") || format_name.ends_with("WithNamesAndTypes")) + additional_message = " (formats -WithNames(AndTypes) support reading subset of columns only when setting input_format_with_names_use_header is enabled)"; + + throw Exception(ErrorCodes::BAD_ARGUMENTS, "UNION schema inference mode is not supported for format {}, because it doesn't support reading subset of columns{}", format_name, additional_message); + } if (FormatFactory::instance().checkIfFormatHasExternalSchemaReader(format_name)) { diff --git a/src/Formats/ReadSchemaUtils.h b/src/Formats/ReadSchemaUtils.h index aade6b28fb0..6aa8f3f9c4c 100644 --- a/src/Formats/ReadSchemaUtils.h +++ b/src/Formats/ReadSchemaUtils.h @@ -57,11 +57,18 @@ private: /// use it and won't create a read buffer. /// For formats that have a schema reader from the data, /// read buffer will be created by the provided iterator and -/// the schema will be extracted from the data. If schema reader -/// couldn't determine the schema we will try the next read buffer -/// from the provided iterator if it makes sense. If the format doesn't -/// have any schema reader or we couldn't determine the schema, -/// an exception will be thrown. +/// the schema will be extracted from the data. If the format doesn't +/// have any schema reader an exception will be thrown. +/// Reading schema can be performed in 2 modes depending on setting schema_inference_mode: +/// 1) Default mode. In this mode ClickHouse assumes that all files have the same schema +/// and tries to infer the schema by reading files one by one until it succeeds. +/// If schema reader couldn't determine the schema for some file, ClickHouse will try the next +/// file (next read buffer from the provided iterator) if it makes sense. If ClickHouse couldn't determine +/// the resulting schema, an exception will be thrown. +/// 2) Union mode. In this mode ClickHouse assumes that files can have different schemas, +/// so it infer schemas of all files and then union them to the common schema. In this mode +/// all read buffers from provided iterator will be used. If ClickHouse couldn't determine +/// the schema for some file, an exception will be thrown. ColumnsDescription readSchemaFromFormat( const String & format_name, const std::optional & format_settings, diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 13871904c56..229f033f65a 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -577,7 +577,7 @@ namespace element_types.reserve(names_to_types.size()); for (const auto & name : element_names) { - auto types = names_to_types[name]; + auto & types = names_to_types[name]; transformInferredTypesIfNeededImpl(types, settings, json_info); /// If some element have different types in different tuples, we can't do anything if (!checkIfTypesAreEqual(types)) From f77c770dc9ffa9e54356bcdece54cb5c9d2f2985 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 20 Nov 2023 15:55:11 +0000 Subject: [PATCH 028/331] Don't run test in parallel --- .../queries/0_stateless/02375_system_schema_inference_cache.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02375_system_schema_inference_cache.sql b/tests/queries/0_stateless/02375_system_schema_inference_cache.sql index 6f656b16c69..310e22ed31f 100644 --- a/tests/queries/0_stateless/02375_system_schema_inference_cache.sql +++ b/tests/queries/0_stateless/02375_system_schema_inference_cache.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest +-- Tags: no-fasttest, no-parallel set input_format_json_try_infer_numbers_from_strings=1; insert into function file('02374_data1.jsonl') select number as x, 'str' as s from numbers(10); From 68c72d7e65f2516c74693624d2826f9680d4d473 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 20 Nov 2023 16:47:52 +0000 Subject: [PATCH 029/331] Fix possible deadlock in Template format during sync after error --- src/IO/ReadHelpers.cpp | 2 +- .../02918_template_format_deadlock.reference | 1 + .../02918_template_format_deadlock.sh | 19 +++++++++++++++++++ 3 files changed, 21 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02918_template_format_deadlock.reference create mode 100755 tests/queries/0_stateless/02918_template_format_deadlock.sh diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index 19750906fdb..ff5743a63af 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -1591,7 +1591,7 @@ void skipToNextRowOrEof(PeekableReadBuffer & buf, const String & row_after_delim if (skip_spaces) skipWhitespaceIfAny(buf); - if (checkString(row_between_delimiter, buf)) + if (buf.eof() || checkString(row_between_delimiter, buf)) break; } } diff --git a/tests/queries/0_stateless/02918_template_format_deadlock.reference b/tests/queries/0_stateless/02918_template_format_deadlock.reference new file mode 100644 index 00000000000..83f5da32cc7 --- /dev/null +++ b/tests/queries/0_stateless/02918_template_format_deadlock.reference @@ -0,0 +1 @@ +42 43 diff --git a/tests/queries/0_stateless/02918_template_format_deadlock.sh b/tests/queries/0_stateless/02918_template_format_deadlock.sh new file mode 100755 index 00000000000..344a8b55b0d --- /dev/null +++ b/tests/queries/0_stateless/02918_template_format_deadlock.sh @@ -0,0 +1,19 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +DATA_FILE=$CLICKHOUSE_TEST_UNIQUE_NAME +TEMPLATE_FILE=$CLICKHOUSE_TEST_UNIQUE_NAME.template + +echo "42 | 43 +Error line" > $DATA_FILE +echo '${a:CSV} | ${b:CSV}' > $TEMPLATE_FILE + +$CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', Template, 'a UInt32, b UInt32') settings format_template_row='$TEMPLATE_FILE', input_format_allow_errors_num=1" + +rm $DATA_FILE +rm $TEMPLATE_FILE + From ffa90628f004449fccda70a07897f467ffbd8658 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 20 Nov 2023 17:22:38 +0000 Subject: [PATCH 030/331] Make input format errors logger a bit better --- src/Formats/FormatFactory.cpp | 8 +++- .../Formats/InputFormatErrorsLogger.cpp | 45 +++++++++++++------ .../Formats/InputFormatErrorsLogger.h | 4 ++ 3 files changed, 42 insertions(+), 15 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 7fb355b6c43..c50fa1bcebd 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -346,7 +346,13 @@ InputFormatPtr FormatFactory::getInput( if (owned_buf) format->addBuffer(std::move(owned_buf)); if (!settings.input_format_record_errors_file_path.toString().empty()) - format->setErrorsLogger(std::make_shared(context)); + { + if (parallel_parsing) + format->setErrorsLogger(std::make_shared(context)); + else + format->setErrorsLogger(std::make_shared(context)); + } + /// It's a kludge. Because I cannot remove context from values format. /// (Not needed in the parallel_parsing case above because VALUES format doesn't support it.) diff --git a/src/Processors/Formats/InputFormatErrorsLogger.cpp b/src/Processors/Formats/InputFormatErrorsLogger.cpp index 71d51f0e04a..814c4679cf9 100644 --- a/src/Processors/Formats/InputFormatErrorsLogger.cpp +++ b/src/Processors/Formats/InputFormatErrorsLogger.cpp @@ -20,7 +20,7 @@ namespace const String DEFAULT_OUTPUT_FORMAT = "CSV"; } -InputFormatErrorsLogger::InputFormatErrorsLogger(const ContextPtr & context) +InputFormatErrorsLogger::InputFormatErrorsLogger(const ContextPtr & context) : max_block_size(context->getSettingsRef().max_block_size) { String output_format = context->getSettingsRef().errors_output_format; if (!FormatFactory::instance().isOutputFormat(output_format)) @@ -59,30 +59,47 @@ InputFormatErrorsLogger::InputFormatErrorsLogger(const ContextPtr & context) {std::make_shared(), "offset"}, {std::make_shared(), "reason"}, {std::make_shared(), "raw_data"}}; + errors_columns = header.cloneEmptyColumns(); writer = context->getOutputFormat(output_format, *write_buf, header); } + InputFormatErrorsLogger::~InputFormatErrorsLogger() { - writer->finalize(); - writer->flush(); - write_buf->finalize(); + try + { + if (!errors_columns[0]->empty()) + writeErrors(); + writer->finalize(); + writer->flush(); + write_buf->finalize(); + } + catch (...) + { + tryLogCurrentException("InputFormatErrorsLogger"); + } } void InputFormatErrorsLogger::logErrorImpl(ErrorEntry entry) { - auto error = header.cloneEmpty(); - auto columns = error.mutateColumns(); - columns[0]->insert(entry.time); - database.empty() ? columns[1]->insertDefault() : columns[1]->insert(database); - table.empty() ? columns[2]->insertDefault() : columns[2]->insert(table); - columns[3]->insert(entry.offset); - columns[4]->insert(entry.reason); - columns[5]->insert(entry.raw_data); - error.setColumns(std::move(columns)); + errors_columns[0]->insert(entry.time); + database.empty() ? errors_columns[1]->insertDefault() : errors_columns[1]->insert(database); + table.empty() ? errors_columns[2]->insertDefault() : errors_columns[2]->insert(table); + errors_columns[3]->insert(entry.offset); + errors_columns[4]->insert(entry.reason); + errors_columns[5]->insert(entry.raw_data); - writer->write(error); + if (errors_columns[0]->size() >= max_block_size) + writeErrors(); +} + +void InputFormatErrorsLogger::writeErrors() +{ + auto block = header.cloneEmpty(); + block.setColumns(std::move(errors_columns)); + writer->write(block); + errors_columns = header.cloneEmptyColumns(); } void InputFormatErrorsLogger::logError(ErrorEntry entry) diff --git a/src/Processors/Formats/InputFormatErrorsLogger.h b/src/Processors/Formats/InputFormatErrorsLogger.h index 4b3766f4d37..b30246ff705 100644 --- a/src/Processors/Formats/InputFormatErrorsLogger.h +++ b/src/Processors/Formats/InputFormatErrorsLogger.h @@ -24,6 +24,7 @@ public: virtual void logError(ErrorEntry entry); void logErrorImpl(ErrorEntry entry); + void writeErrors(); private: Block header; @@ -34,6 +35,9 @@ private: String database; String table; + + MutableColumns errors_columns; + size_t max_block_size; }; using InputFormatErrorsLoggerPtr = std::shared_ptr; From 6d3de41dc2a7545f93331c5f7721c428dfbea4d8 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 20 Nov 2023 18:07:54 +0000 Subject: [PATCH 031/331] Fix skipping lots rows with errors --- src/Processors/Formats/IRowInputFormat.cpp | 2 +- .../Impl/ParallelParsingInputFormat.cpp | 83 ++++++++++--------- ...2919_skip_lots_of_parsing_errors.reference | 4 + .../02919_skip_lots_of_parsing_errors.sh | 23 +++++ 4 files changed, 71 insertions(+), 41 deletions(-) create mode 100644 tests/queries/0_stateless/02919_skip_lots_of_parsing_errors.reference create mode 100755 tests/queries/0_stateless/02919_skip_lots_of_parsing_errors.sh diff --git a/src/Processors/Formats/IRowInputFormat.cpp b/src/Processors/Formats/IRowInputFormat.cpp index 87febb6d46b..8c563b6f13b 100644 --- a/src/Processors/Formats/IRowInputFormat.cpp +++ b/src/Processors/Formats/IRowInputFormat.cpp @@ -128,7 +128,7 @@ Chunk IRowInputFormat::generate() RowReadExtension info; bool continue_reading = true; - for (size_t rows = 0; rows < params.max_block_size && continue_reading; ++rows) + for (size_t rows = 0; (rows < params.max_block_size || num_rows == 0) && continue_reading; ++rows) { try { diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp index 06d5c80281f..8d0f7ef984a 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp @@ -125,10 +125,6 @@ void ParallelParsingInputFormat::parserThreadFunction(ThreadGroupPtr thread_grou first_parser_finished.set(); } - // We suppose we will get at least some blocks for a non-empty buffer, - // except at the end of file. Also see a matching assert in readImpl(). - assert(unit.is_last || !unit.chunk_ext.chunk.empty() || parsing_finished); - std::lock_guard lock(mutex); unit.status = READY_TO_READ; reader_condvar.notify_all(); @@ -199,62 +195,69 @@ Chunk ParallelParsingInputFormat::generate() } const auto inserter_unit_number = reader_ticket_number % processing_units.size(); - auto & unit = processing_units[inserter_unit_number]; + auto * unit = &processing_units[inserter_unit_number]; if (!next_block_in_current_unit.has_value()) { - // We have read out all the Blocks from the previous Processing Unit, - // wait for the current one to become ready. - std::unique_lock lock(mutex); - reader_condvar.wait(lock, [&](){ return unit.status == READY_TO_READ || parsing_finished; }); - - if (parsing_finished) + while (true) { - /** - * Check for background exception and rethrow it before we return. - */ - if (background_exception) + // We have read out all the Blocks from the previous Processing Unit, + // wait for the current one to become ready. + std::unique_lock lock(mutex); + reader_condvar.wait(lock, [&]() { return unit->status == READY_TO_READ || parsing_finished; }); + + if (parsing_finished) { - lock.unlock(); - cancel(); - std::rethrow_exception(background_exception); + /// Check for background exception and rethrow it before we return. + if (background_exception) + { + lock.unlock(); + cancel(); + std::rethrow_exception(background_exception); + } + + return {}; } - return {}; + assert(unit->status == READY_TO_READ); + + if (!unit->chunk_ext.chunk.empty()) + break; + + /// If this uint is last, parsing is finished. + if (unit->is_last) + { + parsing_finished = true; + return {}; + } + + /// We can get zero blocks for an entire segment if format parser + /// skipped all rows. For example, it can happen while using settings + /// input_format_allow_errors_num/input_format_allow_errors_ratio + /// and this segment contained only rows with errors. + /// Process the next unit. + ++reader_ticket_number; + unit = &processing_units[reader_ticket_number % processing_units.size()]; } - assert(unit.status == READY_TO_READ); next_block_in_current_unit = 0; } - if (unit.chunk_ext.chunk.empty()) - { - /* - * Can we get zero blocks for an entire segment, when the format parser - * skips it entire content and does not create any blocks? Probably not, - * but if we ever do, we should add a loop around the above if, to skip - * these. Also see a matching assert in the parser thread. - */ - assert(unit.is_last); - parsing_finished = true; - return {}; - } + assert(next_block_in_current_unit.value() < unit->chunk_ext.chunk.size()); - assert(next_block_in_current_unit.value() < unit.chunk_ext.chunk.size()); - - Chunk res = std::move(unit.chunk_ext.chunk.at(*next_block_in_current_unit)); - last_block_missing_values = std::move(unit.chunk_ext.block_missing_values[*next_block_in_current_unit]); - last_approx_bytes_read_for_chunk = unit.chunk_ext.approx_chunk_sizes.at(*next_block_in_current_unit); + Chunk res = std::move(unit->chunk_ext.chunk.at(*next_block_in_current_unit)); + last_block_missing_values = std::move(unit->chunk_ext.block_missing_values[*next_block_in_current_unit]); + last_approx_bytes_read_for_chunk = unit->chunk_ext.approx_chunk_sizes.at(*next_block_in_current_unit); next_block_in_current_unit.value() += 1; - if (*next_block_in_current_unit == unit.chunk_ext.chunk.size()) + if (*next_block_in_current_unit == unit->chunk_ext.chunk.size()) { // parsing_finished reading this Processing Unit, move to the next one. next_block_in_current_unit.reset(); ++reader_ticket_number; - if (unit.is_last) + if (unit->is_last) { // It it was the last unit, we're parsing_finished. parsing_finished = true; @@ -263,7 +266,7 @@ Chunk ParallelParsingInputFormat::generate() { // Pass the unit back to the segmentator. std::lock_guard lock(mutex); - unit.status = READY_TO_INSERT; + unit->status = READY_TO_INSERT; segmentator_condvar.notify_all(); } } diff --git a/tests/queries/0_stateless/02919_skip_lots_of_parsing_errors.reference b/tests/queries/0_stateless/02919_skip_lots_of_parsing_errors.reference new file mode 100644 index 00000000000..4b4c9812f09 --- /dev/null +++ b/tests/queries/0_stateless/02919_skip_lots_of_parsing_errors.reference @@ -0,0 +1,4 @@ +42 +100000 +42 +100000 diff --git a/tests/queries/0_stateless/02919_skip_lots_of_parsing_errors.sh b/tests/queries/0_stateless/02919_skip_lots_of_parsing_errors.sh new file mode 100755 index 00000000000..2c54e9e68da --- /dev/null +++ b/tests/queries/0_stateless/02919_skip_lots_of_parsing_errors.sh @@ -0,0 +1,23 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +FILE=$CLICKHOUSE_TEST_UNIQUE_NAME +ERRORS_FILE=$CLICKHOUSE_TEST_UNIQUE_NAME.errors + +$CLICKHOUSE_LOCAL -q "select 'Error' from numbers(100000) format TSVRaw" > $FILE +echo -e "42" >> $FILE + +$CLICKHOUSE_LOCAL -q "select * from file('$FILE', CSV, 'x UInt32') settings input_format_allow_errors_ratio=1, max_block_size=10000, input_format_parallel_parsing=0, input_format_record_errors_file_path='$ERRORS_FILE'"; +$CLICKHOUSE_LOCAL -q "select count() from file('$ERRORS_FILE', CSV)" +rm $ERRORS_FILE + +$CLICKHOUSE_LOCAL -q "select * from file('$FILE', CSV, 'x UInt32') settings input_format_allow_errors_ratio=1, max_block_size=10000, input_format_parallel_parsing=1, input_format_record_errors_file_path='$ERRORS_FILE'"; +$CLICKHOUSE_LOCAL -q "select count() from file('$ERRORS_FILE', CSV)" +rm $ERRORS_FILE + +rm $FILE + From 42e16bcc49dad255acfd1d3dce14e6a0e854b55b Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 21 Nov 2023 13:32:39 +0100 Subject: [PATCH 032/331] SLRU for filesystem cache --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 4 +- src/Interpreters/Cache/FileCache.cpp | 214 +++------------ src/Interpreters/Cache/FileCache.h | 35 ++- src/Interpreters/Cache/FileCacheSettings.cpp | 6 + src/Interpreters/Cache/FileCacheSettings.h | 3 + src/Interpreters/Cache/FileSegment.cpp | 4 +- src/Interpreters/Cache/FileSegment.h | 2 +- src/Interpreters/Cache/IFileCachePriority.cpp | 104 ++++++++ src/Interpreters/Cache/IFileCachePriority.h | 87 +++--- .../Cache/LRUFileCachePriority.cpp | 137 ++++++++-- src/Interpreters/Cache/LRUFileCachePriority.h | 45 ++-- .../Cache/SLRUFileCachePriority.cpp | 252 ++++++++++++++++++ .../Cache/SLRUFileCachePriority.h | 88 ++++++ 13 files changed, 714 insertions(+), 267 deletions(-) create mode 100644 src/Interpreters/Cache/IFileCachePriority.cpp create mode 100644 src/Interpreters/Cache/SLRUFileCachePriority.cpp create mode 100644 src/Interpreters/Cache/SLRUFileCachePriority.h diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 1cfdd96b271..16e2233d596 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -516,7 +516,7 @@ bool CachedOnDiskReadBufferFromFile::completeFileSegmentAndGetNext() return false; current_file_segment = &file_segments->front(); - current_file_segment->use(); + current_file_segment->increasePriority(); implementation_buffer = getImplementationBuffer(*current_file_segment); LOG_TEST( @@ -842,7 +842,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() else { implementation_buffer = getImplementationBuffer(file_segments->front()); - file_segments->front().use(); + file_segments->front().increasePriority(); } chassert(!internal_buffer.empty()); diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index e4d4802951a..8f75fb6f0b9 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -62,7 +63,12 @@ FileCache::FileCache(const std::string & cache_name, const FileCacheSettings & s , log(&Poco::Logger::get("FileCache(" + cache_name + ")")) , metadata(settings.base_path) { - main_priority = std::make_unique(settings.max_size, settings.max_elements); + if (settings.cache_policy == "LRU") + main_priority = std::make_unique(settings.max_size, settings.max_elements); + else if (settings.cache_policy == "SLRU") + main_priority = std::make_unique(settings.max_size, settings.max_elements, settings.slru_size_ratio); + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown cache policy: {}", settings.cache_policy); if (settings.cache_hits_threshold) stash = std::make_unique(settings.cache_hits_threshold, settings.max_elements); @@ -511,7 +517,7 @@ KeyMetadata::iterator FileCache::addFileSegment( } else { - result_state = record_it->second->use(*lock) >= stash->hits_threshold + result_state = record_it->second->increasePriority(*lock) >= stash->hits_threshold ? FileSegment::State::EMPTY : FileSegment::State::DETACHED; } @@ -572,171 +578,18 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa file_segment.key(), file_segment.offset()); } - struct EvictionCandidates - { - explicit EvictionCandidates(KeyMetadataPtr key_metadata_) : key_metadata(std::move(key_metadata_)) {} - - void add(const FileSegmentMetadataPtr & candidate) - { - candidate->removal_candidate = true; - candidates.push_back(candidate); - } - - ~EvictionCandidates() - { - /// If failed to reserve space, we don't delete the candidates but drop the flag instead - /// so the segments can be used again - for (const auto & candidate : candidates) - candidate->removal_candidate = false; - } - - KeyMetadataPtr key_metadata; - std::vector candidates; - }; - - std::unordered_map to_delete; - size_t freeable_space = 0, freeable_count = 0; - - auto iterate_func = [&](LockedKey & locked_key, const FileSegmentMetadataPtr & segment_metadata) - { - chassert(segment_metadata->file_segment->assertCorrectness()); - - auto & stat_by_kind = reserve_stat.stat_by_kind[segment_metadata->file_segment->getKind()]; - if (segment_metadata->releasable()) - { - const auto & key = segment_metadata->file_segment->key(); - auto it = to_delete.find(key); - if (it == to_delete.end()) - it = to_delete.emplace(key, locked_key.getKeyMetadata()).first; - it->second.add(segment_metadata); - - stat_by_kind.releasable_size += segment_metadata->size(); - ++stat_by_kind.releasable_count; - - freeable_space += segment_metadata->size(); - ++freeable_count; - } - else - { - stat_by_kind.non_releasable_size += segment_metadata->size(); - ++stat_by_kind.non_releasable_count; - - ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictionSkippedFileSegments); - } - - return PriorityIterationResult::CONTINUE; - }; + IFileCachePriority::EvictionCandidates eviction_candidates; + IFileCachePriority::FinalizeEvictionFunc finalize_eviction_func; if (query_priority) { - auto is_query_priority_overflow = [&] - { - const size_t new_size = query_priority->getSize(cache_lock) + size - freeable_space; - return new_size > query_priority->getSizeLimit(); - }; - - if (is_query_priority_overflow()) - { - ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictionTries); - - query_priority->iterate( - [&](LockedKey & locked_key, const FileSegmentMetadataPtr & segment_metadata) - { return is_query_priority_overflow() ? iterate_func(locked_key, segment_metadata) : PriorityIterationResult::BREAK; }, - cache_lock); - - if (is_query_priority_overflow()) - return false; - } - - LOG_TEST( - log, "Query limits satisfied (while reserving for {}:{})", - file_segment.key(), file_segment.offset()); - } - - auto is_main_priority_overflow = [main_priority_size_limit = main_priority->getSizeLimit(), - main_priority_elements_limit = main_priority->getElementsLimit(), - size, - &freeable_space, - &freeable_count, - &file_segment, - &cache_lock, - my_main_priority = this->main_priority.get(), - my_log = this->log] - { - const bool is_overflow = - /// size_limit == 0 means unlimited cache size - (main_priority_size_limit != 0 && (my_main_priority->getSize(cache_lock) + size - freeable_space > main_priority_size_limit)) - /// elements_limit == 0 means unlimited number of cache elements - || (main_priority_elements_limit != 0 && freeable_count == 0 - && my_main_priority->getElementsCount(cache_lock) == main_priority_elements_limit); - - LOG_TEST( - my_log, "Overflow: {}, size: {}, ready to remove: {} ({} in number), current cache size: {}/{}, elements: {}/{}, while reserving for {}:{}", - is_overflow, size, freeable_space, freeable_count, - my_main_priority->getSize(cache_lock), my_main_priority->getSizeLimit(), - my_main_priority->getElementsCount(cache_lock), my_main_priority->getElementsLimit(), - file_segment.key(), file_segment.offset()); - - return is_overflow; - }; - - /// If we have enough space in query_priority, we are not interested about stat there anymore. - /// Clean the stat before iterating main_priority to avoid calculating any segment stat twice. - reserve_stat.stat_by_kind.clear(); - - if (is_main_priority_overflow()) - { - ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictionTries); - - main_priority->iterate( - [&](LockedKey & locked_key, const FileSegmentMetadataPtr & segment_metadata) - { return is_main_priority_overflow() ? iterate_func(locked_key, segment_metadata) : PriorityIterationResult::BREAK; }, - cache_lock); - - if (is_main_priority_overflow()) + if (!query_priority->collectCandidatesForEviction(size, reserve_stat, eviction_candidates, {}, finalize_eviction_func, cache_lock)) return false; - } - if (!file_segment.getKeyMetadata()->createBaseDirectory()) - return false; - - if (!to_delete.empty()) - { - LOG_DEBUG( - log, "Will evict {} file segments (while reserving {} bytes for {}:{})", - to_delete.size(), size, file_segment.key(), file_segment.offset()); - - ProfileEventTimeIncrement evict_watch(ProfileEvents::FilesystemCacheEvictMicroseconds); - - for (auto & [current_key, deletion_info] : to_delete) - { - auto locked_key = deletion_info.key_metadata->tryLock(); - if (!locked_key) - continue; /// key could become invalid after we released the key lock above, just skip it. - - /// delete from vector in reverse order just for efficiency - auto & candidates = deletion_info.candidates; - while (!candidates.empty()) - { - auto & candidate = candidates.back(); - chassert(candidate->releasable()); - - const auto * segment = candidate->file_segment.get(); - auto queue_it = segment->getQueueIterator(); - chassert(queue_it); - - ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedFileSegments); - ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedBytes, segment->range().size()); - - locked_key->removeFileSegment(segment->offset(), segment->lock()); - queue_it->remove(cache_lock); - - if (query_context) - query_context->remove(current_key, segment->offset(), cache_lock); - - candidates.pop_back(); - } - } + LOG_TEST(log, "Query limits satisfied (while reserving for {}:{})", file_segment.key(), file_segment.offset()); + /// If we have enough space in query_priority, we are not interested about stat there anymore. + /// Clean the stat before iterating main_priority to avoid calculating any segment stat twice. + reserve_stat.stat_by_kind.clear(); } /// A file_segment_metadata acquires a LRUQueue iterator on first successful space reservation attempt, @@ -744,6 +597,17 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa auto queue_iterator = file_segment.getQueueIterator(); chassert(!queue_iterator || file_segment.getReservedSize() > 0); + if (!main_priority->collectCandidatesForEviction(size, reserve_stat, eviction_candidates, queue_iterator, finalize_eviction_func, cache_lock)) + return false; + + if (!file_segment.getKeyMetadata()->createBaseDirectory()) + return false; + + eviction_candidates.evict(cache_lock); + + if (finalize_eviction_func) + finalize_eviction_func(); + if (queue_iterator) { queue_iterator->updateSize(size); @@ -812,8 +676,7 @@ void FileCache::removeAllReleasable() { /// Remove all access information. auto lock = lockCache(); - stash->records.clear(); - stash->queue->removeAll(lock); + stash->clear(); } } @@ -1115,15 +978,7 @@ FileSegments FileCache::getSnapshot(const Key & key) FileSegments FileCache::dumpQueue() { assertInitialized(); - - FileSegments file_segments; - main_priority->iterate([&](LockedKey &, const FileSegmentMetadataPtr & segment_metadata) - { - file_segments.push_back(FileSegment::getSnapshot(segment_metadata->file_segment)); - return PriorityIterationResult::CONTINUE; - }, lockCache()); - - return file_segments; + return main_priority->dump(lockCache()); } std::vector FileCache::tryGetCachePaths(const Key & key) @@ -1210,4 +1065,17 @@ FileSegments FileCache::sync() return file_segments; } +FileCache::HitsCountStash::HitsCountStash(size_t hits_threashold_, size_t queue_size_) + : hits_threshold(hits_threashold_), queue_size(queue_size_), queue(std::make_unique(0, queue_size_)) +{ + if (!queue_size_) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Queue size for hits queue must be non-zero"); +} + +void FileCache::HitsCountStash::clear() +{ + records.clear(); + queue = std::make_unique(0, queue_size); +} + } diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index d7295868a0c..7c65c112869 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -40,7 +40,29 @@ struct FileCacheReserveStat size_t non_releasable_count; }; + Stat stat; std::unordered_map stat_by_kind; + + void update(size_t size, FileSegmentKind kind, bool releasable) + { + auto & local_stat = stat_by_kind[kind]; + if (releasable) + { + stat.releasable_size += size; + ++stat.releasable_count; + + local_stat.releasable_size += size; + ++local_stat.releasable_count; + } + else + { + stat.non_releasable_size += size; + ++stat.non_releasable_count; + + local_stat.non_releasable_size += size; + ++local_stat.non_releasable_count; + } + } }; /// Local cache for remote filesystem files, represented as a set of non-overlapping non-empty file segments. @@ -53,7 +75,6 @@ public: using Priority = IFileCachePriority; using PriorityEntry = IFileCachePriority::Entry; using PriorityIterator = IFileCachePriority::Iterator; - using PriorityIterationResult = IFileCachePriority::IterationResult; FileCache(const std::string & cache_name, const FileCacheSettings & settings); @@ -172,15 +193,13 @@ private: struct HitsCountStash { - HitsCountStash(size_t hits_threashold_, size_t queue_size_) - : hits_threshold(hits_threashold_), queue(std::make_unique(0, queue_size_)) - { - if (!queue_size_) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Queue size for hits queue must be non-zero"); - } + HitsCountStash(size_t hits_threashold_, size_t queue_size_); + void clear(); const size_t hits_threshold; - FileCachePriorityPtr queue; + const size_t queue_size; + + std::unique_ptr queue; using Records = std::unordered_map; Records records; }; diff --git a/src/Interpreters/Cache/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp index 2ac54006a8e..4fe29e241a4 100644 --- a/src/Interpreters/Cache/FileCacheSettings.cpp +++ b/src/Interpreters/Cache/FileCacheSettings.cpp @@ -58,6 +58,12 @@ void FileCacheSettings::loadImpl(FuncHas has, FuncGetUInt get_uint, FuncGetStrin if (has("load_metadata_threads")) load_metadata_threads = get_uint("load_metadata_threads"); + + if (has("cache_policy")) + cache_policy = get_string("cache_policy"); + + // if (has("slru_size_ratio")) + // slru_size_ratio = get_double("slru_size_ratio"); } void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) diff --git a/src/Interpreters/Cache/FileCacheSettings.h b/src/Interpreters/Cache/FileCacheSettings.h index bf0dd492bc9..bb9c4d17435 100644 --- a/src/Interpreters/Cache/FileCacheSettings.h +++ b/src/Interpreters/Cache/FileCacheSettings.h @@ -31,6 +31,9 @@ struct FileCacheSettings size_t load_metadata_threads = FILECACHE_DEFAULT_LOAD_METADATA_THREADS; + std::string cache_policy = "LRU"; + double slru_size_ratio = 0.5; + void loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); void loadFromCollection(const NamedCollection & collection); diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 9a0c1ac5654..8216d7a9a81 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -906,7 +906,7 @@ void FileSegment::detach(const FileSegmentGuard::Lock & lock, const LockedKey &) setDetachedState(lock); } -void FileSegment::use() +void FileSegment::increasePriority() { ProfileEventTimeIncrement watch(ProfileEvents::FileSegmentUseMicroseconds); @@ -920,7 +920,7 @@ void FileSegment::use() if (it) { auto cache_lock = cache->lockCache(); - hits_count = it->use(cache_lock); + hits_count = it->increasePriority(cache_lock); } } diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 7f84a3ea177..d509b45e35c 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -209,7 +209,7 @@ public: /// is not going to be changed. Completed states: DOWNALODED, DETACHED. bool isCompleted(bool sync = false) const; - void use(); + void increasePriority(); /** * ========== Methods used by `cache` ======================== diff --git a/src/Interpreters/Cache/IFileCachePriority.cpp b/src/Interpreters/Cache/IFileCachePriority.cpp new file mode 100644 index 00000000000..342a9589a1d --- /dev/null +++ b/src/Interpreters/Cache/IFileCachePriority.cpp @@ -0,0 +1,104 @@ +#include +#include +#include + + +namespace CurrentMetrics +{ + extern const Metric FilesystemCacheSizeLimit; +} + +namespace ProfileEvents +{ + extern const Event FilesystemCacheEvictMicroseconds; + extern const Event FilesystemCacheEvictedBytes; + extern const Event FilesystemCacheEvictedFileSegments; +} + +namespace DB +{ + +IFileCachePriority::IFileCachePriority(size_t max_size_, size_t max_elements_) + : max_size(max_size_), max_elements(max_elements_) +{ + CurrentMetrics::set(CurrentMetrics::FilesystemCacheSizeLimit, max_size_); +} + +IFileCachePriority::Entry::Entry( + const Key & key_, + size_t offset_, + size_t size_, + KeyMetadataPtr key_metadata_) + : key(key_) + , offset(offset_) + , key_metadata(key_metadata_) + , size(size_) +{ +} + +IFileCachePriority::Entry::Entry(const Entry & other) + : key(other.key) + , offset(other.offset) + , key_metadata(other.key_metadata) + , size(other.size.load()) + , hits(other.hits) +{ +} + +IFileCachePriority::EvictionCandidates::~EvictionCandidates() +{ + /// If failed to reserve space, we don't delete the candidates but drop the flag instead + /// so the segments can be used again + for (const auto & [key, key_candidates] : candidates) + { + for (const auto & candidate : key_candidates.candidates) + candidate->removal_candidate = false; + } +} + +void IFileCachePriority::EvictionCandidates::add(const KeyMetadataPtr & key, const FileSegmentMetadataPtr & candidate) +{ + auto it = candidates.emplace(key->key, KeyCandidates{}).first; + it->second.candidates.push_back(candidate); + candidate->removal_candidate = true; +} + +void IFileCachePriority::EvictionCandidates::evict(const CacheGuard::Lock & lock) +{ + if (candidates.empty()) + return; + + auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::FilesystemCacheEvictMicroseconds); + + for (auto & [key, key_candidates] : candidates) + { + auto locked_key = key_candidates.key_metadata->tryLock(); + if (!locked_key) + continue; /// key could become invalid after we released the key lock above, just skip it. + + /// delete from vector in reverse order just for efficiency + auto & to_evict = key_candidates.candidates; + while (!to_evict.empty()) + { + auto & candidate = to_evict.back(); + chassert(candidate->releasable()); + + const auto * segment = candidate->file_segment.get(); + auto queue_it = segment->getQueueIterator(); + chassert(queue_it); + + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedFileSegments); + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedBytes, segment->range().size()); + + locked_key->removeFileSegment(segment->offset(), segment->lock()); + queue_it->remove(lock); + + // if (query_context) + // query_context->remove(current_key, segment->offset(), cache_lock); + + to_evict.pop_back(); + } + } +} + +} diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index 7de380c163b..da09e927840 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include @@ -10,6 +9,7 @@ namespace DB { +struct FileCacheReserveStat; /// IFileCachePriority is used to maintain the priority of cached data. class IFileCachePriority : private boost::noncopyable @@ -20,53 +20,35 @@ public: struct Entry { - Entry(const Key & key_, size_t offset_, size_t size_, KeyMetadataPtr key_metadata_) - : key(key_), offset(offset_), size(size_), key_metadata(key_metadata_) {} - - Entry(const Entry & other) - : key(other.key), offset(other.offset), size(other.size.load()), hits(other.hits), key_metadata(other.key_metadata) {} + Entry(const Key & key_, size_t offset_, size_t size_, KeyMetadataPtr key_metadata_); + Entry(const Entry & other); const Key key; const size_t offset; + const KeyMetadataPtr key_metadata; + std::atomic size; size_t hits = 0; - const KeyMetadataPtr key_metadata; }; - /// Provides an iterator to traverse the cache priority. Under normal circumstances, - /// the iterator can only return the records that have been directly swapped out. - /// For example, in the LRU algorithm, it can traverse all records, but in the LRU-K, it - /// can only traverse the records in the low priority queue. class IIterator { public: virtual ~IIterator() = default; - virtual size_t use(const CacheGuard::Lock &) = 0; + virtual const Entry & getEntry() const = 0; + + virtual size_t increasePriority(const CacheGuard::Lock &) = 0; + + virtual void updateSize(int64_t size) = 0; virtual void remove(const CacheGuard::Lock &) = 0; - virtual const Entry & getEntry() const = 0; - - virtual Entry & getEntry() = 0; - virtual void invalidate() = 0; - - virtual void updateSize(int64_t size) = 0; }; - using Iterator = std::shared_ptr; - using ConstIterator = std::shared_ptr; - enum class IterationResult - { - BREAK, - CONTINUE, - REMOVE_AND_CONTINUE, - }; - using IterateFunc = std::function; - - IFileCachePriority(size_t max_size_, size_t max_elements_) : max_size(max_size_), max_elements(max_elements_) {} + IFileCachePriority(size_t max_size_, size_t max_elements_); virtual ~IFileCachePriority() = default; @@ -78,19 +60,46 @@ public: virtual size_t getElementsCount(const CacheGuard::Lock &) const = 0; - virtual Iterator add( - KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) = 0; - - virtual void pop(const CacheGuard::Lock &) = 0; - - virtual void removeAll(const CacheGuard::Lock &) = 0; - - /// From lowest to highest priority. - virtual void iterate(IterateFunc && func, const CacheGuard::Lock &) = 0; + virtual Iterator add(KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) = 0; virtual void shuffle(const CacheGuard::Lock &) = 0; -private: + virtual FileSegments dump(const CacheGuard::Lock &) = 0; + + class EvictionCandidates + { + public: + ~EvictionCandidates(); + + void add(const KeyMetadataPtr & key, const FileSegmentMetadataPtr & candidate); + + void evict(const CacheGuard::Lock &); + + auto begin() const { return candidates.begin(); } + auto end() const { return candidates.end(); } + + private: + struct KeyCandidates + { + KeyMetadataPtr key_metadata; + std::vector candidates; + }; + + std::unordered_map candidates; + }; + + using EvictionCandidatesPtr = std::unique_ptr; + using FinalizeEvictionFunc = std::function; + + virtual bool collectCandidatesForEviction( + size_t size, + FileCacheReserveStat & stat, + IFileCachePriority::EvictionCandidates & res, + IFileCachePriority::Iterator it, + FinalizeEvictionFunc & finalize_eviction_func, + const CacheGuard::Lock &) = 0; + +protected: const size_t max_size = 0; const size_t max_elements = 0; }; diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index 5ecea95b1db..addbb55e22d 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -11,6 +11,12 @@ namespace CurrentMetrics extern const Metric FilesystemCacheElements; } +namespace ProfileEvents +{ + extern const Event FilesystemCacheEvictionSkippedFileSegments; + extern const Event FilesystemCacheEvictionTries; +} + namespace DB { @@ -68,21 +74,7 @@ IFileCachePriority::Iterator LRUFileCachePriority::add( return std::make_shared(this, iter); } -void LRUFileCachePriority::removeAll(const CacheGuard::Lock &) -{ - LOG_TEST(log, "Removed all entries from LRU queue"); - - updateSize(-current_size); - updateElementsCount(-current_elements_num); - queue.clear(); -} - -void LRUFileCachePriority::pop(const CacheGuard::Lock &) -{ - remove(queue.begin()); -} - -LRUFileCachePriority::LRUQueueIterator LRUFileCachePriority::remove(LRUQueueIterator it) +LRUFileCachePriority::LRUQueueIterator LRUFileCachePriority::remove(LRUQueueIterator it, const CacheGuard::Lock &) { /// If size is 0, entry is invalidated, current_elements_num was already updated. if (it->size) @@ -119,21 +111,21 @@ LRUFileCachePriority::LRUFileCacheIterator::LRUFileCacheIterator( { } -void LRUFileCachePriority::iterate(IterateFunc && func, const CacheGuard::Lock &) +void LRUFileCachePriority::iterate(IterateFunc && func, const CacheGuard::Lock & lock) { for (auto it = queue.begin(); it != queue.end();) { auto locked_key = it->key_metadata->tryLock(); if (!locked_key || it->size == 0) { - it = remove(it); + it = remove(it, lock); continue; } auto metadata = locked_key->tryGetByOffset(it->offset); if (!metadata) { - it = remove(it); + it = remove(it, lock); continue; } @@ -160,17 +152,115 @@ void LRUFileCachePriority::iterate(IterateFunc && func, const CacheGuard::Lock & } case IterationResult::REMOVE_AND_CONTINUE: { - it = remove(it); + it = remove(it, lock); break; } } } } -void LRUFileCachePriority::LRUFileCacheIterator::remove(const CacheGuard::Lock &) +bool LRUFileCachePriority::collectCandidatesForEviction( + size_t size, + FileCacheReserveStat & stat, + IFileCachePriority::EvictionCandidates & res, + IFileCachePriority::Iterator, + FinalizeEvictionFunc &, + const CacheGuard::Lock & lock) +{ + auto is_overflow = [&] + { + return (max_size != 0 && (current_size + size - stat.stat.releasable_size > max_size)) + || (max_elements != 0 && stat.stat.releasable_count == 0 && current_elements_num == max_elements); + }; + + if (!is_overflow()) + return false; + + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictionTries); + + IterateFunc iterate_func = [&](LockedKey & locked_key, const FileSegmentMetadataPtr & segment_metadata) + { + const auto & file_segment = segment_metadata->file_segment; + chassert(file_segment->assertCorrectness()); + + if (segment_metadata->releasable()) + { + res.add(locked_key.getKeyMetadata(), segment_metadata); + stat.update(segment_metadata->size(), file_segment->getKind(), true); + } + else + { + stat.update(segment_metadata->size(), file_segment->getKind(), false); + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictionSkippedFileSegments); + } + + return IterationResult::CONTINUE; + }; + + iterate( + [&](LockedKey & locked_key, const FileSegmentMetadataPtr & segment_metadata) + { return is_overflow() ? iterate_func(locked_key, segment_metadata) : IterationResult::BREAK; }, + lock); + + return is_overflow(); +} + +size_t LRUFileCachePriority::increasePriority(LRUQueueIterator it, const CacheGuard::Lock &) +{ + queue.splice(queue.end(), queue, it); + return ++it->hits; +} + +LRUFileCachePriority::LRUQueueIterator +LRUFileCachePriority::move(LRUQueueIterator it, LRUFileCachePriority & other, const CacheGuard::Lock &) +{ + const size_t size = it->size; + if (size == 0) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Adding zero size entries to LRU queue is not allowed " + "(key: {}, offset: {})", it->key, it->offset); + } +#ifndef NDEBUG + for (const auto & entry : queue) + { + /// entry.size == 0 means entry was invalidated. + if (entry.size != 0 && entry.key == it->key && entry.offset == it->offset) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Attempt to add duplicate queue entry to queue. " + "(Key: {}, offset: {}, size: {})", + entry.key, entry.offset, entry.size); + } +#endif + + queue.splice(queue.end(), other.queue, it); + + updateSize(size); + updateElementsCount(1); + + other.updateSize(-size); + other.updateElementsCount(-1); + + return queue.end(); +} + +FileSegments LRUFileCachePriority::dump(const CacheGuard::Lock & lock) +{ + FileSegments res; + iterate([&](LockedKey &, const FileSegmentMetadataPtr & segment_metadata) + { + res.push_back(FileSegment::getSnapshot(segment_metadata->file_segment)); + return IterationResult::CONTINUE; + }, lock); + return res; +} + +void LRUFileCachePriority::LRUFileCacheIterator::remove(const CacheGuard::Lock & lock) { checkUsable(); - cache_priority->remove(queue_iter); + cache_priority->remove(queue_iter, lock); queue_iter = LRUQueueIterator{}; } @@ -201,11 +291,10 @@ void LRUFileCachePriority::LRUFileCacheIterator::updateSize(int64_t size) queue_iter->size += size; } -size_t LRUFileCachePriority::LRUFileCacheIterator::use(const CacheGuard::Lock &) +size_t LRUFileCachePriority::LRUFileCacheIterator::increasePriority(const CacheGuard::Lock & lock) { checkUsable(); - cache_priority->queue.splice(cache_priority->queue.end(), cache_priority->queue, queue_iter); - return ++queue_iter->hits; + return cache_priority->increasePriority(queue_iter, lock); } void LRUFileCachePriority::LRUFileCacheIterator::checkUsable() const diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 89f86961811..8e882fe5d9a 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -6,11 +6,6 @@ #include #include "Interpreters/Cache/Guards.h" -namespace CurrentMetrics -{ - extern const Metric FilesystemCacheSizeLimit; -} - namespace DB { @@ -22,12 +17,10 @@ private: class LRUFileCacheIterator; using LRUQueue = std::list; using LRUQueueIterator = typename LRUQueue::iterator; + friend class SLRUFileCachePriority; public: - LRUFileCachePriority(size_t max_size_, size_t max_elements_) : IFileCachePriority(max_size_, max_elements_) - { - CurrentMetrics::set(CurrentMetrics::FilesystemCacheSizeLimit, max_size_); - } + LRUFileCachePriority(size_t max_size_, size_t max_elements_) : IFileCachePriority(max_size_, max_elements_) {} size_t getSize(const CacheGuard::Lock &) const override { return current_size; } @@ -35,14 +28,20 @@ public: Iterator add(KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) override; - void pop(const CacheGuard::Lock &) override; - - void removeAll(const CacheGuard::Lock &) override; - - void iterate(IterateFunc && func, const CacheGuard::Lock &) override; + bool collectCandidatesForEviction( + size_t size, + FileCacheReserveStat & stat, + IFileCachePriority::EvictionCandidates & res, + IFileCachePriority::Iterator it, + FinalizeEvictionFunc & finalize_eviction_func, + const CacheGuard::Lock &) override; void shuffle(const CacheGuard::Lock &) override; + FileSegments dump(const CacheGuard::Lock &) override; + + void pop(const CacheGuard::Lock & lock) { remove(queue.begin(), lock); } + private: void updateElementsCount(int64_t num); void updateSize(int64_t size); @@ -55,7 +54,19 @@ private: /// because of invalidated entries. std::atomic current_elements_num = 0; - LRUQueueIterator remove(LRUQueueIterator it); + LRUQueueIterator remove(LRUQueueIterator it, const CacheGuard::Lock &); + + enum class IterationResult + { + BREAK, + CONTINUE, + REMOVE_AND_CONTINUE, + }; + using IterateFunc = std::function; + void iterate(IterateFunc && func, const CacheGuard::Lock &); + + size_t increasePriority(LRUQueueIterator it, const CacheGuard::Lock &); + LRUQueueIterator move(LRUQueueIterator it, LRUFileCachePriority & other, const CacheGuard::Lock &); }; class LRUFileCachePriority::LRUFileCacheIterator : public IFileCachePriority::IIterator @@ -67,9 +78,7 @@ public: const Entry & getEntry() const override { return *queue_iter; } - Entry & getEntry() override { return *queue_iter; } - - size_t use(const CacheGuard::Lock &) override; + size_t increasePriority(const CacheGuard::Lock &) override; void remove(const CacheGuard::Lock &) override; diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp new file mode 100644 index 00000000000..01a98452e7a --- /dev/null +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -0,0 +1,252 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace CurrentMetrics +{ + extern const Metric FilesystemCacheSize; + extern const Metric FilesystemCacheElements; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace +{ + size_t getRatio(size_t total, double ratio) + { + return static_cast(total * std::max(0.0, std::min(1.0, ratio))); + } +} + +SLRUFileCachePriority::SLRUFileCachePriority( + size_t max_size_, + size_t max_elements_, + double size_ratio) + : IFileCachePriority(max_size_, max_elements_) + , protected_queue(LRUFileCachePriority(getRatio(max_size_, size_ratio), getRatio(max_elements_, size_ratio))) + , probationary_queue(LRUFileCachePriority(getRatio(max_size_, 1 - size_ratio), getRatio(max_elements_, 1 - size_ratio))) +{ + LOG_DEBUG( + log, "Using probationary queue size: {}, protected queue size: {}", + probationary_queue.getSizeLimit(), protected_queue.getSizeLimit()); +} + +size_t SLRUFileCachePriority::getSize(const CacheGuard::Lock & lock) const +{ + return protected_queue.getSize(lock) + probationary_queue.getSize(lock); +} + +size_t SLRUFileCachePriority::getElementsCount(const CacheGuard::Lock & lock) const +{ + return protected_queue.getElementsCount(lock) + probationary_queue.getElementsCount(lock); +} + +IFileCachePriority::Iterator SLRUFileCachePriority::add( + KeyMetadataPtr key_metadata, + size_t offset, + size_t size, + const CacheGuard::Lock & lock) +{ + return probationary_queue.add(key_metadata, offset, size, lock); +} + +SLRUFileCachePriority::SLRUQueueIterator +SLRUFileCachePriority::remove(SLRUQueueIterator it, bool is_protected, const CacheGuard::Lock & lock) +{ + if (is_protected) + return protected_queue.remove(it, lock); + else + return probationary_queue.remove(it, lock); +} + +void SLRUFileCachePriority::updateSize(int64_t size, bool is_protected) +{ + if (is_protected) + protected_queue.updateSize(size); + else + probationary_queue.updateSize(size); +} + +void SLRUFileCachePriority::updateElementsCount(int64_t num, bool is_protected) +{ + if (is_protected) + protected_queue.updateElementsCount(num); + else + probationary_queue.updateElementsCount(num); +} + +bool SLRUFileCachePriority::collectCandidatesForEviction( + size_t size, + FileCacheReserveStat & stat, + IFileCachePriority::EvictionCandidates & res, + IFileCachePriority::Iterator it, + FinalizeEvictionFunc & finalize_eviction_func, + const CacheGuard::Lock & lock) +{ + bool is_protected = false; + if (it) + is_protected = assert_cast(it.get())->is_protected; + + if (!is_protected) + { + return probationary_queue.collectCandidatesForEviction(size, stat, res, it, finalize_eviction_func, lock); + } + + auto downgrade_candidates = std::make_shared(); + FileCacheReserveStat downgrade_stat; + FinalizeEvictionFunc noop; + + if (!protected_queue.collectCandidatesForEviction(size, downgrade_stat, *downgrade_candidates, it, noop, lock)) + return false; + + if (!probationary_queue.collectCandidatesForEviction(downgrade_stat.stat.releasable_size, stat, res, it, noop, lock)) + return false; + + finalize_eviction_func = [=, lk = &lock, this]() mutable + { + for (const auto & [key, key_candidates] : *downgrade_candidates) + { + for (const auto & candidate : key_candidates.candidates) + { + auto * candidate_it = assert_cast(candidate->getQueueIterator().get()); + probationary_queue.move(candidate_it->queue_iter, protected_queue, *lk); + } + } + }; + + return true; +} + +SLRUFileCachePriority::SLRUQueueIterator +SLRUFileCachePriority::increasePriority(SLRUQueueIterator & it, bool is_protected, const CacheGuard::Lock & lock) +{ + if (is_protected) + { + protected_queue.increasePriority(it, lock); + return it; + } + + if (it->size > protected_queue.getSizeLimit()) + { + /// This is only possible if protected_queue_size_limit is less than max_file_segment_size, + /// which is not possible in any realistic cache configuration. + return {}; + } + + IFileCachePriority::EvictionCandidates downgrade_candidates; + FileCacheReserveStat downgrade_stat; + FinalizeEvictionFunc noop; + + if (!protected_queue.collectCandidatesForEviction(it->size, downgrade_stat, downgrade_candidates, {}, noop, lock)) + { + probationary_queue.increasePriority(it, lock); + return it; + } + + IFileCachePriority::EvictionCandidates eviction_candidates; + FileCacheReserveStat stat; + + if (it->size < downgrade_stat.stat.releasable_size + && !probationary_queue.collectCandidatesForEviction( + downgrade_stat.stat.releasable_size - it->size, stat, eviction_candidates, {}, noop, lock)) + { + probationary_queue.increasePriority(it, lock); + return it; + } + + eviction_candidates.evict(lock); + + for (const auto & [key, key_candidates] : downgrade_candidates) + { + for (const auto & candidate : key_candidates.candidates) + { + auto * candidate_it = assert_cast(candidate->getQueueIterator().get()); + probationary_queue.move(candidate_it->queue_iter, protected_queue, lock); + } + } + + return protected_queue.move(it, probationary_queue, lock); +} + +FileSegments SLRUFileCachePriority::dump(const CacheGuard::Lock & lock) +{ + auto res = probationary_queue.dump(lock); + auto part_res = protected_queue.dump(lock); + res.insert(res.end(), part_res.begin(), part_res.end()); + return res; +} + +SLRUFileCachePriority::SLRUFileCacheIterator::SLRUFileCacheIterator( + SLRUFileCachePriority * cache_priority_, + SLRUFileCachePriority::SLRUQueueIterator queue_iter_, + bool is_protected_) + : cache_priority(cache_priority_) + , queue_iter(queue_iter_) + , is_protected(is_protected_) +{ +} + +void SLRUFileCachePriority::SLRUFileCacheIterator::remove(const CacheGuard::Lock & lock) +{ + checkUsable(); + cache_priority->remove(queue_iter, is_protected, lock); + queue_iter = SLRUQueueIterator{}; +} + +void SLRUFileCachePriority::SLRUFileCacheIterator::invalidate() +{ + checkUsable(); + + LOG_TEST( + cache_priority->log, + "Invalidating entry in SLRU queue. Key: {}, offset: {}, previous size: {}", + queue_iter->key, queue_iter->offset, queue_iter->size); + + cache_priority->updateSize(-queue_iter->size, is_protected); + cache_priority->updateElementsCount(-1, is_protected); + queue_iter->size = 0; +} + +void SLRUFileCachePriority::SLRUFileCacheIterator::updateSize(int64_t size) +{ + checkUsable(); + + LOG_TEST( + cache_priority->log, + "Update size with {} in SLRU queue for key: {}, offset: {}, previous size: {}", + size, queue_iter->key, queue_iter->offset, queue_iter->size); + + cache_priority->updateSize(size, is_protected); + queue_iter->size += size; +} + +size_t SLRUFileCachePriority::SLRUFileCacheIterator::increasePriority(const CacheGuard::Lock & lock) +{ + checkUsable(); + queue_iter = cache_priority->increasePriority(queue_iter, is_protected, lock); + return ++queue_iter->hits; +} + +void SLRUFileCachePriority::SLRUFileCacheIterator::checkUsable() const +{ + if (queue_iter == SLRUQueueIterator{}) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to use invalid iterator"); +} + +void SLRUFileCachePriority::shuffle(const CacheGuard::Lock & lock) +{ + protected_queue.shuffle(lock); + probationary_queue.shuffle(lock); +} + +} diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h new file mode 100644 index 00000000000..cc9ab70e4a3 --- /dev/null +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -0,0 +1,88 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace CurrentMetrics +{ + extern const Metric FilesystemCacheSizeLimit; +} + +namespace DB +{ + +/// Based on the SLRU algorithm implementation, the record with the lowest priority is stored at +/// the head of the queue, and the record with the highest priority is stored at the tail. +class SLRUFileCachePriority : public IFileCachePriority +{ +private: + class SLRUFileCacheIterator; + using LRUQueue = std::list; + using SLRUQueueIterator = typename LRUQueue::iterator; + +public: + SLRUFileCachePriority(size_t max_size_, size_t max_elements_, double size_ratio); + + size_t getSize(const CacheGuard::Lock & lock) const override; + + size_t getElementsCount(const CacheGuard::Lock &) const override; + + Iterator add(KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) override; + + bool collectCandidatesForEviction( + size_t size, + FileCacheReserveStat & stat, + IFileCachePriority::EvictionCandidates & res, + IFileCachePriority::Iterator, + FinalizeEvictionFunc & finalize_eviction_func, + const CacheGuard::Lock &) override; + + void shuffle(const CacheGuard::Lock &) override; + + FileSegments dump(const CacheGuard::Lock &) override; + +private: + void updateElementsCount(int64_t num, bool is_protected); + void updateSize(int64_t size, bool is_protected); + + LRUFileCachePriority protected_queue; + LRUFileCachePriority probationary_queue; + + Poco::Logger * log = &Poco::Logger::get("SLRUFileCachePriority"); + + SLRUQueueIterator remove(SLRUQueueIterator it, bool is_protected, const CacheGuard::Lock & lock); + SLRUQueueIterator increasePriority(SLRUQueueIterator & it, bool is_protected, const CacheGuard::Lock & lock); +}; + +class SLRUFileCachePriority::SLRUFileCacheIterator : public IFileCachePriority::IIterator +{ + friend class SLRUFileCachePriority; +public: + SLRUFileCacheIterator( + SLRUFileCachePriority * cache_priority_, + SLRUFileCachePriority::SLRUQueueIterator queue_iter_, + bool is_protected_); + + const Entry & getEntry() const override { return *queue_iter; } + + size_t increasePriority(const CacheGuard::Lock &) override; + + void remove(const CacheGuard::Lock &) override; + + void invalidate() override; + + void updateSize(int64_t size) override; + +private: + void checkUsable() const; + + SLRUFileCachePriority * cache_priority; + mutable SLRUFileCachePriority::SLRUQueueIterator queue_iter; + const bool is_protected; +}; + +} From e3e7e6c879d9a7f243693ebfa9f22c02d4e7b201 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 22 Nov 2023 15:53:19 +0100 Subject: [PATCH 033/331] Better, add comments --- src/Interpreters/Cache/EvictionCandidates.cpp | 71 +++++++++++++++ src/Interpreters/Cache/EvictionCandidates.h | 35 ++++++++ src/Interpreters/Cache/FileCache.cpp | 7 +- src/Interpreters/Cache/FileCache.h | 8 +- src/Interpreters/Cache/FileCacheSettings.cpp | 18 ++-- src/Interpreters/Cache/FileCacheSettings.h | 3 +- src/Interpreters/Cache/IFileCachePriority.cpp | 64 -------------- src/Interpreters/Cache/IFileCachePriority.h | 28 +----- .../Cache/LRUFileCachePriority.cpp | 70 +++++++++------ src/Interpreters/Cache/LRUFileCachePriority.h | 5 +- src/Interpreters/Cache/QueryLimit.h | 4 +- .../Cache/SLRUFileCachePriority.cpp | 87 +++++++++++++++---- .../Cache/SLRUFileCachePriority.h | 2 +- 13 files changed, 248 insertions(+), 154 deletions(-) create mode 100644 src/Interpreters/Cache/EvictionCandidates.cpp create mode 100644 src/Interpreters/Cache/EvictionCandidates.h diff --git a/src/Interpreters/Cache/EvictionCandidates.cpp b/src/Interpreters/Cache/EvictionCandidates.cpp new file mode 100644 index 00000000000..676a02a35da --- /dev/null +++ b/src/Interpreters/Cache/EvictionCandidates.cpp @@ -0,0 +1,71 @@ +#include +#include + + +namespace ProfileEvents +{ + extern const Event FilesystemCacheEvictMicroseconds; + extern const Event FilesystemCacheEvictedBytes; + extern const Event FilesystemCacheEvictedFileSegments; +} + +namespace DB +{ + +EvictionCandidates::~EvictionCandidates() +{ + for (const auto & [key, key_candidates] : candidates) + { + for (const auto & candidate : key_candidates.candidates) + candidate->removal_candidate = false; + } +} + +void EvictionCandidates::add(LockedKey & locked_key, const FileSegmentMetadataPtr & candidate) +{ + auto it = candidates.emplace(locked_key.getKey(), KeyCandidates{}).first; + it->second.key_metadata = locked_key.getKeyMetadata(); + it->second.candidates.push_back(candidate); + + candidate->removal_candidate = true; + ++candidates_size; +} + +void EvictionCandidates::evict(FileCacheQueryLimit::QueryContext * query_context, const CacheGuard::Lock & lock) +{ + if (candidates.empty()) + return; + + auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::FilesystemCacheEvictMicroseconds); + + for (auto & [key, key_candidates] : candidates) + { + auto locked_key = key_candidates.key_metadata->tryLock(); + if (!locked_key) + continue; /// key could become invalid after we released the key lock above, just skip it. + + auto & to_evict = key_candidates.candidates; + while (!to_evict.empty()) + { + auto & candidate = to_evict.back(); + chassert(candidate->releasable()); + + const auto segment = candidate->file_segment; + auto queue_it = segment->getQueueIterator(); + chassert(queue_it); + + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedFileSegments); + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedBytes, segment->range().size()); + + locked_key->removeFileSegment(segment->offset(), segment->lock()); + queue_it->remove(lock); + + if (query_context) + query_context->remove(segment->key(), segment->offset(), lock); + + to_evict.pop_back(); + } + } +} + +} diff --git a/src/Interpreters/Cache/EvictionCandidates.h b/src/Interpreters/Cache/EvictionCandidates.h new file mode 100644 index 00000000000..0557962d97f --- /dev/null +++ b/src/Interpreters/Cache/EvictionCandidates.h @@ -0,0 +1,35 @@ +#pragma once +#include + +namespace DB +{ + +class EvictionCandidates +{ +public: + ~EvictionCandidates(); + + void add(LockedKey & locked_key, const FileSegmentMetadataPtr & candidate); + + void evict(FileCacheQueryLimit::QueryContext * query_context, const CacheGuard::Lock &); + + size_t size() const { return candidates_size; } + + auto begin() const { return candidates.begin(); } + + auto end() const { return candidates.end(); } + +private: + struct KeyCandidates + { + KeyMetadataPtr key_metadata; + std::vector candidates; + }; + + std::unordered_map candidates; + size_t candidates_size = 0; +}; + +using EvictionCandidatesPtr = std::unique_ptr; + +} diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 8f75fb6f0b9..4bfd60cac03 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -578,7 +579,7 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa file_segment.key(), file_segment.offset()); } - IFileCachePriority::EvictionCandidates eviction_candidates; + EvictionCandidates eviction_candidates; IFileCachePriority::FinalizeEvictionFunc finalize_eviction_func; if (query_priority) @@ -603,10 +604,10 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa if (!file_segment.getKeyMetadata()->createBaseDirectory()) return false; - eviction_candidates.evict(cache_lock); + eviction_candidates.evict(query_context.get(), cache_lock); if (finalize_eviction_func) - finalize_eviction_func(); + finalize_eviction_func(cache_lock); if (queue_iterator) { diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 7c65c112869..d5e18486f33 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -33,11 +33,11 @@ struct FileCacheReserveStat { struct Stat { - size_t releasable_size; - size_t releasable_count; + size_t releasable_size = 0; + size_t releasable_count = 0; - size_t non_releasable_size; - size_t non_releasable_count; + size_t non_releasable_size = 0; + size_t non_releasable_count = 0; }; Stat stat; diff --git a/src/Interpreters/Cache/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp index 4fe29e241a4..0c86fb82f0d 100644 --- a/src/Interpreters/Cache/FileCacheSettings.cpp +++ b/src/Interpreters/Cache/FileCacheSettings.cpp @@ -13,7 +13,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -void FileCacheSettings::loadImpl(FuncHas has, FuncGetUInt get_uint, FuncGetString get_string) +void FileCacheSettings::loadImpl(FuncHas has, FuncGetUInt get_uint, FuncGetString get_string, FuncGetDouble get_double) { auto config_parse_size = [&](std::string_view key) { return parseWithSizeSuffix(get_string(key)); }; @@ -62,8 +62,8 @@ void FileCacheSettings::loadImpl(FuncHas has, FuncGetUInt get_uint, FuncGetStrin if (has("cache_policy")) cache_policy = get_string("cache_policy"); - // if (has("slru_size_ratio")) - // slru_size_ratio = get_double("slru_size_ratio"); + if (has("slru_size_ratio")) + slru_size_ratio = get_double("slru_size_ratio"); } void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) @@ -71,15 +71,17 @@ void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & auto config_has = [&](std::string_view key) { return config.has(fmt::format("{}.{}", config_prefix, key)); }; auto config_get_uint = [&](std::string_view key) { return config.getUInt(fmt::format("{}.{}", config_prefix, key)); }; auto config_get_string = [&](std::string_view key) { return config.getString(fmt::format("{}.{}", config_prefix, key)); }; - loadImpl(std::move(config_has), std::move(config_get_uint), std::move(config_get_string)); + auto config_get_double = [&](std::string_view key) { return config.getDouble(fmt::format("{}.{}", config_prefix, key)); }; + loadImpl(std::move(config_has), std::move(config_get_uint), std::move(config_get_string), std::move(config_get_double)); } void FileCacheSettings::loadFromCollection(const NamedCollection & collection) { - auto config_has = [&](std::string_view key) { return collection.has(std::string(key)); }; - auto config_get_uint = [&](std::string_view key) { return collection.get(std::string(key)); }; - auto config_get_string = [&](std::string_view key) { return collection.get(std::string(key)); }; - loadImpl(std::move(config_has), std::move(config_get_uint), std::move(config_get_string)); + auto collection_has = [&](std::string_view key) { return collection.has(std::string(key)); }; + auto collection_get_uint = [&](std::string_view key) { return collection.get(std::string(key)); }; + auto collection_get_string = [&](std::string_view key) { return collection.get(std::string(key)); }; + auto collection_get_double = [&](std::string_view key) { return collection.get(std::string(key)); }; + loadImpl(std::move(collection_has), std::move(collection_get_uint), std::move(collection_get_string), std::move(collection_get_double)); } } diff --git a/src/Interpreters/Cache/FileCacheSettings.h b/src/Interpreters/Cache/FileCacheSettings.h index bb9c4d17435..7b83639f444 100644 --- a/src/Interpreters/Cache/FileCacheSettings.h +++ b/src/Interpreters/Cache/FileCacheSettings.h @@ -41,7 +41,8 @@ private: using FuncHas = std::function; using FuncGetUInt = std::function; using FuncGetString = std::function; - void loadImpl(FuncHas has, FuncGetUInt get_uint, FuncGetString get_string); + using FuncGetDouble = std::function; + void loadImpl(FuncHas has, FuncGetUInt get_uint, FuncGetString get_string, FuncGetDouble get_double); }; } diff --git a/src/Interpreters/Cache/IFileCachePriority.cpp b/src/Interpreters/Cache/IFileCachePriority.cpp index 342a9589a1d..9109e76562f 100644 --- a/src/Interpreters/Cache/IFileCachePriority.cpp +++ b/src/Interpreters/Cache/IFileCachePriority.cpp @@ -1,5 +1,4 @@ #include -#include #include @@ -8,13 +7,6 @@ namespace CurrentMetrics extern const Metric FilesystemCacheSizeLimit; } -namespace ProfileEvents -{ - extern const Event FilesystemCacheEvictMicroseconds; - extern const Event FilesystemCacheEvictedBytes; - extern const Event FilesystemCacheEvictedFileSegments; -} - namespace DB { @@ -45,60 +37,4 @@ IFileCachePriority::Entry::Entry(const Entry & other) { } -IFileCachePriority::EvictionCandidates::~EvictionCandidates() -{ - /// If failed to reserve space, we don't delete the candidates but drop the flag instead - /// so the segments can be used again - for (const auto & [key, key_candidates] : candidates) - { - for (const auto & candidate : key_candidates.candidates) - candidate->removal_candidate = false; - } -} - -void IFileCachePriority::EvictionCandidates::add(const KeyMetadataPtr & key, const FileSegmentMetadataPtr & candidate) -{ - auto it = candidates.emplace(key->key, KeyCandidates{}).first; - it->second.candidates.push_back(candidate); - candidate->removal_candidate = true; -} - -void IFileCachePriority::EvictionCandidates::evict(const CacheGuard::Lock & lock) -{ - if (candidates.empty()) - return; - - auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::FilesystemCacheEvictMicroseconds); - - for (auto & [key, key_candidates] : candidates) - { - auto locked_key = key_candidates.key_metadata->tryLock(); - if (!locked_key) - continue; /// key could become invalid after we released the key lock above, just skip it. - - /// delete from vector in reverse order just for efficiency - auto & to_evict = key_candidates.candidates; - while (!to_evict.empty()) - { - auto & candidate = to_evict.back(); - chassert(candidate->releasable()); - - const auto * segment = candidate->file_segment.get(); - auto queue_it = segment->getQueueIterator(); - chassert(queue_it); - - ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedFileSegments); - ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedBytes, segment->range().size()); - - locked_key->removeFileSegment(segment->offset(), segment->lock()); - queue_it->remove(lock); - - // if (query_context) - // query_context->remove(current_key, segment->offset(), cache_lock); - - to_evict.pop_back(); - } - } -} - } diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index da09e927840..1703ed09139 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -10,6 +10,7 @@ namespace DB { struct FileCacheReserveStat; +class EvictionCandidates; /// IFileCachePriority is used to maintain the priority of cached data. class IFileCachePriority : private boost::noncopyable @@ -66,35 +67,12 @@ public: virtual FileSegments dump(const CacheGuard::Lock &) = 0; - class EvictionCandidates - { - public: - ~EvictionCandidates(); - - void add(const KeyMetadataPtr & key, const FileSegmentMetadataPtr & candidate); - - void evict(const CacheGuard::Lock &); - - auto begin() const { return candidates.begin(); } - auto end() const { return candidates.end(); } - - private: - struct KeyCandidates - { - KeyMetadataPtr key_metadata; - std::vector candidates; - }; - - std::unordered_map candidates; - }; - - using EvictionCandidatesPtr = std::unique_ptr; - using FinalizeEvictionFunc = std::function; + using FinalizeEvictionFunc = std::function; virtual bool collectCandidatesForEviction( size_t size, FileCacheReserveStat & stat, - IFileCachePriority::EvictionCandidates & res, + EvictionCandidates & res, IFileCachePriority::Iterator it, FinalizeEvictionFunc & finalize_eviction_func, const CacheGuard::Lock &) = 0; diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index addbb55e22d..c42e44f21b1 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -29,22 +30,27 @@ IFileCachePriority::Iterator LRUFileCachePriority::add( KeyMetadataPtr key_metadata, size_t offset, size_t size, - const CacheGuard::Lock &) + const CacheGuard::Lock & lock) { - const auto & key = key_metadata->key; - if (size == 0) + auto it = add(Entry(key_metadata->key, offset, size, key_metadata), lock); + return std::make_shared(this, it); +} + +LRUFileCachePriority::LRUQueueIterator LRUFileCachePriority::add(Entry && entry, const CacheGuard::Lock &) +{ + if (entry.size == 0) { throw Exception( ErrorCodes::LOGICAL_ERROR, "Adding zero size entries to LRU queue is not allowed " - "(key: {}, offset: {})", key, offset); + "(key: {}, offset: {})", entry.key, entry.offset); } #ifndef NDEBUG - for (const auto & entry : queue) + for (const auto & queue_entry : queue) { /// entry.size == 0 means entry was invalidated. - if (entry.size != 0 && entry.key == key && entry.offset == offset) + if (queue_entry.size != 0 && queue_entry.key == entry.key && queue_entry.offset == entry.offset) throw Exception( ErrorCodes::LOGICAL_ERROR, "Attempt to add duplicate queue entry to queue. " @@ -54,24 +60,24 @@ IFileCachePriority::Iterator LRUFileCachePriority::add( #endif const auto & size_limit = getSizeLimit(); - if (size_limit && current_size + size > size_limit) + if (size_limit && current_size + entry.size > size_limit) { throw Exception( ErrorCodes::LOGICAL_ERROR, "Not enough space to add {}:{} with size {}: current size: {}/{}", - key, offset, size, current_size, size_limit); + entry.key, entry.offset, entry.size, current_size, size_limit); } - auto iter = queue.insert(queue.end(), Entry(key, offset, size, key_metadata)); + auto it = queue.insert(queue.end(), entry); - updateSize(size); + updateSize(entry.size); updateElementsCount(1); LOG_TEST( log, "Added entry into LRU queue, key: {}, offset: {}, size: {}", - key, offset, size); + entry.key, entry.offset, entry.size); - return std::make_shared(this, iter); + return it; } LRUFileCachePriority::LRUQueueIterator LRUFileCachePriority::remove(LRUQueueIterator it, const CacheGuard::Lock &) @@ -159,22 +165,27 @@ void LRUFileCachePriority::iterate(IterateFunc && func, const CacheGuard::Lock & } } +bool LRUFileCachePriority::canFit(size_t size, const CacheGuard::Lock & lock) const +{ + return canFit(size, 0, 0, lock); +} + +bool LRUFileCachePriority::canFit(size_t size, size_t released_size_assumption, size_t released_elements_assumption, const CacheGuard::Lock &) const +{ + return (max_size == 0 || (current_size + size - released_size_assumption <= max_size)) + && (max_elements == 0 || current_elements_num + 1 - released_elements_assumption <= max_elements); +} + bool LRUFileCachePriority::collectCandidatesForEviction( size_t size, FileCacheReserveStat & stat, - IFileCachePriority::EvictionCandidates & res, + EvictionCandidates & res, IFileCachePriority::Iterator, FinalizeEvictionFunc &, const CacheGuard::Lock & lock) { - auto is_overflow = [&] - { - return (max_size != 0 && (current_size + size - stat.stat.releasable_size > max_size)) - || (max_elements != 0 && stat.stat.releasable_count == 0 && current_elements_num == max_elements); - }; - - if (!is_overflow()) - return false; + if (canFit(size, lock)) + return true; ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictionTries); @@ -185,7 +196,7 @@ bool LRUFileCachePriority::collectCandidatesForEviction( if (segment_metadata->releasable()) { - res.add(locked_key.getKeyMetadata(), segment_metadata); + res.add(locked_key, segment_metadata); stat.update(segment_metadata->size(), file_segment->getKind(), true); } else @@ -197,12 +208,17 @@ bool LRUFileCachePriority::collectCandidatesForEviction( return IterationResult::CONTINUE; }; - iterate( - [&](LockedKey & locked_key, const FileSegmentMetadataPtr & segment_metadata) - { return is_overflow() ? iterate_func(locked_key, segment_metadata) : IterationResult::BREAK; }, - lock); + auto can_fit = [&] + { + return canFit(size, stat.stat.releasable_size, stat.stat.releasable_count, lock); + }; - return is_overflow(); + iterate([&](LockedKey & locked_key, const FileSegmentMetadataPtr & segment_metadata) + { + return can_fit() ? IterationResult::BREAK : iterate_func(locked_key, segment_metadata); + }, lock); + + return can_fit(); } size_t LRUFileCachePriority::increasePriority(LRUQueueIterator it, const CacheGuard::Lock &) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 8e882fe5d9a..99011aca06c 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -31,7 +31,7 @@ public: bool collectCandidatesForEviction( size_t size, FileCacheReserveStat & stat, - IFileCachePriority::EvictionCandidates & res, + EvictionCandidates & res, IFileCachePriority::Iterator it, FinalizeEvictionFunc & finalize_eviction_func, const CacheGuard::Lock &) override; @@ -67,6 +67,9 @@ private: size_t increasePriority(LRUQueueIterator it, const CacheGuard::Lock &); LRUQueueIterator move(LRUQueueIterator it, LRUFileCachePriority & other, const CacheGuard::Lock &); + LRUQueueIterator add(Entry && entry, const CacheGuard::Lock &); + bool canFit(size_t size, const CacheGuard::Lock &) const; + bool canFit(size_t size, size_t released_size_assumption, size_t released_elements_assumption, const CacheGuard::Lock &) const; }; class LRUFileCachePriority::LRUFileCacheIterator : public IFileCachePriority::IIterator diff --git a/src/Interpreters/Cache/QueryLimit.h b/src/Interpreters/Cache/QueryLimit.h index f8247e8c520..c252cd2dccc 100644 --- a/src/Interpreters/Cache/QueryLimit.h +++ b/src/Interpreters/Cache/QueryLimit.h @@ -36,7 +36,7 @@ public: bool recacheOnFileCacheQueryLimitExceeded() const { return recache_on_query_limit_exceeded; } - IFileCachePriority::Iterator tryGet( + Priority::Iterator tryGet( const Key & key, size_t offset, const CacheGuard::Lock &); @@ -53,7 +53,7 @@ public: const CacheGuard::Lock &); private: - using Records = std::unordered_map; + using Records = std::unordered_map; Records records; LRUFileCachePriority priority; const bool recache_on_query_limit_exceeded; diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index 01a98452e7a..12119c23ce6 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -88,38 +89,58 @@ void SLRUFileCachePriority::updateElementsCount(int64_t num, bool is_protected) bool SLRUFileCachePriority::collectCandidatesForEviction( size_t size, FileCacheReserveStat & stat, - IFileCachePriority::EvictionCandidates & res, + EvictionCandidates & res, IFileCachePriority::Iterator it, FinalizeEvictionFunc & finalize_eviction_func, const CacheGuard::Lock & lock) { - bool is_protected = false; - if (it) - is_protected = assert_cast(it.get())->is_protected; - - if (!is_protected) + /// `it` is a pointer to entry we want to evict in favour of. + /// If `it` is nullptr, then it is the first space reservation attempt + /// for a corresponding file segment, so it will be directly put into probationary queue. + if (!it) { return probationary_queue.collectCandidatesForEviction(size, stat, res, it, finalize_eviction_func, lock); } - auto downgrade_candidates = std::make_shared(); + /// If `it` not nullptr (e.g. is already in some queue), + /// we need to check in which queue (protected/probationary) it currently is + /// (in order to know where we need to free space). + if (!assert_cast(it.get())->is_protected) + { + return probationary_queue.collectCandidatesForEviction(size, stat, res, it, finalize_eviction_func, lock); + } + + /// Entry is in protected queue. + /// Check if we have enough space in protected queue to fit a new size of entry. + /// `size` is the increment to the current entry.size we want to increase. + if (protected_queue.canFit(size, lock)) + return true; + + /// If not enough space - we need to "downgrade" lowest priority entries from protected + /// queue to probationary queue. + /// The amount of such "downgraded" entries is equal to the amount + /// required to make space for additionary `size` bytes for entry. + auto downgrade_candidates = std::make_shared(); FileCacheReserveStat downgrade_stat; FinalizeEvictionFunc noop; if (!protected_queue.collectCandidatesForEviction(size, downgrade_stat, *downgrade_candidates, it, noop, lock)) return false; - if (!probationary_queue.collectCandidatesForEviction(downgrade_stat.stat.releasable_size, stat, res, it, noop, lock)) + const size_t size_to_downgrade = downgrade_stat.stat.releasable_size; + + if (!probationary_queue.canFit(size_to_downgrade, lock) + && !probationary_queue.collectCandidatesForEviction(size_to_downgrade, stat, res, it, noop, lock)) return false; - finalize_eviction_func = [=, lk = &lock, this]() mutable + finalize_eviction_func = [=, this](const CacheGuard::Lock & lk) mutable { for (const auto & [key, key_candidates] : *downgrade_candidates) { for (const auto & candidate : key_candidates.candidates) { auto * candidate_it = assert_cast(candidate->getQueueIterator().get()); - probationary_queue.move(candidate_it->queue_iter, protected_queue, *lk); + probationary_queue.move(candidate_it->queue_iter, protected_queue, lk); } } }; @@ -130,41 +151,71 @@ bool SLRUFileCachePriority::collectCandidatesForEviction( SLRUFileCachePriority::SLRUQueueIterator SLRUFileCachePriority::increasePriority(SLRUQueueIterator & it, bool is_protected, const CacheGuard::Lock & lock) { + /// If entry (`it` is the pointer to the entry) is already in protected queue, + /// we only need to increase its priority within the protected queue. if (is_protected) { protected_queue.increasePriority(it, lock); return it; } + /// Entry is in probationary queue. + /// We need to move it to protected queue. + if (it->size > protected_queue.getSizeLimit()) { + /// Entry size is bigger than the whole protected queue limit. /// This is only possible if protected_queue_size_limit is less than max_file_segment_size, /// which is not possible in any realistic cache configuration. - return {}; + probationary_queue.increasePriority(it, lock); + return it; } - IFileCachePriority::EvictionCandidates downgrade_candidates; + /// Check if there is enough space in protected queue to move entry there. + /// If not - we need to "downgrade" lowest priority entries from protected + /// queue to probationary queue. + /// The amount of such "downgraded" entries is equal to the amount + /// required to make space for entry we want to insert. + EvictionCandidates downgrade_candidates; FileCacheReserveStat downgrade_stat; FinalizeEvictionFunc noop; if (!protected_queue.collectCandidatesForEviction(it->size, downgrade_stat, downgrade_candidates, {}, noop, lock)) { + /// We cannot make space for entry to be moved to protected queue + /// (not enough releasable file segments). + /// Then just increase its priority within probationary queue. probationary_queue.increasePriority(it, lock); return it; } - IFileCachePriority::EvictionCandidates eviction_candidates; + /// Now we need to check if those "downgrade" candidates can actually + /// be moved to probationary queue. + const size_t size_to_downgrade = downgrade_stat.stat.releasable_count; + size_t size_to_free = 0; + if (size_to_downgrade && size_to_downgrade > it->size) + size_to_free = size_to_downgrade - it->size; + + EvictionCandidates eviction_candidates; FileCacheReserveStat stat; - if (it->size < downgrade_stat.stat.releasable_size - && !probationary_queue.collectCandidatesForEviction( - downgrade_stat.stat.releasable_size - it->size, stat, eviction_candidates, {}, noop, lock)) + if (size_to_free + && !probationary_queue.collectCandidatesForEviction(size_to_free, stat, eviction_candidates, {}, noop, lock)) { + /// "downgrade" canidates cannot be moved to probationary queue, + /// so entry cannot be moved to protected queue as well. + /// Then just increase its priority within probationary queue. probationary_queue.increasePriority(it, lock); return it; } - eviction_candidates.evict(lock); + /// Make space for "downgrade" candidates. + eviction_candidates.evict(nullptr, lock); + + /// All checks passed, now we can move downgrade candidates to + /// probationary queue and our entry to protected queue. + Entry entry = *it; + probationary_queue.remove(it, lock); for (const auto & [key, key_candidates] : downgrade_candidates) { @@ -175,7 +226,7 @@ SLRUFileCachePriority::increasePriority(SLRUQueueIterator & it, bool is_protecte } } - return protected_queue.move(it, probationary_queue, lock); + return protected_queue.add(std::move(entry), lock); } FileSegments SLRUFileCachePriority::dump(const CacheGuard::Lock & lock) diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index cc9ab70e4a3..a179a5285b2 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -36,7 +36,7 @@ public: bool collectCandidatesForEviction( size_t size, FileCacheReserveStat & stat, - IFileCachePriority::EvictionCandidates & res, + EvictionCandidates & res, IFileCachePriority::Iterator, FinalizeEvictionFunc & finalize_eviction_func, const CacheGuard::Lock &) override; From c22e77d8aaade3ca79872c868bb52516bcbce00c Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 22 Nov 2023 22:54:38 +0100 Subject: [PATCH 034/331] Better --- src/Interpreters/Cache/EvictionCandidates.cpp | 5 +- .../Cache/LRUFileCachePriority.cpp | 63 +++--- src/Interpreters/Cache/LRUFileCachePriority.h | 25 ++- .../Cache/SLRUFileCachePriority.cpp | 183 ++++++++---------- .../Cache/SLRUFileCachePriority.h | 23 +-- 5 files changed, 128 insertions(+), 171 deletions(-) diff --git a/src/Interpreters/Cache/EvictionCandidates.cpp b/src/Interpreters/Cache/EvictionCandidates.cpp index 676a02a35da..7dceab4f95f 100644 --- a/src/Interpreters/Cache/EvictionCandidates.cpp +++ b/src/Interpreters/Cache/EvictionCandidates.cpp @@ -23,8 +23,9 @@ EvictionCandidates::~EvictionCandidates() void EvictionCandidates::add(LockedKey & locked_key, const FileSegmentMetadataPtr & candidate) { - auto it = candidates.emplace(locked_key.getKey(), KeyCandidates{}).first; - it->second.key_metadata = locked_key.getKeyMetadata(); + auto [it, inserted] = candidates.emplace(locked_key.getKey(), KeyCandidates{}); + if (inserted) + it->second.key_metadata = locked_key.getKeyMetadata(); it->second.candidates.push_back(candidate); candidate->removal_candidate = true; diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index c42e44f21b1..5cd44a67d89 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -32,11 +32,10 @@ IFileCachePriority::Iterator LRUFileCachePriority::add( size_t size, const CacheGuard::Lock & lock) { - auto it = add(Entry(key_metadata->key, offset, size, key_metadata), lock); - return std::make_shared(this, it); + return add(Entry(key_metadata->key, offset, size, key_metadata), lock); } -LRUFileCachePriority::LRUQueueIterator LRUFileCachePriority::add(Entry && entry, const CacheGuard::Lock &) +std::unique_ptr LRUFileCachePriority::add(Entry && entry, const CacheGuard::Lock &) { if (entry.size == 0) { @@ -77,10 +76,10 @@ LRUFileCachePriority::LRUQueueIterator LRUFileCachePriority::add(Entry && entry, log, "Added entry into LRU queue, key: {}, offset: {}, size: {}", entry.key, entry.offset, entry.size); - return it; + return std::make_unique(this, it); } -LRUFileCachePriority::LRUQueueIterator LRUFileCachePriority::remove(LRUQueueIterator it, const CacheGuard::Lock &) +LRUFileCachePriority::LRUQueue::iterator LRUFileCachePriority::remove(LRUQueue::iterator it, const CacheGuard::Lock &) { /// If size is 0, entry is invalidated, current_elements_num was already updated. if (it->size) @@ -109,11 +108,8 @@ void LRUFileCachePriority::updateElementsCount(int64_t num) } -LRUFileCachePriority::LRUFileCacheIterator::LRUFileCacheIterator( - LRUFileCachePriority * cache_priority_, - LRUFileCachePriority::LRUQueueIterator queue_iter_) - : cache_priority(cache_priority_) - , queue_iter(queue_iter_) +LRUFileCachePriority::LRUIterator::LRUIterator(LRUFileCachePriority * cache_priority_, LRUQueue::iterator queue_iter_) + : cache_priority(cache_priority_), queue_iter(queue_iter_) { } @@ -221,28 +217,21 @@ bool LRUFileCachePriority::collectCandidatesForEviction( return can_fit(); } -size_t LRUFileCachePriority::increasePriority(LRUQueueIterator it, const CacheGuard::Lock &) +std::unique_ptr LRUFileCachePriority::move(LRUIterator & it, LRUFileCachePriority & other, const CacheGuard::Lock &) { - queue.splice(queue.end(), queue, it); - return ++it->hits; -} - -LRUFileCachePriority::LRUQueueIterator -LRUFileCachePriority::move(LRUQueueIterator it, LRUFileCachePriority & other, const CacheGuard::Lock &) -{ - const size_t size = it->size; - if (size == 0) + const auto & entry = it.getEntry(); + if (entry.size == 0) { throw Exception( ErrorCodes::LOGICAL_ERROR, "Adding zero size entries to LRU queue is not allowed " - "(key: {}, offset: {})", it->key, it->offset); + "(key: {}, offset: {})", entry.key, entry.offset); } #ifndef NDEBUG - for (const auto & entry : queue) + for (const auto & queue_entry : queue) { /// entry.size == 0 means entry was invalidated. - if (entry.size != 0 && entry.key == it->key && entry.offset == it->offset) + if (queue_entry.size != 0 && queue_entry.key == entry.key && queue_entry.offset == entry.offset) throw Exception( ErrorCodes::LOGICAL_ERROR, "Attempt to add duplicate queue entry to queue. " @@ -251,15 +240,14 @@ LRUFileCachePriority::move(LRUQueueIterator it, LRUFileCachePriority & other, co } #endif - queue.splice(queue.end(), other.queue, it); + queue.splice(queue.end(), other.queue, it.queue_iter); - updateSize(size); + updateSize(entry.size); updateElementsCount(1); - other.updateSize(-size); + other.updateSize(-entry.size); other.updateElementsCount(-1); - - return queue.end(); + return std::make_unique(this, it.queue_iter); } FileSegments LRUFileCachePriority::dump(const CacheGuard::Lock & lock) @@ -273,14 +261,14 @@ FileSegments LRUFileCachePriority::dump(const CacheGuard::Lock & lock) return res; } -void LRUFileCachePriority::LRUFileCacheIterator::remove(const CacheGuard::Lock & lock) +void LRUFileCachePriority::LRUIterator::remove(const CacheGuard::Lock & lock) { checkUsable(); cache_priority->remove(queue_iter, lock); - queue_iter = LRUQueueIterator{}; + queue_iter = LRUQueue::iterator{}; } -void LRUFileCachePriority::LRUFileCacheIterator::invalidate() +void LRUFileCachePriority::LRUIterator::invalidate() { checkUsable(); @@ -294,7 +282,7 @@ void LRUFileCachePriority::LRUFileCacheIterator::invalidate() queue_iter->size = 0; } -void LRUFileCachePriority::LRUFileCacheIterator::updateSize(int64_t size) +void LRUFileCachePriority::LRUIterator::updateSize(int64_t size) { checkUsable(); @@ -307,21 +295,22 @@ void LRUFileCachePriority::LRUFileCacheIterator::updateSize(int64_t size) queue_iter->size += size; } -size_t LRUFileCachePriority::LRUFileCacheIterator::increasePriority(const CacheGuard::Lock & lock) +size_t LRUFileCachePriority::LRUIterator::increasePriority(const CacheGuard::Lock &) { checkUsable(); - return cache_priority->increasePriority(queue_iter, lock); + cache_priority->queue.splice(cache_priority->queue.end(), cache_priority->queue, queue_iter); + return ++queue_iter->hits; } -void LRUFileCachePriority::LRUFileCacheIterator::checkUsable() const +void LRUFileCachePriority::LRUIterator::checkUsable() const { - if (queue_iter == LRUQueueIterator{}) + if (queue_iter == LRUQueue::iterator{}) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to use invalid iterator"); } void LRUFileCachePriority::shuffle(const CacheGuard::Lock &) { - std::vector its; + std::vector its; its.reserve(queue.size()); for (auto it = queue.begin(); it != queue.end(); ++it) its.push_back(it); diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 99011aca06c..acc8a5a9a76 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -14,9 +14,8 @@ namespace DB class LRUFileCachePriority : public IFileCachePriority { private: - class LRUFileCacheIterator; + class LRUIterator; using LRUQueue = std::list; - using LRUQueueIterator = typename LRUQueue::iterator; friend class SLRUFileCachePriority; public: @@ -54,7 +53,10 @@ private: /// because of invalidated entries. std::atomic current_elements_num = 0; - LRUQueueIterator remove(LRUQueueIterator it, const CacheGuard::Lock &); + bool canFit(size_t size, const CacheGuard::Lock &) const; + bool canFit(size_t size, size_t released_size_assumption, size_t released_elements_assumption, const CacheGuard::Lock &) const; + + LRUQueue::iterator remove(LRUQueue::iterator it, const CacheGuard::Lock &); enum class IterationResult { @@ -65,19 +67,16 @@ private: using IterateFunc = std::function; void iterate(IterateFunc && func, const CacheGuard::Lock &); - size_t increasePriority(LRUQueueIterator it, const CacheGuard::Lock &); - LRUQueueIterator move(LRUQueueIterator it, LRUFileCachePriority & other, const CacheGuard::Lock &); - LRUQueueIterator add(Entry && entry, const CacheGuard::Lock &); - bool canFit(size_t size, const CacheGuard::Lock &) const; - bool canFit(size_t size, size_t released_size_assumption, size_t released_elements_assumption, const CacheGuard::Lock &) const; + std::unique_ptr move(LRUIterator & it, LRUFileCachePriority & other, const CacheGuard::Lock &); + std::unique_ptr add(Entry && entry, const CacheGuard::Lock &); }; -class LRUFileCachePriority::LRUFileCacheIterator : public IFileCachePriority::IIterator +class LRUFileCachePriority::LRUIterator : public IFileCachePriority::IIterator { + friend class LRUFileCachePriority; + friend class SLRUFileCachePriority; public: - LRUFileCacheIterator( - LRUFileCachePriority * cache_priority_, - LRUFileCachePriority::LRUQueueIterator queue_iter_); + LRUIterator(LRUFileCachePriority * cache_priority_, LRUQueue::iterator queue_iter_); const Entry & getEntry() const override { return *queue_iter; } @@ -93,7 +92,7 @@ private: void checkUsable() const; LRUFileCachePriority * cache_priority; - mutable LRUFileCachePriority::LRUQueueIterator queue_iter; + mutable LRUQueue::iterator queue_iter; }; } diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index 12119c23ce6..ee8da79ee6f 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -58,32 +58,8 @@ IFileCachePriority::Iterator SLRUFileCachePriority::add( size_t size, const CacheGuard::Lock & lock) { - return probationary_queue.add(key_metadata, offset, size, lock); -} - -SLRUFileCachePriority::SLRUQueueIterator -SLRUFileCachePriority::remove(SLRUQueueIterator it, bool is_protected, const CacheGuard::Lock & lock) -{ - if (is_protected) - return protected_queue.remove(it, lock); - else - return probationary_queue.remove(it, lock); -} - -void SLRUFileCachePriority::updateSize(int64_t size, bool is_protected) -{ - if (is_protected) - protected_queue.updateSize(size); - else - probationary_queue.updateSize(size); -} - -void SLRUFileCachePriority::updateElementsCount(int64_t num, bool is_protected) -{ - if (is_protected) - protected_queue.updateElementsCount(num); - else - probationary_queue.updateElementsCount(num); + auto it = probationary_queue.add(Entry(key_metadata->key, offset, size, key_metadata), lock); + return std::make_shared(this, std::move(it), false); } bool SLRUFileCachePriority::collectCandidatesForEviction( @@ -105,7 +81,7 @@ bool SLRUFileCachePriority::collectCandidatesForEviction( /// If `it` not nullptr (e.g. is already in some queue), /// we need to check in which queue (protected/probationary) it currently is /// (in order to know where we need to free space). - if (!assert_cast(it.get())->is_protected) + if (!assert_cast(it.get())->is_protected) { return probationary_queue.collectCandidatesForEviction(size, stat, res, it, finalize_eviction_func, lock); } @@ -139,8 +115,9 @@ bool SLRUFileCachePriority::collectCandidatesForEviction( { for (const auto & candidate : key_candidates.candidates) { - auto * candidate_it = assert_cast(candidate->getQueueIterator().get()); - probationary_queue.move(candidate_it->queue_iter, protected_queue, lk); + auto * candidate_it = assert_cast(candidate->getQueueIterator().get()); + candidate_it->lru_iterator = probationary_queue.move(*candidate_it->lru_iterator, protected_queue, lk); + candidate_it->is_protected = false; } } }; @@ -148,27 +125,30 @@ bool SLRUFileCachePriority::collectCandidatesForEviction( return true; } -SLRUFileCachePriority::SLRUQueueIterator -SLRUFileCachePriority::increasePriority(SLRUQueueIterator & it, bool is_protected, const CacheGuard::Lock & lock) +void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const CacheGuard::Lock & lock) { + auto & lru_it = iterator.lru_iterator; + const bool is_protected = iterator.is_protected; + const auto & entry = lru_it->getEntry(); + /// If entry (`it` is the pointer to the entry) is already in protected queue, /// we only need to increase its priority within the protected queue. if (is_protected) { - protected_queue.increasePriority(it, lock); - return it; + lru_it->increasePriority(lock); + return; } /// Entry is in probationary queue. /// We need to move it to protected queue. - if (it->size > protected_queue.getSizeLimit()) + if (entry.size > protected_queue.getSizeLimit()) { /// Entry size is bigger than the whole protected queue limit. /// This is only possible if protected_queue_size_limit is less than max_file_segment_size, /// which is not possible in any realistic cache configuration. - probationary_queue.increasePriority(it, lock); - return it; + lru_it->increasePriority(lock); + return; } /// Check if there is enough space in protected queue to move entry there. @@ -180,21 +160,21 @@ SLRUFileCachePriority::increasePriority(SLRUQueueIterator & it, bool is_protecte FileCacheReserveStat downgrade_stat; FinalizeEvictionFunc noop; - if (!protected_queue.collectCandidatesForEviction(it->size, downgrade_stat, downgrade_candidates, {}, noop, lock)) + if (!protected_queue.collectCandidatesForEviction(entry.size, downgrade_stat, downgrade_candidates, {}, noop, lock)) { /// We cannot make space for entry to be moved to protected queue /// (not enough releasable file segments). /// Then just increase its priority within probationary queue. - probationary_queue.increasePriority(it, lock); - return it; + lru_it->increasePriority(lock); + return; } /// Now we need to check if those "downgrade" candidates can actually /// be moved to probationary queue. const size_t size_to_downgrade = downgrade_stat.stat.releasable_count; size_t size_to_free = 0; - if (size_to_downgrade && size_to_downgrade > it->size) - size_to_free = size_to_downgrade - it->size; + if (size_to_downgrade && size_to_downgrade > entry.size) + size_to_free = size_to_downgrade - entry.size; EvictionCandidates eviction_candidates; FileCacheReserveStat stat; @@ -205,8 +185,8 @@ SLRUFileCachePriority::increasePriority(SLRUQueueIterator & it, bool is_protecte /// "downgrade" canidates cannot be moved to probationary queue, /// so entry cannot be moved to protected queue as well. /// Then just increase its priority within probationary queue. - probationary_queue.increasePriority(it, lock); - return it; + lru_it->increasePriority(lock); + return; } /// Make space for "downgrade" candidates. @@ -214,19 +194,21 @@ SLRUFileCachePriority::increasePriority(SLRUQueueIterator & it, bool is_protecte /// All checks passed, now we can move downgrade candidates to /// probationary queue and our entry to protected queue. - Entry entry = *it; - probationary_queue.remove(it, lock); + Entry entry_copy = lru_it->getEntry(); + lru_it->remove(lock); for (const auto & [key, key_candidates] : downgrade_candidates) { for (const auto & candidate : key_candidates.candidates) { - auto * candidate_it = assert_cast(candidate->getQueueIterator().get()); - probationary_queue.move(candidate_it->queue_iter, protected_queue, lock); + auto * candidate_it = assert_cast(candidate->getQueueIterator().get()); + candidate_it->lru_iterator = probationary_queue.move(*candidate_it->lru_iterator, protected_queue, lock); + candidate_it->is_protected = false; } } - return protected_queue.add(std::move(entry), lock); + iterator.lru_iterator = protected_queue.add(std::move(entry_copy), lock); + iterator.is_protected = true; } FileSegments SLRUFileCachePriority::dump(const CacheGuard::Lock & lock) @@ -237,67 +219,58 @@ FileSegments SLRUFileCachePriority::dump(const CacheGuard::Lock & lock) return res; } -SLRUFileCachePriority::SLRUFileCacheIterator::SLRUFileCacheIterator( - SLRUFileCachePriority * cache_priority_, - SLRUFileCachePriority::SLRUQueueIterator queue_iter_, - bool is_protected_) - : cache_priority(cache_priority_) - , queue_iter(queue_iter_) - , is_protected(is_protected_) -{ -} - -void SLRUFileCachePriority::SLRUFileCacheIterator::remove(const CacheGuard::Lock & lock) -{ - checkUsable(); - cache_priority->remove(queue_iter, is_protected, lock); - queue_iter = SLRUQueueIterator{}; -} - -void SLRUFileCachePriority::SLRUFileCacheIterator::invalidate() -{ - checkUsable(); - - LOG_TEST( - cache_priority->log, - "Invalidating entry in SLRU queue. Key: {}, offset: {}, previous size: {}", - queue_iter->key, queue_iter->offset, queue_iter->size); - - cache_priority->updateSize(-queue_iter->size, is_protected); - cache_priority->updateElementsCount(-1, is_protected); - queue_iter->size = 0; -} - -void SLRUFileCachePriority::SLRUFileCacheIterator::updateSize(int64_t size) -{ - checkUsable(); - - LOG_TEST( - cache_priority->log, - "Update size with {} in SLRU queue for key: {}, offset: {}, previous size: {}", - size, queue_iter->key, queue_iter->offset, queue_iter->size); - - cache_priority->updateSize(size, is_protected); - queue_iter->size += size; -} - -size_t SLRUFileCachePriority::SLRUFileCacheIterator::increasePriority(const CacheGuard::Lock & lock) -{ - checkUsable(); - queue_iter = cache_priority->increasePriority(queue_iter, is_protected, lock); - return ++queue_iter->hits; -} - -void SLRUFileCachePriority::SLRUFileCacheIterator::checkUsable() const -{ - if (queue_iter == SLRUQueueIterator{}) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to use invalid iterator"); -} - void SLRUFileCachePriority::shuffle(const CacheGuard::Lock & lock) { protected_queue.shuffle(lock); probationary_queue.shuffle(lock); } +SLRUFileCachePriority::SLRUIterator::SLRUIterator( + SLRUFileCachePriority * cache_priority_, + std::unique_ptr lru_iterator_, + bool is_protected_) + : cache_priority(cache_priority_) + , lru_iterator(std::move(lru_iterator_)) + , is_protected(is_protected_) +{ +} + +const SLRUFileCachePriority::Entry & SLRUFileCachePriority::SLRUIterator::getEntry() const +{ + checkUsable(); + return lru_iterator->getEntry(); +} + +size_t SLRUFileCachePriority::SLRUIterator::increasePriority(const CacheGuard::Lock & lock) +{ + checkUsable(); + cache_priority->increasePriority(*this, lock); + return getEntry().hits; +} + +void SLRUFileCachePriority::SLRUIterator::updateSize(int64_t size) +{ + checkUsable(); + lru_iterator->updateSize(size); +} + +void SLRUFileCachePriority::SLRUIterator::invalidate() +{ + checkUsable(); + lru_iterator->invalidate(); +} + +void SLRUFileCachePriority::SLRUIterator::remove(const CacheGuard::Lock & lock) +{ + checkUsable(); + lru_iterator->remove(lock); + lru_iterator = nullptr; +} + +void SLRUFileCachePriority::SLRUIterator::checkUsable() const +{ + if (!lru_iterator) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to use invalid iterator"); +} + } diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index a179a5285b2..079bc342d1b 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -20,9 +20,9 @@ namespace DB class SLRUFileCachePriority : public IFileCachePriority { private: - class SLRUFileCacheIterator; + class SLRUIterator; + using LRUIterator = LRUFileCachePriority::LRUIterator; using LRUQueue = std::list; - using SLRUQueueIterator = typename LRUQueue::iterator; public: SLRUFileCachePriority(size_t max_size_, size_t max_elements_, double size_ratio); @@ -46,28 +46,23 @@ public: FileSegments dump(const CacheGuard::Lock &) override; private: - void updateElementsCount(int64_t num, bool is_protected); - void updateSize(int64_t size, bool is_protected); - LRUFileCachePriority protected_queue; LRUFileCachePriority probationary_queue; - Poco::Logger * log = &Poco::Logger::get("SLRUFileCachePriority"); - SLRUQueueIterator remove(SLRUQueueIterator it, bool is_protected, const CacheGuard::Lock & lock); - SLRUQueueIterator increasePriority(SLRUQueueIterator & it, bool is_protected, const CacheGuard::Lock & lock); + void increasePriority(SLRUIterator & iterator, const CacheGuard::Lock & lock); }; -class SLRUFileCachePriority::SLRUFileCacheIterator : public IFileCachePriority::IIterator +class SLRUFileCachePriority::SLRUIterator : public IFileCachePriority::IIterator { friend class SLRUFileCachePriority; public: - SLRUFileCacheIterator( + SLRUIterator( SLRUFileCachePriority * cache_priority_, - SLRUFileCachePriority::SLRUQueueIterator queue_iter_, + std::unique_ptr lru_iterator_, bool is_protected_); - const Entry & getEntry() const override { return *queue_iter; } + const Entry & getEntry() const override; size_t increasePriority(const CacheGuard::Lock &) override; @@ -81,8 +76,8 @@ private: void checkUsable() const; SLRUFileCachePriority * cache_priority; - mutable SLRUFileCachePriority::SLRUQueueIterator queue_iter; - const bool is_protected; + mutable std::unique_ptr lru_iterator; + bool is_protected; }; } From b5eff71959fd1e6873f7686a69bab48933d51e9d Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 23 Nov 2023 11:18:56 +0100 Subject: [PATCH 035/331] Add a test --- src/Interpreters/Cache/FileSegment.cpp | 3 +- .../Cache/SLRUFileCachePriority.h | 5 +- ...lru_file_cache.cpp => gtest_filecache.cpp} | 155 +++++++++++++++++- 3 files changed, 158 insertions(+), 5 deletions(-) rename src/Interpreters/tests/{gtest_lru_file_cache.cpp => gtest_filecache.cpp} (86%) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 8216d7a9a81..10c277772be 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -844,7 +844,8 @@ FileSegmentPtr FileSegment::getSnapshot(const FileSegmentPtr & file_segment) file_segment->offset(), file_segment->range().size(), State::DETACHED, - CreateFileSegmentSettings(file_segment->getKind(), file_segment->is_unbound)); + CreateFileSegmentSettings(file_segment->getKind(), file_segment->is_unbound), + false, file_segment->cache, file_segment->key_metadata, file_segment->queue_iterator); snapshot->hits_count = file_segment->getHitsCount(); snapshot->downloaded_size = file_segment->getDownloadedSize(); diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index 079bc342d1b..f03a145c533 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -20,11 +20,12 @@ namespace DB class SLRUFileCachePriority : public IFileCachePriority { private: - class SLRUIterator; using LRUIterator = LRUFileCachePriority::LRUIterator; using LRUQueue = std::list; public: + class SLRUIterator; + SLRUFileCachePriority(size_t max_size_, size_t max_elements_, double size_ratio); size_t getSize(const CacheGuard::Lock & lock) const override; @@ -72,6 +73,8 @@ public: void updateSize(int64_t size) override; + bool isProtected() const { return is_protected; } + private: void checkUsable() const; diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_filecache.cpp similarity index 86% rename from src/Interpreters/tests/gtest_lru_file_cache.cpp rename to src/Interpreters/tests/gtest_filecache.cpp index ab2a128de34..f9a18afa618 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_filecache.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -124,6 +125,42 @@ void assertEqual(const FileSegmentsHolderPtr & file_segments, const Ranges & exp assertEqual(file_segments->begin(), file_segments->end(), file_segments->size(), expected_ranges, expected_states); } +void assertProtectedOrProbationary(const FileSegments & file_segments, const Ranges & expected, bool assert_protected) +{ + std::cerr << "File segments: "; + std::vector res; + for (const auto & f : file_segments) + { + std::cerr << f->range().toString() << ", "; + if (auto it = f->getQueueIterator()) + { + if (auto * slru_it = dynamic_cast(it.get())) + { + if ((slru_it->isProtected() && assert_protected) || (!slru_it->isProtected() && !assert_protected)) + { + res.push_back(f->range()); + } + } + } + } + + ASSERT_EQ(res.size(), expected.size()); + for (size_t i = 0; i < res.size(); ++i) + { + ASSERT_EQ(res[i], expected[i]); + } +} + +void assertProtected(const FileSegments & file_segments, const Ranges & expected) +{ + assertProtectedOrProbationary(file_segments, expected, true); +} + +void assertProbationary(const FileSegments & file_segments, const Ranges & expected) +{ + assertProtectedOrProbationary(file_segments, expected, false); +} + FileSegment & get(const HolderPtr & holder, int i) { auto it = std::next(holder->begin(), i); @@ -134,7 +171,7 @@ FileSegment & get(const HolderPtr & holder, int i) void download(FileSegment & file_segment) { - std::cerr << "Downloading range " << file_segment.range().toString() << "\n"; + std::cerr << "\nDownloading range " << file_segment.range().toString() << "\n"; ASSERT_EQ(file_segment.getOrSetDownloader(), FileSegment::getCallerId()); ASSERT_EQ(file_segment.state(), State::DOWNLOADING); @@ -167,7 +204,7 @@ void download(const HolderPtr & holder) void increasePriority(const HolderPtr & holder) { for (auto & it : *holder) - it->use(); + it->increasePriority(); } class FileCacheTest : public ::testing::Test @@ -216,7 +253,7 @@ public: pcg64 rng; }; -TEST_F(FileCacheTest, get) +TEST_F(FileCacheTest, LRUPolicy) { DB::ThreadStatus thread_status; @@ -1040,3 +1077,115 @@ TEST_F(FileCacheTest, TemporaryDataReadBufferSize) ASSERT_EQ(stream.getSize(), 62); } } + +TEST_F(FileCacheTest, SLRUPolicy) +{ + DB::ThreadStatus thread_status; + std::string query_id = "query_id"; /// To work with cache need query_id and query context. + + Poco::XML::DOMParser dom_parser; + std::string xml(R"CONFIG( +)CONFIG"); + Poco::AutoPtr document = dom_parser.parseString(xml); + Poco::AutoPtr config = new Poco::Util::XMLConfiguration(document); + getMutableContext().context->setConfig(config); + + auto query_context = DB::Context::createCopy(getContext().context); + query_context->makeQueryContext(); + query_context->setCurrentQueryId(query_id); + chassert(&DB::CurrentThread::get() == &thread_status); + DB::CurrentThread::QueryScope query_scope_holder(query_context); + + DB::FileCacheSettings settings; + settings.base_path = cache_base_path; + settings.max_size = 40; + settings.max_elements = 6; + settings.boundary_alignment = 1; + + settings.cache_policy = "SLRU"; + settings.slru_size_ratio = 0.5; + + const size_t file_size = -1; // the value doesn't really matter because boundary_alignment == 1. + size_t file_cache_name = 0; + + { + auto cache = DB::FileCache(std::to_string(++file_cache_name), settings); + cache.initialize(); + auto key = cache.createKeyForPath("key1"); + + auto add_range = [&](size_t offset, size_t size) + { + std::cerr << "Add [" << offset << ", " << offset + size - 1 << "]" << std::endl; + + auto holder = cache.getOrSet(key, offset, size, file_size, {}); + assertEqual(holder, { Range(offset, offset + size - 1) }, { State::EMPTY }); + download(holder->front()); + assertEqual(holder, { Range(offset, offset + size - 1) }, { State::DOWNLOADED }); + }; + + auto check_covering_range = [&](size_t offset, size_t size, Ranges covering_ranges) + { + auto holder = cache.getOrSet(key, offset, size, file_size, {}); + std::vector states(covering_ranges.size(), State::DOWNLOADED); + assertEqual(holder, covering_ranges, states); + increasePriority(holder); + }; + + add_range(0, 10); + add_range(10, 5); + + assertEqual(cache.getSnapshot(key), { Range(0, 9), Range(10, 14) }); + assertEqual(cache.dumpQueue(), { Range(0, 9), Range(10, 14) }); + + ASSERT_EQ(cache.getFileSegmentsNum(), 2); + ASSERT_EQ(cache.getUsedCacheSize(), 15); + + assertProbationary(cache.dumpQueue(), { Range(0, 9), Range(10, 14) }); + assertProtected(cache.dumpQueue(), Ranges{}); + + check_covering_range(9, 1, { Range(0, 9) }); + assertEqual(cache.dumpQueue(), { Range(10, 14), Range(0, 9) }); + + check_covering_range(10, 1, { Range(10, 14) }); + assertEqual(cache.dumpQueue(), { Range(0, 9), Range(10, 14) }); + + assertProbationary(cache.dumpQueue(), Ranges{}); + assertProtected(cache.dumpQueue(), { Range(0, 9), Range(10, 14) }); + + add_range(17, 4); + assertEqual(cache.dumpQueue(), { Range(17, 20), Range(0, 9), Range(10, 14) }); + + add_range(24, 3); + assertEqual(cache.dumpQueue(), { Range(17, 20), Range(24, 26), Range(0, 9), Range(10, 14) }); + + add_range(27, 1); + assertEqual(cache.dumpQueue(), { Range(17, 20), Range(24, 26), Range(27, 27), Range(0, 9), Range(10, 14) }); + + assertProbationary(cache.dumpQueue(), { Range(17, 20), Range(24, 26), Range(27, 27) }); + assertProtected(cache.dumpQueue(), { Range(0, 9), Range(10, 14) }); + + assertEqual(cache.getSnapshot(key), { Range(0, 9), Range(10, 14), Range(17, 20), Range(24, 26), Range(27, 27) }); + ASSERT_EQ(cache.getFileSegmentsNum(), 5); + ASSERT_EQ(cache.getUsedCacheSize(), 23); + + add_range(28, 3); + assertEqual(cache.dumpQueue(), { Range(24, 26), Range(27, 27), Range(28, 30), Range(0, 9), Range(10, 14) }); + + assertProbationary(cache.dumpQueue(), { Range(24, 26), Range(27, 27), Range(28, 30) }); + assertProtected(cache.dumpQueue(), { Range(0, 9), Range(10, 14) }); + + check_covering_range(4, 1, { Range(0, 9) }); + + assertProbationary(cache.dumpQueue(), { Range(24, 26), Range(27, 27), Range(28, 30) }); + assertProtected(cache.dumpQueue(), { Range(10, 14), Range(0, 9) }); + + check_covering_range(27, 3, { Range(27, 27), Range(28, 30) }); + + assertProbationary(cache.dumpQueue(), { Range(24, 26), Range(10, 14) }); + assertProtected(cache.dumpQueue(), { Range(0, 9), Range(27, 27), Range(28, 30) }); + + assertEqual(cache.getSnapshot(key), { Range(0, 9), Range(10, 14), Range(24, 26), Range(27, 27), Range(28, 30) }); + ASSERT_EQ(cache.getFileSegmentsNum(), 5); + ASSERT_EQ(cache.getUsedCacheSize(), 22); + } +} From dc5e5f3b2046e14a2cce429c389cc5b49a98a99c Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 23 Nov 2023 13:16:04 +0100 Subject: [PATCH 036/331] Refactoring --- src/Interpreters/Cache/FileCache.cpp | 2 +- src/Interpreters/Cache/FileCache.h | 3 +- src/Interpreters/Cache/FileSegment.cpp | 12 ++- src/Interpreters/Cache/FileSegment.h | 8 +- src/Interpreters/Cache/IFileCachePriority.h | 13 +-- .../Cache/LRUFileCachePriority.cpp | 59 ++++++------ src/Interpreters/Cache/LRUFileCachePriority.h | 16 ++-- src/Interpreters/Cache/Metadata.h | 2 +- src/Interpreters/Cache/QueryLimit.cpp | 2 +- src/Interpreters/Cache/QueryLimit.h | 5 +- .../Cache/SLRUFileCachePriority.cpp | 91 +++++++++---------- .../Cache/SLRUFileCachePriority.h | 8 +- 12 files changed, 112 insertions(+), 109 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index b47e84895c5..c29f01745e4 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -1046,7 +1046,7 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir) } bool limits_satisfied; - IFileCachePriority::Iterator cache_it; + IFileCachePriority::IteratorPtr cache_it; { auto lock = lockCache(); limits_satisfied = (size_limit == 0 || main_priority->getSize(lock) + size <= size_limit) diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 879ee2c133b..493cde9652a 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -74,7 +74,6 @@ public: using QueryLimit = DB::FileCacheQueryLimit; using Priority = IFileCachePriority; using PriorityEntry = IFileCachePriority::Entry; - using PriorityIterator = IFileCachePriority::Iterator; FileCache(const std::string & cache_name, const FileCacheSettings & settings); @@ -205,7 +204,7 @@ private: const size_t queue_size; std::unique_ptr queue; - using Records = std::unordered_map; + using Records = std::unordered_map; Records records; }; diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index de23406cf4a..3ad76131fcc 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -54,7 +54,7 @@ FileSegment::FileSegment( bool background_download_enabled_, FileCache * cache_, std::weak_ptr key_metadata_, - Priority::Iterator queue_iterator_) + Priority::IteratorPtr queue_iterator_) : file_key(key_) , segment_range(offset_, offset_ + size_ - 1) , segment_kind(settings.kind) @@ -146,13 +146,13 @@ size_t FileSegment::getReservedSize() const return reserved_size; } -FileSegment::Priority::Iterator FileSegment::getQueueIterator() const +FileSegment::Priority::IteratorPtr FileSegment::getQueueIterator() const { auto lock = lockFileSegment(); return queue_iterator; } -void FileSegment::setQueueIterator(Priority::Iterator iterator) +void FileSegment::setQueueIterator(Priority::IteratorPtr iterator) { auto lock = lockFileSegment(); if (queue_iterator) @@ -775,7 +775,7 @@ bool FileSegment::assertCorrectness() const bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock &) const { - auto check_iterator = [this](const Priority::Iterator & it) + auto check_iterator = [this](const Priority::IteratorPtr & it) { UNUSED(this); if (!it) @@ -917,6 +917,10 @@ void FileSegment::increasePriority() return; } + /// Priority can be increased only for downloaded file segments. + if (download_state != State::DOWNLOADED) + return; + auto it = getQueueIterator(); if (it) { diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 59008f2e9c9..42b8f10c943 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -113,7 +113,7 @@ public: bool background_download_enabled_ = false, FileCache * cache_ = nullptr, std::weak_ptr key_metadata_ = std::weak_ptr(), - Priority::Iterator queue_iterator_ = Priority::Iterator{}); + Priority::IteratorPtr queue_iterator_ = nullptr); ~FileSegment() = default; @@ -219,9 +219,9 @@ public: FileSegmentGuard::Lock lock() const { return segment_guard.lock(); } - Priority::Iterator getQueueIterator() const; + Priority::IteratorPtr getQueueIterator() const; - void setQueueIterator(Priority::Iterator iterator); + void setQueueIterator(Priority::IteratorPtr iterator); KeyMetadataPtr tryGetKeyMetadata() const; @@ -309,7 +309,7 @@ private: mutable FileSegmentGuard segment_guard; std::weak_ptr key_metadata; - mutable Priority::Iterator queue_iterator; /// Iterator is put here on first reservation attempt, if successful. + mutable Priority::IteratorPtr queue_iterator; /// Iterator is put here on first reservation attempt, if successful. FileCache * cache; std::condition_variable cv; diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index 1703ed09139..fe5de21ec48 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -12,12 +12,10 @@ namespace DB struct FileCacheReserveStat; class EvictionCandidates; -/// IFileCachePriority is used to maintain the priority of cached data. class IFileCachePriority : private boost::noncopyable { public: using Key = FileCacheKey; - using KeyAndOffset = FileCacheKeyAndOffset; struct Entry { @@ -32,10 +30,10 @@ public: size_t hits = 0; }; - class IIterator + class Iterator { public: - virtual ~IIterator() = default; + virtual ~Iterator() = default; virtual const Entry & getEntry() const = 0; @@ -47,7 +45,7 @@ public: virtual void invalidate() = 0; }; - using Iterator = std::shared_ptr; + using IteratorPtr = std::shared_ptr; IFileCachePriority(size_t max_size_, size_t max_elements_); @@ -61,19 +59,18 @@ public: virtual size_t getElementsCount(const CacheGuard::Lock &) const = 0; - virtual Iterator add(KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) = 0; + virtual IteratorPtr add(KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) = 0; virtual void shuffle(const CacheGuard::Lock &) = 0; virtual FileSegments dump(const CacheGuard::Lock &) = 0; using FinalizeEvictionFunc = std::function; - virtual bool collectCandidatesForEviction( size_t size, FileCacheReserveStat & stat, EvictionCandidates & res, - IFileCachePriority::Iterator it, + IFileCachePriority::IteratorPtr reservee, FinalizeEvictionFunc & finalize_eviction_func, const CacheGuard::Lock &) = 0; diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index 5cd44a67d89..625be890cd3 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -26,7 +26,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -IFileCachePriority::Iterator LRUFileCachePriority::add( +IFileCachePriority::IteratorPtr LRUFileCachePriority::add( KeyMetadataPtr key_metadata, size_t offset, size_t size, @@ -67,7 +67,7 @@ std::unique_ptr LRUFileCachePriority::add(Ent entry.key, entry.offset, entry.size, current_size, size_limit); } - auto it = queue.insert(queue.end(), entry); + auto iterator = queue.insert(queue.end(), entry); updateSize(entry.size); updateElementsCount(1); @@ -76,7 +76,7 @@ std::unique_ptr LRUFileCachePriority::add(Ent log, "Added entry into LRU queue, key: {}, offset: {}, size: {}", entry.key, entry.offset, entry.size); - return std::make_unique(this, it); + return std::make_unique(this, iterator); } LRUFileCachePriority::LRUQueue::iterator LRUFileCachePriority::remove(LRUQueue::iterator it, const CacheGuard::Lock &) @@ -107,9 +107,11 @@ void LRUFileCachePriority::updateElementsCount(int64_t num) CurrentMetrics::add(CurrentMetrics::FilesystemCacheElements, num); } - -LRUFileCachePriority::LRUIterator::LRUIterator(LRUFileCachePriority * cache_priority_, LRUQueue::iterator queue_iter_) - : cache_priority(cache_priority_), queue_iter(queue_iter_) +LRUFileCachePriority::LRUIterator::LRUIterator( + LRUFileCachePriority * cache_priority_, + LRUQueue::iterator iterator_) + : cache_priority(cache_priority_) + , iterator(iterator_) { } @@ -166,7 +168,11 @@ bool LRUFileCachePriority::canFit(size_t size, const CacheGuard::Lock & lock) co return canFit(size, 0, 0, lock); } -bool LRUFileCachePriority::canFit(size_t size, size_t released_size_assumption, size_t released_elements_assumption, const CacheGuard::Lock &) const +bool LRUFileCachePriority::canFit( + size_t size, + size_t released_size_assumption, + size_t released_elements_assumption, + const CacheGuard::Lock &) const { return (max_size == 0 || (current_size + size - released_size_assumption <= max_size)) && (max_elements == 0 || current_elements_num + 1 - released_elements_assumption <= max_elements); @@ -176,7 +182,7 @@ bool LRUFileCachePriority::collectCandidatesForEviction( size_t size, FileCacheReserveStat & stat, EvictionCandidates & res, - IFileCachePriority::Iterator, + IFileCachePriority::IteratorPtr, FinalizeEvictionFunc &, const CacheGuard::Lock & lock) { @@ -217,7 +223,8 @@ bool LRUFileCachePriority::collectCandidatesForEviction( return can_fit(); } -std::unique_ptr LRUFileCachePriority::move(LRUIterator & it, LRUFileCachePriority & other, const CacheGuard::Lock &) +std::unique_ptr +LRUFileCachePriority::move(LRUIterator & it, LRUFileCachePriority & other, const CacheGuard::Lock &) { const auto & entry = it.getEntry(); if (entry.size == 0) @@ -240,14 +247,14 @@ std::unique_ptr LRUFileCachePriority::move(LR } #endif - queue.splice(queue.end(), other.queue, it.queue_iter); + queue.splice(queue.end(), other.queue, it.iterator); updateSize(entry.size); updateElementsCount(1); other.updateSize(-entry.size); other.updateElementsCount(-1); - return std::make_unique(this, it.queue_iter); + return std::make_unique(this, it.iterator); } FileSegments LRUFileCachePriority::dump(const CacheGuard::Lock & lock) @@ -263,48 +270,48 @@ FileSegments LRUFileCachePriority::dump(const CacheGuard::Lock & lock) void LRUFileCachePriority::LRUIterator::remove(const CacheGuard::Lock & lock) { - checkUsable(); - cache_priority->remove(queue_iter, lock); - queue_iter = LRUQueue::iterator{}; + assertValid(); + cache_priority->remove(iterator, lock); + iterator = LRUQueue::iterator{}; } void LRUFileCachePriority::LRUIterator::invalidate() { - checkUsable(); + assertValid(); LOG_TEST( cache_priority->log, "Invalidating entry in LRU queue. Key: {}, offset: {}, previous size: {}", - queue_iter->key, queue_iter->offset, queue_iter->size); + iterator->key, iterator->offset, iterator->size); - cache_priority->updateSize(-queue_iter->size); + cache_priority->updateSize(-iterator->size); cache_priority->updateElementsCount(-1); - queue_iter->size = 0; + iterator->size = 0; } void LRUFileCachePriority::LRUIterator::updateSize(int64_t size) { - checkUsable(); + assertValid(); LOG_TEST( cache_priority->log, "Update size with {} in LRU queue for key: {}, offset: {}, previous size: {}", - size, queue_iter->key, queue_iter->offset, queue_iter->size); + size, iterator->key, iterator->offset, iterator->size); cache_priority->updateSize(size); - queue_iter->size += size; + iterator->size += size; } size_t LRUFileCachePriority::LRUIterator::increasePriority(const CacheGuard::Lock &) { - checkUsable(); - cache_priority->queue.splice(cache_priority->queue.end(), cache_priority->queue, queue_iter); - return ++queue_iter->hits; + assertValid(); + cache_priority->queue.splice(cache_priority->queue.end(), cache_priority->queue, iterator); + return ++iterator->hits; } -void LRUFileCachePriority::LRUIterator::checkUsable() const +void LRUFileCachePriority::LRUIterator::assertValid() const { - if (queue_iter == LRUQueue::iterator{}) + if (iterator == LRUQueue::iterator{}) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to use invalid iterator"); } diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index acc8a5a9a76..63b93de76e4 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -11,7 +11,7 @@ namespace DB /// Based on the LRU algorithm implementation, the record with the lowest priority is stored at /// the head of the queue, and the record with the highest priority is stored at the tail. -class LRUFileCachePriority : public IFileCachePriority +class LRUFileCachePriority final : public IFileCachePriority { private: class LRUIterator; @@ -25,13 +25,13 @@ public: size_t getElementsCount(const CacheGuard::Lock &) const override { return current_elements_num; } - Iterator add(KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) override; + IteratorPtr add(KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) override; bool collectCandidatesForEviction( size_t size, FileCacheReserveStat & stat, EvictionCandidates & res, - IFileCachePriority::Iterator it, + IFileCachePriority::IteratorPtr reservee, FinalizeEvictionFunc & finalize_eviction_func, const CacheGuard::Lock &) override; @@ -71,14 +71,14 @@ private: std::unique_ptr add(Entry && entry, const CacheGuard::Lock &); }; -class LRUFileCachePriority::LRUIterator : public IFileCachePriority::IIterator +class LRUFileCachePriority::LRUIterator : public IFileCachePriority::Iterator { friend class LRUFileCachePriority; friend class SLRUFileCachePriority; public: - LRUIterator(LRUFileCachePriority * cache_priority_, LRUQueue::iterator queue_iter_); + LRUIterator(LRUFileCachePriority * cache_priority_, LRUQueue::iterator iterator_); - const Entry & getEntry() const override { return *queue_iter; } + const Entry & getEntry() const override { return *iterator; } size_t increasePriority(const CacheGuard::Lock &) override; @@ -89,10 +89,10 @@ public: void updateSize(int64_t size) override; private: - void checkUsable() const; + void assertValid() const; LRUFileCachePriority * cache_priority; - mutable LRUQueue::iterator queue_iter; + mutable LRUQueue::iterator iterator; }; } diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index d68af87e8b4..c487e7a7b16 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -29,7 +29,7 @@ struct FileSegmentMetadata : private boost::noncopyable bool evicting() const { return removal_candidate.load(); } - Priority::Iterator getQueueIterator() const { return file_segment->getQueueIterator(); } + Priority::IteratorPtr getQueueIterator() const { return file_segment->getQueueIterator(); } FileSegmentPtr file_segment; std::atomic removal_candidate{false}; diff --git a/src/Interpreters/Cache/QueryLimit.cpp b/src/Interpreters/Cache/QueryLimit.cpp index 8b5b42cca1a..109e94b2971 100644 --- a/src/Interpreters/Cache/QueryLimit.cpp +++ b/src/Interpreters/Cache/QueryLimit.cpp @@ -95,7 +95,7 @@ void FileCacheQueryLimit::QueryContext::remove( records.erase({key, offset}); } -IFileCachePriority::Iterator FileCacheQueryLimit::QueryContext::tryGet( +IFileCachePriority::IteratorPtr FileCacheQueryLimit::QueryContext::tryGet( const Key & key, size_t offset, const CacheGuard::Lock &) diff --git a/src/Interpreters/Cache/QueryLimit.h b/src/Interpreters/Cache/QueryLimit.h index c252cd2dccc..cab1ccc63ba 100644 --- a/src/Interpreters/Cache/QueryLimit.h +++ b/src/Interpreters/Cache/QueryLimit.h @@ -27,7 +27,6 @@ public: public: using Key = FileCacheKey; using Priority = IFileCachePriority; - using PriorityIterator = IFileCachePriority::Iterator; QueryContext(size_t query_cache_size, bool recache_on_query_limit_exceeded_); @@ -36,7 +35,7 @@ public: bool recacheOnFileCacheQueryLimitExceeded() const { return recache_on_query_limit_exceeded; } - Priority::Iterator tryGet( + Priority::IteratorPtr tryGet( const Key & key, size_t offset, const CacheGuard::Lock &); @@ -53,7 +52,7 @@ public: const CacheGuard::Lock &); private: - using Records = std::unordered_map; + using Records = std::unordered_map; Records records; LRUFileCachePriority priority; const bool recache_on_query_limit_exceeded; diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index ee8da79ee6f..ee311232515 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -52,38 +52,37 @@ size_t SLRUFileCachePriority::getElementsCount(const CacheGuard::Lock & lock) co return protected_queue.getElementsCount(lock) + probationary_queue.getElementsCount(lock); } -IFileCachePriority::Iterator SLRUFileCachePriority::add( +IFileCachePriority::IteratorPtr SLRUFileCachePriority::add( KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock & lock) { - auto it = probationary_queue.add(Entry(key_metadata->key, offset, size, key_metadata), lock); - return std::make_shared(this, std::move(it), false); + auto lru_iterator = probationary_queue.add(Entry(key_metadata->key, offset, size, key_metadata), lock); + return std::make_shared(this, std::move(lru_iterator), false); } bool SLRUFileCachePriority::collectCandidatesForEviction( size_t size, FileCacheReserveStat & stat, EvictionCandidates & res, - IFileCachePriority::Iterator it, + IFileCachePriority::IteratorPtr reservee, FinalizeEvictionFunc & finalize_eviction_func, const CacheGuard::Lock & lock) { - /// `it` is a pointer to entry we want to evict in favour of. /// If `it` is nullptr, then it is the first space reservation attempt /// for a corresponding file segment, so it will be directly put into probationary queue. - if (!it) + if (!reservee) { - return probationary_queue.collectCandidatesForEviction(size, stat, res, it, finalize_eviction_func, lock); + return probationary_queue.collectCandidatesForEviction(size, stat, res, reservee, finalize_eviction_func, lock); } /// If `it` not nullptr (e.g. is already in some queue), /// we need to check in which queue (protected/probationary) it currently is /// (in order to know where we need to free space). - if (!assert_cast(it.get())->is_protected) + if (!assert_cast(reservee.get())->is_protected) { - return probationary_queue.collectCandidatesForEviction(size, stat, res, it, finalize_eviction_func, lock); + return probationary_queue.collectCandidatesForEviction(size, stat, res, reservee, finalize_eviction_func, lock); } /// Entry is in protected queue. @@ -100,13 +99,13 @@ bool SLRUFileCachePriority::collectCandidatesForEviction( FileCacheReserveStat downgrade_stat; FinalizeEvictionFunc noop; - if (!protected_queue.collectCandidatesForEviction(size, downgrade_stat, *downgrade_candidates, it, noop, lock)) + if (!protected_queue.collectCandidatesForEviction(size, downgrade_stat, *downgrade_candidates, reservee, noop, lock)) return false; const size_t size_to_downgrade = downgrade_stat.stat.releasable_size; if (!probationary_queue.canFit(size_to_downgrade, lock) - && !probationary_queue.collectCandidatesForEviction(size_to_downgrade, stat, res, it, noop, lock)) + && !probationary_queue.collectCandidatesForEviction(size_to_downgrade, stat, res, reservee, noop, lock)) return false; finalize_eviction_func = [=, this](const CacheGuard::Lock & lk) mutable @@ -127,75 +126,73 @@ bool SLRUFileCachePriority::collectCandidatesForEviction( void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const CacheGuard::Lock & lock) { - auto & lru_it = iterator.lru_iterator; - const bool is_protected = iterator.is_protected; - const auto & entry = lru_it->getEntry(); - - /// If entry (`it` is the pointer to the entry) is already in protected queue, + /// If entry is already in protected queue, /// we only need to increase its priority within the protected queue. - if (is_protected) + if (iterator.is_protected) { - lru_it->increasePriority(lock); + iterator.lru_iterator->increasePriority(lock); return; } /// Entry is in probationary queue. /// We need to move it to protected queue. - if (entry.size > protected_queue.getSizeLimit()) + const size_t size = iterator.getEntry().size; + if (size > protected_queue.getSizeLimit()) { /// Entry size is bigger than the whole protected queue limit. /// This is only possible if protected_queue_size_limit is less than max_file_segment_size, /// which is not possible in any realistic cache configuration. - lru_it->increasePriority(lock); + iterator.lru_iterator->increasePriority(lock); return; } /// Check if there is enough space in protected queue to move entry there. /// If not - we need to "downgrade" lowest priority entries from protected /// queue to probationary queue. - /// The amount of such "downgraded" entries is equal to the amount - /// required to make space for entry we want to insert. EvictionCandidates downgrade_candidates; FileCacheReserveStat downgrade_stat; FinalizeEvictionFunc noop; - if (!protected_queue.collectCandidatesForEviction(entry.size, downgrade_stat, downgrade_candidates, {}, noop, lock)) + if (!protected_queue.collectCandidatesForEviction(size, downgrade_stat, downgrade_candidates, {}, noop, lock)) { /// We cannot make space for entry to be moved to protected queue /// (not enough releasable file segments). /// Then just increase its priority within probationary queue. - lru_it->increasePriority(lock); + iterator.lru_iterator->increasePriority(lock); return; } + /// The amount of such "downgraded" entries is equal to the amount + /// required to make space for entry we want to insert. + const size_t size_to_downgrade = downgrade_stat.stat.releasable_count; + size_t size_to_free = 0; + if (size_to_downgrade && size_to_downgrade > size) + size_to_free = size_to_downgrade - size; + /// Now we need to check if those "downgrade" candidates can actually /// be moved to probationary queue. - const size_t size_to_downgrade = downgrade_stat.stat.releasable_count; - size_t size_to_free = 0; - if (size_to_downgrade && size_to_downgrade > entry.size) - size_to_free = size_to_downgrade - entry.size; - EvictionCandidates eviction_candidates; FileCacheReserveStat stat; - if (size_to_free - && !probationary_queue.collectCandidatesForEviction(size_to_free, stat, eviction_candidates, {}, noop, lock)) + if (size_to_free) { - /// "downgrade" canidates cannot be moved to probationary queue, - /// so entry cannot be moved to protected queue as well. - /// Then just increase its priority within probationary queue. - lru_it->increasePriority(lock); - return; + if (!probationary_queue.collectCandidatesForEviction(size_to_free, stat, eviction_candidates, {}, noop, lock)) + { + /// "downgrade" canidates cannot be moved to probationary queue, + /// so entry cannot be moved to protected queue as well. + /// Then just increase its priority within probationary queue. + iterator.lru_iterator->increasePriority(lock); + return; + } + /// Make space for "downgrade" candidates. + eviction_candidates.evict(nullptr, lock); } - /// Make space for "downgrade" candidates. - eviction_candidates.evict(nullptr, lock); - /// All checks passed, now we can move downgrade candidates to /// probationary queue and our entry to protected queue. - Entry entry_copy = lru_it->getEntry(); - lru_it->remove(lock); + Entry entry_copy = iterator.getEntry(); + iterator.lru_iterator->remove(lock); for (const auto & [key, key_candidates] : downgrade_candidates) { @@ -237,37 +234,37 @@ SLRUFileCachePriority::SLRUIterator::SLRUIterator( const SLRUFileCachePriority::Entry & SLRUFileCachePriority::SLRUIterator::getEntry() const { - checkUsable(); + assertValid(); return lru_iterator->getEntry(); } size_t SLRUFileCachePriority::SLRUIterator::increasePriority(const CacheGuard::Lock & lock) { - checkUsable(); + assertValid(); cache_priority->increasePriority(*this, lock); return getEntry().hits; } void SLRUFileCachePriority::SLRUIterator::updateSize(int64_t size) { - checkUsable(); + assertValid(); lru_iterator->updateSize(size); } void SLRUFileCachePriority::SLRUIterator::invalidate() { - checkUsable(); + assertValid(); lru_iterator->invalidate(); } void SLRUFileCachePriority::SLRUIterator::remove(const CacheGuard::Lock & lock) { - checkUsable(); + assertValid(); lru_iterator->remove(lock); lru_iterator = nullptr; } -void SLRUFileCachePriority::SLRUIterator::checkUsable() const +void SLRUFileCachePriority::SLRUIterator::assertValid() const { if (!lru_iterator) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to use invalid iterator"); diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index f03a145c533..7b323353d70 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -32,13 +32,13 @@ public: size_t getElementsCount(const CacheGuard::Lock &) const override; - Iterator add(KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) override; + IteratorPtr add(KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) override; bool collectCandidatesForEviction( size_t size, FileCacheReserveStat & stat, EvictionCandidates & res, - IFileCachePriority::Iterator, + IFileCachePriority::IteratorPtr reservee, FinalizeEvictionFunc & finalize_eviction_func, const CacheGuard::Lock &) override; @@ -54,7 +54,7 @@ private: void increasePriority(SLRUIterator & iterator, const CacheGuard::Lock & lock); }; -class SLRUFileCachePriority::SLRUIterator : public IFileCachePriority::IIterator +class SLRUFileCachePriority::SLRUIterator : public IFileCachePriority::Iterator { friend class SLRUFileCachePriority; public: @@ -76,7 +76,7 @@ public: bool isProtected() const { return is_protected; } private: - void checkUsable() const; + void assertValid() const; SLRUFileCachePriority * cache_priority; mutable std::unique_ptr lru_iterator; From 0f09a69b0528f1cd36eb614dacbbb2223317ac9a Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 23 Nov 2023 15:57:32 +0100 Subject: [PATCH 037/331] Fix style check --- src/Interpreters/Cache/FileCache.cpp | 6 +----- src/Interpreters/Cache/FileCache.h | 5 ----- src/Interpreters/Cache/SLRUFileCachePriority.cpp | 8 +------- src/Interpreters/Cache/SLRUFileCachePriority.h | 8 -------- 4 files changed, 2 insertions(+), 25 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index c29f01745e4..2e12d6ef9bf 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -22,13 +22,8 @@ namespace fs = std::filesystem; namespace ProfileEvents { extern const Event FilesystemCacheLoadMetadataMicroseconds; - extern const Event FilesystemCacheEvictedBytes; - extern const Event FilesystemCacheEvictedFileSegments; - extern const Event FilesystemCacheEvictionSkippedFileSegments; - extern const Event FilesystemCacheEvictionTries; extern const Event FilesystemCacheLockCacheMicroseconds; extern const Event FilesystemCacheReserveMicroseconds; - extern const Event FilesystemCacheEvictMicroseconds; extern const Event FilesystemCacheGetOrSetMicroseconds; extern const Event FilesystemCacheGetMicroseconds; } @@ -53,6 +48,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; } FileCache::FileCache(const std::string & cache_name, const FileCacheSettings & settings) diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 493cde9652a..097a63b0abe 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -22,11 +22,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - /// Track acquired space in cache during reservation /// to make error messages when no space left more informative. struct FileCacheReserveStat diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index ee311232515..a9e017c62e4 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -5,13 +5,7 @@ #include #include #include -#include -namespace CurrentMetrics -{ - extern const Metric FilesystemCacheSize; - extern const Metric FilesystemCacheElements; -} namespace DB { @@ -179,7 +173,7 @@ void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const Cach { if (!probationary_queue.collectCandidatesForEviction(size_to_free, stat, eviction_candidates, {}, noop, lock)) { - /// "downgrade" canidates cannot be moved to probationary queue, + /// "downgrade" candidates cannot be moved to probationary queue, /// so entry cannot be moved to protected queue as well. /// Then just increase its priority within probationary queue. iterator.lru_iterator->increasePriority(lock); diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index 7b323353d70..46e8f37819f 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -1,16 +1,8 @@ #pragma once -#include -#include -#include #include #include -#include -namespace CurrentMetrics -{ - extern const Metric FilesystemCacheSizeLimit; -} namespace DB { From 89656723fbd332e240dd40d630fa8a87cf22f0a2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Nov 2023 16:02:33 +0100 Subject: [PATCH 038/331] 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 039/331] 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 040/331] 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 041/331] 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 07b11534bc324f73488da6d8d037dc941bead7d6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 23 Nov 2023 16:36:18 +0100 Subject: [PATCH 042/331] Tiny refactoring --- src/Interpreters/Cache/FileCache.cpp | 3 +- src/Interpreters/Cache/FileCacheSettings.cpp | 4 +++ .../Cache/LRUFileCachePriority.cpp | 31 +++++++++++++++---- src/Interpreters/Cache/LRUFileCachePriority.h | 9 ++++-- .../Cache/SLRUFileCachePriority.cpp | 30 +++++++++--------- .../Cache/SLRUFileCachePriority.h | 4 +-- 6 files changed, 53 insertions(+), 28 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 2e12d6ef9bf..da996443e68 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -771,8 +771,7 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa reserve_stat.stat_by_kind.clear(); } - /// A file_segment_metadata acquires a LRUQueue iterator on first successful space reservation attempt, - /// e.g. queue_iteratir is std::nullopt here if no space has been reserved yet. + /// A file_segment_metadata acquires a priority iterator on first successful space reservation attempt, auto queue_iterator = file_segment.getQueueIterator(); chassert(!queue_iterator || file_segment.getReservedSize() > 0); diff --git a/src/Interpreters/Cache/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp index 6055cec5ae5..564a0f2aacd 100644 --- a/src/Interpreters/Cache/FileCacheSettings.cpp +++ b/src/Interpreters/Cache/FileCacheSettings.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include namespace DB @@ -63,7 +64,10 @@ void FileCacheSettings::loadImpl(FuncHas has, FuncGetUInt get_uint, FuncGetStrin throw Exception(ErrorCodes::BAD_ARGUMENTS, "Setting `boundary_alignment` cannot exceed `max_file_segment_size`"); if (has("cache_policy")) + { cache_policy = get_string("cache_policy"); + boost::to_upper(cache_policy); + } if (has("slru_size_ratio")) slru_size_ratio = get_double("slru_size_ratio"); diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index 625be890cd3..f9b0ddfce15 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -32,10 +32,10 @@ IFileCachePriority::IteratorPtr LRUFileCachePriority::add( size_t size, const CacheGuard::Lock & lock) { - return add(Entry(key_metadata->key, offset, size, key_metadata), lock); + return std::make_shared(add(Entry(key_metadata->key, offset, size, key_metadata), lock)); } -std::unique_ptr LRUFileCachePriority::add(Entry && entry, const CacheGuard::Lock &) +LRUFileCachePriority::LRUIterator LRUFileCachePriority::add(Entry && entry, const CacheGuard::Lock &) { if (entry.size == 0) { @@ -76,7 +76,7 @@ std::unique_ptr LRUFileCachePriority::add(Ent log, "Added entry into LRU queue, key: {}, offset: {}, size: {}", entry.key, entry.offset, entry.size); - return std::make_unique(this, iterator); + return LRUIterator(this, iterator); } LRUFileCachePriority::LRUQueue::iterator LRUFileCachePriority::remove(LRUQueue::iterator it, const CacheGuard::Lock &) @@ -115,6 +115,26 @@ LRUFileCachePriority::LRUIterator::LRUIterator( { } +LRUFileCachePriority::LRUIterator::LRUIterator(const LRUIterator & other) +{ + *this = other; +} + +LRUFileCachePriority::LRUIterator & LRUFileCachePriority::LRUIterator::operator =(const LRUIterator & other) +{ + if (this == &other) + return *this; + + cache_priority = other.cache_priority; + iterator = other.iterator; + return *this; +} + +bool LRUFileCachePriority::LRUIterator::operator ==(const LRUIterator & other) const +{ + return cache_priority == other.cache_priority && iterator == other.iterator; +} + void LRUFileCachePriority::iterate(IterateFunc && func, const CacheGuard::Lock & lock) { for (auto it = queue.begin(); it != queue.end();) @@ -223,8 +243,7 @@ bool LRUFileCachePriority::collectCandidatesForEviction( return can_fit(); } -std::unique_ptr -LRUFileCachePriority::move(LRUIterator & it, LRUFileCachePriority & other, const CacheGuard::Lock &) +LRUFileCachePriority::LRUIterator LRUFileCachePriority::move(LRUIterator & it, LRUFileCachePriority & other, const CacheGuard::Lock &) { const auto & entry = it.getEntry(); if (entry.size == 0) @@ -254,7 +273,7 @@ LRUFileCachePriority::move(LRUIterator & it, LRUFileCachePriority & other, const other.updateSize(-entry.size); other.updateElementsCount(-1); - return std::make_unique(this, it.iterator); + return LRUIterator(this, it.iterator); } FileSegments LRUFileCachePriority::dump(const CacheGuard::Lock & lock) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 63b93de76e4..289968602ca 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -67,17 +67,22 @@ private: using IterateFunc = std::function; void iterate(IterateFunc && func, const CacheGuard::Lock &); - std::unique_ptr move(LRUIterator & it, LRUFileCachePriority & other, const CacheGuard::Lock &); - std::unique_ptr add(Entry && entry, const CacheGuard::Lock &); + LRUIterator move(LRUIterator & it, LRUFileCachePriority & other, const CacheGuard::Lock &); + LRUIterator add(Entry && entry, const CacheGuard::Lock &); }; class LRUFileCachePriority::LRUIterator : public IFileCachePriority::Iterator { friend class LRUFileCachePriority; friend class SLRUFileCachePriority; + public: LRUIterator(LRUFileCachePriority * cache_priority_, LRUQueue::iterator iterator_); + LRUIterator(const LRUIterator & other); + LRUIterator & operator =(const LRUIterator & other); + bool operator ==(const LRUIterator & other) const; + const Entry & getEntry() const override { return *iterator; } size_t increasePriority(const CacheGuard::Lock &) override; diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index a9e017c62e4..dfc3686683d 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -109,7 +109,7 @@ bool SLRUFileCachePriority::collectCandidatesForEviction( for (const auto & candidate : key_candidates.candidates) { auto * candidate_it = assert_cast(candidate->getQueueIterator().get()); - candidate_it->lru_iterator = probationary_queue.move(*candidate_it->lru_iterator, protected_queue, lk); + candidate_it->lru_iterator = probationary_queue.move(candidate_it->lru_iterator, protected_queue, lk); candidate_it->is_protected = false; } } @@ -124,7 +124,7 @@ void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const Cach /// we only need to increase its priority within the protected queue. if (iterator.is_protected) { - iterator.lru_iterator->increasePriority(lock); + iterator.lru_iterator.increasePriority(lock); return; } @@ -137,7 +137,7 @@ void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const Cach /// Entry size is bigger than the whole protected queue limit. /// This is only possible if protected_queue_size_limit is less than max_file_segment_size, /// which is not possible in any realistic cache configuration. - iterator.lru_iterator->increasePriority(lock); + iterator.lru_iterator.increasePriority(lock); return; } @@ -153,7 +153,7 @@ void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const Cach /// We cannot make space for entry to be moved to protected queue /// (not enough releasable file segments). /// Then just increase its priority within probationary queue. - iterator.lru_iterator->increasePriority(lock); + iterator.lru_iterator.increasePriority(lock); return; } @@ -176,7 +176,7 @@ void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const Cach /// "downgrade" candidates cannot be moved to probationary queue, /// so entry cannot be moved to protected queue as well. /// Then just increase its priority within probationary queue. - iterator.lru_iterator->increasePriority(lock); + iterator.lru_iterator.increasePriority(lock); return; } /// Make space for "downgrade" candidates. @@ -186,14 +186,14 @@ void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const Cach /// All checks passed, now we can move downgrade candidates to /// probationary queue and our entry to protected queue. Entry entry_copy = iterator.getEntry(); - iterator.lru_iterator->remove(lock); + iterator.lru_iterator.remove(lock); for (const auto & [key, key_candidates] : downgrade_candidates) { for (const auto & candidate : key_candidates.candidates) { auto * candidate_it = assert_cast(candidate->getQueueIterator().get()); - candidate_it->lru_iterator = probationary_queue.move(*candidate_it->lru_iterator, protected_queue, lock); + candidate_it->lru_iterator = probationary_queue.move(candidate_it->lru_iterator, protected_queue, lock); candidate_it->is_protected = false; } } @@ -218,10 +218,10 @@ void SLRUFileCachePriority::shuffle(const CacheGuard::Lock & lock) SLRUFileCachePriority::SLRUIterator::SLRUIterator( SLRUFileCachePriority * cache_priority_, - std::unique_ptr lru_iterator_, + LRUFileCachePriority::LRUIterator && lru_iterator_, bool is_protected_) : cache_priority(cache_priority_) - , lru_iterator(std::move(lru_iterator_)) + , lru_iterator(lru_iterator_) , is_protected(is_protected_) { } @@ -229,7 +229,7 @@ SLRUFileCachePriority::SLRUIterator::SLRUIterator( const SLRUFileCachePriority::Entry & SLRUFileCachePriority::SLRUIterator::getEntry() const { assertValid(); - return lru_iterator->getEntry(); + return lru_iterator.getEntry(); } size_t SLRUFileCachePriority::SLRUIterator::increasePriority(const CacheGuard::Lock & lock) @@ -242,26 +242,24 @@ size_t SLRUFileCachePriority::SLRUIterator::increasePriority(const CacheGuard::L void SLRUFileCachePriority::SLRUIterator::updateSize(int64_t size) { assertValid(); - lru_iterator->updateSize(size); + lru_iterator.updateSize(size); } void SLRUFileCachePriority::SLRUIterator::invalidate() { assertValid(); - lru_iterator->invalidate(); + lru_iterator.invalidate(); } void SLRUFileCachePriority::SLRUIterator::remove(const CacheGuard::Lock & lock) { assertValid(); - lru_iterator->remove(lock); - lru_iterator = nullptr; + lru_iterator.remove(lock); } void SLRUFileCachePriority::SLRUIterator::assertValid() const { - if (!lru_iterator) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to use invalid iterator"); + lru_iterator.assertValid(); } } diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index 46e8f37819f..9dad6c15fee 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -52,7 +52,7 @@ class SLRUFileCachePriority::SLRUIterator : public IFileCachePriority::Iterator public: SLRUIterator( SLRUFileCachePriority * cache_priority_, - std::unique_ptr lru_iterator_, + LRUIterator && lru_iterator_, bool is_protected_); const Entry & getEntry() const override; @@ -71,7 +71,7 @@ private: void assertValid() const; SLRUFileCachePriority * cache_priority; - mutable std::unique_ptr lru_iterator; + mutable LRUIterator lru_iterator; bool is_protected; }; From 69260e322e7db6e2e7d85f0eca846df89c58ce28 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 23 Nov 2023 16:53:58 +0100 Subject: [PATCH 043/331] Randomize cache policy in ci --- docker/test/stateful/run.sh | 12 ++++++++++++ docker/test/stress/run.sh | 21 +++++++++++++++++++++ src/Interpreters/Cache/FileCache.cpp | 2 ++ tests/config/config.d/storage_conf.xml | 4 +++- 4 files changed, 38 insertions(+), 1 deletion(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index ad3c3477b37..829b3547856 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -24,6 +24,18 @@ azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log & config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml +cache_policy="SLRU" +#TODO: uncomment this before merge, for testing purposes it is SLRU only before merge. +#if [$(($RANDOM%2)) -eq 1]; then +# cache_policy="LRU" +#fi +if [ "$cache_policy" = "SLRU" ]; then + sudo cat /etc/clickhouse-server/config.d/storage_conf.xml \ + | sed "s|LRU|SLRU|" \ + > /etc/clickhouse-server/config.d/storage_conf.xml.tmp + mv /etc/clickhouse-server/config.d/storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml +fi + function start() { if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index b5092fd40df..02de3c67682 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -65,9 +65,23 @@ chmod 777 -R /var/lib/clickhouse clickhouse-client --query "ATTACH DATABASE IF NOT EXISTS datasets ENGINE = Ordinary" clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test" + stop mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.initial.log +# Randomly choose between LRU and SLRU policies. +cache_policy="SLRU" +#TODO: uncomment this before merge, for testing purposes it is SLRU only before merge. +#if [$(($RANDOM%2)) -eq 1]; then +# cache_policy="LRU" +#fi +if [ "$cache_policy" = "SLRU" ]; then + sudo cat /etc/clickhouse-server/config.d/storage_conf.xml \ + | sed "s|LRU|SLRU|" \ + > /etc/clickhouse-server/config.d/storage_conf.xml.tmp + mv /etc/clickhouse-server/config.d/storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml +fi + start clickhouse-client --query "SHOW TABLES FROM datasets" @@ -191,6 +205,13 @@ sudo cat /etc/clickhouse-server/config.d/logger_trace.xml \ > /etc/clickhouse-server/config.d/logger_trace.xml.tmp mv /etc/clickhouse-server/config.d/logger_trace.xml.tmp /etc/clickhouse-server/config.d/logger_trace.xml +if [ "$cache_policy" = "SLRU" ]; then + sudo cat /etc/clickhouse-server/config.d/storage_conf.xml \ + | sed "s|LRU|SLRU|" \ + > /etc/clickhouse-server/config.d/storage_conf.xml.tmp + mv /etc/clickhouse-server/config.d/storage_conf.xml.tmp /etc/clickhouse-server/config.d/storage_conf.xml +fi + start stress --hung-check --drop-databases --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" --global-time-limit 1200 \ diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index da996443e68..0c818e10c9c 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -67,6 +67,8 @@ FileCache::FileCache(const std::string & cache_name, const FileCacheSettings & s else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown cache policy: {}", settings.cache_policy); + LOG_DEBUG(log, "Using {} cache policy", settings.cache_policy); + if (settings.cache_hits_threshold) stash = std::make_unique(settings.cache_hits_threshold, settings.max_elements); diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index d976e46ff7b..b8599ef3081 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -13,9 +13,11 @@ cache s3_disk s3_cache/ - 128Mi + 64Mi 1 100 + LRU + 0.3 From c5fabcc4727d6137f0f54d58d91a38aba9e03d8d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Nov 2023 17:50:50 +0100 Subject: [PATCH 044/331] 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 0f8d9515ec637bd346baf65d10c1f6b883d4c37c Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 23 Nov 2023 17:53:53 +0100 Subject: [PATCH 045/331] Fix style check --- src/Interpreters/Cache/SLRUFileCachePriority.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index dfc3686683d..02ef91c6aa2 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -10,11 +10,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - namespace { size_t getRatio(size_t total, double ratio) From 81a50703c3e73f64660790a6686450121366e5b4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Nov 2023 23:39:40 +0100 Subject: [PATCH 046/331] 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 047/331] 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 8fac59b18e15a9d2854a08ffb2b485ff81a2902e Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Nov 2023 12:58:18 +0100 Subject: [PATCH 048/331] Fix cache load on server startup with slru policy --- src/Interpreters/Cache/FileCache.cpp | 9 ++---- src/Interpreters/Cache/IFileCachePriority.h | 10 +++++- .../Cache/LRUFileCachePriority.cpp | 3 +- src/Interpreters/Cache/LRUFileCachePriority.h | 10 ++++-- .../Cache/SLRUFileCachePriority.cpp | 31 +++++++++++++++++-- .../Cache/SLRUFileCachePriority.h | 9 +++++- 6 files changed, 58 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 0c818e10c9c..f1b3d24dbc1 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -1001,9 +1001,6 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir) const auto key = Key::fromKeyString(key_directory.filename().string()); auto key_metadata = metadata.getKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY, /* is_initial_load */true); - const size_t size_limit = main_priority->getSizeLimit(); - const size_t elements_limit = main_priority->getElementsLimit(); - for (fs::directory_iterator offset_it{key_directory}; offset_it != fs::directory_iterator(); ++offset_it) { auto offset_with_suffix = offset_it->path().filename().string(); @@ -1044,13 +1041,13 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir) bool limits_satisfied; IFileCachePriority::IteratorPtr cache_it; + { auto lock = lockCache(); - limits_satisfied = (size_limit == 0 || main_priority->getSize(lock) + size <= size_limit) - && (elements_limit == 0 || main_priority->getElementsCount(lock) + 1 <= elements_limit); + limits_satisfied = main_priority->canFit(size, lock); if (limits_satisfied) - cache_it = main_priority->add(key_metadata, offset, size, lock); + cache_it = main_priority->add(key_metadata, offset, size, lock, /* is_startup */true); /// TODO: we can get rid of this lockCache() if we first load everything in parallel /// without any mutual lock between loading threads, and only after do removeOverflow(). diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index fe5de21ec48..44fb450658b 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -59,7 +59,15 @@ public: virtual size_t getElementsCount(const CacheGuard::Lock &) const = 0; - virtual IteratorPtr add(KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) = 0; + /// Throws exception if there is not enought size to fit it. + virtual IteratorPtr add( /// NOLINT + KeyMetadataPtr key_metadata, + size_t offset, + size_t size, + const CacheGuard::Lock &, + bool is_startup = false) = 0; + + virtual bool canFit(size_t size, const CacheGuard::Lock &) const = 0; virtual void shuffle(const CacheGuard::Lock &) = 0; diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index f9b0ddfce15..cb4da79af59 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -30,7 +30,8 @@ IFileCachePriority::IteratorPtr LRUFileCachePriority::add( KeyMetadataPtr key_metadata, size_t offset, size_t size, - const CacheGuard::Lock & lock) + const CacheGuard::Lock & lock, + bool /* is_startup */) { return std::make_shared(add(Entry(key_metadata->key, offset, size, key_metadata), lock)); } diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 289968602ca..31a5c527ca8 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -25,7 +25,14 @@ public: size_t getElementsCount(const CacheGuard::Lock &) const override { return current_elements_num; } - IteratorPtr add(KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) override; + bool canFit(size_t size, const CacheGuard::Lock &) const override; + + IteratorPtr add( /// NOLINT + KeyMetadataPtr key_metadata, + size_t offset, + size_t size, + const CacheGuard::Lock &, + bool is_startup = false) override; bool collectCandidatesForEviction( size_t size, @@ -53,7 +60,6 @@ private: /// because of invalidated entries. std::atomic current_elements_num = 0; - bool canFit(size_t size, const CacheGuard::Lock &) const; bool canFit(size_t size, size_t released_size_assumption, size_t released_elements_assumption, const CacheGuard::Lock &) const; LRUQueue::iterator remove(LRUQueue::iterator it, const CacheGuard::Lock &); diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index 02ef91c6aa2..f2c96c4df24 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -41,14 +41,39 @@ size_t SLRUFileCachePriority::getElementsCount(const CacheGuard::Lock & lock) co return protected_queue.getElementsCount(lock) + probationary_queue.getElementsCount(lock); } +bool SLRUFileCachePriority::canFit(size_t size, const CacheGuard::Lock & lock) const +{ + return probationary_queue.canFit(size, lock) || protected_queue.canFit(size, lock); +} + IFileCachePriority::IteratorPtr SLRUFileCachePriority::add( KeyMetadataPtr key_metadata, size_t offset, size_t size, - const CacheGuard::Lock & lock) + const CacheGuard::Lock & lock, + bool is_startup) { - auto lru_iterator = probationary_queue.add(Entry(key_metadata->key, offset, size, key_metadata), lock); - return std::make_shared(this, std::move(lru_iterator), false); + if (is_startup) + { + /// If it is server startup, we put entries in any queue it will fit in, + /// but with preference for probationary queue, + /// because we do not know the distribution between queues after server restart. + if (probationary_queue.canFit(size, lock)) + { + auto lru_iterator = probationary_queue.add(Entry(key_metadata->key, offset, size, key_metadata), lock); + return std::make_shared(this, std::move(lru_iterator), false); + } + else + { + auto lru_iterator = protected_queue.add(Entry(key_metadata->key, offset, size, key_metadata), lock); + return std::make_shared(this, std::move(lru_iterator), true); + } + } + else + { + auto lru_iterator = probationary_queue.add(Entry(key_metadata->key, offset, size, key_metadata), lock); + return std::make_shared(this, std::move(lru_iterator), false); + } } bool SLRUFileCachePriority::collectCandidatesForEviction( diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index 9dad6c15fee..7753f6f23b2 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -24,7 +24,14 @@ public: size_t getElementsCount(const CacheGuard::Lock &) const override; - IteratorPtr add(KeyMetadataPtr key_metadata, size_t offset, size_t size, const CacheGuard::Lock &) override; + bool canFit(size_t size, const CacheGuard::Lock &) const override; + + IteratorPtr add( /// NOLINT + KeyMetadataPtr key_metadata, + size_t offset, + size_t size, + const CacheGuard::Lock &, + bool is_startup = false) override; bool collectCandidatesForEviction( size_t size, From 5c5cbc9ef507256a13f78aa8e57ea1981b0453ec Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Nov 2023 15:46:45 +0100 Subject: [PATCH 049/331] Fix typo --- src/Interpreters/Cache/IFileCachePriority.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index 44fb450658b..e7a05b0e46d 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -59,7 +59,7 @@ public: virtual size_t getElementsCount(const CacheGuard::Lock &) const = 0; - /// Throws exception if there is not enought size to fit it. + /// Throws exception if there is not enough size to fit it. virtual IteratorPtr add( /// NOLINT KeyMetadataPtr key_metadata, size_t offset, From 411660c267a5c378069d209067a225801bd9b3fa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 24 Nov 2023 17:35:56 +0100 Subject: [PATCH 050/331] 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 0f7b5ebd68d44f1789a9186082a9e8fdfa906b34 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Nov 2023 18:42:56 +0100 Subject: [PATCH 051/331] Fix clang-tidy --- src/Interpreters/Cache/LRUFileCachePriority.cpp | 2 +- src/Interpreters/Cache/SLRUFileCachePriority.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index cb4da79af59..a596d041941 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -26,7 +26,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -IFileCachePriority::IteratorPtr LRUFileCachePriority::add( +IFileCachePriority::IteratorPtr LRUFileCachePriority::add( /// NOLINT KeyMetadataPtr key_metadata, size_t offset, size_t size, diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index f2c96c4df24..8f7f5e4aa32 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -46,7 +46,7 @@ bool SLRUFileCachePriority::canFit(size_t size, const CacheGuard::Lock & lock) c return probationary_queue.canFit(size, lock) || protected_queue.canFit(size, lock); } -IFileCachePriority::IteratorPtr SLRUFileCachePriority::add( +IFileCachePriority::IteratorPtr SLRUFileCachePriority::add( /// NOLINT KeyMetadataPtr key_metadata, size_t offset, size_t size, From 1790626406ae707c551fccce6ceb166d81d360bc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 1 Oct 2022 16:32:04 +0200 Subject: [PATCH 052/331] 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 053/331] 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 054/331] 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 055/331] 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 056/331] 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 057/331] 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 058/331] 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 059/331] 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 060/331] 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 061/331] 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 ca0aa935c1d364eaf04c7fab227731d882d852a1 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Mon, 27 Nov 2023 23:56:33 +0100 Subject: [PATCH 062/331] Add Enums support to `substring` function --- src/DataTypes/IDataType.h | 2 + src/Functions/GatherUtils/Sources.h | 75 +++++++++ src/Functions/substring.cpp | 58 ++++--- .../00493_substring_of_enum.reference | 148 ++++++++++++++++++ .../0_stateless/00493_substring_of_enum.sql | 24 +++ 5 files changed, 285 insertions(+), 22 deletions(-) create mode 100644 tests/queries/0_stateless/00493_substring_of_enum.reference create mode 100644 tests/queries/0_stateless/00493_substring_of_enum.sql diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 4ee615f5f70..98f7e0cb06f 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -447,6 +447,8 @@ template inline bool isNativeNumber(const T & data_type) { return W template inline bool isNumber(const T & data_type) { return WhichDataType(data_type).isNumber(); } template inline bool isEnum(const T & data_type) { return WhichDataType(data_type).isEnum(); } +template inline bool isEnum8(const T & data_type) { return WhichDataType(data_type).isEnum8(); } +template inline bool isEnum16(const T & data_type) { return WhichDataType(data_type).isEnum16(); } template inline bool isDate(const T & data_type) { return WhichDataType(data_type).isDate(); } template inline bool isDate32(const T & data_type) { return WhichDataType(data_type).isDate32(); } diff --git a/src/Functions/GatherUtils/Sources.h b/src/Functions/GatherUtils/Sources.h index fad2be1f622..a3b5c21b89c 100644 --- a/src/Functions/GatherUtils/Sources.h +++ b/src/Functions/GatherUtils/Sources.h @@ -11,6 +11,8 @@ #include #include +#include + #include "IArraySource.h" #include "IValueSource.h" #include "Slices.h" @@ -313,6 +315,79 @@ struct StringSource } }; +template +struct EnumSource { + using Column = ColumnVector; + using Slice = NumericArraySlice; + + using SinkType = StringSink; + + const typename ColumnVector::Container & data; + const DataTypeEnum & data_type; + + size_t row_num = 0; + + explicit EnumSource(const Column & col, const DataTypeEnum & data_type_) : data(col.getData()), data_type(data_type_) { } + + void next() { ++row_num; } + + bool isEnd() const { return row_num == data.size(); } + + size_t rowNum() const { return row_num; } + + size_t getSizeForReserve() const { return data.size(); } + + size_t getElementSize() const + { + StringRef name = data_type.getNameForValue(data[row_num]); + return name.size; + } + + size_t getColumnSize() const { return data.size(); } + + Slice getWhole() const { + StringRef name = data_type.getNameForValue(data[row_num]); + const UInt8 * name_data = reinterpret_cast(name.data); + return {name_data, name.size}; + } + + Slice getSliceFromLeft(size_t offset) const + { + StringRef name = data_type.getNameForValue(data[row_num]); + if (offset >= name.size) + return {nullptr, 0}; + const UInt8 * name_data = reinterpret_cast(name.data); + return {name_data + offset, name.size - offset}; + } + + Slice getSliceFromLeft(size_t offset, size_t length) const + { + StringRef name = data_type.getNameForValue(data[row_num]); + if (offset >= name.size) + return {nullptr, 0}; + const UInt8 * name_data = reinterpret_cast(name.data); + return {name_data + offset, std::min(length, name.size - offset)}; + } + + Slice getSliceFromRight(size_t offset) const + { + StringRef name = data_type.getNameForValue(data[row_num]); + const UInt8 * name_data = reinterpret_cast(name.data); + if (offset > name.size) + return {name_data, name.size}; + return {name_data + name.size - offset, offset}; + } + + Slice getSliceFromRight(size_t offset, size_t length) const + { + StringRef name = data_type.getNameForValue(data[row_num]); + const UInt8 * name_data = reinterpret_cast(name.data); + if (offset > name.size) + return {name_data, length + name.size > offset ? std::min(name.size, length + name.size - offset) : 0}; + return {name_data + name.size - offset, std::min(length, offset)}; + } +}; + /// Differs to StringSource by having 'offset' and 'length' in code points instead of bytes in getSlice* methods. /** NOTE: The behaviour of substring and substringUTF8 is inconsistent when negative offset is greater than string size: diff --git a/src/Functions/substring.cpp b/src/Functions/substring.cpp index 7678692f612..f42452c9d99 100644 --- a/src/Functions/substring.cpp +++ b/src/Functions/substring.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -20,10 +21,10 @@ using namespace GatherUtils; namespace ErrorCodes { - extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int ZERO_ARRAY_OR_TUPLE_INDEX; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +extern const int ILLEGAL_COLUMN; +extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +extern const int ZERO_ARRAY_OR_TUPLE_INDEX; } namespace @@ -61,7 +62,7 @@ public: throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Number of arguments for function {} doesn't match: " "passed {}, should be 2 or 3", getName(), number_of_arguments); - if ((is_utf8 && !isString(arguments[0])) || !isStringOrFixedString(arguments[0])) + if ((is_utf8 && !isString(arguments[0])) || (!isStringOrFixedString(arguments[0]) && !isEnum(arguments[0]))) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", arguments[0]->getName(), getName()); @@ -124,7 +125,7 @@ public: { size_t number_of_arguments = arguments.size(); - ColumnPtr column_string = arguments[0].column; + ColumnPtr column_arg0 = arguments[0].column; ColumnPtr column_start = arguments[1].column; ColumnPtr column_length; @@ -147,33 +148,46 @@ public: if constexpr (is_utf8) { - if (const ColumnString * col = checkAndGetColumn(column_string.get())) + if (const ColumnString * col = checkAndGetColumn(column_arg0.get())) return executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, length_value, UTF8StringSource(*col), input_rows_count); - else if (const ColumnConst * col_const = checkAndGetColumnConst(column_string.get())) + if (const ColumnConst * col_const = checkAndGetColumnConst(column_arg0.get())) return executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, length_value, ConstSource(*col_const), input_rows_count); - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", - arguments[0].column->getName(), getName()); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", + arguments[0].column->getName(), getName()); } else { - if (const ColumnString * col = checkAndGetColumn(column_string.get())) + if (const ColumnString * col = checkAndGetColumn(column_arg0.get())) return executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, - length_value, StringSource(*col), input_rows_count); - else if (const ColumnFixedString * col_fixed = checkAndGetColumn(column_string.get())) + length_value, StringSource(*col), input_rows_count); + if (const ColumnFixedString * col_fixed = checkAndGetColumn(column_arg0.get())) return executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, - length_value, FixedStringSource(*col_fixed), input_rows_count); - else if (const ColumnConst * col_const = checkAndGetColumnConst(column_string.get())) + length_value, FixedStringSource(*col_fixed), input_rows_count); + if (const ColumnConst * col_const = checkAndGetColumnConst(column_arg0.get())) return executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, - length_value, ConstSource(*col_const), input_rows_count); - else if (const ColumnConst * col_const_fixed = checkAndGetColumnConst(column_string.get())) + length_value, ConstSource(*col_const), input_rows_count); + if (const ColumnConst * col_const_fixed = checkAndGetColumnConst(column_arg0.get())) return executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, - length_value, ConstSource(*col_const_fixed), input_rows_count); - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", - arguments[0].column->getName(), getName()); + length_value, ConstSource(*col_const_fixed), input_rows_count); + if (isEnum8(arguments[0].type)) + if (const ColumnVector * col_enum8 = checkAndGetColumn>(column_arg0.get())) + { + const auto * enum_type = typeid_cast *>(arguments[0].type.get()); + return executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, + length_value, EnumSource(*col_enum8, *enum_type), input_rows_count); + } + if (isEnum16(arguments[0].type)) + if (const ColumnVector * col_enum16 = checkAndGetColumn>(column_arg0.get())) + { + const auto * enum_type = typeid_cast *>(arguments[0].type.get()); + return executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, + length_value, EnumSource(*col_enum16, *enum_type), input_rows_count); + } + + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", + arguments[0].column->getName(), getName()); } } }; diff --git a/tests/queries/0_stateless/00493_substring_of_enum.reference b/tests/queries/0_stateless/00493_substring_of_enum.reference new file mode 100644 index 00000000000..427d9c9eafb --- /dev/null +++ b/tests/queries/0_stateless/00493_substring_of_enum.reference @@ -0,0 +1,148 @@ +-- Positive offsets (slice from left) +Offset: 1 Length: 1 hello shark h s +Offset: 1 Length: 2 hello shark he sh +Offset: 1 Length: 3 hello shark hel sha +Offset: 1 Length: 4 hello shark hell shar +Offset: 1 Length: 5 hello shark hello shark +Offset: 1 Length: 6 hello shark hello shark +Offset: 2 Length: 1 ello hark e h +Offset: 2 Length: 2 ello hark el ha +Offset: 2 Length: 3 ello hark ell har +Offset: 2 Length: 4 ello hark ello hark +Offset: 2 Length: 5 ello hark ello hark +Offset: 2 Length: 6 ello hark ello hark +Offset: 3 Length: 1 llo ark l a +Offset: 3 Length: 2 llo ark ll ar +Offset: 3 Length: 3 llo ark llo ark +Offset: 3 Length: 4 llo ark llo ark +Offset: 3 Length: 5 llo ark llo ark +Offset: 3 Length: 6 llo ark llo ark +Offset: 4 Length: 1 lo rk l r +Offset: 4 Length: 2 lo rk lo rk +Offset: 4 Length: 3 lo rk lo rk +Offset: 4 Length: 4 lo rk lo rk +Offset: 4 Length: 5 lo rk lo rk +Offset: 4 Length: 6 lo rk lo rk +Offset: 5 Length: 1 o k o k +Offset: 5 Length: 2 o k o k +Offset: 5 Length: 3 o k o k +Offset: 5 Length: 4 o k o k +Offset: 5 Length: 5 o k o k +Offset: 5 Length: 6 o k o k +Offset: 6 Length: 1 +Offset: 6 Length: 2 +Offset: 6 Length: 3 +Offset: 6 Length: 4 +Offset: 6 Length: 5 +Offset: 6 Length: 6 +Offset: 1 Length: 1 world eagle w e +Offset: 1 Length: 2 world eagle wo ea +Offset: 1 Length: 3 world eagle wor eag +Offset: 1 Length: 4 world eagle worl eagl +Offset: 1 Length: 5 world eagle world eagle +Offset: 1 Length: 6 world eagle world eagle +Offset: 2 Length: 1 orld agle o a +Offset: 2 Length: 2 orld agle or ag +Offset: 2 Length: 3 orld agle orl agl +Offset: 2 Length: 4 orld agle orld agle +Offset: 2 Length: 5 orld agle orld agle +Offset: 2 Length: 6 orld agle orld agle +Offset: 3 Length: 1 rld gle r g +Offset: 3 Length: 2 rld gle rl gl +Offset: 3 Length: 3 rld gle rld gle +Offset: 3 Length: 4 rld gle rld gle +Offset: 3 Length: 5 rld gle rld gle +Offset: 3 Length: 6 rld gle rld gle +Offset: 4 Length: 1 ld le l l +Offset: 4 Length: 2 ld le ld le +Offset: 4 Length: 3 ld le ld le +Offset: 4 Length: 4 ld le ld le +Offset: 4 Length: 5 ld le ld le +Offset: 4 Length: 6 ld le ld le +Offset: 5 Length: 1 d e d e +Offset: 5 Length: 2 d e d e +Offset: 5 Length: 3 d e d e +Offset: 5 Length: 4 d e d e +Offset: 5 Length: 5 d e d e +Offset: 5 Length: 6 d e d e +Offset: 6 Length: 1 +Offset: 6 Length: 2 +Offset: 6 Length: 3 +Offset: 6 Length: 4 +Offset: 6 Length: 5 +Offset: 6 Length: 6 +-- Negative offsets (slice from right) +Offset: -1 Length: 1 o k o k +Offset: -1 Length: 2 o k o k +Offset: -1 Length: 3 o k o k +Offset: -1 Length: 4 o k o k +Offset: -1 Length: 5 o k o k +Offset: -1 Length: 6 o k o k +Offset: -2 Length: 1 lo rk l r +Offset: -2 Length: 2 lo rk lo rk +Offset: -2 Length: 3 lo rk lo rk +Offset: -2 Length: 4 lo rk lo rk +Offset: -2 Length: 5 lo rk lo rk +Offset: -2 Length: 6 lo rk lo rk +Offset: -3 Length: 1 llo ark l a +Offset: -3 Length: 2 llo ark ll ar +Offset: -3 Length: 3 llo ark llo ark +Offset: -3 Length: 4 llo ark llo ark +Offset: -3 Length: 5 llo ark llo ark +Offset: -3 Length: 6 llo ark llo ark +Offset: -4 Length: 1 ello hark e h +Offset: -4 Length: 2 ello hark el ha +Offset: -4 Length: 3 ello hark ell har +Offset: -4 Length: 4 ello hark ello hark +Offset: -4 Length: 5 ello hark ello hark +Offset: -4 Length: 6 ello hark ello hark +Offset: -5 Length: 1 hello shark h s +Offset: -5 Length: 2 hello shark he sh +Offset: -5 Length: 3 hello shark hel sha +Offset: -5 Length: 4 hello shark hell shar +Offset: -5 Length: 5 hello shark hello shark +Offset: -5 Length: 6 hello shark hello shark +Offset: -6 Length: 1 hello shark +Offset: -6 Length: 2 hello shark h s +Offset: -6 Length: 3 hello shark he sh +Offset: -6 Length: 4 hello shark hel sha +Offset: -6 Length: 5 hello shark hell shar +Offset: -6 Length: 6 hello shark hello shark +Offset: -1 Length: 1 d e d e +Offset: -1 Length: 2 d e d e +Offset: -1 Length: 3 d e d e +Offset: -1 Length: 4 d e d e +Offset: -1 Length: 5 d e d e +Offset: -1 Length: 6 d e d e +Offset: -2 Length: 1 ld le l l +Offset: -2 Length: 2 ld le ld le +Offset: -2 Length: 3 ld le ld le +Offset: -2 Length: 4 ld le ld le +Offset: -2 Length: 5 ld le ld le +Offset: -2 Length: 6 ld le ld le +Offset: -3 Length: 1 rld gle r g +Offset: -3 Length: 2 rld gle rl gl +Offset: -3 Length: 3 rld gle rld gle +Offset: -3 Length: 4 rld gle rld gle +Offset: -3 Length: 5 rld gle rld gle +Offset: -3 Length: 6 rld gle rld gle +Offset: -4 Length: 1 orld agle o a +Offset: -4 Length: 2 orld agle or ag +Offset: -4 Length: 3 orld agle orl agl +Offset: -4 Length: 4 orld agle orld agle +Offset: -4 Length: 5 orld agle orld agle +Offset: -4 Length: 6 orld agle orld agle +Offset: -5 Length: 1 world eagle w e +Offset: -5 Length: 2 world eagle wo ea +Offset: -5 Length: 3 world eagle wor eag +Offset: -5 Length: 4 world eagle worl eagl +Offset: -5 Length: 5 world eagle world eagle +Offset: -5 Length: 6 world eagle world eagle +Offset: -6 Length: 1 world eagle +Offset: -6 Length: 2 world eagle w e +Offset: -6 Length: 3 world eagle wo ea +Offset: -6 Length: 4 world eagle wor eag +Offset: -6 Length: 5 world eagle worl eagl +Offset: -6 Length: 6 world eagle world eagle +-- Constant enums +f fo diff --git a/tests/queries/0_stateless/00493_substring_of_enum.sql b/tests/queries/0_stateless/00493_substring_of_enum.sql new file mode 100644 index 00000000000..39d0014bde9 --- /dev/null +++ b/tests/queries/0_stateless/00493_substring_of_enum.sql @@ -0,0 +1,24 @@ +DROP TABLE IF EXISTS substring_enums_test; +CREATE TABLE substring_enums_test(e8 Enum('hello' = -5, 'world' = 15), e16 Enum('shark' = -999, 'eagle' = 9999)) ENGINE MergeTree ORDER BY tuple(); +INSERT INTO TABLE substring_enums_test VALUES ('hello', 'shark'), ('world', 'eagle'); + +SELECT '-- Positive offsets (slice from left)'; +WITH cte AS (SELECT number + 1 AS n FROM system.numbers LIMIT 6), + permutations AS (SELECT c1.n AS offset, c2.n AS length FROM cte AS c1 CROSS JOIN cte AS c2) +SELECT 'Offset: ', p.offset, 'Length: ', p.length, + substring(e8, p.offset) AS s1, substring(e16, p.offset) AS s2, + substring(e8, p.offset, p.length) AS s3, substring(e16, p.offset, p.length) AS s4 +FROM substring_enums_test LEFT JOIN permutations AS p ON true; + +SELECT '-- Negative offsets (slice from right)'; +WITH cte AS (SELECT number + 1 AS n FROM system.numbers LIMIT 6), + permutations AS (SELECT -c1.n AS offset, c2.n AS length FROM cte AS c1 CROSS JOIN cte AS c2) +SELECT 'Offset: ', p.offset, 'Length: ', p.length, + substring(e8, p.offset) AS s1, substring(e16, p.offset) AS s2, + substring(e8, p.offset, p.length) AS s3, substring(e16, p.offset, p.length) AS s4 +FROM substring_enums_test LEFT JOIN permutations AS p ON true; + +SELECT '-- Constant enums'; +SELECT substring(CAST('foo', 'Enum8(\'foo\' = 1)'), 1, 1), substring(CAST('foo', 'Enum16(\'foo\' = 1111)'), 1, 2); + +DROP TABLE substring_enums_test; From 497e5abc4793a8c486192593fc48c0b67db4d483 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Tue, 28 Nov 2023 00:09:36 +0100 Subject: [PATCH 063/331] Add a doc entry. --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 1940993ce0b..8ecd5af9258 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -579,7 +579,7 @@ A function is called injective if it returns for different arguments different r ## substring(s, offset, length) -Returns a substring with `length` many bytes, starting at the byte at index `offset`. Character indexing starts from 1. +Returns a substring with `length` many bytes, starting at the byte at index `offset`. Character indexing starts from 1. Can be also used with [Enum](../../sql-reference/data-types/enum.md) types. **Syntax** From ff6bdfe8576b632b68700a8b8b220602fbc6b041 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Tue, 28 Nov 2023 00:43:29 +0100 Subject: [PATCH 064/331] Add more substring with enums tests --- .../queries/0_stateless/00493_substring_of_enum.reference | 7 +++++++ tests/queries/0_stateless/00493_substring_of_enum.sql | 8 ++++++++ 2 files changed, 15 insertions(+) diff --git a/tests/queries/0_stateless/00493_substring_of_enum.reference b/tests/queries/0_stateless/00493_substring_of_enum.reference index 427d9c9eafb..17bc960b318 100644 --- a/tests/queries/0_stateless/00493_substring_of_enum.reference +++ b/tests/queries/0_stateless/00493_substring_of_enum.reference @@ -144,5 +144,12 @@ Offset: -6 Length: 3 world eagle wo ea Offset: -6 Length: 4 world eagle wor eag Offset: -6 Length: 5 world eagle worl eagl Offset: -6 Length: 6 world eagle world eagle +-- Zero offset/length +Offset: 0 Length: 0 +Offset: 0 Length: 1 +Offset: 1 Length: 0 hello shark +Offset: 0 Length: 0 +Offset: 0 Length: 1 +Offset: 1 Length: 0 world eagle -- Constant enums f fo diff --git a/tests/queries/0_stateless/00493_substring_of_enum.sql b/tests/queries/0_stateless/00493_substring_of_enum.sql index 39d0014bde9..ba9fc630490 100644 --- a/tests/queries/0_stateless/00493_substring_of_enum.sql +++ b/tests/queries/0_stateless/00493_substring_of_enum.sql @@ -18,6 +18,14 @@ SELECT 'Offset: ', p.offset, 'Length: ', p.length, substring(e8, p.offset, p.length) AS s3, substring(e16, p.offset, p.length) AS s4 FROM substring_enums_test LEFT JOIN permutations AS p ON true; +SELECT '-- Zero offset/length'; +WITH cte AS (SELECT number AS n FROM system.numbers LIMIT 2), + permutations AS (SELECT c1.n AS offset, c2.n AS length FROM cte AS c1 CROSS JOIN cte AS c2 LIMIT 3) +SELECT 'Offset: ', p.offset, 'Length: ', p.length, + substring(e8, p.offset) AS s1, substring(e16, p.offset) AS s2, + substring(e8, p.offset, p.length) AS s3, substring(e16, p.offset, p.length) AS s4 +FROM substring_enums_test LEFT JOIN permutations AS p ON true; + SELECT '-- Constant enums'; SELECT substring(CAST('foo', 'Enum8(\'foo\' = 1)'), 1, 1), substring(CAST('foo', 'Enum16(\'foo\' = 1111)'), 1, 2); From 98a03ac36c62782116b390a09bcf75b30251eea1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Tue, 28 Nov 2023 18:28:29 +0800 Subject: [PATCH 065/331] support table --- src/Common/CurrentMetrics.cpp | 2 ++ src/Databases/DatabaseLazy.cpp | 1 + src/Databases/DatabaseOnDisk.cpp | 1 + src/Databases/DatabasesCommon.cpp | 10 ++++++++++ 4 files changed, 14 insertions(+) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 5a4b6e80f75..c592ae75150 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -212,6 +212,8 @@ M(PartsCommitted, "Deprecated. See PartsActive.") \ M(PartsPreActive, "The part is in data_parts, but not used for SELECTs.") \ M(PartsActive, "Active data part, used by current and upcoming SELECTs.") \ + M(AttachedDatabase, "Active database, used by current and upcoming SELECTs.") \ + M(AttachedTable, "Active table, used by current and upcoming SELECTs.") \ M(PartsOutdated, "Not active data part, but could be used by only current SELECTs, could be deleted after SELECTs finishes.") \ M(PartsDeleting, "Not active data part with identity refcounter, it is deleting right now by a cleaner.") \ M(PartsDeleteOnDestroy, "Part was moved to another disk and should be deleted in own destructor.") \ diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index 896ae99656f..9070c0b6ee7 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -64,6 +64,7 @@ void DatabaseLazy::createTable( SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); }); if (!endsWith(table->getName(), "Log")) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Lazy engine can be used only with *Log tables."); + std::cout<<"======= Flag 1"< #include #include +#include #include #include #include #include +namespace CurrentMetrics +{ + const extern Metric AttachedTable; +} + namespace DB { @@ -243,6 +249,7 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n res = it->second; tables.erase(it); res->is_detached = true; + CurrentMetrics::sub(CurrentMetrics::AttachedTable, 1); auto table_id = res->getStorageID(); if (table_id.hasUUID()) @@ -256,12 +263,14 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n void DatabaseWithOwnTablesBase::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & table, const String &) { + std::cout<<"========= Flag 5"<getStorageID(); if (table_id.database_name != database_name) throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database was renamed to `{}`, cannot create table in `{}`", @@ -283,6 +292,7 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, c /// It is important to reset is_detached here since in case of RENAME in /// non-Atomic database the is_detached is set to true before RENAME. table->is_detached = false; + CurrentMetrics::add(CurrentMetrics::AttachedTable, 1); } void DatabaseWithOwnTablesBase::shutdown() From ade60b5b45283adddfcf166d5a5c865c7a4da16f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 28 Nov 2023 14:12:26 +0100 Subject: [PATCH 066/331] Randomize disabled optimizations in CI --- tests/clickhouse-test | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 115e5ac7ba3..bd2f2ca1d5d 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -555,8 +555,15 @@ class SettingsRandomizer: "prefer_localhost_replica": lambda: random.randint(0, 1), "max_block_size": lambda: random.randint(8000, 100000), "max_threads": lambda: random.randint(1, 64), - "optimize_or_like_chain": lambda: random.randint(0, 1), + "optimize_append_index": lambda: random.randint(0, 1), + "optimize_functions_to_subcolumns": lambda: random.randint(0, 1), + "optimize_syntax_fuse_functions": lambda: random.randint(0, 1), + "optimize_if_chain_to_multiif": lambda: random.randint(0, 1), + "optimize_if_transform_strings_to_enum": lambda: random.randint(0, 1), "optimize_read_in_order": lambda: random.randint(0, 1), + "optimize_or_like_chain": lambda: random.randint(0, 1), + "optimize_substitute_columns": lambda: random.randint(0, 1), + "optimize_using_constraints": lambda: random.randint(0, 1), "enable_multiple_prewhere_read_steps": lambda: random.randint(0, 1), "read_in_order_two_level_merge_threshold": lambda: random.randint(0, 100), "optimize_aggregation_in_order": lambda: random.randint(0, 1), From 3aa4a6ae2c7ae50d96a56e9624fa46fa1c233cc7 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 28 Nov 2023 10:47:22 +0100 Subject: [PATCH 067/331] 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 b56b48d2de0aced8c40c3e00591d3e0c8448812f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 28 Nov 2023 14:59:20 +0000 Subject: [PATCH 068/331] Update docs --- .../functions/string-functions.md | 36 ++++++++++++++++--- 1 file changed, 31 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 8ecd5af9258..baf08e18f11 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -577,26 +577,52 @@ Like `concatWithSeparator` but assumes that `concatWithSeparator(sep, expr1, exp A function is called injective if it returns for different arguments different results. In other words: different arguments never produce identical result. -## substring(s, offset, length) +## substring -Returns a substring with `length` many bytes, starting at the byte at index `offset`. Character indexing starts from 1. Can be also used with [Enum](../../sql-reference/data-types/enum.md) types. +Returns the substring of a string `s` which starts at the specified byte index `offset`. Byte counting starts from 1. If `offset` is 0, an empty string is returned. If `offset` is negative, the substring starts `pos` characters from the end of the string, rather than from the beginning. An optional argument `length` specifies the maximum number of bytes the returned substring may have. **Syntax** ```sql -substring(s, offset, length) +substring(s, offset[, length]) ``` Alias: - `substr` - `mid` +**Arguments** + +- `s` — The string to calculate a substring from. [String](../../sql-reference/data-types/string.md), [FixedString](../../sql-reference/data-types/fixedstring.md) or [Enum](../../sql-reference/data-types/enum.md) +- `offset` — The starting position of the substring in `s` . [(U)Int*](../../sql-reference/data-types/int-uint.md). +- `length` — The maximum length of the substring. [(U)Int*](../../sql-reference/data-types/int-uint.md). Optional. + +**Returned value** + +A substring of `s` with `length` many bytes, starting at index `offset`. + +Type: `String`. + +**Example** + +``` sql +SELECT 'database' AS db, substr(db, 5), substr(db, 5, 1) +``` + +Result: + +```result +┌─db───────┬─substring('database', 5)─┬─substring('database', 5, 1)─┐ +│ database │ base │ b │ +└──────────┴──────────────────────────┴─────────────────────────────┘ +``` + ## substringUTF8 Like `substring` but for Unicode code points. Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. -## substringIndex(s, delim, count) +## substringIndex Returns the substring of `s` before `count` occurrences of the delimiter `delim`, as in Spark or MySQL. @@ -627,7 +653,7 @@ Result: └──────────────────────────────────────────────┘ ``` -## substringIndexUTF8(s, delim, count) +## substringIndexUTF8 Like `substringIndex` but for Unicode code points. Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. From 025fcd3aa4ca3867c1c3510adb4804d7b85076c6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 28 Nov 2023 18:22:49 +0100 Subject: [PATCH 069/331] Enable the settings to do a full run with them enabled --- tests/clickhouse-test | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index bd2f2ca1d5d..dc6614471a4 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -555,15 +555,15 @@ class SettingsRandomizer: "prefer_localhost_replica": lambda: random.randint(0, 1), "max_block_size": lambda: random.randint(8000, 100000), "max_threads": lambda: random.randint(1, 64), - "optimize_append_index": lambda: random.randint(0, 1), - "optimize_functions_to_subcolumns": lambda: random.randint(0, 1), - "optimize_syntax_fuse_functions": lambda: random.randint(0, 1), - "optimize_if_chain_to_multiif": lambda: random.randint(0, 1), - "optimize_if_transform_strings_to_enum": lambda: random.randint(0, 1), + "optimize_append_index": lambda: random.randint(1, 1), + "optimize_functions_to_subcolumns": lambda: random.randint(1, 1), + "optimize_syntax_fuse_functions": lambda: random.randint(1, 1), + "optimize_if_chain_to_multiif": lambda: random.randint(1, 1), + "optimize_if_transform_strings_to_enum": lambda: random.randint(1, 1), "optimize_read_in_order": lambda: random.randint(0, 1), - "optimize_or_like_chain": lambda: random.randint(0, 1), - "optimize_substitute_columns": lambda: random.randint(0, 1), - "optimize_using_constraints": lambda: random.randint(0, 1), + "optimize_or_like_chain": lambda: random.randint(1, 1), + "optimize_substitute_columns": lambda: random.randint(1, 1), + "optimize_using_constraints": lambda: random.randint(1, 1), "enable_multiple_prewhere_read_steps": lambda: random.randint(0, 1), "read_in_order_two_level_merge_threshold": lambda: random.randint(0, 100), "optimize_aggregation_in_order": lambda: random.randint(0, 1), From 157555a45bcd11949a496603b224b5f07883fad8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 28 Nov 2023 16:59:12 +0000 Subject: [PATCH 070/331] Some fixups and consistency fixes --- src/DataTypes/IDataType.h | 2 +- src/Functions/GatherUtils/Sources.h | 113 +++++++----- src/Functions/substring.cpp | 161 ++++++++---------- .../0_stateless/00493_substring_of_enum.sql | 14 +- 4 files changed, 150 insertions(+), 140 deletions(-) diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 98f7e0cb06f..e287b5879a2 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -446,9 +446,9 @@ template inline bool isFloat(const T & data_type) { return WhichDat template inline bool isNativeNumber(const T & data_type) { return WhichDataType(data_type).isNativeNumber(); } template inline bool isNumber(const T & data_type) { return WhichDataType(data_type).isNumber(); } -template inline bool isEnum(const T & data_type) { return WhichDataType(data_type).isEnum(); } template inline bool isEnum8(const T & data_type) { return WhichDataType(data_type).isEnum8(); } template inline bool isEnum16(const T & data_type) { return WhichDataType(data_type).isEnum16(); } +template inline bool isEnum(const T & data_type) { return WhichDataType(data_type).isEnum(); } template inline bool isDate(const T & data_type) { return WhichDataType(data_type).isDate(); } template inline bool isDate32(const T & data_type) { return WhichDataType(data_type).isDate32(); } diff --git a/src/Functions/GatherUtils/Sources.h b/src/Functions/GatherUtils/Sources.h index a3b5c21b89c..329f71203bf 100644 --- a/src/Functions/GatherUtils/Sources.h +++ b/src/Functions/GatherUtils/Sources.h @@ -58,8 +58,8 @@ struct NumericArraySource : public ArraySourceImpl> } explicit NumericArraySource(const ColumnArray & arr) - : column(typeid_cast(arr.getData())) - , elements(typeid_cast(arr.getData()).getData()), offsets(arr.getOffsets()) + : column(typeid_cast(arr.getData())) + , elements(typeid_cast(arr.getData()).getData()), offsets(arr.getOffsets()) { } @@ -156,17 +156,22 @@ struct ConstSource : public Base size_t row_num = 0; explicit ConstSource(const ColumnConst & col_) - : Base(static_cast(col_.getDataColumn())), total_rows(col_.size()) + : Base(static_cast(col_.getDataColumn())) + , total_rows(col_.size()) { } template - ConstSource(const ColumnType & col_, size_t total_rows_) : Base(col_), total_rows(total_rows_) + ConstSource(const ColumnType & col_, size_t total_rows_) + : Base(col_) + , total_rows(total_rows_) { } template - ConstSource(const ColumnType & col_, const NullMap & null_map_, size_t total_rows_) : Base(col_, null_map_), total_rows(total_rows_) + ConstSource(const ColumnType & col_, const NullMap & null_map_, size_t total_rows_) + : Base(col_, null_map_) + , total_rows(total_rows_) { } @@ -242,7 +247,8 @@ struct StringSource ColumnString::Offset prev_offset = 0; explicit StringSource(const ColumnString & col) - : elements(col.getChars()), offsets(col.getOffsets()) + : elements(col.getChars()) + , offsets(col.getOffsets()) { } @@ -315,76 +321,91 @@ struct StringSource } }; -template +/// Treats Enum values as Strings, modeled after StringSource +template struct EnumSource { - using Column = ColumnVector; + using Column = typename EnumDataType::ColumnType; using Slice = NumericArraySlice; using SinkType = StringSink; - const typename ColumnVector::Container & data; - const DataTypeEnum & data_type; + const typename Column::Container & data; + const EnumDataType & data_type; size_t row_num = 0; - explicit EnumSource(const Column & col, const DataTypeEnum & data_type_) : data(col.getData()), data_type(data_type_) { } + EnumSource(const Column & col, const EnumDataType & data_type_) + : data(col.getData()) + , data_type(data_type_) + { + } - void next() { ++row_num; } + void next() + { + ++row_num; + } - bool isEnd() const { return row_num == data.size(); } + bool isEnd() const + { + return row_num == data.size(); + } - size_t rowNum() const { return row_num; } + size_t rowNum() const + { + return row_num; + } - size_t getSizeForReserve() const { return data.size(); } + size_t getSizeForReserve() const + { + return data.size(); + } size_t getElementSize() const { - StringRef name = data_type.getNameForValue(data[row_num]); - return name.size; + std::string_view name = data_type.getNameForValue(data[row_num]).toView(); + return name.size(); } - size_t getColumnSize() const { return data.size(); } + size_t getColumnSize() const + { + return data.size(); + } Slice getWhole() const { - StringRef name = data_type.getNameForValue(data[row_num]); - const UInt8 * name_data = reinterpret_cast(name.data); - return {name_data, name.size}; + std::string_view name = data_type.getNameForValue(data[row_num]).toView(); + return {reinterpret_cast(name.data()), name.size()}; } Slice getSliceFromLeft(size_t offset) const { - StringRef name = data_type.getNameForValue(data[row_num]); - if (offset >= name.size) - return {nullptr, 0}; - const UInt8 * name_data = reinterpret_cast(name.data); - return {name_data + offset, name.size - offset}; + std::string_view name = data_type.getNameForValue(data[row_num]).toView(); + if (offset >= name.size()) + return {reinterpret_cast(name.data()), 0}; + return {reinterpret_cast(name.data()) + offset, name.size() - offset}; } Slice getSliceFromLeft(size_t offset, size_t length) const { - StringRef name = data_type.getNameForValue(data[row_num]); - if (offset >= name.size) - return {nullptr, 0}; - const UInt8 * name_data = reinterpret_cast(name.data); - return {name_data + offset, std::min(length, name.size - offset)}; + std::string_view name = data_type.getNameForValue(data[row_num]).toView(); + if (offset >= name.size()) + return {reinterpret_cast(name.data()), 0}; + return {reinterpret_cast(name.data()) + offset, std::min(length, name.size() - offset)}; } Slice getSliceFromRight(size_t offset) const { - StringRef name = data_type.getNameForValue(data[row_num]); - const UInt8 * name_data = reinterpret_cast(name.data); - if (offset > name.size) - return {name_data, name.size}; - return {name_data + name.size - offset, offset}; + std::string_view name = data_type.getNameForValue(data[row_num]).toView(); + if (offset > name.size()) + return {reinterpret_cast(name.data()), name.size()}; + return {reinterpret_cast(name.data()) + name.size() - offset, offset}; } Slice getSliceFromRight(size_t offset, size_t length) const { - StringRef name = data_type.getNameForValue(data[row_num]); - const UInt8 * name_data = reinterpret_cast(name.data); - if (offset > name.size) - return {name_data, length + name.size > offset ? std::min(name.size, length + name.size - offset) : 0}; - return {name_data + name.size - offset, std::min(length, offset)}; + std::string_view name = data_type.getNameForValue(data[row_num]).toView(); + if (offset > name.size()) + return {reinterpret_cast(name.data()), length + name.size() > offset ? std::min(name.size(), length + name.size() - offset) : 0}; + return {reinterpret_cast(name.data()) + name.size() - offset, std::min(length, offset)}; } }; @@ -494,7 +515,7 @@ struct FixedStringSource size_t column_size = 0; explicit FixedStringSource(const ColumnFixedString & col) - : string_size(col.getN()) + : string_size(col.getN()) { const auto & chars = col.getChars(); pos = chars.data(); @@ -628,7 +649,8 @@ struct GenericArraySource : public ArraySourceImpl } explicit GenericArraySource(const ColumnArray & arr) - : elements(arr.getData()), offsets(arr.getOffsets()) + : elements(arr.getData()) + , offsets(arr.getOffsets()) { } @@ -888,7 +910,10 @@ struct NullableValueSource : public ValueSource const NullMap & null_map; template - explicit NullableValueSource(const Column & col, const NullMap & null_map_) : ValueSource(col), null_map(null_map_) {} + NullableValueSource(const Column & col, const NullMap & null_map_) + : ValueSource(col) + , null_map(null_map_) + {} void accept(ValueSourceVisitor & visitor) override { visitor.visit(*this); } diff --git a/src/Functions/substring.cpp b/src/Functions/substring.cpp index f42452c9d99..ac6a24fbc11 100644 --- a/src/Functions/substring.cpp +++ b/src/Functions/substring.cpp @@ -1,16 +1,16 @@ -#include -#include -#include -#include #include +#include +#include +#include +#include #include #include -#include +#include #include -#include #include #include -#include +#include +#include #include @@ -31,40 +31,40 @@ namespace { /// If 'is_utf8' - measure offset and length in code points instead of bytes. -/// UTF8 variant is not available for FixedString arguments. template class FunctionSubstring : public IFunction { public: static constexpr auto name = is_utf8 ? "substringUTF8" : "substring"; - static FunctionPtr create(ContextPtr) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + String getName() const override { return name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - bool useDefaultImplementationForConstants() const override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - size_t number_of_arguments = arguments.size(); + const size_t number_of_arguments = arguments.size(); if (number_of_arguments < 2 || number_of_arguments > 3) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Number of arguments for function {} doesn't match: " "passed {}, should be 2 or 3", getName(), number_of_arguments); - if ((is_utf8 && !isString(arguments[0])) || (!isStringOrFixedString(arguments[0]) && !isEnum(arguments[0]))) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", - arguments[0]->getName(), getName()); + if constexpr (is_utf8) + { + /// UTF8 variant is not available for FixedString and Enum arguments. + if (!isString(arguments[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}", + arguments[0]->getName(), getName()); + } + else + { + if (!isStringOrFixedString(arguments[0]) && !isEnum(arguments[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}", + arguments[0]->getName(), getName()); + } if (!isNativeNumber(arguments[1])) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of second argument of function {}", @@ -78,44 +78,40 @@ public: } template - ColumnPtr executeForSource(const ColumnPtr & column_start, const ColumnPtr & column_length, - const ColumnConst * column_start_const, const ColumnConst * column_length_const, - Int64 start_value, Int64 length_value, Source && source, - size_t input_rows_count) const + ColumnPtr executeForSource(const ColumnPtr & column_offset, const ColumnPtr & column_length, + bool column_offset_const, bool column_length_const, + Int64 offset, Int64 length, + Source && source, size_t input_rows_count) const { auto col_res = ColumnString::create(); if (!column_length) { - if (column_start_const) + if (column_offset_const) { - if (start_value > 0) - sliceFromLeftConstantOffsetUnbounded( - source, StringSink(*col_res, input_rows_count), static_cast(start_value - 1)); - else if (start_value < 0) - sliceFromRightConstantOffsetUnbounded( - source, StringSink(*col_res, input_rows_count), -static_cast(start_value)); + if (offset > 0) + sliceFromLeftConstantOffsetUnbounded(source, StringSink(*col_res, input_rows_count), static_cast(offset - 1)); + else if (offset < 0) + sliceFromRightConstantOffsetUnbounded(source, StringSink(*col_res, input_rows_count), -static_cast(offset)); else throw Exception(ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX, "Indices in strings are 1-based"); } else - sliceDynamicOffsetUnbounded(source, StringSink(*col_res, input_rows_count), *column_start); + sliceDynamicOffsetUnbounded(source, StringSink(*col_res, input_rows_count), *column_offset); } else { - if (column_start_const && column_length_const) + if (column_offset_const && column_length_const) { - if (start_value > 0) - sliceFromLeftConstantOffsetBounded( - source, StringSink(*col_res, input_rows_count), static_cast(start_value - 1), length_value); - else if (start_value < 0) - sliceFromRightConstantOffsetBounded( - source, StringSink(*col_res, input_rows_count), -static_cast(start_value), length_value); + if (offset > 0) + sliceFromLeftConstantOffsetBounded(source, StringSink(*col_res, input_rows_count), static_cast(offset - 1), length); + else if (offset < 0) + sliceFromRightConstantOffsetBounded(source, StringSink(*col_res, input_rows_count), -static_cast(offset), length); else throw Exception(ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX, "Indices in strings are 1-based"); } else - sliceDynamicOffsetBounded(source, StringSink(*col_res, input_rows_count), *column_start, *column_length); + sliceDynamicOffsetBounded(source, StringSink(*col_res, input_rows_count), *column_offset, *column_length); } return col_res; @@ -123,71 +119,60 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - size_t number_of_arguments = arguments.size(); + const size_t number_of_arguments = arguments.size(); - ColumnPtr column_arg0 = arguments[0].column; - ColumnPtr column_start = arguments[1].column; + ColumnPtr column_string = arguments[0].column; + ColumnPtr column_offset = arguments[1].column; ColumnPtr column_length; - if (number_of_arguments == 3) column_length = arguments[2].column; - const ColumnConst * column_start_const = checkAndGetColumn(column_start.get()); + const ColumnConst * column_offset_const = checkAndGetColumn(column_offset.get()); const ColumnConst * column_length_const = nullptr; - if (number_of_arguments == 3) column_length_const = checkAndGetColumn(column_length.get()); - Int64 start_value = 0; - Int64 length_value = 0; + Int64 offset = 0; + Int64 length = 0; - if (column_start_const) - start_value = column_start_const->getInt(0); + if (column_offset_const) + offset = column_offset_const->getInt(0); if (column_length_const) - length_value = column_length_const->getInt(0); + length = column_length_const->getInt(0); if constexpr (is_utf8) { - if (const ColumnString * col = checkAndGetColumn(column_arg0.get())) - return executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, - length_value, UTF8StringSource(*col), input_rows_count); - if (const ColumnConst * col_const = checkAndGetColumnConst(column_arg0.get())) - return executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, - length_value, ConstSource(*col_const), input_rows_count); - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", - arguments[0].column->getName(), getName()); + if (const ColumnString * col = checkAndGetColumn(column_string.get())) + return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, UTF8StringSource(*col), input_rows_count); + if (const ColumnConst * col_const = checkAndGetColumnConst(column_string.get())) + return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, ConstSource(*col_const), input_rows_count); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", arguments[0].column->getName(), getName()); } else { - if (const ColumnString * col = checkAndGetColumn(column_arg0.get())) - return executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, - length_value, StringSource(*col), input_rows_count); - if (const ColumnFixedString * col_fixed = checkAndGetColumn(column_arg0.get())) - return executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, - length_value, FixedStringSource(*col_fixed), input_rows_count); - if (const ColumnConst * col_const = checkAndGetColumnConst(column_arg0.get())) - return executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, - length_value, ConstSource(*col_const), input_rows_count); - if (const ColumnConst * col_const_fixed = checkAndGetColumnConst(column_arg0.get())) - return executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, - length_value, ConstSource(*col_const_fixed), input_rows_count); - if (isEnum8(arguments[0].type)) - if (const ColumnVector * col_enum8 = checkAndGetColumn>(column_arg0.get())) + if (const ColumnString * col = checkAndGetColumn(column_string.get())) + return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, StringSource(*col), input_rows_count); + if (const ColumnFixedString * col_fixed = checkAndGetColumn(column_string.get())) + return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, FixedStringSource(*col_fixed), input_rows_count); + if (const ColumnConst * col_const = checkAndGetColumnConst(column_string.get())) + return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, ConstSource(*col_const), input_rows_count); + if (const ColumnConst * col_const_fixed = checkAndGetColumnConst(column_string.get())) + return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, ConstSource(*col_const_fixed), input_rows_count); + if (isEnum(arguments[0].type)) + { + if (const typename DataTypeEnum8::ColumnType * col_enum8 = checkAndGetColumn(column_string.get())) { - const auto * enum_type = typeid_cast *>(arguments[0].type.get()); - return executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, - length_value, EnumSource(*col_enum8, *enum_type), input_rows_count); + const auto * type_enum8 = assert_cast(arguments[0].type.get()); + return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, EnumSource(*col_enum8, *type_enum8), input_rows_count); } - if (isEnum16(arguments[0].type)) - if (const ColumnVector * col_enum16 = checkAndGetColumn>(column_arg0.get())) + if (const typename DataTypeEnum16::ColumnType * col_enum16 = checkAndGetColumn(column_string.get())) { - const auto * enum_type = typeid_cast *>(arguments[0].type.get()); - return executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, - length_value, EnumSource(*col_enum16, *enum_type), input_rows_count); + const auto * type_enum16 = assert_cast(arguments[0].type.get()); + return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, EnumSource(*col_enum16, *type_enum16), input_rows_count); } + } - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", - arguments[0].column->getName(), getName()); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", arguments[0].column->getName(), getName()); } } }; @@ -197,8 +182,8 @@ public: REGISTER_FUNCTION(Substring) { factory.registerFunction>({}, FunctionFactory::CaseInsensitive); - factory.registerAlias("substr", "substring", FunctionFactory::CaseInsensitive); - factory.registerAlias("mid", "substring", FunctionFactory::CaseInsensitive); /// from MySQL dialect + factory.registerAlias("substr", "substring", FunctionFactory::CaseInsensitive); // MySQL alias + factory.registerAlias("mid", "substring", FunctionFactory::CaseInsensitive); /// MySQL alias factory.registerFunction>({}, FunctionFactory::CaseSensitive); } diff --git a/tests/queries/0_stateless/00493_substring_of_enum.sql b/tests/queries/0_stateless/00493_substring_of_enum.sql index ba9fc630490..cdda76e88a7 100644 --- a/tests/queries/0_stateless/00493_substring_of_enum.sql +++ b/tests/queries/0_stateless/00493_substring_of_enum.sql @@ -1,6 +1,6 @@ -DROP TABLE IF EXISTS substring_enums_test; -CREATE TABLE substring_enums_test(e8 Enum('hello' = -5, 'world' = 15), e16 Enum('shark' = -999, 'eagle' = 9999)) ENGINE MergeTree ORDER BY tuple(); -INSERT INTO TABLE substring_enums_test VALUES ('hello', 'shark'), ('world', 'eagle'); +DROP TABLE IF EXISTS tab; +CREATE TABLE tab(e8 Enum8('hello' = -5, 'world' = 15), e16 Enum16('shark' = -999, 'eagle' = 9999)) ENGINE MergeTree ORDER BY tuple(); +INSERT INTO TABLE tab VALUES ('hello', 'shark'), ('world', 'eagle'); SELECT '-- Positive offsets (slice from left)'; WITH cte AS (SELECT number + 1 AS n FROM system.numbers LIMIT 6), @@ -8,7 +8,7 @@ WITH cte AS (SELECT number + 1 AS n FROM system.numbers LIMIT 6), SELECT 'Offset: ', p.offset, 'Length: ', p.length, substring(e8, p.offset) AS s1, substring(e16, p.offset) AS s2, substring(e8, p.offset, p.length) AS s3, substring(e16, p.offset, p.length) AS s4 -FROM substring_enums_test LEFT JOIN permutations AS p ON true; +FROM tab LEFT JOIN permutations AS p ON true; SELECT '-- Negative offsets (slice from right)'; WITH cte AS (SELECT number + 1 AS n FROM system.numbers LIMIT 6), @@ -16,7 +16,7 @@ WITH cte AS (SELECT number + 1 AS n FROM system.numbers LIMIT 6), SELECT 'Offset: ', p.offset, 'Length: ', p.length, substring(e8, p.offset) AS s1, substring(e16, p.offset) AS s2, substring(e8, p.offset, p.length) AS s3, substring(e16, p.offset, p.length) AS s4 -FROM substring_enums_test LEFT JOIN permutations AS p ON true; +FROM tab LEFT JOIN permutations AS p ON true; SELECT '-- Zero offset/length'; WITH cte AS (SELECT number AS n FROM system.numbers LIMIT 2), @@ -24,9 +24,9 @@ WITH cte AS (SELECT number AS n FROM system.numbers LIMIT 2), SELECT 'Offset: ', p.offset, 'Length: ', p.length, substring(e8, p.offset) AS s1, substring(e16, p.offset) AS s2, substring(e8, p.offset, p.length) AS s3, substring(e16, p.offset, p.length) AS s4 -FROM substring_enums_test LEFT JOIN permutations AS p ON true; +FROM tab LEFT JOIN permutations AS p ON true; SELECT '-- Constant enums'; SELECT substring(CAST('foo', 'Enum8(\'foo\' = 1)'), 1, 1), substring(CAST('foo', 'Enum16(\'foo\' = 1111)'), 1, 2); -DROP TABLE substring_enums_test; +DROP TABLE tab; From b493ce23852dc74e9001832485a2ad00e966e6c2 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 29 Nov 2023 14:19:38 +0000 Subject: [PATCH 071/331] Better JSON -> JSONEachRow fallback without catching exceptions --- src/Formats/JSONUtils.cpp | 116 ++++++++++++++++++ src/Formats/JSONUtils.h | 3 + .../Formats/Impl/JSONRowInputFormat.cpp | 50 +++----- 3 files changed, 135 insertions(+), 34 deletions(-) diff --git a/src/Formats/JSONUtils.cpp b/src/Formats/JSONUtils.cpp index 349945bbd54..7ddfdb6b572 100644 --- a/src/Formats/JSONUtils.cpp +++ b/src/Formats/JSONUtils.cpp @@ -564,6 +564,15 @@ namespace JSONUtils skipWhitespaceIfAny(in); } + bool checkAndSkipColon(ReadBuffer & in) + { + skipWhitespaceIfAny(in); + if (!checkChar(':', in)) + return false; + skipWhitespaceIfAny(in); + return true; + } + String readFieldName(ReadBuffer & in) { skipWhitespaceIfAny(in); @@ -573,6 +582,12 @@ namespace JSONUtils return field; } + bool tryReadFieldName(ReadBuffer & in, String & field) + { + skipWhitespaceIfAny(in); + return tryReadJSONStringInto(field, in) && checkAndSkipColon(in); + } + String readStringField(ReadBuffer & in) { skipWhitespaceIfAny(in); @@ -582,6 +597,15 @@ namespace JSONUtils return value; } + bool tryReadStringField(ReadBuffer & in, String & value) + { + skipWhitespaceIfAny(in); + if (!tryReadJSONStringInto(value, in)) + return false; + skipWhitespaceIfAny(in); + return true; + } + void skipArrayStart(ReadBuffer & in) { skipWhitespaceIfAny(in); @@ -628,6 +652,15 @@ namespace JSONUtils skipWhitespaceIfAny(in); } + bool checkAndSkipObjectStart(ReadBuffer & in) + { + skipWhitespaceIfAny(in); + if (!checkChar('{', in)) + return false; + skipWhitespaceIfAny(in); + return true; + } + bool checkAndSkipObjectEnd(ReadBuffer & in) { skipWhitespaceIfAny(in); @@ -644,6 +677,15 @@ namespace JSONUtils skipWhitespaceIfAny(in); } + bool checkAndSkipComma(ReadBuffer & in) + { + skipWhitespaceIfAny(in); + if (!checkChar(',', in)) + return false; + skipWhitespaceIfAny(in); + return true; + } + std::pair readStringFieldNameAndValue(ReadBuffer & in) { auto field_name = readFieldName(in); @@ -651,6 +693,11 @@ namespace JSONUtils return {field_name, field_value}; } + bool tryReadStringFieldNameAndValue(ReadBuffer & in, std::pair & field_and_value) + { + return tryReadFieldName(in, field_and_value.first) && tryReadStringField(in, field_and_value.second); + } + NameAndTypePair readObjectWithNameAndType(ReadBuffer & in) { skipObjectStart(in); @@ -673,6 +720,44 @@ namespace JSONUtils return name_and_type; } + bool tryReadObjectWithNameAndType(ReadBuffer & in, NameAndTypePair & name_and_type) + { + if (!checkAndSkipObjectStart(in)) + return false; + + std::pair first_field_and_value; + if (!tryReadStringFieldNameAndValue(in, first_field_and_value)) + return false; + + if (!checkAndSkipComma(in)) + return false; + + std::pair second_field_and_value; + if (!tryReadStringFieldNameAndValue(in, second_field_and_value)) + return false; + + if (first_field_and_value.first == "name" && second_field_and_value.first == "type") + { + auto type = DataTypeFactory::instance().tryGet(second_field_and_value.second); + if (!type) + return false; + name_and_type = {first_field_and_value.second, type}; + } + else if (second_field_and_value.first == "name" && first_field_and_value.first == "type") + { + auto type = DataTypeFactory::instance().tryGet(first_field_and_value.second); + if (!type) + return false; + name_and_type = {second_field_and_value.second, type}; + } + else + { + return false; + } + + return checkAndSkipObjectEnd(in); + } + NamesAndTypesList readMetadata(ReadBuffer & in) { auto field_name = readFieldName(in); @@ -693,6 +778,37 @@ namespace JSONUtils return names_and_types; } + bool tryReadMetadata(ReadBuffer & in, NamesAndTypesList & names_and_types) + { + String field_name; + if (!tryReadFieldName(in, field_name) || field_name != "meta") + return false; + + if (!checkAndSkipArrayStart(in)) + return false; + + bool first = true; + while (!checkAndSkipArrayEnd(in)) + { + if (!first) + { + if (!checkAndSkipComma(in)) + return false; + } + else + { + first = false; + } + + NameAndTypePair name_and_type; + if (!tryReadObjectWithNameAndType(in, name_and_type)) + return false; + names_and_types.push_back(name_and_type); + } + + return !names_and_types.empty(); + } + void validateMetadataByHeader(const NamesAndTypesList & names_and_types_from_metadata, const Block & header) { for (const auto & [name, type] : names_and_types_from_metadata) diff --git a/src/Formats/JSONUtils.h b/src/Formats/JSONUtils.h index cd6b5ff8171..a770ded9687 100644 --- a/src/Formats/JSONUtils.h +++ b/src/Formats/JSONUtils.h @@ -112,6 +112,7 @@ namespace JSONUtils void skipColon(ReadBuffer & in); void skipComma(ReadBuffer & in); + bool checkAndSkipComma(ReadBuffer & in); String readFieldName(ReadBuffer & in); @@ -122,9 +123,11 @@ namespace JSONUtils void skipObjectStart(ReadBuffer & in); void skipObjectEnd(ReadBuffer & in); + bool checkAndSkipObjectStart(ReadBuffer & in); bool checkAndSkipObjectEnd(ReadBuffer & in); NamesAndTypesList readMetadata(ReadBuffer & in); + bool tryReadMetadata(ReadBuffer & in, NamesAndTypesList & names_and_types); NamesAndTypesList readMetadataAndValidateHeader(ReadBuffer & in, const Block & header); void validateMetadataByHeader(const NamesAndTypesList & names_and_types_from_metadata, const Block & header); diff --git a/src/Processors/Formats/Impl/JSONRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONRowInputFormat.cpp index f083a00f766..fc4c868b604 100644 --- a/src/Processors/Formats/Impl/JSONRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowInputFormat.cpp @@ -30,38 +30,24 @@ void JSONRowInputFormat::readPrefix() NamesAndTypesList names_and_types_from_metadata; /// Try to parse metadata, if failed, try to parse data as JSONEachRow format. - try + if (JSONUtils::checkAndSkipObjectStart(*peekable_buf) + && JSONUtils::tryReadMetadata(*peekable_buf, names_and_types_from_metadata) + && JSONUtils::checkAndSkipComma(*peekable_buf) + && JSONUtils::skipUntilFieldInObject(*peekable_buf, "data") + && JSONUtils::checkAndSkipArrayStart(*peekable_buf)) { - JSONUtils::skipObjectStart(*peekable_buf); - names_and_types_from_metadata = JSONUtils::readMetadata(*peekable_buf); - JSONUtils::skipComma(*peekable_buf); - if (!JSONUtils::skipUntilFieldInObject(*peekable_buf, "data")) - throw Exception(ErrorCodes::INCORRECT_DATA, "Expected field \"data\" with table content"); - - JSONUtils::skipArrayStart(*peekable_buf); data_in_square_brackets = true; + if (validate_types_from_metadata) + { + JSONUtils::validateMetadataByHeader(names_and_types_from_metadata, getPort().getHeader()); + } } - catch (const ParsingException &) + else { parse_as_json_each_row = true; - } - catch (const Exception & e) - { - if (e.code() != ErrorCodes::INCORRECT_DATA) - throw; - - parse_as_json_each_row = true; - } - - if (parse_as_json_each_row) - { peekable_buf->rollbackToCheckpoint(); JSONEachRowRowInputFormat::readPrefix(); } - else if (validate_types_from_metadata) - { - JSONUtils::validateMetadataByHeader(names_and_types_from_metadata, getPort().getHeader()); - } } void JSONRowInputFormat::readSuffix() @@ -103,16 +89,12 @@ NamesAndTypesList JSONRowSchemaReader::readSchema() skipBOMIfExists(*peekable_buf); PeekableReadBufferCheckpoint checkpoint(*peekable_buf); /// Try to parse metadata, if failed, try to parse data as JSONEachRow format - try - { - JSONUtils::skipObjectStart(*peekable_buf); - return JSONUtils::readMetadata(*peekable_buf); - } - catch (...) - { - peekable_buf->rollbackToCheckpoint(true); - return JSONEachRowSchemaReader::readSchema(); - } + NamesAndTypesList names_and_types; + if (JSONUtils::checkAndSkipObjectStart(*peekable_buf) && JSONUtils::tryReadMetadata(*peekable_buf, names_and_types)) + return names_and_types; + + peekable_buf->rollbackToCheckpoint(true); + return JSONEachRowSchemaReader::readSchema(); } void registerInputFormatJSON(FormatFactory & factory) From 1d840ca02693795890d8bd2ab447a7f7536ce888 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 29 Nov 2023 20:49:53 +0100 Subject: [PATCH 072/331] Fix style --- src/Processors/Formats/Impl/JSONRowInputFormat.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Processors/Formats/Impl/JSONRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONRowInputFormat.cpp index fc4c868b604..fcc7f0f8381 100644 --- a/src/Processors/Formats/Impl/JSONRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowInputFormat.cpp @@ -7,11 +7,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int INCORRECT_DATA; -} - JSONRowInputFormat::JSONRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_) : JSONRowInputFormat(std::make_unique(in_), header_, params_, format_settings_) { From d9f4b4d2c05d553d287f4f3551c406fe599b50d2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Thu, 30 Nov 2023 10:56:31 +0800 Subject: [PATCH 073/331] support parts, tables, databases --- programs/server/Server.cpp | 3 + src/Core/ServerSettings.h | 3 + src/Databases/DatabaseLazy.cpp | 7 ++- src/Databases/DatabaseOnDisk.cpp | 1 - src/Databases/DatabasesCommon.cpp | 2 - src/Databases/IDatabase.cpp | 15 +++++ src/Databases/IDatabase.h | 4 +- src/Interpreters/Context.cpp | 34 +++++++++++ src/Interpreters/Context.h | 3 + tests/config/config.d/max_num_to_warn.xml | 5 ++ tests/config/install.sh | 1 + .../02931_max_num_to_warn.reference | 3 + .../0_stateless/02931_max_num_to_warn.sql | 61 +++++++++++++++++++ 13 files changed, 136 insertions(+), 6 deletions(-) create mode 100644 tests/config/config.d/max_num_to_warn.xml create mode 100644 tests/queries/0_stateless/02931_max_num_to_warn.reference create mode 100644 tests/queries/0_stateless/02931_max_num_to_warn.sql diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 11ad06640c8..c3d7bff00d6 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1280,6 +1280,9 @@ try global_context->setMaxTableSizeToDrop(server_settings_.max_table_size_to_drop); global_context->setMaxPartitionSizeToDrop(server_settings_.max_partition_size_to_drop); + global_context->setMaxTableNumToWarn(server_settings_.max_table_num_to_warn); + global_context->setMaxDatabaseNumToWarn(server_settings_.max_database_num_to_warn); + global_context->setMaxPartNumToWarn(server_settings_.max_part_num_to_warn); ConcurrencyControl::SlotCount concurrent_threads_soft_limit = ConcurrencyControl::Unlimited; if (server_settings_.concurrent_threads_soft_limit_num > 0 && server_settings_.concurrent_threads_soft_limit_num < concurrent_threads_soft_limit) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index eca4b7424de..0a48c73c1fd 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -79,6 +79,9 @@ namespace DB \ M(UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0) \ M(UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0) \ + M(UInt64, max_table_num_to_warn, 5000lu, "If number of tables is greater than this value, server will create a warning that will displayed to user.", 0) \ + M(UInt64, max_database_num_to_warn, 1000lu, "If number of databases is greater than this value, server will create a warning that will displayed to user.", 0) \ + M(UInt64, max_part_num_to_warn, 100000lu, "If number of databases is greater than this value, server will create a warning that will displayed to user.", 0) \ M(UInt64, concurrent_threads_soft_limit_num, 0, "Sets how many concurrent thread can be allocated before applying CPU pressure. Zero means unlimited.", 0) \ M(UInt64, concurrent_threads_soft_limit_ratio_to_cores, 0, "Same as concurrent_threads_soft_limit_num, but with ratio to cores.", 0) \ \ diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index 9070c0b6ee7..75e4b19d628 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -18,6 +18,10 @@ namespace fs = std::filesystem; +namespace CurrentMetrics { + extern const Metric AttachedTable; +} + namespace DB { @@ -64,7 +68,6 @@ void DatabaseLazy::createTable( SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); }); if (!endsWith(table->getName(), "Log")) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Lazy engine can be used only with *Log tables."); - std::cout<<"======= Flag 1"<second.expiration_iterator = cache_expiration_queue.emplace(cache_expiration_queue.end(), current_time, table_name); + CurrentMetrics::add(CurrentMetrics::AttachedTable, 1); } StoragePtr DatabaseLazy::detachTable(ContextPtr /* context */, const String & table_name) @@ -190,6 +194,7 @@ StoragePtr DatabaseLazy::detachTable(ContextPtr /* context */, const String & ta if (it->second.expiration_iterator != cache_expiration_queue.end()) cache_expiration_queue.erase(it->second.expiration_iterator); tables_cache.erase(it); + CurrentMetrics::sub(CurrentMetrics::AttachedTable, 1); } return res; } diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f1fc6bbdd7a..549711150b8 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -263,7 +263,6 @@ void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const Stora try { /// Add a table to the map of known tables. - std::cout<<"===== flag 3"<getStorageID(); if (table_id.database_name != database_name) throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database was renamed to `{}`, cannot create table in `{}`", diff --git a/src/Databases/IDatabase.cpp b/src/Databases/IDatabase.cpp index 09640d2f86e..b4fd5ea4612 100644 --- a/src/Databases/IDatabase.cpp +++ b/src/Databases/IDatabase.cpp @@ -5,8 +5,14 @@ #include #include #include +#include +namespace CurrentMetrics +{ + extern const Metric AttachedDatabase; +} + namespace DB { @@ -29,6 +35,15 @@ StoragePtr IDatabase::getTable(const String & name, ContextPtr context) const throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} does not exist. Maybe you meant {}?", backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(name), backQuoteIfNeed(names[0])); } +IDatabase::IDatabase(String database_name_) : database_name(std::move(database_name_)) { + CurrentMetrics::add(CurrentMetrics::AttachedDatabase, 1); +} + +IDatabase::~IDatabase() +{ + CurrentMetrics::sub(CurrentMetrics::AttachedDatabase, 1); +} + std::vector> IDatabase::getTablesForBackup(const FilterByNameFunction &, const ContextPtr &) const { /// Cannot backup any table because IDatabase doesn't own any tables. diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index e886f1adae3..7a8f9cb2e2b 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -122,7 +122,7 @@ class IDatabase : public std::enable_shared_from_this { public: IDatabase() = delete; - explicit IDatabase(String database_name_) : database_name(std::move(database_name_)) {} + explicit IDatabase(String database_name_); /// Get name of database engine. virtual String getEngineName() const = 0; @@ -357,7 +357,7 @@ public: /// Creates a table restored from backup. virtual void createTableRestoredFromBackup(const ASTPtr & create_table_query, ContextMutablePtr context, std::shared_ptr restore_coordination, UInt64 timeout_ms); - virtual ~IDatabase() = default; + virtual ~IDatabase(); protected: virtual ASTPtr getCreateTableQueryImpl(const String & /*name*/, ContextPtr /*context*/, bool throw_on_error) const diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 185f9782da5..a2c94d59826 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -142,6 +142,9 @@ namespace CurrentMetrics extern const Metric IOWriterThreads; extern const Metric IOWriterThreadsActive; extern const Metric IOWriterThreadsScheduled; + extern const Metric AttachedTable; + extern const Metric AttachedDatabase; + extern const Metric PartsActive; } @@ -323,6 +326,10 @@ struct ContextSharedPart : boost::noncopyable std::optional replicated_merge_tree_settings TSA_GUARDED_BY(mutex); /// Settings of ReplicatedMergeTree* engines. std::atomic_size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default) std::atomic_size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default) + /// std::atomic_size_t max_table_size_to_warn; + std::atomic_size_t max_database_num_to_warn = 1000lu; + std::atomic_size_t max_table_num_to_warn = 5000lu; + std::atomic_size_t max_part_num_to_warn = 100000lu; /// No lock required for format_schema_path modified only during initialization String format_schema_path; /// Path to a directory that contains schema files used by input formats. mutable OnceFlag action_locks_manager_initialized; @@ -829,6 +836,15 @@ Strings Context::getWarnings() const { SharedLockGuard lock(shared->mutex); common_warnings = shared->warnings; + + if (CurrentMetrics::get(CurrentMetrics::AttachedTable) > static_cast(shared->max_table_num_to_warn)) + common_warnings.emplace_back(fmt::format("Attached tables is more than {}", shared->max_table_num_to_warn)); + + if (CurrentMetrics::get(CurrentMetrics::AttachedDatabase) > static_cast(shared->max_database_num_to_warn)) + common_warnings.emplace_back(fmt::format("Attached databases is more than {}", shared->max_table_num_to_warn)); + + if (CurrentMetrics::get(CurrentMetrics::PartsActive) > static_cast(shared->max_part_num_to_warn)) + common_warnings.emplace_back(fmt::format("Active parts is more than {}", shared->max_part_num_to_warn)); } /// Make setting's name ordered std::set obsolete_settings; @@ -3322,6 +3338,24 @@ UInt16 Context::getServerPort(const String & port_name) const return it->second; } +void Context::setMaxPartNumToWarn(size_t max_part_to_warn) +{ + SharedLockGuard lock(shared->mutex); + shared->max_part_num_to_warn = max_part_to_warn; +} + +void Context::setMaxTableNumToWarn(size_t max_table_to_warn) +{ + SharedLockGuard lock(shared->mutex); + shared->max_table_num_to_warn= max_table_to_warn; +} + +void Context::setMaxDatabaseNumToWarn(size_t max_database_to_warn) +{ + SharedLockGuard lock(shared->mutex); + shared->max_database_num_to_warn= max_database_to_warn; +} + std::shared_ptr Context::getCluster(const std::string & cluster_name) const { if (auto res = tryGetCluster(cluster_name)) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 7ae567548dd..517ccded6bd 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -838,6 +838,9 @@ public: void setHTTPHeaderFilter(const Poco::Util::AbstractConfiguration & config); const HTTPHeaderFilter & getHTTPHeaderFilter() const; + void setMaxTableNumToWarn(size_t max_table_to_warn); + void setMaxDatabaseNumToWarn(size_t max_database_to_warn); + void setMaxPartNumToWarn(size_t max_part_to_warn); /// The port that the server listens for executing SQL queries. UInt16 getTCPPort() const; diff --git a/tests/config/config.d/max_num_to_warn.xml b/tests/config/config.d/max_num_to_warn.xml new file mode 100644 index 00000000000..77d68998f8e --- /dev/null +++ b/tests/config/config.d/max_num_to_warn.xml @@ -0,0 +1,5 @@ + + 10 + 10 + 10 + diff --git a/tests/config/install.sh b/tests/config/install.sh index 417a413bbec..9538b59ad9d 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -16,6 +16,7 @@ mkdir -p $DEST_SERVER_PATH/users.d/ mkdir -p $DEST_CLIENT_PATH ln -sf $SRC_PATH/config.d/zookeeper_write.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/max_num_to_warn.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/listen.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/text_log.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/blob_storage_log.xml $DEST_SERVER_PATH/config.d/ diff --git a/tests/queries/0_stateless/02931_max_num_to_warn.reference b/tests/queries/0_stateless/02931_max_num_to_warn.reference new file mode 100644 index 00000000000..76d86352bfc --- /dev/null +++ b/tests/queries/0_stateless/02931_max_num_to_warn.reference @@ -0,0 +1,3 @@ +Attached tables is more than 10 +Attached databases is more than 10 +Active parts is more than 10 diff --git a/tests/queries/0_stateless/02931_max_num_to_warn.sql b/tests/queries/0_stateless/02931_max_num_to_warn.sql new file mode 100644 index 00000000000..cd9a4ebe5fa --- /dev/null +++ b/tests/queries/0_stateless/02931_max_num_to_warn.sql @@ -0,0 +1,61 @@ +CREATE TABLE test_max_num_to_warn_1 (id Int32, str String) Engine=Memory; +CREATE TABLE test_max_num_to_warn_2 (id Int32, str String) Engine=Memory; +CREATE TABLE test_max_num_to_warn_3 (id Int32, str String) Engine=Memory; +CREATE TABLE test_max_num_to_warn_4 (id Int32, str String) Engine=Memory; +CREATE TABLE test_max_num_to_warn_5 (id Int32, str String) Engine=Memory; +CREATE TABLE test_max_num_to_warn_6 (id Int32, str String) Engine=Memory; +CREATE TABLE test_max_num_to_warn_7 (id Int32, str String) Engine=Memory; +CREATE TABLE test_max_num_to_warn_8 (id Int32, str String) Engine=Memory; +CREATE TABLE test_max_num_to_warn_9 (id Int32, str String) Engine=Memory; +CREATE TABLE test_max_num_to_warn_10 (id Int32, str String) Engine=Memory; +CREATE TABLE test_max_num_to_warn_11 (id Int32, str String) Engine=Memory; + +CREATE DATABASE test_max_num_to_warn_1; +CREATE DATABASE test_max_num_to_warn_2; +CREATE DATABASE test_max_num_to_warn_3; +CREATE DATABASE test_max_num_to_warn_4; +CREATE DATABASE test_max_num_to_warn_5; +CREATE DATABASE test_max_num_to_warn_6; +CREATE DATABASE test_max_num_to_warn_7; +CREATE DATABASE test_max_num_to_warn_8; +CREATE DATABASE test_max_num_to_warn_9; +CREATE DATABASE test_max_num_to_warn_10; +CREATE DATABASE test_max_num_to_warn_11; + +INSERT INTO test_max_num_to_warn_1 VALUES (1, 'Hello'); +INSERT INTO test_max_num_to_warn_2 VALUES (1, 'Hello'); +INSERT INTO test_max_num_to_warn_3 VALUES (1, 'Hello'); +INSERT INTO test_max_num_to_warn_4 VALUES (1, 'Hello'); +INSERT INTO test_max_num_to_warn_5 VALUES (1, 'Hello'); +INSERT INTO test_max_num_to_warn_6 VALUES (1, 'Hello'); +INSERT INTO test_max_num_to_warn_7 VALUES (1, 'Hello'); +INSERT INTO test_max_num_to_warn_8 VALUES (1, 'Hello'); +INSERT INTO test_max_num_to_warn_9 VALUES (1, 'Hello'); +INSERT INTO test_max_num_to_warn_10 VALUES (1, 'Hello'); +INSERT INTO test_max_num_to_warn_11 VALUES (1, 'Hello'); + +SELECT * FROM system.warnings where message in ('Attached tables is more than 10', 'Attached databases is more than 10', 'Active parts is more than 10'); + +DROP TABLE test_max_num_to_warn_1; +DROP TABLE test_max_num_to_warn_2; +DROP TABLE test_max_num_to_warn_3; +DROP TABLE test_max_num_to_warn_4; +DROP TABLE test_max_num_to_warn_5; +DROP TABLE test_max_num_to_warn_6; +DROP TABLE test_max_num_to_warn_7; +DROP TABLE test_max_num_to_warn_8; +DROP TABLE test_max_num_to_warn_9; +DROP TABLE test_max_num_to_warn_10; +DROP TABLE test_max_num_to_warn_11; + +DROP DATABASE test_max_num_to_warn_1; +DROP DATABASE test_max_num_to_warn_2; +DROP DATABASE test_max_num_to_warn_3; +DROP DATABASE test_max_num_to_warn_4; +DROP DATABASE test_max_num_to_warn_5; +DROP DATABASE test_max_num_to_warn_6; +DROP DATABASE test_max_num_to_warn_7; +DROP DATABASE test_max_num_to_warn_8; +DROP DATABASE test_max_num_to_warn_9; +DROP DATABASE test_max_num_to_warn_10; +DROP DATABASE test_max_num_to_warn_11; From 64bd77fe9c0131ba9d9699c421e231e92242023a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 30 Nov 2023 14:05:19 +0100 Subject: [PATCH 074/331] Do not randomize optimize_functions_to_subcolumns --- tests/clickhouse-test | 1 - tests/queries/0_stateless/02498_analyzer_settings_push_down.sql | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index dc6614471a4..fcd8c7ba0b4 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -556,7 +556,6 @@ class SettingsRandomizer: "max_block_size": lambda: random.randint(8000, 100000), "max_threads": lambda: random.randint(1, 64), "optimize_append_index": lambda: random.randint(1, 1), - "optimize_functions_to_subcolumns": lambda: random.randint(1, 1), "optimize_syntax_fuse_functions": lambda: random.randint(1, 1), "optimize_if_chain_to_multiif": lambda: random.randint(1, 1), "optimize_if_transform_strings_to_enum": lambda: random.randint(1, 1), diff --git a/tests/queries/0_stateless/02498_analyzer_settings_push_down.sql b/tests/queries/0_stateless/02498_analyzer_settings_push_down.sql index 91bdce2cca9..67623869f0a 100644 --- a/tests/queries/0_stateless/02498_analyzer_settings_push_down.sql +++ b/tests/queries/0_stateless/02498_analyzer_settings_push_down.sql @@ -1,4 +1,5 @@ SET allow_experimental_analyzer = 1; +SET optimize_functions_to_subcolumns = 0; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table (id UInt64, value Tuple(a UInt64)) ENGINE=MergeTree ORDER BY id; From 3867adbb10a83631a06e0b40c74b1bd4fee7ba3e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 30 Nov 2023 19:19:26 +0100 Subject: [PATCH 075/331] Adapt tests and avoid using optimize_syntax_fuse_functions at all --- tests/clickhouse-test | 1 - tests/queries/0_stateless/01300_group_by_other_keys_having.sql | 1 + .../queries/0_stateless/01622_constraints_where_optimization.sql | 1 + .../0_stateless/02481_analyzer_optimize_grouping_sets_keys.sql | 1 + 4 files changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index fcd8c7ba0b4..fe93864b202 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -556,7 +556,6 @@ class SettingsRandomizer: "max_block_size": lambda: random.randint(8000, 100000), "max_threads": lambda: random.randint(1, 64), "optimize_append_index": lambda: random.randint(1, 1), - "optimize_syntax_fuse_functions": lambda: random.randint(1, 1), "optimize_if_chain_to_multiif": lambda: random.randint(1, 1), "optimize_if_transform_strings_to_enum": lambda: random.randint(1, 1), "optimize_read_in_order": lambda: random.randint(0, 1), diff --git a/tests/queries/0_stateless/01300_group_by_other_keys_having.sql b/tests/queries/0_stateless/01300_group_by_other_keys_having.sql index 911f61a62e2..203e8322ad9 100644 --- a/tests/queries/0_stateless/01300_group_by_other_keys_having.sql +++ b/tests/queries/0_stateless/01300_group_by_other_keys_having.sql @@ -1,4 +1,5 @@ set optimize_group_by_function_keys = 1; +set optimize_syntax_fuse_functions = 0; set allow_experimental_analyzer = 1; -- { echoOn } diff --git a/tests/queries/0_stateless/01622_constraints_where_optimization.sql b/tests/queries/0_stateless/01622_constraints_where_optimization.sql index 2818351a120..d41b1988bdd 100644 --- a/tests/queries/0_stateless/01622_constraints_where_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_where_optimization.sql @@ -1,5 +1,6 @@ SET convert_query_to_cnf = 1; SET optimize_using_constraints = 1; +SET optimize_append_index = 0; DROP TABLE IF EXISTS t_constraints_where; diff --git a/tests/queries/0_stateless/02481_analyzer_optimize_grouping_sets_keys.sql b/tests/queries/0_stateless/02481_analyzer_optimize_grouping_sets_keys.sql index de9208ef009..fef71fdf94f 100644 --- a/tests/queries/0_stateless/02481_analyzer_optimize_grouping_sets_keys.sql +++ b/tests/queries/0_stateless/02481_analyzer_optimize_grouping_sets_keys.sql @@ -1,4 +1,5 @@ set allow_experimental_analyzer = 1; +set optimize_syntax_fuse_functions = 0; EXPLAIN QUERY TREE run_passes=1 SELECT avg(log(2) * number) AS k FROM numbers(10000000) From 08f943462fa243ef0a18fa27a60be543e25cda74 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 30 Nov 2023 18:23:05 +0000 Subject: [PATCH 076/331] Delay totals port for creating sets as well. --- src/Processors/QueryPlan/CreatingSetsStep.cpp | 2 + src/QueryPipeline/Pipe.cpp | 211 ++++++++---------- src/QueryPipeline/Pipe.h | 5 + src/QueryPipeline/QueryPipelineBuilder.cpp | 14 +- 4 files changed, 110 insertions(+), 122 deletions(-) diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 3e4dfb0c7d1..37f81ffd160 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -111,6 +111,8 @@ QueryPipelineBuilderPtr CreatingSetsStep::updatePipeline(QueryPipelineBuilders p else delayed_pipeline = std::move(*pipelines.front()); + delayed_pipeline.dropTotalsAndExtremes(); + QueryPipelineProcessorsCollector collector(*main_pipeline, this); main_pipeline->addPipelineBefore(std::move(delayed_pipeline)); auto added_processors = collector.detachProcessors(); diff --git a/src/QueryPipeline/Pipe.cpp b/src/QueryPipeline/Pipe.cpp index b1c82d7a7e8..fd433638252 100644 --- a/src/QueryPipeline/Pipe.cpp +++ b/src/QueryPipeline/Pipe.cpp @@ -434,68 +434,130 @@ void Pipe::addTransform(ProcessorPtr transform) } void Pipe::addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort * extremes) +{ + addTransform(std::move(transform), + static_cast(nullptr), static_cast(nullptr), + totals, extremes); +} + +void Pipe::addTransform(ProcessorPtr transform, InputPort * totals, InputPort * extremes) +{ + addTransform(std::move(transform), + totals, extremes, + static_cast(nullptr), static_cast(nullptr)); +} + +void Pipe::addTransform( + ProcessorPtr transform, + InputPort * totals_in, InputPort * extremes_in, + OutputPort * totals_out, OutputPort * extremes_out) { if (output_ports.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add transform to empty Pipe"); + if (totals_in && !totals_port) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add transform consuming totals to Pipe because Pipe does not have totals"); + + if (extremes_in && !extremes_port) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add transform consuming extremes to Pipe because Pipe does not have extremes"); + + if (totals_out && !totals_in && totals_port) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add transform with totals to Pipe because it already has totals"); + + if (extremes_out && !extremes_in && extremes_port) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add transform with extremes to Pipe because it already has extremes"); + auto & inputs = transform->getInputs(); - if (inputs.size() != output_ports.size()) + auto & outputs = transform->getOutputs(); + + size_t expected_inputs = output_ports.size() + (totals_in ? 1 : 0) + (extremes_in ? 1 : 0); + if (inputs.size() != expected_inputs) throw Exception( ErrorCodes::LOGICAL_ERROR, "Cannot add transform {} to Pipe because it has {} input ports, but {} expected", transform->getName(), inputs.size(), - output_ports.size()); + expected_inputs); - if (totals && totals_port) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add transform with totals to Pipe because it already has totals"); + if (outputs.size() <= (totals_out ? 1 : 0) + (extremes_out ? 1 : 0)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add transform {} to Pipes because it has no outputs", + transform->getName()); - if (extremes && extremes_port) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add transform with extremes to Pipe because it already has extremes"); + bool found_totals_in = false; + bool found_extremes_in = false; - if (totals) - totals_port = totals; - if (extremes) - extremes_port = extremes; - - size_t next_output = 0; for (auto & input : inputs) { - connect(*output_ports[next_output], input); - ++next_output; + if (&input == totals_in) + found_totals_in = true; + else if (&input == extremes_in) + found_extremes_in = true; } - auto & outputs = transform->getOutputs(); + if (totals_in && !found_totals_in) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot add transform {} to Pipes because specified totals port does not belong to it", + transform->getName()); - output_ports.clear(); - output_ports.reserve(outputs.size()); + if (extremes_in && !found_extremes_in) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot add transform {} to Pipes because specified extremes port does not belong to it", + transform->getName()); - bool found_totals = false; - bool found_extremes = false; + bool found_totals_out = false; + bool found_extremes_out = false; for (auto & output : outputs) { - if (&output == totals) - found_totals = true; - else if (&output == extremes) - found_extremes = true; - else - output_ports.emplace_back(&output); + if (&output == totals_out) + found_totals_out = true; + else if (&output == extremes_out) + found_extremes_out = true; } - if (totals && !found_totals) + if (totals_out && !found_totals_out) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add transform {} to Pipes because specified totals port does not belong to it", transform->getName()); - if (extremes && !found_extremes) + if (extremes_out && !found_extremes_out) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add transform {} to Pipes because specified extremes port does not belong to it", transform->getName()); - if (output_ports.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add transform {} to Pipes because it has no outputs", - transform->getName()); + if (totals_in) + { + connect(*totals_port, *totals_in); + totals_port = nullptr; + } + if (extremes_in) + { + connect(*extremes_port, *extremes_in); + extremes_port = nullptr; + } + + totals_port = totals_out ? totals_out : totals_port; + extremes_port = extremes_out ? extremes_out : extremes_port; + + size_t next_output = 0; + for (auto & input : inputs) + { + if (&input != totals_in && &input != extremes_in) + { + connect(*output_ports[next_output], input); + ++next_output; + } + } + + output_ports.clear(); + output_ports.reserve(outputs.size()); + for (auto & output : outputs) + { + if (&output != totals_out && &output != extremes_out) + output_ports.emplace_back(&output); + } header = output_ports.front()->getHeader(); for (size_t i = 1; i < output_ports.size(); ++i) @@ -508,100 +570,11 @@ void Pipe::addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort if (extremes_port) assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "Pipes"); - if (collected_processors) - collected_processors->emplace_back(transform); - processors->emplace_back(std::move(transform)); - max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); -} - -void Pipe::addTransform(ProcessorPtr transform, InputPort * totals, InputPort * extremes) -{ - if (output_ports.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add transform to empty Pipe"); - - auto & inputs = transform->getInputs(); - size_t expected_inputs = output_ports.size() + (totals ? 1 : 0) + (extremes ? 1 : 0); - if (inputs.size() != expected_inputs) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Cannot add transform {} to Pipe because it has {} input ports, but {} expected", - transform->getName(), - inputs.size(), - expected_inputs); - - if (totals && !totals_port) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add transform consuming totals to Pipe because Pipe does not have totals"); - - if (extremes && !extremes_port) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add transform consuming extremes to Pipe because Pipe does not have extremes"); - - if (totals) - { - connect(*totals_port, *totals); - totals_port = nullptr; - } - if (extremes) - { - connect(*extremes_port, *extremes); - extremes_port = nullptr; - } - - bool found_totals = false; - bool found_extremes = false; - - size_t next_output = 0; - for (auto & input : inputs) - { - if (&input == totals) - found_totals = true; - else if (&input == extremes) - found_extremes = true; - else - { - connect(*output_ports[next_output], input); - ++next_output; - } - } - - if (totals && !found_totals) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Cannot add transform {} to Pipes because specified totals port does not belong to it", - transform->getName()); - - if (extremes && !found_extremes) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Cannot add transform {} to Pipes because specified extremes port does not belong to it", - transform->getName()); - - auto & outputs = transform->getOutputs(); - if (outputs.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add transform {} to Pipes because it has no outputs", transform->getName()); - - output_ports.clear(); - output_ports.reserve(outputs.size()); - - for (auto & output : outputs) - output_ports.emplace_back(&output); - - header = output_ports.front()->getHeader(); - for (size_t i = 1; i < output_ports.size(); ++i) - assertBlocksHaveEqualStructure(header, output_ports[i]->getHeader(), "Pipes"); - - if (totals_port) - assertBlocksHaveEqualStructure(header, totals_port->getHeader(), "Pipes"); - - if (extremes_port) - assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "Pipes"); - if (collected_processors) collected_processors->emplace_back(transform); - processors->emplace_back(std::move(transform)); - max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); } diff --git a/src/QueryPipeline/Pipe.h b/src/QueryPipeline/Pipe.h index 09931e38578..ec102605677 100644 --- a/src/QueryPipeline/Pipe.h +++ b/src/QueryPipeline/Pipe.h @@ -69,6 +69,11 @@ public: void addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort * extremes); void addTransform(ProcessorPtr transform, InputPort * totals, InputPort * extremes); + void addTransform( + ProcessorPtr transform, + InputPort * totals_in, InputPort * extremes_in, + OutputPort * totals_out, OutputPort * extremes_out); + enum class StreamType { Main = 0, /// Stream for query data. There may be several streams of this type. diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index f13d1c56d7f..401987d46ba 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -602,7 +602,9 @@ void QueryPipelineBuilder::addPipelineBefore(QueryPipelineBuilder pipeline) throw Exception(ErrorCodes::LOGICAL_ERROR, "Pipeline for CreatingSets should have empty header. Got: {}", pipeline.getHeader().dumpStructure()); - IProcessor::PortNumbers delayed_streams(pipe.numOutputPorts()); + bool has_totals = pipe.getTotalsPort(); + bool has_extremes = pipe.getExtremesPort(); + IProcessor::PortNumbers delayed_streams(pipe.numOutputPorts() + (has_totals ? 1 : 0) + (has_extremes ? 1 : 0)); for (size_t i = 0; i < delayed_streams.size(); ++i) delayed_streams[i] = i; @@ -613,8 +615,14 @@ void QueryPipelineBuilder::addPipelineBefore(QueryPipelineBuilder pipeline) pipes.emplace_back(QueryPipelineBuilder::getPipe(std::move(pipeline), resources)); pipe = Pipe::unitePipes(std::move(pipes), collected_processors, true); - auto processor = std::make_shared(getHeader(), pipe.numOutputPorts(), delayed_streams, true); - addTransform(std::move(processor)); + auto processor = std::make_shared(getHeader(), delayed_streams.size(), delayed_streams, true); + auto in = processor->getInputs().begin(); + auto out = processor->getOutputs().begin(); + InputPort * totals_in = has_totals ? &*(in++) : nullptr; + InputPort * extremes_in = has_extremes ? &*(in++) : nullptr; + OutputPort * totals_out = has_totals ? &*(out++) : nullptr; + OutputPort * extremes_out = has_extremes ? &*(out++) : nullptr; + pipe.addTransform(std::move(processor), totals_in, extremes_in, totals_out, extremes_out); } void QueryPipelineBuilder::setProcessListElement(QueryStatusPtr elem) From 1751524dafc04f5bca14b8ea9434fd870ab6cc57 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Fri, 1 Dec 2023 03:34:54 +0100 Subject: [PATCH 077/331] Fix style --- src/Functions/GatherUtils/Sources.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Functions/GatherUtils/Sources.h b/src/Functions/GatherUtils/Sources.h index 329f71203bf..222f9f19168 100644 --- a/src/Functions/GatherUtils/Sources.h +++ b/src/Functions/GatherUtils/Sources.h @@ -323,7 +323,8 @@ struct StringSource /// Treats Enum values as Strings, modeled after StringSource template -struct EnumSource { +struct EnumSource +{ using Column = typename EnumDataType::ColumnType; using Slice = NumericArraySlice; @@ -371,7 +372,8 @@ struct EnumSource { return data.size(); } - Slice getWhole() const { + Slice getWhole() const + { std::string_view name = data_type.getNameForValue(data[row_num]).toView(); return {reinterpret_cast(name.data()), name.size()}; } From 31894bc9dff4df698acaa79aaa841db2840bbdcf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Fri, 1 Dec 2023 10:38:42 +0800 Subject: [PATCH 078/331] Update settings.md --- .../settings.md | 33 +++++++++++++++++++ 1 file changed, 33 insertions(+) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 2b73c4ec624..3c2604aed0f 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -472,6 +472,39 @@ The value 0 means that you can delete all tables without any restrictions. ``` xml 0 ``` + + +## max\_database\_num\_to\_warn {#max-database-num-to-warn} +If the number of attached databases exceeds the specified value, clickhouse server will add warning message to `system.warnings` table. +Default value: 1000 + +**Example** + +``` xml +50 +``` + +## max\_table\_num\_to\_warn {#max-table-num-to-warn} +If the number of attached tables exceeds the specified value, clickhouse server will add warning message to `system.warnings` table. +Default value: 5000 + +**Example** + +``` xml +400 +``` + + +## max\_part\_num\_to\_warn {#max-part-num-to-warn} +If the number of active parts exceeds the specified value, clickhouse server will add warning message to `system.warnings` table. +Default value: 100000 + +**Example** + +``` xml +400 +``` + ## max_temporary_data_on_disk_size From 2ae934ec05ea797a23ea2d3e8ee4c04db5c39827 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Fri, 1 Dec 2023 10:51:32 +0800 Subject: [PATCH 079/331] Update DatabasesCommon.cpp --- src/Databases/DatabasesCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index e451f3a388e..6f3a6f547e4 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -16,7 +16,7 @@ namespace CurrentMetrics { - const extern Metric AttachedTable; + extern const Metric AttachedTable; } From 0b3cfcc8d4adf47bb3365aeb3bf33f9068144818 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Fri, 1 Dec 2023 11:23:31 +0800 Subject: [PATCH 080/331] Update DatabaseLazy.cpp --- src/Databases/DatabaseLazy.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index 75e4b19d628..caf14aa9b15 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -18,10 +18,12 @@ namespace fs = std::filesystem; + namespace CurrentMetrics { extern const Metric AttachedTable; } + namespace DB { From a30f63802eff4a7203a09baa50e51b2035c07adb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 1 Dec 2023 13:26:30 +0000 Subject: [PATCH 081/331] Fix tests --- src/Processors/QueryPlan/CreatingSetsStep.cpp | 2 -- src/QueryPipeline/Pipe.cpp | 4 ++-- src/QueryPipeline/QueryPipelineBuilder.cpp | 7 +++++-- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 37f81ffd160..3e4dfb0c7d1 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -111,8 +111,6 @@ QueryPipelineBuilderPtr CreatingSetsStep::updatePipeline(QueryPipelineBuilders p else delayed_pipeline = std::move(*pipelines.front()); - delayed_pipeline.dropTotalsAndExtremes(); - QueryPipelineProcessorsCollector collector(*main_pipeline, this); main_pipeline->addPipelineBefore(std::move(delayed_pipeline)); auto added_processors = collector.detachProcessors(); diff --git a/src/QueryPipeline/Pipe.cpp b/src/QueryPipeline/Pipe.cpp index fd433638252..8050c7cc671 100644 --- a/src/QueryPipeline/Pipe.cpp +++ b/src/QueryPipeline/Pipe.cpp @@ -570,11 +570,11 @@ void Pipe::addTransform( if (extremes_port) assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "Pipes"); - processors->emplace_back(std::move(transform)); - if (collected_processors) collected_processors->emplace_back(transform); + processors->emplace_back(std::move(transform)); + max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); } diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 401987d46ba..21eb07a5acd 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -602,9 +602,12 @@ void QueryPipelineBuilder::addPipelineBefore(QueryPipelineBuilder pipeline) throw Exception(ErrorCodes::LOGICAL_ERROR, "Pipeline for CreatingSets should have empty header. Got: {}", pipeline.getHeader().dumpStructure()); + pipeline.dropTotalsAndExtremes(); + bool has_totals = pipe.getTotalsPort(); bool has_extremes = pipe.getExtremesPort(); - IProcessor::PortNumbers delayed_streams(pipe.numOutputPorts() + (has_totals ? 1 : 0) + (has_extremes ? 1 : 0)); + size_t num_extra_ports = (has_totals ? 1 : 0) + (has_extremes ? 1 : 0); + IProcessor::PortNumbers delayed_streams(pipe.numOutputPorts() + num_extra_ports); for (size_t i = 0; i < delayed_streams.size(); ++i) delayed_streams[i] = i; @@ -615,7 +618,7 @@ void QueryPipelineBuilder::addPipelineBefore(QueryPipelineBuilder pipeline) pipes.emplace_back(QueryPipelineBuilder::getPipe(std::move(pipeline), resources)); pipe = Pipe::unitePipes(std::move(pipes), collected_processors, true); - auto processor = std::make_shared(getHeader(), delayed_streams.size(), delayed_streams, true); + auto processor = std::make_shared(getHeader(), pipe.numOutputPorts() + num_extra_ports, delayed_streams, true); auto in = processor->getInputs().begin(); auto out = processor->getOutputs().begin(); InputPort * totals_in = has_totals ? &*(in++) : nullptr; From 6d9e7f98a1c572e847f1acfa2ac765f85a2529f4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 1 Dec 2023 13:28:51 +0000 Subject: [PATCH 082/331] Add a test --- tests/queries/0_stateless/02932_non_ready_set_stuck.reference | 2 ++ tests/queries/0_stateless/02932_non_ready_set_stuck.sql | 2 ++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/02932_non_ready_set_stuck.reference create mode 100644 tests/queries/0_stateless/02932_non_ready_set_stuck.sql diff --git a/tests/queries/0_stateless/02932_non_ready_set_stuck.reference b/tests/queries/0_stateless/02932_non_ready_set_stuck.reference new file mode 100644 index 00000000000..fc39e7c9b45 --- /dev/null +++ b/tests/queries/0_stateless/02932_non_ready_set_stuck.reference @@ -0,0 +1,2 @@ + +0 0 diff --git a/tests/queries/0_stateless/02932_non_ready_set_stuck.sql b/tests/queries/0_stateless/02932_non_ready_set_stuck.sql new file mode 100644 index 00000000000..c04f8f18751 --- /dev/null +++ b/tests/queries/0_stateless/02932_non_ready_set_stuck.sql @@ -0,0 +1,2 @@ +CREATE TABLE tab (item_id UInt64, price_sold Nullable(Float32), date Date) ENGINE = MergeTree ORDER BY item_id; +SELECT * FROM (SELECT item_id FROM tab GROUP BY item_id WITH TOTALS ORDER BY '922337203.6854775806' IN (SELECT NULL)) AS l RIGHT JOIN (SELECT item_id FROM tab) AS r ON l.item_id = r.item_id WHERE NULL; From e4da317f056d31715358d5b806f4b00d5fe04b83 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 1 Dec 2023 18:07:45 +0100 Subject: [PATCH 083/331] 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 084/331] 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 a90458eb65f3743f262df1241915d3046b163a26 Mon Sep 17 00:00:00 2001 From: Ryan Jacobs Date: Thu, 30 Nov 2023 23:46:42 -0800 Subject: [PATCH 085/331] Fix several issues regarding PostgreSQL `array_ndims` usage. 1. Properly quote column identifiers. 2. Handle empty tables that have array columns. 3. Throw a more user friendly error when column value = NULL when calling array_dims(column) 4. Handle column value being the empty array {} --- .../fetchPostgreSQLTableStructure.cpp | 48 +++++++++++++++++-- 1 file changed, 44 insertions(+), 4 deletions(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index dec3f1ffe5a..9890936007f 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -158,6 +158,17 @@ static DataTypePtr convertPostgreSQLDataType(String & type, Fn auto && r return res; } +/// Check if PostgreSQL relation is empty. +/// postgres_table must be already quoted + schema-qualified. +template +bool isTableEmpty(T &tx, const String & postgres_table) { + auto query = fmt::format( + "SELECT NOT EXISTS (SELECT * FROM {} LIMIT 1);", + postgres_table + ); + pqxx::result result{tx.exec(query)}; + return result[0][0].as(); +} template PostgreSQLTableStructure::ColumnsInfoPtr readNamesAndTypesList( @@ -213,10 +224,39 @@ PostgreSQLTableStructure::ColumnsInfoPtr readNamesAndTypesList( { const auto & name_and_type = columns[i]; - /// All rows must contain the same number of dimensions, so limit 1 is ok. If number of dimensions in all rows is not the same - - /// such arrays are not able to be used as ClickHouse Array at all. - pqxx::result result{tx.exec(fmt::format("SELECT array_ndims({}) FROM {} LIMIT 1", name_and_type.name, postgres_table))}; - auto dimensions = result[0][0].as(); + /// NOTE: If the relation is empty, then array_ndims returns NULL. + /// If this is the case, then assume dimensions=1. This covers most + /// use cases, but will be incorrect for empty tables with + /// multi-dimension arrays. The other solutions would be to drop + /// support for empty tables OR attempt fallback to a discovered + /// array_ndims CHECK constraint. + int dimensions; + if (isTableEmpty(tx, postgres_table)) { + dimensions = 1; + } else { + /// All rows must contain the same number of dimensions. + /// 1 is ok. If number of dimensions in all rows is not the same - + /// such arrays are not able to be used as ClickHouse Array at all. + /// + /// Assume dimensions=1 for empty arrays. + auto postgres_column = doubleQuoteString(name_and_type.name); + pqxx::result result{tx.exec(fmt::format( + "SELECT {} IS NULL, COALESCE(array_ndims({}), 1) " + "FROM {} LIMIT 1;", + postgres_column, postgres_column, postgres_table + ))}; + + /// Nullable(Array) is not supported. + auto is_null = result[0][0].as(); + if (is_null) { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "PostgreSQL array cannot be NULL. Column: {}", postgres_column + ); + } + + dimensions = result[0][1].as(); + } /// It is always 1d array if it is in recheck. DataTypePtr type = assert_cast(name_and_type.type.get())->getNestedType(); From 17aacda9e53bfa9359ad808c85b81bc299c86694 Mon Sep 17 00:00:00 2001 From: Ryan Jacobs Date: Sat, 2 Dec 2023 06:44:34 -0800 Subject: [PATCH 086/331] fix: format code for stylecheck --- .../fetchPostgreSQLTableStructure.cpp | 60 +++++++++---------- 1 file changed, 29 insertions(+), 31 deletions(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 9890936007f..0a35bc8c2b5 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -160,14 +160,12 @@ static DataTypePtr convertPostgreSQLDataType(String & type, Fn auto && r /// Check if PostgreSQL relation is empty. /// postgres_table must be already quoted + schema-qualified. -template -bool isTableEmpty(T &tx, const String & postgres_table) { - auto query = fmt::format( - "SELECT NOT EXISTS (SELECT * FROM {} LIMIT 1);", - postgres_table - ); - pqxx::result result{tx.exec(query)}; - return result[0][0].as(); +template +bool isTableEmpty(T & tx, const String & postgres_table) +{ + auto query = fmt::format("SELECT NOT EXISTS (SELECT * FROM {} LIMIT 1);", postgres_table); + pqxx::result result{tx.exec(query)}; + return result[0][0].as(); } template @@ -231,31 +229,31 @@ PostgreSQLTableStructure::ColumnsInfoPtr readNamesAndTypesList( /// support for empty tables OR attempt fallback to a discovered /// array_ndims CHECK constraint. int dimensions; - if (isTableEmpty(tx, postgres_table)) { - dimensions = 1; - } else { - /// All rows must contain the same number of dimensions. - /// 1 is ok. If number of dimensions in all rows is not the same - - /// such arrays are not able to be used as ClickHouse Array at all. - /// - /// Assume dimensions=1 for empty arrays. - auto postgres_column = doubleQuoteString(name_and_type.name); - pqxx::result result{tx.exec(fmt::format( - "SELECT {} IS NULL, COALESCE(array_ndims({}), 1) " - "FROM {} LIMIT 1;", - postgres_column, postgres_column, postgres_table - ))}; + if (isTableEmpty(tx, postgres_table)) + { + dimensions = 1; + } + else + { + /// All rows must contain the same number of dimensions. + /// 1 is ok. If number of dimensions in all rows is not the same - + /// such arrays are not able to be used as ClickHouse Array at all. + /// + /// Assume dimensions=1 for empty arrays. + auto postgres_column = doubleQuoteString(name_and_type.name); + pqxx::result result{tx.exec(fmt::format( + "SELECT {} IS NULL, COALESCE(array_ndims({}), 1) " + "FROM {} LIMIT 1;", + postgres_column, + postgres_column, + postgres_table))}; - /// Nullable(Array) is not supported. - auto is_null = result[0][0].as(); - if (is_null) { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "PostgreSQL array cannot be NULL. Column: {}", postgres_column - ); - } + /// Nullable(Array) is not supported. + auto is_null = result[0][0].as(); + if (is_null) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "PostgreSQL array cannot be NULL. Column: {}", postgres_column); - dimensions = result[0][1].as(); + dimensions = result[0][1].as(); } /// It is always 1d array if it is in recheck. From 9a68461cdb1dde1fdd9f22e6b8958f497590d25d Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 3 Dec 2023 12:43:16 +0100 Subject: [PATCH 087/331] 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 088/331] 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 089/331] 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 090/331] 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 091/331] 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 092/331] 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 093/331] 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 3eb68a67403944a8ea096448d16cb8b2be91eebf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 4 Dec 2023 10:08:46 +0000 Subject: [PATCH 094/331] optimize_using_constraints is broken --- tests/clickhouse-test | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index fe93864b202..006cc71bb4c 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -561,7 +561,6 @@ class SettingsRandomizer: "optimize_read_in_order": lambda: random.randint(0, 1), "optimize_or_like_chain": lambda: random.randint(1, 1), "optimize_substitute_columns": lambda: random.randint(1, 1), - "optimize_using_constraints": lambda: random.randint(1, 1), "enable_multiple_prewhere_read_steps": lambda: random.randint(0, 1), "read_in_order_two_level_merge_threshold": lambda: random.randint(0, 100), "optimize_aggregation_in_order": lambda: random.randint(0, 1), From 434c2113d207b4507cac7f7cac88dbf553cc3474 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 4 Dec 2023 10:51:17 +0000 Subject: [PATCH 095/331] 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 096/331] 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 097/331] 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 098/331] 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 7ce33b073783bd0eb204d737ad8bd7d6e535b5e8 Mon Sep 17 00:00:00 2001 From: johnnymatthews <9611008+johnnymatthews@users.noreply.github.com> Date: Mon, 4 Dec 2023 17:57:33 -0400 Subject: [PATCH 099/331] Adds 'not available on cloud' to Distributed Table Engine. --- docs/en/engines/table-engines/special/distributed.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/engines/table-engines/special/distributed.md b/docs/en/engines/table-engines/special/distributed.md index 14431c4c43b..e0d952c65e4 100644 --- a/docs/en/engines/table-engines/special/distributed.md +++ b/docs/en/engines/table-engines/special/distributed.md @@ -4,6 +4,10 @@ sidebar_position: 10 sidebar_label: Distributed --- +import SelfManaged from '@site/docs/en/_snippets/_self_managed_only_no_roadmap.md'; + + + # Distributed Table Engine Tables with Distributed engine do not store any data of their own, but allow distributed query processing on multiple servers. From e2eb47b2ec8a7a2377c2b1f11c93fcc5b1e52da7 Mon Sep 17 00:00:00 2001 From: johnnymatthews <9611008+johnnymatthews@users.noreply.github.com> Date: Mon, 4 Dec 2023 17:58:19 -0400 Subject: [PATCH 100/331] Reverts last commit. --- docs/en/engines/table-engines/special/distributed.md | 4 ---- 1 file changed, 4 deletions(-) diff --git a/docs/en/engines/table-engines/special/distributed.md b/docs/en/engines/table-engines/special/distributed.md index e0d952c65e4..14431c4c43b 100644 --- a/docs/en/engines/table-engines/special/distributed.md +++ b/docs/en/engines/table-engines/special/distributed.md @@ -4,10 +4,6 @@ sidebar_position: 10 sidebar_label: Distributed --- -import SelfManaged from '@site/docs/en/_snippets/_self_managed_only_no_roadmap.md'; - - - # Distributed Table Engine Tables with Distributed engine do not store any data of their own, but allow distributed query processing on multiple servers. From 40062405fb3b8e4f881c0c7258001dce18d8b59a Mon Sep 17 00:00:00 2001 From: johnnymatthews <9611008+johnnymatthews@users.noreply.github.com> Date: Mon, 4 Dec 2023 17:59:11 -0400 Subject: [PATCH 101/331] Adds 'not available on cloud' to Distributed Table Engine. --- docs/en/engines/table-engines/special/distributed.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/engines/table-engines/special/distributed.md b/docs/en/engines/table-engines/special/distributed.md index 14431c4c43b..e0d952c65e4 100644 --- a/docs/en/engines/table-engines/special/distributed.md +++ b/docs/en/engines/table-engines/special/distributed.md @@ -4,6 +4,10 @@ sidebar_position: 10 sidebar_label: Distributed --- +import SelfManaged from '@site/docs/en/_snippets/_self_managed_only_no_roadmap.md'; + + + # Distributed Table Engine Tables with Distributed engine do not store any data of their own, but allow distributed query processing on multiple servers. From c6ca43b3418de24069c2544b0f0abe33b0147694 Mon Sep 17 00:00:00 2001 From: johnnymatthews <9611008+johnnymatthews@users.noreply.github.com> Date: Mon, 4 Dec 2023 18:05:34 -0400 Subject: [PATCH 102/331] Moves self-hosted-only box under page title. --- docs/en/engines/table-engines/special/distributed.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/engines/table-engines/special/distributed.md b/docs/en/engines/table-engines/special/distributed.md index e0d952c65e4..c484d0803c3 100644 --- a/docs/en/engines/table-engines/special/distributed.md +++ b/docs/en/engines/table-engines/special/distributed.md @@ -1,15 +1,15 @@ --- -slug: /en/engines/table-engines/special/distributed +sidebar_label: "Distributed" sidebar_position: 10 -sidebar_label: Distributed +slug: /en/engines/table-engines/special/distributed --- +# Distributed Table Engine + import SelfManaged from '@site/docs/en/_snippets/_self_managed_only_no_roadmap.md'; -# Distributed Table Engine - Tables with Distributed engine do not store any data of their own, but allow distributed query processing on multiple servers. Reading is automatically parallelized. During a read, the table indexes on remote servers are used, if there are any. From 70f436483c295eea0ab4ebed481da8fd80fb50d7 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 5 Dec 2023 00:40:15 +0100 Subject: [PATCH 103/331] 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 5a51fdac954513a23422bb901d533fecc77bad0d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Tue, 5 Dec 2023 11:33:40 +0800 Subject: [PATCH 104/331] modify --- src/Interpreters/Context.cpp | 1152 ++++++++++------- .../02931_max_num_to_warn.reference | 6 +- .../0_stateless/02931_max_num_to_warn.sql | 2 +- 3 files changed, 673 insertions(+), 487 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a2c94d59826..bc7fc8a0cb4 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1,150 +1,150 @@ +#include #include -#include -#include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include +#include #include #include #include #include -#include -#include -#include -#include -#include #include #include -#include +#include +#include +#include +#include +#include #include +#include +#include +#include +#include +#include +#include #include -#include -#include +#include +#include +#include +#include +#include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include +#include +#include +#include +#include #include -#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include +#include +#include #include -#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include -#include -#include -#include -#include -#include +#include +#include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include namespace fs = std::filesystem; namespace ProfileEvents { - extern const Event ContextLock; - extern const Event ContextLockWaitMicroseconds; +extern const Event ContextLock; +extern const Event ContextLockWaitMicroseconds; } namespace CurrentMetrics { - extern const Metric ContextLockWait; - extern const Metric BackgroundMovePoolTask; - extern const Metric BackgroundMovePoolSize; - extern const Metric BackgroundSchedulePoolTask; - extern const Metric BackgroundSchedulePoolSize; - extern const Metric BackgroundBufferFlushSchedulePoolTask; - extern const Metric BackgroundBufferFlushSchedulePoolSize; - extern const Metric BackgroundDistributedSchedulePoolTask; - extern const Metric BackgroundDistributedSchedulePoolSize; - extern const Metric BackgroundMessageBrokerSchedulePoolTask; - extern const Metric BackgroundMessageBrokerSchedulePoolSize; - extern const Metric BackgroundMergesAndMutationsPoolTask; - extern const Metric BackgroundMergesAndMutationsPoolSize; - extern const Metric BackgroundFetchesPoolTask; - extern const Metric BackgroundFetchesPoolSize; - extern const Metric BackgroundCommonPoolTask; - extern const Metric BackgroundCommonPoolSize; - extern const Metric MarksLoaderThreads; - extern const Metric MarksLoaderThreadsActive; - extern const Metric MarksLoaderThreadsScheduled; - extern const Metric IOPrefetchThreads; - extern const Metric IOPrefetchThreadsActive; - extern const Metric IOPrefetchThreadsScheduled; - extern const Metric IOWriterThreads; - extern const Metric IOWriterThreadsActive; - extern const Metric IOWriterThreadsScheduled; - extern const Metric AttachedTable; - extern const Metric AttachedDatabase; - extern const Metric PartsActive; +extern const Metric ContextLockWait; +extern const Metric BackgroundMovePoolTask; +extern const Metric BackgroundMovePoolSize; +extern const Metric BackgroundSchedulePoolTask; +extern const Metric BackgroundSchedulePoolSize; +extern const Metric BackgroundBufferFlushSchedulePoolTask; +extern const Metric BackgroundBufferFlushSchedulePoolSize; +extern const Metric BackgroundDistributedSchedulePoolTask; +extern const Metric BackgroundDistributedSchedulePoolSize; +extern const Metric BackgroundMessageBrokerSchedulePoolTask; +extern const Metric BackgroundMessageBrokerSchedulePoolSize; +extern const Metric BackgroundMergesAndMutationsPoolTask; +extern const Metric BackgroundMergesAndMutationsPoolSize; +extern const Metric BackgroundFetchesPoolTask; +extern const Metric BackgroundFetchesPoolSize; +extern const Metric BackgroundCommonPoolTask; +extern const Metric BackgroundCommonPoolSize; +extern const Metric MarksLoaderThreads; +extern const Metric MarksLoaderThreadsActive; +extern const Metric MarksLoaderThreadsScheduled; +extern const Metric IOPrefetchThreads; +extern const Metric IOPrefetchThreadsActive; +extern const Metric IOPrefetchThreadsScheduled; +extern const Metric IOWriterThreads; +extern const Metric IOWriterThreadsActive; +extern const Metric IOWriterThreadsScheduled; +extern const Metric AttachedTable; +extern const Metric AttachedDatabase; +extern const Metric PartsActive; } @@ -153,32 +153,33 @@ namespace DB namespace ErrorCodes { - extern const int BAD_ARGUMENTS; - extern const int UNKNOWN_DATABASE; - extern const int UNKNOWN_TABLE; - extern const int TABLE_ALREADY_EXISTS; - extern const int THERE_IS_NO_SESSION; - extern const int THERE_IS_NO_QUERY; - extern const int NO_ELEMENTS_IN_CONFIG; - extern const int TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT; - extern const int LOGICAL_ERROR; - extern const int INVALID_SETTING_VALUE; - extern const int UNKNOWN_READ_METHOD; - extern const int NOT_IMPLEMENTED; - extern const int UNKNOWN_FUNCTION; - extern const int ILLEGAL_COLUMN; - extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; - extern const int CLUSTER_DOESNT_EXIST; +extern const int BAD_ARGUMENTS; +extern const int UNKNOWN_DATABASE; +extern const int UNKNOWN_TABLE; +extern const int TABLE_ALREADY_EXISTS; +extern const int THERE_IS_NO_SESSION; +extern const int THERE_IS_NO_QUERY; +extern const int NO_ELEMENTS_IN_CONFIG; +extern const int TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT; +extern const int LOGICAL_ERROR; +extern const int INVALID_SETTING_VALUE; +extern const int UNKNOWN_READ_METHOD; +extern const int NOT_IMPLEMENTED; +extern const int UNKNOWN_FUNCTION; +extern const int ILLEGAL_COLUMN; +extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; +extern const int CLUSTER_DOESNT_EXIST; } -#define SHUTDOWN(log, desc, ptr, method) do \ -{ \ - if (ptr) \ - { \ - LOG_DEBUG(log, "Shutting down " desc); \ - (ptr)->method; \ - } \ -} while (false) \ +#define SHUTDOWN(log, desc, ptr, method) \ + do \ + { \ + if (ptr) \ + { \ + LOG_DEBUG(log, "Shutting down " desc); \ + (ptr)->method; \ + } \ + } while (false) /** Set of known objects (environment), that could be used in query. * Shared (global) part. Order of members (especially, order of destruction) is very important. @@ -200,8 +201,8 @@ struct ContextSharedPart : boost::noncopyable /// Separate mutex for re-initialization of zookeeper session. This operation could take a long time and must not interfere with another operations. mutable std::mutex zookeeper_mutex; - mutable zkutil::ZooKeeperPtr zookeeper TSA_GUARDED_BY(zookeeper_mutex); /// Client for ZooKeeper. - ConfigurationPtr zookeeper_config TSA_GUARDED_BY(zookeeper_mutex); /// Stores zookeeper configs + mutable zkutil::ZooKeeperPtr zookeeper TSA_GUARDED_BY(zookeeper_mutex); /// Client for ZooKeeper. + ConfigurationPtr zookeeper_config TSA_GUARDED_BY(zookeeper_mutex); /// Stores zookeeper configs ConfigurationPtr sensitive_data_masker_config; @@ -210,37 +211,43 @@ struct ContextSharedPart : boost::noncopyable mutable std::shared_ptr keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_mutex); #endif mutable std::mutex auxiliary_zookeepers_mutex; - mutable std::map auxiliary_zookeepers TSA_GUARDED_BY(auxiliary_zookeepers_mutex); /// Map for auxiliary ZooKeeper clients. - ConfigurationPtr auxiliary_zookeepers_config TSA_GUARDED_BY(auxiliary_zookeepers_mutex); /// Stores auxiliary zookeepers configs + mutable std::map + auxiliary_zookeepers TSA_GUARDED_BY(auxiliary_zookeepers_mutex); /// Map for auxiliary ZooKeeper clients. + ConfigurationPtr auxiliary_zookeepers_config TSA_GUARDED_BY(auxiliary_zookeepers_mutex); /// Stores auxiliary zookeepers configs /// No lock required for interserver_io_host, interserver_io_port, interserver_scheme modified only during initialization - String interserver_io_host; /// The host name by which this server is available for other servers. - UInt16 interserver_io_port = 0; /// and port. - String interserver_scheme; /// http or https + String interserver_io_host; /// The host name by which this server is available for other servers. + UInt16 interserver_io_port = 0; /// and port. + String interserver_scheme; /// http or https MultiVersion interserver_io_credentials; - String path TSA_GUARDED_BY(mutex); /// Path to the data directory, with a slash at the end. - String flags_path TSA_GUARDED_BY(mutex); /// Path to the directory with some control flags for server maintenance. - String user_files_path TSA_GUARDED_BY(mutex); /// Path to the directory with user provided files, usable by 'file' table function. - String dictionaries_lib_path TSA_GUARDED_BY(mutex); /// Path to the directory with user provided binaries and libraries for external dictionaries. - String user_scripts_path TSA_GUARDED_BY(mutex); /// Path to the directory with user provided scripts. - String filesystem_caches_path TSA_GUARDED_BY(mutex); /// Path to the directory with filesystem caches. - ConfigurationPtr config TSA_GUARDED_BY(mutex); /// Global configuration settings. - String tmp_path TSA_GUARDED_BY(mutex); /// Path to the temporary files that occur when processing the request. + String path TSA_GUARDED_BY(mutex); /// Path to the data directory, with a slash at the end. + String flags_path TSA_GUARDED_BY(mutex); /// Path to the directory with some control flags for server maintenance. + String user_files_path TSA_GUARDED_BY(mutex); /// Path to the directory with user provided files, usable by 'file' table function. + String dictionaries_lib_path + TSA_GUARDED_BY(mutex); /// Path to the directory with user provided binaries and libraries for external dictionaries. + String user_scripts_path TSA_GUARDED_BY(mutex); /// Path to the directory with user provided scripts. + String filesystem_caches_path TSA_GUARDED_BY(mutex); /// Path to the directory with filesystem caches. + ConfigurationPtr config TSA_GUARDED_BY(mutex); /// Global configuration settings. + String tmp_path TSA_GUARDED_BY(mutex); /// Path to the temporary files that occur when processing the request. /// All temporary files that occur when processing the requests accounted here. /// Child scopes for more fine-grained accounting are created per user/query/etc. /// Initialized once during server startup. TemporaryDataOnDiskScopePtr root_temp_data_on_disk TSA_GUARDED_BY(mutex); - mutable std::unique_ptr embedded_dictionaries TSA_GUARDED_BY(embedded_dictionaries_mutex); /// Metrica's dictionaries. Have lazy initialization. + mutable std::unique_ptr + embedded_dictionaries TSA_GUARDED_BY(embedded_dictionaries_mutex); /// Metrica's dictionaries. Have lazy initialization. mutable std::unique_ptr external_dictionaries_loader TSA_GUARDED_BY(external_dictionaries_mutex); ExternalLoaderXMLConfigRepository * external_dictionaries_config_repository TSA_GUARDED_BY(external_dictionaries_mutex) = nullptr; scope_guard dictionaries_xmls TSA_GUARDED_BY(external_dictionaries_mutex); - mutable std::unique_ptr external_user_defined_executable_functions_loader TSA_GUARDED_BY(external_user_defined_executable_functions_mutex); - ExternalLoaderXMLConfigRepository * user_defined_executable_functions_config_repository TSA_GUARDED_BY(external_user_defined_executable_functions_mutex) = nullptr; + mutable std::unique_ptr + external_user_defined_executable_functions_loader TSA_GUARDED_BY(external_user_defined_executable_functions_mutex); + ExternalLoaderXMLConfigRepository * + user_defined_executable_functions_config_repository TSA_GUARDED_BY(external_user_defined_executable_functions_mutex) + = nullptr; scope_guard user_defined_executable_functions_xmls TSA_GUARDED_BY(external_user_defined_executable_functions_mutex); mutable OnceFlag user_defined_sql_objects_loader_initialized; @@ -258,39 +265,44 @@ struct ContextSharedPart : boost::noncopyable std::optional backups_worker; /// No lock required for default_profile_name, system_profile_name, buffer_profile_name modified only during initialization - String default_profile_name; /// Default profile name used for default values. - String system_profile_name; /// Profile used by system processes - String buffer_profile_name; /// Profile used by Buffer engine for flushing to the underlying + String default_profile_name; /// Default profile name used for default values. + String system_profile_name; /// Profile used by system processes + String buffer_profile_name; /// Profile used by Buffer engine for flushing to the underlying std::unique_ptr access_control TSA_GUARDED_BY(mutex); mutable OnceFlag resource_manager_initialized; mutable ResourceManagerPtr resource_manager; - mutable UncompressedCachePtr uncompressed_cache TSA_GUARDED_BY(mutex); /// The cache of decompressed blocks. - mutable MarkCachePtr mark_cache TSA_GUARDED_BY(mutex); /// Cache of marks in compressed files. + mutable UncompressedCachePtr uncompressed_cache TSA_GUARDED_BY(mutex); /// The cache of decompressed blocks. + mutable MarkCachePtr mark_cache TSA_GUARDED_BY(mutex); /// Cache of marks in compressed files. mutable OnceFlag load_marks_threadpool_initialized; - mutable std::unique_ptr load_marks_threadpool; /// Threadpool for loading marks cache. + mutable std::unique_ptr load_marks_threadpool; /// Threadpool for loading marks cache. mutable OnceFlag prefetch_threadpool_initialized; - mutable std::unique_ptr prefetch_threadpool; /// Threadpool for loading marks cache. - mutable UncompressedCachePtr index_uncompressed_cache TSA_GUARDED_BY(mutex); /// The cache of decompressed blocks for MergeTree indices. - mutable QueryCachePtr query_cache TSA_GUARDED_BY(mutex); /// Cache of query results. - mutable MarkCachePtr index_mark_cache TSA_GUARDED_BY(mutex); /// Cache of marks in compressed files of MergeTree indices. - mutable MMappedFileCachePtr mmap_cache TSA_GUARDED_BY(mutex); /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads. - ProcessList process_list; /// Executing queries at the moment. + mutable std::unique_ptr prefetch_threadpool; /// Threadpool for loading marks cache. + mutable UncompressedCachePtr index_uncompressed_cache TSA_GUARDED_BY(mutex); /// The cache of decompressed blocks for MergeTree indices. + mutable QueryCachePtr query_cache TSA_GUARDED_BY(mutex); /// Cache of query results. + mutable MarkCachePtr index_mark_cache TSA_GUARDED_BY(mutex); /// Cache of marks in compressed files of MergeTree indices. + mutable MMappedFileCachePtr mmap_cache + TSA_GUARDED_BY(mutex); /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads. + ProcessList process_list; /// Executing queries at the moment. SessionTracker session_tracker; GlobalOvercommitTracker global_overcommit_tracker; - MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree) - MovesList moves_list; /// The list of executing moves (for (Replicated)?MergeTree) + MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree) + MovesList moves_list; /// The list of executing moves (for (Replicated)?MergeTree) ReplicatedFetchList replicated_fetch_list; - ConfigurationPtr users_config TSA_GUARDED_BY(mutex); /// Config with the users, profiles and quotas sections. - InterserverIOHandler interserver_io_handler; /// Handler for interserver communication. + ConfigurationPtr users_config TSA_GUARDED_BY(mutex); /// Config with the users, profiles and quotas sections. + InterserverIOHandler interserver_io_handler; /// Handler for interserver communication. OnceFlag buffer_flush_schedule_pool_initialized; - mutable std::unique_ptr buffer_flush_schedule_pool; /// A thread pool that can do background flush for Buffer tables. + mutable std::unique_ptr + buffer_flush_schedule_pool; /// A thread pool that can do background flush for Buffer tables. OnceFlag schedule_pool_initialized; - mutable std::unique_ptr schedule_pool; /// A thread pool that can run different jobs in background (used in replicated tables) + mutable std::unique_ptr + schedule_pool; /// A thread pool that can run different jobs in background (used in replicated tables) OnceFlag distributed_schedule_pool_initialized; - mutable std::unique_ptr distributed_schedule_pool; /// A thread pool that can run different jobs in background (used for distributed sends) + mutable std::unique_ptr + distributed_schedule_pool; /// A thread pool that can run different jobs in background (used for distributed sends) OnceFlag message_broker_schedule_pool_initialized; - mutable std::unique_ptr message_broker_schedule_pool; /// A thread pool that can run different jobs in background (used for message brokers, like RabbitMQ and Kafka) + mutable std::unique_ptr + message_broker_schedule_pool; /// A thread pool that can run different jobs in background (used for message brokers, like RabbitMQ and Kafka) mutable OnceFlag readers_initialized; mutable std::unique_ptr asynchronous_remote_fs_reader; @@ -300,19 +312,19 @@ struct ContextSharedPart : boost::noncopyable mutable OnceFlag threadpool_writer_initialized; mutable std::unique_ptr threadpool_writer; - mutable ThrottlerPtr replicated_fetches_throttler; /// A server-wide throttler for replicated fetches - mutable ThrottlerPtr replicated_sends_throttler; /// A server-wide throttler for replicated sends + mutable ThrottlerPtr replicated_fetches_throttler; /// A server-wide throttler for replicated fetches + mutable ThrottlerPtr replicated_sends_throttler; /// A server-wide throttler for replicated sends - mutable ThrottlerPtr remote_read_throttler; /// A server-wide throttler for remote IO reads - mutable ThrottlerPtr remote_write_throttler; /// A server-wide throttler for remote IO writes + mutable ThrottlerPtr remote_read_throttler; /// A server-wide throttler for remote IO reads + mutable ThrottlerPtr remote_write_throttler; /// A server-wide throttler for remote IO writes - mutable ThrottlerPtr local_read_throttler; /// A server-wide throttler for local IO reads - mutable ThrottlerPtr local_write_throttler; /// A server-wide throttler for local IO writes + mutable ThrottlerPtr local_read_throttler; /// A server-wide throttler for local IO reads + mutable ThrottlerPtr local_write_throttler; /// A server-wide throttler for local IO writes - mutable ThrottlerPtr backups_server_throttler; /// A server-wide throttler for BACKUPs + mutable ThrottlerPtr backups_server_throttler; /// A server-wide throttler for BACKUPs - MultiVersion macros; /// Substitutions extracted from config. - std::unique_ptr ddl_worker TSA_GUARDED_BY(mutex); /// Process ddl commands from zk. + MultiVersion macros; /// Substitutions extracted from config. + std::unique_ptr ddl_worker TSA_GUARDED_BY(mutex); /// Process ddl commands from zk. /// Rules for selecting the compression settings, depending on the size of the part. mutable std::unique_ptr compression_codec_selector TSA_GUARDED_BY(mutex); /// Storage disk chooser for MergeTree engines @@ -322,22 +334,21 @@ struct ContextSharedPart : boost::noncopyable ServerSettings server_settings; - std::optional merge_tree_settings TSA_GUARDED_BY(mutex); /// Settings of MergeTree* engines. - std::optional replicated_merge_tree_settings TSA_GUARDED_BY(mutex); /// Settings of ReplicatedMergeTree* engines. + std::optional merge_tree_settings TSA_GUARDED_BY(mutex); /// Settings of MergeTree* engines. + std::optional replicated_merge_tree_settings TSA_GUARDED_BY(mutex); /// Settings of ReplicatedMergeTree* engines. std::atomic_size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default) std::atomic_size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default) - /// std::atomic_size_t max_table_size_to_warn; std::atomic_size_t max_database_num_to_warn = 1000lu; std::atomic_size_t max_table_num_to_warn = 5000lu; std::atomic_size_t max_part_num_to_warn = 100000lu; /// No lock required for format_schema_path modified only during initialization - String format_schema_path; /// Path to a directory that contains schema files used by input formats. + String format_schema_path; /// Path to a directory that contains schema files used by input formats. mutable OnceFlag action_locks_manager_initialized; - ActionLocksManagerPtr action_locks_manager; /// Set of storages' action lockers + ActionLocksManagerPtr action_locks_manager; /// Set of storages' action lockers OnceFlag system_logs_initialized; - std::unique_ptr system_logs TSA_GUARDED_BY(mutex); /// Used to log queries and operations on parts - std::optional storage_s3_settings TSA_GUARDED_BY(mutex); /// Settings of S3 storage - std::vector warnings TSA_GUARDED_BY(mutex); /// Store warning messages about server configuration. + std::unique_ptr system_logs TSA_GUARDED_BY(mutex); /// Used to log queries and operations on parts + std::optional storage_s3_settings TSA_GUARDED_BY(mutex); /// Settings of S3 storage + std::vector warnings TSA_GUARDED_BY(mutex); /// Store warning messages about server configuration. /// Background executors for *MergeTree tables /// Has background executors for MergeTree tables been initialized? @@ -348,17 +359,17 @@ struct ContextSharedPart : boost::noncopyable OrdinaryBackgroundExecutorPtr fetch_executor TSA_GUARDED_BY(background_executors_mutex); OrdinaryBackgroundExecutorPtr common_executor TSA_GUARDED_BY(background_executors_mutex); - RemoteHostFilter remote_host_filter TSA_GUARDED_BY(mutex); /// Allowed URL from config.xml - HTTPHeaderFilter http_header_filter TSA_GUARDED_BY(mutex); /// Forbidden HTTP headers from config.xml + RemoteHostFilter remote_host_filter TSA_GUARDED_BY(mutex); /// Allowed URL from config.xml + HTTPHeaderFilter http_header_filter TSA_GUARDED_BY(mutex); /// Forbidden HTTP headers from config.xml /// No lock required for trace_collector modified only during initialization - std::optional trace_collector; /// Thread collecting traces from threads executing queries + std::optional trace_collector; /// Thread collecting traces from threads executing queries /// Clusters for distributed tables /// Initialized on demand (on distributed storages initialization) since Settings should be initialized - mutable std::mutex clusters_mutex; /// Guards clusters, clusters_config and cluster_discovery + mutable std::mutex clusters_mutex; /// Guards clusters, clusters_config and cluster_discovery std::shared_ptr clusters TSA_GUARDED_BY(clusters_mutex); - ConfigurationPtr clusters_config TSA_GUARDED_BY(clusters_mutex); /// Stores updated configs + ConfigurationPtr clusters_config TSA_GUARDED_BY(clusters_mutex); /// Stores updated configs std::unique_ptr cluster_discovery TSA_GUARDED_BY(clusters_mutex); /// No lock required for async_insert_queue modified only during initialization @@ -384,9 +395,7 @@ struct ContextSharedPart : boost::noncopyable bool is_server_completely_started TSA_GUARDED_BY(mutex) = false; ContextSharedPart() - : access_control(std::make_unique()) - , global_overcommit_tracker(&process_list) - , macros(std::make_unique()) + : access_control(std::make_unique()), global_overcommit_tracker(&process_list), macros(std::make_unique()) { /// TODO: make it singleton (?) static std::atomic num_calls{0}; @@ -507,7 +516,8 @@ struct ContextSharedPart : boost::noncopyable access_control->setExternalAuthenticatorsConfig(*config_value); } - const Poco::Util::AbstractConfiguration & getConfigRefWithLock(const std::lock_guard &) const TSA_REQUIRES(this->mutex) + const Poco::Util::AbstractConfiguration & getConfigRefWithLock(const std::lock_guard &) const + TSA_REQUIRES(this->mutex) { return config ? *config : Poco::Util::Application::instance().config(); } @@ -649,10 +659,7 @@ struct ContextSharedPart : boost::noncopyable total_memory_tracker.resetOvercommitTracker(); } - bool hasTraceCollector() const - { - return trace_collector.has_value(); - } + bool hasTraceCollector() const { return trace_collector.has_value(); } void initializeTraceCollector(std::shared_ptr trace_log) { @@ -718,16 +725,22 @@ ContextData::ContextData() = default; ContextData::ContextData(const ContextData &) = default; Context::Context() = default; -Context::Context(const Context & rhs) : ContextData(rhs), std::enable_shared_from_this(rhs) {} +Context::Context(const Context & rhs) : ContextData(rhs), std::enable_shared_from_this(rhs) +{ +} SharedContextHolder::SharedContextHolder(SharedContextHolder &&) noexcept = default; SharedContextHolder & SharedContextHolder::operator=(SharedContextHolder &&) noexcept = default; SharedContextHolder::SharedContextHolder() = default; SharedContextHolder::~SharedContextHolder() = default; -SharedContextHolder::SharedContextHolder(std::unique_ptr shared_context) - : shared(std::move(shared_context)) {} +SharedContextHolder::SharedContextHolder(std::unique_ptr shared_context) : shared(std::move(shared_context)) +{ +} -void SharedContextHolder::reset() { shared.reset(); } +void SharedContextHolder::reset() +{ + shared.reset(); +} ContextMutablePtr Context::createGlobal(ContextSharedPart * shared_part) { @@ -770,21 +783,57 @@ ContextMutablePtr Context::createCopy(const ContextMutablePtr & other) Context::~Context() = default; -InterserverIOHandler & Context::getInterserverIOHandler() { return shared->interserver_io_handler; } -const InterserverIOHandler & Context::getInterserverIOHandler() const { return shared->interserver_io_handler; } +InterserverIOHandler & Context::getInterserverIOHandler() +{ + return shared->interserver_io_handler; +} +const InterserverIOHandler & Context::getInterserverIOHandler() const +{ + return shared->interserver_io_handler; +} -ProcessList & Context::getProcessList() { return shared->process_list; } -const ProcessList & Context::getProcessList() const { return shared->process_list; } -OvercommitTracker * Context::getGlobalOvercommitTracker() const { return &shared->global_overcommit_tracker; } +ProcessList & Context::getProcessList() +{ + return shared->process_list; +} +const ProcessList & Context::getProcessList() const +{ + return shared->process_list; +} +OvercommitTracker * Context::getGlobalOvercommitTracker() const +{ + return &shared->global_overcommit_tracker; +} -SessionTracker & Context::getSessionTracker() { return shared->session_tracker; } +SessionTracker & Context::getSessionTracker() +{ + return shared->session_tracker; +} -MergeList & Context::getMergeList() { return shared->merge_list; } -const MergeList & Context::getMergeList() const { return shared->merge_list; } -MovesList & Context::getMovesList() { return shared->moves_list; } -const MovesList & Context::getMovesList() const { return shared->moves_list; } -ReplicatedFetchList & Context::getReplicatedFetchList() { return shared->replicated_fetch_list; } -const ReplicatedFetchList & Context::getReplicatedFetchList() const { return shared->replicated_fetch_list; } +MergeList & Context::getMergeList() +{ + return shared->merge_list; +} +const MergeList & Context::getMergeList() const +{ + return shared->merge_list; +} +MovesList & Context::getMovesList() +{ + return shared->moves_list; +} +const MovesList & Context::getMovesList() const +{ + return shared->moves_list; +} +ReplicatedFetchList & Context::getReplicatedFetchList() +{ + return shared->replicated_fetch_list; +} +const ReplicatedFetchList & Context::getReplicatedFetchList() const +{ + return shared->replicated_fetch_list; +} String Context::resolveDatabase(const String & database_name) const { @@ -838,21 +887,19 @@ Strings Context::getWarnings() const common_warnings = shared->warnings; if (CurrentMetrics::get(CurrentMetrics::AttachedTable) > static_cast(shared->max_table_num_to_warn)) - common_warnings.emplace_back(fmt::format("Attached tables is more than {}", shared->max_table_num_to_warn)); + common_warnings.emplace_back(fmt::format("The number of attached tables is more than {}", shared->max_table_num_to_warn)); if (CurrentMetrics::get(CurrentMetrics::AttachedDatabase) > static_cast(shared->max_database_num_to_warn)) - common_warnings.emplace_back(fmt::format("Attached databases is more than {}", shared->max_table_num_to_warn)); + common_warnings.emplace_back(fmt::format("The number of attached databases is more than {}", shared->max_table_num_to_warn)); if (CurrentMetrics::get(CurrentMetrics::PartsActive) > static_cast(shared->max_part_num_to_warn)) - common_warnings.emplace_back(fmt::format("Active parts is more than {}", shared->max_part_num_to_warn)); + common_warnings.emplace_back(fmt::format("The number of active parts is more than {}", shared->max_part_num_to_warn)); } /// Make setting's name ordered std::set obsolete_settings; for (const auto & setting : settings) - { if (setting.isValueChanged() && setting.isObsolete()) obsolete_settings.emplace(setting.getName()); - } if (!obsolete_settings.empty()) { @@ -868,7 +915,8 @@ Strings Context::getWarnings() const } res = res + "]" + (single_element ? " is" : " are") + " changed. " - "Please check 'SELECT * FROM system.settings WHERE changed AND is_obsolete' and read the changelog at https://github.com/ClickHouse/ClickHouse/blob/master/CHANGELOG.md"; + "Please check 'SELECT * FROM system.settings WHERE changed AND is_obsolete' and read the changelog at " + "https://github.com/ClickHouse/ClickHouse/blob/master/CHANGELOG.md"; common_warnings.emplace_back(res); } @@ -966,9 +1014,12 @@ try } catch (...) { - DB::tryLogCurrentException(log, fmt::format( - "Caught exception while setup temporary path: {}. " - "It is ok to skip this exception as cleaning old temporary files is not necessary", path)); + DB::tryLogCurrentException( + log, + fmt::format( + "Caught exception while setup temporary path: {}. " + "It is ok to skip this exception as cleaning old temporary files is not necessary", + path)); } static VolumePtr createLocalSingleDiskVolume(const std::string & path, const Poco::Util::AbstractConfiguration & config_) @@ -992,9 +1043,7 @@ void Context::setTemporaryStoragePath(const String & path, size_t max_size) VolumePtr volume = createLocalSingleDiskVolume(shared->tmp_path, shared->getConfigRefWithLock(lock)); for (const auto & disk : volume->getDisks()) - { setupTmpPath(shared->log, disk->getPath()); - } shared->root_temp_data_on_disk = std::make_shared(volume, max_size); } @@ -1010,13 +1059,15 @@ void Context::setTemporaryStoragePolicy(const String & policy_name, size_t max_s } if (tmp_policy->getVolumes().size() != 1) - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, - "Policy '{}' is used temporary files, such policy should have exactly one volume", policy_name); + throw Exception( + ErrorCodes::NO_ELEMENTS_IN_CONFIG, + "Policy '{}' is used temporary files, such policy should have exactly one volume", + policy_name); VolumePtr volume = tmp_policy->getVolume(0); if (volume->getDisks().empty()) - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "No disks volume for temporary files"); + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "No disks volume for temporary files"); for (const auto & disk : volume->getDisks()) { @@ -1029,9 +1080,11 @@ void Context::setTemporaryStoragePolicy(const String & policy_name, size_t max_s if (dynamic_cast(disk_ptr.get()) == nullptr) { const auto * disk_raw_ptr = disk_ptr.get(); - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, + throw Exception( + ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Disk '{}' ({}) is not local and can't be used for temporary files", - disk_ptr->getName(), typeid(*disk_raw_ptr).name()); + disk_ptr->getName(), + typeid(*disk_raw_ptr).name()); } setupTmpPath(shared->log, disk->getPath()); @@ -1156,9 +1209,11 @@ void Context::setUser(const UUID & user_id_, const std::optional(user_id_); - auto new_current_roles = current_roles_ ? user->granted_roles.findGranted(*current_roles_) : user->granted_roles.findGranted(user->default_roles); + auto new_current_roles + = current_roles_ ? user->granted_roles.findGranted(*current_roles_) : user->granted_roles.findGranted(user->default_roles); auto enabled_roles = access_control.getEnabledRolesInfo(new_current_roles, {}); - auto enabled_profiles = access_control.getEnabledSettingsInfo(user_id_, user->settings, enabled_roles->enabled_roles, enabled_roles->settings_from_enabled_roles); + auto enabled_profiles = access_control.getEnabledSettingsInfo( + user_id_, user->settings, enabled_roles->enabled_roles, enabled_roles->settings_from_enabled_roles); const auto & database = user->default_database; /// Apply user's profiles, constraints, settings, roles. @@ -1249,18 +1304,55 @@ void Context::checkAccessImpl(const Args &... args) const return getAccess()->checkAccess(args...); } -void Context::checkAccess(const AccessFlags & flags) const { return checkAccessImpl(flags); } -void Context::checkAccess(const AccessFlags & flags, std::string_view database) const { return checkAccessImpl(flags, database); } -void Context::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table) const { return checkAccessImpl(flags, database, table); } -void Context::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { return checkAccessImpl(flags, database, table, column); } -void Context::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { return checkAccessImpl(flags, database, table, columns); } -void Context::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { return checkAccessImpl(flags, database, table, columns); } -void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName()); } -void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, std::string_view column) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), column); } -void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::vector & columns) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), columns); } -void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const Strings & columns) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), columns); } -void Context::checkAccess(const AccessRightsElement & element) const { return checkAccessImpl(element); } -void Context::checkAccess(const AccessRightsElements & elements) const { return checkAccessImpl(elements); } +void Context::checkAccess(const AccessFlags & flags) const +{ + return checkAccessImpl(flags); +} +void Context::checkAccess(const AccessFlags & flags, std::string_view database) const +{ + return checkAccessImpl(flags, database); +} +void Context::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table) const +{ + return checkAccessImpl(flags, database, table); +} +void Context::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const +{ + return checkAccessImpl(flags, database, table, column); +} +void Context::checkAccess( + const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const +{ + return checkAccessImpl(flags, database, table, columns); +} +void Context::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const +{ + return checkAccessImpl(flags, database, table, columns); +} +void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id) const +{ + checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName()); +} +void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, std::string_view column) const +{ + checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), column); +} +void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::vector & columns) const +{ + checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), columns); +} +void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const Strings & columns) const +{ + checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), columns); +} +void Context::checkAccess(const AccessRightsElement & element) const +{ + return checkAccessImpl(element); +} +void Context::checkAccess(const AccessRightsElements & elements) const +{ + return checkAccessImpl(elements); +} std::shared_ptr Context::getAccess() const { @@ -1270,7 +1362,8 @@ std::shared_ptr Context::getAccess() const /// If setUserID() was never called then this must be the global context with the full access. bool full_access = !user_id; - return ContextAccessParams{user_id, full_access, /* use_default_roles= */ false, current_roles, settings, current_database, client_info}; + return ContextAccessParams{ + user_id, full_access, /* use_default_roles= */ false, current_roles, settings, current_database, client_info}; }; /// Check if the current access rights are still valid, otherwise get parameters for recalculating access rights. @@ -1325,7 +1418,8 @@ std::optional Context::getQuotaUsage() const return getAccess()->getQuotaUsage(); } -void Context::setCurrentProfileWithLock(const String & profile_name, bool check_constraints, const std::lock_guard & lock) +void Context::setCurrentProfileWithLock( + const String & profile_name, bool check_constraints, const std::lock_guard & lock) { try { @@ -1345,7 +1439,8 @@ void Context::setCurrentProfileWithLock(const UUID & profile_id, bool check_cons setCurrentProfilesWithLock(*profile_info, check_constraints, lock); } -void Context::setCurrentProfilesWithLock(const SettingsProfilesInfo & profiles_info, bool check_constraints, const std::lock_guard & lock) +void Context::setCurrentProfilesWithLock( + const SettingsProfilesInfo & profiles_info, bool check_constraints, const std::lock_guard & lock) { if (check_constraints) checkSettingsConstraintsWithLock(profiles_info.settings, SettingSource::PROFILE); @@ -1386,9 +1481,10 @@ std::vector Context::getEnabledProfiles() const ResourceManagerPtr Context::getResourceManager() const { - callOnce(shared->resource_manager_initialized, [&] { - shared->resource_manager = ResourceManagerFactory::instance().get(getConfigRef().getString("resource_manager", "dynamic")); - }); + callOnce( + shared->resource_manager_initialized, + [&] + { shared->resource_manager = ResourceManagerFactory::instance().get(getConfigRef().getString("resource_manager", "dynamic")); }); return shared->resource_manager; } @@ -1656,17 +1752,18 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const ASTCreateQuery create; create.select = query->as(); auto sample_block = InterpreterSelectWithUnionQuery::getSampleBlock(query, getQueryContext()); - auto res = std::make_shared(StorageID(database_name, table_name), - create, - ColumnsDescription(sample_block.getNamesAndTypesList()), - /* comment */ "", - /* is_parameterized_view */ true); + auto res = std::make_shared( + StorageID(database_name, table_name), + create, + ColumnsDescription(sample_block.getNamesAndTypesList()), + /* comment */ "", + /* is_parameterized_view */ true); res->startup(); function->prefer_subquery_to_function_formatting = true; return res; } } - auto hash = table_expression->getTreeHash(/*ignore_aliases=*/ true); + auto hash = table_expression->getTreeHash(/*ignore_aliases=*/true); auto key = toString(hash); StoragePtr & res = table_function_results[key]; if (!res) @@ -1679,21 +1776,19 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const catch (Exception & e) { if (e.code() == ErrorCodes::UNKNOWN_FUNCTION) - { e.addMessage(" or incorrect parameterized view"); - } throw; } - uint64_t use_structure_from_insertion_table_in_table_functions = getSettingsRef().use_structure_from_insertion_table_in_table_functions; + uint64_t use_structure_from_insertion_table_in_table_functions + = getSettingsRef().use_structure_from_insertion_table_in_table_functions; if (use_structure_from_insertion_table_in_table_functions && table_function_ptr->needStructureHint() && hasInsertionTable()) { - const auto & insert_columns = DatabaseCatalog::instance() - .getTable(getInsertionTable(), shared_from_this()) - ->getInMemoryMetadataPtr() - ->getColumns(); + const auto & insert_columns + = DatabaseCatalog::instance().getTable(getInsertionTable(), shared_from_this())->getInMemoryMetadataPtr()->getColumns(); - const auto & insert_column_names = hasInsertionTableColumnNames() ? *getInsertionTableColumnNames() : insert_columns.getOrdinary().getNames(); + const auto & insert_column_names + = hasInsertionTableColumnNames() ? *getInsertionTableColumnNames() : insert_columns.getOrdinary().getNames(); DB::ColumnsDescription structure_hint; bool use_columns_from_insert_query = true; @@ -1702,7 +1797,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const /// insert table columns to table function columns through names from SELECT expression. auto insert_column_name_it = insert_column_names.begin(); - auto insert_column_names_end = insert_column_names.end(); /// end iterator of the range covered by possible asterisk + auto insert_column_names_end = insert_column_names.end(); /// end iterator of the range covered by possible asterisk auto virtual_column_names = table_function_ptr->getVirtualsToCheckBeforeUsingStructureHint(); bool asterisk = false; const auto & expression_list = select_query_hint->select()->as()->children; @@ -1719,7 +1814,8 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const if (asterisk) { if (use_structure_from_insertion_table_in_table_functions == 1) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Asterisk cannot be mixed with column list in INSERT SELECT query."); + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, "Asterisk cannot be mixed with column list in INSERT SELECT query."); use_columns_from_insert_query = false; break; @@ -1752,7 +1848,8 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const if (!structure_hint.empty()) { if (use_structure_from_insertion_table_in_table_functions == 1) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Asterisk cannot be mixed with column list in INSERT SELECT query."); + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, "Asterisk cannot be mixed with column list in INSERT SELECT query."); use_columns_from_insert_query = false; break; @@ -1790,7 +1887,8 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const { /// For input function we should check if input format supports reading subset of columns. if (table_function_ptr->getName() == "input") - use_columns_from_insert_query = FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(getInsertFormat(), shared_from_this()); + use_columns_from_insert_query + = FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(getInsertFormat(), shared_from_this()); else use_columns_from_insert_query = table_function_ptr->supportsReadingSubsetOfColumns(shared_from_this()); } @@ -1814,9 +1912,11 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const if (!structure_hint.empty()) table_function_ptr->setStructureHint(structure_hint); - - } else if (use_structure_from_insertion_table_in_table_functions == 1) - throw Exception(ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH, "Number of columns in insert table less than required by SELECT expression."); + } + else if (use_structure_from_insertion_table_in_table_functions == 1) + throw Exception( + ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH, + "Number of columns in insert table less than required by SELECT expression."); } } @@ -1826,7 +1926,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const /// /// remote('127.1', system.one) -> remote('127.1', 'system.one'), /// - auto new_hash = table_expression->getTreeHash(/*ignore_aliases=*/ true); + auto new_hash = table_expression->getTreeHash(/*ignore_aliases=*/true); if (hash != new_hash) { key = toString(new_hash); @@ -1838,14 +1938,12 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const TableFunctionPtr & table_function_ptr) { - const auto hash = table_expression->getTreeHash(/*ignore_aliases=*/ true); + const auto hash = table_expression->getTreeHash(/*ignore_aliases=*/true); const auto key = toString(hash); StoragePtr & res = table_function_results[key]; if (!res) - { res = table_function_ptr->execute(table_expression, shared_from_this(), table_function_ptr->getName()); - } return res; } @@ -1854,8 +1952,8 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const void Context::addViewSource(const StoragePtr & storage) { if (view_source) - throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Temporary view source storage {} already exists.", - backQuoteIfNeed(view_source->getName())); + throw Exception( + ErrorCodes::TABLE_ALREADY_EXISTS, "Temporary view source storage {} already exists.", backQuoteIfNeed(view_source->getName())); view_source = storage; } @@ -1916,13 +2014,12 @@ void Context::applySettingChangeWithLock(const SettingChange & change, const std catch (Exception & e) { e.addMessage(fmt::format( - "in attempt to set the value of setting '{}' to {}", - change.name, applyVisitor(FieldVisitorToString(), change.value))); + "in attempt to set the value of setting '{}' to {}", change.name, applyVisitor(FieldVisitorToString(), change.value))); throw; } } -void Context::applySettingsChangesWithLock(const SettingsChanges & changes, const std::lock_guard& lock) +void Context::applySettingsChangesWithLock(const SettingsChanges & changes, const std::lock_guard & lock) { for (const SettingChange & change : changes) applySettingChangeWithLock(change, lock); @@ -1950,8 +2047,7 @@ void Context::applySettingChange(const SettingChange & change) catch (Exception & e) { e.addMessage(fmt::format( - "in attempt to set the value of setting '{}' to {}", - change.name, applyVisitor(FieldVisitorToString(), change.value))); + "in attempt to set the value of setting '{}' to {}", change.name, applyVisitor(FieldVisitorToString(), change.value))); throw; } } @@ -1988,7 +2084,8 @@ void Context::clampToSettingsConstraintsWithLock(SettingsChanges & changes, Sett getSettingsConstraintsAndCurrentProfilesWithLock()->constraints.clamp(settings, changes, source); } -void Context::checkMergeTreeSettingsConstraintsWithLock(const MergeTreeSettings & merge_tree_settings, const SettingsChanges & changes) const +void Context::checkMergeTreeSettingsConstraintsWithLock( + const MergeTreeSettings & merge_tree_settings, const SettingsChanges & changes) const { getSettingsConstraintsAndCurrentProfilesWithLock()->constraints.check(merge_tree_settings, changes); } @@ -2032,7 +2129,7 @@ void Context::checkMergeTreeSettingsConstraints(const MergeTreeSettings & merge_ void Context::resetSettingsToDefaultValue(const std::vector & names) { std::lock_guard lock(mutex); - for (const String & name: names) + for (const String & name : names) settings.setDefaultValue(name); } @@ -2066,9 +2163,10 @@ String Context::getInitialQueryId() const void Context::setCurrentDatabaseNameInGlobalContext(const String & name) { if (!isGlobalContext()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Cannot set current database for non global context, this method should " - "be used during server initialization"); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot set current database for non global context, this method should " + "be used during server initialization"); std::lock_guard lock(mutex); if (!current_database.empty()) @@ -2112,13 +2210,12 @@ void Context::setCurrentQueryId(const String & query_id) String query_id_to_set = query_id; - if (query_id_to_set.empty()) /// If the user did not submit his query_id, then we generate it ourselves. + if (query_id_to_set.empty()) /// If the user did not submit his query_id, then we generate it ourselves. { /// Use protected constructor. struct QueryUUID : Poco::UUID { - QueryUUID(const char * bytes, Poco::UUID::Version version) - : Poco::UUID(bytes, version) {} + QueryUUID(const char * bytes, Poco::UUID::Version version) : Poco::UUID(bytes, version) { } }; query_id_to_set = QueryUUID(random.bytes, Poco::UUID::UUID_RANDOM).toString(); @@ -2184,7 +2281,8 @@ void Context::setMacros(std::unique_ptr && macros) ContextMutablePtr Context::getQueryContext() const { auto ptr = query_context.lock(); - if (!ptr) throw Exception(ErrorCodes::THERE_IS_NO_QUERY, "There is no query or query context has expired"); + if (!ptr) + throw Exception(ErrorCodes::THERE_IS_NO_QUERY, "There is no query or query context has expired"); return ptr; } @@ -2197,20 +2295,23 @@ bool Context::isInternalSubquery() const ContextMutablePtr Context::getSessionContext() const { auto ptr = session_context.lock(); - if (!ptr) throw Exception(ErrorCodes::THERE_IS_NO_SESSION, "There is no session or session context has expired"); + if (!ptr) + throw Exception(ErrorCodes::THERE_IS_NO_SESSION, "There is no session or session context has expired"); return ptr; } ContextMutablePtr Context::getGlobalContext() const { auto ptr = global_context.lock(); - if (!ptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no global context or global context has expired"); + if (!ptr) + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no global context or global context has expired"); return ptr; } ContextMutablePtr Context::getBufferContext() const { - if (!buffer_context) throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no buffer context"); + if (!buffer_context) + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no buffer context"); return buffer_context; } @@ -2269,11 +2370,11 @@ ExternalDictionariesLoader & Context::getExternalDictionariesLoader() return getExternalDictionariesLoaderWithLock(lock); } -ExternalDictionariesLoader & Context::getExternalDictionariesLoaderWithLock(const std::lock_guard &) TSA_REQUIRES(shared->external_dictionaries_mutex) +ExternalDictionariesLoader & Context::getExternalDictionariesLoaderWithLock(const std::lock_guard &) + TSA_REQUIRES(shared->external_dictionaries_mutex) { if (!shared->external_dictionaries_loader) - shared->external_dictionaries_loader = - std::make_unique(getGlobalContext()); + shared->external_dictionaries_loader = std::make_unique(getGlobalContext()); return *shared->external_dictionaries_loader; } @@ -2289,11 +2390,12 @@ ExternalUserDefinedExecutableFunctionsLoader & Context::getExternalUserDefinedEx } ExternalUserDefinedExecutableFunctionsLoader & -Context::getExternalUserDefinedExecutableFunctionsLoaderWithLock(const std::lock_guard &) TSA_REQUIRES(shared->external_user_defined_executable_functions_mutex) +Context::getExternalUserDefinedExecutableFunctionsLoaderWithLock(const std::lock_guard &) + TSA_REQUIRES(shared->external_user_defined_executable_functions_mutex) { if (!shared->external_user_defined_executable_functions_loader) - shared->external_user_defined_executable_functions_loader = - std::make_unique(getGlobalContext()); + shared->external_user_defined_executable_functions_loader + = std::make_unique(getGlobalContext()); return *shared->external_user_defined_executable_functions_loader; } @@ -2305,10 +2407,8 @@ EmbeddedDictionaries & Context::getEmbeddedDictionariesImpl(const bool throw_on_ { auto geo_dictionaries_loader = std::make_unique(); - shared->embedded_dictionaries = std::make_unique( - std::move(geo_dictionaries_loader), - getGlobalContext(), - throw_on_error); + shared->embedded_dictionaries + = std::make_unique(std::move(geo_dictionaries_loader), getGlobalContext(), throw_on_error); } return *shared->embedded_dictionaries; @@ -2377,7 +2477,8 @@ void Context::loadOrReloadUserDefinedExecutableFunctions(const Poco::Util::Abstr if (shared->user_defined_executable_functions_config_repository) { shared->user_defined_executable_functions_config_repository->updatePatterns(patterns); - external_user_defined_executable_functions_loader.reloadConfig(shared->user_defined_executable_functions_config_repository->getName()); + external_user_defined_executable_functions_loader.reloadConfig( + shared->user_defined_executable_functions_config_repository->getName()); return; } @@ -2385,14 +2486,15 @@ void Context::loadOrReloadUserDefinedExecutableFunctions(const Poco::Util::Abstr auto config_path = getConfigRef().getString("config-file", "config.xml"); auto repository = std::make_unique(app_path, config_path, patterns); shared->user_defined_executable_functions_config_repository = repository.get(); - shared->user_defined_executable_functions_xmls = external_user_defined_executable_functions_loader.addConfigRepository(std::move(repository)); + shared->user_defined_executable_functions_xmls + = external_user_defined_executable_functions_loader.addConfigRepository(std::move(repository)); } const IUserDefinedSQLObjectsLoader & Context::getUserDefinedSQLObjectsLoader() const { - callOnce(shared->user_defined_sql_objects_loader_initialized, [&] { - shared->user_defined_sql_objects_loader = createUserDefinedSQLObjectsLoader(getGlobalContext()); - }); + callOnce( + shared->user_defined_sql_objects_loader_initialized, + [&] { shared->user_defined_sql_objects_loader = createUserDefinedSQLObjectsLoader(getGlobalContext()); }); SharedLockGuard lock(shared->mutex); return *shared->user_defined_sql_objects_loader; @@ -2400,9 +2502,9 @@ const IUserDefinedSQLObjectsLoader & Context::getUserDefinedSQLObjectsLoader() c IUserDefinedSQLObjectsLoader & Context::getUserDefinedSQLObjectsLoader() { - callOnce(shared->user_defined_sql_objects_loader_initialized, [&] { - shared->user_defined_sql_objects_loader = createUserDefinedSQLObjectsLoader(getGlobalContext()); - }); + callOnce( + shared->user_defined_sql_objects_loader_initialized, + [&] { shared->user_defined_sql_objects_loader = createUserDefinedSQLObjectsLoader(getGlobalContext()); }); SharedLockGuard lock(shared->mutex); return *shared->user_defined_sql_objects_loader; @@ -2412,18 +2514,14 @@ IUserDefinedSQLObjectsLoader & Context::getUserDefinedSQLObjectsLoader() SynonymsExtensions & Context::getSynonymsExtensions() const { - callOnce(shared->synonyms_extensions_initialized, [&] { - shared->synonyms_extensions.emplace(getConfigRef()); - }); + callOnce(shared->synonyms_extensions_initialized, [&] { shared->synonyms_extensions.emplace(getConfigRef()); }); return *shared->synonyms_extensions; } Lemmatizers & Context::getLemmatizers() const { - callOnce(shared->lemmatizers_initialized, [&] { - shared->lemmatizers.emplace(getConfigRef()); - }); + callOnce(shared->lemmatizers_initialized, [&] { shared->lemmatizers.emplace(getConfigRef()); }); return *shared->lemmatizers; } @@ -2431,17 +2529,21 @@ Lemmatizers & Context::getLemmatizers() const BackupsWorker & Context::getBackupsWorker() const { - callOnce(shared->backups_worker_initialized, [&] { - const auto & config = getConfigRef(); - const bool allow_concurrent_backups = config.getBool("backups.allow_concurrent_backups", true); - const bool allow_concurrent_restores = config.getBool("backups.allow_concurrent_restores", true); + callOnce( + shared->backups_worker_initialized, + [&] + { + const auto & config = getConfigRef(); + const bool allow_concurrent_backups = config.getBool("backups.allow_concurrent_backups", true); + const bool allow_concurrent_restores = config.getBool("backups.allow_concurrent_restores", true); - const auto & settings_ref = getSettingsRef(); - UInt64 backup_threads = config.getUInt64("backup_threads", settings_ref.backup_threads); - UInt64 restore_threads = config.getUInt64("restore_threads", settings_ref.restore_threads); + const auto & settings_ref = getSettingsRef(); + UInt64 backup_threads = config.getUInt64("backup_threads", settings_ref.backup_threads); + UInt64 restore_threads = config.getUInt64("restore_threads", settings_ref.restore_threads); - shared->backups_worker.emplace(getGlobalContext(), backup_threads, restore_threads, allow_concurrent_backups, allow_concurrent_restores); - }); + shared->backups_worker.emplace( + getGlobalContext(), backup_threads, restore_threads, allow_concurrent_backups, allow_concurrent_restores); + }); return *shared->backups_worker; } @@ -2556,13 +2658,21 @@ void Context::clearMarkCache() const ThreadPool & Context::getLoadMarksThreadpool() const { - callOnce(shared->load_marks_threadpool_initialized, [&] { - const auto & config = getConfigRef(); - auto pool_size = config.getUInt(".load_marks_threadpool_pool_size", 50); - auto queue_size = config.getUInt(".load_marks_threadpool_queue_size", 1000000); - shared->load_marks_threadpool = std::make_unique( - CurrentMetrics::MarksLoaderThreads, CurrentMetrics::MarksLoaderThreadsActive, CurrentMetrics::MarksLoaderThreadsScheduled, pool_size, pool_size, queue_size); - }); + callOnce( + shared->load_marks_threadpool_initialized, + [&] + { + const auto & config = getConfigRef(); + auto pool_size = config.getUInt(".load_marks_threadpool_pool_size", 50); + auto queue_size = config.getUInt(".load_marks_threadpool_queue_size", 1000000); + shared->load_marks_threadpool = std::make_unique( + CurrentMetrics::MarksLoaderThreads, + CurrentMetrics::MarksLoaderThreadsActive, + CurrentMetrics::MarksLoaderThreadsScheduled, + pool_size, + pool_size, + queue_size); + }); return *shared->load_marks_threadpool; } @@ -2739,13 +2849,21 @@ void Context::clearCaches() const ThreadPool & Context::getPrefetchThreadpool() const { - callOnce(shared->prefetch_threadpool_initialized, [&] { - const auto & config = getConfigRef(); - auto pool_size = config.getUInt(".prefetch_threadpool_pool_size", 100); - auto queue_size = config.getUInt(".prefetch_threadpool_queue_size", 1000000); - shared->prefetch_threadpool = std::make_unique( - CurrentMetrics::IOPrefetchThreads, CurrentMetrics::IOPrefetchThreadsActive, CurrentMetrics::IOPrefetchThreadsScheduled, pool_size, pool_size, queue_size); - }); + callOnce( + shared->prefetch_threadpool_initialized, + [&] + { + const auto & config = getConfigRef(); + auto pool_size = config.getUInt(".prefetch_threadpool_pool_size", 100); + auto queue_size = config.getUInt(".prefetch_threadpool_queue_size", 1000000); + shared->prefetch_threadpool = std::make_unique( + CurrentMetrics::IOPrefetchThreads, + CurrentMetrics::IOPrefetchThreadsActive, + CurrentMetrics::IOPrefetchThreadsScheduled, + pool_size, + pool_size, + queue_size); + }); return *shared->prefetch_threadpool; } @@ -2758,13 +2876,16 @@ size_t Context::getPrefetchThreadpoolSize() const BackgroundSchedulePool & Context::getBufferFlushSchedulePool() const { - callOnce(shared->buffer_flush_schedule_pool_initialized, [&] { - shared->buffer_flush_schedule_pool = std::make_unique( - shared->server_settings.background_buffer_flush_schedule_pool_size, - CurrentMetrics::BackgroundBufferFlushSchedulePoolTask, - CurrentMetrics::BackgroundBufferFlushSchedulePoolSize, - "BgBufSchPool"); - }); + callOnce( + shared->buffer_flush_schedule_pool_initialized, + [&] + { + shared->buffer_flush_schedule_pool = std::make_unique( + shared->server_settings.background_buffer_flush_schedule_pool_size, + CurrentMetrics::BackgroundBufferFlushSchedulePoolTask, + CurrentMetrics::BackgroundBufferFlushSchedulePoolSize, + "BgBufSchPool"); + }); return *shared->buffer_flush_schedule_pool; } @@ -2776,11 +2897,16 @@ BackgroundTaskSchedulingSettings Context::getBackgroundProcessingTaskSchedulingS const auto & config = getConfigRef(); task_settings.thread_sleep_seconds = config.getDouble("background_processing_pool_thread_sleep_seconds", 10); task_settings.thread_sleep_seconds_random_part = config.getDouble("background_processing_pool_thread_sleep_seconds_random_part", 1.0); - task_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); - task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 10); - task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 600); - task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); - task_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); + task_settings.thread_sleep_seconds_if_nothing_to_do + = config.getDouble("background_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); + task_settings.task_sleep_seconds_when_no_work_min + = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 10); + task_settings.task_sleep_seconds_when_no_work_max + = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 600); + task_settings.task_sleep_seconds_when_no_work_multiplier + = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); + task_settings.task_sleep_seconds_when_no_work_random_part + = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); return task_settings; } @@ -2790,51 +2916,66 @@ BackgroundTaskSchedulingSettings Context::getBackgroundMoveTaskSchedulingSetting const auto & config = getConfigRef(); task_settings.thread_sleep_seconds = config.getDouble("background_move_processing_pool_thread_sleep_seconds", 10); - task_settings.thread_sleep_seconds_random_part = config.getDouble("background_move_processing_pool_thread_sleep_seconds_random_part", 1.0); - task_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_move_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); - task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_min", 10); - task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_max", 600); - task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); - task_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); + task_settings.thread_sleep_seconds_random_part + = config.getDouble("background_move_processing_pool_thread_sleep_seconds_random_part", 1.0); + task_settings.thread_sleep_seconds_if_nothing_to_do + = config.getDouble("background_move_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); + task_settings.task_sleep_seconds_when_no_work_min + = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_min", 10); + task_settings.task_sleep_seconds_when_no_work_max + = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_max", 600); + task_settings.task_sleep_seconds_when_no_work_multiplier + = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); + task_settings.task_sleep_seconds_when_no_work_random_part + = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); return task_settings; } BackgroundSchedulePool & Context::getSchedulePool() const { - callOnce(shared->schedule_pool_initialized, [&] { - shared->schedule_pool = std::make_unique( - shared->server_settings.background_schedule_pool_size, - CurrentMetrics::BackgroundSchedulePoolTask, - CurrentMetrics::BackgroundSchedulePoolSize, - "BgSchPool"); - }); + callOnce( + shared->schedule_pool_initialized, + [&] + { + shared->schedule_pool = std::make_unique( + shared->server_settings.background_schedule_pool_size, + CurrentMetrics::BackgroundSchedulePoolTask, + CurrentMetrics::BackgroundSchedulePoolSize, + "BgSchPool"); + }); return *shared->schedule_pool; } BackgroundSchedulePool & Context::getDistributedSchedulePool() const { - callOnce(shared->distributed_schedule_pool_initialized, [&] { - shared->distributed_schedule_pool = std::make_unique( - shared->server_settings.background_distributed_schedule_pool_size, - CurrentMetrics::BackgroundDistributedSchedulePoolTask, - CurrentMetrics::BackgroundDistributedSchedulePoolSize, - "BgDistSchPool"); - }); + callOnce( + shared->distributed_schedule_pool_initialized, + [&] + { + shared->distributed_schedule_pool = std::make_unique( + shared->server_settings.background_distributed_schedule_pool_size, + CurrentMetrics::BackgroundDistributedSchedulePoolTask, + CurrentMetrics::BackgroundDistributedSchedulePoolSize, + "BgDistSchPool"); + }); return *shared->distributed_schedule_pool; } BackgroundSchedulePool & Context::getMessageBrokerSchedulePool() const { - callOnce(shared->message_broker_schedule_pool_initialized, [&] { - shared->message_broker_schedule_pool = std::make_unique( - shared->server_settings.background_message_broker_schedule_pool_size, - CurrentMetrics::BackgroundMessageBrokerSchedulePoolTask, - CurrentMetrics::BackgroundMessageBrokerSchedulePoolSize, - "BgMBSchPool"); - }); + callOnce( + shared->message_broker_schedule_pool_initialized, + [&] + { + shared->message_broker_schedule_pool = std::make_unique( + shared->server_settings.background_message_broker_schedule_pool_size, + CurrentMetrics::BackgroundMessageBrokerSchedulePoolTask, + CurrentMetrics::BackgroundMessageBrokerSchedulePoolSize, + "BgMBSchPool"); + }); return *shared->message_broker_schedule_pool; } @@ -3013,7 +3154,10 @@ bool Context::tryCheckClientConnectionToMyKeeperCluster() const { if (checkZooKeeperConfigIsLocal(getConfigRef(), "auxiliary_zookeepers." + aux_zk_name)) { - LOG_DEBUG(shared->log, "Our Keeper server is participant of the auxiliary zookeeper cluster ({}), will try to connect to it", aux_zk_name); + LOG_DEBUG( + shared->log, + "Our Keeper server is participant of the auxiliary zookeeper cluster ({}), will try to connect to it", + aux_zk_name); getAuxiliaryZooKeeper(aux_zk_name); /// Connected, return true return true; @@ -3086,13 +3230,17 @@ void Context::initializeKeeperDispatcher([[maybe_unused]] bool start_async) cons if (start_async) { assert(!is_standalone_app); - LOG_INFO(shared->log, "Connected to ZooKeeper (or Keeper) before internal Keeper start or we don't depend on our Keeper cluster, " - "will wait for Keeper asynchronously"); + LOG_INFO( + shared->log, + "Connected to ZooKeeper (or Keeper) before internal Keeper start or we don't depend on our Keeper cluster, " + "will wait for Keeper asynchronously"); } else { - LOG_INFO(shared->log, "Cannot connect to ZooKeeper (or Keeper) before internal Keeper start, " - "will wait for Keeper synchronously"); + LOG_INFO( + shared->log, + "Cannot connect to ZooKeeper (or Keeper) before internal Keeper start, " + "will wait for Keeper synchronously"); } shared->keeper_dispatcher = std::make_shared(); @@ -3161,8 +3309,9 @@ zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const "config.xml", name); - zookeeper = shared->auxiliary_zookeepers.emplace(name, - std::make_shared(config, "auxiliary_zookeepers." + name, getZooKeeperLog())).first; + zookeeper = shared->auxiliary_zookeepers + .emplace(name, std::make_shared(config, "auxiliary_zookeepers." + name, getZooKeeperLog())) + .first; } else if (zookeeper->second->expired()) zookeeper->second = zookeeper->second->startNewSession(); @@ -3269,11 +3418,12 @@ void Context::setInterserverIOAddress(const String & host, UInt16 port) std::pair Context::getInterserverIOAddress() const { if (shared->interserver_io_host.empty() || shared->interserver_io_port == 0) - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, - "Parameter 'interserver_http(s)_port' required for replication is not specified " - "in configuration file."); + throw Exception( + ErrorCodes::NO_ELEMENTS_IN_CONFIG, + "Parameter 'interserver_http(s)_port' required for replication is not specified " + "in configuration file."); - return { shared->interserver_io_host, shared->interserver_io_port }; + return {shared->interserver_io_host, shared->interserver_io_port}; } void Context::setInterserverScheme(const String & scheme) @@ -3340,20 +3490,20 @@ UInt16 Context::getServerPort(const String & port_name) const void Context::setMaxPartNumToWarn(size_t max_part_to_warn) { - SharedLockGuard lock(shared->mutex); - shared->max_part_num_to_warn = max_part_to_warn; + SharedLockGuard lock(shared->mutex); + shared->max_part_num_to_warn = max_part_to_warn; } void Context::setMaxTableNumToWarn(size_t max_table_to_warn) { - SharedLockGuard lock(shared->mutex); - shared->max_table_num_to_warn= max_table_to_warn; + SharedLockGuard lock(shared->mutex); + shared->max_table_num_to_warn = max_table_to_warn; } void Context::setMaxDatabaseNumToWarn(size_t max_database_to_warn) { - SharedLockGuard lock(shared->mutex); - shared->max_database_num_to_warn= max_database_to_warn; + SharedLockGuard lock(shared->mutex); + shared->max_database_num_to_warn = max_database_to_warn; } std::shared_ptr Context::getCluster(const std::string & cluster_name) const @@ -3449,9 +3599,7 @@ void Context::setClustersConfig(const ConfigurationPtr & config, bool enable_dis { std::lock_guard lock(shared->clusters_mutex); if (ConfigHelper::getBool(*config, "allow_experimental_cluster_discovery") && enable_discovery && !shared->cluster_discovery) - { shared->cluster_discovery = std::make_unique(*config, getGlobalContext()); - } /// Do not update clusters if this part of config wasn't changed. if (shared->clusters && isSameConfiguration(*config, *shared->clusters_config, config_name)) @@ -3484,11 +3632,14 @@ void Context::initializeSystemLogs() /// triggered from another thread, that is launched while initializing the system logs, /// for example, system.filesystem_cache_log will be triggered by parts loading /// of any other table if it is stored on a disk with cache. - callOnce(shared->system_logs_initialized, [&] { - auto system_logs = std::make_unique(getGlobalContext(), getConfigRef()); - std::lock_guard lock(shared->mutex); - shared->system_logs = std::move(system_logs); - }); + callOnce( + shared->system_logs_initialized, + [&] + { + auto system_logs = std::make_unique(getGlobalContext(), getConfigRef()); + std::lock_guard lock(shared->mutex); + shared->system_logs = std::move(system_logs); + }); } void Context::initializeTraceCollector() @@ -3829,7 +3980,8 @@ DiskSelectorPtr Context::getDiskSelector(std::lock_guard & /* lock * return shared->merge_tree_disk_selector; } -StoragePolicySelectorPtr Context::getStoragePolicySelector(std::lock_guard & lock) const TSA_REQUIRES(shared->storage_policies_mutex) +StoragePolicySelectorPtr Context::getStoragePolicySelector(std::lock_guard & lock) const + TSA_REQUIRES(shared->storage_policies_mutex) { if (!shared->merge_tree_storage_policy_selector) { @@ -3861,7 +4013,9 @@ void Context::updateStorageConfiguration(const Poco::Util::AbstractConfiguration catch (Exception & e) { LOG_ERROR( - shared->log, "An error has occurred while reloading storage policies, storage policies were not applied: {}", e.message()); + shared->log, + "An error has occurred while reloading storage policies, storage policies were not applied: {}", + e.message()); } } @@ -3877,7 +4031,6 @@ void Context::updateStorageConfiguration(const Poco::Util::AbstractConfiguration if (shared->storage_s3_settings) shared->storage_s3_settings->loadFromConfig("s3", config, getSettingsRef()); } - } @@ -3949,19 +4102,24 @@ void Context::checkCanBeDropped(const String & database, const String & table, c String size_str = formatReadableSizeWithDecimalSuffix(size); String max_size_to_drop_str = formatReadableSizeWithDecimalSuffix(max_size_to_drop); - throw Exception(ErrorCodes::TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT, - "Table or Partition in {}.{} was not dropped.\nReason:\n" - "1. Size ({}) is greater than max_[table/partition]_size_to_drop ({})\n" - "2. File '{}' intended to force DROP {}\n" - "How to fix this:\n" - "1. Either increase (or set to zero) max_[table/partition]_size_to_drop in server config\n" - "2. Either create forcing file {} and make sure that ClickHouse has write permission for it.\n" - "Example:\nsudo touch '{}' && sudo chmod 666 '{}'", - backQuoteIfNeed(database), backQuoteIfNeed(table), - size_str, max_size_to_drop_str, - force_file.string(), force_file_exists ? "exists but not writeable (could not be removed)" : "doesn't exist", - force_file.string(), - force_file.string(), force_file.string()); + throw Exception( + ErrorCodes::TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT, + "Table or Partition in {}.{} was not dropped.\nReason:\n" + "1. Size ({}) is greater than max_[table/partition]_size_to_drop ({})\n" + "2. File '{}' intended to force DROP {}\n" + "How to fix this:\n" + "1. Either increase (or set to zero) max_[table/partition]_size_to_drop in server config\n" + "2. Either create forcing file {} and make sure that ClickHouse has write permission for it.\n" + "Example:\nsudo touch '{}' && sudo chmod 666 '{}'", + backQuoteIfNeed(database), + backQuoteIfNeed(table), + size_str, + max_size_to_drop_str, + force_file.string(), + force_file_exists ? "exists but not writeable (could not be removed)" : "doesn't exist", + force_file.string(), + force_file.string(), + force_file.string()); } @@ -4003,7 +4161,13 @@ void Context::checkPartitionCanBeDropped(const String & database, const String & } -InputFormatPtr Context::getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size, const std::optional & format_settings, const std::optional max_parsing_threads) const +InputFormatPtr Context::getInputFormat( + const String & name, + ReadBuffer & buf, + const Block & sample, + UInt64 max_block_size, + const std::optional & format_settings, + const std::optional max_parsing_threads) const { return FormatFactory::instance().getInput(name, buf, sample, shared_from_this(), max_block_size, format_settings, max_parsing_threads); } @@ -4202,9 +4366,9 @@ const IHostContextPtr & Context::getHostContext() const std::shared_ptr Context::getActionLocksManager() const { - callOnce(shared->action_locks_manager_initialized, [&] { - shared->action_locks_manager = std::make_shared(shared_from_this()); - }); + callOnce( + shared->action_locks_manager_initialized, + [&] { shared->action_locks_manager = std::make_shared(shared_from_this()); }); return shared->action_locks_manager; } @@ -4291,7 +4455,8 @@ void Context::setClientInterface(ClientInfo::Interface interface) need_recalculate_access = true; } -void Context::setClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version) +void Context::setClientVersion( + UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version) { client_info.client_version_major = client_version_major; client_info.client_version_minor = client_version_minor; @@ -4376,7 +4541,8 @@ void Context::setQuotaClientKey(const String & quota_key_) need_recalculate_access = true; } -void Context::setConnectionClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version) +void Context::setConnectionClientVersion( + UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version) { client_info.connection_client_version_major = client_version_major; client_info.connection_client_version_minor = client_version_minor; @@ -4457,10 +4623,12 @@ StorageID Context::resolveStorageIDImpl(StorageID storage_id, StorageNamespace w if (!storage_id.database_name.empty()) { if (in_specified_database) - return storage_id; /// NOTE There is no guarantees that table actually exists in database. + return storage_id; /// NOTE There is no guarantees that table actually exists in database. if (exception) - exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "External and temporary tables have no database, but {} is specified", - storage_id.database_name)); + exception->emplace(Exception( + ErrorCodes::UNKNOWN_TABLE, + "External and temporary tables have no database, but {} is specified", + storage_id.database_name)); return StorageID::createEmpty(); } @@ -4545,9 +4713,11 @@ void Context::checkTransactionsAreAllowed(bool explicit_tcl_query /* = false */) if (explicit_tcl_query) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Transactions are not supported"); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Experimental support for transactions is disabled, " - "however, some query or background task tried to access TransactionLog. " - "If you have not enabled this feature explicitly, then it's a bug."); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Experimental support for transactions is disabled, " + "however, some query or background task tried to access TransactionLog. " + "If you have not enabled this feature explicitly, then it's a bug."); } void Context::initCurrentTransaction(MergeTreeTransactionPtr txn) @@ -4708,47 +4878,55 @@ void Context::initializeBackgroundExecutorsIfNeeded() size_t background_common_pool_size = server_settings.background_common_pool_size; /// With this executor we can execute more tasks than threads we have - shared->merge_mutate_executor = std::make_shared - ( + shared->merge_mutate_executor = std::make_shared( "MergeMutate", - /*max_threads_count*/background_pool_size, - /*max_tasks_count*/background_pool_max_tasks_count, + /*max_threads_count*/ background_pool_size, + /*max_tasks_count*/ background_pool_max_tasks_count, CurrentMetrics::BackgroundMergesAndMutationsPoolTask, CurrentMetrics::BackgroundMergesAndMutationsPoolSize, - background_merges_mutations_scheduling_policy - ); - LOG_INFO(shared->log, "Initialized background executor for merges and mutations with num_threads={}, num_tasks={}, scheduling_policy={}", - background_pool_size, background_pool_max_tasks_count, background_merges_mutations_scheduling_policy); + background_merges_mutations_scheduling_policy); + LOG_INFO( + shared->log, + "Initialized background executor for merges and mutations with num_threads={}, num_tasks={}, scheduling_policy={}", + background_pool_size, + background_pool_max_tasks_count, + background_merges_mutations_scheduling_policy); - shared->moves_executor = std::make_shared - ( + shared->moves_executor = std::make_shared( "Move", background_move_pool_size, background_move_pool_size, CurrentMetrics::BackgroundMovePoolTask, - CurrentMetrics::BackgroundMovePoolSize - ); - LOG_INFO(shared->log, "Initialized background executor for move operations with num_threads={}, num_tasks={}", background_move_pool_size, background_move_pool_size); + CurrentMetrics::BackgroundMovePoolSize); + LOG_INFO( + shared->log, + "Initialized background executor for move operations with num_threads={}, num_tasks={}", + background_move_pool_size, + background_move_pool_size); - shared->fetch_executor = std::make_shared - ( + shared->fetch_executor = std::make_shared( "Fetch", background_fetches_pool_size, background_fetches_pool_size, CurrentMetrics::BackgroundFetchesPoolTask, - CurrentMetrics::BackgroundFetchesPoolSize - ); - LOG_INFO(shared->log, "Initialized background executor for fetches with num_threads={}, num_tasks={}", background_fetches_pool_size, background_fetches_pool_size); + CurrentMetrics::BackgroundFetchesPoolSize); + LOG_INFO( + shared->log, + "Initialized background executor for fetches with num_threads={}, num_tasks={}", + background_fetches_pool_size, + background_fetches_pool_size); - shared->common_executor = std::make_shared - ( + shared->common_executor = std::make_shared( "Common", background_common_pool_size, background_common_pool_size, CurrentMetrics::BackgroundCommonPoolTask, - CurrentMetrics::BackgroundCommonPoolSize - ); - LOG_INFO(shared->log, "Initialized background executor for common operations (e.g. clearing old parts) with num_threads={}, num_tasks={}", background_common_pool_size, background_common_pool_size); + CurrentMetrics::BackgroundCommonPoolSize); + LOG_INFO( + shared->log, + "Initialized background executor for common operations (e.g. clearing old parts) with num_threads={}, num_tasks={}", + background_common_pool_size, + background_common_pool_size); shared->are_background_executors_initialized = true; } @@ -4785,12 +4963,15 @@ OrdinaryBackgroundExecutorPtr Context::getCommonExecutor() const IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) const { - callOnce(shared->readers_initialized, [&] { - const auto & config = getConfigRef(); - shared->asynchronous_remote_fs_reader = createThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER, config); - shared->asynchronous_local_fs_reader = createThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER, config); - shared->synchronous_local_fs_reader = createThreadPoolReader(FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER, config); - }); + callOnce( + shared->readers_initialized, + [&] + { + const auto & config = getConfigRef(); + shared->asynchronous_remote_fs_reader = createThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER, config); + shared->asynchronous_local_fs_reader = createThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER, config); + shared->synchronous_local_fs_reader = createThreadPoolReader(FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER, config); + }); switch (type) { @@ -4805,14 +4986,22 @@ IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) co ThreadPool & Context::getThreadPoolWriter() const { - callOnce(shared->threadpool_writer_initialized, [&] { - const auto & config = getConfigRef(); - auto pool_size = config.getUInt(".threadpool_writer_pool_size", 100); - auto queue_size = config.getUInt(".threadpool_writer_queue_size", 1000000); + callOnce( + shared->threadpool_writer_initialized, + [&] + { + const auto & config = getConfigRef(); + auto pool_size = config.getUInt(".threadpool_writer_pool_size", 100); + auto queue_size = config.getUInt(".threadpool_writer_queue_size", 1000000); - shared->threadpool_writer = std::make_unique( - CurrentMetrics::IOWriterThreads, CurrentMetrics::IOWriterThreadsActive, CurrentMetrics::IOWriterThreadsScheduled, pool_size, pool_size, queue_size); - }); + shared->threadpool_writer = std::make_unique( + CurrentMetrics::IOWriterThreads, + CurrentMetrics::IOWriterThreadsActive, + CurrentMetrics::IOWriterThreadsScheduled, + pool_size, + pool_size, + queue_size); + }); return *shared->threadpool_writer; } @@ -4856,10 +5045,7 @@ ReadSettings Context::getReadSettings() const /// Zero read buffer will not make progress. if (!settings.max_read_buffer_size) - { - throw Exception(ErrorCodes::INVALID_SETTING_VALUE, - "Invalid value '{}' for max_read_buffer_size", settings.max_read_buffer_size); - } + throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Invalid value '{}' for max_read_buffer_size", settings.max_read_buffer_size); res.local_fs_buffer_size = settings.max_read_buffer_size_local_fs ? settings.max_read_buffer_size_local_fs : settings.max_read_buffer_size; diff --git a/tests/queries/0_stateless/02931_max_num_to_warn.reference b/tests/queries/0_stateless/02931_max_num_to_warn.reference index 76d86352bfc..c0ad7354039 100644 --- a/tests/queries/0_stateless/02931_max_num_to_warn.reference +++ b/tests/queries/0_stateless/02931_max_num_to_warn.reference @@ -1,3 +1,3 @@ -Attached tables is more than 10 -Attached databases is more than 10 -Active parts is more than 10 +The number of attached tables is more than 10 +The number of attached databases is more than 10 +The number of active parts is more than 10 diff --git a/tests/queries/0_stateless/02931_max_num_to_warn.sql b/tests/queries/0_stateless/02931_max_num_to_warn.sql index cd9a4ebe5fa..2e357a6080c 100644 --- a/tests/queries/0_stateless/02931_max_num_to_warn.sql +++ b/tests/queries/0_stateless/02931_max_num_to_warn.sql @@ -34,7 +34,7 @@ INSERT INTO test_max_num_to_warn_9 VALUES (1, 'Hello'); INSERT INTO test_max_num_to_warn_10 VALUES (1, 'Hello'); INSERT INTO test_max_num_to_warn_11 VALUES (1, 'Hello'); -SELECT * FROM system.warnings where message in ('Attached tables is more than 10', 'Attached databases is more than 10', 'Active parts is more than 10'); +SELECT * FROM system.warnings where message in ('The number of attached tables is more than 10', 'The number of attached databases is more than 10', 'The number of active parts is more than 10'); DROP TABLE test_max_num_to_warn_1; DROP TABLE test_max_num_to_warn_2; From 8ea9403d0f6072427f0bf61ec2a963d9c65726b8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Tue, 5 Dec 2023 11:39:54 +0800 Subject: [PATCH 105/331] modify docs --- .../operations/server-configuration-parameters/settings.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 3c2604aed0f..8d5f0dd3c50 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -475,7 +475,7 @@ The value 0 means that you can delete all tables without any restrictions. ## max\_database\_num\_to\_warn {#max-database-num-to-warn} -If the number of attached databases exceeds the specified value, clickhouse server will add warning message to `system.warnings` table. +If the number of attached databases exceeds the specified value, clickhouse server will add warning messages to `system.warnings` table. Default value: 1000 **Example** @@ -485,7 +485,7 @@ Default value: 1000 ``` ## max\_table\_num\_to\_warn {#max-table-num-to-warn} -If the number of attached tables exceeds the specified value, clickhouse server will add warning message to `system.warnings` table. +If the number of attached tables exceeds the specified value, clickhouse server will add warning messages to `system.warnings` table. Default value: 5000 **Example** @@ -496,7 +496,7 @@ Default value: 5000 ## max\_part\_num\_to\_warn {#max-part-num-to-warn} -If the number of active parts exceeds the specified value, clickhouse server will add warning message to `system.warnings` table. +If the number of active parts exceeds the specified value, clickhouse server will add warning messages to `system.warnings` table. Default value: 100000 **Example** From 60ba1abc176169c67a7709ae9c688bd49cf79d1c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 5 Dec 2023 12:58:29 +0000 Subject: [PATCH 106/331] 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 c41511e51c47c0b0bd1c49a79540bb870fd6dea0 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Tue, 5 Dec 2023 15:11:32 +0000 Subject: [PATCH 107/331] ignore_on_cluster_for_grants --- .../Access/InterpreterGrantQuery.cpp | 6 ++-- .../removeOnClusterClauseIfNeeded.cpp | 4 ++- .../integration/test_replicated_users/test.py | 35 +++++++++++++++++++ 3 files changed, 42 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Access/InterpreterGrantQuery.cpp b/src/Interpreters/Access/InterpreterGrantQuery.cpp index 45e8ba9ea0d..259c6b39524 100644 --- a/src/Interpreters/Access/InterpreterGrantQuery.cpp +++ b/src/Interpreters/Access/InterpreterGrantQuery.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -396,7 +397,8 @@ namespace BlockIO InterpreterGrantQuery::execute() { - auto & query = query_ptr->as(); + const auto updated_query = removeOnClusterClauseIfNeeded(query_ptr, getContext()); + auto & query = updated_query->as(); query.replaceCurrentUserTag(getContext()->getUserName()); query.access_rights_elements.eraseNonGrantable(); @@ -430,7 +432,7 @@ BlockIO InterpreterGrantQuery::execute() current_user_access->checkGranteesAreAllowed(grantees); DDLQueryOnClusterParams params; params.access_to_check = std::move(required_access); - return executeDDLQueryOnCluster(query_ptr, getContext(), params); + return executeDDLQueryOnCluster(updated_query, getContext(), params); } /// Check if the current user has corresponding access rights granted with grant option. diff --git a/src/Interpreters/removeOnClusterClauseIfNeeded.cpp b/src/Interpreters/removeOnClusterClauseIfNeeded.cpp index 7dc452a0fcb..da3930d62a6 100644 --- a/src/Interpreters/removeOnClusterClauseIfNeeded.cpp +++ b/src/Interpreters/removeOnClusterClauseIfNeeded.cpp @@ -14,6 +14,7 @@ #include #include #include +#include namespace DB @@ -33,7 +34,8 @@ static bool isAccessControlQuery(const ASTPtr & query) || query->as() || query->as() || query->as() - || query->as(); + || query->as() + || query->as(); } ASTPtr removeOnClusterClauseIfNeeded(const ASTPtr & query, ContextPtr context, const WithoutOnClusterASTRewriteParams & params) diff --git a/tests/integration/test_replicated_users/test.py b/tests/integration/test_replicated_users/test.py index 489724ed4fb..e34495a0071 100644 --- a/tests/integration/test_replicated_users/test.py +++ b/tests/integration/test_replicated_users/test.py @@ -114,6 +114,41 @@ def test_create_replicated_on_cluster_ignore(started_cluster, entity): node1.query(f"DROP {entity.keyword} {entity.name} {entity.options}") +@pytest.mark.parametrize( + "use_on_cluster", + [ + pytest.param(False, id="Without_on_cluster"), + pytest.param(True, id="With_ignored_on_cluster"), + ], +) +def test_grant_revoke_replicated(started_cluster, use_on_cluster: bool): + node1.replace_config( + "/etc/clickhouse-server/users.d/users.xml", + inspect.cleandoc( + f""" + + + + {int(use_on_cluster)} + + + + """ + ), + ) + node1.query("SYSTEM RELOAD CONFIG") + on_cluster = "ON CLUSTER default" if use_on_cluster else "" + + node1.query(f"CREATE USER theuser {on_cluster}") + + assert node1.query(f"GRANT {on_cluster} SELECT ON *.* to theuser") == "" + + assert node2.query(f"SHOW GRANTS FOR theuser") == "GRANT SELECT ON *.* TO theuser\n" + + assert node1.query(f"REVOKE {on_cluster} SELECT ON *.* from theuser") == "" + node1.query(f"DROP USER theuser {on_cluster}") + + @pytest.mark.parametrize("entity", entities, ids=get_entity_id) def test_create_replicated_if_not_exists_on_cluster(started_cluster, entity): node1.query( From 89b937339845741c5a564c420d1d101cf8c62759 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 5 Dec 2023 18:17:47 +0000 Subject: [PATCH 108/331] More random after tests are successful --- tests/clickhouse-test | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 006cc71bb4c..b596ac2a5fc 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -555,12 +555,12 @@ class SettingsRandomizer: "prefer_localhost_replica": lambda: random.randint(0, 1), "max_block_size": lambda: random.randint(8000, 100000), "max_threads": lambda: random.randint(1, 64), - "optimize_append_index": lambda: random.randint(1, 1), - "optimize_if_chain_to_multiif": lambda: random.randint(1, 1), - "optimize_if_transform_strings_to_enum": lambda: random.randint(1, 1), + "optimize_append_index": lambda: random.randint(0, 1), + "optimize_if_chain_to_multiif": lambda: random.randint(0, 1), + "optimize_if_transform_strings_to_enum": lambda: random.randint(0, 1), "optimize_read_in_order": lambda: random.randint(0, 1), - "optimize_or_like_chain": lambda: random.randint(1, 1), - "optimize_substitute_columns": lambda: random.randint(1, 1), + "optimize_or_like_chain": lambda: random.randint(0, 1), + "optimize_substitute_columns": lambda: random.randint(0, 1), "enable_multiple_prewhere_read_steps": lambda: random.randint(0, 1), "read_in_order_two_level_merge_threshold": lambda: random.randint(0, 100), "optimize_aggregation_in_order": lambda: random.randint(0, 1), From afa6e4388b986f308fa9cd33cd78e35931abc426 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 5 Dec 2023 20:04:42 +0000 Subject: [PATCH 109/331] Move docs of format() from general string functions to replacing string functions --- .../functions/string-functions.md | 34 ------------------- .../functions/string-replace-functions.md | 34 +++++++++++++++++++ 2 files changed, 34 insertions(+), 34 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 1940993ce0b..0f85b81278d 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -393,40 +393,6 @@ Reverses the sequence of bytes in a string. Reverses a sequence of Unicode code points in a string. Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. -## format - -Format the `pattern` string with the strings listed in the arguments, similar to formatting in Python. The pattern string can contain replacement fields surrounded by curly braces `{}`. Anything not contained in braces is considered literal text and copied verbatim into the output. Literal brace character can be escaped by two braces: `{{ '{{' }}` and `{{ '}}' }}`. Field names can be numbers (starting from zero) or empty (then they are implicitly given monotonically increasing numbers). - -**Syntax** - -```sql -format(pattern, s0, s1, …) -``` - -**Example** - -``` sql -SELECT format('{1} {0} {1}', 'World', 'Hello') -``` - -```result -┌─format('{1} {0} {1}', 'World', 'Hello')─┐ -│ Hello World Hello │ -└─────────────────────────────────────────┘ -``` - -With implicit numbers: - -``` sql -SELECT format('{} {}', 'Hello', 'World') -``` - -```result -┌─format('{} {}', 'Hello', 'World')─┐ -│ Hello World │ -└───────────────────────────────────┘ -``` - ## concat Concatenates the given arguments. diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 74d5d747193..0f6d10a1bc0 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -132,6 +132,40 @@ For more information, see [RE2](https://github.com/google/re2/blob/master/re2/re regexpQuoteMeta(s) ``` +## format + +Format the `pattern` string with the strings listed in the arguments, similar to formatting in Python. The pattern string can contain replacement fields surrounded by curly braces `{}`. Anything not contained in braces is considered literal text and copied verbatim into the output. Literal brace character can be escaped by two braces: `{{ '{{' }}` and `{{ '}}' }}`. Field names can be numbers (starting from zero) or empty (then they are implicitly given monotonically increasing numbers). + +**Syntax** + +```sql +format(pattern, s0, s1, …) +``` + +**Example** + +``` sql +SELECT format('{1} {0} {1}', 'World', 'Hello') +``` + +```result +┌─format('{1} {0} {1}', 'World', 'Hello')─┐ +│ Hello World Hello │ +└─────────────────────────────────────────┘ +``` + +With implicit numbers: + +``` sql +SELECT format('{} {}', 'Hello', 'World') +``` + +```result +┌─format('{} {}', 'Hello', 'World')─┐ +│ Hello World │ +└───────────────────────────────────┘ +``` + ## translate Replaces characters in the string `s` using a one-to-one character mapping defined by `from` and `to` strings. `from` and `to` must be constant ASCII strings of the same size. Non-ASCII characters in the original string are not modified. From a7f04880591d7dc3d015ff1f82e1d8fbb3f5ef45 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 5 Dec 2023 20:07:08 +0000 Subject: [PATCH 110/331] Rename Functions/formatString.cpp --> Functions/format.cpp --- src/Functions/{formatString.cpp => format.cpp} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename src/Functions/{formatString.cpp => format.cpp} (100%) diff --git a/src/Functions/formatString.cpp b/src/Functions/format.cpp similarity index 100% rename from src/Functions/formatString.cpp rename to src/Functions/format.cpp From 7fed06158b8cd548503c0b306e1d81d7d1e58253 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 5 Dec 2023 20:08:22 +0000 Subject: [PATCH 111/331] Remove unnecessary template + whitespace --- src/Functions/format.cpp | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/src/Functions/format.cpp b/src/Functions/format.cpp index 8e0b3a238cb..e2bbb3030f3 100644 --- a/src/Functions/format.cpp +++ b/src/Functions/format.cpp @@ -25,11 +25,10 @@ namespace ErrorCodes namespace { -template class FormatFunction : public IFunction { public: - static constexpr auto name = Name::name; + static constexpr auto name = "format"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } @@ -127,11 +126,7 @@ public: }; -struct NameFormat -{ - static constexpr auto name = "format"; -}; -using FunctionFormat = FormatFunction; +using FunctionFormat = FormatFunction; } From 4acfbe948d17b926bc889c9c1d025bb9030755a0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 5 Dec 2023 20:10:49 +0000 Subject: [PATCH 112/331] Iterate less awkwardly --- src/Functions/format.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Functions/format.cpp b/src/Functions/format.cpp index e2bbb3030f3..dd8e17ad03d 100644 --- a/src/Functions/format.cpp +++ b/src/Functions/format.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include #include @@ -51,15 +50,15 @@ public: getName(), arguments.size()); - for (const auto arg_idx : collections::range(0, arguments.size())) + for (size_t i = 0; i < arguments.size(); ++i) { - const auto * arg = arguments[arg_idx].get(); + const auto * arg = arguments[i].get(); if (!isStringOrFixedString(arg)) throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument {} of function {}", arg->getName(), - arg_idx + 1, + i + 1, getName()); } From 3305f46676715a8a8ed515879938285293a8a489 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 5 Dec 2023 20:14:11 +0000 Subject: [PATCH 113/331] Less silly #includes --- src/Functions/concat.cpp | 2 +- src/Functions/format.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/concat.cpp b/src/Functions/concat.cpp index 6403c4b8416..4d7d9ffb56c 100644 --- a/src/Functions/concat.cpp +++ b/src/Functions/concat.cpp @@ -7,10 +7,10 @@ #include #include #include +#include #include #include -#include "formatString.h" namespace DB { diff --git a/src/Functions/format.cpp b/src/Functions/format.cpp index dd8e17ad03d..d6f1930a85a 100644 --- a/src/Functions/format.cpp +++ b/src/Functions/format.cpp @@ -4,13 +4,13 @@ #include #include #include +#include #include #include #include #include -#include "formatString.h" namespace DB { From e9d9048903a7bfb1dd61129ea4d3107ac2f7ff85 Mon Sep 17 00:00:00 2001 From: johnnymatthews <9611008+johnnymatthews@users.noreply.github.com> Date: Tue, 5 Dec 2023 17:14:10 -0400 Subject: [PATCH 114/331] Changes 'cannot run on cloud' message. --- .../table-engines/special/distributed.md | 24 +++++++------------ 1 file changed, 9 insertions(+), 15 deletions(-) diff --git a/docs/en/engines/table-engines/special/distributed.md b/docs/en/engines/table-engines/special/distributed.md index c484d0803c3..6224c450ea2 100644 --- a/docs/en/engines/table-engines/special/distributed.md +++ b/docs/en/engines/table-engines/special/distributed.md @@ -6,12 +6,11 @@ slug: /en/engines/table-engines/special/distributed # Distributed Table Engine -import SelfManaged from '@site/docs/en/_snippets/_self_managed_only_no_roadmap.md'; +:::warning +To create a distributed table engine in the cloud, you can use the [remote and remoteSecure](../../../sql-reference/table-functions/remote) table functions. The `Distributed(...)` syntax cannot be used in ClickHouse Cloud. +::: - - -Tables with Distributed engine do not store any data of their own, but allow distributed query processing on multiple servers. -Reading is automatically parallelized. During a read, the table indexes on remote servers are used, if there are any. +Tables with Distributed engine do not store any data of their own, but allow distributed query processing on multiple servers. Reading is automatically parallelized. During a read, the table indexes on remote servers are used, if there are any. ## Creating a Table {#distributed-creating-a-table} @@ -26,6 +25,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ``` ### From a Table {#distributed-from-a-table} + When the `Distributed` table is pointing to a table on the current server you can adopt that table's schema: ``` sql @@ -52,7 +52,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] AS [db2.]name2 Specifying the `sharding_key` is necessary for the following: -- For `INSERTs` into a distributed table (as the table engine needs the `sharding_key` to determine how to split the data). However, if `insert_distributed_one_random_shard` setting is enabled, then `INSERTs` do not need the sharding key +- For `INSERTs` into a distributed table (as the table engine needs the `sharding_key` to determine how to split the data). However, if `insert_distributed_one_random_shard` setting is enabled, then `INSERTs` do not need the sharding key. - For use with `optimize_skip_unused_shards` as the `sharding_key` is necessary to determine what shards should be queried #### policy_name @@ -126,9 +126,7 @@ SETTINGS fsync_directories=0; ``` -Data will be read from all servers in the `logs` cluster, from the `default.hits` table located on every server in the cluster. -Data is not only read but is partially processed on the remote servers (to the extent that this is possible). -For example, for a query with `GROUP BY`, data will be aggregated on remote servers, and the intermediate states of aggregate functions will be sent to the requestor server. Then data will be further aggregated. +Data will be read from all servers in the `logs` cluster, from the `default.hits` table located on every server in the cluster. Data is not only read but is partially processed on the remote servers (to the extent that this is possible). For example, for a query with `GROUP BY`, data will be aggregated on remote servers, and the intermediate states of aggregate functions will be sent to the requestor server. Then data will be further aggregated. Instead of the database name, you can use a constant expression that returns a string. For example: `currentDatabase()`. @@ -187,9 +185,7 @@ Clusters are configured in the [server configuration file](../../../operations/c ``` -Here a cluster is defined with the name `logs` that consists of two shards, each of which contains two replicas. -Shards refer to the servers that contain different parts of the data (in order to read all the data, you must access all the shards). -Replicas are duplicating servers (in order to read all the data, you can access the data on any one of the replicas). +Here a cluster is defined with the name `logs` that consists of two shards, each of which contains two replicas. Shards refer to the servers that contain different parts of the data (in order to read all the data, you must access all the shards). Replicas are duplicating servers (in order to read all the data, you can access the data on any one of the replicas). Cluster names must not contain dots. @@ -202,9 +198,7 @@ The parameters `host`, `port`, and optionally `user`, `password`, `secure`, `com - `secure` - Whether to use a secure SSL/TLS connection. Usually also requires specifying the port (the default secure port is `9440`). The server should listen on `9440` and be configured with correct certificates. - `compression` - Use data compression. Default value: `true`. -When specifying replicas, one of the available replicas will be selected for each of the shards when reading. You can configure the algorithm for load balancing (the preference for which replica to access) – see the [load_balancing](../../../operations/settings/settings.md#settings-load_balancing) setting. -If the connection with the server is not established, there will be an attempt to connect with a short timeout. If the connection failed, the next replica will be selected, and so on for all the replicas. If the connection attempt failed for all the replicas, the attempt will be repeated the same way, several times. -This works in favour of resiliency, but does not provide complete fault tolerance: a remote server might accept the connection, but might not work, or work poorly. +When specifying replicas, one of the available replicas will be selected for each of the shards when reading. You can configure the algorithm for load balancing (the preference for which replica to access) – see the [load_balancing](../../../operations/settings/settings.md#settings-load_balancing) setting. If the connection with the server is not established, there will be an attempt to connect with a short timeout. If the connection failed, the next replica will be selected, and so on for all the replicas. If the connection attempt failed for all the replicas, the attempt will be repeated the same way, several times. This works in favour of resiliency, but does not provide complete fault tolerance: a remote server might accept the connection, but might not work, or work poorly. You can specify just one of the shards (in this case, query processing should be called remote, rather than distributed) or up to any number of shards. In each shard, you can specify from one to any number of replicas. You can specify a different number of replicas for each shard. From c0d56c7bee76e44f369dac3c5f3f138b9c73f89e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 5 Dec 2023 20:43:34 +0000 Subject: [PATCH 115/331] Implement format() with arbitrary arguments --- .../functions/string-replace-functions.md | 2 +- src/Functions/concatWithSeparator.cpp | 2 +- src/Functions/format.cpp | 39 ++++++--- ...2935_format_with_arbitrary_types.reference | 70 +++++++++++++++ .../02935_format_with_arbitrary_types.sql | 85 +++++++++++++++++++ 5 files changed, 182 insertions(+), 16 deletions(-) create mode 100644 tests/queries/0_stateless/02935_format_with_arbitrary_types.reference create mode 100644 tests/queries/0_stateless/02935_format_with_arbitrary_types.sql diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 0f6d10a1bc0..c7bd16cad4a 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -134,7 +134,7 @@ regexpQuoteMeta(s) ## format -Format the `pattern` string with the strings listed in the arguments, similar to formatting in Python. The pattern string can contain replacement fields surrounded by curly braces `{}`. Anything not contained in braces is considered literal text and copied verbatim into the output. Literal brace character can be escaped by two braces: `{{ '{{' }}` and `{{ '}}' }}`. Field names can be numbers (starting from zero) or empty (then they are implicitly given monotonically increasing numbers). +Format the `pattern` string with the values (strings, integers, etc.) listed in the arguments, similar to formatting in Python. The pattern string can contain replacement fields surrounded by curly braces `{}`. Anything not contained in braces is considered literal text and copied verbatim into the output. Literal brace character can be escaped by two braces: `{{ '{{' }}` and `{{ '}}' }}`. Field names can be numbers (starting from zero) or empty (then they are implicitly given monotonically increasing numbers). **Syntax** diff --git a/src/Functions/concatWithSeparator.cpp b/src/Functions/concatWithSeparator.cpp index f295d86943f..b4f3732710f 100644 --- a/src/Functions/concatWithSeparator.cpp +++ b/src/Functions/concatWithSeparator.cpp @@ -4,11 +4,11 @@ #include #include #include +#include #include #include #include -#include "formatString.h" namespace DB { diff --git a/src/Functions/format.cpp b/src/Functions/format.cpp index d6f1930a85a..d948de1c1a2 100644 --- a/src/Functions/format.cpp +++ b/src/Functions/format.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -50,18 +51,6 @@ public: getName(), arguments.size()); - for (size_t i = 0; i < arguments.size(); ++i) - { - const auto * arg = arguments[i].get(); - if (!isStringOrFixedString(arg)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of argument {} of function {}", - arg->getName(), - i + 1, - getName()); - } - return std::make_shared(); } @@ -81,6 +70,7 @@ public: std::vector offsets(arguments.size() - 1); std::vector fixed_string_sizes(arguments.size() - 1); std::vector> constant_strings(arguments.size() - 1); + std::vector converted_col_ptrs(arguments.size() - 1); bool has_column_string = false; bool has_column_fixed_string = false; @@ -104,8 +94,29 @@ public: constant_strings[i - 1] = const_col->getValue(); } else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", - column->getName(), getName()); + { + /// A non-String/non-FixedString-type argument: use the default serialization to convert it to String + auto full_column = column->convertToFullIfNeeded(); + auto serialization = arguments[i].type->getDefaultSerialization(); + auto converted_col_str = ColumnString::create(); + ColumnStringHelpers::WriteHelper write_helper(*converted_col_str, column->size()); + auto & write_buffer = write_helper.getWriteBuffer(); + FormatSettings format_settings; + for (size_t row = 0; row < column->size(); ++row) + { + serialization->serializeText(*full_column, row, write_buffer, format_settings); + write_helper.rowWritten(); + } + write_helper.finalize(); + + /// Same as the normal `ColumnString` branch + has_column_string = true; + data[i - 1] = &converted_col_str->getChars(); + offsets[i - 1] = &converted_col_str->getOffsets(); + + /// Keep the pointer alive + converted_col_ptrs[i - 1] = std::move(converted_col_str); + } } FormatStringImpl::formatExecute( diff --git a/tests/queries/0_stateless/02935_format_with_arbitrary_types.reference b/tests/queries/0_stateless/02935_format_with_arbitrary_types.reference new file mode 100644 index 00000000000..3455adc8723 --- /dev/null +++ b/tests/queries/0_stateless/02935_format_with_arbitrary_types.reference @@ -0,0 +1,70 @@ +-- Const string + non-const arbitrary type +The answer to all questions is 42. +The answer to all questions is 43. +The answer to all questions is 44. +The answer to all questions is 45. +The answer to all questions is 46. +The answer to all questions is 47. +The answer to all questions is 48. +The answer to all questions is 49. +The answer to all questions is 50. +The answer to all questions is 51. +The answer to all questions is 52. +The answer to all questions is 53. +The answer to all questions is 42.42. +The answer to all questions is 43.43. +The answer to all questions is 44. +The answer to all questions is true. +The answer to all questions is false. +The answer to all questions is foo. +The answer to all questions is bar. +The answer to all questions is foo. +The answer to all questions is bar. +The answer to all questions is foo. +The answer to all questions is bar. +The answer to all questions is foo. +The answer to all questions is bar. +The answer to all questions is 42. +The answer to all questions is 42. +The answer to all questions is fae310ca-d52a-4923-9e9b-02bf67f4b009. +The answer to all questions is 2023-11-14. +The answer to all questions is 2123-11-14. +The answer to all questions is 2023-11-14 05:50:12. +The answer to all questions is 2023-11-14 05:50:12.123. +The answer to all questions is hallo. +The answer to all questions is [\'foo\',\'bar\']. +The answer to all questions is {"foo":"bar"}. +The answer to all questions is (42,\'foo\'). +The answer to all questions is {42:\'foo\'}. +The answer to all questions is 122.233.64.201. +The answer to all questions is 2001:1:130f:2:3:9c0:876a:130b. +The answer to all questions is (42,43). +The answer to all questions is [(0,0),(10,0),(10,10),(0,10)]. +The answer to all questions is [[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]]. +The answer to all questions is [[[(0,0),(10,0),(10,10),(0,10)]],[[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]]]. +-- Nested +The [\'foo\',\'bar\'] to all questions is [\'qaz\',\'qux\']. +-- NULL arguments +\N +\N +\N +\N +\N +\N +\N +-- Various arguments tests +The Non-const to all questions is strings +The Two arguments to all questions is test +The Three to all questions is arguments and test +The 3 to all questions is arguments test and with int type +The 42 to all questions is 144 +The 42 to all questions is 144 and 255 +The 42 to all questions is 144 +The 42 to all questions is 144 and 255 +-- Single argument tests +The answer to all questions is 42. +The answer to all questions is 42. +The answer to all questions is foo. +The answer to all questions is foo. +\N +\N diff --git a/tests/queries/0_stateless/02935_format_with_arbitrary_types.sql b/tests/queries/0_stateless/02935_format_with_arbitrary_types.sql new file mode 100644 index 00000000000..ad1de2bec6d --- /dev/null +++ b/tests/queries/0_stateless/02935_format_with_arbitrary_types.sql @@ -0,0 +1,85 @@ + +-- Tags: no-fasttest +-- no-fasttest: json type needs rapidjson library, geo types need s2 geometry + +SET allow_experimental_object_type = 1; +SET allow_suspicious_low_cardinality_types=1; + +SELECT '-- Const string + non-const arbitrary type'; +SELECT format('The {0} to all questions is {1}.', 'answer', materialize(42 :: Int8)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize(43 :: Int16)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize(44 :: Int32)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize(45 :: Int64)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize(46 :: Int128)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize(47 :: Int256)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize(48 :: UInt8)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize(49 :: UInt16)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize(50 :: UInt32)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize(51 :: UInt64)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize(52 :: UInt128)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize(53 :: UInt256)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize(42.42 :: Float32)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize(43.43 :: Float64)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize(44.44 :: Decimal(2))); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize(true :: Bool)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize(false :: Bool)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize('foo' :: String)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize('bar' :: FixedString(3))); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize('foo' :: Nullable(String))); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize('bar' :: Nullable(FixedString(3)))); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize('foo' :: LowCardinality(String))); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize('bar' :: LowCardinality(FixedString(3)))); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize('foo' :: LowCardinality(Nullable(String)))); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize('bar' :: LowCardinality(Nullable(FixedString(3))))); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize(42 :: LowCardinality(Nullable(UInt32)))); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize(42 :: LowCardinality(UInt32))); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize('fae310ca-d52a-4923-9e9b-02bf67f4b009' :: UUID)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize('2023-11-14' :: Date)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize('2123-11-14' :: Date32)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize('2023-11-14 05:50:12' :: DateTime('Europe/Amsterdam'))); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize('2023-11-14 05:50:12.123' :: DateTime64(3, 'Europe/Amsterdam'))); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize('hallo' :: Enum('hallo' = 1))); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize(['foo', 'bar'] :: Array(String))); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize('{"foo": "bar"}' :: JSON)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize((42, 'foo') :: Tuple(Int32, String))); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize(map(42, 'foo') :: Map(Int32, String))); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize('122.233.64.201' :: IPv4)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize('2001:0001:130F:0002:0003:09C0:876A:130B' :: IPv6)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize((42, 43) :: Point)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize([(0,0),(10,0),(10,10),(0,10)] :: Ring)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize([[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]] :: Polygon)); +SELECT format('The {0} to all questions is {1}.', 'answer', materialize([[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]] :: MultiPolygon)); + +SELECT '-- Nested'; +DROP TABLE IF EXISTS format_nested; +CREATE TABLE format_nested(attrs Nested(k String, v String)) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO format_nested VALUES (['foo', 'bar'], ['qaz', 'qux']); +SELECT format('The {0} to all questions is {1}.', attrs.k, attrs.v) FROM format_nested; +DROP TABLE format_nested; + +SELECT '-- NULL arguments'; +SELECT format('The {0} to all questions is {1}', NULL, NULL); +SELECT format('The {0} to all questions is {1}', NULL, materialize(NULL :: Nullable(UInt64))); +SELECT format('The {0} to all questions is {1}', materialize(NULL :: Nullable(UInt64)), materialize(NULL :: Nullable(UInt64))); +SELECT format('The {0} to all questions is {1}', 42, materialize(NULL :: Nullable(UInt64))); +SELECT format('The {0} to all questions is {1}', '42', materialize(NULL :: Nullable(UInt64))); +SELECT format('The {0} to all questions is {1}', 42, materialize(NULL :: Nullable(UInt64)), materialize(NULL :: Nullable(UInt64))); +SELECT format('The {0} to all questions is {1}', '42', materialize(NULL :: Nullable(UInt64)), materialize(NULL :: Nullable(UInt64))); + +SELECT '-- Various arguments tests'; +SELECT format('The {0} to all questions is {1}', materialize('Non-const'), materialize(' strings')); +SELECT format('The {0} to all questions is {1}', 'Two arguments ', 'test'); +SELECT format('The {0} to all questions is {1} and {2}', 'Three ', 'arguments', ' test'); +SELECT format('The {0} to all questions is {1} and {2}', materialize(3 :: Int64), ' arguments test', ' with int type'); +SELECT format('The {0} to all questions is {1}', materialize(42 :: Int32), materialize(144 :: UInt64)); +SELECT format('The {0} to all questions is {1} and {2}', materialize(42 :: Int32), materialize(144 :: UInt64), materialize(255 :: UInt32)); +SELECT format('The {0} to all questions is {1}', 42, 144); +SELECT format('The {0} to all questions is {1} and {2}', 42, 144, 255); + +SELECT '-- Single argument tests'; +SELECT format('The answer to all questions is {0}.', 42); +SELECT format('The answer to all questions is {0}.', materialize(42)); +SELECT format('The answer to all questions is {0}.', 'foo'); +SELECT format('The answer to all questions is {0}.', materialize('foo')); +SELECT format('The answer to all questions is {0}.', NULL); +SELECT format('The answer to all questions is {0}.', materialize(NULL :: Nullable(UInt64))); From ed618d3be1e7b171a1e19b79fd493460418b9954 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 6 Dec 2023 04:24:43 +0000 Subject: [PATCH 116/331] 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 cfdfde68799b4b70cecdc494f0942b989177cd80 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 6 Dec 2023 08:34:32 +0000 Subject: [PATCH 117/331] Fix style --- src/Functions/format.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/format.cpp b/src/Functions/format.cpp index d948de1c1a2..f1f73cfe438 100644 --- a/src/Functions/format.cpp +++ b/src/Functions/format.cpp @@ -18,7 +18,6 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } From 806061642a4e45bae2d733d27b6c6482156f7360 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 6 Dec 2023 11:39:04 +0100 Subject: [PATCH 118/331] Support GENERATED and DEFAULT columns --- src/Core/ExternalResultDescription.cpp | 5 + src/Core/ExternalResultDescription.h | 3 + src/Core/PostgreSQL/insertPostgreSQLValue.cpp | 8 +- src/Core/PostgreSQL/insertPostgreSQLValue.h | 2 +- .../fetchPostgreSQLTableStructure.cpp | 71 +++- .../fetchPostgreSQLTableStructure.h | 6 +- .../MaterializedPostgreSQLConsumer.cpp | 331 ++++++++++-------- .../MaterializedPostgreSQLConsumer.h | 48 +-- .../PostgreSQLReplicationHandler.cpp | 19 +- .../StorageMaterializedPostgreSQL.cpp | 38 +- .../StorageMaterializedPostgreSQL.h | 3 +- .../test.py | 144 ++++++++ 12 files changed, 471 insertions(+), 207 deletions(-) diff --git a/src/Core/ExternalResultDescription.cpp b/src/Core/ExternalResultDescription.cpp index 0700200a9ec..f7e8a69d355 100644 --- a/src/Core/ExternalResultDescription.cpp +++ b/src/Core/ExternalResultDescription.cpp @@ -20,6 +20,11 @@ namespace ErrorCodes extern const int UNKNOWN_TYPE; } +ExternalResultDescription::ExternalResultDescription(const Block & sample_block_) +{ + init(sample_block_); +} + void ExternalResultDescription::init(const Block & sample_block_) { sample_block = sample_block_; diff --git a/src/Core/ExternalResultDescription.h b/src/Core/ExternalResultDescription.h index a9ffe8b2ed2..b7d852b99cf 100644 --- a/src/Core/ExternalResultDescription.h +++ b/src/Core/ExternalResultDescription.h @@ -41,6 +41,9 @@ struct ExternalResultDescription Block sample_block; std::vector> types; + ExternalResultDescription() = default; + explicit ExternalResultDescription(const Block & sample_block_); + void init(const Block & sample_block_); }; diff --git a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp index d2e8071c5de..2f041134f06 100644 --- a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp +++ b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp @@ -36,7 +36,7 @@ void insertDefaultPostgreSQLValue(IColumn & column, const IColumn & sample_colum void insertPostgreSQLValue( IColumn & column, std::string_view value, const ExternalResultDescription::ValueType type, const DataTypePtr data_type, - std::unordered_map & array_info, size_t idx) + const std::unordered_map & array_info, size_t idx) { switch (type) { @@ -125,8 +125,8 @@ void insertPostgreSQLValue( pqxx::array_parser parser{value}; std::pair parsed = parser.get_next(); - size_t dimension = 0, max_dimension = 0, expected_dimensions = array_info[idx].num_dimensions; - const auto parse_value = array_info[idx].pqxx_parser; + size_t dimension = 0, max_dimension = 0, expected_dimensions = array_info.at(idx).num_dimensions; + const auto parse_value = array_info.at(idx).pqxx_parser; std::vector dimensions(expected_dimensions + 1); while (parsed.first != pqxx::array_parser::juncture::done) @@ -138,7 +138,7 @@ void insertPostgreSQLValue( dimensions[dimension].emplace_back(parse_value(parsed.second)); else if (parsed.first == pqxx::array_parser::juncture::null_value) - dimensions[dimension].emplace_back(array_info[idx].default_value); + dimensions[dimension].emplace_back(array_info.at(idx).default_value); else if (parsed.first == pqxx::array_parser::juncture::row_end) { diff --git a/src/Core/PostgreSQL/insertPostgreSQLValue.h b/src/Core/PostgreSQL/insertPostgreSQLValue.h index b842d86ed47..3bc83292b96 100644 --- a/src/Core/PostgreSQL/insertPostgreSQLValue.h +++ b/src/Core/PostgreSQL/insertPostgreSQLValue.h @@ -23,7 +23,7 @@ struct PostgreSQLArrayInfo void insertPostgreSQLValue( IColumn & column, std::string_view value, const ExternalResultDescription::ValueType type, const DataTypePtr data_type, - std::unordered_map & array_info, size_t idx); + const std::unordered_map & array_info, size_t idx); void preparePostgreSQLArrayInfo( std::unordered_map & array_info, size_t column_idx, const DataTypePtr data_type); diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 2c97c92ba99..527936f1c19 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -186,20 +186,25 @@ PostgreSQLTableStructure::ColumnsInfoPtr readNamesAndTypesList( } else { - std::tuple row; + std::tuple row; while (stream >> row) { - auto data_type = convertPostgreSQLDataType( + const auto column_name = std::get<0>(row); + const auto data_type = convertPostgreSQLDataType( std::get<1>(row), recheck_array, use_nulls && (std::get<2>(row) == /* not nullable */"f"), std::get<3>(row)); - columns.push_back(NameAndTypePair(std::get<0>(row), data_type)); + columns.push_back(NameAndTypePair(column_name, data_type)); + auto attgenerated = std::get<6>(row); + LOG_TEST(&Poco::Logger::get("kssenii"), "KSSENII: attgenerated: {}", attgenerated); - attributes.emplace_back( - PostgreSQLTableStructure::PGAttribute{ - .atttypid = parse(std::get<4>(row)), - .atttypmod = parse(std::get<5>(row)), + attributes.emplace( + column_name, + PostgreSQLTableStructure::PGAttribute{ + .atttypid = parse(std::get<4>(row)), + .atttypmod = parse(std::get<5>(row)), + .attgenerated = attgenerated.empty() ? char{} : char(attgenerated[0]) }); ++i; @@ -253,14 +258,19 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( PostgreSQLTableStructure table; auto where = fmt::format("relname = {}", quoteString(postgres_table)); - if (postgres_schema.empty()) - where += " AND relnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'public')"; - else - where += fmt::format(" AND relnamespace = (SELECT oid FROM pg_namespace WHERE nspname = {})", quoteString(postgres_schema)); + + where += postgres_schema.empty() + ? " AND relnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'public')" + : fmt::format(" AND relnamespace = (SELECT oid FROM pg_namespace WHERE nspname = {})", quoteString(postgres_schema)); std::string query = fmt::format( - "SELECT attname AS name, format_type(atttypid, atttypmod) AS type, " - "attnotnull AS not_null, attndims AS dims, atttypid as type_id, atttypmod as type_modifier " + "SELECT attname AS name, " /// column name + "format_type(atttypid, atttypmod) AS type, " /// data type + "attnotnull AS not_null, " /// is nullable + "attndims AS dims, " /// array dimensions + "atttypid as type_id, " + "atttypmod as type_modifier, " + "attgenerated as generated " /// if column has GENERATED "FROM pg_attribute " "WHERE attrelid = (SELECT oid FROM pg_class WHERE {}) " "AND NOT attisdropped AND attnum > 0", where); @@ -271,11 +281,44 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( if (!table.physical_columns) throw Exception(ErrorCodes::UNKNOWN_TABLE, "PostgreSQL table {} does not exist", postgres_table_with_schema); + for (const auto & column : table.physical_columns->columns) + { + table.physical_columns->names.push_back(column.name); + } + + bool check_generated = table.physical_columns->attributes.end() != std::find_if( + table.physical_columns->attributes.begin(), + table.physical_columns->attributes.end(), + [](const auto & attr){ return attr.second.attgenerated == 's'; }); + + if (check_generated) + { + std::string attrdef_query = fmt::format( + "SELECT adnum, pg_get_expr(adbin, adrelid) as generated_expression " + "FROM pg_attrdef " + "WHERE adrelid = (SELECT oid FROM pg_class WHERE {});", where); + + pqxx::result result{tx.exec(attrdef_query)}; + for (const auto row : result) + { + size_t adnum = row[0].as(); + if (!adnum || adnum > table.physical_columns->names.size()) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Received adnum {}, but currently fetched columns list has {} columns", + adnum, table.physical_columns->attributes.size()); + } + const auto column_name = table.physical_columns->names[adnum - 1]; + table.physical_columns->attributes.at(column_name).attr_def = row[1].as(); + } + } + if (with_primary_key) { /// wiki.postgresql.org/wiki/Retrieve_primary_key_columns query = fmt::format( - "SELECT a.attname, format_type(a.atttypid, a.atttypmod) AS data_type " + "SELECT a.attname, " /// column name + "format_type(a.atttypid, a.atttypmod) AS data_type " /// data type "FROM pg_index i " "JOIN pg_attribute a ON a.attrelid = i.indrelid " "AND a.attnum = ANY(i.indkey) " diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h index 7cd21d353a2..81bf7b278fc 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h @@ -16,13 +16,17 @@ struct PostgreSQLTableStructure { Int32 atttypid; Int32 atttypmod; + bool atthasdef; + char attgenerated; + std::string attr_def; }; - using Attributes = std::vector; + using Attributes = std::unordered_map; struct ColumnsInfo { NamesAndTypesList columns; Attributes attributes; + std::vector names; ColumnsInfo(NamesAndTypesList && columns_, Attributes && attributes_) : columns(columns_), attributes(attributes_) {} }; using ColumnsInfoPtr = std::shared_ptr; diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index e7bd6be7b2b..6be1563d16c 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -24,6 +24,22 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +namespace +{ + using ArrayInfo = std::unordered_map; + + ArrayInfo createArrayInfos(const NamesAndTypesList & columns, const ExternalResultDescription & columns_description) + { + ArrayInfo array_info; + for (size_t i = 0; i < columns.size(); ++i) + { + if (columns_description.types[i].first == ExternalResultDescription::ValueType::vtArray) + preparePostgreSQLArrayInfo(array_info, i, columns_description.sample_block.getByPosition(i).type); + } + return array_info; + } +} + MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer( ContextPtr context_, std::shared_ptr connection_, @@ -40,126 +56,160 @@ MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer( , publication_name(publication_name_) , connection(connection_) , current_lsn(start_lsn) + , final_lsn(start_lsn) , lsn_value(getLSNValue(start_lsn)) , max_block_size(max_block_size_) , schema_as_a_part_of_table_name(schema_as_a_part_of_table_name_) { - final_lsn = start_lsn; - auto tx = std::make_shared(connection->getRef()); - current_lsn = advanceLSN(tx); - LOG_TRACE(log, "Starting replication. LSN: {} (last: {})", getLSNValue(current_lsn), getLSNValue(final_lsn)); - tx->commit(); - - for (const auto & [table_name, storage_info] : storages_info_) - storages.emplace(table_name, storage_info); -} - - -MaterializedPostgreSQLConsumer::StorageData::StorageData(const StorageInfo & storage_info) - : storage(storage_info.storage), buffer(storage_info.storage->getInMemoryMetadataPtr(), storage_info.attributes) -{ - auto table_id = storage_info.storage->getStorageID(); - LOG_TRACE(&Poco::Logger::get("StorageMaterializedPostgreSQL"), - "New buffer for table {}, number of attributes: {}, number if columns: {}, structure: {}", - table_id.getNameForLogs(), buffer.attributes.size(), buffer.getColumnsNum(), buffer.description.sample_block.dumpStructure()); -} - - -MaterializedPostgreSQLConsumer::StorageData::Buffer::Buffer( - StorageMetadataPtr storage_metadata, const PostgreSQLTableStructure::Attributes & attributes_) - : attributes(attributes_) -{ - const Block sample_block = storage_metadata->getSampleBlock(); - - /// Need to clear type, because in description.init() the types are appended - description.types.clear(); - description.init(sample_block); - - columns = description.sample_block.cloneEmptyColumns(); - const auto & storage_columns = storage_metadata->getColumns().getAllPhysical(); - auto insert_columns = std::make_shared(); - - auto columns_num = description.sample_block.columns(); - assert(columns_num == storage_columns.size()); - if (attributes.size() + 2 != columns_num) /// +2 because sign and version columns - throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns number mismatch. Attributes: {}, buffer: {}", - attributes.size(), columns_num); - - size_t idx = 0; - for (const auto & column : storage_columns) { - if (description.types[idx].first == ExternalResultDescription::ValueType::vtArray) - preparePostgreSQLArrayInfo(array_info, idx, description.sample_block.getByPosition(idx).type); - idx++; - - insert_columns->children.emplace_back(std::make_shared(column.name)); + auto tx = std::make_shared(connection->getRef()); + current_lsn = advanceLSN(tx); + tx->commit(); } - columns_ast = std::move(insert_columns); + for (const auto & [table_name, storage_info] : storages_info_) + storages.emplace(table_name, StorageData(storage_info, log)); + + LOG_TRACE(log, "Starting replication. LSN: {} (last: {}), storages: {}", + getLSNValue(current_lsn), getLSNValue(final_lsn), storages.size()); } -void MaterializedPostgreSQLConsumer::assertCorrectInsertion(StorageData::Buffer & buffer, size_t column_idx) +MaterializedPostgreSQLConsumer::StorageData::StorageData(const StorageInfo & storage_info, Poco::Logger * log_) + : storage(storage_info.storage) + , table_description(storage_info.storage->getInMemoryMetadataPtr()->getSampleBlock()) + , columns_attributes(storage_info.attributes) + , array_info(createArrayInfos(storage_info.storage->getInMemoryMetadataPtr()->getColumns().getAllPhysical(), table_description)) { - if (column_idx >= buffer.description.sample_block.columns() - || column_idx >= buffer.description.types.size() - || column_idx >= buffer.columns.size()) - throw Exception( - ErrorCodes::LOGICAL_ERROR, + auto columns_num = table_description.sample_block.columns(); + /// +2 because of _sign and _version columns + if (columns_attributes.size() + 2 != columns_num) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Columns number mismatch. Attributes: {}, buffer: {}", + columns_attributes.size(), columns_num); + } + + LOG_TRACE(log_, "Adding definition for table {}, structure: {}", + storage_info.storage->getStorageID().getNameForLogs(), + table_description.sample_block.dumpStructure()); +} + +MaterializedPostgreSQLConsumer::StorageData::Buffer::Buffer( + ColumnsWithTypeAndName && columns_, + const ExternalResultDescription & table_description_) +{ + if (columns_.end() != std::find_if( + columns_.begin(), columns_.end(), + [](const auto & col) { return col.name == "_sign" || col.name == "_version"; })) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "PostgreSQL table cannot contain `_sign` or `_version` columns " + "as they are reserved for internal usage"); + } + + columns_.push_back(table_description_.sample_block.getByName("_sign")); + columns_.push_back(table_description_.sample_block.getByName("_version")); + + for (const auto & col : columns_) + { + if (!table_description_.sample_block.has(col.name)) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Having column {}, but no such column in table ({})", + col.name, table_description_.sample_block.dumpStructure()); + } + + const auto & actual_column = table_description_.sample_block.getByName(col.name); + if (col.type != actual_column.type) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Having column {} of type {}, but expected {}", + col.name, col.type->getName(), actual_column.type->getName()); + } + } + + sample_block = Block(columns_); + columns = sample_block.cloneEmptyColumns(); + + for (const auto & name : sample_block.getNames()) + columns_ast.children.emplace_back(std::make_shared(name)); +} + +MaterializedPostgreSQLConsumer::StorageData::Buffer & MaterializedPostgreSQLConsumer::StorageData::getBuffer() +{ + if (!buffer) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Data buffer not initialized for {}", + storage->getStorageID().getNameForLogs()); + } + + return *buffer; +} + +void MaterializedPostgreSQLConsumer::StorageData::Buffer::assertInsertIsPossible(size_t col_idx) const +{ + if (col_idx >= columns.size()) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to insert into buffer at position: " - "{}, but block columns size is {}, types size: {}, columns size: {}, buffer structure: {}", - column_idx, - buffer.description.sample_block.columns(), - buffer.description.types.size(), buffer.columns.size(), - buffer.description.sample_block.dumpStructure()); + "{}, but block columns size is {} (full structure: {})", + col_idx, columns.size(), sample_block.dumpStructure()); + } } -void MaterializedPostgreSQLConsumer::insertValue(StorageData::Buffer & buffer, const std::string & value, size_t column_idx) +void MaterializedPostgreSQLConsumer::insertValue(StorageData & storage_data, const std::string & value, size_t column_idx) { - assertCorrectInsertion(buffer, column_idx); + auto & buffer = storage_data.getBuffer(); + buffer.assertInsertIsPossible(column_idx); - const auto & sample = buffer.description.sample_block.getByPosition(column_idx); - bool is_nullable = buffer.description.types[column_idx].second; + const auto & column_type_and_name = buffer.sample_block.getByPosition(column_idx); + auto & column = buffer.columns[column_idx]; + + const size_t column_idx_in_table = storage_data.table_description.sample_block.getPositionByName(column_type_and_name.name); + const auto & type_description = storage_data.table_description.types[column_idx_in_table]; try { - if (is_nullable) + if (column_type_and_name.type->isNullable()) { - ColumnNullable & column_nullable = assert_cast(*buffer.columns[column_idx]); - const auto & data_type = assert_cast(*sample.type); + ColumnNullable & column_nullable = assert_cast(*column); + const auto & data_type = assert_cast(*column_type_and_name.type); insertPostgreSQLValue( - column_nullable.getNestedColumn(), value, - buffer.description.types[column_idx].first, data_type.getNestedType(), buffer.array_info, column_idx); + column_nullable.getNestedColumn(), value, type_description.first, + data_type.getNestedType(), storage_data.array_info, column_idx_in_table); column_nullable.getNullMapData().emplace_back(0); } else { insertPostgreSQLValue( - *buffer.columns[column_idx], value, - buffer.description.types[column_idx].first, sample.type, - buffer.array_info, column_idx); + *column, value, type_description.first, column_type_and_name.type, + storage_data.array_info, column_idx_in_table); } } catch (const pqxx::conversion_error & e) { - LOG_ERROR(log, "Conversion failed while inserting PostgreSQL value {}, will insert default value. Error: {}", value, e.what()); - insertDefaultValue(buffer, column_idx); + LOG_ERROR(log, "Conversion failed while inserting PostgreSQL value {}, " + "will insert default value. Error: {}", value, e.what()); + + insertDefaultPostgreSQLValue(*column, *column_type_and_name.column); } } - -void MaterializedPostgreSQLConsumer::insertDefaultValue(StorageData::Buffer & buffer, size_t column_idx) +void MaterializedPostgreSQLConsumer::insertDefaultValue(StorageData & storage_data, size_t column_idx) { - assertCorrectInsertion(buffer, column_idx); + auto & buffer = storage_data.getBuffer(); + buffer.assertInsertIsPossible(column_idx); - const auto & sample = buffer.description.sample_block.getByPosition(column_idx); - insertDefaultPostgreSQLValue(*buffer.columns[column_idx], *sample.column); + const auto & column_type_and_name = buffer.sample_block.getByPosition(column_idx); + auto & column = buffer.columns[column_idx]; + + insertDefaultPostgreSQLValue(*column, *column_type_and_name.column); } - void MaterializedPostgreSQLConsumer::readString(const char * message, size_t & pos, size_t size, String & result) { assert(size > pos + 2); @@ -173,7 +223,6 @@ void MaterializedPostgreSQLConsumer::readString(const char * message, size_t & p } } - template T MaterializedPostgreSQLConsumer::unhexN(const char * message, size_t pos, size_t n) { @@ -186,7 +235,6 @@ T MaterializedPostgreSQLConsumer::unhexN(const char * message, size_t pos, size_ return result; } - Int64 MaterializedPostgreSQLConsumer::readInt64(const char * message, size_t & pos, [[maybe_unused]] size_t size) { assert(size >= pos + 16); @@ -195,7 +243,6 @@ Int64 MaterializedPostgreSQLConsumer::readInt64(const char * message, size_t & p return result; } - Int32 MaterializedPostgreSQLConsumer::readInt32(const char * message, size_t & pos, [[maybe_unused]] size_t size) { assert(size >= pos + 8); @@ -204,7 +251,6 @@ Int32 MaterializedPostgreSQLConsumer::readInt32(const char * message, size_t & p return result; } - Int16 MaterializedPostgreSQLConsumer::readInt16(const char * message, size_t & pos, [[maybe_unused]] size_t size) { assert(size >= pos + 4); @@ -213,7 +259,6 @@ Int16 MaterializedPostgreSQLConsumer::readInt16(const char * message, size_t & p return result; } - Int8 MaterializedPostgreSQLConsumer::readInt8(const char * message, size_t & pos, [[maybe_unused]] size_t size) { assert(size >= pos + 2); @@ -222,25 +267,23 @@ Int8 MaterializedPostgreSQLConsumer::readInt8(const char * message, size_t & pos return result; } - void MaterializedPostgreSQLConsumer::readTupleData( - StorageData::Buffer & buffer, const char * message, size_t & pos, [[maybe_unused]] size_t size, PostgreSQLQuery type, bool old_value) + StorageData & storage_data, + const char * message, + size_t & pos, + size_t size, + PostgreSQLQuery type, + bool old_value) { Int16 num_columns = readInt16(message, pos, size); - /// Sanity check. In fact, it was already checked. - if (static_cast(num_columns) + 2 != buffer.getColumnsNum()) /// +2 -- sign and version columns - throw Exception(ErrorCodes::POSTGRESQL_REPLICATION_INTERNAL_ERROR, - "Number of columns does not match. Got: {}, expected {}, current buffer structure: {}", - num_columns, buffer.getColumnsNum(), buffer.description.sample_block.dumpStructure()); - auto proccess_column_value = [&](Int8 identifier, Int16 column_idx) { switch (identifier) // NOLINT(bugprone-switch-missing-default-case) { case 'n': /// NULL { - insertDefaultValue(buffer, column_idx); + insertDefaultValue(storage_data, column_idx); break; } case 't': /// Text formatted value @@ -250,7 +293,7 @@ void MaterializedPostgreSQLConsumer::readTupleData( for (Int32 i = 0; i < col_len; ++i) value += readInt8(message, pos, size); - insertValue(buffer, value, column_idx); + insertValue(storage_data, value, column_idx); break; } case 'u': /// TOAST value && unchanged at the same time. Actual value is not sent. @@ -258,13 +301,13 @@ void MaterializedPostgreSQLConsumer::readTupleData( /// TOAST values are not supported. (TOAST values are values that are considered in postgres /// to be too large to be stored directly) LOG_WARNING(log, "Got TOAST value, which is not supported, default value will be used instead."); - insertDefaultValue(buffer, column_idx); + insertDefaultValue(storage_data, column_idx); break; } case 'b': /// Binary data. { LOG_WARNING(log, "We do not yet process this format of data, will insert default value"); - insertDefaultValue(buffer, column_idx); + insertDefaultValue(storage_data, column_idx); break; } default: @@ -272,7 +315,7 @@ void MaterializedPostgreSQLConsumer::readTupleData( LOG_WARNING(log, "Unexpected identifier: {}. This is a bug! Please report an issue on github", identifier); chassert(false); - insertDefaultValue(buffer, column_idx); + insertDefaultValue(storage_data, column_idx); break; } } @@ -291,7 +334,7 @@ void MaterializedPostgreSQLConsumer::readTupleData( "Got error while receiving value for column {}, will insert default value. Error: {}", column_idx, getCurrentExceptionMessage(true)); - insertDefaultValue(buffer, column_idx); + insertDefaultValue(storage_data, column_idx); /// Let's collect only the first exception. /// This delaying of error throw is needed because /// some errors can be ignored and just logged, @@ -301,19 +344,20 @@ void MaterializedPostgreSQLConsumer::readTupleData( } } + auto & columns = storage_data.getBuffer().columns; switch (type) { case PostgreSQLQuery::INSERT: { - buffer.columns[num_columns]->insert(static_cast(1)); - buffer.columns[num_columns + 1]->insert(lsn_value); + columns[num_columns]->insert(static_cast(1)); + columns[num_columns + 1]->insert(lsn_value); break; } case PostgreSQLQuery::DELETE: { - buffer.columns[num_columns]->insert(static_cast(-1)); - buffer.columns[num_columns + 1]->insert(lsn_value); + columns[num_columns]->insert(static_cast(-1)); + columns[num_columns + 1]->insert(lsn_value); break; } @@ -321,11 +365,11 @@ void MaterializedPostgreSQLConsumer::readTupleData( { /// Process old value in case changed value is a primary key. if (old_value) - buffer.columns[num_columns]->insert(static_cast(-1)); + columns[num_columns]->insert(static_cast(-1)); else - buffer.columns[num_columns]->insert(static_cast(1)); + columns[num_columns]->insert(static_cast(1)); - buffer.columns[num_columns + 1]->insert(lsn_value); + columns[num_columns + 1]->insert(lsn_value); break; } @@ -335,7 +379,6 @@ void MaterializedPostgreSQLConsumer::readTupleData( std::rethrow_exception(error); } - /// https://www.postgresql.org/docs/13/protocol-logicalrep-message-formats.html void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * replication_message, size_t size) { @@ -366,10 +409,10 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl return; Int8 new_tuple = readInt8(replication_message, pos, size); - auto & buffer = storages.find(table_name)->second.buffer; + auto & storage_data = storages.find(table_name)->second; if (new_tuple) - readTupleData(buffer, replication_message, pos, size, PostgreSQLQuery::INSERT); + readTupleData(storage_data, replication_message, pos, size, PostgreSQLQuery::INSERT); break; } @@ -386,7 +429,7 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl if (!isSyncAllowed(relation_id, table_name)) return; - auto & buffer = storages.find(table_name)->second.buffer; + auto & storage_data = storages.find(table_name)->second; auto proccess_identifier = [&](Int8 identifier) -> bool { @@ -401,13 +444,13 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl /// it is much more efficient to use replica identity index, but support all possible cases. case 'O': { - readTupleData(buffer, replication_message, pos, size, PostgreSQLQuery::UPDATE, true); + readTupleData(storage_data, replication_message, pos, size, PostgreSQLQuery::UPDATE, true); break; } case 'N': { /// New row. - readTupleData(buffer, replication_message, pos, size, PostgreSQLQuery::UPDATE); + readTupleData(storage_data, replication_message, pos, size, PostgreSQLQuery::UPDATE); read_next = false; break; } @@ -441,8 +484,8 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl /// 0 or 1 if replica identity is set to full. For now only default replica identity is supported (with primary keys). readInt8(replication_message, pos, size); - auto & buffer = storages.find(table_name)->second.buffer; - readTupleData(buffer, replication_message, pos, size, PostgreSQLQuery::DELETE); + auto & storage_data = storages.find(table_name)->second; + readTupleData(storage_data, replication_message, pos, size, PostgreSQLQuery::DELETE); break; } case 'C': // Commit @@ -490,8 +533,6 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl return; } - auto & buffer = storage_iter->second.buffer; - /// 'd' - default (primary key if any) /// 'n' - nothing /// 'f' - all columns (set replica identity full) @@ -509,29 +550,13 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl Int16 num_columns = readInt16(replication_message, pos, size); - if (static_cast(num_columns) + 2 != buffer.getColumnsNum()) /// +2 -- sign and version columns - { - markTableAsSkipped(relation_id, table_name); - return; - } - - if (static_cast(num_columns) != buffer.attributes.size()) - { -#ifndef NDEBUG - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Mismatch in attributes size. Got {}, expected {}. It's a bug. Current buffer structure: {}", - num_columns, buffer.attributes.size(), buffer.description.sample_block.dumpStructure()); -#else - LOG_ERROR(log, "Mismatch in attributes size. Got {}, expected {}. It's a bug. Current buffer structure: {}", - num_columns, buffer.attributes.size(), buffer.description.sample_block.dumpStructure()); - markTableAsSkipped(relation_id, table_name); - return; -#endif - } - Int32 data_type_id; Int32 type_modifier; /// For example, n in varchar(n) + auto & storage_data = storage_iter->second; + const auto & description = storage_data.table_description; + + ColumnsWithTypeAndName columns; for (uint16_t i = 0; i < num_columns; ++i) { String column_name; @@ -541,13 +566,22 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl data_type_id = readInt32(replication_message, pos, size); type_modifier = readInt32(replication_message, pos, size); - if (buffer.attributes[i].atttypid != data_type_id || buffer.attributes[i].atttypmod != type_modifier) + columns.push_back(description.sample_block.getByName(column_name)); + + const auto & attributes_it = storage_data.columns_attributes.find(column_name); + if (attributes_it == storage_data.columns_attributes.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown column: {}", column_name); + + const auto & attributes = attributes_it->second; + if (attributes.atttypid != data_type_id || attributes.atttypmod != type_modifier) { + LOG_TEST(log, "Column {} has a different type", column_name); markTableAsSkipped(relation_id, table_name); return; } } + storage_data.setBuffer(std::make_unique(std::move(columns), description)); tables_to_sync.insert(table_name); break; } @@ -563,7 +597,6 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl } } - void MaterializedPostgreSQLConsumer::syncTables() { size_t synced_tables = 0; @@ -571,8 +604,8 @@ void MaterializedPostgreSQLConsumer::syncTables() { auto table_name = *tables_to_sync.begin(); auto & storage_data = storages.find(table_name)->second; - Block result_rows = storage_data.buffer.description.sample_block.cloneWithColumns(std::move(storage_data.buffer.columns)); - storage_data.buffer.columns = storage_data.buffer.description.sample_block.cloneEmptyColumns(); + auto & buffer = storage_data.getBuffer(); + Block result_rows = buffer.sample_block.cloneWithColumns(std::move(buffer.columns)); try { @@ -585,7 +618,7 @@ void MaterializedPostgreSQLConsumer::syncTables() auto insert = std::make_shared(); insert->table_id = storage->getStorageID(); - insert->columns = storage_data.buffer.columns_ast; + insert->columns = std::make_shared(buffer.columns_ast); InterpreterInsertQuery interpreter(insert, insert_context, true); auto io = interpreter.execute(); @@ -603,10 +636,11 @@ void MaterializedPostgreSQLConsumer::syncTables() catch (...) { /// Retry this buffer later. - storage_data.buffer.columns = result_rows.mutateColumns(); + buffer.columns = result_rows.mutateColumns(); throw; } + storage_data.setBuffer(nullptr); tables_to_sync.erase(tables_to_sync.begin()); } @@ -616,7 +650,6 @@ void MaterializedPostgreSQLConsumer::syncTables() updateLsn(); } - void MaterializedPostgreSQLConsumer::updateLsn() { try @@ -632,7 +665,6 @@ void MaterializedPostgreSQLConsumer::updateLsn() } } - String MaterializedPostgreSQLConsumer::advanceLSN(std::shared_ptr tx) { std::string query_str = fmt::format("SELECT end_lsn FROM pg_replication_slot_advance('{}', '{}')", replication_slot_name, final_lsn); @@ -644,7 +676,6 @@ String MaterializedPostgreSQLConsumer::advanceLSN(std::shared_ptr; + + const StoragePtr storage; + const ExternalResultDescription table_description; + const PostgreSQLTableStructure::Attributes columns_attributes; + const ArrayInfo array_info; + struct Buffer { - ExternalResultDescription description; + Block sample_block; MutableColumns columns; + ASTExpressionList columns_ast; - /// Needed to pass to insert query columns list in syncTables(). - std::shared_ptr columns_ast; - /// Needed for insertPostgreSQLValue() method to parse array - std::unordered_map array_info; - /// To validate ddl. - PostgreSQLTableStructure::Attributes attributes; + explicit Buffer(ColumnsWithTypeAndName && columns_, const ExternalResultDescription & table_description_); - Buffer(StorageMetadataPtr storage_metadata, const PostgreSQLTableStructure::Attributes & attributes_); - - size_t getColumnsNum() const - { - const auto & sample_block = description.sample_block; - return sample_block.columns(); - } + void assertInsertIsPossible(size_t col_idx) const; }; - StoragePtr storage; - Buffer buffer; + Buffer & getBuffer(); - explicit StorageData(const StorageInfo & storage_info); - StorageData(const StorageData & other) = delete; + void setBuffer(std::unique_ptr buffer_) { buffer = std::move(buffer_); } + + private: + std::unique_ptr buffer; }; using Storages = std::unordered_map; @@ -97,8 +101,8 @@ private: bool isSyncAllowed(Int32 relation_id, const String & relation_name); - static void insertDefaultValue(StorageData::Buffer & buffer, size_t column_idx); - void insertValue(StorageData::Buffer & buffer, const std::string & value, size_t column_idx); + static void insertDefaultValue(StorageData & storage_data, size_t column_idx); + void insertValue(StorageData & storage_data, const std::string & value, size_t column_idx); enum class PostgreSQLQuery { @@ -107,7 +111,7 @@ private: DELETE }; - void readTupleData(StorageData::Buffer & buffer, const char * message, size_t & pos, size_t size, PostgreSQLQuery type, bool old_value = false); + void readTupleData(StorageData & storage_data, const char * message, size_t & pos, size_t size, PostgreSQLQuery type, bool old_value = false); template static T unhexN(const char * message, size_t pos, size_t n); @@ -119,8 +123,6 @@ private: void markTableAsSkipped(Int32 relation_id, const String & relation_name); - static void assertCorrectInsertion(StorageData::Buffer & buffer, size_t column_idx); - /// lsn - log sequence number, like wal offset (64 bit). static Int64 getLSNValue(const std::string & lsn) { diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index ee38dcb44d4..7a73bdf153b 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -337,6 +337,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) dropReplicationSlot(tx); initial_sync(); + LOG_DEBUG(log, "Loaded {} tables", nested_storages.size()); } /// Synchronization and initial load already took place - do not create any new tables, just fetch StoragePtr's /// and pass them to replication consumer. @@ -414,16 +415,18 @@ StorageInfo PostgreSQLReplicationHandler::loadFromSnapshot(postgres::Connection std::string query_str = fmt::format("SET TRANSACTION SNAPSHOT '{}'", snapshot_name); tx->exec(query_str); + auto table_structure = fetchTableStructure(*tx, table_name); + if (!table_structure->physical_columns) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No table attributes"); + + auto table_attributes = table_structure->physical_columns->attributes; + /// Load from snapshot, which will show table state before creation of replication slot. /// Already connected to needed database, no need to add it to query. auto quoted_name = doubleQuoteWithSchema(table_name); query_str = fmt::format("SELECT * FROM ONLY {}", quoted_name); - LOG_DEBUG(log, "Loading PostgreSQL table {}.{}", postgres_database, quoted_name); - auto table_structure = fetchTableStructure(*tx, table_name); - if (!table_structure->physical_columns) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No table attributes"); - auto table_attributes = table_structure->physical_columns->attributes; + LOG_DEBUG(log, "Loading PostgreSQL table {}.{}", postgres_database, quoted_name); auto table_override = tryGetTableOverride(current_database_name, table_name); materialized_storage->createNestedIfNeeded(std::move(table_structure), table_override ? table_override->as() : nullptr); @@ -444,12 +447,16 @@ StorageInfo PostgreSQLReplicationHandler::loadFromSnapshot(postgres::Connection assertBlocksHaveEqualStructure(input->getPort().getHeader(), block_io.pipeline.getHeader(), "postgresql replica load from snapshot"); block_io.pipeline.complete(Pipe(std::move(input))); + /// TODO: make a test when we fail in the middle of inserting data from source. + CompletedPipelineExecutor executor(block_io.pipeline); executor.execute(); materialized_storage->set(nested_storage); auto nested_table_id = nested_storage->getStorageID(); - LOG_DEBUG(log, "Loaded table {}.{} (uuid: {})", nested_table_id.database_name, nested_table_id.table_name, toString(nested_table_id.uuid)); + + LOG_DEBUG(log, "Loaded table {}.{} (uuid: {})", + nested_table_id.database_name, nested_table_id.table_name, toString(nested_table_id.uuid)); return StorageInfo(nested_storage, std::move(table_attributes)); } diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index d83722dba6c..c753a41be40 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -25,6 +25,8 @@ #include #include #include +#include +#include #include #include @@ -195,7 +197,8 @@ void StorageMaterializedPostgreSQL::createNestedIfNeeded(PostgreSQLTableStructur const auto ast_create = getCreateNestedTableQuery(std::move(table_structure), table_override); auto table_id = getStorageID(); auto tmp_nested_table_id = StorageID(table_id.database_name, getNestedTableName()); - LOG_DEBUG(log, "Creating clickhouse table for postgresql table {}", table_id.getNameForLogs()); + LOG_DEBUG(log, "Creating clickhouse table for postgresql table {} (ast: {})", + table_id.getNameForLogs(), serializeAST(*ast_create)); InterpreterCreateQuery interpreter(ast_create, nested_context); interpreter.execute(); @@ -359,7 +362,8 @@ ASTPtr StorageMaterializedPostgreSQL::getColumnDeclaration(const DataTypePtr & d } -std::shared_ptr StorageMaterializedPostgreSQL::getColumnsExpressionList(const NamesAndTypesList & columns) const +std::shared_ptr +StorageMaterializedPostgreSQL::getColumnsExpressionList(const NamesAndTypesList & columns, std::unordered_map defaults) const { auto columns_expression_list = std::make_shared(); for (const auto & [name, type] : columns) @@ -369,6 +373,12 @@ std::shared_ptr StorageMaterializedPostgreSQL::getColumnsExpr column_declaration->name = name; column_declaration->type = getColumnDeclaration(type); + if (auto it = defaults.find(name); it != defaults.end()) + { + column_declaration->default_expression = it->second; + column_declaration->default_specifier = "DEFAULT"; + } + columns_expression_list->children.emplace_back(column_declaration); } return columns_expression_list; @@ -460,8 +470,28 @@ ASTPtr StorageMaterializedPostgreSQL::getCreateNestedTableQuery( } else { - ordinary_columns_and_types = table_structure->physical_columns->columns; - columns_declare_list->set(columns_declare_list->columns, getColumnsExpressionList(ordinary_columns_and_types)); + const auto columns = table_structure->physical_columns; + std::unordered_map defaults; + for (const auto & col : columns->columns) + { + const auto & attr = columns->attributes.at(col.name); + if (!attr.attr_def.empty()) + { + ParserExpression expr_parser; + Expected expected; + ASTPtr result; + + Tokens tokens(attr.attr_def.data(), attr.attr_def.data() + attr.attr_def.size()); + IParser::Pos pos(tokens, DBMS_DEFAULT_MAX_PARSER_DEPTH); + if (!expr_parser.parse(pos, result, expected)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to parse default expression: {}", attr.attr_def); + } + defaults.emplace(col.name, result); + } + } + ordinary_columns_and_types = columns->columns; + columns_declare_list->set(columns_declare_list->columns, getColumnsExpressionList(ordinary_columns_and_types, defaults)); } if (ordinary_columns_and_types.empty()) diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h index af0adb10f9f..9c3c195e34f 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h @@ -109,7 +109,8 @@ public: ASTPtr getCreateNestedTableQuery(PostgreSQLTableStructurePtr table_structure, const ASTTableOverride * table_override); - std::shared_ptr getColumnsExpressionList(const NamesAndTypesList & columns) const; + std::shared_ptr getColumnsExpressionList( + const NamesAndTypesList & columns, std::unordered_map defaults = {}) const; StoragePtr getNested() const; 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..2a72c3591e9 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -810,6 +810,150 @@ def test_replica_consumer(started_cluster): pg_manager_instance2.clear() +def test_replica_consumer(started_cluster): + table = "test_replica_consumer" + pg_manager_instance2.restart() + + pg_manager.create_postgres_table(table) + instance.query( + f"INSERT INTO postgres_database.{table} SELECT number, number from numbers(0, 50)" + ) + + for pm in [pg_manager, pg_manager_instance2]: + pm.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", + "materialized_postgresql_use_unique_replication_consumer_identifier = 1", + ], + ) + + check_tables_are_synchronized( + instance, table, postgres_database=pg_manager.get_default_database() + ) + check_tables_are_synchronized( + instance2, table, postgres_database=pg_manager_instance2.get_default_database() + ) + + assert 50 == int(instance.query(f"SELECT count() FROM test_database.{table}")) + assert 50 == int(instance2.query(f"SELECT count() FROM test_database.{table}")) + + instance.query( + f"INSERT INTO postgres_database.{table} SELECT number, number from numbers(1000, 1000)" + ) + + check_tables_are_synchronized( + instance, table, postgres_database=pg_manager.get_default_database() + ) + check_tables_are_synchronized( + instance2, table, postgres_database=pg_manager_instance2.get_default_database() + ) + + assert 1050 == int(instance.query(f"SELECT count() FROM test_database.{table}")) + assert 1050 == int(instance2.query(f"SELECT count() FROM test_database.{table}")) + + for pm in [pg_manager, pg_manager_instance2]: + pm.drop_materialized_db() + pg_manager_instance2.clear() + + +def test_generated_columns(started_cluster): + table = "test_generated_columns" + + pg_manager.create_postgres_table( + table, + "", + f"""CREATE TABLE {table} ( + key integer PRIMARY KEY, + x integer, + y integer GENERATED ALWAYS AS (x*2) STORED, + z text); + """, + ) + + pg_manager.execute(f"insert into {table} (key, x, z) values (1,1,'1');") + pg_manager.execute(f"insert into {table} (key, x, z) values (2,2,'2');") + + pg_manager.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_manager.get_default_database() + ) + + pg_manager.execute(f"insert into {table} (key, x, z) values (3,3,'3');") + pg_manager.execute(f"insert into {table} (key, x, z) values (4,4,'4');") + + check_tables_are_synchronized( + instance, table, postgres_database=pg_manager.get_default_database() + ) + + pg_manager.execute(f"insert into {table} (key, x, z) values (5,5,'5');") + pg_manager.execute(f"insert into {table} (key, x, z) values (6,6,'6');") + + check_tables_are_synchronized( + instance, table, postgres_database=pg_manager.get_default_database() + ) + + +def test_default_columns(started_cluster): + table = "test_default_columns" + + pg_manager.create_postgres_table( + table, + "", + f"""CREATE TABLE {table} ( + key integer PRIMARY KEY, + x integer, + y text DEFAULT 'y1', + z integer, + a text DEFAULT 'a1', + b integer); + """, + ) + + pg_manager.execute(f"insert into {table} (key, x, z, b) values (1,1,1,1);") + pg_manager.execute(f"insert into {table} (key, x, z, b) values (2,2,2,2);") + + pg_manager.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_manager.get_default_database() + ) + + pg_manager.execute(f"insert into {table} (key, x, z, b) values (3,3,3,3);") + pg_manager.execute(f"insert into {table} (key, x, z, b) values (4,4,4,4);") + + check_tables_are_synchronized( + instance, table, postgres_database=pg_manager.get_default_database() + ) + + pg_manager.execute(f"insert into {table} (key, x, z, b) values (5,5,5,5);") + pg_manager.execute(f"insert into {table} (key, x, z, b) values (6,6,6,6);") + + check_tables_are_synchronized( + instance, table, postgres_database=pg_manager.get_default_database() + ) + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From 9d8e3f8bd913aa4e7160ebc7c8045a41550cd4fe Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 6 Dec 2023 16:47:58 +0100 Subject: [PATCH 119/331] Update fetchPostgreSQLTableStructure.cpp --- src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 527936f1c19..a3ae864db85 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -25,6 +25,7 @@ namespace ErrorCodes { extern const int UNKNOWN_TABLE; extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; } From 6321576ff8dc32cd97508f989e169f65527a3d13 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 6 Dec 2023 20:23:09 +0100 Subject: [PATCH 120/331] 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 121/331] 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 b57340bde214855c9e04f77483bdf571d122d822 Mon Sep 17 00:00:00 2001 From: Ryan Jacobs Date: Wed, 6 Dec 2023 11:54:45 -0800 Subject: [PATCH 122/331] postgresql integration: Throw errors instead of assuming array_ndim == 1 --- .../fetchPostgreSQLTableStructure.cpp | 60 ++++++++++--------- 1 file changed, 31 insertions(+), 29 deletions(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 0a35bc8c2b5..6c25514418e 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -222,40 +222,42 @@ PostgreSQLTableStructure::ColumnsInfoPtr readNamesAndTypesList( { const auto & name_and_type = columns[i]; - /// NOTE: If the relation is empty, then array_ndims returns NULL. - /// If this is the case, then assume dimensions=1. This covers most - /// use cases, but will be incorrect for empty tables with - /// multi-dimension arrays. The other solutions would be to drop - /// support for empty tables OR attempt fallback to a discovered - /// array_ndims CHECK constraint. - int dimensions; + /// If the relation is empty, then array_ndims returns NULL. + /// ClickHouse cannot support this use case. if (isTableEmpty(tx, postgres_table)) - { - dimensions = 1; - } - else - { - /// All rows must contain the same number of dimensions. - /// 1 is ok. If number of dimensions in all rows is not the same - - /// such arrays are not able to be used as ClickHouse Array at all. - /// - /// Assume dimensions=1 for empty arrays. - auto postgres_column = doubleQuoteString(name_and_type.name); - pqxx::result result{tx.exec(fmt::format( - "SELECT {} IS NULL, COALESCE(array_ndims({}), 1) " - "FROM {} LIMIT 1;", - postgres_column, - postgres_column, - postgres_table))}; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "PostgreSQL relation containing arrays cannot be empty: {}", postgres_table); - /// Nullable(Array) is not supported. - auto is_null = result[0][0].as(); - if (is_null) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "PostgreSQL array cannot be NULL. Column: {}", postgres_column); + /// All rows must contain the same number of dimensions. + /// 1 is ok. If number of dimensions in all rows is not the same - + /// such arrays are not able to be used as ClickHouse Array at all. + /// + /// For empty arrays, array_ndims([]) will return NULL. + auto postgres_column = doubleQuoteString(name_and_type.name); + pqxx::result result{tx.exec(fmt::format( + "SELECT {} IS NULL, array_ndims({}) " + "FROM {} LIMIT 1;", + postgres_column, + postgres_column, + postgres_table))}; - dimensions = result[0][1].as(); + /// Nullable(Array) is not supported. + auto is_null_array = result[0][0].as(); + if (is_null_array) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "PostgreSQL array cannot be NULL: {}.{}", postgres_table, postgres_column); + + /// Cannot infer dimension of empty arrays. + auto is_empty_array = result[0][1].is_null(); + if (is_empty_array) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "PostgreSQL cannot infer dimensions of an empty array: {}.{}", + postgres_table, + postgres_column); } + int dimensions = result[0][1].as(); + /// It is always 1d array if it is in recheck. DataTypePtr type = assert_cast(name_and_type.type.get())->getNestedType(); while (dimensions--) From 119c2864a07e7ef83a68add87020bbfad869a237 Mon Sep 17 00:00:00 2001 From: Ryan Jacobs Date: Wed, 6 Dec 2023 12:59:28 -0800 Subject: [PATCH 123/331] test_storage_postgresql: mixed-case identifier on array column --- .../test_storage_postgresql/test.py | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 11729a5ab18..39896c57b59 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -90,20 +90,20 @@ def test_postgres_conversions(started_cluster): cursor.execute( """CREATE TABLE test_types ( a smallint, b integer, c bigint, d real, e double precision, f serial, g bigserial, - h timestamp, i date, j decimal(5, 3), k numeric, l boolean)""" + h timestamp, i date, j decimal(5, 3), k numeric, l boolean, "M" integer)""" ) node1.query( """ INSERT INTO TABLE FUNCTION postgresql('postgres1:5432', 'postgres', 'test_types', 'postgres', 'mysecretpassword') VALUES - (-32768, -2147483648, -9223372036854775808, 1.12345, 1.1234567890, 2147483647, 9223372036854775807, '2000-05-12 12:12:12.012345', '2000-05-12', 22.222, 22.222, 1)""" + (-32768, -2147483648, -9223372036854775808, 1.12345, 1.1234567890, 2147483647, 9223372036854775807, '2000-05-12 12:12:12.012345', '2000-05-12', 22.222, 22.222, 1, 42)""" ) result = node1.query( """ - SELECT a, b, c, d, e, f, g, h, i, j, toDecimal128(k, 3), l FROM postgresql('postgres1:5432', 'postgres', 'test_types', 'postgres', 'mysecretpassword')""" + SELECT a, b, c, d, e, f, g, h, i, j, toDecimal128(k, 3), l, "M" FROM postgresql('postgres1:5432', 'postgres', 'test_types', 'postgres', 'mysecretpassword')""" ) assert ( result - == "-32768\t-2147483648\t-9223372036854775808\t1.12345\t1.123456789\t2147483647\t9223372036854775807\t2000-05-12 12:12:12.012345\t2000-05-12\t22.222\t22.222\t1\n" + == "-32768\t-2147483648\t-9223372036854775808\t1.12345\t1.123456789\t2147483647\t9223372036854775807\t2000-05-12 12:12:12.012345\t2000-05-12\t22.222\t22.222\t1\t42\n" ) cursor.execute( @@ -132,7 +132,8 @@ def test_postgres_conversions(started_cluster): i Char(2)[][][][], -- Nullable(String) j Char(2)[], -- Nullable(String) k UUID[], -- Nullable(UUID) - l UUID[][] -- Nullable(UUID) + l UUID[][], -- Nullable(UUID) + "M" integer[] NOT NULL -- Int32 (mixed-case identifier) )""" ) @@ -152,7 +153,8 @@ def test_postgres_conversions(started_cluster): "i\tArray(Array(Array(Array(Nullable(String)))))\t\t\t\t\t\n" "j\tArray(Nullable(String))\t\t\t\t\t\n" "k\tArray(Nullable(UUID))\t\t\t\t\t\n" - "l\tArray(Array(Nullable(UUID)))" + "l\tArray(Array(Nullable(UUID)))\t\t\t\t\t\n" + "M\tArray(Int32)" "" ) assert result.rstrip() == expected @@ -171,7 +173,8 @@ def test_postgres_conversions(started_cluster): "[[[[NULL]]]], " "[], " "['2a0c0bfc-4fec-4e32-ae3a-7fc8eea6626a', '42209d53-d641-4d73-a8b6-c038db1e75d6', NULL], " - "[[NULL, '42209d53-d641-4d73-a8b6-c038db1e75d6'], ['2a0c0bfc-4fec-4e32-ae3a-7fc8eea6626a', NULL], [NULL, NULL]]" + "[[NULL, '42209d53-d641-4d73-a8b6-c038db1e75d6'], ['2a0c0bfc-4fec-4e32-ae3a-7fc8eea6626a', NULL], [NULL, NULL]]," + "[42, 42, 42]" ")" ) @@ -191,7 +194,8 @@ def test_postgres_conversions(started_cluster): "[[[[NULL]]]]\t" "[]\t" "['2a0c0bfc-4fec-4e32-ae3a-7fc8eea6626a','42209d53-d641-4d73-a8b6-c038db1e75d6',NULL]\t" - "[[NULL,'42209d53-d641-4d73-a8b6-c038db1e75d6'],['2a0c0bfc-4fec-4e32-ae3a-7fc8eea6626a',NULL],[NULL,NULL]]\n" + "[[NULL,'42209d53-d641-4d73-a8b6-c038db1e75d6'],['2a0c0bfc-4fec-4e32-ae3a-7fc8eea6626a',NULL],[NULL,NULL]]\t" + "[42,42,42]\n" ) assert result == expected From b44dadc5c6cd7671e39cee69d139730a1c3fea62 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 7 Dec 2023 00:45:57 +0100 Subject: [PATCH 124/331] Identify failed jobs in lambda and mark as steps=0 --- tests/ci/workflow_jobs_lambda/app.py | 28 ++++++++++++++++++++++++---- 1 file changed, 24 insertions(+), 4 deletions(-) diff --git a/tests/ci/workflow_jobs_lambda/app.py b/tests/ci/workflow_jobs_lambda/app.py index c624a492604..6931835f601 100644 --- a/tests/ci/workflow_jobs_lambda/app.py +++ b/tests/ci/workflow_jobs_lambda/app.py @@ -8,11 +8,11 @@ Then it either posts it as is to the play.clickhouse.com, or anonymizes the sens fields for private repositories """ +import json +import logging from base64 import b64decode from dataclasses import dataclass from typing import Any, List, Optional -import json -import logging from lambda_shared import ClickHouseHelper, InsertException, get_parameter_from_ssm @@ -126,6 +126,20 @@ def send_event_workflow_job(workflow_job: WorkflowJob) -> None: ) +def killed_job(wf_job: dict) -> bool: + """a hack to identify the killed runner if "Complete job" is omit""" + if ( + wf_job.get("status", "") != "completed" + or wf_job.get("conclusion", "") != "failure" + ): + # The task either success or in progress + return False + return not any( + step["name"] == "Complete job" and step["conclusion"] is not None + for step in wf_job["steps"] + ) + + def handler(event: dict, context: Any) -> dict: if event["isBase64Encoded"]: event_data = json.loads(b64decode(event["body"])) @@ -141,8 +155,14 @@ def handler(event: dict, context: Any) -> dict: logging.error("The event data: %s", event) logging.error("The context data: %s", context) - # We record only finished steps - steps = len([step for step in wf_job["steps"] if step["conclusion"] is not None]) + if killed_job(wf_job): + # for killed job we record 0 + steps = 0 + else: + # We record only finished steps + steps = len( + [step for step in wf_job["steps"] if step["conclusion"] is not None] + ) workflow_job = WorkflowJob( wf_job["id"], From 7d4142693513b0cccfedc0d1398e849e53f36107 Mon Sep 17 00:00:00 2001 From: Ryan Jacobs Date: Wed, 6 Dec 2023 18:25:45 -0800 Subject: [PATCH 125/331] test_storage_postgresql: Add test_postgres_array_ndim_error_messges() Tests: 1. View with array column cannot be empty --> error message 2. View cannot have empty array --> error message 3. View cannot have NULL array value --> error message 4. Ensures PG identifiers that require quoting do not crash ClickHouse. These apply to views that contain arrays. --- .../test_storage_postgresql/test.py | 47 +++++++++++++++++++ 1 file changed, 47 insertions(+) diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 39896c57b59..8e1be600687 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -202,6 +202,53 @@ def test_postgres_conversions(started_cluster): cursor.execute(f"DROP TABLE test_types") cursor.execute(f"DROP TABLE test_array_dimensions") +def test_postgres_array_ndim_error_messges(started_cluster): + cursor = started_cluster.postgres_conn.cursor() + + # cleanup + cursor.execute('DROP VIEW IF EXISTS array_ndim_view;') + cursor.execute('DROP TABLE IF EXISTS array_ndim_table;') + + # setup + cursor.execute('CREATE TABLE array_ndim_table (x INTEGER, "Mixed-case with spaces" INTEGER[]);') + cursor.execute('CREATE VIEW array_ndim_view AS SELECT * FROM array_ndim_table;') + describe_table = """ + DESCRIBE TABLE postgresql( + 'postgres1:5432', 'postgres', 'array_ndim_view', + 'postgres', 'mysecretpassword' + ) + """ + + # View with array column cannot be empty. Should throw a useful error message. + # (Cannot infer array dimension.) + try: + node1.query(describe_table) + assert False + except Exception as error: + assert ('PostgreSQL relation containing arrays cannot be empty: array_ndim_view' in str(error)) + + # View cannot have empty array. Should throw useful error message. + # (Cannot infer array dimension.) + cursor.execute('TRUNCATE array_ndim_table;') + cursor.execute("INSERT INTO array_ndim_table VALUES (1234, '{}');") + try: + node1.query(describe_table) + assert False + except Exception as error: + assert ('PostgreSQL cannot infer dimensions of an empty array: array_ndim_view."Mixed-case with spaces"' in str(error)) + + # View cannot have NULL array value. Should throw useful error message. + cursor.execute('TRUNCATE array_ndim_table;') + cursor.execute('INSERT INTO array_ndim_table VALUES (1234, NULL);') + try: + node1.query(describe_table) + assert False + except Exception as error: + assert ('PostgreSQL array cannot be NULL: array_ndim_view."Mixed-case with spaces"' in str(error)) + + # cleanup + cursor.execute('DROP VIEW IF EXISTS array_ndim_view;') + cursor.execute('DROP TABLE IF EXISTS array_ndim_table;') def test_non_default_schema(started_cluster): node1.query("DROP TABLE IF EXISTS test_pg_table_schema") From 6a698d94804c2fd165c29ea168ec64383e333d3e Mon Sep 17 00:00:00 2001 From: Ryan Jacobs Date: Wed, 6 Dec 2023 18:56:54 -0800 Subject: [PATCH 126/331] clang-tidy --- .../PostgreSQL/fetchPostgreSQLTableStructure.cpp | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 6c25514418e..d4f9bb6dcf4 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -233,12 +233,8 @@ PostgreSQLTableStructure::ColumnsInfoPtr readNamesAndTypesList( /// /// For empty arrays, array_ndims([]) will return NULL. auto postgres_column = doubleQuoteString(name_and_type.name); - pqxx::result result{tx.exec(fmt::format( - "SELECT {} IS NULL, array_ndims({}) " - "FROM {} LIMIT 1;", - postgres_column, - postgres_column, - postgres_table))}; + pqxx::result result{tx.exec( + fmt::format("SELECT {} IS NULL, array_ndims({}) FROM {} LIMIT 1;", postgres_column, postgres_column, postgres_table))}; /// Nullable(Array) is not supported. auto is_null_array = result[0][0].as(); From d0675488acf65d2391c9dbcff8f58ade93c73384 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Tue, 5 Dec 2023 16:16:11 +0800 Subject: [PATCH 127/331] rebase --- src/Databases/DatabaseLazy.cpp | 3 +- src/Databases/IDatabase.cpp | 3 +- src/Interpreters/Context.cpp | 1042 +++++++---------- .../0_stateless/02931_max_num_to_warn.sql | 58 +- 4 files changed, 432 insertions(+), 674 deletions(-) diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index caf14aa9b15..c6249c68933 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -19,7 +19,8 @@ namespace fs = std::filesystem; -namespace CurrentMetrics { +namespace CurrentMetrics +{ extern const Metric AttachedTable; } diff --git a/src/Databases/IDatabase.cpp b/src/Databases/IDatabase.cpp index b4fd5ea4612..95fcf0c7939 100644 --- a/src/Databases/IDatabase.cpp +++ b/src/Databases/IDatabase.cpp @@ -35,7 +35,8 @@ StoragePtr IDatabase::getTable(const String & name, ContextPtr context) const throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} does not exist. Maybe you meant {}?", backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(name), backQuoteIfNeed(names[0])); } -IDatabase::IDatabase(String database_name_) : database_name(std::move(database_name_)) { +IDatabase::IDatabase(String database_name_) : database_name(std::move(database_name_)) +{ CurrentMetrics::add(CurrentMetrics::AttachedDatabase, 1); } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index dd1f5c76370..1e732083c9d 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1,5 +1,6 @@ -#include #include +#include +#include #include #include #include @@ -51,119 +52,74 @@ #include #include #include -#include -#include #include -#include +#include #include #include -#include +#include +#include +#include +#include #include -#include -#include -#include -#include -#include -#include #include -#include -#include -#include -#include -#include +#include +#include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include +#include +#include +#include +#include +#include +#include #include -#include -#include -#include #include -#include -#include +#include +#include +#include +#include #include -#include +#include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include #include -#include -#include -#include +#include #include -#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include namespace fs = std::filesystem; namespace ProfileEvents { -extern const Event ContextLock; -extern const Event ContextLockWaitMicroseconds; + extern const Event ContextLock; + extern const Event ContextLockWaitMicroseconds; } namespace CurrentMetrics { - extern const Metric AttachedTable; - extern const Metric AttachedDatabase; - extern const Metric PartsActive; extern const Metric ContextLockWait; extern const Metric BackgroundMovePoolTask; extern const Metric BackgroundMovePoolSize; @@ -196,6 +152,9 @@ namespace CurrentMetrics extern const Metric TablesLoaderForegroundThreadsActive; extern const Metric TablesLoaderForegroundThreadsScheduled; extern const Metric IOWriterThreadsScheduled; + extern const Metric AttachedTable; + extern const Metric AttachedDatabase; + extern const Metric PartsActive; } @@ -204,33 +163,32 @@ namespace DB namespace ErrorCodes { -extern const int BAD_ARGUMENTS; -extern const int UNKNOWN_DATABASE; -extern const int UNKNOWN_TABLE; -extern const int TABLE_ALREADY_EXISTS; -extern const int THERE_IS_NO_SESSION; -extern const int THERE_IS_NO_QUERY; -extern const int NO_ELEMENTS_IN_CONFIG; -extern const int TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT; -extern const int LOGICAL_ERROR; -extern const int INVALID_SETTING_VALUE; -extern const int UNKNOWN_READ_METHOD; -extern const int NOT_IMPLEMENTED; -extern const int UNKNOWN_FUNCTION; -extern const int ILLEGAL_COLUMN; -extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; -extern const int CLUSTER_DOESNT_EXIST; + extern const int BAD_ARGUMENTS; + extern const int UNKNOWN_DATABASE; + extern const int UNKNOWN_TABLE; + extern const int TABLE_ALREADY_EXISTS; + extern const int THERE_IS_NO_SESSION; + extern const int THERE_IS_NO_QUERY; + extern const int NO_ELEMENTS_IN_CONFIG; + extern const int TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT; + extern const int LOGICAL_ERROR; + extern const int INVALID_SETTING_VALUE; + extern const int UNKNOWN_READ_METHOD; + extern const int NOT_IMPLEMENTED; + extern const int UNKNOWN_FUNCTION; + extern const int ILLEGAL_COLUMN; + extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; + extern const int CLUSTER_DOESNT_EXIST; } -#define SHUTDOWN(log, desc, ptr, method) \ - do \ - { \ - if (ptr) \ - { \ - LOG_DEBUG(log, "Shutting down " desc); \ - (ptr)->method; \ - } \ - } while (false) +#define SHUTDOWN(log, desc, ptr, method) do \ +{ \ + if (ptr) \ + { \ + LOG_DEBUG(log, "Shutting down " desc); \ + (ptr)->method; \ + } \ +} while (false) \ /** Set of known objects (environment), that could be used in query. * Shared (global) part. Order of members (especially, order of destruction) is very important. @@ -252,8 +210,8 @@ struct ContextSharedPart : boost::noncopyable /// Separate mutex for re-initialization of zookeeper session. This operation could take a long time and must not interfere with another operations. mutable std::mutex zookeeper_mutex; - mutable zkutil::ZooKeeperPtr zookeeper TSA_GUARDED_BY(zookeeper_mutex); /// Client for ZooKeeper. - ConfigurationPtr zookeeper_config TSA_GUARDED_BY(zookeeper_mutex); /// Stores zookeeper configs + mutable zkutil::ZooKeeperPtr zookeeper TSA_GUARDED_BY(zookeeper_mutex); /// Client for ZooKeeper. + ConfigurationPtr zookeeper_config TSA_GUARDED_BY(zookeeper_mutex); /// Stores zookeeper configs ConfigurationPtr sensitive_data_masker_config; @@ -262,25 +220,23 @@ struct ContextSharedPart : boost::noncopyable mutable std::shared_ptr keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_mutex); #endif mutable std::mutex auxiliary_zookeepers_mutex; - mutable std::map - auxiliary_zookeepers TSA_GUARDED_BY(auxiliary_zookeepers_mutex); /// Map for auxiliary ZooKeeper clients. - ConfigurationPtr auxiliary_zookeepers_config TSA_GUARDED_BY(auxiliary_zookeepers_mutex); /// Stores auxiliary zookeepers configs + mutable std::map auxiliary_zookeepers TSA_GUARDED_BY(auxiliary_zookeepers_mutex); /// Map for auxiliary ZooKeeper clients. + ConfigurationPtr auxiliary_zookeepers_config TSA_GUARDED_BY(auxiliary_zookeepers_mutex); /// Stores auxiliary zookeepers configs /// No lock required for interserver_io_host, interserver_io_port, interserver_scheme modified only during initialization - String interserver_io_host; /// The host name by which this server is available for other servers. - UInt16 interserver_io_port = 0; /// and port. - String interserver_scheme; /// http or https + String interserver_io_host; /// The host name by which this server is available for other servers. + UInt16 interserver_io_port = 0; /// and port. + String interserver_scheme; /// http or https MultiVersion interserver_io_credentials; - String path TSA_GUARDED_BY(mutex); /// Path to the data directory, with a slash at the end. - String flags_path TSA_GUARDED_BY(mutex); /// Path to the directory with some control flags for server maintenance. - String user_files_path TSA_GUARDED_BY(mutex); /// Path to the directory with user provided files, usable by 'file' table function. - String dictionaries_lib_path - TSA_GUARDED_BY(mutex); /// Path to the directory with user provided binaries and libraries for external dictionaries. - String user_scripts_path TSA_GUARDED_BY(mutex); /// Path to the directory with user provided scripts. - String filesystem_caches_path TSA_GUARDED_BY(mutex); /// Path to the directory with filesystem caches. - ConfigurationPtr config TSA_GUARDED_BY(mutex); /// Global configuration settings. - String tmp_path TSA_GUARDED_BY(mutex); /// Path to the temporary files that occur when processing the request. + String path TSA_GUARDED_BY(mutex); /// Path to the data directory, with a slash at the end. + String flags_path TSA_GUARDED_BY(mutex); /// Path to the directory with some control flags for server maintenance. + String user_files_path TSA_GUARDED_BY(mutex); /// Path to the directory with user provided files, usable by 'file' table function. + String dictionaries_lib_path TSA_GUARDED_BY(mutex); /// Path to the directory with user provided binaries and libraries for external dictionaries. + String user_scripts_path TSA_GUARDED_BY(mutex); /// Path to the directory with user provided scripts. + String filesystem_caches_path TSA_GUARDED_BY(mutex); /// Path to the directory with filesystem caches. + ConfigurationPtr config TSA_GUARDED_BY(mutex); /// Global configuration settings. + String tmp_path TSA_GUARDED_BY(mutex); /// Path to the temporary files that occur when processing the request. /// All temporary files that occur when processing the requests accounted here. /// Child scopes for more fine-grained accounting are created per user/query/etc. @@ -296,11 +252,8 @@ struct ContextSharedPart : boost::noncopyable ExternalLoaderXMLConfigRepository * external_dictionaries_config_repository TSA_GUARDED_BY(external_dictionaries_mutex) = nullptr; scope_guard dictionaries_xmls TSA_GUARDED_BY(external_dictionaries_mutex); - mutable std::unique_ptr - external_user_defined_executable_functions_loader TSA_GUARDED_BY(external_user_defined_executable_functions_mutex); - ExternalLoaderXMLConfigRepository * - user_defined_executable_functions_config_repository TSA_GUARDED_BY(external_user_defined_executable_functions_mutex) - = nullptr; + mutable std::unique_ptr external_user_defined_executable_functions_loader TSA_GUARDED_BY(external_user_defined_executable_functions_mutex); + ExternalLoaderXMLConfigRepository * user_defined_executable_functions_config_repository TSA_GUARDED_BY(external_user_defined_executable_functions_mutex) = nullptr; scope_guard user_defined_executable_functions_xmls TSA_GUARDED_BY(external_user_defined_executable_functions_mutex); mutable OnceFlag user_defined_sql_objects_loader_initialized; @@ -318,44 +271,39 @@ struct ContextSharedPart : boost::noncopyable std::optional backups_worker; /// No lock required for default_profile_name, system_profile_name, buffer_profile_name modified only during initialization - String default_profile_name; /// Default profile name used for default values. - String system_profile_name; /// Profile used by system processes - String buffer_profile_name; /// Profile used by Buffer engine for flushing to the underlying + String default_profile_name; /// Default profile name used for default values. + String system_profile_name; /// Profile used by system processes + String buffer_profile_name; /// Profile used by Buffer engine for flushing to the underlying std::unique_ptr access_control TSA_GUARDED_BY(mutex); mutable OnceFlag resource_manager_initialized; mutable ResourceManagerPtr resource_manager; - mutable UncompressedCachePtr uncompressed_cache TSA_GUARDED_BY(mutex); /// The cache of decompressed blocks. - mutable MarkCachePtr mark_cache TSA_GUARDED_BY(mutex); /// Cache of marks in compressed files. + mutable UncompressedCachePtr uncompressed_cache TSA_GUARDED_BY(mutex); /// The cache of decompressed blocks. + mutable MarkCachePtr mark_cache TSA_GUARDED_BY(mutex); /// Cache of marks in compressed files. mutable OnceFlag load_marks_threadpool_initialized; - mutable std::unique_ptr load_marks_threadpool; /// Threadpool for loading marks cache. + mutable std::unique_ptr load_marks_threadpool; /// Threadpool for loading marks cache. mutable OnceFlag prefetch_threadpool_initialized; - mutable std::unique_ptr prefetch_threadpool; /// Threadpool for loading marks cache. - mutable UncompressedCachePtr index_uncompressed_cache TSA_GUARDED_BY(mutex); /// The cache of decompressed blocks for MergeTree indices. - mutable QueryCachePtr query_cache TSA_GUARDED_BY(mutex); /// Cache of query results. - mutable MarkCachePtr index_mark_cache TSA_GUARDED_BY(mutex); /// Cache of marks in compressed files of MergeTree indices. - mutable MMappedFileCachePtr mmap_cache - TSA_GUARDED_BY(mutex); /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads. - ProcessList process_list; /// Executing queries at the moment. + mutable std::unique_ptr prefetch_threadpool; /// Threadpool for loading marks cache. + mutable UncompressedCachePtr index_uncompressed_cache TSA_GUARDED_BY(mutex); /// The cache of decompressed blocks for MergeTree indices. + mutable QueryCachePtr query_cache TSA_GUARDED_BY(mutex); /// Cache of query results. + mutable MarkCachePtr index_mark_cache TSA_GUARDED_BY(mutex); /// Cache of marks in compressed files of MergeTree indices. + mutable MMappedFileCachePtr mmap_cache TSA_GUARDED_BY(mutex); /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads. + ProcessList process_list; /// Executing queries at the moment. SessionTracker session_tracker; GlobalOvercommitTracker global_overcommit_tracker; - MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree) - MovesList moves_list; /// The list of executing moves (for (Replicated)?MergeTree) + MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree) + MovesList moves_list; /// The list of executing moves (for (Replicated)?MergeTree) ReplicatedFetchList replicated_fetch_list; - ConfigurationPtr users_config TSA_GUARDED_BY(mutex); /// Config with the users, profiles and quotas sections. - InterserverIOHandler interserver_io_handler; /// Handler for interserver communication. + ConfigurationPtr users_config TSA_GUARDED_BY(mutex); /// Config with the users, profiles and quotas sections. + InterserverIOHandler interserver_io_handler; /// Handler for interserver communication. OnceFlag buffer_flush_schedule_pool_initialized; - mutable std::unique_ptr - buffer_flush_schedule_pool; /// A thread pool that can do background flush for Buffer tables. + mutable std::unique_ptr buffer_flush_schedule_pool; /// A thread pool that can do background flush for Buffer tables. OnceFlag schedule_pool_initialized; - mutable std::unique_ptr - schedule_pool; /// A thread pool that can run different jobs in background (used in replicated tables) + mutable std::unique_ptr schedule_pool; /// A thread pool that can run different jobs in background (used in replicated tables) OnceFlag distributed_schedule_pool_initialized; - mutable std::unique_ptr - distributed_schedule_pool; /// A thread pool that can run different jobs in background (used for distributed sends) + mutable std::unique_ptr distributed_schedule_pool; /// A thread pool that can run different jobs in background (used for distributed sends) OnceFlag message_broker_schedule_pool_initialized; - mutable std::unique_ptr - message_broker_schedule_pool; /// A thread pool that can run different jobs in background (used for message brokers, like RabbitMQ and Kafka) + mutable std::unique_ptr message_broker_schedule_pool; /// A thread pool that can run different jobs in background (used for message brokers, like RabbitMQ and Kafka) mutable OnceFlag readers_initialized; mutable std::unique_ptr asynchronous_remote_fs_reader; @@ -365,16 +313,16 @@ struct ContextSharedPart : boost::noncopyable mutable OnceFlag threadpool_writer_initialized; mutable std::unique_ptr threadpool_writer; - mutable ThrottlerPtr replicated_fetches_throttler; /// A server-wide throttler for replicated fetches - mutable ThrottlerPtr replicated_sends_throttler; /// A server-wide throttler for replicated sends + mutable ThrottlerPtr replicated_fetches_throttler; /// A server-wide throttler for replicated fetches + mutable ThrottlerPtr replicated_sends_throttler; /// A server-wide throttler for replicated sends - mutable ThrottlerPtr remote_read_throttler; /// A server-wide throttler for remote IO reads - mutable ThrottlerPtr remote_write_throttler; /// A server-wide throttler for remote IO writes + mutable ThrottlerPtr remote_read_throttler; /// A server-wide throttler for remote IO reads + mutable ThrottlerPtr remote_write_throttler; /// A server-wide throttler for remote IO writes - mutable ThrottlerPtr local_read_throttler; /// A server-wide throttler for local IO reads - mutable ThrottlerPtr local_write_throttler; /// A server-wide throttler for local IO writes + mutable ThrottlerPtr local_read_throttler; /// A server-wide throttler for local IO reads + mutable ThrottlerPtr local_write_throttler; /// A server-wide throttler for local IO writes - mutable ThrottlerPtr backups_server_throttler; /// A server-wide throttler for BACKUPs + mutable ThrottlerPtr backups_server_throttler; /// A server-wide throttler for BACKUPs MultiVersion macros; /// Substitutions extracted from config. std::unique_ptr ddl_worker TSA_GUARDED_BY(mutex); /// Process ddl commands from zk. @@ -388,24 +336,24 @@ struct ContextSharedPart : boost::noncopyable ServerSettings server_settings; - std::optional merge_tree_settings TSA_GUARDED_BY(mutex); /// Settings of MergeTree* engines. - std::optional replicated_merge_tree_settings TSA_GUARDED_BY(mutex); /// Settings of ReplicatedMergeTree* engines. + std::optional merge_tree_settings TSA_GUARDED_BY(mutex); /// Settings of MergeTree* engines. + std::optional replicated_merge_tree_settings TSA_GUARDED_BY(mutex); /// Settings of ReplicatedMergeTree* engines. std::atomic_size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default) std::unordered_set get_client_http_header_forbidden_headers; bool allow_get_client_http_header; std::atomic_size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default) + /// No lock required for format_schema_path modified only during initialization std::atomic_size_t max_database_num_to_warn = 1000lu; std::atomic_size_t max_table_num_to_warn = 5000lu; std::atomic_size_t max_part_num_to_warn = 100000lu; - /// No lock required for format_schema_path modified only during initialization - String format_schema_path; /// Path to a directory that contains schema files used by input formats. + String format_schema_path; /// Path to a directory that contains schema files used by input formats. String google_protos_path; /// Path to a directory that contains the proto files for the well-known Protobuf types. mutable OnceFlag action_locks_manager_initialized; - ActionLocksManagerPtr action_locks_manager; /// Set of storages' action lockers + ActionLocksManagerPtr action_locks_manager; /// Set of storages' action lockers OnceFlag system_logs_initialized; - std::unique_ptr system_logs TSA_GUARDED_BY(mutex); /// Used to log queries and operations on parts - std::optional storage_s3_settings TSA_GUARDED_BY(mutex); /// Settings of S3 storage - std::vector warnings TSA_GUARDED_BY(mutex); /// Store warning messages about server configuration. + std::unique_ptr system_logs TSA_GUARDED_BY(mutex); /// Used to log queries and operations on parts + std::optional storage_s3_settings TSA_GUARDED_BY(mutex); /// Settings of S3 storage + std::vector warnings TSA_GUARDED_BY(mutex); /// Store warning messages about server configuration. /// Background executors for *MergeTree tables /// Has background executors for MergeTree tables been initialized? @@ -416,17 +364,17 @@ struct ContextSharedPart : boost::noncopyable OrdinaryBackgroundExecutorPtr fetch_executor TSA_GUARDED_BY(background_executors_mutex); OrdinaryBackgroundExecutorPtr common_executor TSA_GUARDED_BY(background_executors_mutex); - RemoteHostFilter remote_host_filter TSA_GUARDED_BY(mutex); /// Allowed URL from config.xml - HTTPHeaderFilter http_header_filter TSA_GUARDED_BY(mutex); /// Forbidden HTTP headers from config.xml + RemoteHostFilter remote_host_filter TSA_GUARDED_BY(mutex); /// Allowed URL from config.xml + HTTPHeaderFilter http_header_filter TSA_GUARDED_BY(mutex); /// Forbidden HTTP headers from config.xml /// No lock required for trace_collector modified only during initialization - std::optional trace_collector; /// Thread collecting traces from threads executing queries + std::optional trace_collector; /// Thread collecting traces from threads executing queries /// Clusters for distributed tables /// Initialized on demand (on distributed storages initialization) since Settings should be initialized - mutable std::mutex clusters_mutex; /// Guards clusters, clusters_config and cluster_discovery + mutable std::mutex clusters_mutex; /// Guards clusters, clusters_config and cluster_discovery std::shared_ptr clusters TSA_GUARDED_BY(clusters_mutex); - ConfigurationPtr clusters_config TSA_GUARDED_BY(clusters_mutex); /// Stores updated configs + ConfigurationPtr clusters_config TSA_GUARDED_BY(clusters_mutex); /// Stores updated configs std::unique_ptr cluster_discovery TSA_GUARDED_BY(clusters_mutex); /// No lock required for async_insert_queue modified only during initialization @@ -452,7 +400,9 @@ struct ContextSharedPart : boost::noncopyable bool is_server_completely_started TSA_GUARDED_BY(mutex) = false; ContextSharedPart() - : access_control(std::make_unique()), global_overcommit_tracker(&process_list), macros(std::make_unique()) + : access_control(std::make_unique()) + , global_overcommit_tracker(&process_list) + , macros(std::make_unique()) { /// TODO: make it singleton (?) static std::atomic num_calls{0}; @@ -573,8 +523,7 @@ struct ContextSharedPart : boost::noncopyable access_control->setExternalAuthenticatorsConfig(*config_value); } - const Poco::Util::AbstractConfiguration & getConfigRefWithLock(const std::lock_guard &) const - TSA_REQUIRES(this->mutex) + const Poco::Util::AbstractConfiguration & getConfigRefWithLock(const std::lock_guard &) const TSA_REQUIRES(this->mutex) { return config ? *config : Poco::Util::Application::instance().config(); } @@ -738,7 +687,10 @@ struct ContextSharedPart : boost::noncopyable total_memory_tracker.resetOvercommitTracker(); } - bool hasTraceCollector() const { return trace_collector.has_value(); } + bool hasTraceCollector() const + { + return trace_collector.has_value(); + } void initializeTraceCollector(std::shared_ptr trace_log) { @@ -804,22 +756,16 @@ ContextData::ContextData() = default; ContextData::ContextData(const ContextData &) = default; Context::Context() = default; -Context::Context(const Context & rhs) : ContextData(rhs), std::enable_shared_from_this(rhs) -{ -} +Context::Context(const Context & rhs) : ContextData(rhs), std::enable_shared_from_this(rhs) {} SharedContextHolder::SharedContextHolder(SharedContextHolder &&) noexcept = default; SharedContextHolder & SharedContextHolder::operator=(SharedContextHolder &&) noexcept = default; SharedContextHolder::SharedContextHolder() = default; SharedContextHolder::~SharedContextHolder() = default; -SharedContextHolder::SharedContextHolder(std::unique_ptr shared_context) : shared(std::move(shared_context)) -{ -} +SharedContextHolder::SharedContextHolder(std::unique_ptr shared_context) + : shared(std::move(shared_context)) {} -void SharedContextHolder::reset() -{ - shared.reset(); -} +void SharedContextHolder::reset() { shared.reset(); } ContextMutablePtr Context::createGlobal(ContextSharedPart * shared_part) { @@ -862,57 +808,21 @@ ContextMutablePtr Context::createCopy(const ContextMutablePtr & other) Context::~Context() = default; -InterserverIOHandler & Context::getInterserverIOHandler() -{ - return shared->interserver_io_handler; -} -const InterserverIOHandler & Context::getInterserverIOHandler() const -{ - return shared->interserver_io_handler; -} +InterserverIOHandler & Context::getInterserverIOHandler() { return shared->interserver_io_handler; } +const InterserverIOHandler & Context::getInterserverIOHandler() const { return shared->interserver_io_handler; } -ProcessList & Context::getProcessList() -{ - return shared->process_list; -} -const ProcessList & Context::getProcessList() const -{ - return shared->process_list; -} -OvercommitTracker * Context::getGlobalOvercommitTracker() const -{ - return &shared->global_overcommit_tracker; -} +ProcessList & Context::getProcessList() { return shared->process_list; } +const ProcessList & Context::getProcessList() const { return shared->process_list; } +OvercommitTracker * Context::getGlobalOvercommitTracker() const { return &shared->global_overcommit_tracker; } -SessionTracker & Context::getSessionTracker() -{ - return shared->session_tracker; -} +SessionTracker & Context::getSessionTracker() { return shared->session_tracker; } -MergeList & Context::getMergeList() -{ - return shared->merge_list; -} -const MergeList & Context::getMergeList() const -{ - return shared->merge_list; -} -MovesList & Context::getMovesList() -{ - return shared->moves_list; -} -const MovesList & Context::getMovesList() const -{ - return shared->moves_list; -} -ReplicatedFetchList & Context::getReplicatedFetchList() -{ - return shared->replicated_fetch_list; -} -const ReplicatedFetchList & Context::getReplicatedFetchList() const -{ - return shared->replicated_fetch_list; -} +MergeList & Context::getMergeList() { return shared->merge_list; } +const MergeList & Context::getMergeList() const { return shared->merge_list; } +MovesList & Context::getMovesList() { return shared->moves_list; } +const MovesList & Context::getMovesList() const { return shared->moves_list; } +ReplicatedFetchList & Context::getReplicatedFetchList() { return shared->replicated_fetch_list; } +const ReplicatedFetchList & Context::getReplicatedFetchList() const { return shared->replicated_fetch_list; } String Context::resolveDatabase(const String & database_name) const { @@ -964,21 +874,20 @@ Strings Context::getWarnings() const { SharedLockGuard lock(shared->mutex); common_warnings = shared->warnings; - if (CurrentMetrics::get(CurrentMetrics::AttachedTable) > static_cast(shared->max_table_num_to_warn)) common_warnings.emplace_back(fmt::format("The number of attached tables is more than {}", shared->max_table_num_to_warn)); - if (CurrentMetrics::get(CurrentMetrics::AttachedDatabase) > static_cast(shared->max_database_num_to_warn)) common_warnings.emplace_back(fmt::format("The number of attached databases is more than {}", shared->max_table_num_to_warn)); - if (CurrentMetrics::get(CurrentMetrics::PartsActive) > static_cast(shared->max_part_num_to_warn)) common_warnings.emplace_back(fmt::format("The number of active parts is more than {}", shared->max_part_num_to_warn)); } /// Make setting's name ordered std::set obsolete_settings; for (const auto & setting : settings) + { if (setting.isValueChanged() && setting.isObsolete()) obsolete_settings.emplace(setting.getName()); + } if (!obsolete_settings.empty()) { @@ -994,8 +903,7 @@ Strings Context::getWarnings() const } res = res + "]" + (single_element ? " is" : " are") + " changed. " - "Please check 'SELECT * FROM system.settings WHERE changed AND is_obsolete' and read the changelog at " - "https://github.com/ClickHouse/ClickHouse/blob/master/CHANGELOG.md"; + "Please check 'SELECT * FROM system.settings WHERE changed AND is_obsolete' and read the changelog at https://github.com/ClickHouse/ClickHouse/blob/master/CHANGELOG.md"; common_warnings.emplace_back(res); } @@ -1093,12 +1001,9 @@ try } catch (...) { - DB::tryLogCurrentException( - log, - fmt::format( - "Caught exception while setup temporary path: {}. " - "It is ok to skip this exception as cleaning old temporary files is not necessary", - path)); + DB::tryLogCurrentException(log, fmt::format( + "Caught exception while setup temporary path: {}. " + "It is ok to skip this exception as cleaning old temporary files is not necessary", path)); } static VolumePtr createLocalSingleDiskVolume(const std::string & path, const Poco::Util::AbstractConfiguration & config_) @@ -1122,7 +1027,9 @@ void Context::setTemporaryStoragePath(const String & path, size_t max_size) VolumePtr volume = createLocalSingleDiskVolume(shared->tmp_path, shared->getConfigRefWithLock(lock)); for (const auto & disk : volume->getDisks()) + { setupTmpPath(shared->log, disk->getPath()); + } shared->root_temp_data_on_disk = std::make_shared(volume, max_size); } @@ -1138,15 +1045,13 @@ void Context::setTemporaryStoragePolicy(const String & policy_name, size_t max_s } if (tmp_policy->getVolumes().size() != 1) - throw Exception( - ErrorCodes::NO_ELEMENTS_IN_CONFIG, - "Policy '{}' is used temporary files, such policy should have exactly one volume", - policy_name); + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, + "Policy '{}' is used temporary files, such policy should have exactly one volume", policy_name); VolumePtr volume = tmp_policy->getVolume(0); if (volume->getDisks().empty()) - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "No disks volume for temporary files"); + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "No disks volume for temporary files"); for (const auto & disk : volume->getDisks()) { @@ -1159,11 +1064,9 @@ void Context::setTemporaryStoragePolicy(const String & policy_name, size_t max_s if (dynamic_cast(disk_ptr.get()) == nullptr) { const auto * disk_raw_ptr = disk_ptr.get(); - throw Exception( - ErrorCodes::NO_ELEMENTS_IN_CONFIG, + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Disk '{}' ({}) is not local and can't be used for temporary files", - disk_ptr->getName(), - typeid(*disk_raw_ptr).name()); + disk_ptr->getName(), typeid(*disk_raw_ptr).name()); } setupTmpPath(shared->log, disk->getPath()); @@ -1288,11 +1191,9 @@ void Context::setUser(const UUID & user_id_, const std::optional(user_id_); - auto new_current_roles - = current_roles_ ? user->granted_roles.findGranted(*current_roles_) : user->granted_roles.findGranted(user->default_roles); + auto new_current_roles = current_roles_ ? user->granted_roles.findGranted(*current_roles_) : user->granted_roles.findGranted(user->default_roles); auto enabled_roles = access_control.getEnabledRolesInfo(new_current_roles, {}); - auto enabled_profiles = access_control.getEnabledSettingsInfo( - user_id_, user->settings, enabled_roles->enabled_roles, enabled_roles->settings_from_enabled_roles); + auto enabled_profiles = access_control.getEnabledSettingsInfo(user_id_, user->settings, enabled_roles->enabled_roles, enabled_roles->settings_from_enabled_roles); const auto & database = user->default_database; /// Apply user's profiles, constraints, settings, roles. @@ -1383,55 +1284,18 @@ void Context::checkAccessImpl(const Args &... args) const return getAccess()->checkAccess(args...); } -void Context::checkAccess(const AccessFlags & flags) const -{ - return checkAccessImpl(flags); -} -void Context::checkAccess(const AccessFlags & flags, std::string_view database) const -{ - return checkAccessImpl(flags, database); -} -void Context::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table) const -{ - return checkAccessImpl(flags, database, table); -} -void Context::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const -{ - return checkAccessImpl(flags, database, table, column); -} -void Context::checkAccess( - const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const -{ - return checkAccessImpl(flags, database, table, columns); -} -void Context::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const -{ - return checkAccessImpl(flags, database, table, columns); -} -void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id) const -{ - checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName()); -} -void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, std::string_view column) const -{ - checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), column); -} -void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::vector & columns) const -{ - checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), columns); -} -void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const Strings & columns) const -{ - checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), columns); -} -void Context::checkAccess(const AccessRightsElement & element) const -{ - return checkAccessImpl(element); -} -void Context::checkAccess(const AccessRightsElements & elements) const -{ - return checkAccessImpl(elements); -} +void Context::checkAccess(const AccessFlags & flags) const { return checkAccessImpl(flags); } +void Context::checkAccess(const AccessFlags & flags, std::string_view database) const { return checkAccessImpl(flags, database); } +void Context::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table) const { return checkAccessImpl(flags, database, table); } +void Context::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { return checkAccessImpl(flags, database, table, column); } +void Context::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { return checkAccessImpl(flags, database, table, columns); } +void Context::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { return checkAccessImpl(flags, database, table, columns); } +void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName()); } +void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, std::string_view column) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), column); } +void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::vector & columns) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), columns); } +void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const Strings & columns) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), columns); } +void Context::checkAccess(const AccessRightsElement & element) const { return checkAccessImpl(element); } +void Context::checkAccess(const AccessRightsElements & elements) const { return checkAccessImpl(elements); } std::shared_ptr Context::getAccess() const { @@ -1441,8 +1305,7 @@ std::shared_ptr Context::getAccess() const /// If setUserID() was never called then this must be the global context with the full access. bool full_access = !user_id; - return ContextAccessParams{ - user_id, full_access, /* use_default_roles= */ false, current_roles, settings, current_database, client_info}; + return ContextAccessParams{user_id, full_access, /* use_default_roles= */ false, current_roles, settings, current_database, client_info}; }; /// Check if the current access rights are still valid, otherwise get parameters for recalculating access rights. @@ -1497,8 +1360,7 @@ std::optional Context::getQuotaUsage() const return getAccess()->getQuotaUsage(); } -void Context::setCurrentProfileWithLock( - const String & profile_name, bool check_constraints, const std::lock_guard & lock) +void Context::setCurrentProfileWithLock(const String & profile_name, bool check_constraints, const std::lock_guard & lock) { try { @@ -1518,8 +1380,7 @@ void Context::setCurrentProfileWithLock(const UUID & profile_id, bool check_cons setCurrentProfilesWithLock(*profile_info, check_constraints, lock); } -void Context::setCurrentProfilesWithLock( - const SettingsProfilesInfo & profiles_info, bool check_constraints, const std::lock_guard & lock) +void Context::setCurrentProfilesWithLock(const SettingsProfilesInfo & profiles_info, bool check_constraints, const std::lock_guard & lock) { if (check_constraints) checkSettingsConstraintsWithLock(profiles_info.settings, SettingSource::PROFILE); @@ -1560,10 +1421,9 @@ std::vector Context::getEnabledProfiles() const ResourceManagerPtr Context::getResourceManager() const { - callOnce( - shared->resource_manager_initialized, - [&] - { shared->resource_manager = ResourceManagerFactory::instance().get(getConfigRef().getString("resource_manager", "dynamic")); }); + callOnce(shared->resource_manager_initialized, [&] { + shared->resource_manager = ResourceManagerFactory::instance().get(getConfigRef().getString("resource_manager", "dynamic")); + }); return shared->resource_manager; } @@ -1831,18 +1691,17 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const ASTCreateQuery create; create.select = query->as(); auto sample_block = InterpreterSelectWithUnionQuery::getSampleBlock(query, getQueryContext()); - auto res = std::make_shared( - StorageID(database_name, table_name), - create, - ColumnsDescription(sample_block.getNamesAndTypesList()), - /* comment */ "", - /* is_parameterized_view */ true); + auto res = std::make_shared(StorageID(database_name, table_name), + create, + ColumnsDescription(sample_block.getNamesAndTypesList()), + /* comment */ "", + /* is_parameterized_view */ true); res->startup(); function->prefer_subquery_to_function_formatting = true; return res; } } - auto hash = table_expression->getTreeHash(/*ignore_aliases=*/true); + auto hash = table_expression->getTreeHash(/*ignore_aliases=*/ true); auto key = toString(hash); StoragePtr & res = table_function_results[key]; if (!res) @@ -1855,19 +1714,21 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const catch (Exception & e) { if (e.code() == ErrorCodes::UNKNOWN_FUNCTION) + { e.addMessage(" or incorrect parameterized view"); + } throw; } - uint64_t use_structure_from_insertion_table_in_table_functions - = getSettingsRef().use_structure_from_insertion_table_in_table_functions; + uint64_t use_structure_from_insertion_table_in_table_functions = getSettingsRef().use_structure_from_insertion_table_in_table_functions; if (use_structure_from_insertion_table_in_table_functions && table_function_ptr->needStructureHint() && hasInsertionTable()) { - const auto & insert_columns - = DatabaseCatalog::instance().getTable(getInsertionTable(), shared_from_this())->getInMemoryMetadataPtr()->getColumns(); + const auto & insert_columns = DatabaseCatalog::instance() + .getTable(getInsertionTable(), shared_from_this()) + ->getInMemoryMetadataPtr() + ->getColumns(); - const auto & insert_column_names - = hasInsertionTableColumnNames() ? *getInsertionTableColumnNames() : insert_columns.getOrdinary().getNames(); + const auto & insert_column_names = hasInsertionTableColumnNames() ? *getInsertionTableColumnNames() : insert_columns.getOrdinary().getNames(); DB::ColumnsDescription structure_hint; bool use_columns_from_insert_query = true; @@ -1876,7 +1737,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const /// insert table columns to table function columns through names from SELECT expression. auto insert_column_name_it = insert_column_names.begin(); - auto insert_column_names_end = insert_column_names.end(); /// end iterator of the range covered by possible asterisk + auto insert_column_names_end = insert_column_names.end(); /// end iterator of the range covered by possible asterisk auto virtual_column_names = table_function_ptr->getVirtualsToCheckBeforeUsingStructureHint(); bool asterisk = false; const auto & expression_list = select_query_hint->select()->as()->children; @@ -1893,8 +1754,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const if (asterisk) { if (use_structure_from_insertion_table_in_table_functions == 1) - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, "Asterisk cannot be mixed with column list in INSERT SELECT query."); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Asterisk cannot be mixed with column list in INSERT SELECT query."); use_columns_from_insert_query = false; break; @@ -1927,8 +1787,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const if (!structure_hint.empty()) { if (use_structure_from_insertion_table_in_table_functions == 1) - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, "Asterisk cannot be mixed with column list in INSERT SELECT query."); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Asterisk cannot be mixed with column list in INSERT SELECT query."); use_columns_from_insert_query = false; break; @@ -1966,8 +1825,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const { /// For input function we should check if input format supports reading subset of columns. if (table_function_ptr->getName() == "input") - use_columns_from_insert_query - = FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(getInsertFormat(), shared_from_this()); + use_columns_from_insert_query = FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(getInsertFormat(), shared_from_this()); else use_columns_from_insert_query = table_function_ptr->supportsReadingSubsetOfColumns(shared_from_this()); } @@ -1991,11 +1849,9 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const if (!structure_hint.empty()) table_function_ptr->setStructureHint(structure_hint); - } - else if (use_structure_from_insertion_table_in_table_functions == 1) - throw Exception( - ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH, - "Number of columns in insert table less than required by SELECT expression."); + + } else if (use_structure_from_insertion_table_in_table_functions == 1) + throw Exception(ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH, "Number of columns in insert table less than required by SELECT expression."); } } @@ -2005,7 +1861,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const /// /// remote('127.1', system.one) -> remote('127.1', 'system.one'), /// - auto new_hash = table_expression->getTreeHash(/*ignore_aliases=*/true); + auto new_hash = table_expression->getTreeHash(/*ignore_aliases=*/ true); if (hash != new_hash) { key = toString(new_hash); @@ -2017,12 +1873,14 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const TableFunctionPtr & table_function_ptr) { - const auto hash = table_expression->getTreeHash(/*ignore_aliases=*/true); + const auto hash = table_expression->getTreeHash(/*ignore_aliases=*/ true); const auto key = toString(hash); StoragePtr & res = table_function_results[key]; if (!res) + { res = table_function_ptr->execute(table_expression, shared_from_this(), table_function_ptr->getName()); + } return res; } @@ -2031,8 +1889,8 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const void Context::addViewSource(const StoragePtr & storage) { if (view_source) - throw Exception( - ErrorCodes::TABLE_ALREADY_EXISTS, "Temporary view source storage {} already exists.", backQuoteIfNeed(view_source->getName())); + throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Temporary view source storage {} already exists.", + backQuoteIfNeed(view_source->getName())); view_source = storage; } @@ -2093,12 +1951,13 @@ void Context::applySettingChangeWithLock(const SettingChange & change, const std catch (Exception & e) { e.addMessage(fmt::format( - "in attempt to set the value of setting '{}' to {}", change.name, applyVisitor(FieldVisitorToString(), change.value))); + "in attempt to set the value of setting '{}' to {}", + change.name, applyVisitor(FieldVisitorToString(), change.value))); throw; } } -void Context::applySettingsChangesWithLock(const SettingsChanges & changes, const std::lock_guard & lock) +void Context::applySettingsChangesWithLock(const SettingsChanges & changes, const std::lock_guard& lock) { for (const SettingChange & change : changes) applySettingChangeWithLock(change, lock); @@ -2126,7 +1985,8 @@ void Context::applySettingChange(const SettingChange & change) catch (Exception & e) { e.addMessage(fmt::format( - "in attempt to set the value of setting '{}' to {}", change.name, applyVisitor(FieldVisitorToString(), change.value))); + "in attempt to set the value of setting '{}' to {}", + change.name, applyVisitor(FieldVisitorToString(), change.value))); throw; } } @@ -2163,8 +2023,7 @@ void Context::clampToSettingsConstraintsWithLock(SettingsChanges & changes, Sett getSettingsConstraintsAndCurrentProfilesWithLock()->constraints.clamp(settings, changes, source); } -void Context::checkMergeTreeSettingsConstraintsWithLock( - const MergeTreeSettings & merge_tree_settings, const SettingsChanges & changes) const +void Context::checkMergeTreeSettingsConstraintsWithLock(const MergeTreeSettings & merge_tree_settings, const SettingsChanges & changes) const { getSettingsConstraintsAndCurrentProfilesWithLock()->constraints.check(merge_tree_settings, changes); } @@ -2208,7 +2067,7 @@ void Context::checkMergeTreeSettingsConstraints(const MergeTreeSettings & merge_ void Context::resetSettingsToDefaultValue(const std::vector & names) { std::lock_guard lock(mutex); - for (const String & name : names) + for (const String & name: names) settings.setDefaultValue(name); } @@ -2242,10 +2101,9 @@ String Context::getInitialQueryId() const void Context::setCurrentDatabaseNameInGlobalContext(const String & name) { if (!isGlobalContext()) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Cannot set current database for non global context, this method should " - "be used during server initialization"); + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot set current database for non global context, this method should " + "be used during server initialization"); std::lock_guard lock(mutex); if (!current_database.empty()) @@ -2289,12 +2147,13 @@ void Context::setCurrentQueryId(const String & query_id) String query_id_to_set = query_id; - if (query_id_to_set.empty()) /// If the user did not submit his query_id, then we generate it ourselves. + if (query_id_to_set.empty()) /// If the user did not submit his query_id, then we generate it ourselves. { /// Use protected constructor. struct QueryUUID : Poco::UUID { - QueryUUID(const char * bytes, Poco::UUID::Version version) : Poco::UUID(bytes, version) { } + QueryUUID(const char * bytes, Poco::UUID::Version version) + : Poco::UUID(bytes, version) {} }; query_id_to_set = QueryUUID(random.bytes, Poco::UUID::UUID_RANDOM).toString(); @@ -2360,8 +2219,7 @@ void Context::setMacros(std::unique_ptr && macros) ContextMutablePtr Context::getQueryContext() const { auto ptr = query_context.lock(); - if (!ptr) - throw Exception(ErrorCodes::THERE_IS_NO_QUERY, "There is no query or query context has expired"); + if (!ptr) throw Exception(ErrorCodes::THERE_IS_NO_QUERY, "There is no query or query context has expired"); return ptr; } @@ -2374,23 +2232,20 @@ bool Context::isInternalSubquery() const ContextMutablePtr Context::getSessionContext() const { auto ptr = session_context.lock(); - if (!ptr) - throw Exception(ErrorCodes::THERE_IS_NO_SESSION, "There is no session or session context has expired"); + if (!ptr) throw Exception(ErrorCodes::THERE_IS_NO_SESSION, "There is no session or session context has expired"); return ptr; } ContextMutablePtr Context::getGlobalContext() const { auto ptr = global_context.lock(); - if (!ptr) - throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no global context or global context has expired"); + if (!ptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no global context or global context has expired"); return ptr; } ContextMutablePtr Context::getBufferContext() const { - if (!buffer_context) - throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no buffer context"); + if (!buffer_context) throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no buffer context"); return buffer_context; } @@ -2486,11 +2341,11 @@ ExternalDictionariesLoader & Context::getExternalDictionariesLoader() return getExternalDictionariesLoaderWithLock(lock); } -ExternalDictionariesLoader & Context::getExternalDictionariesLoaderWithLock(const std::lock_guard &) - TSA_REQUIRES(shared->external_dictionaries_mutex) +ExternalDictionariesLoader & Context::getExternalDictionariesLoaderWithLock(const std::lock_guard &) TSA_REQUIRES(shared->external_dictionaries_mutex) { if (!shared->external_dictionaries_loader) - shared->external_dictionaries_loader = std::make_unique(getGlobalContext()); + shared->external_dictionaries_loader = + std::make_unique(getGlobalContext()); return *shared->external_dictionaries_loader; } @@ -2506,12 +2361,11 @@ ExternalUserDefinedExecutableFunctionsLoader & Context::getExternalUserDefinedEx } ExternalUserDefinedExecutableFunctionsLoader & -Context::getExternalUserDefinedExecutableFunctionsLoaderWithLock(const std::lock_guard &) - TSA_REQUIRES(shared->external_user_defined_executable_functions_mutex) +Context::getExternalUserDefinedExecutableFunctionsLoaderWithLock(const std::lock_guard &) TSA_REQUIRES(shared->external_user_defined_executable_functions_mutex) { if (!shared->external_user_defined_executable_functions_loader) - shared->external_user_defined_executable_functions_loader - = std::make_unique(getGlobalContext()); + shared->external_user_defined_executable_functions_loader = + std::make_unique(getGlobalContext()); return *shared->external_user_defined_executable_functions_loader; } @@ -2523,8 +2377,10 @@ EmbeddedDictionaries & Context::getEmbeddedDictionariesImpl(const bool throw_on_ { auto geo_dictionaries_loader = std::make_unique(); - shared->embedded_dictionaries - = std::make_unique(std::move(geo_dictionaries_loader), getGlobalContext(), throw_on_error); + shared->embedded_dictionaries = std::make_unique( + std::move(geo_dictionaries_loader), + getGlobalContext(), + throw_on_error); } return *shared->embedded_dictionaries; @@ -2593,8 +2449,7 @@ void Context::loadOrReloadUserDefinedExecutableFunctions(const Poco::Util::Abstr if (shared->user_defined_executable_functions_config_repository) { shared->user_defined_executable_functions_config_repository->updatePatterns(patterns); - external_user_defined_executable_functions_loader.reloadConfig( - shared->user_defined_executable_functions_config_repository->getName()); + external_user_defined_executable_functions_loader.reloadConfig(shared->user_defined_executable_functions_config_repository->getName()); return; } @@ -2602,15 +2457,14 @@ void Context::loadOrReloadUserDefinedExecutableFunctions(const Poco::Util::Abstr auto config_path = getConfigRef().getString("config-file", "config.xml"); auto repository = std::make_unique(app_path, config_path, patterns); shared->user_defined_executable_functions_config_repository = repository.get(); - shared->user_defined_executable_functions_xmls - = external_user_defined_executable_functions_loader.addConfigRepository(std::move(repository)); + shared->user_defined_executable_functions_xmls = external_user_defined_executable_functions_loader.addConfigRepository(std::move(repository)); } const IUserDefinedSQLObjectsLoader & Context::getUserDefinedSQLObjectsLoader() const { - callOnce( - shared->user_defined_sql_objects_loader_initialized, - [&] { shared->user_defined_sql_objects_loader = createUserDefinedSQLObjectsLoader(getGlobalContext()); }); + callOnce(shared->user_defined_sql_objects_loader_initialized, [&] { + shared->user_defined_sql_objects_loader = createUserDefinedSQLObjectsLoader(getGlobalContext()); + }); SharedLockGuard lock(shared->mutex); return *shared->user_defined_sql_objects_loader; @@ -2618,9 +2472,9 @@ const IUserDefinedSQLObjectsLoader & Context::getUserDefinedSQLObjectsLoader() c IUserDefinedSQLObjectsLoader & Context::getUserDefinedSQLObjectsLoader() { - callOnce( - shared->user_defined_sql_objects_loader_initialized, - [&] { shared->user_defined_sql_objects_loader = createUserDefinedSQLObjectsLoader(getGlobalContext()); }); + callOnce(shared->user_defined_sql_objects_loader_initialized, [&] { + shared->user_defined_sql_objects_loader = createUserDefinedSQLObjectsLoader(getGlobalContext()); + }); SharedLockGuard lock(shared->mutex); return *shared->user_defined_sql_objects_loader; @@ -2630,14 +2484,18 @@ IUserDefinedSQLObjectsLoader & Context::getUserDefinedSQLObjectsLoader() SynonymsExtensions & Context::getSynonymsExtensions() const { - callOnce(shared->synonyms_extensions_initialized, [&] { shared->synonyms_extensions.emplace(getConfigRef()); }); + callOnce(shared->synonyms_extensions_initialized, [&] { + shared->synonyms_extensions.emplace(getConfigRef()); + }); return *shared->synonyms_extensions; } Lemmatizers & Context::getLemmatizers() const { - callOnce(shared->lemmatizers_initialized, [&] { shared->lemmatizers.emplace(getConfigRef()); }); + callOnce(shared->lemmatizers_initialized, [&] { + shared->lemmatizers.emplace(getConfigRef()); + }); return *shared->lemmatizers; } @@ -2645,21 +2503,17 @@ Lemmatizers & Context::getLemmatizers() const BackupsWorker & Context::getBackupsWorker() const { - callOnce( - shared->backups_worker_initialized, - [&] - { - const auto & config = getConfigRef(); - const bool allow_concurrent_backups = config.getBool("backups.allow_concurrent_backups", true); - const bool allow_concurrent_restores = config.getBool("backups.allow_concurrent_restores", true); + callOnce(shared->backups_worker_initialized, [&] { + const auto & config = getConfigRef(); + const bool allow_concurrent_backups = config.getBool("backups.allow_concurrent_backups", true); + const bool allow_concurrent_restores = config.getBool("backups.allow_concurrent_restores", true); - const auto & settings_ref = getSettingsRef(); - UInt64 backup_threads = config.getUInt64("backup_threads", settings_ref.backup_threads); - UInt64 restore_threads = config.getUInt64("restore_threads", settings_ref.restore_threads); + const auto & settings_ref = getSettingsRef(); + UInt64 backup_threads = config.getUInt64("backup_threads", settings_ref.backup_threads); + UInt64 restore_threads = config.getUInt64("restore_threads", settings_ref.restore_threads); - shared->backups_worker.emplace( - getGlobalContext(), backup_threads, restore_threads, allow_concurrent_backups, allow_concurrent_restores); - }); + shared->backups_worker.emplace(getGlobalContext(), backup_threads, restore_threads, allow_concurrent_backups, allow_concurrent_restores); + }); return *shared->backups_worker; } @@ -2774,21 +2628,13 @@ void Context::clearMarkCache() const ThreadPool & Context::getLoadMarksThreadpool() const { - callOnce( - shared->load_marks_threadpool_initialized, - [&] - { - const auto & config = getConfigRef(); - auto pool_size = config.getUInt(".load_marks_threadpool_pool_size", 50); - auto queue_size = config.getUInt(".load_marks_threadpool_queue_size", 1000000); - shared->load_marks_threadpool = std::make_unique( - CurrentMetrics::MarksLoaderThreads, - CurrentMetrics::MarksLoaderThreadsActive, - CurrentMetrics::MarksLoaderThreadsScheduled, - pool_size, - pool_size, - queue_size); - }); + callOnce(shared->load_marks_threadpool_initialized, [&] { + const auto & config = getConfigRef(); + auto pool_size = config.getUInt(".load_marks_threadpool_pool_size", 50); + auto queue_size = config.getUInt(".load_marks_threadpool_queue_size", 1000000); + shared->load_marks_threadpool = std::make_unique( + CurrentMetrics::MarksLoaderThreads, CurrentMetrics::MarksLoaderThreadsActive, CurrentMetrics::MarksLoaderThreadsScheduled, pool_size, pool_size, queue_size); + }); return *shared->load_marks_threadpool; } @@ -2965,21 +2811,13 @@ void Context::clearCaches() const ThreadPool & Context::getPrefetchThreadpool() const { - callOnce( - shared->prefetch_threadpool_initialized, - [&] - { - const auto & config = getConfigRef(); - auto pool_size = config.getUInt(".prefetch_threadpool_pool_size", 100); - auto queue_size = config.getUInt(".prefetch_threadpool_queue_size", 1000000); - shared->prefetch_threadpool = std::make_unique( - CurrentMetrics::IOPrefetchThreads, - CurrentMetrics::IOPrefetchThreadsActive, - CurrentMetrics::IOPrefetchThreadsScheduled, - pool_size, - pool_size, - queue_size); - }); + callOnce(shared->prefetch_threadpool_initialized, [&] { + const auto & config = getConfigRef(); + auto pool_size = config.getUInt(".prefetch_threadpool_pool_size", 100); + auto queue_size = config.getUInt(".prefetch_threadpool_queue_size", 1000000); + shared->prefetch_threadpool = std::make_unique( + CurrentMetrics::IOPrefetchThreads, CurrentMetrics::IOPrefetchThreadsActive, CurrentMetrics::IOPrefetchThreadsScheduled, pool_size, pool_size, queue_size); + }); return *shared->prefetch_threadpool; } @@ -2992,16 +2830,13 @@ size_t Context::getPrefetchThreadpoolSize() const BackgroundSchedulePool & Context::getBufferFlushSchedulePool() const { - callOnce( - shared->buffer_flush_schedule_pool_initialized, - [&] - { - shared->buffer_flush_schedule_pool = std::make_unique( - shared->server_settings.background_buffer_flush_schedule_pool_size, - CurrentMetrics::BackgroundBufferFlushSchedulePoolTask, - CurrentMetrics::BackgroundBufferFlushSchedulePoolSize, - "BgBufSchPool"); - }); + callOnce(shared->buffer_flush_schedule_pool_initialized, [&] { + shared->buffer_flush_schedule_pool = std::make_unique( + shared->server_settings.background_buffer_flush_schedule_pool_size, + CurrentMetrics::BackgroundBufferFlushSchedulePoolTask, + CurrentMetrics::BackgroundBufferFlushSchedulePoolSize, + "BgBufSchPool"); + }); return *shared->buffer_flush_schedule_pool; } @@ -3013,16 +2848,11 @@ BackgroundTaskSchedulingSettings Context::getBackgroundProcessingTaskSchedulingS const auto & config = getConfigRef(); task_settings.thread_sleep_seconds = config.getDouble("background_processing_pool_thread_sleep_seconds", 10); task_settings.thread_sleep_seconds_random_part = config.getDouble("background_processing_pool_thread_sleep_seconds_random_part", 1.0); - task_settings.thread_sleep_seconds_if_nothing_to_do - = config.getDouble("background_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); - task_settings.task_sleep_seconds_when_no_work_min - = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 10); - task_settings.task_sleep_seconds_when_no_work_max - = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 600); - task_settings.task_sleep_seconds_when_no_work_multiplier - = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); - task_settings.task_sleep_seconds_when_no_work_random_part - = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); + task_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); + task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 10); + task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 600); + task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); + task_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); return task_settings; } @@ -3032,66 +2862,51 @@ BackgroundTaskSchedulingSettings Context::getBackgroundMoveTaskSchedulingSetting const auto & config = getConfigRef(); task_settings.thread_sleep_seconds = config.getDouble("background_move_processing_pool_thread_sleep_seconds", 10); - task_settings.thread_sleep_seconds_random_part - = config.getDouble("background_move_processing_pool_thread_sleep_seconds_random_part", 1.0); - task_settings.thread_sleep_seconds_if_nothing_to_do - = config.getDouble("background_move_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); - task_settings.task_sleep_seconds_when_no_work_min - = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_min", 10); - task_settings.task_sleep_seconds_when_no_work_max - = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_max", 600); - task_settings.task_sleep_seconds_when_no_work_multiplier - = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); - task_settings.task_sleep_seconds_when_no_work_random_part - = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); + task_settings.thread_sleep_seconds_random_part = config.getDouble("background_move_processing_pool_thread_sleep_seconds_random_part", 1.0); + task_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_move_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); + task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_min", 10); + task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_max", 600); + task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); + task_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); return task_settings; } BackgroundSchedulePool & Context::getSchedulePool() const { - callOnce( - shared->schedule_pool_initialized, - [&] - { - shared->schedule_pool = std::make_unique( - shared->server_settings.background_schedule_pool_size, - CurrentMetrics::BackgroundSchedulePoolTask, - CurrentMetrics::BackgroundSchedulePoolSize, - "BgSchPool"); - }); + callOnce(shared->schedule_pool_initialized, [&] { + shared->schedule_pool = std::make_unique( + shared->server_settings.background_schedule_pool_size, + CurrentMetrics::BackgroundSchedulePoolTask, + CurrentMetrics::BackgroundSchedulePoolSize, + "BgSchPool"); + }); return *shared->schedule_pool; } BackgroundSchedulePool & Context::getDistributedSchedulePool() const { - callOnce( - shared->distributed_schedule_pool_initialized, - [&] - { - shared->distributed_schedule_pool = std::make_unique( - shared->server_settings.background_distributed_schedule_pool_size, - CurrentMetrics::BackgroundDistributedSchedulePoolTask, - CurrentMetrics::BackgroundDistributedSchedulePoolSize, - "BgDistSchPool"); - }); + callOnce(shared->distributed_schedule_pool_initialized, [&] { + shared->distributed_schedule_pool = std::make_unique( + shared->server_settings.background_distributed_schedule_pool_size, + CurrentMetrics::BackgroundDistributedSchedulePoolTask, + CurrentMetrics::BackgroundDistributedSchedulePoolSize, + "BgDistSchPool"); + }); return *shared->distributed_schedule_pool; } BackgroundSchedulePool & Context::getMessageBrokerSchedulePool() const { - callOnce( - shared->message_broker_schedule_pool_initialized, - [&] - { - shared->message_broker_schedule_pool = std::make_unique( - shared->server_settings.background_message_broker_schedule_pool_size, - CurrentMetrics::BackgroundMessageBrokerSchedulePoolTask, - CurrentMetrics::BackgroundMessageBrokerSchedulePoolSize, - "BgMBSchPool"); - }); + callOnce(shared->message_broker_schedule_pool_initialized, [&] { + shared->message_broker_schedule_pool = std::make_unique( + shared->server_settings.background_message_broker_schedule_pool_size, + CurrentMetrics::BackgroundMessageBrokerSchedulePoolTask, + CurrentMetrics::BackgroundMessageBrokerSchedulePoolSize, + "BgMBSchPool"); + }); return *shared->message_broker_schedule_pool; } @@ -3290,10 +3105,7 @@ bool Context::tryCheckClientConnectionToMyKeeperCluster() const { if (checkZooKeeperConfigIsLocal(getConfigRef(), "auxiliary_zookeepers." + aux_zk_name)) { - LOG_DEBUG( - shared->log, - "Our Keeper server is participant of the auxiliary zookeeper cluster ({}), will try to connect to it", - aux_zk_name); + LOG_DEBUG(shared->log, "Our Keeper server is participant of the auxiliary zookeeper cluster ({}), will try to connect to it", aux_zk_name); getAuxiliaryZooKeeper(aux_zk_name); /// Connected, return true return true; @@ -3366,17 +3178,13 @@ void Context::initializeKeeperDispatcher([[maybe_unused]] bool start_async) cons if (start_async) { assert(!is_standalone_app); - LOG_INFO( - shared->log, - "Connected to ZooKeeper (or Keeper) before internal Keeper start or we don't depend on our Keeper cluster, " - "will wait for Keeper asynchronously"); + LOG_INFO(shared->log, "Connected to ZooKeeper (or Keeper) before internal Keeper start or we don't depend on our Keeper cluster, " + "will wait for Keeper asynchronously"); } else { - LOG_INFO( - shared->log, - "Cannot connect to ZooKeeper (or Keeper) before internal Keeper start, " - "will wait for Keeper synchronously"); + LOG_INFO(shared->log, "Cannot connect to ZooKeeper (or Keeper) before internal Keeper start, " + "will wait for Keeper synchronously"); } shared->keeper_dispatcher = std::make_shared(); @@ -3445,9 +3253,8 @@ zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const "config.xml", name); - zookeeper = shared->auxiliary_zookeepers - .emplace(name, std::make_shared(config, "auxiliary_zookeepers." + name, getZooKeeperLog())) - .first; + zookeeper = shared->auxiliary_zookeepers.emplace(name, + std::make_shared(config, "auxiliary_zookeepers." + name, getZooKeeperLog())).first; } else if (zookeeper->second->expired()) zookeeper->second = zookeeper->second->startNewSession(); @@ -3554,12 +3361,11 @@ void Context::setInterserverIOAddress(const String & host, UInt16 port) std::pair Context::getInterserverIOAddress() const { if (shared->interserver_io_host.empty() || shared->interserver_io_port == 0) - throw Exception( - ErrorCodes::NO_ELEMENTS_IN_CONFIG, - "Parameter 'interserver_http(s)_port' required for replication is not specified " - "in configuration file."); + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, + "Parameter 'interserver_http(s)_port' required for replication is not specified " + "in configuration file."); - return {shared->interserver_io_host, shared->interserver_io_port}; + return { shared->interserver_io_host, shared->interserver_io_port }; } void Context::setInterserverScheme(const String & scheme) @@ -3633,13 +3439,13 @@ void Context::setMaxPartNumToWarn(size_t max_part_to_warn) void Context::setMaxTableNumToWarn(size_t max_table_to_warn) { SharedLockGuard lock(shared->mutex); - shared->max_table_num_to_warn = max_table_to_warn; + shared->max_table_num_to_warn= max_table_to_warn; } void Context::setMaxDatabaseNumToWarn(size_t max_database_to_warn) { SharedLockGuard lock(shared->mutex); - shared->max_database_num_to_warn = max_database_to_warn; + shared->max_database_num_to_warn= max_database_to_warn; } std::shared_ptr Context::getCluster(const std::string & cluster_name) const @@ -3735,7 +3541,9 @@ void Context::setClustersConfig(const ConfigurationPtr & config, bool enable_dis { std::lock_guard lock(shared->clusters_mutex); if (ConfigHelper::getBool(*config, "allow_experimental_cluster_discovery") && enable_discovery && !shared->cluster_discovery) + { shared->cluster_discovery = std::make_unique(*config, getGlobalContext()); + } /// Do not update clusters if this part of config wasn't changed. if (shared->clusters && isSameConfiguration(*config, *shared->clusters_config, config_name)) @@ -3768,14 +3576,11 @@ void Context::initializeSystemLogs() /// triggered from another thread, that is launched while initializing the system logs, /// for example, system.filesystem_cache_log will be triggered by parts loading /// of any other table if it is stored on a disk with cache. - callOnce( - shared->system_logs_initialized, - [&] - { - auto system_logs = std::make_unique(getGlobalContext(), getConfigRef()); - std::lock_guard lock(shared->mutex); - shared->system_logs = std::move(system_logs); - }); + callOnce(shared->system_logs_initialized, [&] { + auto system_logs = std::make_unique(getGlobalContext(), getConfigRef()); + std::lock_guard lock(shared->mutex); + shared->system_logs = std::move(system_logs); + }); } void Context::initializeTraceCollector() @@ -4116,8 +3921,7 @@ DiskSelectorPtr Context::getDiskSelector(std::lock_guard & /* lock * return shared->merge_tree_disk_selector; } -StoragePolicySelectorPtr Context::getStoragePolicySelector(std::lock_guard & lock) const - TSA_REQUIRES(shared->storage_policies_mutex) +StoragePolicySelectorPtr Context::getStoragePolicySelector(std::lock_guard & lock) const TSA_REQUIRES(shared->storage_policies_mutex) { if (!shared->merge_tree_storage_policy_selector) { @@ -4149,9 +3953,7 @@ void Context::updateStorageConfiguration(const Poco::Util::AbstractConfiguration catch (Exception & e) { LOG_ERROR( - shared->log, - "An error has occurred while reloading storage policies, storage policies were not applied: {}", - e.message()); + shared->log, "An error has occurred while reloading storage policies, storage policies were not applied: {}", e.message()); } } @@ -4167,6 +3969,7 @@ void Context::updateStorageConfiguration(const Poco::Util::AbstractConfiguration if (shared->storage_s3_settings) shared->storage_s3_settings->loadFromConfig("s3", config, getSettingsRef()); } + } @@ -4238,24 +4041,19 @@ void Context::checkCanBeDropped(const String & database, const String & table, c String size_str = formatReadableSizeWithDecimalSuffix(size); String max_size_to_drop_str = formatReadableSizeWithDecimalSuffix(max_size_to_drop); - throw Exception( - ErrorCodes::TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT, - "Table or Partition in {}.{} was not dropped.\nReason:\n" - "1. Size ({}) is greater than max_[table/partition]_size_to_drop ({})\n" - "2. File '{}' intended to force DROP {}\n" - "How to fix this:\n" - "1. Either increase (or set to zero) max_[table/partition]_size_to_drop in server config\n" - "2. Either create forcing file {} and make sure that ClickHouse has write permission for it.\n" - "Example:\nsudo touch '{}' && sudo chmod 666 '{}'", - backQuoteIfNeed(database), - backQuoteIfNeed(table), - size_str, - max_size_to_drop_str, - force_file.string(), - force_file_exists ? "exists but not writeable (could not be removed)" : "doesn't exist", - force_file.string(), - force_file.string(), - force_file.string()); + throw Exception(ErrorCodes::TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT, + "Table or Partition in {}.{} was not dropped.\nReason:\n" + "1. Size ({}) is greater than max_[table/partition]_size_to_drop ({})\n" + "2. File '{}' intended to force DROP {}\n" + "How to fix this:\n" + "1. Either increase (or set to zero) max_[table/partition]_size_to_drop in server config\n" + "2. Either create forcing file {} and make sure that ClickHouse has write permission for it.\n" + "Example:\nsudo touch '{}' && sudo chmod 666 '{}'", + backQuoteIfNeed(database), backQuoteIfNeed(table), + size_str, max_size_to_drop_str, + force_file.string(), force_file_exists ? "exists but not writeable (could not be removed)" : "doesn't exist", + force_file.string(), + force_file.string(), force_file.string()); } @@ -4287,7 +4085,7 @@ void Context::setClientHTTPHeaderForbiddenHeaders(const String & forbidden_heade void Context::setAllowGetHTTPHeaderFunction(bool allow_get_http_header_function) { - shared->allow_get_client_http_header = allow_get_http_header_function; + shared->allow_get_client_http_header= allow_get_http_header_function; } const std::unordered_set & Context::getClientHTTPHeaderForbiddenHeaders() const @@ -4319,13 +4117,7 @@ void Context::checkPartitionCanBeDropped(const String & database, const String & } -InputFormatPtr Context::getInputFormat( - const String & name, - ReadBuffer & buf, - const Block & sample, - UInt64 max_block_size, - const std::optional & format_settings, - const std::optional max_parsing_threads) const +InputFormatPtr Context::getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size, const std::optional & format_settings, const std::optional max_parsing_threads) const { return FormatFactory::instance().getInput(name, buf, sample, shared_from_this(), max_block_size, format_settings, max_parsing_threads); } @@ -4514,9 +4306,9 @@ const IHostContextPtr & Context::getHostContext() const std::shared_ptr Context::getActionLocksManager() const { - callOnce( - shared->action_locks_manager_initialized, - [&] { shared->action_locks_manager = std::make_shared(shared_from_this()); }); + callOnce(shared->action_locks_manager_initialized, [&] { + shared->action_locks_manager = std::make_shared(shared_from_this()); + }); return shared->action_locks_manager; } @@ -4603,8 +4395,7 @@ void Context::setClientInterface(ClientInfo::Interface interface) need_recalculate_access = true; } -void Context::setClientVersion( - UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version) +void Context::setClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version) { client_info.client_version_major = client_version_major; client_info.client_version_minor = client_version_minor; @@ -4617,11 +4408,7 @@ void Context::setClientConnectionId(uint32_t connection_id_) client_info.connection_id = connection_id_; } -void Context::setHttpClientInfo( - ClientInfo::HTTPMethod http_method, - const String & http_user_agent, - const String & http_referer, - const Poco::Net::NameValueCollection & http_headers) +void Context::setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer, const Poco::Net::NameValueCollection & http_headers) { client_info.http_method = http_method; client_info.http_user_agent = http_user_agent; @@ -4696,8 +4483,7 @@ void Context::setQuotaClientKey(const String & quota_key_) need_recalculate_access = true; } -void Context::setConnectionClientVersion( - UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version) +void Context::setConnectionClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version) { client_info.connection_client_version_major = client_version_major; client_info.connection_client_version_minor = client_version_minor; @@ -4778,12 +4564,10 @@ StorageID Context::resolveStorageIDImpl(StorageID storage_id, StorageNamespace w if (!storage_id.database_name.empty()) { if (in_specified_database) - return storage_id; /// NOTE There is no guarantees that table actually exists in database. + return storage_id; /// NOTE There is no guarantees that table actually exists in database. if (exception) - exception->emplace(Exception( - ErrorCodes::UNKNOWN_TABLE, - "External and temporary tables have no database, but {} is specified", - storage_id.database_name)); + exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "External and temporary tables have no database, but {} is specified", + storage_id.database_name)); return StorageID::createEmpty(); } @@ -4868,11 +4652,9 @@ void Context::checkTransactionsAreAllowed(bool explicit_tcl_query /* = false */) if (explicit_tcl_query) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Transactions are not supported"); - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Experimental support for transactions is disabled, " - "however, some query or background task tried to access TransactionLog. " - "If you have not enabled this feature explicitly, then it's a bug."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Experimental support for transactions is disabled, " + "however, some query or background task tried to access TransactionLog. " + "If you have not enabled this feature explicitly, then it's a bug."); } void Context::initCurrentTransaction(MergeTreeTransactionPtr txn) @@ -5033,55 +4815,47 @@ void Context::initializeBackgroundExecutorsIfNeeded() size_t background_common_pool_size = server_settings.background_common_pool_size; /// With this executor we can execute more tasks than threads we have - shared->merge_mutate_executor = std::make_shared( + shared->merge_mutate_executor = std::make_shared + ( "MergeMutate", - /*max_threads_count*/ background_pool_size, - /*max_tasks_count*/ background_pool_max_tasks_count, + /*max_threads_count*/background_pool_size, + /*max_tasks_count*/background_pool_max_tasks_count, CurrentMetrics::BackgroundMergesAndMutationsPoolTask, CurrentMetrics::BackgroundMergesAndMutationsPoolSize, - background_merges_mutations_scheduling_policy); - LOG_INFO( - shared->log, - "Initialized background executor for merges and mutations with num_threads={}, num_tasks={}, scheduling_policy={}", - background_pool_size, - background_pool_max_tasks_count, - background_merges_mutations_scheduling_policy); + background_merges_mutations_scheduling_policy + ); + LOG_INFO(shared->log, "Initialized background executor for merges and mutations with num_threads={}, num_tasks={}, scheduling_policy={}", + background_pool_size, background_pool_max_tasks_count, background_merges_mutations_scheduling_policy); - shared->moves_executor = std::make_shared( + shared->moves_executor = std::make_shared + ( "Move", background_move_pool_size, background_move_pool_size, CurrentMetrics::BackgroundMovePoolTask, - CurrentMetrics::BackgroundMovePoolSize); - LOG_INFO( - shared->log, - "Initialized background executor for move operations with num_threads={}, num_tasks={}", - background_move_pool_size, - background_move_pool_size); + CurrentMetrics::BackgroundMovePoolSize + ); + LOG_INFO(shared->log, "Initialized background executor for move operations with num_threads={}, num_tasks={}", background_move_pool_size, background_move_pool_size); - shared->fetch_executor = std::make_shared( + shared->fetch_executor = std::make_shared + ( "Fetch", background_fetches_pool_size, background_fetches_pool_size, CurrentMetrics::BackgroundFetchesPoolTask, - CurrentMetrics::BackgroundFetchesPoolSize); - LOG_INFO( - shared->log, - "Initialized background executor for fetches with num_threads={}, num_tasks={}", - background_fetches_pool_size, - background_fetches_pool_size); + CurrentMetrics::BackgroundFetchesPoolSize + ); + LOG_INFO(shared->log, "Initialized background executor for fetches with num_threads={}, num_tasks={}", background_fetches_pool_size, background_fetches_pool_size); - shared->common_executor = std::make_shared( + shared->common_executor = std::make_shared + ( "Common", background_common_pool_size, background_common_pool_size, CurrentMetrics::BackgroundCommonPoolTask, - CurrentMetrics::BackgroundCommonPoolSize); - LOG_INFO( - shared->log, - "Initialized background executor for common operations (e.g. clearing old parts) with num_threads={}, num_tasks={}", - background_common_pool_size, - background_common_pool_size); + CurrentMetrics::BackgroundCommonPoolSize + ); + LOG_INFO(shared->log, "Initialized background executor for common operations (e.g. clearing old parts) with num_threads={}, num_tasks={}", background_common_pool_size, background_common_pool_size); shared->are_background_executors_initialized = true; } @@ -5118,15 +4892,12 @@ OrdinaryBackgroundExecutorPtr Context::getCommonExecutor() const IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) const { - callOnce( - shared->readers_initialized, - [&] - { - const auto & config = getConfigRef(); - shared->asynchronous_remote_fs_reader = createThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER, config); - shared->asynchronous_local_fs_reader = createThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER, config); - shared->synchronous_local_fs_reader = createThreadPoolReader(FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER, config); - }); + callOnce(shared->readers_initialized, [&] { + const auto & config = getConfigRef(); + shared->asynchronous_remote_fs_reader = createThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER, config); + shared->asynchronous_local_fs_reader = createThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER, config); + shared->synchronous_local_fs_reader = createThreadPoolReader(FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER, config); + }); switch (type) { @@ -5141,22 +4912,14 @@ IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) co ThreadPool & Context::getThreadPoolWriter() const { - callOnce( - shared->threadpool_writer_initialized, - [&] - { - const auto & config = getConfigRef(); - auto pool_size = config.getUInt(".threadpool_writer_pool_size", 100); - auto queue_size = config.getUInt(".threadpool_writer_queue_size", 1000000); + callOnce(shared->threadpool_writer_initialized, [&] { + const auto & config = getConfigRef(); + auto pool_size = config.getUInt(".threadpool_writer_pool_size", 100); + auto queue_size = config.getUInt(".threadpool_writer_queue_size", 1000000); - shared->threadpool_writer = std::make_unique( - CurrentMetrics::IOWriterThreads, - CurrentMetrics::IOWriterThreadsActive, - CurrentMetrics::IOWriterThreadsScheduled, - pool_size, - pool_size, - queue_size); - }); + shared->threadpool_writer = std::make_unique( + CurrentMetrics::IOWriterThreads, CurrentMetrics::IOWriterThreadsActive, CurrentMetrics::IOWriterThreadsScheduled, pool_size, pool_size, queue_size); + }); return *shared->threadpool_writer; } @@ -5200,7 +4963,10 @@ ReadSettings Context::getReadSettings() const /// Zero read buffer will not make progress. if (!settings.max_read_buffer_size) - throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Invalid value '{}' for max_read_buffer_size", settings.max_read_buffer_size); + { + throw Exception(ErrorCodes::INVALID_SETTING_VALUE, + "Invalid value '{}' for max_read_buffer_size", settings.max_read_buffer_size); + } res.local_fs_buffer_size = settings.max_read_buffer_size_local_fs ? settings.max_read_buffer_size_local_fs : settings.max_read_buffer_size; diff --git a/tests/queries/0_stateless/02931_max_num_to_warn.sql b/tests/queries/0_stateless/02931_max_num_to_warn.sql index 2e357a6080c..f8043e9d2a5 100644 --- a/tests/queries/0_stateless/02931_max_num_to_warn.sql +++ b/tests/queries/0_stateless/02931_max_num_to_warn.sql @@ -1,14 +1,15 @@ -CREATE TABLE test_max_num_to_warn_1 (id Int32, str String) Engine=Memory; -CREATE TABLE test_max_num_to_warn_2 (id Int32, str String) Engine=Memory; -CREATE TABLE test_max_num_to_warn_3 (id Int32, str String) Engine=Memory; -CREATE TABLE test_max_num_to_warn_4 (id Int32, str String) Engine=Memory; -CREATE TABLE test_max_num_to_warn_5 (id Int32, str String) Engine=Memory; -CREATE TABLE test_max_num_to_warn_6 (id Int32, str String) Engine=Memory; -CREATE TABLE test_max_num_to_warn_7 (id Int32, str String) Engine=Memory; -CREATE TABLE test_max_num_to_warn_8 (id Int32, str String) Engine=Memory; -CREATE TABLE test_max_num_to_warn_9 (id Int32, str String) Engine=Memory; -CREATE TABLE test_max_num_to_warn_10 (id Int32, str String) Engine=Memory; -CREATE TABLE test_max_num_to_warn_11 (id Int32, str String) Engine=Memory; +CREATE DATABASE test_max_num_to_warn_02931; +CREATE TABLE test_max_num_to_warn_02931.test_max_num_to_warn_1 (id Int32, str String) Engine=Memory; +CREATE TABLE test_max_num_to_warn_02931.test_max_num_to_warn_2 (id Int32, str String) Engine=Memory; +CREATE TABLE test_max_num_to_warn_02931.test_max_num_to_warn_3 (id Int32, str String) Engine=Memory; +CREATE TABLE test_max_num_to_warn_02931.test_max_num_to_warn_4 (id Int32, str String) Engine=Memory; +CREATE TABLE test_max_num_to_warn_02931.test_max_num_to_warn_5 (id Int32, str String) Engine=Memory; +CREATE TABLE test_max_num_to_warn_02931.test_max_num_to_warn_6 (id Int32, str String) Engine=Memory; +CREATE TABLE test_max_num_to_warn_02931.test_max_num_to_warn_7 (id Int32, str String) Engine=Memory; +CREATE TABLE test_max_num_to_warn_02931.test_max_num_to_warn_8 (id Int32, str String) Engine=Memory; +CREATE TABLE test_max_num_to_warn_02931.test_max_num_to_warn_9 (id Int32, str String) Engine=Memory; +CREATE TABLE test_max_num_to_warn_02931.test_max_num_to_warn_10 (id Int32, str String) Engine=Memory; +CREATE TABLE test_max_num_to_warn_02931.test_max_num_to_warn_11 (id Int32, str String) Engine=Memory; CREATE DATABASE test_max_num_to_warn_1; CREATE DATABASE test_max_num_to_warn_2; @@ -22,32 +23,21 @@ CREATE DATABASE test_max_num_to_warn_9; CREATE DATABASE test_max_num_to_warn_10; CREATE DATABASE test_max_num_to_warn_11; -INSERT INTO test_max_num_to_warn_1 VALUES (1, 'Hello'); -INSERT INTO test_max_num_to_warn_2 VALUES (1, 'Hello'); -INSERT INTO test_max_num_to_warn_3 VALUES (1, 'Hello'); -INSERT INTO test_max_num_to_warn_4 VALUES (1, 'Hello'); -INSERT INTO test_max_num_to_warn_5 VALUES (1, 'Hello'); -INSERT INTO test_max_num_to_warn_6 VALUES (1, 'Hello'); -INSERT INTO test_max_num_to_warn_7 VALUES (1, 'Hello'); -INSERT INTO test_max_num_to_warn_8 VALUES (1, 'Hello'); -INSERT INTO test_max_num_to_warn_9 VALUES (1, 'Hello'); -INSERT INTO test_max_num_to_warn_10 VALUES (1, 'Hello'); -INSERT INTO test_max_num_to_warn_11 VALUES (1, 'Hello'); +INSERT INTO test_max_num_to_warn_02931.test_max_num_to_warn_1 VALUES (1, 'Hello'); +INSERT INTO test_max_num_to_warn_02931.test_max_num_to_warn_2 VALUES (1, 'Hello'); +INSERT INTO test_max_num_to_warn_02931.test_max_num_to_warn_3 VALUES (1, 'Hello'); +INSERT INTO test_max_num_to_warn_02931.test_max_num_to_warn_4 VALUES (1, 'Hello'); +INSERT INTO test_max_num_to_warn_02931.test_max_num_to_warn_5 VALUES (1, 'Hello'); +INSERT INTO test_max_num_to_warn_02931.test_max_num_to_warn_6 VALUES (1, 'Hello'); +INSERT INTO test_max_num_to_warn_02931.test_max_num_to_warn_7 VALUES (1, 'Hello'); +INSERT INTO test_max_num_to_warn_02931.test_max_num_to_warn_8 VALUES (1, 'Hello'); +INSERT INTO test_max_num_to_warn_02931.test_max_num_to_warn_9 VALUES (1, 'Hello'); +INSERT INTO test_max_num_to_warn_02931.test_max_num_to_warn_10 VALUES (1, 'Hello'); +INSERT INTO test_max_num_to_warn_02931.test_max_num_to_warn_11 VALUES (1, 'Hello'); SELECT * FROM system.warnings where message in ('The number of attached tables is more than 10', 'The number of attached databases is more than 10', 'The number of active parts is more than 10'); -DROP TABLE test_max_num_to_warn_1; -DROP TABLE test_max_num_to_warn_2; -DROP TABLE test_max_num_to_warn_3; -DROP TABLE test_max_num_to_warn_4; -DROP TABLE test_max_num_to_warn_5; -DROP TABLE test_max_num_to_warn_6; -DROP TABLE test_max_num_to_warn_7; -DROP TABLE test_max_num_to_warn_8; -DROP TABLE test_max_num_to_warn_9; -DROP TABLE test_max_num_to_warn_10; -DROP TABLE test_max_num_to_warn_11; - +DROP DATABASE test_max_num_to_warn_02931; DROP DATABASE test_max_num_to_warn_1; DROP DATABASE test_max_num_to_warn_2; DROP DATABASE test_max_num_to_warn_3; From ca790b6eecade9f23a7cbbfaf755332182e263ce Mon Sep 17 00:00:00 2001 From: Ryan Jacobs Date: Wed, 6 Dec 2023 19:26:50 -0800 Subject: [PATCH 128/331] apply python style check --- .../test_storage_postgresql/test.py | 37 +++++++++++++------ 1 file changed, 25 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 8e1be600687..a1b13739b5b 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -202,16 +202,19 @@ def test_postgres_conversions(started_cluster): cursor.execute(f"DROP TABLE test_types") cursor.execute(f"DROP TABLE test_array_dimensions") + def test_postgres_array_ndim_error_messges(started_cluster): cursor = started_cluster.postgres_conn.cursor() # cleanup - cursor.execute('DROP VIEW IF EXISTS array_ndim_view;') - cursor.execute('DROP TABLE IF EXISTS array_ndim_table;') + cursor.execute("DROP VIEW IF EXISTS array_ndim_view;") + cursor.execute("DROP TABLE IF EXISTS array_ndim_table;") # setup - cursor.execute('CREATE TABLE array_ndim_table (x INTEGER, "Mixed-case with spaces" INTEGER[]);') - cursor.execute('CREATE VIEW array_ndim_view AS SELECT * FROM array_ndim_table;') + cursor.execute( + 'CREATE TABLE array_ndim_table (x INTEGER, "Mixed-case with spaces" INTEGER[]);' + ) + cursor.execute("CREATE VIEW array_ndim_view AS SELECT * FROM array_ndim_table;") describe_table = """ DESCRIBE TABLE postgresql( 'postgres1:5432', 'postgres', 'array_ndim_view', @@ -225,30 +228,40 @@ def test_postgres_array_ndim_error_messges(started_cluster): node1.query(describe_table) assert False except Exception as error: - assert ('PostgreSQL relation containing arrays cannot be empty: array_ndim_view' in str(error)) + assert ( + "PostgreSQL relation containing arrays cannot be empty: array_ndim_view" + in str(error) + ) # View cannot have empty array. Should throw useful error message. # (Cannot infer array dimension.) - cursor.execute('TRUNCATE array_ndim_table;') + cursor.execute("TRUNCATE array_ndim_table;") cursor.execute("INSERT INTO array_ndim_table VALUES (1234, '{}');") try: node1.query(describe_table) assert False except Exception as error: - assert ('PostgreSQL cannot infer dimensions of an empty array: array_ndim_view."Mixed-case with spaces"' in str(error)) + assert ( + 'PostgreSQL cannot infer dimensions of an empty array: array_ndim_view."Mixed-case with spaces"' + in str(error) + ) # View cannot have NULL array value. Should throw useful error message. - cursor.execute('TRUNCATE array_ndim_table;') - cursor.execute('INSERT INTO array_ndim_table VALUES (1234, NULL);') + cursor.execute("TRUNCATE array_ndim_table;") + cursor.execute("INSERT INTO array_ndim_table VALUES (1234, NULL);") try: node1.query(describe_table) assert False except Exception as error: - assert ('PostgreSQL array cannot be NULL: array_ndim_view."Mixed-case with spaces"' in str(error)) + assert ( + 'PostgreSQL array cannot be NULL: array_ndim_view."Mixed-case with spaces"' + in str(error) + ) # cleanup - cursor.execute('DROP VIEW IF EXISTS array_ndim_view;') - cursor.execute('DROP TABLE IF EXISTS array_ndim_table;') + cursor.execute("DROP VIEW IF EXISTS array_ndim_view;") + cursor.execute("DROP TABLE IF EXISTS array_ndim_table;") + def test_non_default_schema(started_cluster): node1.query("DROP TABLE IF EXISTS test_pg_table_schema") 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 129/331] 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 130/331] 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 131/331] 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 132/331] 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 133/331] 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 7205e4edbeee134dde63451883c21c2f67cf5e01 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 7 Dec 2023 18:47:14 +0100 Subject: [PATCH 134/331] Review fix --- src/Interpreters/Cache/FileCache.cpp | 21 +++++++++++++++++++ src/Interpreters/Cache/FileCache.h | 21 +------------------ .../Cache/SLRUFileCachePriority.cpp | 2 +- 3 files changed, 23 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index f1b3d24dbc1..29f2467bd12 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -51,6 +51,27 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +void FileCacheReserveStat::update(size_t size, FileSegmentKind kind, bool releasable) +{ + auto & local_stat = stat_by_kind[kind]; + if (releasable) + { + stat.releasable_size += size; + ++stat.releasable_count; + + local_stat.releasable_size += size; + ++local_stat.releasable_count; + } + else + { + stat.non_releasable_size += size; + ++stat.non_releasable_count; + + local_stat.non_releasable_size += size; + ++local_stat.non_releasable_count; + } +} + FileCache::FileCache(const std::string & cache_name, const FileCacheSettings & settings) : max_file_segment_size(settings.max_file_segment_size) , bypass_cache_threshold(settings.enable_bypass_cache_with_threshold ? settings.bypass_cache_threshold : 0) diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 097a63b0abe..9cd2232bf7e 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -38,26 +38,7 @@ struct FileCacheReserveStat Stat stat; std::unordered_map stat_by_kind; - void update(size_t size, FileSegmentKind kind, bool releasable) - { - auto & local_stat = stat_by_kind[kind]; - if (releasable) - { - stat.releasable_size += size; - ++stat.releasable_count; - - local_stat.releasable_size += size; - ++local_stat.releasable_count; - } - else - { - stat.non_releasable_size += size; - ++stat.non_releasable_count; - - local_stat.non_releasable_size += size; - ++local_stat.non_releasable_count; - } - } + void update(size_t size, FileSegmentKind kind, bool releasable); }; /// Local cache for remote filesystem files, represented as a set of non-overlapping non-empty file segments. diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index 8f7f5e4aa32..75987bccf6a 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -14,7 +14,7 @@ namespace { size_t getRatio(size_t total, double ratio) { - return static_cast(total * std::max(0.0, std::min(1.0, ratio))); + return static_cast(total * std::clamp(ratio, 0.0, 1.0)); } } From ff65d0e72778c66947889f5675d61199845906a1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 7 Dec 2023 19:49:30 +0100 Subject: [PATCH 135/331] Adjustments after merge with master --- src/Interpreters/Cache/FileCache.cpp | 4 +- src/Interpreters/Cache/FileCache.h | 2 +- src/Interpreters/Cache/FileSegment.cpp | 2 +- src/Interpreters/Cache/FileSegment.h | 72 +----------------- src/Interpreters/Cache/FileSegmentInfo.h | 73 +++++++++++++++++++ src/Interpreters/Cache/IFileCachePriority.h | 11 ++- .../Cache/LRUFileCachePriority.cpp | 6 +- src/Interpreters/Cache/LRUFileCachePriority.h | 2 +- .../Cache/SLRUFileCachePriority.cpp | 12 ++- .../Cache/SLRUFileCachePriority.h | 2 +- src/Interpreters/Cache/test | 6 ++ src/Interpreters/tests/gtest_filecache.cpp | 57 +++++++++++---- 12 files changed, 152 insertions(+), 97 deletions(-) create mode 100644 src/Interpreters/Cache/FileSegmentInfo.h create mode 100644 src/Interpreters/Cache/test diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index e3d0e1585b2..1ca654f7b66 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -1173,10 +1173,10 @@ std::vector FileCache::getFileSegmentInfos(const Key & key) return file_segments; } -std::vector FileCache::dumpQueue() +IFileCachePriority::QueueEntriesDumps FileCache::dumpQueue() { assertInitialized(); - return main_priority->dump(lockCache()); + return main_priority->dump(*this, lockCache()); } std::vector FileCache::tryGetCachePaths(const Key & key) diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 0398383f662..02979b72b98 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -126,7 +126,7 @@ public: std::vector getFileSegmentInfos(const Key & key); - std::vector dumpQueue(); + IFileCachePriority::QueueEntriesDumps dumpQueue(); void deactivateBackgroundOperations(); diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index b9399595304..8d2776258ae 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -480,7 +480,7 @@ bool FileSegment::reserve(size_t size_to_reserve, FileCacheReserveStat * reserve bool is_file_segment_size_exceeded; { - auto lock = segment_guard.lock(); + auto lock = lockFileSegment(); assertNotDetachedUnlocked(lock); assertIsDownloaderUnlocked("reserve", lock); diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index bdeec44237c..1183878fa1c 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -11,8 +11,8 @@ #include #include #include +#include #include -#include namespace Poco { class Logger; } @@ -28,23 +28,6 @@ namespace DB class ReadBufferFromFileBase; struct FileCacheReserveStat; -/* - * FileSegmentKind is used to specify the eviction policy for file segments. - */ -enum class FileSegmentKind -{ - /* `Regular` file segment is still in cache after usage, and can be evicted - * (unless there're some holders). - */ - Regular, - - /* `Temporary` file segment is removed right after releasing. - * Also corresponding files are removed during cache loading (if any). - */ - Temporary, -}; - -String toString(FileSegmentKind kind); struct CreateFileSegmentSettings { @@ -69,40 +52,8 @@ public: using Downloader = std::string; using DownloaderId = std::string; using Priority = IFileCachePriority; - - enum class State - { - DOWNLOADED, - /** - * When file segment is first created and returned to user, it has state EMPTY. - * EMPTY state can become DOWNLOADING when getOrSetDownaloder is called successfully - * by any owner of EMPTY state file segment. - */ - EMPTY, - /** - * A newly created file segment never has DOWNLOADING state until call to getOrSetDownloader - * because each cache user might acquire multiple file segments and read them one by one, - * so only user which actually needs to read this segment earlier than others - becomes a downloader. - */ - DOWNLOADING, - /** - * Space reservation for a file segment is incremental, i.e. downloader reads buffer_size bytes - * from remote fs -> tries to reserve buffer_size bytes to put them to cache -> writes to cache - * on successful reservation and stops cache write otherwise. Those, who waited for the same file - * segment, will read downloaded part from cache and remaining part directly from remote fs. - */ - PARTIALLY_DOWNLOADED_NO_CONTINUATION, - /** - * If downloader did not finish download of current file segment for any reason apart from running - * out of cache space, then download can be continued by other owners of this file segment. - */ - PARTIALLY_DOWNLOADED, - /** - * If file segment cannot possibly be downloaded (first space reservation attempt failed), mark - * this file segment as out of cache scope. - */ - DETACHED, - }; + using State = FileSegmentState; + using Info = FileSegmentInfo; FileSegment( const Key & key_, @@ -205,22 +156,7 @@ public: /// exception. void detach(const FileSegmentGuard::Lock &, const LockedKey &); - struct Info - { - FileSegment::Key key; - size_t offset; - std::string path; - uint64_t range_left; - uint64_t range_right; - FileSegmentKind kind; - State state; - uint64_t size; - uint64_t downloaded_size; - uint64_t cache_hits; - uint64_t references; - bool is_unbound; - }; - static Info getInfo(const FileSegmentPtr & file_segment, FileCache & cache); + static FileSegmentInfo getInfo(const FileSegmentPtr & file_segment, FileCache & cache); bool isDetached() const; diff --git a/src/Interpreters/Cache/FileSegmentInfo.h b/src/Interpreters/Cache/FileSegmentInfo.h new file mode 100644 index 00000000000..c1a38e28b1d --- /dev/null +++ b/src/Interpreters/Cache/FileSegmentInfo.h @@ -0,0 +1,73 @@ +#pragma once +#include +#include + +namespace DB +{ + enum class FileSegmentState + { + DOWNLOADED, + /** + * When file segment is first created and returned to user, it has state EMPTY. + * EMPTY state can become DOWNLOADING when getOrSetDownaloder is called successfully + * by any owner of EMPTY state file segment. + */ + EMPTY, + /** + * A newly created file segment never has DOWNLOADING state until call to getOrSetDownloader + * because each cache user might acquire multiple file segments and read them one by one, + * so only user which actually needs to read this segment earlier than others - becomes a downloader. + */ + DOWNLOADING, + /** + * Space reservation for a file segment is incremental, i.e. downloader reads buffer_size bytes + * from remote fs -> tries to reserve buffer_size bytes to put them to cache -> writes to cache + * on successful reservation and stops cache write otherwise. Those, who waited for the same file + * segment, will read downloaded part from cache and remaining part directly from remote fs. + */ + PARTIALLY_DOWNLOADED_NO_CONTINUATION, + /** + * If downloader did not finish download of current file segment for any reason apart from running + * out of cache space, then download can be continued by other owners of this file segment. + */ + PARTIALLY_DOWNLOADED, + /** + * If file segment cannot possibly be downloaded (first space reservation attempt failed), mark + * this file segment as out of cache scope. + */ + DETACHED, + }; + + enum class FileSegmentKind + { + /** + * `Regular` file segment is still in cache after usage, and can be evicted + * (unless there're some holders). + */ + Regular, + + /** + * Temporary` file segment is removed right after releasing. + * Also corresponding files are removed during cache loading (if any). + */ + Temporary, + }; + + std::string toString(FileSegmentKind kind); + + struct FileSegmentInfo + { + FileCacheKey key; + size_t offset; + std::string path; + uint64_t range_left; + uint64_t range_right; + FileSegmentKind kind; + FileSegmentState state; + uint64_t size; + uint64_t downloaded_size; + uint64_t cache_hits; + uint64_t references; + bool is_unbound; + }; +} diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index e7a05b0e46d..dfc57328242 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -3,8 +3,9 @@ #include #include #include -#include +#include #include +#include #include namespace DB @@ -71,7 +72,13 @@ public: virtual void shuffle(const CacheGuard::Lock &) = 0; - virtual FileSegments dump(const CacheGuard::Lock &) = 0; + struct QueueEntryDump + { + FileSegmentInfo info; + bool is_protected = false; + }; + using QueueEntriesDumps = std::vector; + virtual QueueEntriesDumps dump(FileCache & cache, const CacheGuard::Lock &) = 0; using FinalizeEvictionFunc = std::function; virtual bool collectCandidatesForEviction( diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index a596d041941..ea0ce168913 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -277,12 +277,12 @@ LRUFileCachePriority::LRUIterator LRUFileCachePriority::move(LRUIterator & it, L return LRUIterator(this, it.iterator); } -FileSegments LRUFileCachePriority::dump(const CacheGuard::Lock & lock) +IFileCachePriority::QueueEntriesDumps LRUFileCachePriority::dump(FileCache & cache, const CacheGuard::Lock & lock) { - FileSegments res; + QueueEntriesDumps res; iterate([&](LockedKey &, const FileSegmentMetadataPtr & segment_metadata) { - res.push_back(FileSegment::getSnapshot(segment_metadata->file_segment)); + res.emplace_back(FileSegment::getInfo(segment_metadata->file_segment, cache)); return IterationResult::CONTINUE; }, lock); return res; diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 31a5c527ca8..a9d823313d9 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -44,7 +44,7 @@ public: void shuffle(const CacheGuard::Lock &) override; - FileSegments dump(const CacheGuard::Lock &) override; + QueueEntriesDumps dump(FileCache & cache, const CacheGuard::Lock &) override; void pop(const CacheGuard::Lock & lock) { remove(queue.begin(), lock); } diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index 75987bccf6a..88d90e4dede 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -222,10 +222,16 @@ void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const Cach iterator.is_protected = true; } -FileSegments SLRUFileCachePriority::dump(const CacheGuard::Lock & lock) +IFileCachePriority::QueueEntriesDumps SLRUFileCachePriority::dump(FileCache & cache, const CacheGuard::Lock & lock) { - auto res = probationary_queue.dump(lock); - auto part_res = protected_queue.dump(lock); + auto res = probationary_queue.dump(cache, lock); + for (auto & entry : res) + entry.is_protected = false; + + auto part_res = protected_queue.dump(cache, lock); + for (auto & entry : part_res) + entry.is_protected = true; + res.insert(res.end(), part_res.begin(), part_res.end()); return res; } diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index 7753f6f23b2..b08fce50f00 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -43,7 +43,7 @@ public: void shuffle(const CacheGuard::Lock &) override; - FileSegments dump(const CacheGuard::Lock &) override; + QueueEntriesDumps dump(FileCache & cache, const CacheGuard::Lock &) override; private: LRUFileCachePriority protected_queue; diff --git a/src/Interpreters/Cache/test b/src/Interpreters/Cache/test new file mode 100644 index 00000000000..5b1f397194b --- /dev/null +++ b/src/Interpreters/Cache/test @@ -0,0 +1,6 @@ +CREATE DICTIONARY dict (`id` String, `timestamp` DateTime) +PRIMARY KEY id SOURCE(CLICKHOUSE(QUERY 'SELECT \'test\' as id, now() as timestamp')) LAYOUT(DIRECT()); + +CREATE table t (id LowCardinality(String)) engine = MergeTree() ORDER BY id; + +CREATE VIEW v AS select dictGet(dict, 'timestamp', id) from t; diff --git a/src/Interpreters/tests/gtest_filecache.cpp b/src/Interpreters/tests/gtest_filecache.cpp index 15a7598bfd5..3bf7d1b9a14 100644 --- a/src/Interpreters/tests/gtest_filecache.cpp +++ b/src/Interpreters/tests/gtest_filecache.cpp @@ -142,22 +142,49 @@ void assertEqual(const std::vector & file_segments, const Ran } } -void assertProtectedOrProbationary(const FileSegments & file_segments, const Ranges & expected, bool assert_protected) +void assertEqual(const IFileCachePriority::QueueEntriesDumps & file_segments, const Ranges & expected_ranges, const States & expected_states = {}) +{ + std::cerr << "File segments: "; + for (const auto & f : file_segments) + { + auto range = FileSegment::Range(f.info.range_left, f.info.range_right); + std::cerr << range.toString() << ", "; + } + + ASSERT_EQ(file_segments.size(), expected_ranges.size()); + + if (!expected_states.empty()) + ASSERT_EQ(file_segments.size(), expected_states.size()); + + auto get_expected_state = [&](size_t i) + { + if (expected_states.empty()) + return State::DOWNLOADED; + else + return expected_states[i]; + }; + + size_t i = 0; + for (const auto & f : file_segments) + { + auto range = FileSegment::Range(f.info.range_left, f.info.range_right); + ASSERT_EQ(range, expected_ranges[i]); + ASSERT_EQ(f.info.state, get_expected_state(i)); + ++i; + } +} + +void assertProtectedOrProbationary(const IFileCachePriority::QueueEntriesDumps & file_segments, const Ranges & expected, bool assert_protected) { std::cerr << "File segments: "; std::vector res; for (const auto & f : file_segments) { - std::cerr << f->range().toString() << ", "; - if (auto it = f->getQueueIterator()) + auto range = FileSegment::Range(f.info.range_left, f.info.range_right); + std::cerr << range.toString() << ", "; + if ((f.is_protected && assert_protected) || (!f.is_protected && !assert_protected)) { - if (auto * slru_it = dynamic_cast(it.get())) - { - if ((slru_it->isProtected() && assert_protected) || (!slru_it->isProtected() && !assert_protected)) - { - res.push_back(f->range()); - } - } + res.push_back(range); } } @@ -168,12 +195,12 @@ void assertProtectedOrProbationary(const FileSegments & file_segments, const Ran } } -void assertProtected(const FileSegments & file_segments, const Ranges & expected) +void assertProtected(const IFileCachePriority::QueueEntriesDumps & file_segments, const Ranges & expected) { assertProtectedOrProbationary(file_segments, expected, true); } -void assertProbationary(const FileSegments & file_segments, const Ranges & expected) +void assertProbationary(const IFileCachePriority::QueueEntriesDumps & file_segments, const Ranges & expected) { assertProtectedOrProbationary(file_segments, expected, false); } @@ -1151,7 +1178,7 @@ TEST_F(FileCacheTest, SLRUPolicy) add_range(0, 10); add_range(10, 5); - assertEqual(cache.getSnapshot(key), { Range(0, 9), Range(10, 14) }); + assertEqual(cache.getFileSegmentInfos(key), { Range(0, 9), Range(10, 14) }); assertEqual(cache.dumpQueue(), { Range(0, 9), Range(10, 14) }); ASSERT_EQ(cache.getFileSegmentsNum(), 2); @@ -1181,7 +1208,7 @@ TEST_F(FileCacheTest, SLRUPolicy) assertProbationary(cache.dumpQueue(), { Range(17, 20), Range(24, 26), Range(27, 27) }); assertProtected(cache.dumpQueue(), { Range(0, 9), Range(10, 14) }); - assertEqual(cache.getSnapshot(key), { Range(0, 9), Range(10, 14), Range(17, 20), Range(24, 26), Range(27, 27) }); + assertEqual(cache.getFileSegmentInfos(key), { Range(0, 9), Range(10, 14), Range(17, 20), Range(24, 26), Range(27, 27) }); ASSERT_EQ(cache.getFileSegmentsNum(), 5); ASSERT_EQ(cache.getUsedCacheSize(), 23); @@ -1201,7 +1228,7 @@ TEST_F(FileCacheTest, SLRUPolicy) assertProbationary(cache.dumpQueue(), { Range(24, 26), Range(10, 14) }); assertProtected(cache.dumpQueue(), { Range(0, 9), Range(27, 27), Range(28, 30) }); - assertEqual(cache.getSnapshot(key), { Range(0, 9), Range(10, 14), Range(24, 26), Range(27, 27), Range(28, 30) }); + assertEqual(cache.getFileSegmentInfos(key), { Range(0, 9), Range(10, 14), Range(24, 26), Range(27, 27), Range(28, 30) }); ASSERT_EQ(cache.getFileSegmentsNum(), 5); ASSERT_EQ(cache.getUsedCacheSize(), 22); } From 5de99eceec87771c2441bc2042bf134feb10c048 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Dec 2023 21:38:59 +0100 Subject: [PATCH 136/331] 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 137/331] 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 138/331] 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 139/331] 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 140/331] 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 141/331] 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 142/331] 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 143/331] 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 144/331] 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 145/331] 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 146/331] 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 4616ecb53652495111db72479463ced149f280ad Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Fri, 8 Dec 2023 10:11:53 +0800 Subject: [PATCH 147/331] fix tests --- .../0_stateless/02931_max_num_to_warn.sql | 70 +++++++++---------- 1 file changed, 35 insertions(+), 35 deletions(-) diff --git a/tests/queries/0_stateless/02931_max_num_to_warn.sql b/tests/queries/0_stateless/02931_max_num_to_warn.sql index f8043e9d2a5..dfb6e4ceac5 100644 --- a/tests/queries/0_stateless/02931_max_num_to_warn.sql +++ b/tests/queries/0_stateless/02931_max_num_to_warn.sql @@ -1,27 +1,27 @@ -CREATE DATABASE test_max_num_to_warn_02931; -CREATE TABLE test_max_num_to_warn_02931.test_max_num_to_warn_1 (id Int32, str String) Engine=Memory; -CREATE TABLE test_max_num_to_warn_02931.test_max_num_to_warn_2 (id Int32, str String) Engine=Memory; -CREATE TABLE test_max_num_to_warn_02931.test_max_num_to_warn_3 (id Int32, str String) Engine=Memory; -CREATE TABLE test_max_num_to_warn_02931.test_max_num_to_warn_4 (id Int32, str String) Engine=Memory; -CREATE TABLE test_max_num_to_warn_02931.test_max_num_to_warn_5 (id Int32, str String) Engine=Memory; -CREATE TABLE test_max_num_to_warn_02931.test_max_num_to_warn_6 (id Int32, str String) Engine=Memory; -CREATE TABLE test_max_num_to_warn_02931.test_max_num_to_warn_7 (id Int32, str String) Engine=Memory; -CREATE TABLE test_max_num_to_warn_02931.test_max_num_to_warn_8 (id Int32, str String) Engine=Memory; -CREATE TABLE test_max_num_to_warn_02931.test_max_num_to_warn_9 (id Int32, str String) Engine=Memory; -CREATE TABLE test_max_num_to_warn_02931.test_max_num_to_warn_10 (id Int32, str String) Engine=Memory; -CREATE TABLE test_max_num_to_warn_02931.test_max_num_to_warn_11 (id Int32, str String) Engine=Memory; +CREATE DATABASE IF NOT EXISTS test_max_num_to_warn_02931; +CREATE TABLE IF NOT EXISTS test_max_num_to_warn_02931.test_max_num_to_warn_1 (id Int32, str String) Engine=Memory; +CREATE TABLE IF NOT EXISTS test_max_num_to_warn_02931.test_max_num_to_warn_2 (id Int32, str String) Engine=Memory; +CREATE TABLE IF NOT EXISTS test_max_num_to_warn_02931.test_max_num_to_warn_3 (id Int32, str String) Engine=Memory; +CREATE TABLE IF NOT EXISTS test_max_num_to_warn_02931.test_max_num_to_warn_4 (id Int32, str String) Engine=Memory; +CREATE TABLE IF NOT EXISTS test_max_num_to_warn_02931.test_max_num_to_warn_5 (id Int32, str String) Engine=Memory; +CREATE TABLE IF NOT EXISTS test_max_num_to_warn_02931.test_max_num_to_warn_6 (id Int32, str String) Engine=Memory; +CREATE TABLE IF NOT EXISTS test_max_num_to_warn_02931.test_max_num_to_warn_7 (id Int32, str String) Engine=Memory; +CREATE TABLE IF NOT EXISTS test_max_num_to_warn_02931.test_max_num_to_warn_8 (id Int32, str String) Engine=Memory; +CREATE TABLE IF NOT EXISTS test_max_num_to_warn_02931.test_max_num_to_warn_9 (id Int32, str String) Engine=Memory; +CREATE TABLE IF NOT EXISTS test_max_num_to_warn_02931.test_max_num_to_warn_10 (id Int32, str String) Engine=Memory; +CREATE TABLE IF NOT EXISTS test_max_num_to_warn_02931.test_max_num_to_warn_11 (id Int32, str String) Engine=Memory; -CREATE DATABASE test_max_num_to_warn_1; -CREATE DATABASE test_max_num_to_warn_2; -CREATE DATABASE test_max_num_to_warn_3; -CREATE DATABASE test_max_num_to_warn_4; -CREATE DATABASE test_max_num_to_warn_5; -CREATE DATABASE test_max_num_to_warn_6; -CREATE DATABASE test_max_num_to_warn_7; -CREATE DATABASE test_max_num_to_warn_8; -CREATE DATABASE test_max_num_to_warn_9; -CREATE DATABASE test_max_num_to_warn_10; -CREATE DATABASE test_max_num_to_warn_11; +CREATE DATABASE IF NOT EXISTS test_max_num_to_warn_1; +CREATE DATABASE IF NOT EXISTS test_max_num_to_warn_2; +CREATE DATABASE IF NOT EXISTS test_max_num_to_warn_3; +CREATE DATABASE IF NOT EXISTS test_max_num_to_warn_4; +CREATE DATABASE IF NOT EXISTS test_max_num_to_warn_5; +CREATE DATABASE IF NOT EXISTS test_max_num_to_warn_6; +CREATE DATABASE IF NOT EXISTS test_max_num_to_warn_7; +CREATE DATABASE IF NOT EXISTS test_max_num_to_warn_8; +CREATE DATABASE IF NOT EXISTS test_max_num_to_warn_9; +CREATE DATABASE IF NOT EXISTS test_max_num_to_warn_10; +CREATE DATABASE IF NOT EXISTS test_max_num_to_warn_11; INSERT INTO test_max_num_to_warn_02931.test_max_num_to_warn_1 VALUES (1, 'Hello'); INSERT INTO test_max_num_to_warn_02931.test_max_num_to_warn_2 VALUES (1, 'Hello'); @@ -37,15 +37,15 @@ INSERT INTO test_max_num_to_warn_02931.test_max_num_to_warn_11 VALUES (1, 'Hello SELECT * FROM system.warnings where message in ('The number of attached tables is more than 10', 'The number of attached databases is more than 10', 'The number of active parts is more than 10'); -DROP DATABASE test_max_num_to_warn_02931; -DROP DATABASE test_max_num_to_warn_1; -DROP DATABASE test_max_num_to_warn_2; -DROP DATABASE test_max_num_to_warn_3; -DROP DATABASE test_max_num_to_warn_4; -DROP DATABASE test_max_num_to_warn_5; -DROP DATABASE test_max_num_to_warn_6; -DROP DATABASE test_max_num_to_warn_7; -DROP DATABASE test_max_num_to_warn_8; -DROP DATABASE test_max_num_to_warn_9; -DROP DATABASE test_max_num_to_warn_10; -DROP DATABASE test_max_num_to_warn_11; +DROP DATABASE IF EXISTS test_max_num_to_warn_02931; +DROP DATABASE IF EXISTS test_max_num_to_warn_1; +DROP DATABASE IF EXISTS test_max_num_to_warn_2; +DROP DATABASE IF EXISTS test_max_num_to_warn_3; +DROP DATABASE IF EXISTS test_max_num_to_warn_4; +DROP DATABASE IF EXISTS test_max_num_to_warn_5; +DROP DATABASE IF EXISTS test_max_num_to_warn_6; +DROP DATABASE IF EXISTS test_max_num_to_warn_7; +DROP DATABASE IF EXISTS test_max_num_to_warn_8; +DROP DATABASE IF EXISTS test_max_num_to_warn_9; +DROP DATABASE IF EXISTS test_max_num_to_warn_10; +DROP DATABASE IF EXISTS test_max_num_to_warn_11; From 01135211e9134b2a4ce02b6209affa3bcc64a81e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Dec 2023 03:12:54 +0100 Subject: [PATCH 148/331] 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 149/331] 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 150/331] 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 151/331] 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 152/331] 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 153/331] 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 154/331] 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 f2ef3ed5c9750fd8661455d7b9fd31370df038ba Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Fri, 8 Dec 2023 14:22:33 +0800 Subject: [PATCH 155/331] fix tests --- tests/queries/0_stateless/02931_max_num_to_warn.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02931_max_num_to_warn.sql b/tests/queries/0_stateless/02931_max_num_to_warn.sql index dfb6e4ceac5..49b981fc355 100644 --- a/tests/queries/0_stateless/02931_max_num_to_warn.sql +++ b/tests/queries/0_stateless/02931_max_num_to_warn.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel + CREATE DATABASE IF NOT EXISTS test_max_num_to_warn_02931; CREATE TABLE IF NOT EXISTS test_max_num_to_warn_02931.test_max_num_to_warn_1 (id Int32, str String) Engine=Memory; CREATE TABLE IF NOT EXISTS test_max_num_to_warn_02931.test_max_num_to_warn_2 (id Int32, str String) Engine=Memory; From 7598ae1c041457c4040c17d5d2ed63f0f1bdde7e Mon Sep 17 00:00:00 2001 From: Bharat Nallan Chakravarthy Date: Thu, 7 Dec 2023 23:53:08 -0800 Subject: [PATCH 156/331] 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 157/331] 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 158/331] 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 d77938c3ff86175a6e10025aea72452bc40a5b90 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 8 Dec 2023 12:42:38 +0100 Subject: [PATCH 159/331] Add validation of columns --- .../MaterializedPostgreSQLConsumer.cpp | 65 +++++++++++++++++-- .../MaterializedPostgreSQLConsumer.h | 1 + 2 files changed, 61 insertions(+), 5 deletions(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index 6be1563d16c..d369b8f3788 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -79,6 +79,7 @@ MaterializedPostgreSQLConsumer::StorageData::StorageData(const StorageInfo & sto : storage(storage_info.storage) , table_description(storage_info.storage->getInMemoryMetadataPtr()->getSampleBlock()) , columns_attributes(storage_info.attributes) + , column_names(storage_info.storage->getInMemoryMetadataPtr()->getColumns().getNamesOfPhysical()) , array_info(createArrayInfos(storage_info.storage->getInMemoryMetadataPtr()->getColumns().getAllPhysical(), table_description)) { auto columns_num = table_description.sample_block.columns(); @@ -548,34 +549,88 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl return; } - Int16 num_columns = readInt16(replication_message, pos, size); + auto log_table_structure_changed = [&](std::string_view reason) + { + LOG_INFO(log, "Table structure of the table {} changed ({}), " + "will mark it as skipped from replication. " + "Please perform manual DETACH and ATTACH of the table to bring it back", + table_name, reason); + }; - Int32 data_type_id; - Int32 type_modifier; /// For example, n in varchar(n) + Int16 num_columns = readInt16(replication_message, pos, size); auto & storage_data = storage_iter->second; const auto & description = storage_data.table_description; + const size_t actual_columns_num = storage_data.getColumnsNum(); + if (size_t(num_columns) > actual_columns_num - 2) + { + log_table_structure_changed(fmt::format("received {} columns, expected {}", num_columns, actual_columns_num - 2)); + markTableAsSkipped(relation_id, table_name); + return; + } + + Int32 data_type_id; + Int32 type_modifier; /// For example, n in varchar(n) + + std::set all_columns(storage_data.column_names.begin(), storage_data.column_names.end()); + std::set received_columns; ColumnsWithTypeAndName columns; + for (uint16_t i = 0; i < num_columns; ++i) { String column_name; readInt8(replication_message, pos, size); /// Marks column as part of replica identity index readString(replication_message, pos, size, column_name); + if (!all_columns.contains(column_name)) + { + log_table_structure_changed(fmt::format("column {} is not known", column_name)); + markTableAsSkipped(relation_id, table_name); + return; + } + data_type_id = readInt32(replication_message, pos, size); type_modifier = readInt32(replication_message, pos, size); columns.push_back(description.sample_block.getByName(column_name)); + received_columns.emplace(column_name); const auto & attributes_it = storage_data.columns_attributes.find(column_name); if (attributes_it == storage_data.columns_attributes.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown column: {}", column_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "No column {} in attributes", column_name); const auto & attributes = attributes_it->second; if (attributes.atttypid != data_type_id || attributes.atttypmod != type_modifier) { - LOG_TEST(log, "Column {} has a different type", column_name); + log_table_structure_changed(fmt::format("column {} has a different type", column_name)); + markTableAsSkipped(relation_id, table_name); + return; + } + } + + + if (size_t(num_columns) < actual_columns_num) + { + std::vector absent_columns; + std::set_difference( + all_columns.begin(), all_columns.end(), + received_columns.begin(), received_columns.end(), std::back_inserter(absent_columns)); + + for (const auto & name : absent_columns) + { + if (name == "_sign" || name == "_version") + continue; + + const auto & attributes_it = storage_data.columns_attributes.find(name); + if (attributes_it == storage_data.columns_attributes.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No column {} in attributes", name); + + /// Column has a default value or it is a GENERATED columns. + if (!attributes_it->second.attr_def.empty()) + continue; + + log_table_structure_changed(fmt::format("column {} was not found", name)); markTableAsSkipped(relation_id, table_name); return; } diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h index d29236b8123..1614cb4bdbc 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h @@ -43,6 +43,7 @@ private: const StoragePtr storage; const ExternalResultDescription table_description; const PostgreSQLTableStructure::Attributes columns_attributes; + const Names column_names; const ArrayInfo array_info; struct Buffer From 0c40465fec34d4e3af6560332f7e171d3b1b666b Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 8 Dec 2023 12:46:59 +0100 Subject: [PATCH 160/331] Test --- .../test.py | 94 +++++++++++++++++++ 1 file changed, 94 insertions(+) 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 965cca54a23..ddfb5608336 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -915,6 +915,100 @@ def test_failed_load_from_snapshot(started_cluster): ) +def test_generated_columns(started_cluster): + table = "test_generated_columns" + + pg_manager.create_postgres_table( + table, + "", + f"""CREATE TABLE {table} ( + key integer PRIMARY KEY, + x integer, + y integer GENERATED ALWAYS AS (x*2) STORED, + z text); + """, + ) + + pg_manager.execute(f"insert into {table} (key, x, z) values (1,1,'1');") + pg_manager.execute(f"insert into {table} (key, x, z) values (2,2,'2');") + + pg_manager.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_manager.get_default_database() + ) + + pg_manager.execute(f"insert into {table} (key, x, z) values (3,3,'3');") + pg_manager.execute(f"insert into {table} (key, x, z) values (4,4,'4');") + + check_tables_are_synchronized( + instance, table, postgres_database=pg_manager.get_default_database() + ) + + pg_manager.execute(f"insert into {table} (key, x, z) values (5,5,'5');") + pg_manager.execute(f"insert into {table} (key, x, z) values (6,6,'6');") + + check_tables_are_synchronized( + instance, table, postgres_database=pg_manager.get_default_database() + ) + + +def test_default_columns(started_cluster): + table = "test_default_columns" + + pg_manager.create_postgres_table( + table, + "", + f"""CREATE TABLE {table} ( + key integer PRIMARY KEY, + x integer, + y text DEFAULT 'y1', + z integer, + a text DEFAULT 'a1', + b integer); + """, + ) + + pg_manager.execute(f"insert into {table} (key, x, z, b) values (1,1,1,1);") + pg_manager.execute(f"insert into {table} (key, x, z, b) values (2,2,2,2);") + + pg_manager.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_manager.get_default_database() + ) + + pg_manager.execute(f"insert into {table} (key, x, z, b) values (3,3,3,3);") + pg_manager.execute(f"insert into {table} (key, x, z, b) values (4,4,4,4);") + + check_tables_are_synchronized( + instance, table, postgres_database=pg_manager.get_default_database() + ) + + pg_manager.execute(f"insert into {table} (key, x, z, b) values (5,5,5,5);") + pg_manager.execute(f"insert into {table} (key, x, z, b) values (6,6,6,6);") + + check_tables_are_synchronized( + instance, table, postgres_database=pg_manager.get_default_database() + ) + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From ea63819c76b5884f63573a408720a1a862999759 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 8 Dec 2023 15:21:42 +0100 Subject: [PATCH 161/331] Better --- src/Interpreters/Cache/FileCache.cpp | 2 +- src/Interpreters/Cache/FileCache.h | 2 +- src/Interpreters/Cache/FileSegment.cpp | 5 +- src/Interpreters/Cache/FileSegment.h | 1 + src/Interpreters/Cache/FileSegmentInfo.h | 9 + src/Interpreters/Cache/IFileCachePriority.h | 11 +- .../Cache/LRUFileCachePriority.cpp | 4 +- src/Interpreters/Cache/LRUFileCachePriority.h | 4 +- .../Cache/SLRUFileCachePriority.cpp | 8 +- .../Cache/SLRUFileCachePriority.h | 4 +- src/Interpreters/Cache/test | 6 - src/Interpreters/tests/gtest_filecache.cpp | 167 +++++++++++++----- 12 files changed, 151 insertions(+), 72 deletions(-) delete mode 100644 src/Interpreters/Cache/test diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 1ca654f7b66..23768102616 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -1173,7 +1173,7 @@ std::vector FileCache::getFileSegmentInfos(const Key & key) return file_segments; } -IFileCachePriority::QueueEntriesDumps FileCache::dumpQueue() +std::vector FileCache::dumpQueue() { assertInitialized(); return main_priority->dump(*this, lockCache()); diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 02979b72b98..0398383f662 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -126,7 +126,7 @@ public: std::vector getFileSegmentInfos(const Key & key); - IFileCachePriority::QueueEntriesDumps dumpQueue(); + std::vector dumpQueue(); void deactivateBackgroundOperations(); diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 8d2776258ae..9d4c79c96fb 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -849,6 +849,7 @@ FileSegment::Info FileSegment::getInfo(const FileSegmentPtr & file_segment, File .cache_hits = file_segment->hits_count, .references = static_cast(file_segment.use_count()), .is_unbound = file_segment->is_unbound, + .queue_entry_type = file_segment->queue_iterator ? file_segment->queue_iterator->getType() : QueueEntryType::None, }; } @@ -914,10 +915,6 @@ void FileSegment::increasePriority() return; } - /// Priority can be increased only for downloaded file segments. - if (download_state != State::DOWNLOADED) - return; - auto it = getQueueIterator(); if (it) { diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 1183878fa1c..3cd5604c9bf 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -54,6 +54,7 @@ public: using Priority = IFileCachePriority; using State = FileSegmentState; using Info = FileSegmentInfo; + using QueueEntryType = FileCacheQueueEntryType; FileSegment( const Key & key_, diff --git a/src/Interpreters/Cache/FileSegmentInfo.h b/src/Interpreters/Cache/FileSegmentInfo.h index c1a38e28b1d..bb87cbbc15d 100644 --- a/src/Interpreters/Cache/FileSegmentInfo.h +++ b/src/Interpreters/Cache/FileSegmentInfo.h @@ -53,6 +53,14 @@ namespace DB Temporary, }; + enum class FileCacheQueueEntryType + { + None, + LRU, + SLRU_Protected, + SLRU_Probationary, + }; + std::string toString(FileSegmentKind kind); struct FileSegmentInfo @@ -69,5 +77,6 @@ namespace DB uint64_t cache_hits; uint64_t references; bool is_unbound; + FileCacheQueueEntryType queue_entry_type; }; } diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index dfc57328242..0f407a3082c 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -17,6 +17,7 @@ class IFileCachePriority : private boost::noncopyable { public: using Key = FileCacheKey; + using QueueEntryType = FileCacheQueueEntryType; struct Entry { @@ -45,6 +46,8 @@ public: virtual void remove(const CacheGuard::Lock &) = 0; virtual void invalidate() = 0; + + virtual QueueEntryType getType() const = 0; }; using IteratorPtr = std::shared_ptr; @@ -72,13 +75,7 @@ public: virtual void shuffle(const CacheGuard::Lock &) = 0; - struct QueueEntryDump - { - FileSegmentInfo info; - bool is_protected = false; - }; - using QueueEntriesDumps = std::vector; - virtual QueueEntriesDumps dump(FileCache & cache, const CacheGuard::Lock &) = 0; + virtual std::vector dump(FileCache & cache, const CacheGuard::Lock &) = 0; using FinalizeEvictionFunc = std::function; virtual bool collectCandidatesForEviction( diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index ea0ce168913..a6abaea11c3 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -277,9 +277,9 @@ LRUFileCachePriority::LRUIterator LRUFileCachePriority::move(LRUIterator & it, L return LRUIterator(this, it.iterator); } -IFileCachePriority::QueueEntriesDumps LRUFileCachePriority::dump(FileCache & cache, const CacheGuard::Lock & lock) +std::vector LRUFileCachePriority::dump(FileCache & cache, const CacheGuard::Lock & lock) { - QueueEntriesDumps res; + std::vector res; iterate([&](LockedKey &, const FileSegmentMetadataPtr & segment_metadata) { res.emplace_back(FileSegment::getInfo(segment_metadata->file_segment, cache)); diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index a9d823313d9..5ff6c61eb4d 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -44,7 +44,7 @@ public: void shuffle(const CacheGuard::Lock &) override; - QueueEntriesDumps dump(FileCache & cache, const CacheGuard::Lock &) override; + std::vector dump(FileCache & cache, const CacheGuard::Lock &) override; void pop(const CacheGuard::Lock & lock) { remove(queue.begin(), lock); } @@ -99,6 +99,8 @@ public: void updateSize(int64_t size) override; + QueueEntryType getType() const override { return QueueEntryType::LRU; } + private: void assertValid() const; diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index 88d90e4dede..7b3e666f595 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -222,16 +222,10 @@ void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const Cach iterator.is_protected = true; } -IFileCachePriority::QueueEntriesDumps SLRUFileCachePriority::dump(FileCache & cache, const CacheGuard::Lock & lock) +std::vector SLRUFileCachePriority::dump(FileCache & cache, const CacheGuard::Lock & lock) { auto res = probationary_queue.dump(cache, lock); - for (auto & entry : res) - entry.is_protected = false; - auto part_res = protected_queue.dump(cache, lock); - for (auto & entry : part_res) - entry.is_protected = true; - res.insert(res.end(), part_res.begin(), part_res.end()); return res; } diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index b08fce50f00..45fc7ad8333 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -43,7 +43,7 @@ public: void shuffle(const CacheGuard::Lock &) override; - QueueEntriesDumps dump(FileCache & cache, const CacheGuard::Lock &) override; + std::vector dump(FileCache & cache, const CacheGuard::Lock &) override; private: LRUFileCachePriority protected_queue; @@ -72,7 +72,7 @@ public: void updateSize(int64_t size) override; - bool isProtected() const { return is_protected; } + QueueEntryType getType() const override { return is_protected ? QueueEntryType::SLRU_Protected : QueueEntryType::SLRU_Probationary; } private: void assertValid() const; diff --git a/src/Interpreters/Cache/test b/src/Interpreters/Cache/test deleted file mode 100644 index 5b1f397194b..00000000000 --- a/src/Interpreters/Cache/test +++ /dev/null @@ -1,6 +0,0 @@ -CREATE DICTIONARY dict (`id` String, `timestamp` DateTime) -PRIMARY KEY id SOURCE(CLICKHOUSE(QUERY 'SELECT \'test\' as id, now() as timestamp')) LAYOUT(DIRECT()); - -CREATE table t (id LowCardinality(String)) engine = MergeTree() ORDER BY id; - -CREATE VIEW v AS select dictGet(dict, 'timestamp', id) from t; diff --git a/src/Interpreters/tests/gtest_filecache.cpp b/src/Interpreters/tests/gtest_filecache.cpp index 3bf7d1b9a14..1005e6090b8 100644 --- a/src/Interpreters/tests/gtest_filecache.cpp +++ b/src/Interpreters/tests/gtest_filecache.cpp @@ -84,11 +84,12 @@ using HolderPtr = FileSegmentsHolderPtr; fs::path caches_dir = fs::current_path() / "lru_cache_test"; std::string cache_base_path = caches_dir / "cache1" / ""; +std::string cache_base_path2 = caches_dir / "cache2" / ""; void assertEqual(const FileSegmentsHolderPtr & file_segments, const Ranges & expected_ranges, const States & expected_states = {}) { - std::cerr << "File segments: "; + std::cerr << "\nFile segments: "; for (const auto & file_segment : *file_segments) std::cerr << file_segment->range().toString() << ", "; @@ -116,9 +117,12 @@ void assertEqual(const FileSegmentsHolderPtr & file_segments, const Ranges & exp void assertEqual(const std::vector & file_segments, const Ranges & expected_ranges, const States & expected_states = {}) { - std::cerr << "File segments: "; + std::cerr << "\nFile segments: "; for (const auto & file_segment : file_segments) std::cerr << FileSegment::Range(file_segment.range_left, file_segment.range_right).toString() << ", "; + std::cerr << "\nExpected: "; + for (const auto & r : expected_ranges) + std::cerr << r.toString() << ", "; ASSERT_EQ(file_segments.size(), expected_ranges.size()); @@ -142,51 +146,29 @@ void assertEqual(const std::vector & file_segments, const Ran } } -void assertEqual(const IFileCachePriority::QueueEntriesDumps & file_segments, const Ranges & expected_ranges, const States & expected_states = {}) +void assertProtectedOrProbationary(const std::vector & file_segments, const Ranges & expected, bool assert_protected) { - std::cerr << "File segments: "; - for (const auto & f : file_segments) - { - auto range = FileSegment::Range(f.info.range_left, f.info.range_right); - std::cerr << range.toString() << ", "; - } - - ASSERT_EQ(file_segments.size(), expected_ranges.size()); - - if (!expected_states.empty()) - ASSERT_EQ(file_segments.size(), expected_states.size()); - - auto get_expected_state = [&](size_t i) - { - if (expected_states.empty()) - return State::DOWNLOADED; - else - return expected_states[i]; - }; - - size_t i = 0; - for (const auto & f : file_segments) - { - auto range = FileSegment::Range(f.info.range_left, f.info.range_right); - ASSERT_EQ(range, expected_ranges[i]); - ASSERT_EQ(f.info.state, get_expected_state(i)); - ++i; - } -} - -void assertProtectedOrProbationary(const IFileCachePriority::QueueEntriesDumps & file_segments, const Ranges & expected, bool assert_protected) -{ - std::cerr << "File segments: "; + std::cerr << "\nFile segments: "; std::vector res; for (const auto & f : file_segments) { - auto range = FileSegment::Range(f.info.range_left, f.info.range_right); - std::cerr << range.toString() << ", "; - if ((f.is_protected && assert_protected) || (!f.is_protected && !assert_protected)) + auto range = FileSegment::Range(f.range_left, f.range_right); + bool is_protected = (f.queue_entry_type == FileCacheQueueEntryType::SLRU_Protected); + bool is_probationary = (f.queue_entry_type == FileCacheQueueEntryType::SLRU_Probationary); + ASSERT_TRUE(is_probationary || is_protected); + + std::cerr << fmt::format("{} (protected: {})", range.toString(), is_protected) << ", "; + + if ((is_protected && assert_protected) || (!is_protected && !assert_protected)) { res.push_back(range); } } + std::cerr << "\nExpected: "; + for (const auto & range : expected) + { + std::cerr << range.toString() << ", "; + } ASSERT_EQ(res.size(), expected.size()); for (size_t i = 0; i < res.size(); ++i) @@ -195,13 +177,15 @@ void assertProtectedOrProbationary(const IFileCachePriority::QueueEntriesDumps & } } -void assertProtected(const IFileCachePriority::QueueEntriesDumps & file_segments, const Ranges & expected) +void assertProtected(const std::vector & file_segments, const Ranges & expected) { + std::cerr << "\nAssert protected"; assertProtectedOrProbationary(file_segments, expected, true); } -void assertProbationary(const IFileCachePriority::QueueEntriesDumps & file_segments, const Ranges & expected) +void assertProbationary(const std::vector & file_segments, const Ranges & expected) { + std::cerr << "\nAssert probationary"; assertProtectedOrProbationary(file_segments, expected, false); } @@ -251,6 +235,13 @@ void increasePriority(const HolderPtr & holder) it->increasePriority(); } +void increasePriority(const HolderPtr & holder, size_t pos) +{ + FileSegments::iterator it = holder->begin(); + std::advance(it, pos); + (*it)->increasePriority(); +} + class FileCacheTest : public ::testing::Test { public: @@ -285,7 +276,10 @@ public: if (fs::exists(cache_base_path)) fs::remove_all(cache_base_path); + if (fs::exists(cache_base_path2)) + fs::remove_all(cache_base_path2); fs::create_directories(cache_base_path); + fs::create_directories(cache_base_path2); } void TearDown() override @@ -1232,4 +1226,95 @@ TEST_F(FileCacheTest, SLRUPolicy) ASSERT_EQ(cache.getFileSegmentsNum(), 5); ASSERT_EQ(cache.getUsedCacheSize(), 22); } + + { + ReadSettings read_settings; + read_settings.enable_filesystem_cache = true; + read_settings.local_fs_method = LocalFSReadMethod::pread; + + auto write_file = [](const std::string & filename, const std::string & s) + { + std::string file_path = fs::current_path() / filename; + auto wb = std::make_unique(file_path, DBMS_DEFAULT_BUFFER_SIZE); + wb->write(s.data(), s.size()); + wb->next(); + wb->finalize(); + return file_path; + }; + + DB::FileCacheSettings settings2; + settings2.base_path = cache_base_path2; + settings2.max_file_segment_size = 5; + settings2.max_size = 30; + settings2.max_elements = 6; + settings2.boundary_alignment = 1; + settings2.cache_policy = "SLRU"; + settings2.slru_size_ratio = 0.5; + + auto cache = std::make_shared("slru_2", settings2); + cache->initialize(); + + auto read_and_check = [&](const std::string & file, const FileCacheKey & key, const std::string & expect_result) + { + auto read_buffer_creator = [&]() + { + return createReadBufferFromFileBase(file, read_settings, std::nullopt, std::nullopt); + }; + + auto cached_buffer = std::make_shared( + file, key, cache, read_buffer_creator, read_settings, "test", expect_result.size(), false, false, std::nullopt, nullptr); + + WriteBufferFromOwnString result; + copyData(*cached_buffer, result); + ASSERT_EQ(result.str(), expect_result); + }; + + std::string data1(15, '*'); + auto file1 = write_file("test1", data1); + auto key1 = cache->createKeyForPath(file1); + + read_and_check(file1, key1, data1); + + assertEqual(cache->dumpQueue(), { Range(0, 4), Range(5, 9), Range(10, 14) }); + assertProbationary(cache->dumpQueue(), { Range(0, 4), Range(5, 9), Range(10, 14) }); + assertProtected(cache->dumpQueue(), Ranges{}); + + read_and_check(file1, key1, data1); + + assertEqual(cache->dumpQueue(), { Range(0, 4), Range(5, 9), Range(10, 14) }); + assertProbationary(cache->dumpQueue(), Ranges{}); + assertProtected(cache->dumpQueue(), { Range(0, 4), Range(5, 9), Range(10, 14) }); + + std::string data2(10, '*'); + auto file2 = write_file("test2", data2); + auto key2 = cache->createKeyForPath(file2); + + read_and_check(file2, key2, data2); + + auto dump = cache->dumpQueue(); + assertEqual(dump, { Range(0, 4), Range(5, 9), Range(0, 4), Range(5, 9), Range(10, 14) }); + + ASSERT_EQ(dump[0].key, key2); + ASSERT_EQ(dump[1].key, key2); + ASSERT_EQ(dump[2].key, key1); + ASSERT_EQ(dump[3].key, key1); + ASSERT_EQ(dump[4].key, key1); + + assertProbationary(cache->dumpQueue(), { Range(0, 4), Range(5, 9) }); + assertProtected(cache->dumpQueue(), { Range(0, 4), Range(5, 9), Range(10, 14) }); + + read_and_check(file2, key2, data2); + + dump = cache->dumpQueue(); + assertEqual(dump, { Range(0, 4), Range(5, 9), Range(10, 14), Range(0, 4), Range(5, 9) }); + + ASSERT_EQ(dump[0].key, key1); + ASSERT_EQ(dump[1].key, key1); + ASSERT_EQ(dump[2].key, key1); + ASSERT_EQ(dump[3].key, key2); + ASSERT_EQ(dump[4].key, key2); + + assertProbationary(cache->dumpQueue(), { Range(0, 4), Range(5, 9) }); + assertProtected(cache->dumpQueue(), { Range(10, 14), Range(0, 4), Range(5, 9) }); + } } From b1043b7ad3ec102261ee40ed2348dd656406698e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 8 Dec 2023 12:11:00 +0100 Subject: [PATCH 162/331] 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 163/331] 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 8ebd5a2d6c75d81a79c07a2463179314756852b9 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 7 Dec 2023 12:10:30 +0000 Subject: [PATCH 164/331] Fix type correction in HashJoin for nested low cardinality --- src/Interpreters/HashJoin.cpp | 71 +++++++++++-------- src/Interpreters/JoinUtils.cpp | 10 +-- src/Interpreters/JoinUtils.h | 1 - ...oin_with_totals_and_subquery_bug.reference | 17 +++++ ...2516_join_with_totals_and_subquery_bug.sql | 46 +++++++++++- 5 files changed, 103 insertions(+), 42 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 060fe95958f..71e9d4bba80 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -32,7 +32,7 @@ #include #include - +#include namespace DB { @@ -217,7 +217,7 @@ static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nulla } } -static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable, const ColumnUInt8 & negative_null_map) +static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable, const IColumn::Filter & negative_null_map) { if (nullable) { @@ -1548,6 +1548,40 @@ IColumn::Filter switchJoinRightColumns( } } +/** Since we do not store right key columns, + * this function is used to copy left key columns to right key columns. + * If the user requests some right columns, we just copy left key columns to right, since they are equal. + * Example: SELECT t1.key, t2.key FROM t1 FULL JOIN t2 ON t1.key = t2.key; + * In that case for matched rows in t2.key we will use values from t1.key. + * However, in some cases we might need to adjust the type of column, e.g. t1.key :: LowCardinality(String) and t2.key :: String + * Also, the nullability of the column might be different. + * Returns the right column after with necessary adjustments. + */ +ColumnWithTypeAndName copyLeftKeyColumnToRight( + const DataTypePtr & right_key_type, const String & renamed_right_column, const ColumnWithTypeAndName & left_column, const IColumn::Filter * null_map_filter = nullptr) +{ + ColumnWithTypeAndName right_column = left_column; + right_column.name = renamed_right_column; + + if (null_map_filter) + right_column.column = JoinCommon::filterWithBlanks(right_column.column, *null_map_filter); + + bool should_be_nullable = isNullableOrLowCardinalityNullable(right_key_type); + if (null_map_filter) + correctNullabilityInplace(right_column, should_be_nullable, *null_map_filter); + else + correctNullabilityInplace(right_column, should_be_nullable); + + if (!right_column.type->equals(*right_key_type)) + { + right_column.column = castColumnAccurate(right_column, right_key_type); + right_column.type = right_key_type; + } + + right_column.column = right_column.column->convertToFullColumnIfConst(); + return right_column; +} + } /// nameless template @@ -1614,31 +1648,19 @@ void HashJoin::joinBlockImpl( // renamed ??? if (!block.findByName(right_key.name)) { - const auto & left_name = required_right_keys_sources[i]; - /// asof column is already in block. if (join_features.is_asof_join && right_key.name == table_join->getOnlyClause().key_names_right.back()) continue; - const auto & col = block.getByName(left_name); - bool is_nullable = JoinCommon::isNullable(right_key.type); - auto right_col_name = getTableJoin().renamedRightColumnName(right_key.name); - ColumnWithTypeAndName right_col(col.column, col.type, right_col_name); - if (right_col.type->lowCardinality() != right_key.type->lowCardinality()) - JoinCommon::changeLowCardinalityInplace(right_col); - correctNullabilityInplace(right_col, is_nullable); + const auto & left_column = block.getByName(required_right_keys_sources[i]); + const auto & right_col_name = getTableJoin().renamedRightColumnName(right_key.name); + auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column); block.insert(std::move(right_col)); } } } else if (has_required_right_keys) { - /// Some trash to represent IColumn::Filter as ColumnUInt8 needed for ColumnNullable::applyNullMap() - auto null_map_filter_ptr = ColumnUInt8::create(); - ColumnUInt8 & null_map_filter = assert_cast(*null_map_filter_ptr); - null_map_filter.getData().swap(row_filter); - const IColumn::Filter & filter = null_map_filter.getData(); - /// Add join key columns from right block if needed. for (size_t i = 0; i < required_right_keys.columns(); ++i) { @@ -1646,21 +1668,12 @@ void HashJoin::joinBlockImpl( auto right_col_name = getTableJoin().renamedRightColumnName(right_key.name); if (!block.findByName(right_col_name)) { - const auto & left_name = required_right_keys_sources[i]; - /// asof column is already in block. if (join_features.is_asof_join && right_key.name == table_join->getOnlyClause().key_names_right.back()) continue; - const auto & col = block.getByName(left_name); - bool is_nullable = JoinCommon::isNullable(right_key.type); - - ColumnPtr thin_column = JoinCommon::filterWithBlanks(col.column, filter); - - ColumnWithTypeAndName right_col(thin_column, col.type, right_col_name); - if (right_col.type->lowCardinality() != right_key.type->lowCardinality()) - JoinCommon::changeLowCardinalityInplace(right_col); - correctNullabilityInplace(right_col, is_nullable, null_map_filter); + const auto & left_column = block.getByName(required_right_keys_sources[i]); + auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column, &row_filter); block.insert(std::move(right_col)); if constexpr (join_features.need_replication) @@ -2179,7 +2192,7 @@ BlocksList HashJoin::releaseJoinedBlocks(bool restructure) for (const auto & sample_column : right_sample_block) { positions.emplace_back(tmp_block.getPositionByName(sample_column.name)); - is_nullable.emplace_back(JoinCommon::isNullable(sample_column.type)); + is_nullable.emplace_back(isNullableOrLowCardinalityNullable(sample_column.type)); } } diff --git a/src/Interpreters/JoinUtils.cpp b/src/Interpreters/JoinUtils.cpp index 33c9dfa76ca..be5ee80bd53 100644 --- a/src/Interpreters/JoinUtils.cpp +++ b/src/Interpreters/JoinUtils.cpp @@ -120,19 +120,11 @@ bool canBecomeNullable(const DataTypePtr & type) return can_be_inside; } -bool isNullable(const DataTypePtr & type) -{ - bool is_nullable = type->isNullable(); - if (const auto * low_cardinality_type = typeid_cast(type.get())) - is_nullable |= low_cardinality_type->getDictionaryType()->isNullable(); - return is_nullable; -} - /// Add nullability to type. /// Note: LowCardinality(T) transformed to LowCardinality(Nullable(T)) DataTypePtr convertTypeToNullable(const DataTypePtr & type) { - if (isNullable(type)) + if (isNullableOrLowCardinalityNullable(type)) return type; if (const auto * low_cardinality_type = typeid_cast(type.get())) diff --git a/src/Interpreters/JoinUtils.h b/src/Interpreters/JoinUtils.h index f112ca22e5b..7daed6b7f7e 100644 --- a/src/Interpreters/JoinUtils.h +++ b/src/Interpreters/JoinUtils.h @@ -59,7 +59,6 @@ private: }; -bool isNullable(const DataTypePtr & type); bool canBecomeNullable(const DataTypePtr & type); DataTypePtr convertTypeToNullable(const DataTypePtr & type); void convertColumnToNullable(ColumnWithTypeAndName & column); diff --git a/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference b/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference index 19da8828c30..51e8394a2f7 100644 --- a/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference +++ b/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference @@ -9,3 +9,20 @@ \N 100000000000000000000 +--- +['0'] 2 ['0'] +['0'] 2 ['0'] +['1'] 1 ['1'] + +[] 3 [] +--- +['0'] 2 ['0'] 2 +['1'] 1 ['1'] 1 + +[] 3 [] 3 +--- +['0'] ['0'] 2 +['0'] ['0'] 2 +['1'] ['1'] 1 + +[] [] 3 diff --git a/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.sql b/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.sql index 6b58d737a3e..243dceffc43 100644 --- a/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.sql +++ b/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.sql @@ -1,5 +1,3 @@ -SET allow_experimental_analyzer = 1; - SELECT * FROM ( @@ -34,7 +32,6 @@ INNER JOIN ) AS t2 USING (a) SETTINGS allow_experimental_analyzer=1; - SELECT a FROM ( @@ -72,3 +69,46 @@ ALL LEFT JOIN WITH TOTALS ) AS js2 USING (a) ORDER BY b ASC NULLS FIRST; + +SELECT '---'; +SELECT + * +FROM ( + SELECT ([toString(number % 2)] :: Array(LowCardinality(String))) AS item_id, count() FROM numbers(3) GROUP BY item_id + WITH TOTALS +) AS l +FULL JOIN ( + SELECT ([toString(number % 2)] :: Array(String)) AS item_id FROM numbers(3) +) AS r +ON l.item_id = r.item_id +ORDER BY 1,2,3 +; + +SELECT '---'; +SELECT + * +FROM ( + SELECT ([toString(number % 2)] :: Array(LowCardinality(String))) AS item_id, count() FROM numbers(3) GROUP BY item_id + WITH TOTALS +) AS l +FULL JOIN ( + SELECT ([toString(number % 2)] :: Array(String)) AS item_id, count() FROM numbers(3) GROUP BY item_id + WITH TOTALS +) AS r +ON l.item_id = r.item_id +ORDER BY 1,2,3 +; + +SELECT '---'; +SELECT + * +FROM ( + SELECT ([toString(number % 2)] :: Array(String)) AS item_id FROM numbers(3) +) AS l +FULL JOIN ( + SELECT ([toString(number % 2)] :: Array(LowCardinality(String))) AS item_id, count() FROM numbers(3) GROUP BY item_id + WITH TOTALS +) AS r +ON l.item_id = r.item_id +ORDER BY 1,2,3 +; From ff0340b0fca45ec77ac2f7273b71875c13e357ce Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 8 Dec 2023 18:18:45 +0000 Subject: [PATCH 165/331] Release memory for aggregation earlier. --- src/Processors/Transforms/AggregatingTransform.cpp | 13 +++++++++++++ .../02933_group_by_memory_usage.reference | 3 +++ .../0_stateless/02933_group_by_memory_usage.sh | 13 +++++++++++++ 3 files changed, 29 insertions(+) create mode 100644 tests/queries/0_stateless/02933_group_by_memory_usage.reference create mode 100755 tests/queries/0_stateless/02933_group_by_memory_usage.sh diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index bf475c57d36..6bcd9b2f686 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -123,7 +123,10 @@ protected: UInt32 bucket_num = shared_data->next_bucket_to_merge.fetch_add(1); if (bucket_num >= NUM_BUCKETS) + { + data.reset(); return {}; + } Block block = params->aggregator.mergeAndConvertOneBucketToBlock(*data, arena, params->final, bucket_num, &shared_data->is_cancelled); Chunk chunk = convertToChunk(block); @@ -170,6 +173,8 @@ protected: return convertToChunk(block); } + variant.reset(); + return {}; } @@ -489,6 +494,7 @@ private: single_level_chunks.emplace_back(convertToChunk(block)); finished = true; + data.reset(); } void createSources() @@ -504,6 +510,8 @@ private: processors.emplace_back(std::move(source)); } + + data.reset(); } }; @@ -710,7 +718,10 @@ void AggregatingTransform::initGenerate() } if (many_data->num_finished.fetch_add(1) + 1 < many_data->variants.size()) + { + many_data.reset(); return; + } if (!params->aggregator.hasTemporaryData()) { @@ -807,6 +818,8 @@ void AggregatingTransform::initGenerate() processors = Pipe::detachProcessors(std::move(pipe)); } + + many_data.reset(); } } diff --git a/tests/queries/0_stateless/02933_group_by_memory_usage.reference b/tests/queries/0_stateless/02933_group_by_memory_usage.reference new file mode 100644 index 00000000000..1481fcaa297 --- /dev/null +++ b/tests/queries/0_stateless/02933_group_by_memory_usage.reference @@ -0,0 +1,3 @@ +Spin up a long running query +1 1 1 1 1 +0 diff --git a/tests/queries/0_stateless/02933_group_by_memory_usage.sh b/tests/queries/0_stateless/02933_group_by_memory_usage.sh new file mode 100755 index 00000000000..96ed2f3f0da --- /dev/null +++ b/tests/queries/0_stateless/02933_group_by_memory_usage.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash +# Tags: long, no-random-settings + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +query_id="group-by-mem-usage-$CLICKHOUSE_DATABASE" + +echo "Spin up a long running query" +${CLICKHOUSE_CLIENT} --query "with q as (select length(groupArray(toString(number))) as x from numbers_mt(2e6) group by number order by x limit 1), q1 as (select * from q), q2 as (select * from q), q3 as (select * from q), q4 as (select * from q) select * from q, q1, q2, q3, q4 settings max_bytes_before_external_group_by='1G', max_memory_usage='2G'" --query_id "$query_id" +${CLICKHOUSE_CLIENT} --query "system flush logs" +${CLICKHOUSE_CLIENT} --query "select ProfileEvents['ExternalAggregationWritePart'] from system.query_log where type = 'QueryFinish' and query_id = '$query_id' and event_date >= today() - 1" From 75441d9812078eada13a275a3a148ee4b28facb9 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 8 Dec 2023 14:44:30 -0400 Subject: [PATCH 166/331] 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 167/331] 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 168/331] 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 169/331] 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 881e58dfcbcc9f5326a9650b7af373fe8f853136 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 8 Dec 2023 21:46:31 +0100 Subject: [PATCH 170/331] support soft limit ratio --- programs/keeper/Keeper.cpp | 30 +++++++++++++++++++ src/Coordination/CoordinationSettings.h | 1 - src/Coordination/KeeperContext.cpp | 8 +++++ src/Coordination/KeeperContext.h | 5 ++++ src/Coordination/KeeperDispatcher.cpp | 4 ++- .../configs/keeper_config1.xml | 2 +- .../configs/keeper_config2.xml | 2 +- .../configs/keeper_config3.xml | 3 +- 8 files changed, 50 insertions(+), 5 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index e04e669abae..9db9e1a3a26 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -289,6 +290,33 @@ try if (!config().has("keeper_server")) throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Keeper configuration ( section) not found in config"); + auto updateMemorySoftLimitInConfig = [&](Poco::Util::AbstractConfiguration & config) + { + UInt64 memory_soft_limit = 0; + if (config.has("keeper_server.max_memory_usage_soft_limit")) + { + memory_soft_limit = config.getUInt64("keeper_server.max_memory_usage_soft_limit"); + } + + /// if memory soft limit is not set, we will use default value + if (memory_soft_limit == 0) + { + Float64 ratio = 0.9; + if (config.has("keeper_server.max_memory_usage_soft_limit_ratio")) + ratio = config.getDouble("keeper_server.max_memory_usage_soft_limit_ratio"); + + size_t physical_server_memory = getMemoryAmount(); + if (ratio > 0 && physical_server_memory > 0) + { + memory_soft_limit = static_cast(physical_server_memory * ratio); + config.setUInt64("keeper_server.max_memory_usage_soft_limit", memory_soft_limit); + } + } + LOG_INFO(log, "keeper_server.max_memory_usage_soft_limit is set to {}", formatReadableSizeWithBinarySuffix(memory_soft_limit)); + }; + + updateMemorySoftLimitInConfig(config()); + std::string path; if (config().has("keeper_server.storage_path")) @@ -492,6 +520,8 @@ try { updateLevels(*config, logger()); + updateMemorySoftLimitInConfig(*config); + if (config->has("keeper_server")) global_context->updateKeeperConfiguration(*config); diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 13ef304b353..a58f2b04797 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -43,7 +43,6 @@ 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/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 7e0b75a6353..6bb5b066d9f 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -59,6 +59,8 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, } } + updateKeeperMemorySoftLimit(config); + digest_enabled = config.getBool("keeper_server.digest_enabled", false); ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false); @@ -375,4 +377,10 @@ void KeeperContext::initializeFeatureFlags(const Poco::Util::AbstractConfigurati feature_flags.logFlags(&Poco::Logger::get("KeeperContext")); } +void KeeperContext::updateKeeperMemorySoftLimit(const Poco::Util::AbstractConfiguration & config) +{ + if (config.hasProperty("keeper_server.max_memory_usage_soft_limit")) + memory_soft_limit = config.getUInt64("keeper_server.max_memory_usage_soft_limit"); +} + } diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 1af34b19ccf..c1c34db2c4b 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -53,6 +53,9 @@ public: constexpr KeeperDispatcher * getDispatcher() const { return dispatcher; } + UInt64 getKeeperMemorySoftLimit() const { return memory_soft_limit; } + void updateKeeperMemorySoftLimit(const Poco::Util::AbstractConfiguration & config); + /// set to true when we have preprocessed or committed all the logs /// that were already present locally during startup std::atomic local_logs_preprocessed = false; @@ -92,6 +95,8 @@ private: KeeperFeatureFlags feature_flags; KeeperDispatcher * dispatcher{nullptr}; + + std::atomic memory_soft_limit = 0; }; using KeeperContextPtr = std::shared_ptr; diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 1299e9c9f20..dcd22552fe3 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -143,7 +143,7 @@ void KeeperDispatcher::requestThread() if (shutdown_called) break; - Int64 mem_soft_limit = configuration_and_settings->coordination_settings->max_memory_usage_soft_limit; + Int64 mem_soft_limit = keeper_context->getKeeperMemorySoftLimit(); 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 max_memory_usage_soft_limit {}, the total used memory is {}, request type is {}", mem_soft_limit, total_memory_tracker.get(), request.request->getOpNum()); @@ -930,6 +930,8 @@ void KeeperDispatcher::updateConfiguration(const Poco::Util::AbstractConfigurati throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push configuration update to queue"); snapshot_s3.updateS3Configuration(config, macros); + + keeper_context->updateKeeperMemorySoftLimit(config); } void KeeperDispatcher::updateKeeperStatLatency(uint64_t process_time_ms) 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 index fe45d09d915..642cf16414e 100644 --- a/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config1.xml +++ b/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config1.xml @@ -15,6 +15,7 @@ az-zoo1 1 + 200000000 10000 @@ -23,7 +24,6 @@ false 2000 4000 - 200000000 1 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 index f7f6a5718b5..25ececea3e8 100644 --- a/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config2.xml +++ b/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config2.xml @@ -16,6 +16,7 @@ az-zoo2 1 + 20000000 10000 @@ -24,7 +25,6 @@ false 2000 4000 - 20000000 1 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 index 82345aebc46..81e343b77c9 100644 --- a/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config3.xml +++ b/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config3.xml @@ -13,6 +13,8 @@ 2181 3 + 20000000 + 10000 15000 @@ -20,7 +22,6 @@ false 2000 4000 - 20000000 1 From bb084466c4af93fb6ddd8ba5223ef1ddf89c0c71 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Dec 2023 23:51:12 +0300 Subject: [PATCH 171/331] 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 a76d9c259f690baee094e024d78059db40b4e644 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 8 Dec 2023 21:06:18 +0000 Subject: [PATCH 172/331] Fixing style --- src/Processors/Transforms/AggregatingTransform.cpp | 2 +- tests/queries/0_stateless/02933_group_by_memory_usage.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 6bcd9b2f686..dacd6261477 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -289,7 +289,7 @@ public: void work() override { - if (data->empty()) + if (!data || data->empty()) { finished = true; return; diff --git a/tests/queries/0_stateless/02933_group_by_memory_usage.sh b/tests/queries/0_stateless/02933_group_by_memory_usage.sh index 96ed2f3f0da..bb1bbbf16a2 100755 --- a/tests/queries/0_stateless/02933_group_by_memory_usage.sh +++ b/tests/queries/0_stateless/02933_group_by_memory_usage.sh @@ -10,4 +10,4 @@ query_id="group-by-mem-usage-$CLICKHOUSE_DATABASE" echo "Spin up a long running query" ${CLICKHOUSE_CLIENT} --query "with q as (select length(groupArray(toString(number))) as x from numbers_mt(2e6) group by number order by x limit 1), q1 as (select * from q), q2 as (select * from q), q3 as (select * from q), q4 as (select * from q) select * from q, q1, q2, q3, q4 settings max_bytes_before_external_group_by='1G', max_memory_usage='2G'" --query_id "$query_id" ${CLICKHOUSE_CLIENT} --query "system flush logs" -${CLICKHOUSE_CLIENT} --query "select ProfileEvents['ExternalAggregationWritePart'] from system.query_log where type = 'QueryFinish' and query_id = '$query_id' and event_date >= today() - 1" +${CLICKHOUSE_CLIENT} --query "select ProfileEvents['ExternalAggregationWritePart'] from system.query_log where current_database = currentDatabase() and type = 'QueryFinish' and query_id = '$query_id' and event_date >= today() - 1" From 57b740f84b512ab0b91db5ea243ce058e7f67dfe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Dec 2023 22:14:24 +0100 Subject: [PATCH 173/331] 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 174/331] 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 5e1dccf1f423cb9d2738c89b858ec1eaa5698601 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 8 Dec 2023 22:36:01 +0000 Subject: [PATCH 175/331] Fixing tests --- src/Processors/Transforms/AggregatingTransform.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index dacd6261477..ec8ea9396e4 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -289,7 +289,7 @@ public: void work() override { - if (!data || data->empty()) + if (data->empty()) { finished = true; return; @@ -424,7 +424,7 @@ private: return Status::Finished; } - return has_rows ? Status::PortFull : Status::Ready; + return has_rows ? Status::PortFull : Status::NeedData; } AggregatingTransformParamsPtr params; From 64ae04095406c93abd5974dfd63e1144249fab9e Mon Sep 17 00:00:00 2001 From: Bharat Nallan Chakravarthy Date: Fri, 8 Dec 2023 07:37:38 -0800 Subject: [PATCH 176/331] 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 177/331] 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 178/331] 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 179/331] 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 63619b59353699b5975acedffe0f2ca85e3985d7 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 9 Dec 2023 17:57:32 +0000 Subject: [PATCH 180/331] allow IPv6 to UInt128 conversion and binary arithmetic --- src/Functions/FunctionBinaryArithmetic.h | 30 ++++++++++++++++++++++++ src/Functions/FunctionsConversion.h | 12 ++++++++++ 2 files changed, 42 insertions(+) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 33f62b8da8c..1b2519d1ec5 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -1483,6 +1483,17 @@ public: return getReturnTypeImplStatic(new_arguments, context); } + /// Special case - one or both arguments are IPv6 + if (isIPv6(arguments[0]) || isIPv6(arguments[1])) + { + DataTypes new_arguments { + isIPv6(arguments[0]) ? std::make_shared() : arguments[0], + isIPv6(arguments[1]) ? std::make_shared() : arguments[1], + }; + + return getReturnTypeImplStatic(new_arguments, context); + } + if constexpr (is_plus || is_minus) { @@ -2181,6 +2192,25 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A return executeImpl2(new_arguments, result_type, input_rows_count, right_nullmap); } + /// Special case - one or both arguments are IPv6 + if (isIPv6(arguments[0].type) || isIPv6(arguments[1].type)) + { + ColumnsWithTypeAndName new_arguments { + { + isIPv6(arguments[0].type) ? castColumn(arguments[0], std::make_shared()) : arguments[0].column, + isIPv6(arguments[0].type) ? std::make_shared() : arguments[0].type, + arguments[0].name, + }, + { + isIPv6(arguments[1].type) ? castColumn(arguments[1], std::make_shared()) : arguments[1].column, + isIPv6(arguments[1].type) ? std::make_shared() : arguments[1].type, + arguments[1].name + } + }; + + return executeImpl2(new_arguments, result_type, input_rows_count, right_nullmap); + } + const auto * const left_generic = left_argument.type.get(); const auto * const right_generic = right_argument.type.get(); ColumnPtr res; diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index d7c2c70884b..bef1e7b420a 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -221,6 +221,18 @@ struct ConvertImpl continue; } + if constexpr (std::is_same_v && std::is_same_v) + { + static_assert( + std::is_same_v, + "UInt128 and IPv6 types must be same"); + + vec_to[i].items[1] = std::byteswap(vec_from[i].toUnderType().items[0]); + vec_to[i].items[0] = std::byteswap(vec_from[i].toUnderType().items[1]); + + continue; + } + if constexpr (std::is_same_v != std::is_same_v) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, From b6e3e5691cadbe1a60a6202ebdecc9da18b0823b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Dec 2023 18:58:35 +0100 Subject: [PATCH 181/331] 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 182/331] 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 183/331] 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 184/331] 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 185/331] 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 186/331] 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 187/331] 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 188/331] 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 189/331] 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 190/331] 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 191/331] 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 3ac3a06561f8f98fd3f38b9048a5b1a44c263377 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Dec 2023 23:31:05 +0100 Subject: [PATCH 192/331] Add ClickBench to CI --- .github/workflows/pull_request.yml | 25 ++ docker/images.json | 5 + docker/test/clickbench/Dockerfile | 10 + docker/test/clickbench/create.sql | 112 +++++++++ docker/test/clickbench/queries.sql | 43 ++++ docker/test/clickbench/run.sh | 52 ++++ tests/ci/clickbench.py | 261 ++++++++++++++++++++ tests/ci/fast_test_check.py | 2 +- tests/ci/functional_test_check.py | 2 +- tests/ci/integration_test_check.py | 2 +- tests/ci/test_docker.py | 6 + tests/ci/tests/docker_images_for_tests.json | 5 + 12 files changed, 522 insertions(+), 3 deletions(-) create mode 100644 docker/test/clickbench/Dockerfile create mode 100644 docker/test/clickbench/create.sql create mode 100644 docker/test/clickbench/queries.sql create mode 100755 docker/test/clickbench/run.sh create mode 100644 tests/ci/clickbench.py diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index a6631a93766..3db382b1fd6 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -701,6 +701,31 @@ jobs: cd "$REPO_COPY/tests/ci" python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" ############################################################################################## +########################### ClickBench ####################################################### +############################################################################################## + ClickBenchAMD64: + needs: [BuilderDebRelease] + uses: ./.github/workflows/reusable_test.yml + with: + test_name: ClickBench (amd64) + runner_type: func-tester + additional_envs: | + KILL_TIMEOUT=1800 + run_command: | + cd "$REPO_COPY/tests/ci" + python3 clickbench.py "$CHECK_NAME" "$KILL_TIMEOUT" + ClickBenchAarch64: + needs: [ BuilderDebAarch64 ] + uses: ./.github/workflows/reusable_test.yml + with: + test_name: ClickBench (aarch64) + runner_type: func-tester-aarch64 + additional_envs: | + KILL_TIMEOUT=1800 + run_command: | + cd "$REPO_COPY/tests/ci" + python3 clickbench.py "$CHECK_NAME" "$KILL_TIMEOUT" +############################################################################################## ######################################### STRESS TESTS ####################################### ############################################################################################## StressTestAsan: diff --git a/docker/images.json b/docker/images.json index 1535715648c..d2f098f53d7 100644 --- a/docker/images.json +++ b/docker/images.json @@ -125,6 +125,7 @@ "docker/test/server-jepsen", "docker/test/sqllogic", "docker/test/sqltest", + "docker/test/clickbench", "docker/test/stateless" ] }, @@ -145,6 +146,10 @@ "name": "clickhouse/server-jepsen-test", "dependent": [] }, + "docker/test/clickbench": { + "name": "clickhouse/clickbench", + "dependent": [] + }, "docker/test/install/deb": { "name": "clickhouse/install-deb-test", "dependent": [] diff --git a/docker/test/clickbench/Dockerfile b/docker/test/clickbench/Dockerfile new file mode 100644 index 00000000000..0b6b1736e03 --- /dev/null +++ b/docker/test/clickbench/Dockerfile @@ -0,0 +1,10 @@ +ARG FROM_TAG=latest +FROM clickhouse/test-base:$FROM_TAG + +ENV TZ=Europe/Amsterdam +RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone + +COPY *.sh / +COPY *.sql / + +CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/clickbench/create.sql b/docker/test/clickbench/create.sql new file mode 100644 index 00000000000..620bdf09331 --- /dev/null +++ b/docker/test/clickbench/create.sql @@ -0,0 +1,112 @@ +ATTACH TABLE hits UUID 'c449dfbf-ba06-4d13-abec-8396559eb955' +( + WatchID BIGINT NOT NULL, + JavaEnable SMALLINT NOT NULL, + Title TEXT NOT NULL, + GoodEvent SMALLINT NOT NULL, + EventTime TIMESTAMP NOT NULL, + EventDate Date NOT NULL, + CounterID INTEGER NOT NULL, + ClientIP INTEGER NOT NULL, + RegionID INTEGER NOT NULL, + UserID BIGINT NOT NULL, + CounterClass SMALLINT NOT NULL, + OS SMALLINT NOT NULL, + UserAgent SMALLINT NOT NULL, + URL TEXT NOT NULL, + Referer TEXT NOT NULL, + IsRefresh SMALLINT NOT NULL, + RefererCategoryID SMALLINT NOT NULL, + RefererRegionID INTEGER NOT NULL, + URLCategoryID SMALLINT NOT NULL, + URLRegionID INTEGER NOT NULL, + ResolutionWidth SMALLINT NOT NULL, + ResolutionHeight SMALLINT NOT NULL, + ResolutionDepth SMALLINT NOT NULL, + FlashMajor SMALLINT NOT NULL, + FlashMinor SMALLINT NOT NULL, + FlashMinor2 TEXT NOT NULL, + NetMajor SMALLINT NOT NULL, + NetMinor SMALLINT NOT NULL, + UserAgentMajor SMALLINT NOT NULL, + UserAgentMinor VARCHAR(255) NOT NULL, + CookieEnable SMALLINT NOT NULL, + JavascriptEnable SMALLINT NOT NULL, + IsMobile SMALLINT NOT NULL, + MobilePhone SMALLINT NOT NULL, + MobilePhoneModel TEXT NOT NULL, + Params TEXT NOT NULL, + IPNetworkID INTEGER NOT NULL, + TraficSourceID SMALLINT NOT NULL, + SearchEngineID SMALLINT NOT NULL, + SearchPhrase TEXT NOT NULL, + AdvEngineID SMALLINT NOT NULL, + IsArtifical SMALLINT NOT NULL, + WindowClientWidth SMALLINT NOT NULL, + WindowClientHeight SMALLINT NOT NULL, + ClientTimeZone SMALLINT NOT NULL, + ClientEventTime TIMESTAMP NOT NULL, + SilverlightVersion1 SMALLINT NOT NULL, + SilverlightVersion2 SMALLINT NOT NULL, + SilverlightVersion3 INTEGER NOT NULL, + SilverlightVersion4 SMALLINT NOT NULL, + PageCharset TEXT NOT NULL, + CodeVersion INTEGER NOT NULL, + IsLink SMALLINT NOT NULL, + IsDownload SMALLINT NOT NULL, + IsNotBounce SMALLINT NOT NULL, + FUniqID BIGINT NOT NULL, + OriginalURL TEXT NOT NULL, + HID INTEGER NOT NULL, + IsOldCounter SMALLINT NOT NULL, + IsEvent SMALLINT NOT NULL, + IsParameter SMALLINT NOT NULL, + DontCountHits SMALLINT NOT NULL, + WithHash SMALLINT NOT NULL, + HitColor CHAR NOT NULL, + LocalEventTime TIMESTAMP NOT NULL, + Age SMALLINT NOT NULL, + Sex SMALLINT NOT NULL, + Income SMALLINT NOT NULL, + Interests SMALLINT NOT NULL, + Robotness SMALLINT NOT NULL, + RemoteIP INTEGER NOT NULL, + WindowName INTEGER NOT NULL, + OpenerName INTEGER NOT NULL, + HistoryLength SMALLINT NOT NULL, + BrowserLanguage TEXT NOT NULL, + BrowserCountry TEXT NOT NULL, + SocialNetwork TEXT NOT NULL, + SocialAction TEXT NOT NULL, + HTTPError SMALLINT NOT NULL, + SendTiming INTEGER NOT NULL, + DNSTiming INTEGER NOT NULL, + ConnectTiming INTEGER NOT NULL, + ResponseStartTiming INTEGER NOT NULL, + ResponseEndTiming INTEGER NOT NULL, + FetchTiming INTEGER NOT NULL, + SocialSourceNetworkID SMALLINT NOT NULL, + SocialSourcePage TEXT NOT NULL, + ParamPrice BIGINT NOT NULL, + ParamOrderID TEXT NOT NULL, + ParamCurrency TEXT NOT NULL, + ParamCurrencyID SMALLINT NOT NULL, + OpenstatServiceName TEXT NOT NULL, + OpenstatCampaignID TEXT NOT NULL, + OpenstatAdID TEXT NOT NULL, + OpenstatSourceID TEXT NOT NULL, + UTMSource TEXT NOT NULL, + UTMMedium TEXT NOT NULL, + UTMCampaign TEXT NOT NULL, + UTMContent TEXT NOT NULL, + UTMTerm TEXT NOT NULL, + FromTag TEXT NOT NULL, + HasGCLID SMALLINT NOT NULL, + RefererHash BIGINT NOT NULL, + URLHash BIGINT NOT NULL, + CLID INTEGER NOT NULL, + PRIMARY KEY (CounterID, EventDate, UserID, EventTime, WatchID) +) +ENGINE = MergeTree +SETTINGS disk = disk(type = cache, path = '/dev/shm/clickhouse/', max_size = '16G', + disk = disk(type = web, endpoint = 'https://clickhouse-public-datasets.s3.amazonaws.com/web/')); diff --git a/docker/test/clickbench/queries.sql b/docker/test/clickbench/queries.sql new file mode 100644 index 00000000000..31f65fc898d --- /dev/null +++ b/docker/test/clickbench/queries.sql @@ -0,0 +1,43 @@ +SELECT COUNT(*) FROM hits; +SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0; +SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits; +SELECT AVG(UserID) FROM hits; +SELECT COUNT(DISTINCT UserID) FROM hits; +SELECT COUNT(DISTINCT SearchPhrase) FROM hits; +SELECT MIN(EventDate), MAX(EventDate) FROM hits; +SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC; +SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10; +SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10; +SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10; +SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10; +SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10; +SELECT UserID FROM hits WHERE UserID = 435090932899640449; +SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'; +SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; +SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10; +SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10; +SELECT CounterID, AVG(length(URL)) AS l, COUNT(*) AS c FROM hits WHERE URL <> '' GROUP BY CounterID HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; +SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer <> '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25; +SELECT SUM(ResolutionWidth), SUM(ResolutionWidth + 1), SUM(ResolutionWidth + 2), SUM(ResolutionWidth + 3), SUM(ResolutionWidth + 4), SUM(ResolutionWidth + 5), SUM(ResolutionWidth + 6), SUM(ResolutionWidth + 7), SUM(ResolutionWidth + 8), SUM(ResolutionWidth + 9), SUM(ResolutionWidth + 10), SUM(ResolutionWidth + 11), SUM(ResolutionWidth + 12), SUM(ResolutionWidth + 13), SUM(ResolutionWidth + 14), SUM(ResolutionWidth + 15), SUM(ResolutionWidth + 16), SUM(ResolutionWidth + 17), SUM(ResolutionWidth + 18), SUM(ResolutionWidth + 19), SUM(ResolutionWidth + 20), SUM(ResolutionWidth + 21), SUM(ResolutionWidth + 22), SUM(ResolutionWidth + 23), SUM(ResolutionWidth + 24), SUM(ResolutionWidth + 25), SUM(ResolutionWidth + 26), SUM(ResolutionWidth + 27), SUM(ResolutionWidth + 28), SUM(ResolutionWidth + 29), SUM(ResolutionWidth + 30), SUM(ResolutionWidth + 31), SUM(ResolutionWidth + 32), SUM(ResolutionWidth + 33), SUM(ResolutionWidth + 34), SUM(ResolutionWidth + 35), SUM(ResolutionWidth + 36), SUM(ResolutionWidth + 37), SUM(ResolutionWidth + 38), SUM(ResolutionWidth + 39), SUM(ResolutionWidth + 40), SUM(ResolutionWidth + 41), SUM(ResolutionWidth + 42), SUM(ResolutionWidth + 43), SUM(ResolutionWidth + 44), SUM(ResolutionWidth + 45), SUM(ResolutionWidth + 46), SUM(ResolutionWidth + 47), SUM(ResolutionWidth + 48), SUM(ResolutionWidth + 49), SUM(ResolutionWidth + 50), SUM(ResolutionWidth + 51), SUM(ResolutionWidth + 52), SUM(ResolutionWidth + 53), SUM(ResolutionWidth + 54), SUM(ResolutionWidth + 55), SUM(ResolutionWidth + 56), SUM(ResolutionWidth + 57), SUM(ResolutionWidth + 58), SUM(ResolutionWidth + 59), SUM(ResolutionWidth + 60), SUM(ResolutionWidth + 61), SUM(ResolutionWidth + 62), SUM(ResolutionWidth + 63), SUM(ResolutionWidth + 64), SUM(ResolutionWidth + 65), SUM(ResolutionWidth + 66), SUM(ResolutionWidth + 67), SUM(ResolutionWidth + 68), SUM(ResolutionWidth + 69), SUM(ResolutionWidth + 70), SUM(ResolutionWidth + 71), SUM(ResolutionWidth + 72), SUM(ResolutionWidth + 73), SUM(ResolutionWidth + 74), SUM(ResolutionWidth + 75), SUM(ResolutionWidth + 76), SUM(ResolutionWidth + 77), SUM(ResolutionWidth + 78), SUM(ResolutionWidth + 79), SUM(ResolutionWidth + 80), SUM(ResolutionWidth + 81), SUM(ResolutionWidth + 82), SUM(ResolutionWidth + 83), SUM(ResolutionWidth + 84), SUM(ResolutionWidth + 85), SUM(ResolutionWidth + 86), SUM(ResolutionWidth + 87), SUM(ResolutionWidth + 88), SUM(ResolutionWidth + 89) FROM hits; +SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10; +SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10; +SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, COUNT(*) AS c FROM hits GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10; +SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10; +SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN (SearchEngineID = 0 AND AdvEngineID = 0) THEN Referer ELSE '' END AS Src, URL AS Dst, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000; +SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100; +SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000; +SELECT DATE_TRUNC('minute', EventTime) AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_TRUNC('minute', EventTime) ORDER BY DATE_TRUNC('minute', EventTime) LIMIT 10 OFFSET 1000; diff --git a/docker/test/clickbench/run.sh b/docker/test/clickbench/run.sh new file mode 100755 index 00000000000..6e02a346f04 --- /dev/null +++ b/docker/test/clickbench/run.sh @@ -0,0 +1,52 @@ +#!/bin/bash + +# shellcheck disable=SC1091 +source /setup_export_logs.sh + +# fail on errors, verbose and export all env variables +set -e -x -a + +dpkg -i package_folder/clickhouse-common-static_*.deb +dpkg -i package_folder/clickhouse-server_*.deb +dpkg -i package_folder/clickhouse-client_*.deb + +# A directory for cache +sudo mkdir /dev/shm/clickhouse +sudo chown clickhouse:clickhouse /dev/shm/clickhouse + +sudo clickhouse start + +# Wait for the server to start, but not for too long. +for _ in {1..100} +do + clickhouse-client --query "SELECT 1" && break + sleep 1 +done + +setup_logs_replication + +# Load the data + +clickhouse-client --time < /create.sql + +# Run the queries + +TRIES=3 +QUERY_NUM=1 +cat /queries.sql | while read query; do + echo -n "[" + for i in $(seq 1 $TRIES); do + RES=$(clickhouse-client --time --format Null --query "$query" --progress 0 2>&1 ||:) + [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" + [[ "$i" != $TRIES ]] && echo -n ", " + + echo "${QUERY_NUM},${i},${RES}" >> /test_output/test_results.tsv + done + echo "]," + + QUERY_NUM=$((QUERY_NUM + 1)) +done + +clickhouse-client --query "SELECT total_bytes FROM system.tables WHERE name = 'hits' AND database = 'default'" + +echo -e "success\tClickBench finished" > /test_output/check_status.tsv diff --git a/tests/ci/clickbench.py b/tests/ci/clickbench.py new file mode 100644 index 00000000000..0d3452b7a64 --- /dev/null +++ b/tests/ci/clickbench.py @@ -0,0 +1,261 @@ +#!/usr/bin/env python3 + +import argparse +import csv +import logging +import os +import re +import subprocess +import sys +import atexit +from pathlib import Path +from typing import List, Tuple + +from github import Github + +from build_download_helper import download_all_deb_packages +from clickhouse_helper import ( + CiLogsCredentials, + ClickHouseHelper, + prepare_tests_results_for_clickhouse, +) +from commit_status_helper import ( + NotSet, + RerunHelper, + get_commit, + override_status, + post_commit_status, + post_commit_status_to_file, + update_mergeable_check, +) +from docker_pull_helper import DockerImage, get_image_with_version +from download_release_packages import download_last_release +from env_helper import TEMP_PATH, REPO_COPY, REPORTS_PATH +from get_robot_token import get_best_robot_token +from pr_info import FORCE_TESTS_LABEL, PRInfo +from s3_helper import S3Helper +from stopwatch import Stopwatch +from tee_popen import TeePopen +from upload_result_helper import upload_results + +NO_CHANGES_MSG = "Nothing to run" + + +def get_image_name(check_name: str) -> str: + return "clickhouse/clickbench" + + +def get_run_command( + check_name: str, + builds_path: Path, + result_path: Path, + server_log_path: Path, + additional_envs: List[str], + ci_logs_args: str, + image: DockerImage, +) -> str: + + envs = [f"-e {e}" for e in additional_envs] + + env_str = " ".join(envs) + + return ( + f"docker run --volume={builds_path}:/package_folder " + f"{ci_logs_args}" + f"--volume={result_path}:/test_output " + f"--volume={server_log_path}:/var/log/clickhouse-server " + f"--cap-add=SYS_PTRACE {env_str} {image}" + ) + +def process_results( + result_directory: Path, + server_log_path: Path, +) -> Tuple[str, str, TestResults, List[Path]]: + test_results = [] # type: TestResults + additional_files = [] + # Just upload all files from result_directory. + # If task provides processed results, then it's responsible for content of result_directory. + if result_directory.exists(): + additional_files = [p for p in result_directory.iterdir() if p.is_file()] + + if server_log_path.exists(): + additional_files = additional_files + [ + p for p in server_log_path.iterdir() if p.is_file() + ] + + status = [] + status_path = result_directory / "check_status.tsv" + if status_path.exists(): + logging.info("Found check_status.tsv") + with open(status_path, "r", encoding="utf-8") as status_file: + status = list(csv.reader(status_file, delimiter="\t")) + + if len(status) != 1 or len(status[0]) != 2: + logging.info("Files in result folder %s", os.listdir(result_directory)) + return "error", "Invalid check_status.tsv", test_results, additional_files + state, description = status[0][0], status[0][1] + + try: + results_path = result_directory / "test_results.tsv" + + if results_path.exists(): + logging.info("Found test_results.tsv") + else: + logging.info("Files in result folder %s", os.listdir(result_directory)) + return "error", "Not found test_results.tsv", test_results, additional_files + + except Exception as e: + return ( + "error", + f"Cannot parse test_results.tsv ({e})", + test_results, + additional_files, + ) + + return state, description, test_results, additional_files + + +def parse_args(): + parser = argparse.ArgumentParser() + parser.add_argument("check_name") + parser.add_argument("kill_timeout", type=int) + parser.add_argument( + "--post-commit-status", + default="commit_status", + choices=["commit_status", "file"], + help="Where to public post commit status", + ) + return parser.parse_args() + + +def main(): + logging.basicConfig(level=logging.INFO) + + stopwatch = Stopwatch() + + temp_path = Path(TEMP_PATH) + temp_path.mkdir(parents=True, exist_ok=True) + + repo_path = Path(REPO_COPY) + reports_path = Path(REPORTS_PATH) + post_commit_path = temp_path / "clickbench_status.tsv" + + args = parse_args() + check_name = args.check_name + kill_timeout = args.kill_timeout + + gh = Github(get_best_robot_token(), per_page=100) + + pr_info = PRInfo() + + commit = get_commit(gh, pr_info.sha) + atexit.register(update_mergeable_check, gh, pr_info, check_name) + + rerun_helper = RerunHelper(commit, check_name) + if rerun_helper.is_already_finished_by_status(): + logging.info("Check is already finished according to github status, exiting") + sys.exit(0) + + image_name = get_image_name(check_name) + docker_image = get_image_with_version(reports_path, image_name) + + packages_path = temp_path / "packages" + packages_path.mkdir(parents=True, exist_ok=True) + + download_all_deb_packages(check_name, reports_path, packages_path) + + server_log_path = temp_path / "server_log" + server_log_path.mkdir(parents=True, exist_ok=True) + + result_path = temp_path / "result_path" + result_path.mkdir(parents=True, exist_ok=True) + + run_log_path = result_path / "run.log" + + additional_envs = [] + + ci_logs_credentials = CiLogsCredentials(temp_path / "export-logs-config.sh") + ci_logs_args = ci_logs_credentials.get_docker_arguments( + pr_info, stopwatch.start_time_str, check_name + ) + + run_command = get_run_command( + check_name, + packages_path, + repo_path, + result_path, + server_log_path, + kill_timeout, + additional_envs, + ci_logs_args, + docker_image, + ) + logging.info("Going to run ClickBench: %s", run_command) + + with TeePopen(run_command, run_log_path) as process: + retcode = process.wait() + if retcode == 0: + logging.info("Run successfully") + else: + logging.info("Run failed") + + try: + subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) + except subprocess.CalledProcessError: + logging.warning("Failed to change files owner in %s, ignoring it", temp_path) + + ci_logs_credentials.clean_ci_logs_from_credentials(run_log_path) + s3_helper = S3Helper() + + state, description, test_results, additional_logs = process_results( + result_path, server_log_path + ) + state = override_status(state, check_name) + + ch_helper = ClickHouseHelper() + + report_url = upload_results( + s3_helper, + pr_info.number, + pr_info.sha, + test_results, + [run_log_path] + additional_logs, + check_name, + ) + + print(f"::notice:: {check_name} Report url: {report_url}") + if args.post_commit_status == "commit_status": + post_commit_status( + commit, state, report_url, description, check_name_with_group, pr_info + ) + elif args.post_commit_status == "file": + post_commit_status_to_file( + post_commit_path, + description, + state, + report_url, + ) + else: + raise Exception( + f'Unknown post_commit_status option "{args.post_commit_status}"' + ) + + prepared_events = prepare_tests_results_for_clickhouse( + pr_info, + test_results, + state, + stopwatch.duration_seconds, + stopwatch.start_time_str, + report_url, + ) + ch_helper.insert_events_into(db="default", table="checks", events=prepared_events) + + if state != "success": + if FORCE_TESTS_LABEL in pr_info.labels: + print(f"'{FORCE_TESTS_LABEL}' enabled, will report success") + else: + sys.exit(1) + + +if __name__ == "__main__": + main() diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index ee046d6dfef..f5c7342d6f4 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -72,7 +72,7 @@ def process_results(result_directory: Path) -> Tuple[str, str, TestResults]: status = [] status_path = result_directory / "check_status.tsv" if status_path.exists(): - logging.info("Found test_results.tsv") + logging.info("Found check_status.tsv") with open(status_path, "r", encoding="utf-8") as status_file: status = list(csv.reader(status_file, delimiter="\t")) if len(status) != 1 or len(status[0]) != 2: diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index f85c44f617d..c8b3e42eed4 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -169,7 +169,7 @@ def process_results( status = [] status_path = result_directory / "check_status.tsv" if status_path.exists(): - logging.info("Found test_results.tsv") + logging.info("Found check_status.tsv") with open(status_path, "r", encoding="utf-8") as status_file: status = list(csv.reader(status_file, delimiter="\t")) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 8535d71c5c4..b22aa08354c 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -118,7 +118,7 @@ def process_results( status = [] status_path = result_directory / "check_status.tsv" if status_path.exists(): - logging.info("Found test_results.tsv") + logging.info("Found check_status.tsv") with open(status_path, "r", encoding="utf-8") as status_file: status = list(csv.reader(status_file, delimiter="\t")) diff --git a/tests/ci/test_docker.py b/tests/ci/test_docker.py index 8aab50ed082..f15fd1ae63f 100644 --- a/tests/ci/test_docker.py +++ b/tests/ci/test_docker.py @@ -64,6 +64,12 @@ class TestDockerImageCheck(unittest.TestCase): False, "clickhouse/test-base", # type: ignore ), + di.DockerImage( + "docker/test/clickbench", + "clickhouse/clickbench", + False, + "clickhouse/test-base", # type: ignore + ), di.DockerImage( "docker/test/keeper-jepsen", "clickhouse/keeper-jepsen-test", diff --git a/tests/ci/tests/docker_images_for_tests.json b/tests/ci/tests/docker_images_for_tests.json index 70db8760561..19e4b94bcdf 100644 --- a/tests/ci/tests/docker_images_for_tests.json +++ b/tests/ci/tests/docker_images_for_tests.json @@ -120,6 +120,7 @@ "docker/test/integration/base", "docker/test/fuzzer", "docker/test/keeper-jepsen", + "docker/test/clickbench", "docker/test/sqltest" ] }, @@ -131,6 +132,10 @@ "name": "clickhouse/sqlancer-test", "dependent": [] }, + "docker/test/clickbench": { + "name": "clickhouse/clickbench", + "dependent": [] + }, "docker/test/keeper-jepsen": { "name": "clickhouse/keeper-jepsen-test", "dependent": [] From fc8656ee9ea27cc486caec79adee2dc8adca2870 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sat, 9 Dec 2023 22:44:06 +0000 Subject: [PATCH 193/331] Fix unary operators parsing --- src/Parsers/ExpressionListParsers.cpp | 13 +++++++++++-- .../02920_unary_operators_functions.reference | 1 + .../0_stateless/02920_unary_operators_functions.sql | 1 + 3 files changed, 13 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02920_unary_operators_functions.reference create mode 100644 tests/queries/0_stateless/02920_unary_operators_functions.sql diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 344f9b868b0..a7095111dda 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2614,8 +2614,17 @@ Action ParserExpressionImpl::tryParseOperand(Layers & layers, IParser::Pos & pos if (cur_op != unary_operators_table.end()) { - layers.back()->pushOperator(cur_op->second); - return Action::OPERAND; + if (pos->type == TokenType::OpeningRoundBracket) + { + ++pos; + auto identifier = std::make_shared(cur_op->second.function_name); + layers.push_back(getFunctionLayer(identifier, layers.front()->is_table_function)); + return Action::OPERAND; + } + else + { + layers.back()->pushOperator(cur_op->second); + } } auto old_pos = pos; diff --git a/tests/queries/0_stateless/02920_unary_operators_functions.reference b/tests/queries/0_stateless/02920_unary_operators_functions.reference new file mode 100644 index 00000000000..0cfbf08886f --- /dev/null +++ b/tests/queries/0_stateless/02920_unary_operators_functions.reference @@ -0,0 +1 @@ +2 diff --git a/tests/queries/0_stateless/02920_unary_operators_functions.sql b/tests/queries/0_stateless/02920_unary_operators_functions.sql new file mode 100644 index 00000000000..3f3c3a1618a --- /dev/null +++ b/tests/queries/0_stateless/02920_unary_operators_functions.sql @@ -0,0 +1 @@ +SELECT NOT (0) + NOT (0); \ No newline at end of file From f8f42d82ddcdb3f48f4ceab0154486ad4b80a8b0 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sat, 9 Dec 2023 22:52:58 +0000 Subject: [PATCH 194/331] Automatic style fix --- tests/ci/clickbench.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/clickbench.py b/tests/ci/clickbench.py index 0d3452b7a64..2b320d69e7d 100644 --- a/tests/ci/clickbench.py +++ b/tests/ci/clickbench.py @@ -54,7 +54,6 @@ def get_run_command( ci_logs_args: str, image: DockerImage, ) -> str: - envs = [f"-e {e}" for e in additional_envs] env_str = " ".join(envs) @@ -67,6 +66,7 @@ def get_run_command( f"--cap-add=SYS_PTRACE {env_str} {image}" ) + def process_results( result_directory: Path, server_log_path: Path, From 17772dc41c3098c5a8411e80cd970e6d73c45ce2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Dec 2023 00:03:33 +0100 Subject: [PATCH 195/331] Recommendation from @felixoid --- tests/ci/tests/docker_images_for_tests.json | 5 ----- 1 file changed, 5 deletions(-) diff --git a/tests/ci/tests/docker_images_for_tests.json b/tests/ci/tests/docker_images_for_tests.json index 19e4b94bcdf..70db8760561 100644 --- a/tests/ci/tests/docker_images_for_tests.json +++ b/tests/ci/tests/docker_images_for_tests.json @@ -120,7 +120,6 @@ "docker/test/integration/base", "docker/test/fuzzer", "docker/test/keeper-jepsen", - "docker/test/clickbench", "docker/test/sqltest" ] }, @@ -132,10 +131,6 @@ "name": "clickhouse/sqlancer-test", "dependent": [] }, - "docker/test/clickbench": { - "name": "clickhouse/clickbench", - "dependent": [] - }, "docker/test/keeper-jepsen": { "name": "clickhouse/keeper-jepsen-test", "dependent": [] From f2875068be79c2b1838aad693e338e0bbf8b3b96 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Dec 2023 00:19:30 +0100 Subject: [PATCH 196/331] Style --- docker/test/clickbench/run.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docker/test/clickbench/run.sh b/docker/test/clickbench/run.sh index 6e02a346f04..eadaa2b5939 100755 --- a/docker/test/clickbench/run.sh +++ b/docker/test/clickbench/run.sh @@ -33,19 +33,19 @@ clickhouse-client --time < /create.sql TRIES=3 QUERY_NUM=1 -cat /queries.sql | while read query; do +while read query; do echo -n "[" for i in $(seq 1 $TRIES); do RES=$(clickhouse-client --time --format Null --query "$query" --progress 0 2>&1 ||:) - [[ "$?" == "0" ]] && echo -n "${RES}" || echo -n "null" - [[ "$i" != $TRIES ]] && echo -n ", " + echo -n "${RES}" + [[ "$i" != "$TRIES" ]] && echo -n ", " echo "${QUERY_NUM},${i},${RES}" >> /test_output/test_results.tsv done echo "]," QUERY_NUM=$((QUERY_NUM + 1)) -done +done < /queries.sql clickhouse-client --query "SELECT total_bytes FROM system.tables WHERE name = 'hits' AND database = 'default'" From 2602a7ba817b23668f46aade018601da3214669b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Dec 2023 00:22:19 +0100 Subject: [PATCH 197/331] Python --- tests/ci/clickbench.py | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/tests/ci/clickbench.py b/tests/ci/clickbench.py index 2b320d69e7d..5f41d3d497f 100644 --- a/tests/ci/clickbench.py +++ b/tests/ci/clickbench.py @@ -41,12 +41,11 @@ from upload_result_helper import upload_results NO_CHANGES_MSG = "Nothing to run" -def get_image_name(check_name: str) -> str: +def get_image_name() -> str: return "clickhouse/clickbench" def get_run_command( - check_name: str, builds_path: Path, result_path: Path, server_log_path: Path, @@ -70,8 +69,8 @@ def get_run_command( def process_results( result_directory: Path, server_log_path: Path, -) -> Tuple[str, str, TestResults, List[Path]]: - test_results = [] # type: TestResults +) -> Tuple[str, str, List[str], List[Path]]: + test_results = [] additional_files = [] # Just upload all files from result_directory. # If task provides processed results, then it's responsible for content of result_directory. @@ -156,7 +155,7 @@ def main(): logging.info("Check is already finished according to github status, exiting") sys.exit(0) - image_name = get_image_name(check_name) + image_name = get_image_name() docker_image = get_image_with_version(reports_path, image_name) packages_path = temp_path / "packages" @@ -180,9 +179,7 @@ def main(): ) run_command = get_run_command( - check_name, packages_path, - repo_path, result_path, server_log_path, kill_timeout, From ea31c8471f11ccc4c753ae98643fbac3715c2b29 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Dec 2023 00:26:44 +0100 Subject: [PATCH 198/331] Python --- tests/ci/clickbench.py | 5 ----- 1 file changed, 5 deletions(-) diff --git a/tests/ci/clickbench.py b/tests/ci/clickbench.py index 5f41d3d497f..82209acd2c4 100644 --- a/tests/ci/clickbench.py +++ b/tests/ci/clickbench.py @@ -4,7 +4,6 @@ import argparse import csv import logging import os -import re import subprocess import sys import atexit @@ -20,7 +19,6 @@ from clickhouse_helper import ( prepare_tests_results_for_clickhouse, ) from commit_status_helper import ( - NotSet, RerunHelper, get_commit, override_status, @@ -29,7 +27,6 @@ from commit_status_helper import ( update_mergeable_check, ) from docker_pull_helper import DockerImage, get_image_with_version -from download_release_packages import download_last_release from env_helper import TEMP_PATH, REPO_COPY, REPORTS_PATH from get_robot_token import get_best_robot_token from pr_info import FORCE_TESTS_LABEL, PRInfo @@ -38,8 +35,6 @@ from stopwatch import Stopwatch from tee_popen import TeePopen from upload_result_helper import upload_results -NO_CHANGES_MSG = "Nothing to run" - def get_image_name() -> str: return "clickhouse/clickbench" From 9a3860581c4a8e3b01026c6355cb9ea6110a7ea1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Dec 2023 00:27:11 +0100 Subject: [PATCH 199/331] YAML --- .github/workflows/pull_request.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 3db382b1fd6..5bb62b04c32 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -715,7 +715,7 @@ jobs: cd "$REPO_COPY/tests/ci" python3 clickbench.py "$CHECK_NAME" "$KILL_TIMEOUT" ClickBenchAarch64: - needs: [ BuilderDebAarch64 ] + needs: [BuilderDebAarch64] uses: ./.github/workflows/reusable_test.yml with: test_name: ClickBench (aarch64) From af48cb97322541495e496bfc1a346186584d936e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Dec 2023 00:35:04 +0100 Subject: [PATCH 200/331] Python --- tests/ci/test_docker.py | 6 ------ 1 file changed, 6 deletions(-) diff --git a/tests/ci/test_docker.py b/tests/ci/test_docker.py index f15fd1ae63f..8aab50ed082 100644 --- a/tests/ci/test_docker.py +++ b/tests/ci/test_docker.py @@ -64,12 +64,6 @@ class TestDockerImageCheck(unittest.TestCase): False, "clickhouse/test-base", # type: ignore ), - di.DockerImage( - "docker/test/clickbench", - "clickhouse/clickbench", - False, - "clickhouse/test-base", # type: ignore - ), di.DockerImage( "docker/test/keeper-jepsen", "clickhouse/keeper-jepsen-test", From 100b36e8a912b090e79a3181058149afc1ddf47a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Dec 2023 01:32:42 +0100 Subject: [PATCH 201/331] 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 202/331] 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 ab2f60cceffa588e9f46359e142273aefdddba95 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 10 Dec 2023 02:45:51 +0100 Subject: [PATCH 203/331] sleep() function now can be cancelled with KILL QUERY. --- src/Functions/sleep.h | 25 ++++++++++-- .../02932_kill_query_sleep.reference | 2 + .../0_stateless/02932_kill_query_sleep.sh | 38 +++++++++++++++++++ 3 files changed, 62 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02932_kill_query_sleep.reference create mode 100755 tests/queries/0_stateless/02932_kill_query_sleep.sh diff --git a/src/Functions/sleep.h b/src/Functions/sleep.h index 160a8a2afe2..36fa14cd36e 100644 --- a/src/Functions/sleep.h +++ b/src/Functions/sleep.h @@ -10,6 +10,7 @@ #include #include #include +#include namespace ProfileEvents @@ -27,6 +28,7 @@ namespace ErrorCodes extern const int TOO_SLOW; extern const int ILLEGAL_COLUMN; extern const int BAD_ARGUMENTS; + extern const int QUERY_WAS_CANCELLED; } /** sleep(seconds) - the specified number of seconds sleeps each columns. @@ -43,15 +45,20 @@ class FunctionSleep : public IFunction { private: UInt64 max_microseconds; + QueryStatusPtr query_status; + public: static constexpr auto name = variant == FunctionSleepVariant::PerBlock ? "sleep" : "sleepEachRow"; static FunctionPtr create(ContextPtr context) { - return std::make_shared>(context->getSettingsRef().function_sleep_max_microseconds_per_block); + return std::make_shared>( + context->getSettingsRef().function_sleep_max_microseconds_per_block, + context->getProcessListElementSafe()); } - FunctionSleep(UInt64 max_microseconds_) + FunctionSleep(UInt64 max_microseconds_, QueryStatusPtr query_status_) : max_microseconds(std::min(max_microseconds_, static_cast(std::numeric_limits::max()))) + , query_status(query_status_) { } @@ -128,7 +135,19 @@ public: "The maximum sleep time is {} microseconds. Requested: {} microseconds per block (of size {})", max_microseconds, microseconds, size); - sleepForMicroseconds(microseconds); + while (microseconds) + { + UInt64 sleep_ms = microseconds; + if (query_status) + sleep_ms = std::min(sleep_ms, /* 1 second */ static_cast(1000000)); + + sleepForMicroseconds(sleep_ms); + microseconds -= sleep_ms; + + if (query_status && query_status->isKilled()) + throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled"); + } + ProfileEvents::increment(ProfileEvents::SleepFunctionCalls, count); ProfileEvents::increment(ProfileEvents::SleepFunctionMicroseconds, microseconds); } diff --git a/tests/queries/0_stateless/02932_kill_query_sleep.reference b/tests/queries/0_stateless/02932_kill_query_sleep.reference new file mode 100644 index 00000000000..9c19635a83f --- /dev/null +++ b/tests/queries/0_stateless/02932_kill_query_sleep.reference @@ -0,0 +1,2 @@ +Cancelling query +QUERY_WAS_CANCELLED diff --git a/tests/queries/0_stateless/02932_kill_query_sleep.sh b/tests/queries/0_stateless/02932_kill_query_sleep.sh new file mode 100755 index 00000000000..81bb892bc15 --- /dev/null +++ b/tests/queries/0_stateless/02932_kill_query_sleep.sh @@ -0,0 +1,38 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +function wait_query_started() +{ + local query_id="$1" + while [[ $($CLICKHOUSE_CLIENT --query="SELECT count() FROM system.query_log WHERE query_id='$query_id'") == 0 ]]; do + sleep 0.1; + $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS;" + done +} + +function kill_query() +{ + local query_id="$1" + $CLICKHOUSE_CLIENT --query "KILL QUERY WHERE query_id='$query_id'" >/dev/null + while [[ $($CLICKHOUSE_CLIENT --query="SELECT count() FROM system.processes WHERE query_id='$query_id'") != 0 ]]; do sleep 0.1; done +} + + +sleep_query_id="sleep_query_id_02932_kill_query_sleep_${CLICKHOUSE_DATABASE}_$RANDOM" + +# This sleep query wants to sleep for 1000 seconds (which is too long). +# We're going to cancel this query later. +sleep_query="SELECT sleep(1000)" + +$CLICKHOUSE_CLIENT --query_id="$sleep_query_id" --function_sleep_max_microseconds_per_block="1000000000" --query "$sleep_query" >/dev/null 2>&1 & +wait_query_started "$sleep_query_id" + +echo "Cancelling query" +kill_query "$sleep_query_id" + +$CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS;" +$CLICKHOUSE_CLIENT --query "SELECT exception FROM system.query_log WHERE query_id='$sleep_query_id'" | grep -oF "QUERY_WAS_CANCELLED" From bfe2ffbce9fcc4650f6350aa1f26adc4067362f0 Mon Sep 17 00:00:00 2001 From: Bharat Nallan Chakravarthy Date: Sat, 9 Dec 2023 16:41:54 -0800 Subject: [PATCH 204/331] 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 205/331] 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 206/331] 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 207/331] 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 208/331] 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 209/331] 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 210/331] 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 8b5c69468493619b4ae51d0049440175d57a224c Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sun, 10 Dec 2023 16:45:17 +0000 Subject: [PATCH 211/331] Fix --- src/Parsers/ExpressionListParsers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index a7095111dda..b5db06e96cb 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2619,12 +2619,12 @@ Action ParserExpressionImpl::tryParseOperand(Layers & layers, IParser::Pos & pos ++pos; auto identifier = std::make_shared(cur_op->second.function_name); layers.push_back(getFunctionLayer(identifier, layers.front()->is_table_function)); - return Action::OPERAND; } else { layers.back()->pushOperator(cur_op->second); } + return Action::OPERAND; } auto old_pos = pos; From 450e8da7ce60acfca52c8fb5ae1801c492ad2cc7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Dec 2023 18:17:41 +0100 Subject: [PATCH 212/331] 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 bb501a0e136c41c5ac8573c010822ecaa30d136d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Dec 2023 19:06:38 +0100 Subject: [PATCH 213/331] Shellcheck --- docker/test/clickbench/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/clickbench/run.sh b/docker/test/clickbench/run.sh index eadaa2b5939..8244497a36c 100755 --- a/docker/test/clickbench/run.sh +++ b/docker/test/clickbench/run.sh @@ -33,7 +33,7 @@ clickhouse-client --time < /create.sql TRIES=3 QUERY_NUM=1 -while read query; do +while read -r query; do echo -n "[" for i in $(seq 1 $TRIES); do RES=$(clickhouse-client --time --format Null --query "$query" --progress 0 2>&1 ||:) From 11dda11f22f252d4996171bcc2887a6664878588 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Dec 2023 19:08:50 +0100 Subject: [PATCH 214/331] Python --- tests/ci/clickbench.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/ci/clickbench.py b/tests/ci/clickbench.py index 82209acd2c4..661a6ba50dc 100644 --- a/tests/ci/clickbench.py +++ b/tests/ci/clickbench.py @@ -130,7 +130,6 @@ def main(): temp_path = Path(TEMP_PATH) temp_path.mkdir(parents=True, exist_ok=True) - repo_path = Path(REPO_COPY) reports_path = Path(REPORTS_PATH) post_commit_path = temp_path / "clickbench_status.tsv" @@ -177,7 +176,6 @@ def main(): packages_path, result_path, server_log_path, - kill_timeout, additional_envs, ci_logs_args, docker_image, @@ -218,7 +216,7 @@ def main(): print(f"::notice:: {check_name} Report url: {report_url}") if args.post_commit_status == "commit_status": post_commit_status( - commit, state, report_url, description, check_name_with_group, pr_info + commit, state, report_url, description, check_name, pr_info ) elif args.post_commit_status == "file": post_commit_status_to_file( @@ -239,6 +237,7 @@ def main(): stopwatch.duration_seconds, stopwatch.start_time_str, report_url, + check_name ) ch_helper.insert_events_into(db="default", table="checks", events=prepared_events) From 99c8430c3e3b25ef4098e7006ec751fa34cabdd9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Dec 2023 19:10:44 +0100 Subject: [PATCH 215/331] mypy --- tests/ci/clickbench.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/clickbench.py b/tests/ci/clickbench.py index 661a6ba50dc..657695a57a1 100644 --- a/tests/ci/clickbench.py +++ b/tests/ci/clickbench.py @@ -65,8 +65,8 @@ def process_results( result_directory: Path, server_log_path: Path, ) -> Tuple[str, str, List[str], List[Path]]: - test_results = [] - additional_files = [] + test_results = [] # type: List[str] + additional_files = [] # type: List[str] # Just upload all files from result_directory. # If task provides processed results, then it's responsible for content of result_directory. if result_directory.exists(): From c1e387d682df837782ea1b5dde828e38f6d18795 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 10 Dec 2023 18:32:09 +0000 Subject: [PATCH 216/331] Automatic style fix --- tests/ci/clickbench.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/tests/ci/clickbench.py b/tests/ci/clickbench.py index 657695a57a1..c2efcacefad 100644 --- a/tests/ci/clickbench.py +++ b/tests/ci/clickbench.py @@ -215,9 +215,7 @@ def main(): print(f"::notice:: {check_name} Report url: {report_url}") if args.post_commit_status == "commit_status": - post_commit_status( - commit, state, report_url, description, check_name, pr_info - ) + post_commit_status(commit, state, report_url, description, check_name, pr_info) elif args.post_commit_status == "file": post_commit_status_to_file( post_commit_path, @@ -237,7 +235,7 @@ def main(): stopwatch.duration_seconds, stopwatch.start_time_str, report_url, - check_name + check_name, ) ch_helper.insert_events_into(db="default", table="checks", events=prepared_events) From 75df8db29870de080c60684889f9a7440492d248 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Dec 2023 19:52:52 +0100 Subject: [PATCH 217/331] Python --- tests/ci/clickbench.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/ci/clickbench.py b/tests/ci/clickbench.py index 657695a57a1..8e6b9a621fe 100644 --- a/tests/ci/clickbench.py +++ b/tests/ci/clickbench.py @@ -27,7 +27,7 @@ from commit_status_helper import ( update_mergeable_check, ) from docker_pull_helper import DockerImage, get_image_with_version -from env_helper import TEMP_PATH, REPO_COPY, REPORTS_PATH +from env_helper import TEMP_PATH, REPORTS_PATH from get_robot_token import get_best_robot_token from pr_info import FORCE_TESTS_LABEL, PRInfo from s3_helper import S3Helper @@ -135,7 +135,6 @@ def main(): args = parse_args() check_name = args.check_name - kill_timeout = args.kill_timeout gh = Github(get_best_robot_token(), per_page=100) From ce894c28f3c22ac52f359442c6f583608f771503 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Dec 2023 19:54:59 +0100 Subject: [PATCH 218/331] mypy --- tests/ci/clickbench.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/clickbench.py b/tests/ci/clickbench.py index 8e6b9a621fe..870d83ecded 100644 --- a/tests/ci/clickbench.py +++ b/tests/ci/clickbench.py @@ -65,8 +65,8 @@ def process_results( result_directory: Path, server_log_path: Path, ) -> Tuple[str, str, List[str], List[Path]]: - test_results = [] # type: List[str] - additional_files = [] # type: List[str] + test_results = [] # type: List[TestResult] + additional_files = [] # type: List[Path] # Just upload all files from result_directory. # If task provides processed results, then it's responsible for content of result_directory. if result_directory.exists(): @@ -164,7 +164,7 @@ def main(): run_log_path = result_path / "run.log" - additional_envs = [] + additional_envs = [] # type: List[str] ci_logs_credentials = CiLogsCredentials(temp_path / "export-logs-config.sh") ci_logs_args = ci_logs_credentials.get_docker_arguments( From 6b55c16b4ed40864aa0577fa61a9a6a41c12912d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Dec 2023 20:49:03 +0100 Subject: [PATCH 219/331] mypy --- tests/ci/clickbench.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/ci/clickbench.py b/tests/ci/clickbench.py index 3cce429a4e3..9c700adb398 100644 --- a/tests/ci/clickbench.py +++ b/tests/ci/clickbench.py @@ -34,6 +34,7 @@ from s3_helper import S3Helper from stopwatch import Stopwatch from tee_popen import TeePopen from upload_result_helper import upload_results +from report import TestResults def get_image_name() -> str: @@ -64,8 +65,8 @@ def get_run_command( def process_results( result_directory: Path, server_log_path: Path, -) -> Tuple[str, str, List[str], List[Path]]: - test_results = [] # type: List[TestResult] +) -> Tuple[str, str, TestResults, List[Path]]: + test_results = [] # type: TestResults additional_files = [] # type: List[Path] # Just upload all files from result_directory. # If task provides processed results, then it's responsible for content of result_directory. From 76ae90c5ae5f102a62c09731a311a6adaed4169f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 10 Dec 2023 20:25:08 +0000 Subject: [PATCH 220/331] test added --- .../0_stateless/02935_ipv6_bit_operations.reference | 1 + tests/queries/0_stateless/02935_ipv6_bit_operations.sql | 7 +++++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/02935_ipv6_bit_operations.reference create mode 100644 tests/queries/0_stateless/02935_ipv6_bit_operations.sql diff --git a/tests/queries/0_stateless/02935_ipv6_bit_operations.reference b/tests/queries/0_stateless/02935_ipv6_bit_operations.reference new file mode 100644 index 00000000000..22d5cda0a39 --- /dev/null +++ b/tests/queries/0_stateless/02935_ipv6_bit_operations.reference @@ -0,0 +1 @@ +11111111111111110000000000000000111111111111111100000000000000001111111111111111000000000000000011111111111111110000000000000000 00000000000000001111111111111111000000000000000011111111111111110000000000000000111111111111111100000000000000001111111111111111 10101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010 01010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101 10101010101010100000000000000000101010101010101000000000000000001010101010101010000000000000000010101010101010100000000000000000 10101010101010100000000000000000101010101010101000000000000000001010101010101010000000000000000010101010101010100000000000000000 1010101010101010000000000000000010101010101010100000000000000000101010101010101000000000000000001010101010101010 1010101010101010000000000000000010101010101010100000000000000000101010101010101000000000000000001010101010101010 01010101010101010000000000000000010101010101010100000000000000000101010101010101000000000000000001010101010101010000000000000000 01010101010101010000000000000000010101010101010100000000000000000101010101010101000000000000000001010101010101010000000000000000 0101010101010101000000000000000001010101010101010000000000000000010101010101010100000000000000000101010101010101 0101010101010101000000000000000001010101010101010000000000000000010101010101010100000000000000000101010101010101 11111111111111111010101010101010111111111111111110101010101010101111111111111111101010101010101011111111111111111010101010101010 11111111111111111010101010101010111111111111111110101010101010101111111111111111101010101010101011111111111111111010101010101010 10101010101010101111111111111111101010101010101011111111111111111010101010101010111111111111111110101010101010101111111111111111 10101010101010101111111111111111101010101010101011111111111111111010101010101010111111111111111110101010101010101111111111111111 11111111111111110101010101010101111111111111111101010101010101011111111111111111010101010101010111111111111111110101010101010101 11111111111111110101010101010101111111111111111101010101010101011111111111111111010101010101010111111111111111110101010101010101 01010101010101011111111111111111010101010101010111111111111111110101010101010101111111111111111101010101010101011111111111111111 01010101010101011111111111111111010101010101010111111111111111110101010101010101111111111111111101010101010101011111111111111111 diff --git a/tests/queries/0_stateless/02935_ipv6_bit_operations.sql b/tests/queries/0_stateless/02935_ipv6_bit_operations.sql new file mode 100644 index 00000000000..6598c2ac539 --- /dev/null +++ b/tests/queries/0_stateless/02935_ipv6_bit_operations.sql @@ -0,0 +1,7 @@ +WITH toIPv6('FFFF:0000:FFFF:0000:FFFF:0000:FFFF:0000') AS ip1, toIPv6('0000:FFFF:0000:FFFF:0000:FFFF:0000:FFFF') AS ip2, + CAST('226854911280625642308916404954512140970', 'UInt128') AS n1, CAST('113427455640312821154458202477256070485', 'UInt128') AS n2 +SELECT bin(ip1), bin(ip2), bin(n1), bin(n2), + bin(bitAnd(ip1, n1)), bin(bitAnd(n1, ip1)), bin(bitAnd(ip2, n1)), bin(bitAnd(n1, ip2)), + bin(bitAnd(ip1, n2)), bin(bitAnd(n2, ip1)), bin(bitAnd(ip2, n2)), bin(bitAnd(n2, ip2)), + bin(bitOr(ip1, n1)), bin(bitOr(n1, ip1)), bin(bitOr(ip2, n1)), bin(bitOr(n1, ip2)), + bin(bitOr(ip1, n2)), bin(bitOr(n2, ip1)), bin(bitOr(ip2, n2)), bin(bitOr(n2, ip2)); From 644ef7b63f7a96a7675d178f1e329e0895824c4f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Dec 2023 23:51:59 +0300 Subject: [PATCH 221/331] 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 = [] From 4b13a6d08f35daf09ee1f6ef951f260f68523ea5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Dec 2023 22:02:05 +0100 Subject: [PATCH 222/331] Fix CI --- tests/ci/ci_config.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index aa3aa5654aa..e9f75d66b2e 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -329,6 +329,8 @@ CI_CONFIG = CiConfig( "SQLancer (debug)": TestConfig("package_debug"), "Sqllogic test (release)": TestConfig("package_release"), "SQLTest": TestConfig("package_release"), + "ClickBench (amd64)": TestConfig("package_release"), + "ClickBench (aarch64)": TestConfig("package_aarch64"), "libFuzzer tests": TestConfig("fuzzers"), }, ) @@ -507,6 +509,11 @@ CHECK_DESCRIPTIONS = [ "successfully startup without any errors, crashes or sanitizer asserts", lambda x: x.startswith("Upgrade check ("), ), + CheckDescription( + "ClickBench", + "Runs [ClickBench](https://github.com/ClickHouse/ClickBench/) with instant-attach table", + lambda x: x.startswith("Upgrade check ("), + ), CheckDescription( "Falback for unknown", "There's no description for the check yet, please add it to " From 7df7793724c7ff30c0f6c0d475eeda121d9c3cab Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Dec 2023 23:53:22 +0100 Subject: [PATCH 223/331] Docker --- docker/test/clickbench/run.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/clickbench/run.sh b/docker/test/clickbench/run.sh index 8244497a36c..82eb06dffcd 100755 --- a/docker/test/clickbench/run.sh +++ b/docker/test/clickbench/run.sh @@ -11,10 +11,10 @@ dpkg -i package_folder/clickhouse-server_*.deb dpkg -i package_folder/clickhouse-client_*.deb # A directory for cache -sudo mkdir /dev/shm/clickhouse -sudo chown clickhouse:clickhouse /dev/shm/clickhouse +mkdir /dev/shm/clickhouse +chown clickhouse:clickhouse /dev/shm/clickhouse -sudo clickhouse start +clickhouse start # Wait for the server to start, but not for too long. for _ in {1..100} From ed23f47e72ccf3095ddc896cf1a2886c1bbdc45a Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sun, 10 Dec 2023 23:04:29 +0000 Subject: [PATCH 224/331] Yet another fix --- src/Parsers/ExpressionListParsers.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index b5db06e96cb..20f7fa89e51 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -466,7 +466,8 @@ enum class OperatorType StartIf, FinishIf, Cast, - Lambda + Lambda, + Not }; /** Operator struct stores parameters of the operator: @@ -2448,7 +2449,7 @@ const std::vector> ParserExpressionImpl::o const std::vector> ParserExpressionImpl::unary_operators_table { - {"NOT", Operator("not", 5, 1)}, + {"NOT", Operator("not", 5, 1, OperatorType::Not)}, {"-", Operator("negate", 13, 1)}, {"−", Operator("negate", 13, 1)} }; @@ -2614,7 +2615,7 @@ Action ParserExpressionImpl::tryParseOperand(Layers & layers, IParser::Pos & pos if (cur_op != unary_operators_table.end()) { - if (pos->type == TokenType::OpeningRoundBracket) + if (cur_op->second.type == OperatorType::Not && pos->type == TokenType::OpeningRoundBracket) { ++pos; auto identifier = std::make_shared(cur_op->second.function_name); From a39f3ea9f4e81c342879d2bb352e9a39ca00c765 Mon Sep 17 00:00:00 2001 From: Tanya Bragin Date: Sun, 10 Dec 2023 15:33:48 -0800 Subject: [PATCH 225/331] Update developer-instruction.md - add MacOSX link to general guide Refer to MacOSX (and other OS & arch specific guides) in the build section of the general guide. cc @rschu1ze Per our chat, feel free to reject if you think it's not helpful. --- docs/en/development/developer-instruction.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index 9d6a80de904..a5c66d4dde3 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -74,6 +74,10 @@ You can also add original ClickHouse repo address to your local repository to pu After successfully running this command you will be able to pull updates from the main ClickHouse repo by running `git pull upstream master`. +:::note +Instructions below work on most Linux compatible systems, but before proceeding please also check for operating system and architecture specific guides, such as building on (MacOS)[https://clickhouse.com/docs/en/development/build-osx], (Linux for MacOS)[https://clickhouse.com/docs/en/development/build-cross-osx], (Linux for RISC-V 64)[https://clickhouse.com/docs/en/development/build-cross-riscv] and so on. +::: + ## Build System {#build-system} ClickHouse uses CMake and Ninja for building. From fc00569db6b1b9d45464ecbcf42b82e6f2150d21 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 3 Dec 2023 13:44:35 +0100 Subject: [PATCH 226/331] 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 dc83e2716461e1e3694ae4e6dc6a86cdf68325d9 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 3 Dec 2023 13:46:52 +0100 Subject: [PATCH 227/331] 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 f1ccf4e29f37bf7153874f2d95e5f40e017c7cff Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 3 Dec 2023 14:04:54 +0100 Subject: [PATCH 228/331] 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 e2a7f4004ff8cc6113095bd31d77d58537741594 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 3 Dec 2023 19:59:26 +0100 Subject: [PATCH 229/331] 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 4aba3ec685d8d7ccef42cb59b35c39e56302156e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 7 Dec 2023 17:52:30 +0100 Subject: [PATCH 230/331] 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 73547e6d8a383b4e6a28ba7a87884cde610028b7 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 9 Dec 2023 13:50:54 +0100 Subject: [PATCH 231/331] 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 952e58287a18e3ec2f525228e0ce513bf41059da Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 11 Dec 2023 00:48:10 +0100 Subject: [PATCH 232/331] Fix flaky test RWLockWriteLockTimeoutDuringTwoReads, add more comments. --- src/Common/tests/gtest_rw_lock.cpp | 68 ++++++++++++++++++++++++++++-- 1 file changed, 65 insertions(+), 3 deletions(-) diff --git a/src/Common/tests/gtest_rw_lock.cpp b/src/Common/tests/gtest_rw_lock.cpp index 16ba01d02c6..08a14aba8fb 100644 --- a/src/Common/tests/gtest_rw_lock.cpp +++ b/src/Common/tests/gtest_rw_lock.cpp @@ -31,11 +31,13 @@ namespace public: Events() : start_time(std::chrono::steady_clock::now()) {} - void add(String && event) + void add(String && event, std::chrono::milliseconds correction = std::chrono::milliseconds::zero()) { 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(), ' '); + if (correction.count()) + std::this_thread::sleep_for(correction); std::lock_guard lock{mutex}; //std::cout << timepoint << " : " << event << std::endl; events.emplace_back(std::move(event)); @@ -324,6 +326,22 @@ TEST(Common, RWLockNotUpgradeableWithNoQuery) TEST(Common, RWLockWriteLockTimeoutDuringRead) { + /// 0 100 200 300 400 + /// <---------------------------------------- ra ----------------------------------------------> + /// <----- wc (acquiring lock, failed by timeout) -----> + /// + /// + /// 0 : Locking ra + /// 0 : Locked ra + /// 100 : Locking wc + /// 300 : Failed to lock wc + /// 400 : Unlocking ra + /// 400 : Unlocked ra + /// 400 : Locking wd + /// 400 : Locked wd + /// 400 : Unlocking wd + /// 400 : Unlocked wd + static auto rw_lock = RWLockImpl::create(); Events events; @@ -379,6 +397,27 @@ TEST(Common, RWLockWriteLockTimeoutDuringRead) TEST(Common, RWLockWriteLockTimeoutDuringTwoReads) { + /// 0 100 200 300 400 500 + /// <---------------------------------------- ra -----------------------------------------------> + /// <------ wc (acquiring lock, failed by timeout) -------> + /// <-- rb (acquiring lock) --><---------- rb (locked) ------------> + /// + /// + /// 0 : Locking ra + /// 0 : Locked ra + /// 100 : Locking wc + /// 200 : Locking rb + /// 300 : Failed to lock wc + /// 300 : Locked rb + /// 400 : Unlocking ra + /// 400 : Unlocked ra + /// 500 : Unlocking rb + /// 500 : Unlocked rb + /// 501 : Locking wd + /// 501 : Locked wd + /// 501 : Unlocking wd + /// 501 : Unlocked wd + static auto rw_lock = RWLockImpl::create(); Events events; @@ -402,10 +441,14 @@ TEST(Common, RWLockWriteLockTimeoutDuringTwoReads) events.add("Locking rb"); auto rb = rw_lock->getLock(RWLockImpl::Read, "rb"); - events.add(rb ? "Locked rb" : "Failed to lock rb"); + + /// `correction` is used here to add an event to `events` a little later. + /// (Because the event "Locked rb" happens at nearly the same time as "Failed to lock wc" and we don't want our test to be flaky.) + auto correction = std::chrono::duration(50); + events.add(rb ? "Locked rb" : "Failed to lock rb", correction); EXPECT_NE(rb, nullptr); - std::this_thread::sleep_for(std::chrono::duration(200)); + std::this_thread::sleep_for(std::chrono::duration(200) - correction); events.add("Unlocking rb"); rb.reset(); events.add("Unlocked rb"); @@ -454,6 +497,25 @@ TEST(Common, RWLockWriteLockTimeoutDuringTwoReads) TEST(Common, RWLockWriteLockTimeoutDuringWriteWithWaitingRead) { + /// 0 100 200 300 400 500 + /// <--------------------------------------------------- wa --------------------------------------------------------> + /// <------ wb (acquiring lock, failed by timeout) ------> + /// <-- rc (acquiring lock, failed by timeout) --> + /// + /// + /// 0 : Locking wa + /// 0 : Locked wa + /// 100 : Locking wb + /// 200 : Locking rc + /// 300 : Failed to lock wb + /// 400 : Failed to lock rc + /// 500 : Unlocking wa + /// 500 : Unlocked wa + /// 501 : Locking wd + /// 501 : Locked wd + /// 501 : Unlocking wd + /// 501 : Unlocked wd + static auto rw_lock = RWLockImpl::create(); Events events; From 740ceea108f61d8cda725d6a1a5c0f26b4b399cd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Dec 2023 03:10:17 +0100 Subject: [PATCH 233/331] Docker --- docker/test/base/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index b55baa0e0fc..b48017fdacc 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -12,6 +12,7 @@ RUN apt-get update \ ripgrep \ zstd \ locales \ + sudo \ --yes --no-install-recommends # Sanitizer options for services (clickhouse-server) From e006454eb2dd4b3e65ba81a830dbb88bed54943a Mon Sep 17 00:00:00 2001 From: Bharat Nallan Chakravarthy Date: Sun, 10 Dec 2023 16:43:45 -0800 Subject: [PATCH 234/331] add inline docs for hash functions --- src/Functions/FunctionsHashingSSL.cpp | 149 ++++++++++++++++++++++++-- 1 file changed, 141 insertions(+), 8 deletions(-) diff --git a/src/Functions/FunctionsHashingSSL.cpp b/src/Functions/FunctionsHashingSSL.cpp index b716a11f9c3..3e109b8a11d 100644 --- a/src/Functions/FunctionsHashingSSL.cpp +++ b/src/Functions/FunctionsHashingSSL.cpp @@ -14,14 +14,147 @@ namespace DB REGISTER_FUNCTION(HashingSSL) { - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(FunctionDocumentation{ + .description = R"(Calculates the MD4 hash of the given string.)", + .syntax = "SELECT MD4(s);", + .arguments = {{"s", "The input [String](../../sql-reference/data-types/string.md)."}}, + .returned_value + = "The MD4 hash of the given input string returned as a [FixedString(16)](../../sql-reference/data-types/fixedstring.md).", + .examples + = {{"", + "SELECT HEX(MD4('abc'));", + R"( +┌─hex(MD4('abc'))──────────────────┐ +│ A448017AAF21D8525FC10AE87AA6729D │ +└──────────────────────────────────┘ + )" + }} + }); + factory.registerFunction(FunctionDocumentation{ + .description = R"( +[Interprets](../..//sql-reference/functions/type-conversion-functions.md/#type_conversion_functions-reinterpretAsString) all the input +parameters as strings and calculates the MD5 hash value for each of them. Then combines hashes, takes the first 8 bytes of the hash of the +resulting string, and interprets them as [UInt64](../../../sql-reference/data-types/int-uint.md) in big-endian byte order. The function is +relatively slow (5 million short strings per second per processor core). + +Consider using the [sipHash64](../../sql-reference/functions/hash-functions.md/#hash_functions-siphash64) function instead. + )", + .syntax = "SELECT halfMD5(par1,par2,...,parN);", + .arguments = {{"par1,par2,...,parN", + R"( +The function takes a variable number of input parameters. Arguments can be any of the supported data types. For some data types calculated +value of hash function may be the same for the same values even if types of arguments differ (integers of different size, named and unnamed +Tuple with the same data, Map and the corresponding Array(Tuple(key, value)) type with the same data). + )" + }}, + .returned_value + = "The computed half MD5 hash of the given input params returned as a [UInt64](../../../sql-reference/data-types/int-uint.md) in big-endian byte order.", + .examples + = {{"", + "SELECT HEX(halfMD5('abc', 'cde', 'fgh'));", + R"( +┌─hex(halfMD5('abc', 'cde', 'fgh'))─┐ +│ 2C9506B7374CFAF4 │ +└───────────────────────────────────┘ + )" + }} + }); + factory.registerFunction(FunctionDocumentation{ + .description = R"(Calculates the MD5 hash of the given string.)", + .syntax = "SELECT MD5(s);", + .arguments = {{"s", "The input [String](../../sql-reference/data-types/string.md)."}}, + .returned_value + = "The MD5 hash of the given input string returned as a [FixedString(16)](../../sql-reference/data-types/fixedstring.md).", + .examples + = {{"", + "SELECT HEX(MD5('abc'));", + R"( +┌─hex(MD5('abc'))──────────────────┐ +│ 900150983CD24FB0D6963F7D28E17F72 │ +└──────────────────────────────────┘ + )" + }} + }); + factory.registerFunction(FunctionDocumentation{ + .description = R"(Calculates the SHA1 hash of the given string.)", + .syntax = "SELECT SHA1(s);", + .arguments = {{"s", "The input [String](../../sql-reference/data-types/string.md)."}}, + .returned_value + = "The SHA1 hash of the given input string returned as a [FixedString](../../sql-reference/data-types/fixedstring.md).", + .examples + = {{"", + "SELECT HEX(SHA1('abc'));", + R"( +┌─hex(SHA1('abc'))─────────────────────────┐ +│ A9993E364706816ABA3E25717850C26C9CD0D89D │ +└──────────────────────────────────────────┘ + )" + }} + }); + factory.registerFunction(FunctionDocumentation{ + .description = R"(Calculates the SHA224 hash of the given string.)", + .syntax = "SELECT SHA224(s);", + .arguments = {{"s", "The input [String](../../sql-reference/data-types/string.md)."}}, + .returned_value + = "The SHA224 hash of the given input string returned as a [FixedString](../../sql-reference/data-types/fixedstring.md).", + .examples + = {{"", + "SELECT HEX(SHA224('abc'));", + R"( +┌─hex(SHA224('abc'))───────────────────────────────────────┐ +│ 23097D223405D8228642A477BDA255B32AADBCE4BDA0B3F7E36C9DA7 │ +└──────────────────────────────────────────────────────────┘ + )" + }} + }); + factory.registerFunction(FunctionDocumentation{ + .description = R"(Calculates the SHA256 hash of the given string.)", + .syntax = "SELECT SHA256(s);", + .arguments = {{"s", "The input [String](../../sql-reference/data-types/string.md)."}}, + .returned_value + = "The SHA256 hash of the given input string returned as a [FixedString](../../sql-reference/data-types/fixedstring.md).", + .examples + = {{"", + "SELECT HEX(SHA256('abc'));", + R"( +┌─hex(SHA256('abc'))───────────────────────────────────────────────┐ +│ BA7816BF8F01CFEA414140DE5DAE2223B00361A396177A9CB410FF61F20015AD │ +└──────────────────────────────────────────────────────────────────┘ + )" + }} + }); + factory.registerFunction(FunctionDocumentation{ + .description = R"(Calculates the SHA384 hash of the given string.)", + .syntax = "SELECT SHA384(s);", + .arguments = {{"s", "The input [String](../../sql-reference/data-types/string.md)."}}, + .returned_value + = "The SHA384 hash of the given input string returned as a [FixedString](../../sql-reference/data-types/fixedstring.md).", + .examples + = {{"", + "SELECT HEX(SHA384('abc'));", + R"( +┌─hex(SHA384('abc'))───────────────────────────────────────────────────────────────────────────────┐ +│ CB00753F45A35E8BB5A03D699AC65007272C32AB0EDED1631A8B605A43FF5BED8086072BA1E7CC2358BAECA134C825A7 │ +└──────────────────────────────────────────────────────────────────────────────────────────────────┘ + )" + }} + }); + factory.registerFunction(FunctionDocumentation{ + .description = R"(Calculates the SHA512 hash of the given string.)", + .syntax = "SELECT SHA512(s);", + .arguments = {{"s", "The input [String](../../sql-reference/data-types/string.md)."}}, + .returned_value + = "The SHA512 hash of the given input string returned as a [FixedString](../../sql-reference/data-types/fixedstring.md).", + .examples + = {{"", + "SELECT HEX(SHA512('abc'));", + R"( +┌─hex(SHA512('abc'))───────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ DDAF35A193617ABACC417349AE20413112E6FA4E89A97EA20A9EEEE64B55D39A2192992A274FC1A836BA3C23A3FEEBBD454D4423643CE80E2A9AC94FA54CA49F │ +└──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ + )" + }} + }); factory.registerFunction(FunctionDocumentation{ .description = R"(Calculates the SHA512_256 hash of the given string.)", .syntax = "SELECT SHA512_256(s);", From 972e8c1a983c47c84eea3c6015e1f3ce7e43ec4f Mon Sep 17 00:00:00 2001 From: Bharat Nallan Chakravarthy Date: Sun, 10 Dec 2023 18:39:40 -0800 Subject: [PATCH 235/331] fix tests --- .../0_stateless/02415_all_new_functions_must_be_documented.sql | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql index 148ad303bd4..cabcd230eb6 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql @@ -2,7 +2,6 @@ -- Please help shorten this list down to zero elements. SELECT name FROM system.functions WHERE NOT is_aggregate AND origin = 'System' AND alias_to = '' AND length(description) < 10 AND name NOT IN ( - 'MD4', 'MD5', 'SHA1', 'SHA224', 'SHA256', 'SHA384', 'SHA512', 'halfMD5', 'aes_decrypt_mysql', 'aes_encrypt_mysql', 'decrypt', 'encrypt', 'base64Decode', 'base64Encode', 'tryBase64Decode', 'convertCharset', From 89ba4a845232849b7bd2dc807b63328d853f0451 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 7 Dec 2023 07:28:41 +0100 Subject: [PATCH 236/331] Add HTTP readiness endpoint --- programs/keeper/Keeper.cpp | 23 +++++ programs/server/Server.cpp | 29 ++++++ src/Server/HTTPHandlerFactory.h | 7 ++ src/Server/KeeperReadinessHandler.cpp | 95 +++++++++++++++++++ src/Server/KeeperReadinessHandler.h | 31 ++++++ tests/integration/helpers/keeper_utils.py | 5 + .../test_keeper_http_control/__init__.py | 0 .../configs/enable_keeper1.xml | 37 ++++++++ .../configs/enable_keeper2.xml | 37 ++++++++ .../configs/enable_keeper3.xml | 37 ++++++++ .../test_keeper_http_control/test.py | 62 ++++++++++++ 11 files changed, 363 insertions(+) create mode 100644 src/Server/KeeperReadinessHandler.cpp create mode 100644 src/Server/KeeperReadinessHandler.h create mode 100644 tests/integration/test_keeper_http_control/__init__.py create mode 100644 tests/integration/test_keeper_http_control/configs/enable_keeper1.xml create mode 100644 tests/integration/test_keeper_http_control/configs/enable_keeper2.xml create mode 100644 tests/integration/test_keeper_http_control/configs/enable_keeper3.xml create mode 100644 tests/integration/test_keeper_http_control/test.py diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 1acf7e39b04..4f45b09b682 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -466,6 +466,29 @@ try std::make_unique( std::move(my_http_context), createPrometheusMainHandlerFactory(*this, config_getter(), async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); }); + + /// HTTP control endpoints + port_name = "keeper_server.http_control.port"; + createServer(listen_host, port_name, listen_try, [&](UInt16 port) mutable + { + auto my_http_context = httpContext(); + Poco::Timespan my_keep_alive_timeout(config.getUInt("keep_alive_timeout", 10), 0); + Poco::Net::HTTPServerParams::Ptr my_http_params = new Poco::Net::HTTPServerParams; + my_http_params->setTimeout(my_http_context->getReceiveTimeout()); + my_http_params->setKeepAliveTimeout(my_keep_alive_timeout); + + Poco::Net::ServerSocket socket; + auto address = socketBindListen(socket, listen_host, port); + socket.setReceiveTimeout(my_http_context->getReceiveTimeout()); + socket.setSendTimeout(my_http_context->getSendTimeout()); + servers->emplace_back( + listen_host, + port_name, + "HTTP Control: http://" + address.toString(), + std::make_unique( + std::move(my_http_context), createKeeperHTTPControlMainHandlerFactory(*this, config_getter(), global_context->getKeeperDispatcher(), "KeeperHTTPControlHandler-factory"), server_pool, socket, http_params) + ); + }); } for (auto & server : *servers) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 8076d108083..ee3c7a72edc 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1551,6 +1551,35 @@ try throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); #endif }); + + /// HTTP control endpoints + port_name = "keeper_server.http_control.port"; + createServer(config(), listen_host, port_name, listen_try, /* start_server: */ false, + servers_to_start_before_tables, + [&](UInt16 port) -> ProtocolServerAdapter + { + auto http_context = httpContext(); + Poco::Timespan keep_alive_timeout(config().getUInt("keep_alive_timeout", 10), 0); + Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; + http_params->setTimeout(http_context->getReceiveTimeout()); + http_params->setKeepAliveTimeout(keep_alive_timeout); + + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config(), socket, listen_host, port); + socket.setReceiveTimeout(http_context->getReceiveTimeout()); + socket.setSendTimeout(http_context->getSendTimeout()); + return ProtocolServerAdapter( + listen_host, + port_name, + "HTTP Control: http://" + address.toString(), + std::make_unique( + std::move(http_context), + createKeeperHTTPControlMainHandlerFactory( + *this, + config_getter(), + global_context->getKeeperDispatcher(), + "KeeperHTTPControlHandler-factory"), server_pool, socket, http_params)); + }); } #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "ClickHouse server built without NuRaft library. Cannot use internal coordination."); diff --git a/src/Server/HTTPHandlerFactory.h b/src/Server/HTTPHandlerFactory.h index 94b02e52277..fd927d480fd 100644 --- a/src/Server/HTTPHandlerFactory.h +++ b/src/Server/HTTPHandlerFactory.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -136,6 +137,12 @@ createPrometheusMainHandlerFactory(IServer & server, AsynchronousMetrics & async_metrics, const std::string & name); +HTTPRequestHandlerFactoryPtr +createKeeperHTTPControlMainHandlerFactory(IServer & server, + const Poco::Util::AbstractConfiguration & config, + std::shared_ptr keeper_dispatcher, + const std::string & name); + /// @param server - used in handlers to check IServer::isCancelled() /// @param config - not the same as server.config(), since it can be newer /// @param async_metrics - used for prometheus (in case of prometheus.asynchronous_metrics=true) diff --git a/src/Server/KeeperReadinessHandler.cpp b/src/Server/KeeperReadinessHandler.cpp new file mode 100644 index 00000000000..f69f6cb377a --- /dev/null +++ b/src/Server/KeeperReadinessHandler.cpp @@ -0,0 +1,95 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +void KeeperReadinessHandler::handleRequest(HTTPServerRequest & /*request*/, HTTPServerResponse & response) +{ + try + { + auto is_leader = keeper_dispatcher->isLeader(); + auto is_follower = keeper_dispatcher->isFollower() && keeper_dispatcher->hasLeader(); + + auto status = is_leader || is_follower; + + Poco::JSON::Object json, details; + + details.set("leader", is_leader); + details.set("follower", is_follower); + json.set("details", details); + json.set("status", status ? "ok": "fail"); + + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(json, oss); + + if (!status) + response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_SERVICE_UNAVAILABLE); + + *response.send() << oss.str(); + } + catch (...) + { + tryLogCurrentException("KeeperReadinessHandler"); + + try + { + response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR); + + if (!response.sent()) + { + /// We have not sent anything yet and we don't even know if we need to compress response. + *response.send() << getCurrentExceptionMessage(false) << std::endl; + } + } + catch (...) + { + LOG_ERROR((&Poco::Logger::get("KeeperReadinessHandler")), "Cannot send exception to client"); + } + } +} + + +HTTPRequestHandlerFactoryPtr createKeeperHTTPControlMainHandlerFactory( + IServer & server, + const Poco::Util::AbstractConfiguration & config, + std::shared_ptr keeper_dispatcher, + const std::string & name) +{ + auto factory = std::make_shared(name); + using Factory = HandlingRuleHTTPHandlerFactory; + Factory::Creator creator = [&server, keeper_dispatcher]() -> std::unique_ptr + { + return std::make_unique(server, keeper_dispatcher); + }; + + auto readiness_handler = std::make_shared(std::move(creator)); + + readiness_handler->attachStrictPath(config.getString("keeper_server.http_control.readiness.endpoint", "/ready")); + readiness_handler->allowGetAndHeadRequest(); + factory->addHandler(readiness_handler); + + return factory; +} + +} diff --git a/src/Server/KeeperReadinessHandler.h b/src/Server/KeeperReadinessHandler.h new file mode 100644 index 00000000000..143751c5d67 --- /dev/null +++ b/src/Server/KeeperReadinessHandler.h @@ -0,0 +1,31 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class Context; +class IServer; + +class KeeperReadinessHandler : public HTTPRequestHandler, WithContext +{ +private: + IServer & server; + std::shared_ptr keeper_dispatcher; + +public: + explicit KeeperReadinessHandler(IServer & server_, std::shared_ptr keeper_dispatcher_) + : server(server_) + , keeper_dispatcher(keeper_dispatcher_) + { + } + + void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) override; +}; + + +} diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index 1ca17e923e4..6e4c1c1d417 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -278,6 +278,11 @@ def get_leader(cluster, nodes): return node raise Exception("No leader in Keeper cluster.") +def get_follower(cluster, nodes): + for node in nodes: + if is_follower(cluster, node): + return node + raise Exception("No followers in Keeper cluster.") def get_fake_zk(cluster, node, timeout: float = 30.0) -> KazooClient: _fake = KazooClient( diff --git a/tests/integration/test_keeper_http_control/__init__.py b/tests/integration/test_keeper_http_control/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_http_control/configs/enable_keeper1.xml b/tests/integration/test_keeper_http_control/configs/enable_keeper1.xml new file mode 100644 index 00000000000..20e3c307f31 --- /dev/null +++ b/tests/integration/test_keeper_http_control/configs/enable_keeper1.xml @@ -0,0 +1,37 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + 9182 + + + diff --git a/tests/integration/test_keeper_http_control/configs/enable_keeper2.xml b/tests/integration/test_keeper_http_control/configs/enable_keeper2.xml new file mode 100644 index 00000000000..b9002eb2436 --- /dev/null +++ b/tests/integration/test_keeper_http_control/configs/enable_keeper2.xml @@ -0,0 +1,37 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + 9182 + + + diff --git a/tests/integration/test_keeper_http_control/configs/enable_keeper3.xml b/tests/integration/test_keeper_http_control/configs/enable_keeper3.xml new file mode 100644 index 00000000000..6e4e17399f7 --- /dev/null +++ b/tests/integration/test_keeper_http_control/configs/enable_keeper3.xml @@ -0,0 +1,37 @@ + + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + 9182 + + + diff --git a/tests/integration/test_keeper_http_control/test.py b/tests/integration/test_keeper_http_control/test.py new file mode 100644 index 00000000000..04d84671de2 --- /dev/null +++ b/tests/integration/test_keeper_http_control/test.py @@ -0,0 +1,62 @@ +#!/usr/bin/env python3 + +import os +import pytest +import requests + +import helpers.keeper_utils as keeper_utils +from kazoo.client import KazooClient +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), "configs") + +node1 = cluster.add_instance( + "node1", main_configs=["configs/enable_keeper1.xml"], stay_alive=True +) +node2 = cluster.add_instance( + "node2", main_configs=["configs/enable_keeper2.xml"], stay_alive=True +) +node3 = cluster.add_instance( + "node3", main_configs=["configs/enable_keeper3.xml"], stay_alive=True +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def get_fake_zk(node, timeout=30.0): + _fake_zk_instance = KazooClient( + hosts=cluster.get_instance_ip(node.name) + ":9181", timeout=timeout + ) + _fake_zk_instance.start() + return _fake_zk_instance + +def test_http_readiness(started_cluster): + leader = keeper_utils.get_leader(cluster, [node1, node2, node3]) + response = requests.get( + "http://{host}:{port}/ready".format(host=leader.ip_address, port=9182) + ) + assert(response.status_code == 200) + + readiness_data = response.json() + assert(readiness_data["status"] == "ok") + assert(readiness_data["details"]["leader"] == True) + assert(readiness_data["details"]["follower"] == False) + + follower = keeper_utils.get_follower(cluster, [node1, node2, node3]) + response = requests.get( + "http://{host}:{port}/ready".format(host=follower.ip_address, port=9182) + ) + assert(response.status_code == 200) + + readiness_data = response.json() + assert(readiness_data["status"] == "ok") + assert(readiness_data["details"]["leader"] == False) + assert(readiness_data["details"]["follower"] == True) From 3adb83e406027edc92a9792aa0f5587e33451765 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 7 Dec 2023 06:49:10 +0000 Subject: [PATCH 237/331] Automatic style fix --- tests/integration/helpers/keeper_utils.py | 2 ++ .../test_keeper_http_control/test.py | 21 ++++++++++--------- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index 6e4c1c1d417..e07bce901d2 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -278,12 +278,14 @@ def get_leader(cluster, nodes): return node raise Exception("No leader in Keeper cluster.") + def get_follower(cluster, nodes): for node in nodes: if is_follower(cluster, node): return node raise Exception("No followers in Keeper cluster.") + def get_fake_zk(cluster, node, timeout: float = 30.0) -> KazooClient: _fake = KazooClient( hosts=cluster.get_instance_ip(node.name) + ":9181", timeout=timeout diff --git a/tests/integration/test_keeper_http_control/test.py b/tests/integration/test_keeper_http_control/test.py index 04d84671de2..b415a03a5c4 100644 --- a/tests/integration/test_keeper_http_control/test.py +++ b/tests/integration/test_keeper_http_control/test.py @@ -38,25 +38,26 @@ def get_fake_zk(node, timeout=30.0): _fake_zk_instance.start() return _fake_zk_instance + def test_http_readiness(started_cluster): leader = keeper_utils.get_leader(cluster, [node1, node2, node3]) response = requests.get( - "http://{host}:{port}/ready".format(host=leader.ip_address, port=9182) + "http://{host}:{port}/ready".format(host=leader.ip_address, port=9182) ) - assert(response.status_code == 200) + assert response.status_code == 200 readiness_data = response.json() - assert(readiness_data["status"] == "ok") - assert(readiness_data["details"]["leader"] == True) - assert(readiness_data["details"]["follower"] == False) + assert readiness_data["status"] == "ok" + assert readiness_data["details"]["leader"] == True + assert readiness_data["details"]["follower"] == False follower = keeper_utils.get_follower(cluster, [node1, node2, node3]) response = requests.get( - "http://{host}:{port}/ready".format(host=follower.ip_address, port=9182) + "http://{host}:{port}/ready".format(host=follower.ip_address, port=9182) ) - assert(response.status_code == 200) + assert response.status_code == 200 readiness_data = response.json() - assert(readiness_data["status"] == "ok") - assert(readiness_data["details"]["leader"] == False) - assert(readiness_data["details"]["follower"] == True) + assert readiness_data["status"] == "ok" + assert readiness_data["details"]["leader"] == False + assert readiness_data["details"]["follower"] == True From 771d5c58ea49271b96e70fbfe5c992362eae6c13 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Fri, 8 Dec 2023 05:28:01 +0100 Subject: [PATCH 238/331] Fix Keeper standalone build --- programs/keeper/CMakeLists.txt | 1 + programs/keeper/Keeper.cpp | 3 ++- programs/server/Server.cpp | 2 +- src/Server/HTTPHandlerFactory.h | 6 ------ src/Server/KeeperReadinessHandler.cpp | 17 ++++------------- src/Server/KeeperReadinessHandler.h | 15 +++++++++------ 6 files changed, 17 insertions(+), 27 deletions(-) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index cdb1d89b18e..f3d82b6029b 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -68,6 +68,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/waitServersToFinish.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/ServerType.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTPRequestHandlerFactoryMain.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/KeeperReadinessHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/HTTPServer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/ReadHeaders.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/HTTPServerConnection.cpp diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 4f45b09b682..7585f147161 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include "Core/Defines.h" #include "config.h" @@ -486,7 +487,7 @@ try port_name, "HTTP Control: http://" + address.toString(), std::make_unique( - std::move(my_http_context), createKeeperHTTPControlMainHandlerFactory(*this, config_getter(), global_context->getKeeperDispatcher(), "KeeperHTTPControlHandler-factory"), server_pool, socket, http_params) + std::move(my_http_context), createKeeperHTTPControlMainHandlerFactory(config_getter(), global_context->getKeeperDispatcher(), "KeeperHTTPControlHandler-factory"), server_pool, socket, http_params) ); }); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index ee3c7a72edc..5abaf67e3d8 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -92,6 +92,7 @@ #include #include #include +#include #include #include #include @@ -1575,7 +1576,6 @@ try std::make_unique( std::move(http_context), createKeeperHTTPControlMainHandlerFactory( - *this, config_getter(), global_context->getKeeperDispatcher(), "KeeperHTTPControlHandler-factory"), server_pool, socket, http_params)); diff --git a/src/Server/HTTPHandlerFactory.h b/src/Server/HTTPHandlerFactory.h index fd927d480fd..459d54e27a5 100644 --- a/src/Server/HTTPHandlerFactory.h +++ b/src/Server/HTTPHandlerFactory.h @@ -137,12 +137,6 @@ createPrometheusMainHandlerFactory(IServer & server, AsynchronousMetrics & async_metrics, const std::string & name); -HTTPRequestHandlerFactoryPtr -createKeeperHTTPControlMainHandlerFactory(IServer & server, - const Poco::Util::AbstractConfiguration & config, - std::shared_ptr keeper_dispatcher, - const std::string & name); - /// @param server - used in handlers to check IServer::isCancelled() /// @param config - not the same as server.config(), since it can be newer /// @param async_metrics - used for prometheus (in case of prometheus.asynchronous_metrics=true) diff --git a/src/Server/KeeperReadinessHandler.cpp b/src/Server/KeeperReadinessHandler.cpp index f69f6cb377a..9ba4fe44ab4 100644 --- a/src/Server/KeeperReadinessHandler.cpp +++ b/src/Server/KeeperReadinessHandler.cpp @@ -1,18 +1,11 @@ #include -#include -#include #include -#include -#include +#include +#include #include #include -#include -#include -#include -#include #include - #include #include #include @@ -69,18 +62,16 @@ void KeeperReadinessHandler::handleRequest(HTTPServerRequest & /*request*/, HTTP } } - HTTPRequestHandlerFactoryPtr createKeeperHTTPControlMainHandlerFactory( - IServer & server, const Poco::Util::AbstractConfiguration & config, std::shared_ptr keeper_dispatcher, const std::string & name) { auto factory = std::make_shared(name); using Factory = HandlingRuleHTTPHandlerFactory; - Factory::Creator creator = [&server, keeper_dispatcher]() -> std::unique_ptr + Factory::Creator creator = [keeper_dispatcher]() -> std::unique_ptr { - return std::make_unique(server, keeper_dispatcher); + return std::make_unique(keeper_dispatcher); }; auto readiness_handler = std::make_shared(std::move(creator)); diff --git a/src/Server/KeeperReadinessHandler.h b/src/Server/KeeperReadinessHandler.h index 143751c5d67..03fd58e831f 100644 --- a/src/Server/KeeperReadinessHandler.h +++ b/src/Server/KeeperReadinessHandler.h @@ -1,8 +1,8 @@ #pragma once -#include +#include #include -#include +#include #include namespace DB @@ -14,18 +14,21 @@ class IServer; class KeeperReadinessHandler : public HTTPRequestHandler, WithContext { private: - IServer & server; std::shared_ptr keeper_dispatcher; public: - explicit KeeperReadinessHandler(IServer & server_, std::shared_ptr keeper_dispatcher_) - : server(server_) - , keeper_dispatcher(keeper_dispatcher_) + explicit KeeperReadinessHandler(std::shared_ptr keeper_dispatcher_) + : keeper_dispatcher(keeper_dispatcher_) { } void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) override; }; +HTTPRequestHandlerFactoryPtr +createKeeperHTTPControlMainHandlerFactory( + const Poco::Util::AbstractConfiguration & config, + std::shared_ptr keeper_dispatcher, + const std::string & name); } From 76966818bade118a0e003469314fbb196f806c66 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Fri, 8 Dec 2023 13:57:22 +0100 Subject: [PATCH 239/331] Fix FreeBSD build --- src/Server/KeeperReadinessHandler.cpp | 7 ++++++- src/Server/KeeperReadinessHandler.h | 10 ++++++---- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/src/Server/KeeperReadinessHandler.cpp b/src/Server/KeeperReadinessHandler.cpp index 9ba4fe44ab4..37afd8e9898 100644 --- a/src/Server/KeeperReadinessHandler.cpp +++ b/src/Server/KeeperReadinessHandler.cpp @@ -1,8 +1,11 @@ +#include + +#if USE_NURAFT + #include #include #include -#include #include #include #include @@ -84,3 +87,5 @@ HTTPRequestHandlerFactoryPtr createKeeperHTTPControlMainHandlerFactory( } } + +#endif diff --git a/src/Server/KeeperReadinessHandler.h b/src/Server/KeeperReadinessHandler.h index 03fd58e831f..caa59098427 100644 --- a/src/Server/KeeperReadinessHandler.h +++ b/src/Server/KeeperReadinessHandler.h @@ -1,6 +1,9 @@ #pragma once -#include +#include "config.h" + +#if USE_NURAFT + #include #include #include @@ -8,9 +11,6 @@ namespace DB { -class Context; -class IServer; - class KeeperReadinessHandler : public HTTPRequestHandler, WithContext { private: @@ -32,3 +32,5 @@ createKeeperHTTPControlMainHandlerFactory( const std::string & name); } + +#endif From 7172a8ec9a87b43097db50f64914f2991329856c Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Fri, 8 Dec 2023 13:22:46 +0100 Subject: [PATCH 240/331] Remove redundant include --- src/Server/HTTPHandlerFactory.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Server/HTTPHandlerFactory.h b/src/Server/HTTPHandlerFactory.h index 459d54e27a5..94b02e52277 100644 --- a/src/Server/HTTPHandlerFactory.h +++ b/src/Server/HTTPHandlerFactory.h @@ -5,7 +5,6 @@ #include #include #include -#include #include #include From 81250d1a901040e18c8ad96486a27dd9f78bc9dc Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 10 Dec 2023 16:05:29 +0300 Subject: [PATCH 241/331] SerializationString improve performance --- src/DataTypes/Serializations/SerializationString.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index 89fa09781ef..c59c642cf04 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -152,6 +152,8 @@ template static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnString::Offsets & offsets, ReadBuffer & istr, size_t limit) { size_t offset = data.size(); + data.resize(std::max(data.capacity(), static_cast(4096))); + for (size_t i = 0; i < limit; ++i) { if (istr.eof()) @@ -171,7 +173,8 @@ static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnSt offset += size + 1; offsets.push_back(offset); - data.resize(offset); + if (unlikely(offset > data.size())) + data.resize(data.size() * 2); if (size) { @@ -203,6 +206,8 @@ static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnSt data[offset - 1] = 0; } + + data.resize(offset); } From 3652419873e3c940170c95970c5a973e6201b05d Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Dec 2023 11:01:13 +0100 Subject: [PATCH 242/331] Update run.sh --- docker/test/stress/run.sh | 16 ++++++++++------ src/Interpreters/Cache/SLRUFileCachePriority.cpp | 1 - 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 5e12ade24d5..67056cc1bc1 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -69,12 +69,16 @@ clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test" stop mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.initial.log -# Randomly choose between LRU and SLRU policies. -cache_policy="SLRU" -#TODO: uncomment this before merge, for testing purposes it is SLRU only before merge. -#if [$(($RANDOM%2)) -eq 1]; then -# cache_policy="LRU" -#fi +# Randomize cache policies. +cache_policy="" +if [ $(( $(date +%-d) % 2 )) -eq 1 ]; then + cache_policy="SLRU" +else + cache_policy="LRU" +fi + +echo "Using cache policy: $cache_policy" + if [ "$cache_policy" = "SLRU" ]; then sudo cat /etc/clickhouse-server/config.d/storage_conf.xml \ | sed "s|LRU|SLRU|" \ diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index 7b3e666f595..71b8d44d438 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -150,7 +150,6 @@ void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const Cach /// Entry is in probationary queue. /// We need to move it to protected queue. - const size_t size = iterator.getEntry().size; if (size > protected_queue.getSizeLimit()) { From 4c14758f237f4bf0154f219652a083d9b04585e1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 11 Dec 2023 10:19:09 +0000 Subject: [PATCH 243/331] Some fixups --- docs/en/development/build-osx.md | 2 +- docs/en/development/developer-instruction.md | 50 ++++++++++---------- 2 files changed, 25 insertions(+), 27 deletions(-) diff --git a/docs/en/development/build-osx.md b/docs/en/development/build-osx.md index e65de4a37e0..39ccc9a78c3 100644 --- a/docs/en/development/build-osx.md +++ b/docs/en/development/build-osx.md @@ -3,7 +3,7 @@ slug: /en/development/build-osx sidebar_position: 65 sidebar_label: Build on macOS title: How to Build ClickHouse on macOS -description: How to build ClickHouse on macOS +description: How to build ClickHouse on macOS for macOS --- :::info You don't have to build ClickHouse yourself! diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index a5c66d4dde3..645756a46c7 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -7,42 +7,39 @@ description: Prerequisites and an overview of how to build ClickHouse # Getting Started Guide for Building ClickHouse -The building of ClickHouse is supported on Linux, FreeBSD and macOS. +ClickHouse can be build on Linux, FreeBSD and macOS. If you use Windows, you can still build ClickHouse in a virtual machine running Linux, e.g. [VirtualBox](https://www.virtualbox.org/) with Ubuntu. -If you use Windows, you need to create a virtual machine with Ubuntu. To start working with a virtual machine please install VirtualBox. You can download Ubuntu from the website: https://www.ubuntu.com/#download. Please create a virtual machine from the downloaded image (you should reserve at least 4GB of RAM for it). To run a command-line terminal in Ubuntu, please locate a program containing the word “terminal” in its name (gnome-terminal, konsole etc.) or just press Ctrl+Alt+T. - -ClickHouse cannot work or build on a 32-bit system. You should acquire access to a 64-bit system and you can continue reading. +ClickHouse requires a 64-bit system to compile and run, 32-bit systems do not work. ## Creating a Repository on GitHub {#creating-a-repository-on-github} -To start working with ClickHouse repository you will need a GitHub account. +To start developing for ClickHouse you will need a [GitHub](https://www.virtualbox.org/) account. Please also generate a SSH key locally (if you don't have one already) and upload the public key to GitHub as this is a prerequisite for contributing patches. -You probably already have one, but if you do not, please register at https://github.com. In case you do not have SSH keys, you should generate them and then upload them on GitHub. It is required for sending over your patches. It is also possible to use the same SSH keys that you use with any other SSH servers - probably you already have those. +Next, create a fork of the [ClickHouse repository](https://github.com/ClickHouse/ClickHouse/) in your personal account by clicking the "fork" button in the upper right corner. -Create a fork of ClickHouse repository. To do that please click on the “fork” button in the upper right corner at https://github.com/ClickHouse/ClickHouse. It will fork your own copy of ClickHouse/ClickHouse to your account. +To contribute, e.g. a fix for an issue or a feature, please commit your changes to a branch in your fork, then create a "pull request" with the changes to the main repository. -The development process consists of first committing the intended changes into your fork of ClickHouse and then creating a “pull request” for these changes to be accepted into the main repository (ClickHouse/ClickHouse). +For working with Git repositories, please install `git`. In Ubuntu run these commands in a terminal: -To work with Git repositories, please install `git`. To do that in Ubuntu you would run in the command line terminal: +```sh +sudo apt update +sudo apt install git +``` - sudo apt update - sudo apt install git - -A brief manual on using Git can be found [here](https://education.github.com/git-cheat-sheet-education.pdf). -For a detailed manual on Git see [here](https://git-scm.com/book/en/v2). +A cheatsheet for using Git can be found [here](https://education.github.com/git-cheat-sheet-education.pdf). The detailed manual for Git is [here](https://git-scm.com/book/en/v2). ## Cloning a Repository to Your Development Machine {#cloning-a-repository-to-your-development-machine} -Next, you need to download the source files onto your working machine. This is called “to clone a repository” because it creates a local copy of the repository on your working machine. +First, download the source files to your working machine, i.e. clone the repository: -Run in your terminal: +```sh +git clone git@github.com:your_github_username/ClickHouse.git # replace placeholder with your GitHub user name +cd ClickHouse +``` - git clone git@github.com:your_github_username/ClickHouse.git # replace placeholder with your GitHub user name - cd ClickHouse +This command creates a directory `ClickHouse/` containing the source code of ClickHouse. If you specify a custom checkout directory after the URL but it is important that this path does not contain whitespaces as it may lead to problems with the build later on. -This command will create a directory `ClickHouse/` containing the source code of ClickHouse. If you specify a custom checkout directory (after the URL), it is important that this path does not contain whitespaces as it may lead to problems with the build system. - -To make library dependencies available for the build, the ClickHouse repository uses Git submodules, i.e. references to external repositories. These are not checked out by default. To do so, you can either +The ClickHouse repository uses Git submodules, i.e. references to external repositories (usually 3rd party libraries used by ClickHouse). These are not checked out by default. To do so, you can either - run `git clone` with option `--recurse-submodules`, @@ -52,7 +49,7 @@ To make library dependencies available for the build, the ClickHouse repository You can check the Git status with the command: `git submodule status`. -If you get the following error message: +If you get the following error message Permission denied (publickey). fatal: Could not read from remote repository. @@ -60,7 +57,7 @@ If you get the following error message: Please make sure you have the correct access rights and the repository exists. -It generally means that the SSH keys for connecting to GitHub are missing. These keys are normally located in `~/.ssh`. For SSH keys to be accepted you need to upload them in the settings section of GitHub UI. +it generally means that the SSH keys for connecting to GitHub are missing. These keys are normally located in `~/.ssh`. For SSH keys to be accepted you need to upload them in GitHub's settings. You can also clone the repository via https protocol: @@ -75,15 +72,16 @@ You can also add original ClickHouse repo address to your local repository to pu After successfully running this command you will be able to pull updates from the main ClickHouse repo by running `git pull upstream master`. :::note -Instructions below work on most Linux compatible systems, but before proceeding please also check for operating system and architecture specific guides, such as building on (MacOS)[https://clickhouse.com/docs/en/development/build-osx], (Linux for MacOS)[https://clickhouse.com/docs/en/development/build-cross-osx], (Linux for RISC-V 64)[https://clickhouse.com/docs/en/development/build-cross-riscv] and so on. +Instructions below assume you are building on Linux. If you are cross-compiling or using building on macOS, please also check for operating system and architecture specific guides, such as building [on macOS for macOS](build-osx.md), [on Linux for macOS](build-cross-osx.md), [on Linux for Linux/RISC-V](build-cross-riscv.md) and so on. ::: ## Build System {#build-system} ClickHouse uses CMake and Ninja for building. -CMake - a meta-build system that can generate Ninja files (build tasks). -Ninja - a smaller build system with a focus on the speed used to execute those cmake generated tasks. +- CMake - a meta-build system that can generate Ninja files (build tasks). + +- Ninja - a smaller build system with a focus on the speed used to execute those cmake generated tasks. To install on Ubuntu, Debian or Mint run `sudo apt install cmake ninja-build`. From c78649bdcffd9ac1b1cf547f1b8d29e4b4deea38 Mon Sep 17 00:00:00 2001 From: flynn Date: Mon, 11 Dec 2023 10:36:26 +0000 Subject: [PATCH 244/331] Support negative positional arguments --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 10 +- .../replaceForPositionalArguments.cpp | 10 +- .../02006_test_positional_arguments.reference | 94 +++++++++++++++++++ .../02006_test_positional_arguments.sql | 21 +++++ 4 files changed, 127 insertions(+), 8 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 1e63d5ca8e4..a52a0fac232 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2153,18 +2153,20 @@ void QueryAnalyzer::replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_ node_to_replace = &sort_node->getExpression(); auto * constant_node = (*node_to_replace)->as(); - if (!constant_node || constant_node->getValue().getType() != Field::Types::UInt64) + if (!constant_node + || (constant_node->getValue().getType() != Field::Types::UInt64 && constant_node->getValue().getType() != Field::Types::Int64)) continue; - UInt64 positional_argument_number = constant_node->getValue().get(); - if (positional_argument_number == 0 || positional_argument_number > projection_nodes.size()) + auto positional_argument_number = constant_node->getValue().get(); + if (positional_argument_number == 0 || static_cast(std::abs(positional_argument_number)) > projection_nodes.size()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional argument number {} is out of bounds. Expected in range [1, {}]. In scope {}", positional_argument_number, projection_nodes.size(), scope.scope_node->formatASTForErrorMessage()); - --positional_argument_number; + positional_argument_number + = (positional_argument_number > 0) ? --positional_argument_number : projection_nodes.size() + positional_argument_number; *node_to_replace = projection_nodes[positional_argument_number]; } } diff --git a/src/Interpreters/replaceForPositionalArguments.cpp b/src/Interpreters/replaceForPositionalArguments.cpp index 241dd7cf92c..f5a77dacd4c 100644 --- a/src/Interpreters/replaceForPositionalArguments.cpp +++ b/src/Interpreters/replaceForPositionalArguments.cpp @@ -27,16 +27,18 @@ bool replaceForPositionalArguments(ASTPtr & argument, const ASTSelectQuery * sel return false; auto which = ast_literal->value.getType(); - if (which != Field::Types::UInt64) + if (which != Field::Types::UInt64 && which != Field::Types::Int64) return false; - auto pos = ast_literal->value.get(); - if (!pos || pos > columns.size()) + auto pos = ast_literal->value.get(); + if (!pos || static_cast(std::abs(pos)) > columns.size()) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Positional argument out of bounds: {} (expected in range [1, {}]", pos, columns.size()); - const auto & column = columns[--pos]; + pos = (pos > 0) ? --pos : columns.size() + pos; + + const auto & column = columns[pos]; if (typeid_cast(column.get()) || typeid_cast(column.get())) { argument = column->clone(); diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.reference b/tests/queries/0_stateless/02006_test_positional_arguments.reference index 40100e8d5be..079bd071103 100644 --- a/tests/queries/0_stateless/02006_test_positional_arguments.reference +++ b/tests/queries/0_stateless/02006_test_positional_arguments.reference @@ -3,18 +3,50 @@ select x3, x2, x1 from test order by 1; 1 100 100 10 1 10 100 10 1 +select x3, x2, x1 from test order by -3; +1 100 100 +10 1 10 +100 10 1 select x3, x2, x1 from test order by x3; 1 100 100 10 1 10 100 10 1 +select x3, x2, x1 from test order by 3; +100 10 1 +10 1 10 +1 100 100 +select x3, x2, x1 from test order by -1; +100 10 1 +10 1 10 +1 100 100 +select x3, x2, x1 from test order by x1; +100 10 1 +10 1 10 +1 100 100 select x3, x2, x1 from test order by 1 desc; 100 10 1 10 1 10 1 100 100 +select x3, x2, x1 from test order by -3 desc; +100 10 1 +10 1 10 +1 100 100 select x3, x2, x1 from test order by x3 desc; 100 10 1 10 1 10 1 100 100 +select x3, x2, x1 from test order by 3 desc; +1 100 100 +10 1 10 +100 10 1 +select x3, x2, x1 from test order by -1 desc; +1 100 100 +10 1 10 +100 10 1 +select x3, x2, x1 from test order by x1 desc; +1 100 100 +10 1 10 +100 10 1 insert into test values (1, 10, 100), (10, 1, 10), (100, 100, 1); select x3, x2 from test group by x3, x2 order by x3; 1 100 @@ -54,6 +86,20 @@ SELECT x1 FROM test ORDER BY x3 + 1 ASC +explain syntax select x3, x2, x1 from test order by -1; +SELECT + x3, + x2, + x1 +FROM test +ORDER BY x1 ASC +explain syntax select x3 + 1, x2, x1 from test order by -1; +SELECT + x3 + 1, + x2, + x1 +FROM test +ORDER BY x1 ASC explain syntax select x3, x3 - x2, x2, x1 from test order by 2; SELECT x3, @@ -62,6 +108,14 @@ SELECT x1 FROM test ORDER BY x3 - x2 ASC +explain syntax select x3, x3 - x2, x2, x1 from test order by -2; +SELECT + x3, + x3 - x2, + x2, + x1 +FROM test +ORDER BY x2 ASC explain syntax select x3, if(x3 > 10, x3, plus(x1, x2)), x1 + x2 from test order by 2; SELECT x3, @@ -69,12 +123,28 @@ SELECT x1 + x2 FROM test ORDER BY if(x3 > 10, x3, x1 + x2) ASC +explain syntax select x3, if(x3 > 10, x3, plus(x1, x2)), x1 + x2 from test order by -2; +SELECT + x3, + if(x3 > 10, x3, x1 + x2), + x1 + x2 +FROM test +ORDER BY if(x3 > 10, x3, x1 + x2) ASC explain syntax select max(x1), x2 from test group by 2 order by 1, 2; SELECT max(x1), x2 FROM test GROUP BY x2 +ORDER BY + max(x1) ASC, + x2 ASC +explain syntax select max(x1), x2 from test group by -1 order by -2, -1; +SELECT + max(x1), + x2 +FROM test +GROUP BY x2 ORDER BY max(x1) ASC, x2 ASC @@ -83,16 +153,34 @@ SELECT 1 + greatest(x1, 1), x2 FROM test +GROUP BY + 1 + greatest(x1, 1), + x2 +explain syntax select 1 + greatest(x1, 1), x2 from test group by -2, -1; +SELECT + 1 + greatest(x1, 1), + x2 +FROM test GROUP BY 1 + greatest(x1, 1), x2 select max(x1), x2 from test group by 1, 2; -- { serverError 43, 184 } select 1 + max(x1), x2 from test group by 1, 2; -- { serverError 43, 184 } +select max(x1), x2 from test group by -2, -1; -- { serverError 43, 184 } +select 1 + max(x1), x2 from test group by -2, -1; -- { serverError 43, 184 } explain syntax select x1 + x3, x3 from test group by 1, 2; SELECT x1 + x3, x3 FROM test +GROUP BY + x1 + x3, + x3 +explain syntax select x1 + x3, x3 from test group by -2, -1; +SELECT + x1 + x3, + x3 +FROM test GROUP BY x1 + x3, x3 @@ -102,8 +190,14 @@ select x1, x1 * 2, max(x2), max(x3) from test2 group by 2, 1, x1 order by 1, 2, 1 2 10 100 10 20 1 10 100 200 100 1 +select x1, x1 * 2, max(x2), max(x3) from test2 group by 2, 1, x1 order by 1, 2, -1 desc, -2 asc; +1 2 10 100 +10 20 1 10 +100 200 100 1 select a, b, c, d, e, f from (select 44 a, 88 b, 13 c, 14 d, 15 e, 16 f) t group by 1,2,3,4,5,6 order by a; 44 88 13 14 15 16 +select a, b, c, d, e, f from (select 44 a, 88 b, 13 c, 14 d, 15 e, 16 f) t group by 1,2,3,-3,-2,-1 order by a; +44 88 13 14 15 16 explain syntax select plus(1, 1) as a group by a; SELECT 1 + 1 AS a GROUP BY a diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.sql b/tests/queries/0_stateless/02006_test_positional_arguments.sql index 159ad6bd427..6f427e0298d 100644 --- a/tests/queries/0_stateless/02006_test_positional_arguments.sql +++ b/tests/queries/0_stateless/02006_test_positional_arguments.sql @@ -9,11 +9,21 @@ insert into test values (1, 10, 100), (10, 1, 10), (100, 100, 1); -- { echo } select x3, x2, x1 from test order by 1; +select x3, x2, x1 from test order by -3; select x3, x2, x1 from test order by x3; +select x3, x2, x1 from test order by 3; +select x3, x2, x1 from test order by -1; +select x3, x2, x1 from test order by x1; + select x3, x2, x1 from test order by 1 desc; +select x3, x2, x1 from test order by -3 desc; select x3, x2, x1 from test order by x3 desc; +select x3, x2, x1 from test order by 3 desc; +select x3, x2, x1 from test order by -1 desc; +select x3, x2, x1 from test order by x1 desc; + insert into test values (1, 10, 100), (10, 1, 10), (100, 100, 1); select x3, x2 from test group by x3, x2 order by x3; select x3, x2 from test group by 1, 2 order by x3; @@ -25,21 +35,32 @@ select x1, x2, x3 from test order by 3 limit 1 by 1; explain syntax select x3, x2, x1 from test order by 1; explain syntax select x3 + 1, x2, x1 from test order by 1; +explain syntax select x3, x2, x1 from test order by -1; +explain syntax select x3 + 1, x2, x1 from test order by -1; explain syntax select x3, x3 - x2, x2, x1 from test order by 2; +explain syntax select x3, x3 - x2, x2, x1 from test order by -2; explain syntax select x3, if(x3 > 10, x3, plus(x1, x2)), x1 + x2 from test order by 2; +explain syntax select x3, if(x3 > 10, x3, plus(x1, x2)), x1 + x2 from test order by -2; explain syntax select max(x1), x2 from test group by 2 order by 1, 2; +explain syntax select max(x1), x2 from test group by -1 order by -2, -1; explain syntax select 1 + greatest(x1, 1), x2 from test group by 1, 2; +explain syntax select 1 + greatest(x1, 1), x2 from test group by -2, -1; select max(x1), x2 from test group by 1, 2; -- { serverError 43, 184 } select 1 + max(x1), x2 from test group by 1, 2; -- { serverError 43, 184 } +select max(x1), x2 from test group by -2, -1; -- { serverError 43, 184 } +select 1 + max(x1), x2 from test group by -2, -1; -- { serverError 43, 184 } explain syntax select x1 + x3, x3 from test group by 1, 2; +explain syntax select x1 + x3, x3 from test group by -2, -1; create table test2(x1 Int, x2 Int, x3 Int) engine=Memory; insert into test2 values (1, 10, 100), (10, 1, 10), (100, 100, 1); select x1, x1 * 2, max(x2), max(x3) from test2 group by 2, 1, x1 order by 1, 2, 4 desc, 3 asc; +select x1, x1 * 2, max(x2), max(x3) from test2 group by 2, 1, x1 order by 1, 2, -1 desc, -2 asc; select a, b, c, d, e, f from (select 44 a, 88 b, 13 c, 14 d, 15 e, 16 f) t group by 1,2,3,4,5,6 order by a; +select a, b, c, d, e, f from (select 44 a, 88 b, 13 c, 14 d, 15 e, 16 f) t group by 1,2,3,-3,-2,-1 order by a; explain syntax select plus(1, 1) as a group by a; select substr('aaaaaaaaaaaaaa', 8) as a group by a order by a; From d24f389a0a1e694eb8712ee0a2c18b57aaeb8587 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 11 Dec 2023 11:28:38 +0000 Subject: [PATCH 245/331] Fix spelling --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index a07c2648f92..bed1b278b6f 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -30,6 +30,7 @@ AppleClang Approximative ArrayJoin ArrowStream +AsyncInsertCacheSize AsynchronousHeavyMetricsCalculationTimeSpent AsynchronousHeavyMetricsUpdateInterval AsynchronousInsert @@ -38,11 +39,6 @@ AsynchronousInsertThreadsActive AsynchronousMetricsCalculationTimeSpent AsynchronousMetricsUpdateInterval AsynchronousReadWait -AsyncInsertCacheSize -TablesLoaderBackgroundThreads -TablesLoaderBackgroundThreadsActive -TablesLoaderForegroundThreads -TablesLoaderForegroundThreadsActive Authenticator Authenticators AutoFDO @@ -887,6 +883,10 @@ TabSeparatedRawWithNamesAndTypes TabSeparatedWithNames TabSeparatedWithNamesAndTypes Tabix +TablesLoaderBackgroundThreads +TablesLoaderBackgroundThreadsActive +TablesLoaderForegroundThreads +TablesLoaderForegroundThreadsActive TablesToDropQueueSize TargetSpecific Telegraf @@ -1233,6 +1233,7 @@ changelogs charset charsets chconn +cheatsheet checkouting checksummed checksumming From cdfe99c380861caa2b09ccd937c2964153758518 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 11 Dec 2023 11:34:54 +0000 Subject: [PATCH 246/331] Remove unused template parameter from fillColumnsFromMap --- src/Interpreters/HashJoin.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 71e9d4bba80..c79db392eb4 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1954,9 +1954,9 @@ public: } else { - auto fill_callback = [&](auto, auto strictness, auto & map) + auto fill_callback = [&](auto, auto, auto & map) { - rows_added = fillColumnsFromMap(map, columns_right); + rows_added = fillColumnsFromMap(map, columns_right); }; if (!joinDispatch(parent.kind, parent.strictness, parent.data->maps.front(), fill_callback)) @@ -2017,14 +2017,14 @@ private: return rows_added; } - template + template size_t fillColumnsFromMap(const Maps & maps, MutableColumns & columns_keys_and_right) { switch (parent.data->type) { #define M(TYPE) \ case HashJoin::Type::TYPE: \ - return fillColumns(*maps.TYPE, columns_keys_and_right); + return fillColumns(*maps.TYPE, columns_keys_and_right); APPLY_FOR_JOIN_VARIANTS(M) #undef M default: @@ -2034,7 +2034,7 @@ private: UNREACHABLE(); } - template + template size_t fillColumns(const Map & map, MutableColumns & columns_keys_and_right) { size_t rows_added = 0; From 966a09fb70465a161353ac005600608428da7847 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Dec 2023 12:42:23 +0100 Subject: [PATCH 247/331] Review fixes --- src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp | 2 +- src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp | 2 -- src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp | 2 +- 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index d369b8f3788..bb7e108a34e 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -549,7 +549,7 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl return; } - auto log_table_structure_changed = [&](std::string_view reason) + auto log_table_structure_changed = [&](const std::string & reason) { LOG_INFO(log, "Table structure of the table {} changed ({}), " "will mark it as skipped from replication. " diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 35daced0fa9..43de2069b19 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -447,8 +447,6 @@ StorageInfo PostgreSQLReplicationHandler::loadFromSnapshot(postgres::Connection assertBlocksHaveEqualStructure(input->getPort().getHeader(), block_io.pipeline.getHeader(), "postgresql replica load from snapshot"); block_io.pipeline.complete(Pipe(std::move(input))); - /// TODO: make a test when we fail in the middle of inserting data from source. - CompletedPipelineExecutor executor(block_io.pipeline); executor.execute(); diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index 9eb35d14506..21bb5d9316f 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -198,7 +198,7 @@ void StorageMaterializedPostgreSQL::createNestedIfNeeded(PostgreSQLTableStructur auto table_id = getStorageID(); auto tmp_nested_table_id = StorageID(table_id.database_name, getNestedTableName()); LOG_DEBUG(log, "Creating clickhouse table for postgresql table {} (ast: {})", - table_id.getNameForLogs(), serializeAST(*ast_create)); + table_id.getNameForLogs(), ast_create->formatForLogging()); InterpreterCreateQuery interpreter(ast_create, nested_context); interpreter.execute(); From a62a0b92de558ac2376eecf54f883353b6a1afb9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 11 Dec 2023 12:46:22 +0100 Subject: [PATCH 248/331] Increase async block cache deduplication timeout --- src/Storages/MergeTree/AsyncBlockIDsCache.cpp | 36 +++++++++---------- src/Storages/MergeTree/AsyncBlockIDsCache.h | 1 + src/Storages/MergeTree/MergeTreeSettings.h | 3 +- 3 files changed, 21 insertions(+), 19 deletions(-) diff --git a/src/Storages/MergeTree/AsyncBlockIDsCache.cpp b/src/Storages/MergeTree/AsyncBlockIDsCache.cpp index a9cdd09e061..4f3a8f16366 100644 --- a/src/Storages/MergeTree/AsyncBlockIDsCache.cpp +++ b/src/Storages/MergeTree/AsyncBlockIDsCache.cpp @@ -79,11 +79,12 @@ catch (...) template AsyncBlockIDsCache::AsyncBlockIDsCache(TStorage & storage_) - : storage(storage_), - update_min_interval(storage.getSettings()->async_block_ids_cache_min_update_interval_ms), - path(storage.getZooKeeperPath() + "/async_blocks"), - log_name(storage.getStorageID().getFullTableName() + " (AsyncBlockIDsCache)"), - log(&Poco::Logger::get(log_name)) + : storage(storage_) + , update_min_interval(storage.getSettings()->async_block_ids_cache_min_update_interval_ms) + , update_wait(storage.getSettings()->async_block_ids_cache_update_wait_ms) + , path(storage.getZooKeeperPath() + "/async_blocks") + , log_name(storage.getStorageID().getFullTableName() + " (AsyncBlockIDsCache)") + , log(&Poco::Logger::get(log_name)) { task = storage.getContext()->getSchedulePool().createTask(log_name, [this]{ update(); }); } @@ -102,21 +103,20 @@ Strings AsyncBlockIDsCache::detectConflicts(const Strings & paths, UIn if (!storage.getSettings()->use_async_block_ids_cache) return {}; - std::unique_lock lk(mu); - /// For first time access of this cache, the `last_version` is zero, so it will not block here. - /// For retrying request, We compare the request version and cache version, because zk only returns - /// incomplete information of duplication, we need to update the cache to find out more duplication. - /// The timeout here is to prevent deadlock, just in case. - cv.wait_for(lk, update_min_interval * 2, [&]{return version != last_version;}); - - if (version == last_version) - LOG_INFO(log, "Read cache with a old version {}", last_version); - CachePtr cur_cache; - cur_cache = cache_ptr; - last_version = version; + { + std::unique_lock lk(mu); + /// For first time access of this cache, the `last_version` is zero, so it will not block here. + /// For retrying request, We compare the request version and cache version, because zk only returns + /// incomplete information of duplication, we need to update the cache to find out more duplication. + cv.wait_for(lk, update_wait, [&]{return version != last_version;}); - lk.unlock(); + if (version == last_version) + LOG_INFO(log, "Read cache with a old version {}", last_version); + + cur_cache = cache_ptr; + last_version = version; + } if (cur_cache == nullptr) return {}; diff --git a/src/Storages/MergeTree/AsyncBlockIDsCache.h b/src/Storages/MergeTree/AsyncBlockIDsCache.h index fbd97fd00ff..8ce65ec4927 100644 --- a/src/Storages/MergeTree/AsyncBlockIDsCache.h +++ b/src/Storages/MergeTree/AsyncBlockIDsCache.h @@ -33,6 +33,7 @@ private: std::atomic last_updatetime; const std::chrono::milliseconds update_min_interval; + const std::chrono::milliseconds update_wait; std::mutex mu; CachePtr cache_ptr; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 5bb712ea786..c71951503c8 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -95,7 +95,8 @@ struct Settings; M(UInt64, replicated_deduplication_window_seconds, 7 * 24 * 60 * 60 /* one week */, "Similar to \"replicated_deduplication_window\", but determines old blocks by their lifetime. Hash of an inserted block will be deleted (and the block will not be deduplicated after) if it outside of one \"window\". You can set very big replicated_deduplication_window to avoid duplicating INSERTs during that period of time.", 0) \ M(UInt64, replicated_deduplication_window_for_async_inserts, 10000, "How many last hash values of async_insert blocks should be kept in ZooKeeper (old blocks will be deleted).", 0) \ M(UInt64, replicated_deduplication_window_seconds_for_async_inserts, 7 * 24 * 60 * 60 /* one week */, "Similar to \"replicated_deduplication_window_for_async_inserts\", but determines old blocks by their lifetime. Hash of an inserted block will be deleted (and the block will not be deduplicated after) if it outside of one \"window\". You can set very big replicated_deduplication_window to avoid duplicating INSERTs during that period of time.", 0) \ - M(Milliseconds, async_block_ids_cache_min_update_interval_ms, 100, "Minimum interval between updates of async_block_ids_cache", 0) \ + M(Milliseconds, async_block_ids_cache_min_update_interval_ms, 1000, "Minimum interval between updates of async_block_ids_cache", 0) \ + M(Milliseconds, async_block_ids_cache_update_wait_ms, 100, "How long each insert iteration will wait for async_block_ids_cache update", 0) \ M(Bool, use_async_block_ids_cache, true, "Use in-memory cache to filter duplicated async inserts based on block ids", 0) \ M(UInt64, max_replicated_logs_to_keep, 1000, "How many records may be in log, if there is inactive replica. Inactive replica becomes lost when when this number exceed.", 0) \ M(UInt64, min_replicated_logs_to_keep, 10, "Keep about this number of last records in ZooKeeper log, even if they are obsolete. It doesn't affect work of tables: used only to diagnose ZooKeeper log before cleaning.", 0) \ From 9307a871187121b00ec75eeaab223a15741d037b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 11 Dec 2023 12:56:52 +0100 Subject: [PATCH 249/331] Tune network memory for dockerhub proxy hosts --- tests/ci/worker/dockerhub_proxy_template.sh | 22 ++++++++++++++++++++- 1 file changed, 21 insertions(+), 1 deletion(-) diff --git a/tests/ci/worker/dockerhub_proxy_template.sh b/tests/ci/worker/dockerhub_proxy_template.sh index 38f2bc6486a..7ca8d581df5 100644 --- a/tests/ci/worker/dockerhub_proxy_template.sh +++ b/tests/ci/worker/dockerhub_proxy_template.sh @@ -15,6 +15,19 @@ if [[ "$ETH_DNS" ]] && [[ "${ETH_DNS#*: }" != *"$CLOUDFLARE_NS"* ]]; then resolvectl dns "$IFACE" "${new_dns[@]}" fi +# tune sysctl for network performance +cat > /etc/sysctl.d/10-network-memory.conf << EOF +net.core.netdev_max_backlog=2000 +net.core.rmem_max=1048576 +net.core.wmem_max=1048576 +net.ipv4.tcp_max_syn_backlog=1024 +net.ipv4.tcp_rmem=4096 131072 16777216 +net.ipv4.tcp_wmem=4096 87380 16777216 +net.ipv4.tcp_mem=4096 131072 16777216 +EOF + +sysctl -p /etc/sysctl.d/10-network-memory.conf + mkdir /home/ubuntu/registrystorage sed -i 's/preserve_hostname: false/preserve_hostname: true/g' /etc/cloud/cloud.cfg @@ -22,4 +35,11 @@ sed -i 's/preserve_hostname: false/preserve_hostname: true/g' /etc/cloud/cloud.c REGISTRY_PROXY_USERNAME=robotclickhouse REGISTRY_PROXY_PASSWORD=$(aws ssm get-parameter --name dockerhub_robot_password --with-decryption | jq '.Parameter.Value' -r) -docker run -d --network=host -p 5000:5000 -v /home/ubuntu/registrystorage:/var/lib/registry -e REGISTRY_HTTP_ADDR=0.0.0.0:5000 -e REGISTRY_STORAGE_DELETE_ENABLED=true -e REGISTRY_PROXY_REMOTEURL=https://registry-1.docker.io -e REGISTRY_PROXY_PASSWORD="$REGISTRY_PROXY_PASSWORD" -e REGISTRY_PROXY_USERNAME="$REGISTRY_PROXY_USERNAME" --restart=always --name registry registry:2 +docker run -d --network=host -p 5000:5000 -v /home/ubuntu/registrystorage:/var/lib/registry \ + -e REGISTRY_STORAGE_CACHE='' \ + -e REGISTRY_HTTP_ADDR=0.0.0.0:5000 \ + -e REGISTRY_STORAGE_DELETE_ENABLED=true \ + -e REGISTRY_PROXY_REMOTEURL=https://registry-1.docker.io \ + -e REGISTRY_PROXY_PASSWORD="$REGISTRY_PROXY_PASSWORD" \ + -e REGISTRY_PROXY_USERNAME="$REGISTRY_PROXY_USERNAME" \ + --restart=always --name registry registry:2 From f5ba4ef9b33aa5bddbb43a5f86de5f8e3bf6be2d Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 11 Dec 2023 13:07:56 +0100 Subject: [PATCH 250/331] Tune autoscale to scale for single job in the queue (#57742) * tune setting desired capacity * Automatic style fix --------- Co-authored-by: robot-clickhouse --- tests/ci/autoscale_runners_lambda/app.py | 9 +++----- .../test_autoscale.py | 22 +++++-------------- 2 files changed, 8 insertions(+), 23 deletions(-) diff --git a/tests/ci/autoscale_runners_lambda/app.py b/tests/ci/autoscale_runners_lambda/app.py index b872f9edb79..120126b404a 100644 --- a/tests/ci/autoscale_runners_lambda/app.py +++ b/tests/ci/autoscale_runners_lambda/app.py @@ -66,11 +66,6 @@ def get_scales(runner_type: str) -> Tuple[int, int]: # Let's have it the same as the other ASG # UPDATE THE COMMENT ON CHANGES scale_up = 3 - if runner_type.startswith("private-"): - scale_up = 1 - elif runner_type == "limited-tester": - # The limited runners should inflate and deflate faster - scale_up = 2 return scale_down, scale_up @@ -120,7 +115,9 @@ def set_capacity( # Are we already at the capacity limits stop = stop or asg["MaxSize"] <= asg["DesiredCapacity"] # Let's calculate a new desired capacity - desired_capacity = asg["DesiredCapacity"] + (capacity_deficit // scale_up) + desired_capacity = ( + asg["DesiredCapacity"] + (capacity_deficit + scale_up - 1) // scale_up + ) desired_capacity = max(desired_capacity, asg["MinSize"]) desired_capacity = min(desired_capacity, asg["MaxSize"]) # Finally, should the capacity be even changed diff --git a/tests/ci/autoscale_runners_lambda/test_autoscale.py b/tests/ci/autoscale_runners_lambda/test_autoscale.py index 53e519c32f1..464e5695556 100644 --- a/tests/ci/autoscale_runners_lambda/test_autoscale.py +++ b/tests/ci/autoscale_runners_lambda/test_autoscale.py @@ -69,14 +69,14 @@ class TestSetCapacity(unittest.TestCase): # Do not change capacity TestCase("noqueue", 1, 13, 20, [Queue("in_progress", 155, "noqueue")], -1), TestCase( - "w/reserve-1", 1, 13, 20, [Queue("queued", 15, "w/reserve-1")], -1 + "w/reserve-1", 1, 13, 20, [Queue("queued", 15, "w/reserve-1")], 14 ), # Increase capacity - TestCase("increase-1", 1, 13, 20, [Queue("queued", 23, "increase-1")], 16), + TestCase("increase-1", 1, 13, 20, [Queue("queued", 23, "increase-1")], 17), TestCase( - "style-checker", 1, 13, 20, [Queue("queued", 33, "style-checker")], 19 + "style-checker", 1, 13, 20, [Queue("queued", 33, "style-checker")], 20 ), - TestCase("increase-2", 1, 13, 20, [Queue("queued", 18, "increase-2")], 14), + TestCase("increase-2", 1, 13, 20, [Queue("queued", 18, "increase-2")], 15), TestCase("increase-3", 1, 13, 20, [Queue("queued", 183, "increase-3")], 20), TestCase( "increase-w/o reserve", @@ -87,21 +87,9 @@ class TestSetCapacity(unittest.TestCase): Queue("in_progress", 11, "increase-w/o reserve"), Queue("queued", 12, "increase-w/o reserve"), ], - 16, + 17, ), TestCase("lower-min", 10, 5, 20, [Queue("queued", 5, "lower-min")], 10), - # scale up group with prefix private- - TestCase( - "private-increase", - 1, - 13, - 20, - [ - Queue("in_progress", 12, "private-increase"), - Queue("queued", 11, "private-increase"), - ], - 20, - ), # Decrease capacity TestCase("w/reserve", 1, 13, 20, [Queue("queued", 5, "w/reserve")], 5), TestCase( From af45e138ad2f5f17414bd81262c7b319632a40c6 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 11 Dec 2023 12:33:11 +0000 Subject: [PATCH 251/331] fix --- src/Interpreters/HashJoin.cpp | 8 ++++---- .../02516_join_with_totals_and_subquery_bug.reference | 9 ++++++--- .../02516_join_with_totals_and_subquery_bug.sql | 6 +++--- 3 files changed, 13 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index c79db392eb4..6e62e595caa 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -269,7 +269,7 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s sample_block_with_columns_to_add = right_table_keys = materializeBlock(right_sample_block); } - JoinCommon::convertToFullColumnsInplace(right_table_keys); + materializeBlockInplace(right_table_keys); initRightBlockStructure(data->sample_block); JoinCommon::createMissedColumns(sample_block_with_columns_to_add); @@ -2028,7 +2028,7 @@ private: APPLY_FOR_JOIN_VARIANTS(M) #undef M default: - throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys (type: {})", parent.data->type) ; + throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys (type: {})", parent.data->type); } UNREACHABLE(); @@ -2080,8 +2080,8 @@ private: { const Mapped & mapped = it->getMapped(); - size_t off = map.offsetInternal(it.getPtr()); - if (parent.isUsed(off)) + size_t offset = map.offsetInternal(it.getPtr()); + if (parent.isUsed(offset)) continue; AdderNonJoined::add(mapped, rows_added, columns_keys_and_right); diff --git a/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference b/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference index 51e8394a2f7..83571fd9005 100644 --- a/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference +++ b/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference @@ -10,19 +10,22 @@ 100000000000000000000 --- +[] 0 ['2'] ['0'] 2 ['0'] ['0'] 2 ['0'] -['1'] 1 ['1'] +['1'] 1 [] [] 3 [] --- +[] 0 ['2'] 1 ['0'] 2 ['0'] 2 -['1'] 1 ['1'] 1 +['1'] 1 [] 0 [] 3 [] 3 --- +[] ['2'] 1 ['0'] ['0'] 2 ['0'] ['0'] 2 -['1'] ['1'] 1 +['1'] [] 0 [] [] 3 diff --git a/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.sql b/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.sql index 243dceffc43..d39efb0b193 100644 --- a/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.sql +++ b/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.sql @@ -78,7 +78,7 @@ FROM ( WITH TOTALS ) AS l FULL JOIN ( - SELECT ([toString(number % 2)] :: Array(String)) AS item_id FROM numbers(3) + SELECT ([toString((number % 2) * 2)] :: Array(String)) AS item_id FROM numbers(3) ) AS r ON l.item_id = r.item_id ORDER BY 1,2,3 @@ -92,7 +92,7 @@ FROM ( WITH TOTALS ) AS l FULL JOIN ( - SELECT ([toString(number % 2)] :: Array(String)) AS item_id, count() FROM numbers(3) GROUP BY item_id + SELECT ([toString((number % 2) * 2)] :: Array(String)) AS item_id, count() FROM numbers(3) GROUP BY item_id WITH TOTALS ) AS r ON l.item_id = r.item_id @@ -106,7 +106,7 @@ FROM ( SELECT ([toString(number % 2)] :: Array(String)) AS item_id FROM numbers(3) ) AS l FULL JOIN ( - SELECT ([toString(number % 2)] :: Array(LowCardinality(String))) AS item_id, count() FROM numbers(3) GROUP BY item_id + SELECT ([toString((number % 2) * 2)] :: Array(LowCardinality(String))) AS item_id, count() FROM numbers(3) GROUP BY item_id WITH TOTALS ) AS r ON l.item_id = r.item_id From 1e4d61d55a7a83821583b53d842886bcaee371ec Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Dec 2023 13:44:12 +0100 Subject: [PATCH 252/331] Change error code --- src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index bb7e108a34e..a7ac609a4c5 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -22,6 +22,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int POSTGRESQL_REPLICATION_INTERNAL_ERROR; extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_COLUMN; } namespace @@ -104,7 +105,7 @@ MaterializedPostgreSQLConsumer::StorageData::Buffer::Buffer( columns_.begin(), columns_.end(), [](const auto & col) { return col.name == "_sign" || col.name == "_version"; })) { - throw Exception(ErrorCodes::LOGICAL_ERROR, + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "PostgreSQL table cannot contain `_sign` or `_version` columns " "as they are reserved for internal usage"); } From 821c7d65271f8319e0867d003a71bdcca6f65bf7 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 11 Dec 2023 15:48:52 +0300 Subject: [PATCH 253/331] Fixed tests --- src/DataTypes/Serializations/SerializationString.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index c59c642cf04..308bdce0507 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -174,7 +174,7 @@ static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnSt offsets.push_back(offset); if (unlikely(offset > data.size())) - data.resize(data.size() * 2); + data.resize(roundUpToPowerOfTwoOrZero(std::max(offset, data.size() * 2))); if (size) { From fc4e6d70d863cdc6c714ee12d91c3958efd882e3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Dec 2023 14:24:05 +0100 Subject: [PATCH 254/331] Update run.sh --- docker/test/stateful/run.sh | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 829b3547856..a0def50bfb5 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -24,11 +24,15 @@ azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log & config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml -cache_policy="SLRU" -#TODO: uncomment this before merge, for testing purposes it is SLRU only before merge. -#if [$(($RANDOM%2)) -eq 1]; then -# cache_policy="LRU" -#fi +cache_policy="" +if [ $(( $(date +%-d) % 2 )) -eq 1 ]; then + cache_policy="SLRU" +else + cache_policy="LRU" +fi + +echo "Using cache policy: $cache_policy" + if [ "$cache_policy" = "SLRU" ]; then sudo cat /etc/clickhouse-server/config.d/storage_conf.xml \ | sed "s|LRU|SLRU|" \ From fcaa556bf79e4821e4c1a82b3fc8919c21127158 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 11 Dec 2023 13:39:10 +0000 Subject: [PATCH 255/331] Fixing test. --- .../Transforms/AggregatingTransform.cpp | 36 ++++++++++--------- 1 file changed, 19 insertions(+), 17 deletions(-) diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index ec8ea9396e4..7b1d51bb320 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -405,26 +405,28 @@ private: } } - if (!shared_data->is_bucket_processed[current_bucket_num]) - return Status::NeedData; - - if (!two_level_chunks[current_bucket_num]) - return Status::NeedData; - - auto chunk = std::move(two_level_chunks[current_bucket_num]); - const auto has_rows = chunk.hasRows(); - if (has_rows) - output.push(std::move(chunk)); - - ++current_bucket_num; - if (current_bucket_num == NUM_BUCKETS) + while (current_bucket_num < NUM_BUCKETS) { - output.finish(); - /// Do not close inputs, they must be finished. - return Status::Finished; + if (!shared_data->is_bucket_processed[current_bucket_num]) + return Status::NeedData; + + if (!two_level_chunks[current_bucket_num]) + return Status::NeedData; + + auto chunk = std::move(two_level_chunks[current_bucket_num]); + ++current_bucket_num; + + const auto has_rows = chunk.hasRows(); + if (has_rows) + { + output.push(std::move(chunk)); + return Status::PortFull; + } } - return has_rows ? Status::PortFull : Status::NeedData; + output.finish(); + /// Do not close inputs, they must be finished. + return Status::Finished; } AggregatingTransformParamsPtr params; From 6b1acf7e9b7697b28215450cb8a86c2a9284c687 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 11 Dec 2023 17:06:13 +0300 Subject: [PATCH 256/331] Test --- tests/queries/0_stateless/01926_order_by_desc_limit.sql | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01926_order_by_desc_limit.sql b/tests/queries/0_stateless/01926_order_by_desc_limit.sql index a0047a2925a..6854e6c1e84 100644 --- a/tests/queries/0_stateless/01926_order_by_desc_limit.sql +++ b/tests/queries/0_stateless/01926_order_by_desc_limit.sql @@ -11,11 +11,9 @@ SETTINGS index_granularity = 1024, index_granularity_bytes = '10Mi'; INSERT INTO order_by_desc SELECT number, repeat('a', 1024) FROM numbers(1024 * 300); OPTIMIZE TABLE order_by_desc FINAL; -SELECT s FROM order_by_desc ORDER BY u DESC LIMIT 10 FORMAT Null -SETTINGS max_memory_usage = '400M'; +SELECT s FROM order_by_desc ORDER BY u DESC LIMIT 10 FORMAT Null; -SELECT s FROM order_by_desc ORDER BY u LIMIT 10 FORMAT Null -SETTINGS max_memory_usage = '400M'; +SELECT s FROM order_by_desc ORDER BY u LIMIT 10 FORMAT Null; SYSTEM FLUSH LOGS; From 2c033487323796c8f35b1400817e5b9571d4a0c7 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 11 Dec 2023 15:20:46 +0100 Subject: [PATCH 257/331] Use checkTimeLimit() to check if query is cancelled. --- src/Functions/sleep.h | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Functions/sleep.h b/src/Functions/sleep.h index 36fa14cd36e..11b8e48a295 100644 --- a/src/Functions/sleep.h +++ b/src/Functions/sleep.h @@ -28,7 +28,6 @@ namespace ErrorCodes extern const int TOO_SLOW; extern const int ILLEGAL_COLUMN; extern const int BAD_ARGUMENTS; - extern const int QUERY_WAS_CANCELLED; } /** sleep(seconds) - the specified number of seconds sleeps each columns. @@ -144,8 +143,8 @@ public: sleepForMicroseconds(sleep_ms); microseconds -= sleep_ms; - if (query_status && query_status->isKilled()) - throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled"); + if (query_status && !query_status->checkTimeLimit()) + break; } ProfileEvents::increment(ProfileEvents::SleepFunctionCalls, count); From ff1e95c3575c4b3956e2f7eacda3af7a41151891 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 11 Dec 2023 15:37:19 +0100 Subject: [PATCH 258/331] Fix style. --- tests/queries/0_stateless/02932_kill_query_sleep.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02932_kill_query_sleep.sh b/tests/queries/0_stateless/02932_kill_query_sleep.sh index 81bb892bc15..836d7f2c686 100755 --- a/tests/queries/0_stateless/02932_kill_query_sleep.sh +++ b/tests/queries/0_stateless/02932_kill_query_sleep.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function wait_query_started() { local query_id="$1" - while [[ $($CLICKHOUSE_CLIENT --query="SELECT count() FROM system.query_log WHERE query_id='$query_id'") == 0 ]]; do + while [[ $($CLICKHOUSE_CLIENT --query="SELECT count() FROM system.query_log WHERE query_id='$query_id' AND current_database = currentDatabase()") == 0 ]]; do sleep 0.1; $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS;" done @@ -35,4 +35,4 @@ echo "Cancelling query" kill_query "$sleep_query_id" $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS;" -$CLICKHOUSE_CLIENT --query "SELECT exception FROM system.query_log WHERE query_id='$sleep_query_id'" | grep -oF "QUERY_WAS_CANCELLED" +$CLICKHOUSE_CLIENT --query "SELECT exception FROM system.query_log WHERE query_id='$sleep_query_id' AND current_database = currentDatabase()" | grep -oF "QUERY_WAS_CANCELLED" From a87a8e91cf1ae8bf449d4f7d95278ab79114d6a3 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 11 Dec 2023 14:46:12 +0000 Subject: [PATCH 259/331] Slightly better inference of unnamed tupes in JSON formats --- src/Formats/SchemaInferenceUtils.cpp | 14 ++++++++++++-- ...son_array_of_unnamed_tuples_inference.reference | 1 + ...2940_json_array_of_unnamed_tuples_inference.sql | 2 ++ 3 files changed, 15 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02940_json_array_of_unnamed_tuples_inference.reference create mode 100644 tests/queries/0_stateless/02940_json_array_of_unnamed_tuples_inference.sql diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 94166aa9002..e6586d2d271 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -1247,11 +1247,22 @@ void transformFinalInferredJSONTypeIfNeededImpl(DataTypePtr & data_type, const F return; } + /// First, try to transform nested types without final transformations to see if there is a common type. + auto nested_types_copy = nested_types; + transformInferredTypesIfNeededImpl(nested_types_copy, settings, json_info); + if (checkIfTypesAreEqual(nested_types_copy)) + { + data_type = std::make_shared(nested_types_copy.back()); + transformFinalInferredJSONTypeIfNeededImpl(data_type, settings, json_info); + return; + } + + /// Apply final transformation to nested types, and then try to find common type. for (auto & nested_type : nested_types) /// Don't change Nothing to String in nested types here, because we are not sure yet if it's Array or actual Tuple transformFinalInferredJSONTypeIfNeededImpl(nested_type, settings, json_info, /*remain_nothing_types=*/ true); - auto nested_types_copy = nested_types; + nested_types_copy = nested_types; transformInferredTypesIfNeededImpl(nested_types_copy, settings, json_info); if (checkIfTypesAreEqual(nested_types_copy)) { @@ -1381,7 +1392,6 @@ DataTypePtr makeNullableRecursively(DataTypePtr type) return std::make_shared(std::move(nested_types), tuple_type->getElementNames()); return std::make_shared(std::move(nested_types)); - } if (which.isMap()) diff --git a/tests/queries/0_stateless/02940_json_array_of_unnamed_tuples_inference.reference b/tests/queries/0_stateless/02940_json_array_of_unnamed_tuples_inference.reference new file mode 100644 index 00000000000..aac3e471264 --- /dev/null +++ b/tests/queries/0_stateless/02940_json_array_of_unnamed_tuples_inference.reference @@ -0,0 +1 @@ +data Array(Tuple(Nullable(Int64), Tuple(a Nullable(Int64), b Nullable(Int64)), Nullable(Int64), Nullable(String))) diff --git a/tests/queries/0_stateless/02940_json_array_of_unnamed_tuples_inference.sql b/tests/queries/0_stateless/02940_json_array_of_unnamed_tuples_inference.sql new file mode 100644 index 00000000000..a8a7af1f96c --- /dev/null +++ b/tests/queries/0_stateless/02940_json_array_of_unnamed_tuples_inference.sql @@ -0,0 +1,2 @@ +desc format(JSONEachRow, '{"data" : [[1, null, 3, null], [null, {"a" : 12, "b" : 12}, null, "string"], [null, null, 4, "string"]]}'); + From ca262d6e10af713f5d296cf90d05e59ad00be50d Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 11 Dec 2023 15:55:34 +0100 Subject: [PATCH 260/331] Fix profile events. --- src/Common/ProfileEvents.cpp | 3 ++- src/Functions/sleep.h | 13 ++++++++----- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index f9ea03f4947..7d6abd587c5 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -456,7 +456,8 @@ The server successfully detected this situation and will download merged part fr M(ReadBufferSeekCancelConnection, "Number of seeks which lead to new connection (s3, http)") \ \ M(SleepFunctionCalls, "Number of times a sleep function (sleep, sleepEachRow) has been called.") \ - M(SleepFunctionMicroseconds, "Time spent sleeping due to a sleep function call.") \ + M(SleepFunctionMicroseconds, "Time set to sleep in a sleep function (sleep, sleepEachRow).") \ + M(SleepFunctionElapsedMicroseconds, "Time spent sleeping in a sleep function (sleep, sleepEachRow).") \ \ M(ThreadPoolReaderPageCacheHit, "Number of times the read inside ThreadPoolReader was done from page cache.") \ M(ThreadPoolReaderPageCacheHitBytes, "Number of bytes read inside ThreadPoolReader when it was done from page cache.") \ diff --git a/src/Functions/sleep.h b/src/Functions/sleep.h index 11b8e48a295..f5d3b6f29cd 100644 --- a/src/Functions/sleep.h +++ b/src/Functions/sleep.h @@ -17,6 +17,7 @@ namespace ProfileEvents { extern const Event SleepFunctionCalls; extern const Event SleepFunctionMicroseconds; +extern const Event SleepFunctionElapsedMicroseconds; } namespace DB @@ -134,14 +135,15 @@ public: "The maximum sleep time is {} microseconds. Requested: {} microseconds per block (of size {})", max_microseconds, microseconds, size); - while (microseconds) + UInt64 elapsed = 0; + while (elapsed < microseconds) { - UInt64 sleep_ms = microseconds; + UInt64 sleep_time = microseconds - elapsed; if (query_status) - sleep_ms = std::min(sleep_ms, /* 1 second */ static_cast(1000000)); + sleep_time = std::min(sleep_time, /* 1 second */ static_cast(1000000)); - sleepForMicroseconds(sleep_ms); - microseconds -= sleep_ms; + sleepForMicroseconds(sleep_time); + elapsed += sleep_time; if (query_status && !query_status->checkTimeLimit()) break; @@ -149,6 +151,7 @@ public: ProfileEvents::increment(ProfileEvents::SleepFunctionCalls, count); ProfileEvents::increment(ProfileEvents::SleepFunctionMicroseconds, microseconds); + ProfileEvents::increment(ProfileEvents::SleepFunctionElapsedMicroseconds, elapsed); } } From ea09080c73102e118fc01c4286eff74bcf904573 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 11 Dec 2023 15:50:27 +0000 Subject: [PATCH 261/331] fix --- src/Interpreters/HashJoin.cpp | 62 +++++++++++++--------------------- src/Interpreters/JoinUtils.cpp | 14 -------- src/Interpreters/JoinUtils.h | 1 - src/Storages/StorageJoin.cpp | 3 +- 4 files changed, 24 insertions(+), 56 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 6e62e595caa..ce4236317ac 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -373,10 +373,20 @@ HashJoin::Type HashJoin::chooseMethod(JoinKind kind, const ColumnRawPtrs & key_c return Type::keys256; /// If there is single string key, use hash table of it's values. - if (keys_size == 1 - && (typeid_cast(key_columns[0]) - || (isColumnConst(*key_columns[0]) && typeid_cast(&assert_cast(key_columns[0])->getDataColumn())))) - return Type::key_string; + if (keys_size == 1) + { + auto is_string_column = [](const IColumn * column_ptr) -> bool + { + if (const auto * lc_column_ptr = typeid_cast(column_ptr)) + return typeid_cast(lc_column_ptr->getDictionary().getNestedColumn().get()); + return typeid_cast(column_ptr); + }; + + const auto * key_column = key_columns[0]; + if (is_string_column(key_column) || + (isColumnConst(*key_column) && is_string_column(assert_cast(key_column)->getDataColumnPtr().get()))) + return Type::key_string; + } if (keys_size == 1 && typeid_cast(key_columns[0])) return Type::key_fixed_string; @@ -791,7 +801,13 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) size_t rows = source_block.rows(); - ColumnPtrMap all_key_columns = JoinCommon::materializeColumnsInplaceMap(source_block, table_join->getAllNames(JoinTableSide::Right)); + const auto & right_key_names = table_join->getAllNames(JoinTableSide::Right); + ColumnPtrMap all_key_columns(right_key_names.size()); + for (const auto & column_name : right_key_names) + { + const auto & column = source_block.getByName(column_name).column; + all_key_columns[column_name] = recursiveRemoveLowCardinality(recursiveRemoveSparse(column->convertToFullColumnIfConst())); + } Block block_to_save = prepareRightBlock(source_block); if (shrink_blocks) @@ -804,6 +820,8 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "addBlockToJoin called when HashJoin locked to prevent updates"); data->blocks_allocated_size += block_to_save.allocatedBytes(); + + assertBlocksHaveEqualStructure(data->sample_block, block_to_save, "Saved joined block structure mismatch"); data->blocks.emplace_back(std::move(block_to_save)); Block * stored_block = &data->blocks.back(); @@ -1061,33 +1079,6 @@ public: return ColumnWithTypeAndName(std::move(columns[i]), type_name[i].type, type_name[i].qualified_name); } - static void assertBlockEqualsStructureUpToLowCard(const Block & lhs_block, const Block & rhs_block) - { - if (lhs_block.columns() != rhs_block.columns()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Different number of columns in blocks [{}] and [{}]", - lhs_block.dumpStructure(), rhs_block.dumpStructure()); - - for (size_t i = 0; i < lhs_block.columns(); ++i) - { - const auto & lhs = lhs_block.getByPosition(i); - const auto & rhs = rhs_block.getByPosition(i); - if (lhs.name != rhs.name) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Block structure mismatch: [{}] != [{}] ({} != {})", - lhs_block.dumpStructure(), rhs_block.dumpStructure(), lhs.name, rhs.name); - - const auto & ltype = recursiveRemoveLowCardinality(lhs.type); - const auto & rtype = recursiveRemoveLowCardinality(rhs.type); - if (!ltype->equals(*rtype)) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Block structure mismatch: [{}] != [{}] ({} != {})", - lhs_block.dumpStructure(), rhs_block.dumpStructure(), ltype->getName(), rtype->getName()); - - const auto & lcol = recursiveRemoveLowCardinality(lhs.column); - const auto & rcol = recursiveRemoveLowCardinality(rhs.column); - if (lcol->getDataType() != rcol->getDataType()) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Block structure mismatch: [{}] != [{}] ({} != {})", - lhs_block.dumpStructure(), rhs_block.dumpStructure(), lcol->getDataType(), rcol->getDataType()); - } - } template void appendFromBlock(const Block & block, size_t row_num) @@ -1095,13 +1086,6 @@ public: if constexpr (has_defaults) applyLazyDefaults(); -#ifndef NDEBUG - /// Like assertBlocksHaveEqualStructure but doesn't check low cardinality - assertBlockEqualsStructureUpToLowCard(sample_block, block); -#else - UNUSED(assertBlockEqualsStructureUpToLowCard); -#endif - if (is_join_get) { /// If it's joinGetOrNull, we need to wrap not-nullable columns in StorageJoin. diff --git a/src/Interpreters/JoinUtils.cpp b/src/Interpreters/JoinUtils.cpp index be5ee80bd53..949a97d5748 100644 --- a/src/Interpreters/JoinUtils.cpp +++ b/src/Interpreters/JoinUtils.cpp @@ -315,20 +315,6 @@ ColumnRawPtrs materializeColumnsInplace(Block & block, const Names & names) return ptrs; } -ColumnPtrMap materializeColumnsInplaceMap(const Block & block, const Names & names) -{ - ColumnPtrMap ptrs; - ptrs.reserve(names.size()); - - for (const auto & column_name : names) - { - ColumnPtr column = block.getByName(column_name).column; - ptrs[column_name] = materializeColumn(column); - } - - return ptrs; -} - ColumnPtr materializeColumn(const Block & block, const String & column_name) { const auto & src_column = block.getByName(column_name).column; diff --git a/src/Interpreters/JoinUtils.h b/src/Interpreters/JoinUtils.h index 7daed6b7f7e..a88fca02bd8 100644 --- a/src/Interpreters/JoinUtils.h +++ b/src/Interpreters/JoinUtils.h @@ -70,7 +70,6 @@ ColumnPtr emptyNotNullableClone(const ColumnPtr & column); ColumnPtr materializeColumn(const Block & block, const String & name); Columns materializeColumns(const Block & block, const Names & names); ColumnRawPtrs materializeColumnsInplace(Block & block, const Names & names); -ColumnPtrMap materializeColumnsInplaceMap(const Block & block, const Names & names); ColumnRawPtrs getRawPointers(const Columns & columns); void convertToFullColumnsInplace(Block & block); void convertToFullColumnsInplace(Block & block, const Names & names, bool change_type = true); diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 121d859a3f2..efe446a8ccd 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -535,8 +535,7 @@ private: #undef M default: - throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys in StorageJoin. Type: {}", - static_cast(join->data->type)); + throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys of type {} in StorageJoin", join->data->type); } if (!rows_added) From 380e34226fbfbbf6dcbc22188a50bb5afdce42c3 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 11 Dec 2023 16:57:18 +0100 Subject: [PATCH 262/331] Fix compilation. --- src/Functions/sleep.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/sleep.h b/src/Functions/sleep.h index f5d3b6f29cd..73d58ca6b5b 100644 --- a/src/Functions/sleep.h +++ b/src/Functions/sleep.h @@ -140,7 +140,7 @@ public: { UInt64 sleep_time = microseconds - elapsed; if (query_status) - sleep_time = std::min(sleep_time, /* 1 second */ static_cast(1000000)); + sleep_time = std::min(sleep_time, /* 1 second */ static_cast(1000000)); sleepForMicroseconds(sleep_time); elapsed += sleep_time; From c76ceb29fd3619b755c631d0d0aa4a60a602a97b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Dec 2023 17:21:02 +0100 Subject: [PATCH 263/331] Minor changes --- docker/test/clickbench/run.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docker/test/clickbench/run.sh b/docker/test/clickbench/run.sh index 82eb06dffcd..7357fa6df86 100755 --- a/docker/test/clickbench/run.sh +++ b/docker/test/clickbench/run.sh @@ -31,6 +31,8 @@ clickhouse-client --time < /create.sql # Run the queries +set +x + TRIES=3 QUERY_NUM=1 while read -r query; do @@ -47,6 +49,8 @@ while read -r query; do QUERY_NUM=$((QUERY_NUM + 1)) done < /queries.sql +set -x + clickhouse-client --query "SELECT total_bytes FROM system.tables WHERE name = 'hits' AND database = 'default'" echo -e "success\tClickBench finished" > /test_output/check_status.tsv From 48ce04062125b93c0fa3e83785ef18f4d7f54dce Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Dec 2023 17:30:23 +0100 Subject: [PATCH 264/331] Fix --- src/Databases/DDLLoadingDependencyVisitor.cpp | 7 +++ src/Databases/DatabaseOrdinary.cpp | 2 + .../StorageMaterializedPostgreSQL.cpp | 3 - .../StorageMaterializedPostgreSQL.h | 3 + .../configs/log_conf.xml | 6 ++ .../test.py | 56 +++++++++++++++++++ 6 files changed, 74 insertions(+), 3 deletions(-) diff --git a/src/Databases/DDLLoadingDependencyVisitor.cpp b/src/Databases/DDLLoadingDependencyVisitor.cpp index 77a40f674fd..8f1b82941f6 100644 --- a/src/Databases/DDLLoadingDependencyVisitor.cpp +++ b/src/Databases/DDLLoadingDependencyVisitor.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -131,6 +132,12 @@ void DDLLoadingDependencyVisitor::visit(const ASTStorage & storage, Data & data) extractTableNameFromArgument(*storage.engine, data, 3); else if (storage.engine->name == "Dictionary") extractTableNameFromArgument(*storage.engine, data, 0); + else if (storage.engine->name == "MaterializedPostgreSQL") + { + const auto * create_query = data.create_query->as(); + auto nested_table = toString(create_query->uuid) + StorageMaterializedPostgreSQL::NESTED_TABLE_SUFFIX; + data.dependencies.emplace(QualifiedTableName{ .database = create_query->getDatabase(), .table = nested_table }); + } } diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 1f344551c5e..9a9dcf22c88 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -139,6 +139,8 @@ void DatabaseOrdinary::loadTableFromMetadata( assert(name.database == TSA_SUPPRESS_WARNING_FOR_READ(database_name)); const auto & query = ast->as(); + LOG_TRACE(log, "Loading table {}", name.getFullName()); + try { auto [table_name, table] = createTableFromAST( diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index a287b96fe51..9cceb3ec503 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -45,9 +45,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -static const auto NESTED_TABLE_SUFFIX = "_nested"; -static const auto TMP_SUFFIX = "_tmp"; - /// For the case of single storage. StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL( diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h index ca7b801cb7c..6bc0856e4dd 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h @@ -63,6 +63,9 @@ namespace DB class StorageMaterializedPostgreSQL final : public IStorage, WithContext { public: + static constexpr auto NESTED_TABLE_SUFFIX = "_nested"; + static constexpr auto TMP_SUFFIX = "_tmp"; + StorageMaterializedPostgreSQL(const StorageID & table_id_, ContextPtr context_, const String & postgres_database_name, const String & postgres_table_name); diff --git a/tests/integration/test_postgresql_replica_database_engine_2/configs/log_conf.xml b/tests/integration/test_postgresql_replica_database_engine_2/configs/log_conf.xml index 6cc1128e130..c9f6195a014 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/configs/log_conf.xml +++ b/tests/integration/test_postgresql_replica_database_engine_2/configs/log_conf.xml @@ -24,4 +24,10 @@ postgres_database + + system + text_log
+ 7500 + Test +
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 df72a2f705c..aa28db9a56d 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -944,6 +944,62 @@ def test_symbols_in_publication_name(started_cluster): ) +def test_dependent_loading(started_cluster): + table = "test_dependent_loading" + + pg_manager.create_postgres_table(table) + instance.query( + f"INSERT INTO postgres_database.{table} SELECT number, number from numbers(0, 50)" + ) + + instance.query( + f""" + SET allow_experimental_materialized_postgresql_table=1; + CREATE TABLE {table} (key Int32, value Int32) + ENGINE=MaterializedPostgreSQL('{started_cluster.postgres_ip}:{started_cluster.postgres_port}', 'postgres_database', '{table}', 'postgres', 'mysecretpassword') ORDER BY key + """ + ) + + check_tables_are_synchronized( + instance, + table, + postgres_database=pg_manager.get_default_database(), + materialized_database="default", + ) + + assert 50 == int(instance.query(f"SELECT count() FROM {table}")) + + instance.restart_clickhouse() + + check_tables_are_synchronized( + instance, + table, + postgres_database=pg_manager.get_default_database(), + materialized_database="default", + ) + + assert 50 == int(instance.query(f"SELECT count() FROM {table}")) + + uuid = instance.query( + f"SELECT uuid FROM system.tables WHERE name='{table}' and database='default' limit 1" + ).strip() + nested_table = f"default.`{uuid}_nested`" + instance.contains_in_log( + f"Table default.{table} has 1 dependencies: {nested_table} (level 1)" + ) + + instance.query("SYSTEM FLUSH LOGS") + nested_time = instance.query( + f"SELECT event_time_microseconds FROM system.text_log WHERE message like 'Loading table default.{uuid}_nested' and message not like '%like%'" + ).strip() + time = instance.query( + f"SELECT event_time_microseconds FROM system.text_log WHERE message like 'Loading table default.{table}' and message not like '%like%'" + ).strip() + instance.query( + f"SELECT toDateTime64('{nested_time}', 6) < toDateTime64('{time}', 6)" + ) + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From c3a76fcc0888f0cb2db67a694157d9bb17aeb267 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 16 Nov 2023 13:25:31 +0000 Subject: [PATCH 265/331] Allow to infer numbers from strings in CSV format --- src/Core/Settings.h | 1 + src/Formats/EscapingRuleUtils.cpp | 4 ++-- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + .../02916_csv_infer_numbers_from_strings.reference | 6 ++++++ .../0_stateless/02916_csv_infer_numbers_from_strings.sql | 4 ++++ 6 files changed, 15 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02916_csv_infer_numbers_from_strings.reference create mode 100644 tests/queries/0_stateless/02916_csv_infer_numbers_from_strings.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9601cd3e398..1e3b05bedb0 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -942,6 +942,7 @@ class IColumn; M(UInt64, input_format_max_rows_to_read_for_schema_inference, 25000, "The maximum rows of data to read for automatic schema inference", 0) \ M(UInt64, input_format_max_bytes_to_read_for_schema_inference, 32 * 1024 * 1024, "The maximum bytes of data to read for automatic schema inference", 0) \ M(Bool, input_format_csv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in CSV format", 0) \ + M(Bool, input_format_csv_try_infer_numbers_from_strings, false, "Try to infer numbers from string fields while schema inference in CSV format", 0) \ M(Bool, input_format_tsv_use_best_effort_in_schema_inference, true, "Use some tweaks and heuristics to infer schema in TSV format", 0) \ M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index d429985e52a..9cc7cb3b89e 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -303,8 +303,8 @@ DataTypePtr tryInferDataTypeByEscapingRule(const String & field, const FormatSet /// Try to determine the type of value inside quotes auto type = tryInferDataTypeForSingleField(data, format_settings); - /// If we couldn't infer any type or it's a number or tuple in quotes, we determine it as a string. - if (!type || isNumber(removeNullable(type)) || isTuple(type)) + /// If we couldn't infer any type or it's tuple in quotes or it's a number and csv.try_infer_numbers_from_strings = 0, we determine it as a string. + if (!type || isTuple(type) || (isNumber(type) && !format_settings.csv.try_infer_numbers_from_strings)) return std::make_shared(); return type; diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index b2ff9b1c3f7..1d62e58176b 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -74,6 +74,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.allow_whitespace_or_tab_as_delimiter = settings.input_format_csv_allow_whitespace_or_tab_as_delimiter; format_settings.csv.allow_variable_number_of_columns = settings.input_format_csv_allow_variable_number_of_columns; format_settings.csv.use_default_on_bad_values = settings.input_format_csv_use_default_on_bad_values; + format_settings.csv.try_infer_numbers_from_strings = settings.input_format_csv_try_infer_numbers_from_strings; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index d9e3a420502..8d5c044a311 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -164,6 +164,7 @@ struct FormatSettings bool allow_whitespace_or_tab_as_delimiter = false; bool allow_variable_number_of_columns = false; bool use_default_on_bad_values = false; + bool try_infer_numbers_from_strings = true; } csv; struct HiveText diff --git a/tests/queries/0_stateless/02916_csv_infer_numbers_from_strings.reference b/tests/queries/0_stateless/02916_csv_infer_numbers_from_strings.reference new file mode 100644 index 00000000000..f64557f1b70 --- /dev/null +++ b/tests/queries/0_stateless/02916_csv_infer_numbers_from_strings.reference @@ -0,0 +1,6 @@ +c1 Nullable(Int64) +c2 Nullable(Float64) +c3 Nullable(Bool) +c1 Nullable(String) +c2 Nullable(String) +c3 Nullable(String) diff --git a/tests/queries/0_stateless/02916_csv_infer_numbers_from_strings.sql b/tests/queries/0_stateless/02916_csv_infer_numbers_from_strings.sql new file mode 100644 index 00000000000..713d3d7190c --- /dev/null +++ b/tests/queries/0_stateless/02916_csv_infer_numbers_from_strings.sql @@ -0,0 +1,4 @@ +set input_format_csv_try_infer_numbers_from_strings=1; +desc format(CSV, '"42","42.42","True"'); +desc format(CSV, '"42","42.42","True"\n"abc","def","ghk"'); + From 1db000dae5e555723d9ad6a6d2ff1617db159af2 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 16 Nov 2023 13:31:04 +0000 Subject: [PATCH 266/331] Add docs --- docs/en/interfaces/formats.md | 1 + docs/en/interfaces/schema-inference.md | 21 +++++++++++++++++++ .../operations/settings/settings-formats.md | 7 +++++++ 3 files changed, 29 insertions(+) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 57de0555bf6..836b1f2f637 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -478,6 +478,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. - [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - allow variable number of columns in CSV format, ignore extra columns and use default values on missing columns. Default value - `false`. - [input_format_csv_use_default_on_bad_values](/docs/en/operations/settings/settings-formats.md/#input_format_csv_use_default_on_bad_values) - Allow to set default value to column when CSV field deserialization failed on bad value. Default value - `false`. +- [input_format_csv_try_infer_numbers_from_strings](/docs/en/operations/settings/settings-formats.md/#input_format_csv_try_infer_numbers_from_strings) - Try to infer numbers from string fields while schema inference. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/interfaces/schema-inference.md b/docs/en/interfaces/schema-inference.md index 0aadb09730a..34b98181355 100644 --- a/docs/en/interfaces/schema-inference.md +++ b/docs/en/interfaces/schema-inference.md @@ -834,6 +834,27 @@ $$) └──────────────┴───────────────┘ ``` +#### CSV settings {#csv-settings} + +##### input_format_csv_try_infer_numbers_from_strings + +Enabling this setting allows inferring numbers from string values. + +This setting is disabled by default. + +**Example:** + +```sql +SET input_format_json_try_infer_numbers_from_strings = 1; +DESC format(CSV, '"42","42.42"'); +``` +```reponse +┌─name─┬─type──────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐ +│ c1 │ Nullable(Int64) │ │ │ │ │ │ +│ c2 │ Nullable(Float64) │ │ │ │ │ │ +└──────┴───────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ +``` + ### TSV/TSKV {#tsv-tskv} In TSV/TSKV formats ClickHouse extracts column value from the row according to tabular delimiters and then parses extracted value using diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 344e6dda680..3d76bd9df73 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -1130,6 +1130,13 @@ Result a 0 1971-01-01 ``` +## input_format_csv_try_infer_numbers_from_strings {#input_format_csv_try_infer_numbers_from_strings} + +If enabled, during schema inference ClickHouse will try to infer numbers from string fields. +It can be useful if CSV data contains quoted UInt64 numbers. + +Disabled by default. + ## Values format settings {#values-format-settings} ### input_format_values_interpret_expressions {#input_format_values_interpret_expressions} From eaf2c347616913c2e12a87c445381ff1bdb25b00 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 11 Dec 2023 18:35:38 +0000 Subject: [PATCH 267/331] better lowcard handling in hash join --- src/Interpreters/HashJoin.cpp | 67 +++++++++++++++++++++++------------ 1 file changed, 45 insertions(+), 22 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index ce4236317ac..078797910af 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -271,6 +271,7 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s materializeBlockInplace(right_table_keys); initRightBlockStructure(data->sample_block); + data->sample_block = prepareRightBlock(data->sample_block); JoinCommon::createMissedColumns(sample_block_with_columns_to_add); @@ -806,7 +807,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) for (const auto & column_name : right_key_names) { const auto & column = source_block.getByName(column_name).column; - all_key_columns[column_name] = recursiveRemoveLowCardinality(recursiveRemoveSparse(column->convertToFullColumnIfConst())); + all_key_columns[column_name] = recursiveRemoveSparse(column->convertToFullColumnIfConst())->convertToFullColumnIfLowCardinality(); } Block block_to_save = prepareRightBlock(source_block); @@ -821,7 +822,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) data->blocks_allocated_size += block_to_save.allocatedBytes(); - assertBlocksHaveEqualStructure(data->sample_block, block_to_save, "Saved joined block structure mismatch"); + assertBlocksHaveEqualStructure(data->sample_block, block_to_save, "joined block"); data->blocks.emplace_back(std::move(block_to_save)); Block * stored_block = &data->blocks.back(); @@ -1030,16 +1031,15 @@ public: }; AddedColumns( + const Block & left_block, const Block & block_with_columns_to_add, - const Block & block, const Block & saved_block_sample, const HashJoin & join, std::vector && join_on_keys_, bool is_asof_join, bool is_join_get_) : join_on_keys(join_on_keys_) - , rows_to_add(block.rows()) - , sample_block(saved_block_sample) + , rows_to_add(left_block.rows()) , is_join_get(is_join_get_) { size_t num_columns_to_add = block_with_columns_to_add.columns(); @@ -1056,7 +1056,7 @@ public: /// because it uses not qualified right block column names auto qualified_name = join.getTableJoin().renamedRightColumnName(src_column.name); /// Don't insert column if it's in left block - if (!block.has(qualified_name)) + if (!left_block.has(qualified_name)) addColumn(src_column, qualified_name); } @@ -1070,6 +1070,17 @@ public: for (auto & tn : type_name) right_indexes.push_back(saved_block_sample.getPositionByName(tn.name)); + + nullable_column_ptrs.resize(right_indexes.size(), nullptr); + for (size_t j = 0; j < right_indexes.size(); ++j) + { + /** If it's joinGetOrNull, we will have nullable columns in result block + * even if right column is not nullable in storage (saved_block_sample). + */ + const auto & saved_column = saved_block_sample.getByPosition(right_indexes[j]).column; + if (columns[j]->isNullable() && !saved_column->isNullable()) + nullable_column_ptrs[j] = typeid_cast(columns[j].get()); + } } size_t size() const { return columns.size(); } @@ -1086,32 +1097,43 @@ public: if constexpr (has_defaults) applyLazyDefaults(); +#ifndef NDEBUG + for (size_t j = 0; j < right_indexes.size(); ++j) + { + const auto & column_from_block = block.getByPosition(right_indexes[j]); + const auto * dest_column = columns[j].get(); + if (auto * nullable_col = nullable_column_ptrs[j]) + { + if (!is_join_get) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Columns {} and {} can have different nullability only in joinGetOrNull", + dest_column->getName(), column_from_block.column->getName()); + dest_column = nullable_col->getNestedColumnPtr().get(); + } + if (!dest_column->structureEquals(*column_from_block.column)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns {} and {} are not structure equals", dest_column->getName(), column_from_block.column->getName()); + } +#endif + if (is_join_get) { - /// If it's joinGetOrNull, we need to wrap not-nullable columns in StorageJoin. - for (size_t j = 0, size = right_indexes.size(); j < size; ++j) + size_t right_indexes_size = right_indexes.size(); + for (size_t j = 0; j < right_indexes_size; ++j) { const auto & column_from_block = block.getByPosition(right_indexes[j]); - if (auto * nullable_col = typeid_cast(columns[j].get()); - nullable_col && !column_from_block.column->isNullable()) + if (auto * nullable_col = nullable_column_ptrs[j]) nullable_col->insertFromNotNullable(*column_from_block.column, row_num); - else if (auto * lowcard_col = typeid_cast(columns[j].get()); - lowcard_col && !typeid_cast(column_from_block.column.get())) - lowcard_col->insertFromFullColumn(*column_from_block.column, row_num); else columns[j]->insertFrom(*column_from_block.column, row_num); } } else { - for (size_t j = 0, size = right_indexes.size(); j < size; ++j) + size_t right_indexes_size = right_indexes.size(); + for (size_t j = 0; j < right_indexes_size; ++j) { const auto & column_from_block = block.getByPosition(right_indexes[j]); - if (auto * lowcard_col = typeid_cast(columns[j].get()); - lowcard_col && !typeid_cast(column_from_block.column.get())) - lowcard_col->insertFromFullColumn(*column_from_block.column, row_num); - else - columns[j]->insertFrom(*column_from_block.column, row_num); + columns[j]->insertFrom(*column_from_block.column, row_num); } } } @@ -1142,11 +1164,12 @@ public: private: std::vector type_name; MutableColumns columns; + std::vector nullable_column_ptrs; + std::vector right_indexes; size_t lazy_defaults_count = 0; /// for ASOF const IColumn * left_asof_key = nullptr; - Block sample_block; bool is_join_get; @@ -1601,8 +1624,8 @@ void HashJoin::joinBlockImpl( * For ASOF, the last column is used as the ASOF column */ AddedColumns added_columns( - block_with_columns_to_add, block, + block_with_columns_to_add, savedBlockSample(), *this, std::move(join_on_keys), @@ -1811,7 +1834,7 @@ ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block std::vector maps_vector; maps_vector.push_back(&std::get(data->maps[0])); joinBlockImpl( - keys, block_with_columns_to_add, maps_vector, true); + keys, block_with_columns_to_add, maps_vector, /* is_join_get = */ true); return keys.getByPosition(keys.columns() - 1); } From 76be22e8e87630eddd9a97363fdcb8291acdedd5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Dec 2023 19:55:14 +0100 Subject: [PATCH 268/331] Fix build without libpqxx --- src/Databases/DDLLoadingDependencyVisitor.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Databases/DDLLoadingDependencyVisitor.cpp b/src/Databases/DDLLoadingDependencyVisitor.cpp index 8f1b82941f6..b8690125aaa 100644 --- a/src/Databases/DDLLoadingDependencyVisitor.cpp +++ b/src/Databases/DDLLoadingDependencyVisitor.cpp @@ -1,7 +1,10 @@ #include #include #include +#include "config.h" +#if USE_LIBPQXX #include +#endif #include #include #include @@ -132,12 +135,14 @@ void DDLLoadingDependencyVisitor::visit(const ASTStorage & storage, Data & data) extractTableNameFromArgument(*storage.engine, data, 3); else if (storage.engine->name == "Dictionary") extractTableNameFromArgument(*storage.engine, data, 0); +#if USE_LIBPQXX else if (storage.engine->name == "MaterializedPostgreSQL") { const auto * create_query = data.create_query->as(); auto nested_table = toString(create_query->uuid) + StorageMaterializedPostgreSQL::NESTED_TABLE_SUFFIX; data.dependencies.emplace(QualifiedTableName{ .database = create_query->getDatabase(), .table = nested_table }); } +#endif } From 1e0d8e098fcbf2a70194ce4b8a87465566140de0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 11 Dec 2023 20:48:48 +0100 Subject: [PATCH 269/331] Fix building Rust with Musl Signed-off-by: Azat Khuzhin --- cmake/target.cmake | 5 ----- rust/CMakeLists.txt | 4 ++++ 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/cmake/target.cmake b/cmake/target.cmake index 1680715d15f..0d6993142b3 100644 --- a/cmake/target.cmake +++ b/cmake/target.cmake @@ -73,8 +73,3 @@ if (CMAKE_CROSSCOMPILING) 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/rust/CMakeLists.txt b/rust/CMakeLists.txt index 6aa25e95679..5ea806baa3b 100644 --- a/rust/CMakeLists.txt +++ b/rust/CMakeLists.txt @@ -14,6 +14,10 @@ macro(configure_rustc) set(RUST_CFLAGS "${RUST_CFLAGS} --sysroot ${CMAKE_SYSROOT}") endif() + if (USE_MUSL) + set(RUST_CXXFLAGS "${RUST_CXXFLAGS} -D_LIBCPP_HAS_MUSL_LIBC=1") + endif () + if(CCACHE_EXECUTABLE MATCHES "/sccache$") message(STATUS "Using RUSTC_WRAPPER: ${CCACHE_EXECUTABLE}") set(RUSTCWRAPPER "rustc-wrapper = \"${CCACHE_EXECUTABLE}\"") From c007025ee3c06779b3a69903f6b10de21bd20bf1 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 11 Dec 2023 19:30:33 +0000 Subject: [PATCH 270/331] Fix flaky test_parallel_replicas_distributed_read_from_all --- .../test_parallel_replicas_distributed_read_from_all/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py b/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py index fa1dfbefe52..88dabedb3f5 100644 --- a/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py +++ b/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py @@ -143,6 +143,8 @@ def test_read_equally_from_each_replica(start_cluster, prefer_localhost_replica) nodes[0].query(f"system start fetches {table_name}") nodes[1].query(f"system start fetches {table_name}") nodes[2].query(f"system start fetches {table_name}") + # ensure that replica in sync before querying it to get stable result + nodes[0].query(f"system sync replica {table_name} strict") assert ( nodes[0].query( f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d" From bfc2a7bb752cdb231ebfd53b0fc7d71928ba99ae Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 11 Dec 2023 21:27:48 +0100 Subject: [PATCH 271/331] Update cache only after conflicts --- src/Storages/MergeTree/AsyncBlockIDsCache.cpp | 32 +++++++------------ src/Storages/MergeTree/AsyncBlockIDsCache.h | 4 +-- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 5 ++- 4 files changed, 18 insertions(+), 25 deletions(-) diff --git a/src/Storages/MergeTree/AsyncBlockIDsCache.cpp b/src/Storages/MergeTree/AsyncBlockIDsCache.cpp index 4f3a8f16366..c2e7b4cbad5 100644 --- a/src/Storages/MergeTree/AsyncBlockIDsCache.cpp +++ b/src/Storages/MergeTree/AsyncBlockIDsCache.cpp @@ -18,6 +18,7 @@ namespace CurrentMetrics namespace DB { +static constexpr int FAILURE_RETRY_MS = 3000; template struct AsyncBlockIDsCache::Cache : public std::unordered_set @@ -33,24 +34,7 @@ template std::vector AsyncBlockIDsCache::getChildren() { auto zookeeper = storage.getZooKeeper(); - - auto watch_callback = [last_time = this->last_updatetime.load() - , my_update_min_interval = this->update_min_interval - , my_task = task->shared_from_this()](const Coordination::WatchResponse &) - { - auto now = std::chrono::steady_clock::now(); - if (now - last_time < my_update_min_interval) - { - std::chrono::milliseconds sleep_time = std::chrono::duration_cast(my_update_min_interval - (now - last_time)); - my_task->scheduleAfter(sleep_time.count()); - } - else - my_task->schedule(); - }; - std::vector children; - Coordination::Stat stat; - zookeeper->tryGetChildrenWatch(path, children, &stat, watch_callback); - return children; + return zookeeper->getChildren(path); } template @@ -69,18 +53,16 @@ try ++version; } cv.notify_all(); - last_updatetime = std::chrono::steady_clock::now(); } catch (...) { LOG_INFO(log, "Updating async block ids cache failed. Reason: {}", getCurrentExceptionMessage(false)); - task->scheduleAfter(update_min_interval.count()); + task->scheduleAfter(FAILURE_RETRY_MS); } template AsyncBlockIDsCache::AsyncBlockIDsCache(TStorage & storage_) : storage(storage_) - , update_min_interval(storage.getSettings()->async_block_ids_cache_min_update_interval_ms) , update_wait(storage.getSettings()->async_block_ids_cache_update_wait_ms) , path(storage.getZooKeeperPath() + "/async_blocks") , log_name(storage.getStorageID().getFullTableName() + " (AsyncBlockIDsCache)") @@ -96,6 +78,14 @@ void AsyncBlockIDsCache::start() task->activateAndSchedule(); } +template +void AsyncBlockIDsCache::triggerCacheUpdate() +{ + /// Trigger task update + if (!task->schedule()) + LOG_TRACE(log, "Task is already scheduled, will wait for update for {}ms", update_wait.count()); +} + /// Caller will keep the version of last call. When the caller calls again, it will wait util gets a newer version. template Strings AsyncBlockIDsCache::detectConflicts(const Strings & paths, UInt64 & last_version) diff --git a/src/Storages/MergeTree/AsyncBlockIDsCache.h b/src/Storages/MergeTree/AsyncBlockIDsCache.h index 8ce65ec4927..d505b0fe8be 100644 --- a/src/Storages/MergeTree/AsyncBlockIDsCache.h +++ b/src/Storages/MergeTree/AsyncBlockIDsCache.h @@ -27,12 +27,12 @@ public: Strings detectConflicts(const Strings & paths, UInt64 & last_version); + void triggerCacheUpdate(); + private: TStorage & storage; - std::atomic last_updatetime; - const std::chrono::milliseconds update_min_interval; const std::chrono::milliseconds update_wait; std::mutex mu; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index c71951503c8..b921458feab 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -95,7 +95,6 @@ struct Settings; M(UInt64, replicated_deduplication_window_seconds, 7 * 24 * 60 * 60 /* one week */, "Similar to \"replicated_deduplication_window\", but determines old blocks by their lifetime. Hash of an inserted block will be deleted (and the block will not be deduplicated after) if it outside of one \"window\". You can set very big replicated_deduplication_window to avoid duplicating INSERTs during that period of time.", 0) \ M(UInt64, replicated_deduplication_window_for_async_inserts, 10000, "How many last hash values of async_insert blocks should be kept in ZooKeeper (old blocks will be deleted).", 0) \ M(UInt64, replicated_deduplication_window_seconds_for_async_inserts, 7 * 24 * 60 * 60 /* one week */, "Similar to \"replicated_deduplication_window_for_async_inserts\", but determines old blocks by their lifetime. Hash of an inserted block will be deleted (and the block will not be deduplicated after) if it outside of one \"window\". You can set very big replicated_deduplication_window to avoid duplicating INSERTs during that period of time.", 0) \ - M(Milliseconds, async_block_ids_cache_min_update_interval_ms, 1000, "Minimum interval between updates of async_block_ids_cache", 0) \ M(Milliseconds, async_block_ids_cache_update_wait_ms, 100, "How long each insert iteration will wait for async_block_ids_cache update", 0) \ M(Bool, use_async_block_ids_cache, true, "Use in-memory cache to filter duplicated async inserts based on block ids", 0) \ M(UInt64, max_replicated_logs_to_keep, 1000, "How many records may be in log, if there is inactive replica. Inactive replica becomes lost when when this number exceed.", 0) \ @@ -215,6 +214,7 @@ struct Settings; MAKE_OBSOLETE_MERGE_TREE_SETTING(M, Bool, use_metadata_cache, false) \ MAKE_OBSOLETE_MERGE_TREE_SETTING(M, UInt64, merge_tree_enable_clear_old_broken_detached, 0) \ MAKE_OBSOLETE_MERGE_TREE_SETTING(M, UInt64, merge_tree_clear_old_broken_detached_parts_ttl_timeout_seconds, 1ULL * 3600 * 24 * 30) \ + MAKE_OBSOLETE_MERGE_TREE_SETTING(M, UInt64, async_block_ids_cache_min_update_interval_ms, 1000) \ /// Settings that should not change after the creation of a table. /// NOLINTNEXTLINE diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 4fa473da813..9fb575c0213 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -456,7 +456,7 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithFa if (!delayed_chunk) return; - for (auto & partition: delayed_chunk->partitions) + for (auto & partition : delayed_chunk->partitions) { int retry_times = 0; /// users may have lots of same inserts. It will be helpful to deduplicate in advance. @@ -469,6 +469,7 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithFa } /// reset the cache version to zero for every partition write. + /// Version zero allows to avoid wait on first iteration cache_version = 0; while (true) { @@ -476,6 +477,8 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithFa auto conflict_block_ids = commitPart(zookeeper, partition.temp_part.part, partition.block_id, delayed_chunk->replicas_num, false).first; if (conflict_block_ids.empty()) break; + + storage.async_block_ids_cache.triggerCacheUpdate(); ++retry_times; LOG_DEBUG(log, "Found duplicate block IDs: {}, retry times {}", toString(conflict_block_ids), retry_times); /// partition clean conflict From 7cdc3d29fbc6693cf98eac943723cb2bbb2f4fed Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 11 Dec 2023 22:29:51 +0100 Subject: [PATCH 272/331] Get rid of tiny function --- src/Storages/MergeTree/AsyncBlockIDsCache.cpp | 14 +++++--------- src/Storages/MergeTree/AsyncBlockIDsCache.h | 2 -- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- 3 files changed, 6 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/AsyncBlockIDsCache.cpp b/src/Storages/MergeTree/AsyncBlockIDsCache.cpp index c2e7b4cbad5..cc3bc8fc2a8 100644 --- a/src/Storages/MergeTree/AsyncBlockIDsCache.cpp +++ b/src/Storages/MergeTree/AsyncBlockIDsCache.cpp @@ -30,18 +30,12 @@ struct AsyncBlockIDsCache::Cache : public std::unordered_set {} }; -template -std::vector AsyncBlockIDsCache::getChildren() -{ - auto zookeeper = storage.getZooKeeper(); - return zookeeper->getChildren(path); -} - template void AsyncBlockIDsCache::update() try { - std::vector paths = getChildren(); + auto zookeeper = storage.getZooKeeper(); + std::vector paths = zookeeper->getChildren(path); std::unordered_set set; for (String & p : paths) { @@ -81,7 +75,9 @@ void AsyncBlockIDsCache::start() template void AsyncBlockIDsCache::triggerCacheUpdate() { - /// Trigger task update + /// Trigger task update. Watch-based updates may produce a lot of + /// redundant work in case of multiple replicas, so we use manually controlled updates + /// in case of duplicates if (!task->schedule()) LOG_TRACE(log, "Task is already scheduled, will wait for update for {}ms", update_wait.count()); } diff --git a/src/Storages/MergeTree/AsyncBlockIDsCache.h b/src/Storages/MergeTree/AsyncBlockIDsCache.h index d505b0fe8be..38c38da0033 100644 --- a/src/Storages/MergeTree/AsyncBlockIDsCache.h +++ b/src/Storages/MergeTree/AsyncBlockIDsCache.h @@ -14,8 +14,6 @@ class AsyncBlockIDsCache struct Cache; using CachePtr = std::shared_ptr; - std::vector getChildren(); - void update(); public: diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 9fb575c0213..90cbe6b797d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -307,7 +307,7 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) auto profile_events_scope = std::make_unique(&part_counters); /// Some merging algorithms can mofidy the block which loses the information about the async insert offsets - /// when preprocessing or filtering data for asnyc inserts deduplication we want to use the initial, unmerged block + /// when preprocessing or filtering data for async inserts deduplication we want to use the initial, unmerged block std::optional unmerged_block; if constexpr (async_insert) From da43d49ffc7b637baca40fd9fbdac606a4033ac4 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 11 Dec 2023 21:30:25 +0000 Subject: [PATCH 273/331] Fix: sync is failed with stopped merges --- .../test.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py b/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py index 88dabedb3f5..58abe1247e6 100644 --- a/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py +++ b/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py @@ -144,10 +144,14 @@ def test_read_equally_from_each_replica(start_cluster, prefer_localhost_replica) nodes[1].query(f"system start fetches {table_name}") nodes[2].query(f"system start fetches {table_name}") # ensure that replica in sync before querying it to get stable result - nodes[0].query(f"system sync replica {table_name} strict") + nodes[0].query(f"system start merges {table_name}") + nodes[0].query(f"system sync replica {table_name}") assert ( nodes[0].query( - f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d" + f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d", + settings={ + "allow_experimental_parallel_reading_from_replicas": 0, + } ) == expected_result ) From e95041a3afcdbc8473b44f537eca254e2a413e7d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 11 Dec 2023 21:40:23 +0000 Subject: [PATCH 274/331] Automatic style fix --- .../test_parallel_replicas_distributed_read_from_all/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py b/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py index 58abe1247e6..7e12da956ea 100644 --- a/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py +++ b/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py @@ -151,7 +151,7 @@ def test_read_equally_from_each_replica(start_cluster, prefer_localhost_replica) f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d", settings={ "allow_experimental_parallel_reading_from_replicas": 0, - } + }, ) == expected_result ) From fcb8ab9b6375907cf0b0e5ee498f11972925d503 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Dec 2023 00:42:23 +0300 Subject: [PATCH 275/331] Update tests/ci/functional_test_check.py Co-authored-by: Mikhail f. Shiryaev --- tests/ci/functional_test_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index c8b3e42eed4..0dea2c5476f 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -169,7 +169,7 @@ def process_results( status = [] status_path = result_directory / "check_status.tsv" if status_path.exists(): - logging.info("Found check_status.tsv") + logging.info("Found %s", status_path.name) with open(status_path, "r", encoding="utf-8") as status_file: status = list(csv.reader(status_file, delimiter="\t")) From b9b6e7584e547a358098a6a174f302cb2d7b7774 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Dec 2023 00:42:41 +0300 Subject: [PATCH 276/331] Update tests/ci/fast_test_check.py Co-authored-by: Mikhail f. Shiryaev --- tests/ci/fast_test_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index f5c7342d6f4..265fc81ccb3 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -72,7 +72,7 @@ def process_results(result_directory: Path) -> Tuple[str, str, TestResults]: status = [] status_path = result_directory / "check_status.tsv" if status_path.exists(): - logging.info("Found check_status.tsv") + logging.info("Found %s", status_path.name) with open(status_path, "r", encoding="utf-8") as status_file: status = list(csv.reader(status_file, delimiter="\t")) if len(status) != 1 or len(status[0]) != 2: From a3262003f3bba9921e160d1abf8a05027b92a69d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Dec 2023 00:42:47 +0300 Subject: [PATCH 277/331] Update tests/ci/integration_test_check.py Co-authored-by: Mikhail f. Shiryaev --- tests/ci/integration_test_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index b22aa08354c..e49cec6d694 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -118,7 +118,7 @@ def process_results( status = [] status_path = result_directory / "check_status.tsv" if status_path.exists(): - logging.info("Found check_status.tsv") + logging.info("Found %s", status_path.name) with open(status_path, "r", encoding="utf-8") as status_file: status = list(csv.reader(status_file, delimiter="\t")) From 0c81892463428e06b30b1d7a3a9729f86618a3a3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Dec 2023 00:43:06 +0300 Subject: [PATCH 278/331] Update tests/ci/clickbench.py Co-authored-by: Mikhail f. Shiryaev --- tests/ci/clickbench.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/clickbench.py b/tests/ci/clickbench.py index 9c700adb398..359c10eeb9d 100644 --- a/tests/ci/clickbench.py +++ b/tests/ci/clickbench.py @@ -94,7 +94,7 @@ def process_results( results_path = result_directory / "test_results.tsv" if results_path.exists(): - logging.info("Found test_results.tsv") + logging.info("Found %s", results_path.name) else: logging.info("Files in result folder %s", os.listdir(result_directory)) return "error", "Not found test_results.tsv", test_results, additional_files From bb4d9f7ae5fc7cef1442ac2d27657fcbad92f916 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 11 Dec 2023 15:27:41 +0000 Subject: [PATCH 279/331] Refactor toStartOfInterval() --- src/Functions/toStartOfInterval.cpp | 200 ++++++++++++++-------------- 1 file changed, 100 insertions(+), 100 deletions(-) diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 48f60dddb33..ea0ad139481 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -1,7 +1,7 @@ -#include -#include #include #include +#include +#include #include #include #include @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB @@ -24,9 +25,6 @@ namespace ErrorCodes } -namespace -{ - class FunctionToStartOfInterval : public IFunction { public: @@ -34,86 +32,90 @@ public: static constexpr auto name = "toStartOfInterval"; String getName() const override { return name; } - bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } - bool hasInformationAboutMonotonicity() const override { return true; } - Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override - { - return { .is_monotonic = true, .is_always_monotonic = true }; - } + Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override { return { .is_monotonic = true, .is_always_monotonic = true }; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - bool first_argument_is_date = false; + bool value_is_date = false; auto check_first_argument = [&] { - if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " - "Should be a date or a date with time", arguments[0].type->getName(), getName()); - first_argument_is_date = isDate(arguments[0].type); + const DataTypePtr & type_arg1 = arguments[0].type; + if (!isDate(type_arg1) && !isDateTime(type_arg1) && !isDateTime64(type_arg1)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of 1st argument of function {}, expected a Date, DateTime or DateTime64", + type_arg1->getName(), getName()); + value_is_date = isDate(type_arg1); }; const DataTypeInterval * interval_type = nullptr; - bool result_type_is_date = false; - bool result_type_is_datetime = false; - bool result_type_is_datetime_64 = false; - auto check_interval_argument = [&] + enum class ResultType { - interval_type = checkAndGetDataType(arguments[1].type.get()); + Date, + DateTime, + DateTime64 + }; + ResultType result_type; + auto check_second_argument = [&] + { + const DataTypePtr & type_arg2 = arguments[1].type; + + interval_type = checkAndGetDataType(type_arg2.get()); if (!interval_type) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " - "Should be an interval of time", arguments[1].type->getName(), getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of 2nd argument of function {}, expected a time interval", + type_arg2->getName(), getName()); + switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case) { case IntervalKind::Nanosecond: case IntervalKind::Microsecond: case IntervalKind::Millisecond: - result_type_is_datetime_64 = true; + result_type = ResultType::DateTime64; break; case IntervalKind::Second: case IntervalKind::Minute: case IntervalKind::Hour: - case IntervalKind::Day: - result_type_is_datetime = true; + case IntervalKind::Day: /// weird why Day leads to DateTime but too afraid to change it + result_type = ResultType::DateTime; break; case IntervalKind::Week: case IntervalKind::Month: case IntervalKind::Quarter: case IntervalKind::Year: - result_type_is_date = true; + result_type = ResultType::Date; break; } }; - auto check_timezone_argument = [&] + auto check_third_argument = [&] { - if (!isString(arguments[2].type)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " - "This argument is optional and must be a constant string with timezone name", - arguments[2].type->getName(), getName()); - if (first_argument_is_date && result_type_is_date) + const DataTypePtr & type_arg3 = arguments[2].type; + if (!isString(type_arg3)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "The timezone argument of function {} with interval type {} is allowed only when the 1st argument " - "has the type DateTime or DateTime64", - getName(), interval_type->getKind().toString()); + "Illegal type {} of 3rd argument of function {}, expected a constant timezone string", + type_arg3->getName(), getName()); + if (value_is_date && result_type == ResultType::Date) /// weird why this is && instead of || but too afraid to change it + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "The timezone argument of function {} with interval type {} is allowed only when the 1st argument has type DateTime or DateTimt64", + getName(), interval_type->getKind().toString()); }; if (arguments.size() == 2) { check_first_argument(); - check_interval_argument(); + check_second_argument(); } else if (arguments.size() == 3) { check_first_argument(); - check_interval_argument(); - check_timezone_argument(); + check_second_argument(); + check_third_argument(); } else { @@ -122,25 +124,27 @@ public: getName(), arguments.size()); } - if (result_type_is_date) - return std::make_shared(); - else if (result_type_is_datetime) - return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); - else if (result_type_is_datetime_64) + switch (result_type) { - auto scale = 0; + case ResultType::Date: + return std::make_shared(); + case ResultType::DateTime: + return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); + case ResultType::DateTime64: + { + UInt32 scale = 0; + if (interval_type->getKind() == IntervalKind::Nanosecond) + scale = 9; + else if (interval_type->getKind() == IntervalKind::Microsecond) + scale = 6; + else if (interval_type->getKind() == IntervalKind::Millisecond) + scale = 3; - if (interval_type->getKind() == IntervalKind::Nanosecond) - scale = 9; - else if (interval_type->getKind() == IntervalKind::Microsecond) - scale = 6; - else if (interval_type->getKind() == IntervalKind::Millisecond) - scale = 3; - - return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); + return std::make_shared(scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); + } } - UNREACHABLE(); + std::unreachable(); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /* input_rows_count */) const override @@ -154,110 +158,106 @@ public: private: ColumnPtr dispatchForTimeColumn( - const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone) const + const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, + const DataTypePtr & result_type, const DateLUTImpl & time_zone) const { - const auto & from_datatype = *time_column.type.get(); + const auto & time_column_type = *time_column.type.get(); + const auto & time_column_col = *time_column.column.get(); - if (isDateTime64(from_datatype)) + if (isDateTime64(time_column_type)) { - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); - auto scale = assert_cast(from_datatype).getScale(); + const auto * time_column_vec = checkAndGetColumn(time_column_col); + auto scale = assert_cast(time_column_type).getScale(); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(from_datatype), *time_column_vec, interval_column, result_type, time_zone, scale); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone, scale); } - if (isDateTime(from_datatype)) + else if (isDateTime(time_column_type)) { - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const auto * time_column_vec = checkAndGetColumn(time_column_col); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(from_datatype), *time_column_vec, interval_column, result_type, time_zone); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone); } - if (isDate(from_datatype)) + else if (isDate(time_column_type)) { - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const auto * time_column_vec = checkAndGetColumn(time_column_col); if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(from_datatype), *time_column_vec, interval_column, result_type, time_zone); + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone); } - if (isDate32(from_datatype)) - { - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); - if (time_column_vec) - return dispatchForIntervalColumn(assert_cast(from_datatype), *time_column_vec, interval_column, result_type, time_zone); - } - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for first argument of function {}. Must contain dates or dates with time", getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for 1st argument of function {}, expected a Date, DateTime or DateTime64", getName()); } - template + template ColumnPtr dispatchForIntervalColumn( const TimeDataType & time_data_type, const TimeColumnType & time_column, const ColumnWithTypeAndName & interval_column, - const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale = 1) const + const DataTypePtr & result_type, const DateLUTImpl & time_zone, UInt16 scale = 1) const { const auto * interval_type = checkAndGetDataType(interval_column.type.get()); if (!interval_type) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column for second argument of function {}, must be an interval of time.", getName()); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column for 2nd argument of function {}, must be a time interval", getName()); const auto * interval_column_const_int64 = checkAndGetColumnConst(interval_column.column.get()); if (!interval_column_const_int64) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column for second argument of function {}, must be a const interval of time.", getName()); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column for 2nd argument of function {}, must be a const time interval", getName()); - Int64 num_units = interval_column_const_int64->getValue(); + const Int64 num_units = interval_column_const_int64->getValue(); if (num_units <= 0) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Value for second argument of function {} must be positive.", getName()); + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Value for 2nd argument of function {} must be positive", getName()); switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case) { case IntervalKind::Nanosecond: - return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); case IntervalKind::Microsecond: - return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); case IntervalKind::Millisecond: - return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); case IntervalKind::Second: - return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); case IntervalKind::Minute: - return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); case IntervalKind::Hour: - return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); case IntervalKind::Day: - return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); case IntervalKind::Week: - return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); case IntervalKind::Month: - return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); case IntervalKind::Quarter: - return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); case IntervalKind::Year: - return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); + return execute(time_data_type, time_column, num_units, result_type, time_zone, scale); } - UNREACHABLE(); + std::unreachable(); } - template - ColumnPtr execute(const TimeDataType &, const ColumnType & time_column_type, Int64 num_units, const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale) const + template + ColumnPtr execute( + const TimeDataType &, const TimeColumnType & time_column_type, Int64 num_units, + const DataTypePtr & result_type, const DateLUTImpl & time_zone, UInt16 scale) const { - using ToColumnType = typename ToDataType::ColumnType; - using ToFieldType = typename ToDataType::FieldType; + using ResultColumnType = typename ResultDataType::ColumnType; + using ResultFieldType = typename ResultDataType::FieldType; const auto & time_data = time_column_type.getData(); size_t size = time_data.size(); auto result_col = result_type->createColumn(); - auto *col_to = assert_cast(result_col.get()); + auto * col_to = assert_cast(result_col.get()); auto & result_data = col_to->getData(); result_data.resize(size); Int64 scale_multiplier = DecimalUtils::scaleMultiplier(scale); for (size_t i = 0; i != size; ++i) - result_data[i] = static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_multiplier)); + result_data[i] = static_cast(ToStartOfInterval::execute(time_data[i], num_units, time_zone, scale_multiplier)); return result_col; } }; -} - REGISTER_FUNCTION(ToStartOfInterval) { factory.registerFunction(); From 6b0936553c9d8c34ce32fef5b4f64669ae25946e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 11 Dec 2023 22:58:39 +0100 Subject: [PATCH 280/331] Don't run test 02919_skip_lots_of_parsing_errors on aarch64 --- tests/queries/0_stateless/02919_skip_lots_of_parsing_errors.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02919_skip_lots_of_parsing_errors.sh b/tests/queries/0_stateless/02919_skip_lots_of_parsing_errors.sh index 2c54e9e68da..7ddb55fb39b 100755 --- a/tests/queries/0_stateless/02919_skip_lots_of_parsing_errors.sh +++ b/tests/queries/0_stateless/02919_skip_lots_of_parsing_errors.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-cpu-aarch64 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 16afd81322a15e4cdde07ea0007d45bbdbccd1b9 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 12 Dec 2023 00:19:16 +0100 Subject: [PATCH 281/331] Fix retries for disconnected nodes for BACKUP/RESTORE ON CLUSTER. --- src/Backups/BackupCoordinationRemote.cpp | 6 +- src/Backups/BackupCoordinationStageSync.cpp | 107 ++++++++------------ src/Backups/BackupCoordinationStageSync.h | 2 +- 3 files changed, 43 insertions(+), 72 deletions(-) diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 2633e1bedd2..7319b1aba58 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -184,11 +184,9 @@ BackupCoordinationRemote::BackupCoordinationRemote( if (my_is_internal) { String alive_node_path = my_zookeeper_path + "/stage/alive|" + my_current_host; + zk->createAncestors(alive_node_path); auto code = zk->tryCreate(alive_node_path, "", zkutil::CreateMode::Ephemeral); - - if (code == Coordination::Error::ZNODEEXISTS) - zk->handleEphemeralNodeExistenceNoFailureInjection(alive_node_path, ""); - else if (code != Coordination::Error::ZOK) + if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) throw zkutil::KeeperException::fromPath(code, alive_node_path); } }) diff --git a/src/Backups/BackupCoordinationStageSync.cpp b/src/Backups/BackupCoordinationStageSync.cpp index 9b9ddc8515c..e4dac7dbbe9 100644 --- a/src/Backups/BackupCoordinationStageSync.cpp +++ b/src/Backups/BackupCoordinationStageSync.cpp @@ -60,12 +60,6 @@ void BackupCoordinationStageSync::set(const String & current_host, const String } else { - /// Make an ephemeral node so the initiator can track if the current host is still working. - String alive_node_path = zookeeper_path + "/alive|" + current_host; - auto code = zookeeper->tryCreate(alive_node_path, "", zkutil::CreateMode::Ephemeral); - if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNODEEXISTS) - throw zkutil::KeeperException::fromPath(code, alive_node_path); - zookeeper->createIfNotExists(zookeeper_path + "/started|" + current_host, ""); zookeeper->createIfNotExists(zookeeper_path + "/current|" + current_host + "|" + new_stage, message); } @@ -118,27 +112,24 @@ struct BackupCoordinationStageSync::State Strings results; std::map unready_hosts; std::optional> error; - std::optional host_terminated; + std::optional disconnected_host; }; BackupCoordinationStageSync::State BackupCoordinationStageSync::readCurrentState( - const Strings & zk_nodes, const Strings & all_hosts, const String & stage_to_wait) const + WithRetries::RetriesControlHolder & retries_control_holder, + const Strings & zk_nodes, + const Strings & all_hosts, + const String & stage_to_wait) const { + auto zookeeper = retries_control_holder.faulty_zookeeper; + auto & retries_ctl = retries_control_holder.retries_ctl; + std::unordered_set zk_nodes_set{zk_nodes.begin(), zk_nodes.end()}; State state; if (zk_nodes_set.contains("error")) { - String errors; - { - auto holder = with_retries.createRetriesControlHolder("readCurrentState"); - holder.retries_ctl.retryLoop( - [&, &zookeeper = holder.faulty_zookeeper]() - { - with_retries.renewZooKeeper(zookeeper); - errors = zookeeper->get(zookeeper_path + "/error"); - }); - } + String errors = zookeeper->get(zookeeper_path + "/error"); ReadBufferFromOwnString buf{errors}; String host; readStringBinary(host, buf); @@ -150,60 +141,40 @@ BackupCoordinationStageSync::State BackupCoordinationStageSync::readCurrentState { if (!zk_nodes_set.contains("current|" + host + "|" + stage_to_wait)) { - UnreadyHostState unready_host_state; const String started_node_name = "started|" + host; const String alive_node_name = "alive|" + host; const String alive_node_path = zookeeper_path + "/" + alive_node_name; + + UnreadyHostState unready_host_state; unready_host_state.started = zk_nodes_set.contains(started_node_name); - - /// Because we do retries everywhere we can't fully rely on ephemeral nodes anymore. - /// Though we recreate "alive" node when reconnecting it might be not enough and race condition is possible. - /// And everything we can do here - just retry. - /// In worst case when we won't manage to see the alive node for a long time we will just abort the backup. unready_host_state.alive = zk_nodes_set.contains(alive_node_name); - if (!unready_host_state.alive) - { - LOG_TRACE(log, "Seems like host ({}) is dead. Will retry the check to confirm", host); - auto holder = with_retries.createRetriesControlHolder("readCurrentState::checkAliveNode"); - holder.retries_ctl.retryLoop( - [&, &zookeeper = holder.faulty_zookeeper]() - { - with_retries.renewZooKeeper(zookeeper); - - if (zookeeper->existsNoFailureInjection(alive_node_path)) - { - unready_host_state.alive = true; - return; - } - - // Retry with backoff. We also check whether it is last retry or no, because we won't to rethrow an exception. - if (!holder.retries_ctl.isLastRetry()) - holder.retries_ctl.setKeeperError(Coordination::Error::ZNONODE, "There is no alive node for host {}. Will retry", host); - }); - } - LOG_TRACE(log, "Host ({}) appeared to be {}", host, unready_host_state.alive ? "alive" : "dead"); - state.unready_hosts.emplace(host, unready_host_state); - if (!unready_host_state.alive && unready_host_state.started && !state.host_terminated) - state.host_terminated = host; + + if (!unready_host_state.alive && !state.disconnected_host) + { + /// If the "alive" node doesn't exist then we don't have connection to the corresponding host. + /// This node is ephemeral so probably it will be recreated soon. We use zookeeper retries to wait. + /// In worst case when we won't manage to see the alive node for a long time we will just abort the backup. + state.disconnected_host = host; + String message; + if (unready_host_state.started) + message = fmt::format("Lost connection to host {}", host); + else + message = fmt::format("No connection to host {} yet", host); + if (!retries_ctl.isLastRetry()) + message += ", will retry"; + retries_ctl.setUserError(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, message); + } } } - if (state.host_terminated || !state.unready_hosts.empty()) + if (state.disconnected_host || !state.unready_hosts.empty()) return state; - auto holder = with_retries.createRetriesControlHolder("waitImpl::collectStagesToWait"); - holder.retries_ctl.retryLoop( - [&, &zookeeper = holder.faulty_zookeeper]() - { - with_retries.renewZooKeeper(zookeeper); - Strings results; - - for (const auto & host : all_hosts) - results.emplace_back(zookeeper->get(zookeeper_path + "/current|" + host + "|" + stage_to_wait)); - - state.results = std::move(results); - }); + Strings results; + for (const auto & host : all_hosts) + results.emplace_back(zookeeper->get(zookeeper_path + "/current|" + host + "|" + stage_to_wait)); + state.results = std::move(results); return state; } @@ -229,7 +200,7 @@ Strings BackupCoordinationStageSync::waitImpl( auto watch = std::make_shared(); Strings zk_nodes; { - auto holder = with_retries.createRetriesControlHolder("waitImpl::getChildren"); + auto holder = with_retries.createRetriesControlHolder("waitImpl"); holder.retries_ctl.retryLoop( [&, &zookeeper = holder.faulty_zookeeper]() { @@ -237,12 +208,14 @@ Strings BackupCoordinationStageSync::waitImpl( watch->reset(); /// Get zk nodes and subscribe on their changes. zk_nodes = zookeeper->getChildren(zookeeper_path, nullptr, watch); + + /// Read the current state of zk nodes. + state = readCurrentState(holder, zk_nodes, all_hosts, stage_to_wait); }); } - /// Read and analyze the current state of zk nodes. - state = readCurrentState(zk_nodes, all_hosts, stage_to_wait); - if (state.error || state.host_terminated || state.unready_hosts.empty()) + /// Analyze the current state of zk nodes. + if (state.error || state.disconnected_host || state.unready_hosts.empty()) break; /// Error happened or everything is ready. /// Log that we will wait @@ -270,8 +243,8 @@ Strings BackupCoordinationStageSync::waitImpl( state.error->second.rethrow(); /// Another host terminated without errors. - if (state.host_terminated) - throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, "Host {} suddenly stopped working", *state.host_terminated); + if (state.disconnected_host) + throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, "No connection to host {}", *state.disconnected_host); /// Something's unready, timeout is probably not enough. if (!state.unready_hosts.empty()) diff --git a/src/Backups/BackupCoordinationStageSync.h b/src/Backups/BackupCoordinationStageSync.h index 2efaec46b3a..e34fbcc099b 100644 --- a/src/Backups/BackupCoordinationStageSync.h +++ b/src/Backups/BackupCoordinationStageSync.h @@ -29,7 +29,7 @@ private: void createRootNodes(); struct State; - State readCurrentState(const Strings & zk_nodes, const Strings & all_hosts, const String & stage_to_wait) const; + State readCurrentState(WithRetries::RetriesControlHolder & retries_control_holder, const Strings & zk_nodes, const Strings & all_hosts, const String & stage_to_wait) const; Strings waitImpl(const Strings & all_hosts, const String & stage_to_wait, std::optional timeout) const; From cab764fff9f6522be9684a5dbbce49c44c18e4f4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Dec 2023 03:44:26 +0100 Subject: [PATCH 282/331] Add a test --- ...2_window_functions_logical_error.reference | 216 ++++++++++++++++++ .../02942_window_functions_logical_error.sql | 158 +++++++++++++ 2 files changed, 374 insertions(+) create mode 100644 tests/queries/0_stateless/02942_window_functions_logical_error.reference create mode 100644 tests/queries/0_stateless/02942_window_functions_logical_error.sql diff --git a/tests/queries/0_stateless/02942_window_functions_logical_error.reference b/tests/queries/0_stateless/02942_window_functions_logical_error.reference new file mode 100644 index 00000000000..73f8351d9df --- /dev/null +++ b/tests/queries/0_stateless/02942_window_functions_logical_error.reference @@ -0,0 +1,216 @@ +1 901 19 +1 911 19 +1 921 19 +1 931 19 +1 941 19 +1 951 20 +1 961 20 +1 971 20 +1 981 20 +1 991 20 +2 902 19 +2 912 19 +2 922 19 +2 932 19 +2 942 19 +2 952 20 +2 962 20 +2 972 20 +2 982 20 +2 992 20 +3 903 19 +3 913 19 +3 923 19 +3 933 19 +3 943 19 +3 953 20 +3 963 20 +3 973 20 +3 983 20 +3 993 20 +4 904 19 +4 914 19 +4 924 19 +4 934 19 +4 944 19 +4 954 20 +4 964 20 +4 974 20 +4 984 20 +4 994 20 +5 905 19 +5 915 19 +5 925 19 +5 935 19 +5 945 19 +5 955 20 +5 965 20 +5 975 20 +5 985 20 +5 995 20 +6 906 19 +6 916 19 +6 926 19 +6 936 19 +6 946 19 +6 956 20 +6 966 20 +6 976 20 +6 986 20 +6 996 20 +7 907 19 +7 917 19 +7 927 19 +7 937 19 +7 947 19 +7 957 20 +7 967 20 +7 977 20 +7 987 20 +7 997 20 +8 908 19 +8 918 19 +8 928 19 +8 938 19 +8 948 19 +8 958 20 +8 968 20 +8 978 20 +8 988 20 +8 998 20 +9 909 19 +9 919 19 +9 929 19 +9 939 19 +9 949 19 +9 959 20 +9 969 20 +9 979 20 +9 989 20 +9 999 20 +1 1301 19 +1 1311 19 +1 1321 19 +1 1331 19 +1 1341 19 +1 1351 19 +1 1361 19 +1 1371 20 +1 1381 20 +1 1391 20 +1 1401 20 +1 1411 20 +1 1421 20 +1 1431 20 +2 1302 19 +2 1312 19 +2 1322 19 +2 1332 19 +2 1342 19 +2 1352 19 +2 1362 19 +2 1372 20 +2 1382 20 +2 1392 20 +2 1402 20 +2 1412 20 +2 1422 20 +2 1432 20 +3 1303 19 +3 1313 19 +3 1323 19 +3 1333 19 +3 1343 19 +3 1353 19 +3 1363 19 +3 1373 20 +3 1383 20 +3 1393 20 +3 1403 20 +3 1413 20 +3 1423 20 +3 1433 20 +4 1304 19 +4 1314 19 +4 1324 19 +4 1334 19 +4 1344 19 +4 1354 19 +4 1364 19 +4 1374 20 +4 1384 20 +4 1394 20 +4 1404 20 +4 1414 20 +4 1424 20 +4 1434 20 +5 1305 19 +5 1315 19 +5 1325 19 +5 1335 19 +5 1345 19 +5 1355 19 +5 1365 19 +5 1375 20 +5 1385 20 +5 1395 20 +5 1405 20 +5 1415 20 +5 1425 20 +5 1435 20 +6 1306 19 +6 1316 19 +6 1326 19 +6 1336 19 +6 1346 19 +6 1356 19 +6 1366 19 +6 1376 20 +6 1386 20 +6 1396 20 +6 1406 20 +6 1416 20 +6 1426 20 +6 1436 20 +7 1307 19 +7 1317 19 +7 1327 19 +7 1337 19 +7 1347 19 +7 1357 19 +7 1367 19 +7 1377 20 +7 1387 20 +7 1397 20 +7 1407 20 +7 1417 20 +7 1427 20 +7 1437 20 +8 1308 19 +8 1318 19 +8 1328 19 +8 1338 19 +8 1348 19 +8 1358 19 +8 1368 19 +8 1378 20 +8 1388 20 +8 1398 20 +8 1408 20 +8 1418 20 +8 1428 20 +8 1438 20 +9 1309 19 +9 1319 19 +9 1329 19 +9 1339 19 +9 1349 19 +9 1359 19 +9 1369 19 +9 1379 20 +9 1389 20 +9 1399 20 +9 1409 20 +9 1419 20 +9 1429 20 +9 1439 20 diff --git a/tests/queries/0_stateless/02942_window_functions_logical_error.sql b/tests/queries/0_stateless/02942_window_functions_logical_error.sql new file mode 100644 index 00000000000..1e4371a134f --- /dev/null +++ b/tests/queries/0_stateless/02942_window_functions_logical_error.sql @@ -0,0 +1,158 @@ +DROP TABLE IF EXISTS posts; +DROP TABLE IF EXISTS post_metrics; + +CREATE TABLE IF NOT EXISTS posts +( + `page_id` LowCardinality(String), + `post_id` String CODEC(LZ4), + `host_id` UInt32 CODEC(T64, LZ4), + `path_id` UInt32, + `created` DateTime CODEC(T64, LZ4), + `as_of` DateTime CODEC(T64, LZ4) +) +ENGINE = ReplacingMergeTree(as_of) +PARTITION BY toStartOfMonth(created) +ORDER BY (page_id, post_id) +TTL created + toIntervalMonth(26); + + +INSERT INTO posts SELECT + repeat('a', (number % 10) + 1), + toString(number), + number % 10, + number, + now() - toIntervalMinute(number), + now() +FROM numbers(1000); + + +CREATE TABLE IF NOT EXISTS post_metrics +( + `page_id` LowCardinality(String), + `post_id` String CODEC(LZ4), + `created` DateTime CODEC(T64, LZ4), + `impressions` UInt32 CODEC(T64, LZ4), + `clicks` UInt32 CODEC(T64, LZ4), + `as_of` DateTime CODEC(T64, LZ4) +) +ENGINE = ReplacingMergeTree(as_of) +PARTITION BY toStartOfMonth(created) +ORDER BY (page_id, post_id) +TTL created + toIntervalMonth(26); + + +INSERT INTO post_metrics SELECT + repeat('a', (number % 10) + 1), + toString(number), + now() - toIntervalMinute(number), + number * 100, + number * 10, + now() +FROM numbers(1000); + + +SELECT + host_id, + path_id, + max(rank) AS rank +FROM +( + WITH + as_of_posts AS + ( + SELECT + *, + row_number() OVER (PARTITION BY (page_id, post_id) ORDER BY as_of DESC) AS row_num + FROM posts + WHERE (created >= subtractHours(now(), 24)) AND (host_id > 0) + ), + as_of_post_metrics AS + ( + SELECT + *, + row_number() OVER (PARTITION BY (page_id, post_id) ORDER BY as_of DESC) AS row_num + FROM post_metrics + WHERE created >= subtractHours(now(), 24) + ) + SELECT + page_id, + post_id, + host_id, + path_id, + impressions, + clicks, + ntile(20) OVER (PARTITION BY page_id ORDER BY clicks ASC ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS rank + FROM as_of_posts + GLOBAL LEFT JOIN as_of_post_metrics USING (page_id, post_id, row_num) + WHERE (row_num = 1) AND (impressions > 0) +) AS t +WHERE t.rank > 18 +GROUP BY + host_id, + path_id +ORDER BY host_id, path_id; + + +INSERT INTO posts SELECT + repeat('a', (number % 10) + 1), + toString(number), + number % 10, + number, + now() - toIntervalMinute(number), + now() +FROM numbers(100000); + + +INSERT INTO post_metrics SELECT + repeat('a', (number % 10) + 1), + toString(number), + now() - toIntervalMinute(number), + number * 100, + number * 10, + now() +FROM numbers(100000); + + +SELECT + host_id, + path_id, + max(rank) AS rank +FROM +( + WITH + as_of_posts AS + ( + SELECT + *, + row_number() OVER (PARTITION BY (page_id, post_id) ORDER BY as_of DESC) AS row_num + FROM posts + WHERE (created >= subtractHours(now(), 24)) AND (host_id > 0) + ), + as_of_post_metrics AS + ( + SELECT + *, + row_number() OVER (PARTITION BY (page_id, post_id) ORDER BY as_of DESC) AS row_num + FROM post_metrics + WHERE created >= subtractHours(now(), 24) + ) + SELECT + page_id, + post_id, + host_id, + path_id, + impressions, + clicks, + ntile(20) OVER (PARTITION BY page_id ORDER BY clicks ASC ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS rank + FROM as_of_posts + GLOBAL LEFT JOIN as_of_post_metrics USING (page_id, post_id, row_num) + WHERE (row_num = 1) AND (impressions > 0) +) AS t +WHERE t.rank > 18 +GROUP BY + host_id, + path_id +ORDER BY host_id, path_id; + +DROP TABLE posts; +DROP TABLE post_metrics; From 22200566c8fbadee0f6c820a9e5320bde9549165 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Dec 2023 03:45:00 +0100 Subject: [PATCH 283/331] Revert "Merge pull request #39631 from ClickHouse/parallel-window" This reverts commit 33cc853d6193a7ad0e79aafb3c22b5effeeb9b17, reversing changes made to b05a3d02ed5bbd8304bc48763a7c58b6f419eb6d. --- src/Interpreters/InterpreterSelectQuery.cpp | 1 - src/Planner/Planner.cpp | 1 - src/Processors/QueryPlan/SortingStep.cpp | 78 +---------- src/Processors/QueryPlan/SortingStep.h | 30 +--- src/Processors/QueryPlan/WindowStep.cpp | 3 +- .../ScatterByPartitionTransform.cpp | 129 ------------------ .../Transforms/ScatterByPartitionTransform.h | 34 ----- ...568_window_functions_distributed.reference | 29 ---- .../01568_window_functions_distributed.sql | 4 - .../02884_parallel_window_functions.reference | 100 -------------- .../02884_parallel_window_functions.sql | 119 ---------------- 11 files changed, 7 insertions(+), 521 deletions(-) delete mode 100644 src/Processors/Transforms/ScatterByPartitionTransform.cpp delete mode 100644 src/Processors/Transforms/ScatterByPartitionTransform.h delete mode 100644 tests/queries/0_stateless/02884_parallel_window_functions.reference delete mode 100644 tests/queries/0_stateless/02884_parallel_window_functions.sql diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 4f4e96a9be7..67245438156 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2942,7 +2942,6 @@ void InterpreterSelectQuery::executeWindow(QueryPlan & query_plan) auto sorting_step = std::make_unique( query_plan.getCurrentDataStream(), window.full_sort_description, - window.partition_by, 0 /* LIMIT */, sort_settings, settings.optimize_sorting_by_input_stream_properties); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index d6e0f42a06d..12e8d795347 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -915,7 +915,6 @@ void addWindowSteps(QueryPlan & query_plan, auto sorting_step = std::make_unique( query_plan.getCurrentDataStream(), window_description.full_sort_description, - window_description.partition_by, 0 /*limit*/, sort_settings, settings.optimize_sorting_by_input_stream_properties); diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index 641b9036d4c..55ce763575e 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -1,4 +1,3 @@ -#include #include #include #include @@ -10,8 +9,6 @@ #include #include -#include -#include namespace CurrentMetrics { @@ -79,21 +76,6 @@ SortingStep::SortingStep( output_stream->sort_scope = DataStream::SortScope::Global; } -SortingStep::SortingStep( - const DataStream & input_stream, - const SortDescription & description_, - const SortDescription & partition_by_description_, - UInt64 limit_, - const Settings & settings_, - bool optimize_sorting_by_input_stream_properties_) - : SortingStep(input_stream, description_, limit_, settings_, optimize_sorting_by_input_stream_properties_) -{ - partition_by_description = partition_by_description_; - - output_stream->sort_description = result_description; - output_stream->sort_scope = DataStream::SortScope::Stream; -} - SortingStep::SortingStep( const DataStream & input_stream_, SortDescription prefix_description_, @@ -135,11 +117,7 @@ void SortingStep::updateOutputStream() { output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits()); output_stream->sort_description = result_description; - - if (partition_by_description.empty()) - output_stream->sort_scope = DataStream::SortScope::Global; - else - output_stream->sort_scope = DataStream::SortScope::Stream; + output_stream->sort_scope = DataStream::SortScope::Global; } void SortingStep::updateLimit(size_t limit_) @@ -157,55 +135,6 @@ void SortingStep::convertToFinishSorting(SortDescription prefix_description_) prefix_description = std::move(prefix_description_); } -void SortingStep::scatterByPartitionIfNeeded(QueryPipelineBuilder& pipeline) -{ - size_t threads = pipeline.getNumThreads(); - size_t streams = pipeline.getNumStreams(); - - if (!partition_by_description.empty() && threads > 1) - { - Block stream_header = pipeline.getHeader(); - - ColumnNumbers key_columns; - key_columns.reserve(partition_by_description.size()); - for (auto & col : partition_by_description) - { - key_columns.push_back(stream_header.getPositionByName(col.column_name)); - } - - pipeline.transform([&](OutputPortRawPtrs ports) - { - Processors processors; - for (auto * port : ports) - { - auto scatter = std::make_shared(stream_header, threads, key_columns); - connect(*port, scatter->getInputs().front()); - processors.push_back(scatter); - } - return processors; - }); - - if (streams > 1) - { - pipeline.transform([&](OutputPortRawPtrs ports) - { - Processors processors; - for (size_t i = 0; i < threads; ++i) - { - size_t output_it = i; - auto resize = std::make_shared(stream_header, streams, 1); - auto & inputs = resize->getInputs(); - - for (auto input_it = inputs.begin(); input_it != inputs.end(); output_it += threads, ++input_it) - connect(*ports[output_it], *input_it); - processors.push_back(resize); - } - return processors; - }); - } - } -} - void SortingStep::finishSorting( QueryPipelineBuilder & pipeline, const SortDescription & input_sort_desc, const SortDescription & result_sort_desc, const UInt64 limit_) { @@ -331,12 +260,10 @@ void SortingStep::fullSortStreams( void SortingStep::fullSort( QueryPipelineBuilder & pipeline, const SortDescription & result_sort_desc, const UInt64 limit_, const bool skip_partial_sort) { - scatterByPartitionIfNeeded(pipeline); - fullSortStreams(pipeline, sort_settings, result_sort_desc, limit_, skip_partial_sort); /// If there are several streams, then we merge them into one - if (pipeline.getNumStreams() > 1 && (partition_by_description.empty() || pipeline.getNumThreads() == 1)) + if (pipeline.getNumStreams() > 1) { auto transform = std::make_shared( pipeline.getHeader(), @@ -368,7 +295,6 @@ void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build { bool need_finish_sorting = (prefix_description.size() < result_description.size()); mergingSorted(pipeline, prefix_description, (need_finish_sorting ? 0 : limit)); - if (need_finish_sorting) { finishSorting(pipeline, prefix_description, result_description, limit); diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h index 52f48f66a32..371a24ac6f2 100644 --- a/src/Processors/QueryPlan/SortingStep.h +++ b/src/Processors/QueryPlan/SortingStep.h @@ -40,15 +40,6 @@ public: const Settings & settings_, bool optimize_sorting_by_input_stream_properties_); - /// Full with partitioning - SortingStep( - const DataStream & input_stream, - const SortDescription & description_, - const SortDescription & partition_by_description_, - UInt64 limit_, - const Settings & settings_, - bool optimize_sorting_by_input_stream_properties_); - /// FinishSorting SortingStep( const DataStream & input_stream_, @@ -92,24 +83,14 @@ public: bool skip_partial_sort = false); private: - void scatterByPartitionIfNeeded(QueryPipelineBuilder& pipeline); void updateOutputStream() override; - static void mergeSorting( - QueryPipelineBuilder & pipeline, - const Settings & sort_settings, - const SortDescription & result_sort_desc, - UInt64 limit_); + static void + mergeSorting(QueryPipelineBuilder & pipeline, const Settings & sort_settings, const SortDescription & result_sort_desc, UInt64 limit_); - void mergingSorted( - QueryPipelineBuilder & pipeline, - const SortDescription & result_sort_desc, - UInt64 limit_); + void mergingSorted(QueryPipelineBuilder & pipeline, const SortDescription & result_sort_desc, UInt64 limit_); void finishSorting( - QueryPipelineBuilder & pipeline, - const SortDescription & input_sort_desc, - const SortDescription & result_sort_desc, - UInt64 limit_); + QueryPipelineBuilder & pipeline, const SortDescription & input_sort_desc, const SortDescription & result_sort_desc, UInt64 limit_); void fullSort( QueryPipelineBuilder & pipeline, const SortDescription & result_sort_desc, @@ -120,9 +101,6 @@ private: SortDescription prefix_description; const SortDescription result_description; - - SortDescription partition_by_description; - UInt64 limit; bool always_read_till_end = false; diff --git a/src/Processors/QueryPlan/WindowStep.cpp b/src/Processors/QueryPlan/WindowStep.cpp index bb4f429d626..9c68a4b73d1 100644 --- a/src/Processors/QueryPlan/WindowStep.cpp +++ b/src/Processors/QueryPlan/WindowStep.cpp @@ -67,8 +67,7 @@ void WindowStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQ // This resize is needed for cases such as `over ()` when we don't have a // sort node, and the input might have multiple streams. The sort node would // have resized it. - if (window_description.full_sort_description.empty()) - pipeline.resize(1); + pipeline.resize(1); pipeline.addSimpleTransform( [&](const Block & /*header*/) diff --git a/src/Processors/Transforms/ScatterByPartitionTransform.cpp b/src/Processors/Transforms/ScatterByPartitionTransform.cpp deleted file mode 100644 index 6e3cdc0fda1..00000000000 --- a/src/Processors/Transforms/ScatterByPartitionTransform.cpp +++ /dev/null @@ -1,129 +0,0 @@ -#include - -#include -#include - -namespace DB -{ -ScatterByPartitionTransform::ScatterByPartitionTransform(Block header, size_t output_size_, ColumnNumbers key_columns_) - : IProcessor(InputPorts{header}, OutputPorts{output_size_, header}) - , output_size(output_size_) - , key_columns(std::move(key_columns_)) - , hash(0) -{} - -IProcessor::Status ScatterByPartitionTransform::prepare() -{ - auto & input = getInputs().front(); - - /// Check all outputs are finished or ready to get data. - - bool all_finished = true; - for (auto & output : outputs) - { - if (output.isFinished()) - continue; - - all_finished = false; - } - - if (all_finished) - { - input.close(); - return Status::Finished; - } - - if (!all_outputs_processed) - { - auto output_it = outputs.begin(); - bool can_push = false; - for (size_t i = 0; i < output_size; ++i, ++output_it) - if (!was_output_processed[i] && output_it->canPush()) - can_push = true; - if (!can_push) - return Status::PortFull; - return Status::Ready; - } - /// Try get chunk from input. - - if (input.isFinished()) - { - for (auto & output : outputs) - output.finish(); - - return Status::Finished; - } - - input.setNeeded(); - if (!input.hasData()) - return Status::NeedData; - - chunk = input.pull(); - has_data = true; - was_output_processed.assign(outputs.size(), false); - - return Status::Ready; -} - -void ScatterByPartitionTransform::work() -{ - if (all_outputs_processed) - generateOutputChunks(); - all_outputs_processed = true; - - size_t chunk_number = 0; - for (auto & output : outputs) - { - auto & was_processed = was_output_processed[chunk_number]; - auto & output_chunk = output_chunks[chunk_number]; - ++chunk_number; - - if (was_processed) - continue; - - if (output.isFinished()) - continue; - - if (!output.canPush()) - { - all_outputs_processed = false; - continue; - } - - output.push(std::move(output_chunk)); - was_processed = true; - } - - if (all_outputs_processed) - { - has_data = false; - output_chunks.clear(); - } -} - -void ScatterByPartitionTransform::generateOutputChunks() -{ - auto num_rows = chunk.getNumRows(); - const auto & columns = chunk.getColumns(); - - hash.reset(num_rows); - - for (const auto & column_number : key_columns) - columns[column_number]->updateWeakHash32(hash); - - const auto & hash_data = hash.getData(); - IColumn::Selector selector(num_rows); - - for (size_t row = 0; row < num_rows; ++row) - selector[row] = hash_data[row] % output_size; - - output_chunks.resize(output_size); - for (const auto & column : columns) - { - auto filtered_columns = column->scatter(output_size, selector); - for (size_t i = 0; i < output_size; ++i) - output_chunks[i].addColumn(std::move(filtered_columns[i])); - } -} - -} diff --git a/src/Processors/Transforms/ScatterByPartitionTransform.h b/src/Processors/Transforms/ScatterByPartitionTransform.h deleted file mode 100644 index 327f6dd62b4..00000000000 --- a/src/Processors/Transforms/ScatterByPartitionTransform.h +++ /dev/null @@ -1,34 +0,0 @@ -#pragma once -#include -#include -#include - -namespace DB -{ - -struct ScatterByPartitionTransform : IProcessor -{ - ScatterByPartitionTransform(Block header, size_t output_size_, ColumnNumbers key_columns_); - - String getName() const override { return "ScatterByPartitionTransform"; } - - Status prepare() override; - void work() override; - -private: - - void generateOutputChunks(); - - size_t output_size; - ColumnNumbers key_columns; - - bool has_data = false; - bool all_outputs_processed = true; - std::vector was_output_processed; - Chunk chunk; - - WeakHash32 hash; - Chunks output_chunks; -}; - -} diff --git a/tests/queries/0_stateless/01568_window_functions_distributed.reference b/tests/queries/0_stateless/01568_window_functions_distributed.reference index 29ff2e7133c..13ac0769a24 100644 --- a/tests/queries/0_stateless/01568_window_functions_distributed.reference +++ b/tests/queries/0_stateless/01568_window_functions_distributed.reference @@ -22,16 +22,6 @@ select sum(number) over w as x, max(number) over w as y from t_01568 window w as 21 8 21 8 21 8 -select sum(number) over w, max(number) over w from t_01568 window w as (partition by p) order by p; -3 2 -3 2 -3 2 -12 5 -12 5 -12 5 -21 8 -21 8 -21 8 select sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by x, y; 6 2 6 2 @@ -51,25 +41,6 @@ select sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1, 42 8 42 8 42 8 -select sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by x, y SETTINGS max_threads = 1; -6 2 -6 2 -6 2 -6 2 -6 2 -6 2 -24 5 -24 5 -24 5 -24 5 -24 5 -24 5 -42 8 -42 8 -42 8 -42 8 -42 8 -42 8 select distinct sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by x, y; 6 2 24 5 diff --git a/tests/queries/0_stateless/01568_window_functions_distributed.sql b/tests/queries/0_stateless/01568_window_functions_distributed.sql index ecce7b412ba..95072d6460f 100644 --- a/tests/queries/0_stateless/01568_window_functions_distributed.sql +++ b/tests/queries/0_stateless/01568_window_functions_distributed.sql @@ -15,12 +15,8 @@ from numbers(9); select sum(number) over w as x, max(number) over w as y from t_01568 window w as (partition by p) order by x, y; -select sum(number) over w, max(number) over w from t_01568 window w as (partition by p) order by p; - select sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by x, y; -select sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by x, y SETTINGS max_threads = 1; - select distinct sum(number) over w as x, max(number) over w as y from remote('127.0.0.{1,2}', '', t_01568) window w as (partition by p) order by x, y; -- window functions + aggregation w/shards diff --git a/tests/queries/0_stateless/02884_parallel_window_functions.reference b/tests/queries/0_stateless/02884_parallel_window_functions.reference deleted file mode 100644 index bac15838dc2..00000000000 --- a/tests/queries/0_stateless/02884_parallel_window_functions.reference +++ /dev/null @@ -1,100 +0,0 @@ -1 --- { echoOn } - -SELECT - nw, - sum(WR) AS R, - sumIf(WR, uniq_rows = 1) AS UNR -FROM -( - SELECT - uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, - AVG(wg) AS WR, - ac, - nw - FROM window_funtion_threading - GROUP BY ac, nw -) -GROUP BY nw -ORDER BY nw ASC, R DESC -LIMIT 10; -0 2 0 -1 2 0 -2 2 0 -SELECT - nw, - sum(WR) AS R, - sumIf(WR, uniq_rows = 1) AS UNR -FROM -( - SELECT - uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, - AVG(wg) AS WR, - ac, - nw - FROM window_funtion_threading - GROUP BY ac, nw -) -GROUP BY nw -ORDER BY nw ASC, R DESC -LIMIT 10 -SETTINGS max_threads = 1; -0 2 0 -1 2 0 -2 2 0 -SELECT - nw, - sum(WR) AS R, - sumIf(WR, uniq_rows = 1) AS UNR -FROM -( - SELECT - uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, - AVG(wg) AS WR, - ac, - nw - FROM window_funtion_threading - WHERE (ac % 4) = 0 - GROUP BY - ac, - nw - UNION ALL - SELECT - uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, - AVG(wg) AS WR, - ac, - nw - FROM window_funtion_threading - WHERE (ac % 4) = 1 - GROUP BY - ac, - nw - UNION ALL - SELECT - uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, - AVG(wg) AS WR, - ac, - nw - FROM window_funtion_threading - WHERE (ac % 4) = 2 - GROUP BY - ac, - nw - UNION ALL - SELECT - uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, - AVG(wg) AS WR, - ac, - nw - FROM window_funtion_threading - WHERE (ac % 4) = 3 - GROUP BY - ac, - nw -) -GROUP BY nw -ORDER BY nw ASC, R DESC -LIMIT 10; -0 2 0 -1 2 0 -2 2 0 diff --git a/tests/queries/0_stateless/02884_parallel_window_functions.sql b/tests/queries/0_stateless/02884_parallel_window_functions.sql deleted file mode 100644 index 3151b42f896..00000000000 --- a/tests/queries/0_stateless/02884_parallel_window_functions.sql +++ /dev/null @@ -1,119 +0,0 @@ -CREATE TABLE window_funtion_threading -Engine = MergeTree -ORDER BY (ac, nw) -AS SELECT - toUInt64(toFloat32(number % 2) % 20000000) as ac, - toFloat32(1) as wg, - toUInt16(toFloat32(number % 3) % 400) as nw -FROM numbers_mt(10000000); - -SELECT count() FROM (EXPLAIN PIPELINE SELECT - nw, - sum(WR) AS R, - sumIf(WR, uniq_rows = 1) AS UNR -FROM -( - SELECT - uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, - AVG(wg) AS WR, - ac, - nw - FROM window_funtion_threading - GROUP BY ac, nw -) -GROUP BY nw -ORDER BY nw ASC, R DESC -LIMIT 10) where explain ilike '%ScatterByPartitionTransform%' SETTINGS max_threads = 4; - --- { echoOn } - -SELECT - nw, - sum(WR) AS R, - sumIf(WR, uniq_rows = 1) AS UNR -FROM -( - SELECT - uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, - AVG(wg) AS WR, - ac, - nw - FROM window_funtion_threading - GROUP BY ac, nw -) -GROUP BY nw -ORDER BY nw ASC, R DESC -LIMIT 10; - -SELECT - nw, - sum(WR) AS R, - sumIf(WR, uniq_rows = 1) AS UNR -FROM -( - SELECT - uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, - AVG(wg) AS WR, - ac, - nw - FROM window_funtion_threading - GROUP BY ac, nw -) -GROUP BY nw -ORDER BY nw ASC, R DESC -LIMIT 10 -SETTINGS max_threads = 1; - -SELECT - nw, - sum(WR) AS R, - sumIf(WR, uniq_rows = 1) AS UNR -FROM -( - SELECT - uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, - AVG(wg) AS WR, - ac, - nw - FROM window_funtion_threading - WHERE (ac % 4) = 0 - GROUP BY - ac, - nw - UNION ALL - SELECT - uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, - AVG(wg) AS WR, - ac, - nw - FROM window_funtion_threading - WHERE (ac % 4) = 1 - GROUP BY - ac, - nw - UNION ALL - SELECT - uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, - AVG(wg) AS WR, - ac, - nw - FROM window_funtion_threading - WHERE (ac % 4) = 2 - GROUP BY - ac, - nw - UNION ALL - SELECT - uniq(nw) OVER (PARTITION BY ac) AS uniq_rows, - AVG(wg) AS WR, - ac, - nw - FROM window_funtion_threading - WHERE (ac % 4) = 3 - GROUP BY - ac, - nw -) -GROUP BY nw -ORDER BY nw ASC, R DESC -LIMIT 10; From 6cab0124cd51511baa151bb981c54305cc0c49f9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Dec 2023 06:02:44 +0300 Subject: [PATCH 284/331] Update SerializationString.cpp --- src/DataTypes/Serializations/SerializationString.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index 308bdce0507..a87c5e7d880 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -152,6 +152,7 @@ template static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnString::Offsets & offsets, ReadBuffer & istr, size_t limit) { size_t offset = data.size(); + /// Avoiding calling resize in a loop improves the performance. data.resize(std::max(data.capacity(), static_cast(4096))); for (size_t i = 0; i < limit; ++i) From 4240e48a5e0230316c928a5999646a37b10137b8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Dec 2023 04:55:16 +0100 Subject: [PATCH 285/331] Sending the logs better --- docker/test/clickbench/run.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docker/test/clickbench/run.sh b/docker/test/clickbench/run.sh index 7357fa6df86..e3c56ed9a75 100755 --- a/docker/test/clickbench/run.sh +++ b/docker/test/clickbench/run.sh @@ -53,4 +53,7 @@ set -x clickhouse-client --query "SELECT total_bytes FROM system.tables WHERE name = 'hits' AND database = 'default'" +clickhouse-client -q "system flush logs" ||: +stop_logs_replication + echo -e "success\tClickBench finished" > /test_output/check_status.tsv From b40e04a8beafb013e1dadd81971764313bb6d5fb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Dec 2023 05:04:37 +0100 Subject: [PATCH 286/331] Export the logs --- docker/test/clickbench/run.sh | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/docker/test/clickbench/run.sh b/docker/test/clickbench/run.sh index e3c56ed9a75..b3b4ea85e24 100755 --- a/docker/test/clickbench/run.sh +++ b/docker/test/clickbench/run.sh @@ -14,6 +14,15 @@ dpkg -i package_folder/clickhouse-client_*.deb mkdir /dev/shm/clickhouse chown clickhouse:clickhouse /dev/shm/clickhouse +# Allow introspection functions, needed for sending the logs +echo " +profiles: + default: + allow_introspection_functions: 1 +" > /etc/clickhouse-server/allow_introspection_functions.yaml + +config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml + clickhouse start # Wait for the server to start, but not for too long. From 02720cde010ea7a3a8ace0a4b97def2222cc8bc3 Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 12 Dec 2023 04:06:43 +0000 Subject: [PATCH 287/331] Fix --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 25 ++++++++++++----- .../replaceForPositionalArguments.cpp | 27 ++++++++++++++----- 2 files changed, 38 insertions(+), 14 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index a52a0fac232..a3b461f32ea 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2153,21 +2153,32 @@ void QueryAnalyzer::replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_ node_to_replace = &sort_node->getExpression(); auto * constant_node = (*node_to_replace)->as(); + if (!constant_node || (constant_node->getValue().getType() != Field::Types::UInt64 && constant_node->getValue().getType() != Field::Types::Int64)) continue; - auto positional_argument_number = constant_node->getValue().get(); - if (positional_argument_number == 0 || static_cast(std::abs(positional_argument_number)) > projection_nodes.size()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, + UInt64 pos; + if (constant_node->getValue().getType() == Field::Types::UInt64) + { + pos = constant_node->getValue().get(); + } + else // Int64 + { + auto value = constant_node->getValue().get(); + pos = value > 0 ? value : projection_nodes.size() + value + 1; + } + + + if (!pos || pos > projection_nodes.size()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Positional argument number {} is out of bounds. Expected in range [1, {}]. In scope {}", - positional_argument_number, + pos, projection_nodes.size(), scope.scope_node->formatASTForErrorMessage()); - positional_argument_number - = (positional_argument_number > 0) ? --positional_argument_number : projection_nodes.size() + positional_argument_number; - *node_to_replace = projection_nodes[positional_argument_number]; + *node_to_replace = projection_nodes[--pos]; } } diff --git a/src/Interpreters/replaceForPositionalArguments.cpp b/src/Interpreters/replaceForPositionalArguments.cpp index f5a77dacd4c..8306da17f52 100644 --- a/src/Interpreters/replaceForPositionalArguments.cpp +++ b/src/Interpreters/replaceForPositionalArguments.cpp @@ -30,15 +30,28 @@ bool replaceForPositionalArguments(ASTPtr & argument, const ASTSelectQuery * sel if (which != Field::Types::UInt64 && which != Field::Types::Int64) return false; - auto pos = ast_literal->value.get(); - if (!pos || static_cast(std::abs(pos)) > columns.size()) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Positional argument out of bounds: {} (expected in range [1, {}]", - pos, columns.size()); + UInt64 pos; - pos = (pos > 0) ? --pos : columns.size() + pos; + if (which == Field::Types::UInt64) + { + pos = ast_literal->value.get(); + } + else if (which == Field::Types::Int64) + { + auto value = ast_literal->value.get(); + pos = value > 0 ? value : columns.size() + value + 1; + } + else + { + return false; + } - const auto & column = columns[pos]; + + if (!pos || pos > columns.size()) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Positional argument out of bounds: {} (expected in range [1, {}]", pos, columns.size()); + + const auto & column = columns[--pos]; if (typeid_cast(column.get()) || typeid_cast(column.get())) { argument = column->clone(); From 9789c2caa214e17bb8323c9d67b6cc62c56eb350 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Dec 2023 05:48:09 +0100 Subject: [PATCH 288/331] Review fixes --- .github/workflows/pull_request.yml | 8 ++----- docker/test/clickbench/run.sh | 2 ++ src/Common/parseRemoteDescription.cpp | 2 +- src/Common/parseRemoteDescription.h | 4 ++++ .../Cached/registerDiskCache.cpp | 3 +-- tests/ci/clickbench.py | 22 +------------------ 6 files changed, 11 insertions(+), 30 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 5bb62b04c32..0be703e1196 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -709,22 +709,18 @@ jobs: with: test_name: ClickBench (amd64) runner_type: func-tester - additional_envs: | - KILL_TIMEOUT=1800 run_command: | cd "$REPO_COPY/tests/ci" - python3 clickbench.py "$CHECK_NAME" "$KILL_TIMEOUT" + python3 clickbench.py "$CHECK_NAME" ClickBenchAarch64: needs: [BuilderDebAarch64] uses: ./.github/workflows/reusable_test.yml with: test_name: ClickBench (aarch64) runner_type: func-tester-aarch64 - additional_envs: | - KILL_TIMEOUT=1800 run_command: | cd "$REPO_COPY/tests/ci" - python3 clickbench.py "$CHECK_NAME" "$KILL_TIMEOUT" + python3 clickbench.py "$CHECK_NAME" ############################################################################################## ######################################### STRESS TESTS ####################################### ############################################################################################## diff --git a/docker/test/clickbench/run.sh b/docker/test/clickbench/run.sh index b3b4ea85e24..a344e0ec27c 100755 --- a/docker/test/clickbench/run.sh +++ b/docker/test/clickbench/run.sh @@ -65,4 +65,6 @@ clickhouse-client --query "SELECT total_bytes FROM system.tables WHERE name = 'h clickhouse-client -q "system flush logs" ||: stop_logs_replication +mv /var/log/clickhouse-server/* /test_output/ + echo -e "success\tClickBench finished" > /test_output/check_status.tsv diff --git a/src/Common/parseRemoteDescription.cpp b/src/Common/parseRemoteDescription.cpp index 8ea3f4a0aa5..7b2045b9de1 100644 --- a/src/Common/parseRemoteDescription.cpp +++ b/src/Common/parseRemoteDescription.cpp @@ -184,7 +184,7 @@ std::vector> parseRemoteDescriptionForExternalDataba } else { - result.emplace_back(std::make_pair(address.substr(0, colon), DB::parseFromString(address.substr(colon + 1)))); + result.emplace_back(std::make_pair(address.substr(0, colon), parseFromString(address.substr(colon + 1)))); } } diff --git a/src/Common/parseRemoteDescription.h b/src/Common/parseRemoteDescription.h index d97558c4728..12435bc68a0 100644 --- a/src/Common/parseRemoteDescription.h +++ b/src/Common/parseRemoteDescription.h @@ -1,8 +1,12 @@ #pragma once + #include #include + + namespace DB { + /* Parse a string that generates shards and replicas. Separator - one of two characters '|' or ',' * depending on whether shards or replicas are generated. * For example: diff --git a/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp b/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp index 182326bbdc3..99fd2c932af 100644 --- a/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp +++ b/src/Disks/ObjectStorages/Cached/registerDiskCache.cpp @@ -6,9 +6,8 @@ #include #include #include -#include #include -#include + namespace DB { diff --git a/tests/ci/clickbench.py b/tests/ci/clickbench.py index 359c10eeb9d..061d36f02fa 100644 --- a/tests/ci/clickbench.py +++ b/tests/ci/clickbench.py @@ -23,7 +23,6 @@ from commit_status_helper import ( get_commit, override_status, post_commit_status, - post_commit_status_to_file, update_mergeable_check, ) from docker_pull_helper import DockerImage, get_image_with_version @@ -113,13 +112,6 @@ def process_results( def parse_args(): parser = argparse.ArgumentParser() parser.add_argument("check_name") - parser.add_argument("kill_timeout", type=int) - parser.add_argument( - "--post-commit-status", - default="commit_status", - choices=["commit_status", "file"], - help="Where to public post commit status", - ) return parser.parse_args() @@ -214,19 +206,7 @@ def main(): ) print(f"::notice:: {check_name} Report url: {report_url}") - if args.post_commit_status == "commit_status": - post_commit_status(commit, state, report_url, description, check_name, pr_info) - elif args.post_commit_status == "file": - post_commit_status_to_file( - post_commit_path, - description, - state, - report_url, - ) - else: - raise Exception( - f'Unknown post_commit_status option "{args.post_commit_status}"' - ) + post_commit_status(commit, state, report_url, description, check_name, pr_info) prepared_events = prepare_tests_results_for_clickhouse( pr_info, From 928659b1ae127bcfb9ec166ada6109540a7af3c4 Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 12 Dec 2023 05:36:07 +0000 Subject: [PATCH 289/331] update test --- tests/queries/0_stateless/01162_strange_mutations.sh | 2 +- tests/queries/0_stateless/01798_having_push_down.sql | 3 ++- tests/queries/0_stateless/02932_group_by_null_fuzzer.sql | 1 + 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01162_strange_mutations.sh b/tests/queries/0_stateless/01162_strange_mutations.sh index eea9ea5f7e5..f6b31847c1e 100755 --- a/tests/queries/0_stateless/01162_strange_mutations.sh +++ b/tests/queries/0_stateless/01162_strange_mutations.sh @@ -28,7 +28,7 @@ do $CLICKHOUSE_CLIENT -q "CREATE TABLE test ENGINE=$engine AS SELECT number + 100 AS n, 0 AS test FROM numbers(50)" 2>&1| grep -Ev "Removing leftovers from table|removed by another replica" $CLICKHOUSE_CLIENT -q "select count(), sum(n), sum(test) from test" if [[ $engine == *"ReplicatedMergeTree"* ]]; then - $CLICKHOUSE_CLIENT -q "ALTER TABLE test + $CLICKHOUSE_CLIENT --enable_positional_arguments=0 -q "ALTER TABLE test UPDATE test = (SELECT groupArray(id) FROM t1 GROUP BY 'dummy')[n - 99] WHERE 1" 2>&1| grep -Fa "DB::Exception: " | grep -Fv "statement with subquery may be nondeterministic" $CLICKHOUSE_CLIENT --allow_nondeterministic_mutations=1 --mutations_sync=1 -q "ALTER TABLE test UPDATE test = (SELECT groupArray(id) FROM t1)[n - 99] WHERE 1" diff --git a/tests/queries/0_stateless/01798_having_push_down.sql b/tests/queries/0_stateless/01798_having_push_down.sql index b3a77c8f5b5..c0c3447f5ab 100644 --- a/tests/queries/0_stateless/01798_having_push_down.sql +++ b/tests/queries/0_stateless/01798_having_push_down.sql @@ -8,11 +8,12 @@ SELECT sum(c0 = 0), min(c0 + 1), sum(c0 + 2) FROM t_having GROUP BY c0 HAVING c0 = 0 SETTINGS enable_optimize_predicate_expression=0; +SET enable_positional_arguments=0; + SELECT c0 + -1, sum(intDivOrZero(intDivOrZero(NULL, NULL), '2'), intDivOrZero(10000000000., intDivOrZero(intDivOrZero(intDivOrZero(NULL, NULL), 10), NULL))) FROM t_having GROUP BY c0 = 2, c0 = 10, intDivOrZero(intDivOrZero(intDivOrZero(NULL, NULL), NULL), NULL), c0 HAVING c0 = 2 SETTINGS enable_optimize_predicate_expression = 0; SELECT sum(c0 + 257) FROM t_having GROUP BY c0 = -9223372036854775808, NULL, -2147483649, c0 HAVING c0 = -9223372036854775808 SETTINGS enable_optimize_predicate_expression = 0; -SET enable_positional_arguments=0; SELECT c0 + -2, c0 + -9223372036854775807, c0 = NULL FROM t_having GROUP BY c0 = 0.9998999834060669, 1023, c0 HAVING c0 = 0.9998999834060669 SETTINGS enable_optimize_predicate_expression = 0; DROP TABLE t_having; diff --git a/tests/queries/0_stateless/02932_group_by_null_fuzzer.sql b/tests/queries/0_stateless/02932_group_by_null_fuzzer.sql index 0c28c120d40..603c7783ef8 100644 --- a/tests/queries/0_stateless/02932_group_by_null_fuzzer.sql +++ b/tests/queries/0_stateless/02932_group_by_null_fuzzer.sql @@ -1,5 +1,6 @@ -- https://github.com/ClickHouse/ClickHouse/issues/43202 -- Queries are generated by the fuzzer, so don't expect them to make sense +SET enable_positional_arguments=0; SELECT NULL, '' FROM (SELECT toNullable(''), NULL AS key GROUP BY GROUPING SETS ((NULL))) AS s1 ALL LEFT JOIN (SELECT '' AS key, NULL AS value GROUP BY GROUPING SETS (('')) WITH TOTALS UNION ALL SELECT NULL AS key, toNullable(NULL) AS value GROUP BY '', NULL, '' WITH TOTALS) AS s2 USING (key); SELECT NULL GROUP BY NULL WITH TOTALS; SELECT 1048575, NULL, b FROM (SELECT '25.5' AS a, NULL, NULL AS b GROUP BY GROUPING SETS ((0.0001)) WITH TOTALS) AS js1 ANY RIGHT JOIN (SELECT NULL AS a, NULL AS b WHERE NULL GROUP BY NULL, -9223372036854775807 WITH CUBE WITH TOTALS UNION ALL SELECT NULL AS a, NULL AS b GROUP BY 1, '21474836.46' WITH TOTALS) AS js2 USING (a, b) ORDER BY nan DESC NULLS LAST, '9223372036854775807' DESC NULLS LAST, a ASC NULLS LAST; From 8a68a4247e164dd822460fc73e03f4f2ad8b8a2c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Dec 2023 07:58:54 +0100 Subject: [PATCH 290/331] Style --- tests/ci/clickbench.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/ci/clickbench.py b/tests/ci/clickbench.py index 061d36f02fa..096309eaf92 100644 --- a/tests/ci/clickbench.py +++ b/tests/ci/clickbench.py @@ -124,7 +124,6 @@ def main(): temp_path.mkdir(parents=True, exist_ok=True) reports_path = Path(REPORTS_PATH) - post_commit_path = temp_path / "clickbench_status.tsv" args = parse_args() check_name = args.check_name From be9fac3a55392da05dba36b7a8adc949ae5da593 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 12 Dec 2023 09:26:06 +0100 Subject: [PATCH 291/331] Lint includes Co-authored-by: alesapin --- src/Server/KeeperReadinessHandler.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/KeeperReadinessHandler.h b/src/Server/KeeperReadinessHandler.h index caa59098427..00b51b886f9 100644 --- a/src/Server/KeeperReadinessHandler.h +++ b/src/Server/KeeperReadinessHandler.h @@ -1,6 +1,6 @@ #pragma once -#include "config.h" +#include #if USE_NURAFT From 1f9c7336a97b88a070d0ce783ff5e687c8abcfb7 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 12 Dec 2023 09:32:16 +0100 Subject: [PATCH 292/331] Fix test helper naming --- tests/integration/helpers/keeper_utils.py | 2 +- tests/integration/test_keeper_http_control/test.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index e07bce901d2..39fa0d0f074 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -279,7 +279,7 @@ def get_leader(cluster, nodes): raise Exception("No leader in Keeper cluster.") -def get_follower(cluster, nodes): +def get_any_follower(cluster, nodes): for node in nodes: if is_follower(cluster, node): return node diff --git a/tests/integration/test_keeper_http_control/test.py b/tests/integration/test_keeper_http_control/test.py index b415a03a5c4..8bffaa6763c 100644 --- a/tests/integration/test_keeper_http_control/test.py +++ b/tests/integration/test_keeper_http_control/test.py @@ -51,7 +51,7 @@ def test_http_readiness(started_cluster): assert readiness_data["details"]["leader"] == True assert readiness_data["details"]["follower"] == False - follower = keeper_utils.get_follower(cluster, [node1, node2, node3]) + follower = keeper_utils.get_any_follower(cluster, [node1, node2, node3]) response = requests.get( "http://{host}:{port}/ready".format(host=follower.ip_address, port=9182) ) From 69a022f72a35b214b8305ae2cd5bca90dcb6f099 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 12 Dec 2023 09:42:32 +0100 Subject: [PATCH 293/331] Add `observer` status --- src/Server/KeeperReadinessHandler.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Server/KeeperReadinessHandler.cpp b/src/Server/KeeperReadinessHandler.cpp index 37afd8e9898..148a209fb12 100644 --- a/src/Server/KeeperReadinessHandler.cpp +++ b/src/Server/KeeperReadinessHandler.cpp @@ -25,13 +25,15 @@ void KeeperReadinessHandler::handleRequest(HTTPServerRequest & /*request*/, HTTP { auto is_leader = keeper_dispatcher->isLeader(); auto is_follower = keeper_dispatcher->isFollower() && keeper_dispatcher->hasLeader(); + auto is_observer = keeper_dispatcher->isObserver() && keeper_dispatcher->hasLeader(); - auto status = is_leader || is_follower; + auto status = is_leader || is_follower || is_observer; Poco::JSON::Object json, details; details.set("leader", is_leader); details.set("follower", is_follower); + details.set("observer", is_observer); json.set("details", details); json.set("status", status ? "ok": "fail"); From efa2e0341ab66004ce3c6695b43f5d15213941c7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 12 Dec 2023 08:45:25 +0000 Subject: [PATCH 294/331] Docs: Fix typo --- docs/en/development/developer-instruction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index 645756a46c7..31346c77949 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -72,7 +72,7 @@ You can also add original ClickHouse repo address to your local repository to pu After successfully running this command you will be able to pull updates from the main ClickHouse repo by running `git pull upstream master`. :::note -Instructions below assume you are building on Linux. If you are cross-compiling or using building on macOS, please also check for operating system and architecture specific guides, such as building [on macOS for macOS](build-osx.md), [on Linux for macOS](build-cross-osx.md), [on Linux for Linux/RISC-V](build-cross-riscv.md) and so on. +Instructions below assume you are building on Linux. If you are cross-compiling or building on macOS, please also check for operating system and architecture specific guides, such as building [on macOS for macOS](build-osx.md), [on Linux for macOS](build-cross-osx.md), [on Linux for Linux/RISC-V](build-cross-riscv.md) and so on. ::: ## Build System {#build-system} From ea123ed5c143ee221fb372d462ede73a1492c317 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 12 Dec 2023 11:35:01 +0100 Subject: [PATCH 295/331] Change response structure --- src/Server/KeeperReadinessHandler.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Server/KeeperReadinessHandler.cpp b/src/Server/KeeperReadinessHandler.cpp index 148a209fb12..ed972055aee 100644 --- a/src/Server/KeeperReadinessHandler.cpp +++ b/src/Server/KeeperReadinessHandler.cpp @@ -27,15 +27,16 @@ void KeeperReadinessHandler::handleRequest(HTTPServerRequest & /*request*/, HTTP auto is_follower = keeper_dispatcher->isFollower() && keeper_dispatcher->hasLeader(); auto is_observer = keeper_dispatcher->isObserver() && keeper_dispatcher->hasLeader(); + auto data = keeper_dispatcher->getKeeper4LWInfo(); + auto status = is_leader || is_follower || is_observer; Poco::JSON::Object json, details; - details.set("leader", is_leader); - details.set("follower", is_follower); - details.set("observer", is_observer); + details.set("role", data.getRole()); + details.set("hasLeader", keeper_dispatcher->hasLeader()); json.set("details", details); - json.set("status", status ? "ok": "fail"); + json.set("status", status ? "ok" : "fail"); std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); From 8fe2cd1a7effaca3676b44158004fe5747a8bbc2 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 12 Dec 2023 11:35:17 +0100 Subject: [PATCH 296/331] Update tests --- .../test_keeper_http_control/test.py | 44 ++++++++++++------- 1 file changed, 28 insertions(+), 16 deletions(-) diff --git a/tests/integration/test_keeper_http_control/test.py b/tests/integration/test_keeper_http_control/test.py index 8bffaa6763c..ed86e06c626 100644 --- a/tests/integration/test_keeper_http_control/test.py +++ b/tests/integration/test_keeper_http_control/test.py @@ -1,12 +1,13 @@ #!/usr/bin/env python3 import os +import time import pytest import requests -import helpers.keeper_utils as keeper_utils -from kazoo.client import KazooClient from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager +import helpers.keeper_utils as keeper_utils cluster = ClickHouseCluster(__file__) CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), "configs") @@ -30,16 +31,7 @@ def started_cluster(): finally: cluster.shutdown() - -def get_fake_zk(node, timeout=30.0): - _fake_zk_instance = KazooClient( - hosts=cluster.get_instance_ip(node.name) + ":9181", timeout=timeout - ) - _fake_zk_instance.start() - return _fake_zk_instance - - -def test_http_readiness(started_cluster): +def test_http_readiness_basic_responses(started_cluster): leader = keeper_utils.get_leader(cluster, [node1, node2, node3]) response = requests.get( "http://{host}:{port}/ready".format(host=leader.ip_address, port=9182) @@ -48,8 +40,7 @@ def test_http_readiness(started_cluster): readiness_data = response.json() assert readiness_data["status"] == "ok" - assert readiness_data["details"]["leader"] == True - assert readiness_data["details"]["follower"] == False + assert readiness_data["details"]["role"] == "leader" follower = keeper_utils.get_any_follower(cluster, [node1, node2, node3]) response = requests.get( @@ -59,5 +50,26 @@ def test_http_readiness(started_cluster): readiness_data = response.json() assert readiness_data["status"] == "ok" - assert readiness_data["details"]["leader"] == False - assert readiness_data["details"]["follower"] == True + assert readiness_data["details"]["role"] == "follower" + assert readiness_data["details"]["hasLeader"] == True + +def test_http_readiness_partitioned_cluster(started_cluster): + with PartitionManager() as pm: + leader = keeper_utils.get_leader(cluster, [node1, node2, node3]) + follower = keeper_utils.get_any_follower(cluster, [node1, node2, node3]) + + pm.partition_instances( + leader, follower + ) + time.sleep(3) + + response = requests.get( + "http://{host}:{port}/ready".format(host=follower.ip_address, port=9182) + ) + print(response.json()) + assert response.status_code == 503 + + readiness_data = response.json() + assert readiness_data["status"] == "fail" + assert readiness_data["details"]["role"] == "follower" + assert readiness_data["details"]["hasLeader"] == False From b49452fb45e7b68575411e1bd9479d2c7e9531cb Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 12 Dec 2023 10:44:55 +0000 Subject: [PATCH 297/331] Automatic style fix --- tests/integration/test_keeper_http_control/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_keeper_http_control/test.py b/tests/integration/test_keeper_http_control/test.py index ed86e06c626..49d2f70e6b0 100644 --- a/tests/integration/test_keeper_http_control/test.py +++ b/tests/integration/test_keeper_http_control/test.py @@ -31,6 +31,7 @@ def started_cluster(): finally: cluster.shutdown() + def test_http_readiness_basic_responses(started_cluster): leader = keeper_utils.get_leader(cluster, [node1, node2, node3]) response = requests.get( @@ -53,14 +54,13 @@ def test_http_readiness_basic_responses(started_cluster): assert readiness_data["details"]["role"] == "follower" assert readiness_data["details"]["hasLeader"] == True + def test_http_readiness_partitioned_cluster(started_cluster): with PartitionManager() as pm: leader = keeper_utils.get_leader(cluster, [node1, node2, node3]) follower = keeper_utils.get_any_follower(cluster, [node1, node2, node3]) - pm.partition_instances( - leader, follower - ) + pm.partition_instances(leader, follower) time.sleep(3) response = requests.get( From 511cfb393dcc7765c30a0ff50d909ca88bbfa35a Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 12 Dec 2023 12:27:49 +0100 Subject: [PATCH 298/331] Remove `time.sleep` from test --- tests/integration/test_keeper_http_control/test.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/integration/test_keeper_http_control/test.py b/tests/integration/test_keeper_http_control/test.py index 49d2f70e6b0..65dc5bea909 100644 --- a/tests/integration/test_keeper_http_control/test.py +++ b/tests/integration/test_keeper_http_control/test.py @@ -1,7 +1,6 @@ #!/usr/bin/env python3 import os -import time import pytest import requests @@ -61,7 +60,7 @@ def test_http_readiness_partitioned_cluster(started_cluster): follower = keeper_utils.get_any_follower(cluster, [node1, node2, node3]) pm.partition_instances(leader, follower) - time.sleep(3) + keeper_utils.wait_until_quorum_lost(cluster, follower) response = requests.get( "http://{host}:{port}/ready".format(host=follower.ip_address, port=9182) From cfe6bc2cc547d91759b69f071d4ad96cec20ab87 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 7 Dec 2023 00:57:01 +0100 Subject: [PATCH 299/331] Replace len by sum for generator --- tests/ci/workflow_jobs_lambda/app.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/ci/workflow_jobs_lambda/app.py b/tests/ci/workflow_jobs_lambda/app.py index 6931835f601..4210ca22da9 100644 --- a/tests/ci/workflow_jobs_lambda/app.py +++ b/tests/ci/workflow_jobs_lambda/app.py @@ -160,9 +160,7 @@ def handler(event: dict, context: Any) -> dict: steps = 0 else: # We record only finished steps - steps = len( - [step for step in wf_job["steps"] if step["conclusion"] is not None] - ) + steps = sum(1 for st in wf_job["steps"] if st["conclusion"] is not None) workflow_job = WorkflowJob( wf_job["id"], From 7ff30211128d08a82cd830d4d1ed16321d58fa47 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Dec 2023 14:12:10 +0100 Subject: [PATCH 300/331] Fix Docker --- tests/ci/ci_config.py | 2 +- tests/ci/clickbench.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index e9f75d66b2e..de2ba3dc1ce 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -512,7 +512,7 @@ CHECK_DESCRIPTIONS = [ CheckDescription( "ClickBench", "Runs [ClickBench](https://github.com/ClickHouse/ClickBench/) with instant-attach table", - lambda x: x.startswith("Upgrade check ("), + lambda x: x.startswith("ClickBench"), ), CheckDescription( "Falback for unknown", diff --git a/tests/ci/clickbench.py b/tests/ci/clickbench.py index 096309eaf92..2ea5e39ce8e 100644 --- a/tests/ci/clickbench.py +++ b/tests/ci/clickbench.py @@ -53,7 +53,7 @@ def get_run_command( env_str = " ".join(envs) return ( - f"docker run --volume={builds_path}:/package_folder " + f"docker run --shm-size=16g --volume={builds_path}:/package_folder " f"{ci_logs_args}" f"--volume={result_path}:/test_output " f"--volume={server_log_path}:/var/log/clickhouse-server " From f1a330e95de085781e75840891060528c9ec301d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Dec 2023 13:46:29 +0000 Subject: [PATCH 301/331] Add a comment --- src/Processors/Transforms/AggregatingTransform.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 7b1d51bb320..0f0fa38727d 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -721,6 +721,8 @@ void AggregatingTransform::initGenerate() if (many_data->num_finished.fetch_add(1) + 1 < many_data->variants.size()) { + /// Note: we reset aggregation state here to release memory earlier. + /// It might cause extra memory usage for complex queries othervise. many_data.reset(); return; } From e77cb18d184c3c06d4765d5cb320d8b4920b9f60 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 12 Dec 2023 15:27:42 +0100 Subject: [PATCH 302/331] Better test --- tests/integration/helpers/postgres_utility.py | 4 +- .../test.py | 55 ++++++++++--------- 2 files changed, 31 insertions(+), 28 deletions(-) diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index 4bf549174e8..690a833f37f 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -280,9 +280,9 @@ class PostgresManager: f"INSERT INTO {database_name}.{table_name} SELECT number, number from numbers(50)" ) - def create_and_fill_postgres_tables(self, tables_num, numbers=50, database_name=""): + def create_and_fill_postgres_tables(self, tables_num, numbers=50, database_name="", table_name_base="postgresql_replica"): for i in range(tables_num): - table_name = f"postgresql_replica_{i}" + table_name = f"{table_name_base}_{i}" create_postgres_table(self.cursor, table_name, database_name) if numbers > 0: db = self.database_or_default(database_name) diff --git a/tests/integration/test_postgresql_replica_database_engine_1/test.py b/tests/integration/test_postgresql_replica_database_engine_1/test.py index c118080a572..e4cce96244f 100644 --- a/tests/integration/test_postgresql_replica_database_engine_1/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_1/test.py @@ -393,18 +393,19 @@ def test_table_schema_changes(started_cluster): def test_many_concurrent_queries(started_cluster): + table = "test_many_conc" query_pool = [ - "DELETE FROM postgresql_replica_{} WHERE (value*value) % 3 = 0;", - "UPDATE postgresql_replica_{} SET value = value - 125 WHERE key % 2 = 0;", - "DELETE FROM postgresql_replica_{} WHERE key % 10 = 0;", - "UPDATE postgresql_replica_{} SET value = value*5 WHERE key % 2 = 1;", - "DELETE FROM postgresql_replica_{} WHERE value % 2 = 0;", - "UPDATE postgresql_replica_{} SET value = value + 2000 WHERE key % 5 = 0;", - "DELETE FROM postgresql_replica_{} WHERE value % 3 = 0;", - "UPDATE postgresql_replica_{} SET value = value * 2 WHERE key % 3 = 0;", - "DELETE FROM postgresql_replica_{} WHERE value % 9 = 2;", - "UPDATE postgresql_replica_{} SET value = value + 2 WHERE key % 3 = 1;", - "DELETE FROM postgresql_replica_{} WHERE value%5 = 0;", + "DELETE FROM {} WHERE (value*value) % 3 = 0;", + "UPDATE {} SET value = value - 125 WHERE key % 2 = 0;", + "DELETE FROM {} WHERE key % 10 = 0;", + "UPDATE {} SET value = value*5 WHERE key % 2 = 1;", + "DELETE FROM {} WHERE value % 2 = 0;", + "UPDATE {} SET value = value + 2000 WHERE key % 5 = 0;", + "DELETE FROM {} WHERE value % 3 = 0;", + "UPDATE {} SET value = value * 2 WHERE key % 3 = 0;", + "DELETE FROM {} WHERE value % 9 = 2;", + "UPDATE {} SET value = value + 2 WHERE key % 3 = 1;", + "DELETE FROM {} WHERE value%5 = 0;", ] NUM_TABLES = 5 @@ -415,7 +416,7 @@ def test_many_concurrent_queries(started_cluster): database=True, ) cursor = conn.cursor() - pg_manager.create_and_fill_postgres_tables(NUM_TABLES, numbers=10000) + pg_manager.create_and_fill_postgres_tables(NUM_TABLES, numbers=10000, table_name_base=table) def attack(thread_id): print("thread {}".format(thread_id)) @@ -423,17 +424,19 @@ def test_many_concurrent_queries(started_cluster): for i in range(20): query_id = random.randrange(0, len(query_pool) - 1) table_id = random.randrange(0, 5) # num tables + random_table_name = f"{table}_{table_id}" + table_name = f"{table}_{thread_id}" # random update / delete query - cursor.execute(query_pool[query_id].format(table_id)) - print("table {} query {} ok".format(table_id, query_id)) + cursor.execute(query_pool[query_id].format(random_table_name)) + print("table {} query {} ok".format(random_table_name, query_id)) # allow some thread to do inserts (not to violate key constraints) if thread_id < 5: print("try insert table {}".format(thread_id)) instance.query( - "INSERT INTO postgres_database.postgresql_replica_{} SELECT {}*10000*({} + number), number from numbers(1000)".format( - i, thread_id, k + "INSERT INTO postgres_database.{} SELECT {}*10000*({} + number), number from numbers(1000)".format( + table_name, thread_id, k ) ) k += 1 @@ -443,8 +446,8 @@ def test_many_concurrent_queries(started_cluster): # also change primary key value print("try update primary key {}".format(thread_id)) cursor.execute( - "UPDATE postgresql_replica_{} SET key=key%100000+100000*{} WHERE key%{}=0".format( - thread_id, i + 1, i + 1 + "UPDATE {table}_{} SET key=key%100000+100000*{} WHERE key%{}=0".format( + table_name, i + 1, i + 1 ) ) print("update primary key {} ok".format(thread_id)) @@ -467,25 +470,25 @@ def test_many_concurrent_queries(started_cluster): n[0] = 50000 for table_id in range(NUM_TABLES): n[0] += 1 + table_name = f"{table}_{table_id}" instance.query( - "INSERT INTO postgres_database.postgresql_replica_{} SELECT {} + number, number from numbers(5000)".format( - table_id, n[0] + "INSERT INTO postgres_database.{} SELECT {} + number, number from numbers(5000)".format( + table_name, n[0] ) ) - # cursor.execute("UPDATE postgresql_replica_{} SET key=key%100000+100000*{} WHERE key%{}=0".format(table_id, table_id+1, table_id+1)) + # cursor.execute("UPDATE {table}_{} SET key=key%100000+100000*{} WHERE key%{}=0".format(table_id, table_id+1, table_id+1)) for thread in threads: thread.join() for i in range(NUM_TABLES): - check_tables_are_synchronized(instance, "postgresql_replica_{}".format(i)) + table_name = f"{table}_{i}" + check_tables_are_synchronized(instance, table_name) count1 = instance.query( - "SELECT count() FROM postgres_database.postgresql_replica_{}".format(i) + "SELECT count() FROM postgres_database.{}".format(table_name) ) count2 = instance.query( - "SELECT count() FROM (SELECT * FROM test_database.postgresql_replica_{})".format( - i - ) + "SELECT count() FROM (SELECT * FROM test_database.{})".format(table_name) ) assert int(count1) == int(count2) print(count1, count2) From f2336ff0253703c755dae8f53cc7c0604fb7f450 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 12 Dec 2023 14:43:44 +0000 Subject: [PATCH 303/331] Automatic style fix --- tests/integration/helpers/postgres_utility.py | 8 +++++++- .../test_postgresql_replica_database_engine_1/test.py | 4 +++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index 690a833f37f..468c3b3bb63 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -280,7 +280,13 @@ class PostgresManager: f"INSERT INTO {database_name}.{table_name} SELECT number, number from numbers(50)" ) - def create_and_fill_postgres_tables(self, tables_num, numbers=50, database_name="", table_name_base="postgresql_replica"): + def create_and_fill_postgres_tables( + self, + tables_num, + numbers=50, + database_name="", + table_name_base="postgresql_replica", + ): for i in range(tables_num): table_name = f"{table_name_base}_{i}" create_postgres_table(self.cursor, table_name, database_name) diff --git a/tests/integration/test_postgresql_replica_database_engine_1/test.py b/tests/integration/test_postgresql_replica_database_engine_1/test.py index e4cce96244f..2e0a597f885 100644 --- a/tests/integration/test_postgresql_replica_database_engine_1/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_1/test.py @@ -416,7 +416,9 @@ def test_many_concurrent_queries(started_cluster): database=True, ) cursor = conn.cursor() - pg_manager.create_and_fill_postgres_tables(NUM_TABLES, numbers=10000, table_name_base=table) + pg_manager.create_and_fill_postgres_tables( + NUM_TABLES, numbers=10000, table_name_base=table + ) def attack(thread_id): print("thread {}".format(thread_id)) From 7142eacad3c8d2e793f21a36bb58d94d6d9b5656 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 12 Dec 2023 17:54:26 +0300 Subject: [PATCH 304/331] SerializationString reduce memory usage --- src/DataTypes/Serializations/SerializationString.cpp | 2 +- tests/queries/0_stateless/01926_order_by_desc_limit.sql | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index a87c5e7d880..788ff429088 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -175,7 +175,7 @@ static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnSt offsets.push_back(offset); if (unlikely(offset > data.size())) - data.resize(roundUpToPowerOfTwoOrZero(std::max(offset, data.size() * 2))); + data.resize_exact(roundUpToPowerOfTwoOrZero(std::max(offset, data.size() * 2))); if (size) { diff --git a/tests/queries/0_stateless/01926_order_by_desc_limit.sql b/tests/queries/0_stateless/01926_order_by_desc_limit.sql index 6854e6c1e84..a0047a2925a 100644 --- a/tests/queries/0_stateless/01926_order_by_desc_limit.sql +++ b/tests/queries/0_stateless/01926_order_by_desc_limit.sql @@ -11,9 +11,11 @@ SETTINGS index_granularity = 1024, index_granularity_bytes = '10Mi'; INSERT INTO order_by_desc SELECT number, repeat('a', 1024) FROM numbers(1024 * 300); OPTIMIZE TABLE order_by_desc FINAL; -SELECT s FROM order_by_desc ORDER BY u DESC LIMIT 10 FORMAT Null; +SELECT s FROM order_by_desc ORDER BY u DESC LIMIT 10 FORMAT Null +SETTINGS max_memory_usage = '400M'; -SELECT s FROM order_by_desc ORDER BY u LIMIT 10 FORMAT Null; +SELECT s FROM order_by_desc ORDER BY u LIMIT 10 FORMAT Null +SETTINGS max_memory_usage = '400M'; SYSTEM FLUSH LOGS; From 54676707412d586b1f97a773a22a540b7eb40d85 Mon Sep 17 00:00:00 2001 From: johnnymatthews <9611008+johnnymatthews@users.noreply.github.com> Date: Tue, 12 Dec 2023 11:13:34 -0400 Subject: [PATCH 305/331] Mentions that APPEND or TRUNCATE should be used with INTO-OUTFILE. --- docs/en/sql-reference/statements/select/into-outfile.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/sql-reference/statements/select/into-outfile.md b/docs/en/sql-reference/statements/select/into-outfile.md index 352af16042a..985f5e25b05 100644 --- a/docs/en/sql-reference/statements/select/into-outfile.md +++ b/docs/en/sql-reference/statements/select/into-outfile.md @@ -26,6 +26,7 @@ SELECT INTO OUTFILE file_name [AND STDOUT] [APPEND] [COMPRESSION typ - The default [output format](../../../interfaces/formats.md) is `TabSeparated` (like in the command-line client batch mode). Use [FORMAT](format.md) clause to change it. - If `AND STDOUT` is mentioned in the query then the output that is written to the file is also displayed on standard output. If used with compression, the plaintext is displayed on standard output. - If `APPEND` is mentioned in the query then the output is appended to an existing file. If compression is used, append cannot be used. +- When writing to a file that already exists, `APPEND` or `TRUNCATE` must be used. **Example** From af4f1abadd1563b00d5ff572142b59039eee76c3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Dec 2023 16:41:38 +0100 Subject: [PATCH 306/331] Fix error --- docker/test/clickbench/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/clickbench/run.sh b/docker/test/clickbench/run.sh index a344e0ec27c..471e1fd6714 100755 --- a/docker/test/clickbench/run.sh +++ b/docker/test/clickbench/run.sh @@ -21,7 +21,7 @@ profiles: allow_introspection_functions: 1 " > /etc/clickhouse-server/allow_introspection_functions.yaml -config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml +config_logs_export_cluster /etc/clickhouse-server/users.d/system_logs_export.yaml clickhouse start From 12561c0c9b7abaee9c7bf0d469de909b70af9c84 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Dec 2023 16:42:31 +0100 Subject: [PATCH 307/331] Maybe better --- docker/test/clickbench/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/clickbench/run.sh b/docker/test/clickbench/run.sh index 471e1fd6714..5d2312c22c5 100755 --- a/docker/test/clickbench/run.sh +++ b/docker/test/clickbench/run.sh @@ -64,6 +64,7 @@ clickhouse-client --query "SELECT total_bytes FROM system.tables WHERE name = 'h clickhouse-client -q "system flush logs" ||: stop_logs_replication +clickhouse stop mv /var/log/clickhouse-server/* /test_output/ From 7196103be5bf2d937ca0422f01e21dfad94978ba Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 12 Dec 2023 17:41:16 +0100 Subject: [PATCH 308/331] Always recreate ephemeral "alive" node on reconnection. --- src/Backups/BackupCoordinationRemote.cpp | 8 +++++--- src/Backups/RestoreCoordinationRemote.cpp | 10 +++++----- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 7319b1aba58..b659887e0da 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -184,10 +184,12 @@ BackupCoordinationRemote::BackupCoordinationRemote( if (my_is_internal) { String alive_node_path = my_zookeeper_path + "/stage/alive|" + my_current_host; + + /// Delete the ephemeral node from the previous connection so we don't have to wait for keeper to do it automatically. + zk->tryRemove(alive_node_path); + zk->createAncestors(alive_node_path); - auto code = zk->tryCreate(alive_node_path, "", zkutil::CreateMode::Ephemeral); - if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) - throw zkutil::KeeperException::fromPath(code, alive_node_path); + zk->create(alive_node_path, "", zkutil::CreateMode::Ephemeral); } }) { diff --git a/src/Backups/RestoreCoordinationRemote.cpp b/src/Backups/RestoreCoordinationRemote.cpp index 60a83c580f0..190634de4a9 100644 --- a/src/Backups/RestoreCoordinationRemote.cpp +++ b/src/Backups/RestoreCoordinationRemote.cpp @@ -43,12 +43,12 @@ RestoreCoordinationRemote::RestoreCoordinationRemote( if (my_is_internal) { String alive_node_path = my_zookeeper_path + "/stage/alive|" + my_current_host; - auto code = zk->tryCreate(alive_node_path, "", zkutil::CreateMode::Ephemeral); - if (code == Coordination::Error::ZNODEEXISTS) - zk->handleEphemeralNodeExistenceNoFailureInjection(alive_node_path, ""); - else if (code != Coordination::Error::ZOK) - throw zkutil::KeeperException::fromPath(code, alive_node_path); + /// Delete the ephemeral node from the previous connection so we don't have to wait for keeper to do it automatically. + zk->tryRemove(alive_node_path); + + zk->createAncestors(alive_node_path); + zk->create(alive_node_path, "", zkutil::CreateMode::Ephemeral); } }) { From b31816cc901707ec87eb7c531487467146468d12 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 12 Dec 2023 16:48:13 +0000 Subject: [PATCH 309/331] atomic_set_in_librdkafka: update librdkafka submodule --- contrib/librdkafka | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/librdkafka b/contrib/librdkafka index 6f3b483426a..2d2aab6f5b7 160000 --- a/contrib/librdkafka +++ b/contrib/librdkafka @@ -1 +1 @@ -Subproject commit 6f3b483426a8c8ec950e27e446bec175cf8b553f +Subproject commit 2d2aab6f5b79db1cfca15d7bf0dee75d00d82082 From 64d7abde099bfc516e0fb630b581b19c3a548279 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 12 Dec 2023 16:51:23 +0000 Subject: [PATCH 310/331] fix result of external aggregation in case of partially materialized projection --- .../Transforms/AggregatingTransform.cpp | 22 ++++--- ...projections_external_aggregation.reference | 41 ++++++++++++ ...02941_projections_external_aggregation.sql | 66 +++++++++++++++++++ 3 files changed, 121 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/02941_projections_external_aggregation.reference create mode 100644 tests/queries/0_stateless/02941_projections_external_aggregation.sql diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index bf475c57d36..1f52ed97491 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -726,8 +726,11 @@ void AggregatingTransform::initGenerate() auto prepared_data = params->aggregator.prepareVariantsToMerge(many_data->variants); Pipes pipes; for (auto & variant : prepared_data) + { /// Converts hash tables to blocks with data (finalized or not). pipes.emplace_back(std::make_shared(params, variant)); + } + Pipe pipe = Pipe::unitePipes(std::move(pipes)); if (!pipe.empty()) { @@ -781,21 +784,23 @@ void AggregatingTransform::initGenerate() } } - const auto & tmp_data = params->aggregator.getTemporaryData(); + size_t num_streams = 0; + size_t compressed_size = 0; + size_t uncompressed_size = 0; - Pipe pipe; + Pipes pipes; + /// Merge external data from all aggregators used in query. + for (const auto & aggregator : *params->aggregator_list_ptr) { - Pipes pipes; - + const auto & tmp_data = aggregator.getTemporaryData(); for (auto * tmp_stream : tmp_data.getStreams()) pipes.emplace_back(Pipe(std::make_unique(tmp_stream))); - pipe = Pipe::unitePipes(std::move(pipes)); + num_streams += tmp_data.getStreams().size(); + compressed_size += tmp_data.getStat().compressed_size; + uncompressed_size += tmp_data.getStat().uncompressed_size; } - size_t num_streams = tmp_data.getStreams().size(); - size_t compressed_size = tmp_data.getStat().compressed_size; - size_t uncompressed_size = tmp_data.getStat().uncompressed_size; LOG_DEBUG( log, "Will merge {} temporary files of size {} compressed, {} uncompressed.", @@ -803,6 +808,7 @@ void AggregatingTransform::initGenerate() ReadableSize(compressed_size), ReadableSize(uncompressed_size)); + auto pipe = Pipe::unitePipes(std::move(pipes)); addMergingAggregatedMemoryEfficientTransform(pipe, params, temporary_data_merge_threads); processors = Pipe::detachProcessors(std::move(pipe)); diff --git a/tests/queries/0_stateless/02941_projections_external_aggregation.reference b/tests/queries/0_stateless/02941_projections_external_aggregation.reference new file mode 100644 index 00000000000..4b1a62520cd --- /dev/null +++ b/tests/queries/0_stateless/02941_projections_external_aggregation.reference @@ -0,0 +1,41 @@ +*** correct aggregation *** +1 0 0 1249950000 +1 0 2 1250000000 +1 1 1 1249975000 +1 1 3 1250025000 +*** correct aggregation with projection *** +1 0 0 1249950000 +1 0 2 1250000000 +1 1 1 1249975000 +1 1 3 1250025000 +*** optimize_aggregation_in_order = 0, max_bytes_before_external_group_by = 1, group_by_two_level_threshold = 1 *** +1 0 0 1249950000 +1 0 2 1250000000 +1 1 1 1249975000 +1 1 3 1250025000 +*** optimize_aggregation_in_order = 1, max_bytes_before_external_group_by = 1, group_by_two_level_threshold = 1 *** +1 0 0 1249950000 +1 0 2 1250000000 +1 1 1 1249975000 +1 1 3 1250025000 +*** after materialization *** +*** correct aggregation *** +1 0 0 1249950000 +1 0 2 1250000000 +1 1 1 1249975000 +1 1 3 1250025000 +*** correct aggregation with projection *** +1 0 0 1249950000 +1 0 2 1250000000 +1 1 1 1249975000 +1 1 3 1250025000 +*** optimize_aggregation_in_order = 0, max_bytes_before_external_group_by = 1, group_by_two_level_threshold = 1 *** +1 0 0 1249950000 +1 0 2 1250000000 +1 1 1 1249975000 +1 1 3 1250025000 +*** optimize_aggregation_in_order = 1, max_bytes_before_external_group_by = 1, group_by_two_level_threshold = 1 *** +1 0 0 1249950000 +1 0 2 1250000000 +1 1 1 1249975000 +1 1 3 1250025000 diff --git a/tests/queries/0_stateless/02941_projections_external_aggregation.sql b/tests/queries/0_stateless/02941_projections_external_aggregation.sql new file mode 100644 index 00000000000..5053773f142 --- /dev/null +++ b/tests/queries/0_stateless/02941_projections_external_aggregation.sql @@ -0,0 +1,66 @@ +DROP TABLE IF EXISTS t_proj_external; + +CREATE TABLE t_proj_external +( + k1 UInt32, + k2 UInt32, + k3 UInt32, + value UInt32 +) +ENGINE = MergeTree +ORDER BY tuple(); + +INSERT INTO t_proj_external SELECT 1, number%2, number%4, number FROM numbers(50000); + +SYSTEM STOP MERGES t_proj_external; + +ALTER TABLE t_proj_external ADD PROJECTION aaaa ( + SELECT + k1, + k2, + k3, + sum(value) + GROUP BY k1, k2, k3 +); + +INSERT INTO t_proj_external SELECT 1, number%2, number%4, number FROM numbers(100000) LIMIT 50000, 100000; + +SELECT '*** correct aggregation ***'; + +SELECT k1, k2, k3, sum(value) v FROM t_proj_external GROUP BY k1, k2, k3 ORDER BY k1, k2, k3 SETTINGS optimize_use_projections = 0; + +SELECT '*** correct aggregation with projection ***'; + +SELECT k1, k2, k3, sum(value) v FROM t_proj_external GROUP BY k1, k2, k3 ORDER BY k1, k2, k3; + +SELECT '*** optimize_aggregation_in_order = 0, max_bytes_before_external_group_by = 1, group_by_two_level_threshold = 1 ***'; + +SELECT k1, k2, k3, sum(value) v FROM t_proj_external GROUP BY k1, k2, k3 ORDER BY k1, k2, k3 SETTINGS optimize_aggregation_in_order = 0, max_bytes_before_external_group_by = 1, group_by_two_level_threshold = 1; + +SELECT '*** optimize_aggregation_in_order = 1, max_bytes_before_external_group_by = 1, group_by_two_level_threshold = 1 ***'; + +SELECT k1, k2, k3, sum(value) v FROM t_proj_external GROUP BY k1, k2, k3 ORDER BY k1, k2, k3 SETTINGS optimize_aggregation_in_order = 1, max_bytes_before_external_group_by = 1, group_by_two_level_threshold = 1; + +SYSTEM START MERGES t_proj_external; + +ALTER TABLE t_proj_external MATERIALIZE PROJECTION aaaa SETTINGS mutations_sync = 2; + +SELECT '*** after materialization ***'; + +SELECT '*** correct aggregation ***'; + +SELECT k1, k2, k3, sum(value) v FROM t_proj_external GROUP BY k1, k2, k3 ORDER BY k1, k2, k3 SETTINGS optimize_use_projections = 0; + +SELECT '*** correct aggregation with projection ***'; + +SELECT k1, k2, k3, sum(value) v FROM t_proj_external GROUP BY k1, k2, k3 ORDER BY k1, k2, k3; + +SELECT '*** optimize_aggregation_in_order = 0, max_bytes_before_external_group_by = 1, group_by_two_level_threshold = 1 ***'; + +SELECT k1, k2, k3, sum(value) v FROM t_proj_external GROUP BY k1, k2, k3 ORDER BY k1, k2, k3 SETTINGS optimize_aggregation_in_order = 0, max_bytes_before_external_group_by = 1, group_by_two_level_threshold = 1; + +SELECT '*** optimize_aggregation_in_order = 1, max_bytes_before_external_group_by = 1, group_by_two_level_threshold = 1 ***'; + +SELECT k1, k2, k3, sum(value) v FROM t_proj_external GROUP BY k1, k2, k3 ORDER BY k1, k2, k3 SETTINGS optimize_aggregation_in_order = 1, max_bytes_before_external_group_by = 1, group_by_two_level_threshold = 1; + +DROP TABLE IF EXISTS t_proj_external; From e4e97471a6b2a6b7617473c7be62494d6098f0d6 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 12 Dec 2023 17:31:56 +0000 Subject: [PATCH 311/331] fix totals in aggregation functions with Map combinator --- .../AggregateFunctionSumMap.cpp | 11 +- .../02480_max_map_null_totals.reference | 108 +++++++++--------- .../0_stateless/02480_max_map_null_totals.sql | 54 ++++----- 3 files changed, 91 insertions(+), 82 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.cpp b/src/AggregateFunctions/AggregateFunctionSumMap.cpp index 04bc908396a..9f0873a6c9c 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionSumMap.cpp @@ -254,11 +254,20 @@ public: if (it != merged_maps.end()) { for (size_t col = 0; col < values_types.size(); ++col) + { if (!elem.second[col].isNull()) - applyVisitor(Visitor(elem.second[col]), it->second[col]); + { + if (it->second[col].isNull()) + it->second[col] = elem.second[col]; + else + applyVisitor(Visitor(elem.second[col]), it->second[col]); + } + } } else + { merged_maps[elem.first] = elem.second; + } } } diff --git a/tests/queries/0_stateless/02480_max_map_null_totals.reference b/tests/queries/0_stateless/02480_max_map_null_totals.reference index 5cc9b5a495f..8fa02ad2a39 100644 --- a/tests/queries/0_stateless/02480_max_map_null_totals.reference +++ b/tests/queries/0_stateless/02480_max_map_null_totals.reference @@ -1,119 +1,119 @@ ([-1,0],[0,0]) -([1,2],[0,2]) ([0,1],[0,1]) +([1,2],[0,2]) -([-1,0,1,2],[0,0,0,2]) +([-1,0,1,2],[0,0,1,2]) ([-1,0],[0,0]) -([1,2],[0,2]) +([-1,0,1,2],[0,0,1,2]) ([0,1],[0,1]) -([-1,0,1,2],[0,0,0,2]) +([1,2],[0,2]) ([-1,0],[0,0]) -([1,2],[0,2]) +([-1,0,1,2],[0,0,1,2]) ([0,1],[0,1]) -([-1,0,1,2],[0,0,0,2]) +([1,2],[0,2]) ([-1,0],[0,0]) -([1,2],[0,2]) ([0,1],[0,1]) +([1,2],[0,2]) -([-1,0,1,2],[0,0,0,2]) +([-1,0,1,2],[0,0,1,2]) ([-1,0],[0,0]) -([1,2],[0,2]) +([-1,0,1,2],[0,0,1,2]) ([0,1],[0,1]) -([-1,0,1,2],[0,0,0,2]) +([1,2],[0,2]) ([-1,0],[0,0]) -([1,2],[0,2]) +([-1,0,1,2],[0,0,1,2]) ([0,1],[0,1]) -([-1,0,1,2],[0,0,0,2]) +([1,2],[0,2]) ([0],[0]) -([2],[2]) ([1],[1]) +([2],[2]) -([0,2],[0,2]) +([0,1,2],[0,1,2]) ([0],[0]) -([2],[2]) +([0,1,2],[0,1,2]) ([1],[1]) -([0,2],[0,2]) +([2],[2]) ([0],[0]) -([2],[2]) +([0,1,2],[0,1,2]) ([1],[1]) -([0,2],[0,2]) +([2],[2]) - ([-1,0],[0,0]) -([1,2],[0,2]) ([0,1],[0,1]) +([1,2],[0,2]) -([-1,0,1,2],[0,0,0,2]) +([-1,0,1,2],[0,0,1,2]) ([-1,0],[0,0]) -([1,2],[0,2]) +([-1,0,1,2],[0,0,1,2]) ([0,1],[0,1]) -([-1,0,1,2],[0,0,0,2]) +([1,2],[0,2]) ([-1,0],[0,0]) -([1,2],[0,2]) +([-1,0,1,2],[0,0,1,2]) ([0,1],[0,1]) -([-1,0,1,2],[0,0,0,2]) +([1,2],[0,2]) ([-1,0],[0,0]) -([1,2],[0,2]) ([0,1],[0,1]) +([1,2],[0,2]) -([-1,0,1,2],[0,0,0,2]) +([-1,0,1,2],[0,0,1,2]) ([-1,0],[0,0]) -([1,2],[0,2]) +([-1,0,1,2],[0,0,1,2]) ([0,1],[0,1]) -([-1,0,1,2],[0,0,0,2]) +([1,2],[0,2]) ([-1,0],[0,0]) -([1,2],[0,2]) +([-1,0,1,2],[0,0,1,2]) ([0,1],[0,1]) -([-1,0,1,2],[0,0,0,2]) +([1,2],[0,2]) ([0],[0]) -([2],[2]) ([1],[1]) +([2],[2]) -([0,2],[0,2]) +([0,1,2],[0,1,2]) ([0],[0]) -([2],[2]) +([0,1,2],[0,1,2]) ([1],[1]) -([0,2],[0,2]) +([2],[2]) ([0],[0]) -([2],[2]) +([0,1,2],[0,1,2]) ([1],[1]) -([0,2],[0,2]) +([2],[2]) - ([-1,0],[0,0]) -([1,2],[0,2]) ([0,1],[0,1]) +([1,2],[0,2]) -([-1,0,1,2],[0,0,0,2]) +([-1,0,1,2],[0,0,1,2]) ([-1,0],[0,0]) -([1,2],[0,2]) +([-1,0,1,2],[0,0,1,2]) ([0,1],[0,1]) -([-1,0,1,2],[0,0,0,2]) +([1,2],[0,2]) ([-1,0],[0,0]) -([1,2],[0,2]) +([-1,0,1,2],[0,0,1,2]) ([0,1],[0,1]) -([-1,0,1,2],[0,0,0,2]) +([1,2],[0,2]) ([-1,0],[0,0]) -([1,2],[0,2]) ([0,1],[0,1]) +([1,2],[0,2]) -([-1,0,1,2],[0,0,0,2]) +([-1,0,1,2],[0,0,1,2]) ([-1,0],[0,0]) -([1,2],[0,2]) +([-1,0,1,2],[0,0,1,2]) ([0,1],[0,1]) -([-1,0,1,2],[0,0,0,2]) +([1,2],[0,2]) ([-1,0],[0,0]) -([1,2],[0,2]) +([-1,0,1,2],[0,0,1,2]) ([0,1],[0,1]) -([-1,0,1,2],[0,0,0,2]) +([1,2],[0,2]) ([0],[0]) -([2],[2]) ([1],[1]) +([2],[2]) -([0,2],[0,2]) +([0,1,2],[0,1,2]) ([0],[0]) -([2],[2]) +([0,1,2],[0,1,2]) ([1],[1]) -([0,2],[0,2]) +([2],[2]) ([0],[0]) -([2],[2]) +([0,1,2],[0,1,2]) ([1],[1]) -([0,2],[0,2]) +([2],[2]) diff --git a/tests/queries/0_stateless/02480_max_map_null_totals.sql b/tests/queries/0_stateless/02480_max_map_null_totals.sql index 81e2a5c4243..2c970e25fd5 100644 --- a/tests/queries/0_stateless/02480_max_map_null_totals.sql +++ b/tests/queries/0_stateless/02480_max_map_null_totals.sql @@ -1,39 +1,39 @@ -SELECT maxMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS; -SELECT maxMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP; -SELECT maxMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH CUBE; +SELECT maxMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS ORDER BY number; +SELECT maxMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP ORDER BY number; +SELECT maxMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH CUBE ORDER BY number; -SELECT minMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS; -SELECT minMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP; -SELECT minMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH CUBE; +SELECT minMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS ORDER BY number; +SELECT minMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP ORDER BY number; +SELECT minMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH CUBE ORDER BY number; -SELECT sumMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS; -SELECT sumMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP; -SELECT sumMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH CUBE; +SELECT sumMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS ORDER BY number; +SELECT sumMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP ORDER BY number; +SELECT sumMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH CUBE ORDER BY number; SELECT '-'; -SELECT maxMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS; -SELECT maxMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP; -SELECT maxMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH CUBE; +SELECT maxMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS ORDER BY number; +SELECT maxMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP ORDER BY number; +SELECT maxMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH CUBE ORDER BY number; -SELECT minMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS; -SELECT minMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP; -SELECT minMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH CUBE; +SELECT minMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS ORDER BY number; +SELECT minMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP ORDER BY number; +SELECT minMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH CUBE ORDER BY number; -SELECT sumMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS; -SELECT sumMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP; -SELECT sumMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH CUBE; +SELECT sumMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS ORDER BY number; +SELECT sumMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP ORDER BY number; +SELECT sumMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH CUBE ORDER BY number; SELECT '-'; -SELECT maxMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS; -SELECT maxMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP; -SELECT maxMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH CUBE; +SELECT maxMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS ORDER BY number; +SELECT maxMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP ORDER BY number; +SELECT maxMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH CUBE ORDER BY number; -SELECT minMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS; -SELECT minMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP; -SELECT minMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH CUBE; +SELECT minMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS ORDER BY number; +SELECT minMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP ORDER BY number; +SELECT minMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH CUBE ORDER BY number; -SELECT sumMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS; -SELECT sumMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP; -SELECT sumMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH CUBE; +SELECT sumMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS ORDER BY number; +SELECT sumMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP ORDER BY number; +SELECT sumMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH CUBE ORDER BY number; \ No newline at end of file From 028763def5313debef322ffabaedbb4c3a9cdcd6 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 12 Dec 2023 18:53:52 +0100 Subject: [PATCH 312/331] Simplify logic in BackupCoordinationStageSync::readCurrentState() and return earlier from the cycly on a connection problem. --- src/Backups/BackupCoordinationStageSync.cpp | 56 ++++++++++++--------- 1 file changed, 33 insertions(+), 23 deletions(-) diff --git a/src/Backups/BackupCoordinationStageSync.cpp b/src/Backups/BackupCoordinationStageSync.cpp index e4dac7dbbe9..cedcecfd35c 100644 --- a/src/Backups/BackupCoordinationStageSync.cpp +++ b/src/Backups/BackupCoordinationStageSync.cpp @@ -100,19 +100,19 @@ Strings BackupCoordinationStageSync::waitFor(const Strings & all_hosts, const St namespace { - struct UnreadyHostState + struct UnreadyHost { + String host; bool started = false; - bool alive = false; }; } struct BackupCoordinationStageSync::State { - Strings results; - std::map unready_hosts; + std::optional results; std::optional> error; std::optional disconnected_host; + std::optional unready_host; }; BackupCoordinationStageSync::State BackupCoordinationStageSync::readCurrentState( @@ -137,39 +137,45 @@ BackupCoordinationStageSync::State BackupCoordinationStageSync::readCurrentState return state; } + std::optional unready_host; + for (const auto & host : all_hosts) { if (!zk_nodes_set.contains("current|" + host + "|" + stage_to_wait)) { const String started_node_name = "started|" + host; const String alive_node_name = "alive|" + host; - const String alive_node_path = zookeeper_path + "/" + alive_node_name; - UnreadyHostState unready_host_state; - unready_host_state.started = zk_nodes_set.contains(started_node_name); - unready_host_state.alive = zk_nodes_set.contains(alive_node_name); - state.unready_hosts.emplace(host, unready_host_state); + bool started = zk_nodes_set.contains(started_node_name); + bool alive = zk_nodes_set.contains(alive_node_name); - if (!unready_host_state.alive && !state.disconnected_host) + if (!alive) { /// If the "alive" node doesn't exist then we don't have connection to the corresponding host. /// This node is ephemeral so probably it will be recreated soon. We use zookeeper retries to wait. /// In worst case when we won't manage to see the alive node for a long time we will just abort the backup. - state.disconnected_host = host; String message; - if (unready_host_state.started) + if (started) message = fmt::format("Lost connection to host {}", host); else message = fmt::format("No connection to host {} yet", host); if (!retries_ctl.isLastRetry()) message += ", will retry"; retries_ctl.setUserError(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, message); + state.disconnected_host = host; + return state; } + + if (!unready_host) + unready_host.emplace(UnreadyHost{.host = host, .started = started}); } } - if (state.disconnected_host || !state.unready_hosts.empty()) + if (unready_host) + { + state.unready_host = std::move(unready_host); return state; + } Strings results; for (const auto & host : all_hosts) @@ -215,12 +221,16 @@ Strings BackupCoordinationStageSync::waitImpl( } /// Analyze the current state of zk nodes. - if (state.error || state.disconnected_host || state.unready_hosts.empty()) - break; /// Error happened or everything is ready. + chassert(state.results || state.error || state.disconnected_host || state.unready_host); - /// Log that we will wait - const auto & unready_host = state.unready_hosts.begin()->first; - LOG_INFO(log, "Waiting on ZooKeeper watch for any node to be changed (currently waiting for host {})", unready_host); + if (state.results || state.error || state.disconnected_host) + break; /// Everything is ready or error happened. + + /// Log what we will wait. + const auto & unready_host = *state.unready_host; + LOG_INFO(log, "Waiting on ZooKeeper watch for any node to be changed (currently waiting for host {}{})", + unready_host.host, + (!unready_host.started ? " which didn't start the operation yet" : "")); /// Wait until `watch_callback` is called by ZooKeeper meaning that zk nodes have changed. { @@ -247,19 +257,19 @@ Strings BackupCoordinationStageSync::waitImpl( throw Exception(ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, "No connection to host {}", *state.disconnected_host); /// Something's unready, timeout is probably not enough. - if (!state.unready_hosts.empty()) + if (state.unready_host) { - const auto & [unready_host, unready_host_state] = *state.unready_hosts.begin(); + const auto & unready_host = *state.unready_host; throw Exception( ErrorCodes::FAILED_TO_SYNC_BACKUP_OR_RESTORE, "Waited for host {} too long (> {}){}", - unready_host, + unready_host.host, to_string(*timeout), - unready_host_state.started ? "" : ": Operation didn't start"); + unready_host.started ? "" : ": Operation didn't start"); } LOG_TRACE(log, "Everything is Ok. All hosts achieved stage {}", stage_to_wait); - return state.results; + return std::move(*state.results); } } From 49aad9c88e9ce4aea771b28a1fa8a4816cb481a4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Dec 2023 19:33:08 +0100 Subject: [PATCH 313/331] Maybe better --- docker/test/clickbench/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/clickbench/run.sh b/docker/test/clickbench/run.sh index 5d2312c22c5..255ff46b0bc 100755 --- a/docker/test/clickbench/run.sh +++ b/docker/test/clickbench/run.sh @@ -19,9 +19,9 @@ echo " profiles: default: allow_introspection_functions: 1 -" > /etc/clickhouse-server/allow_introspection_functions.yaml +" > /etc/clickhouse-server/users.d/allow_introspection_functions.yaml -config_logs_export_cluster /etc/clickhouse-server/users.d/system_logs_export.yaml +config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml clickhouse start From 4c1860b9b4e499c736f8e85cef98afe27e35db65 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Dec 2023 21:21:58 +0100 Subject: [PATCH 314/331] Fix a mistake --- docker/test/base/setup_export_logs.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 6e3721956c0..ea82e071112 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -21,7 +21,7 @@ EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:-"check_name, "} # trace_log needs more columns for symbolization EXTRA_COLUMNS_TRACE_LOG="${EXTRA_COLUMNS} symbols Array(LowCardinality(String)), lines Array(LowCardinality(String)), " -EXTRA_COLUMNS_EXPRESSION_TRACE_LOG="${EXTRA_COLUMNS_EXPRESSION}, arrayMap(x -> toLowCardinality(demangle(addressToSymbol(x))), trace) AS symbols, arrayMap(x -> toLowCardinality(addressToLine(x)), trace) AS lines" +EXTRA_COLUMNS_EXPRESSION_TRACE_LOG="${EXTRA_COLUMNS_EXPRESSION}, arrayMap(x -> demangle(addressToSymbol(x)), trace)::Array(LowCardinality(String)) AS symbols, arrayMap(x -> addressToLine(x), trace)::Array(LowCardinality(String)) AS lines" function __set_connection_args From 2099130bd2d66b8f2d9b87e27c833fdddaebc723 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Dec 2023 21:28:28 +0100 Subject: [PATCH 315/331] Enable text_log --- docker/test/clickbench/run.sh | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docker/test/clickbench/run.sh b/docker/test/clickbench/run.sh index 255ff46b0bc..921d2023fd7 100755 --- a/docker/test/clickbench/run.sh +++ b/docker/test/clickbench/run.sh @@ -21,6 +21,11 @@ profiles: allow_introspection_functions: 1 " > /etc/clickhouse-server/users.d/allow_introspection_functions.yaml +# Enable text_log +echo " +text_log: +" > /etc/clickhouse-server/config.d/text_log.yaml + config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml clickhouse start From d51aaddf12119e45525a12112557c0595422a2b3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Dec 2023 22:15:19 +0100 Subject: [PATCH 316/331] Use the local region --- docker/test/clickbench/create.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/clickbench/create.sql b/docker/test/clickbench/create.sql index 620bdf09331..a57faf35837 100644 --- a/docker/test/clickbench/create.sql +++ b/docker/test/clickbench/create.sql @@ -109,4 +109,4 @@ ATTACH TABLE hits UUID 'c449dfbf-ba06-4d13-abec-8396559eb955' ) ENGINE = MergeTree SETTINGS disk = disk(type = cache, path = '/dev/shm/clickhouse/', max_size = '16G', - disk = disk(type = web, endpoint = 'https://clickhouse-public-datasets.s3.amazonaws.com/web/')); + disk = disk(type = web, endpoint = 'https://clickhouse-datasets.s3.amazonaws.com/web/')); From 2043791ed76d040c8f05f5ad856bb599512da15c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Dec 2023 23:37:55 +0100 Subject: [PATCH 317/331] Fix typo --- docker/test/stateful/s3downloader | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stateful/s3downloader b/docker/test/stateful/s3downloader index 96f2aa96dd5..77601fb5af6 100755 --- a/docker/test/stateful/s3downloader +++ b/docker/test/stateful/s3downloader @@ -30,7 +30,7 @@ def build_url(base_url, dataset): return os.path.join(base_url, dataset, "partitions", AVAILABLE_DATASETS[dataset]) -def dowload_with_progress(url, path): +def download_with_progress(url, path): logging.info("Downloading from %s to temp path %s", url, path) for i in range(RETRIES_COUNT): try: @@ -110,7 +110,7 @@ if __name__ == "__main__": temp_archive_path = _get_temp_file_name() try: download_url_for_dataset = build_url(args.url_prefix, dataset) - dowload_with_progress(download_url_for_dataset, temp_archive_path) + download_with_progress(download_url_for_dataset, temp_archive_path) unpack_to_clickhouse_directory(temp_archive_path, args.clickhouse_data_path) except Exception as ex: logging.info("Some exception occured %s", str(ex)) From f11b90e7bbec13ba27989442e359f818d8e85088 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 13 Dec 2023 00:10:58 +0100 Subject: [PATCH 318/331] Allow buckets without List access --- src/Storages/StorageMergeTree.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index e9a0dd5fbf3..16f4122d605 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -829,8 +829,13 @@ void StorageMergeTree::loadDeduplicationLog() auto disk = getDisks()[0]; std::string path = fs::path(relative_data_path) / "deduplication_logs"; - deduplication_log = std::make_unique(path, settings->non_replicated_deduplication_window, format_version, disk); - deduplication_log->load(); + + /// If either there is already a deduplication log, or we will be able to use it. + if (disk->exists(path) || !disk->isReadOnly()) + { + deduplication_log = std::make_unique(path, settings->non_replicated_deduplication_window, format_version, disk); + deduplication_log->load(); + } } void StorageMergeTree::loadMutations() From 9f5299e118fc536f0ec9deb224c6ed6028362743 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 13 Dec 2023 00:11:10 +0100 Subject: [PATCH 319/331] Use a new bucket --- docker/test/clickbench/create.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/clickbench/create.sql b/docker/test/clickbench/create.sql index a57faf35837..9f18a47474b 100644 --- a/docker/test/clickbench/create.sql +++ b/docker/test/clickbench/create.sql @@ -109,4 +109,4 @@ ATTACH TABLE hits UUID 'c449dfbf-ba06-4d13-abec-8396559eb955' ) ENGINE = MergeTree SETTINGS disk = disk(type = cache, path = '/dev/shm/clickhouse/', max_size = '16G', - disk = disk(type = web, endpoint = 'https://clickhouse-datasets.s3.amazonaws.com/web/')); + disk = disk(type = web, endpoint = 'https://clickhouse-datasets-web.s3.us-east-1.amazonaws.com/')); From bb22ce6ec87063086ef5f6525d7d40dd72cfe88b Mon Sep 17 00:00:00 2001 From: Mikhail Koviazin Date: Wed, 13 Dec 2023 07:17:56 +0000 Subject: [PATCH 320/331] fix clickhouse-client invocation in 02327_capnproto_protobuf_empty_messages The test relies on `clickhouse-client` to be in `$PATH`, which is a wrong assumption. This commit makes it use `$CLICKHOUSE_CLIENT_BINARY` instead. --- .../0_stateless/02327_capnproto_protobuf_empty_messages.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh b/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh index 69e65112305..dfc0dedeaf1 100755 --- a/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh +++ b/tests/queries/0_stateless/02327_capnproto_protobuf_empty_messages.sh @@ -5,10 +5,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +USER_FILES_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') touch $USER_FILES_PATH/data.capnp -SCHEMADIR=$(clickhouse-client --query "select * from file('data.capnp', 'CapnProto', 'val1 char') settings format_schema='nonexist:Message'" 2>&1 | grep Exception | grep -oP "file \K.*(?=/nonexist.capnp)") +SCHEMADIR=$($CLICKHOUSE_CLIENT_BINARY --query "select * from file('data.capnp', 'CapnProto', 'val1 char') settings format_schema='nonexist:Message'" 2>&1 | grep Exception | grep -oP "file \K.*(?=/nonexist.capnp)") CLIENT_SCHEMADIR=$CURDIR/format_schemas SERVER_SCHEMADIR=test_02327 mkdir -p $SCHEMADIR/$SERVER_SCHEMADIR From 7762beaf6cd64f2553e81db5f7b1e5ba4ea4d8bd Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 13 Dec 2023 11:23:53 +0000 Subject: [PATCH 321/331] Fix: w/o replicas sync query result can vary --- .../test_parallel_replicas_over_distributed/test.py | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_parallel_replicas_over_distributed/test.py b/tests/integration/test_parallel_replicas_over_distributed/test.py index ecfc2ddea63..aecc0fcdcb8 100644 --- a/tests/integration/test_parallel_replicas_over_distributed/test.py +++ b/tests/integration/test_parallel_replicas_over_distributed/test.py @@ -129,6 +129,9 @@ def test_parallel_replicas_over_distributed( node = nodes[0] expected_result = f"6003\t-1999\t1999\t3\n" + # sync all replicas to get consistent result + node.query(f"SYSTEM SYNC REPLICA ON CLUSTER {cluster} {table_name}") + # parallel replicas assert ( node.query( @@ -143,11 +146,12 @@ def test_parallel_replicas_over_distributed( == expected_result ) - # sync all replicas to get consistent result by next distributed query - node.query(f"SYSTEM SYNC REPLICA ON CLUSTER {cluster} {table_name}") - # w/o parallel replicas assert ( - node.query(f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d") + node.query(f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d", + settings={ + "allow_experimental_parallel_reading_from_replicas": 0, + } + ) == expected_result ) From 7d9e9fd42eab9ae926d7bbd748627e8272c6afec Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 13 Dec 2023 11:38:41 +0000 Subject: [PATCH 322/331] Automatic style fix --- .../test_parallel_replicas_over_distributed/test.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_parallel_replicas_over_distributed/test.py b/tests/integration/test_parallel_replicas_over_distributed/test.py index aecc0fcdcb8..ebff0309a4f 100644 --- a/tests/integration/test_parallel_replicas_over_distributed/test.py +++ b/tests/integration/test_parallel_replicas_over_distributed/test.py @@ -148,10 +148,11 @@ def test_parallel_replicas_over_distributed( # w/o parallel replicas assert ( - node.query(f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d", + node.query( + f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d", settings={ "allow_experimental_parallel_reading_from_replicas": 0, - } + }, ) == expected_result ) From 8c2137e0c62721d6867cc252d9f2985e6b9d5339 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 13 Dec 2023 13:09:36 +0100 Subject: [PATCH 323/331] Revert "Merge pull request #57741 from ucasfl/negtive-position" This reverts commit 3d846800e0bdd94916ed8b8faf1c1bc7868ca933, reversing changes made to b31b4c932f78c8ea4f65657f88d65b494de15db0. --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 27 ++---- .../replaceForPositionalArguments.cpp | 25 +---- .../0_stateless/01162_strange_mutations.sh | 2 +- .../0_stateless/01798_having_push_down.sql | 3 +- .../02006_test_positional_arguments.reference | 94 ------------------- .../02006_test_positional_arguments.sql | 21 ----- .../02932_group_by_null_fuzzer.sql | 1 - 7 files changed, 14 insertions(+), 159 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index a3b461f32ea..1e63d5ca8e4 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2153,32 +2153,19 @@ void QueryAnalyzer::replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_ node_to_replace = &sort_node->getExpression(); auto * constant_node = (*node_to_replace)->as(); - - if (!constant_node - || (constant_node->getValue().getType() != Field::Types::UInt64 && constant_node->getValue().getType() != Field::Types::Int64)) + if (!constant_node || constant_node->getValue().getType() != Field::Types::UInt64) continue; - UInt64 pos; - if (constant_node->getValue().getType() == Field::Types::UInt64) - { - pos = constant_node->getValue().get(); - } - else // Int64 - { - auto value = constant_node->getValue().get(); - pos = value > 0 ? value : projection_nodes.size() + value + 1; - } - - - if (!pos || pos > projection_nodes.size()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, + UInt64 positional_argument_number = constant_node->getValue().get(); + if (positional_argument_number == 0 || positional_argument_number > projection_nodes.size()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional argument number {} is out of bounds. Expected in range [1, {}]. In scope {}", - pos, + positional_argument_number, projection_nodes.size(), scope.scope_node->formatASTForErrorMessage()); - *node_to_replace = projection_nodes[--pos]; + --positional_argument_number; + *node_to_replace = projection_nodes[positional_argument_number]; } } diff --git a/src/Interpreters/replaceForPositionalArguments.cpp b/src/Interpreters/replaceForPositionalArguments.cpp index 8306da17f52..241dd7cf92c 100644 --- a/src/Interpreters/replaceForPositionalArguments.cpp +++ b/src/Interpreters/replaceForPositionalArguments.cpp @@ -27,29 +27,14 @@ bool replaceForPositionalArguments(ASTPtr & argument, const ASTSelectQuery * sel return false; auto which = ast_literal->value.getType(); - if (which != Field::Types::UInt64 && which != Field::Types::Int64) + if (which != Field::Types::UInt64) return false; - UInt64 pos; - - if (which == Field::Types::UInt64) - { - pos = ast_literal->value.get(); - } - else if (which == Field::Types::Int64) - { - auto value = ast_literal->value.get(); - pos = value > 0 ? value : columns.size() + value + 1; - } - else - { - return false; - } - - + auto pos = ast_literal->value.get(); if (!pos || pos > columns.size()) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Positional argument out of bounds: {} (expected in range [1, {}]", pos, columns.size()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Positional argument out of bounds: {} (expected in range [1, {}]", + pos, columns.size()); const auto & column = columns[--pos]; if (typeid_cast(column.get()) || typeid_cast(column.get())) diff --git a/tests/queries/0_stateless/01162_strange_mutations.sh b/tests/queries/0_stateless/01162_strange_mutations.sh index f6b31847c1e..eea9ea5f7e5 100755 --- a/tests/queries/0_stateless/01162_strange_mutations.sh +++ b/tests/queries/0_stateless/01162_strange_mutations.sh @@ -28,7 +28,7 @@ do $CLICKHOUSE_CLIENT -q "CREATE TABLE test ENGINE=$engine AS SELECT number + 100 AS n, 0 AS test FROM numbers(50)" 2>&1| grep -Ev "Removing leftovers from table|removed by another replica" $CLICKHOUSE_CLIENT -q "select count(), sum(n), sum(test) from test" if [[ $engine == *"ReplicatedMergeTree"* ]]; then - $CLICKHOUSE_CLIENT --enable_positional_arguments=0 -q "ALTER TABLE test + $CLICKHOUSE_CLIENT -q "ALTER TABLE test UPDATE test = (SELECT groupArray(id) FROM t1 GROUP BY 'dummy')[n - 99] WHERE 1" 2>&1| grep -Fa "DB::Exception: " | grep -Fv "statement with subquery may be nondeterministic" $CLICKHOUSE_CLIENT --allow_nondeterministic_mutations=1 --mutations_sync=1 -q "ALTER TABLE test UPDATE test = (SELECT groupArray(id) FROM t1)[n - 99] WHERE 1" diff --git a/tests/queries/0_stateless/01798_having_push_down.sql b/tests/queries/0_stateless/01798_having_push_down.sql index c0c3447f5ab..b3a77c8f5b5 100644 --- a/tests/queries/0_stateless/01798_having_push_down.sql +++ b/tests/queries/0_stateless/01798_having_push_down.sql @@ -8,12 +8,11 @@ SELECT sum(c0 = 0), min(c0 + 1), sum(c0 + 2) FROM t_having GROUP BY c0 HAVING c0 = 0 SETTINGS enable_optimize_predicate_expression=0; -SET enable_positional_arguments=0; - SELECT c0 + -1, sum(intDivOrZero(intDivOrZero(NULL, NULL), '2'), intDivOrZero(10000000000., intDivOrZero(intDivOrZero(intDivOrZero(NULL, NULL), 10), NULL))) FROM t_having GROUP BY c0 = 2, c0 = 10, intDivOrZero(intDivOrZero(intDivOrZero(NULL, NULL), NULL), NULL), c0 HAVING c0 = 2 SETTINGS enable_optimize_predicate_expression = 0; SELECT sum(c0 + 257) FROM t_having GROUP BY c0 = -9223372036854775808, NULL, -2147483649, c0 HAVING c0 = -9223372036854775808 SETTINGS enable_optimize_predicate_expression = 0; +SET enable_positional_arguments=0; SELECT c0 + -2, c0 + -9223372036854775807, c0 = NULL FROM t_having GROUP BY c0 = 0.9998999834060669, 1023, c0 HAVING c0 = 0.9998999834060669 SETTINGS enable_optimize_predicate_expression = 0; DROP TABLE t_having; diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.reference b/tests/queries/0_stateless/02006_test_positional_arguments.reference index 079bd071103..40100e8d5be 100644 --- a/tests/queries/0_stateless/02006_test_positional_arguments.reference +++ b/tests/queries/0_stateless/02006_test_positional_arguments.reference @@ -3,50 +3,18 @@ select x3, x2, x1 from test order by 1; 1 100 100 10 1 10 100 10 1 -select x3, x2, x1 from test order by -3; -1 100 100 -10 1 10 -100 10 1 select x3, x2, x1 from test order by x3; 1 100 100 10 1 10 100 10 1 -select x3, x2, x1 from test order by 3; -100 10 1 -10 1 10 -1 100 100 -select x3, x2, x1 from test order by -1; -100 10 1 -10 1 10 -1 100 100 -select x3, x2, x1 from test order by x1; -100 10 1 -10 1 10 -1 100 100 select x3, x2, x1 from test order by 1 desc; 100 10 1 10 1 10 1 100 100 -select x3, x2, x1 from test order by -3 desc; -100 10 1 -10 1 10 -1 100 100 select x3, x2, x1 from test order by x3 desc; 100 10 1 10 1 10 1 100 100 -select x3, x2, x1 from test order by 3 desc; -1 100 100 -10 1 10 -100 10 1 -select x3, x2, x1 from test order by -1 desc; -1 100 100 -10 1 10 -100 10 1 -select x3, x2, x1 from test order by x1 desc; -1 100 100 -10 1 10 -100 10 1 insert into test values (1, 10, 100), (10, 1, 10), (100, 100, 1); select x3, x2 from test group by x3, x2 order by x3; 1 100 @@ -86,20 +54,6 @@ SELECT x1 FROM test ORDER BY x3 + 1 ASC -explain syntax select x3, x2, x1 from test order by -1; -SELECT - x3, - x2, - x1 -FROM test -ORDER BY x1 ASC -explain syntax select x3 + 1, x2, x1 from test order by -1; -SELECT - x3 + 1, - x2, - x1 -FROM test -ORDER BY x1 ASC explain syntax select x3, x3 - x2, x2, x1 from test order by 2; SELECT x3, @@ -108,14 +62,6 @@ SELECT x1 FROM test ORDER BY x3 - x2 ASC -explain syntax select x3, x3 - x2, x2, x1 from test order by -2; -SELECT - x3, - x3 - x2, - x2, - x1 -FROM test -ORDER BY x2 ASC explain syntax select x3, if(x3 > 10, x3, plus(x1, x2)), x1 + x2 from test order by 2; SELECT x3, @@ -123,28 +69,12 @@ SELECT x1 + x2 FROM test ORDER BY if(x3 > 10, x3, x1 + x2) ASC -explain syntax select x3, if(x3 > 10, x3, plus(x1, x2)), x1 + x2 from test order by -2; -SELECT - x3, - if(x3 > 10, x3, x1 + x2), - x1 + x2 -FROM test -ORDER BY if(x3 > 10, x3, x1 + x2) ASC explain syntax select max(x1), x2 from test group by 2 order by 1, 2; SELECT max(x1), x2 FROM test GROUP BY x2 -ORDER BY - max(x1) ASC, - x2 ASC -explain syntax select max(x1), x2 from test group by -1 order by -2, -1; -SELECT - max(x1), - x2 -FROM test -GROUP BY x2 ORDER BY max(x1) ASC, x2 ASC @@ -153,34 +83,16 @@ SELECT 1 + greatest(x1, 1), x2 FROM test -GROUP BY - 1 + greatest(x1, 1), - x2 -explain syntax select 1 + greatest(x1, 1), x2 from test group by -2, -1; -SELECT - 1 + greatest(x1, 1), - x2 -FROM test GROUP BY 1 + greatest(x1, 1), x2 select max(x1), x2 from test group by 1, 2; -- { serverError 43, 184 } select 1 + max(x1), x2 from test group by 1, 2; -- { serverError 43, 184 } -select max(x1), x2 from test group by -2, -1; -- { serverError 43, 184 } -select 1 + max(x1), x2 from test group by -2, -1; -- { serverError 43, 184 } explain syntax select x1 + x3, x3 from test group by 1, 2; SELECT x1 + x3, x3 FROM test -GROUP BY - x1 + x3, - x3 -explain syntax select x1 + x3, x3 from test group by -2, -1; -SELECT - x1 + x3, - x3 -FROM test GROUP BY x1 + x3, x3 @@ -190,14 +102,8 @@ select x1, x1 * 2, max(x2), max(x3) from test2 group by 2, 1, x1 order by 1, 2, 1 2 10 100 10 20 1 10 100 200 100 1 -select x1, x1 * 2, max(x2), max(x3) from test2 group by 2, 1, x1 order by 1, 2, -1 desc, -2 asc; -1 2 10 100 -10 20 1 10 -100 200 100 1 select a, b, c, d, e, f from (select 44 a, 88 b, 13 c, 14 d, 15 e, 16 f) t group by 1,2,3,4,5,6 order by a; 44 88 13 14 15 16 -select a, b, c, d, e, f from (select 44 a, 88 b, 13 c, 14 d, 15 e, 16 f) t group by 1,2,3,-3,-2,-1 order by a; -44 88 13 14 15 16 explain syntax select plus(1, 1) as a group by a; SELECT 1 + 1 AS a GROUP BY a diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.sql b/tests/queries/0_stateless/02006_test_positional_arguments.sql index 6f427e0298d..159ad6bd427 100644 --- a/tests/queries/0_stateless/02006_test_positional_arguments.sql +++ b/tests/queries/0_stateless/02006_test_positional_arguments.sql @@ -9,21 +9,11 @@ insert into test values (1, 10, 100), (10, 1, 10), (100, 100, 1); -- { echo } select x3, x2, x1 from test order by 1; -select x3, x2, x1 from test order by -3; select x3, x2, x1 from test order by x3; -select x3, x2, x1 from test order by 3; -select x3, x2, x1 from test order by -1; -select x3, x2, x1 from test order by x1; - select x3, x2, x1 from test order by 1 desc; -select x3, x2, x1 from test order by -3 desc; select x3, x2, x1 from test order by x3 desc; -select x3, x2, x1 from test order by 3 desc; -select x3, x2, x1 from test order by -1 desc; -select x3, x2, x1 from test order by x1 desc; - insert into test values (1, 10, 100), (10, 1, 10), (100, 100, 1); select x3, x2 from test group by x3, x2 order by x3; select x3, x2 from test group by 1, 2 order by x3; @@ -35,32 +25,21 @@ select x1, x2, x3 from test order by 3 limit 1 by 1; explain syntax select x3, x2, x1 from test order by 1; explain syntax select x3 + 1, x2, x1 from test order by 1; -explain syntax select x3, x2, x1 from test order by -1; -explain syntax select x3 + 1, x2, x1 from test order by -1; explain syntax select x3, x3 - x2, x2, x1 from test order by 2; -explain syntax select x3, x3 - x2, x2, x1 from test order by -2; explain syntax select x3, if(x3 > 10, x3, plus(x1, x2)), x1 + x2 from test order by 2; -explain syntax select x3, if(x3 > 10, x3, plus(x1, x2)), x1 + x2 from test order by -2; explain syntax select max(x1), x2 from test group by 2 order by 1, 2; -explain syntax select max(x1), x2 from test group by -1 order by -2, -1; explain syntax select 1 + greatest(x1, 1), x2 from test group by 1, 2; -explain syntax select 1 + greatest(x1, 1), x2 from test group by -2, -1; select max(x1), x2 from test group by 1, 2; -- { serverError 43, 184 } select 1 + max(x1), x2 from test group by 1, 2; -- { serverError 43, 184 } -select max(x1), x2 from test group by -2, -1; -- { serverError 43, 184 } -select 1 + max(x1), x2 from test group by -2, -1; -- { serverError 43, 184 } explain syntax select x1 + x3, x3 from test group by 1, 2; -explain syntax select x1 + x3, x3 from test group by -2, -1; create table test2(x1 Int, x2 Int, x3 Int) engine=Memory; insert into test2 values (1, 10, 100), (10, 1, 10), (100, 100, 1); select x1, x1 * 2, max(x2), max(x3) from test2 group by 2, 1, x1 order by 1, 2, 4 desc, 3 asc; -select x1, x1 * 2, max(x2), max(x3) from test2 group by 2, 1, x1 order by 1, 2, -1 desc, -2 asc; select a, b, c, d, e, f from (select 44 a, 88 b, 13 c, 14 d, 15 e, 16 f) t group by 1,2,3,4,5,6 order by a; -select a, b, c, d, e, f from (select 44 a, 88 b, 13 c, 14 d, 15 e, 16 f) t group by 1,2,3,-3,-2,-1 order by a; explain syntax select plus(1, 1) as a group by a; select substr('aaaaaaaaaaaaaa', 8) as a group by a order by a; diff --git a/tests/queries/0_stateless/02932_group_by_null_fuzzer.sql b/tests/queries/0_stateless/02932_group_by_null_fuzzer.sql index 603c7783ef8..0c28c120d40 100644 --- a/tests/queries/0_stateless/02932_group_by_null_fuzzer.sql +++ b/tests/queries/0_stateless/02932_group_by_null_fuzzer.sql @@ -1,6 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/43202 -- Queries are generated by the fuzzer, so don't expect them to make sense -SET enable_positional_arguments=0; SELECT NULL, '' FROM (SELECT toNullable(''), NULL AS key GROUP BY GROUPING SETS ((NULL))) AS s1 ALL LEFT JOIN (SELECT '' AS key, NULL AS value GROUP BY GROUPING SETS (('')) WITH TOTALS UNION ALL SELECT NULL AS key, toNullable(NULL) AS value GROUP BY '', NULL, '' WITH TOTALS) AS s2 USING (key); SELECT NULL GROUP BY NULL WITH TOTALS; SELECT 1048575, NULL, b FROM (SELECT '25.5' AS a, NULL, NULL AS b GROUP BY GROUPING SETS ((0.0001)) WITH TOTALS) AS js1 ANY RIGHT JOIN (SELECT NULL AS a, NULL AS b WHERE NULL GROUP BY NULL, -9223372036854775807 WITH CUBE WITH TOTALS UNION ALL SELECT NULL AS a, NULL AS b GROUP BY 1, '21474836.46' WITH TOTALS) AS js2 USING (a, b) ORDER BY nan DESC NULLS LAST, '9223372036854775807' DESC NULLS LAST, a ASC NULLS LAST; From 090d412d7cc37104ba90355c880a357fbd34e091 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 13 Dec 2023 13:14:00 +0100 Subject: [PATCH 324/331] Add tests for 46628 --- .../02943_positional_arguments_bugs.reference | 2 ++ .../02943_positional_arguments_bugs.sql | 23 +++++++++++++++++++ 2 files changed, 25 insertions(+) create mode 100644 tests/queries/0_stateless/02943_positional_arguments_bugs.reference create mode 100644 tests/queries/0_stateless/02943_positional_arguments_bugs.sql diff --git a/tests/queries/0_stateless/02943_positional_arguments_bugs.reference b/tests/queries/0_stateless/02943_positional_arguments_bugs.reference new file mode 100644 index 00000000000..702e1261186 --- /dev/null +++ b/tests/queries/0_stateless/02943_positional_arguments_bugs.reference @@ -0,0 +1,2 @@ +45 1 +processed 99 0 diff --git a/tests/queries/0_stateless/02943_positional_arguments_bugs.sql b/tests/queries/0_stateless/02943_positional_arguments_bugs.sql new file mode 100644 index 00000000000..b8cf73da42d --- /dev/null +++ b/tests/queries/0_stateless/02943_positional_arguments_bugs.sql @@ -0,0 +1,23 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/46628 +DROP TABLE IF EXISTS t; +CREATE TABLE t +( + `n` int +) + ENGINE = MergeTree + ORDER BY n AS +SELECT * +FROM numbers(10); + +SELECT + sum(n), + 1 AS x +FROM t +GROUP BY x; + +SELECT + 'processed' AS type, + max(number) AS max_date, + min(number) AS min_date +FROM numbers(100) +GROUP BY type; From 54abbf146d1d7dfcdab8da50a4292e2079d40bc8 Mon Sep 17 00:00:00 2001 From: Johnny <9611008+johnnymatthews@users.noreply.github.com> Date: Wed, 13 Dec 2023 09:04:07 -0400 Subject: [PATCH 325/331] Update into-outfile.md --- docs/en/sql-reference/statements/select/into-outfile.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/into-outfile.md b/docs/en/sql-reference/statements/select/into-outfile.md index 985f5e25b05..5b7196f13e3 100644 --- a/docs/en/sql-reference/statements/select/into-outfile.md +++ b/docs/en/sql-reference/statements/select/into-outfile.md @@ -12,7 +12,7 @@ Compressed files are supported. Compression type is detected by the extension of **Syntax** ```sql -SELECT INTO OUTFILE file_name [AND STDOUT] [APPEND] [COMPRESSION type [LEVEL level]] +SELECT INTO OUTFILE file_name [AND STDOUT] [APPEND | TRUNCATE] [COMPRESSION type [LEVEL level]] ``` `file_name` and `type` are string literals. Supported compression types are: `'none'`, `'gzip'`, `'deflate'`, `'br'`, `'xz'`, `'zstd'`, `'lz4'`, `'bz2'`. From c165be76abfb51a2dca9ee9a7baec9e46ce52d34 Mon Sep 17 00:00:00 2001 From: Igor Nikonov <954088+devcrafter@users.noreply.github.com> Date: Wed, 13 Dec 2023 14:42:06 +0100 Subject: [PATCH 326/331] Parallel replicas: friendly settings (#57542) --- docker/test/stateful/run.sh | 2 +- .../ClusterProxy/executeQuery.cpp | 38 ++++++++++++++++++- src/Interpreters/ClusterProxy/executeQuery.h | 3 +- src/Interpreters/Context.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 22 +++++------ src/Storages/StorageReplicatedMergeTree.cpp | 17 +++------ src/Storages/StorageReplicatedMergeTree.h | 4 +- .../test.py | 1 - .../test.py | 3 -- .../test.py | 1 - .../test.py | 2 - ...arallel_reading_from_replicas_benchmark.sh | 1 - .../02731_parallel_replicas_join_subquery.sql | 1 - ...arallel_replicas_bug_chunkinfo_not_set.sql | 2 +- ...764_parallel_replicas_plain_merge_tree.sql | 2 +- ...02765_parallel_replicas_final_modifier.sql | 2 +- ...9_parallel_replicas_unavailable_shards.sql | 2 +- ...02771_parallel_replicas_analyzer.reference | 4 +- .../02771_parallel_replicas_analyzer.sql | 3 +- ...lel_replicas_trivial_count_optimization.sh | 4 -- ...84_parallel_replicas_automatic_decision.sh | 1 - ...rallel_replicas_automatic_decision_join.sh | 1 - ...02811_parallel_replicas_prewhere_count.sql | 1 - ...835_parallel_replicas_over_distributed.sql | 8 ++-- .../02841_parallel_replicas_summary.sh | 2 - .../02861_index_set_incorrect_args.sql | 2 +- ...69_parallel_replicas_read_from_several.sql | 2 +- ...parallel_replicas_cluster_all_replicas.sql | 2 +- .../02875_parallel_replicas_remote.sql | 2 +- .../02898_parallel_replicas_progress_bar.sql | 2 +- .../02901_parallel_replicas_rollup.sh | 2 - ...02935_parallel_replicas_settings.reference | 4 ++ .../02935_parallel_replicas_settings.sql | 35 +++++++++++++++++ .../1_stateful/00177_memory_bound_merging.sh | 6 +-- 34 files changed, 114 insertions(+), 72 deletions(-) create mode 100644 tests/queries/0_stateless/02935_parallel_replicas_settings.reference create mode 100644 tests/queries/0_stateless/02935_parallel_replicas_settings.sql diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index a0def50bfb5..806b57c4616 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -151,7 +151,7 @@ function run_tests() set +e if [[ -n "$USE_PARALLEL_REPLICAS" ]] && [[ "$USE_PARALLEL_REPLICAS" -eq 1 ]]; then - clickhouse-test --client="clickhouse-client --use_hedged_requests=0 --allow_experimental_parallel_reading_from_replicas=1 --parallel_replicas_for_non_replicated_merge_tree=1 \ + clickhouse-test --client="clickhouse-client --allow_experimental_parallel_reading_from_replicas=1 --parallel_replicas_for_non_replicated_merge_tree=1 \ --max_parallel_replicas=100 --cluster_for_parallel_replicas='parallel_replicas'" \ -j 2 --testname --shard --zookeeper --check-zookeeper-session --no-stateless --no-parallel-replicas --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \ "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 8a2f7e3205a..f3b7e371f38 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -30,6 +30,7 @@ namespace ErrorCodes { extern const int TOO_LARGE_DISTRIBUTED_DEPTH; extern const int LOGICAL_ERROR; + extern const int CLUSTER_DOESNT_EXIST; } namespace ClusterProxy @@ -322,11 +323,44 @@ void executeQueryWithParallelReplicas( SelectStreamFactory & stream_factory, const ASTPtr & query_ast, ContextPtr context, - std::shared_ptr storage_limits, - const ClusterPtr & not_optimized_cluster) + std::shared_ptr storage_limits) { const auto & settings = context->getSettingsRef(); + + /// check cluster for parallel replicas + if (settings.cluster_for_parallel_replicas.value.empty()) + { + throw Exception( + ErrorCodes::CLUSTER_DOESNT_EXIST, + "Reading in parallel from replicas is enabled but cluster to execute query is not provided. Please set " + "'cluster_for_parallel_replicas' setting"); + } + auto not_optimized_cluster = context->getCluster(settings.cluster_for_parallel_replicas); + auto new_context = Context::createCopy(context); + + /// check hedged connections setting + if (settings.use_hedged_requests.value) + { + if (settings.use_hedged_requests.changed) + { + LOG_WARNING( + &Poco::Logger::get("executeQueryWithParallelReplicas"), + "Setting 'use_hedged_requests' explicitly with enabled 'allow_experimental_parallel_reading_from_replicas' has no effect. " + "Hedged connections are not used for parallel reading from replicas"); + } + else + { + LOG_INFO( + &Poco::Logger::get("executeQueryWithParallelReplicas"), + "Disabling 'use_hedged_requests' in favor of 'allow_experimental_parallel_reading_from_replicas'. Hedged connections are " + "not used for parallel reading from replicas"); + } + + /// disable hedged connections -> parallel replicas uses own logic to choose replicas + new_context->setSetting("use_hedged_requests", Field{false}); + } + auto scalars = new_context->hasQueryContext() ? new_context->getQueryContext()->getScalars() : Scalars{}; UInt64 shard_num = 0; /// shard_num is 1-based, so 0 - no shard specified diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index 7ffaa3ae62c..2149d8c1640 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -71,8 +71,7 @@ void executeQueryWithParallelReplicas( SelectStreamFactory & stream_factory, const ASTPtr & query_ast, ContextPtr context, - std::shared_ptr storage_limits, - const ClusterPtr & not_optimized_cluster); + std::shared_ptr storage_limits); } } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index f0f20e171af..79cfe9a9546 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -5020,7 +5020,7 @@ Context::ParallelReplicasMode Context::getParallelReplicasMode() const if (!settings_ref.parallel_replicas_custom_key.value.empty()) return CUSTOM_KEY; - if (settings_ref.allow_experimental_parallel_reading_from_replicas > 0 && !settings_ref.use_hedged_requests) + if (settings_ref.allow_experimental_parallel_reading_from_replicas > 0) return READ_TASKS; return SAMPLE_KEY; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index e9a0dd5fbf3..22d72902e8d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -211,17 +211,12 @@ void StorageMergeTree::read( { if (local_context->canUseParallelReplicasOnInitiator() && local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree) { - auto table_id = getStorageID(); - + const auto table_id = getStorageID(); const auto & modified_query_ast = ClusterProxy::rewriteSelectQuery( local_context, query_info.query, table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); - String cluster_for_parallel_replicas = local_context->getSettingsRef().cluster_for_parallel_replicas; - auto cluster = local_context->getCluster(cluster_for_parallel_replicas); - Block header; - if (local_context->getSettingsRef().allow_experimental_analyzer) header = InterpreterSelectQueryAnalyzer::getSampleBlock(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()); else @@ -240,17 +235,22 @@ void StorageMergeTree::read( select_stream_factory, modified_query_ast, local_context, - query_info.storage_limits, - cluster); + query_info.storage_limits); } else { const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower() && local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree; if (auto plan = reader.read( - column_names, storage_snapshot, query_info, - local_context, max_block_size, num_streams, - processed_stage, nullptr, enable_parallel_reading)) + column_names, + storage_snapshot, + query_info, + local_context, + max_block_size, + num_streams, + processed_stage, + nullptr, + enable_parallel_reading)) query_plan = std::move(*plan); } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 4fb21705534..307870aaf4c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5338,7 +5338,7 @@ void StorageReplicatedMergeTree::read( return readLocalSequentialConsistencyImpl(query_plan, column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams); if (local_context->canUseParallelReplicasOnInitiator()) - return readParallelReplicasImpl(query_plan, column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams); + return readParallelReplicasImpl(query_plan, column_names, storage_snapshot, query_info, local_context, processed_stage); readLocalImpl(query_plan, column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams); } @@ -5367,18 +5367,11 @@ void StorageReplicatedMergeTree::readParallelReplicasImpl( const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, ContextPtr local_context, - QueryProcessingStage::Enum processed_stage, - const size_t /*max_block_size*/, - const size_t /*num_streams*/) + QueryProcessingStage::Enum processed_stage) { - auto table_id = getStorageID(); - - auto scalars = local_context->hasQueryContext() ? local_context->getQueryContext()->getScalars() : Scalars{}; - String cluster_for_parallel_replicas = local_context->getSettingsRef().cluster_for_parallel_replicas; - auto parallel_replicas_cluster = local_context->getCluster(cluster_for_parallel_replicas); - ASTPtr modified_query_ast; Block header; + if (local_context->getSettingsRef().allow_experimental_analyzer) { auto modified_query_tree = buildQueryTreeForShard(query_info, query_info.query_tree); @@ -5389,6 +5382,7 @@ void StorageReplicatedMergeTree::readParallelReplicasImpl( } else { + const auto table_id = getStorageID(); modified_query_ast = ClusterProxy::rewriteSelectQuery(local_context, query_info.query, table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); header @@ -5407,8 +5401,7 @@ void StorageReplicatedMergeTree::readParallelReplicasImpl( select_stream_factory, modified_query_ast, local_context, - query_info.storage_limits, - parallel_replicas_cluster); + query_info.storage_limits); } void StorageReplicatedMergeTree::readLocalImpl( diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index a8ab8eb7013..159828effcf 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -582,9 +582,7 @@ private: const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, ContextPtr local_context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - size_t num_streams); + QueryProcessingStage::Enum processed_stage); template void foreachActiveParts(Func && func, bool select_sequential_consistency) const; diff --git a/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py b/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py index 7e12da956ea..8af7bb12595 100644 --- a/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py +++ b/tests/integration/test_parallel_replicas_distributed_read_from_all/test.py @@ -119,7 +119,6 @@ def test_read_equally_from_each_replica(start_cluster, prefer_localhost_replica) "allow_experimental_parallel_reading_from_replicas": 2, "prefer_localhost_replica": prefer_localhost_replica, "max_parallel_replicas": 3, - "use_hedged_requests": 0, }, ) == expected_result diff --git a/tests/integration/test_parallel_replicas_distributed_skip_shards/test.py b/tests/integration/test_parallel_replicas_distributed_skip_shards/test.py index 315a9781c8b..af114ade2d7 100644 --- a/tests/integration/test_parallel_replicas_distributed_skip_shards/test.py +++ b/tests/integration/test_parallel_replicas_distributed_skip_shards/test.py @@ -84,7 +84,6 @@ def test_skip_unavailable_shards(start_cluster, prefer_localhost_replica): settings={ "allow_experimental_parallel_reading_from_replicas": 2, "max_parallel_replicas": 3, - "use_hedged_requests": 0, "prefer_localhost_replica": prefer_localhost_replica, "skip_unavailable_shards": 1, "connections_with_failover_max_tries": 0, # just don't wait for unavailable replicas @@ -119,7 +118,6 @@ def test_error_on_unavailable_shards(start_cluster, prefer_localhost_replica): settings={ "allow_experimental_parallel_reading_from_replicas": 2, "max_parallel_replicas": 3, - "use_hedged_requests": 0, "prefer_localhost_replica": prefer_localhost_replica, "skip_unavailable_shards": 0, }, @@ -155,7 +153,6 @@ def test_no_unavailable_shards(start_cluster, skip_unavailable_shards): settings={ "allow_experimental_parallel_reading_from_replicas": 2, "max_parallel_replicas": 3, - "use_hedged_requests": 0, "prefer_localhost_replica": 0, "skip_unavailable_shards": skip_unavailable_shards, }, diff --git a/tests/integration/test_parallel_replicas_over_distributed/test.py b/tests/integration/test_parallel_replicas_over_distributed/test.py index ecfc2ddea63..00b95965b65 100644 --- a/tests/integration/test_parallel_replicas_over_distributed/test.py +++ b/tests/integration/test_parallel_replicas_over_distributed/test.py @@ -137,7 +137,6 @@ def test_parallel_replicas_over_distributed( "allow_experimental_parallel_reading_from_replicas": 2, "prefer_localhost_replica": prefer_localhost_replica, "max_parallel_replicas": max_parallel_replicas, - "use_hedged_requests": 0, }, ) == expected_result diff --git a/tests/integration/test_parallel_replicas_skip_shards/test.py b/tests/integration/test_parallel_replicas_skip_shards/test.py index 3df80ba061e..a18c82a53a9 100644 --- a/tests/integration/test_parallel_replicas_skip_shards/test.py +++ b/tests/integration/test_parallel_replicas_skip_shards/test.py @@ -38,7 +38,6 @@ def test_skip_unavailable_shards(start_cluster): settings={ "allow_experimental_parallel_reading_from_replicas": 2, "max_parallel_replicas": 3, - "use_hedged_requests": 0, "skip_unavailable_shards": 1, # "async_socket_for_remote" : 0, # "async_query_sending_for_remote" : 0, @@ -65,7 +64,6 @@ def test_error_on_unavailable_shards(start_cluster): settings={ "allow_experimental_parallel_reading_from_replicas": 2, "max_parallel_replicas": 3, - "use_hedged_requests": 0, "skip_unavailable_shards": 0, }, ) diff --git a/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh b/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh index 941f024825a..bc90f4b2c11 100755 --- a/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh +++ b/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh @@ -17,7 +17,6 @@ opts=( --allow_experimental_parallel_reading_from_replicas 1 --parallel_replicas_for_non_replicated_merge_tree 1 --max_parallel_replicas 3 - --use_hedged_requests 0 --cluster_for_parallel_replicas parallel_replicas --iterations 1 diff --git a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql index 29c20980c14..fa40c96048c 100644 --- a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql +++ b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql @@ -23,7 +23,6 @@ SET allow_experimental_analyzer = 0; SET max_parallel_replicas = 3; SET prefer_localhost_replica = 1; SET cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; -SET use_hedged_requests = 0; SET joined_subquery_requires_alias = 0; SELECT '=============== INNER QUERY (NO PARALLEL) ==============='; diff --git a/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.sql b/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.sql index 2ea2cecc7b5..5ec0a1fcc31 100644 --- a/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.sql +++ b/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.sql @@ -18,7 +18,7 @@ INSERT INTO join_inner_table__fuzz_1 SELECT FROM generateRandom('number Int64, value1 String, value2 String, time Int64', 1, 10, 2) LIMIT 100; -SET max_parallel_replicas = 3, prefer_localhost_replica = 1, use_hedged_requests = 0, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_parallel_reading_from_replicas = 1; +SET max_parallel_replicas = 3, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_parallel_reading_from_replicas = 1; -- SELECT query will write a Warning to the logs SET send_logs_level='error'; diff --git a/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql b/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql index aaf68dfd300..9caa6f76e89 100644 --- a/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql +++ b/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql @@ -1,7 +1,7 @@ CREATE TABLE IF NOT EXISTS parallel_replicas_plain (x String) ENGINE=MergeTree() ORDER BY x; INSERT INTO parallel_replicas_plain SELECT toString(number) FROM numbers(10); -SET max_parallel_replicas=3, allow_experimental_parallel_reading_from_replicas=1, use_hedged_requests=0, cluster_for_parallel_replicas='parallel_replicas'; +SET max_parallel_replicas=3, allow_experimental_parallel_reading_from_replicas=1, cluster_for_parallel_replicas='parallel_replicas'; SET send_logs_level='error'; SET parallel_replicas_for_non_replicated_merge_tree = 0; diff --git a/tests/queries/0_stateless/02765_parallel_replicas_final_modifier.sql b/tests/queries/0_stateless/02765_parallel_replicas_final_modifier.sql index f447051e1e5..6c121802b06 100644 --- a/tests/queries/0_stateless/02765_parallel_replicas_final_modifier.sql +++ b/tests/queries/0_stateless/02765_parallel_replicas_final_modifier.sql @@ -2,7 +2,7 @@ CREATE TABLE IF NOT EXISTS parallel_replicas_final (x String) ENGINE=ReplacingMe INSERT INTO parallel_replicas_final SELECT toString(number) FROM numbers(10); -SET max_parallel_replicas=3, allow_experimental_parallel_reading_from_replicas=1, use_hedged_requests=0, cluster_for_parallel_replicas='parallel_replicas'; +SET max_parallel_replicas=3, allow_experimental_parallel_reading_from_replicas=1, cluster_for_parallel_replicas='parallel_replicas'; SET parallel_replicas_for_non_replicated_merge_tree = 1; SELECT * FROM parallel_replicas_final FINAL FORMAT Null; diff --git a/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql index 020a429c109..38d592201e3 100644 --- a/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql +++ b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql @@ -4,7 +4,7 @@ INSERT INTO test_parallel_replicas_unavailable_shards SELECT * FROM numbers(10); SYSTEM FLUSH LOGS; -SET allow_experimental_parallel_reading_from_replicas=2, max_parallel_replicas=11, use_hedged_requests=0, cluster_for_parallel_replicas='parallel_replicas', parallel_replicas_for_non_replicated_merge_tree=1; +SET allow_experimental_parallel_reading_from_replicas=2, max_parallel_replicas=11, cluster_for_parallel_replicas='parallel_replicas', parallel_replicas_for_non_replicated_merge_tree=1; SET send_logs_level='error'; SELECT count() FROM test_parallel_replicas_unavailable_shards WHERE NOT ignore(*); diff --git a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference index f688db940d9..35573110550 100644 --- a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference +++ b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference @@ -8,5 +8,5 @@ 5935810273536892891 7885388429666205427 8124171311239967992 -1 1 -- Simple query with analyzer and pure parallel replicas\nSELECT number\nFROM join_inner_table__fuzz_146_replicated\n SETTINGS\n allow_experimental_analyzer = 1,\n max_parallel_replicas = 2,\n cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\',\n allow_experimental_parallel_reading_from_replicas = 1,\n use_hedged_requests = 0; -0 2 SELECT `join_inner_table__fuzz_146_replicated`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated` SETTINGS allow_experimental_analyzer = 1, max_parallel_replicas = 2, cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\', allow_experimental_parallel_reading_from_replicas = 1, use_hedged_requests = 0 +1 1 -- Simple query with analyzer and pure parallel replicas\nSELECT number\nFROM join_inner_table__fuzz_146_replicated\n SETTINGS\n allow_experimental_analyzer = 1,\n max_parallel_replicas = 2,\n cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\',\n allow_experimental_parallel_reading_from_replicas = 1; +0 2 SELECT `join_inner_table__fuzz_146_replicated`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated` SETTINGS allow_experimental_analyzer = 1, max_parallel_replicas = 2, cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\', allow_experimental_parallel_reading_from_replicas = 1 diff --git a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql index 35089c0cedb..88a0d2163d6 100644 --- a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql +++ b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql @@ -24,8 +24,7 @@ FROM join_inner_table__fuzz_146_replicated allow_experimental_analyzer = 1, max_parallel_replicas = 2, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', - allow_experimental_parallel_reading_from_replicas = 1, - use_hedged_requests = 0; + allow_experimental_parallel_reading_from_replicas = 1; SYSTEM FLUSH LOGS; -- There should be 2 different queries diff --git a/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh b/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh index 9cfd3a392c8..6c697095b57 100755 --- a/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh +++ b/tests/queries/0_stateless/02783_parallel_replicas_trivial_count_optimization.sh @@ -30,7 +30,6 @@ function run_query_with_pure_parallel_replicas () { --query_id "${1}_pure" \ --max_parallel_replicas 3 \ --prefer_localhost_replica 1 \ - --use_hedged_requests 0 \ --cluster_for_parallel_replicas 'test_cluster_one_shard_three_replicas_localhost' \ --allow_experimental_parallel_reading_from_replicas 1 \ --allow_experimental_analyzer 0 @@ -40,7 +39,6 @@ function run_query_with_pure_parallel_replicas () { --query_id "${1}_pure_analyzer" \ --max_parallel_replicas 3 \ --prefer_localhost_replica 1 \ - --use_hedged_requests 0 \ --cluster_for_parallel_replicas 'test_cluster_one_shard_three_replicas_localhost' \ --allow_experimental_parallel_reading_from_replicas 1 \ --allow_experimental_analyzer 1 @@ -56,7 +54,6 @@ function run_query_with_custom_key_parallel_replicas () { --query "$2" \ --query_id "${1}_custom_key" \ --max_parallel_replicas 3 \ - --use_hedged_requests 0 \ --parallel_replicas_custom_key_filter_type 'default' \ --parallel_replicas_custom_key "$2" \ --allow_experimental_analyzer 0 @@ -65,7 +62,6 @@ function run_query_with_custom_key_parallel_replicas () { --query "$2" \ --query_id "${1}_custom_key_analyzer" \ --max_parallel_replicas 3 \ - --use_hedged_requests 0 \ --parallel_replicas_custom_key_filter_type 'default' \ --parallel_replicas_custom_key "$2" \ --allow_experimental_analyzer 1 diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh index 741b51284fe..8a3b34e5cfa 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh @@ -49,7 +49,6 @@ function run_query_with_pure_parallel_replicas () { --query_id "${1}_pure" \ --max_parallel_replicas 3 \ --prefer_localhost_replica 1 \ - --use_hedged_requests 0 \ --cluster_for_parallel_replicas "parallel_replicas" \ --allow_experimental_parallel_reading_from_replicas 1 \ --parallel_replicas_for_non_replicated_merge_tree 1 \ diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh index ed68a304b85..baeeb820da5 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh @@ -64,7 +64,6 @@ function run_query_with_pure_parallel_replicas () { --query_id "${1}_pure" \ --max_parallel_replicas 3 \ --prefer_localhost_replica 1 \ - --use_hedged_requests 0 \ --cluster_for_parallel_replicas "parallel_replicas" \ --allow_experimental_parallel_reading_from_replicas 1 \ --parallel_replicas_for_non_replicated_merge_tree 1 \ diff --git a/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql b/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql index 374d73d7d03..14edeecf57e 100644 --- a/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql +++ b/tests/queries/0_stateless/02811_parallel_replicas_prewhere_count.sql @@ -13,7 +13,6 @@ SET skip_unavailable_shards=1, allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=3, -use_hedged_requests=0, cluster_for_parallel_replicas='parallel_replicas', parallel_replicas_for_non_replicated_merge_tree=1, parallel_replicas_min_number_of_rows_per_replica=1000; diff --git a/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql b/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql index 60aa5748575..1e6f9304c0c 100644 --- a/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql +++ b/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql @@ -14,13 +14,13 @@ insert into test select *, today() from numbers(100); SELECT count(), min(id), max(id), avg(id) FROM test_d -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1; insert into test select *, today() from numbers(100); SELECT count(), min(id), max(id), avg(id) FROM test_d -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1; -- 2 shards @@ -38,10 +38,10 @@ insert into test2 select *, today() from numbers(100); SELECT count(), min(id), max(id), avg(id) FROM test2_d -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1; insert into test2 select *, today() from numbers(100); SELECT count(), min(id), max(id), avg(id) FROM test2_d -SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0; +SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1; diff --git a/tests/queries/0_stateless/02841_parallel_replicas_summary.sh b/tests/queries/0_stateless/02841_parallel_replicas_summary.sh index 792c45b06d6..c82d2c8b0c0 100755 --- a/tests/queries/0_stateless/02841_parallel_replicas_summary.sh +++ b/tests/queries/0_stateless/02841_parallel_replicas_summary.sh @@ -36,7 +36,6 @@ echo " cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, - use_hedged_requests = 0, interactive_delay=0 "\ | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query_id=${query_id_base}_interactive_0" --data-binary @- -vvv 2>&1 \ @@ -51,7 +50,6 @@ echo " cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, - use_hedged_requests = 0, interactive_delay=99999999999 "\ | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query_id=${query_id_base}_interactive_high" --data-binary @- -vvv 2>&1 \ diff --git a/tests/queries/0_stateless/02861_index_set_incorrect_args.sql b/tests/queries/0_stateless/02861_index_set_incorrect_args.sql index fa51f5c9abc..17b505cd051 100644 --- a/tests/queries/0_stateless/02861_index_set_incorrect_args.sql +++ b/tests/queries/0_stateless/02861_index_set_incorrect_args.sql @@ -2,5 +2,5 @@ DROP TABLE IF EXISTS set_index__fuzz_41; CREATE TABLE set_index__fuzz_41 (`a` Date, `b` Nullable(DateTime64(3)), INDEX b_set b TYPE set(0) GRANULARITY 1) ENGINE = MergeTree ORDER BY tuple(); INSERT INTO set_index__fuzz_41 (a) VALUES (today()); -SELECT b FROM set_index__fuzz_41 WHERE and(b = 256) SETTINGS force_data_skipping_indices = 'b_set', optimize_move_to_prewhere = 0, max_parallel_replicas=2, parallel_replicas_for_non_replicated_merge_tree=1, allow_experimental_parallel_reading_from_replicas=2, use_hedged_requests=0; -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } +SELECT b FROM set_index__fuzz_41 WHERE and(b = 256) SETTINGS force_data_skipping_indices = 'b_set', optimize_move_to_prewhere = 0, max_parallel_replicas=2, parallel_replicas_for_non_replicated_merge_tree=1, allow_experimental_parallel_reading_from_replicas=2; -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } DROP TABLE set_index__fuzz_41; diff --git a/tests/queries/0_stateless/02869_parallel_replicas_read_from_several.sql b/tests/queries/0_stateless/02869_parallel_replicas_read_from_several.sql index 9559b46fa08..e040fae1fa6 100644 --- a/tests/queries/0_stateless/02869_parallel_replicas_read_from_several.sql +++ b/tests/queries/0_stateless/02869_parallel_replicas_read_from_several.sql @@ -24,5 +24,5 @@ system sync replica t3; SELECT count(), min(k), max(k), avg(k) FROM t1 -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, use_hedged_requests=0, +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost', parallel_replicas_single_task_marks_count_multiplier = 0.001; diff --git a/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql b/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql index 1201a156246..f59d38ceb04 100644 --- a/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql +++ b/tests/queries/0_stateless/02875_parallel_replicas_cluster_all_replicas.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS tt; CREATE TABLE tt (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); INSERT INTO tt SELECT * FROM numbers(10); -SET allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=3, use_hedged_requests=0, parallel_replicas_for_non_replicated_merge_tree=1; +SET allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=3, parallel_replicas_for_non_replicated_merge_tree=1; SELECT count() FROM clusterAllReplicas('test_cluster_two_shard_three_replicas_localhost', currentDatabase(), tt) settings log_comment='02875_190aed82-2423-413b-ad4c-24dcca50f65b'; SYSTEM FLUSH LOGS; diff --git a/tests/queries/0_stateless/02875_parallel_replicas_remote.sql b/tests/queries/0_stateless/02875_parallel_replicas_remote.sql index f47fc559df9..5fbaf34b621 100644 --- a/tests/queries/0_stateless/02875_parallel_replicas_remote.sql +++ b/tests/queries/0_stateless/02875_parallel_replicas_remote.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS tt; CREATE TABLE tt (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); INSERT INTO tt SELECT * FROM numbers(10); -SET allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=3, use_hedged_requests=0, parallel_replicas_for_non_replicated_merge_tree=1; +SET allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=3, parallel_replicas_for_non_replicated_merge_tree=1; SELECT count() FROM remote('127.0.0.{1..6}', currentDatabase(), tt) settings log_comment='02875_89f3c39b-1919-48cb-b66e-ef9904e73146'; SYSTEM FLUSH LOGS; diff --git a/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql b/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql index 70a1cedf663..6b2f146efd0 100644 --- a/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql +++ b/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql @@ -14,7 +14,7 @@ system sync replica t1; system sync replica t2; system sync replica t3; -SET allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=3, use_hedged_requests=0, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; +SET allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=3, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; -- default coordinator SELECT count(), min(k), max(k), avg(k) FROM t1 SETTINGS log_comment='02898_default_190aed82-2423-413b-ad4c-24dcca50f65b'; diff --git a/tests/queries/0_stateless/02901_parallel_replicas_rollup.sh b/tests/queries/0_stateless/02901_parallel_replicas_rollup.sh index f23b80348c1..9c922ec4723 100755 --- a/tests/queries/0_stateless/02901_parallel_replicas_rollup.sh +++ b/tests/queries/0_stateless/02901_parallel_replicas_rollup.sh @@ -29,7 +29,6 @@ $CLICKHOUSE_CLIENT \ --query_id "${query_id}" \ --max_parallel_replicas 3 \ --prefer_localhost_replica 1 \ - --use_hedged_requests 0 \ --cluster_for_parallel_replicas "parallel_replicas" \ --allow_experimental_parallel_reading_from_replicas 1 \ --parallel_replicas_for_non_replicated_merge_tree 1 \ @@ -63,7 +62,6 @@ $CLICKHOUSE_CLIENT \ --query_id "${query_id}" \ --max_parallel_replicas 3 \ --prefer_localhost_replica 1 \ - --use_hedged_requests 0 \ --cluster_for_parallel_replicas "parallel_replicas" \ --allow_experimental_parallel_reading_from_replicas 1 \ --parallel_replicas_for_non_replicated_merge_tree 1 \ diff --git a/tests/queries/0_stateless/02935_parallel_replicas_settings.reference b/tests/queries/0_stateless/02935_parallel_replicas_settings.reference new file mode 100644 index 00000000000..846d77bfa57 --- /dev/null +++ b/tests/queries/0_stateless/02935_parallel_replicas_settings.reference @@ -0,0 +1,4 @@ +10 +1 +10 +1 diff --git a/tests/queries/0_stateless/02935_parallel_replicas_settings.sql b/tests/queries/0_stateless/02935_parallel_replicas_settings.sql new file mode 100644 index 00000000000..be6f1c2958c --- /dev/null +++ b/tests/queries/0_stateless/02935_parallel_replicas_settings.sql @@ -0,0 +1,35 @@ +DROP TABLE IF EXISTS test_parallel_replicas_settings; +CREATE TABLE test_parallel_replicas_settings (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); +INSERT INTO test_parallel_replicas_settings SELECT * FROM numbers(10); + +SET allow_experimental_parallel_reading_from_replicas=2, max_parallel_replicas=3, parallel_replicas_for_non_replicated_merge_tree=1; + +SET cluster_for_parallel_replicas=''; +SELECT count() FROM test_parallel_replicas_settings WHERE NOT ignore(*); -- { serverError CLUSTER_DOESNT_EXIST } + +SET cluster_for_parallel_replicas='parallel_replicas'; +SELECT count() FROM test_parallel_replicas_settings WHERE NOT ignore(*) settings log_comment='0_f621c4f2-4da7-4a7c-bb6d-052c442d0f7f'; + +SYSTEM FLUSH LOGS; + +SELECT count() > 0 FROM system.text_log +WHERE yesterday() <= event_date + AND query_id in (select query_id from system.query_log where current_database=currentDatabase() AND log_comment='0_f621c4f2-4da7-4a7c-bb6d-052c442d0f7f') + AND level = 'Information' + AND message ILIKE '%Disabling ''use_hedged_requests'' in favor of ''allow_experimental_parallel_reading_from_replicas''%' +SETTINGS allow_experimental_parallel_reading_from_replicas=0; + +SET use_hedged_requests=1; +SELECT count() FROM test_parallel_replicas_settings WHERE NOT ignore(*) settings log_comment='1_f621c4f2-4da7-4a7c-bb6d-052c442d0f7f'; + +SYSTEM FLUSH LOGS; + +SET allow_experimental_parallel_reading_from_replicas=0; +SELECT count() > 0 FROM system.text_log +WHERE yesterday() <= event_date + AND query_id in (select query_id from system.query_log where current_database = currentDatabase() AND log_comment = '1_f621c4f2-4da7-4a7c-bb6d-052c442d0f7f') + AND level = 'Warning' + AND message ILIKE '%Setting ''use_hedged_requests'' explicitly with enabled ''allow_experimental_parallel_reading_from_replicas'' has no effect%' +SETTINGS allow_experimental_parallel_reading_from_replicas=0; + +DROP TABLE test_parallel_replicas_settings; diff --git a/tests/queries/1_stateful/00177_memory_bound_merging.sh b/tests/queries/1_stateful/00177_memory_bound_merging.sh index ce889b338d6..d5cd1a05cd8 100755 --- a/tests/queries/1_stateful/00177_memory_bound_merging.sh +++ b/tests/queries/1_stateful/00177_memory_bound_merging.sh @@ -31,7 +31,7 @@ test1() { GROUP BY CounterID, URL, EventDate ORDER BY URL, EventDate LIMIT 5 OFFSET 10 - SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3, use_hedged_requests = 0" + SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3" check_replicas_read_in_order $query_id } @@ -48,7 +48,7 @@ test2() { GROUP BY URL, EventDate ORDER BY URL, EventDate LIMIT 5 OFFSET 10 - SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3, use_hedged_requests = 0, query_plan_aggregation_in_order = 1" + SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3, query_plan_aggregation_in_order = 1" check_replicas_read_in_order $query_id } @@ -64,7 +64,7 @@ test3() { FROM test.hits WHERE CounterID = 1704509 AND UserID = 4322253409885123546 GROUP BY URL, EventDate - SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3, use_hedged_requests = 0 + SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3 ) WHERE explain LIKE '%Aggr%Transform%' OR explain LIKE '%InOrder%'" } From fa7190805a64aa447b9bb568db26f614f15adfb5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 13 Dec 2023 17:59:55 +0100 Subject: [PATCH 327/331] A timeout --- docker/test/clickbench/run.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docker/test/clickbench/run.sh b/docker/test/clickbench/run.sh index 921d2023fd7..ba9580f55ae 100755 --- a/docker/test/clickbench/run.sh +++ b/docker/test/clickbench/run.sh @@ -1,5 +1,8 @@ #!/bin/bash +SCRIPT_PID=$! +(sleep 1200 && kill -9 $SCRIPT_PID) & + # shellcheck disable=SC1091 source /setup_export_logs.sh From 1d70c9464e6e23dde1e5a9370bb9ae1d57bec9c9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 13 Dec 2023 18:01:18 +0100 Subject: [PATCH 328/331] Add it to master checks --- .github/workflows/master.yml | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 05654926fd7..771de46be13 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -555,6 +555,27 @@ jobs: cd "$REPO_COPY/tests/ci" python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" ############################################################################################## +########################### ClickBench ####################################################### +############################################################################################## + ClickBenchAMD64: + needs: [BuilderDebRelease] + uses: ./.github/workflows/reusable_test.yml + with: + test_name: ClickBench (amd64) + runner_type: func-tester + run_command: | + cd "$REPO_COPY/tests/ci" + python3 clickbench.py "$CHECK_NAME" + ClickBenchAarch64: + needs: [BuilderDebAarch64] + uses: ./.github/workflows/reusable_test.yml + with: + test_name: ClickBench (aarch64) + runner_type: func-tester-aarch64 + run_command: | + cd "$REPO_COPY/tests/ci" + python3 clickbench.py "$CHECK_NAME" +############################################################################################## ######################################### STRESS TESTS ####################################### ############################################################################################## StressTestAsan: From 30e6797c7e63488056405dc69c8f46ec22d737fd Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 13 Dec 2023 17:19:14 +0000 Subject: [PATCH 329/331] Update version_date.tsv and changelogs after v23.11.2.11-stable --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.11.2.11-stable.md | 22 ++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 5 files changed, 26 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v23.11.2.11-stable.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 8fc639af1a7..a238a9851d9 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="23.11.1.2711" +ARG VERSION="23.11.2.11" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 76b03218eab..31dbc38708f 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="23.11.1.2711" +ARG VERSION="23.11.2.11" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index c6dfcf9f679..6bbec625300 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -30,7 +30,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="23.11.1.2711" +ARG VERSION="23.11.2.11" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.11.2.11-stable.md b/docs/changelogs/v23.11.2.11-stable.md new file mode 100644 index 00000000000..490cc9a4590 --- /dev/null +++ b/docs/changelogs/v23.11.2.11-stable.md @@ -0,0 +1,22 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.11.2.11-stable (6e5411358c8) FIXME as compared to v23.11.1.2711-stable (05bc8ef1e02) + +#### Improvement +* Backported in [#57661](https://github.com/ClickHouse/ClickHouse/issues/57661): Handle sigabrt case when getting PostgreSQl table structure with empty array. [#57618](https://github.com/ClickHouse/ClickHouse/pull/57618) ([Mike Kot (Михаил Кот)](https://github.com/myrrc)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Ignore ON CLUSTER clause in grant/revoke queries for management of replicated access entities. [#57538](https://github.com/ClickHouse/ClickHouse/pull/57538) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Fix SIGSEGV for aggregation of sparse columns with any() RESPECT NULL [#57710](https://github.com/ClickHouse/ClickHouse/pull/57710) ([Azat Khuzhin](https://github.com/azat)). +* Fix bug window functions: revert [#39631](https://github.com/ClickHouse/ClickHouse/issues/39631) [#57766](https://github.com/ClickHouse/ClickHouse/pull/57766) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Pin alpine version of integration tests helper container [#57669](https://github.com/ClickHouse/ClickHouse/pull/57669) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index cb4102b3072..f319f57e0b9 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v23.11.2.11-stable 2023-12-13 v23.11.1.2711-stable 2023-12-06 v23.10.5.20-stable 2023-11-25 v23.10.4.25-stable 2023-11-17 From 560e66f3ca5f76475b18ef4d6a3efe2b9a21a9af Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 13 Dec 2023 19:51:49 +0100 Subject: [PATCH 330/331] More respect to `min_number_of_marks` in `ParallelReplicasReadingCoordinator` (#57763) --- .../ParallelReplicasReadingCoordinator.cpp | 62 +++++++++---------- 1 file changed, 29 insertions(+), 33 deletions(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 9137dc89705..757d1461769 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -300,20 +300,20 @@ void DefaultCoordinator::selectPartsAndRanges(const PartRefs & container, size_t while (!part->description.ranges.empty() && current_mark_size < min_number_of_marks) { auto & range = part->description.ranges.front(); + const size_t needed = min_number_of_marks - current_mark_size; - if (range.getNumberOfMarks() > min_number_of_marks) + if (range.getNumberOfMarks() > needed) { - auto new_range = range; - range.begin += min_number_of_marks; - new_range.end = new_range.begin + min_number_of_marks; + auto range_we_take = MarkRange{range.begin, range.begin + needed}; + response.description.back().ranges.emplace_back(range_we_take); + current_mark_size += range_we_take.getNumberOfMarks(); - response.description.back().ranges.emplace_back(new_range); - current_mark_size += new_range.getNumberOfMarks(); - continue; + range.begin += needed; + break; } - current_mark_size += part->description.ranges.front().getNumberOfMarks(); - response.description.back().ranges.emplace_back(part->description.ranges.front()); + response.description.back().ranges.emplace_back(range); + current_mark_size += range.getNumberOfMarks(); part->description.ranges.pop_front(); } } @@ -473,23 +473,21 @@ ParallelReadResponse InOrderCoordinator::handleRequest(ParallelReadRequest { while (!global_part_it->description.ranges.empty() && current_mark_size < request.min_number_of_marks) { - auto range = global_part_it->description.ranges.back(); + auto & range = global_part_it->description.ranges.back(); + const size_t needed = request.min_number_of_marks - current_mark_size; - if (range.getNumberOfMarks() > request.min_number_of_marks) + if (range.getNumberOfMarks() > needed) { - auto new_range = range; - range.end -= request.min_number_of_marks; - new_range.begin = new_range.end - request.min_number_of_marks; + auto range_we_take = MarkRange{range.end - needed, range.end}; + part.ranges.emplace_front(range_we_take); + current_mark_size += range_we_take.getNumberOfMarks(); - global_part_it->description.ranges.back() = range; - - part.ranges.emplace_front(new_range); - current_mark_size += new_range.getNumberOfMarks(); - continue; + range.end -= needed; + break; } - current_mark_size += global_part_it->description.ranges.back().getNumberOfMarks(); - part.ranges.emplace_front(global_part_it->description.ranges.back()); + part.ranges.emplace_front(range); + current_mark_size += range.getNumberOfMarks(); global_part_it->description.ranges.pop_back(); } } @@ -497,23 +495,21 @@ ParallelReadResponse InOrderCoordinator::handleRequest(ParallelReadRequest { while (!global_part_it->description.ranges.empty() && current_mark_size < request.min_number_of_marks) { - auto range = global_part_it->description.ranges.front(); + auto & range = global_part_it->description.ranges.front(); + const size_t needed = request.min_number_of_marks - current_mark_size; - if (range.getNumberOfMarks() > request.min_number_of_marks) + if (range.getNumberOfMarks() > needed) { - auto new_range = range; - range.begin += request.min_number_of_marks; - new_range.end = new_range.begin + request.min_number_of_marks; + auto range_we_take = MarkRange{range.begin, range.begin + needed}; + part.ranges.emplace_back(range_we_take); + current_mark_size += range_we_take.getNumberOfMarks(); - global_part_it->description.ranges.front() = range; - - part.ranges.emplace_back(new_range); - current_mark_size += new_range.getNumberOfMarks(); - continue; + range.begin += needed; + break; } - current_mark_size += global_part_it->description.ranges.front().getNumberOfMarks(); - part.ranges.emplace_back(global_part_it->description.ranges.front()); + part.ranges.emplace_back(range); + current_mark_size += range.getNumberOfMarks(); global_part_it->description.ranges.pop_front(); } } From b8d274d070b89bdfee578492f8210cd96859fdd8 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Wed, 13 Dec 2023 10:59:31 -0800 Subject: [PATCH 331/331] Add malformed output generation to JSON fuzzer (#57646) Randomly modify structural characters of a valid JSON ('{', '}', '[', ']', ':', '"', ',') to generate output that cannot be parsed as JSON. Follow-up to https://github.com/ClickHouse/ClickHouse/pull/56490 --- .../sql-reference/table-functions/fuzzJSON.md | 11 ++++ src/Storages/StorageFuzzJSON.cpp | 48 ++++++++++---- src/Storages/StorageFuzzJSON.h | 1 + .../02918_fuzzjson_table_function.reference | 1 + .../02918_fuzzjson_table_function.sql | 65 +++++++++++++++++-- 5 files changed, 107 insertions(+), 19 deletions(-) diff --git a/docs/en/sql-reference/table-functions/fuzzJSON.md b/docs/en/sql-reference/table-functions/fuzzJSON.md index 74ccb0bcb8a..a64f35691f6 100644 --- a/docs/en/sql-reference/table-functions/fuzzJSON.md +++ b/docs/en/sql-reference/table-functions/fuzzJSON.md @@ -19,6 +19,7 @@ fuzzJSON({ named_collection [option=value [,..]] | json_str[, random_seed] }) - `json_str` (String) - The source string representing structured data in JSON format. - `random_seed` (UInt64) - Manual random seed for producing stable results. - `reuse_output` (boolean) - Reuse the output from a fuzzing process as input for the next fuzzer. + - `malform_output` (boolean) - Generate a string that cannot be parsed as a JSON object. - `max_output_length` (UInt64) - Maximum allowable length of the generated or perturbed JSON string. - `probability` (Float64) - The probability to fuzz a JSON field (a key-value pair). Must be within [0, 1] range. - `max_nesting_level` (UInt64) - The maximum allowed depth of nested structures within the JSON data. @@ -84,3 +85,13 @@ SELECT * FROM fuzzJSON('{"id":1}', 1234) LIMIT 3; {"BRjE":16137826149911306846} {"XjKE":15076727133550123563} ``` + +``` sql +SELECT * FROM fuzzJSON(json_nc, json_str='{"name" : "FuzzJSON"}', random_seed=1337, malform_output=true) LIMIT 3; +``` + +``` text +U"name":"FuzzJSON*"SpByjZKtr2VAyHCO"falseh +{"name"keFuzzJSON, "g6vVO7TCIk":jTt^ +{"DBhz":YFuzzJSON5} +``` diff --git a/src/Storages/StorageFuzzJSON.cpp b/src/Storages/StorageFuzzJSON.cpp index 8b2dd7ac692..6bf69efa1dd 100644 --- a/src/Storages/StorageFuzzJSON.cpp +++ b/src/Storages/StorageFuzzJSON.cpp @@ -248,10 +248,10 @@ Field generateRandomFixedValue(const StorageFuzzJSON::Configuration & config, pc return f; } -String fuzzJSONKey(const StorageFuzzJSON::Configuration & config, pcg64 & rnd, const String & source) +String fuzzString(UInt64 min_length, UInt64 max_length, pcg64 & rnd, const String & source, std::function charGen) { String result; - result.reserve(config.max_key_length); + result.reserve(max_length); using FA = FuzzAction; auto get_action = [&]() -> FuzzAction @@ -261,7 +261,7 @@ String fuzzJSONKey(const StorageFuzzJSON::Configuration & config, pcg64 & rnd, c }; size_t i = 0; - while (i < source.size() && result.size() < config.max_key_length) + while (i < source.size() && result.size() < max_length) { auto action = get_action(); switch (action) @@ -271,12 +271,12 @@ String fuzzJSONKey(const StorageFuzzJSON::Configuration & config, pcg64 & rnd, c } break; case FA::Edit: { - result.push_back(generateRandomKeyCharacter(rnd)); + result.push_back(charGen(rnd)); ++i; } break; case FA::Add: { - result.push_back(generateRandomKeyCharacter(rnd)); + result.push_back(charGen(rnd)); } break; default: @@ -284,12 +284,24 @@ String fuzzJSONKey(const StorageFuzzJSON::Configuration & config, pcg64 & rnd, c } } - while (result.size() < config.min_key_length) - result.push_back(generateRandomKeyCharacter(rnd)); + while (result.size() < min_length) + result.push_back(charGen(rnd)); return result; } +String fuzzJSONKey(const StorageFuzzJSON::Configuration & config, pcg64 & rnd, const String & key) +{ + return fuzzString(config.min_key_length, config.max_key_length, rnd, key, generateRandomKeyCharacter); +} + +// Randomly modify structural characters (e.g. '{', '}', '[', ']', ':', '"') to generate output that cannot be parsed as JSON. +String fuzzJSONStructure(const StorageFuzzJSON::Configuration & config, pcg64 & rnd, const String & s) +{ + return config.should_malform_output ? fuzzString(/*min_length*/ 0, /*max_length*/ s.size(), rnd, s, generateRandomStringValueCharacter) + : s; +} + std::shared_ptr generateRandomJSONNode(const StorageFuzzJSON::Configuration & config, pcg64 & rnd, bool with_key, JSONValue::Type type) { @@ -397,7 +409,7 @@ void fuzzJSONObject( if (next_node->key) { writeDoubleQuoted(*next_node->key, out); - out << ":"; + out << fuzzJSONStructure(config, rnd, ":"); } auto & val = next_node->value; @@ -405,7 +417,11 @@ void fuzzJSONObject( if (val.fixed) { if (val.fixed->getType() == Field::Types::Which::String) - writeDoubleQuoted(val.fixed->get(), out); + { + out << fuzzJSONStructure(config, rnd, "\""); + writeText(val.fixed->get(), out); + out << fuzzJSONStructure(config, rnd, "\""); + } else writeFieldText(*val.fixed, out); } @@ -414,9 +430,9 @@ void fuzzJSONObject( if (!val.array && !val.object) return; - const auto & [op, cl, node_list] = val.array ? std::make_tuple('[', ']', *val.array) : std::make_tuple('{', '}', *val.object); + const auto & [op, cl, node_list] = val.array ? std::make_tuple("[", "]", *val.array) : std::make_tuple("{", "}", *val.object); - out << op; + out << fuzzJSONStructure(config, rnd, op); bool first = true; for (const auto & ptr : node_list) @@ -426,7 +442,7 @@ void fuzzJSONObject( WriteBufferFromOwnString child_out; if (!first) - child_out << ", "; + child_out << fuzzJSONStructure(config, rnd, ", "); first = false; fuzzJSONObject(ptr, child_out, config, rnd, depth + 1, node_count); @@ -435,7 +451,7 @@ void fuzzJSONObject( break; out << child_out.str(); } - out << cl; + out << fuzzJSONStructure(config, rnd, cl); } } @@ -554,10 +570,11 @@ Pipe StorageFuzzJSON::read( return Pipe::unitePipes(std::move(pipes)); } -static constexpr std::array optional_configuration_keys +static constexpr std::array optional_configuration_keys = {"json_str", "random_seed", "reuse_output", + "malform_output", "probability", "max_output_length", "max_nesting_level", @@ -583,6 +600,9 @@ void StorageFuzzJSON::processNamedCollectionResult(Configuration & configuration if (collection.has("reuse_output")) configuration.should_reuse_output = static_cast(collection.get("reuse_output")); + if (collection.has("malform_output")) + configuration.should_malform_output = static_cast(collection.get("malform_output")); + if (collection.has("probability")) { configuration.probability = collection.get("probability"); diff --git a/src/Storages/StorageFuzzJSON.h b/src/Storages/StorageFuzzJSON.h index f1d78fba85c..027c5a98634 100644 --- a/src/Storages/StorageFuzzJSON.h +++ b/src/Storages/StorageFuzzJSON.h @@ -27,6 +27,7 @@ public: String json_str = "{}"; UInt64 random_seed = randomSeed(); bool should_reuse_output = false; + bool should_malform_output = false; Float64 probability = 0.25; UInt64 max_output_length = 1024; diff --git a/tests/queries/0_stateless/02918_fuzzjson_table_function.reference b/tests/queries/0_stateless/02918_fuzzjson_table_function.reference index 1b5c6f46f77..8ad9e886b49 100644 --- a/tests/queries/0_stateless/02918_fuzzjson_table_function.reference +++ b/tests/queries/0_stateless/02918_fuzzjson_table_function.reference @@ -150,3 +150,4 @@ {} 730 200 +50 diff --git a/tests/queries/0_stateless/02918_fuzzjson_table_function.sql b/tests/queries/0_stateless/02918_fuzzjson_table_function.sql index 6db0c69dbac..398b3572587 100644 --- a/tests/queries/0_stateless/02918_fuzzjson_table_function.sql +++ b/tests/queries/0_stateless/02918_fuzzjson_table_function.sql @@ -92,15 +92,70 @@ SELECT * FROM fuzzJSON(02918_json_fuzzer, max_key_length=10, min_key_length=0) L SELECT * FROM fuzzJSON(02918_json_fuzzer, max_key_length=10, min_key_length=11) LIMIT 10; -- { serverError BAD_ARGUMENTS } -- -DROP TABLE IF EXISTS 02918_table_obj; -CREATE TABLE 02918_table_obj (json_obj Object('json')) Engine=Memory; +DROP TABLE IF EXISTS 02918_table_obj1; +CREATE TABLE 02918_table_obj1 (json_obj Object('json')) Engine=Memory; -INSERT INTO 02918_table_obj SELECT * FROM fuzzJSON( +INSERT INTO 02918_table_obj1 SELECT * FROM fuzzJSON( 02918_json_fuzzer, json_str='{"name": "John Doe", "age": 27, "address": {"city": "Citiville", "zip": "12345"}, "hobbies": ["reading", "traveling", "coding"]}', random_seed=12345) LIMIT 200; -SELECT count() FROM 02918_table_obj; -DROP TABLE IF EXISTS 02918_table_obj; +SELECT count() FROM 02918_table_obj1; + +DROP TABLE IF EXISTS 02918_table_obj1; + +-- +DROP TABLE IF EXISTS 02918_table_obj2; +CREATE TABLE 02918_table_obj2 (json_obj Object('json')) Engine=Memory; + +INSERT INTO 02918_table_obj2 SELECT * FROM fuzzJSON( + 02918_json_fuzzer, + json_str= + '{ + "name": { + "first": "Joan", + "last": "of Arc" + }, + "birth": {"date": "January 6, 1412", "place": "Domremy, France"}, + "death": {"date": "May 30, 1431", "place": "Rouen, France"}, + "occupation": "Military Leader", + "achievements": ["Lifted Siege of Orleans", "Assisted in Charles VII\'s Coronation"], + "legacy": { + "honors": ["Canonized Saint", "National Heroine of France"], + "memorials": [ + {"name": "Joan of Arc Memorial", "location": "Domremy"}, + {"name": "Place Jeanne d\'Arc", "location": "Rouen"} + ] + } + }', + random_seed=12345, + max_output_length=1024) LIMIT 50; + +INSERT INTO 02918_table_obj2 SELECT * FROM fuzzJSON( + 02918_json_fuzzer, + json_str= + '{ + "name": { + "first": "Joan", + "last": "of Arc" + }, + "birth": {"date": "January 6, 1412", "place": "Domremy, France"}, + "death": {"date": "May 30, 1431", "place": "Rouen, France"}, + "occupation": "Military Leader", + "achievements": ["Lifted Siege of Orleans", "Assisted in Charles VII\'s Coronation"], + "legacy": { + "honors": ["Canonized Saint", "National Heroine of France"], + "memorials": [ + {"name": "Joan of Arc Memorial", "location": "Domremy"}, + {"name": "Place Jeanne d\'Arc", "location": "Rouen"} + ] + } + }', + random_seed=12345, + max_output_length=1024, malform_output=true) LIMIT 50; -- {serverError INCORRECT_DATA } + +SELECT count() FROM 02918_table_obj2; + +DROP TABLE IF EXISTS 02918_table_obj2; DROP NAMED COLLECTION IF EXISTS 02918_json_fuzzer;