From 0b014e0caaf8b808f2240c398ecc7bb1e5941f65 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 2 Sep 2022 11:06:32 -0300 Subject: [PATCH 001/253] Add MT select final setting --- src/Interpreters/InterpreterSelectQuery.cpp | 6 ++++++ src/Storages/IStorage.h | 2 ++ src/Storages/MergeTree/MergeTreeData.h | 2 ++ src/Storages/MergeTree/MergeTreeSettings.h | 1 + src/Storages/MergeTree/registerStorageMergeTree.cpp | 7 +++++++ 5 files changed, 18 insertions(+) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index c73db82a27b..7bf7dd920ac 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -495,6 +495,12 @@ InterpreterSelectQuery::InterpreterSelectQuery( query_info.additional_filter_ast = parseAdditionalFilterConditionForTable( settings.additional_table_filters, joined_tables.tablesWithColumns().front().table, *context); + + if (!query.final() && storage && storage->forceQueryWithFinal()) + { + query.setFinal(); + } + auto analyze = [&] (bool try_move_to_prewhere) { /// Allow push down and other optimizations for VIEW: replace with subquery and rewrite it. diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index a61bfeaff57..a9709fd7610 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -325,6 +325,8 @@ public: /// It's needed for ReplacingMergeTree wrappers such as MaterializedMySQL and MaterializedPostrgeSQL virtual bool needRewriteQueryWithFinal(const Names & /*column_names*/) const { return false; } + virtual bool forceQueryWithFinal() const { return false; } + private: /** Read a set of columns from the table. * Accepts a list of columns to read, as well as a description of the query, diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index c91c7ba02a8..ad04752bd0b 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -449,6 +449,8 @@ public: bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, ContextPtr, const StorageMetadataPtr & metadata_snapshot) const override; + bool forceQueryWithFinal() const override { return getSettings()->force_select_final && supportsFinal(); } + /// Snapshot for MergeTree contains the current set of data parts /// at the moment of the start of query. struct SnapshotData : public StorageSnapshot::Data diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 07659b1c9dc..34810d38645 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -144,6 +144,7 @@ struct Settings; M(Bool, allow_remote_fs_zero_copy_replication, false, "Don't use this setting in production, because it is not ready.", 0) \ M(String, remote_fs_zero_copy_zookeeper_path, "/clickhouse/zero_copy", "ZooKeeper path for Zero-copy table-independet info.", 0) \ M(Bool, remote_fs_zero_copy_path_compatible_mode, false, "Run zero-copy in compatible mode during conversion process.", 0) \ + M(Bool, force_select_final, false, "Query with the FINAL modifier by default", 0) \ \ /** Obsolete settings. Kept for backward compatibility only. */ \ M(UInt64, min_relative_delay_to_yield_leadership, 120, "Obsolete setting, does nothing.", 0) \ diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index e52a0fed674..41075e29a9d 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -34,6 +34,7 @@ namespace ErrorCodes extern const int NO_REPLICA_NAME_GIVEN; extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; extern const int NOT_IMPLEMENTED; + extern const int ILLEGAL_FINAL; } @@ -677,6 +678,12 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (arg_num != arg_cnt) throw Exception("Wrong number of engine arguments.", ErrorCodes::BAD_ARGUMENTS); + + if (merging_params.mode == MergeTreeData::MergingParams::Mode::Ordinary && storage_settings->force_select_final) + { + throw Exception("Storage MergeTree doesn't support FINAL", ErrorCodes::ILLEGAL_FINAL); + } + if (replicated) { auto storage_policy = args.getContext()->getStoragePolicy(storage_settings->storage_policy); From 2c0d337733a6169d4f6a125947be1c8cde337b65 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 7 Sep 2022 09:07:29 -0300 Subject: [PATCH 002/253] Add tests & fix select count --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../02420_mt_ordinary_select_final_setting.reference | 0 .../0_stateless/02420_mt_ordinary_select_final_setting.sql | 1 + .../02420_mt_replacing_select_final_setting.reference | 1 + .../0_stateless/02420_mt_replacing_select_final_setting.sql | 6 ++++++ 5 files changed, 9 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02420_mt_ordinary_select_final_setting.reference create mode 100644 tests/queries/0_stateless/02420_mt_ordinary_select_final_setting.sql create mode 100644 tests/queries/0_stateless/02420_mt_replacing_select_final_setting.reference create mode 100644 tests/queries/0_stateless/02420_mt_replacing_select_final_setting.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b7b68367e98..21f6c98e309 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5558,7 +5558,7 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg if (settings.parallel_replicas_count > 1 || settings.max_parallel_replicas > 1) return std::nullopt; - auto query_ptr = query_info.original_query; + auto query_ptr = query_info.query; auto * select_query = query_ptr->as(); if (!select_query) return std::nullopt; diff --git a/tests/queries/0_stateless/02420_mt_ordinary_select_final_setting.reference b/tests/queries/0_stateless/02420_mt_ordinary_select_final_setting.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02420_mt_ordinary_select_final_setting.sql b/tests/queries/0_stateless/02420_mt_ordinary_select_final_setting.sql new file mode 100644 index 00000000000..ec4f17fbcf4 --- /dev/null +++ b/tests/queries/0_stateless/02420_mt_ordinary_select_final_setting.sql @@ -0,0 +1 @@ +create table if not exists ordinary_mt (x String) engine=MergeTree() ORDER BY x SETTINGS force_select_final=1; -- { serverError 181 } \ No newline at end of file diff --git a/tests/queries/0_stateless/02420_mt_replacing_select_final_setting.reference b/tests/queries/0_stateless/02420_mt_replacing_select_final_setting.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02420_mt_replacing_select_final_setting.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02420_mt_replacing_select_final_setting.sql b/tests/queries/0_stateless/02420_mt_replacing_select_final_setting.sql new file mode 100644 index 00000000000..cc5cf9b3123 --- /dev/null +++ b/tests/queries/0_stateless/02420_mt_replacing_select_final_setting.sql @@ -0,0 +1,6 @@ +create table if not exists replacing_mt (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; + +insert into replacing_mt values ('abc'); +insert into replacing_mt values ('abc'); + +select count() from replacing_mt \ No newline at end of file From a6d106c31250a4b294016801a334ca8b29866e84 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 7 Sep 2022 10:25:16 -0300 Subject: [PATCH 003/253] tmp --- src/Storages/MergeTree/MergeTreeData.h | 5 ++++- src/Storages/ReadFinalForExternalReplicaStorage.cpp | 3 +++ src/Storages/StorageMaterializedMySQL.cpp | 3 +-- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index ad04752bd0b..fba09854603 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -449,8 +449,11 @@ public: bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, ContextPtr, const StorageMetadataPtr & metadata_snapshot) const override; - bool forceQueryWithFinal() const override { return getSettings()->force_select_final && supportsFinal(); } + virtual bool needRewriteQueryWithFinal(const Names & /*column_names*/) const override { return getSettings()->force_select_final && supportsFinal(); } + bool forceQueryWithFinal() const override { + return getSettings()->force_select_final && supportsFinal(); + } /// Snapshot for MergeTree contains the current set of data parts /// at the moment of the start of query. struct SnapshotData : public StorageSnapshot::Data diff --git a/src/Storages/ReadFinalForExternalReplicaStorage.cpp b/src/Storages/ReadFinalForExternalReplicaStorage.cpp index 3ec7a074fd4..14bef0167ba 100644 --- a/src/Storages/ReadFinalForExternalReplicaStorage.cpp +++ b/src/Storages/ReadFinalForExternalReplicaStorage.cpp @@ -21,6 +21,9 @@ namespace DB bool needRewriteQueryWithFinalForStorage(const Names & column_names, const StoragePtr & storage) { + if (storage->needRewriteQueryWithFinal(column_names)) { + return true; + } const StorageMetadataPtr & metadata = storage->getInMemoryMetadataPtr(); Block header = metadata->getSampleBlock(); ColumnWithTypeAndName & version_column = header.getByPosition(header.columns() - 1); diff --git a/src/Storages/StorageMaterializedMySQL.cpp b/src/Storages/StorageMaterializedMySQL.cpp index a7e54960563..de7f8d7e589 100644 --- a/src/Storages/StorageMaterializedMySQL.cpp +++ b/src/Storages/StorageMaterializedMySQL.cpp @@ -27,8 +27,7 @@ StorageMaterializedMySQL::StorageMaterializedMySQL(const StoragePtr & nested_sto setInMemoryMetadata(in_memory_metadata); } -bool StorageMaterializedMySQL::needRewriteQueryWithFinal(const Names & column_names) const -{ +bool StorageMaterializedMySQL::needRewriteQueryWithFinal(const Names & column_names) const { return needRewriteQueryWithFinalForStorage(column_names, nested_storage); } From 64ce1922aba83cf852bb59ac65f0071c210376e9 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 21 Sep 2022 14:45:43 -0300 Subject: [PATCH 004/253] Minor fixes --- src/Storages/MergeTree/MergeTreeData.cpp | 18 +++++++++++------- .../02420_mt_ordinary_select_final_setting.sql | 2 +- ...02420_mt_replacing_select_final_setting.sql | 2 +- 3 files changed, 13 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 21f6c98e309..a89201ecbe3 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5559,16 +5559,20 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg return std::nullopt; auto query_ptr = query_info.query; + auto original_query_ptr = query_info.original_query; + auto * select_query = query_ptr->as(); - if (!select_query) + auto * original_select_query = original_query_ptr->as(); + + if (!original_select_query || select_query) return std::nullopt; // Currently projections don't support final yet. - if (select_query->final()) + if (select_query->final() || original_select_query->final()) return std::nullopt; // Currently projections don't support sample yet. - if (select_query->sampleSize()) + if (original_select_query->sampleSize()) return std::nullopt; // Currently projection don't support deduplication when moving parts between shards. @@ -5576,21 +5580,21 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg return std::nullopt; // Currently projections don't support ARRAY JOIN yet. - if (select_query->arrayJoinExpressionList().first) + if (original_select_query->arrayJoinExpressionList().first) return std::nullopt; // In order to properly analyze joins, aliases should be recognized. However, aliases get lost during projection analysis. // Let's disable projection if there are any JOIN clauses. // TODO: We need a better identifier resolution mechanism for projection analysis. - if (select_query->hasJoin()) + if (original_select_query->hasJoin()) return std::nullopt; // INTERPOLATE expressions may include aliases, so aliases should be preserved - if (select_query->interpolate() && !select_query->interpolate()->children.empty()) + if (original_select_query->interpolate() && !original_select_query->interpolate()->children.empty()) return std::nullopt; // Currently projections don't support GROUPING SET yet. - if (select_query->group_by_with_grouping_sets) + if (original_select_query->group_by_with_grouping_sets) return std::nullopt; auto query_options = SelectQueryOptions( diff --git a/tests/queries/0_stateless/02420_mt_ordinary_select_final_setting.sql b/tests/queries/0_stateless/02420_mt_ordinary_select_final_setting.sql index ec4f17fbcf4..6575de9c59b 100644 --- a/tests/queries/0_stateless/02420_mt_ordinary_select_final_setting.sql +++ b/tests/queries/0_stateless/02420_mt_ordinary_select_final_setting.sql @@ -1 +1 @@ -create table if not exists ordinary_mt (x String) engine=MergeTree() ORDER BY x SETTINGS force_select_final=1; -- { serverError 181 } \ No newline at end of file +create table if not exists ordinary_mt (x String) engine=MergeTree() ORDER BY x SETTINGS force_select_final=1; -- { serverError 181 } diff --git a/tests/queries/0_stateless/02420_mt_replacing_select_final_setting.sql b/tests/queries/0_stateless/02420_mt_replacing_select_final_setting.sql index cc5cf9b3123..3db1d5999b8 100644 --- a/tests/queries/0_stateless/02420_mt_replacing_select_final_setting.sql +++ b/tests/queries/0_stateless/02420_mt_replacing_select_final_setting.sql @@ -3,4 +3,4 @@ create table if not exists replacing_mt (x String) engine=ReplacingMergeTree() O insert into replacing_mt values ('abc'); insert into replacing_mt values ('abc'); -select count() from replacing_mt \ No newline at end of file +select count() from replacing_mt From f97dc8bf6ad4880fc82dc2135cf60413797be758 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 28 Sep 2022 08:21:17 -0300 Subject: [PATCH 005/253] Fix styling and remove experiment --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.h | 5 ++--- src/Storages/ReadFinalForExternalReplicaStorage.cpp | 3 --- 3 files changed, 3 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a89201ecbe3..70e46dcfa70 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5564,7 +5564,7 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg auto * select_query = query_ptr->as(); auto * original_select_query = original_query_ptr->as(); - if (!original_select_query || select_query) + if (!original_select_query || !select_query) return std::nullopt; // Currently projections don't support final yet. diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index fba09854603..0ae382ccb73 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -449,9 +449,8 @@ public: bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, ContextPtr, const StorageMetadataPtr & metadata_snapshot) const override; - virtual bool needRewriteQueryWithFinal(const Names & /*column_names*/) const override { return getSettings()->force_select_final && supportsFinal(); } - - bool forceQueryWithFinal() const override { + bool forceQueryWithFinal() const override + { return getSettings()->force_select_final && supportsFinal(); } /// Snapshot for MergeTree contains the current set of data parts diff --git a/src/Storages/ReadFinalForExternalReplicaStorage.cpp b/src/Storages/ReadFinalForExternalReplicaStorage.cpp index 14bef0167ba..3ec7a074fd4 100644 --- a/src/Storages/ReadFinalForExternalReplicaStorage.cpp +++ b/src/Storages/ReadFinalForExternalReplicaStorage.cpp @@ -21,9 +21,6 @@ namespace DB bool needRewriteQueryWithFinalForStorage(const Names & column_names, const StoragePtr & storage) { - if (storage->needRewriteQueryWithFinal(column_names)) { - return true; - } const StorageMetadataPtr & metadata = storage->getInMemoryMetadataPtr(); Block header = metadata->getSampleBlock(); ColumnWithTypeAndName & version_column = header.getByPosition(header.columns() - 1); From ca4e109c0908aa0a097f8300fa31f72ddf454168 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 28 Sep 2022 08:44:12 -0300 Subject: [PATCH 006/253] fix style --- src/Storages/StorageMaterializedMySQL.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageMaterializedMySQL.cpp b/src/Storages/StorageMaterializedMySQL.cpp index de7f8d7e589..a7e54960563 100644 --- a/src/Storages/StorageMaterializedMySQL.cpp +++ b/src/Storages/StorageMaterializedMySQL.cpp @@ -27,7 +27,8 @@ StorageMaterializedMySQL::StorageMaterializedMySQL(const StoragePtr & nested_sto setInMemoryMetadata(in_memory_metadata); } -bool StorageMaterializedMySQL::needRewriteQueryWithFinal(const Names & column_names) const { +bool StorageMaterializedMySQL::needRewriteQueryWithFinal(const Names & column_names) const +{ return needRewriteQueryWithFinalForStorage(column_names, nested_storage); } From 6a57ea922797c7cdd3334fceb13f072d360aa6fa Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 18 Oct 2022 17:13:54 -0300 Subject: [PATCH 007/253] Add tests to validate select final setting works on joins --- ...ing_join_select_final_setting_both_tables.reference | 1 + ...replacing_join_select_final_setting_both_tables.sql | 10 ++++++++++ ...lacing_join_select_final_setting_rhs_only.reference | 1 + ...mt_replacing_join_select_final_setting_rhs_only.sql | 10 ++++++++++ 4 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_both_tables.reference create mode 100644 tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_both_tables.sql create mode 100644 tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_rhs_only.reference create mode 100644 tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_rhs_only.sql diff --git a/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_both_tables.reference b/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_both_tables.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_both_tables.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_both_tables.sql b/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_both_tables.sql new file mode 100644 index 00000000000..5f7fdaa2f70 --- /dev/null +++ b/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_both_tables.sql @@ -0,0 +1,10 @@ +create table if not exists lhs (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; +create table if not exists rhs (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; + +insert into lhs values ('abc'); +insert into lhs values ('abc'); + +insert into rhs values ('abc'); +insert into rhs values ('abc'); + +select count() from lhs inner join rhs on lhs.x = rhs.x; diff --git a/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_rhs_only.reference b/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_rhs_only.reference new file mode 100644 index 00000000000..0cfbf08886f --- /dev/null +++ b/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_rhs_only.reference @@ -0,0 +1 @@ +2 diff --git a/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_rhs_only.sql b/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_rhs_only.sql new file mode 100644 index 00000000000..c03b8db9e83 --- /dev/null +++ b/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_rhs_only.sql @@ -0,0 +1,10 @@ +create table if not exists lhs (x String) engine=ReplacingMergeTree() ORDER BY x; +create table if not exists rhs (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; + +insert into lhs values ('abc'); +insert into lhs values ('abc'); + +insert into rhs values ('abc'); +insert into rhs values ('abc'); + +select count() from lhs inner join rhs on lhs.x = rhs.x; From 0f6b8770734c306f701d847b97cc34d577d873a0 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 21 Oct 2022 16:27:59 -0300 Subject: [PATCH 008/253] Add ignore_force_select_final setting --- src/Core/Settings.h | 2 ++ src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5dedc6117aa..e015806b562 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -269,6 +269,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Milliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \ M(Milliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \ \ + M(Bool, ignore_force_select_final, false, "Ignores the MT force_select_final setting", 0) \ + \ /** Settings for testing hedged requests */ \ M(Milliseconds, sleep_in_send_tables_status_ms, 0, "Time to sleep in sending tables status response in TCPHandler", 0) \ M(Milliseconds, sleep_in_send_data_ms, 0, "Time to sleep in sending data in TCPHandler", 0) \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 1229ac75083..303302d8259 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -496,7 +496,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( settings.additional_table_filters, joined_tables.tablesWithColumns().front().table, *context); - if (!query.final() && storage && storage->forceQueryWithFinal()) + if (!query.final() && storage && storage->forceQueryWithFinal() && !context->getSettingsRef().ignore_force_select_final) { query.setFinal(); } From dcbd5d9c3d34461a5c58e5be47d36109c9329a0f Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 25 Oct 2022 09:49:48 -0300 Subject: [PATCH 009/253] Add ignore force select final setting test --- ...gnore_force_select_final_setting.reference | 16 ++++++++++++++++ ...0_mt_ignore_force_select_final_setting.sql | 19 +++++++++++++++++++ 2 files changed, 35 insertions(+) create mode 100644 tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.reference create mode 100644 tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.sql diff --git a/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.reference b/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.reference new file mode 100644 index 00000000000..ac177354f4a --- /dev/null +++ b/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.reference @@ -0,0 +1,16 @@ +-- { echoOn } + +SELECT * FROM tbl WHERE id = 10000; -- single row expected (bar), because the force_select_final is in action +10000 bar +SELECT * FROM tbl final WHERE id = 10000; -- single row expected (bar), because the force_select_final is in action and FINAL is there. +10000 bar +SELECT * FROM tbl WHERE id = 10000 SETTINGS ignore_force_select_final=1; -- now we see 2 'real' rows +10000 bar +10000 foo +SELECT * FROM tbl FINAL WHERE id = 10000 SETTINGS ignore_force_select_final=1; -- now we see single row again. +10000 bar +SYSTEM START MERGES tbl; +OPTIMIZE TABLE tbl FINAL; +SELECT * FROM tbl WHERE id = 10000 SETTINGS ignore_force_select_final=1; -- now we see single row DROP TABLE tbl; +10000 bar +DROP TABLE tbl; diff --git a/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.sql b/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.sql new file mode 100644 index 00000000000..fdcdea2bc23 --- /dev/null +++ b/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.sql @@ -0,0 +1,19 @@ +CREATE TABLE tbl (id UInt64, val String) engine=ReplacingMergeTree() ORDER BY id SETTINGS force_select_final=1; + +SYSTEM STOP MERGES tbl; +INSERT INTO tbl SELECT number as id, 'foo' AS val FROM numbers(100000); +INSERT INTO tbl SELECT number as id, 'bar' AS val FROM numbers(100000); + +-- { echoOn } + +SELECT * FROM tbl WHERE id = 10000; -- single row expected (bar), because the force_select_final is in action +SELECT * FROM tbl final WHERE id = 10000; -- single row expected (bar), because the force_select_final is in action and FINAL is there. + +SELECT * FROM tbl WHERE id = 10000 SETTINGS ignore_force_select_final=1; -- now we see 2 'real' rows +SELECT * FROM tbl FINAL WHERE id = 10000 SETTINGS ignore_force_select_final=1; -- now we see single row again. + +SYSTEM START MERGES tbl; +OPTIMIZE TABLE tbl FINAL; + +SELECT * FROM tbl WHERE id = 10000 SETTINGS ignore_force_select_final=1; -- now we see single row DROP TABLE tbl; +DROP TABLE tbl; \ No newline at end of file From 687677399e28ed4e2e027e540832cd75d60d4ccc Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 25 Oct 2022 10:06:15 -0300 Subject: [PATCH 010/253] group force select final setting tests together --- ...02420_force_select_final_setting.reference | 32 ++++++++++++++ .../02420_force_select_final_setting.sql | 42 +++++++++++++++++++ ...mt_ordinary_select_final_setting.reference | 0 ...02420_mt_ordinary_select_final_setting.sql | 1 - ...select_final_setting_both_tables.reference | 1 - ..._join_select_final_setting_both_tables.sql | 10 ----- ...in_select_final_setting_rhs_only.reference | 1 - ...ing_join_select_final_setting_rhs_only.sql | 10 ----- ...t_replacing_select_final_setting.reference | 1 - ...2420_mt_replacing_select_final_setting.sql | 6 --- 10 files changed, 74 insertions(+), 30 deletions(-) create mode 100644 tests/queries/0_stateless/02420_force_select_final_setting.reference create mode 100644 tests/queries/0_stateless/02420_force_select_final_setting.sql delete mode 100644 tests/queries/0_stateless/02420_mt_ordinary_select_final_setting.reference delete mode 100644 tests/queries/0_stateless/02420_mt_ordinary_select_final_setting.sql delete mode 100644 tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_both_tables.reference delete mode 100644 tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_both_tables.sql delete mode 100644 tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_rhs_only.reference delete mode 100644 tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_rhs_only.sql delete mode 100644 tests/queries/0_stateless/02420_mt_replacing_select_final_setting.reference delete mode 100644 tests/queries/0_stateless/02420_mt_replacing_select_final_setting.sql diff --git a/tests/queries/0_stateless/02420_force_select_final_setting.reference b/tests/queries/0_stateless/02420_force_select_final_setting.reference new file mode 100644 index 00000000000..466be527e6f --- /dev/null +++ b/tests/queries/0_stateless/02420_force_select_final_setting.reference @@ -0,0 +1,32 @@ +-- { echoOn } +create table if not exists ordinary_mt (x String) engine=MergeTree() ORDER BY x SETTINGS force_select_final=1; -- { serverError 181 } +-- simple test case +create table if not exists replacing_mt (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; +insert into replacing_mt values ('abc'); +insert into replacing_mt values ('abc'); +-- expected output is 1 because force_select_final is turned on +select count() from replacing_mt; +1 +-- JOIN test cases +create table if not exists lhs (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; +create table if not exists rhs (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; +insert into lhs values ('abc'); +insert into lhs values ('abc'); +insert into rhs values ('abc'); +insert into rhs values ('abc'); +-- expected output is 1 because both tables have force_select_final = 1 +select count() from lhs inner join rhs on lhs.x = rhs.x; +1 +drop table lhs; +drop table rhs; +create table if not exists lhs (x String) engine=ReplacingMergeTree() ORDER BY x; +create table if not exists rhs (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; +insert into lhs values ('abc'); +insert into lhs values ('abc'); +insert into rhs values ('abc'); +insert into rhs values ('abc'); +-- expected output is 2 because lhs table doesn't have final applied +select count() from lhs inner join rhs on lhs.x = rhs.x; +2 +-- Engine that does not support final +create table if not exists ordinary_mt (x String) engine=MergeTree() ORDER BY x SETTINGS force_select_final=1; -- { serverError 181 } diff --git a/tests/queries/0_stateless/02420_force_select_final_setting.sql b/tests/queries/0_stateless/02420_force_select_final_setting.sql new file mode 100644 index 00000000000..fae3a74e465 --- /dev/null +++ b/tests/queries/0_stateless/02420_force_select_final_setting.sql @@ -0,0 +1,42 @@ +-- { echoOn } +create table if not exists ordinary_mt (x String) engine=MergeTree() ORDER BY x SETTINGS force_select_final=1; -- { serverError 181 } + +-- simple test case +create table if not exists replacing_mt (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; + +insert into replacing_mt values ('abc'); +insert into replacing_mt values ('abc'); + +-- expected output is 1 because force_select_final is turned on +select count() from replacing_mt; + +-- JOIN test cases +create table if not exists lhs (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; +create table if not exists rhs (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; + +insert into lhs values ('abc'); +insert into lhs values ('abc'); + +insert into rhs values ('abc'); +insert into rhs values ('abc'); + +-- expected output is 1 because both tables have force_select_final = 1 +select count() from lhs inner join rhs on lhs.x = rhs.x; + +drop table lhs; +drop table rhs; + +create table if not exists lhs (x String) engine=ReplacingMergeTree() ORDER BY x; +create table if not exists rhs (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; + +insert into lhs values ('abc'); +insert into lhs values ('abc'); + +insert into rhs values ('abc'); +insert into rhs values ('abc'); + +-- expected output is 2 because lhs table doesn't have final applied +select count() from lhs inner join rhs on lhs.x = rhs.x; + +-- Engine that does not support final +create table if not exists ordinary_mt (x String) engine=MergeTree() ORDER BY x SETTINGS force_select_final=1; -- { serverError 181 } diff --git a/tests/queries/0_stateless/02420_mt_ordinary_select_final_setting.reference b/tests/queries/0_stateless/02420_mt_ordinary_select_final_setting.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02420_mt_ordinary_select_final_setting.sql b/tests/queries/0_stateless/02420_mt_ordinary_select_final_setting.sql deleted file mode 100644 index 6575de9c59b..00000000000 --- a/tests/queries/0_stateless/02420_mt_ordinary_select_final_setting.sql +++ /dev/null @@ -1 +0,0 @@ -create table if not exists ordinary_mt (x String) engine=MergeTree() ORDER BY x SETTINGS force_select_final=1; -- { serverError 181 } diff --git a/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_both_tables.reference b/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_both_tables.reference deleted file mode 100644 index d00491fd7e5..00000000000 --- a/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_both_tables.reference +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_both_tables.sql b/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_both_tables.sql deleted file mode 100644 index 5f7fdaa2f70..00000000000 --- a/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_both_tables.sql +++ /dev/null @@ -1,10 +0,0 @@ -create table if not exists lhs (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; -create table if not exists rhs (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; - -insert into lhs values ('abc'); -insert into lhs values ('abc'); - -insert into rhs values ('abc'); -insert into rhs values ('abc'); - -select count() from lhs inner join rhs on lhs.x = rhs.x; diff --git a/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_rhs_only.reference b/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_rhs_only.reference deleted file mode 100644 index 0cfbf08886f..00000000000 --- a/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_rhs_only.reference +++ /dev/null @@ -1 +0,0 @@ -2 diff --git a/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_rhs_only.sql b/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_rhs_only.sql deleted file mode 100644 index c03b8db9e83..00000000000 --- a/tests/queries/0_stateless/02420_mt_replacing_join_select_final_setting_rhs_only.sql +++ /dev/null @@ -1,10 +0,0 @@ -create table if not exists lhs (x String) engine=ReplacingMergeTree() ORDER BY x; -create table if not exists rhs (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; - -insert into lhs values ('abc'); -insert into lhs values ('abc'); - -insert into rhs values ('abc'); -insert into rhs values ('abc'); - -select count() from lhs inner join rhs on lhs.x = rhs.x; diff --git a/tests/queries/0_stateless/02420_mt_replacing_select_final_setting.reference b/tests/queries/0_stateless/02420_mt_replacing_select_final_setting.reference deleted file mode 100644 index d00491fd7e5..00000000000 --- a/tests/queries/0_stateless/02420_mt_replacing_select_final_setting.reference +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/tests/queries/0_stateless/02420_mt_replacing_select_final_setting.sql b/tests/queries/0_stateless/02420_mt_replacing_select_final_setting.sql deleted file mode 100644 index 3db1d5999b8..00000000000 --- a/tests/queries/0_stateless/02420_mt_replacing_select_final_setting.sql +++ /dev/null @@ -1,6 +0,0 @@ -create table if not exists replacing_mt (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; - -insert into replacing_mt values ('abc'); -insert into replacing_mt values ('abc'); - -select count() from replacing_mt From 1423eb0cbcd0d9e038370025e3f6a4f9b571c4f8 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 25 Oct 2022 11:09:59 -0300 Subject: [PATCH 011/253] Make ignore_force_select_final setting description more descriptive Co-authored-by: filimonov <1549571+filimonov@users.noreply.github.com> --- 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 e015806b562..04f444e2ab8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -269,7 +269,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Milliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \ M(Milliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \ \ - M(Bool, ignore_force_select_final, false, "Ignores the MT force_select_final setting", 0) \ + M(Bool, ignore_force_select_final, false, "Ignores the mergetree setting force_select_final, allowing to read the raw data (without applying FINAL automatically) is force_select_final is enabled", 0) \ \ /** Settings for testing hedged requests */ \ M(Milliseconds, sleep_in_send_tables_status_ms, 0, "Time to sleep in sending tables status response in TCPHandler", 0) \ From f2f165b612ae1c6e9f5f8aae290dcb4299c21a7d Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 26 Oct 2022 07:54:33 -0300 Subject: [PATCH 012/253] fix test --- .../02420_mt_ignore_force_select_final_setting.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.reference b/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.reference index ac177354f4a..1d270752668 100644 --- a/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.reference +++ b/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.reference @@ -5,8 +5,8 @@ SELECT * FROM tbl WHERE id = 10000; -- single row expected (bar), because the fo SELECT * FROM tbl final WHERE id = 10000; -- single row expected (bar), because the force_select_final is in action and FINAL is there. 10000 bar SELECT * FROM tbl WHERE id = 10000 SETTINGS ignore_force_select_final=1; -- now we see 2 'real' rows -10000 bar 10000 foo +10000 bar SELECT * FROM tbl FINAL WHERE id = 10000 SETTINGS ignore_force_select_final=1; -- now we see single row again. 10000 bar SYSTEM START MERGES tbl; From 5708780c44dbc9dbb952a149435826caeb3cfa42 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 31 Oct 2022 11:09:25 -0300 Subject: [PATCH 013/253] add order by to tests to guarantee ordering --- .../02420_mt_ignore_force_select_final_setting.reference | 2 +- .../0_stateless/02420_mt_ignore_force_select_final_setting.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.reference b/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.reference index 1d270752668..1ae1def4cb1 100644 --- a/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.reference +++ b/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.reference @@ -4,7 +4,7 @@ SELECT * FROM tbl WHERE id = 10000; -- single row expected (bar), because the fo 10000 bar SELECT * FROM tbl final WHERE id = 10000; -- single row expected (bar), because the force_select_final is in action and FINAL is there. 10000 bar -SELECT * FROM tbl WHERE id = 10000 SETTINGS ignore_force_select_final=1; -- now we see 2 'real' rows +SELECT * FROM tbl WHERE id = 10000 ORDER BY val DESC SETTINGS ignore_force_select_final=1; -- now we see 2 'real' rows 10000 foo 10000 bar SELECT * FROM tbl FINAL WHERE id = 10000 SETTINGS ignore_force_select_final=1; -- now we see single row again. diff --git a/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.sql b/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.sql index fdcdea2bc23..87ff5c789bd 100644 --- a/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.sql +++ b/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.sql @@ -9,7 +9,7 @@ INSERT INTO tbl SELECT number as id, 'bar' AS val FROM numbers(100000); SELECT * FROM tbl WHERE id = 10000; -- single row expected (bar), because the force_select_final is in action SELECT * FROM tbl final WHERE id = 10000; -- single row expected (bar), because the force_select_final is in action and FINAL is there. -SELECT * FROM tbl WHERE id = 10000 SETTINGS ignore_force_select_final=1; -- now we see 2 'real' rows +SELECT * FROM tbl WHERE id = 10000 ORDER BY val DESC SETTINGS ignore_force_select_final=1; -- now we see 2 'real' rows SELECT * FROM tbl FINAL WHERE id = 10000 SETTINGS ignore_force_select_final=1; -- now we see single row again. SYSTEM START MERGES tbl; From 27d63405f28867b4f87300c0e34422d1037c5fec Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 17 Nov 2022 10:21:46 -0300 Subject: [PATCH 014/253] Silently fail when force_select_final setting is used with MT engines that do not support it --- src/Storages/MergeTree/registerStorageMergeTree.cpp | 6 ------ .../0_stateless/02420_force_select_final_setting.sql | 4 ---- 2 files changed, 10 deletions(-) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index f375a53b42a..659fe5246f1 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -682,12 +682,6 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (arg_num != arg_cnt) throw Exception("Wrong number of engine arguments.", ErrorCodes::BAD_ARGUMENTS); - - if (merging_params.mode == MergeTreeData::MergingParams::Mode::Ordinary && storage_settings->force_select_final) - { - throw Exception("Storage MergeTree doesn't support FINAL", ErrorCodes::ILLEGAL_FINAL); - } - if (replicated) { return std::make_shared( diff --git a/tests/queries/0_stateless/02420_force_select_final_setting.sql b/tests/queries/0_stateless/02420_force_select_final_setting.sql index fae3a74e465..8dcaa6bf934 100644 --- a/tests/queries/0_stateless/02420_force_select_final_setting.sql +++ b/tests/queries/0_stateless/02420_force_select_final_setting.sql @@ -1,5 +1,4 @@ -- { echoOn } -create table if not exists ordinary_mt (x String) engine=MergeTree() ORDER BY x SETTINGS force_select_final=1; -- { serverError 181 } -- simple test case create table if not exists replacing_mt (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; @@ -37,6 +36,3 @@ insert into rhs values ('abc'); -- expected output is 2 because lhs table doesn't have final applied select count() from lhs inner join rhs on lhs.x = rhs.x; - --- Engine that does not support final -create table if not exists ordinary_mt (x String) engine=MergeTree() ORDER BY x SETTINGS force_select_final=1; -- { serverError 181 } From 5b84a4fb090990d54195a96f47826dcbb69ca312 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 17 Nov 2022 10:32:50 -0300 Subject: [PATCH 015/253] remove illegal final definition --- src/Storages/MergeTree/registerStorageMergeTree.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 659fe5246f1..6982521f76a 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -34,7 +34,6 @@ namespace ErrorCodes extern const int UNKNOWN_STORAGE; extern const int NO_REPLICA_NAME_GIVEN; extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; - extern const int ILLEGAL_FINAL; } From e33b1a39da83330f43c1b0aab4866ddbb993ba4c Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 17 Nov 2022 13:32:22 -0300 Subject: [PATCH 016/253] fix tst --- .../0_stateless/02420_force_select_final_setting.reference | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/queries/0_stateless/02420_force_select_final_setting.reference b/tests/queries/0_stateless/02420_force_select_final_setting.reference index 466be527e6f..f9044ba05aa 100644 --- a/tests/queries/0_stateless/02420_force_select_final_setting.reference +++ b/tests/queries/0_stateless/02420_force_select_final_setting.reference @@ -1,5 +1,4 @@ -- { echoOn } -create table if not exists ordinary_mt (x String) engine=MergeTree() ORDER BY x SETTINGS force_select_final=1; -- { serverError 181 } -- simple test case create table if not exists replacing_mt (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; insert into replacing_mt values ('abc'); @@ -28,5 +27,3 @@ insert into rhs values ('abc'); -- expected output is 2 because lhs table doesn't have final applied select count() from lhs inner join rhs on lhs.x = rhs.x; 2 --- Engine that does not support final -create table if not exists ordinary_mt (x String) engine=MergeTree() ORDER BY x SETTINGS force_select_final=1; -- { serverError 181 } From e9b19ffa847e8f19c0ad4b63ed127c9847221f05 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 17 Nov 2022 14:05:21 -0300 Subject: [PATCH 017/253] Update reference file --- tests/queries/0_stateless/02420_force_select_final_setting.sql | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02420_force_select_final_setting.sql b/tests/queries/0_stateless/02420_force_select_final_setting.sql index 8dcaa6bf934..de91db071e1 100644 --- a/tests/queries/0_stateless/02420_force_select_final_setting.sql +++ b/tests/queries/0_stateless/02420_force_select_final_setting.sql @@ -1,5 +1,4 @@ -- { echoOn } - -- simple test case create table if not exists replacing_mt (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; From 6ee21e63d7dc1e84cfa7e40b25545937f356d73f Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 25 Nov 2022 13:31:08 -0300 Subject: [PATCH 018/253] Move force select final to user lvel setting and remove ignore force select final --- src/Core/Settings.h | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Storages/IStorage.h | 2 -- src/Storages/MergeTree/MergeTreeData.h | 4 --- src/Storages/MergeTree/MergeTreeSettings.h | 2 -- ...02420_force_select_final_setting.reference | 30 ++++++++--------- .../02420_force_select_final_setting.sql | 32 ++++++++----------- ...gnore_force_select_final_setting.reference | 16 ---------- ...0_mt_ignore_force_select_final_setting.sql | 19 ----------- 9 files changed, 31 insertions(+), 78 deletions(-) delete mode 100644 tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.reference delete mode 100644 tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 591557a6031..8e5b9d5c690 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -274,7 +274,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Milliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \ M(Milliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \ \ - M(Bool, ignore_force_select_final, false, "Ignores the mergetree setting force_select_final, allowing to read the raw data (without applying FINAL automatically) is force_select_final is enabled", 0) \ + M(Bool, force_select_final, false, "Query with the FINAL modifier by default", 0) \ \ /** Settings for testing hedged requests */ \ M(Milliseconds, sleep_in_send_tables_status_ms, 0, "Time to sleep in sending tables status response in TCPHandler", 0) \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 766c595b2f1..d4183da2f11 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -500,7 +500,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( settings.additional_table_filters, joined_tables.tablesWithColumns().front().table, *context); - if (!query.final() && storage && storage->forceQueryWithFinal() && !context->getSettingsRef().ignore_force_select_final) + if (!query.final() && context->getSettingsRef().force_select_final) { query.setFinal(); } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 186addbbd71..fd48d22b12b 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -329,8 +329,6 @@ public: /// It's needed for ReplacingMergeTree wrappers such as MaterializedMySQL and MaterializedPostrgeSQL virtual bool needRewriteQueryWithFinal(const Names & /*column_names*/) const { return false; } - virtual bool forceQueryWithFinal() const { return false; } - private: /** Read a set of columns from the table. * Accepts a list of columns to read, as well as a description of the query, diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index b572423859e..8bd0fc1f280 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -437,10 +437,6 @@ public: bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, ContextPtr, const StorageMetadataPtr & metadata_snapshot) const override; - bool forceQueryWithFinal() const override - { - return getSettings()->force_select_final && supportsFinal(); - } /// Snapshot for MergeTree contains the current set of data parts /// at the moment of the start of query. struct SnapshotData : public StorageSnapshot::Data diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 3099adaa48b..998e870484c 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -147,8 +147,6 @@ struct Settings; M(Bool, allow_remote_fs_zero_copy_replication, false, "Don't use this setting in production, because it is not ready.", 0) \ M(String, remote_fs_zero_copy_zookeeper_path, "/clickhouse/zero_copy", "ZooKeeper path for Zero-copy table-independet info.", 0) \ M(Bool, remote_fs_zero_copy_path_compatible_mode, false, "Run zero-copy in compatible mode during conversion process.", 0) \ - M(Bool, force_select_final, false, "Query with the FINAL modifier by default", 0) \ - \ /** Compress marks and primary key. */ \ M(Bool, compress_marks, false, "Marks support compression, reduce mark file size and speed up network transmission.", 0) \ M(Bool, compress_primary_key, false, "Primary key support compression, reduce primary key file size and speed up network transmission.", 0) \ diff --git a/tests/queries/0_stateless/02420_force_select_final_setting.reference b/tests/queries/0_stateless/02420_force_select_final_setting.reference index f9044ba05aa..ab9cf9225ef 100644 --- a/tests/queries/0_stateless/02420_force_select_final_setting.reference +++ b/tests/queries/0_stateless/02420_force_select_final_setting.reference @@ -1,29 +1,29 @@ -- { echoOn } + +SYSTEM STOP MERGES tbl; -- simple test case -create table if not exists replacing_mt (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; +create table if not exists replacing_mt (x String) engine=ReplacingMergeTree() ORDER BY x; insert into replacing_mt values ('abc'); insert into replacing_mt values ('abc'); +-- expected output is 2 because force_select_final is turned off +select count() from replacing_mt; +2 +set force_select_final = 1; -- expected output is 1 because force_select_final is turned on select count() from replacing_mt; 1 -- JOIN test cases -create table if not exists lhs (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; -create table if not exists rhs (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; +create table if not exists lhs (x String) engine=ReplacingMergeTree() ORDER BY x; +create table if not exists rhs (x String) engine=ReplacingMergeTree() ORDER BY x; insert into lhs values ('abc'); insert into lhs values ('abc'); insert into rhs values ('abc'); insert into rhs values ('abc'); --- expected output is 1 because both tables have force_select_final = 1 +set force_select_final = 0; +-- expected output is 4 because select_final == 0 +select count() from lhs inner join rhs on lhs.x = rhs.x; +4 +set force_select_final = 1; +-- expected output is 1 because force_select_final == 1 select count() from lhs inner join rhs on lhs.x = rhs.x; 1 -drop table lhs; -drop table rhs; -create table if not exists lhs (x String) engine=ReplacingMergeTree() ORDER BY x; -create table if not exists rhs (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; -insert into lhs values ('abc'); -insert into lhs values ('abc'); -insert into rhs values ('abc'); -insert into rhs values ('abc'); --- expected output is 2 because lhs table doesn't have final applied -select count() from lhs inner join rhs on lhs.x = rhs.x; -2 diff --git a/tests/queries/0_stateless/02420_force_select_final_setting.sql b/tests/queries/0_stateless/02420_force_select_final_setting.sql index de91db071e1..ae925a5dd14 100644 --- a/tests/queries/0_stateless/02420_force_select_final_setting.sql +++ b/tests/queries/0_stateless/02420_force_select_final_setting.sql @@ -1,31 +1,22 @@ -- { echoOn } +SYSTEM STOP MERGES tbl; + -- simple test case -create table if not exists replacing_mt (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; +create table if not exists replacing_mt (x String) engine=ReplacingMergeTree() ORDER BY x; insert into replacing_mt values ('abc'); insert into replacing_mt values ('abc'); +-- expected output is 2 because force_select_final is turned off +select count() from replacing_mt; + +set force_select_final = 1; -- expected output is 1 because force_select_final is turned on select count() from replacing_mt; -- JOIN test cases -create table if not exists lhs (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; -create table if not exists rhs (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; - -insert into lhs values ('abc'); -insert into lhs values ('abc'); - -insert into rhs values ('abc'); -insert into rhs values ('abc'); - --- expected output is 1 because both tables have force_select_final = 1 -select count() from lhs inner join rhs on lhs.x = rhs.x; - -drop table lhs; -drop table rhs; - create table if not exists lhs (x String) engine=ReplacingMergeTree() ORDER BY x; -create table if not exists rhs (x String) engine=ReplacingMergeTree() ORDER BY x SETTINGS force_select_final=1; +create table if not exists rhs (x String) engine=ReplacingMergeTree() ORDER BY x; insert into lhs values ('abc'); insert into lhs values ('abc'); @@ -33,5 +24,10 @@ insert into lhs values ('abc'); insert into rhs values ('abc'); insert into rhs values ('abc'); --- expected output is 2 because lhs table doesn't have final applied +set force_select_final = 0; +-- expected output is 4 because select_final == 0 +select count() from lhs inner join rhs on lhs.x = rhs.x; + +set force_select_final = 1; +-- expected output is 1 because force_select_final == 1 select count() from lhs inner join rhs on lhs.x = rhs.x; diff --git a/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.reference b/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.reference deleted file mode 100644 index 1ae1def4cb1..00000000000 --- a/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.reference +++ /dev/null @@ -1,16 +0,0 @@ --- { echoOn } - -SELECT * FROM tbl WHERE id = 10000; -- single row expected (bar), because the force_select_final is in action -10000 bar -SELECT * FROM tbl final WHERE id = 10000; -- single row expected (bar), because the force_select_final is in action and FINAL is there. -10000 bar -SELECT * FROM tbl WHERE id = 10000 ORDER BY val DESC SETTINGS ignore_force_select_final=1; -- now we see 2 'real' rows -10000 foo -10000 bar -SELECT * FROM tbl FINAL WHERE id = 10000 SETTINGS ignore_force_select_final=1; -- now we see single row again. -10000 bar -SYSTEM START MERGES tbl; -OPTIMIZE TABLE tbl FINAL; -SELECT * FROM tbl WHERE id = 10000 SETTINGS ignore_force_select_final=1; -- now we see single row DROP TABLE tbl; -10000 bar -DROP TABLE tbl; diff --git a/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.sql b/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.sql deleted file mode 100644 index 87ff5c789bd..00000000000 --- a/tests/queries/0_stateless/02420_mt_ignore_force_select_final_setting.sql +++ /dev/null @@ -1,19 +0,0 @@ -CREATE TABLE tbl (id UInt64, val String) engine=ReplacingMergeTree() ORDER BY id SETTINGS force_select_final=1; - -SYSTEM STOP MERGES tbl; -INSERT INTO tbl SELECT number as id, 'foo' AS val FROM numbers(100000); -INSERT INTO tbl SELECT number as id, 'bar' AS val FROM numbers(100000); - --- { echoOn } - -SELECT * FROM tbl WHERE id = 10000; -- single row expected (bar), because the force_select_final is in action -SELECT * FROM tbl final WHERE id = 10000; -- single row expected (bar), because the force_select_final is in action and FINAL is there. - -SELECT * FROM tbl WHERE id = 10000 ORDER BY val DESC SETTINGS ignore_force_select_final=1; -- now we see 2 'real' rows -SELECT * FROM tbl FINAL WHERE id = 10000 SETTINGS ignore_force_select_final=1; -- now we see single row again. - -SYSTEM START MERGES tbl; -OPTIMIZE TABLE tbl FINAL; - -SELECT * FROM tbl WHERE id = 10000 SETTINGS ignore_force_select_final=1; -- now we see single row DROP TABLE tbl; -DROP TABLE tbl; \ No newline at end of file From 21449847361438f018e3e22bae0bc923e432592d Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 25 Nov 2022 15:01:38 -0300 Subject: [PATCH 019/253] fix tests --- .../0_stateless/02420_force_select_final_setting.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02420_force_select_final_setting.reference b/tests/queries/0_stateless/02420_force_select_final_setting.reference index ab9cf9225ef..36b8905289b 100644 --- a/tests/queries/0_stateless/02420_force_select_final_setting.reference +++ b/tests/queries/0_stateless/02420_force_select_final_setting.reference @@ -1,5 +1,4 @@ -- { echoOn } - SYSTEM STOP MERGES tbl; -- simple test case create table if not exists replacing_mt (x String) engine=ReplacingMergeTree() ORDER BY x; From eb3f4c9fe25792924b8bfeb8356728d0d2c6f09f Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 29 Nov 2022 13:14:06 -0300 Subject: [PATCH 020/253] prevent force select final from affecting non table based queries --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d4183da2f11..1d5eeb706f6 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -500,7 +500,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( settings.additional_table_filters, joined_tables.tablesWithColumns().front().table, *context); - if (!query.final() && context->getSettingsRef().force_select_final) + if (context->getSettingsRef().force_select_final && !query.final() && query.tables()) { query.setFinal(); } From 252575ee23d8151b3a7c5435c82616cd6ff7fbe9 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 29 Nov 2022 13:14:52 -0300 Subject: [PATCH 021/253] doc --- src/Interpreters/InterpreterSelectQuery.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 1d5eeb706f6..da08ee8d389 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -500,6 +500,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( settings.additional_table_filters, joined_tables.tablesWithColumns().front().table, *context); + // query.tables() is required because not all queries have tables in it, it could be a function. if (context->getSettingsRef().force_select_final && !query.final() && query.tables()) { query.setFinal(); From 32395165219fbe37ceffde0eed4482368273d755 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 5 Dec 2022 10:18:44 -0300 Subject: [PATCH 022/253] apply final only on storages that support it --- src/Interpreters/InterpreterSelectQuery.cpp | 13 ++++++++++--- src/Interpreters/InterpreterSelectQuery.h | 1 + .../02420_force_select_final_setting.reference | 7 +++++++ .../02420_force_select_final_setting.sql | 7 +++++++ 4 files changed, 25 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index da08ee8d389..6e7259a98e3 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -499,9 +499,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( query_info.additional_filter_ast = parseAdditionalFilterConditionForTable( settings.additional_table_filters, joined_tables.tablesWithColumns().front().table, *context); - - // query.tables() is required because not all queries have tables in it, it could be a function. - if (context->getSettingsRef().force_select_final && !query.final() && query.tables()) + if (forceSelectFinalOnSelectQuery(query)) { query.setFinal(); } @@ -2916,6 +2914,15 @@ void InterpreterSelectQuery::ignoreWithTotals() getSelectQuery().group_by_with_totals = false; } +bool InterpreterSelectQuery::forceSelectFinalOnSelectQuery(ASTSelectQuery & query) +{ + // query.tables() is required because not all queries have tables in it, it could be a function. + auto isFinalSupported = storage && storage->supportsFinal() && query.tables(); + auto isForceSelectFinalSettingOn = context->getSettingsRef().force_select_final; + auto isQueryAlreadyFinal = query.final(); + + return isForceSelectFinalSettingOn && !isQueryAlreadyFinal && isFinalSupported; +} void InterpreterSelectQuery::initSettings() { diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 761eea8e1b8..f679051532c 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -194,6 +194,7 @@ private: void executeDistinct(QueryPlan & query_plan, bool before_order, Names columns, bool pre_distinct); void executeExtremes(QueryPlan & query_plan); void executeSubqueriesInSetsAndJoins(QueryPlan & query_plan); + bool forceSelectFinalOnSelectQuery(ASTSelectQuery & select_query); enum class Modificator { diff --git a/tests/queries/0_stateless/02420_force_select_final_setting.reference b/tests/queries/0_stateless/02420_force_select_final_setting.reference index 36b8905289b..ad1a0bc1567 100644 --- a/tests/queries/0_stateless/02420_force_select_final_setting.reference +++ b/tests/queries/0_stateless/02420_force_select_final_setting.reference @@ -26,3 +26,10 @@ set force_select_final = 1; -- expected output is 1 because force_select_final == 1 select count() from lhs inner join rhs on lhs.x = rhs.x; 1 +-- regular non final table +set force_select_final = 1; +create table if not exists regular_mt_table (x String) engine=MergeTree() ORDER BY x; +insert into regular_mt_table values ('abc'); +-- expected output is 1, it should silently ignore final modifier +select count() from regular_mt_table; +1 diff --git a/tests/queries/0_stateless/02420_force_select_final_setting.sql b/tests/queries/0_stateless/02420_force_select_final_setting.sql index ae925a5dd14..7dd7eb05b12 100644 --- a/tests/queries/0_stateless/02420_force_select_final_setting.sql +++ b/tests/queries/0_stateless/02420_force_select_final_setting.sql @@ -31,3 +31,10 @@ select count() from lhs inner join rhs on lhs.x = rhs.x; set force_select_final = 1; -- expected output is 1 because force_select_final == 1 select count() from lhs inner join rhs on lhs.x = rhs.x; + +-- regular non final table +set force_select_final = 1; +create table if not exists regular_mt_table (x String) engine=MergeTree() ORDER BY x; +insert into regular_mt_table values ('abc'); +-- expected output is 1, it should silently ignore final modifier +select count() from regular_mt_table; From 1cf8dc6daaf1876064ca2754a621b96ee02ef8b7 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 5 Dec 2022 13:21:50 -0300 Subject: [PATCH 023/253] add view test --- .../02420_force_select_final_setting.reference | 9 ++++++++- .../0_stateless/02420_force_select_final_setting.sql | 8 ++++++++ 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02420_force_select_final_setting.reference b/tests/queries/0_stateless/02420_force_select_final_setting.reference index ad1a0bc1567..8be3da3545c 100644 --- a/tests/queries/0_stateless/02420_force_select_final_setting.reference +++ b/tests/queries/0_stateless/02420_force_select_final_setting.reference @@ -30,6 +30,13 @@ select count() from lhs inner join rhs on lhs.x = rhs.x; set force_select_final = 1; create table if not exists regular_mt_table (x String) engine=MergeTree() ORDER BY x; insert into regular_mt_table values ('abc'); +insert into regular_mt_table values ('abc'); -- expected output is 1, it should silently ignore final modifier select count() from regular_mt_table; -1 +2 +-- view test +create materialized VIEW mv_regular_mt_table TO regular_mt_table AS SELECT * FROM regular_mt_table; +create view nv_regular_mt_table AS SELECT * FROM mv_regular_mt_table; +set force_select_final=1; +select count() from nv_regular_mt_table; +2 diff --git a/tests/queries/0_stateless/02420_force_select_final_setting.sql b/tests/queries/0_stateless/02420_force_select_final_setting.sql index 7dd7eb05b12..84d4c40445a 100644 --- a/tests/queries/0_stateless/02420_force_select_final_setting.sql +++ b/tests/queries/0_stateless/02420_force_select_final_setting.sql @@ -36,5 +36,13 @@ select count() from lhs inner join rhs on lhs.x = rhs.x; set force_select_final = 1; create table if not exists regular_mt_table (x String) engine=MergeTree() ORDER BY x; insert into regular_mt_table values ('abc'); +insert into regular_mt_table values ('abc'); -- expected output is 1, it should silently ignore final modifier select count() from regular_mt_table; + +-- view test +create materialized VIEW mv_regular_mt_table TO regular_mt_table AS SELECT * FROM regular_mt_table; +create view nv_regular_mt_table AS SELECT * FROM mv_regular_mt_table; + +set force_select_final=1; +select count() from nv_regular_mt_table; From 7f8a4f8d8779bb1192e53b7a7605885e7bac28e5 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 6 Dec 2022 17:28:30 -0300 Subject: [PATCH 024/253] do not apply final on distributed engine, only on underlying --- src/Interpreters/InterpreterSelectQuery.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6e7259a98e3..9e5b462cef4 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2917,11 +2917,11 @@ void InterpreterSelectQuery::ignoreWithTotals() bool InterpreterSelectQuery::forceSelectFinalOnSelectQuery(ASTSelectQuery & query) { // query.tables() is required because not all queries have tables in it, it could be a function. - auto isFinalSupported = storage && storage->supportsFinal() && query.tables(); - auto isForceSelectFinalSettingOn = context->getSettingsRef().force_select_final; - auto isQueryAlreadyFinal = query.final(); + auto is_force_select_final_setting_on = context->getSettingsRef().force_select_final; + auto is_final_supported = storage && storage->supportsFinal() && !storage->isRemote() && query.tables(); + auto is_query_already_final = query.final(); - return isForceSelectFinalSettingOn && !isQueryAlreadyFinal && isFinalSupported; + return is_force_select_final_setting_on && !is_query_already_final && is_final_supported; } void InterpreterSelectQuery::initSettings() From 301b684933f54a6ec1c75958c7eba443e0b3c9ca Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 12 Jan 2023 10:00:18 -0300 Subject: [PATCH 025/253] remove trialing whitespace? --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index bc4e8c27b82..d8242953d60 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6087,7 +6087,7 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg /// Cannot use projections in case of additional filter. if (query_info.additional_filter_ast) return std::nullopt; - + auto query_ptr = query_info.query; auto original_query_ptr = query_info.original_query; From f2fab484409e49b77fcbcf45dcc3f52ff75d7522 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 30 Jan 2023 20:33:18 -0300 Subject: [PATCH 026/253] add join on mix of tables supporting/ not supporting final --- ...02420_force_select_final_setting.reference | 17 +++++++++++++++ .../02420_force_select_final_setting.sql | 21 +++++++++++++++++++ 2 files changed, 38 insertions(+) diff --git a/tests/queries/0_stateless/02420_force_select_final_setting.reference b/tests/queries/0_stateless/02420_force_select_final_setting.reference index 8be3da3545c..1fe561cd267 100644 --- a/tests/queries/0_stateless/02420_force_select_final_setting.reference +++ b/tests/queries/0_stateless/02420_force_select_final_setting.reference @@ -40,3 +40,20 @@ create view nv_regular_mt_table AS SELECT * FROM mv_regular_mt_table; set force_select_final=1; select count() from nv_regular_mt_table; 2 +-- join on mix of tables that support / do not support select final +create table if not exists left_table (x String) engine=ReplacingMergeTree() ORDER BY x; +create table if not exists middle_table (x String) engine=MergeTree() ORDER BY x; +create table if not exists right_table (x String) engine=ReplacingMergeTree() ORDER BY x; +insert into left_table values ('abc'); +insert into left_table values ('abc'); +insert into left_table values ('abc'); +insert into middle_table values ('abc'); +insert into middle_table values ('abc'); +insert into right_table values ('abc'); +insert into right_table values ('abc'); +insert into right_table values ('abc'); +-- Expected output is 2 because middle table does not support final +select count() from left_table + inner join middle_table on left_table.x = middle_table.x + inner join right_table on middle_table.x = right_table.x; +2 diff --git a/tests/queries/0_stateless/02420_force_select_final_setting.sql b/tests/queries/0_stateless/02420_force_select_final_setting.sql index 84d4c40445a..c074d1a8926 100644 --- a/tests/queries/0_stateless/02420_force_select_final_setting.sql +++ b/tests/queries/0_stateless/02420_force_select_final_setting.sql @@ -46,3 +46,24 @@ create view nv_regular_mt_table AS SELECT * FROM mv_regular_mt_table; set force_select_final=1; select count() from nv_regular_mt_table; + +-- join on mix of tables that support / do not support select final +create table if not exists left_table (x String) engine=ReplacingMergeTree() ORDER BY x; +create table if not exists middle_table (x String) engine=MergeTree() ORDER BY x; +create table if not exists right_table (x String) engine=ReplacingMergeTree() ORDER BY x; + +insert into left_table values ('abc'); +insert into left_table values ('abc'); +insert into left_table values ('abc'); + +insert into middle_table values ('abc'); +insert into middle_table values ('abc'); + +insert into right_table values ('abc'); +insert into right_table values ('abc'); +insert into right_table values ('abc'); + +-- Expected output is 2 because middle table does not support final +select count() from left_table + inner join middle_table on left_table.x = middle_table.x + inner join right_table on middle_table.x = right_table.x; From f194b452e7c5120c31d1108ef17957565aad24b5 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 31 Jan 2023 09:13:57 -0300 Subject: [PATCH 027/253] auto -> bool --- src/Interpreters/InterpreterSelectQuery.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 250d92cbe4c..12b6f6f3297 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -3010,9 +3010,9 @@ void InterpreterSelectQuery::ignoreWithTotals() bool InterpreterSelectQuery::forceSelectFinalOnSelectQuery(ASTSelectQuery & query) { // query.tables() is required because not all queries have tables in it, it could be a function. - auto is_force_select_final_setting_on = context->getSettingsRef().force_select_final; - auto is_final_supported = storage && storage->supportsFinal() && !storage->isRemote() && query.tables(); - auto is_query_already_final = query.final(); + bool is_force_select_final_setting_on = context->getSettingsRef().force_select_final; + bool is_final_supported = storage && storage->supportsFinal() && !storage->isRemote() && query.tables(); + bool is_query_already_final = query.final(); return is_force_select_final_setting_on && !is_query_already_final && is_final_supported; } From aef4154b8b51f2ce34c9c11de7727b232cbf2a42 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 31 Jan 2023 12:32:26 -0300 Subject: [PATCH 028/253] add and improve tests, mix of join, distributed and subqueries --- ...02420_force_select_final_setting.reference | 144 +++++++++++++++--- .../02420_force_select_final_setting.sql | 100 ++++++++++-- 2 files changed, 211 insertions(+), 33 deletions(-) diff --git a/tests/queries/0_stateless/02420_force_select_final_setting.reference b/tests/queries/0_stateless/02420_force_select_final_setting.reference index 1fe561cd267..7194b90f775 100644 --- a/tests/queries/0_stateless/02420_force_select_final_setting.reference +++ b/tests/queries/0_stateless/02420_force_select_final_setting.reference @@ -40,20 +40,130 @@ create view nv_regular_mt_table AS SELECT * FROM mv_regular_mt_table; set force_select_final=1; select count() from nv_regular_mt_table; 2 --- join on mix of tables that support / do not support select final -create table if not exists left_table (x String) engine=ReplacingMergeTree() ORDER BY x; -create table if not exists middle_table (x String) engine=MergeTree() ORDER BY x; -create table if not exists right_table (x String) engine=ReplacingMergeTree() ORDER BY x; -insert into left_table values ('abc'); -insert into left_table values ('abc'); -insert into left_table values ('abc'); -insert into middle_table values ('abc'); -insert into middle_table values ('abc'); -insert into right_table values ('abc'); -insert into right_table values ('abc'); -insert into right_table values ('abc'); --- Expected output is 2 because middle table does not support final -select count() from left_table - inner join middle_table on left_table.x = middle_table.x - inner join right_table on middle_table.x = right_table.x; -2 +-- join on mix of tables that support / do not support select final with explain +create table if not exists left_table (id UInt64, val_left String) engine=ReplacingMergeTree() ORDER BY id; +create table if not exists middle_table (id UInt64, val_middle String) engine=MergeTree() ORDER BY id; +create table if not exists right_table (id UInt64, val_right String) engine=ReplacingMergeTree() ORDER BY id; +insert into left_table values (1,'a'); +insert into left_table values (1,'b'); +insert into left_table values (1,'c'); +insert into middle_table values (1,'a'); +insert into middle_table values (1,'b'); +insert into right_table values (1,'a'); +insert into right_table values (1,'b'); +insert into right_table values (1,'c'); +-- expected output +-- 1 c a c +-- 1 c b c +select left_table.id,val_left, val_middle, val_right from left_table + inner join middle_table on left_table.id = middle_table.id + inner join right_table on middle_table.id = right_table.id +ORDER BY left_table.id, val_left, val_middle, val_right; +1 c a c +1 c b c +explain syntax select left_table.id,val_left, val_middle, val_right from left_table + inner join middle_table on left_table.id = middle_table.id + inner join right_table on middle_table.id = right_table.id +ORDER BY left_table.id, val_left, val_middle, val_right; +SELECT + `--left_table.id` AS `left_table.id`, + val_left, + val_middle, + val_right +FROM +( + SELECT + val_left, + id AS `--left_table.id`, + val_middle, + middle_table.id AS `--middle_table.id` + FROM left_table + FINAL + ALL INNER JOIN + ( + SELECT + val_middle, + id + FROM middle_table + ) AS middle_table ON `--left_table.id` = `--middle_table.id` +) AS `--.s` +ALL INNER JOIN +( + SELECT + val_right, + id + FROM right_table + FINAL +) AS right_table ON `--middle_table.id` = id +ORDER BY + `--left_table.id` ASC, + val_left ASC, + val_middle ASC, + val_right ASC +-- extra: same with subquery +select left_table.id,val_left, val_middle, val_right from left_table + inner join middle_table on left_table.id = middle_table.id + inner join (SELECT * FROM right_table WHERE id = 1) r on middle_table.id = r.id +ORDER BY left_table.id, val_left, val_middle, val_right; +1 c a c +1 c b c +-- distributed tables +drop table if exists left_table; +drop table if exists middle_table; +drop table if exists right_table; +create table if not exists left_table (id UInt64, val_left String) engine=ReplacingMergeTree() ORDER BY id; +create table if not exists middle_table (id UInt64, val_middle String) engine=MergeTree() ORDER BY id; +create table if not exists right_table_local (id UInt64, val_right String) engine=ReplacingMergeTree() ORDER BY id; +create table if not exists right_table engine=Distributed('test_shard_localhost', currentDatabase(), right_table_local) AS right_table_local; +insert into left_table values (1,'a'); +insert into left_table values (1,'b'); +insert into left_table values (1,'c'); +insert into middle_table values (1,'a'); +insert into middle_table values (1,'b'); +insert into right_table_local values (1,'a'); +insert into right_table_local values (1,'b'); +insert into right_table_local values (1,'c'); +SET prefer_localhost_replica=0; +-- expected output: +-- 1 c 1 a 1 c +-- 1 c 1 b 1 c +select left_table.*,middle_table.*, right_table.* from left_table + inner join middle_table on left_table.id = middle_table.id + inner join right_table on middle_table.id = right_table.id +ORDER BY left_table.id, val_left, val_middle, val_right; +1 c 1 a 1 c +1 c 1 b 1 c +SET prefer_localhost_replica=1; +-- expected output: +-- 1 c 1 a 1 c +-- 1 c 1 b 1 c +select left_table.*,middle_table.*, right_table.* from left_table + inner join middle_table on left_table.id = middle_table.id + inner join right_table on middle_table.id = right_table.id +ORDER BY left_table.id, val_left, val_middle, val_right; +1 c 1 a 1 c +1 c 1 b 1 c +-- Quite exotic with Merge engine +DROP TABLE IF EXISTS table_to_merge_a; +DROP TABLE IF EXISTS table_to_merge_b; +DROP TABLE IF EXISTS table_to_merge_c; +DROP TABLE IF EXISTS merge_table; +create table if not exists table_to_merge_a (id UInt64, val String) engine=ReplacingMergeTree() ORDER BY id; +create table if not exists table_to_merge_b (id UInt64, val String) engine=MergeTree() ORDER BY id; +create table if not exists table_to_merge_c (id UInt64, val String) engine=ReplacingMergeTree() ORDER BY id; +CREATE TABLE merge_table Engine=Merge(currentDatabase(), '^(table_to_merge_[a-z])$') AS table_to_merge_a; +insert into table_to_merge_a values (1,'a'); +insert into table_to_merge_a values (1,'b'); +insert into table_to_merge_a values (1,'c'); +insert into table_to_merge_b values (2,'a'); +insert into table_to_merge_b values (2,'b'); +insert into table_to_merge_c values (3,'a'); +insert into table_to_merge_c values (3,'b'); +insert into table_to_merge_c values (3,'c'); +-- expected output: +-- 1 c, 2 a, 2 b, 3 c +SELECT * FROM merge_table ORDER BY id, val; +1 c +2 a +2 b +3 c diff --git a/tests/queries/0_stateless/02420_force_select_final_setting.sql b/tests/queries/0_stateless/02420_force_select_final_setting.sql index c074d1a8926..c60bc3c7f53 100644 --- a/tests/queries/0_stateless/02420_force_select_final_setting.sql +++ b/tests/queries/0_stateless/02420_force_select_final_setting.sql @@ -47,23 +47,91 @@ create view nv_regular_mt_table AS SELECT * FROM mv_regular_mt_table; set force_select_final=1; select count() from nv_regular_mt_table; --- join on mix of tables that support / do not support select final -create table if not exists left_table (x String) engine=ReplacingMergeTree() ORDER BY x; -create table if not exists middle_table (x String) engine=MergeTree() ORDER BY x; -create table if not exists right_table (x String) engine=ReplacingMergeTree() ORDER BY x; +-- join on mix of tables that support / do not support select final with explain +create table if not exists left_table (id UInt64, val_left String) engine=ReplacingMergeTree() ORDER BY id; +create table if not exists middle_table (id UInt64, val_middle String) engine=MergeTree() ORDER BY id; +create table if not exists right_table (id UInt64, val_right String) engine=ReplacingMergeTree() ORDER BY id; +insert into left_table values (1,'a'); +insert into left_table values (1,'b'); +insert into left_table values (1,'c'); +insert into middle_table values (1,'a'); +insert into middle_table values (1,'b'); +insert into right_table values (1,'a'); +insert into right_table values (1,'b'); +insert into right_table values (1,'c'); +-- expected output +-- 1 c a c +-- 1 c b c +select left_table.id,val_left, val_middle, val_right from left_table + inner join middle_table on left_table.id = middle_table.id + inner join right_table on middle_table.id = right_table.id +ORDER BY left_table.id, val_left, val_middle, val_right; -insert into left_table values ('abc'); -insert into left_table values ('abc'); -insert into left_table values ('abc'); +explain syntax select left_table.id,val_left, val_middle, val_right from left_table + inner join middle_table on left_table.id = middle_table.id + inner join right_table on middle_table.id = right_table.id +ORDER BY left_table.id, val_left, val_middle, val_right; -insert into middle_table values ('abc'); -insert into middle_table values ('abc'); +-- extra: same with subquery +select left_table.id,val_left, val_middle, val_right from left_table + inner join middle_table on left_table.id = middle_table.id + inner join (SELECT * FROM right_table WHERE id = 1) r on middle_table.id = r.id +ORDER BY left_table.id, val_left, val_middle, val_right; -insert into right_table values ('abc'); -insert into right_table values ('abc'); -insert into right_table values ('abc'); +-- distributed tables +drop table if exists left_table; +drop table if exists middle_table; +drop table if exists right_table; +create table if not exists left_table (id UInt64, val_left String) engine=ReplacingMergeTree() ORDER BY id; +create table if not exists middle_table (id UInt64, val_middle String) engine=MergeTree() ORDER BY id; +create table if not exists right_table_local (id UInt64, val_right String) engine=ReplacingMergeTree() ORDER BY id; +create table if not exists right_table engine=Distributed('test_shard_localhost', currentDatabase(), right_table_local) AS right_table_local; +insert into left_table values (1,'a'); +insert into left_table values (1,'b'); +insert into left_table values (1,'c'); +insert into middle_table values (1,'a'); +insert into middle_table values (1,'b'); +insert into right_table_local values (1,'a'); +insert into right_table_local values (1,'b'); +insert into right_table_local values (1,'c'); +SET prefer_localhost_replica=0; +-- expected output: +-- 1 c 1 a 1 c +-- 1 c 1 b 1 c +select left_table.*,middle_table.*, right_table.* from left_table + inner join middle_table on left_table.id = middle_table.id + inner join right_table on middle_table.id = right_table.id +ORDER BY left_table.id, val_left, val_middle, val_right; --- Expected output is 2 because middle table does not support final -select count() from left_table - inner join middle_table on left_table.x = middle_table.x - inner join right_table on middle_table.x = right_table.x; +SET prefer_localhost_replica=1; +-- expected output: +-- 1 c 1 a 1 c +-- 1 c 1 b 1 c +select left_table.*,middle_table.*, right_table.* from left_table + inner join middle_table on left_table.id = middle_table.id + inner join right_table on middle_table.id = right_table.id +ORDER BY left_table.id, val_left, val_middle, val_right; + +-- Quite exotic with Merge engine +DROP TABLE IF EXISTS table_to_merge_a; +DROP TABLE IF EXISTS table_to_merge_b; +DROP TABLE IF EXISTS table_to_merge_c; +DROP TABLE IF EXISTS merge_table; + +create table if not exists table_to_merge_a (id UInt64, val String) engine=ReplacingMergeTree() ORDER BY id; +create table if not exists table_to_merge_b (id UInt64, val String) engine=MergeTree() ORDER BY id; +create table if not exists table_to_merge_c (id UInt64, val String) engine=ReplacingMergeTree() ORDER BY id; +CREATE TABLE merge_table Engine=Merge(currentDatabase(), '^(table_to_merge_[a-z])$') AS table_to_merge_a; + +insert into table_to_merge_a values (1,'a'); +insert into table_to_merge_a values (1,'b'); +insert into table_to_merge_a values (1,'c'); +insert into table_to_merge_b values (2,'a'); +insert into table_to_merge_b values (2,'b'); +insert into table_to_merge_c values (3,'a'); +insert into table_to_merge_c values (3,'b'); +insert into table_to_merge_c values (3,'c'); + +-- expected output: +-- 1 c, 2 a, 2 b, 3 c +SELECT * FROM merge_table ORDER BY id, val; From 2c15ede37c422ff995a227ae64264610fa725e1b Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 31 Jan 2023 15:21:00 -0300 Subject: [PATCH 029/253] Change setting name from force_select_final to final --- src/Core/Settings.h | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 8 ++++---- src/Interpreters/InterpreterSelectQuery.h | 2 +- .../02420_force_select_final_setting.reference | 16 ++++++++-------- .../02420_force_select_final_setting.sql | 16 ++++++++-------- ...force_select_final_setting_analyzer.reference | 0 ...02420_force_select_final_setting_analyzer.sql | 0 7 files changed, 22 insertions(+), 22 deletions(-) create mode 100644 tests/queries/0_stateless/02420_force_select_final_setting_analyzer.reference create mode 100644 tests/queries/0_stateless/02420_force_select_final_setting_analyzer.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3dfbfa292d8..413d128564a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -275,7 +275,7 @@ class IColumn; M(Milliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \ M(Milliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \ \ - M(Bool, force_select_final, false, "Query with the FINAL modifier by default", 0) \ + M(Bool, final, false, "Query with the FINAL modifier by default", 0) \ \ /** Settings for testing hedged requests */ \ M(Milliseconds, sleep_in_send_tables_status_ms, 0, "Time to sleep in sending tables status response in TCPHandler", 0) \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 12b6f6f3297..ae4ccc34f78 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -503,7 +503,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( query_info.additional_filter_ast = parseAdditionalFilterConditionForTable( settings.additional_table_filters, joined_tables.tablesWithColumns().front().table, *context); - if (forceSelectFinalOnSelectQuery(query)) + if (autoFinalOnQuery(query)) { query.setFinal(); } @@ -3007,14 +3007,14 @@ void InterpreterSelectQuery::ignoreWithTotals() getSelectQuery().group_by_with_totals = false; } -bool InterpreterSelectQuery::forceSelectFinalOnSelectQuery(ASTSelectQuery & query) +bool InterpreterSelectQuery::autoFinalOnQuery(ASTSelectQuery & query) { // query.tables() is required because not all queries have tables in it, it could be a function. - bool is_force_select_final_setting_on = context->getSettingsRef().force_select_final; + bool is_auto_final_setting_on = context->getSettingsRef().final; bool is_final_supported = storage && storage->supportsFinal() && !storage->isRemote() && query.tables(); bool is_query_already_final = query.final(); - return is_force_select_final_setting_on && !is_query_already_final && is_final_supported; + return is_auto_final_setting_on && !is_query_already_final && is_final_supported; } void InterpreterSelectQuery::initSettings() diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index f679051532c..88db1360683 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -194,7 +194,7 @@ private: void executeDistinct(QueryPlan & query_plan, bool before_order, Names columns, bool pre_distinct); void executeExtremes(QueryPlan & query_plan); void executeSubqueriesInSetsAndJoins(QueryPlan & query_plan); - bool forceSelectFinalOnSelectQuery(ASTSelectQuery & select_query); + bool autoFinalOnQuery(ASTSelectQuery & select_query); enum class Modificator { diff --git a/tests/queries/0_stateless/02420_force_select_final_setting.reference b/tests/queries/0_stateless/02420_force_select_final_setting.reference index 7194b90f775..5b9780e27cd 100644 --- a/tests/queries/0_stateless/02420_force_select_final_setting.reference +++ b/tests/queries/0_stateless/02420_force_select_final_setting.reference @@ -4,11 +4,11 @@ SYSTEM STOP MERGES tbl; create table if not exists replacing_mt (x String) engine=ReplacingMergeTree() ORDER BY x; insert into replacing_mt values ('abc'); insert into replacing_mt values ('abc'); --- expected output is 2 because force_select_final is turned off +-- expected output is 2 because final is turned off select count() from replacing_mt; 2 -set force_select_final = 1; --- expected output is 1 because force_select_final is turned on +set final = 1; +-- expected output is 1 because final is turned on select count() from replacing_mt; 1 -- JOIN test cases @@ -18,16 +18,16 @@ insert into lhs values ('abc'); insert into lhs values ('abc'); insert into rhs values ('abc'); insert into rhs values ('abc'); -set force_select_final = 0; +set final = 0; -- expected output is 4 because select_final == 0 select count() from lhs inner join rhs on lhs.x = rhs.x; 4 -set force_select_final = 1; --- expected output is 1 because force_select_final == 1 +set final = 1; +-- expected output is 1 because final == 1 select count() from lhs inner join rhs on lhs.x = rhs.x; 1 -- regular non final table -set force_select_final = 1; +set final = 1; create table if not exists regular_mt_table (x String) engine=MergeTree() ORDER BY x; insert into regular_mt_table values ('abc'); insert into regular_mt_table values ('abc'); @@ -37,7 +37,7 @@ select count() from regular_mt_table; -- view test create materialized VIEW mv_regular_mt_table TO regular_mt_table AS SELECT * FROM regular_mt_table; create view nv_regular_mt_table AS SELECT * FROM mv_regular_mt_table; -set force_select_final=1; +set final=1; select count() from nv_regular_mt_table; 2 -- join on mix of tables that support / do not support select final with explain diff --git a/tests/queries/0_stateless/02420_force_select_final_setting.sql b/tests/queries/0_stateless/02420_force_select_final_setting.sql index c60bc3c7f53..d8ca24da3da 100644 --- a/tests/queries/0_stateless/02420_force_select_final_setting.sql +++ b/tests/queries/0_stateless/02420_force_select_final_setting.sql @@ -7,11 +7,11 @@ create table if not exists replacing_mt (x String) engine=ReplacingMergeTree() O insert into replacing_mt values ('abc'); insert into replacing_mt values ('abc'); --- expected output is 2 because force_select_final is turned off +-- expected output is 2 because final is turned off select count() from replacing_mt; -set force_select_final = 1; --- expected output is 1 because force_select_final is turned on +set final = 1; +-- expected output is 1 because final is turned on select count() from replacing_mt; -- JOIN test cases @@ -24,16 +24,16 @@ insert into lhs values ('abc'); insert into rhs values ('abc'); insert into rhs values ('abc'); -set force_select_final = 0; +set final = 0; -- expected output is 4 because select_final == 0 select count() from lhs inner join rhs on lhs.x = rhs.x; -set force_select_final = 1; --- expected output is 1 because force_select_final == 1 +set final = 1; +-- expected output is 1 because final == 1 select count() from lhs inner join rhs on lhs.x = rhs.x; -- regular non final table -set force_select_final = 1; +set final = 1; create table if not exists regular_mt_table (x String) engine=MergeTree() ORDER BY x; insert into regular_mt_table values ('abc'); insert into regular_mt_table values ('abc'); @@ -44,7 +44,7 @@ select count() from regular_mt_table; create materialized VIEW mv_regular_mt_table TO regular_mt_table AS SELECT * FROM regular_mt_table; create view nv_regular_mt_table AS SELECT * FROM mv_regular_mt_table; -set force_select_final=1; +set final=1; select count() from nv_regular_mt_table; -- join on mix of tables that support / do not support select final with explain diff --git a/tests/queries/0_stateless/02420_force_select_final_setting_analyzer.reference b/tests/queries/0_stateless/02420_force_select_final_setting_analyzer.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02420_force_select_final_setting_analyzer.sql b/tests/queries/0_stateless/02420_force_select_final_setting_analyzer.sql new file mode 100644 index 00000000000..e69de29bb2d From b9e46c02ddf92c12bf3211f4a23abe9975a63a09 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 31 Jan 2023 15:24:06 -0300 Subject: [PATCH 030/253] rename tests --- ...lect_final_setting.reference => 02420_final_setting.reference} | 0 ...420_force_select_final_setting.sql => 02420_final_setting.sql} | 0 .../02420_force_select_final_setting_analyzer.reference | 0 .../0_stateless/02420_force_select_final_setting_analyzer.sql | 0 4 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{02420_force_select_final_setting.reference => 02420_final_setting.reference} (100%) rename tests/queries/0_stateless/{02420_force_select_final_setting.sql => 02420_final_setting.sql} (100%) delete mode 100644 tests/queries/0_stateless/02420_force_select_final_setting_analyzer.reference delete mode 100644 tests/queries/0_stateless/02420_force_select_final_setting_analyzer.sql diff --git a/tests/queries/0_stateless/02420_force_select_final_setting.reference b/tests/queries/0_stateless/02420_final_setting.reference similarity index 100% rename from tests/queries/0_stateless/02420_force_select_final_setting.reference rename to tests/queries/0_stateless/02420_final_setting.reference diff --git a/tests/queries/0_stateless/02420_force_select_final_setting.sql b/tests/queries/0_stateless/02420_final_setting.sql similarity index 100% rename from tests/queries/0_stateless/02420_force_select_final_setting.sql rename to tests/queries/0_stateless/02420_final_setting.sql diff --git a/tests/queries/0_stateless/02420_force_select_final_setting_analyzer.reference b/tests/queries/0_stateless/02420_force_select_final_setting_analyzer.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02420_force_select_final_setting_analyzer.sql b/tests/queries/0_stateless/02420_force_select_final_setting_analyzer.sql deleted file mode 100644 index e69de29bb2d..00000000000 From a8489fbef3ab7a7a27c4474f1b9bf37203f14adb Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 31 Jan 2023 15:29:30 -0300 Subject: [PATCH 031/253] improve docs --- 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 413d128564a..b2af210124d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -275,7 +275,7 @@ class IColumn; M(Milliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \ M(Milliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \ \ - M(Bool, final, false, "Query with the FINAL modifier by default", 0) \ + M(Bool, final, false, "Query with the FINAL modifier by default. If the engine does not support final, it does not have any effect. On queries with multiple tables final is applied only on those that support it. It also works on distributed tables", 0) \ \ /** Settings for testing hedged requests */ \ M(Milliseconds, sleep_in_send_tables_status_ms, 0, "Time to sleep in sending tables status response in TCPHandler", 0) \ From 7e7b1829bea49ab271158c7e87270b5e1e82e3cf Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 1 Feb 2023 14:29:59 -0300 Subject: [PATCH 032/253] temp --- src/Interpreters/AutoFinalOnQueryVisitor.cpp | 37 ++++++++++++++++++++ src/Interpreters/AutoFinalOnQueryVisitor.h | 31 ++++++++++++++++ src/Interpreters/InterpreterSelectQuery.cpp | 13 ++++--- 3 files changed, 77 insertions(+), 4 deletions(-) create mode 100644 src/Interpreters/AutoFinalOnQueryVisitor.cpp create mode 100644 src/Interpreters/AutoFinalOnQueryVisitor.h diff --git a/src/Interpreters/AutoFinalOnQueryVisitor.cpp b/src/Interpreters/AutoFinalOnQueryVisitor.cpp new file mode 100644 index 00000000000..f2b3eb147c1 --- /dev/null +++ b/src/Interpreters/AutoFinalOnQueryVisitor.cpp @@ -0,0 +1,37 @@ +#include "AutoFinalOnQueryVisitor.h" +#include +#include + +namespace DB +{ + +void AutoFinalOnQuery::visit(ASTPtr & query, Data & data) +{ + if (auto * select_query = query->as()) + visit(*select_query, data.storage, data.context); +} + +void AutoFinalOnQuery::visit(ASTSelectQuery & query, StoragePtr storage, ContextPtr context) +{ + if (autoFinalOnQuery(query, storage, context)) + { + query.setFinal(); + } +} + +bool AutoFinalOnQuery::needChildVisit(ASTPtr &, const ASTPtr &) +{ + return true; +} + +bool AutoFinalOnQuery::autoFinalOnQuery(ASTSelectQuery & query, StoragePtr storage, ContextPtr context) +{ + // query.tables() is required because not all queries have tables in it, it could be a function. + bool is_auto_final_setting_on = context->getSettingsRef().final; + bool is_final_supported = storage && storage->supportsFinal() && !storage->isRemote() && query.tables(); + bool is_query_already_final = query.final(); + + return is_auto_final_setting_on && !is_query_already_final && is_final_supported; +} + +} diff --git a/src/Interpreters/AutoFinalOnQueryVisitor.h b/src/Interpreters/AutoFinalOnQueryVisitor.h new file mode 100644 index 00000000000..850fb9fc0e4 --- /dev/null +++ b/src/Interpreters/AutoFinalOnQueryVisitor.h @@ -0,0 +1,31 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class AutoFinalOnQuery +{ +public: + struct Data + { + StoragePtr storage; + ContextPtr context; + }; + + static bool needChildVisit(ASTPtr &, const ASTPtr &); + static void visit(ASTPtr & query, Data & data); + +private: + static void visit(ASTSelectQuery & select, StoragePtr storage, ContextPtr context); + static bool autoFinalOnQuery(ASTSelectQuery & select_query, StoragePtr storage, ContextPtr context); + +}; + +using AutoFinalOnQueryVisitor = InDepthNodeVisitor; + +} diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index ae4ccc34f78..8114b7a5375 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -38,6 +38,7 @@ #include #include #include +#include #include #include @@ -503,10 +504,14 @@ InterpreterSelectQuery::InterpreterSelectQuery( query_info.additional_filter_ast = parseAdditionalFilterConditionForTable( settings.additional_table_filters, joined_tables.tablesWithColumns().front().table, *context); - if (autoFinalOnQuery(query)) - { - query.setFinal(); - } + AutoFinalOnQuery::Data abc{storage, context}; + + AutoFinalOnQueryVisitor(abc).visit(query_ptr); + +// if (autoFinalOnQuery(query)) +// { +// query.setFinal(); +// } auto analyze = [&] (bool try_move_to_prewhere) { From ef546833860f32440859789e2103ad05a58e0ca4 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 2 Feb 2023 19:25:14 +0100 Subject: [PATCH 033/253] Use cluster state data to check concurrent backup/restore Implementation: * BackupWorker checks the if any backup/restore which has a path in zookeeper has status not completed, if yes, new backup/restore is stopped. * For not on cluster only active backup / restore is checked. * Removed restore_uuid from RestoreSettings, as it is no longer used. --- src/Backups/BackupCoordinationRemote.cpp | 7 +- src/Backups/BackupCoordinationRemote.h | 2 + src/Backups/BackupCoordinationStage.h | 4 + src/Backups/BackupsWorker.cpp | 133 +++++++++++++++++------ src/Backups/BackupsWorker.h | 4 +- src/Backups/RestoreSettings.cpp | 3 +- src/Backups/RestoreSettings.h | 5 - 7 files changed, 110 insertions(+), 48 deletions(-) diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 18789802769..ad164768333 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -160,9 +160,6 @@ namespace { return fmt::format("{:03}", counter); /// Outputs 001, 002, 003, ... } - - /// We try to store data to zookeeper several times due to possible version conflicts. - constexpr size_t NUM_ATTEMPTS = 10; } BackupCoordinationRemote::BackupCoordinationRemote( @@ -468,7 +465,7 @@ void BackupCoordinationRemote::updateFileInfo(const FileInfo & file_info) auto zk = getZooKeeper(); String size_and_checksum = serializeSizeAndChecksum(std::pair{file_info.size, file_info.checksum}); String full_path = zookeeper_path + "/file_infos/" + size_and_checksum; - for (size_t attempt = 0; attempt < NUM_ATTEMPTS; ++attempt) + for (size_t attempt = 0; attempt < MAX_ZOOKEEPER_ATTEMPTS; ++attempt) { Coordination::Stat stat; auto new_info = deserializeFileInfo(zk->get(full_path, &stat)); @@ -476,7 +473,7 @@ void BackupCoordinationRemote::updateFileInfo(const FileInfo & file_info) auto code = zk->trySet(full_path, serializeFileInfo(new_info), stat.version); if (code == Coordination::Error::ZOK) return; - bool is_last_attempt = (attempt == NUM_ATTEMPTS - 1); + bool is_last_attempt = (attempt == MAX_ZOOKEEPER_ATTEMPTS - 1); if ((code != Coordination::Error::ZBADVERSION) || is_last_attempt) throw zkutil::KeeperException(code, full_path); } diff --git a/src/Backups/BackupCoordinationRemote.h b/src/Backups/BackupCoordinationRemote.h index 711fadb539e..b1a7a12e109 100644 --- a/src/Backups/BackupCoordinationRemote.h +++ b/src/Backups/BackupCoordinationRemote.h @@ -5,6 +5,8 @@ #include #include +/// We try to store data to zookeeper several times due to possible version conflicts. +constexpr size_t MAX_ZOOKEEPER_ATTEMPTS = 10; namespace DB { diff --git a/src/Backups/BackupCoordinationStage.h b/src/Backups/BackupCoordinationStage.h index 091c1f11463..2c02b651851 100644 --- a/src/Backups/BackupCoordinationStage.h +++ b/src/Backups/BackupCoordinationStage.h @@ -8,6 +8,10 @@ namespace DB namespace BackupCoordinationStage { + /// This stage is set after concurrency check so ensure we dont start other backup/restores + /// when concurrent backup/restores are not allowed + constexpr const char * SCHEDULED_TO_START = "scheduled to start"; + /// Finding all tables and databases which we're going to put to the backup and collecting their metadata. constexpr const char * GATHERING_METADATA = "gathering metadata"; diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 5348f4ec81a..743f3329f5c 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -173,13 +173,6 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context String backup_name_for_logging = backup_info.toStringForLogging(); try { - if (!allow_concurrent_backups && hasConcurrentBackups(backup_settings)) - { - /// addInfo is called here to record the failed backup details - addInfo(backup_id, backup_name_for_logging, backup_settings.internal, BackupStatus::BACKUP_FAILED); - throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Concurrent backups not supported, turn on setting 'allow_concurrent_backups'"); - } - addInfo(backup_id, backup_name_for_logging, backup_settings.internal, BackupStatus::CREATING_BACKUP); /// Prepare context to use. @@ -259,6 +252,7 @@ void BackupsWorker::doBackup( } bool on_cluster = !backup_query->cluster.empty(); + assert(mutable_context || (!on_cluster && !called_async)); /// Checks access rights if this is not ON CLUSTER query. @@ -284,6 +278,9 @@ void BackupsWorker::doBackup( if (!backup_coordination) backup_coordination = makeBackupCoordination(backup_settings.coordination_zk_path, context, backup_settings.internal); + if (!allow_concurrent_backups && !backup_settings.internal && hasConcurrentBackups(backup_id, context, on_cluster)) + throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Concurrent backups not supported, turn on setting 'allow_concurrent_backups'"); + /// Opens a backup for writing. BackupFactory::CreateParams backup_create_params; backup_create_params.open_mode = IBackup::OpenMode::WRITE; @@ -384,9 +381,6 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt auto restore_query = std::static_pointer_cast(query->clone()); auto restore_settings = RestoreSettings::fromRestoreQuery(*restore_query); - if (!restore_settings.restore_uuid) - restore_settings.restore_uuid = UUIDHelpers::generateV4(); - /// `restore_id` will be used as a key to the `infos` map, so it should be unique. OperationID restore_id; if (restore_settings.internal) @@ -394,7 +388,7 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt else if (!restore_settings.id.empty()) restore_id = restore_settings.id; else - restore_id = toString(*restore_settings.restore_uuid); + restore_id = toString(UUIDHelpers::generateV4()); std::shared_ptr restore_coordination; if (restore_settings.internal) @@ -410,13 +404,6 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt auto backup_info = BackupInfo::fromAST(*restore_query->backup_name); String backup_name_for_logging = backup_info.toStringForLogging(); - if (!allow_concurrent_restores && hasConcurrentRestores(restore_settings)) - { - /// addInfo is called here to record the failed restore details - addInfo(restore_id, backup_name_for_logging, restore_settings.internal, BackupStatus::RESTORING); - throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Concurrent restores not supported, turn on setting 'allow_concurrent_restores'"); - } - addInfo(restore_id, backup_name_for_logging, restore_settings.internal, BackupStatus::RESTORING); /// Prepare context to use. @@ -496,7 +483,6 @@ void BackupsWorker::doRestore( backup_open_params.context = context; backup_open_params.backup_info = backup_info; backup_open_params.base_backup_info = restore_settings.base_backup_info; - backup_open_params.backup_uuid = restore_settings.restore_uuid; backup_open_params.password = restore_settings.password; BackupPtr backup = BackupFactory::instance().createBackup(backup_open_params); @@ -532,12 +518,15 @@ void BackupsWorker::doRestore( if (on_cluster && restore_settings.coordination_zk_path.empty()) { String root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); - restore_settings.coordination_zk_path = root_zk_path + "/restore-" + toString(UUIDHelpers::generateV4()); + restore_settings.coordination_zk_path = root_zk_path + "/restore-" + toString(restore_id); } if (!restore_coordination) restore_coordination = makeRestoreCoordination(restore_settings.coordination_zk_path, context, restore_settings.internal); + if (!allow_concurrent_restores && !restore_settings.internal && hasConcurrentRestores(restore_id, context, on_cluster)) + throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Concurrent restores not supported, turn on setting 'allow_concurrent_restores'"); + /// Do RESTORE. if (on_cluster) { @@ -744,32 +733,108 @@ std::vector BackupsWorker::getAllActiveRestoreInfos() const return res_infos; } -bool BackupsWorker::hasConcurrentBackups(const BackupSettings & backup_settings) const +bool BackupsWorker::hasConcurrentBackups(const OperationID & backup_id, const ContextPtr & context, bool on_cluster) const { - /// Check if there are no concurrent backups - if (num_active_backups) + if (on_cluster) { - /// If its an internal backup and we currently have 1 active backup, it could be the original query, validate using backup_uuid - if (!(num_active_backups == 1 && backup_settings.internal && getAllActiveBackupInfos().at(0).id == toString(*backup_settings.backup_uuid))) + String common_backup_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups") ; + auto zookeeper = context->getGlobalContext()->getZooKeeper(); + std::string backup_stage_path = common_backup_path + "/backup-" + toString(backup_id) +"/stage"; + + if (!zookeeper->exists(common_backup_path)) + zookeeper->createAncestors(common_backup_path); + + for (size_t attempt = 0; attempt < MAX_ZOOKEEPER_ATTEMPTS; ++attempt) { - return true; + Coordination::Stat stat; + zookeeper->get(common_backup_path, &stat); + Strings existing_backup_paths = zookeeper->getChildren(common_backup_path); + + for (const auto & existing_backup_path : existing_backup_paths) + { + if (startsWith(existing_backup_path, "restore-")) + continue; + + String existing_backup_id = existing_backup_path; + existing_backup_id.erase(0, String("backup-").size()); + + if (existing_backup_id == toString(backup_id)) + continue; + + const auto status = zookeeper->get(common_backup_path + "/" + existing_backup_path + "/stage"); + if (status != Stage::COMPLETED) + return true; + } + + zookeeper->createIfNotExists(backup_stage_path, ""); + auto code = zookeeper->trySet(backup_stage_path, Stage::SCHEDULED_TO_START, stat.version); + if (code == Coordination::Error::ZOK) + break; + bool is_last_attempt = (attempt == MAX_ZOOKEEPER_ATTEMPTS - 1); + if ((code != Coordination::Error::ZBADVERSION) || is_last_attempt) + throw zkutil::KeeperException(code, backup_stage_path); } + return false; + } + else + { + if (num_active_backups == 1) + return false; + else + return true; } - return false; } -bool BackupsWorker::hasConcurrentRestores(const RestoreSettings & restore_settings) const +bool BackupsWorker::hasConcurrentRestores(const OperationID & restore_id, const ContextPtr & context, bool on_cluster) const { - /// Check if there are no concurrent restores - if (num_active_restores) + if (on_cluster) { - /// If its an internal restore and we currently have 1 active restore, it could be the original query, validate using iz - if (!(num_active_restores == 1 && restore_settings.internal && getAllActiveRestoreInfos().at(0).id == toString(*restore_settings.restore_uuid))) + String common_restore_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups") ; + auto zookeeper = context->getGlobalContext()->getZooKeeper(); + std::string path = common_restore_path + "/restore-" + toString(restore_id) +"/stage"; + + if (!zookeeper->exists(common_restore_path)) + zookeeper->createAncestors(common_restore_path); + + for (size_t attempt = 0; attempt < MAX_ZOOKEEPER_ATTEMPTS; ++attempt) { - return true; + Coordination::Stat stat; + zookeeper->get(common_restore_path, &stat); + Strings existing_restore_paths = zookeeper->getChildren(common_restore_path); + for (const auto & existing_restore_path : existing_restore_paths) + { + if (startsWith(existing_restore_path, "backup-")) + continue; + + String existing_restore_id = existing_restore_path; + existing_restore_id.erase(0, String("restore-").size()); + + if (existing_restore_id == toString(restore_id)) + continue; + + + const auto status = zookeeper->get(common_restore_path + "/" + existing_restore_path + "/stage"); + if (status != Stage::COMPLETED) + return true; + } + + zookeeper->createIfNotExists(path, ""); + auto code = zookeeper->trySet(path, Stage::SCHEDULED_TO_START, stat.version); + if (code == Coordination::Error::ZOK) + break; + bool is_last_attempt = (attempt == MAX_ZOOKEEPER_ATTEMPTS - 1); + if ((code != Coordination::Error::ZBADVERSION) || is_last_attempt) + throw zkutil::KeeperException(code, path); } + return false; + } + else + { + if (num_active_restores == 1) + return false; + else + return true; } - return false; } void BackupsWorker::shutdown() diff --git a/src/Backups/BackupsWorker.h b/src/Backups/BackupsWorker.h index 462a9033251..994e5f1f8b7 100644 --- a/src/Backups/BackupsWorker.h +++ b/src/Backups/BackupsWorker.h @@ -113,8 +113,8 @@ private: void setNumFilesAndSize(const OperationID & id, size_t num_files, size_t num_processed_files, UInt64 processed_files_size, UInt64 uncompressed_size, UInt64 compressed_size); std::vector getAllActiveBackupInfos() const; std::vector getAllActiveRestoreInfos() const; - bool hasConcurrentBackups(const BackupSettings & backup_settings) const; - bool hasConcurrentRestores(const RestoreSettings & restore_settings) const; + bool hasConcurrentBackups(const OperationID & backup_id, const ContextPtr & context, bool on_cluster) const; + bool hasConcurrentRestores(const OperationID & restore_id, const ContextPtr & context, bool on_cluster) const; ThreadPool backups_thread_pool; ThreadPool restores_thread_pool; diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 5e06764a247..bbcefb819cf 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -163,8 +163,7 @@ namespace M(RestoreUDFCreationMode, create_function) \ M(Bool, internal) \ M(String, host_id) \ - M(String, coordination_zk_path) \ - M(OptionalUUID, restore_uuid) + M(String, coordination_zk_path) RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index 3bfe9f869ea..713adbe8029 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -122,11 +122,6 @@ struct RestoreSettings /// Path in Zookeeper used to coordinate restoring process while executing by RESTORE ON CLUSTER. String coordination_zk_path; - /// Internal, should not be specified by user. - /// UUID of the restore. If it's not set it will be generated randomly. - /// This is used to validate internal restores when allow_concurrent_restores is turned off - std::optional restore_uuid; - static RestoreSettings fromRestoreQuery(const ASTBackupQuery & query); void copySettingsToQuery(ASTBackupQuery & query) const; }; From 74298f111e3f5922698c66c66c6823967710471f Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 2 Feb 2023 17:25:04 -0300 Subject: [PATCH 034/253] Implement AutoFinalQueryOnPass to enable it on Analyzer as well --- src/Analyzer/Passes/AutoFinalOnQueryPass.cpp | 64 ++++++++ src/Analyzer/Passes/AutoFinalOnQueryPass.h | 21 +++ src/Analyzer/QueryTreePassManager.cpp | 4 + .../02420_final_setting_analyzer.reference | 138 ++++++++++++++++++ .../02420_final_setting_analyzer.sql | 107 ++++++++++++++ 5 files changed, 334 insertions(+) create mode 100644 src/Analyzer/Passes/AutoFinalOnQueryPass.cpp create mode 100644 src/Analyzer/Passes/AutoFinalOnQueryPass.h create mode 100644 tests/queries/0_stateless/02420_final_setting_analyzer.reference create mode 100644 tests/queries/0_stateless/02420_final_setting_analyzer.sql diff --git a/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp b/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp new file mode 100644 index 00000000000..97a770a4a6c --- /dev/null +++ b/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp @@ -0,0 +1,64 @@ +#include "AutoFinalOnQueryPass.h" + +#include +#include +#include +#include + +namespace DB +{ + + +namespace +{ + class AutoFinalOnQueryPassVisitor : public InDepthQueryTreeVisitorWithContext + { + public: + using Base = InDepthQueryTreeVisitorWithContext; + using Base::Base; + + void visitImpl(QueryTreeNodePtr & node) + { + if (auto * table_node = node->as()) + { + if (autoFinalOnQuery(*table_node, table_node->getStorage(), getContext())) + { + auto modifier = TableExpressionModifiers(true, std::nullopt, std::nullopt); + table_node->setTableExpressionModifiers(modifier); + } + } + } + + private: + bool autoFinalOnQuery(TableNode & table_node, StoragePtr storage, ContextPtr context) + { + // query.tables() is required because not all queries have tables in it, it could be a function. + bool is_auto_final_setting_on = context->getSettingsRef().final; + bool is_final_supported = storage && storage->supportsFinal() && !storage->isRemote(); + bool is_query_already_final = table_node.hasTableExpressionModifiers() ? table_node.getTableExpressionModifiers().has_value() : false; + + return is_auto_final_setting_on && !is_query_already_final && is_final_supported; + } + + }; + +} + +String AutoFinalOnQueryPass::getName() +{ + return "AutoFinalOnQueryPass"; +} + +String AutoFinalOnQueryPass::getDescription() +{ + return "Automatically applies final modifier to queries if it is supported and if user level final setting is set."; +} + +void AutoFinalOnQueryPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) +{ + auto visitor = AutoFinalOnQueryPassVisitor(std::move(context)); + + visitor.visit(query_tree_node); +} + +} diff --git a/src/Analyzer/Passes/AutoFinalOnQueryPass.h b/src/Analyzer/Passes/AutoFinalOnQueryPass.h new file mode 100644 index 00000000000..eacbe0f8235 --- /dev/null +++ b/src/Analyzer/Passes/AutoFinalOnQueryPass.h @@ -0,0 +1,21 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + + +class AutoFinalOnQueryPass final : public IQueryTreePass +{ +public: + String getName() override; + + String getDescription() override; + + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; +}; + +} diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 0280831a255..62bf5b7de5b 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -34,6 +34,7 @@ #include #include #include +#include namespace DB { @@ -239,6 +240,9 @@ void addQueryTreePasses(QueryTreePassManager & manager) manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); + + manager.addPass(std::make_unique()); + } } diff --git a/tests/queries/0_stateless/02420_final_setting_analyzer.reference b/tests/queries/0_stateless/02420_final_setting_analyzer.reference new file mode 100644 index 00000000000..dfc64fa39a1 --- /dev/null +++ b/tests/queries/0_stateless/02420_final_setting_analyzer.reference @@ -0,0 +1,138 @@ +-- { echoOn } +set allow_experimental_analyzer=1; +SYSTEM STOP MERGES tbl; +-- simple test case +create table if not exists replacing_mt (x String) engine=ReplacingMergeTree() ORDER BY x; +insert into replacing_mt values ('abc'); +insert into replacing_mt values ('abc'); +-- expected output is 2 because final is turned off +select count() from replacing_mt; +2 +set final = 1; +-- expected output is 1 because final is turned on +select count() from replacing_mt; +1 +-- JOIN test cases +create table if not exists lhs (x String) engine=ReplacingMergeTree() ORDER BY x; +create table if not exists rhs (x String) engine=ReplacingMergeTree() ORDER BY x; +insert into lhs values ('abc'); +insert into lhs values ('abc'); +insert into rhs values ('abc'); +insert into rhs values ('abc'); +set final = 0; +-- expected output is 4 because select_final == 0 +select count() from lhs inner join rhs on lhs.x = rhs.x; +4 +set final = 1; +-- expected output is 1 because final == 1 +select count() from lhs inner join rhs on lhs.x = rhs.x; +1 +-- regular non final table +set final = 1; +create table if not exists regular_mt_table (x String) engine=MergeTree() ORDER BY x; +insert into regular_mt_table values ('abc'); +insert into regular_mt_table values ('abc'); +-- expected output is 2, it should silently ignore final modifier +select count() from regular_mt_table; +2 +-- view test +create materialized VIEW mv_regular_mt_table TO regular_mt_table AS SELECT * FROM regular_mt_table; +create view nv_regular_mt_table AS SELECT * FROM mv_regular_mt_table; +set final=1; +select count() from nv_regular_mt_table; +2 +-- join on mix of tables that support / do not support select final with explain +create table if not exists left_table (id UInt64, val_left String) engine=ReplacingMergeTree() ORDER BY id; +create table if not exists middle_table (id UInt64, val_middle String) engine=MergeTree() ORDER BY id; +create table if not exists right_table (id UInt64, val_right String) engine=ReplacingMergeTree() ORDER BY id; +insert into left_table values (1,'a'); +insert into left_table values (1,'b'); +insert into left_table values (1,'c'); +insert into middle_table values (1,'a'); +insert into middle_table values (1,'b'); +insert into right_table values (1,'a'); +insert into right_table values (1,'b'); +insert into right_table values (1,'c'); +-- expected output +-- 1 c a c +-- 1 c b c +select left_table.id,val_left, val_middle, val_right from left_table + inner join middle_table on left_table.id = middle_table.id + inner join right_table on middle_table.id = right_table.id +ORDER BY left_table.id, val_left, val_middle, val_right; +1 c a c +1 c b c +explain syntax select left_table.id,val_left, val_middle, val_right from left_table + inner join middle_table on left_table.id = middle_table.id + inner join right_table on middle_table.id = right_table.id + ORDER BY left_table.id, val_left, val_middle, val_right; +SELECT + `--left_table.id` AS `left_table.id`, + val_left, + val_middle, + val_right +FROM +( + SELECT + val_left, + id AS `--left_table.id`, + val_middle, + middle_table.id AS `--middle_table.id` + FROM left_table + FINAL + ALL INNER JOIN + ( + SELECT + val_middle, + id + FROM middle_table + FINAL + ) AS middle_table ON `--left_table.id` = `--middle_table.id` +) AS `--.s` +ALL INNER JOIN +( + SELECT + val_right, + id + FROM right_table + FINAL +) AS right_table ON `--middle_table.id` = id +ORDER BY + `--left_table.id` ASC, + val_left ASC, + val_middle ASC, + val_right ASC +-- extra: same with subquery +select left_table.id,val_left, val_middle, val_right from left_table + inner join middle_table on left_table.id = middle_table.id + inner join (SELECT * FROM right_table WHERE id = 1) r on middle_table.id = r.id +ORDER BY left_table.id, val_left, val_middle, val_right; +1 c a c +1 c b c +-- no distributed tests because it is not currently supported: +-- JOIN with remote storages is unsupported. + +-- Quite exotic with Merge engine +DROP TABLE IF EXISTS table_to_merge_a; +DROP TABLE IF EXISTS table_to_merge_b; +DROP TABLE IF EXISTS table_to_merge_c; +DROP TABLE IF EXISTS merge_table; +create table if not exists table_to_merge_a (id UInt64, val String) engine=ReplacingMergeTree() ORDER BY id; +create table if not exists table_to_merge_b (id UInt64, val String) engine=MergeTree() ORDER BY id; +create table if not exists table_to_merge_c (id UInt64, val String) engine=ReplacingMergeTree() ORDER BY id; +CREATE TABLE merge_table Engine=Merge(currentDatabase(), '^(table_to_merge_[a-z])$') AS table_to_merge_a; +insert into table_to_merge_a values (1,'a'); +insert into table_to_merge_a values (1,'b'); +insert into table_to_merge_a values (1,'c'); +insert into table_to_merge_b values (2,'a'); +insert into table_to_merge_b values (2,'b'); +insert into table_to_merge_c values (3,'a'); +insert into table_to_merge_c values (3,'b'); +insert into table_to_merge_c values (3,'c'); +-- expected output: +-- 1 c, 2 a, 2 b, 3 c +SELECT * FROM merge_table ORDER BY id, val; +1 c +2 a +2 b +3 c diff --git a/tests/queries/0_stateless/02420_final_setting_analyzer.sql b/tests/queries/0_stateless/02420_final_setting_analyzer.sql new file mode 100644 index 00000000000..5937e536239 --- /dev/null +++ b/tests/queries/0_stateless/02420_final_setting_analyzer.sql @@ -0,0 +1,107 @@ +-- { echoOn } +set allow_experimental_analyzer=1; +SYSTEM STOP MERGES tbl; + +-- simple test case +create table if not exists replacing_mt (x String) engine=ReplacingMergeTree() ORDER BY x; + +insert into replacing_mt values ('abc'); +insert into replacing_mt values ('abc'); + +-- expected output is 2 because final is turned off +select count() from replacing_mt; + +set final = 1; +-- expected output is 1 because final is turned on +select count() from replacing_mt; + +-- JOIN test cases +create table if not exists lhs (x String) engine=ReplacingMergeTree() ORDER BY x; +create table if not exists rhs (x String) engine=ReplacingMergeTree() ORDER BY x; + +insert into lhs values ('abc'); +insert into lhs values ('abc'); + +insert into rhs values ('abc'); +insert into rhs values ('abc'); + +set final = 0; +-- expected output is 4 because select_final == 0 +select count() from lhs inner join rhs on lhs.x = rhs.x; + +set final = 1; +-- expected output is 1 because final == 1 +select count() from lhs inner join rhs on lhs.x = rhs.x; + +-- regular non final table +set final = 1; +create table if not exists regular_mt_table (x String) engine=MergeTree() ORDER BY x; +insert into regular_mt_table values ('abc'); +insert into regular_mt_table values ('abc'); +-- expected output is 2, it should silently ignore final modifier +select count() from regular_mt_table; + +-- view test +create materialized VIEW mv_regular_mt_table TO regular_mt_table AS SELECT * FROM regular_mt_table; +create view nv_regular_mt_table AS SELECT * FROM mv_regular_mt_table; + +set final=1; +select count() from nv_regular_mt_table; + +-- join on mix of tables that support / do not support select final with explain +create table if not exists left_table (id UInt64, val_left String) engine=ReplacingMergeTree() ORDER BY id; +create table if not exists middle_table (id UInt64, val_middle String) engine=MergeTree() ORDER BY id; +create table if not exists right_table (id UInt64, val_right String) engine=ReplacingMergeTree() ORDER BY id; +insert into left_table values (1,'a'); +insert into left_table values (1,'b'); +insert into left_table values (1,'c'); +insert into middle_table values (1,'a'); +insert into middle_table values (1,'b'); +insert into right_table values (1,'a'); +insert into right_table values (1,'b'); +insert into right_table values (1,'c'); +-- expected output +-- 1 c a c +-- 1 c b c +select left_table.id,val_left, val_middle, val_right from left_table + inner join middle_table on left_table.id = middle_table.id + inner join right_table on middle_table.id = right_table.id +ORDER BY left_table.id, val_left, val_middle, val_right; + +explain syntax select left_table.id,val_left, val_middle, val_right from left_table + inner join middle_table on left_table.id = middle_table.id + inner join right_table on middle_table.id = right_table.id + ORDER BY left_table.id, val_left, val_middle, val_right; + +-- extra: same with subquery +select left_table.id,val_left, val_middle, val_right from left_table + inner join middle_table on left_table.id = middle_table.id + inner join (SELECT * FROM right_table WHERE id = 1) r on middle_table.id = r.id +ORDER BY left_table.id, val_left, val_middle, val_right; + +-- no distributed tests because it is not currently supported: +-- JOIN with remote storages is unsupported. + +-- Quite exotic with Merge engine +DROP TABLE IF EXISTS table_to_merge_a; +DROP TABLE IF EXISTS table_to_merge_b; +DROP TABLE IF EXISTS table_to_merge_c; +DROP TABLE IF EXISTS merge_table; + +create table if not exists table_to_merge_a (id UInt64, val String) engine=ReplacingMergeTree() ORDER BY id; +create table if not exists table_to_merge_b (id UInt64, val String) engine=MergeTree() ORDER BY id; +create table if not exists table_to_merge_c (id UInt64, val String) engine=ReplacingMergeTree() ORDER BY id; +CREATE TABLE merge_table Engine=Merge(currentDatabase(), '^(table_to_merge_[a-z])$') AS table_to_merge_a; + +insert into table_to_merge_a values (1,'a'); +insert into table_to_merge_a values (1,'b'); +insert into table_to_merge_a values (1,'c'); +insert into table_to_merge_b values (2,'a'); +insert into table_to_merge_b values (2,'b'); +insert into table_to_merge_c values (3,'a'); +insert into table_to_merge_c values (3,'b'); +insert into table_to_merge_c values (3,'c'); + +-- expected output: +-- 1 c, 2 a, 2 b, 3 c +SELECT * FROM merge_table ORDER BY id, val; From 2de89f62e2ce90ec24ba98312fa152c33516eed4 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 2 Feb 2023 17:30:09 -0300 Subject: [PATCH 035/253] make function static --- src/Analyzer/Passes/AutoFinalOnQueryPass.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp b/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp index 97a770a4a6c..613a7b36261 100644 --- a/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp +++ b/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp @@ -8,7 +8,6 @@ namespace DB { - namespace { class AutoFinalOnQueryPassVisitor : public InDepthQueryTreeVisitorWithContext @@ -30,7 +29,7 @@ namespace } private: - bool autoFinalOnQuery(TableNode & table_node, StoragePtr storage, ContextPtr context) + static bool autoFinalOnQuery(TableNode & table_node, StoragePtr storage, ContextPtr context) { // query.tables() is required because not all queries have tables in it, it could be a function. bool is_auto_final_setting_on = context->getSettingsRef().final; From 75e3faed851adf9939dcdb5a1a0f4758d4a5538b Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 2 Feb 2023 17:32:07 -0300 Subject: [PATCH 036/253] remove unnecessary comment --- src/Analyzer/Passes/AutoFinalOnQueryPass.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp b/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp index 613a7b36261..10efebe0731 100644 --- a/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp +++ b/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp @@ -31,7 +31,6 @@ namespace private: static bool autoFinalOnQuery(TableNode & table_node, StoragePtr storage, ContextPtr context) { - // query.tables() is required because not all queries have tables in it, it could be a function. bool is_auto_final_setting_on = context->getSettingsRef().final; bool is_final_supported = storage && storage->supportsFinal() && !storage->isRemote(); bool is_query_already_final = table_node.hasTableExpressionModifiers() ? table_node.getTableExpressionModifiers().has_value() : false; From 0f2b9612e3a183b4a8cabd0792fe758e51ac4f1c Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 3 Feb 2023 10:03:39 -0300 Subject: [PATCH 037/253] revert old architecture visitor until I get it working if needed --- src/Interpreters/AutoFinalOnQueryVisitor.cpp | 37 -------------------- src/Interpreters/AutoFinalOnQueryVisitor.h | 31 ---------------- src/Interpreters/InterpreterSelectQuery.cpp | 13 +++---- 3 files changed, 4 insertions(+), 77 deletions(-) delete mode 100644 src/Interpreters/AutoFinalOnQueryVisitor.cpp delete mode 100644 src/Interpreters/AutoFinalOnQueryVisitor.h diff --git a/src/Interpreters/AutoFinalOnQueryVisitor.cpp b/src/Interpreters/AutoFinalOnQueryVisitor.cpp deleted file mode 100644 index f2b3eb147c1..00000000000 --- a/src/Interpreters/AutoFinalOnQueryVisitor.cpp +++ /dev/null @@ -1,37 +0,0 @@ -#include "AutoFinalOnQueryVisitor.h" -#include -#include - -namespace DB -{ - -void AutoFinalOnQuery::visit(ASTPtr & query, Data & data) -{ - if (auto * select_query = query->as()) - visit(*select_query, data.storage, data.context); -} - -void AutoFinalOnQuery::visit(ASTSelectQuery & query, StoragePtr storage, ContextPtr context) -{ - if (autoFinalOnQuery(query, storage, context)) - { - query.setFinal(); - } -} - -bool AutoFinalOnQuery::needChildVisit(ASTPtr &, const ASTPtr &) -{ - return true; -} - -bool AutoFinalOnQuery::autoFinalOnQuery(ASTSelectQuery & query, StoragePtr storage, ContextPtr context) -{ - // query.tables() is required because not all queries have tables in it, it could be a function. - bool is_auto_final_setting_on = context->getSettingsRef().final; - bool is_final_supported = storage && storage->supportsFinal() && !storage->isRemote() && query.tables(); - bool is_query_already_final = query.final(); - - return is_auto_final_setting_on && !is_query_already_final && is_final_supported; -} - -} diff --git a/src/Interpreters/AutoFinalOnQueryVisitor.h b/src/Interpreters/AutoFinalOnQueryVisitor.h deleted file mode 100644 index 850fb9fc0e4..00000000000 --- a/src/Interpreters/AutoFinalOnQueryVisitor.h +++ /dev/null @@ -1,31 +0,0 @@ -#pragma once - -#include -#include -#include -#include - -namespace DB -{ - -class AutoFinalOnQuery -{ -public: - struct Data - { - StoragePtr storage; - ContextPtr context; - }; - - static bool needChildVisit(ASTPtr &, const ASTPtr &); - static void visit(ASTPtr & query, Data & data); - -private: - static void visit(ASTSelectQuery & select, StoragePtr storage, ContextPtr context); - static bool autoFinalOnQuery(ASTSelectQuery & select_query, StoragePtr storage, ContextPtr context); - -}; - -using AutoFinalOnQueryVisitor = InDepthNodeVisitor; - -} diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8114b7a5375..ae4ccc34f78 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -38,7 +38,6 @@ #include #include #include -#include #include #include @@ -504,14 +503,10 @@ InterpreterSelectQuery::InterpreterSelectQuery( query_info.additional_filter_ast = parseAdditionalFilterConditionForTable( settings.additional_table_filters, joined_tables.tablesWithColumns().front().table, *context); - AutoFinalOnQuery::Data abc{storage, context}; - - AutoFinalOnQueryVisitor(abc).visit(query_ptr); - -// if (autoFinalOnQuery(query)) -// { -// query.setFinal(); -// } + if (autoFinalOnQuery(query)) + { + query.setFinal(); + } auto analyze = [&] (bool try_move_to_prewhere) { From 6f47c36389dc0c761c880ea221e91d165032ebcd Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 3 Feb 2023 14:31:41 -0300 Subject: [PATCH 038/253] fix test --- tests/queries/0_stateless/02420_final_setting_analyzer.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02420_final_setting_analyzer.reference b/tests/queries/0_stateless/02420_final_setting_analyzer.reference index dfc64fa39a1..2f8bc48fc65 100644 --- a/tests/queries/0_stateless/02420_final_setting_analyzer.reference +++ b/tests/queries/0_stateless/02420_final_setting_analyzer.reference @@ -86,7 +86,6 @@ FROM val_middle, id FROM middle_table - FINAL ) AS middle_table ON `--left_table.id` = `--middle_table.id` ) AS `--.s` ALL INNER JOIN From 7fee8995d373e47ef06257a40a1c72709cafd88a Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 10 Feb 2023 12:04:05 +0100 Subject: [PATCH 039/253] Addressed review comments and moved concurrency check to Backup/Restore Coordination - Use cluster state data to check concurrent backup/restore --- src/Backups/BackupCoordinationLocal.cpp | 5 + src/Backups/BackupCoordinationLocal.h | 2 + src/Backups/BackupCoordinationRemote.cpp | 55 ++++++++- src/Backups/BackupCoordinationRemote.h | 11 +- src/Backups/BackupsWorker.cpp | 144 ++-------------------- src/Backups/BackupsWorker.h | 4 - src/Backups/IBackupCoordination.h | 4 + src/Backups/IRestoreCoordination.h | 4 + src/Backups/RestoreCoordinationLocal.cpp | 5 + src/Backups/RestoreCoordinationLocal.h | 2 + src/Backups/RestoreCoordinationRemote.cpp | 56 ++++++++- src/Backups/RestoreCoordinationRemote.h | 6 +- 12 files changed, 146 insertions(+), 152 deletions(-) diff --git a/src/Backups/BackupCoordinationLocal.cpp b/src/Backups/BackupCoordinationLocal.cpp index e199e43fe01..dd5afbed8e8 100644 --- a/src/Backups/BackupCoordinationLocal.cpp +++ b/src/Backups/BackupCoordinationLocal.cpp @@ -202,4 +202,9 @@ Strings BackupCoordinationLocal::getAllArchiveSuffixes() const return archive_suffixes; } +bool BackupCoordinationLocal::hasConcurrentBackups(const String &, const String &, const std::atomic & num_active_backups) const +{ + return (num_active_backups > 1); +} + } diff --git a/src/Backups/BackupCoordinationLocal.h b/src/Backups/BackupCoordinationLocal.h index 43145a42bf6..6e8a793ccd4 100644 --- a/src/Backups/BackupCoordinationLocal.h +++ b/src/Backups/BackupCoordinationLocal.h @@ -52,6 +52,8 @@ public: String getNextArchiveSuffix() override; Strings getAllArchiveSuffixes() const override; + bool hasConcurrentBackups(const String & backup_id, const String & common_backups_path, const std::atomic & num_active_backups) const override; + private: mutable std::mutex mutex; BackupCoordinationReplicatedTables replicated_tables TSA_GUARDED_BY(mutex); diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index ad164768333..f613eb7d198 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -18,6 +19,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +namespace Stage = BackupCoordinationStage; + /// zookeeper_path/file_names/file_name->checksum_and_size /// zookeeper_path/file_infos/checksum_and_size->info /// zookeeper_path/archive_suffixes @@ -163,10 +166,10 @@ namespace } BackupCoordinationRemote::BackupCoordinationRemote( - const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, bool remove_zk_nodes_in_destructor_) + const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, bool is_internal_) : zookeeper_path(zookeeper_path_) , get_zookeeper(get_zookeeper_) - , remove_zk_nodes_in_destructor(remove_zk_nodes_in_destructor_) + , is_internal(is_internal_) { createRootNodes(); stage_sync.emplace( @@ -177,7 +180,7 @@ BackupCoordinationRemote::~BackupCoordinationRemote() { try { - if (remove_zk_nodes_in_destructor) + if (!is_internal) removeAllNodes(); } catch (...) @@ -592,4 +595,50 @@ Strings BackupCoordinationRemote::getAllArchiveSuffixes() const return node_names; } +bool BackupCoordinationRemote::hasConcurrentBackups(const String & backup_id, const String & common_backup_path, const std::atomic &) const +{ + /// If its internal concurrency will be checked for the base backup + if (is_internal) + return false; + + auto zk = getZooKeeper(); + std::string backup_stage_path = common_backup_path + "/backup-" + toString(backup_id) +"/stage"; + + if (!zk->exists(common_backup_path)) + zk->createAncestors(common_backup_path); + + for (size_t attempt = 0; attempt < MAX_ZOOKEEPER_ATTEMPTS; ++attempt) + { + Coordination::Stat stat; + zk->get(common_backup_path, &stat); + Strings existing_backup_paths = zk->getChildren(common_backup_path); + + for (const auto & existing_backup_path : existing_backup_paths) + { + if (startsWith(existing_backup_path, "restore-")) + continue; + + String existing_backup_id = existing_backup_path; + existing_backup_id.erase(0, String("backup-").size()); + + if (existing_backup_id == toString(backup_id)) + continue; + + const auto status = zk->get(common_backup_path + "/" + existing_backup_path + "/stage"); + if (status != Stage::COMPLETED) + return true; + } + + zk->createIfNotExists(backup_stage_path, ""); + auto code = zk->trySet(backup_stage_path, Stage::SCHEDULED_TO_START, stat.version); + if (code == Coordination::Error::ZOK) + break; + bool is_last_attempt = (attempt == MAX_ZOOKEEPER_ATTEMPTS - 1); + if ((code != Coordination::Error::ZBADVERSION) || is_last_attempt) + throw zkutil::KeeperException(code, backup_stage_path); + } + return false; +} + + } diff --git a/src/Backups/BackupCoordinationRemote.h b/src/Backups/BackupCoordinationRemote.h index b1a7a12e109..50b98fcd77d 100644 --- a/src/Backups/BackupCoordinationRemote.h +++ b/src/Backups/BackupCoordinationRemote.h @@ -5,17 +5,18 @@ #include #include -/// We try to store data to zookeeper several times due to possible version conflicts. -constexpr size_t MAX_ZOOKEEPER_ATTEMPTS = 10; namespace DB { +/// We try to store data to zookeeper several times due to possible version conflicts. +constexpr size_t MAX_ZOOKEEPER_ATTEMPTS = 10; + /// Implementation of the IBackupCoordination interface performing coordination via ZooKeeper. It's necessary for "BACKUP ON CLUSTER". class BackupCoordinationRemote : public IBackupCoordination { public: - BackupCoordinationRemote(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, bool remove_zk_nodes_in_destructor_); + BackupCoordinationRemote(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, bool is_internal_); ~BackupCoordinationRemote() override; void setStage(const String & current_host, const String & new_stage, const String & message) override; @@ -57,6 +58,8 @@ public: String getNextArchiveSuffix() override; Strings getAllArchiveSuffixes() const override; + bool hasConcurrentBackups(const String & backup_id, const String & common_backups_path, const std::atomic & num_active_backups) const override; + private: zkutil::ZooKeeperPtr getZooKeeper() const; zkutil::ZooKeeperPtr getZooKeeperNoLock() const; @@ -67,7 +70,7 @@ private: const String zookeeper_path; const zkutil::GetZooKeeper get_zookeeper; - const bool remove_zk_nodes_in_destructor; + const bool is_internal; std::optional stage_sync; diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 743f3329f5c..133e6989ff5 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -43,7 +43,7 @@ namespace if (!coordination_zk_path.empty()) { auto get_zookeeper = [global_context = context->getGlobalContext()] { return global_context->getZooKeeper(); }; - return std::make_shared(coordination_zk_path, get_zookeeper, !is_internal_backup); + return std::make_shared(coordination_zk_path, get_zookeeper, is_internal_backup); } else { @@ -56,7 +56,7 @@ namespace if (!coordination_zk_path.empty()) { auto get_zookeeper = [global_context = context->getGlobalContext()] { return global_context->getZooKeeper(); }; - return std::make_shared(coordination_zk_path, get_zookeeper, !is_internal_backup); + return std::make_shared(coordination_zk_path, get_zookeeper, is_internal_backup); } else { @@ -261,15 +261,17 @@ void BackupsWorker::doBackup( if (!on_cluster) context->checkAccess(required_access); + String root_zk_path; + ClusterPtr cluster; if (on_cluster) { + root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); backup_query->cluster = context->getMacros()->expand(backup_query->cluster); cluster = context->getCluster(backup_query->cluster); backup_settings.cluster_host_ids = cluster->getHostIDs(); if (backup_settings.coordination_zk_path.empty()) { - String root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); backup_settings.coordination_zk_path = root_zk_path + "/backup-" + toString(*backup_settings.backup_uuid); } } @@ -278,7 +280,7 @@ void BackupsWorker::doBackup( if (!backup_coordination) backup_coordination = makeBackupCoordination(backup_settings.coordination_zk_path, context, backup_settings.internal); - if (!allow_concurrent_backups && !backup_settings.internal && hasConcurrentBackups(backup_id, context, on_cluster)) + if (!allow_concurrent_backups && backup_coordination->hasConcurrentBackups(backup_id, root_zk_path, std::ref(num_active_backups))) throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Concurrent backups not supported, turn on setting 'allow_concurrent_backups'"); /// Opens a backup for writing. @@ -487,13 +489,14 @@ void BackupsWorker::doRestore( BackupPtr backup = BackupFactory::instance().createBackup(backup_open_params); String current_database = context->getCurrentDatabase(); - + String root_zk_path; /// Checks access rights if this is ON CLUSTER query. /// (If this isn't ON CLUSTER query RestorerFromBackup will check access rights later.) ClusterPtr cluster; bool on_cluster = !restore_query->cluster.empty(); if (on_cluster) { + root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); restore_query->cluster = context->getMacros()->expand(restore_query->cluster); cluster = context->getCluster(restore_query->cluster); restore_settings.cluster_host_ids = cluster->getHostIDs(); @@ -517,14 +520,13 @@ void BackupsWorker::doRestore( /// Make a restore coordination. if (on_cluster && restore_settings.coordination_zk_path.empty()) { - String root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); restore_settings.coordination_zk_path = root_zk_path + "/restore-" + toString(restore_id); } if (!restore_coordination) restore_coordination = makeRestoreCoordination(restore_settings.coordination_zk_path, context, restore_settings.internal); - if (!allow_concurrent_restores && !restore_settings.internal && hasConcurrentRestores(restore_id, context, on_cluster)) + if (!allow_concurrent_restores && restore_coordination->hasConcurrentRestores(restore_id, root_zk_path, std::ref(num_active_restores))) throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Concurrent restores not supported, turn on setting 'allow_concurrent_restores'"); /// Do RESTORE. @@ -709,134 +711,6 @@ std::vector BackupsWorker::getAllInfos() const return res_infos; } -std::vector BackupsWorker::getAllActiveBackupInfos() const -{ - std::vector res_infos; - std::lock_guard lock{infos_mutex}; - for (const auto & info : infos | boost::adaptors::map_values) - { - if (info.status==BackupStatus::CREATING_BACKUP) - res_infos.push_back(info); - } - return res_infos; -} - -std::vector BackupsWorker::getAllActiveRestoreInfos() const -{ - std::vector res_infos; - std::lock_guard lock{infos_mutex}; - for (const auto & info : infos | boost::adaptors::map_values) - { - if (info.status==BackupStatus::RESTORING) - res_infos.push_back(info); - } - return res_infos; -} - -bool BackupsWorker::hasConcurrentBackups(const OperationID & backup_id, const ContextPtr & context, bool on_cluster) const -{ - if (on_cluster) - { - String common_backup_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups") ; - auto zookeeper = context->getGlobalContext()->getZooKeeper(); - std::string backup_stage_path = common_backup_path + "/backup-" + toString(backup_id) +"/stage"; - - if (!zookeeper->exists(common_backup_path)) - zookeeper->createAncestors(common_backup_path); - - for (size_t attempt = 0; attempt < MAX_ZOOKEEPER_ATTEMPTS; ++attempt) - { - Coordination::Stat stat; - zookeeper->get(common_backup_path, &stat); - Strings existing_backup_paths = zookeeper->getChildren(common_backup_path); - - for (const auto & existing_backup_path : existing_backup_paths) - { - if (startsWith(existing_backup_path, "restore-")) - continue; - - String existing_backup_id = existing_backup_path; - existing_backup_id.erase(0, String("backup-").size()); - - if (existing_backup_id == toString(backup_id)) - continue; - - const auto status = zookeeper->get(common_backup_path + "/" + existing_backup_path + "/stage"); - if (status != Stage::COMPLETED) - return true; - } - - zookeeper->createIfNotExists(backup_stage_path, ""); - auto code = zookeeper->trySet(backup_stage_path, Stage::SCHEDULED_TO_START, stat.version); - if (code == Coordination::Error::ZOK) - break; - bool is_last_attempt = (attempt == MAX_ZOOKEEPER_ATTEMPTS - 1); - if ((code != Coordination::Error::ZBADVERSION) || is_last_attempt) - throw zkutil::KeeperException(code, backup_stage_path); - } - return false; - } - else - { - if (num_active_backups == 1) - return false; - else - return true; - } -} - -bool BackupsWorker::hasConcurrentRestores(const OperationID & restore_id, const ContextPtr & context, bool on_cluster) const -{ - if (on_cluster) - { - String common_restore_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups") ; - auto zookeeper = context->getGlobalContext()->getZooKeeper(); - std::string path = common_restore_path + "/restore-" + toString(restore_id) +"/stage"; - - if (!zookeeper->exists(common_restore_path)) - zookeeper->createAncestors(common_restore_path); - - for (size_t attempt = 0; attempt < MAX_ZOOKEEPER_ATTEMPTS; ++attempt) - { - Coordination::Stat stat; - zookeeper->get(common_restore_path, &stat); - Strings existing_restore_paths = zookeeper->getChildren(common_restore_path); - for (const auto & existing_restore_path : existing_restore_paths) - { - if (startsWith(existing_restore_path, "backup-")) - continue; - - String existing_restore_id = existing_restore_path; - existing_restore_id.erase(0, String("restore-").size()); - - if (existing_restore_id == toString(restore_id)) - continue; - - - const auto status = zookeeper->get(common_restore_path + "/" + existing_restore_path + "/stage"); - if (status != Stage::COMPLETED) - return true; - } - - zookeeper->createIfNotExists(path, ""); - auto code = zookeeper->trySet(path, Stage::SCHEDULED_TO_START, stat.version); - if (code == Coordination::Error::ZOK) - break; - bool is_last_attempt = (attempt == MAX_ZOOKEEPER_ATTEMPTS - 1); - if ((code != Coordination::Error::ZBADVERSION) || is_last_attempt) - throw zkutil::KeeperException(code, path); - } - return false; - } - else - { - if (num_active_restores == 1) - return false; - else - return true; - } -} - void BackupsWorker::shutdown() { bool has_active_backups_and_restores = (num_active_backups || num_active_restores); diff --git a/src/Backups/BackupsWorker.h b/src/Backups/BackupsWorker.h index 994e5f1f8b7..e169a9a2597 100644 --- a/src/Backups/BackupsWorker.h +++ b/src/Backups/BackupsWorker.h @@ -111,10 +111,6 @@ private: void setStatus(const OperationID & id, BackupStatus status, bool throw_if_error = true); void setStatusSafe(const String & id, BackupStatus status) { setStatus(id, status, false); } void setNumFilesAndSize(const OperationID & id, size_t num_files, size_t num_processed_files, UInt64 processed_files_size, UInt64 uncompressed_size, UInt64 compressed_size); - std::vector getAllActiveBackupInfos() const; - std::vector getAllActiveRestoreInfos() const; - bool hasConcurrentBackups(const OperationID & backup_id, const ContextPtr & context, bool on_cluster) const; - bool hasConcurrentRestores(const OperationID & restore_id, const ContextPtr & context, bool on_cluster) const; ThreadPool backups_thread_pool; ThreadPool restores_thread_pool; diff --git a/src/Backups/IBackupCoordination.h b/src/Backups/IBackupCoordination.h index 7ff911488aa..787b86f8cbd 100644 --- a/src/Backups/IBackupCoordination.h +++ b/src/Backups/IBackupCoordination.h @@ -114,6 +114,10 @@ public: /// Returns the list of all the archive suffixes which were generated. virtual Strings getAllArchiveSuffixes() const = 0; + + /// This function is used to check if concurrent backups are running + /// other than the backup passed to the function + virtual bool hasConcurrentBackups(const String & backup_id, const String & common_backups_path, const std::atomic & num_active_backups) const = 0; }; } diff --git a/src/Backups/IRestoreCoordination.h b/src/Backups/IRestoreCoordination.h index 692054ae267..a2506e5c157 100644 --- a/src/Backups/IRestoreCoordination.h +++ b/src/Backups/IRestoreCoordination.h @@ -34,6 +34,10 @@ public: /// Sets that this replica is going to restore a ReplicatedAccessStorage. /// The function returns false if this access storage is being already restored by another replica. virtual bool acquireReplicatedAccessStorage(const String & access_storage_zk_path) = 0; + + /// This function is used to check if concurrent restores are running + /// other than the restore passed to the function + virtual bool hasConcurrentRestores(const String & restore_id, const String & common_restores_path, const std::atomic & num_active_restores) const = 0; }; } diff --git a/src/Backups/RestoreCoordinationLocal.cpp b/src/Backups/RestoreCoordinationLocal.cpp index b2a9849c38d..56951c56baa 100644 --- a/src/Backups/RestoreCoordinationLocal.cpp +++ b/src/Backups/RestoreCoordinationLocal.cpp @@ -42,4 +42,9 @@ bool RestoreCoordinationLocal::acquireReplicatedAccessStorage(const String &) return true; } +bool RestoreCoordinationLocal::hasConcurrentRestores(const String &, const String &, const std::atomic & num_active_restores) const +{ + return (num_active_restores > 1); +} + } diff --git a/src/Backups/RestoreCoordinationLocal.h b/src/Backups/RestoreCoordinationLocal.h index b4e70d83b72..fa6941a7577 100644 --- a/src/Backups/RestoreCoordinationLocal.h +++ b/src/Backups/RestoreCoordinationLocal.h @@ -35,6 +35,8 @@ public: /// The function returns false if this access storage is being already restored by another replica. bool acquireReplicatedAccessStorage(const String & access_storage_zk_path) override; + bool hasConcurrentRestores(const String & restore_id, const String & common_restores_path, const std::atomic & num_active_restores) const override; + private: std::set> acquired_tables_in_replicated_databases; std::unordered_set acquired_data_in_replicated_tables; diff --git a/src/Backups/RestoreCoordinationRemote.cpp b/src/Backups/RestoreCoordinationRemote.cpp index 89a9950aad2..95766bfcae3 100644 --- a/src/Backups/RestoreCoordinationRemote.cpp +++ b/src/Backups/RestoreCoordinationRemote.cpp @@ -1,16 +1,19 @@ #include #include #include - +#include +#include namespace DB { +namespace Stage = BackupCoordinationStage; + RestoreCoordinationRemote::RestoreCoordinationRemote( - const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, bool remove_zk_nodes_in_destructor_) + const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, bool is_internal_) : zookeeper_path(zookeeper_path_) , get_zookeeper(get_zookeeper_) - , remove_zk_nodes_in_destructor(remove_zk_nodes_in_destructor_) + , is_internal(is_internal_) { createRootNodes(); @@ -22,7 +25,7 @@ RestoreCoordinationRemote::~RestoreCoordinationRemote() { try { - if (remove_zk_nodes_in_destructor) + if (!is_internal) removeAllNodes(); } catch (...) @@ -129,4 +132,49 @@ void RestoreCoordinationRemote::removeAllNodes() zk->removeRecursive(zookeeper_path); } +bool RestoreCoordinationRemote::hasConcurrentRestores(const String & restore_id, const String & common_restores_path, const std::atomic &) const +{ + /// If its internal concurrency will be checked for the base restore + if (is_internal) + return false; + + auto zk = getZooKeeper(); + std::string path = common_restores_path + "/restore-" + toString(restore_id) +"/stage"; + + if (! zk->exists(common_restores_path)) + zk->createAncestors(common_restores_path); + + for (size_t attempt = 0; attempt < MAX_ZOOKEEPER_ATTEMPTS; ++attempt) + { + Coordination::Stat stat; + zk->get(common_restores_path, &stat); + Strings existing_restore_paths = zk->getChildren(common_restores_path); + for (const auto & existing_restore_path : existing_restore_paths) + { + if (startsWith(existing_restore_path, "backup-")) + continue; + + String existing_restore_id = existing_restore_path; + existing_restore_id.erase(0, String("restore-").size()); + + if (existing_restore_id == toString(restore_id)) + continue; + + + const auto status = zk->get(common_restores_path + "/" + existing_restore_path + "/stage"); + if (status != Stage::COMPLETED) + return true; + } + + zk->createIfNotExists(path, ""); + auto code = zk->trySet(path, Stage::SCHEDULED_TO_START, stat.version); + if (code == Coordination::Error::ZOK) + break; + bool is_last_attempt = (attempt == MAX_ZOOKEEPER_ATTEMPTS - 1); + if ((code != Coordination::Error::ZBADVERSION) || is_last_attempt) + throw zkutil::KeeperException(code, path); + } + return false; +} + } diff --git a/src/Backups/RestoreCoordinationRemote.h b/src/Backups/RestoreCoordinationRemote.h index 83760a2d883..67d78192e1e 100644 --- a/src/Backups/RestoreCoordinationRemote.h +++ b/src/Backups/RestoreCoordinationRemote.h @@ -11,7 +11,7 @@ namespace DB class RestoreCoordinationRemote : public IRestoreCoordination { public: - RestoreCoordinationRemote(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, bool remove_zk_nodes_in_destructor_); + RestoreCoordinationRemote(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, bool is_internal_); ~RestoreCoordinationRemote() override; /// Sets the current stage and waits for other hosts to come to this stage too. @@ -31,6 +31,8 @@ public: /// The function returns false if this access storage is being already restored by another replica. bool acquireReplicatedAccessStorage(const String & access_storage_zk_path) override; + bool hasConcurrentRestores(const String & restore_id, const String & common_restores_path, const std::atomic & num_active_restores) const override; + private: zkutil::ZooKeeperPtr getZooKeeper() const; void createRootNodes(); @@ -40,7 +42,7 @@ private: const String zookeeper_path; const zkutil::GetZooKeeper get_zookeeper; - const bool remove_zk_nodes_in_destructor; + const bool is_internal; std::optional stage_sync; From 94fba0b6649b7425ef6b59eeaad4274999e04bd2 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 10 Feb 2023 13:53:21 +0100 Subject: [PATCH 040/253] Fixed build issue caused after merge master in BackupsWorker.h - Use cluster state data to check concurrent backup/restore --- src/Backups/BackupsWorker.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Backups/BackupsWorker.h b/src/Backups/BackupsWorker.h index a704db720b5..d0f3ad99b5f 100644 --- a/src/Backups/BackupsWorker.h +++ b/src/Backups/BackupsWorker.h @@ -114,8 +114,9 @@ private: void addInfo(const OperationID & id, const String & name, bool internal, BackupStatus status); void setStatus(const OperationID & id, BackupStatus status, bool throw_if_error = true); void setStatusSafe(const String & id, BackupStatus status) { setStatus(id, status, false); } - void setNumFilesAndSize(const OperationID & id, size_t num_files, size_t num_processed_files, UInt64 processed_files_size, UInt64 uncompressed_size, UInt64 compressed_size); - + void setNumFilesAndSize(const OperationID & id, size_t num_files, UInt64 total_size, size_t num_entries, + UInt64 uncompressed_size, UInt64 compressed_size, size_t num_read_files, UInt64 num_read_bytes); + ThreadPool backups_thread_pool; ThreadPool restores_thread_pool; From 2ce67830c8c4b822ddf98fc7c4ca77966ec88b61 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 10 Feb 2023 14:41:43 +0100 Subject: [PATCH 041/253] Fixed style check by removing trailing whitespaces in BackupsWorker.h - Use cluster state data to check concurrent backup/restore --- src/Backups/BackupsWorker.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Backups/BackupsWorker.h b/src/Backups/BackupsWorker.h index d0f3ad99b5f..0f5c16cd71f 100644 --- a/src/Backups/BackupsWorker.h +++ b/src/Backups/BackupsWorker.h @@ -116,7 +116,7 @@ private: void setStatusSafe(const String & id, BackupStatus status) { setStatus(id, status, false); } void setNumFilesAndSize(const OperationID & id, size_t num_files, UInt64 total_size, size_t num_entries, UInt64 uncompressed_size, UInt64 compressed_size, size_t num_read_files, UInt64 num_read_bytes); - + ThreadPool backups_thread_pool; ThreadPool restores_thread_pool; From d1efd024809cea35b2bb099e32208ad88603c37a Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 10 Feb 2023 16:40:14 +0000 Subject: [PATCH 042/253] Extend setting input_format_null_as_default for more formats --- src/Columns/ColumnLowCardinality.cpp | 28 ++++++++ src/Columns/ColumnLowCardinality.h | 2 + src/Columns/ColumnNullable.cpp | 22 ++++++ src/Columns/ColumnNullable.h | 2 + src/Core/Settings.h | 3 +- src/DataTypes/IDataType.h | 5 ++ src/Formats/FormatFactory.cpp | 1 - src/Formats/FormatSettings.h | 1 - src/Formats/NativeReader.cpp | 31 +++++++-- src/Formats/NativeReader.h | 10 ++- src/Formats/insertNullAsDefaultIfNeeded.cpp | 37 ++++++++++ src/Formats/insertNullAsDefaultIfNeeded.h | 10 +++ .../Formats/Impl/ArrowBlockInputFormat.cpp | 9 +-- .../Formats/Impl/ArrowBlockInputFormat.h | 1 - .../Formats/Impl/ArrowColumnToCHColumn.cpp | 39 ++++------- .../Formats/Impl/ArrowColumnToCHColumn.h | 9 ++- .../Formats/Impl/AvroRowInputFormat.cpp | 69 +++++++++++++++---- .../Formats/Impl/AvroRowInputFormat.h | 7 +- .../Formats/Impl/MsgPackRowInputFormat.cpp | 44 +++++++----- .../Formats/Impl/MsgPackRowInputFormat.h | 10 ++- src/Processors/Formats/Impl/NativeFormat.cpp | 12 +++- .../Formats/Impl/ORCBlockInputFormat.cpp | 8 +-- .../Formats/Impl/ORCBlockInputFormat.h | 1 - .../Formats/Impl/ParquetBlockInputFormat.cpp | 13 ++-- .../Formats/Impl/ParquetBlockInputFormat.h | 1 - ...561_null_as_default_more_formats.reference | 36 ++++++++++ .../02561_null_as_default_more_formats.sh | 21 ++++++ ...2_native_null_on_missing_columns.reference | 4 ++ .../02562_native_null_on_missing_columns.sh | 16 +++++ 29 files changed, 352 insertions(+), 100 deletions(-) create mode 100644 src/Formats/insertNullAsDefaultIfNeeded.cpp create mode 100644 src/Formats/insertNullAsDefaultIfNeeded.h create mode 100644 tests/queries/0_stateless/02561_null_as_default_more_formats.reference create mode 100755 tests/queries/0_stateless/02561_null_as_default_more_formats.sh create mode 100644 tests/queries/0_stateless/02562_native_null_on_missing_columns.reference create mode 100755 tests/queries/0_stateless/02562_native_null_on_missing_columns.sh diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index ecdaf240e5e..109bf201836 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -830,4 +830,32 @@ void ColumnLowCardinality::Dictionary::compact(ColumnPtr & positions) shared = false; } +ColumnPtr ColumnLowCardinality::cloneWithDefaultOnNull() const +{ + if (!nestedIsNullable()) + return getPtr(); + + auto res = cloneEmpty(); + auto & lc_res = assert_cast(*res); + lc_res.nestedRemoveNullable(); + size_t end = size(); + size_t start = 0; + while (start < end) + { + size_t next_null_index = start; + while (next_null_index < end && !isNullAt(next_null_index)) + ++next_null_index; + + if (next_null_index != start) + lc_res.insertRangeFrom(*this, start, next_null_index - start); + + if (next_null_index < end) + lc_res.insertDefault(); + + start = next_null_index + 1; + } + + return res; +} + } diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index e895bc6b54e..3d42f82a867 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -220,6 +220,8 @@ public: void nestedToNullable() { dictionary.getColumnUnique().nestedToNullable(); } void nestedRemoveNullable() { dictionary.getColumnUnique().nestedRemoveNullable(); } + ColumnPtr cloneWithDefaultOnNull() const; + const IColumnUnique & getDictionary() const { return dictionary.getColumnUnique(); } IColumnUnique & getDictionary() { return dictionary.getColumnUnique(); } const ColumnPtr & getDictionaryPtr() const { return dictionary.getColumnUniquePtr(); } diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 9398c66bef0..99d377f10eb 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -781,6 +781,28 @@ ColumnPtr ColumnNullable::createWithOffsets(const IColumn::Offsets & offsets, co return ColumnNullable::create(new_values, new_null_map); } +ColumnPtr ColumnNullable::getNestedColumnWithDefaultOnNull() const +{ + auto res = nested_column->cloneEmpty(); + const auto & null_map_data = getNullMapData(); + size_t start = 0; + while (start < nested_column->size()) + { + size_t next_null_index = start; + while (next_null_index < null_map->size() && !null_map_data[next_null_index]) + ++next_null_index; + + if (next_null_index != start) + res->insertRangeFrom(*nested_column, start, next_null_index - start); + + if (next_null_index < null_map->size()) + res->insertDefault(); + + start = next_null_index + 1; + } + return res; +} + ColumnPtr makeNullable(const ColumnPtr & column) { if (isColumnNullable(*column)) diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 85bf095a9d1..1ec037092b5 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -188,6 +188,8 @@ public: NullMap & getNullMapData() { return getNullMapColumn().getData(); } const NullMap & getNullMapData() const { return getNullMapColumn().getData(); } + ColumnPtr getNestedColumnWithDefaultOnNull() const; + /// Apply the null byte map of a specified nullable column onto the /// null byte map of the current column by performing an element-wise OR /// between both byte maps. This method is used to determine the null byte diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6e085fd27ac..481929d915f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -751,7 +751,7 @@ class IColumn; M(Bool, input_format_csv_empty_as_default, true, "Treat empty fields in CSV input as default values.", 0) \ M(Bool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.", 0) \ M(Bool, input_format_tsv_enum_as_number, false, "Treat inserted enum values in TSV formats as enum indices.", 0) \ - M(Bool, input_format_null_as_default, true, "For text input formats initialize null fields with default values if data type of this field is not nullable", 0) \ + M(Bool, input_format_null_as_default, true, "For most input formats initialize null fields with default values if data type of this field is not nullable", 0) \ M(Bool, input_format_arrow_import_nested, false, "Allow to insert array of structs into Nested table in Arrow input format.", 0) \ M(Bool, input_format_arrow_case_insensitive_column_matching, false, "Ignore case when matching Arrow columns with CH columns.", 0) \ M(Bool, input_format_orc_import_nested, false, "Allow to insert array of structs into Nested table in ORC input format.", 0) \ @@ -811,6 +811,7 @@ class IColumn; M(Bool, input_format_values_deduce_templates_of_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser, deduce template of the SQL expression, try to parse all rows using template and then interpret expression for all rows.", 0) \ M(Bool, input_format_values_accurate_types_of_literals, true, "For Values format: when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues.", 0) \ M(Bool, input_format_avro_allow_missing_fields, false, "For Avro/AvroConfluent format: when field is not found in schema use default value instead of error", 0) \ + /** This setting is obsolete and do nothing, left for compatibility reasons. */ \ M(Bool, input_format_avro_null_as_default, false, "For Avro/AvroConfluent format: insert default in case of null and non Nullable column", 0) \ M(UInt64, format_binary_max_string_size, 1_GiB, "The maximum allowed size for String in RowBinary format. It prevents allocating large amount of memory in case of corrupted data. 0 means there is no limit", 0) \ M(URI, format_avro_schema_registry_url, "", "For AvroConfluent format: Confluent Schema Registry URL.", 0) \ diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index bafe03dbc3a..f2230b70cab 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -548,6 +548,11 @@ inline bool isAggregateFunction(const DataTypePtr & data_type) return which.isAggregateFunction(); } +inline bool isNullableOrLowCardinalityNullable(const DataTypePtr & data_type) +{ + return data_type->isNullable() || data_type->isLowCardinalityNullable(); +} + template constexpr bool IsDataTypeDecimal = false; template constexpr bool IsDataTypeNumber = false; template constexpr bool IsDataTypeDateOrDateTime = false; diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 3fcecd23f5b..8d8ffebe270 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -56,7 +56,6 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.avro.schema_registry_url = settings.format_avro_schema_registry_url.toString(); format_settings.avro.string_column_pattern = settings.output_format_avro_string_column_pattern.toString(); format_settings.avro.output_rows_in_file = settings.output_format_avro_rows_in_file; - format_settings.avro.null_as_default = settings.input_format_avro_null_as_default; format_settings.csv.allow_double_quotes = settings.format_csv_allow_double_quotes; format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes; format_settings.csv.crlf_end_of_line = settings.output_format_csv_crlf_end_of_line; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 92e499abb10..2bf8e136c63 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -104,7 +104,6 @@ struct FormatSettings bool allow_missing_fields = false; String string_column_pattern; UInt64 output_rows_in_file = 1; - bool null_as_default = false; } avro; String bool_true_representation = "true"; diff --git a/src/Formats/NativeReader.cpp b/src/Formats/NativeReader.cpp index 58baee5931b..9f8d4ba1930 100644 --- a/src/Formats/NativeReader.cpp +++ b/src/Formats/NativeReader.cpp @@ -10,6 +10,7 @@ #include #include +#include #include #include #include @@ -32,8 +33,19 @@ NativeReader::NativeReader(ReadBuffer & istr_, UInt64 server_revision_) { } -NativeReader::NativeReader(ReadBuffer & istr_, const Block & header_, UInt64 server_revision_, bool skip_unknown_columns_) - : istr(istr_), header(header_), server_revision(server_revision_), skip_unknown_columns(skip_unknown_columns_) +NativeReader::NativeReader( + ReadBuffer & istr_, + const Block & header_, + UInt64 server_revision_, + bool skip_unknown_columns_, + bool null_as_default_, + BlockMissingValues * block_missing_values_) + : istr(istr_) + , header(header_) + , server_revision(server_revision_) + , skip_unknown_columns(skip_unknown_columns_) + , null_as_default(null_as_default_) + , block_missing_values(block_missing_values_) { } @@ -187,8 +199,12 @@ Block NativeReader::read() { if (header.has(column.name)) { - /// Support insert from old clients without low cardinality type. auto & header_column = header.getByName(column.name); + + if (null_as_default) + insertNullAsDefaultIfNeeded(column, header_column, header.getPositionByName(column.name), block_missing_values); + + /// Support insert from old clients without low cardinality type. if (!header_column.type->equals(*column.type)) { column.column = recursiveTypeConversion(column.column, column.type, header.safeGetByPosition(i).type); @@ -225,12 +241,19 @@ Block NativeReader::read() /// Allow to skip columns. Fill them with default values. Block tmp_res; - for (auto & col : header) + for (size_t column_i = 0; column_i != header.columns(); ++column_i) { + auto & col = header.getByPosition(column_i); if (res.has(col.name)) + { tmp_res.insert(res.getByName(col.name)); + } else + { tmp_res.insert({col.type->createColumn()->cloneResized(rows), col.type, col.name}); + if (block_missing_values) + block_missing_values->setBits(column_i, rows); + } } tmp_res.info = res.info; diff --git a/src/Formats/NativeReader.h b/src/Formats/NativeReader.h index 3ae53d45faf..64d3e4d6df0 100644 --- a/src/Formats/NativeReader.h +++ b/src/Formats/NativeReader.h @@ -24,7 +24,13 @@ public: /// For cases when data structure (header) is known in advance. /// NOTE We may use header for data validation and/or type conversions. It is not implemented. - NativeReader(ReadBuffer & istr_, const Block & header_, UInt64 server_revision_, bool skip_unknown_columns_ = false); + NativeReader( + ReadBuffer & istr_, + const Block & header_, + UInt64 server_revision_, + bool skip_unknown_columns_ = false, + bool null_as_default_ = false, + BlockMissingValues * block_missing_values_ = nullptr); /// For cases when we have an index. It allows to skip columns. Only columns specified in the index will be read. NativeReader(ReadBuffer & istr_, UInt64 server_revision_, @@ -44,6 +50,8 @@ private: Block header; UInt64 server_revision; bool skip_unknown_columns; + bool null_as_default; + BlockMissingValues * block_missing_values; bool use_index = false; IndexForNativeFormat::Blocks::const_iterator index_block_it; diff --git a/src/Formats/insertNullAsDefaultIfNeeded.cpp b/src/Formats/insertNullAsDefaultIfNeeded.cpp new file mode 100644 index 00000000000..767892718c5 --- /dev/null +++ b/src/Formats/insertNullAsDefaultIfNeeded.cpp @@ -0,0 +1,37 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +void insertNullAsDefaultIfNeeded(ColumnWithTypeAndName & input_column, const ColumnWithTypeAndName & header_column, size_t column_i, BlockMissingValues * block_missing_values) +{ + if (!isNullableOrLowCardinalityNullable(input_column.type) || isNullableOrLowCardinalityNullable(header_column.type)) + return; + + if (block_missing_values) + { + for (size_t i = 0; i < input_column.column->size(); ++i) + { + if (input_column.column->isNullAt(i)) + block_missing_values->setBit(column_i, i); + } + } + + if (input_column.type->isNullable()) + { + input_column.column = assert_cast(input_column.column.get())->getNestedColumnWithDefaultOnNull(); + input_column.type = removeNullable(input_column.type); + } + else + { + input_column.column = assert_cast(input_column.column.get())->cloneWithDefaultOnNull(); + const auto * lc_type = assert_cast(input_column.type.get()); + input_column.type = std::make_shared(removeNullable(lc_type->getDictionaryType())); + } +} + +} diff --git a/src/Formats/insertNullAsDefaultIfNeeded.h b/src/Formats/insertNullAsDefaultIfNeeded.h new file mode 100644 index 00000000000..3e4dcd1e74a --- /dev/null +++ b/src/Formats/insertNullAsDefaultIfNeeded.h @@ -0,0 +1,10 @@ +#pragma once + +#include + +namespace DB +{ + +void insertNullAsDefaultIfNeeded(ColumnWithTypeAndName & input_column, const ColumnWithTypeAndName & header_column, size_t column_i, BlockMissingValues * block_missing_values); + +} diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index ed963d8a500..cd8facb83eb 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -71,13 +71,10 @@ Chunk ArrowBlockInputFormat::generate() ++record_batch_current; - arrow_column_to_ch_column->arrowTableToCHChunk(res, *table_result, (*table_result)->num_rows()); - /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. /// Otherwise fill the missing columns with zero values of its type. - if (format_settings.defaults_for_omitted_fields) - for (const auto & column_idx : missing_columns) - block_missing_values.setBits(column_idx, res.getNumRows()); + BlockMissingValues * block_missing_values_ptr = format_settings.defaults_for_omitted_fields ? &block_missing_values : nullptr; + arrow_column_to_ch_column->arrowTableToCHChunk(res, *table_result, (*table_result)->num_rows(), block_missing_values_ptr); return res; } @@ -143,8 +140,8 @@ void ArrowBlockInputFormat::prepareReader() "Arrow", format_settings.arrow.import_nested, format_settings.arrow.allow_missing_columns, + format_settings.null_as_default, format_settings.arrow.case_insensitive_column_matching); - missing_columns = arrow_column_to_ch_column->getMissingColumns(*schema); if (stream) record_batch_total = -1; diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h index 02648d28048..3db76777891 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h @@ -47,7 +47,6 @@ private: int record_batch_total = 0; int record_batch_current = 0; - std::vector missing_columns; BlockMissingValues block_missing_values; const FormatSettings format_settings; diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 68c40527097..80172ca9c05 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -30,6 +30,7 @@ #include #include #include +#include #include #include #include @@ -384,9 +385,10 @@ static ColumnWithTypeAndName readColumnWithIndexesDataImpl(std::shared_ptr(buffer->data()); /// Check that indexes are correct (protection against corrupted files) + /// Note that on null values index can be arbitrary value. for (int64_t i = 0; i != chunk->length(); ++i) { - if (data[i] < 0 || data[i] >= dict_size) + if (!chunk->IsNull(i) && (data[i] < 0 || data[i] >= dict_size)) throw Exception(ErrorCodes::INCORRECT_DATA, "Index {} in Dictionary column is out of bounds, dictionary size is {}", Int64(data[i]), UInt64(dict_size)); @@ -805,16 +807,18 @@ ArrowColumnToCHColumn::ArrowColumnToCHColumn( const std::string & format_name_, bool import_nested_, bool allow_missing_columns_, + bool null_as_default_, bool case_insensitive_matching_) : header(header_) , format_name(format_name_) , import_nested(import_nested_) , allow_missing_columns(allow_missing_columns_) + , null_as_default(null_as_default_) , case_insensitive_matching(case_insensitive_matching_) { } -void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr & table, size_t num_rows) +void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr & table, size_t num_rows, BlockMissingValues * block_missing_values) { NameToColumnPtr name_to_column_ptr; for (auto column_name : table->ColumnNames()) @@ -828,10 +832,10 @@ void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptrcloneResized(num_rows); columns_list.push_back(std::move(column.column)); + if (block_missing_values) + block_missing_values->setBits(column_i, num_rows); continue; } } @@ -906,6 +912,9 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & arrow_column, header_column.name, format_name, false, dictionary_infos, true, false, skipped, header_column.type); } + if (null_as_default) + insertNullAsDefaultIfNeeded(column, header_column, column_i, block_missing_values); + try { column.column = castColumn(column, header_column.type); @@ -927,28 +936,6 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & res.setColumns(columns_list, num_rows); } -std::vector ArrowColumnToCHColumn::getMissingColumns(const arrow::Schema & schema) const -{ - std::vector missing_columns; - auto block_from_arrow = arrowSchemaToCHHeader(schema, format_name, false, &header, case_insensitive_matching); - NestedColumnExtractHelper nested_columns_extractor(block_from_arrow, case_insensitive_matching); - - for (size_t i = 0, columns = header.columns(); i < columns; ++i) - { - const auto & header_column = header.getByPosition(i); - if (!block_from_arrow.has(header_column.name, case_insensitive_matching)) - { - if (!import_nested || !nested_columns_extractor.extractColumn(header_column.name)) - { - if (!allow_missing_columns) - throw Exception{ErrorCodes::THERE_IS_NO_COLUMN, "Column '{}' is not presented in input data.", header_column.name}; - missing_columns.push_back(i); - } - } - } - return missing_columns; -} - } #endif diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index dd9f44eb94e..64ff99c70ac 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -26,14 +26,12 @@ public: const std::string & format_name_, bool import_nested_, bool allow_missing_columns_, + bool null_as_default_, bool case_insensitive_matching_ = false); - void arrowTableToCHChunk(Chunk & res, std::shared_ptr & table, size_t num_rows); + void arrowTableToCHChunk(Chunk & res, std::shared_ptr & table, size_t num_rows, BlockMissingValues * block_missing_values = nullptr); - void arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & name_to_column_ptr, size_t num_rows); - - /// Get missing columns that exists in header but not in arrow::Schema - std::vector getMissingColumns(const arrow::Schema & schema) const; + void arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & name_to_column_ptr, size_t num_rows, BlockMissingValues * block_missing_values = nullptr); /// Transform arrow schema to ClickHouse header. If hint_header is provided, /// we will skip columns in schema that are not in hint_header. @@ -58,6 +56,7 @@ private: bool import_nested; /// If false, throw exception if some columns in header not exists in arrow table. bool allow_missing_columns; + bool null_as_default; bool case_insensitive_matching; /// Map {column name : dictionary column}. diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 9a475efa195..e77f4132100 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -176,8 +176,9 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node { auto & lc_column = assert_cast(column); auto tmp_column = lc_column.getDictionary().getNestedColumn()->cloneEmpty(); - dict_deserialize(*tmp_column, decoder); + auto res = dict_deserialize(*tmp_column, decoder); lc_column.insertFromFullColumn(*tmp_column, 0); + return res; }; } @@ -198,6 +199,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node UUID uuid; parseUUID(reinterpret_cast(tmp.data()), std::reverse_iterator(reinterpret_cast(&uuid) + 16)); assert_cast(column).insertValue(uuid); + return true; }; } if (target.isString() || target.isFixedString()) @@ -206,6 +208,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node { decoder.decodeString(tmp); column.insertData(tmp.c_str(), tmp.length()); + return true; }; } break; @@ -215,6 +218,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node return [target](IColumn & column, avro::Decoder & decoder) { insertNumber(column, target, decoder.decodeInt()); + return true; }; } break; @@ -224,6 +228,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node return [target](IColumn & column, avro::Decoder & decoder) { insertNumber(column, target, decoder.decodeLong()); + return true; }; } break; @@ -233,6 +238,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node return [target](IColumn & column, avro::Decoder & decoder) { insertNumber(column, target, decoder.decodeFloat()); + return true; }; } break; @@ -242,6 +248,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node return [target](IColumn & column, avro::Decoder & decoder) { insertNumber(column, target, decoder.decodeDouble()); + return true; }; } break; @@ -251,6 +258,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node return [target](IColumn & column, avro::Decoder & decoder) { insertNumber(column, target, decoder.decodeBool()); + return true; }; } break; @@ -275,6 +283,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node } } offsets.push_back(offsets.back() + total); + return true; }; } break; @@ -301,24 +310,33 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node { col.insertDefault(); } + return true; }; } - - /// If the Union is ['Null', Nested-Type], since the Nested-Type can not be inside - /// Nullable, so we will get Nested-Type, instead of Nullable type. - if (null_as_default || !target.isNullable()) + else if (null_as_default) { auto nested_deserialize = this->createDeserializeFn(root_node->leafAt(non_null_union_index), target_type); return [non_null_union_index, nested_deserialize](IColumn & column, avro::Decoder & decoder) { int union_index = static_cast(decoder.decodeUnionIndex()); if (union_index == non_null_union_index) + { nested_deserialize(column, decoder); - else - column.insertDefault(); + return true; + } + column.insertDefault(); + return false; }; } - + else + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot insert Avro Union(Null, {}) into non-nullable type {}. To use default value on NULL, enable setting " + "input_format_null_as_default", + avro::toString(root_node->leafAt(non_null_union_index)->type()), + target_type->getName()); + } } break; } @@ -331,6 +349,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node return [](IColumn &, avro::Decoder & decoder) { decoder.decodeNull(); + return true; }; } else @@ -340,10 +359,26 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node ColumnNullable & col = assert_cast(column); decoder.decodeNull(); col.insertDefault(); + return true; }; } } - break; + else if (null_as_default) + { + return [](IColumn & column, avro::Decoder & decoder) + { + decoder.decodeNull(); + column.insertDefault(); + return false; + }; + } + else + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot insert Avro Null into non-nullable type {}. To use default value on NULL, enable setting " + "input_format_null_as_default", target_type->getName()); + } case avro::AVRO_ENUM: if (target.isString()) { @@ -358,6 +393,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node size_t enum_index = decoder.decodeEnum(); const auto & enum_symbol = symbols[enum_index]; column.insertData(enum_symbol.c_str(), enum_symbol.length()); + return true; }; } if (target.isEnum()) @@ -372,6 +408,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node { size_t enum_index = decoder.decodeEnum(); column.insert(symbol_mapping[enum_index]); + return true; }; } break; @@ -384,6 +421,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node { decoder.decodeFixed(tmp_fixed.size(), tmp_fixed); column.insertData(reinterpret_cast(tmp_fixed.data()), tmp_fixed.size()); + return true; }; } break; @@ -415,6 +453,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node auto nested_columns = column_tuple.getColumns(); for (const auto & [nested_deserializer, pos] : nested_deserializers) nested_deserializer(*nested_columns[pos], decoder); + return true; }; } break; @@ -449,6 +488,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node } } offsets.push_back(offsets.back() + total); + return true; }; } break; @@ -465,6 +505,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node ColumnNullable & col = assert_cast(column); nested_deserialize(col.getNestedColumn(), decoder); col.getNullMapData().push_back(0); + return true; }; } @@ -593,7 +634,6 @@ void AvroDeserializer::Action::deserializeNested(MutableColumns & columns, avro: ColumnArray & column_array = assert_cast(*columns[index]); arrays_offsets.push_back(&column_array.getOffsets()); nested_columns.push_back(&column_array.getData()); - ext.read_columns[index] = true; } size_t total = 0; @@ -603,7 +643,7 @@ void AvroDeserializer::Action::deserializeNested(MutableColumns & columns, avro: for (size_t i = 0; i < n; ++i) { for (size_t j = 0; j != nested_deserializers.size(); ++j) - nested_deserializers[j](*nested_columns[j], decoder); + ext.read_columns[nested_column_indexes[j]] = nested_deserializers[j](*nested_columns[j], decoder); } } @@ -742,7 +782,8 @@ void AvroDeserializer::deserializeRow(MutableColumns & columns, avro::Decoder & row_action.execute(columns, decoder, ext); for (size_t i = 0; i < ext.read_columns.size(); ++i) { - if (!ext.read_columns[i]) + /// Insert default in missing columns. + if (!column_found[i]) { columns[i]->insertDefault(); } @@ -759,7 +800,7 @@ void AvroRowInputFormat::readPrefix() { file_reader_ptr = std::make_unique(std::make_unique(*in)); deserializer_ptr = std::make_unique( - output.getHeader(), file_reader_ptr->dataSchema(), format_settings.avro.allow_missing_fields, format_settings.avro.null_as_default); + output.getHeader(), file_reader_ptr->dataSchema(), format_settings.avro.allow_missing_fields, format_settings.null_as_default); file_reader_ptr->init(); } @@ -950,7 +991,7 @@ const AvroDeserializer & AvroConfluentRowInputFormat::getOrCreateDeserializer(Sc { auto schema = schema_registry->getSchema(schema_id); AvroDeserializer deserializer( - output.getHeader(), schema, format_settings.avro.allow_missing_fields, format_settings.avro.null_as_default); + output.getHeader(), schema, format_settings.avro.allow_missing_fields, format_settings.null_as_default); it = deserializer_cache.emplace(schema_id, deserializer).first; } return it->second; diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.h b/src/Processors/Formats/Impl/AvroRowInputFormat.h index 96370b8c4c7..dcd51398032 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.h +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.h @@ -35,8 +35,8 @@ public: void deserializeRow(MutableColumns & columns, avro::Decoder & decoder, RowReadExtension & ext) const; private: - using DeserializeFn = std::function; - using DeserializeNestedFn = std::function; + using DeserializeFn = std::function; + using DeserializeNestedFn = std::function; using SkipFn = std::function; DeserializeFn createDeserializeFn(avro::NodePtr root_node, DataTypePtr target_type); @@ -86,8 +86,7 @@ private: case Noop: break; case Deserialize: - deserialize_fn(*columns[target_column_idx], decoder); - ext.read_columns[target_column_idx] = true; + ext.read_columns[target_column_idx] = deserialize_fn(*columns[target_column_idx], decoder); break; case Skip: skip_fn(decoder); diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp index f337eedbb05..488f4ff9a73 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp @@ -45,11 +45,11 @@ namespace ErrorCodes extern const int UNEXPECTED_END_OF_FILE; } -MsgPackRowInputFormat::MsgPackRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_) - : MsgPackRowInputFormat(header_, std::make_unique(in_), params_) {} +MsgPackRowInputFormat::MsgPackRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & settings) + : MsgPackRowInputFormat(header_, std::make_unique(in_), params_, settings) {} -MsgPackRowInputFormat::MsgPackRowInputFormat(const Block & header_, std::unique_ptr buf_, Params params_) - : IRowInputFormat(header_, *buf_, std::move(params_)), buf(std::move(buf_)), parser(visitor), data_types(header_.getDataTypes()) {} +MsgPackRowInputFormat::MsgPackRowInputFormat(const Block & header_, std::unique_ptr buf_, Params params_, const FormatSettings & settings) + : IRowInputFormat(header_, *buf_, std::move(params_)), buf(std::move(buf_)), visitor(settings.null_as_default), parser(visitor), data_types(header_.getDataTypes()) {} void MsgPackRowInputFormat::resetParser() { @@ -58,13 +58,13 @@ void MsgPackRowInputFormat::resetParser() visitor.reset(); } -void MsgPackVisitor::set_info(IColumn & column, DataTypePtr type) // NOLINT +void MsgPackVisitor::set_info(IColumn & column, DataTypePtr type, UInt8 & read) // NOLINT { while (!info_stack.empty()) { info_stack.pop(); } - info_stack.push(Info{column, type}); + info_stack.push(Info{column, type, &read}); } void MsgPackVisitor::reset() @@ -228,11 +228,11 @@ static void insertFloat64(IColumn & column, DataTypePtr type, Float64 value) // assert_cast(column).insertValue(value); } -static void insertNull(IColumn & column, DataTypePtr type) +static void insertNull(IColumn & column, DataTypePtr type, UInt8 * read, bool null_as_default) { auto insert_func = [&](IColumn & column_, DataTypePtr type_) { - insertNull(column_, type_); + insertNull(column_, type_, read, null_as_default); }; /// LowCardinality(Nullable(...)) @@ -240,7 +240,16 @@ static void insertNull(IColumn & column, DataTypePtr type) return; if (!type->isNullable()) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack null into non-nullable column with type {}.", type->getName()); + { + if (!null_as_default) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack null into non-nullable column with type {}.", type->getName()); + column.insertDefault(); + /// In case of default on null column can have defined DEFAULT expression that should be used. + if (read) + *read = false; + return; + } assert_cast(column).insertDefault(); } @@ -316,7 +325,7 @@ bool MsgPackVisitor::start_array(size_t size) // NOLINT ColumnArray::Offsets & offsets = column_array.getOffsets(); IColumn & nested_column = column_array.getData(); offsets.push_back(offsets.back() + size); - info_stack.push(Info{nested_column, nested_type}); + info_stack.push(Info{nested_column, nested_type, nullptr}); return true; } @@ -340,7 +349,7 @@ bool MsgPackVisitor::start_map_key() // NOLINT { auto key_column = assert_cast(info_stack.top().column).getNestedData().getColumns()[0]; auto key_type = assert_cast(*info_stack.top().type).getKeyType(); - info_stack.push(Info{*key_column, key_type}); + info_stack.push(Info{*key_column, key_type, nullptr}); return true; } @@ -354,7 +363,7 @@ bool MsgPackVisitor::start_map_value() // NOLINT { auto value_column = assert_cast(info_stack.top().column).getNestedData().getColumns()[1]; auto value_type = assert_cast(*info_stack.top().type).getValueType(); - info_stack.push(Info{*value_column, value_type}); + info_stack.push(Info{*value_column, value_type, nullptr}); return true; } @@ -366,7 +375,7 @@ bool MsgPackVisitor::end_map_value() // NOLINT bool MsgPackVisitor::visit_nil() { - insertNull(info_stack.top().column, info_stack.top().type); + insertNull(info_stack.top().column, info_stack.top().type, info_stack.top().read, null_as_default); return true; } @@ -407,13 +416,14 @@ bool MsgPackRowInputFormat::readObject() return true; } -bool MsgPackRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) +bool MsgPackRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext) { size_t column_index = 0; bool has_more_data = true; + ext.read_columns.resize(columns.size(), true); for (; column_index != columns.size(); ++column_index) { - visitor.set_info(*columns[column_index], data_types[column_index]); + visitor.set_info(*columns[column_index], data_types[column_index], ext.read_columns[column_index]); has_more_data = readObject(); if (!has_more_data) break; @@ -547,9 +557,9 @@ void registerInputFormatMsgPack(FormatFactory & factory) ReadBuffer & buf, const Block & sample, const RowInputFormatParams & params, - const FormatSettings &) + const FormatSettings & settings) { - return std::make_shared(sample, buf, params); + return std::make_shared(sample, buf, params, settings); }); factory.registerFileExtension("messagepack", "MsgPack"); } diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.h b/src/Processors/Formats/Impl/MsgPackRowInputFormat.h index 64bb8b569e0..5eaa3719d0c 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.h +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.h @@ -19,10 +19,13 @@ class ReadBuffer; class MsgPackVisitor : public msgpack::null_visitor { public: + MsgPackVisitor(bool null_as_default_) : null_as_default(null_as_default_) {} + struct Info { IColumn & column; DataTypePtr type; + UInt8 * read; }; /// These functions are called when parser meets corresponding object in parsed data @@ -47,25 +50,26 @@ public: [[noreturn]] void parse_error(size_t parsed_offset, size_t error_offset); /// Update info_stack - void set_info(IColumn & column, DataTypePtr type); + void set_info(IColumn & column, DataTypePtr type, UInt8 & read); void reset(); private: /// Stack is needed to process arrays and maps std::stack info_stack; + bool null_as_default; }; class MsgPackRowInputFormat : public IRowInputFormat { public: - MsgPackRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_); + MsgPackRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & settings); String getName() const override { return "MagPackRowInputFormat"; } void resetParser() override; void setReadBuffer(ReadBuffer & in_) override; private: - MsgPackRowInputFormat(const Block & header_, std::unique_ptr buf_, Params params_); + MsgPackRowInputFormat(const Block & header_, std::unique_ptr buf_, Params params_, const FormatSettings & settings); bool readRow(MutableColumns & columns, RowReadExtension & ext) override; diff --git a/src/Processors/Formats/Impl/NativeFormat.cpp b/src/Processors/Formats/Impl/NativeFormat.cpp index 959b86ec051..3c1a2bd5965 100644 --- a/src/Processors/Formats/Impl/NativeFormat.cpp +++ b/src/Processors/Formats/Impl/NativeFormat.cpp @@ -17,7 +17,13 @@ class NativeInputFormat final : public IInputFormat public: NativeInputFormat(ReadBuffer & buf, const Block & header_, const FormatSettings & settings) : IInputFormat(header_, buf) - , reader(std::make_unique(buf, header_, 0, settings.skip_unknown_fields)) + , reader(std::make_unique( + buf, + header_, + 0, + settings.skip_unknown_fields, + settings.null_as_default, + settings.defaults_for_omitted_fields ? &block_missing_values : nullptr)) , header(header_) {} String getName() const override { return "Native"; } @@ -30,6 +36,7 @@ public: Chunk generate() override { + block_missing_values.clear(); auto block = reader->read(); if (!block) return {}; @@ -47,9 +54,12 @@ public: IInputFormat::setReadBuffer(in_); } + const BlockMissingValues & getMissingValues() const override { return block_missing_values; } + private: std::unique_ptr reader; Block header; + BlockMissingValues block_missing_values; }; class NativeOutputFormat final : public IOutputFormat diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 2e45d817506..03f056e22b3 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -67,12 +67,10 @@ Chunk ORCBlockInputFormat::generate() ++stripe_current; Chunk res; - arrow_column_to_ch_column->arrowTableToCHChunk(res, table, num_rows); /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. /// Otherwise fill the missing columns with zero values of its type. - if (format_settings.defaults_for_omitted_fields) - for (const auto & column_idx : missing_columns) - block_missing_values.setBits(column_idx, res.getNumRows()); + BlockMissingValues * block_missing_values_ptr = format_settings.defaults_for_omitted_fields ? &block_missing_values : nullptr; + arrow_column_to_ch_column->arrowTableToCHChunk(res, table, num_rows, block_missing_values_ptr); return res; } @@ -128,8 +126,8 @@ void ORCBlockInputFormat::prepareReader() "ORC", format_settings.orc.import_nested, format_settings.orc.allow_missing_columns, + format_settings.null_as_default, format_settings.orc.case_insensitive_column_matching); - missing_columns = arrow_column_to_ch_column->getMissingColumns(*schema); ArrowFieldIndexUtil field_util( format_settings.orc.case_insensitive_column_matching, diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.h b/src/Processors/Formats/Impl/ORCBlockInputFormat.h index bc2abe41cc1..3d8bc781278 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.h @@ -49,7 +49,6 @@ private: // indices of columns to read from ORC file std::vector include_indices; - std::vector missing_columns; BlockMissingValues block_missing_values; const FormatSettings format_settings; diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index d2ec3c02eed..fca097d8ea7 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -71,7 +71,10 @@ Chunk ParquetBlockInputFormat::generate() if (*batch) { auto tmp_table = arrow::Table::FromRecordBatches({*batch}); - arrow_column_to_ch_column->arrowTableToCHChunk(res, *tmp_table, (*tmp_table)->num_rows()); + /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. + /// Otherwise fill the missing columns with zero values of its type. + BlockMissingValues * block_missing_values_ptr = format_settings.defaults_for_omitted_fields ? &block_missing_values : nullptr; + arrow_column_to_ch_column->arrowTableToCHChunk(res, *tmp_table, (*tmp_table)->num_rows(), block_missing_values_ptr); } else { @@ -80,12 +83,6 @@ Chunk ParquetBlockInputFormat::generate() return {}; } - /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. - /// Otherwise fill the missing columns with zero values of its type. - if (format_settings.defaults_for_omitted_fields) - for (const auto & column_idx : missing_columns) - block_missing_values.setBits(column_idx, res.getNumRows()); - return res; } @@ -133,8 +130,8 @@ void ParquetBlockInputFormat::prepareReader() "Parquet", format_settings.parquet.import_nested, format_settings.parquet.allow_missing_columns, + format_settings.null_as_default, format_settings.parquet.case_insensitive_column_matching); - missing_columns = arrow_column_to_ch_column->getMissingColumns(*schema); ArrowFieldIndexUtil field_util( format_settings.parquet.case_insensitive_column_matching, diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index 37878a94dd9..afc46939c79 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -42,7 +42,6 @@ private: // indices of columns to read from Parquet file std::vector column_indices; std::unique_ptr arrow_column_to_ch_column; - std::vector missing_columns; BlockMissingValues block_missing_values; const FormatSettings format_settings; const std::unordered_set & skip_row_groups; diff --git a/tests/queries/0_stateless/02561_null_as_default_more_formats.reference b/tests/queries/0_stateless/02561_null_as_default_more_formats.reference new file mode 100644 index 00000000000..f5d4f41efe8 --- /dev/null +++ b/tests/queries/0_stateless/02561_null_as_default_more_formats.reference @@ -0,0 +1,36 @@ +Parquet +1 +0 0 0 +0 0 +0 0 0 +42 0 42 +Arrow +1 +0 0 0 +0 0 +0 0 0 +42 0 42 +ORC +1 +0 0 0 +0 0 +0 0 0 +42 0 42 +Avro +1 +0 0 0 +0 0 +0 0 0 +42 0 42 +MsgPack +1 +0 0 0 +0 0 +0 0 0 +42 0 42 +Native +1 +0 0 0 +0 0 +0 0 0 +42 0 42 diff --git a/tests/queries/0_stateless/02561_null_as_default_more_formats.sh b/tests/queries/0_stateless/02561_null_as_default_more_formats.sh new file mode 100755 index 00000000000..eacd8e964a6 --- /dev/null +++ b/tests/queries/0_stateless/02561_null_as_default_more_formats.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "drop table if exists test" +$CLICKHOUSE_CLIENT -q "create table test (x UInt64 default 42, y UInt64, z LowCardinality(String) default '42') engine=Memory"; +for format in Parquet Arrow ORC Avro MsgPack Native +do + echo $format + $CLICKHOUSE_CLIENT -q "select number % 2 ? NULL : number as x, x as y, CAST(number % 2 ? NULL : toString(number), 'LowCardinality(Nullable(String))') as z from numbers(2) format $format" | $CLICKHOUSE_CLIENT -q "insert into test settings input_format_null_as_default=0 format $format" 2>&1 | grep "Exception" -c + $CLICKHOUSE_CLIENT -q "select number % 2 ? NULL : number as x, x as y, CAST(number % 2 ? NULL : toString(number), 'LowCardinality(Nullable(String))') as z from numbers(2) format $format settings output_format_arrow_low_cardinality_as_dictionary=1" | $CLICKHOUSE_CLIENT -q "insert into test settings input_format_null_as_default=1, input_format_defaults_for_omitted_fields=0 format $format" + $CLICKHOUSE_CLIENT -q "select * from test" + $CLICKHOUSE_CLIENT -q "truncate table test" + $CLICKHOUSE_CLIENT -q "select number % 2 ? NULL : number as x, x as y, CAST(number % 2 ? NULL : toString(number), 'LowCardinality(Nullable(String))') as z from numbers(2) format $format settings output_format_arrow_low_cardinality_as_dictionary=1" | $CLICKHOUSE_CLIENT -q "insert into test settings input_format_null_as_default=1, input_format_defaults_for_omitted_fields=1 format $format" + $CLICKHOUSE_CLIENT -q "select * from test" + $CLICKHOUSE_CLIENT -q "truncate table test" +done + diff --git a/tests/queries/0_stateless/02562_native_null_on_missing_columns.reference b/tests/queries/0_stateless/02562_native_null_on_missing_columns.reference new file mode 100644 index 00000000000..e072efc3352 --- /dev/null +++ b/tests/queries/0_stateless/02562_native_null_on_missing_columns.reference @@ -0,0 +1,4 @@ +0 0 +1 0 +0 42 +1 42 diff --git a/tests/queries/0_stateless/02562_native_null_on_missing_columns.sh b/tests/queries/0_stateless/02562_native_null_on_missing_columns.sh new file mode 100755 index 00000000000..c3d174d77e8 --- /dev/null +++ b/tests/queries/0_stateless/02562_native_null_on_missing_columns.sh @@ -0,0 +1,16 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "drop table if exists test" +$CLICKHOUSE_CLIENT -q "create table test (x UInt64, y UInt64 default 42) engine=Memory" + +$CLICKHOUSE_CLIENT -q "select number as x from numbers(2) format Native" | $CLICKHOUSE_CLIENT -q "insert into test settings input_format_defaults_for_omitted_fields=0 format Native" +$CLICKHOUSE_CLIENT -q "select * from test" +$CLICKHOUSE_CLIENT -q "truncate table test" + +$CLICKHOUSE_CLIENT -q "select number as x from numbers(2) format Native" | $CLICKHOUSE_CLIENT -q "insert into test settings input_format_defaults_for_omitted_fields=1 format Native" +$CLICKHOUSE_CLIENT -q "select * from test" +$CLICKHOUSE_CLIENT -q "truncate table test" From 04cf144edc9abba1f9e61ff4f4a949140f8f51e7 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 10 Feb 2023 17:20:51 +0000 Subject: [PATCH 043/253] Fix TSKV, update docs --- .../operations/settings/settings-formats.md | 11 +++++--- .../Formats/Impl/TSKVRowInputFormat.cpp | 5 +++- .../02562_native_null_on_missing_columns.sh | 16 ------------ ...tskv_default_for_omitted_fields.reference} | 4 +-- ..._native_tskv_default_for_omitted_fields.sh | 25 +++++++++++++++++++ 5 files changed, 38 insertions(+), 23 deletions(-) delete mode 100755 tests/queries/0_stateless/02562_native_null_on_missing_columns.sh rename tests/queries/0_stateless/{02562_native_null_on_missing_columns.reference => 02562_native_tskv_default_for_omitted_fields.reference} (50%) create mode 100755 tests/queries/0_stateless/02562_native_tskv_default_for_omitted_fields.sh diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index fd727704710..dda97696de5 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -15,11 +15,12 @@ When writing data, ClickHouse throws an exception if input data contain columns Supported formats: -- [JSONEachRow](../../interfaces/formats.md/#jsoneachrow) +- [JSONEachRow](../../interfaces/formats.md/#jsoneachrow) (and other JSON formats) +- [BSONEachRow](../../interfaces/formats.md/#bsoneachrow) (and other JSON formats) - [TSKV](../../interfaces/formats.md/#tskv) - All formats with suffixes WithNames/WithNamesAndTypes -- [JSONColumns](../../interfaces/formats.md/#jsoncolumns) - [MySQLDump](../../interfaces/formats.md/#mysqldump) +- [Native](../../interfaces/formats.md/#native) Possible values: @@ -78,7 +79,7 @@ Default value: 1. ## input_format_defaults_for_omitted_fields {#input_format_defaults_for_omitted_fields} -When performing `INSERT` queries, replace omitted input column values with default values of the respective columns. This option only applies to [JSONEachRow](../../interfaces/formats.md/#jsoneachrow), [CSV](../../interfaces/formats.md/#csv), [TabSeparated](../../interfaces/formats.md/#tabseparated) formats and formats with `WithNames`/`WithNamesAndTypes` suffixes. +When performing `INSERT` queries, replace omitted input column values with default values of the respective columns. This option applies to [JSONEachRow](../../interfaces/formats.md/#jsoneachrow) (and other JSON formats), [CSV](../../interfaces/formats.md/#csv), [TabSeparated](../../interfaces/formats.md/#tabseparated), [TSKV](../../interfaces/formats.md/#tskv), [Parquet](../../interfaces/formats.md/#parquet), [Arrow](../../interfaces/formats.md/#arrow), [Avro](../../interfaces/formats.md/#avro), [ORC](../../interfaces/formats.md/#orc), [Native](../../interfaces/formats.md/#native) formats and formats with `WithNames`/`WithNamesAndTypes` suffixes. :::note When this option is enabled, extended table metadata are sent from server to client. It consumes additional computing resources on the server and can reduce performance. @@ -96,7 +97,9 @@ Default value: 1. Enables or disables the initialization of [NULL](../../sql-reference/syntax.md/#null-literal) fields with [default values](../../sql-reference/statements/create/table.md/#create-default-values), if data type of these fields is not [nullable](../../sql-reference/data-types/nullable.md/#data_type-nullable). If column type is not nullable and this setting is disabled, then inserting `NULL` causes an exception. If column type is nullable, then `NULL` values are inserted as is, regardless of this setting. -This setting is applicable to [INSERT ... VALUES](../../sql-reference/statements/insert-into.md) queries for text input formats. +This setting is applicable for most input formats. + +For complex default expressions `input_format_defaults_for_omitted_fields` must be enabled too. Possible values: diff --git a/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp b/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp index bf6d0ab88d2..23a8589bd0a 100644 --- a/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp @@ -193,7 +193,10 @@ bool TSKVRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ex header.getByPosition(i).type->insertDefaultInto(*columns[i]); /// return info about defaults set - ext.read_columns = read_columns; + if (format_settings.defaults_for_omitted_fields) + ext.read_columns = read_columns; + else + ext.read_columns.assign(num_columns, true); return true; } diff --git a/tests/queries/0_stateless/02562_native_null_on_missing_columns.sh b/tests/queries/0_stateless/02562_native_null_on_missing_columns.sh deleted file mode 100755 index c3d174d77e8..00000000000 --- a/tests/queries/0_stateless/02562_native_null_on_missing_columns.sh +++ /dev/null @@ -1,16 +0,0 @@ -#!/usr/bin/env bash - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - -$CLICKHOUSE_CLIENT -q "drop table if exists test" -$CLICKHOUSE_CLIENT -q "create table test (x UInt64, y UInt64 default 42) engine=Memory" - -$CLICKHOUSE_CLIENT -q "select number as x from numbers(2) format Native" | $CLICKHOUSE_CLIENT -q "insert into test settings input_format_defaults_for_omitted_fields=0 format Native" -$CLICKHOUSE_CLIENT -q "select * from test" -$CLICKHOUSE_CLIENT -q "truncate table test" - -$CLICKHOUSE_CLIENT -q "select number as x from numbers(2) format Native" | $CLICKHOUSE_CLIENT -q "insert into test settings input_format_defaults_for_omitted_fields=1 format Native" -$CLICKHOUSE_CLIENT -q "select * from test" -$CLICKHOUSE_CLIENT -q "truncate table test" diff --git a/tests/queries/0_stateless/02562_native_null_on_missing_columns.reference b/tests/queries/0_stateless/02562_native_tskv_default_for_omitted_fields.reference similarity index 50% rename from tests/queries/0_stateless/02562_native_null_on_missing_columns.reference rename to tests/queries/0_stateless/02562_native_tskv_default_for_omitted_fields.reference index e072efc3352..17197fa3563 100644 --- a/tests/queries/0_stateless/02562_native_null_on_missing_columns.reference +++ b/tests/queries/0_stateless/02562_native_tskv_default_for_omitted_fields.reference @@ -1,4 +1,4 @@ -0 0 1 0 -0 42 +1 42 +1 0 1 42 diff --git a/tests/queries/0_stateless/02562_native_tskv_default_for_omitted_fields.sh b/tests/queries/0_stateless/02562_native_tskv_default_for_omitted_fields.sh new file mode 100755 index 00000000000..a08c948705d --- /dev/null +++ b/tests/queries/0_stateless/02562_native_tskv_default_for_omitted_fields.sh @@ -0,0 +1,25 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "drop table if exists test" +$CLICKHOUSE_CLIENT -q "insert into function file(02562_data.native) select 1::UInt64 as x settings engine_file_truncate_on_insert=1" +$CLICKHOUSE_CLIENT -q "create table test (x UInt64, y UInt64 default 42) engine=File(Native, '02562_data.native') settings input_format_defaults_for_omitted_fields=0" +$CLICKHOUSE_CLIENT -q "select * from test" +$CLICKHOUSE_CLIENT -q "drop table test" + +$CLICKHOUSE_CLIENT -q "create table test (x UInt64, y UInt64 default 42) engine=File(Native, '02562_data.native') settings input_format_defaults_for_omitted_fields=1" +$CLICKHOUSE_CLIENT -q "select * from test" +$CLICKHOUSE_CLIENT -q "drop table test" + +$CLICKHOUSE_CLIENT -q "insert into function file(02562_data.tskv) select 1::UInt64 as x settings engine_file_truncate_on_insert=1" +$CLICKHOUSE_CLIENT -q "create table test (x UInt64, y UInt64 default 42) engine=File(TSKV, '02562_data.tskv') settings input_format_defaults_for_omitted_fields=0" +$CLICKHOUSE_CLIENT -q "select * from test" +$CLICKHOUSE_CLIENT -q "drop table test" + +$CLICKHOUSE_CLIENT -q "create table test (x UInt64, y UInt64 default 42) engine=File(TSKV, '02562_data.tskv') settings input_format_defaults_for_omitted_fields=1" +$CLICKHOUSE_CLIENT -q "select * from test" +$CLICKHOUSE_CLIENT -q "drop table test" + From 3ae526d4ba9f1d78ce735f3156b7f48e704a082f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 12 Feb 2023 04:57:53 +0100 Subject: [PATCH 044/253] Enable retries for INSERT by default in case of ZooKeeper session loss --- 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 2e0a4f00c64..fd9eba94ec2 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -697,7 +697,7 @@ class IColumn; M(UInt64, grace_hash_join_max_buckets, 1024, "Limit on the number of grace hash join buckets", 0) \ M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \ M(Bool, optimize_sorting_by_input_stream_properties, true, "Optimize sorting by sorting properties of input stream", 0) \ - M(UInt64, insert_keeper_max_retries, 0, "Max retries for keeper operations during insert", 0) \ + M(UInt64, insert_keeper_max_retries, 20, "Max retries for keeper operations during insert", 0) \ M(UInt64, insert_keeper_retry_initial_backoff_ms, 100, "Initial backoff timeout for keeper operations during insert", 0) \ M(UInt64, insert_keeper_retry_max_backoff_ms, 10000, "Max backoff timeout for keeper operations during insert", 0) \ M(Float, insert_keeper_fault_injection_probability, 0.0f, "Approximate probability of failure for a keeper request during insert. Valid value is in interval [0.0f, 1.0f]", 0) \ From 45ef2f6d60af170c070fde363bdfc9d0997bff48 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Tue, 7 Feb 2023 20:45:40 +0100 Subject: [PATCH 045/253] Split prewhere actions into separate conjuctive steps --- src/Interpreters/ActionsDAG.cpp | 23 +++++---- src/Interpreters/ActionsDAG.h | 14 ++++++ .../MergeTreeBaseSelectProcessor.cpp | 49 +++++++++++++++++++ .../MergeTree/MergeTreeBaseSelectProcessor.h | 2 + .../MergeTree/MergeTreeBlockReadUtils.cpp | 40 ++++++++++++++- .../MergeTree/MergeTreeWhereOptimizer.cpp | 4 +- 6 files changed, 118 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 5f1398fed39..913a891c6bb 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1017,6 +1017,9 @@ std::string ActionsDAG::dumpDAG() const out << ' ' << map[node]; out << '\n'; + out << "Project input: " << project_input << '\n'; + out << "Projected output: " << projected_output << '\n'; + return out.str(); } @@ -1660,20 +1663,20 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsForFilter(const std::string & co return res; } -namespace -{ - -struct ConjunctionNodes -{ - ActionsDAG::NodeRawConstPtrs allowed; - ActionsDAG::NodeRawConstPtrs rejected; -}; +//namespace +//{ +// +//struct ConjunctionNodes +//{ +// ActionsDAG::NodeRawConstPtrs allowed; +// ActionsDAG::NodeRawConstPtrs rejected; +//}; /// Take a node which result is predicate. /// Assuming predicate is a conjunction (probably, trivial). /// Find separate conjunctions nodes. Split nodes into allowed and rejected sets. /// Allowed predicate is a predicate which can be calculated using only nodes from allowed_nodes set. -ConjunctionNodes getConjunctionNodes(ActionsDAG::Node * predicate, std::unordered_set allowed_nodes) +ConjunctionNodes getConjunctionNodes(const ActionsDAG::Node * predicate, std::unordered_set allowed_nodes) { ConjunctionNodes conjunction; std::unordered_set allowed; @@ -1795,7 +1798,7 @@ ColumnsWithTypeAndName prepareFunctionArguments(const ActionsDAG::NodeRawConstPt return arguments; } -} +//} /// Create actions which calculate conjunction of selected nodes. /// Assume conjunction nodes are predicates (and may be used as arguments of function AND). diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 40bc76fe057..98c5d36c69a 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -363,6 +363,7 @@ private: void compileFunctions(size_t min_count_to_compile_expression, const std::unordered_set & lazy_executed_nodes = {}); #endif +public: static ActionsDAGPtr cloneActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs); }; @@ -372,4 +373,17 @@ struct ActionDAGNodes ActionsDAG::NodeRawConstPtrs nodes; }; +struct ConjunctionNodes +{ + ActionsDAG::NodeRawConstPtrs allowed; + ActionsDAG::NodeRawConstPtrs rejected; +}; + +/// Take a node which result is predicate. +/// Assuming predicate is a conjunction (probably, trivial). +/// Find separate conjunctions nodes. Split nodes into allowed and rejected sets. +/// Allowed predicate is a predicate which can be calculated using only nodes from allowed_nodes set. +ConjunctionNodes getConjunctionNodes(const ActionsDAG::Node * predicate, std::unordered_set allowed_nodes); + + } diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 49458be4232..072bf5c2cba 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -7,6 +7,7 @@ #include #include #include +#include "Core/Names.h" #include #include #include @@ -102,6 +103,53 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( prewhere_actions->steps.emplace_back(std::move(row_level_filter_step)); } +#if 1 + auto conjunctions = getConjunctionNodes( + prewhere_info->prewhere_actions->tryFindInOutputs(prewhere_info->prewhere_column_name), + {}); + + auto original_outputs = prewhere_info->prewhere_actions->getOutputs(); + NameSet original_output_names; + for (const auto & output : original_outputs) + original_output_names.insert(output->result_name); + + auto inputs = prewhere_info->prewhere_actions->getInputs(); + ColumnsWithTypeAndName all_inputs; + for (const auto & input : inputs) + all_inputs.emplace_back(input->column, input->result_type, input->result_name); + + ActionsDAG::NodeRawConstPtrs all_conjunctions = std::move(conjunctions.allowed); + all_conjunctions.insert(all_conjunctions.end(), conjunctions.rejected.begin(), conjunctions.rejected.end()); + + for (const auto & conjunction : all_conjunctions) + { + auto step_dag = ActionsDAG::cloneActionsForConjunction({conjunction}, all_inputs); + + /// Return the condition columns + Names step_outputs{conjunction->result_name}; + /// Preserve all the original outputs computed at this step + for (const auto & output : original_output_names) + if (step_dag->tryRestoreColumn(output)) + step_outputs.emplace_back(output); + step_dag->removeUnusedActions(step_outputs, true, true); + + //std::cerr << conjunction->result_name << "\n"; + std::cerr << step_dag->dumpDAG() << "\n"; + + PrewhereExprStep prewhere_step + { + .actions = std::make_shared(step_dag, actions_settings), + .column_name = conjunction->result_name, + .remove_column = false, // TODO: properly set this depending on whether the column is used in the next step + .need_filter = false + }; + prewhere_actions->steps.emplace_back(std::move(prewhere_step)); + } + + //prewhere_actions->steps.back().remove_column = prewhere_info->remove_prewhere_column; + prewhere_actions->steps.back().need_filter = prewhere_info->need_filter; +#else + PrewhereExprStep prewhere_step { .actions = std::make_shared(prewhere_info->prewhere_actions, actions_settings), @@ -111,6 +159,7 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( }; prewhere_actions->steps.emplace_back(std::move(prewhere_step)); +#endif } return prewhere_actions; diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index c6680676ce9..0dc6f2f9f9b 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -101,8 +101,10 @@ protected: static void injectVirtualColumns(Block & block, size_t row_count, MergeTreeReadTask * task, const DataTypePtr & partition_value_type, const Names & virtual_columns); +public: static std::unique_ptr getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings); +protected: static void initializeRangeReadersImpl( MergeTreeRangeReader & range_reader, std::deque & pre_range_readers, diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 1f69fcae8dc..d71e7b276ab 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -5,6 +5,7 @@ #include #include #include +#include "Storages/MergeTree/MergeTreeBaseSelectProcessor.h" #include #include #include @@ -291,7 +292,6 @@ MergeTreeReadTaskColumns getReadTaskColumns( bool with_subcolumns) { Names column_names = required_columns; - Names pre_column_names; /// Read system columns such as lightweight delete mask "_row_exists" if it is persisted in the part for (const auto & name : system_columns) @@ -313,6 +313,40 @@ MergeTreeReadTaskColumns getReadTaskColumns( if (prewhere_info) { + auto prewhere_actions = IMergeTreeSelectAlgorithm::getPrewhereActions(prewhere_info, {}); + + NameSet pre_name_set; + + for (const auto & step : prewhere_actions->steps) + { + Names step_column_names = step.actions->getActionsDAG().getRequiredColumnsNames(); + + injectRequiredColumns( + data_part_info_for_reader, storage_snapshot, with_subcolumns, step_column_names); + + Names new_step_column_names; + for (const auto & name : step_column_names) + { + if (pre_name_set.contains(name)) + continue; + new_step_column_names.push_back(name); + pre_name_set.insert(name); + } + + result.pre_columns.push_back(storage_snapshot->getColumnsByNames(options, new_step_column_names)); + } + + /// Remove prewhere columns from the list of columns to read + Names post_column_names; + for (const auto & name : column_names) + if (!pre_name_set.contains(name)) + post_column_names.push_back(name); + + column_names = post_column_names; + + + +#if 0 NameSet pre_name_set; /// Add column reading steps: @@ -346,9 +380,11 @@ MergeTreeReadTaskColumns getReadTaskColumns( post_column_names.push_back(name); column_names = post_column_names; +#endif + } - result.pre_columns.push_back(storage_snapshot->getColumnsByNames(options, pre_column_names)); +// result.pre_columns.push_back(storage_snapshot->getColumnsByNames(options, pre_column_names)); /// 3. Rest of the requested columns result.columns = storage_snapshot->getColumnsByNames(options, column_names); diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index b3ff05a960a..ce73dad48f9 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -274,7 +274,7 @@ void MergeTreeWhereOptimizer::optimize(ASTSelectQuery & select) const if (!it->viable) break; - +#if 0 bool moved_enough = false; if (total_size_of_queried_columns > 0) { @@ -292,7 +292,7 @@ void MergeTreeWhereOptimizer::optimize(ASTSelectQuery & select) const if (moved_enough) break; - +#endif move_condition(it); } From 249e670c9fd0b0cc4a0a3b7ddbd58c4c93a09c57 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 8 Feb 2023 00:07:19 +0100 Subject: [PATCH 046/253] Properly preserve columns that are required by next steps --- .../MergeTreeBaseSelectProcessor.cpp | 44 +++++++++++++++---- .../MergeTree/MergeTreeBlockReadUtils.cpp | 2 - 2 files changed, 35 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 072bf5c2cba..43c4666f1b9 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -8,6 +8,7 @@ #include #include #include "Core/Names.h" +#include "Interpreters/ActionsDAG.h" #include #include #include @@ -121,32 +122,57 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( ActionsDAG::NodeRawConstPtrs all_conjunctions = std::move(conjunctions.allowed); all_conjunctions.insert(all_conjunctions.end(), conjunctions.rejected.begin(), conjunctions.rejected.end()); + struct Step + { + ActionsDAGPtr actions; + String column_name; + }; + std::vector steps; + for (const auto & conjunction : all_conjunctions) { auto step_dag = ActionsDAG::cloneActionsForConjunction({conjunction}, all_inputs); + step_dag->removeUnusedActions(Names{conjunction->result_name}, true, true); + steps.emplace_back(Step{step_dag, conjunction->result_name}); + } + + /// "Rename" the last step result to the combined prewhere column name, because in fact it will be AND of all step results + if (steps.back().column_name != prewhere_info->prewhere_column_name) + steps.back().actions->addAlias(steps.back().actions->findInOutputs(steps.back().column_name), prewhere_info->prewhere_column_name); + + prewhere_actions->steps.resize(steps.size()); + + for (ssize_t i = steps.size() - 1; i >= 0; --i) + { + const auto & step = steps[i]; /// Return the condition columns - Names step_outputs{conjunction->result_name}; + Names step_outputs{step.column_name}; + const bool remove_column = !original_output_names.contains(step.column_name); /// Preserve all the original outputs computed at this step for (const auto & output : original_output_names) - if (step_dag->tryRestoreColumn(output)) + if (step.actions->tryRestoreColumn(output)) step_outputs.emplace_back(output); - step_dag->removeUnusedActions(step_outputs, true, true); + step.actions->removeUnusedActions(step_outputs, true, true); + + /// Add current step columns as outputs for previous steps + for (const auto & input :step.actions->getInputs()) + original_output_names.insert(input->result_name); //std::cerr << conjunction->result_name << "\n"; - std::cerr << step_dag->dumpDAG() << "\n"; + //std::cerr << step.actions->dumpDAG() << "\n"; PrewhereExprStep prewhere_step { - .actions = std::make_shared(step_dag, actions_settings), - .column_name = conjunction->result_name, - .remove_column = false, // TODO: properly set this depending on whether the column is used in the next step + .actions = std::make_shared(step.actions, actions_settings), + .column_name = step.column_name, + .remove_column = remove_column, // TODO: properly set this depending on whether the column is used in the next step .need_filter = false }; - prewhere_actions->steps.emplace_back(std::move(prewhere_step)); + prewhere_actions->steps[i] = std::move(prewhere_step); } - //prewhere_actions->steps.back().remove_column = prewhere_info->remove_prewhere_column; + prewhere_actions->steps.back().remove_column = prewhere_info->remove_prewhere_column; prewhere_actions->steps.back().need_filter = prewhere_info->need_filter; #else diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index d71e7b276ab..c58ab06f08f 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -344,8 +344,6 @@ MergeTreeReadTaskColumns getReadTaskColumns( column_names = post_column_names; - - #if 0 NameSet pre_name_set; From 1b207d9fa099f97cf7b1dcfd637af28430f2db35 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 8 Feb 2023 14:29:12 +0100 Subject: [PATCH 047/253] Fix for overwritten row level filter step --- src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 43c4666f1b9..56929809b42 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -140,7 +140,8 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( if (steps.back().column_name != prewhere_info->prewhere_column_name) steps.back().actions->addAlias(steps.back().actions->findInOutputs(steps.back().column_name), prewhere_info->prewhere_column_name); - prewhere_actions->steps.resize(steps.size()); + const size_t steps_before_prewhere = prewhere_actions->steps.size(); + prewhere_actions->steps.resize(steps_before_prewhere + steps.size()); for (ssize_t i = steps.size() - 1; i >= 0; --i) { @@ -169,7 +170,7 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( .remove_column = remove_column, // TODO: properly set this depending on whether the column is used in the next step .need_filter = false }; - prewhere_actions->steps[i] = std::move(prewhere_step); + prewhere_actions->steps[steps_before_prewhere + i] = std::move(prewhere_step); } prewhere_actions->steps.back().remove_column = prewhere_info->remove_prewhere_column; From 31f5dbe2a3e88e7c530c3284264df995a57a0f9c Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 8 Feb 2023 17:07:23 +0100 Subject: [PATCH 048/253] Cast intermediate step result to Bool if needed --- .../MergeTree/MergeTreeBaseSelectProcessor.cpp | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 56929809b42..2aa23243785 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -131,9 +131,18 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( for (const auto & conjunction : all_conjunctions) { + auto result_name = conjunction->result_name; auto step_dag = ActionsDAG::cloneActionsForConjunction({conjunction}, all_inputs); - step_dag->removeUnusedActions(Names{conjunction->result_name}, true, true); - steps.emplace_back(Step{step_dag, conjunction->result_name}); + const auto & result_node = step_dag->findInOutputs(result_name); + /// Cast to UInt8 if needed + if (result_node.result_type->getTypeId() != TypeIndex::UInt8) + { + const auto & cast_node = step_dag->addCast(result_node, std::make_shared()); + step_dag->addOrReplaceInOutputs(cast_node); + result_name = cast_node.result_name; + } + step_dag->removeUnusedActions(Names{result_name}, true, true); + steps.emplace_back(Step{step_dag, result_name}); } /// "Rename" the last step result to the combined prewhere column name, because in fact it will be AND of all step results From 120b112743ed786cb1a89db22342b9c9b59794fa Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 9 Feb 2023 12:56:59 +0100 Subject: [PATCH 049/253] Cleanup --- .../MergeTree/MergeTreeBlockReadUtils.cpp | 73 +++++-------------- 1 file changed, 17 insertions(+), 56 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index c58ab06f08f..17ef9c91e78 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -291,18 +291,18 @@ MergeTreeReadTaskColumns getReadTaskColumns( const PrewhereInfoPtr & prewhere_info, bool with_subcolumns) { - Names column_names = required_columns; + Names column_to_read_after_prewhere = required_columns; /// Read system columns such as lightweight delete mask "_row_exists" if it is persisted in the part for (const auto & name : system_columns) { if (data_part_info_for_reader.getColumns().contains(name)) - column_names.push_back(name); + column_to_read_after_prewhere.push_back(name); } - /// inject columns required for defaults evaluation + /// Inject columns required for defaults evaluation injectRequiredColumns( - data_part_info_for_reader, storage_snapshot, with_subcolumns, column_names); + data_part_info_for_reader, storage_snapshot, with_subcolumns, column_to_read_after_prewhere); MergeTreeReadTaskColumns result; auto options = GetColumnsOptions(GetColumnsOptions::All) @@ -313,9 +313,9 @@ MergeTreeReadTaskColumns getReadTaskColumns( if (prewhere_info) { - auto prewhere_actions = IMergeTreeSelectAlgorithm::getPrewhereActions(prewhere_info, {}); + auto prewhere_actions = IMergeTreeSelectAlgorithm::getPrewhereActions(prewhere_info, {}); // TODO: pass proper actions_settings - NameSet pre_name_set; + NameSet columns_from_previous_steps; for (const auto & step : prewhere_actions->steps) { @@ -324,68 +324,29 @@ MergeTreeReadTaskColumns getReadTaskColumns( injectRequiredColumns( data_part_info_for_reader, storage_snapshot, with_subcolumns, step_column_names); - Names new_step_column_names; + Names columns_to_read_in_step; for (const auto & name : step_column_names) { - if (pre_name_set.contains(name)) + if (columns_from_previous_steps.contains(name)) continue; - new_step_column_names.push_back(name); - pre_name_set.insert(name); + columns_to_read_in_step.push_back(name); + columns_from_previous_steps.insert(name); } - result.pre_columns.push_back(storage_snapshot->getColumnsByNames(options, new_step_column_names)); + result.pre_columns.push_back(storage_snapshot->getColumnsByNames(options, columns_to_read_in_step)); } - /// Remove prewhere columns from the list of columns to read + /// Remove columns read in prewehere from the list of columns to read Names post_column_names; - for (const auto & name : column_names) - if (!pre_name_set.contains(name)) + for (const auto & name : column_to_read_after_prewhere) + if (!columns_from_previous_steps.contains(name)) post_column_names.push_back(name); - column_names = post_column_names; - -#if 0 - NameSet pre_name_set; - - /// Add column reading steps: - /// 1. Columns for row level filter - if (prewhere_info->row_level_filter) - { - Names row_filter_column_names = prewhere_info->row_level_filter->getRequiredColumnsNames(); - injectRequiredColumns( - data_part_info_for_reader, storage_snapshot, with_subcolumns, row_filter_column_names); - result.pre_columns.push_back(storage_snapshot->getColumnsByNames(options, row_filter_column_names)); - pre_name_set.insert(row_filter_column_names.begin(), row_filter_column_names.end()); - } - - /// 2. Columns for prewhere - Names all_pre_column_names = prewhere_info->prewhere_actions->getRequiredColumnsNames(); - - injectRequiredColumns( - data_part_info_for_reader, storage_snapshot, with_subcolumns, all_pre_column_names); - - for (const auto & name : all_pre_column_names) - { - if (pre_name_set.contains(name)) - continue; - pre_column_names.push_back(name); - pre_name_set.insert(name); - } - - Names post_column_names; - for (const auto & name : column_names) - if (!pre_name_set.contains(name)) - post_column_names.push_back(name); - - column_names = post_column_names; -#endif - + column_to_read_after_prewhere = std::move(post_column_names); } -// result.pre_columns.push_back(storage_snapshot->getColumnsByNames(options, pre_column_names)); - - /// 3. Rest of the requested columns - result.columns = storage_snapshot->getColumnsByNames(options, column_names); + /// Rest of the requested columns + result.columns = storage_snapshot->getColumnsByNames(options, column_to_read_after_prewhere); return result; } From 7634b9f56effe0b716382f0af974677cda88b520 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 9 Feb 2023 15:37:06 +0100 Subject: [PATCH 050/253] Added settings to enable the new logic --- src/Core/Settings.h | 2 + .../QueryPlan/ReadFromMergeTree.cpp | 3 + .../MergeTreeBaseSelectProcessor.cpp | 172 +++++++++--------- .../MergeTree/MergeTreeBaseSelectProcessor.h | 2 +- .../MergeTree/MergeTreeBlockReadUtils.cpp | 3 +- .../MergeTree/MergeTreeBlockReadUtils.h | 2 + src/Storages/MergeTree/MergeTreeIOSettings.h | 2 + src/Storages/MergeTree/MergeTreeReadPool.cpp | 4 +- src/Storages/MergeTree/MergeTreeReadPool.h | 6 + .../MergeTree/MergeTreeSelectProcessor.cpp | 2 +- .../MergeTree/MergeTreeWhereOptimizer.cpp | 37 ++-- .../MergeTree/MergeTreeWhereOptimizer.h | 1 + 12 files changed, 132 insertions(+), 104 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2e0a4f00c64..52cc5eb7ad0 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -114,6 +114,8 @@ class IColumn; \ M(Bool, optimize_move_to_prewhere, true, "Allows disabling WHERE to PREWHERE optimization in SELECT queries from MergeTree.", 0) \ M(Bool, optimize_move_to_prewhere_if_final, false, "If query has `FINAL`, the optimization `move_to_prewhere` is not always correct and it is enabled only if both settings `optimize_move_to_prewhere` and `optimize_move_to_prewhere_if_final` are turned on", 0) \ + M(Bool, move_all_conditions_to_prewhere, false, "Move all viable conditions from WHERE to PREWHERE", 0) \ + M(Bool, enable_multiple_prewhere_read_steps, false, "Move more conditions from WHERE to PREWHERE and do reads from disk and filtering in multiple steps if there are multiple conditions combined with AND", 0) \ \ M(UInt64, replication_alter_partitions_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) \ M(Int64, replication_wait_for_inactive_replica_timeout, 120, "Wait for inactive replica to execute ALTER/OPTIMIZE. Time in seconds, 0 - do not wait, negative - wait for unlimited time.", 0) \ diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index ff0c5002e09..6ff230e9d78 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -73,6 +73,7 @@ static MergeTreeReaderSettings getMergeTreeReaderSettings( .read_in_order = query_info.input_order_info != nullptr, .use_asynchronous_read_from_pool = settings.allow_asynchronous_read_from_io_pool_for_merge_tree && (settings.max_streams_to_max_threads_ratio > 1 || settings.max_streams_for_merge_tree_reading > 1), + .enable_multiple_prewhere_read_steps = settings.enable_multiple_prewhere_read_steps, }; } @@ -225,6 +226,7 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas( extension, parts_with_range, prewhere_info, + reader_settings, required_columns, virt_column_names, min_marks_for_concurrent_read @@ -302,6 +304,7 @@ Pipe ReadFromMergeTree::readFromPool( std::move(parts_with_range), storage_snapshot, prewhere_info, + reader_settings, required_columns, virt_column_names, backoff_settings, diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 2aa23243785..49022dce828 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -59,7 +59,7 @@ IMergeTreeSelectAlgorithm::IMergeTreeSelectAlgorithm( : storage(storage_) , storage_snapshot(storage_snapshot_) , prewhere_info(prewhere_info_) - , prewhere_actions(getPrewhereActions(prewhere_info, actions_settings)) + , prewhere_actions(getPrewhereActions(prewhere_info, actions_settings, reader_settings_.enable_multiple_prewhere_read_steps)) , max_block_size_rows(max_block_size_rows_) , preferred_block_size_bytes(preferred_block_size_bytes_) , preferred_max_column_in_block_size_bytes(preferred_max_column_in_block_size_bytes_) @@ -84,7 +84,7 @@ IMergeTreeSelectAlgorithm::IMergeTreeSelectAlgorithm( } -std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings) +std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, bool enable_multiple_prewhere_read_steps) { std::unique_ptr prewhere_actions; if (prewhere_info) @@ -104,98 +104,102 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( prewhere_actions->steps.emplace_back(std::move(row_level_filter_step)); } -#if 1 - auto conjunctions = getConjunctionNodes( - prewhere_info->prewhere_actions->tryFindInOutputs(prewhere_info->prewhere_column_name), - {}); - - auto original_outputs = prewhere_info->prewhere_actions->getOutputs(); - NameSet original_output_names; - for (const auto & output : original_outputs) - original_output_names.insert(output->result_name); - - auto inputs = prewhere_info->prewhere_actions->getInputs(); - ColumnsWithTypeAndName all_inputs; - for (const auto & input : inputs) - all_inputs.emplace_back(input->column, input->result_type, input->result_name); - - ActionsDAG::NodeRawConstPtrs all_conjunctions = std::move(conjunctions.allowed); - all_conjunctions.insert(all_conjunctions.end(), conjunctions.rejected.begin(), conjunctions.rejected.end()); - - struct Step + if (enable_multiple_prewhere_read_steps) { - ActionsDAGPtr actions; - String column_name; - }; - std::vector steps; + auto conjunctions = getConjunctionNodes( + prewhere_info->prewhere_actions->tryFindInOutputs(prewhere_info->prewhere_column_name), + {}); - for (const auto & conjunction : all_conjunctions) - { - auto result_name = conjunction->result_name; - auto step_dag = ActionsDAG::cloneActionsForConjunction({conjunction}, all_inputs); - const auto & result_node = step_dag->findInOutputs(result_name); - /// Cast to UInt8 if needed - if (result_node.result_type->getTypeId() != TypeIndex::UInt8) + auto original_outputs = prewhere_info->prewhere_actions->getOutputs(); + NameSet outputs_required_by_next_steps; + for (const auto & output : original_outputs) + outputs_required_by_next_steps.insert(output->result_name); + + auto inputs = prewhere_info->prewhere_actions->getInputs(); + ColumnsWithTypeAndName all_inputs; + for (const auto & input : inputs) + all_inputs.emplace_back(input->column, input->result_type, input->result_name); + + ActionsDAG::NodeRawConstPtrs all_conjunctions = std::move(conjunctions.allowed); + all_conjunctions.insert(all_conjunctions.end(), conjunctions.rejected.begin(), conjunctions.rejected.end()); + + struct Step { - const auto & cast_node = step_dag->addCast(result_node, std::make_shared()); - step_dag->addOrReplaceInOutputs(cast_node); - result_name = cast_node.result_name; + ActionsDAGPtr actions; + String column_name; + }; + std::vector steps; + + for (const auto & conjunction : all_conjunctions) + { + auto result_name = conjunction->result_name; + auto step_dag = ActionsDAG::cloneActionsForConjunction({conjunction}, all_inputs); + const auto & result_node = step_dag->findInOutputs(result_name); + /// Cast to UInt8 if needed + if (result_node.result_type->getTypeId() != TypeIndex::UInt8) + { + const auto & cast_node = step_dag->addCast(result_node, std::make_shared()); + step_dag->addOrReplaceInOutputs(cast_node); + result_name = cast_node.result_name; + } + step_dag->removeUnusedActions(Names{result_name}, true, true); + steps.emplace_back(Step{step_dag, result_name}); } - step_dag->removeUnusedActions(Names{result_name}, true, true); - steps.emplace_back(Step{step_dag, result_name}); + + /// "Rename" the last step result to the combined prewhere column name, because in fact it will be AND of all step results + if (steps.back().column_name != prewhere_info->prewhere_column_name) + steps.back().actions->addAlias(steps.back().actions->findInOutputs(steps.back().column_name), prewhere_info->prewhere_column_name); + + const size_t steps_before_prewhere = prewhere_actions->steps.size(); + prewhere_actions->steps.resize(steps_before_prewhere + steps.size()); + + /// Check the steps in the reverse order so that we can maintain the list of outputs used by the next steps + /// and preserve them in the current step. + for (ssize_t i = steps.size() - 1; i >= 0; --i) + { + const auto & step = steps[i]; + + /// Return the condition column + Names step_outputs{step.column_name}; + const bool remove_column = !outputs_required_by_next_steps.contains(step.column_name); + /// Preserve outputs computed at this step that are used by the next steps + for (const auto & output : outputs_required_by_next_steps) + if (step.actions->tryRestoreColumn(output)) + step_outputs.emplace_back(output); + step.actions->removeUnusedActions(step_outputs, true, true); + + /// Add current step columns as outputs that should be preserved from previous steps + for (const auto & input :step.actions->getInputs()) + outputs_required_by_next_steps.insert(input->result_name); + + //std::cerr << conjunction->result_name << "\n"; + //std::cerr << step.actions->dumpDAG() << "\n"; + + PrewhereExprStep prewhere_step + { + .actions = std::make_shared(step.actions, actions_settings), + .column_name = step.column_name, + .remove_column = remove_column, + .need_filter = false + }; + prewhere_actions->steps[steps_before_prewhere + i] = std::move(prewhere_step); + } + + prewhere_actions->steps.back().remove_column = prewhere_info->remove_prewhere_column; + prewhere_actions->steps.back().need_filter = prewhere_info->need_filter; } - - /// "Rename" the last step result to the combined prewhere column name, because in fact it will be AND of all step results - if (steps.back().column_name != prewhere_info->prewhere_column_name) - steps.back().actions->addAlias(steps.back().actions->findInOutputs(steps.back().column_name), prewhere_info->prewhere_column_name); - - const size_t steps_before_prewhere = prewhere_actions->steps.size(); - prewhere_actions->steps.resize(steps_before_prewhere + steps.size()); - - for (ssize_t i = steps.size() - 1; i >= 0; --i) + else { - const auto & step = steps[i]; - - /// Return the condition columns - Names step_outputs{step.column_name}; - const bool remove_column = !original_output_names.contains(step.column_name); - /// Preserve all the original outputs computed at this step - for (const auto & output : original_output_names) - if (step.actions->tryRestoreColumn(output)) - step_outputs.emplace_back(output); - step.actions->removeUnusedActions(step_outputs, true, true); - - /// Add current step columns as outputs for previous steps - for (const auto & input :step.actions->getInputs()) - original_output_names.insert(input->result_name); - - //std::cerr << conjunction->result_name << "\n"; - //std::cerr << step.actions->dumpDAG() << "\n"; - PrewhereExprStep prewhere_step { - .actions = std::make_shared(step.actions, actions_settings), - .column_name = step.column_name, - .remove_column = remove_column, // TODO: properly set this depending on whether the column is used in the next step - .need_filter = false + .actions = std::make_shared(prewhere_info->prewhere_actions, actions_settings), + .column_name = prewhere_info->prewhere_column_name, + .remove_column = prewhere_info->remove_prewhere_column, + .need_filter = prewhere_info->need_filter }; - prewhere_actions->steps[steps_before_prewhere + i] = std::move(prewhere_step); + + prewhere_actions->steps.emplace_back(std::move(prewhere_step)); } - - prewhere_actions->steps.back().remove_column = prewhere_info->remove_prewhere_column; - prewhere_actions->steps.back().need_filter = prewhere_info->need_filter; -#else - - PrewhereExprStep prewhere_step - { - .actions = std::make_shared(prewhere_info->prewhere_actions, actions_settings), - .column_name = prewhere_info->prewhere_column_name, - .remove_column = prewhere_info->remove_prewhere_column, - .need_filter = prewhere_info->need_filter - }; - - prewhere_actions->steps.emplace_back(std::move(prewhere_step)); -#endif } return prewhere_actions; diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index 0dc6f2f9f9b..e66e3d8a741 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -102,7 +102,7 @@ protected: injectVirtualColumns(Block & block, size_t row_count, MergeTreeReadTask * task, const DataTypePtr & partition_value_type, const Names & virtual_columns); public: - static std::unique_ptr getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings); + static std::unique_ptr getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, bool enable_multiple_prewhere_read_steps = true); protected: static void initializeRangeReadersImpl( diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 17ef9c91e78..7fd063826e7 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -289,6 +289,7 @@ MergeTreeReadTaskColumns getReadTaskColumns( const Names & required_columns, const Names & system_columns, const PrewhereInfoPtr & prewhere_info, + const MergeTreeReaderSettings & reader_settings, bool with_subcolumns) { Names column_to_read_after_prewhere = required_columns; @@ -313,7 +314,7 @@ MergeTreeReadTaskColumns getReadTaskColumns( if (prewhere_info) { - auto prewhere_actions = IMergeTreeSelectAlgorithm::getPrewhereActions(prewhere_info, {}); // TODO: pass proper actions_settings + auto prewhere_actions = IMergeTreeSelectAlgorithm::getPrewhereActions(prewhere_info, {}, reader_settings.enable_multiple_prewhere_read_steps); // TODO: pass proper actions_settings NameSet columns_from_previous_steps; diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h index 91c895c197e..7b654a05283 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h @@ -13,6 +13,7 @@ namespace DB class MergeTreeData; struct MergeTreeReadTask; +struct MergeTreeReaderSettings; struct MergeTreeBlockSizePredictor; class IMergeTreeDataPartInfoForReader; @@ -100,6 +101,7 @@ MergeTreeReadTaskColumns getReadTaskColumns( const Names & required_columns, const Names & system_columns, const PrewhereInfoPtr & prewhere_info, + const MergeTreeReaderSettings & reader_settings, bool with_subcolumns); struct MergeTreeBlockSizePredictor diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.h b/src/Storages/MergeTree/MergeTreeIOSettings.h index 65a8efebb8e..dbe5d893444 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.h +++ b/src/Storages/MergeTree/MergeTreeIOSettings.h @@ -29,6 +29,8 @@ struct MergeTreeReaderSettings bool apply_deleted_mask = true; /// Put reading task in a common I/O pool, return Async state on prepare() bool use_asynchronous_read_from_pool = false; + /// If PREWHERE has multiple conditions combined with AND, execute them in separate read/filtering steps. + bool enable_multiple_prewhere_read_steps = false; }; struct MergeTreeWriterSettings diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index ebb87be0c41..54cc44b6503 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -46,7 +46,7 @@ std::vector IMergeTreeReadPool::fillPerPartInfo(const RangesInDataParts auto task_columns = getReadTaskColumns( LoadedMergeTreeDataPartInfoForReader(part.data_part), storage_snapshot, - column_names, virtual_column_names, prewhere_info, /*with_subcolumns=*/ true); + column_names, virtual_column_names, prewhere_info, reader_settings, /*with_subcolumns=*/ true); auto size_predictor = !predict_block_size_bytes ? nullptr : IMergeTreeSelectAlgorithm::getSizePredictor(part.data_part, task_columns, sample_block); @@ -72,6 +72,7 @@ MergeTreeReadPool::MergeTreeReadPool( RangesInDataParts && parts_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, + const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, const BackoffSettings & backoff_settings_, @@ -83,6 +84,7 @@ MergeTreeReadPool::MergeTreeReadPool( virtual_column_names_, min_marks_for_concurrent_read_, prewhere_info_, + reader_settings_, std::move(parts_), (preferred_block_size_bytes_ > 0), do_not_steal_tasks_) diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index ad9d8b2e225..857aa37f783 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -27,6 +27,7 @@ public: Names virtual_column_names_, size_t min_marks_for_concurrent_read_, PrewhereInfoPtr prewhere_info_, + const MergeTreeReaderSettings & reader_settings_, RangesInDataParts parts_ranges_, bool predict_block_size_bytes_, bool do_not_steal_tasks_) @@ -35,6 +36,7 @@ public: , virtual_column_names(virtual_column_names_) , min_marks_for_concurrent_read(min_marks_for_concurrent_read_) , prewhere_info(prewhere_info_) + , reader_settings(reader_settings_) , parts_ranges(parts_ranges_) , predict_block_size_bytes(predict_block_size_bytes_) , do_not_steal_tasks(do_not_steal_tasks_) @@ -55,6 +57,7 @@ protected: const Names virtual_column_names; size_t min_marks_for_concurrent_read{0}; PrewhereInfoPtr prewhere_info; + MergeTreeReaderSettings reader_settings; RangesInDataParts parts_ranges; bool predict_block_size_bytes; bool do_not_steal_tasks; @@ -123,6 +126,7 @@ public: RangesInDataParts && parts_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, + const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, const BackoffSettings & backoff_settings_, @@ -197,6 +201,7 @@ public: ParallelReadingExtension extension_, const RangesInDataParts & parts_, const PrewhereInfoPtr & prewhere_info_, + const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, size_t min_marks_for_concurrent_read_ @@ -207,6 +212,7 @@ public: virtual_column_names_, min_marks_for_concurrent_read_, prewhere_info_, + reader_settings_, parts_, /*predict_block_size*/false, /*do_not_steal_tasks*/false) diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index df2b0ea9e3d..7daccdb95d3 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -46,7 +46,7 @@ void MergeTreeSelectAlgorithm::initializeReaders() { task_columns = getReadTaskColumns( LoadedMergeTreeDataPartInfoForReader(data_part), storage_snapshot, - required_columns, virt_column_names, prewhere_info, /*with_subcolumns=*/ true); + required_columns, virt_column_names, prewhere_info, reader_settings, /*with_subcolumns=*/ true); /// Will be used to distinguish between PREWHERE and WHERE columns when applying filter const auto & column_names = task_columns.columns.getNames(); diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index ce73dad48f9..5be1e514faa 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -11,6 +11,7 @@ #include #include #include +#include "Interpreters/Context_fwd.h" #include #include @@ -41,6 +42,7 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( , block_with_constants{KeyCondition::getBlockWithConstants(query_info.query->clone(), query_info.syntax_analyzer_result, context)} , log{log_} , column_sizes{std::move(column_sizes_)} + , move_all_conditions_to_prewhere(context->getSettingsRef().move_all_conditions_to_prewhere) { const auto & primary_key = metadata_snapshot->getPrimaryKey(); if (!primary_key.column_names.empty()) @@ -274,25 +276,28 @@ void MergeTreeWhereOptimizer::optimize(ASTSelectQuery & select) const if (!it->viable) break; -#if 0 - bool moved_enough = false; - if (total_size_of_queried_columns > 0) + + if (!move_all_conditions_to_prewhere) { - /// If we know size of queried columns use it as threshold. 10% ratio is just a guess. - moved_enough = total_size_of_moved_conditions > 0 - && (total_size_of_moved_conditions + it->columns_size) * 10 > total_size_of_queried_columns; - } - else - { - /// Otherwise, use number of moved columns as a fallback. - /// It can happen, if table has only compact parts. 25% ratio is just a guess. - moved_enough = total_number_of_moved_columns > 0 - && (total_number_of_moved_columns + it->identifiers.size()) * 4 > queried_columns.size(); + bool moved_enough = false; + if (total_size_of_queried_columns > 0) + { + /// If we know size of queried columns use it as threshold. 10% ratio is just a guess. + moved_enough = total_size_of_moved_conditions > 0 + && (total_size_of_moved_conditions + it->columns_size) * 10 > total_size_of_queried_columns; + } + else + { + /// Otherwise, use number of moved columns as a fallback. + /// It can happen, if table has only compact parts. 25% ratio is just a guess. + moved_enough = total_number_of_moved_columns > 0 + && (total_number_of_moved_columns + it->identifiers.size()) * 4 > queried_columns.size(); + } + + if (moved_enough) + break; } - if (moved_enough) - break; -#endif move_condition(it); } diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index fa14fea94d1..b0aa35ea266 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -114,6 +114,7 @@ private: std::unordered_map column_sizes; UInt64 total_size_of_queried_columns = 0; NameSet array_joined_names; + const bool move_all_conditions_to_prewhere = false; }; From 9e656b5c0e130333b8a818e0689a89c40bcfba4d Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 9 Feb 2023 19:40:15 +0100 Subject: [PATCH 051/253] Added a test --- ..._multiple_read_steps_in_prewhere.reference | 31 +++++++++++++++++++ .../02559_multiple_read_steps_in_prewhere.sql | 21 +++++++++++++ 2 files changed, 52 insertions(+) create mode 100644 tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference create mode 100644 tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference new file mode 100644 index 00000000000..fbd143e9b55 --- /dev/null +++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference @@ -0,0 +1,31 @@ +SELECT cast(id as UInt16) AS id16 FROM test_02559 PREWHERE id16 and (id % 40000) LIMIT 10; +1 +2 +3 +4 +5 +6 +7 +8 +9 +SELECT cast(id as UInt16) AS id16, (id % 40000) AS id40000, (id16 AND id40000) AS cond FROM test_02559 PREWHERE cond LIMIT 10; +1 1 1 +2 2 1 +3 3 1 +4 4 1 +5 5 1 +6 6 1 +7 7 1 +8 8 1 +9 9 1 +SELECT cast(id as UInt16) AS id16 FROM test_02559 PREWHERE id16 LIMIT 10; +1 +2 +3 +4 +5 +6 +7 +8 +9 +SELECT count() FROM test_02559 PREWHERE 1 OR ignore(id) WHERE ignore(id)=0; diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql new file mode 100644 index 00000000000..68c61dda6dd --- /dev/null +++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql @@ -0,0 +1,21 @@ +DROP TABLE IF EXISTS test_02559; + +CREATE TABLE test_02559 (id UInt64) ENGINE=MergeTree ORDER BY id; + +INSERT INTO test_02559 SELECT number FROM numbers(10); + +SET enable_multiple_prewhere_read_steps=true, move_all_conditions_to_prewhere=true; + +-- { echoOn } + +SELECT cast(id as UInt16) AS id16 FROM test_02559 PREWHERE id16 and (id % 40000) LIMIT 10; + +SELECT cast(id as UInt16) AS id16, (id % 40000) AS id40000, (id16 AND id40000) AS cond FROM test_02559 PREWHERE cond LIMIT 10; + +SELECT cast(id as UInt16) AS id16 FROM test_02559 PREWHERE id16 LIMIT 10; + +SELECT count() FROM test_02559 PREWHERE 1 OR ignore(id) WHERE ignore(id)=0; + +-- { echoOff } + +DROP TABLE test_02559; From e460d5361911cfc78442932034892030f34e0a2d Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 10 Feb 2023 15:08:15 +0100 Subject: [PATCH 052/253] Use different name when filter column is CASTed to UInt8 --- .../MergeTreeBaseSelectProcessor.cpp | 64 +++++++++++++++---- 1 file changed, 52 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 49022dce828..c13cda6466a 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -7,13 +7,18 @@ #include #include #include -#include "Core/Names.h" -#include "Interpreters/ActionsDAG.h" #include #include #include #include #include + +#include + +/// For CAST to bool +#include +#include + #include namespace ProfileEvents @@ -104,17 +109,22 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( prewhere_actions->steps.emplace_back(std::move(row_level_filter_step)); } + //std::cerr << "ORIGINAL PREWHERE:\n" << prewhere_info->prewhere_actions->dumpDAG() << std::endl; + if (enable_multiple_prewhere_read_steps) { + /// Find all conjunctions in prewhere expression. auto conjunctions = getConjunctionNodes( prewhere_info->prewhere_actions->tryFindInOutputs(prewhere_info->prewhere_column_name), {}); + /// Save the list of outputs from the original prewhere expression. auto original_outputs = prewhere_info->prewhere_actions->getOutputs(); - NameSet outputs_required_by_next_steps; + std::unordered_map outputs_required_by_next_steps; for (const auto & output : original_outputs) - outputs_required_by_next_steps.insert(output->result_name); + outputs_required_by_next_steps[output->result_name] = output->result_type; + /// Save the list of inputs to the original prewhere expression. auto inputs = prewhere_info->prewhere_actions->getInputs(); ColumnsWithTypeAndName all_inputs; for (const auto & input : inputs) @@ -130,15 +140,36 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( }; std::vector steps; + /// Adds a CAST node with the regular name ("CAST(...)") or with the provided name. + /// This is different from ActionsDAG::addCast() becuase it set the name equal to the original name effectively hiding the value before cast, + /// but it might be required for further steps with its original uncasted type. + auto add_cast = [] (ActionsDAGPtr dag, const ActionsDAG::Node & node_to_cast, const String & type_name, const String & new_name = {}) -> const ActionsDAG::Node & + { + Field cast_type_constant_value(type_name); + + ColumnWithTypeAndName column; + column.name = calculateConstantActionNodeName(cast_type_constant_value); + column.column = DataTypeString().createColumnConst(0, cast_type_constant_value); + column.type = std::make_shared(); + + const auto * cast_type_constant_node = &dag->addColumn(std::move(column)); + ActionsDAG::NodeRawConstPtrs children = {&node_to_cast, cast_type_constant_node}; + FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver::createImpl(); + + return dag->addFunction(func_builder_cast, std::move(children), new_name); + }; + + /// Make separate DAG for each step for (const auto & conjunction : all_conjunctions) { auto result_name = conjunction->result_name; auto step_dag = ActionsDAG::cloneActionsForConjunction({conjunction}, all_inputs); const auto & result_node = step_dag->findInOutputs(result_name); - /// Cast to UInt8 if needed + /// Cast result to UInt8 if needed if (result_node.result_type->getTypeId() != TypeIndex::UInt8) { - const auto & cast_node = step_dag->addCast(result_node, std::make_shared()); + const auto & cast_node = add_cast(step_dag, result_node, "UInt8"); + step_dag->addOrReplaceInOutputs(cast_node); result_name = cast_node.result_name; } @@ -147,8 +178,17 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( } /// "Rename" the last step result to the combined prewhere column name, because in fact it will be AND of all step results - if (steps.back().column_name != prewhere_info->prewhere_column_name) - steps.back().actions->addAlias(steps.back().actions->findInOutputs(steps.back().column_name), prewhere_info->prewhere_column_name); + if (steps.back().column_name != prewhere_info->prewhere_column_name && + outputs_required_by_next_steps.contains(prewhere_info->prewhere_column_name)) + { + const auto & prewhere_result_node = add_cast( + steps.back().actions, + steps.back().actions->findInOutputs(steps.back().column_name), + outputs_required_by_next_steps[prewhere_info->prewhere_column_name]->getName(), + prewhere_info->prewhere_column_name); + + steps.back().actions->addOrReplaceInOutputs(prewhere_result_node); + } const size_t steps_before_prewhere = prewhere_actions->steps.size(); prewhere_actions->steps.resize(steps_before_prewhere + steps.size()); @@ -164,16 +204,16 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( const bool remove_column = !outputs_required_by_next_steps.contains(step.column_name); /// Preserve outputs computed at this step that are used by the next steps for (const auto & output : outputs_required_by_next_steps) - if (step.actions->tryRestoreColumn(output)) - step_outputs.emplace_back(output); + if (step.actions->tryRestoreColumn(output.first)) + step_outputs.emplace_back(output.first); step.actions->removeUnusedActions(step_outputs, true, true); /// Add current step columns as outputs that should be preserved from previous steps for (const auto & input :step.actions->getInputs()) - outputs_required_by_next_steps.insert(input->result_name); + outputs_required_by_next_steps[input->result_name] = input->result_type; //std::cerr << conjunction->result_name << "\n"; - //std::cerr << step.actions->dumpDAG() << "\n"; + //std::cerr << "STEP " << i << ":\n" << step.actions->dumpDAG() << "\n"; PrewhereExprStep prewhere_step { From 8fc08c74f0dd60fa7778200da9bcb7e37a2f1197 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 10 Feb 2023 16:50:47 +0100 Subject: [PATCH 053/253] Pass action_settings to getReadTaskColumns() --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 ++ src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp | 5 +++-- src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h | 3 ++- src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp | 4 +++- src/Storages/MergeTree/MergeTreeBlockReadUtils.h | 1 + src/Storages/MergeTree/MergeTreeReadPool.cpp | 4 +++- src/Storages/MergeTree/MergeTreeReadPool.h | 6 ++++++ src/Storages/MergeTree/MergeTreeSelectProcessor.cpp | 6 +++--- src/Storages/MergeTree/MergeTreeSelectProcessor.h | 2 +- src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeThreadSelectProcessor.h | 2 +- 11 files changed, 27 insertions(+), 12 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 6ff230e9d78..734bab4624b 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -226,6 +226,7 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas( extension, parts_with_range, prewhere_info, + actions_settings, reader_settings, required_columns, virt_column_names, @@ -304,6 +305,7 @@ Pipe ReadFromMergeTree::readFromPool( std::move(parts_with_range), storage_snapshot, prewhere_info, + actions_settings, reader_settings, required_columns, virt_column_names, diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index c13cda6466a..00e1eed37e1 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -54,7 +54,7 @@ IMergeTreeSelectAlgorithm::IMergeTreeSelectAlgorithm( const MergeTreeData & storage_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, - ExpressionActionsSettings actions_settings, + const ExpressionActionsSettings & actions_settings_, UInt64 max_block_size_rows_, UInt64 preferred_block_size_bytes_, UInt64 preferred_max_column_in_block_size_bytes_, @@ -64,6 +64,7 @@ IMergeTreeSelectAlgorithm::IMergeTreeSelectAlgorithm( : storage(storage_) , storage_snapshot(storage_snapshot_) , prewhere_info(prewhere_info_) + , actions_settings(actions_settings_) , prewhere_actions(getPrewhereActions(prewhere_info, actions_settings, reader_settings_.enable_multiple_prewhere_read_steps)) , max_block_size_rows(max_block_size_rows_) , preferred_block_size_bytes(preferred_block_size_bytes_) @@ -141,7 +142,7 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( std::vector steps; /// Adds a CAST node with the regular name ("CAST(...)") or with the provided name. - /// This is different from ActionsDAG::addCast() becuase it set the name equal to the original name effectively hiding the value before cast, + /// This is different from ActionsDAG::addCast() because it set the name equal to the original name effectively hiding the value before cast, /// but it might be required for further steps with its original uncasted type. auto add_cast = [] (ActionsDAGPtr dag, const ActionsDAG::Node & node_to_cast, const String & type_name, const String & new_name = {}) -> const ActionsDAG::Node & { diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index e66e3d8a741..b27a7114122 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -43,7 +43,7 @@ public: const MergeTreeData & storage_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, - ExpressionActionsSettings actions_settings, + const ExpressionActionsSettings & actions_settings, UInt64 max_block_size_rows_, UInt64 preferred_block_size_bytes_, UInt64 preferred_max_column_in_block_size_bytes_, @@ -140,6 +140,7 @@ protected: /// This step is added when the part has lightweight delete mask const PrewhereExprStep lightweight_delete_filter_step { nullptr, LightweightDeleteDescription::FILTER_COLUMN.name, true, true }; PrewhereInfoPtr prewhere_info; + ExpressionActionsSettings actions_settings; std::unique_ptr prewhere_actions; UInt64 max_block_size_rows; diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 7fd063826e7..95c76c81665 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -289,6 +289,7 @@ MergeTreeReadTaskColumns getReadTaskColumns( const Names & required_columns, const Names & system_columns, const PrewhereInfoPtr & prewhere_info, + const ExpressionActionsSettings & actions_settings, const MergeTreeReaderSettings & reader_settings, bool with_subcolumns) { @@ -314,7 +315,8 @@ MergeTreeReadTaskColumns getReadTaskColumns( if (prewhere_info) { - auto prewhere_actions = IMergeTreeSelectAlgorithm::getPrewhereActions(prewhere_info, {}, reader_settings.enable_multiple_prewhere_read_steps); // TODO: pass proper actions_settings + auto prewhere_actions = IMergeTreeSelectAlgorithm::getPrewhereActions( + prewhere_info, actions_settings, reader_settings.enable_multiple_prewhere_read_steps); NameSet columns_from_previous_steps; diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h index 7b654a05283..8a8c1ca6a40 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h @@ -101,6 +101,7 @@ MergeTreeReadTaskColumns getReadTaskColumns( const Names & required_columns, const Names & system_columns, const PrewhereInfoPtr & prewhere_info, + const ExpressionActionsSettings & actions_settings, const MergeTreeReaderSettings & reader_settings, bool with_subcolumns); diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 54cc44b6503..f5de75c7471 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -46,7 +46,7 @@ std::vector IMergeTreeReadPool::fillPerPartInfo(const RangesInDataParts auto task_columns = getReadTaskColumns( LoadedMergeTreeDataPartInfoForReader(part.data_part), storage_snapshot, - column_names, virtual_column_names, prewhere_info, reader_settings, /*with_subcolumns=*/ true); + column_names, virtual_column_names, prewhere_info, actions_settings, reader_settings, /*with_subcolumns=*/ true); auto size_predictor = !predict_block_size_bytes ? nullptr : IMergeTreeSelectAlgorithm::getSizePredictor(part.data_part, task_columns, sample_block); @@ -72,6 +72,7 @@ MergeTreeReadPool::MergeTreeReadPool( RangesInDataParts && parts_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, + const ExpressionActionsSettings & actions_settings_, const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, @@ -84,6 +85,7 @@ MergeTreeReadPool::MergeTreeReadPool( virtual_column_names_, min_marks_for_concurrent_read_, prewhere_info_, + actions_settings_, reader_settings_, std::move(parts_), (preferred_block_size_bytes_ > 0), diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index 857aa37f783..988dabb47f0 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -27,6 +27,7 @@ public: Names virtual_column_names_, size_t min_marks_for_concurrent_read_, PrewhereInfoPtr prewhere_info_, + const ExpressionActionsSettings & actions_settings_, const MergeTreeReaderSettings & reader_settings_, RangesInDataParts parts_ranges_, bool predict_block_size_bytes_, @@ -36,6 +37,7 @@ public: , virtual_column_names(virtual_column_names_) , min_marks_for_concurrent_read(min_marks_for_concurrent_read_) , prewhere_info(prewhere_info_) + , actions_settings(actions_settings_) , reader_settings(reader_settings_) , parts_ranges(parts_ranges_) , predict_block_size_bytes(predict_block_size_bytes_) @@ -57,6 +59,7 @@ protected: const Names virtual_column_names; size_t min_marks_for_concurrent_read{0}; PrewhereInfoPtr prewhere_info; + ExpressionActionsSettings actions_settings; MergeTreeReaderSettings reader_settings; RangesInDataParts parts_ranges; bool predict_block_size_bytes; @@ -126,6 +129,7 @@ public: RangesInDataParts && parts_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, + const ExpressionActionsSettings & actions_settings_, const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, @@ -201,6 +205,7 @@ public: ParallelReadingExtension extension_, const RangesInDataParts & parts_, const PrewhereInfoPtr & prewhere_info_, + const ExpressionActionsSettings & actions_settings_, const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const Names & virtual_column_names_, @@ -212,6 +217,7 @@ public: virtual_column_names_, min_marks_for_concurrent_read_, prewhere_info_, + actions_settings_, reader_settings_, parts_, /*predict_block_size*/false, diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 7daccdb95d3..83e9cc90547 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -19,7 +19,7 @@ MergeTreeSelectAlgorithm::MergeTreeSelectAlgorithm( MarkRanges mark_ranges_, bool use_uncompressed_cache_, const PrewhereInfoPtr & prewhere_info_, - ExpressionActionsSettings actions_settings, + const ExpressionActionsSettings & actions_settings_, const MergeTreeReaderSettings & reader_settings_, MergeTreeInOrderReadPoolParallelReplicasPtr pool_, const Names & virt_column_names_, @@ -27,7 +27,7 @@ MergeTreeSelectAlgorithm::MergeTreeSelectAlgorithm( bool has_limit_below_one_block_) : IMergeTreeSelectAlgorithm{ storage_snapshot_->getSampleBlockForColumns(required_columns_), - storage_, storage_snapshot_, prewhere_info_, std::move(actions_settings), max_block_size_rows_, + storage_, storage_snapshot_, prewhere_info_, actions_settings_, max_block_size_rows_, preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_, reader_settings_, use_uncompressed_cache_, virt_column_names_}, required_columns{std::move(required_columns_)}, @@ -46,7 +46,7 @@ void MergeTreeSelectAlgorithm::initializeReaders() { task_columns = getReadTaskColumns( LoadedMergeTreeDataPartInfoForReader(data_part), storage_snapshot, - required_columns, virt_column_names, prewhere_info, reader_settings, /*with_subcolumns=*/ true); + required_columns, virt_column_names, prewhere_info, actions_settings, reader_settings, /*with_subcolumns=*/ true); /// Will be used to distinguish between PREWHERE and WHERE columns when applying filter const auto & column_names = task_columns.columns.getNames(); diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.h b/src/Storages/MergeTree/MergeTreeSelectProcessor.h index 76c8d81dd0b..981c42574e0 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.h @@ -28,7 +28,7 @@ public: MarkRanges mark_ranges, bool use_uncompressed_cache, const PrewhereInfoPtr & prewhere_info, - ExpressionActionsSettings actions_settings, + const ExpressionActionsSettings & actions_settings_, const MergeTreeReaderSettings & reader_settings, MergeTreeInOrderReadPoolParallelReplicasPtr pool_, const Names & virt_column_names = {}, diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp index 0c7f90b2349..d84bda63421 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp @@ -18,12 +18,12 @@ MergeTreeThreadSelectAlgorithm::MergeTreeThreadSelectAlgorithm( const StorageSnapshotPtr & storage_snapshot_, bool use_uncompressed_cache_, const PrewhereInfoPtr & prewhere_info_, - ExpressionActionsSettings actions_settings, + const ExpressionActionsSettings & actions_settings_, const MergeTreeReaderSettings & reader_settings_, const Names & virt_column_names_) : IMergeTreeSelectAlgorithm{ - pool_->getHeader(), storage_, storage_snapshot_, prewhere_info_, std::move(actions_settings), max_block_size_rows_, + pool_->getHeader(), storage_, storage_snapshot_, prewhere_info_, actions_settings_, max_block_size_rows_, preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_, reader_settings_, use_uncompressed_cache_, virt_column_names_}, thread{thread_}, diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.h b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.h index 37c9375a581..4d9c9c92daf 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.h @@ -25,7 +25,7 @@ public: const StorageSnapshotPtr & storage_snapshot_, bool use_uncompressed_cache_, const PrewhereInfoPtr & prewhere_info_, - ExpressionActionsSettings actions_settings, + const ExpressionActionsSettings & actions_settings_, const MergeTreeReaderSettings & reader_settings_, const Names & virt_column_names_); From 7ee7afe75062ba2fb13d646aa262d30e9e671f94 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 10 Feb 2023 22:09:07 +0100 Subject: [PATCH 054/253] More test cases --- ..._multiple_read_steps_in_prewhere.reference | 33 ++++++++++++------- .../02559_multiple_read_steps_in_prewhere.sql | 10 ++++-- 2 files changed, 30 insertions(+), 13 deletions(-) diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference index fbd143e9b55..8e4056ec660 100644 --- a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference +++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference @@ -1,3 +1,7 @@ +--SET enable_multiple_prewhere_read_steps=true, move_all_conditions_to_prewhere=true; + +-- { echoOn } + SELECT cast(id as UInt16) AS id16 FROM test_02559 PREWHERE id16 and (id % 40000) LIMIT 10; 1 2 @@ -8,7 +12,7 @@ SELECT cast(id as UInt16) AS id16 FROM test_02559 PREWHERE id16 and (id % 40000) 7 8 9 -SELECT cast(id as UInt16) AS id16, (id % 40000) AS id40000, (id16 AND id40000) AS cond FROM test_02559 PREWHERE cond LIMIT 10; +SELECT cast(id as UInt16) AS cond1, (id % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond LIMIT 10; 1 1 1 2 2 1 3 3 1 @@ -18,14 +22,21 @@ SELECT cast(id as UInt16) AS id16, (id % 40000) AS id40000, (id16 AND id40000) A 7 7 1 8 8 1 9 9 1 -SELECT cast(id as UInt16) AS id16 FROM test_02559 PREWHERE id16 LIMIT 10; -1 -2 -3 -4 -5 -6 -7 -8 -9 +SELECT cast(id as UInt16) AS cond1, (id % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond AND id > 4 LIMIT 10; +5 5 1 +6 6 1 +7 7 1 +8 8 1 +9 9 1 +SELECT cast(id as UInt16) AS cond1, (id % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE id > 5 AND cond LIMIT 10; +6 6 1 +7 7 1 +8 8 1 +9 9 1 +SELECT cast(id as UInt16) AS cond1, (id % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond1 AND id > 6 AND cond2 LIMIT 10; +7 7 1 +8 8 1 +9 9 1 +SELECT cast(id as UInt16) AS cond1 FROM test_02559 PREWHERE cond1 LIMIT 10; -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } SELECT count() FROM test_02559 PREWHERE 1 OR ignore(id) WHERE ignore(id)=0; +10 diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql index 68c61dda6dd..5dd09457313 100644 --- a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql +++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql @@ -10,9 +10,15 @@ SET enable_multiple_prewhere_read_steps=true, move_all_conditions_to_prewhere=tr SELECT cast(id as UInt16) AS id16 FROM test_02559 PREWHERE id16 and (id % 40000) LIMIT 10; -SELECT cast(id as UInt16) AS id16, (id % 40000) AS id40000, (id16 AND id40000) AS cond FROM test_02559 PREWHERE cond LIMIT 10; +SELECT cast(id as UInt16) AS cond1, (id % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond LIMIT 10; -SELECT cast(id as UInt16) AS id16 FROM test_02559 PREWHERE id16 LIMIT 10; +SELECT cast(id as UInt16) AS cond1, (id % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond AND id > 4 LIMIT 10; + +SELECT cast(id as UInt16) AS cond1, (id % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE id > 5 AND cond LIMIT 10; + +SELECT cast(id as UInt16) AS cond1, (id % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond1 AND id > 6 AND cond2 LIMIT 10; + +SELECT cast(id as UInt16) AS cond1 FROM test_02559 PREWHERE cond1 LIMIT 10; -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } SELECT count() FROM test_02559 PREWHERE 1 OR ignore(id) WHERE ignore(id)=0; From 12e576cddb18083021b4a5984939265d4d544537 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 10 Feb 2023 22:15:27 +0100 Subject: [PATCH 055/253] Slight refactoring --- .../MergeTreeBaseSelectProcessor.cpp | 70 ++++++++++--------- 1 file changed, 36 insertions(+), 34 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 00e1eed37e1..df7f77261af 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -89,6 +89,24 @@ IMergeTreeSelectAlgorithm::IMergeTreeSelectAlgorithm( LOG_TEST(log, "PREWHERE actions: {}", (prewhere_actions ? prewhere_actions->dump() : std::string(""))); } +/// Adds a CAST node with the regular name ("CAST(...)") or with the provided name. +/// This is different from ActionsDAG::addCast() because it set the name equal to the original name effectively hiding the value before cast, +/// but it might be required for further steps with its original uncasted type. +static const ActionsDAG::Node & addCast(ActionsDAGPtr dag, const ActionsDAG::Node & node_to_cast, const String & type_name, const String & new_name = {}) +{ + Field cast_type_constant_value(type_name); + + ColumnWithTypeAndName column; + column.name = calculateConstantActionNodeName(cast_type_constant_value); + column.column = DataTypeString().createColumnConst(0, cast_type_constant_value); + column.type = std::make_shared(); + + const auto * cast_type_constant_node = &dag->addColumn(std::move(column)); + ActionsDAG::NodeRawConstPtrs children = {&node_to_cast, cast_type_constant_node}; + FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver::createImpl(); + + return dag->addFunction(func_builder_cast, std::move(children), new_name); +}; std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, bool enable_multiple_prewhere_read_steps) { @@ -112,6 +130,13 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( //std::cerr << "ORIGINAL PREWHERE:\n" << prewhere_info->prewhere_actions->dumpDAG() << std::endl; + struct Step + { + ActionsDAGPtr actions; + String column_name; + }; + std::vector steps; + if (enable_multiple_prewhere_read_steps) { /// Find all conjunctions in prewhere expression. @@ -119,12 +144,6 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( prewhere_info->prewhere_actions->tryFindInOutputs(prewhere_info->prewhere_column_name), {}); - /// Save the list of outputs from the original prewhere expression. - auto original_outputs = prewhere_info->prewhere_actions->getOutputs(); - std::unordered_map outputs_required_by_next_steps; - for (const auto & output : original_outputs) - outputs_required_by_next_steps[output->result_name] = output->result_type; - /// Save the list of inputs to the original prewhere expression. auto inputs = prewhere_info->prewhere_actions->getInputs(); ColumnsWithTypeAndName all_inputs; @@ -134,32 +153,6 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( ActionsDAG::NodeRawConstPtrs all_conjunctions = std::move(conjunctions.allowed); all_conjunctions.insert(all_conjunctions.end(), conjunctions.rejected.begin(), conjunctions.rejected.end()); - struct Step - { - ActionsDAGPtr actions; - String column_name; - }; - std::vector steps; - - /// Adds a CAST node with the regular name ("CAST(...)") or with the provided name. - /// This is different from ActionsDAG::addCast() because it set the name equal to the original name effectively hiding the value before cast, - /// but it might be required for further steps with its original uncasted type. - auto add_cast = [] (ActionsDAGPtr dag, const ActionsDAG::Node & node_to_cast, const String & type_name, const String & new_name = {}) -> const ActionsDAG::Node & - { - Field cast_type_constant_value(type_name); - - ColumnWithTypeAndName column; - column.name = calculateConstantActionNodeName(cast_type_constant_value); - column.column = DataTypeString().createColumnConst(0, cast_type_constant_value); - column.type = std::make_shared(); - - const auto * cast_type_constant_node = &dag->addColumn(std::move(column)); - ActionsDAG::NodeRawConstPtrs children = {&node_to_cast, cast_type_constant_node}; - FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver::createImpl(); - - return dag->addFunction(func_builder_cast, std::move(children), new_name); - }; - /// Make separate DAG for each step for (const auto & conjunction : all_conjunctions) { @@ -169,7 +162,7 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( /// Cast result to UInt8 if needed if (result_node.result_type->getTypeId() != TypeIndex::UInt8) { - const auto & cast_node = add_cast(step_dag, result_node, "UInt8"); + const auto & cast_node = addCast(step_dag, result_node, "UInt8"); step_dag->addOrReplaceInOutputs(cast_node); result_name = cast_node.result_name; @@ -177,12 +170,21 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( step_dag->removeUnusedActions(Names{result_name}, true, true); steps.emplace_back(Step{step_dag, result_name}); } + } + + if (steps.size() > 1) + { + /// Save the list of outputs from the original prewhere expression. + auto original_outputs = prewhere_info->prewhere_actions->getOutputs(); + std::unordered_map outputs_required_by_next_steps; + for (const auto & output : original_outputs) + outputs_required_by_next_steps[output->result_name] = output->result_type; /// "Rename" the last step result to the combined prewhere column name, because in fact it will be AND of all step results if (steps.back().column_name != prewhere_info->prewhere_column_name && outputs_required_by_next_steps.contains(prewhere_info->prewhere_column_name)) { - const auto & prewhere_result_node = add_cast( + const auto & prewhere_result_node = addCast( steps.back().actions, steps.back().actions->findInOutputs(steps.back().column_name), outputs_required_by_next_steps[prewhere_info->prewhere_column_name]->getName(), From 943f2ea212db183306429ba99fa0e6508651f581 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Sun, 12 Feb 2023 21:41:13 +0100 Subject: [PATCH 056/253] Use 2 columns in test to have 2 read steps --- ...9_multiple_read_steps_in_prewhere.reference | 16 +++++++--------- .../02559_multiple_read_steps_in_prewhere.sql | 18 +++++++++--------- 2 files changed, 16 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference index 8e4056ec660..0c46a90257d 100644 --- a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference +++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference @@ -1,8 +1,6 @@ ---SET enable_multiple_prewhere_read_steps=true, move_all_conditions_to_prewhere=true; - -- { echoOn } -SELECT cast(id as UInt16) AS id16 FROM test_02559 PREWHERE id16 and (id % 40000) LIMIT 10; +SELECT cast(id1 as UInt16) AS id16 FROM test_02559 PREWHERE id16 and (id2 % 40000) LIMIT 10; 1 2 3 @@ -12,7 +10,7 @@ SELECT cast(id as UInt16) AS id16 FROM test_02559 PREWHERE id16 and (id % 40000) 7 8 9 -SELECT cast(id as UInt16) AS cond1, (id % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond LIMIT 10; +SELECT cast(id1 as UInt16) AS cond1, (id2 % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond LIMIT 10; 1 1 1 2 2 1 3 3 1 @@ -22,21 +20,21 @@ SELECT cast(id as UInt16) AS cond1, (id % 40000) AS cond2, (cond1 AND cond2) AS 7 7 1 8 8 1 9 9 1 -SELECT cast(id as UInt16) AS cond1, (id % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond AND id > 4 LIMIT 10; +SELECT cast(id1 as UInt16) AS cond1, (id2 % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond AND id2 > 4 LIMIT 10; 5 5 1 6 6 1 7 7 1 8 8 1 9 9 1 -SELECT cast(id as UInt16) AS cond1, (id % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE id > 5 AND cond LIMIT 10; +SELECT cast(id1 as UInt16) AS cond1, (id2 % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE id2 > 5 AND cond LIMIT 10; 6 6 1 7 7 1 8 8 1 9 9 1 -SELECT cast(id as UInt16) AS cond1, (id % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond1 AND id > 6 AND cond2 LIMIT 10; +SELECT cast(id1 as UInt16) AS cond1, (id2 % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond1 AND id2 > 6 AND cond2 LIMIT 10; 7 7 1 8 8 1 9 9 1 -SELECT cast(id as UInt16) AS cond1 FROM test_02559 PREWHERE cond1 LIMIT 10; -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } -SELECT count() FROM test_02559 PREWHERE 1 OR ignore(id) WHERE ignore(id)=0; +SELECT cast(id1 as UInt16) AS cond1 FROM test_02559 PREWHERE cond1 LIMIT 10; -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } +SELECT count() FROM test_02559 PREWHERE id2>=0 AND (1 OR ignore(id1)) WHERE ignore(id1)=0; 10 diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql index 5dd09457313..e39f8ac846a 100644 --- a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql +++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql @@ -1,26 +1,26 @@ DROP TABLE IF EXISTS test_02559; -CREATE TABLE test_02559 (id UInt64) ENGINE=MergeTree ORDER BY id; +CREATE TABLE test_02559 (id1 UInt64, id2 UInt64) ENGINE=MergeTree ORDER BY id1; -INSERT INTO test_02559 SELECT number FROM numbers(10); +INSERT INTO test_02559 SELECT number, number FROM numbers(10); SET enable_multiple_prewhere_read_steps=true, move_all_conditions_to_prewhere=true; -- { echoOn } -SELECT cast(id as UInt16) AS id16 FROM test_02559 PREWHERE id16 and (id % 40000) LIMIT 10; +SELECT cast(id1 as UInt16) AS id16 FROM test_02559 PREWHERE id16 and (id2 % 40000) LIMIT 10; -SELECT cast(id as UInt16) AS cond1, (id % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond LIMIT 10; +SELECT cast(id1 as UInt16) AS cond1, (id2 % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond LIMIT 10; -SELECT cast(id as UInt16) AS cond1, (id % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond AND id > 4 LIMIT 10; +SELECT cast(id1 as UInt16) AS cond1, (id2 % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond AND id2 > 4 LIMIT 10; -SELECT cast(id as UInt16) AS cond1, (id % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE id > 5 AND cond LIMIT 10; +SELECT cast(id1 as UInt16) AS cond1, (id2 % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE id2 > 5 AND cond LIMIT 10; -SELECT cast(id as UInt16) AS cond1, (id % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond1 AND id > 6 AND cond2 LIMIT 10; +SELECT cast(id1 as UInt16) AS cond1, (id2 % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond1 AND id2 > 6 AND cond2 LIMIT 10; -SELECT cast(id as UInt16) AS cond1 FROM test_02559 PREWHERE cond1 LIMIT 10; -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } +SELECT cast(id1 as UInt16) AS cond1 FROM test_02559 PREWHERE cond1 LIMIT 10; -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } -SELECT count() FROM test_02559 PREWHERE 1 OR ignore(id) WHERE ignore(id)=0; +SELECT count() FROM test_02559 PREWHERE id2>=0 AND (1 OR ignore(id1)) WHERE ignore(id1)=0; -- { echoOff } From 3ad0683de93691bc8c9745f622a373b8f958db3a Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Sun, 12 Feb 2023 21:42:24 +0100 Subject: [PATCH 057/253] Refactor DAG splitting into steps --- .../MergeTreeBaseSelectProcessor.cpp | 269 +++++++++++++++++- 1 file changed, 268 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index df7f77261af..5f40950c3b2 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -17,6 +17,7 @@ /// For CAST to bool #include +#include #include #include @@ -108,6 +109,255 @@ static const ActionsDAG::Node & addCast(ActionsDAGPtr dag, const ActionsDAG::Nod return dag->addFunction(func_builder_cast, std::move(children), new_name); }; +struct NodeInfo +{ + NameSet required_columns; +}; + +void fillRequiredColumns(const ActionsDAG::Node * node, std::unordered_map & nodes_info) +{ + if (nodes_info.contains(node)) + return; + + auto & node_info = nodes_info[node]; + + if (node->type == ActionsDAG::ActionType::INPUT) + { + node_info.required_columns.insert(node->result_name); + return; + } + + for (const auto & child : node->children) + { + fillRequiredColumns(child, nodes_info); + const auto & child_info = nodes_info[child]; + node_info.required_columns.insert(child_info.required_columns.begin(), child_info.required_columns.end()); + } +} + +struct DAGNodeRef +{ + ActionsDAGPtr dag; + const ActionsDAG::Node * node; +}; + +using OriginalToNewNodeMap = std::unordered_map; + +const ActionsDAG::Node & addClonedDAGToDAG(const ActionsDAG::Node * original_dag_node, ActionsDAGPtr new_dag, OriginalToNewNodeMap & node_remap) +{ + if (node_remap.contains(original_dag_node)) + { + /// If the node is already in the new DAG, return it + const auto & node_ref = node_remap.at(original_dag_node); + if (node_ref.dag == new_dag) + return *node_ref.node; + + /// If the node is known from the previous steps, add it as an input + node_ref.dag->addOrReplaceInOutputs(*node_ref.node); + const auto & new_node = new_dag->addInput(node_ref.node->result_name, node_ref.node->result_type); + node_remap[original_dag_node] = {new_dag, &new_node}; /// TODO: here we update the node reference. Is ti always correct? + return new_node; + } + + /// If the node is an input, add it as an input + if (original_dag_node->type == ActionsDAG::ActionType::INPUT) + { + const auto & new_node = new_dag->addInput(original_dag_node->result_name, original_dag_node->result_type); + node_remap[original_dag_node] = {new_dag, &new_node}; + return new_node; + } + + /// If the node is a column, add it as an input + if (original_dag_node->type == ActionsDAG::ActionType::COLUMN) + { + const auto & new_node = new_dag->addColumn( + ColumnWithTypeAndName(original_dag_node->column, original_dag_node->result_type, original_dag_node->result_name)); + node_remap[original_dag_node] = {new_dag, &new_node}; + return new_node; + } + + /// TODO: Alias node? + + /// If the node is a function, add it as a function and add its children + if (original_dag_node->type == ActionsDAG::ActionType::FUNCTION) + { + ActionsDAG::NodeRawConstPtrs new_children; + for (const auto & child : original_dag_node->children) + { + const auto & new_child = addClonedDAGToDAG(child, new_dag, node_remap); + new_children.push_back(&new_child); + } + + const auto & new_node = new_dag->addFunction(original_dag_node->function_base, new_children, original_dag_node->result_name); + node_remap[original_dag_node] = {new_dag, &new_node}; + return new_node; + } + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected node type in PREWHERE actions: {}", original_dag_node->type); +} + +bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, PrewhereExprInfo & prewhere) +{ + /// We want to build a sequence of steps that will compute parts of the prewhere condition. + /// Each step reads some new columns and computes some new expressions and a filter condition. + /// The last step computes the final filter condition and the remaining expressions that are required for the main query. + /// The steps are built in the following way: + /// 1. List all condition nodes that are combined with AND into PREWHERE condition + /// 2. Collect the set of columns that are used in the condition + /// 3. Sort condition nodes by the number of columns used in them and the overall size of those columns + /// 4. Group conditions with the same set of columns into a single read/compute step + /// 5. Build DAGs for each step: + /// - DFS from the condition root node: + /// - If the node was not computed yet, add it to the DAG and traverse its children + /// - If the node was already computed by one of the previous steps, add it as output for that step and as input for the current step + /// - If the node was already computed by the current step just stop traversing + /// 6. Find all outputs of the original DAG + /// 7. Find all outputs that were computed in the already built DAGs, mark these nodes as outputs in the steps where they were computed + /// 8. Add computation of the remaining outputs to the last step with the procedure similar to 4 + + if (!prewhere_info || !prewhere_info->prewhere_actions) + return true; + + /// 1. List all condition nodes that are combined with AND into PREWHERE condition + const auto & condition_root = prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name); + const bool is_conjunction = (condition_root.type == ActionsDAG::ActionType::FUNCTION && condition_root.function_base->getName() == "and"); + if (!is_conjunction) + return false; + auto condition_nodes = condition_root.children; + + /// 2. Collect the set of columns that are used in the condition + std::unordered_map nodes_info; + for (const auto & node : condition_nodes) + { + fillRequiredColumns(node, nodes_info); + } + + /// 3. Sort condition nodes by the number of columns used in them and the overall size of those columns + /// TODO: not sorting for now because the conditions are already sorted by Where Optimizer + + /// 4. Group conditions with the same set of columns into a single read/compute step + std::vector> condition_groups; + for (const auto & node : condition_nodes) + { + const auto & node_info = nodes_info[node]; + if (!condition_groups.empty() && nodes_info[condition_groups.back().back()].required_columns == node_info.required_columns) + condition_groups.back().push_back(node); /// Add to the last group + else + condition_groups.push_back({node}); /// Start new group + } + + /// 5. Build DAGs for each step + struct Step + { + ActionsDAGPtr actions; + String column_name; + }; + std::vector steps; + + OriginalToNewNodeMap node_remap; + + for (const auto & condition_group : condition_groups) + { +// std::cerr +// << "Conditions: ["; +// +// for (const auto & condition : condition_group) +// std::cerr << " \"" << condition->result_name; +// +// std::cerr << "\" ] Columns: " << boost::algorithm::join(nodes_info[condition_group.front()].required_columns, " ") +// << std::endl; + + ActionsDAGPtr step_dag = std::make_shared(); + String result_name; + + std::vector new_condition_nodes; + for (const auto * node : condition_group) + { + const auto & node_in_new_dag = addClonedDAGToDAG(node, step_dag, node_remap); + new_condition_nodes.push_back(&node_in_new_dag); + } + + if (new_condition_nodes.size() > 1) + { + /// Add AND function to combine the conditions + FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); + const auto & and_function_node = step_dag->addFunction(func_builder_and, new_condition_nodes, ""); + step_dag->addOrReplaceInOutputs(and_function_node); + result_name = and_function_node.result_name; + } + else + { + const auto & result_node = *new_condition_nodes.front(); + /// Add cast to UInt8 if needed + if (result_node.result_type->getTypeId() == TypeIndex::UInt8) + { + step_dag->addOrReplaceInOutputs(result_node); + result_name = result_node.result_name; + } + else + { + const auto & cast_node = addCast(step_dag, result_node, "UInt8"); + step_dag->addOrReplaceInOutputs(cast_node); + result_name = cast_node.result_name; + } + } + +// std::cerr << "Step DAG:\n" << step_dag->dumpDAG() << std::endl; + + steps.push_back({step_dag, result_name}); + } + + /// 6. Find all outputs of the original DAG + auto original_outputs = prewhere_info->prewhere_actions->getOutputs(); + /// 7. Find all outputs that were computed in the already built DAGs, mark these nodes as outputs in the steps where they were computed + /// 8. Add computation of the remaining outputs to the last step with the procedure similar to 4 + for (const auto * output : original_outputs) + { +// std::cerr << "Original output: " << output->result_name << std::endl; + if (node_remap.contains(output)) + { + const auto & new_node_info = node_remap[output]; + new_node_info.dag->addOrReplaceInOutputs(*new_node_info.node); + } + else if (output->result_name == prewhere_info->prewhere_column_name) + { + /// Special case for final PREWHERE column: + /// "Rename" the last step result to the combined PREWHERE column name, because in fact it will be AND of all step results + const auto & prewhere_result_node = addCast( + steps.back().actions, + steps.back().actions->findInOutputs(steps.back().column_name), + output->result_type->getName(), + prewhere_info->prewhere_column_name); + + steps.back().actions->addOrReplaceInOutputs(prewhere_result_node); + } + else + { + const auto & node_in_new_dag = addClonedDAGToDAG(output, steps.back().actions, node_remap); + steps.back().actions->addOrReplaceInOutputs(node_in_new_dag); + } + } + + /// 9. Build PrewhereExprInfo + { + for (const auto & step : steps) + { +// std::cerr << "Step DAG:\n" << step.actions->dumpDAG() << std::endl; + prewhere.steps.push_back( + { + .actions = std::make_shared(step.actions, actions_settings), + .column_name = step.column_name, + .remove_column = true, + .need_filter = false, + }); + } + prewhere.steps.back().remove_column = prewhere_info->remove_prewhere_column; + prewhere.steps.back().need_filter = prewhere_info->need_filter; + } + + return true; +} + std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, bool enable_multiple_prewhere_read_steps) { std::unique_ptr prewhere_actions; @@ -128,7 +378,23 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( prewhere_actions->steps.emplace_back(std::move(row_level_filter_step)); } - //std::cerr << "ORIGINAL PREWHERE:\n" << prewhere_info->prewhere_actions->dumpDAG() << std::endl; +// std::cerr << "ORIGINAL PREWHERE:\n" << prewhere_info->prewhere_actions->dumpDAG() << std::endl; + +#if 1 + if (!enable_multiple_prewhere_read_steps || !tryBuildPrewhereSteps(prewhere_info, actions_settings, *prewhere_actions)) + { + PrewhereExprStep prewhere_step + { + .actions = std::make_shared(prewhere_info->prewhere_actions, actions_settings), + .column_name = prewhere_info->prewhere_column_name, + .remove_column = prewhere_info->remove_prewhere_column, + .need_filter = prewhere_info->need_filter + }; + + prewhere_actions->steps.emplace_back(std::move(prewhere_step)); + } + +#else struct Step { @@ -243,6 +509,7 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( prewhere_actions->steps.emplace_back(std::move(prewhere_step)); } +#endif } return prewhere_actions; From 9817c5601bbb3330b53479ea956e457065b0b5a3 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 12 Feb 2023 22:25:33 +0100 Subject: [PATCH 058/253] Fixed clang tidy build by updating parameter name to common_backups_path - Use cluster state data to check concurrent backup/restore --- src/Backups/BackupCoordinationRemote.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index f613eb7d198..29514b8fe1f 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -595,23 +595,23 @@ Strings BackupCoordinationRemote::getAllArchiveSuffixes() const return node_names; } -bool BackupCoordinationRemote::hasConcurrentBackups(const String & backup_id, const String & common_backup_path, const std::atomic &) const +bool BackupCoordinationRemote::hasConcurrentBackups(const String & backup_id, const String & common_backups_path, const std::atomic &) const { /// If its internal concurrency will be checked for the base backup if (is_internal) return false; auto zk = getZooKeeper(); - std::string backup_stage_path = common_backup_path + "/backup-" + toString(backup_id) +"/stage"; + std::string backup_stage_path = common_backups_path + "/backup-" + toString(backup_id) +"/stage"; - if (!zk->exists(common_backup_path)) - zk->createAncestors(common_backup_path); + if (!zk->exists(common_backups_path)) + zk->createAncestors(common_backups_path); for (size_t attempt = 0; attempt < MAX_ZOOKEEPER_ATTEMPTS; ++attempt) { Coordination::Stat stat; - zk->get(common_backup_path, &stat); - Strings existing_backup_paths = zk->getChildren(common_backup_path); + zk->get(common_backups_path, &stat); + Strings existing_backup_paths = zk->getChildren(common_backups_path); for (const auto & existing_backup_path : existing_backup_paths) { @@ -624,7 +624,7 @@ bool BackupCoordinationRemote::hasConcurrentBackups(const String & backup_id, co if (existing_backup_id == toString(backup_id)) continue; - const auto status = zk->get(common_backup_path + "/" + existing_backup_path + "/stage"); + const auto status = zk->get(common_backups_path + "/" + existing_backup_path + "/stage"); if (status != Stage::COMPLETED) return true; } From a89465d1cb3512be4a440fb6e657e0294e5d5dbf Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Sun, 12 Feb 2023 22:48:28 +0100 Subject: [PATCH 059/253] Fix build after merge --- src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp | 6 +++++- src/Storages/MergeTree/MergeTreePrefetchedReadPool.h | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index d7cbf18c115..73de5685085 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -31,6 +31,7 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( RangesInDataParts && parts_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, + const ExpressionActionsSettings & actions_settings_, const Names & column_names_, const Names & virtual_column_names_, size_t preferred_block_size_bytes_, @@ -45,6 +46,8 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( virtual_column_names_, min_marks_for_concurrent_read_, prewhere_info_, + actions_settings_, + reader_settings_, parts_, (preferred_block_size_bytes_ > 0), /*do_not_steal_tasks_*/false) @@ -53,7 +56,6 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( , header(storage_snapshot_->getSampleBlockForColumns(column_names_)) , mark_cache(context_->getGlobalContext()->getMarkCache().get()) , uncompressed_cache(use_uncompressed_cache_ ? context_->getGlobalContext()->getUncompressedCache().get() : nullptr) - , reader_settings(reader_settings_) , profile_callback([this](ReadBufferFromFileBase::ProfileInfo info_) { profileFeedback(info_); }) , index_granularity_bytes(storage_settings_.index_granularity_bytes) , fixed_index_granularity(storage_settings_.index_granularity) @@ -326,6 +328,8 @@ MergeTreePrefetchedReadPool::PartsInfos MergeTreePrefetchedReadPool::getPartsInf column_names, virtual_column_names, prewhere_info, + actions_settings, + reader_settings, /* with_subcolumns */true); part_info->size_predictor = !predict_block_size_bytes diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h index ca12739ef6b..c3ac6c7c3e7 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h @@ -25,6 +25,7 @@ public: RangesInDataParts && parts_, const StorageSnapshotPtr & storage_snapshot_, const PrewhereInfoPtr & prewhere_info_, + const ExpressionActionsSettings & actions_settings_, const Names & column_names_, const Names & virtual_column_names_, size_t preferred_block_size_bytes_, @@ -80,7 +81,6 @@ private: Block header; MarkCache * mark_cache; UncompressedCache * uncompressed_cache; - MergeTreeReaderSettings reader_settings; ReadBufferFromFileBase::ProfileCallback profile_callback; size_t index_granularity_bytes; size_t fixed_index_granularity; From 3060bee18a321a2cda50de866206abfad827a588 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 13 Feb 2023 12:58:33 +0100 Subject: [PATCH 060/253] Properly handle constants in multiple steps --- .../MergeTree/MergeTreeBaseSelectProcessor.cpp | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 5f40950c3b2..c7f098dcb1c 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -145,6 +145,7 @@ using OriginalToNewNodeMap = std::unordered_mapaddOrReplaceInOutputs(*node_ref.node); - const auto & new_node = new_dag->addInput(node_ref.node->result_name, node_ref.node->result_type); - node_remap[original_dag_node] = {new_dag, &new_node}; /// TODO: here we update the node reference. Is ti always correct? - return new_node; + /// If the node is known from the previous steps, add it as an input, except for constants + if (original_dag_node->type != ActionsDAG::ActionType::COLUMN) + { + node_ref.dag->addOrReplaceInOutputs(*node_ref.node); + const auto & new_node = new_dag->addInput(node_ref.node->result_name, node_ref.node->result_type); + node_remap[original_dag_node] = {new_dag, &new_node}; /// TODO: here we update the node reference. Is ti always correct? + return new_node; + } } /// If the node is an input, add it as an input From d5f413304dfe2b9807c6bbef530f803c14e86bdf Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 13 Feb 2023 13:28:09 +0100 Subject: [PATCH 061/253] Store null map size into a variable --- src/Columns/ColumnNullable.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 99d377f10eb..139323f9770 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -786,16 +786,17 @@ ColumnPtr ColumnNullable::getNestedColumnWithDefaultOnNull() const auto res = nested_column->cloneEmpty(); const auto & null_map_data = getNullMapData(); size_t start = 0; + size_t end = null_map->size(); while (start < nested_column->size()) { size_t next_null_index = start; - while (next_null_index < null_map->size() && !null_map_data[next_null_index]) + while (next_null_index < end && !null_map_data[next_null_index]) ++next_null_index; if (next_null_index != start) res->insertRangeFrom(*nested_column, start, next_null_index - start); - if (next_null_index < null_map->size()) + if (next_null_index < end) res->insertDefault(); start = next_null_index + 1; From d67e7e47f5025744e4fdc291952303e646634684 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 13 Feb 2023 13:28:46 +0100 Subject: [PATCH 062/253] Update src/Core/Settings.h --- 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 481929d915f..7fec350f208 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -751,7 +751,7 @@ class IColumn; M(Bool, input_format_csv_empty_as_default, true, "Treat empty fields in CSV input as default values.", 0) \ M(Bool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.", 0) \ M(Bool, input_format_tsv_enum_as_number, false, "Treat inserted enum values in TSV formats as enum indices.", 0) \ - M(Bool, input_format_null_as_default, true, "For most input formats initialize null fields with default values if data type of this field is not nullable", 0) \ + M(Bool, input_format_null_as_default, true, "Initialize null fields with default values if the data type of this field is not nullable and it is supported by the input format", 0) \ M(Bool, input_format_arrow_import_nested, false, "Allow to insert array of structs into Nested table in Arrow input format.", 0) \ M(Bool, input_format_arrow_case_insensitive_column_matching, false, "Ignore case when matching Arrow columns with CH columns.", 0) \ M(Bool, input_format_orc_import_nested, false, "Allow to insert array of structs into Nested table in ORC input format.", 0) \ From c39cfee9548145f7436f62f51cd91a1ec33eb99e Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 13 Feb 2023 14:11:18 +0100 Subject: [PATCH 063/253] Properly set "remove" flag for step prewhere_columns when they are used in futher steps --- .../MergeTreeBaseSelectProcessor.cpp | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index c7f098dcb1c..fa8b5d40ff2 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -315,9 +315,11 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction auto original_outputs = prewhere_info->prewhere_actions->getOutputs(); /// 7. Find all outputs that were computed in the already built DAGs, mark these nodes as outputs in the steps where they were computed /// 8. Add computation of the remaining outputs to the last step with the procedure similar to 4 + NameSet all_output_names; for (const auto * output : original_outputs) { // std::cerr << "Original output: " << output->result_name << std::endl; + all_output_names.insert(output->result_name); if (node_remap.contains(output)) { const auto & new_node_info = node_remap[output]; @@ -325,14 +327,12 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction } else if (output->result_name == prewhere_info->prewhere_column_name) { - /// Special case for final PREWHERE column: - /// "Rename" the last step result to the combined PREWHERE column name, because in fact it will be AND of all step results - const auto & prewhere_result_node = addCast( - steps.back().actions, - steps.back().actions->findInOutputs(steps.back().column_name), - output->result_type->getName(), - prewhere_info->prewhere_column_name); - + /// Special case for final PREWHERE column: it is an AND combination of all conditions, + /// but we have only the condition for the last step here. + /// However we know that the ultimate result after filtering is constant 1 for the PREWHERE column. + auto const_true = output->result_type->createColumnConst(0, Field{1}); + const auto & prewhere_result_node = + steps.back().actions->addColumn(ColumnWithTypeAndName(const_true, output->result_type, output->result_name)); steps.back().actions->addOrReplaceInOutputs(prewhere_result_node); } else @@ -351,11 +351,10 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction { .actions = std::make_shared(step.actions, actions_settings), .column_name = step.column_name, - .remove_column = true, + .remove_column = !all_output_names.contains(step.column_name), /// Don't remove if it's in the list of original outputs .need_filter = false, }); } - prewhere.steps.back().remove_column = prewhere_info->remove_prewhere_column; prewhere.steps.back().need_filter = prewhere_info->need_filter; } @@ -385,7 +384,8 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( // std::cerr << "ORIGINAL PREWHERE:\n" << prewhere_info->prewhere_actions->dumpDAG() << std::endl; #if 1 - if (!enable_multiple_prewhere_read_steps || !tryBuildPrewhereSteps(prewhere_info, actions_settings, *prewhere_actions)) + if (!enable_multiple_prewhere_read_steps || + !tryBuildPrewhereSteps(prewhere_info, actions_settings, *prewhere_actions)) { PrewhereExprStep prewhere_step { From 39e4ca9850fd488ba21994960daaccb85d695558 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 13 Feb 2023 14:40:54 +0100 Subject: [PATCH 064/253] Cleanup --- src/Interpreters/ActionsDAG.cpp | 20 +-- src/Interpreters/ActionsDAG.h | 14 -- .../MergeTreeBaseSelectProcessor.cpp | 121 ------------------ 3 files changed, 10 insertions(+), 145 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 913a891c6bb..160772d38ea 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1663,20 +1663,20 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsForFilter(const std::string & co return res; } -//namespace -//{ -// -//struct ConjunctionNodes -//{ -// ActionsDAG::NodeRawConstPtrs allowed; -// ActionsDAG::NodeRawConstPtrs rejected; -//}; +namespace +{ + +struct ConjunctionNodes +{ + ActionsDAG::NodeRawConstPtrs allowed; + ActionsDAG::NodeRawConstPtrs rejected; +}; /// Take a node which result is predicate. /// Assuming predicate is a conjunction (probably, trivial). /// Find separate conjunctions nodes. Split nodes into allowed and rejected sets. /// Allowed predicate is a predicate which can be calculated using only nodes from allowed_nodes set. -ConjunctionNodes getConjunctionNodes(const ActionsDAG::Node * predicate, std::unordered_set allowed_nodes) +ConjunctionNodes getConjunctionNodes(ActionsDAG::Node * predicate, std::unordered_set allowed_nodes) { ConjunctionNodes conjunction; std::unordered_set allowed; @@ -1798,7 +1798,7 @@ ColumnsWithTypeAndName prepareFunctionArguments(const ActionsDAG::NodeRawConstPt return arguments; } -//} +} /// Create actions which calculate conjunction of selected nodes. /// Assume conjunction nodes are predicates (and may be used as arguments of function AND). diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 98c5d36c69a..40bc76fe057 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -363,7 +363,6 @@ private: void compileFunctions(size_t min_count_to_compile_expression, const std::unordered_set & lazy_executed_nodes = {}); #endif -public: static ActionsDAGPtr cloneActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs); }; @@ -373,17 +372,4 @@ struct ActionDAGNodes ActionsDAG::NodeRawConstPtrs nodes; }; -struct ConjunctionNodes -{ - ActionsDAG::NodeRawConstPtrs allowed; - ActionsDAG::NodeRawConstPtrs rejected; -}; - -/// Take a node which result is predicate. -/// Assuming predicate is a conjunction (probably, trivial). -/// Find separate conjunctions nodes. Split nodes into allowed and rejected sets. -/// Allowed predicate is a predicate which can be calculated using only nodes from allowed_nodes set. -ConjunctionNodes getConjunctionNodes(const ActionsDAG::Node * predicate, std::unordered_set allowed_nodes); - - } diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index fa8b5d40ff2..3280ec6e487 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -13,9 +13,6 @@ #include #include -#include - -/// For CAST to bool #include #include #include @@ -383,7 +380,6 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( // std::cerr << "ORIGINAL PREWHERE:\n" << prewhere_info->prewhere_actions->dumpDAG() << std::endl; -#if 1 if (!enable_multiple_prewhere_read_steps || !tryBuildPrewhereSteps(prewhere_info, actions_settings, *prewhere_actions)) { @@ -397,123 +393,6 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( prewhere_actions->steps.emplace_back(std::move(prewhere_step)); } - -#else - - struct Step - { - ActionsDAGPtr actions; - String column_name; - }; - std::vector steps; - - if (enable_multiple_prewhere_read_steps) - { - /// Find all conjunctions in prewhere expression. - auto conjunctions = getConjunctionNodes( - prewhere_info->prewhere_actions->tryFindInOutputs(prewhere_info->prewhere_column_name), - {}); - - /// Save the list of inputs to the original prewhere expression. - auto inputs = prewhere_info->prewhere_actions->getInputs(); - ColumnsWithTypeAndName all_inputs; - for (const auto & input : inputs) - all_inputs.emplace_back(input->column, input->result_type, input->result_name); - - ActionsDAG::NodeRawConstPtrs all_conjunctions = std::move(conjunctions.allowed); - all_conjunctions.insert(all_conjunctions.end(), conjunctions.rejected.begin(), conjunctions.rejected.end()); - - /// Make separate DAG for each step - for (const auto & conjunction : all_conjunctions) - { - auto result_name = conjunction->result_name; - auto step_dag = ActionsDAG::cloneActionsForConjunction({conjunction}, all_inputs); - const auto & result_node = step_dag->findInOutputs(result_name); - /// Cast result to UInt8 if needed - if (result_node.result_type->getTypeId() != TypeIndex::UInt8) - { - const auto & cast_node = addCast(step_dag, result_node, "UInt8"); - - step_dag->addOrReplaceInOutputs(cast_node); - result_name = cast_node.result_name; - } - step_dag->removeUnusedActions(Names{result_name}, true, true); - steps.emplace_back(Step{step_dag, result_name}); - } - } - - if (steps.size() > 1) - { - /// Save the list of outputs from the original prewhere expression. - auto original_outputs = prewhere_info->prewhere_actions->getOutputs(); - std::unordered_map outputs_required_by_next_steps; - for (const auto & output : original_outputs) - outputs_required_by_next_steps[output->result_name] = output->result_type; - - /// "Rename" the last step result to the combined prewhere column name, because in fact it will be AND of all step results - if (steps.back().column_name != prewhere_info->prewhere_column_name && - outputs_required_by_next_steps.contains(prewhere_info->prewhere_column_name)) - { - const auto & prewhere_result_node = addCast( - steps.back().actions, - steps.back().actions->findInOutputs(steps.back().column_name), - outputs_required_by_next_steps[prewhere_info->prewhere_column_name]->getName(), - prewhere_info->prewhere_column_name); - - steps.back().actions->addOrReplaceInOutputs(prewhere_result_node); - } - - const size_t steps_before_prewhere = prewhere_actions->steps.size(); - prewhere_actions->steps.resize(steps_before_prewhere + steps.size()); - - /// Check the steps in the reverse order so that we can maintain the list of outputs used by the next steps - /// and preserve them in the current step. - for (ssize_t i = steps.size() - 1; i >= 0; --i) - { - const auto & step = steps[i]; - - /// Return the condition column - Names step_outputs{step.column_name}; - const bool remove_column = !outputs_required_by_next_steps.contains(step.column_name); - /// Preserve outputs computed at this step that are used by the next steps - for (const auto & output : outputs_required_by_next_steps) - if (step.actions->tryRestoreColumn(output.first)) - step_outputs.emplace_back(output.first); - step.actions->removeUnusedActions(step_outputs, true, true); - - /// Add current step columns as outputs that should be preserved from previous steps - for (const auto & input :step.actions->getInputs()) - outputs_required_by_next_steps[input->result_name] = input->result_type; - - //std::cerr << conjunction->result_name << "\n"; - //std::cerr << "STEP " << i << ":\n" << step.actions->dumpDAG() << "\n"; - - PrewhereExprStep prewhere_step - { - .actions = std::make_shared(step.actions, actions_settings), - .column_name = step.column_name, - .remove_column = remove_column, - .need_filter = false - }; - prewhere_actions->steps[steps_before_prewhere + i] = std::move(prewhere_step); - } - - prewhere_actions->steps.back().remove_column = prewhere_info->remove_prewhere_column; - prewhere_actions->steps.back().need_filter = prewhere_info->need_filter; - } - else - { - PrewhereExprStep prewhere_step - { - .actions = std::make_shared(prewhere_info->prewhere_actions, actions_settings), - .column_name = prewhere_info->prewhere_column_name, - .remove_column = prewhere_info->remove_prewhere_column, - .need_filter = prewhere_info->need_filter - }; - - prewhere_actions->steps.emplace_back(std::move(prewhere_step)); - } -#endif } return prewhere_actions; From f0011862a984384c80de005f4dae1c2a554965ad Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 13 Feb 2023 15:28:54 +0100 Subject: [PATCH 065/253] Move prewhere splitting logic into a separate file --- .../MergeTreeBaseSelectProcessor.cpp | 278 +--------------- .../MergeTreeSplitPrewhereIntoReadSteps.cpp | 302 ++++++++++++++++++ 2 files changed, 303 insertions(+), 277 deletions(-) create mode 100644 src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 3280ec6e487..fe649808cbe 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -12,11 +12,6 @@ #include #include #include - -#include -#include -#include - #include namespace ProfileEvents @@ -87,276 +82,7 @@ IMergeTreeSelectAlgorithm::IMergeTreeSelectAlgorithm( LOG_TEST(log, "PREWHERE actions: {}", (prewhere_actions ? prewhere_actions->dump() : std::string(""))); } -/// Adds a CAST node with the regular name ("CAST(...)") or with the provided name. -/// This is different from ActionsDAG::addCast() because it set the name equal to the original name effectively hiding the value before cast, -/// but it might be required for further steps with its original uncasted type. -static const ActionsDAG::Node & addCast(ActionsDAGPtr dag, const ActionsDAG::Node & node_to_cast, const String & type_name, const String & new_name = {}) -{ - Field cast_type_constant_value(type_name); - - ColumnWithTypeAndName column; - column.name = calculateConstantActionNodeName(cast_type_constant_value); - column.column = DataTypeString().createColumnConst(0, cast_type_constant_value); - column.type = std::make_shared(); - - const auto * cast_type_constant_node = &dag->addColumn(std::move(column)); - ActionsDAG::NodeRawConstPtrs children = {&node_to_cast, cast_type_constant_node}; - FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver::createImpl(); - - return dag->addFunction(func_builder_cast, std::move(children), new_name); -}; - -struct NodeInfo -{ - NameSet required_columns; -}; - -void fillRequiredColumns(const ActionsDAG::Node * node, std::unordered_map & nodes_info) -{ - if (nodes_info.contains(node)) - return; - - auto & node_info = nodes_info[node]; - - if (node->type == ActionsDAG::ActionType::INPUT) - { - node_info.required_columns.insert(node->result_name); - return; - } - - for (const auto & child : node->children) - { - fillRequiredColumns(child, nodes_info); - const auto & child_info = nodes_info[child]; - node_info.required_columns.insert(child_info.required_columns.begin(), child_info.required_columns.end()); - } -} - -struct DAGNodeRef -{ - ActionsDAGPtr dag; - const ActionsDAG::Node * node; -}; - -using OriginalToNewNodeMap = std::unordered_map; - -const ActionsDAG::Node & addClonedDAGToDAG(const ActionsDAG::Node * original_dag_node, ActionsDAGPtr new_dag, OriginalToNewNodeMap & node_remap) -{ - /// Look for the node in the map of already known nodes - if (node_remap.contains(original_dag_node)) - { - /// If the node is already in the new DAG, return it - const auto & node_ref = node_remap.at(original_dag_node); - if (node_ref.dag == new_dag) - return *node_ref.node; - - /// If the node is known from the previous steps, add it as an input, except for constants - if (original_dag_node->type != ActionsDAG::ActionType::COLUMN) - { - node_ref.dag->addOrReplaceInOutputs(*node_ref.node); - const auto & new_node = new_dag->addInput(node_ref.node->result_name, node_ref.node->result_type); - node_remap[original_dag_node] = {new_dag, &new_node}; /// TODO: here we update the node reference. Is ti always correct? - return new_node; - } - } - - /// If the node is an input, add it as an input - if (original_dag_node->type == ActionsDAG::ActionType::INPUT) - { - const auto & new_node = new_dag->addInput(original_dag_node->result_name, original_dag_node->result_type); - node_remap[original_dag_node] = {new_dag, &new_node}; - return new_node; - } - - /// If the node is a column, add it as an input - if (original_dag_node->type == ActionsDAG::ActionType::COLUMN) - { - const auto & new_node = new_dag->addColumn( - ColumnWithTypeAndName(original_dag_node->column, original_dag_node->result_type, original_dag_node->result_name)); - node_remap[original_dag_node] = {new_dag, &new_node}; - return new_node; - } - - /// TODO: Alias node? - - /// If the node is a function, add it as a function and add its children - if (original_dag_node->type == ActionsDAG::ActionType::FUNCTION) - { - ActionsDAG::NodeRawConstPtrs new_children; - for (const auto & child : original_dag_node->children) - { - const auto & new_child = addClonedDAGToDAG(child, new_dag, node_remap); - new_children.push_back(&new_child); - } - - const auto & new_node = new_dag->addFunction(original_dag_node->function_base, new_children, original_dag_node->result_name); - node_remap[original_dag_node] = {new_dag, &new_node}; - return new_node; - } - - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected node type in PREWHERE actions: {}", original_dag_node->type); -} - -bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, PrewhereExprInfo & prewhere) -{ - /// We want to build a sequence of steps that will compute parts of the prewhere condition. - /// Each step reads some new columns and computes some new expressions and a filter condition. - /// The last step computes the final filter condition and the remaining expressions that are required for the main query. - /// The steps are built in the following way: - /// 1. List all condition nodes that are combined with AND into PREWHERE condition - /// 2. Collect the set of columns that are used in the condition - /// 3. Sort condition nodes by the number of columns used in them and the overall size of those columns - /// 4. Group conditions with the same set of columns into a single read/compute step - /// 5. Build DAGs for each step: - /// - DFS from the condition root node: - /// - If the node was not computed yet, add it to the DAG and traverse its children - /// - If the node was already computed by one of the previous steps, add it as output for that step and as input for the current step - /// - If the node was already computed by the current step just stop traversing - /// 6. Find all outputs of the original DAG - /// 7. Find all outputs that were computed in the already built DAGs, mark these nodes as outputs in the steps where they were computed - /// 8. Add computation of the remaining outputs to the last step with the procedure similar to 4 - - if (!prewhere_info || !prewhere_info->prewhere_actions) - return true; - - /// 1. List all condition nodes that are combined with AND into PREWHERE condition - const auto & condition_root = prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name); - const bool is_conjunction = (condition_root.type == ActionsDAG::ActionType::FUNCTION && condition_root.function_base->getName() == "and"); - if (!is_conjunction) - return false; - auto condition_nodes = condition_root.children; - - /// 2. Collect the set of columns that are used in the condition - std::unordered_map nodes_info; - for (const auto & node : condition_nodes) - { - fillRequiredColumns(node, nodes_info); - } - - /// 3. Sort condition nodes by the number of columns used in them and the overall size of those columns - /// TODO: not sorting for now because the conditions are already sorted by Where Optimizer - - /// 4. Group conditions with the same set of columns into a single read/compute step - std::vector> condition_groups; - for (const auto & node : condition_nodes) - { - const auto & node_info = nodes_info[node]; - if (!condition_groups.empty() && nodes_info[condition_groups.back().back()].required_columns == node_info.required_columns) - condition_groups.back().push_back(node); /// Add to the last group - else - condition_groups.push_back({node}); /// Start new group - } - - /// 5. Build DAGs for each step - struct Step - { - ActionsDAGPtr actions; - String column_name; - }; - std::vector steps; - - OriginalToNewNodeMap node_remap; - - for (const auto & condition_group : condition_groups) - { -// std::cerr -// << "Conditions: ["; -// -// for (const auto & condition : condition_group) -// std::cerr << " \"" << condition->result_name; -// -// std::cerr << "\" ] Columns: " << boost::algorithm::join(nodes_info[condition_group.front()].required_columns, " ") -// << std::endl; - - ActionsDAGPtr step_dag = std::make_shared(); - String result_name; - - std::vector new_condition_nodes; - for (const auto * node : condition_group) - { - const auto & node_in_new_dag = addClonedDAGToDAG(node, step_dag, node_remap); - new_condition_nodes.push_back(&node_in_new_dag); - } - - if (new_condition_nodes.size() > 1) - { - /// Add AND function to combine the conditions - FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); - const auto & and_function_node = step_dag->addFunction(func_builder_and, new_condition_nodes, ""); - step_dag->addOrReplaceInOutputs(and_function_node); - result_name = and_function_node.result_name; - } - else - { - const auto & result_node = *new_condition_nodes.front(); - /// Add cast to UInt8 if needed - if (result_node.result_type->getTypeId() == TypeIndex::UInt8) - { - step_dag->addOrReplaceInOutputs(result_node); - result_name = result_node.result_name; - } - else - { - const auto & cast_node = addCast(step_dag, result_node, "UInt8"); - step_dag->addOrReplaceInOutputs(cast_node); - result_name = cast_node.result_name; - } - } - -// std::cerr << "Step DAG:\n" << step_dag->dumpDAG() << std::endl; - - steps.push_back({step_dag, result_name}); - } - - /// 6. Find all outputs of the original DAG - auto original_outputs = prewhere_info->prewhere_actions->getOutputs(); - /// 7. Find all outputs that were computed in the already built DAGs, mark these nodes as outputs in the steps where they were computed - /// 8. Add computation of the remaining outputs to the last step with the procedure similar to 4 - NameSet all_output_names; - for (const auto * output : original_outputs) - { -// std::cerr << "Original output: " << output->result_name << std::endl; - all_output_names.insert(output->result_name); - if (node_remap.contains(output)) - { - const auto & new_node_info = node_remap[output]; - new_node_info.dag->addOrReplaceInOutputs(*new_node_info.node); - } - else if (output->result_name == prewhere_info->prewhere_column_name) - { - /// Special case for final PREWHERE column: it is an AND combination of all conditions, - /// but we have only the condition for the last step here. - /// However we know that the ultimate result after filtering is constant 1 for the PREWHERE column. - auto const_true = output->result_type->createColumnConst(0, Field{1}); - const auto & prewhere_result_node = - steps.back().actions->addColumn(ColumnWithTypeAndName(const_true, output->result_type, output->result_name)); - steps.back().actions->addOrReplaceInOutputs(prewhere_result_node); - } - else - { - const auto & node_in_new_dag = addClonedDAGToDAG(output, steps.back().actions, node_remap); - steps.back().actions->addOrReplaceInOutputs(node_in_new_dag); - } - } - - /// 9. Build PrewhereExprInfo - { - for (const auto & step : steps) - { -// std::cerr << "Step DAG:\n" << step.actions->dumpDAG() << std::endl; - prewhere.steps.push_back( - { - .actions = std::make_shared(step.actions, actions_settings), - .column_name = step.column_name, - .remove_column = !all_output_names.contains(step.column_name), /// Don't remove if it's in the list of original outputs - .need_filter = false, - }); - } - prewhere.steps.back().need_filter = prewhere_info->need_filter; - } - - return true; -} +bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, PrewhereExprInfo & prewhere); std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, bool enable_multiple_prewhere_read_steps) { @@ -378,8 +104,6 @@ std::unique_ptr IMergeTreeSelectAlgorithm::getPrewhereActions( prewhere_actions->steps.emplace_back(std::move(row_level_filter_step)); } -// std::cerr << "ORIGINAL PREWHERE:\n" << prewhere_info->prewhere_actions->dumpDAG() << std::endl; - if (!enable_multiple_prewhere_read_steps || !tryBuildPrewhereSteps(prewhere_info, actions_settings, *prewhere_actions)) { diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp new file mode 100644 index 00000000000..8284bda34a7 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -0,0 +1,302 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace +{ + +/// Stores the ist of columns required to compute a node in the DAG. +struct NodeInfo +{ + NameSet required_columns; +}; + +/// Fills the list of required columns for a node in the DAG. +void fillRequiredColumns(const ActionsDAG::Node * node, std::unordered_map & nodes_info) +{ + if (nodes_info.contains(node)) + return; + + auto & node_info = nodes_info[node]; + + if (node->type == ActionsDAG::ActionType::INPUT) + { + node_info.required_columns.insert(node->result_name); + return; + } + + for (const auto & child : node->children) + { + fillRequiredColumns(child, nodes_info); + const auto & child_info = nodes_info[child]; + node_info.required_columns.insert(child_info.required_columns.begin(), child_info.required_columns.end()); + } +} + +/// Stores information about a node that has already been cloned to one of the new DAGs. +/// This allows to avoid cloning the same sub-DAG into multiple step DAGs but reference previously cloned nodes from earliers steps. +struct DAGNodeRef +{ + ActionsDAGPtr dag; + const ActionsDAG::Node * node; +}; + +using OriginalToNewNodeMap = std::unordered_map; + +/// Clones the part of original DAG responsible for computing the original_dag_node and adds it to the new DAG. +const ActionsDAG::Node & addClonedDAGToDAG(const ActionsDAG::Node * original_dag_node, ActionsDAGPtr new_dag, OriginalToNewNodeMap & node_remap) +{ + /// Look for the node in the map of already known nodes + if (node_remap.contains(original_dag_node)) + { + /// If the node is already in the new DAG, return it + const auto & node_ref = node_remap.at(original_dag_node); + if (node_ref.dag == new_dag) + return *node_ref.node; + + /// If the node is known from the previous steps, add it as an input, except for constants + if (original_dag_node->type != ActionsDAG::ActionType::COLUMN) + { + node_ref.dag->addOrReplaceInOutputs(*node_ref.node); + const auto & new_node = new_dag->addInput(node_ref.node->result_name, node_ref.node->result_type); + node_remap[original_dag_node] = {new_dag, &new_node}; /// TODO: here we update the node reference. Is it always correct? + return new_node; + } + } + + /// If the node is an input, add it as an input + if (original_dag_node->type == ActionsDAG::ActionType::INPUT) + { + const auto & new_node = new_dag->addInput(original_dag_node->result_name, original_dag_node->result_type); + node_remap[original_dag_node] = {new_dag, &new_node}; + return new_node; + } + + /// If the node is a column, add it as an input + if (original_dag_node->type == ActionsDAG::ActionType::COLUMN) + { + const auto & new_node = new_dag->addColumn( + ColumnWithTypeAndName(original_dag_node->column, original_dag_node->result_type, original_dag_node->result_name)); + node_remap[original_dag_node] = {new_dag, &new_node}; + return new_node; + } + + /// TODO: Do we need to handle ALIAS nodes in cloning? + + /// If the node is a function, add it as a function and add its children + if (original_dag_node->type == ActionsDAG::ActionType::FUNCTION) + { + ActionsDAG::NodeRawConstPtrs new_children; + for (const auto & child : original_dag_node->children) + { + const auto & new_child = addClonedDAGToDAG(child, new_dag, node_remap); + new_children.push_back(&new_child); + } + + const auto & new_node = new_dag->addFunction(original_dag_node->function_base, new_children, original_dag_node->result_name); + node_remap[original_dag_node] = {new_dag, &new_node}; + return new_node; + } + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected node type in PREWHERE actions: {}", original_dag_node->type); +} + +/// Adds a CAST node with the regular name ("CAST(...)") or with the provided name. +/// This is different from ActionsDAG::addCast() because it set the name equal to the original name effectively hiding the value before cast, +/// but it might be required for further steps with its original uncasted type. +const ActionsDAG::Node & addCast(ActionsDAGPtr dag, const ActionsDAG::Node & node_to_cast, const String & type_name, const String & new_name = {}) +{ + Field cast_type_constant_value(type_name); + + ColumnWithTypeAndName column; + column.name = calculateConstantActionNodeName(cast_type_constant_value); + column.column = DataTypeString().createColumnConst(0, cast_type_constant_value); + column.type = std::make_shared(); + + const auto * cast_type_constant_node = &dag->addColumn(std::move(column)); + ActionsDAG::NodeRawConstPtrs children = {&node_to_cast, cast_type_constant_node}; + FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver::createImpl(); + + return dag->addFunction(func_builder_cast, std::move(children), new_name); +} + +} + +/// We want to build a sequence of steps that will compute parts of the prewhere condition. +/// Each step reads some new columns and computes some new expressions and a filter condition. +/// The last step computes the final filter condition and the remaining expressions that are required for the main query. +/// The goal of this is to, when it is possible, filter out many rows in early steps so that the remaining steps will +/// read less data from the storage. +/// NOTE: The result of executing the steps is exactly the same as if we would execute the original DAG in single step. +/// +/// The steps are built in the following way: +/// 1. List all condition nodes that are combined with AND into PREWHERE condition +/// 2. Collect the set of columns that are used in each condition +/// 3. Sort condition nodes by the number of columns used in them and the overall size of those columns +/// 4. Group conditions with the same set of columns into a single read/compute step +/// 5. Build DAGs for each step: +/// - DFS from the condition root node: +/// - If the node was not computed yet, add it to the DAG and traverse its children +/// - If the node was already computed by one of the previous steps, add it as output for that step and as input for the current step +/// - If the node was already computed by the current step just stop traversing +/// 6. Find all outputs of the original DAG +/// 7. Find all outputs that were computed in the already built DAGs, mark these nodes as outputs in the steps where they were computed +/// 8. Add computation of the remaining outputs to the last step with the procedure similar to 4 +bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, PrewhereExprInfo & prewhere) +{ + if (!prewhere_info || !prewhere_info->prewhere_actions) + return true; + +// std::cerr << "ORIGINAL PREWHERE:\n" << prewhere_info->prewhere_actions->dumpDAG() << std::endl; + + /// 1. List all condition nodes that are combined with AND into PREWHERE condition + const auto & condition_root = prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name); + const bool is_conjunction = (condition_root.type == ActionsDAG::ActionType::FUNCTION && condition_root.function_base->getName() == "and"); + if (!is_conjunction) + return false; + auto condition_nodes = condition_root.children; + + /// 2. Collect the set of columns that are used in the condition + std::unordered_map nodes_info; + for (const auto & node : condition_nodes) + { + fillRequiredColumns(node, nodes_info); + } + + /// 3. Sort condition nodes by the number of columns used in them and the overall size of those columns + /// TODO: not sorting for now because the conditions are already sorted by Where Optimizer + + /// 4. Group conditions with the same set of columns into a single read/compute step + std::vector> condition_groups; + for (const auto & node : condition_nodes) + { + const auto & node_info = nodes_info[node]; + if (!condition_groups.empty() && nodes_info[condition_groups.back().back()].required_columns == node_info.required_columns) + condition_groups.back().push_back(node); /// Add to the last group + else + condition_groups.push_back({node}); /// Start new group + } + + /// 5. Build DAGs for each step + struct Step + { + ActionsDAGPtr actions; + String column_name; + }; + std::vector steps; + + OriginalToNewNodeMap node_remap; + + for (const auto & condition_group : condition_groups) + { +// std::cerr +// << "Conditions: ["; +// +// for (const auto & condition : condition_group) +// std::cerr << " \"" << condition->result_name; +// +// std::cerr << "\" ] Columns: " << boost::algorithm::join(nodes_info[condition_group.front()].required_columns, " ") +// << std::endl; + + ActionsDAGPtr step_dag = std::make_shared(); + String result_name; + + std::vector new_condition_nodes; + for (const auto * node : condition_group) + { + const auto & node_in_new_dag = addClonedDAGToDAG(node, step_dag, node_remap); + new_condition_nodes.push_back(&node_in_new_dag); + } + + if (new_condition_nodes.size() > 1) + { + /// Add AND function to combine the conditions + FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); + const auto & and_function_node = step_dag->addFunction(func_builder_and, new_condition_nodes, ""); + step_dag->addOrReplaceInOutputs(and_function_node); + result_name = and_function_node.result_name; + } + else + { + const auto & result_node = *new_condition_nodes.front(); + /// Add cast to UInt8 if needed + if (result_node.result_type->getTypeId() == TypeIndex::UInt8) + { + step_dag->addOrReplaceInOutputs(result_node); + result_name = result_node.result_name; + } + else + { + const auto & cast_node = addCast(step_dag, result_node, "UInt8"); + step_dag->addOrReplaceInOutputs(cast_node); + result_name = cast_node.result_name; + } + } + +// std::cerr << "Step DAG:\n" << step_dag->dumpDAG() << std::endl; + + steps.push_back({step_dag, result_name}); + } + + /// 6. Find all outputs of the original DAG + auto original_outputs = prewhere_info->prewhere_actions->getOutputs(); + /// 7. Find all outputs that were computed in the already built DAGs, mark these nodes as outputs in the steps where they were computed + /// 8. Add computation of the remaining outputs to the last step with the procedure similar to 4 + NameSet all_output_names; + for (const auto * output : original_outputs) + { +// std::cerr << "Original output: " << output->result_name << std::endl; + all_output_names.insert(output->result_name); + if (node_remap.contains(output)) + { + const auto & new_node_info = node_remap[output]; + new_node_info.dag->addOrReplaceInOutputs(*new_node_info.node); + } + else if (output->result_name == prewhere_info->prewhere_column_name) + { + /// Special case for final PREWHERE column: it is an AND combination of all conditions, + /// but we have only the condition for the last step here. + /// However we know that the ultimate result after filtering is constant 1 for the PREWHERE column. + auto const_true = output->result_type->createColumnConst(0, Field{1}); + const auto & prewhere_result_node = + steps.back().actions->addColumn(ColumnWithTypeAndName(const_true, output->result_type, output->result_name)); + steps.back().actions->addOrReplaceInOutputs(prewhere_result_node); + } + else + { + const auto & node_in_new_dag = addClonedDAGToDAG(output, steps.back().actions, node_remap); + steps.back().actions->addOrReplaceInOutputs(node_in_new_dag); + } + } + + /// 9. Build PrewhereExprInfo + { + for (const auto & step : steps) + { +// std::cerr << "Step DAG:\n" << step.actions->dumpDAG() << std::endl; + prewhere.steps.push_back( + { + .actions = std::make_shared(step.actions, actions_settings), + .column_name = step.column_name, + .remove_column = !all_output_names.contains(step.column_name), /// Don't remove if it's in the list of original outputs + .need_filter = false, + }); + } + prewhere.steps.back().need_filter = prewhere_info->need_filter; + } + + return true; +} + +} From 18a29e81fc20c791fd77671c8129f27b1202eefd Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 13 Feb 2023 17:42:27 +0100 Subject: [PATCH 066/253] Better formatting in PrewhereExprInfo::dump() --- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index f5afc0b37d6..4036d352a54 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #ifdef __SSE2__ #include @@ -1370,13 +1371,16 @@ std::string PrewhereExprInfo::dump() const { WriteBufferFromOwnString s; + const char indent[] = "\n "; for (size_t i = 0; i < steps.size(); ++i) { s << "STEP " << i << ":\n" - << " ACTIONS: " << (steps[i].actions ? steps[i].actions->dumpActions() : "nullptr") << "\n" + << " ACTIONS: " << (steps[i].actions ? + (indent + boost::replace_all_copy(steps[i].actions->dumpActions(), "\n", indent)) : + "nullptr") << "\n" << " COLUMN: " << steps[i].column_name << "\n" << " REMOVE_COLUMN: " << steps[i].remove_column << "\n" - << " NEED_FILTER: " << steps[i].need_filter << "\n"; + << " NEED_FILTER: " << steps[i].need_filter << "\n\n"; } return s.str(); From 0fe080cc6f7a3432c19a044bd869c59bf633d393 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 13 Feb 2023 17:43:41 +0100 Subject: [PATCH 067/253] Logger instead of debug prints --- .../MergeTreeSplitPrewhereIntoReadSteps.cpp | 19 +++++-------------- 1 file changed, 5 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp index 8284bda34a7..90ecf3e25f5 100644 --- a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -158,7 +158,9 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction if (!prewhere_info || !prewhere_info->prewhere_actions) return true; -// std::cerr << "ORIGINAL PREWHERE:\n" << prewhere_info->prewhere_actions->dumpDAG() << std::endl; + Poco::Logger * log = &Poco::Logger::get("tryBuildPrewhereSteps"); + + LOG_TRACE(log, "Original PREWHERE DAG:\n{}", prewhere_info->prewhere_actions->dumpDAG()); /// 1. List all condition nodes that are combined with AND into PREWHERE condition const auto & condition_root = prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name); @@ -200,15 +202,6 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction for (const auto & condition_group : condition_groups) { -// std::cerr -// << "Conditions: ["; -// -// for (const auto & condition : condition_group) -// std::cerr << " \"" << condition->result_name; -// -// std::cerr << "\" ] Columns: " << boost::algorithm::join(nodes_info[condition_group.front()].required_columns, " ") -// << std::endl; - ActionsDAGPtr step_dag = std::make_shared(); String result_name; @@ -244,8 +237,6 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction } } -// std::cerr << "Step DAG:\n" << step_dag->dumpDAG() << std::endl; - steps.push_back({step_dag, result_name}); } @@ -256,7 +247,6 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction NameSet all_output_names; for (const auto * output : original_outputs) { -// std::cerr << "Original output: " << output->result_name << std::endl; all_output_names.insert(output->result_name); if (node_remap.contains(output)) { @@ -284,7 +274,6 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction { for (const auto & step : steps) { -// std::cerr << "Step DAG:\n" << step.actions->dumpDAG() << std::endl; prewhere.steps.push_back( { .actions = std::make_shared(step.actions, actions_settings), @@ -296,6 +285,8 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction prewhere.steps.back().need_filter = prewhere_info->need_filter; } + LOG_TRACE(log, "Resulting PREWHERE:\n{}", prewhere.dump()); + return true; } From 67565aaa792ed38f2d5662b466ce4de6c99bc30e Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 13 Feb 2023 22:50:22 +0100 Subject: [PATCH 068/253] A test for combined condition node --- .../02559_multiple_read_steps_in_prewhere.reference | 6 ++++++ .../02559_multiple_read_steps_in_prewhere.sql | 12 ++++++++++++ 2 files changed, 18 insertions(+) diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference index 0c46a90257d..f2da126fc92 100644 --- a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference +++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference @@ -38,3 +38,9 @@ SELECT cast(id1 as UInt16) AS cond1, (id2 % 40000) AS cond2, (cond1 AND cond2) A SELECT cast(id1 as UInt16) AS cond1 FROM test_02559 PREWHERE cond1 LIMIT 10; -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } SELECT count() FROM test_02559 PREWHERE id2>=0 AND (1 OR ignore(id1)) WHERE ignore(id1)=0; 10 +CREATE ROW POLICY 02559_filter_1 ON test_02559 USING id2=2 AS permissive TO ALL; +SELECT * FROM test_02559; +2 2 +CREATE ROW POLICY 02559_filter_2 ON test_02559 USING id2<=2 AS restrictive TO ALL; +SELECT * FROM test_02559; +2 2 diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql index e39f8ac846a..0b434eae7df 100644 --- a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql +++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql @@ -4,6 +4,9 @@ CREATE TABLE test_02559 (id1 UInt64, id2 UInt64) ENGINE=MergeTree ORDER BY id1; INSERT INTO test_02559 SELECT number, number FROM numbers(10); +DROP ROW POLICY IF EXISTS 02559_filter_1 ON test_02559; +DROP ROW POLICY IF EXISTS 02559_filter_2 ON test_02559; + SET enable_multiple_prewhere_read_steps=true, move_all_conditions_to_prewhere=true; -- { echoOn } @@ -22,6 +25,15 @@ SELECT cast(id1 as UInt16) AS cond1 FROM test_02559 PREWHERE cond1 LIMIT 10; -- SELECT count() FROM test_02559 PREWHERE id2>=0 AND (1 OR ignore(id1)) WHERE ignore(id1)=0; +CREATE ROW POLICY 02559_filter_1 ON test_02559 USING id2=2 AS permissive TO ALL; +SELECT * FROM test_02559; + +CREATE ROW POLICY 02559_filter_2 ON test_02559 USING id2<=2 AS restrictive TO ALL; +SELECT * FROM test_02559; + -- { echoOff } +DROP ROW POLICY IF EXISTS 02559_filter_1 ON test_02559; +DROP ROW POLICY IF EXISTS 02559_filter_2 ON test_02559; + DROP TABLE test_02559; From d76e4f60a83e2ac7c9032d59605ff98aba80abb2 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 13 Feb 2023 23:14:18 +0100 Subject: [PATCH 069/253] Fix for combined node for condition group not found --- .../MergeTreeSplitPrewhereIntoReadSteps.cpp | 47 ++++++++++++------- 1 file changed, 31 insertions(+), 16 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp index 90ecf3e25f5..d41572c9db7 100644 --- a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include #include @@ -44,7 +43,7 @@ void fillRequiredColumns(const ActionsDAG::Node * node, std::unordered_map; +/// Result name -> DAGNodeRef +using OriginalToNewNodeMap = std::unordered_map; /// Clones the part of original DAG responsible for computing the original_dag_node and adds it to the new DAG. const ActionsDAG::Node & addClonedDAGToDAG(const ActionsDAG::Node * original_dag_node, ActionsDAGPtr new_dag, OriginalToNewNodeMap & node_remap) { + const String & node_name = original_dag_node->result_name; /// Look for the node in the map of already known nodes - if (node_remap.contains(original_dag_node)) + if (node_remap.contains(node_name)) { /// If the node is already in the new DAG, return it - const auto & node_ref = node_remap.at(original_dag_node); + const auto & node_ref = node_remap.at(node_name); if (node_ref.dag == new_dag) return *node_ref.node; @@ -70,7 +71,7 @@ const ActionsDAG::Node & addClonedDAGToDAG(const ActionsDAG::Node * original_dag { node_ref.dag->addOrReplaceInOutputs(*node_ref.node); const auto & new_node = new_dag->addInput(node_ref.node->result_name, node_ref.node->result_type); - node_remap[original_dag_node] = {new_dag, &new_node}; /// TODO: here we update the node reference. Is it always correct? + node_remap[node_name] = {new_dag, &new_node}; /// TODO: here we update the node reference. Is it always correct? return new_node; } } @@ -79,7 +80,7 @@ const ActionsDAG::Node & addClonedDAGToDAG(const ActionsDAG::Node * original_dag if (original_dag_node->type == ActionsDAG::ActionType::INPUT) { const auto & new_node = new_dag->addInput(original_dag_node->result_name, original_dag_node->result_type); - node_remap[original_dag_node] = {new_dag, &new_node}; + node_remap[node_name] = {new_dag, &new_node}; return new_node; } @@ -88,7 +89,7 @@ const ActionsDAG::Node & addClonedDAGToDAG(const ActionsDAG::Node * original_dag { const auto & new_node = new_dag->addColumn( ColumnWithTypeAndName(original_dag_node->column, original_dag_node->result_type, original_dag_node->result_name)); - node_remap[original_dag_node] = {new_dag, &new_node}; + node_remap[node_name] = {new_dag, &new_node}; return new_node; } @@ -105,22 +106,36 @@ const ActionsDAG::Node & addClonedDAGToDAG(const ActionsDAG::Node * original_dag } const auto & new_node = new_dag->addFunction(original_dag_node->function_base, new_children, original_dag_node->result_name); - node_remap[original_dag_node] = {new_dag, &new_node}; + node_remap[node_name] = {new_dag, &new_node}; return new_node; } throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected node type in PREWHERE actions: {}", original_dag_node->type); } +const ActionsDAG::Node & addFunction( + ActionsDAGPtr new_dag, + const FunctionOverloadResolverPtr & function, + ActionsDAG::NodeRawConstPtrs children, + OriginalToNewNodeMap & node_remap) +{ + const auto & new_node = new_dag->addFunction(function, children, ""); + node_remap[new_node.result_name] = {new_dag, &new_node}; + return new_node; +} + /// Adds a CAST node with the regular name ("CAST(...)") or with the provided name. /// This is different from ActionsDAG::addCast() because it set the name equal to the original name effectively hiding the value before cast, /// but it might be required for further steps with its original uncasted type. -const ActionsDAG::Node & addCast(ActionsDAGPtr dag, const ActionsDAG::Node & node_to_cast, const String & type_name, const String & new_name = {}) +const ActionsDAG::Node & addCast( + ActionsDAGPtr dag, + const ActionsDAG::Node & node_to_cast, + const String & type_name, + OriginalToNewNodeMap & node_remap) { Field cast_type_constant_value(type_name); ColumnWithTypeAndName column; - column.name = calculateConstantActionNodeName(cast_type_constant_value); column.column = DataTypeString().createColumnConst(0, cast_type_constant_value); column.type = std::make_shared(); @@ -128,7 +143,7 @@ const ActionsDAG::Node & addCast(ActionsDAGPtr dag, const ActionsDAG::Node & nod ActionsDAG::NodeRawConstPtrs children = {&node_to_cast, cast_type_constant_node}; FunctionOverloadResolverPtr func_builder_cast = CastInternalOverloadResolver::createImpl(); - return dag->addFunction(func_builder_cast, std::move(children), new_name); + return addFunction(dag, func_builder_cast, std::move(children), node_remap); } } @@ -216,7 +231,7 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction { /// Add AND function to combine the conditions FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); - const auto & and_function_node = step_dag->addFunction(func_builder_and, new_condition_nodes, ""); + const auto & and_function_node = addFunction(step_dag, func_builder_and, new_condition_nodes, node_remap); step_dag->addOrReplaceInOutputs(and_function_node); result_name = and_function_node.result_name; } @@ -231,7 +246,7 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction } else { - const auto & cast_node = addCast(step_dag, result_node, "UInt8"); + const auto & cast_node = addCast(step_dag, result_node, "UInt8", node_remap); step_dag->addOrReplaceInOutputs(cast_node); result_name = cast_node.result_name; } @@ -248,9 +263,9 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction for (const auto * output : original_outputs) { all_output_names.insert(output->result_name); - if (node_remap.contains(output)) + if (node_remap.contains(output->result_name)) { - const auto & new_node_info = node_remap[output]; + const auto & new_node_info = node_remap[output->result_name]; new_node_info.dag->addOrReplaceInOutputs(*new_node_info.node); } else if (output->result_name == prewhere_info->prewhere_column_name) From bbf94a2664be9b5ba3bdc5f10ccde150cae08f34 Mon Sep 17 00:00:00 2001 From: lzydmxy <13126752315@163.com> Date: Tue, 14 Feb 2023 17:29:44 +0800 Subject: [PATCH 070/253] Apply `ALTER TABLE table_name ON CLUSTER cluster MOVE PARTITION|PART partition_expr TO DISK|VOLUME 'disk_name'` to all replicas. Because `ALTER TABLE t MOVE` is not replicated. --- src/Interpreters/DDLWorker.cpp | 7 +++---- src/Parsers/ASTAlterQuery.cpp | 18 ++++++++++++++++++ src/Parsers/ASTAlterQuery.h | 2 ++ 3 files changed, 23 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 0f91212e6a9..54d1067de31 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -691,10 +691,9 @@ bool DDLWorker::taskShouldBeExecutedOnLeader(const ASTPtr & ast_ddl, const Stora if (auto * alter = ast_ddl->as()) { // Setting alters should be executed on all replicas - if (alter->isSettingsAlter()) - return false; - - if (alter->isFreezeAlter()) + if (alter->isSettingsAlter() || + alter->isFreezeAlter() || + alter->isMovePartitionToDiskOrVolumeAlter()) return false; } diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 5d347446d37..426b63a9d28 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -533,6 +533,24 @@ bool ASTAlterQuery::isDropPartitionAlter() const return isOneCommandTypeOnly(ASTAlterCommand::DROP_PARTITION) || isOneCommandTypeOnly(ASTAlterCommand::DROP_DETACHED_PARTITION); } +bool ASTAlterQuery::isMovePartitionToDiskOrVolumeAlter() const +{ + if (command_list) + { + if (command_list->children.empty()) + return false; + for (const auto & child : command_list->children) + { + const auto & command = child->as(); + if (command.type != ASTAlterCommand::MOVE_PARTITION || + (command.move_destination_type != DataDestinationType::DISK && command.move_destination_type != DataDestinationType::VOLUME)) + return false; + } + return true; + } + return false; +} + /** Get the text that identifies this element. */ String ASTAlterQuery::getID(char delim) const diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index 4a8c9c14ea9..2a48f5bbd9e 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -239,6 +239,8 @@ public: bool isDropPartitionAlter() const; + bool isMovePartitionToDiskOrVolumeAlter() const; + String getID(char) const override; ASTPtr clone() const override; From f7604cc686dd10eaf11b686385f959d14d449af4 Mon Sep 17 00:00:00 2001 From: John Skopis Date: Fri, 19 Nov 2021 15:19:20 +0000 Subject: [PATCH 071/253] [feat] Add ProfileEvents map to PartLog closes #10316 --- src/Common/CurrentMetrics.cpp | 1 + src/Interpreters/PartLog.cpp | 32 ++++++++++++++-- src/Interpreters/PartLog.h | 14 +++++-- src/Storages/MergeTree/IExecutableTask.h | 12 ++++++ .../MergeTree/MergeFromLogEntryTask.cpp | 5 ++- .../MergeTree/MergePlainMergeTreeTask.cpp | 6 ++- .../MergeTree/MergePlainMergeTreeTask.h | 15 ++++++++ .../MergeTree/MergeTreeBackgroundExecutor.cpp | 18 +++++++++ src/Storages/MergeTree/MergeTreeData.cpp | 8 +++- src/Storages/MergeTree/MergeTreeData.h | 3 +- .../MergeTree/MutateFromLogEntryTask.cpp | 5 ++- .../MergeTree/MutatePlainMergeTreeTask.cpp | 6 ++- .../MergeTree/MutatePlainMergeTreeTask.h | 14 +++++++ src/Storages/MergeTree/MutateTask.cpp | 38 +++++++++++++++++++ .../MergeTree/ReplicatedMergeMutateTaskBase.h | 10 +++++ src/Storages/MergeTree/TaskObserverMetrics.h | 37 ++++++++++++++++++ .../MergeTree/tests/gtest_executor.cpp | 15 ++++++++ src/Storages/StorageReplicatedMergeTree.cpp | 15 ++++++-- 18 files changed, 239 insertions(+), 15 deletions(-) create mode 100644 src/Storages/MergeTree/TaskObserverMetrics.h diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 0395f2470af..62d56dead62 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -13,6 +13,7 @@ M(BackgroundMergesAndMutationsPoolTask, "Number of active merges and mutations in an associated background pool") \ M(BackgroundFetchesPoolTask, "Number of active fetches in an associated background pool") \ M(BackgroundCommonPoolTask, "Number of active tasks in an associated background pool") \ + M(BackgroundMaintPoolTask, "Number of active tasks in BackgroundProcessingPool (maint)") \ M(BackgroundMovePoolTask, "Number of active tasks in BackgroundProcessingPool for moves") \ M(BackgroundSchedulePoolTask, "Number of active tasks in BackgroundSchedulePool. This pool is used for periodic ReplicatedMergeTree tasks, like cleaning old data parts, altering data parts, replica re-initialization, etc.") \ M(BackgroundBufferFlushSchedulePoolTask, "Number of active tasks in BackgroundBufferFlushSchedulePool. This pool is used for periodic Buffer flushes") \ diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index 4a1349680fd..b39f7423bbe 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -11,6 +11,9 @@ #include #include #include +#include +#include +#include #include @@ -121,6 +124,17 @@ NamesAndTypesList PartLogElement::getNamesAndTypes() /// Is there an error during the execution or commit {"error", std::make_shared()}, {"exception", std::make_shared()}, + + {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared())}, + }; +} + +NamesAndAliases PartLogElement::getNamesAndAliases() +{ + return + { + {"ProfileEvents.Names", {std::make_shared(std::make_shared())}, "mapKeys(ProfileEvents)"}, + {"ProfileEvents.Values", {std::make_shared(std::make_shared())}, "mapValues(ProfileEvents)"}, }; } @@ -163,18 +177,28 @@ void PartLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(error); columns[i++]->insert(exception); + + if (profile_counters) + { + auto * column = columns[i++].get(); + ProfileEvents::dumpToMapColumn(*profile_counters, column, true); + } + else + { + columns[i++]->insertDefault(); + } } bool PartLog::addNewPart( - ContextPtr current_context, const MutableDataPartPtr & part, UInt64 elapsed_ns, const ExecutionStatus & execution_status) + ContextPtr current_context, const MutableDataPartPtr & part, UInt64 elapsed_ns, const ExecutionStatus & execution_status, std::shared_ptr profile_counters_) { - return addNewParts(current_context, {part}, elapsed_ns, execution_status); + return addNewParts(current_context, {part}, elapsed_ns, execution_status, profile_counters_); } bool PartLog::addNewParts( - ContextPtr current_context, const PartLog::MutableDataPartsVector & parts, UInt64 elapsed_ns, const ExecutionStatus & execution_status) + ContextPtr current_context, const PartLog::MutableDataPartsVector & parts, UInt64 elapsed_ns, const ExecutionStatus & execution_status, std::shared_ptr profile_counters_) { if (parts.empty()) return true; @@ -221,6 +245,8 @@ bool PartLog::addNewParts( elem.error = static_cast(execution_status.code); elem.exception = execution_status.message; + elem.profile_counters = profile_counters_; + part_log->add(elem); } } diff --git a/src/Interpreters/PartLog.h b/src/Interpreters/PartLog.h index 392e76d85d1..75b2539bda9 100644 --- a/src/Interpreters/PartLog.h +++ b/src/Interpreters/PartLog.h @@ -8,6 +8,10 @@ #include #include +namespace ProfileEvents +{ + class Counters; +} namespace DB { @@ -81,13 +85,15 @@ struct PartLogElement UInt16 error = 0; String exception; + std::shared_ptr profile_counters; + static std::string name() { return "PartLog"; } static MergeReasonType getMergeReasonType(MergeType merge_type); static PartMergeAlgorithm getMergeAlgorithm(MergeAlgorithm merge_algorithm_); static NamesAndTypesList getNamesAndTypes(); - static NamesAndAliases getNamesAndAliases() { return {}; } + static NamesAndAliases getNamesAndAliases(); void appendToBlock(MutableColumns & columns) const; static const char * getCustomColumnList() { return nullptr; } }; @@ -106,9 +112,11 @@ class PartLog : public SystemLog public: /// Add a record about creation of new part. static bool addNewPart(ContextPtr context, const MutableDataPartPtr & part, UInt64 elapsed_ns, - const ExecutionStatus & execution_status = {}); + const ExecutionStatus & execution_status = {}, + std::shared_ptr profile_counters_ = {}); static bool addNewParts(ContextPtr context, const MutableDataPartsVector & parts, UInt64 elapsed_ns, - const ExecutionStatus & execution_status = {}); + const ExecutionStatus & execution_status = {}, + std::shared_ptr profile_counters_ = {}); }; } diff --git a/src/Storages/MergeTree/IExecutableTask.h b/src/Storages/MergeTree/IExecutableTask.h index 9617960c182..d878d57dec8 100644 --- a/src/Storages/MergeTree/IExecutableTask.h +++ b/src/Storages/MergeTree/IExecutableTask.h @@ -29,7 +29,9 @@ class IExecutableTask { public: using TaskResultCallback = std::function; + virtual bool onResume() = 0; virtual bool executeStep() = 0; + virtual bool onSuspend() = 0; virtual void onCompleted() = 0; virtual StorageID getStorageID() = 0; virtual UInt64 getPriority() = 0; @@ -54,6 +56,11 @@ public: , job_result_callback(std::forward(job_result_callback_)) , id(id_) {} + bool onResume() override + { + return true; + } + bool executeStep() override { res = job_to_execute(); @@ -61,6 +68,11 @@ public: return false; } + bool onSuspend() override + { + return true; + } + void onCompleted() override { job_result_callback(!res); } StorageID getStorageID() override { return id; } UInt64 getPriority() override diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index 2d2013bd648..fab9b7c0f7e 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -289,9 +289,12 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() return {true, true, [this, stopwatch = *stopwatch_ptr] (const ExecutionStatus & execution_status) { + auto & thread_status = CurrentThread::get(); + thread_status.finalizePerformanceCounters(); + auto profile_counters = std::make_shared(thread_status.performance_counters.getPartiallyAtomicSnapshot()); storage.writePartLog( PartLogElement::MERGE_PARTS, execution_status, stopwatch.elapsed(), - entry.new_part_name, part, parts, merge_mutate_entry.get()); + entry.new_part_name, part, parts, merge_mutate_entry.get(), profile_counters); }}; } diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index 9f24839f1e1..ccd132beb4e 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -85,6 +85,9 @@ void MergePlainMergeTreeTask::prepare() write_part_log = [this] (const ExecutionStatus & execution_status) { + auto & thread_status = CurrentThread::get(); + thread_status.finalizePerformanceCounters(); + auto profile_counters = std::make_shared(thread_status.performance_counters.getPartiallyAtomicSnapshot()); merge_task.reset(); storage.writePartLog( PartLogElement::MERGE_PARTS, @@ -93,7 +96,8 @@ void MergePlainMergeTreeTask::prepare() future_part->name, new_part, future_part->parts, - merge_list_entry.get()); + merge_list_entry.get(), + profile_counters); }; merge_task = storage.merger_mutator.mergePartsToTemporaryPart( diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.h b/src/Storages/MergeTree/MergePlainMergeTreeTask.h index d84db36bac2..472518c2723 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.h +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.h @@ -5,6 +5,8 @@ #include #include #include +#include + namespace DB { @@ -34,7 +36,18 @@ public: priority += item->getBytesOnDisk(); } + bool onResume() override + { + return observer.doResume(); + } + bool executeStep() override; + + bool onSuspend() override + { + return observer.doSuspend(); + } + void onCompleted() override; StorageID getStorageID() override; UInt64 getPriority() override { return priority; } @@ -82,6 +95,8 @@ private: MergeTreeTransactionHolder txn_holder; MergeTreeTransactionPtr txn; + + TaskObserverMetrics observer; }; diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index 5bc3fda88bb..028da3a3b5f 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -130,6 +130,15 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) bool need_execute_again = false; + try + { + item->task->onResume(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + try { ALLOW_ALLOCATIONS_IN_SCOPE; @@ -153,6 +162,15 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) }); } + try + { + item->task->onSuspend(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + if (need_execute_again) { std::lock_guard guard(mutex); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c3c4cd3082d..31c797556f6 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7280,7 +7280,8 @@ void MergeTreeData::writePartLog( const String & new_part_name, const DataPartPtr & result_part, const DataPartsVector & source_parts, - const MergeListEntry * merge_entry) + const MergeListEntry * merge_entry, + std::shared_ptr profile_counters) try { auto table_id = getStorageID(); @@ -7342,6 +7343,11 @@ try part_log_elem.peak_memory_usage = (*merge_entry)->memory_tracker.getPeak(); } + if (profile_counters) + { + part_log_elem.profile_counters = profile_counters; + } + part_log->add(part_log_elem); } catch (...) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 34bc3d24d66..ffedb0ebfdc 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1298,7 +1298,8 @@ protected: const String & new_part_name, const DataPartPtr & result_part, const DataPartsVector & source_parts, - const MergeListEntry * merge_entry); + const MergeListEntry * merge_entry, + std::shared_ptr profile_counters = nullptr); /// If part is assigned to merge or mutation (possibly replicated) /// Should be overridden by children, because they can have different diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index b83c058f7fd..322d8e78585 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -184,9 +184,12 @@ ReplicatedMergeMutateTaskBase::PrepareResult MutateFromLogEntryTask::prepare() return {true, true, [this] (const ExecutionStatus & execution_status) { + auto & thread_status = CurrentThread::get(); + thread_status.finalizePerformanceCounters(); + auto profile_counters = std::make_shared(thread_status.performance_counters.getPartiallyAtomicSnapshot()); storage.writePartLog( PartLogElement::MUTATE_PART, execution_status, stopwatch_ptr->elapsed(), - entry.new_part_name, new_part, future_mutated_part->parts, merge_mutate_entry.get()); + entry.new_part_name, new_part, future_mutated_part->parts, merge_mutate_entry.get(), profile_counters); }}; } diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 76ba921b705..d611773a5d5 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -38,6 +38,9 @@ void MutatePlainMergeTreeTask::prepare() write_part_log = [this] (const ExecutionStatus & execution_status) { + auto & thread_status = CurrentThread::get(); + thread_status.finalizePerformanceCounters(); + auto profile_counters = std::make_shared(thread_status.performance_counters.getPartiallyAtomicSnapshot()); mutate_task.reset(); storage.writePartLog( PartLogElement::MUTATE_PART, @@ -46,7 +49,8 @@ void MutatePlainMergeTreeTask::prepare() future_part->name, new_part, future_part->parts, - merge_list_entry.get()); + merge_list_entry.get(), + profile_counters); }; fake_query_context = Context::createCopy(storage.getContext()); diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.h b/src/Storages/MergeTree/MutatePlainMergeTreeTask.h index e2b019c08ce..577529422de 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.h +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.h @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB { @@ -38,7 +39,18 @@ public: priority += part->getBytesOnDisk(); } + bool onSuspend() override + { + return observer.doSuspend(); + } + bool executeStep() override; + + bool onResume() override + { + return observer.doResume(); + } + void onCompleted() override; StorageID getStorageID() override; UInt64 getPriority() override { return priority; } @@ -76,6 +88,8 @@ private: ContextMutablePtr fake_query_context; MutateTaskPtr mutate_task; + + TaskObserverMetrics observer; }; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 4a7224b0722..5416e731d84 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -812,6 +813,11 @@ public: StorageID getStorageID() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } UInt64 getPriority() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } + bool onSuspend() override + { + return observer.doSuspend(); + } + bool executeStep() override { auto & current_level_parts = level_parts[current_level]; @@ -898,6 +904,12 @@ public: /// Need execute again return true; } + + bool onResume() override + { + return observer.doResume(); + } + private: String name; MergeTreeData::MutableDataPartsVector parts; @@ -913,6 +925,8 @@ private: /// TODO(nikitamikhaylov): make this constant a setting static constexpr size_t max_parts_to_merge_in_one_level = 10; + + TaskObserverMetrics observer; }; @@ -1134,6 +1148,12 @@ public: StorageID getStorageID() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } UInt64 getPriority() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } + + bool onSuspend() override + { + return observer.doSuspend(); + } + bool executeStep() override { switch (state) @@ -1168,6 +1188,11 @@ public: return false; } + bool onResume() override + { + return observer.doResume(); + } + private: void prepare() @@ -1251,6 +1276,8 @@ private: MutationContextPtr ctx; std::unique_ptr part_merger_writer_task; + + TaskObserverMetrics observer; }; class MutateSomePartColumnsTask : public IExecutableTask @@ -1262,6 +1289,11 @@ public: StorageID getStorageID() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } UInt64 getPriority() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } + bool onSuspend() override + { + return observer.doSuspend(); + } + bool executeStep() override { switch (state) @@ -1295,6 +1327,11 @@ public: return false; } + bool onResume() override + { + return observer.doResume(); + } + private: void prepare() @@ -1455,6 +1492,7 @@ private: MergedColumnOnlyOutputStreamPtr out; std::unique_ptr part_merger_writer_task{nullptr}; + TaskObserverMetrics observer; }; diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h index d8495d35d90..6080d488ca4 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h @@ -4,6 +4,7 @@ #include #include +#include namespace DB { @@ -34,7 +35,15 @@ public: ~ReplicatedMergeMutateTaskBase() override = default; void onCompleted() override; StorageID getStorageID() override; + bool onSuspend() override + { + return observer.doSuspend(); + } bool executeStep() override; + bool onResume() override + { + return observer.doResume(); + } protected: using PartLogWriter = std::function; @@ -83,6 +92,7 @@ private: PartLogWriter part_log_writer{}; State state{State::NEED_PREPARE}; IExecutableTask::TaskResultCallback task_result_callback; + TaskObserverMetrics observer; }; } diff --git a/src/Storages/MergeTree/TaskObserverMetrics.h b/src/Storages/MergeTree/TaskObserverMetrics.h new file mode 100644 index 00000000000..c6e49435c7d --- /dev/null +++ b/src/Storages/MergeTree/TaskObserverMetrics.h @@ -0,0 +1,37 @@ +#pragma once + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +class TaskObserverMetrics : public boost::noncopyable +{ +public: + TaskObserverMetrics() : thread_group(std::make_shared()) {} + ~TaskObserverMetrics() {} + + bool doResume() + { + CurrentThread::attachTo(thread_group); + return true; + } + + bool doSuspend() + { + CurrentThread::detachQueryIfNotDetached(); + return true; + } + + +private: + ThreadGroupStatusPtr thread_group; +}; + + +} diff --git a/src/Storages/MergeTree/tests/gtest_executor.cpp b/src/Storages/MergeTree/tests/gtest_executor.cpp index b89692869fd..1ae2123ce5a 100644 --- a/src/Storages/MergeTree/tests/gtest_executor.cpp +++ b/src/Storages/MergeTree/tests/gtest_executor.cpp @@ -24,6 +24,11 @@ public: { } + bool onSuspend() override + { + suspend_calls++ + } + bool executeStep() override { auto sleep_time = distribution(generator); @@ -36,6 +41,11 @@ public: return false; } + bool onSuspend() override + { + resume_calls++ + } + StorageID getStorageID() override { return {"test", name}; @@ -55,7 +65,9 @@ private: std::uniform_int_distribution<> distribution; String name; + size_t suspend_calls; std::function on_completed; + size_t resume_calls; }; @@ -93,6 +105,9 @@ TEST(Executor, RemoveTasks) thread.join(); ASSERT_EQ(CurrentMetrics::values[CurrentMetrics::BackgroundMergesAndMutationsPoolTask], 0); + /// TODO: move to a test by itself + ASSERT_EQ(batch*tasks_kinds, suspend_calls); + ASSERT_EQ(batch*tasks_kinds, resume_calls); executor->wait(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0fb3e25ee85..3672bd44bb2 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1602,8 +1602,11 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) renameTempPartAndReplace(part, transaction); checkPartChecksumsAndCommit(transaction, part); + auto & thread_status = CurrentThread::get(); + thread_status.finalizePerformanceCounters(); + auto profile_counters = std::make_shared(thread_status.performance_counters.getPartiallyAtomicSnapshot()); writePartLog(PartLogElement::Type::NEW_PART, {}, 0 /** log entry is fake so we don't measure the time */, - part->name, part, {} /** log entry is fake so there are no initial parts */, nullptr); + part->name, part, {} /** log entry is fake so there are no initial parts */, nullptr, profile_counters); return true; } @@ -4007,9 +4010,12 @@ bool StorageReplicatedMergeTree::fetchPart( auto write_part_log = [&] (const ExecutionStatus & execution_status) { + auto & thread_status = CurrentThread::get(); + thread_status.finalizePerformanceCounters(); + auto profile_counters = std::make_shared(thread_status.performance_counters.getPartiallyAtomicSnapshot()); writePartLog( PartLogElement::DOWNLOAD_PART, execution_status, stopwatch.elapsed(), - part_name, part, replaced_parts, nullptr); + part_name, part, replaced_parts, nullptr, profile_counters); }; DataPartPtr part_to_clone; @@ -4243,9 +4249,12 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( auto write_part_log = [&] (const ExecutionStatus & execution_status) { + auto & thread_status = CurrentThread::get(); + thread_status.finalizePerformanceCounters(); + auto profile_counters = std::make_shared(thread_status.performance_counters.getPartiallyAtomicSnapshot()); writePartLog( PartLogElement::DOWNLOAD_PART, execution_status, stopwatch.elapsed(), - part_name, part, replaced_parts, nullptr); + part_name, part, replaced_parts, nullptr, profile_counters); }; std::function get_part; From 96e4411694fb1a693414018ba76dab667e35508d Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Wed, 6 Apr 2022 09:08:10 +0000 Subject: [PATCH 072/253] Fix style --- src/Storages/MergeTree/TaskObserverMetrics.h | 32 ++++++++------------ 1 file changed, 13 insertions(+), 19 deletions(-) diff --git a/src/Storages/MergeTree/TaskObserverMetrics.h b/src/Storages/MergeTree/TaskObserverMetrics.h index c6e49435c7d..b518682f0c9 100644 --- a/src/Storages/MergeTree/TaskObserverMetrics.h +++ b/src/Storages/MergeTree/TaskObserverMetrics.h @@ -5,33 +5,27 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - class TaskObserverMetrics : public boost::noncopyable { public: - TaskObserverMetrics() : thread_group(std::make_shared()) {} - ~TaskObserverMetrics() {} + TaskObserverMetrics() : thread_group(std::make_shared()) { } + ~TaskObserverMetrics() { } - bool doResume() - { - CurrentThread::attachTo(thread_group); - return true; - } + bool doResume() + { + CurrentThread::attachTo(thread_group); + return true; + } - bool doSuspend() - { - CurrentThread::detachQueryIfNotDetached(); - return true; - } + bool doSuspend() + { + CurrentThread::detachQueryIfNotDetached(); + return true; + } private: - ThreadGroupStatusPtr thread_group; + ThreadGroupStatusPtr thread_group; }; - } From f554ff9d99893af48a693b08ce4b56eefeae5e7c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 4 Aug 2022 06:07:32 +0200 Subject: [PATCH 073/253] Remove trash --- src/Common/CurrentMetrics.cpp | 1 - src/Common/CurrentThread.h | 16 +++++- src/Interpreters/PartLog.cpp | 2 +- src/Storages/MergeTree/IExecutableTask.h | 12 ----- .../MergeTree/MergeFromLogEntryTask.cpp | 2 +- .../MergeTree/MergePlainMergeTreeTask.cpp | 3 ++ .../MergeTree/MergePlainMergeTreeTask.h | 17 +----- .../MergeTree/MergeTreeBackgroundExecutor.cpp | 18 ------- .../MergeTree/MergeTreeBackgroundExecutor.h | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 3 +- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../MergeTree/MutateFromLogEntryTask.cpp | 2 +- .../MergeTree/MutatePlainMergeTreeTask.cpp | 5 +- .../MergeTree/MutatePlainMergeTreeTask.h | 15 +----- src/Storages/MergeTree/MutateTask.cpp | 54 +++++-------------- .../ReplicatedMergeMutateTaskBase.cpp | 4 +- .../MergeTree/ReplicatedMergeMutateTaskBase.h | 24 ++++----- src/Storages/MergeTree/TaskObserverMetrics.h | 31 ----------- .../MergeTree/tests/gtest_executor.cpp | 18 +------ src/Storages/StorageReplicatedMergeTree.cpp | 6 +-- 20 files changed, 63 insertions(+), 173 deletions(-) delete mode 100644 src/Storages/MergeTree/TaskObserverMetrics.h diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 62d56dead62..0395f2470af 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -13,7 +13,6 @@ M(BackgroundMergesAndMutationsPoolTask, "Number of active merges and mutations in an associated background pool") \ M(BackgroundFetchesPoolTask, "Number of active fetches in an associated background pool") \ M(BackgroundCommonPoolTask, "Number of active tasks in an associated background pool") \ - M(BackgroundMaintPoolTask, "Number of active tasks in BackgroundProcessingPool (maint)") \ M(BackgroundMovePoolTask, "Number of active tasks in BackgroundProcessingPool for moves") \ M(BackgroundSchedulePoolTask, "Number of active tasks in BackgroundSchedulePool. This pool is used for periodic ReplicatedMergeTree tasks, like cleaning old data parts, altering data parts, replica re-initialization, etc.") \ M(BackgroundBufferFlushSchedulePoolTask, "Number of active tasks in BackgroundBufferFlushSchedulePool. This pool is used for periodic Buffer flushes") \ diff --git a/src/Common/CurrentThread.h b/src/Common/CurrentThread.h index cbe60365798..9548a927cfd 100644 --- a/src/Common/CurrentThread.h +++ b/src/Common/CurrentThread.h @@ -92,7 +92,7 @@ public: static void detachQueryIfNotDetached(); /// Initializes query with current thread as master thread in constructor, and detaches it in destructor - struct QueryScope + struct QueryScope : private boost::noncopyable { explicit QueryScope(ContextMutablePtr query_context); explicit QueryScope(ContextPtr query_context); @@ -102,6 +102,20 @@ public: bool log_peak_memory_usage_in_destructor = true; }; + class ScopedAttach : private boost::noncopyable + { + public: + explicit ScopedAttach(const ThreadGroupStatusPtr & thread_group) + { + CurrentThread::attachTo(thread_group); + } + + ~ScopedAttach() + { + CurrentThread::detachQuery(); + } + }; + private: static void defaultThreadDeleter(); diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index b39f7423bbe..b422f4f7f03 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -193,7 +193,7 @@ void PartLogElement::appendToBlock(MutableColumns & columns) const bool PartLog::addNewPart( ContextPtr current_context, const MutableDataPartPtr & part, UInt64 elapsed_ns, const ExecutionStatus & execution_status, std::shared_ptr profile_counters_) { - return addNewParts(current_context, {part}, elapsed_ns, execution_status, profile_counters_); + return addNewParts(current_context, {part}, elapsed_ns, execution_status, std::move(profile_counters_)); } diff --git a/src/Storages/MergeTree/IExecutableTask.h b/src/Storages/MergeTree/IExecutableTask.h index d878d57dec8..9617960c182 100644 --- a/src/Storages/MergeTree/IExecutableTask.h +++ b/src/Storages/MergeTree/IExecutableTask.h @@ -29,9 +29,7 @@ class IExecutableTask { public: using TaskResultCallback = std::function; - virtual bool onResume() = 0; virtual bool executeStep() = 0; - virtual bool onSuspend() = 0; virtual void onCompleted() = 0; virtual StorageID getStorageID() = 0; virtual UInt64 getPriority() = 0; @@ -56,11 +54,6 @@ public: , job_result_callback(std::forward(job_result_callback_)) , id(id_) {} - bool onResume() override - { - return true; - } - bool executeStep() override { res = job_to_execute(); @@ -68,11 +61,6 @@ public: return false; } - bool onSuspend() override - { - return true; - } - void onCompleted() override { job_result_callback(!res); } StorageID getStorageID() override { return id; } UInt64 getPriority() override diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index fab9b7c0f7e..abb871be372 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -294,7 +294,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() auto profile_counters = std::make_shared(thread_status.performance_counters.getPartiallyAtomicSnapshot()); storage.writePartLog( PartLogElement::MERGE_PARTS, execution_status, stopwatch.elapsed(), - entry.new_part_name, part, parts, merge_mutate_entry.get(), profile_counters); + entry.new_part_name, part, parts, merge_mutate_entry.get(), std::move(profile_counters)); }}; } diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index ccd132beb4e..95bcfba937a 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -27,6 +27,9 @@ void MergePlainMergeTreeTask::onCompleted() bool MergePlainMergeTreeTask::executeStep() { + /// Metrics will be saved in the thread_group. + CurrentThread::ScopedAttach scoped_attach(thread_group); + /// Make out memory tracker a parent of current thread memory tracker MemoryTrackerThreadSwitcherPtr switcher; if (merge_list_entry) diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.h b/src/Storages/MergeTree/MergePlainMergeTreeTask.h index 472518c2723..4d0cecf1e55 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.h +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.h @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB @@ -36,18 +35,7 @@ public: priority += item->getBytesOnDisk(); } - bool onResume() override - { - return observer.doResume(); - } - bool executeStep() override; - - bool onSuspend() override - { - return observer.doSuspend(); - } - void onCompleted() override; StorageID getStorageID() override; UInt64 getPriority() override { return priority; } @@ -59,7 +47,6 @@ public: } private: - void prepare(); void finish(); @@ -95,8 +82,8 @@ private: MergeTreeTransactionHolder txn_holder; MergeTreeTransactionPtr txn; - - TaskObserverMetrics observer; + + ThreadGroupStatusPtr thread_group = std::make_shared(); }; diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index 028da3a3b5f..5bc3fda88bb 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -130,15 +130,6 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) bool need_execute_again = false; - try - { - item->task->onResume(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - try { ALLOW_ALLOCATIONS_IN_SCOPE; @@ -162,15 +153,6 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) }); } - try - { - item->task->onSuspend(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - if (need_execute_again) { std::lock_guard guard(mutex); diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h index 5c1178a1bc1..cf4d4b08c4d 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h @@ -17,6 +17,7 @@ #include #include + namespace DB { namespace ErrorCodes diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 31c797556f6..07385067ede 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7345,7 +7345,7 @@ try if (profile_counters) { - part_log_elem.profile_counters = profile_counters; + part_log_elem.profile_counters = profile_counters; } part_log->add(part_log_elem); @@ -7493,6 +7493,7 @@ bool MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagge moving_part.part->name, cloned_part, {moving_part.part}, + nullptr, nullptr); }; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index ffedb0ebfdc..10ba0045826 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1299,7 +1299,7 @@ protected: const DataPartPtr & result_part, const DataPartsVector & source_parts, const MergeListEntry * merge_entry, - std::shared_ptr profile_counters = nullptr); + std::shared_ptr profile_counters); /// If part is assigned to merge or mutation (possibly replicated) /// Should be overridden by children, because they can have different diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index 322d8e78585..2e546574e42 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -189,7 +189,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MutateFromLogEntryTask::prepare() auto profile_counters = std::make_shared(thread_status.performance_counters.getPartiallyAtomicSnapshot()); storage.writePartLog( PartLogElement::MUTATE_PART, execution_status, stopwatch_ptr->elapsed(), - entry.new_part_name, new_part, future_mutated_part->parts, merge_mutate_entry.get(), profile_counters); + entry.new_part_name, new_part, future_mutated_part->parts, merge_mutate_entry.get(), std::move(profile_counters)); }}; } diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index d611773a5d5..4a29291b20d 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -62,8 +62,12 @@ void MutatePlainMergeTreeTask::prepare() time(nullptr), fake_query_context, merge_mutate_entry->txn, merge_mutate_entry->tagger->reserved_space, table_lock_holder); } + bool MutatePlainMergeTreeTask::executeStep() { + /// Metrics will be saved in the thread_group. + CurrentThread::ScopedAttach scoped_attach(thread_group); + /// Make out memory tracker a parent of current thread memory tracker MemoryTrackerThreadSwitcherPtr switcher; if (merge_list_entry) @@ -127,5 +131,4 @@ bool MutatePlainMergeTreeTask::executeStep() return false; } - } diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.h b/src/Storages/MergeTree/MutatePlainMergeTreeTask.h index 577529422de..6356595a16b 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.h +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.h @@ -8,7 +8,7 @@ #include #include #include -#include + namespace DB { @@ -39,18 +39,7 @@ public: priority += part->getBytesOnDisk(); } - bool onSuspend() override - { - return observer.doSuspend(); - } - bool executeStep() override; - - bool onResume() override - { - return observer.doResume(); - } - void onCompleted() override; StorageID getStorageID() override; UInt64 getPriority() override { return priority; } @@ -89,7 +78,7 @@ private: ContextMutablePtr fake_query_context; MutateTaskPtr mutate_task; - TaskObserverMetrics observer; + ThreadGroupStatusPtr thread_group = std::make_shared(); }; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 5416e731d84..295a88d5f76 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -17,7 +17,6 @@ #include #include #include -#include #include #include #include @@ -813,13 +812,11 @@ public: StorageID getStorageID() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } UInt64 getPriority() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } - bool onSuspend() override - { - return observer.doSuspend(); - } - bool executeStep() override { + /// Metrics will be saved in the thread_group. + CurrentThread::ScopedAttach scoped_attach(thread_group); + auto & current_level_parts = level_parts[current_level]; auto & next_level_parts = level_parts[next_level]; @@ -905,11 +902,6 @@ public: return true; } - bool onResume() override - { - return observer.doResume(); - } - private: String name; MergeTreeData::MutableDataPartsVector parts; @@ -926,7 +918,7 @@ private: /// TODO(nikitamikhaylov): make this constant a setting static constexpr size_t max_parts_to_merge_in_one_level = 10; - TaskObserverMetrics observer; + ThreadGroupStatusPtr thread_group = std::make_shared(); }; @@ -937,9 +929,7 @@ private: // In short it executed a mutation for the part an original part and for its every projection -/** - * - * An overview of how the process of mutation works for projections: +/** An overview of how the process of mutation works for projections: * * The mutation for original parts is executed block by block, * but additionally we execute a SELECT query for each projection over a current block. @@ -1148,14 +1138,11 @@ public: StorageID getStorageID() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } UInt64 getPriority() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } - - bool onSuspend() override - { - return observer.doSuspend(); - } - bool executeStep() override { + /// Metrics will be saved in the thread_group. + CurrentThread::ScopedAttach scoped_attach(thread_group); + switch (state) { case State::NEED_PREPARE: @@ -1188,13 +1175,7 @@ public: return false; } - bool onResume() override - { - return observer.doResume(); - } - private: - void prepare() { if (ctx->new_data_part->isStoredOnDisk()) @@ -1277,9 +1258,10 @@ private: std::unique_ptr part_merger_writer_task; - TaskObserverMetrics observer; + ThreadGroupStatusPtr thread_group = std::make_shared(); }; + class MutateSomePartColumnsTask : public IExecutableTask { public: @@ -1289,13 +1271,11 @@ public: StorageID getStorageID() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } UInt64 getPriority() override { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } - bool onSuspend() override - { - return observer.doSuspend(); - } - bool executeStep() override { + /// Metrics will be saved in the thread_group. + CurrentThread::ScopedAttach scoped_attach(thread_group); + switch (state) { case State::NEED_PREPARE: @@ -1327,13 +1307,7 @@ public: return false; } - bool onResume() override - { - return observer.doResume(); - } - private: - void prepare() { if (ctx->execute_ttl_type != ExecuteTTLType::NONE) @@ -1492,7 +1466,7 @@ private: MergedColumnOnlyOutputStreamPtr out; std::unique_ptr part_merger_writer_task{nullptr}; - TaskObserverMetrics observer; + ThreadGroupStatusPtr thread_group = std::make_shared(); }; diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index a22aab8d6ce..2ef36654d95 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -29,6 +29,9 @@ void ReplicatedMergeMutateTaskBase::onCompleted() bool ReplicatedMergeMutateTaskBase::executeStep() { + /// Metrics will be saved in the thread_group. + CurrentThread::ScopedAttach scoped_attach(thread_group); + std::exception_ptr saved_exception; bool retryable_error = false; @@ -83,7 +86,6 @@ bool ReplicatedMergeMutateTaskBase::executeStep() saved_exception = std::current_exception(); } - if (!retryable_error && saved_exception) { std::lock_guard lock(storage.queue.state_mutex); diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h index 6080d488ca4..e08074cd4ff 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h @@ -4,16 +4,16 @@ #include #include -#include + namespace DB { class StorageReplicatedMergeTree; -/** - * This is used as a base of MergeFromLogEntryTask and MutateFromLogEntryTaskBase - */ + +/** This is used as a base of MergeFromLogEntryTask and MutateFromLogEntryTaskBase + */ class ReplicatedMergeMutateTaskBase : public IExecutableTask { public: @@ -33,17 +33,12 @@ public: } ~ReplicatedMergeMutateTaskBase() override = default; + void onCompleted() override; + StorageID getStorageID() override; - bool onSuspend() override - { - return observer.doSuspend(); - } + bool executeStep() override; - bool onResume() override - { - return observer.doResume(); - } protected: using PartLogWriter = std::function; @@ -70,7 +65,6 @@ protected: StorageReplicatedMergeTree & storage; private: - enum class CheckExistingPartResult { PART_EXISTS, @@ -78,7 +72,7 @@ private: }; CheckExistingPartResult checkExistingPart(); - bool executeImpl() ; + bool executeImpl(); enum class State { @@ -92,7 +86,7 @@ private: PartLogWriter part_log_writer{}; State state{State::NEED_PREPARE}; IExecutableTask::TaskResultCallback task_result_callback; - TaskObserverMetrics observer; + ThreadGroupStatusPtr thread_group = std::make_shared(); }; } diff --git a/src/Storages/MergeTree/TaskObserverMetrics.h b/src/Storages/MergeTree/TaskObserverMetrics.h deleted file mode 100644 index b518682f0c9..00000000000 --- a/src/Storages/MergeTree/TaskObserverMetrics.h +++ /dev/null @@ -1,31 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -class TaskObserverMetrics : public boost::noncopyable -{ -public: - TaskObserverMetrics() : thread_group(std::make_shared()) { } - ~TaskObserverMetrics() { } - - bool doResume() - { - CurrentThread::attachTo(thread_group); - return true; - } - - bool doSuspend() - { - CurrentThread::detachQueryIfNotDetached(); - return true; - } - - -private: - ThreadGroupStatusPtr thread_group; -}; - -} diff --git a/src/Storages/MergeTree/tests/gtest_executor.cpp b/src/Storages/MergeTree/tests/gtest_executor.cpp index 1ae2123ce5a..60149220643 100644 --- a/src/Storages/MergeTree/tests/gtest_executor.cpp +++ b/src/Storages/MergeTree/tests/gtest_executor.cpp @@ -24,11 +24,6 @@ public: { } - bool onSuspend() override - { - suspend_calls++ - } - bool executeStep() override { auto sleep_time = distribution(generator); @@ -41,11 +36,6 @@ public: return false; } - bool onSuspend() override - { - resume_calls++ - } - StorageID getStorageID() override { return {"test", name}; @@ -65,9 +55,7 @@ private: std::uniform_int_distribution<> distribution; String name; - size_t suspend_calls; std::function on_completed; - size_t resume_calls; }; @@ -87,8 +75,7 @@ TEST(Executor, RemoveTasks) for (size_t i = 0; i < batch; ++i) for (size_t j = 0; j < tasks_kinds; ++j) ASSERT_TRUE( - executor->trySchedule(std::make_shared(std::to_string(j))) - ); + executor->trySchedule(std::make_shared(std::to_string(j)))); std::vector threads(batch); @@ -105,9 +92,6 @@ TEST(Executor, RemoveTasks) thread.join(); ASSERT_EQ(CurrentMetrics::values[CurrentMetrics::BackgroundMergesAndMutationsPoolTask], 0); - /// TODO: move to a test by itself - ASSERT_EQ(batch*tasks_kinds, suspend_calls); - ASSERT_EQ(batch*tasks_kinds, resume_calls); executor->wait(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 3672bd44bb2..e9958a68406 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1606,7 +1606,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) thread_status.finalizePerformanceCounters(); auto profile_counters = std::make_shared(thread_status.performance_counters.getPartiallyAtomicSnapshot()); writePartLog(PartLogElement::Type::NEW_PART, {}, 0 /** log entry is fake so we don't measure the time */, - part->name, part, {} /** log entry is fake so there are no initial parts */, nullptr, profile_counters); + part->name, part, {} /** log entry is fake so there are no initial parts */, nullptr, std::move(profile_counters)); return true; } @@ -4015,7 +4015,7 @@ bool StorageReplicatedMergeTree::fetchPart( auto profile_counters = std::make_shared(thread_status.performance_counters.getPartiallyAtomicSnapshot()); writePartLog( PartLogElement::DOWNLOAD_PART, execution_status, stopwatch.elapsed(), - part_name, part, replaced_parts, nullptr, profile_counters); + part_name, part, replaced_parts, nullptr, std::move(profile_counters)); }; DataPartPtr part_to_clone; @@ -4254,7 +4254,7 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( auto profile_counters = std::make_shared(thread_status.performance_counters.getPartiallyAtomicSnapshot()); writePartLog( PartLogElement::DOWNLOAD_PART, execution_status, stopwatch.elapsed(), - part_name, part, replaced_parts, nullptr, profile_counters); + part_name, part, replaced_parts, nullptr, std::move(profile_counters)); }; std::function get_part; From f15623141ede304c5de0111c1f7075caef1c74d9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 6 Aug 2022 20:46:42 +0200 Subject: [PATCH 074/253] Fix error --- src/Common/CurrentThread.h | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Common/CurrentThread.h b/src/Common/CurrentThread.h index 9548a927cfd..24c9dac844a 100644 --- a/src/Common/CurrentThread.h +++ b/src/Common/CurrentThread.h @@ -104,15 +104,24 @@ public: class ScopedAttach : private boost::noncopyable { + private: + bool attached = false; public: explicit ScopedAttach(const ThreadGroupStatusPtr & thread_group) { - CurrentThread::attachTo(thread_group); + if (!CurrentThread::getGroup()) + { + CurrentThread::attachToIfDetached(thread_group); + attached = true; + } } ~ScopedAttach() { - CurrentThread::detachQuery(); + if (attached) + { + CurrentThread::detachQuery(); + } } }; From 5f9f72f9376a6c691fc8de75cfdae1e594f8ad76 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 6 Aug 2022 20:53:17 +0200 Subject: [PATCH 075/253] Add a smoke test --- .../02378_part_log_profile_events.reference | 1 + .../0_stateless/02378_part_log_profile_events.sql | 13 +++++++++++++ 2 files changed, 14 insertions(+) create mode 100644 tests/queries/0_stateless/02378_part_log_profile_events.reference create mode 100644 tests/queries/0_stateless/02378_part_log_profile_events.sql diff --git a/tests/queries/0_stateless/02378_part_log_profile_events.reference b/tests/queries/0_stateless/02378_part_log_profile_events.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02378_part_log_profile_events.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02378_part_log_profile_events.sql b/tests/queries/0_stateless/02378_part_log_profile_events.sql new file mode 100644 index 00000000000..9ffd6aa4135 --- /dev/null +++ b/tests/queries/0_stateless/02378_part_log_profile_events.sql @@ -0,0 +1,13 @@ +DROP TABLE IF EXISTS test; +CREATE TABLE test (x UInt64) ENGINE = MergeTree ORDER BY x; + +SET max_block_size = 1; +INSERT INTO test SELECT * FROM system.numbers LIMIT 1000; +OPTIMIZE TABLE test FINAL; + +SYSTEM FLUSH LOGS; + +-- ProfileEvents field exist and contains something plausible: +SELECT count() > 0 FROM system.part_log WHERE ProfileEvents['MergedRows'] = 1000 AND table = 'test' AND database = currentDatabase() AND event_time >= now() - 600; + +DROP TABLE test; From a228f7f419827915b99aca81789aa3f2d0d4c560 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 23 Jan 2023 12:45:28 +0000 Subject: [PATCH 076/253] [wip] ProfileCounters for each part --- src/Common/CurrentThread.cpp | 2 + src/Common/CurrentThread.h | 23 ---------- src/Common/ProfileEventsScope.cpp | 32 ++++++++++++++ src/Common/ProfileEventsScope.h | 35 +++++++++++++++ src/Common/ThreadStatus.cpp | 1 + src/Common/ThreadStatus.h | 4 ++ src/Interpreters/PartLog.cpp | 44 +++++++++++++------ src/Interpreters/PartLog.h | 36 +++++++++++++-- src/Interpreters/ProfileEventsExt.cpp | 16 +++++++ src/Interpreters/ProfileEventsExt.h | 2 + src/Interpreters/ThreadStatusExt.cpp | 6 +++ .../MergeTree/MergePlainMergeTreeTask.cpp | 7 +-- .../MergeTree/MergePlainMergeTreeTask.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 8 +++- src/Storages/MergeTree/MergeTreeSink.cpp | 25 ++++++++--- .../MergeTree/MutateFromLogEntryTask.cpp | 6 +-- .../MergeTree/MutatePlainMergeTreeTask.cpp | 11 +++-- .../MergeTree/MutatePlainMergeTreeTask.h | 2 +- src/Storages/MergeTree/MutateTask.cpp | 25 ++++++----- .../ReplicatedMergeMutateTaskBase.cpp | 5 ++- .../MergeTree/ReplicatedMergeMutateTaskBase.h | 12 +++-- src/Storages/StorageReplicatedMergeTree.cpp | 26 +++++------ .../02378_part_log_profile_events.reference | 2 + .../02378_part_log_profile_events.sql | 44 ++++++++++++++++--- 24 files changed, 277 insertions(+), 99 deletions(-) create mode 100644 src/Common/ProfileEventsScope.cpp create mode 100644 src/Common/ProfileEventsScope.h diff --git a/src/Common/CurrentThread.cpp b/src/Common/CurrentThread.cpp index e54b2c8abe4..14c9f4418e0 100644 --- a/src/Common/CurrentThread.cpp +++ b/src/Common/CurrentThread.cpp @@ -40,6 +40,8 @@ ThreadStatus & CurrentThread::get() ProfileEvents::Counters & CurrentThread::getProfileEvents() { + if (unlikely(subthread_profile_events)) + return *subthread_profile_events; return current_thread ? current_thread->performance_counters : ProfileEvents::global_counters; } diff --git a/src/Common/CurrentThread.h b/src/Common/CurrentThread.h index 24c9dac844a..c07b34acae3 100644 --- a/src/Common/CurrentThread.h +++ b/src/Common/CurrentThread.h @@ -102,29 +102,6 @@ public: bool log_peak_memory_usage_in_destructor = true; }; - class ScopedAttach : private boost::noncopyable - { - private: - bool attached = false; - public: - explicit ScopedAttach(const ThreadGroupStatusPtr & thread_group) - { - if (!CurrentThread::getGroup()) - { - CurrentThread::attachToIfDetached(thread_group); - attached = true; - } - } - - ~ScopedAttach() - { - if (attached) - { - CurrentThread::detachQuery(); - } - } - }; - private: static void defaultThreadDeleter(); diff --git a/src/Common/ProfileEventsScope.cpp b/src/Common/ProfileEventsScope.cpp new file mode 100644 index 00000000000..6dcb5f8fcf3 --- /dev/null +++ b/src/Common/ProfileEventsScope.cpp @@ -0,0 +1,32 @@ +#include + +namespace DB +{ +extern thread_local constinit ProfileEvents::Counters * subthread_profile_events; + + +ScopedProfileEvents::ScopedProfileEvents() + : performance_counters_holder(std::make_unique()) + , performance_counters_scope(performance_counters_holder.get()) +{ + CurrentThread::get().attachProfileCountersScope(performance_counters_scope); +} + +ScopedProfileEvents::ScopedProfileEvents(ProfileEvents::Counters * performance_counters_scope_) + : performance_counters_scope(performance_counters_scope_) +{ + CurrentThread::get().attachProfileCountersScope(performance_counters_scope); +} + +std::shared_ptr ScopedProfileEvents::getSnapshot() +{ + return std::make_shared(performance_counters_scope->getPartiallyAtomicSnapshot()); +} + +ScopedProfileEvents::~ScopedProfileEvents() +{ + subthread_profile_events = nullptr; +} + + +} diff --git a/src/Common/ProfileEventsScope.h b/src/Common/ProfileEventsScope.h new file mode 100644 index 00000000000..59ba4f616d9 --- /dev/null +++ b/src/Common/ProfileEventsScope.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include + +namespace DB +{ + + +/// Use specific performance counters for current thread in the current scope. +class ScopedProfileEvents : private boost::noncopyable +{ +public: + /// Counters are owned by this object. + ScopedProfileEvents(); + + /// Shared counters are stored outisde. + /// Useful when we calculate metrics entering into some scope several times. + explicit ScopedProfileEvents(ProfileEvents::Counters * performance_counters_scope_); + + std::shared_ptr getSnapshot(); + + ~ScopedProfileEvents(); + +private: + /// If set, then performance_counters_scope is owned by this object. + /// Otherwise, counters are passed to the constructor from outside. + std::unique_ptr performance_counters_holder; + + ProfileEvents::Counters * performance_counters_scope; +}; + + +} + diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 9f9a78c4036..6949942f745 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -25,6 +25,7 @@ namespace ErrorCodes } thread_local ThreadStatus constinit * current_thread = nullptr; +thread_local ProfileEvents::Counters constinit * subthread_profile_events = nullptr; #if !defined(SANITIZER) namespace diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 69c5732ddb6..d44cd57b812 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -107,6 +107,7 @@ using ThreadGroupStatusPtr = std::shared_ptr; * - https://github.com/ClickHouse/ClickHouse/pull/40078 */ extern thread_local constinit ThreadStatus * current_thread; +extern thread_local constinit ProfileEvents::Counters * subthread_profile_events; /** Encapsulates all per-thread info (ProfileEvents, MemoryTracker, query_id, query context, etc.). * The object must be created in thread function and destroyed in the same thread before the exit. @@ -139,6 +140,7 @@ public: Deleter deleter; protected: + /// Group of threads, to which this thread attached ThreadGroupStatusPtr thread_group; std::atomic thread_state{ThreadState::DetachedFromQuery}; @@ -244,6 +246,8 @@ public: /// Attaches slave thread to existing thread group void attachQuery(const ThreadGroupStatusPtr & thread_group_, bool check_detached = true); + void attachProfileCountersScope(ProfileEvents::Counters * performance_counters_scope); + InternalTextLogsQueuePtr getInternalTextLogsQueue() const { return thread_state == Died ? nullptr : logs_queue_ptr.lock(); diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index b422f4f7f03..b736fa43e0c 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -189,16 +189,8 @@ void PartLogElement::appendToBlock(MutableColumns & columns) const } } - -bool PartLog::addNewPart( - ContextPtr current_context, const MutableDataPartPtr & part, UInt64 elapsed_ns, const ExecutionStatus & execution_status, std::shared_ptr profile_counters_) -{ - return addNewParts(current_context, {part}, elapsed_ns, execution_status, std::move(profile_counters_)); -} - - bool PartLog::addNewParts( - ContextPtr current_context, const PartLog::MutableDataPartsVector & parts, UInt64 elapsed_ns, const ExecutionStatus & execution_status, std::shared_ptr profile_counters_) + ContextPtr current_context, const PartLog::PartLogEntries & parts, const ExecutionStatus & execution_status) { if (parts.empty()) return true; @@ -207,15 +199,17 @@ bool PartLog::addNewParts( try { - auto table_id = parts.front()->storage.getStorageID(); + auto table_id = parts.front().part->storage.getStorageID(); part_log = current_context->getPartLog(table_id.database_name); // assume parts belong to the same table if (!part_log) return false; auto query_id = CurrentThread::getQueryId(); - for (const auto & part : parts) + for (const auto & part_log_entry : parts) { + const auto & part = part_log_entry.part; + PartLogElement elem; if (!query_id.empty()) @@ -228,7 +222,7 @@ bool PartLog::addNewParts( const auto time_now = std::chrono::system_clock::now(); elem.event_time = timeInSeconds(time_now); elem.event_time_microseconds = timeInMicroseconds(time_now); - elem.duration_ms = elapsed_ns / 1000000; + elem.duration_ms = part_log_entry.elapsed_ns / 1000000; elem.database_name = table_id.database_name; elem.table_name = table_id.table_name; @@ -245,7 +239,7 @@ bool PartLog::addNewParts( elem.error = static_cast(execution_status.code); elem.exception = execution_status.message; - elem.profile_counters = profile_counters_; + elem.profile_counters = part_log_entry.profile_counters; part_log->add(elem); } @@ -259,4 +253,28 @@ bool PartLog::addNewParts( return true; } +bool PartLog::addNewPart(ContextPtr context, const PartLog::PartLogEntry & part, const ExecutionStatus & execution_status) +{ + return addNewParts(context, {part}, execution_status); +} + +bool PartLog::addNewParts(ContextPtr context, const PartLog::MutableDataPartsVector & parts, UInt64 elapsed_ns, + const ExecutionStatus & execution_status) +{ + PartLog::PartLogEntries part_log_entries; + part_log_entries.reserve(parts.size()); + + for (const auto & part : parts) + part_log_entries.emplace_back(part, elapsed_ns); + + return addNewParts(context, part_log_entries, execution_status); +} + +bool PartLog::addNewPart(ContextPtr context, const PartLog::MutableDataPartPtr & part, UInt64 elapsed_ns, const ExecutionStatus & execution_status) +{ + PartLog::PartLogEntries part_log_entries; + part_log_entries.emplace_back(part, elapsed_ns); + return addNewParts(context, part_log_entries, execution_status); +} + } diff --git a/src/Interpreters/PartLog.h b/src/Interpreters/PartLog.h index 75b2539bda9..57044cba844 100644 --- a/src/Interpreters/PartLog.h +++ b/src/Interpreters/PartLog.h @@ -109,14 +109,42 @@ class PartLog : public SystemLog using MutableDataPartPtr = std::shared_ptr; using MutableDataPartsVector = std::vector; + using ProfileCountersSnapshotPtr = std::shared_ptr; + public: + struct PartLogEntry + { + std::shared_ptr part; + ProfileCountersSnapshotPtr profile_counters; + UInt64 elapsed_ns; + + PartLogEntry(std::shared_ptr part_, UInt64 elapsed_ns_) + : part(std::move(part_)), elapsed_ns(elapsed_ns_) + { + } + + PartLogEntry(std::shared_ptr part_, UInt64 elapsed_ns_, ProfileCountersSnapshotPtr profile_counters_) + : part(std::move(part_)) + , profile_counters(std::move(profile_counters_)) + , elapsed_ns(elapsed_ns_) + { + } + }; + + using PartLogEntries = std::vector; + + /// Add a record about creation of new part. + static bool addNewPart(ContextPtr context, const PartLogEntry & part, + const ExecutionStatus & execution_status = {}); + + static bool addNewParts(ContextPtr context, const PartLogEntries & parts, + const ExecutionStatus & execution_status = {}); + /// Add a record about creation of new part. static bool addNewPart(ContextPtr context, const MutableDataPartPtr & part, UInt64 elapsed_ns, - const ExecutionStatus & execution_status = {}, - std::shared_ptr profile_counters_ = {}); + const ExecutionStatus & execution_status = {}); static bool addNewParts(ContextPtr context, const MutableDataPartsVector & parts, UInt64 elapsed_ns, - const ExecutionStatus & execution_status = {}, - std::shared_ptr profile_counters_ = {}); + const ExecutionStatus & execution_status = {}); }; } diff --git a/src/Interpreters/ProfileEventsExt.cpp b/src/Interpreters/ProfileEventsExt.cpp index 0f6b52b2611..b5a49c8f381 100644 --- a/src/Interpreters/ProfileEventsExt.cpp +++ b/src/Interpreters/ProfileEventsExt.cpp @@ -19,6 +19,22 @@ std::shared_ptr TypeEnum = std::make_shared(GAUGE)}, }); +String dumpToString(const Counters::Snapshot & counters, bool nonzero_only) +{ + std::vector ss; + for (Event event = 0; event < Counters::num_counters; ++event) + { + UInt64 value = counters[event]; + + if (nonzero_only && 0 == value) + continue; + + const char * desc = getName(event); + ss.push_back(fmt::format("{}: {}", desc, value)); + } + return fmt::format("[{}]", fmt::join(ss, ", ")); +} + /// Put implementation here to avoid extra linking dependencies for clickhouse_common_io void dumpToMapColumn(const Counters::Snapshot & counters, DB::IColumn * column, bool nonzero_only) { diff --git a/src/Interpreters/ProfileEventsExt.h b/src/Interpreters/ProfileEventsExt.h index 7d9fc512d15..be88ca02593 100644 --- a/src/Interpreters/ProfileEventsExt.h +++ b/src/Interpreters/ProfileEventsExt.h @@ -21,6 +21,8 @@ struct ProfileEventsSnapshot using ThreadIdToCountersSnapshot = std::unordered_map; +String dumpToString(const Counters::Snapshot & counters, bool nonzero_only = true); + /// Dumps profile events to columns Map(String, UInt64) void dumpToMapColumn(const Counters::Snapshot & counters, DB::IColumn * column, bool nonzero_only = true); diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 816b03f3a0e..b4f82b2a215 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -161,6 +161,12 @@ void ThreadStatus::attachQuery(const ThreadGroupStatusPtr & thread_group_, bool setupState(thread_group_); } +void ThreadStatus::attachProfileCountersScope(ProfileEvents::Counters * performance_counters_scope) +{ + subthread_profile_events = performance_counters_scope; + performance_counters_scope->setParent(&performance_counters); +} + void ThreadStatus::initPerformanceCounters() { performance_counters_finalized = false; diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index 95bcfba937a..5346a932fba 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -28,7 +29,7 @@ void MergePlainMergeTreeTask::onCompleted() bool MergePlainMergeTreeTask::executeStep() { /// Metrics will be saved in the thread_group. - CurrentThread::ScopedAttach scoped_attach(thread_group); + ScopedProfileEvents profile_events_scope(&profile_counters); /// Make out memory tracker a parent of current thread memory tracker MemoryTrackerThreadSwitcherPtr switcher; @@ -90,7 +91,7 @@ void MergePlainMergeTreeTask::prepare() { auto & thread_status = CurrentThread::get(); thread_status.finalizePerformanceCounters(); - auto profile_counters = std::make_shared(thread_status.performance_counters.getPartiallyAtomicSnapshot()); + auto profile_counters_snapshot = std::make_shared(profile_counters.getPartiallyAtomicSnapshot()); merge_task.reset(); storage.writePartLog( PartLogElement::MERGE_PARTS, @@ -100,7 +101,7 @@ void MergePlainMergeTreeTask::prepare() new_part, future_part->parts, merge_list_entry.get(), - profile_counters); + std::move(profile_counters_snapshot)); }; merge_task = storage.merger_mutator.mergePartsToTemporaryPart( diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.h b/src/Storages/MergeTree/MergePlainMergeTreeTask.h index 4d0cecf1e55..eb659bf38ec 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.h +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.h @@ -83,7 +83,7 @@ private: MergeTreeTransactionHolder txn_holder; MergeTreeTransactionPtr txn; - ThreadGroupStatusPtr thread_group = std::make_shared(); + ProfileEvents::Counters profile_counters; }; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 07385067ede..c5160a27d48 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -7347,6 +7348,10 @@ try { part_log_elem.profile_counters = profile_counters; } + else + { + LOG_WARNING(log, "Profile counters are not set"); + } part_log->add(part_log_elem); } @@ -7483,6 +7488,7 @@ bool MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagge { Stopwatch stopwatch; MutableDataPartPtr cloned_part; + ScopedProfileEvents profile_events_scope; auto write_part_log = [&](const ExecutionStatus & execution_status) { @@ -7494,7 +7500,7 @@ bool MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagge cloned_part, {moving_part.part}, nullptr, - nullptr); + profile_events_scope.getSnapshot()); }; // Register in global moves list (StorageSystemMoves) diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index 99f6b1855e4..be645cf27c0 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace ProfileEvents { @@ -47,6 +48,7 @@ struct MergeTreeSink::DelayedChunk MergeTreeDataWriter::TemporaryPart temp_part; UInt64 elapsed_ns; String block_dedup_token; + ProfileEvents::Counters part_counters; }; std::vector partitions; @@ -70,12 +72,18 @@ void MergeTreeSink::consume(Chunk chunk) for (auto & current_block : part_blocks) { - Stopwatch watch; - String block_dedup_token; + ProfileEvents::Counters part_counters; - auto temp_part = storage.writer.writeTempPart(current_block, metadata_snapshot, context); + UInt64 elapsed_ns = 0; + MergeTreeDataWriter::TemporaryPart temp_part; - UInt64 elapsed_ns = watch.elapsed(); + { + ScopedProfileEvents scoped_attach(&part_counters); + + Stopwatch watch; + temp_part = storage.writer.writeTempPart(current_block, metadata_snapshot, context); + elapsed_ns = watch.elapsed(); + } /// If optimize_on_insert setting is true, current_block could become empty after merge /// and we didn't create part. @@ -85,6 +93,7 @@ void MergeTreeSink::consume(Chunk chunk) if (!support_parallel_write && temp_part.part->getDataPartStorage().supportParallelWrite()) support_parallel_write = true; + String block_dedup_token; if (storage.getDeduplicationLog()) { const String & dedup_token = settings.insert_deduplication_token; @@ -119,7 +128,8 @@ void MergeTreeSink::consume(Chunk chunk) { .temp_part = std::move(temp_part), .elapsed_ns = elapsed_ns, - .block_dedup_token = std::move(block_dedup_token) + .block_dedup_token = std::move(block_dedup_token), + .part_counters = std::move(part_counters), }); } @@ -135,6 +145,8 @@ void MergeTreeSink::finishDelayedChunk() for (auto & partition : delayed_chunk->partitions) { + ScopedProfileEvents scoped_attach(&partition.part_counters); + partition.temp_part.finalize(); auto & part = partition.temp_part.part; @@ -168,7 +180,8 @@ void MergeTreeSink::finishDelayedChunk() /// Part can be deduplicated, so increment counters and add to part log only if it's really added if (added) { - PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns); + auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); + PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, partition.elapsed_ns, counters_snapshot)); storage.incrementInsertedPartsProfileEvent(part->getType()); /// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'. diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index 2e546574e42..4428f6c2bce 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -184,12 +184,10 @@ ReplicatedMergeMutateTaskBase::PrepareResult MutateFromLogEntryTask::prepare() return {true, true, [this] (const ExecutionStatus & execution_status) { - auto & thread_status = CurrentThread::get(); - thread_status.finalizePerformanceCounters(); - auto profile_counters = std::make_shared(thread_status.performance_counters.getPartiallyAtomicSnapshot()); + auto profile_counters_snapshot = std::make_shared(profile_counters.getPartiallyAtomicSnapshot()); storage.writePartLog( PartLogElement::MUTATE_PART, execution_status, stopwatch_ptr->elapsed(), - entry.new_part_name, new_part, future_mutated_part->parts, merge_mutate_entry.get(), std::move(profile_counters)); + entry.new_part_name, new_part, future_mutated_part->parts, merge_mutate_entry.get(), std::move(profile_counters_snapshot)); }}; } diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 4a29291b20d..944a2d1070b 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -2,6 +2,7 @@ #include #include +#include namespace DB { @@ -38,9 +39,7 @@ void MutatePlainMergeTreeTask::prepare() write_part_log = [this] (const ExecutionStatus & execution_status) { - auto & thread_status = CurrentThread::get(); - thread_status.finalizePerformanceCounters(); - auto profile_counters = std::make_shared(thread_status.performance_counters.getPartiallyAtomicSnapshot()); + auto profile_counters_snapshot = std::make_shared(profile_counters.getPartiallyAtomicSnapshot()); mutate_task.reset(); storage.writePartLog( PartLogElement::MUTATE_PART, @@ -50,7 +49,7 @@ void MutatePlainMergeTreeTask::prepare() new_part, future_part->parts, merge_list_entry.get(), - profile_counters); + std::move(profile_counters_snapshot)); }; fake_query_context = Context::createCopy(storage.getContext()); @@ -65,8 +64,8 @@ void MutatePlainMergeTreeTask::prepare() bool MutatePlainMergeTreeTask::executeStep() { - /// Metrics will be saved in the thread_group. - CurrentThread::ScopedAttach scoped_attach(thread_group); + /// Metrics will be saved in the local profile_counters. + ScopedProfileEvents profile_events_scope(&profile_counters); /// Make out memory tracker a parent of current thread memory tracker MemoryTrackerThreadSwitcherPtr switcher; diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.h b/src/Storages/MergeTree/MutatePlainMergeTreeTask.h index 6356595a16b..ae2ac039543 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.h +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.h @@ -78,7 +78,7 @@ private: ContextMutablePtr fake_query_context; MutateTaskPtr mutate_task; - ThreadGroupStatusPtr thread_group = std::make_shared(); + ProfileEvents::Counters profile_counters; }; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 295a88d5f76..a5ea5c90b95 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -20,6 +20,7 @@ #include #include #include +#include namespace CurrentMetrics @@ -814,8 +815,8 @@ public: bool executeStep() override { - /// Metrics will be saved in the thread_group. - CurrentThread::ScopedAttach scoped_attach(thread_group); + /// Metrics will be saved in the local profile_counters. + ScopedProfileEvents profile_events_scope(&profile_counters); auto & current_level_parts = level_parts[current_level]; auto & next_level_parts = level_parts[next_level]; @@ -918,7 +919,7 @@ private: /// TODO(nikitamikhaylov): make this constant a setting static constexpr size_t max_parts_to_merge_in_one_level = 10; - ThreadGroupStatusPtr thread_group = std::make_shared(); + ProfileEvents::Counters profile_counters; }; @@ -929,7 +930,9 @@ private: // In short it executed a mutation for the part an original part and for its every projection -/** An overview of how the process of mutation works for projections: +/** + * + * An overview of how the process of mutation works for projections: * * The mutation for original parts is executed block by block, * but additionally we execute a SELECT query for each projection over a current block. @@ -1140,8 +1143,8 @@ public: bool executeStep() override { - /// Metrics will be saved in the thread_group. - CurrentThread::ScopedAttach scoped_attach(thread_group); + /// Metrics will be saved in the local profile_counters. + ScopedProfileEvents profile_events_scope(&profile_counters); switch (state) { @@ -1176,6 +1179,7 @@ public: } private: + void prepare() { if (ctx->new_data_part->isStoredOnDisk()) @@ -1258,7 +1262,7 @@ private: std::unique_ptr part_merger_writer_task; - ThreadGroupStatusPtr thread_group = std::make_shared(); + ProfileEvents::Counters profile_counters; }; @@ -1273,8 +1277,8 @@ public: bool executeStep() override { - /// Metrics will be saved in the thread_group. - CurrentThread::ScopedAttach scoped_attach(thread_group); + /// Metrics will be saved in the local profile_counters. + ScopedProfileEvents profile_events_scope(&profile_counters); switch (state) { @@ -1308,6 +1312,7 @@ public: } private: + void prepare() { if (ctx->execute_ttl_type != ExecuteTTLType::NONE) @@ -1466,7 +1471,7 @@ private: MergedColumnOnlyOutputStreamPtr out; std::unique_ptr part_merger_writer_task{nullptr}; - ThreadGroupStatusPtr thread_group = std::make_shared(); + ProfileEvents::Counters profile_counters; }; diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index 2ef36654d95..d81bbd67441 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -29,8 +30,8 @@ void ReplicatedMergeMutateTaskBase::onCompleted() bool ReplicatedMergeMutateTaskBase::executeStep() { - /// Metrics will be saved in the thread_group. - CurrentThread::ScopedAttach scoped_attach(thread_group); + /// Metrics will be saved in the local profile_counters. + ScopedProfileEvents profile_events_scope(&profile_counters); std::exception_ptr saved_exception; diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h index e08074cd4ff..d9a1cbff166 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h @@ -11,9 +11,9 @@ namespace DB class StorageReplicatedMergeTree; - -/** This is used as a base of MergeFromLogEntryTask and MutateFromLogEntryTaskBase - */ +/** + * This is used as a base of MergeFromLogEntryTask and MutateFromLogEntryTaskBase + */ class ReplicatedMergeMutateTaskBase : public IExecutableTask { public: @@ -33,11 +33,8 @@ public: } ~ReplicatedMergeMutateTaskBase() override = default; - void onCompleted() override; - StorageID getStorageID() override; - bool executeStep() override; protected: @@ -63,6 +60,8 @@ protected: MergeList::EntryPtr merge_mutate_entry{nullptr}; Poco::Logger * log; StorageReplicatedMergeTree & storage; + /// ProfileEvents for current part will be stored here + ProfileEvents::Counters profile_counters; private: enum class CheckExistingPartResult @@ -86,7 +85,6 @@ private: PartLogWriter part_log_writer{}; State state{State::NEED_PREPARE}; IExecutableTask::TaskResultCallback task_result_callback; - ThreadGroupStatusPtr thread_group = std::make_shared(); }; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e9958a68406..b9c178cab5c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2,8 +2,10 @@ #include #include -#include "Common/hex.h" + +#include #include +#include #include #include #include @@ -1592,6 +1594,8 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) if (entry.type == LogEntry::ATTACH_PART) { + ScopedProfileEvents profile_events_scope; + if (MutableDataPartPtr part = attachPartHelperFoundValidPart(entry)) { LOG_TRACE(log, "Found valid local part for {}, preparing the transaction", part->name); @@ -1602,11 +1606,9 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) renameTempPartAndReplace(part, transaction); checkPartChecksumsAndCommit(transaction, part); - auto & thread_status = CurrentThread::get(); - thread_status.finalizePerformanceCounters(); - auto profile_counters = std::make_shared(thread_status.performance_counters.getPartiallyAtomicSnapshot()); writePartLog(PartLogElement::Type::NEW_PART, {}, 0 /** log entry is fake so we don't measure the time */, - part->name, part, {} /** log entry is fake so there are no initial parts */, nullptr, std::move(profile_counters)); + part->name, part, {} /** log entry is fake so there are no initial parts */, nullptr, + profile_events_scope.getSnapshot()); return true; } @@ -4007,15 +4009,14 @@ bool StorageReplicatedMergeTree::fetchPart( Stopwatch stopwatch; MutableDataPartPtr part; DataPartsVector replaced_parts; + ScopedProfileEvents profile_events_scope; auto write_part_log = [&] (const ExecutionStatus & execution_status) { - auto & thread_status = CurrentThread::get(); - thread_status.finalizePerformanceCounters(); - auto profile_counters = std::make_shared(thread_status.performance_counters.getPartiallyAtomicSnapshot()); writePartLog( PartLogElement::DOWNLOAD_PART, execution_status, stopwatch.elapsed(), - part_name, part, replaced_parts, nullptr, std::move(profile_counters)); + part_name, part, replaced_parts, nullptr, + profile_events_scope.getSnapshot()); }; DataPartPtr part_to_clone; @@ -4246,15 +4247,14 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( Stopwatch stopwatch; MutableDataPartPtr part; DataPartsVector replaced_parts; + ScopedProfileEvents profile_events_scope; auto write_part_log = [&] (const ExecutionStatus & execution_status) { - auto & thread_status = CurrentThread::get(); - thread_status.finalizePerformanceCounters(); - auto profile_counters = std::make_shared(thread_status.performance_counters.getPartiallyAtomicSnapshot()); writePartLog( PartLogElement::DOWNLOAD_PART, execution_status, stopwatch.elapsed(), - part_name, part, replaced_parts, nullptr, std::move(profile_counters)); + part_name, part, replaced_parts, nullptr, + profile_events_scope.getSnapshot()); }; std::function get_part; diff --git a/tests/queries/0_stateless/02378_part_log_profile_events.reference b/tests/queries/0_stateless/02378_part_log_profile_events.reference index d00491fd7e5..e8183f05f5d 100644 --- a/tests/queries/0_stateless/02378_part_log_profile_events.reference +++ b/tests/queries/0_stateless/02378_part_log_profile_events.reference @@ -1 +1,3 @@ 1 +1 +1 diff --git a/tests/queries/0_stateless/02378_part_log_profile_events.sql b/tests/queries/0_stateless/02378_part_log_profile_events.sql index 9ffd6aa4135..af8fe8a2669 100644 --- a/tests/queries/0_stateless/02378_part_log_profile_events.sql +++ b/tests/queries/0_stateless/02378_part_log_profile_events.sql @@ -1,13 +1,47 @@ DROP TABLE IF EXISTS test; -CREATE TABLE test (x UInt64) ENGINE = MergeTree ORDER BY x; -SET max_block_size = 1; -INSERT INTO test SELECT * FROM system.numbers LIMIT 1000; +CREATE TABLE test (key UInt64, val UInt64) engine = MergeTree Order by key PARTITION BY key >= 128; + +SET max_block_size = 64, max_insert_block_size = 64, min_insert_block_size_rows = 64; + +INSERT INTO test SELECT number AS key, sipHash64(number) AS val FROM numbers(512); + +SYSTEM FLUSH LOGS; +SELECT + count(DISTINCT query_id) == 1 + AND count() >= 512 / 64 -- 512 rows inserted, 64 rows per block + AND SUM(ProfileEvents['MergeTreeDataWriterRows']) == 512 + AND SUM(ProfileEvents['MergeTreeDataWriterUncompressedBytes']) > 1024 + AND SUM(ProfileEvents['MergeTreeDataWriterCompressedBytes']) > 1024 + AND SUM(ProfileEvents['MergeTreeDataWriterBlocks']) >= 8 +FROM system.part_log +WHERE event_time > now() - INTERVAL 10 MINUTE + AND database == currentDatabase() AND table == 'test' + AND event_type == 'NewPart' +; + OPTIMIZE TABLE test FINAL; SYSTEM FLUSH LOGS; +SELECT + count() >= 2 AND SUM(ProfileEvents['MergedRows']) >= 512 +FROM system.part_log +WHERE event_time > now() - INTERVAL 10 MINUTE + AND database == currentDatabase() AND table == 'test' + AND event_type == 'MergeParts' +; --- ProfileEvents field exist and contains something plausible: -SELECT count() > 0 FROM system.part_log WHERE ProfileEvents['MergedRows'] = 1000 AND table = 'test' AND database = currentDatabase() AND event_time >= now() - 600; +ALTER TABLE test UPDATE val = 0 WHERE key % 2 == 0 SETTINGS mutations_sync = 2; + +SYSTEM FLUSH LOGS; +SELECT + count() == 2 + AND SUM(ProfileEvents['SelectedRows']) == 512 + AND SUM(ProfileEvents['FileOpen']) > 2 +FROM system.part_log +WHERE event_time > now() - INTERVAL 10 MINUTE + AND database == currentDatabase() AND table == 'test' + AND event_type == 'MutatePart' +; DROP TABLE test; From 3f77bc0ecf219e60798f551bc92778ce38c4ff74 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 23 Jan 2023 18:35:18 +0000 Subject: [PATCH 077/253] Add 02378_part_log_profile_events_replicated --- ...rt_log_profile_events_replicated.reference | 1 + ...378_part_log_profile_events_replicated.sql | 40 +++++++++++++++++++ 2 files changed, 41 insertions(+) create mode 100644 tests/queries/0_stateless/02378_part_log_profile_events_replicated.reference create mode 100644 tests/queries/0_stateless/02378_part_log_profile_events_replicated.sql diff --git a/tests/queries/0_stateless/02378_part_log_profile_events_replicated.reference b/tests/queries/0_stateless/02378_part_log_profile_events_replicated.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02378_part_log_profile_events_replicated.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02378_part_log_profile_events_replicated.sql b/tests/queries/0_stateless/02378_part_log_profile_events_replicated.sql new file mode 100644 index 00000000000..d61b680bb87 --- /dev/null +++ b/tests/queries/0_stateless/02378_part_log_profile_events_replicated.sql @@ -0,0 +1,40 @@ + +-- Tags: long, replica, no-replicated-database, no-parallel + +DROP TABLE IF EXISTS part_log_profile_events_r1 NO DELAY; +DROP TABLE IF EXISTS part_log_profile_events_r2 NO DELAY; + +CREATE TABLE part_log_profile_events_r1 (x UInt64) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_02378/part_log_profile_events', 'r1') +ORDER BY x +PARTITION BY x >= 128 +; + +CREATE TABLE part_log_profile_events_r2 (x UInt64) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_02378/part_log_profile_events', 'r2') +ORDER BY x +PARTITION BY x >= 128 +; + +-- SYSTEM STOP MERGES part_log_profile_events_r1; +-- SYSTEM STOP MERGES part_log_profile_events_r2; + +SET max_block_size = 64, max_insert_block_size = 64, min_insert_block_size_rows = 64; + +INSERT INTO part_log_profile_events_r1 SELECT number FROM numbers(1000); + +SYSTEM SYNC REPLICA part_log_profile_events_r2; + +SYSTEM FLUSH LOGS; + +SELECT + count() > 1 + AND SUM(ProfileEvents['ZooKeeperTransactions']) >= 4 +FROM system.part_log +WHERE event_time > now() - INTERVAL 10 MINUTE + AND database == currentDatabase() AND table == 'part_log_profile_events_r2' + AND event_type == 'DownloadPart' +; + +DROP TABLE part_log_profile_events_r1 NO DELAY; +DROP TABLE part_log_profile_events_r2 NO DELAY; From 994dcb93f95cf269cf4a1db3aa793cfffe8114d3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 24 Jan 2023 08:24:36 +0000 Subject: [PATCH 078/253] Rename ScopedProfileEvents -> ProfileEventsScope --- src/Common/ProfileEventsScope.cpp | 9 +++++---- src/Common/ProfileEventsScope.h | 11 +++++------ src/Storages/MergeTree/MergePlainMergeTreeTask.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 3 ++- src/Storages/MergeTree/MergeTreeSink.cpp | 4 ++-- src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp | 2 +- src/Storages/MergeTree/MutateTask.cpp | 6 +++--- .../MergeTree/ReplicatedMergeMutateTaskBase.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 6 +++--- 9 files changed, 23 insertions(+), 22 deletions(-) diff --git a/src/Common/ProfileEventsScope.cpp b/src/Common/ProfileEventsScope.cpp index 6dcb5f8fcf3..d5181692cae 100644 --- a/src/Common/ProfileEventsScope.cpp +++ b/src/Common/ProfileEventsScope.cpp @@ -2,28 +2,29 @@ namespace DB { + extern thread_local constinit ProfileEvents::Counters * subthread_profile_events; -ScopedProfileEvents::ScopedProfileEvents() +ProfileEventsScope::ProfileEventsScope() : performance_counters_holder(std::make_unique()) , performance_counters_scope(performance_counters_holder.get()) { CurrentThread::get().attachProfileCountersScope(performance_counters_scope); } -ScopedProfileEvents::ScopedProfileEvents(ProfileEvents::Counters * performance_counters_scope_) +ProfileEventsScope::ProfileEventsScope(ProfileEvents::Counters * performance_counters_scope_) : performance_counters_scope(performance_counters_scope_) { CurrentThread::get().attachProfileCountersScope(performance_counters_scope); } -std::shared_ptr ScopedProfileEvents::getSnapshot() +std::shared_ptr ProfileEventsScope::getSnapshot() { return std::make_shared(performance_counters_scope->getPartiallyAtomicSnapshot()); } -ScopedProfileEvents::~ScopedProfileEvents() +ProfileEventsScope::~ProfileEventsScope() { subthread_profile_events = nullptr; } diff --git a/src/Common/ProfileEventsScope.h b/src/Common/ProfileEventsScope.h index 59ba4f616d9..653a1ed55e7 100644 --- a/src/Common/ProfileEventsScope.h +++ b/src/Common/ProfileEventsScope.h @@ -6,21 +6,20 @@ namespace DB { - /// Use specific performance counters for current thread in the current scope. -class ScopedProfileEvents : private boost::noncopyable +class ProfileEventsScope : private boost::noncopyable { public: /// Counters are owned by this object. - ScopedProfileEvents(); + ProfileEventsScope(); - /// Shared counters are stored outisde. + /// Shared counters are stored outside. /// Useful when we calculate metrics entering into some scope several times. - explicit ScopedProfileEvents(ProfileEvents::Counters * performance_counters_scope_); + explicit ProfileEventsScope(ProfileEvents::Counters * performance_counters_scope_); std::shared_ptr getSnapshot(); - ~ScopedProfileEvents(); + ~ProfileEventsScope(); private: /// If set, then performance_counters_scope is owned by this object. diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index 5346a932fba..de88a1984e7 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -29,7 +29,7 @@ void MergePlainMergeTreeTask::onCompleted() bool MergePlainMergeTreeTask::executeStep() { /// Metrics will be saved in the thread_group. - ScopedProfileEvents profile_events_scope(&profile_counters); + ProfileEventsScope profile_events_scope(&profile_counters); /// Make out memory tracker a parent of current thread memory tracker MemoryTrackerThreadSwitcherPtr switcher; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c5160a27d48..57924331cce 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7488,7 +7488,8 @@ bool MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagge { Stopwatch stopwatch; MutableDataPartPtr cloned_part; - ScopedProfileEvents profile_events_scope; + ProfileEventsScope profile_events_scope; + auto write_part_log = [&](const ExecutionStatus & execution_status) { diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index be645cf27c0..1e607767f86 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -78,7 +78,7 @@ void MergeTreeSink::consume(Chunk chunk) MergeTreeDataWriter::TemporaryPart temp_part; { - ScopedProfileEvents scoped_attach(&part_counters); + ProfileEventsScope scoped_attach(&part_counters); Stopwatch watch; temp_part = storage.writer.writeTempPart(current_block, metadata_snapshot, context); @@ -145,7 +145,7 @@ void MergeTreeSink::finishDelayedChunk() for (auto & partition : delayed_chunk->partitions) { - ScopedProfileEvents scoped_attach(&partition.part_counters); + ProfileEventsScope scoped_attach(&partition.part_counters); partition.temp_part.finalize(); diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 944a2d1070b..9bd0f148d6c 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -65,7 +65,7 @@ void MutatePlainMergeTreeTask::prepare() bool MutatePlainMergeTreeTask::executeStep() { /// Metrics will be saved in the local profile_counters. - ScopedProfileEvents profile_events_scope(&profile_counters); + ProfileEventsScope profile_events_scope(&profile_counters); /// Make out memory tracker a parent of current thread memory tracker MemoryTrackerThreadSwitcherPtr switcher; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index a5ea5c90b95..563137e6a8c 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -816,7 +816,7 @@ public: bool executeStep() override { /// Metrics will be saved in the local profile_counters. - ScopedProfileEvents profile_events_scope(&profile_counters); + ProfileEventsScope profile_events_scope(&profile_counters); auto & current_level_parts = level_parts[current_level]; auto & next_level_parts = level_parts[next_level]; @@ -1144,7 +1144,7 @@ public: bool executeStep() override { /// Metrics will be saved in the local profile_counters. - ScopedProfileEvents profile_events_scope(&profile_counters); + ProfileEventsScope profile_events_scope(&profile_counters); switch (state) { @@ -1278,7 +1278,7 @@ public: bool executeStep() override { /// Metrics will be saved in the local profile_counters. - ScopedProfileEvents profile_events_scope(&profile_counters); + ProfileEventsScope profile_events_scope(&profile_counters); switch (state) { diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index d81bbd67441..9ce7eb42666 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -31,7 +31,7 @@ void ReplicatedMergeMutateTaskBase::onCompleted() bool ReplicatedMergeMutateTaskBase::executeStep() { /// Metrics will be saved in the local profile_counters. - ScopedProfileEvents profile_events_scope(&profile_counters); + ProfileEventsScope profile_events_scope(&profile_counters); std::exception_ptr saved_exception; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b9c178cab5c..8fc2adcad07 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1594,7 +1594,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) if (entry.type == LogEntry::ATTACH_PART) { - ScopedProfileEvents profile_events_scope; + ProfileEventsScope profile_events_scope; if (MutableDataPartPtr part = attachPartHelperFoundValidPart(entry)) { @@ -4009,7 +4009,7 @@ bool StorageReplicatedMergeTree::fetchPart( Stopwatch stopwatch; MutableDataPartPtr part; DataPartsVector replaced_parts; - ScopedProfileEvents profile_events_scope; + ProfileEventsScope profile_events_scope; auto write_part_log = [&] (const ExecutionStatus & execution_status) { @@ -4247,7 +4247,7 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( Stopwatch stopwatch; MutableDataPartPtr part; DataPartsVector replaced_parts; - ScopedProfileEvents profile_events_scope; + ProfileEventsScope profile_events_scope; auto write_part_log = [&] (const ExecutionStatus & execution_status) { From 07d7478bc71e963bca3b9e80ca0a8fe1fe4da7ca Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 24 Jan 2023 11:23:17 +0000 Subject: [PATCH 079/253] wip --- .../MergeTree/MergeFromLogEntryTask.cpp | 10 ++++---- .../MergeTree/MergePlainMergeTreeTask.cpp | 2 -- .../02378_part_log_profile_events.reference | 6 ++--- .../02378_part_log_profile_events.sql | 23 +++++++++++-------- 4 files changed, 22 insertions(+), 19 deletions(-) diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index abb871be372..0d8dc7f68a6 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -2,6 +2,7 @@ #include #include +#include #include namespace ProfileEvents @@ -267,6 +268,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() transaction_ptr = std::make_unique(storage, NO_TRANSACTION_RAW); stopwatch_ptr = std::make_unique(); + ProfileEventsScope profile_events_scope(&profile_counters); merge_task = storage.merger_mutator.mergePartsToTemporaryPart( future_merged_part, @@ -289,18 +291,18 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() return {true, true, [this, stopwatch = *stopwatch_ptr] (const ExecutionStatus & execution_status) { - auto & thread_status = CurrentThread::get(); - thread_status.finalizePerformanceCounters(); - auto profile_counters = std::make_shared(thread_status.performance_counters.getPartiallyAtomicSnapshot()); + auto profile_counters_snapshot = std::make_shared(profile_counters.getPartiallyAtomicSnapshot()); storage.writePartLog( PartLogElement::MERGE_PARTS, execution_status, stopwatch.elapsed(), - entry.new_part_name, part, parts, merge_mutate_entry.get(), std::move(profile_counters)); + entry.new_part_name, part, parts, merge_mutate_entry.get(), std::move(profile_counters_snapshot)); }}; } bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWriter write_part_log) { + ProfileEventsScope profile_events_scope(&profile_counters); + part = merge_task->getFuture().get(); storage.merger_mutator.renameMergedTemporaryPart(part, parts, NO_TRANSACTION_PTR, *transaction_ptr); diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index de88a1984e7..855b93dc90e 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -89,8 +89,6 @@ void MergePlainMergeTreeTask::prepare() write_part_log = [this] (const ExecutionStatus & execution_status) { - auto & thread_status = CurrentThread::get(); - thread_status.finalizePerformanceCounters(); auto profile_counters_snapshot = std::make_shared(profile_counters.getPartiallyAtomicSnapshot()); merge_task.reset(); storage.writePartLog( diff --git a/tests/queries/0_stateless/02378_part_log_profile_events.reference b/tests/queries/0_stateless/02378_part_log_profile_events.reference index e8183f05f5d..c09e6c997c5 100644 --- a/tests/queries/0_stateless/02378_part_log_profile_events.reference +++ b/tests/queries/0_stateless/02378_part_log_profile_events.reference @@ -1,3 +1,3 @@ -1 -1 -1 +Ok Ok Ok Ok Ok Ok +Ok Ok +Ok Ok Ok diff --git a/tests/queries/0_stateless/02378_part_log_profile_events.sql b/tests/queries/0_stateless/02378_part_log_profile_events.sql index af8fe8a2669..af68360dbbb 100644 --- a/tests/queries/0_stateless/02378_part_log_profile_events.sql +++ b/tests/queries/0_stateless/02378_part_log_profile_events.sql @@ -7,13 +7,14 @@ SET max_block_size = 64, max_insert_block_size = 64, min_insert_block_size_rows INSERT INTO test SELECT number AS key, sipHash64(number) AS val FROM numbers(512); SYSTEM FLUSH LOGS; + SELECT - count(DISTINCT query_id) == 1 - AND count() >= 512 / 64 -- 512 rows inserted, 64 rows per block - AND SUM(ProfileEvents['MergeTreeDataWriterRows']) == 512 - AND SUM(ProfileEvents['MergeTreeDataWriterUncompressedBytes']) > 1024 - AND SUM(ProfileEvents['MergeTreeDataWriterCompressedBytes']) > 1024 - AND SUM(ProfileEvents['MergeTreeDataWriterBlocks']) >= 8 + if(count(DISTINCT query_id) == 1, 'Ok', 'Error: ' || toString(count(DISTINCT query_id))), + if(count() == 512 / 64, 'Ok', 'Error: ' || toString(count())), -- 512 rows inserted, 64 rows per block + if(SUM(ProfileEvents['MergeTreeDataWriterRows']) == 512, 'Ok', 'Error: ' || toString(SUM(ProfileEvents['MergeTreeDataWriterRows']))), + if(SUM(ProfileEvents['MergeTreeDataWriterUncompressedBytes']) >= 1024, 'Ok', 'Error: ' || toString(SUM(ProfileEvents['MergeTreeDataWriterUncompressedBytes']))), + if(SUM(ProfileEvents['MergeTreeDataWriterCompressedBytes']) >= 1024, 'Ok', 'Error: ' || toString(SUM(ProfileEvents['MergeTreeDataWriterCompressedBytes']))), + if(SUM(ProfileEvents['MergeTreeDataWriterBlocks']) >= 8, 'Ok', 'Error: ' || toString(SUM(ProfileEvents['MergeTreeDataWriterBlocks']))) FROM system.part_log WHERE event_time > now() - INTERVAL 10 MINUTE AND database == currentDatabase() AND table == 'test' @@ -24,7 +25,8 @@ OPTIMIZE TABLE test FINAL; SYSTEM FLUSH LOGS; SELECT - count() >= 2 AND SUM(ProfileEvents['MergedRows']) >= 512 + if(count() > 2, 'Ok', 'Error: ' || toString(count())), + if(SUM(ProfileEvents['MergedRows']) >= 512, 'Ok', 'Error: ' || toString(SUM(ProfileEvents['MergedRows']))) FROM system.part_log WHERE event_time > now() - INTERVAL 10 MINUTE AND database == currentDatabase() AND table == 'test' @@ -34,10 +36,11 @@ WHERE event_time > now() - INTERVAL 10 MINUTE ALTER TABLE test UPDATE val = 0 WHERE key % 2 == 0 SETTINGS mutations_sync = 2; SYSTEM FLUSH LOGS; + SELECT - count() == 2 - AND SUM(ProfileEvents['SelectedRows']) == 512 - AND SUM(ProfileEvents['FileOpen']) > 2 + if(count() == 2, 'Ok', 'Error: ' || toString(count())), + if(SUM(ProfileEvents['SelectedRows']) == 512, 'Ok', 'Error: ' || toString(SUM(ProfileEvents['SelectedRows']))), + if(SUM(ProfileEvents['FileOpen']) > 1, 'Ok', 'Error: ' || toString(SUM(ProfileEvents['FileOpen']))) FROM system.part_log WHERE event_time > now() - INTERVAL 10 MINUTE AND database == currentDatabase() AND table == 'test' From fa08f72bba61dcf76dc2af93d18604dfbf23481a Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 31 Jan 2023 12:34:07 +0000 Subject: [PATCH 080/253] Move subthread_profile_events to ThreadStatus --- src/Common/CurrentThread.cpp | 4 +--- src/Common/ProfileEventsScope.cpp | 4 +--- src/Common/ThreadStatus.cpp | 1 - src/Common/ThreadStatus.h | 6 +++++- src/Interpreters/ThreadStatusExt.cpp | 7 ++++++- 5 files changed, 13 insertions(+), 9 deletions(-) diff --git a/src/Common/CurrentThread.cpp b/src/Common/CurrentThread.cpp index 14c9f4418e0..a176a19673b 100644 --- a/src/Common/CurrentThread.cpp +++ b/src/Common/CurrentThread.cpp @@ -40,9 +40,7 @@ ThreadStatus & CurrentThread::get() ProfileEvents::Counters & CurrentThread::getProfileEvents() { - if (unlikely(subthread_profile_events)) - return *subthread_profile_events; - return current_thread ? current_thread->performance_counters : ProfileEvents::global_counters; + return current_thread ? *current_thread->current_performance_counters : ProfileEvents::global_counters; } void CurrentThread::updateProgressIn(const Progress & value) diff --git a/src/Common/ProfileEventsScope.cpp b/src/Common/ProfileEventsScope.cpp index d5181692cae..5b996209b76 100644 --- a/src/Common/ProfileEventsScope.cpp +++ b/src/Common/ProfileEventsScope.cpp @@ -3,8 +3,6 @@ namespace DB { -extern thread_local constinit ProfileEvents::Counters * subthread_profile_events; - ProfileEventsScope::ProfileEventsScope() : performance_counters_holder(std::make_unique()) @@ -26,7 +24,7 @@ std::shared_ptr ProfileEventsScope::getSnapsh ProfileEventsScope::~ProfileEventsScope() { - subthread_profile_events = nullptr; + CurrentThread::get().detachProfileCountersScope(); } diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 6949942f745..9f9a78c4036 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -25,7 +25,6 @@ namespace ErrorCodes } thread_local ThreadStatus constinit * current_thread = nullptr; -thread_local ProfileEvents::Counters constinit * subthread_profile_events = nullptr; #if !defined(SANITIZER) namespace diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index d44cd57b812..ac53faa2a75 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -107,7 +107,6 @@ using ThreadGroupStatusPtr = std::shared_ptr; * - https://github.com/ClickHouse/ClickHouse/pull/40078 */ extern thread_local constinit ThreadStatus * current_thread; -extern thread_local constinit ProfileEvents::Counters * subthread_profile_events; /** Encapsulates all per-thread info (ProfileEvents, MemoryTracker, query_id, query context, etc.). * The object must be created in thread function and destroyed in the same thread before the exit. @@ -125,6 +124,10 @@ public: /// TODO: merge them into common entity ProfileEvents::Counters performance_counters{VariableContext::Thread}; + + /// Points to performance_counters by default. + /// Could be changed to point to another object to caclulate performance counters for some narrow scope. + ProfileEvents::Counters * current_performance_counters{&performance_counters}; MemoryTracker memory_tracker{VariableContext::Thread}; /// Small amount of untracked memory (per thread atomic-less counter) @@ -247,6 +250,7 @@ public: void attachQuery(const ThreadGroupStatusPtr & thread_group_, bool check_detached = true); void attachProfileCountersScope(ProfileEvents::Counters * performance_counters_scope); + void detachProfileCountersScope(); InternalTextLogsQueuePtr getInternalTextLogsQueue() const { diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index b4f82b2a215..7c1a1fb7bb4 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -163,8 +163,13 @@ void ThreadStatus::attachQuery(const ThreadGroupStatusPtr & thread_group_, bool void ThreadStatus::attachProfileCountersScope(ProfileEvents::Counters * performance_counters_scope) { - subthread_profile_events = performance_counters_scope; performance_counters_scope->setParent(&performance_counters); + current_performance_counters = performance_counters_scope; +} + +void ThreadStatus::detachProfileCountersScope() +{ + current_performance_counters = &performance_counters; } void ThreadStatus::initPerformanceCounters() From 7a66c0105a0c2641bd632395d16cc1642b5d6268 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 31 Jan 2023 12:59:15 +0000 Subject: [PATCH 081/253] Remove unused ProfileEventsExt::dumpToString --- src/Interpreters/ProfileEventsExt.cpp | 16 ---------------- src/Interpreters/ProfileEventsExt.h | 2 -- 2 files changed, 18 deletions(-) diff --git a/src/Interpreters/ProfileEventsExt.cpp b/src/Interpreters/ProfileEventsExt.cpp index b5a49c8f381..0f6b52b2611 100644 --- a/src/Interpreters/ProfileEventsExt.cpp +++ b/src/Interpreters/ProfileEventsExt.cpp @@ -19,22 +19,6 @@ std::shared_ptr TypeEnum = std::make_shared(GAUGE)}, }); -String dumpToString(const Counters::Snapshot & counters, bool nonzero_only) -{ - std::vector ss; - for (Event event = 0; event < Counters::num_counters; ++event) - { - UInt64 value = counters[event]; - - if (nonzero_only && 0 == value) - continue; - - const char * desc = getName(event); - ss.push_back(fmt::format("{}: {}", desc, value)); - } - return fmt::format("[{}]", fmt::join(ss, ", ")); -} - /// Put implementation here to avoid extra linking dependencies for clickhouse_common_io void dumpToMapColumn(const Counters::Snapshot & counters, DB::IColumn * column, bool nonzero_only) { diff --git a/src/Interpreters/ProfileEventsExt.h b/src/Interpreters/ProfileEventsExt.h index be88ca02593..7d9fc512d15 100644 --- a/src/Interpreters/ProfileEventsExt.h +++ b/src/Interpreters/ProfileEventsExt.h @@ -21,8 +21,6 @@ struct ProfileEventsSnapshot using ThreadIdToCountersSnapshot = std::unordered_map; -String dumpToString(const Counters::Snapshot & counters, bool nonzero_only = true); - /// Dumps profile events to columns Map(String, UInt64) void dumpToMapColumn(const Counters::Snapshot & counters, DB::IColumn * column, bool nonzero_only = true); From 4c1aeb797209999136cee0c9497e3ea014f0db07 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 31 Jan 2023 14:01:12 +0000 Subject: [PATCH 082/253] Use overload of PartLog::addNewPart with profile_events --- src/Common/ThreadStatus.h | 2 +- src/Interpreters/PartLog.cpp | 17 +++------ src/Interpreters/PartLog.h | 8 ++--- src/Storages/MergeTree/MergeTreeData.cpp | 1 - .../MergeTree/ReplicatedMergeTreeSink.cpp | 36 ++++++++++++++----- src/Storages/StorageMergeTree.cpp | 23 ++++++++---- src/Storages/StorageReplicatedMergeTree.cpp | 18 ++++++---- 7 files changed, 64 insertions(+), 41 deletions(-) diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index ac53faa2a75..531dd669e65 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -126,7 +126,7 @@ public: ProfileEvents::Counters performance_counters{VariableContext::Thread}; /// Points to performance_counters by default. - /// Could be changed to point to another object to caclulate performance counters for some narrow scope. + /// Could be changed to point to another object to calculate performance counters for some narrow scope. ProfileEvents::Counters * current_performance_counters{&performance_counters}; MemoryTracker memory_tracker{VariableContext::Thread}; diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index b736fa43e0c..37650196783 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -258,23 +258,16 @@ bool PartLog::addNewPart(ContextPtr context, const PartLog::PartLogEntry & part, return addNewParts(context, {part}, execution_status); } -bool PartLog::addNewParts(ContextPtr context, const PartLog::MutableDataPartsVector & parts, UInt64 elapsed_ns, - const ExecutionStatus & execution_status) + +PartLog::PartLogEntries PartLog::createPartLogEntries(const MutableDataPartsVector & parts, UInt64 elapsed_ns, ProfileCountersSnapshotPtr profile_counters) { - PartLog::PartLogEntries part_log_entries; + PartLogEntries part_log_entries; part_log_entries.reserve(parts.size()); for (const auto & part : parts) - part_log_entries.emplace_back(part, elapsed_ns); + part_log_entries.emplace_back(part, elapsed_ns, profile_counters); - return addNewParts(context, part_log_entries, execution_status); -} - -bool PartLog::addNewPart(ContextPtr context, const PartLog::MutableDataPartPtr & part, UInt64 elapsed_ns, const ExecutionStatus & execution_status) -{ - PartLog::PartLogEntries part_log_entries; - part_log_entries.emplace_back(part, elapsed_ns); - return addNewParts(context, part_log_entries, execution_status); + return part_log_entries; } } diff --git a/src/Interpreters/PartLog.h b/src/Interpreters/PartLog.h index 57044cba844..843792d03a9 100644 --- a/src/Interpreters/PartLog.h +++ b/src/Interpreters/PartLog.h @@ -133,18 +133,14 @@ public: using PartLogEntries = std::vector; + static PartLogEntries createPartLogEntries(const MutableDataPartsVector & parts, UInt64 elapsed_ns, ProfileCountersSnapshotPtr profile_counters = {}); + /// Add a record about creation of new part. static bool addNewPart(ContextPtr context, const PartLogEntry & part, const ExecutionStatus & execution_status = {}); static bool addNewParts(ContextPtr context, const PartLogEntries & parts, const ExecutionStatus & execution_status = {}); - - /// Add a record about creation of new part. - static bool addNewPart(ContextPtr context, const MutableDataPartPtr & part, UInt64 elapsed_ns, - const ExecutionStatus & execution_status = {}); - static bool addNewParts(ContextPtr context, const MutableDataPartsVector & parts, UInt64 elapsed_ns, - const ExecutionStatus & execution_status = {}); }; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 57924331cce..d13452c291e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7490,7 +7490,6 @@ bool MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagge MutableDataPartPtr cloned_part; ProfileEventsScope profile_events_scope; - auto write_part_log = [&](const ExecutionStatus & execution_status) { writePartLog( diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index ee192966758..31fd99f0aa1 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -48,14 +49,21 @@ struct ReplicatedMergeTreeSinkImpl::DelayedChunk BlockIDsType block_id; BlockWithPartition block_with_partition; std::unordered_map> block_id_to_offset_idx; + ProfileEvents::Counters part_counters; Partition() = default; - Partition(Poco::Logger * log_, MergeTreeDataWriter::TemporaryPart && temp_part_, UInt64 elapsed_ns_, BlockIDsType && block_id_, BlockWithPartition && block_) + Partition(Poco::Logger * log_, + MergeTreeDataWriter::TemporaryPart && temp_part_, + UInt64 elapsed_ns_, + BlockIDsType && block_id_, + BlockWithPartition && block_, + ProfileEvents::Counters && part_counters_) : log(log_), temp_part(std::move(temp_part_)), elapsed_ns(elapsed_ns_), block_id(std::move(block_id_)), - block_with_partition(std::move(block_)) + block_with_partition(std::move(block_)), + part_counters(std::move(part_counters_)) { initBlockIDMap(); } @@ -186,8 +194,9 @@ std::vector testSelfDeduplicate(std::vector data, std::vector::DelayedChunk::Partition part( - &Poco::Logger::get("testSelfDeduplicate"), MergeTreeDataWriter::TemporaryPart(), 0, std::move(hashes), std::move(block1)); + &Poco::Logger::get("testSelfDeduplicate"), MergeTreeDataWriter::TemporaryPart(), 0, std::move(hashes), std::move(block1), std::move(profile_counters)); part.filterSelfDuplicate(); @@ -411,6 +420,9 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) { Stopwatch watch; + ProfileEvents::Counters part_counters; + auto profile_events_scope = std::make_unique(&part_counters); + /// Write part to the filesystem under temporary name. Calculate a checksum. auto temp_part = storage.writer.writeTempPart(current_block, metadata_snapshot, context); @@ -452,6 +464,7 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) LOG_DEBUG(log, "Wrote block with {} rows{}", current_block.block.rows(), quorumLogMessage(replicas_num)); } + profile_events_scope.reset(); UInt64 elapsed_ns = watch.elapsed(); size_t max_insert_delayed_streams_for_parallel_write = DEFAULT_DELAYED_STREAMS_FOR_PARALLEL_WRITE; @@ -472,12 +485,14 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) partitions = DelayedPartitions{}; } + partitions.emplace_back(DelayedPartition( log, std::move(temp_part), elapsed_ns, std::move(block_id), - std::move(current_block) + std::move(current_block), + std::move(part_counters) /// profile_events_scope must be reset here. )); } @@ -503,6 +518,8 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF for (auto & partition : delayed_chunk->partitions) { + ProfileEventsScope scoped_attach(&partition.part_counters); + partition.temp_part.finalize(); auto & part = partition.temp_part.part; @@ -515,12 +532,14 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF /// Set a special error code if the block is duplicate int error = (deduplicate && part->is_duplicate) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0; - PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns, ExecutionStatus(error)); + auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); + PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, partition.elapsed_ns, counters_snapshot), ExecutionStatus(error)); storage.incrementInsertedPartsProfileEvent(part->getType()); } catch (...) { - PartLog::addNewPart(storage.getContext(), part, partition.elapsed_ns, ExecutionStatus::fromCurrentException("", true)); + auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); + PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, partition.elapsed_ns, counters_snapshot), ExecutionStatus::fromCurrentException("", true)); throw; } } @@ -579,16 +598,17 @@ void ReplicatedMergeTreeSinkImpl::writeExistingPart(MergeTreeData: size_t replicas_num = checkQuorumPrecondition(zookeeper); Stopwatch watch; + ProfileEventsScope profile_events_scope; try { part->version.setCreationTID(Tx::PrehistoricTID, nullptr); commitPart(zookeeper, part, BlockIDsType(), replicas_num, true); - PartLog::addNewPart(storage.getContext(), part, watch.elapsed()); + PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, watch.elapsed(), profile_events_scope.getSnapshot())); } catch (...) { - PartLog::addNewPart(storage.getContext(), part, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); + PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, watch.elapsed(), profile_events_scope.getSnapshot()), ExecutionStatus::fromCurrentException("", true)); throw; } } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 125322281d0..9e98946baeb 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -1608,6 +1609,7 @@ void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont waitForOutdatedPartsToBeLoaded(); Stopwatch watch; + ProfileEventsScope profile_events_scope; auto txn = query_context->getCurrentTransaction(); MergeTreeData::Transaction transaction(*this, txn.get()); @@ -1628,7 +1630,7 @@ void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont auto new_data_parts = createEmptyDataParts(*this, future_parts, txn); renameAndCommitEmptyParts(new_data_parts, transaction); - PartLog::addNewParts(query_context, new_data_parts, watch.elapsed()); + PartLog::addNewParts(query_context, PartLog::createPartLogEntries(new_data_parts, watch.elapsed(), profile_events_scope.getSnapshot())); LOG_INFO(log, "Truncated table with {} parts by replacing them with new empty {} parts. With txn {}", parts.size(), future_parts.size(), @@ -1650,6 +1652,7 @@ void StorageMergeTree::dropPart(const String & part_name, bool detach, ContextPt auto merge_blocker = stopMergesAndWait(); Stopwatch watch; + ProfileEventsScope profile_events_scope; /// It's important to create it outside of lock scope because /// otherwise it can lock parts in destructor and deadlock is possible. @@ -1681,7 +1684,7 @@ void StorageMergeTree::dropPart(const String & part_name, bool detach, ContextPt auto new_data_parts = createEmptyDataParts(*this, future_parts, txn); renameAndCommitEmptyParts(new_data_parts, transaction); - PartLog::addNewParts(query_context, new_data_parts, watch.elapsed()); + PartLog::addNewParts(query_context, PartLog::createPartLogEntries(new_data_parts, watch.elapsed(), profile_events_scope.getSnapshot())); const auto * op = detach ? "Detached" : "Dropped"; LOG_INFO(log, "{} {} part by replacing it with new empty {} part. With txn {}", @@ -1707,6 +1710,7 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, Cont auto merge_blocker = stopMergesAndWait(); Stopwatch watch; + ProfileEventsScope profile_events_scope; /// It's important to create it outside of lock scope because /// otherwise it can lock parts in destructor and deadlock is possible. @@ -1746,7 +1750,7 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, Cont auto new_data_parts = createEmptyDataParts(*this, future_parts, txn); renameAndCommitEmptyParts(new_data_parts, transaction); - PartLog::addNewParts(query_context, new_data_parts, watch.elapsed()); + PartLog::addNewParts(query_context, PartLog::createPartLogEntries(new_data_parts, watch.elapsed(), profile_events_scope.getSnapshot())); const auto * op = detach ? "Detached" : "Dropped"; LOG_INFO(log, "{} partition with {} parts by replacing them with new empty {} parts. With txn {}", @@ -1814,6 +1818,8 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con auto my_metadata_snapshot = getInMemoryMetadataPtr(); Stopwatch watch; + ProfileEventsScope profile_events_scope; + MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, my_metadata_snapshot); String partition_id = getPartitionIDFromQuery(partition, local_context); @@ -1878,11 +1884,12 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con removePartsInRangeFromWorkingSet(local_context->getCurrentTransaction().get(), drop_range, data_parts_lock); } - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed()); + /// Note: same elapsed time and profile events for all parts is used + PartLog::addNewParts(getContext(), PartLog::createPartLogEntries(dst_parts, watch.elapsed(), profile_events_scope.getSnapshot())); } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); + PartLog::addNewParts(getContext(), PartLog::createPartLogEntries(dst_parts, watch.elapsed()), ExecutionStatus::fromCurrentException("", true)); throw; } } @@ -1909,6 +1916,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const auto dest_metadata_snapshot = dest_table->getInMemoryMetadataPtr(); auto metadata_snapshot = getInMemoryMetadataPtr(); Stopwatch watch; + ProfileEventsScope profile_events_scope; MergeTreeData & src_data = dest_table_storage->checkStructureAndGetMergeTreeData(*this, metadata_snapshot, dest_metadata_snapshot); String partition_id = getPartitionIDFromQuery(partition, local_context); @@ -1961,11 +1969,12 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const clearOldPartsFromFilesystem(); - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed()); + /// Note: same elapsed time and profile events for all parts is used + PartLog::addNewParts(getContext(), PartLog::createPartLogEntries(dst_parts, watch.elapsed(), profile_events_scope.getSnapshot())); } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); + PartLog::addNewParts(getContext(), PartLog::createPartLogEntries(dst_parts, watch.elapsed()), ExecutionStatus::fromCurrentException("", true)); throw; } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 8fc2adcad07..e2844431e34 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1952,6 +1952,8 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry) bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) { Stopwatch watch; + ProfileEventsScope profile_events_scope; + auto & entry_replace = *entry.replace_range_entry; LOG_DEBUG(log, "Executing log entry {} to replace parts range {} with {} parts from {}.{}", entry.znode_name, entry_replace.drop_range_part_name, entry_replace.new_part_names.size(), @@ -2344,11 +2346,11 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) } } - PartLog::addNewParts(getContext(), res_parts, watch.elapsed()); + PartLog::addNewParts(getContext(), PartLog::createPartLogEntries(res_parts, watch.elapsed(), profile_events_scope.getSnapshot())); } catch (...) { - PartLog::addNewParts(getContext(), res_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); + PartLog::addNewParts(getContext(), PartLog::createPartLogEntries(res_parts, watch.elapsed()), ExecutionStatus::fromCurrentException("", true)); for (const auto & res_part : res_parts) unlockSharedData(*res_part); @@ -6887,6 +6889,8 @@ void StorageReplicatedMergeTree::replacePartitionFrom( auto metadata_snapshot = getInMemoryMetadataPtr(); Stopwatch watch; + ProfileEventsScope profile_events_scope; + MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, metadata_snapshot); String partition_id = getPartitionIDFromQuery(partition, query_context); @@ -7063,11 +7067,11 @@ void StorageReplicatedMergeTree::replacePartitionFrom( parts_to_remove = removePartsInRangeFromWorkingSetAndGetPartsToRemoveFromZooKeeper(NO_TRANSACTION_RAW, drop_range, data_parts_lock); } - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed()); + PartLog::addNewParts(getContext(), PartLog::createPartLogEntries(dst_parts, watch.elapsed(), profile_events_scope.getSnapshot())); } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); + PartLog::addNewParts(getContext(), PartLog::createPartLogEntries(dst_parts, watch.elapsed()), ExecutionStatus::fromCurrentException("", true)); for (const auto & dst_part : dst_parts) unlockSharedData(*dst_part); @@ -7122,6 +7126,8 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta auto metadata_snapshot = getInMemoryMetadataPtr(); Stopwatch watch; + ProfileEventsScope profile_events_scope; + MergeTreeData & src_data = dest_table_storage->checkStructureAndGetMergeTreeData(*this, metadata_snapshot, dest_metadata_snapshot); auto src_data_id = src_data.getStorageID(); String partition_id = getPartitionIDFromQuery(partition, query_context); @@ -7292,11 +7298,11 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta transaction.commit(&src_data_parts_lock); } - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed()); + PartLog::addNewParts(getContext(), PartLog::createPartLogEntries(dst_parts, watch.elapsed(), profile_events_scope.getSnapshot())); } catch (...) { - PartLog::addNewParts(getContext(), dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException("", true)); + PartLog::addNewParts(getContext(), PartLog::createPartLogEntries(dst_parts, watch.elapsed()), ExecutionStatus::fromCurrentException("", true)); for (const auto & dst_part : dst_parts) dest_table_storage->unlockSharedData(*dst_part); From 83dea52affdcd7bea9f19b68c7cf9bbd00b1e3ab Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 1 Feb 2023 12:19:35 +0000 Subject: [PATCH 083/253] add sanity check to ThreadStatus::attachProfileCountersScope --- src/Interpreters/ThreadStatusExt.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 7c1a1fb7bb4..d13617b1d93 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -163,7 +163,9 @@ void ThreadStatus::attachQuery(const ThreadGroupStatusPtr & thread_group_, bool void ThreadStatus::attachProfileCountersScope(ProfileEvents::Counters * performance_counters_scope) { - performance_counters_scope->setParent(&performance_counters); + if (current_performance_counters != &performance_counters) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't attach performance counters scope to the thread, it is already attached to another scope"); + performance_counters_scope->setParent(current_performance_counters); current_performance_counters = performance_counters_scope; } From abd9119583b78ff798bb2341115f03f8ef3d54d1 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 2 Feb 2023 09:46:01 +0000 Subject: [PATCH 084/253] Remove unnecessary ProfileEventsScope --- src/Interpreters/ThreadStatusExt.cpp | 5 +++++ src/Storages/MergeTree/MergeFromLogEntryTask.cpp | 3 --- src/Storages/MergeTree/MutateTask.cpp | 14 -------------- 3 files changed, 5 insertions(+), 17 deletions(-) diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index d13617b1d93..ea2d5c4a125 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -163,8 +163,13 @@ void ThreadStatus::attachQuery(const ThreadGroupStatusPtr & thread_group_, bool void ThreadStatus::attachProfileCountersScope(ProfileEvents::Counters * performance_counters_scope) { + if (current_performance_counters == performance_counters_scope) + /// Allow to attach the same scope multiple times + return; + if (current_performance_counters != &performance_counters) throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't attach performance counters scope to the thread, it is already attached to another scope"); + performance_counters_scope->setParent(current_performance_counters); current_performance_counters = performance_counters_scope; } diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index 0d8dc7f68a6..a4e5b1f0575 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -268,7 +268,6 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() transaction_ptr = std::make_unique(storage, NO_TRANSACTION_RAW); stopwatch_ptr = std::make_unique(); - ProfileEventsScope profile_events_scope(&profile_counters); merge_task = storage.merger_mutator.mergePartsToTemporaryPart( future_merged_part, @@ -301,8 +300,6 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWriter write_part_log) { - ProfileEventsScope profile_events_scope(&profile_counters); - part = merge_task->getFuture().get(); storage.merger_mutator.renameMergedTemporaryPart(part, parts, NO_TRANSACTION_PTR, *transaction_ptr); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 563137e6a8c..43cf3d950b2 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -815,9 +815,6 @@ public: bool executeStep() override { - /// Metrics will be saved in the local profile_counters. - ProfileEventsScope profile_events_scope(&profile_counters); - auto & current_level_parts = level_parts[current_level]; auto & next_level_parts = level_parts[next_level]; @@ -918,8 +915,6 @@ private: /// TODO(nikitamikhaylov): make this constant a setting static constexpr size_t max_parts_to_merge_in_one_level = 10; - - ProfileEvents::Counters profile_counters; }; @@ -1143,9 +1138,6 @@ public: bool executeStep() override { - /// Metrics will be saved in the local profile_counters. - ProfileEventsScope profile_events_scope(&profile_counters); - switch (state) { case State::NEED_PREPARE: @@ -1261,8 +1253,6 @@ private: MutationContextPtr ctx; std::unique_ptr part_merger_writer_task; - - ProfileEvents::Counters profile_counters; }; @@ -1277,9 +1267,6 @@ public: bool executeStep() override { - /// Metrics will be saved in the local profile_counters. - ProfileEventsScope profile_events_scope(&profile_counters); - switch (state) { case State::NEED_PREPARE: @@ -1471,7 +1458,6 @@ private: MergedColumnOnlyOutputStreamPtr out; std::unique_ptr part_merger_writer_task{nullptr}; - ProfileEvents::Counters profile_counters; }; From 1b715a33882b34b4962d8fca1e8e28aaddbb0c4e Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 8 Feb 2023 11:12:59 +0000 Subject: [PATCH 085/253] Allow scope override in ThreadStatus::attachProfileCountersScope --- src/Common/ProfileEventsScope.cpp | 7 ++++--- src/Common/ProfileEventsScope.h | 1 + src/Common/ThreadStatus.h | 5 +++-- src/Interpreters/ThreadStatusExt.cpp | 16 ++++++---------- 4 files changed, 14 insertions(+), 15 deletions(-) diff --git a/src/Common/ProfileEventsScope.cpp b/src/Common/ProfileEventsScope.cpp index 5b996209b76..92f75f4f5b0 100644 --- a/src/Common/ProfileEventsScope.cpp +++ b/src/Common/ProfileEventsScope.cpp @@ -7,14 +7,14 @@ namespace DB ProfileEventsScope::ProfileEventsScope() : performance_counters_holder(std::make_unique()) , performance_counters_scope(performance_counters_holder.get()) + , previous_counters_scope(CurrentThread::get().attachProfileCountersScope(performance_counters_scope)) { - CurrentThread::get().attachProfileCountersScope(performance_counters_scope); } ProfileEventsScope::ProfileEventsScope(ProfileEvents::Counters * performance_counters_scope_) : performance_counters_scope(performance_counters_scope_) + , previous_counters_scope(CurrentThread::get().attachProfileCountersScope(performance_counters_scope)) { - CurrentThread::get().attachProfileCountersScope(performance_counters_scope); } std::shared_ptr ProfileEventsScope::getSnapshot() @@ -24,7 +24,8 @@ std::shared_ptr ProfileEventsScope::getSnapsh ProfileEventsScope::~ProfileEventsScope() { - CurrentThread::get().detachProfileCountersScope(); + /// Restore previous performance counters + CurrentThread::get().attachProfileCountersScope(previous_counters_scope); } diff --git a/src/Common/ProfileEventsScope.h b/src/Common/ProfileEventsScope.h index 653a1ed55e7..0444531d02b 100644 --- a/src/Common/ProfileEventsScope.h +++ b/src/Common/ProfileEventsScope.h @@ -27,6 +27,7 @@ private: std::unique_ptr performance_counters_holder; ProfileEvents::Counters * performance_counters_scope; + ProfileEvents::Counters * previous_counters_scope; }; diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 531dd669e65..20550a63312 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -249,8 +249,9 @@ public: /// Attaches slave thread to existing thread group void attachQuery(const ThreadGroupStatusPtr & thread_group_, bool check_detached = true); - void attachProfileCountersScope(ProfileEvents::Counters * performance_counters_scope); - void detachProfileCountersScope(); + /// Returns pointer to the current profile counters to restore them back. + /// Note: consequent call with new scope will detach previous scope. + ProfileEvents::Counters * attachProfileCountersScope(ProfileEvents::Counters * performance_counters_scope); InternalTextLogsQueuePtr getInternalTextLogsQueue() const { diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index ea2d5c4a125..a770c84a3e7 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -161,22 +161,18 @@ void ThreadStatus::attachQuery(const ThreadGroupStatusPtr & thread_group_, bool setupState(thread_group_); } -void ThreadStatus::attachProfileCountersScope(ProfileEvents::Counters * performance_counters_scope) +ProfileEvents::Counters * ThreadStatus::attachProfileCountersScope(ProfileEvents::Counters * performance_counters_scope) { + ProfileEvents::Counters * prev_counters = current_performance_counters; + if (current_performance_counters == performance_counters_scope) /// Allow to attach the same scope multiple times - return; + return prev_counters; - if (current_performance_counters != &performance_counters) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't attach performance counters scope to the thread, it is already attached to another scope"); - - performance_counters_scope->setParent(current_performance_counters); + performance_counters_scope->setParent(&performance_counters); current_performance_counters = performance_counters_scope; -} -void ThreadStatus::detachProfileCountersScope() -{ - current_performance_counters = &performance_counters; + return prev_counters; } void ThreadStatus::initPerformanceCounters() From 94928e7445f5b1eb2069e84d7d47fbeb35e9b429 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 8 Feb 2023 11:16:16 +0000 Subject: [PATCH 086/253] fix 02378_part_log_profile_events --- tests/queries/0_stateless/02378_part_log_profile_events.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02378_part_log_profile_events.sql b/tests/queries/0_stateless/02378_part_log_profile_events.sql index af68360dbbb..38d3f8b4c05 100644 --- a/tests/queries/0_stateless/02378_part_log_profile_events.sql +++ b/tests/queries/0_stateless/02378_part_log_profile_events.sql @@ -39,7 +39,7 @@ SYSTEM FLUSH LOGS; SELECT if(count() == 2, 'Ok', 'Error: ' || toString(count())), - if(SUM(ProfileEvents['SelectedRows']) == 512, 'Ok', 'Error: ' || toString(SUM(ProfileEvents['SelectedRows']))), + if(SUM(ProfileEvents['MergedRows']) == 512, 'Ok', 'Error: ' || toString(SUM(ProfileEvents['MergedRows']))), if(SUM(ProfileEvents['FileOpen']) > 1, 'Ok', 'Error: ' || toString(SUM(ProfileEvents['FileOpen']))) FROM system.part_log WHERE event_time > now() - INTERVAL 10 MINUTE From 850601846927f7968de706232841b8f0d9cde5b7 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Tue, 14 Feb 2023 15:20:39 +0100 Subject: [PATCH 087/253] A test for bug found by fuzzer --- .../02559_multiple_read_steps_in_prewhere_fuzz.reference | 1 + .../02559_multiple_read_steps_in_prewhere_fuzz.sql | 7 +++++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_fuzz.reference create mode 100644 tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_fuzz.sql diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_fuzz.reference b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_fuzz.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_fuzz.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_fuzz.sql b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_fuzz.sql new file mode 100644 index 00000000000..20f159cbe5d --- /dev/null +++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_fuzz.sql @@ -0,0 +1,7 @@ +CREATE TABLE test_02559__fuzz_20(`id1` Int16, `id2` Decimal(18, 14)) ENGINE = MergeTree ORDER BY id1; + +INSERT INTO test_02559__fuzz_20 SELECT number, number FROM numbers(10); + +SET enable_multiple_prewhere_read_steps=true, move_all_conditions_to_prewhere=true; + +SELECT count() FROM test_02559__fuzz_20 PREWHERE (id2 >= 104) AND ((-9223372036854775808 OR (inf OR -2147483649 OR NULL) OR NULL) OR 1 OR ignore(ignore(id1) OR NULL, id1)) WHERE ignore(id1) = 0; From 7ee94d6a900b71c6af94b6e74c2ca131b8d5c088 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Tue, 14 Feb 2023 15:34:04 +0100 Subject: [PATCH 088/253] Fix for 'Result column is not empty' bug found by fuzzer --- src/Interpreters/ExpressionActions.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 97555feb426..9931ae97286 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -584,6 +584,12 @@ static void executeAction(const ExpressionActions::Action & action, ExecutionCon { /// Do not execute function if it's result is already known. res_column.column = action.node->column->cloneResized(num_rows); + /// But still need to remove unused arguments. + for (const auto & argument : action.arguments) + { + if (!argument.needed_later) + columns[argument.pos] = {}; + } break; } From 0d6c6a94de01771ffe237f8e42feb503dacb568e Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 15 Feb 2023 12:18:17 +0000 Subject: [PATCH 089/253] finally fix attachProfileCountersScope --- src/Interpreters/ThreadStatusExt.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index a770c84a3e7..8b84d4cae9b 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -169,7 +169,8 @@ ProfileEvents::Counters * ThreadStatus::attachProfileCountersScope(ProfileEvents /// Allow to attach the same scope multiple times return prev_counters; - performance_counters_scope->setParent(&performance_counters); + if (!performance_counters_scope->getParent()) + performance_counters_scope->setParent(&performance_counters); current_performance_counters = performance_counters_scope; return prev_counters; From 9fd2226c4c5395baf8f9fdea0eced26959ba7a62 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 15 Feb 2023 15:13:04 +0100 Subject: [PATCH 090/253] Update NativeReader.h --- src/Formats/NativeReader.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Formats/NativeReader.h b/src/Formats/NativeReader.h index 64d3e4d6df0..2d8b16e06eb 100644 --- a/src/Formats/NativeReader.h +++ b/src/Formats/NativeReader.h @@ -49,9 +49,9 @@ private: ReadBuffer & istr; Block header; UInt64 server_revision; - bool skip_unknown_columns; - bool null_as_default; - BlockMissingValues * block_missing_values; + bool skip_unknown_columns = false; + bool null_as_default = false; + BlockMissingValues * block_missing_values = nullptr; bool use_index = false; IndexForNativeFormat::Blocks::const_iterator index_block_it; From 3ff0c7edcc3a86f5bc7baa7d5f27d41e6e445843 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 15 Feb 2023 22:37:04 +0100 Subject: [PATCH 091/253] fix typo Co-authored-by: Nikita Taranov --- src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp index d41572c9db7..c93c3ff6b53 100644 --- a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -44,7 +44,7 @@ void fillRequiredColumns(const ActionsDAG::Node * node, std::unordered_map Date: Wed, 15 Feb 2023 22:37:18 +0100 Subject: [PATCH 092/253] fix typo Co-authored-by: Nikita Taranov --- src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp index c93c3ff6b53..268ae5509d1 100644 --- a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -15,7 +15,7 @@ namespace ErrorCodes namespace { -/// Stores the ist of columns required to compute a node in the DAG. +/// Stores the list of columns required to compute a node in the DAG. struct NodeInfo { NameSet required_columns; From 69364711500f603cccb00fa04357c28d29962cfd Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 15 Feb 2023 22:37:40 +0100 Subject: [PATCH 093/253] fix include Co-authored-by: Nikita Taranov --- src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 95c76c81665..f811f71e23b 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -5,7 +5,7 @@ #include #include #include -#include "Storages/MergeTree/MergeTreeBaseSelectProcessor.h" +#include #include #include #include From cdf382857558eeb3091c0fc900f05f215d376733 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 15 Feb 2023 22:41:55 +0100 Subject: [PATCH 094/253] Removed unneeded default value for enable_multiple_prewhere_read_steps --- src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index b27a7114122..ae307a6a35c 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -70,6 +70,8 @@ public: const MergeTreeReaderSettings & getSettings() const { return reader_settings; } virtual std::string getName() const = 0; + + static std::unique_ptr getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, bool enable_multiple_prewhere_read_steps); protected: /// This struct allow to return block with no columns but with non-zero number of rows similar to Chunk @@ -101,9 +103,6 @@ protected: static void injectVirtualColumns(Block & block, size_t row_count, MergeTreeReadTask * task, const DataTypePtr & partition_value_type, const Names & virtual_columns); -public: - static std::unique_ptr getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, bool enable_multiple_prewhere_read_steps = true); - protected: static void initializeRangeReadersImpl( MergeTreeRangeReader & range_reader, From 0506d9289cfae0cb240b01f5aabf6200c424ba12 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 16 Feb 2023 09:30:27 +0100 Subject: [PATCH 095/253] Updated Backup/Restore Coordination construction and removed coordination_path and added uuid in settings - Use cluster state data to check concurrent backup/restore --- src/Backups/BackupCoordinationLocal.cpp | 2 +- src/Backups/BackupCoordinationLocal.h | 2 +- src/Backups/BackupCoordinationRemote.cpp | 29 +++++++++------- src/Backups/BackupCoordinationRemote.h | 6 ++-- src/Backups/BackupSettings.cpp | 1 - src/Backups/BackupSettings.h | 4 --- src/Backups/BackupsWorker.cpp | 42 +++++++++++------------ src/Backups/IBackupCoordination.h | 2 +- src/Backups/IRestoreCoordination.h | 2 +- src/Backups/RestoreCoordinationLocal.cpp | 2 +- src/Backups/RestoreCoordinationLocal.h | 2 +- src/Backups/RestoreCoordinationRemote.cpp | 28 ++++++++------- src/Backups/RestoreCoordinationRemote.h | 6 ++-- src/Backups/RestoreSettings.cpp | 2 +- src/Backups/RestoreSettings.h | 5 +-- 15 files changed, 69 insertions(+), 66 deletions(-) diff --git a/src/Backups/BackupCoordinationLocal.cpp b/src/Backups/BackupCoordinationLocal.cpp index dd5afbed8e8..91da16097cc 100644 --- a/src/Backups/BackupCoordinationLocal.cpp +++ b/src/Backups/BackupCoordinationLocal.cpp @@ -202,7 +202,7 @@ Strings BackupCoordinationLocal::getAllArchiveSuffixes() const return archive_suffixes; } -bool BackupCoordinationLocal::hasConcurrentBackups(const String &, const String &, const std::atomic & num_active_backups) const +bool BackupCoordinationLocal::hasConcurrentBackups(const std::atomic & num_active_backups) const { return (num_active_backups > 1); } diff --git a/src/Backups/BackupCoordinationLocal.h b/src/Backups/BackupCoordinationLocal.h index 6e8a793ccd4..8e54eb6fb27 100644 --- a/src/Backups/BackupCoordinationLocal.h +++ b/src/Backups/BackupCoordinationLocal.h @@ -52,7 +52,7 @@ public: String getNextArchiveSuffix() override; Strings getAllArchiveSuffixes() const override; - bool hasConcurrentBackups(const String & backup_id, const String & common_backups_path, const std::atomic & num_active_backups) const override; + bool hasConcurrentBackups(const std::atomic & num_active_backups) const override; private: mutable std::mutex mutex; diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 29514b8fe1f..8c696057755 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -166,14 +166,16 @@ namespace } BackupCoordinationRemote::BackupCoordinationRemote( - const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, bool is_internal_) - : zookeeper_path(zookeeper_path_) + const String & root_zookeeper_path_, const String & backup_uuid_, zkutil::GetZooKeeper get_zookeeper_, bool is_internal_) + : root_zookeeper_path(root_zookeeper_path_) + , zookeeper_path(root_zookeeper_path_ + "/backup-" + backup_uuid_) + , backup_uuid(backup_uuid_) , get_zookeeper(get_zookeeper_) , is_internal(is_internal_) { createRootNodes(); stage_sync.emplace( - zookeeper_path_ + "/stage", [this] { return getZooKeeper(); }, &Poco::Logger::get("BackupCoordination")); + zookeeper_path + "/stage", [this] { return getZooKeeper(); }, &Poco::Logger::get("BackupCoordination")); } BackupCoordinationRemote::~BackupCoordinationRemote() @@ -595,36 +597,36 @@ Strings BackupCoordinationRemote::getAllArchiveSuffixes() const return node_names; } -bool BackupCoordinationRemote::hasConcurrentBackups(const String & backup_id, const String & common_backups_path, const std::atomic &) const +bool BackupCoordinationRemote::hasConcurrentBackups(const std::atomic &) const { /// If its internal concurrency will be checked for the base backup if (is_internal) return false; auto zk = getZooKeeper(); - std::string backup_stage_path = common_backups_path + "/backup-" + toString(backup_id) +"/stage"; + std::string backup_stage_path = zookeeper_path +"/stage"; - if (!zk->exists(common_backups_path)) - zk->createAncestors(common_backups_path); + if (!zk->exists(root_zookeeper_path)) + zk->createAncestors(root_zookeeper_path); for (size_t attempt = 0; attempt < MAX_ZOOKEEPER_ATTEMPTS; ++attempt) { Coordination::Stat stat; - zk->get(common_backups_path, &stat); - Strings existing_backup_paths = zk->getChildren(common_backups_path); + zk->get(root_zookeeper_path, &stat); + Strings existing_backup_paths = zk->getChildren(root_zookeeper_path); for (const auto & existing_backup_path : existing_backup_paths) { if (startsWith(existing_backup_path, "restore-")) continue; - String existing_backup_id = existing_backup_path; - existing_backup_id.erase(0, String("backup-").size()); + String existing_backup_uuid = existing_backup_path; + existing_backup_uuid.erase(0, String("backup-").size()); - if (existing_backup_id == toString(backup_id)) + if (existing_backup_uuid == toString(backup_uuid)) continue; - const auto status = zk->get(common_backups_path + "/" + existing_backup_path + "/stage"); + const auto status = zk->get(root_zookeeper_path + "/" + existing_backup_path + "/stage"); if (status != Stage::COMPLETED) return true; } @@ -637,6 +639,7 @@ bool BackupCoordinationRemote::hasConcurrentBackups(const String & backup_id, co if ((code != Coordination::Error::ZBADVERSION) || is_last_attempt) throw zkutil::KeeperException(code, backup_stage_path); } + return false; } diff --git a/src/Backups/BackupCoordinationRemote.h b/src/Backups/BackupCoordinationRemote.h index 50b98fcd77d..c7260bcd237 100644 --- a/src/Backups/BackupCoordinationRemote.h +++ b/src/Backups/BackupCoordinationRemote.h @@ -16,7 +16,7 @@ constexpr size_t MAX_ZOOKEEPER_ATTEMPTS = 10; class BackupCoordinationRemote : public IBackupCoordination { public: - BackupCoordinationRemote(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, bool is_internal_); + BackupCoordinationRemote(const String & root_zookeeper_path_, const String & backup_uuid_, zkutil::GetZooKeeper get_zookeeper_, bool is_internal_); ~BackupCoordinationRemote() override; void setStage(const String & current_host, const String & new_stage, const String & message) override; @@ -58,7 +58,7 @@ public: String getNextArchiveSuffix() override; Strings getAllArchiveSuffixes() const override; - bool hasConcurrentBackups(const String & backup_id, const String & common_backups_path, const std::atomic & num_active_backups) const override; + bool hasConcurrentBackups(const std::atomic & num_active_backups) const override; private: zkutil::ZooKeeperPtr getZooKeeper() const; @@ -68,7 +68,9 @@ private: void prepareReplicatedTables() const; void prepareReplicatedAccess() const; + const String root_zookeeper_path; const String zookeeper_path; + const String backup_uuid; const zkutil::GetZooKeeper get_zookeeper; const bool is_internal; diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index c801430cad7..57d85305e25 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -28,7 +28,6 @@ namespace ErrorCodes M(UInt64, replica_num) \ M(Bool, internal) \ M(String, host_id) \ - M(String, coordination_zk_path) \ M(OptionalUUID, backup_uuid) /// M(Int64, compression_level) diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index 2e7717c3afe..1b97256c75b 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -55,10 +55,6 @@ struct BackupSettings /// Cluster's hosts' IDs in the format 'escaped_host_name:port' for all shards and replicas in a cluster specified in BACKUP ON CLUSTER. std::vector cluster_host_ids; - /// Internal, should not be specified by user. - /// Path in Zookeeper used to coordinate a distributed backup created by BACKUP ON CLUSTER. - String coordination_zk_path; - /// Internal, should not be specified by user. /// UUID of the backup. If it's not set it will be generated randomly. std::optional backup_uuid; diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 2bb176efce4..865151cc9ec 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -38,12 +38,12 @@ namespace Stage = BackupCoordinationStage; namespace { - std::shared_ptr makeBackupCoordination(const String & coordination_zk_path, const ContextPtr & context, bool is_internal_backup) + std::shared_ptr makeBackupCoordination(const String & root_zk_path, const String & backup_uuid, const ContextPtr & context, bool is_internal_backup) { - if (!coordination_zk_path.empty()) + if (!root_zk_path.empty()) { auto get_zookeeper = [global_context = context->getGlobalContext()] { return global_context->getZooKeeper(); }; - return std::make_shared(coordination_zk_path, get_zookeeper, is_internal_backup); + return std::make_shared(root_zk_path, backup_uuid, get_zookeeper, is_internal_backup); } else { @@ -51,12 +51,12 @@ namespace } } - std::shared_ptr makeRestoreCoordination(const String & coordination_zk_path, const ContextPtr & context, bool is_internal_backup) + std::shared_ptr makeRestoreCoordination(const String & root_zk_path, const String & restore_uuid, const ContextPtr & context, bool is_internal_backup) { - if (!coordination_zk_path.empty()) + if (!root_zk_path.empty()) { auto get_zookeeper = [global_context = context->getGlobalContext()] { return global_context->getZooKeeper(); }; - return std::make_shared(coordination_zk_path, get_zookeeper, is_internal_backup); + return std::make_shared(root_zk_path, restore_uuid, get_zookeeper, is_internal_backup); } else { @@ -160,13 +160,16 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context else backup_id = toString(*backup_settings.backup_uuid); + String root_zk_path; + std::shared_ptr backup_coordination; if (backup_settings.internal) { /// The following call of makeBackupCoordination() is not essential because doBackup() will later create a backup coordination /// if it's not created here. However to handle errors better it's better to make a coordination here because this way /// if an exception will be thrown in startMakingBackup() other hosts will know about that. - backup_coordination = makeBackupCoordination(backup_settings.coordination_zk_path, context, backup_settings.internal); + root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); + backup_coordination = makeBackupCoordination(root_zk_path, toString(*backup_settings.backup_uuid), context, backup_settings.internal); } auto backup_info = BackupInfo::fromAST(*backup_query->backup_name); @@ -270,17 +273,13 @@ void BackupsWorker::doBackup( backup_query->cluster = context->getMacros()->expand(backup_query->cluster); cluster = context->getCluster(backup_query->cluster); backup_settings.cluster_host_ids = cluster->getHostIDs(); - if (backup_settings.coordination_zk_path.empty()) - { - backup_settings.coordination_zk_path = root_zk_path + "/backup-" + toString(*backup_settings.backup_uuid); - } } /// Make a backup coordination. if (!backup_coordination) - backup_coordination = makeBackupCoordination(backup_settings.coordination_zk_path, context, backup_settings.internal); + backup_coordination = makeBackupCoordination(root_zk_path, toString(*backup_settings.backup_uuid), context, backup_settings.internal); - if (!allow_concurrent_backups && backup_coordination->hasConcurrentBackups(backup_id, root_zk_path, std::ref(num_active_backups))) + if (!allow_concurrent_backups && backup_coordination->hasConcurrentBackups(std::ref(num_active_backups))) throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Concurrent backups not supported, turn on setting 'allow_concurrent_backups'"); /// Opens a backup for writing. @@ -383,6 +382,9 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt auto restore_query = std::static_pointer_cast(query->clone()); auto restore_settings = RestoreSettings::fromRestoreQuery(*restore_query); + if (!restore_settings.restore_uuid) + restore_settings.restore_uuid = UUIDHelpers::generateV4(); + /// `restore_id` will be used as a key to the `infos` map, so it should be unique. OperationID restore_id; if (restore_settings.internal) @@ -390,7 +392,7 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt else if (!restore_settings.id.empty()) restore_id = restore_settings.id; else - restore_id = toString(UUIDHelpers::generateV4()); + restore_id = toString(*restore_settings.restore_uuid); std::shared_ptr restore_coordination; if (restore_settings.internal) @@ -398,7 +400,8 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt /// The following call of makeRestoreCoordination() is not essential because doRestore() will later create a restore coordination /// if it's not created here. However to handle errors better it's better to make a coordination here because this way /// if an exception will be thrown in startRestoring() other hosts will know about that. - restore_coordination = makeRestoreCoordination(restore_settings.coordination_zk_path, context, restore_settings.internal); + auto root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); + restore_coordination = makeRestoreCoordination(root_zk_path, toString(*restore_settings.restore_uuid), context, restore_settings.internal); } try @@ -518,15 +521,10 @@ void BackupsWorker::doRestore( } /// Make a restore coordination. - if (on_cluster && restore_settings.coordination_zk_path.empty()) - { - restore_settings.coordination_zk_path = root_zk_path + "/restore-" + toString(restore_id); - } - if (!restore_coordination) - restore_coordination = makeRestoreCoordination(restore_settings.coordination_zk_path, context, restore_settings.internal); + restore_coordination = makeRestoreCoordination(root_zk_path, toString(*restore_settings.restore_uuid), context, restore_settings.internal); - if (!allow_concurrent_restores && restore_coordination->hasConcurrentRestores(restore_id, root_zk_path, std::ref(num_active_restores))) + if (!allow_concurrent_restores && restore_coordination->hasConcurrentRestores(std::ref(num_active_restores))) throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Concurrent restores not supported, turn on setting 'allow_concurrent_restores'"); /// Do RESTORE. diff --git a/src/Backups/IBackupCoordination.h b/src/Backups/IBackupCoordination.h index 787b86f8cbd..f5fa01a1530 100644 --- a/src/Backups/IBackupCoordination.h +++ b/src/Backups/IBackupCoordination.h @@ -117,7 +117,7 @@ public: /// This function is used to check if concurrent backups are running /// other than the backup passed to the function - virtual bool hasConcurrentBackups(const String & backup_id, const String & common_backups_path, const std::atomic & num_active_backups) const = 0; + virtual bool hasConcurrentBackups(const std::atomic & num_active_backups) const = 0; }; } diff --git a/src/Backups/IRestoreCoordination.h b/src/Backups/IRestoreCoordination.h index a2506e5c157..098d048f6a3 100644 --- a/src/Backups/IRestoreCoordination.h +++ b/src/Backups/IRestoreCoordination.h @@ -37,7 +37,7 @@ public: /// This function is used to check if concurrent restores are running /// other than the restore passed to the function - virtual bool hasConcurrentRestores(const String & restore_id, const String & common_restores_path, const std::atomic & num_active_restores) const = 0; + virtual bool hasConcurrentRestores(const std::atomic & num_active_restores) const = 0; }; } diff --git a/src/Backups/RestoreCoordinationLocal.cpp b/src/Backups/RestoreCoordinationLocal.cpp index 56951c56baa..4e908d5d67d 100644 --- a/src/Backups/RestoreCoordinationLocal.cpp +++ b/src/Backups/RestoreCoordinationLocal.cpp @@ -42,7 +42,7 @@ bool RestoreCoordinationLocal::acquireReplicatedAccessStorage(const String &) return true; } -bool RestoreCoordinationLocal::hasConcurrentRestores(const String &, const String &, const std::atomic & num_active_restores) const +bool RestoreCoordinationLocal::hasConcurrentRestores(const std::atomic & num_active_restores) const { return (num_active_restores > 1); } diff --git a/src/Backups/RestoreCoordinationLocal.h b/src/Backups/RestoreCoordinationLocal.h index fa6941a7577..ab9d1ce0a59 100644 --- a/src/Backups/RestoreCoordinationLocal.h +++ b/src/Backups/RestoreCoordinationLocal.h @@ -35,7 +35,7 @@ public: /// The function returns false if this access storage is being already restored by another replica. bool acquireReplicatedAccessStorage(const String & access_storage_zk_path) override; - bool hasConcurrentRestores(const String & restore_id, const String & common_restores_path, const std::atomic & num_active_restores) const override; + bool hasConcurrentRestores(const std::atomic & num_active_restores) const override; private: std::set> acquired_tables_in_replicated_databases; diff --git a/src/Backups/RestoreCoordinationRemote.cpp b/src/Backups/RestoreCoordinationRemote.cpp index 95766bfcae3..f829cd20c0d 100644 --- a/src/Backups/RestoreCoordinationRemote.cpp +++ b/src/Backups/RestoreCoordinationRemote.cpp @@ -10,15 +10,17 @@ namespace DB namespace Stage = BackupCoordinationStage; RestoreCoordinationRemote::RestoreCoordinationRemote( - const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, bool is_internal_) - : zookeeper_path(zookeeper_path_) + const String & root_zookeeper_path_, const String & restore_uuid_, zkutil::GetZooKeeper get_zookeeper_, bool is_internal_) + : root_zookeeper_path(root_zookeeper_path_) + , zookeeper_path(root_zookeeper_path_ + "/restore-" + restore_uuid_) + , restore_uuid(restore_uuid_) , get_zookeeper(get_zookeeper_) , is_internal(is_internal_) { createRootNodes(); stage_sync.emplace( - zookeeper_path_ + "/stage", [this] { return getZooKeeper(); }, &Poco::Logger::get("RestoreCoordination")); + zookeeper_path + "/stage", [this] { return getZooKeeper(); }, &Poco::Logger::get("RestoreCoordination")); } RestoreCoordinationRemote::~RestoreCoordinationRemote() @@ -132,36 +134,36 @@ void RestoreCoordinationRemote::removeAllNodes() zk->removeRecursive(zookeeper_path); } -bool RestoreCoordinationRemote::hasConcurrentRestores(const String & restore_id, const String & common_restores_path, const std::atomic &) const +bool RestoreCoordinationRemote::hasConcurrentRestores(const std::atomic &) const { /// If its internal concurrency will be checked for the base restore if (is_internal) return false; auto zk = getZooKeeper(); - std::string path = common_restores_path + "/restore-" + toString(restore_id) +"/stage"; + std::string path = zookeeper_path +"/stage"; - if (! zk->exists(common_restores_path)) - zk->createAncestors(common_restores_path); + if (! zk->exists(root_zookeeper_path)) + zk->createAncestors(root_zookeeper_path); for (size_t attempt = 0; attempt < MAX_ZOOKEEPER_ATTEMPTS; ++attempt) { Coordination::Stat stat; - zk->get(common_restores_path, &stat); - Strings existing_restore_paths = zk->getChildren(common_restores_path); + zk->get(root_zookeeper_path, &stat); + Strings existing_restore_paths = zk->getChildren(root_zookeeper_path); for (const auto & existing_restore_path : existing_restore_paths) { if (startsWith(existing_restore_path, "backup-")) continue; - String existing_restore_id = existing_restore_path; - existing_restore_id.erase(0, String("restore-").size()); + String existing_restore_uuid = existing_restore_path; + existing_restore_uuid.erase(0, String("restore-").size()); - if (existing_restore_id == toString(restore_id)) + if (existing_restore_uuid == toString(restore_uuid)) continue; - const auto status = zk->get(common_restores_path + "/" + existing_restore_path + "/stage"); + const auto status = zk->get(root_zookeeper_path + "/" + existing_restore_path + "/stage"); if (status != Stage::COMPLETED) return true; } diff --git a/src/Backups/RestoreCoordinationRemote.h b/src/Backups/RestoreCoordinationRemote.h index 67d78192e1e..d72781bac2f 100644 --- a/src/Backups/RestoreCoordinationRemote.h +++ b/src/Backups/RestoreCoordinationRemote.h @@ -11,7 +11,7 @@ namespace DB class RestoreCoordinationRemote : public IRestoreCoordination { public: - RestoreCoordinationRemote(const String & zookeeper_path_, zkutil::GetZooKeeper get_zookeeper_, bool is_internal_); + RestoreCoordinationRemote(const String & root_zookeeper_path_, const String & restore_uuid_, zkutil::GetZooKeeper get_zookeeper_, bool is_internal_); ~RestoreCoordinationRemote() override; /// Sets the current stage and waits for other hosts to come to this stage too. @@ -31,7 +31,7 @@ public: /// The function returns false if this access storage is being already restored by another replica. bool acquireReplicatedAccessStorage(const String & access_storage_zk_path) override; - bool hasConcurrentRestores(const String & restore_id, const String & common_restores_path, const std::atomic & num_active_restores) const override; + bool hasConcurrentRestores(const std::atomic & num_active_restores) const override; private: zkutil::ZooKeeperPtr getZooKeeper() const; @@ -40,7 +40,9 @@ private: class ReplicatedDatabasesMetadataSync; + const String root_zookeeper_path; const String zookeeper_path; + const String restore_uuid; const zkutil::GetZooKeeper get_zookeeper; const bool is_internal; diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index bbcefb819cf..d12da704b2d 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -163,7 +163,7 @@ namespace M(RestoreUDFCreationMode, create_function) \ M(Bool, internal) \ M(String, host_id) \ - M(String, coordination_zk_path) + M(OptionalUUID, restore_uuid) RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index 713adbe8029..3bce8698620 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -119,8 +119,9 @@ struct RestoreSettings std::vector cluster_host_ids; /// Internal, should not be specified by user. - /// Path in Zookeeper used to coordinate restoring process while executing by RESTORE ON CLUSTER. - String coordination_zk_path; + /// UUID of the restore. If it's not set it will be generated randomly. + /// This is used to generate coordination path and for concurrency check + std::optional restore_uuid; static RestoreSettings fromRestoreQuery(const ASTBackupQuery & query); void copySettingsToQuery(ASTBackupQuery & query) const; From ba6b7b161064c5ccd99fcd3c4e4516eb4d678dab Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 16 Feb 2023 16:01:25 -0300 Subject: [PATCH 096/253] fix test_alias_column --- 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 69401587e41..fe352c59ead 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6427,9 +6427,9 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg // Projections don't support grouping sets yet. if (original_select_query->group_by_with_grouping_sets - || select_query->group_by_with_totals - || select_query->group_by_with_rollup - || select_query->group_by_with_cube) + || original_select_query->group_by_with_totals + || original_select_query->group_by_with_rollup + || original_select_query->group_by_with_cube) return std::nullopt; auto query_options = SelectQueryOptions( @@ -6439,7 +6439,7 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg ).ignoreProjections().ignoreAlias(); InterpreterSelectQuery select( - query_ptr, + original_query_ptr, query_context, query_options, query_info.prepared_sets); From 3a635e428a48b93d4b99f52a263f8ad6a58d71e0 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Thu, 16 Feb 2023 11:03:41 -0800 Subject: [PATCH 097/253] Fix xxhash endian issue for s390x --- src/Functions/FunctionsHashing.h | 106 ++++++++++++++++++++++++++----- 1 file changed, 91 insertions(+), 15 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 69c3a299eea..6bf1a2db3ac 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -55,7 +55,7 @@ #include #include #include - +#include namespace DB { @@ -1025,17 +1025,58 @@ private: if constexpr (Impl::use_int_hash_for_pods) { - if constexpr (std::is_same_v) - h = IntHash64Impl::apply(bit_cast(vec_from[i])); + if constexpr (std::endian::native == std::endian::little) + { + if constexpr (std::is_same_v) + h = IntHash64Impl::apply(bit_cast(vec_from[i])); + else + h = IntHash32Impl::apply(bit_cast(vec_from[i])); + } else - h = IntHash32Impl::apply(bit_cast(vec_from[i])); + { + if constexpr (std::is_same_v) + { + UInt64 v = bit_cast(vec_from[i]); + v = __builtin_bswap64(v); + h = IntHash64Impl::apply(v); + } + else + { + UInt32 v = bit_cast(vec_from[i]); + v = __builtin_bswap32(v); + h = IntHash32Impl::apply(v); + } + } } else { - if (std::is_same_v) - h = JavaHashImpl::apply(vec_from[i]); + if constexpr (std::endian::native == std::endian::little) + { + if (std::is_same_v) + h = JavaHashImpl::apply(vec_from[i]); + else + h = apply(key, reinterpret_cast(&vec_from[i]), sizeof(vec_from[i])); + } else - h = apply(key, reinterpret_cast(&vec_from[i]), sizeof(vec_from[i])); + { + if (std::is_same_v) + h = JavaHashImpl::apply(vec_from[i]); + else + { + if constexpr (std::is_same_v) + { + UInt64 v = bit_cast(vec_from[i]); + v = __builtin_bswap64(v); + h = apply(key, reinterpret_cast(&v), sizeof(vec_from[i])); + } + else + { + UInt32 v = bit_cast(vec_from[i]); + v = __builtin_bswap32(v); + h = apply(key, reinterpret_cast(&v), sizeof(vec_from[i])); + } + } + } } if constexpr (first) @@ -1048,11 +1089,28 @@ private: { auto value = col_from_const->template getValue(); ToType hash; - if constexpr (std::is_same_v) - hash = IntHash64Impl::apply(bit_cast(value)); + if constexpr (std::endian::native == std::endian::little) + { + if constexpr (std::is_same_v) + hash = IntHash64Impl::apply(bit_cast(value)); + else + hash = IntHash32Impl::apply(bit_cast(value)); + } else - hash = IntHash32Impl::apply(bit_cast(value)); - + { + if constexpr (std::is_same_v) + { + UInt64 v = bit_cast(value); + v = __builtin_bswap64(v); + hash = IntHash64Impl::apply(v); + } + else + { + UInt32 v = bit_cast(value); + v = __builtin_bswap32(v); + hash = IntHash32Impl::apply(bit_cast(v)); + } + } size_t size = vec_to.size(); if constexpr (first) { @@ -1080,8 +1138,17 @@ private: size_t size = vec_from.size(); for (size_t i = 0; i < size; ++i) { - ToType h = apply(key, reinterpret_cast(&vec_from[i]), sizeof(vec_from[i])); - + ToType h; + if constexpr (std::endian::native == std::endian::little) + { + h = apply(key, reinterpret_cast(&vec_from[i]), sizeof(vec_from[i])); + } + else + { + char tmp_buffer[sizeof(vec_from[i])]; + reverseMemcpy(tmp_buffer, &vec_from[i], sizeof(vec_from[i])); + h = apply(key, reinterpret_cast(tmp_buffer), sizeof(vec_from[i])); + } if constexpr (first) vec_to[i] = h; else @@ -1092,8 +1159,17 @@ private: { auto value = col_from_const->template getValue(); - ToType h = apply(key, reinterpret_cast(&value), sizeof(value)); - + ToType h; + if constexpr (std::endian::native == std::endian::little) + { + h = apply(key, reinterpret_cast(&value), sizeof(value)); + } + else + { + char tmp_buffer[sizeof(value)]; + reverseMemcpy(tmp_buffer, &value, sizeof(value)); + h = apply(key, reinterpret_cast(tmp_buffer), sizeof(value)); + } size_t size = vec_to.size(); if constexpr (first) { From 4b0beb938890e55784654ed5f8e59eb722fd45c7 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Sat, 18 Feb 2023 17:59:12 +0100 Subject: [PATCH 098/253] Test case for incorrect combined PREWHERE column --- .../02559_multiple_read_steps_in_prewhere.reference | 4 ++++ .../0_stateless/02559_multiple_read_steps_in_prewhere.sql | 2 ++ 2 files changed, 6 insertions(+) diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference index f2da126fc92..1bc2ac005a8 100644 --- a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference +++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference @@ -36,6 +36,10 @@ SELECT cast(id1 as UInt16) AS cond1, (id2 % 40000) AS cond2, (cond1 AND cond2) A 8 8 1 9 9 1 SELECT cast(id1 as UInt16) AS cond1 FROM test_02559 PREWHERE cond1 LIMIT 10; -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } +SELECT * FROM test_02559 PREWHERE id1 <= 3 AND id2 > 0 WHERE (id1 + id2 < 15) LIMIT 10; +1 1 +2 2 +3 3 SELECT count() FROM test_02559 PREWHERE id2>=0 AND (1 OR ignore(id1)) WHERE ignore(id1)=0; 10 CREATE ROW POLICY 02559_filter_1 ON test_02559 USING id2=2 AS permissive TO ALL; diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql index 0b434eae7df..829aea9f865 100644 --- a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql +++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql @@ -23,6 +23,8 @@ SELECT cast(id1 as UInt16) AS cond1, (id2 % 40000) AS cond2, (cond1 AND cond2) A SELECT cast(id1 as UInt16) AS cond1 FROM test_02559 PREWHERE cond1 LIMIT 10; -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } +SELECT * FROM test_02559 PREWHERE id1 <= 3 AND id2 > 0 WHERE (id1 + id2 < 15) LIMIT 10; + SELECT count() FROM test_02559 PREWHERE id2>=0 AND (1 OR ignore(id1)) WHERE ignore(id1)=0; CREATE ROW POLICY 02559_filter_1 ON test_02559 USING id2=2 AS permissive TO ALL; From 88fb6c3c25aa4eb0a48a51152fbbacfe236cbd54 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Sat, 18 Feb 2023 18:29:16 +0100 Subject: [PATCH 099/253] Fix for incorrect combined PREWHERE column --- .../MergeTreeSplitPrewhereIntoReadSteps.cpp | 31 +++++++++++++++---- 1 file changed, 25 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp index 268ae5509d1..0dd91536bb4 100644 --- a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -133,6 +133,9 @@ const ActionsDAG::Node & addCast( const String & type_name, OriginalToNewNodeMap & node_remap) { + if (node_to_cast.result_type->getName() == type_name) + return node_to_cast; + Field cast_type_constant_value(type_name); ColumnWithTypeAndName column; @@ -271,12 +274,28 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction else if (output->result_name == prewhere_info->prewhere_column_name) { /// Special case for final PREWHERE column: it is an AND combination of all conditions, - /// but we have only the condition for the last step here. - /// However we know that the ultimate result after filtering is constant 1 for the PREWHERE column. - auto const_true = output->result_type->createColumnConst(0, Field{1}); - const auto & prewhere_result_node = - steps.back().actions->addColumn(ColumnWithTypeAndName(const_true, output->result_type, output->result_name)); - steps.back().actions->addOrReplaceInOutputs(prewhere_result_node); + /// but we have only the condition for the last step here. We know that the combined filter is equivalent to + /// to the last condition after filters from previous steps are applied. We just need to CAST the last condition + /// to the type of combined filter. We do this in 2 steps: + /// 1. AND the last condition with constant True. This is needed to make sure that in the last step filter has UInt8 type + /// but containes values other than 0 and 1 (e.g. if it is (number%5) it contains 2,3,4) + /// 2. CAST the result to the exact type of the PREWHERE column from the original DAG + const auto & last_step_result_node_info = node_remap[steps.back().column_name]; + auto & last_step_dag = steps.back().actions; + /// Build AND(last_step_result_node, true) + Field true_constant_value(true); + ColumnWithTypeAndName column; + column.column = DataTypeUInt8().createColumnConst(0, true_constant_value); + column.type = std::make_shared(); + const auto * cast_type_constant_node = &last_step_dag->addColumn(std::move(column)); + ActionsDAG::NodeRawConstPtrs children = {last_step_result_node_info.node, cast_type_constant_node}; + FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); + const auto& and_node = addFunction(last_step_dag, func_builder_and, children, node_remap); + /// Build CAST(and_node, type of PREWHERE column) + const auto & cast_node = addCast(last_step_dag, and_node, output->result_type->getName(), node_remap); + /// Add alias for the result with the name of the PREWHERE column + const auto & prewhere_result_node = last_step_dag->addAlias(cast_node, output->result_name); + last_step_dag->addOrReplaceInOutputs(prewhere_result_node); } else { From dbacc57e4b1c53b6995280813ea393d45bb29886 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Sat, 18 Feb 2023 18:31:18 +0100 Subject: [PATCH 100/253] Test case for Nullable step filter column --- .../02559_multiple_read_steps_in_prewhere.reference | 7 +++++++ .../0_stateless/02559_multiple_read_steps_in_prewhere.sql | 2 ++ 2 files changed, 9 insertions(+) diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference index 1bc2ac005a8..580e7d4fef3 100644 --- a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference +++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference @@ -20,6 +20,13 @@ SELECT cast(id1 as UInt16) AS cond1, (id2 % 40000) AS cond2, (cond1 AND cond2) A 7 7 1 8 8 1 9 9 1 +SELECT cast(id1 as UInt16) AS cond1, (if(id2 > 3, id2, NULL) % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond LIMIT 10; +4 4 1 +5 5 1 +6 6 1 +7 7 1 +8 8 1 +9 9 1 SELECT cast(id1 as UInt16) AS cond1, (id2 % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond AND id2 > 4 LIMIT 10; 5 5 1 6 6 1 diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql index 829aea9f865..ee11b6edd55 100644 --- a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql +++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql @@ -15,6 +15,8 @@ SELECT cast(id1 as UInt16) AS id16 FROM test_02559 PREWHERE id16 and (id2 % 4000 SELECT cast(id1 as UInt16) AS cond1, (id2 % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond LIMIT 10; +SELECT cast(id1 as UInt16) AS cond1, (if(id2 > 3, id2, NULL) % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond LIMIT 10; + SELECT cast(id1 as UInt16) AS cond1, (id2 % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE cond AND id2 > 4 LIMIT 10; SELECT cast(id1 as UInt16) AS cond1, (id2 % 40000) AS cond2, (cond1 AND cond2) AS cond FROM test_02559 PREWHERE id2 > 5 AND cond LIMIT 10; From 0ed0b191f8a7edd142ec396fe26d9efe3dea13d3 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Sat, 18 Feb 2023 18:45:38 +0100 Subject: [PATCH 101/253] Fix for Nullable step filter column --- .../MergeTreeSplitPrewhereIntoReadSteps.cpp | 44 ++++++++++++++----- 1 file changed, 32 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp index 0dd91536bb4..b42900d239d 100644 --- a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -149,6 +149,27 @@ const ActionsDAG::Node & addCast( return addFunction(dag, func_builder_cast, std::move(children), node_remap); } +/// Normalizes the filter node by adding AND with a constant true. +/// This: +/// 1. produces a result with the proper Nullable or non-Nullable UInt8 type and +/// 2. makes sure that the result contains only 0 or 1 values even if the source column contains non-boolean values. +const ActionsDAG::Node & addAndTrue( + ActionsDAGPtr dag, + const ActionsDAG::Node & filter_node_to_normalize, + OriginalToNewNodeMap & node_remap) +{ + Field const_true_value(true); + + ColumnWithTypeAndName const_true_column; + const_true_column.column = DataTypeUInt8().createColumnConst(0, const_true_value); + const_true_column.type = std::make_shared(); + + const auto * const_true_node = &dag->addColumn(std::move(const_true_column)); + ActionsDAG::NodeRawConstPtrs children = {&filter_node_to_normalize, const_true_node}; + FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); + return addFunction(dag, func_builder_and, children, node_remap); +} + } /// We want to build a sequence of steps that will compute parts of the prewhere condition. @@ -241,15 +262,21 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction else { const auto & result_node = *new_condition_nodes.front(); - /// Add cast to UInt8 if needed - if (result_node.result_type->getTypeId() == TypeIndex::UInt8) + /// Check if explicit cast is needed for the condition to serve as a filter. + const auto result_type_name = result_node.result_type->getName(); + if (result_type_name == "UInt8" || + result_type_name == "Nullable(UInt8)" || + result_type_name == "LowCardinality(UInt8)" || + result_type_name == "LowCardinality(Nullable(UInt8))") { + /// No need to cast step_dag->addOrReplaceInOutputs(result_node); result_name = result_node.result_name; } else { - const auto & cast_node = addCast(step_dag, result_node, "UInt8", node_remap); + /// Build "condition AND True" expression to "cast" the condition to UInt8 or Nullable(UInt8) depending on its type. + const auto & cast_node = addAndTrue(step_dag, result_node, node_remap); step_dag->addOrReplaceInOutputs(cast_node); result_name = cast_node.result_name; } @@ -278,19 +305,12 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction /// to the last condition after filters from previous steps are applied. We just need to CAST the last condition /// to the type of combined filter. We do this in 2 steps: /// 1. AND the last condition with constant True. This is needed to make sure that in the last step filter has UInt8 type - /// but containes values other than 0 and 1 (e.g. if it is (number%5) it contains 2,3,4) + /// but contains values other than 0 and 1 (e.g. if it is (number%5) it contains 2,3,4) /// 2. CAST the result to the exact type of the PREWHERE column from the original DAG const auto & last_step_result_node_info = node_remap[steps.back().column_name]; auto & last_step_dag = steps.back().actions; /// Build AND(last_step_result_node, true) - Field true_constant_value(true); - ColumnWithTypeAndName column; - column.column = DataTypeUInt8().createColumnConst(0, true_constant_value); - column.type = std::make_shared(); - const auto * cast_type_constant_node = &last_step_dag->addColumn(std::move(column)); - ActionsDAG::NodeRawConstPtrs children = {last_step_result_node_info.node, cast_type_constant_node}; - FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); - const auto& and_node = addFunction(last_step_dag, func_builder_and, children, node_remap); + const auto & and_node = addAndTrue(last_step_dag, *last_step_result_node_info.node, node_remap); /// Build CAST(and_node, type of PREWHERE column) const auto & cast_node = addCast(last_step_dag, and_node, output->result_type->getName(), node_remap); /// Add alias for the result with the name of the PREWHERE column From f91a12717535a0cebd8d269cc7cf80b52969ccfa Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Sat, 18 Feb 2023 23:42:36 +0100 Subject: [PATCH 102/253] Repro for bug found by fuzzer and more similar cases --- ...559_multiple_read_steps_in_prewhere.reference | 16 ++++++++++++++++ .../02559_multiple_read_steps_in_prewhere.sql | 16 ++++++++++++++++ 2 files changed, 32 insertions(+) diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference index 580e7d4fef3..e713c2d01e5 100644 --- a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference +++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.reference @@ -49,6 +49,22 @@ SELECT * FROM test_02559 PREWHERE id1 <= 3 AND id2 > 0 WHERE (id1 + id2 < 15) LI 3 3 SELECT count() FROM test_02559 PREWHERE id2>=0 AND (1 OR ignore(id1)) WHERE ignore(id1)=0; 10 +SELECT count() FROM test_02559 PREWHERE ignore(id1); +0 +SELECT count() FROM test_02559 PREWHERE 1 OR ignore(id1); +10 +SELECT count() FROM test_02559 PREWHERE ignore(id1) AND id2 > 0; +0 +SELECT count() FROM test_02559 PREWHERE (1 OR ignore(id1)) AND id2 > 0; +9 +SELECT count() FROM test_02559 PREWHERE (id1 <= 10 AND id2 > 0) AND ignore(id1); +0 +SELECT count() FROM test_02559 PREWHERE ignore(id1) AND (id1 <= 10 AND id2 > 0); +0 +SELECT count() FROM test_02559 PREWHERE (id1 <= 10 AND id2 > 0) AND (1 OR ignore(id1)); +9 +SELECT count() FROM test_02559 PREWHERE (1 OR ignore(id1)) AND (id1 <= 10 AND id2 > 0); +9 CREATE ROW POLICY 02559_filter_1 ON test_02559 USING id2=2 AS permissive TO ALL; SELECT * FROM test_02559; 2 2 diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql index ee11b6edd55..1e969afac33 100644 --- a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql +++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere.sql @@ -29,6 +29,22 @@ SELECT * FROM test_02559 PREWHERE id1 <= 3 AND id2 > 0 WHERE (id1 + id2 < 15) LI SELECT count() FROM test_02559 PREWHERE id2>=0 AND (1 OR ignore(id1)) WHERE ignore(id1)=0; +SELECT count() FROM test_02559 PREWHERE ignore(id1); + +SELECT count() FROM test_02559 PREWHERE 1 OR ignore(id1); + +SELECT count() FROM test_02559 PREWHERE ignore(id1) AND id2 > 0; + +SELECT count() FROM test_02559 PREWHERE (1 OR ignore(id1)) AND id2 > 0; + +SELECT count() FROM test_02559 PREWHERE (id1 <= 10 AND id2 > 0) AND ignore(id1); + +SELECT count() FROM test_02559 PREWHERE ignore(id1) AND (id1 <= 10 AND id2 > 0); + +SELECT count() FROM test_02559 PREWHERE (id1 <= 10 AND id2 > 0) AND (1 OR ignore(id1)); + +SELECT count() FROM test_02559 PREWHERE (1 OR ignore(id1)) AND (id1 <= 10 AND id2 > 0); + CREATE ROW POLICY 02559_filter_1 ON test_02559 USING id2=2 AS permissive TO ALL; SELECT * FROM test_02559; From 1d28da3263c92a28a1dbfbedfe6f0baf3f8b9489 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Sun, 19 Feb 2023 11:56:14 +0100 Subject: [PATCH 103/253] Test for bug with missing column in PREWHERE step --- ...teps_in_prewhere_missing_columns.reference | 30 +++++++++++++++++++ ...read_steps_in_prewhere_missing_columns.sql | 30 +++++++++++++++++++ 2 files changed, 60 insertions(+) create mode 100644 tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_missing_columns.reference create mode 100644 tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_missing_columns.sql diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_missing_columns.reference b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_missing_columns.reference new file mode 100644 index 00000000000..0c10fa885cd --- /dev/null +++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_missing_columns.reference @@ -0,0 +1,30 @@ +1 Hello, world! 0 +2 Goodbye. 3 +-- { echoOn } +SELECT s FROM test_02559 PREWHERE x AND y ORDER BY s; +Goodbye. +SELECT s, y FROM test_02559 PREWHERE y ORDER BY s; +Goodbye. 3 +SELECT s, y FROM test_02559 PREWHERE NOT y ORDER BY s; +Hello, world! 0 +SELECT s, y FROM test_02559 PREWHERE x AND y ORDER BY s; +Goodbye. 3 +SELECT s, y FROM test_02559 PREWHERE x AND NOT y ORDER BY s; +Hello, world! 0 +SELECT s, y FROM test_02559 PREWHERE y AND x ORDER BY s; +Goodbye. 3 +SELECT s, y FROM test_02559 PREWHERE (NOT y) AND x ORDER BY s; +Hello, world! 0 +ALTER TABLE test_02559 ADD COLUMN z UInt8 DEFAULT 10; +INSERT INTO test_02559 VALUES (3, 'So long, and thanks for all the fish.', 42, 0); +SELECT * FROM test_02559 ORDER BY x; +1 Hello, world! 0 10 +2 Goodbye. 3 10 +3 So long, and thanks for all the fish. 42 0 +SELECT s FROM test_02559 PREWHERE z ORDER BY s; +Goodbye. +Hello, world! +SELECT s FROM test_02559 PREWHERE y AND z ORDER BY s; +Goodbye. +SELECT s, z FROM test_02559 PREWHERE NOT y AND z ORDER BY s; +Hello, world! 10 diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_missing_columns.sql b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_missing_columns.sql new file mode 100644 index 00000000000..f6299122ef2 --- /dev/null +++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_missing_columns.sql @@ -0,0 +1,30 @@ +DROP TABLE IF EXISTS test_02559; +CREATE TABLE test_02559 (x UInt8, s String) ENGINE = MergeTree ORDER BY tuple(); + +INSERT INTO test_02559 VALUES (1, 'Hello, world!'); + +ALTER TABLE test_02559 ADD COLUMN y UInt8 DEFAULT 0; +INSERT INTO test_02559 VALUES (2, 'Goodbye.', 3); +SELECT * FROM test_02559 ORDER BY x; + +SET enable_multiple_prewhere_read_steps=true, move_all_conditions_to_prewhere=true; + +-- { echoOn } +SELECT s FROM test_02559 PREWHERE x AND y ORDER BY s; +SELECT s, y FROM test_02559 PREWHERE y ORDER BY s; +SELECT s, y FROM test_02559 PREWHERE NOT y ORDER BY s; +SELECT s, y FROM test_02559 PREWHERE x AND y ORDER BY s; +SELECT s, y FROM test_02559 PREWHERE x AND NOT y ORDER BY s; +SELECT s, y FROM test_02559 PREWHERE y AND x ORDER BY s; +SELECT s, y FROM test_02559 PREWHERE (NOT y) AND x ORDER BY s; + +ALTER TABLE test_02559 ADD COLUMN z UInt8 DEFAULT 10; +INSERT INTO test_02559 VALUES (3, 'So long, and thanks for all the fish.', 42, 0); +SELECT * FROM test_02559 ORDER BY x; + +SELECT s FROM test_02559 PREWHERE z ORDER BY s; +SELECT s FROM test_02559 PREWHERE y AND z ORDER BY s; +SELECT s, z FROM test_02559 PREWHERE NOT y AND z ORDER BY s; +-- { echoOff } + +DROP TABLE test_02559; From 520b3816758e1908170a123730bd1d52dccd7202 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Sun, 19 Feb 2023 12:14:23 +0100 Subject: [PATCH 104/253] Fixes for executing actions on block with unknown row count --- .../MergeTree/MergeTreeRangeReader.cpp | 44 +++++++++++++++++++ 1 file changed, 44 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 4036d352a54..44f6cf9f70d 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -921,6 +922,39 @@ bool MergeTreeRangeReader::isCurrentRangeFinished() const return prev_reader ? prev_reader->isCurrentRangeFinished() : stream.isFinished(); } + +/// When executing ExpressionActions on an empty block, it is not possible to determine the number of rows +/// in the block for the new columns so the result block will have 0 rows and it will not match the rest of +/// the columns in the ReadResult. +/// The dummy column is added to maintain the information about the number of rows in the block and to produce +/// the result block with the correct number of rows. +String addDummyColumnWithRowCount(Block & block, size_t num_rows) +{ + bool has_columns = false; + for (const auto & column : block) + { + if (column.column) + { + assert(column.column->size() == num_rows); + has_columns = true; + break; + } + } + + if (has_columns) + return {}; + + ColumnWithTypeAndName dummy_column; + dummy_column.column = DataTypeUInt8().createColumnConst(num_rows, Field(1)); + dummy_column.type = std::make_shared(); + /// Generate a random name to avoid collisions with real columns. + dummy_column.name = "....dummy...." + toString(UUIDHelpers::generateV4()); + block.insert(dummy_column); + + return dummy_column.name; +} + + MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, MarkRanges & ranges) { if (max_rows == 0) @@ -988,6 +1022,7 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar for (const auto & col : read_result.additional_columns) additional_columns.insert(col); + addDummyColumnWithRowCount(additional_columns, read_result.num_rows); merge_tree_reader->evaluateMissingDefaults(additional_columns, columns); } @@ -1309,8 +1344,17 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r Block additional_columns = block; if (prewhere_info->actions) + { + const String dummy_column = addDummyColumnWithRowCount(block, result.num_rows); + + LOG_TEST(log, "Executing prewhere actions on block: {}", block.dumpStructure()); + prewhere_info->actions->execute(block); + if (!dummy_column.empty()) + block.erase(dummy_column); + } + result.additional_columns.clear(); /// Additional columns might only be needed if there are more steps in the chain. if (!last_reader_in_chain) From b6394ae55931fe48077a46726ac9dab0874765b1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Feb 2023 00:17:53 +0100 Subject: [PATCH 105/253] Add bug --- .../0_stateless/02667_wrong_structure.reference | 0 .../queries/0_stateless/02667_wrong_structure.sql | 15 +++++++++++++++ 2 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/02667_wrong_structure.reference create mode 100644 tests/queries/0_stateless/02667_wrong_structure.sql diff --git a/tests/queries/0_stateless/02667_wrong_structure.reference b/tests/queries/0_stateless/02667_wrong_structure.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02667_wrong_structure.sql b/tests/queries/0_stateless/02667_wrong_structure.sql new file mode 100644 index 00000000000..e8dc20a1aad --- /dev/null +++ b/tests/queries/0_stateless/02667_wrong_structure.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS test_table__fuzz_0; +DROP TABLE IF EXISTS test_table__fuzz_2; + +SET allow_suspicious_low_cardinality_types = 1; + +CREATE TABLE test_table__fuzz_0 (`id` UInt64, `value` String) ENGINE = TinyLog; +CREATE TABLE test_table__fuzz_2 (`id` LowCardinality(UInt64), `value` LowCardinality(String)) ENGINE = TinyLog; + +INSERT INTO test_table__fuzz_0 VALUES (1911323367950415347,'@]~2|%N'),(15844130449337515313,'),Z?yV'),(10948652031493940763,'RC'),(10996795342436375388,''),(6019500279534407119,''),(3133348157913025617,'{fW!'),(8081599482719673616,'s'),(11115208170338871149,'sg`:}'),(9740456510723628701,'P'),(16586086969122622868,'Bn'); +INSERT INTO test_table__fuzz_2 VALUES (9806329011943062144,'wS6*'),(15996368926351777736,'pg.eA77G+P'),(5001315181732963827,'Qt,1)+pxB'),(10351886378573180883,'h--;'),(12739541939251729647,'>7sm*'),(6903016743298037689,'aVF8.ZU'),(9608529974121700083,',ZT'),(12616722808145461678,'#9Qp]'),(13678566439661733359,' J8'),(13909219891163965895,'.'); + +SELECT arrayMap(x -> (id + (SELECT id FROM test_table__fuzz_0 WHERE arrayMap(x -> (id + 9806329011943062144), [10])[NULL])), [1048575]) FROM test_table__fuzz_2; + +DROP TABLE test_table__fuzz_0; +DROP TABLE test_table__fuzz_2; From 1bd9629ed0c3e7484112e46dbc5f03ae85d430f9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Feb 2023 00:21:14 +0100 Subject: [PATCH 106/253] Simpler --- tests/queries/0_stateless/02667_wrong_structure.sql | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02667_wrong_structure.sql b/tests/queries/0_stateless/02667_wrong_structure.sql index e8dc20a1aad..16e6eff82b7 100644 --- a/tests/queries/0_stateless/02667_wrong_structure.sql +++ b/tests/queries/0_stateless/02667_wrong_structure.sql @@ -1,15 +1,11 @@ -DROP TABLE IF EXISTS test_table__fuzz_0; DROP TABLE IF EXISTS test_table__fuzz_2; SET allow_suspicious_low_cardinality_types = 1; -CREATE TABLE test_table__fuzz_0 (`id` UInt64, `value` String) ENGINE = TinyLog; CREATE TABLE test_table__fuzz_2 (`id` LowCardinality(UInt64), `value` LowCardinality(String)) ENGINE = TinyLog; -INSERT INTO test_table__fuzz_0 VALUES (1911323367950415347,'@]~2|%N'),(15844130449337515313,'),Z?yV'),(10948652031493940763,'RC'),(10996795342436375388,''),(6019500279534407119,''),(3133348157913025617,'{fW!'),(8081599482719673616,'s'),(11115208170338871149,'sg`:}'),(9740456510723628701,'P'),(16586086969122622868,'Bn'); -INSERT INTO test_table__fuzz_2 VALUES (9806329011943062144,'wS6*'),(15996368926351777736,'pg.eA77G+P'),(5001315181732963827,'Qt,1)+pxB'),(10351886378573180883,'h--;'),(12739541939251729647,'>7sm*'),(6903016743298037689,'aVF8.ZU'),(9608529974121700083,',ZT'),(12616722808145461678,'#9Qp]'),(13678566439661733359,' J8'),(13909219891163965895,'.'); +INSERT INTO test_table__fuzz_2 VALUES (9806329011943062144,'wS6*'); -SELECT arrayMap(x -> (id + (SELECT id FROM test_table__fuzz_0 WHERE arrayMap(x -> (id + 9806329011943062144), [10])[NULL])), [1048575]) FROM test_table__fuzz_2; +SELECT arrayMap(x -> (id + (SELECT c1 AS id FROM VALUES((1911323367950415347, '@]~2|%N')) WHERE arrayMap(x -> (id + 9806329011943062144), [10])[NULL])), [1048575]) FROM test_table__fuzz_2; -DROP TABLE test_table__fuzz_0; DROP TABLE test_table__fuzz_2; From f63ed43f216dc34544ab62fc580574d8352ef99a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Feb 2023 00:21:51 +0100 Subject: [PATCH 107/253] Simpler --- tests/queries/0_stateless/02667_wrong_structure.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02667_wrong_structure.sql b/tests/queries/0_stateless/02667_wrong_structure.sql index 16e6eff82b7..384a7b70a43 100644 --- a/tests/queries/0_stateless/02667_wrong_structure.sql +++ b/tests/queries/0_stateless/02667_wrong_structure.sql @@ -6,6 +6,6 @@ CREATE TABLE test_table__fuzz_2 (`id` LowCardinality(UInt64), `value` LowCardina INSERT INTO test_table__fuzz_2 VALUES (9806329011943062144,'wS6*'); -SELECT arrayMap(x -> (id + (SELECT c1 AS id FROM VALUES((1911323367950415347, '@]~2|%N')) WHERE arrayMap(x -> (id + 9806329011943062144), [10])[NULL])), [1048575]) FROM test_table__fuzz_2; +SELECT arrayMap(x -> (id + (SELECT 1911323367950415347 AS id WHERE arrayMap(x -> (id + 9806329011943062144), [10])[NULL])), [1048575]) FROM test_table__fuzz_2; DROP TABLE test_table__fuzz_2; From 77373a9381a8cff730cfd88b712c1763dc6c2a0f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Feb 2023 00:23:30 +0100 Subject: [PATCH 108/253] Simpler --- tests/queries/0_stateless/02667_wrong_structure.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02667_wrong_structure.sql b/tests/queries/0_stateless/02667_wrong_structure.sql index 384a7b70a43..70d84cd9242 100644 --- a/tests/queries/0_stateless/02667_wrong_structure.sql +++ b/tests/queries/0_stateless/02667_wrong_structure.sql @@ -6,6 +6,6 @@ CREATE TABLE test_table__fuzz_2 (`id` LowCardinality(UInt64), `value` LowCardina INSERT INTO test_table__fuzz_2 VALUES (9806329011943062144,'wS6*'); -SELECT arrayMap(x -> (id + (SELECT 1911323367950415347 AS id WHERE arrayMap(x -> (id + 9806329011943062144), [10])[NULL])), [1048575]) FROM test_table__fuzz_2; +SELECT arrayMap(x -> (id + (SELECT 1 AS id WHERE arrayMap(x -> (id + 1), [10])[NULL])), [1048575]) FROM test_table__fuzz_2; DROP TABLE test_table__fuzz_2; From d9c675e1ca3fa84cf81e367906373992ab7662a9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Feb 2023 00:24:41 +0100 Subject: [PATCH 109/253] Simpler --- tests/queries/0_stateless/02667_wrong_structure.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02667_wrong_structure.sql b/tests/queries/0_stateless/02667_wrong_structure.sql index 70d84cd9242..9997143cfdf 100644 --- a/tests/queries/0_stateless/02667_wrong_structure.sql +++ b/tests/queries/0_stateless/02667_wrong_structure.sql @@ -6,6 +6,6 @@ CREATE TABLE test_table__fuzz_2 (`id` LowCardinality(UInt64), `value` LowCardina INSERT INTO test_table__fuzz_2 VALUES (9806329011943062144,'wS6*'); -SELECT arrayMap(x -> (id + (SELECT 1 AS id WHERE arrayMap(x -> (id + 1), [10])[NULL])), [1048575]) FROM test_table__fuzz_2; +SELECT arrayMap(x -> (id + (SELECT 1 AS id WHERE [10][NULL])), [1048575]) FROM test_table__fuzz_2; DROP TABLE test_table__fuzz_2; From 5427368fda5a5cfba9fbdadf56afefbd7ebdfbe3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Feb 2023 00:26:46 +0100 Subject: [PATCH 110/253] Simpler --- tests/queries/0_stateless/02667_wrong_structure.sql | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/02667_wrong_structure.sql b/tests/queries/0_stateless/02667_wrong_structure.sql index 9997143cfdf..04370073c98 100644 --- a/tests/queries/0_stateless/02667_wrong_structure.sql +++ b/tests/queries/0_stateless/02667_wrong_structure.sql @@ -1,11 +1 @@ -DROP TABLE IF EXISTS test_table__fuzz_2; - -SET allow_suspicious_low_cardinality_types = 1; - -CREATE TABLE test_table__fuzz_2 (`id` LowCardinality(UInt64), `value` LowCardinality(String)) ENGINE = TinyLog; - -INSERT INTO test_table__fuzz_2 VALUES (9806329011943062144,'wS6*'); - -SELECT arrayMap(x -> (id + (SELECT 1 AS id WHERE [10][NULL])), [1048575]) FROM test_table__fuzz_2; - -DROP TABLE test_table__fuzz_2; +SELECT arrayMap(x -> (toLowCardinality(1) + (SELECT 1 WHERE 0)), [1]); From e8b4869ec02674ae43601035060cbb7d2d71c157 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Feb 2023 00:28:29 +0100 Subject: [PATCH 111/253] Simpler --- tests/queries/0_stateless/02667_wrong_structure.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02667_wrong_structure.sql b/tests/queries/0_stateless/02667_wrong_structure.sql index 04370073c98..ed3fca5e0f2 100644 --- a/tests/queries/0_stateless/02667_wrong_structure.sql +++ b/tests/queries/0_stateless/02667_wrong_structure.sql @@ -1 +1 @@ -SELECT arrayMap(x -> (toLowCardinality(1) + (SELECT 1 WHERE 0)), [1]); +SELECT arrayMap(x -> (toLowCardinality(1) + 1::Nullable(UInt8)), [1]); From d48ec14cabc454fc54001921d26df3ad1af1795d Mon Sep 17 00:00:00 2001 From: Jus <40656180+jus1096@users.noreply.github.com> Date: Mon, 20 Feb 2023 10:59:57 +0400 Subject: [PATCH 112/253] Add description function Add description function. Fix bugs --- .../functions/other-functions.md | 120 +++++++++++++++++- 1 file changed, 119 insertions(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index af21ccd6bed..6e0c4ba146d 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -295,6 +295,10 @@ SELECT byteSize(NULL, 1, 0.3, ''); Спит seconds секунд на каждый блок данных. Можно указать как целое число, так и число с плавающей запятой. +## sleepEachRow(seconds) {# sleepeachrowseconds} + +Спит seconds секунд на каждую строку. Можно указать как целое число, так и число с плавающей запятой. + ## currentDatabase() {#currentdatabase} Возвращает имя текущей базы данных. @@ -590,6 +594,27 @@ LIMIT 10 └────────────────┴─────────┘ ``` +## formatReadableDecimalSize(x) + +Принимает размер (число байт). Возвращает округленный размер с суффиксом (KiB, MiB и т.д.) в виде строки. + +Пример: + +``` sql +SELECT + arrayJoin([1, 1024, 1024*1024, 192851925]) AS filesize_bytes, + formatReadableDecimalSize(filesize_bytes) AS filesize +``` + +``` text +┌─filesize_bytes─┬─filesize───┐ +│ 1 │ 1.00 B │ +│ 1024 │ 1.02 KB │ +│ 1048576 │ 1.05 MB │ +│ 192851925 │ 192.85 MB │ +└────────────────┴────────────┘ +``` + ## formatReadableSize(x) {#formatreadablesizex} Принимает размер (число байт). Возвращает округленный размер с суффиксом (KiB, MiB и т.д.) в виде строки. @@ -634,6 +659,92 @@ SELECT └────────────────┴───────────────────┘ ``` +## formatReadableTimeDelta {#formatreadabletimedelta} + +Принимает дельту времени в секундах. Возвращает дельту времени с (год, месяц, день, час, минута, секунда) в виде строки. + +**Синтаксис** + +``` sql +formatReadableTimeDelta(column[, maximum_unit]) +``` + +**Аргументы** + +- `column` — Столбец с числовой дельтой времени. +- `maximum_unit` — Опицонально. Максимальная единица измерения для отображения. Допустимые значения: секунды, минуты, часы, дни, месяцы, годы. + +Пример: + +``` sql +SELECT + arrayJoin([100, 12345, 432546534]) AS elapsed, + formatReadableTimeDelta(elapsed) AS time_delta +``` + +``` text +┌────elapsed─┬─time_delta ─────────────────────────────────────────────────────┐ +│ 100 │ 1 minute and 40 seconds │ +│ 12345 │ 3 hours, 25 minutes and 45 seconds │ +│ 432546534 │ 13 years, 8 months, 17 days, 7 hours, 48 minutes and 54 seconds │ +└────────────┴─────────────────────────────────────────────────────────────────┘ +``` + +``` sql +SELECT + arrayJoin([100, 12345, 432546534]) AS elapsed, + formatReadableTimeDelta(elapsed, 'minutes') AS time_delta +``` + +``` text +┌────elapsed─┬─time_delta ─────────────────────────────────────────────────────┐ +│ 100 │ 1 minute and 40 seconds │ +│ 12345 │ 205 minutes and 45 seconds │ +│ 432546534 │ 7209108 minutes and 54 seconds │ +└────────────┴─────────────────────────────────────────────────────────────────┘ +``` + +## parseTimeDelta {#parsetimedelta} + +Преобразует последовательность символов, которая напоминает нечто похожее на единицу времени. + +**Синтаксис** + +```sql +parseTimeDelta(timestr) +``` + +**Аргументы** + +- `timestr` — Последовательность символов, которая напоминает нечто похожее на единицу времени. + + +**Возвращаемое значение** + +- Число с плавающей точкой, содержащее количество секунд. + +**Пример** + +```sql +SELECT parseTimeDelta('11s+22min') +``` + +```text +┌─parseTimeDelta('11s+22min')─┐ +│ 1331 │ +└─────────────────────────────┘ +``` + +```sql +SELECT parseTimeDelta('1yr2mo') +``` + +```text +┌─parseTimeDelta('1yr2mo')─┐ +│ 36806400 │ +└──────────────────────────┘ +``` + ## least(a, b) {#leasta-b} Возвращает наименьшее значение из a и b. @@ -657,6 +768,10 @@ SELECT Возвращает ID сборки, сгенерированный компилятором для данного сервера ClickHouse. Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями, актуальными для каждого шарда. Иначе возвращается константа. +## blockNumber {#function-blocknumber} + +Возвращает порядковый номер блока данных, в котором находится строка. + ## rowNumberInBlock {#function-rownumberinblock} Возвращает порядковый номер строки в блоке данных. Для каждого блока данных нумерация начинается с 0. @@ -679,6 +794,7 @@ neighbor(column, offset[, default_value]) :::danger "Предупреждение" Функция может получить доступ к значению в столбце соседней строки только внутри обрабатываемого в данный момент блока данных. +::: Порядок строк, используемый при вычислении функции `neighbor`, может отличаться от порядка строк, возвращаемых пользователю. Чтобы этого не случилось, вы можете сделать подзапрос с [ORDER BY](../../sql-reference/statements/select/order-by.md) и вызвать функцию извне подзапроса. @@ -788,6 +904,7 @@ FROM numbers(16) :::danger "Предупреждение" Функция может взять значение предыдущей строки только внутри текущего обработанного блока данных. +::: Результат функции зависит от затронутых блоков данных и порядка данных в блоке. @@ -869,7 +986,7 @@ WHERE diff != 1 :::danger "Предупреждение" События должны быть отсортированы по возрастанию времени начала. Если это требование нарушено, то функция вызывает исключение. Каждый блок данных обрабатывается независимо. Если события из разных блоков данных накладываются по времени, они не могут быть корректно обработаны. - +::: **Синтаксис** ``` sql @@ -1560,6 +1677,7 @@ FROM numbers(10); :::danger "Warning" Функция обнуляет состояние для каждого нового блока. +::: **Синтаксис** From 238d44783b10e89584432091a1cdd4ff843b3ba5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 20 Feb 2023 09:05:48 +0100 Subject: [PATCH 113/253] Fix flakiness of expect tests for clickhouse-client by avoiding history overlap Yes, all writes to the history file is done under flock, *but*, before writing the history file there is sort(), and so if you will run the following tests the 01300_client_save_history_when_terminated_long will fail: $ /src/tests/clickhouse-test --print-time -j2 01180_client_syntax_errors 01300_client_save_history_when_terminated_long 0001_select And it has nothing todo with timeouts: expect: does "" (spawn_id exp8) match glob pattern "for the history"? no f8f1dbfdaaca :) select (1, 2 expect: does "\u001b[1Gf8f1dbfdaaca :) select \u001b[0;22;33m(\u001b[0;22;32m1\u001b[0;1m,\u001b[0m \u001b[0;22;32m2\u001b[0m\u001b[J" (spawn_id exp8) match glob pattern "for the history"? no expect: does "\u001b[1Gf8f1dbfdaaca :) select \u001b[0;22;33m(\u001b[0;22;32m1\u001b[0;1m,\u001b[0m \u001b[0;22;32m2\u001b[0m\u001b[J\u001b[29G" (spawn_id exp8) match glob pattern "for the history"? no expect: timed out The "select (1, 2" is from 01180_client_syntax_errors And use real file only when the history should be preserved across runs (i.e. there are multiple invocations of clickhouse-client) CI: https://s3.amazonaws.com/clickhouse-test-reports/0/1adfbac19fed9813725d8b1df14e617b58a45d20/stateless_tests__asan__[2/2].html Signed-off-by: Azat Khuzhin --- .gitignore | 1 + .../01179_insert_values_semicolon.expect | 2 +- .../01180_client_syntax_errors.expect | 2 +- ...client_interactive_vertical_multiline.expect | 5 +++-- ...lient_interactive_vertical_singleline.expect | 2 +- ...ent_save_history_when_terminated_long.expect | 5 +++-- ...nt_autocomplete_word_break_characters.expect | 2 +- .../01520_client_print_query_id.expect | 2 +- .../01565_reconnect_after_client_error.expect | 2 +- ...ghlight_multi_line_comment_regression.expect | 2 +- .../01933_client_replxx_convert_history.expect | 17 ++++++++--------- .../0_stateless/01945_show_debug_warning.expect | 8 ++++---- .../02003_memory_limit_in_client.expect | 6 +++--- .../0_stateless/02047_client_exception.expect | 2 +- .../02105_backslash_letter_commands.expect | 4 ++-- .../0_stateless/02116_interactive_hello.expect | 2 +- .../02132_client_history_navigation.expect | 4 ++-- ...02160_client_autocomplete_parse_query.expect | 2 +- .../02417_repeat_input_commands.expect | 4 ++-- 19 files changed, 38 insertions(+), 36 deletions(-) diff --git a/.gitignore b/.gitignore index 7a513ec1a09..14b860244c2 100644 --- a/.gitignore +++ b/.gitignore @@ -161,6 +161,7 @@ website/package-lock.json tests/queries/0_stateless/test_* tests/queries/0_stateless/*.binary tests/queries/0_stateless/*.generated-expect +tests/queries/0_stateless/*.expect.history # rust /rust/**/target diff --git a/tests/queries/0_stateless/01179_insert_values_semicolon.expect b/tests/queries/0_stateless/01179_insert_values_semicolon.expect index 9d35941ae40..d66a84769f2 100755 --- a/tests/queries/0_stateless/01179_insert_values_semicolon.expect +++ b/tests/queries/0_stateless/01179_insert_values_semicolon.expect @@ -15,7 +15,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=/dev/null" expect ":) " send -- "DROP TABLE IF EXISTS test_01179\r" diff --git a/tests/queries/0_stateless/01180_client_syntax_errors.expect b/tests/queries/0_stateless/01180_client_syntax_errors.expect index da3dfbec6df..a031333313a 100755 --- a/tests/queries/0_stateless/01180_client_syntax_errors.expect +++ b/tests/queries/0_stateless/01180_client_syntax_errors.expect @@ -14,7 +14,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=/dev/null" expect ":) " # Make a query with syntax error diff --git a/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect b/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect index bab1dd224cf..629698b4565 100755 --- a/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect +++ b/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect @@ -3,6 +3,7 @@ set basedir [file dirname $argv0] set basename [file tail $argv0] exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 +set history_file $env(CLICKHOUSE_TMP)/$basename.history log_user 0 set timeout 10 @@ -18,7 +19,7 @@ expect_after { # useful debugging configuration # exp_internal 1 -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file" expect ":) " send -- "SELECT 1\r" @@ -60,7 +61,7 @@ expect ":) " send -- "" expect eof -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --highlight 0 --multiline" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --highlight 0 --multiline --history_file=$history_file" expect ":) " send -- "SELECT 1;\r" diff --git a/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect b/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect index 83eced841ce..e64d71c6319 100755 --- a/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect +++ b/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect @@ -14,7 +14,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=/dev/null" expect ":) " send -- "SELECT 1\r" diff --git a/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect b/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect index 06a60ed95a2..de485383024 100755 --- a/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect +++ b/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect @@ -4,6 +4,7 @@ set basedir [file dirname $argv0] set basename [file tail $argv0] exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 +set history_file $env(CLICKHOUSE_TMP)/$basename.history log_user 0 set timeout 60 @@ -15,7 +16,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file" expect ":) " # Make a query @@ -28,7 +29,7 @@ exec kill -9 [exp_pid] close # Run client one more time and press "up" to see the last recorded query -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --history_file=$history_file" expect ":) " send -- "\[A" expect "for the history" diff --git a/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect b/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect index fff0dd015e1..bbd0fdd223b 100755 --- a/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect +++ b/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect @@ -14,7 +14,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --history_file=/dev/null" expect ":) " # Make a query diff --git a/tests/queries/0_stateless/01520_client_print_query_id.expect b/tests/queries/0_stateless/01520_client_print_query_id.expect index 0e8f660041d..b16cd6d499b 100755 --- a/tests/queries/0_stateless/01520_client_print_query_id.expect +++ b/tests/queries/0_stateless/01520_client_print_query_id.expect @@ -14,7 +14,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=/dev/null" expect ":) " # Make a query diff --git a/tests/queries/0_stateless/01565_reconnect_after_client_error.expect b/tests/queries/0_stateless/01565_reconnect_after_client_error.expect index 035698f524b..076b91390bd 100755 --- a/tests/queries/0_stateless/01565_reconnect_after_client_error.expect +++ b/tests/queries/0_stateless/01565_reconnect_after_client_error.expect @@ -19,7 +19,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion -mn" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion -mn --history_file=/dev/null" expect "\n:) " send -- "DROP TABLE IF EXISTS t01565;\n" diff --git a/tests/queries/0_stateless/01755_client_highlight_multi_line_comment_regression.expect b/tests/queries/0_stateless/01755_client_highlight_multi_line_comment_regression.expect index 3d9c633eb44..293fd2e13f4 100755 --- a/tests/queries/0_stateless/01755_client_highlight_multi_line_comment_regression.expect +++ b/tests/queries/0_stateless/01755_client_highlight_multi_line_comment_regression.expect @@ -14,7 +14,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=/dev/null" expect ":) " # regression for heap-buffer-overflow issue (under ASAN) diff --git a/tests/queries/0_stateless/01933_client_replxx_convert_history.expect b/tests/queries/0_stateless/01933_client_replxx_convert_history.expect index 111389e49b2..0c95b630742 100755 --- a/tests/queries/0_stateless/01933_client_replxx_convert_history.expect +++ b/tests/queries/0_stateless/01933_client_replxx_convert_history.expect @@ -1,10 +1,9 @@ #!/usr/bin/expect -f -# Tags: no-parallel -# Tag no-parallel: Uses non unique history file set basedir [file dirname $argv0] set basename [file tail $argv0] exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 +set history_file $env(CLICKHOUSE_TMP)/$basename.history log_user 0 set timeout 60 @@ -16,17 +15,17 @@ expect_after { timeout { exit 1 } } -exec bash -c "echo select 1 > $argv0.txt" -exec bash -c "echo select 1 >> $argv0.txt" -exec bash -c "echo select 1 >> $argv0.txt" +exec bash -c "echo select 1 > $history_file.txt" +exec bash -c "echo select 1 >> $history_file.txt" +exec bash -c "echo select 1 >> $history_file.txt" -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --history_file=$argv0.txt" -expect "The history file ($argv0.txt) is in old format. 3 lines, 1 unique lines." +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --history_file=$history_file.txt" +expect "The history file ($history_file.txt) is in old format. 3 lines, 1 unique lines." expect ":) " send -- "\4" expect eof -spawn bash -c "wc -l $argv0.txt" +spawn bash -c "wc -l $history_file.txt" # The following lines are expected: # # ### YYYY-MM-DD HH:MM:SS.SSS @@ -35,4 +34,4 @@ spawn bash -c "wc -l $argv0.txt" expect "2" expect eof -exec bash -c "rm $argv0.txt" +exec bash -c "rm $history_file.txt" diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index ca423ee106c..4a0abb358d5 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -20,14 +20,14 @@ expect_after { set Debug_type 0 -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=/dev/null" expect ":) " # Check debug type send -- "SELECT value FROM system.build_options WHERE name='BUILD_TYPE'\r" expect { "Debug" { - set Debug_type 1 + set Debug_type 1 expect ":) " } "RelWithDebInfo" @@ -38,7 +38,7 @@ expect eof if { $Debug_type > 0} { -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=/dev/null" expect "Warnings:" expect " * Server was built in debug mode. It will work slowly." expect ":) " @@ -52,7 +52,7 @@ send -- "q\r" expect eof } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=/dev/null" expect "Warnings:" expect " * Some obsolete setting is changed." expect ":) " diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.expect b/tests/queries/0_stateless/02003_memory_limit_in_client.expect index a8e8c1d5786..e03fb9308f4 100755 --- a/tests/queries/0_stateless/02003_memory_limit_in_client.expect +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.expect @@ -22,7 +22,7 @@ expect_after { # # Check that the query will fail in clickhouse-client # -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1 --history_file=/dev/null" expect ":) " send -- "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000)\r" @@ -37,7 +37,7 @@ expect eof # # Check that the query will fail in clickhouse-client # -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1 --history_file=/dev/null" expect ":) " send -- "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000)\r" @@ -52,7 +52,7 @@ expect eof # # Check that the query will not fail (due to max_untracked_memory) # -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1 --history_file=/dev/null" expect ":) " send -- "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000\r" diff --git a/tests/queries/0_stateless/02047_client_exception.expect b/tests/queries/0_stateless/02047_client_exception.expect index 50ed09d03c5..9ae8495cfcb 100755 --- a/tests/queries/0_stateless/02047_client_exception.expect +++ b/tests/queries/0_stateless/02047_client_exception.expect @@ -15,7 +15,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=/dev/null" expect ":) " send -- "DROP TABLE IF EXISTS test_02047\r" diff --git a/tests/queries/0_stateless/02105_backslash_letter_commands.expect b/tests/queries/0_stateless/02105_backslash_letter_commands.expect index 707a544f6bb..a7340020611 100755 --- a/tests/queries/0_stateless/02105_backslash_letter_commands.expect +++ b/tests/queries/0_stateless/02105_backslash_letter_commands.expect @@ -5,7 +5,7 @@ set basename [file tail $argv0] exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 log_user 0 -set timeout 02 +set timeout 60 match_max 100000 expect_after { # Do not ignore eof from expect @@ -14,7 +14,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=/dev/null" expect ":) " # Send a command diff --git a/tests/queries/0_stateless/02116_interactive_hello.expect b/tests/queries/0_stateless/02116_interactive_hello.expect index 5fa31d33e87..443f835b76d 100755 --- a/tests/queries/0_stateless/02116_interactive_hello.expect +++ b/tests/queries/0_stateless/02116_interactive_hello.expect @@ -16,7 +16,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=/dev/null" expect -re "ClickHouse client version \[\\d\]{2}.\[\\d\]{1,2}.\[\\d\]{1,2}.\[\\d\]{1,2}.\r" expect -re "Connecting to database .* at localhost:9000 as user default.\r" diff --git a/tests/queries/0_stateless/02132_client_history_navigation.expect b/tests/queries/0_stateless/02132_client_history_navigation.expect index 10167fb2e97..82e994f4fc1 100755 --- a/tests/queries/0_stateless/02132_client_history_navigation.expect +++ b/tests/queries/0_stateless/02132_client_history_navigation.expect @@ -5,7 +5,7 @@ set basename [file tail $argv0] exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 log_user 0 -set timeout 3 +set timeout 60 match_max 100000 expect_after { @@ -18,7 +18,7 @@ expect_after { # useful debugging configuration # exp_internal 1 -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --highlight 0" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --highlight 0 --history_file=/dev/null" expect ":) " # Make a query diff --git a/tests/queries/0_stateless/02160_client_autocomplete_parse_query.expect b/tests/queries/0_stateless/02160_client_autocomplete_parse_query.expect index b95f85403e3..12bfcc7c041 100755 --- a/tests/queries/0_stateless/02160_client_autocomplete_parse_query.expect +++ b/tests/queries/0_stateless/02160_client_autocomplete_parse_query.expect @@ -15,7 +15,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --history_file=/dev/null" expect ":) " # Make a query diff --git a/tests/queries/0_stateless/02417_repeat_input_commands.expect b/tests/queries/0_stateless/02417_repeat_input_commands.expect index 119aac68645..85be0a02881 100755 --- a/tests/queries/0_stateless/02417_repeat_input_commands.expect +++ b/tests/queries/0_stateless/02417_repeat_input_commands.expect @@ -5,7 +5,7 @@ set basename [file tail $argv0] exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 log_user 0 -set timeout 10 +set timeout 60 match_max 100000 expect_after { @@ -15,7 +15,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=/dev/null" expect ":) " # ----------------------------------------- From 2018559fa5f93c31af8f524c271504239d3ed56b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 20 Feb 2023 09:10:28 +0100 Subject: [PATCH 114/253] Add style check for using --history_file in expect tests Signed-off-by: Azat Khuzhin --- utils/check-style/check-style | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 1ae003174d7..8436d3378d9 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -333,6 +333,11 @@ expect_tests=( $(find $ROOT_PATH/tests/queries -name '*.expect') ) for test_case in "${expect_tests[@]}"; do pattern="^exp_internal -f \$env(CLICKHOUSE_TMP)/\$basename.debuglog 0$" grep -q "$pattern" "$test_case" || echo "Missing '$pattern' in '$test_case'" + + if grep -q "^spawn.*CLICKHOUSE_CLIENT_BINARY$" "$test_case"; then + pattern="^spawn.*CLICKHOUSE_CLIENT_BINARY.*--history_file$" + grep -q "$pattern" "$test_case" || echo "Missing '$pattern' in '$test_case'" + fi done # Conflict markers From 6e1f284edac731d902b38c8062f895d92b47e9e3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 20 Feb 2023 09:25:28 +0100 Subject: [PATCH 115/253] Improve detection of debug builds in 01945_show_debug_warning Signed-off-by: Azat Khuzhin --- .../0_stateless/01945_show_debug_warning.expect | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index 4a0abb358d5..6714901ab20 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -24,13 +24,13 @@ spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \ expect ":) " # Check debug type -send -- "SELECT value FROM system.build_options WHERE name='BUILD_TYPE'\r" +send -- "SELECT lower(value) FROM system.build_options WHERE name='BUILD_TYPE'\r" expect { -"Debug" { - set Debug_type 1 - expect ":) " + "debug" { + set Debug_type 1 + expect ":) " } -"RelWithDebInfo" + "relwithdebinfo" } send -- "q\r" From ccf87a6afd1fa93ea6ad8fe454ee27edd75c73ef Mon Sep 17 00:00:00 2001 From: lzydmxy <13126752315@163.com> Date: Mon, 20 Feb 2023 16:52:55 +0800 Subject: [PATCH 116/253] add integration test for move partition to disk on cluster --- .../configs/config.d/cluster.xml | 17 ++++ .../config.d/storage_configuration.xml | 28 ++++++ .../test.py | 94 +++++++++++++++++++ 3 files changed, 139 insertions(+) create mode 100644 tests/integration/test_move_partition_to_disk_on_cluster/configs/config.d/cluster.xml create mode 100644 tests/integration/test_move_partition_to_disk_on_cluster/configs/config.d/storage_configuration.xml create mode 100644 tests/integration/test_move_partition_to_disk_on_cluster/test.py diff --git a/tests/integration/test_move_partition_to_disk_on_cluster/configs/config.d/cluster.xml b/tests/integration/test_move_partition_to_disk_on_cluster/configs/config.d/cluster.xml new file mode 100644 index 00000000000..2316050b629 --- /dev/null +++ b/tests/integration/test_move_partition_to_disk_on_cluster/configs/config.d/cluster.xml @@ -0,0 +1,17 @@ + + + + + true + + node1 + 9000 + + + node2 + 9000 + + + + + \ No newline at end of file diff --git a/tests/integration/test_move_partition_to_disk_on_cluster/configs/config.d/storage_configuration.xml b/tests/integration/test_move_partition_to_disk_on_cluster/configs/config.d/storage_configuration.xml new file mode 100644 index 00000000000..3289186c175 --- /dev/null +++ b/tests/integration/test_move_partition_to_disk_on_cluster/configs/config.d/storage_configuration.xml @@ -0,0 +1,28 @@ + + + + + + /jbod1/ + + + /external/ + + + + + + +
+ jbod1 +
+ + external + +
+
+
+ +
+ +
diff --git a/tests/integration/test_move_partition_to_disk_on_cluster/test.py b/tests/integration/test_move_partition_to_disk_on_cluster/test.py new file mode 100644 index 00000000000..fe8606bd549 --- /dev/null +++ b/tests/integration/test_move_partition_to_disk_on_cluster/test.py @@ -0,0 +1,94 @@ +import pytest +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance( + "node1", + main_configs=[ + "configs/config.d/storage_configuration.xml", + "configs/config.d/cluster.xml", + ], + with_zookeeper=True, + stay_alive=True, + tmpfs=["/jbod1:size=10M", "/external:size=10M"], + macros={"shard": 0, "replica": 1}, +) + +node2 = cluster.add_instance( + "node2", + main_configs=[ + "configs/config.d/storage_configuration.xml", + "configs/config.d/cluster.xml", + ], + with_zookeeper=True, + stay_alive=True, + tmpfs=["/jbod1:size=10M", "/external:size=10M"], + macros={"shard": 0, "replica": 2}, +) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_move_partition_to_disk_on_cluster(start_cluster): + for node in [node1, node2]: + node.query( + sql="CREATE TABLE test_local_table" + "(x UInt64) " + "ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_local_table', '{replica}') " + "ORDER BY tuple()" + "SETTINGS storage_policy = 'jbod_with_external';", + ) + + node1.query("INSERT INTO test_local_table VALUES (0)") + node1.query("SYSTEM SYNC REPLICA test_local_table", timeout=30) + + try: + node1.query( + sql="ALTER TABLE test_local_table ON CLUSTER 'test_cluster' MOVE PARTITION tuple() TO DISK 'jbod1';", + ) + except QueryRuntimeException: + pass + + for node in [node1, node2]: + assert ( + node.query( + "SELECT partition_id, disk_name FROM system.parts WHERE table = 'test_local_table' FORMAT Values" + ) + == "('all','jbod1')" + ) + + node1.query( + sql="ALTER TABLE test_local_table ON CLUSTER 'test_cluster' MOVE PARTITION tuple() TO DISK 'external';", + ) + + for node in [node1, node2]: + assert ( + node.query( + "SELECT partition_id, disk_name FROM system.parts WHERE table = 'test_local_table' FORMAT Values" + ) + == "('all','external')" + ) + + node1.query( + sql="ALTER TABLE test_local_table ON CLUSTER 'test_cluster' MOVE PARTITION tuple() TO VOLUME 'main';", + ) + + for node in [node1, node2]: + assert ( + node.query( + "SELECT partition_id, disk_name FROM system.parts WHERE table = 'test_local_table' FORMAT Values" + ) + == "('all','jbod1')" + ) + + From d39cffb3f2832188fcba1ea5f9b6d414152ea145 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 17 Feb 2023 20:16:42 +0100 Subject: [PATCH 117/253] Add rollback commands for the case of broad error --- tests/ci/release.py | 21 +++++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/tests/ci/release.py b/tests/ci/release.py index 52723e46693..d5537f106c5 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -277,8 +277,12 @@ class Release: dry_run=self.dry_run, ) + @property + def has_rollback(self) -> bool: + return bool(self._rollback_stack) + def log_rollback(self): - if self._rollback_stack: + if self.has_rollback: rollback = self._rollback_stack.copy() rollback.reverse() logging.info( @@ -635,7 +639,20 @@ def main(): repo, args.commit, args.release_type, args.dry_run, args.with_stderr ) - release.do(args.check_dirty, args.check_branch) + try: + release.do(args.check_dirty, args.check_branch) + except: + if release.has_rollback: + logging.error( + "!!The release process finished with error, read the output carefully!!" + ) + logging.error( + "Probably, rollback finished with error. " + "If you don't see any of the following commands in the output, " + "execute them manually:" + ) + release.log_rollback() + raise if __name__ == "__main__": From 9c7fc297787e01ca53d3feb8fbece60ddbf31f87 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 17 Feb 2023 20:39:38 +0100 Subject: [PATCH 118/253] Add dedicated key for checking launching release.py from master --- tests/ci/release.py | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/tests/ci/release.py b/tests/ci/release.py index d5537f106c5..a4fe4046572 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -173,7 +173,9 @@ class Release: self.check_commit_release_ready() - def do(self, check_dirty: bool, check_branch: bool) -> None: + def do( + self, check_dirty: bool, check_run_from_master: bool, check_branch: bool + ) -> None: self.check_prerequisites() if check_dirty: @@ -183,8 +185,9 @@ class Release: except subprocess.CalledProcessError: logging.fatal("Repo contains uncommitted changes") raise - if self._git.branch != "master": - raise Exception("the script must be launched only from master") + + if check_run_from_master and self._git.branch != "master": + raise Exception("the script must be launched only from master") self.set_release_info() @@ -606,6 +609,14 @@ def parse_args() -> argparse.Namespace: default=argparse.SUPPRESS, help="(dangerous) if set, skip check repository for uncommited changes", ) + parser.add_argument("--check-run-from-master", default=True, help=argparse.SUPPRESS) + parser.add_argument( + "--no-run-from-master", + dest="check_run_from_master", + action="store_false", + default=argparse.SUPPRESS, + help="(for development) if set, the script could run from non-master branch", + ) parser.add_argument("--check-branch", default=True, help=argparse.SUPPRESS) parser.add_argument( "--no-check-branch", @@ -640,7 +651,7 @@ def main(): ) try: - release.do(args.check_dirty, args.check_branch) + release.do(args.check_dirty, args.check_run_from_master, args.check_branch) except: if release.has_rollback: logging.error( From dbb4bdee1d0f1fdb5f8fbb6c7119cda6036ac81b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 20 Feb 2023 11:40:04 +0100 Subject: [PATCH 119/253] Allow parallel execution of 02003_memory_limit_in_client Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/02003_memory_limit_in_client.expect | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.expect b/tests/queries/0_stateless/02003_memory_limit_in_client.expect index e03fb9308f4..2b59b54b4dc 100755 --- a/tests/queries/0_stateless/02003_memory_limit_in_client.expect +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.expect @@ -1,5 +1,5 @@ #!/usr/bin/expect -f -# Tags: no-parallel, no-fasttest +# Tags: no-fasttest # This is a test for system.warnings. Testing in interactive mode is necessary, # as we want to see certain warnings from client From 9bdb3220b920c50c12a76050840ff6a2240c88c5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 20 Feb 2023 12:28:09 +0100 Subject: [PATCH 120/253] tests: add a note about alacritty for 02132_client_history_navigation Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/02132_client_history_navigation.expect | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02132_client_history_navigation.expect b/tests/queries/0_stateless/02132_client_history_navigation.expect index 82e994f4fc1..ec7b69ad90e 100755 --- a/tests/queries/0_stateless/02132_client_history_navigation.expect +++ b/tests/queries/0_stateless/02132_client_history_navigation.expect @@ -26,6 +26,7 @@ send -- "SELECT 1\r" expect "1" expect ":) " send -- "SELECT 2" +# NOTE: it does not work for alacritty with TERM=xterm send -- "\033\[A" expect "SELECT 1" send -- "\033\[B" From 34cda2bcb255c99c69ce4a5fbeeead3015f55e98 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 20 Feb 2023 12:54:13 +0100 Subject: [PATCH 121/253] Update 00170_s3_cache.sql --- tests/queries/1_stateful/00170_s3_cache.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/1_stateful/00170_s3_cache.sql b/tests/queries/1_stateful/00170_s3_cache.sql index 81592255428..43e85af0bc3 100644 --- a/tests/queries/1_stateful/00170_s3_cache.sql +++ b/tests/queries/1_stateful/00170_s3_cache.sql @@ -2,6 +2,7 @@ -- { echo } +SET allow_prefetched_read_pool_for_remote_filesystem=0; SET enable_filesystem_cache_on_write_operations=0; SET max_memory_usage='20G'; SYSTEM DROP FILESYSTEM CACHE; From 9fb7d4fd059994664744e11610979edcaf6b3c1f Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 20 Feb 2023 12:55:24 +0100 Subject: [PATCH 122/253] Update .reference --- tests/queries/1_stateful/00170_s3_cache.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/1_stateful/00170_s3_cache.reference b/tests/queries/1_stateful/00170_s3_cache.reference index 04d610bc8d2..293fbd7f8cb 100644 --- a/tests/queries/1_stateful/00170_s3_cache.reference +++ b/tests/queries/1_stateful/00170_s3_cache.reference @@ -1,5 +1,6 @@ -- { echo } +SET allow_prefetched_read_pool_for_remote_filesystem=0; SET enable_filesystem_cache_on_write_operations=0; SET max_memory_usage='20G'; SYSTEM DROP FILESYSTEM CACHE; From 78c35ffc45172c8b9976d4dbc164f5c637e96919 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 20 Feb 2023 12:28:44 +0000 Subject: [PATCH 123/253] finally fix attachProfileCountersScope --- src/Interpreters/ThreadStatusExt.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 8b84d4cae9b..84400fc3711 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -169,8 +169,10 @@ ProfileEvents::Counters * ThreadStatus::attachProfileCountersScope(ProfileEvents /// Allow to attach the same scope multiple times return prev_counters; - if (!performance_counters_scope->getParent()) + /// Avoid cycles when exiting local scope and attaching back to current thread counters + if (performance_counters_scope != &performance_counters) performance_counters_scope->setParent(&performance_counters); + current_performance_counters = performance_counters_scope; return prev_counters; From a6d420a1c64e99ddc0977af69e12ae6f7dbc8959 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 20 Feb 2023 13:55:42 +0100 Subject: [PATCH 124/253] One cannot use /dev/null as a history file since replxx run chmod Signed-off-by: Azat Khuzhin --- .../0_stateless/01179_insert_values_semicolon.expect | 3 ++- .../queries/0_stateless/01180_client_syntax_errors.expect | 3 ++- .../01293_client_interactive_vertical_singleline.expect | 3 ++- .../01370_client_autocomplete_word_break_characters.expect | 3 ++- .../queries/0_stateless/01520_client_print_query_id.expect | 3 ++- .../0_stateless/01565_reconnect_after_client_error.expect | 3 ++- ...5_client_highlight_multi_line_comment_regression.expect | 3 ++- tests/queries/0_stateless/01945_show_debug_warning.expect | 7 ++++--- .../0_stateless/02003_memory_limit_in_client.expect | 7 ++++--- tests/queries/0_stateless/02047_client_exception.expect | 3 ++- .../0_stateless/02105_backslash_letter_commands.expect | 3 ++- tests/queries/0_stateless/02116_interactive_hello.expect | 3 ++- .../0_stateless/02132_client_history_navigation.expect | 3 ++- .../02160_client_autocomplete_parse_query.expect | 3 ++- .../queries/0_stateless/02417_repeat_input_commands.expect | 3 ++- 15 files changed, 34 insertions(+), 19 deletions(-) diff --git a/tests/queries/0_stateless/01179_insert_values_semicolon.expect b/tests/queries/0_stateless/01179_insert_values_semicolon.expect index d66a84769f2..35713a90297 100755 --- a/tests/queries/0_stateless/01179_insert_values_semicolon.expect +++ b/tests/queries/0_stateless/01179_insert_values_semicolon.expect @@ -4,6 +4,7 @@ set basedir [file dirname $argv0] set basename [file tail $argv0] exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 +set history_file $env(CLICKHOUSE_TMP)/$basename.history log_user 0 set timeout 60 @@ -15,7 +16,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=/dev/null" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file" expect ":) " send -- "DROP TABLE IF EXISTS test_01179\r" diff --git a/tests/queries/0_stateless/01180_client_syntax_errors.expect b/tests/queries/0_stateless/01180_client_syntax_errors.expect index a031333313a..c1fd0f93510 100755 --- a/tests/queries/0_stateless/01180_client_syntax_errors.expect +++ b/tests/queries/0_stateless/01180_client_syntax_errors.expect @@ -3,6 +3,7 @@ set basedir [file dirname $argv0] set basename [file tail $argv0] exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 +set history_file $env(CLICKHOUSE_TMP)/$basename.history log_user 0 set timeout 60 @@ -14,7 +15,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=/dev/null" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file" expect ":) " # Make a query with syntax error diff --git a/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect b/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect index e64d71c6319..6b11b1eee15 100755 --- a/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect +++ b/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect @@ -3,6 +3,7 @@ set basedir [file dirname $argv0] set basename [file tail $argv0] exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 +set history_file $env(CLICKHOUSE_TMP)/$basename.history log_user 0 set timeout 60 @@ -14,7 +15,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=/dev/null" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file" expect ":) " send -- "SELECT 1\r" diff --git a/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect b/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect index bbd0fdd223b..8547be839d4 100755 --- a/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect +++ b/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect @@ -3,6 +3,7 @@ set basedir [file dirname $argv0] set basename [file tail $argv0] exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 +set history_file $env(CLICKHOUSE_TMP)/$basename.history log_user 0 set timeout 60 @@ -14,7 +15,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --history_file=/dev/null" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --history_file=$history_file" expect ":) " # Make a query diff --git a/tests/queries/0_stateless/01520_client_print_query_id.expect b/tests/queries/0_stateless/01520_client_print_query_id.expect index b16cd6d499b..cbeacc6a4ec 100755 --- a/tests/queries/0_stateless/01520_client_print_query_id.expect +++ b/tests/queries/0_stateless/01520_client_print_query_id.expect @@ -3,6 +3,7 @@ set basedir [file dirname $argv0] set basename [file tail $argv0] exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 +set history_file $env(CLICKHOUSE_TMP)/$basename.history log_user 0 set timeout 60 @@ -14,7 +15,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=/dev/null" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file" expect ":) " # Make a query diff --git a/tests/queries/0_stateless/01565_reconnect_after_client_error.expect b/tests/queries/0_stateless/01565_reconnect_after_client_error.expect index 076b91390bd..255248ba61a 100755 --- a/tests/queries/0_stateless/01565_reconnect_after_client_error.expect +++ b/tests/queries/0_stateless/01565_reconnect_after_client_error.expect @@ -7,6 +7,7 @@ set basedir [file dirname $argv0] set basename [file tail $argv0] exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 +set history_file $env(CLICKHOUSE_TMP)/$basename.history log_user 0 set timeout 60 @@ -19,7 +20,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion -mn --history_file=/dev/null" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion -mn --history_file=$history_file" expect "\n:) " send -- "DROP TABLE IF EXISTS t01565;\n" diff --git a/tests/queries/0_stateless/01755_client_highlight_multi_line_comment_regression.expect b/tests/queries/0_stateless/01755_client_highlight_multi_line_comment_regression.expect index 293fd2e13f4..223690f1f8b 100755 --- a/tests/queries/0_stateless/01755_client_highlight_multi_line_comment_regression.expect +++ b/tests/queries/0_stateless/01755_client_highlight_multi_line_comment_regression.expect @@ -3,6 +3,7 @@ set basedir [file dirname $argv0] set basename [file tail $argv0] exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 +set history_file $env(CLICKHOUSE_TMP)/$basename.history log_user 0 set timeout 60 @@ -14,7 +15,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=/dev/null" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file" expect ":) " # regression for heap-buffer-overflow issue (under ASAN) diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index 6714901ab20..c93635b3b27 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -6,6 +6,7 @@ set basedir [file dirname $argv0] set basename [file tail $argv0] exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 +set history_file $env(CLICKHOUSE_TMP)/$basename.history log_user 0 set timeout 60 @@ -20,7 +21,7 @@ expect_after { set Debug_type 0 -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=/dev/null" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file" expect ":) " # Check debug type @@ -38,7 +39,7 @@ expect eof if { $Debug_type > 0} { -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=/dev/null" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file" expect "Warnings:" expect " * Server was built in debug mode. It will work slowly." expect ":) " @@ -52,7 +53,7 @@ send -- "q\r" expect eof } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=/dev/null" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=$history_file" expect "Warnings:" expect " * Some obsolete setting is changed." expect ":) " diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.expect b/tests/queries/0_stateless/02003_memory_limit_in_client.expect index 2b59b54b4dc..4f28fafc1e6 100755 --- a/tests/queries/0_stateless/02003_memory_limit_in_client.expect +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.expect @@ -7,6 +7,7 @@ set basedir [file dirname $argv0] set basename [file tail $argv0] exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 +set history_file $env(CLICKHOUSE_TMP)/$basename.history log_user 0 set timeout 60 @@ -22,7 +23,7 @@ expect_after { # # Check that the query will fail in clickhouse-client # -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1 --history_file=/dev/null" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1 --history_file=$history_file" expect ":) " send -- "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000)\r" @@ -37,7 +38,7 @@ expect eof # # Check that the query will fail in clickhouse-client # -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1 --history_file=/dev/null" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1 --history_file=$history_file" expect ":) " send -- "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000)\r" @@ -52,7 +53,7 @@ expect eof # # Check that the query will not fail (due to max_untracked_memory) # -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1 --history_file=/dev/null" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1 --history_file=$history_file" expect ":) " send -- "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000\r" diff --git a/tests/queries/0_stateless/02047_client_exception.expect b/tests/queries/0_stateless/02047_client_exception.expect index 9ae8495cfcb..69f468907a3 100755 --- a/tests/queries/0_stateless/02047_client_exception.expect +++ b/tests/queries/0_stateless/02047_client_exception.expect @@ -3,6 +3,7 @@ set basedir [file dirname $argv0] set basename [file tail $argv0] exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 +set history_file $env(CLICKHOUSE_TMP)/$basename.history log_user 0 set timeout 20 @@ -15,7 +16,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=/dev/null" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file" expect ":) " send -- "DROP TABLE IF EXISTS test_02047\r" diff --git a/tests/queries/0_stateless/02105_backslash_letter_commands.expect b/tests/queries/0_stateless/02105_backslash_letter_commands.expect index a7340020611..8f8ec1f5abd 100755 --- a/tests/queries/0_stateless/02105_backslash_letter_commands.expect +++ b/tests/queries/0_stateless/02105_backslash_letter_commands.expect @@ -3,6 +3,7 @@ set basedir [file dirname $argv0] set basename [file tail $argv0] exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 +set history_file $env(CLICKHOUSE_TMP)/$basename.history log_user 0 set timeout 60 @@ -14,7 +15,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=/dev/null" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file" expect ":) " # Send a command diff --git a/tests/queries/0_stateless/02116_interactive_hello.expect b/tests/queries/0_stateless/02116_interactive_hello.expect index 443f835b76d..7e895196304 100755 --- a/tests/queries/0_stateless/02116_interactive_hello.expect +++ b/tests/queries/0_stateless/02116_interactive_hello.expect @@ -4,6 +4,7 @@ set basedir [file dirname $argv0] set basename [file tail $argv0] exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 +set history_file $env(CLICKHOUSE_TMP)/$basename.history log_user 0 set timeout 60 @@ -16,7 +17,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=/dev/null" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file" expect -re "ClickHouse client version \[\\d\]{2}.\[\\d\]{1,2}.\[\\d\]{1,2}.\[\\d\]{1,2}.\r" expect -re "Connecting to database .* at localhost:9000 as user default.\r" diff --git a/tests/queries/0_stateless/02132_client_history_navigation.expect b/tests/queries/0_stateless/02132_client_history_navigation.expect index ec7b69ad90e..3316f26d552 100755 --- a/tests/queries/0_stateless/02132_client_history_navigation.expect +++ b/tests/queries/0_stateless/02132_client_history_navigation.expect @@ -3,6 +3,7 @@ set basedir [file dirname $argv0] set basename [file tail $argv0] exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 +set history_file $env(CLICKHOUSE_TMP)/$basename.history log_user 0 set timeout 60 @@ -18,7 +19,7 @@ expect_after { # useful debugging configuration # exp_internal 1 -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --highlight 0 --history_file=/dev/null" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --highlight 0 --history_file=$history_file" expect ":) " # Make a query diff --git a/tests/queries/0_stateless/02160_client_autocomplete_parse_query.expect b/tests/queries/0_stateless/02160_client_autocomplete_parse_query.expect index 12bfcc7c041..41d32891e98 100755 --- a/tests/queries/0_stateless/02160_client_autocomplete_parse_query.expect +++ b/tests/queries/0_stateless/02160_client_autocomplete_parse_query.expect @@ -3,6 +3,7 @@ set basedir [file dirname $argv0] set basename [file tail $argv0] exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 +set history_file $env(CLICKHOUSE_TMP)/$basename.history log_user 0 set timeout 60 @@ -15,7 +16,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --history_file=/dev/null" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --history_file=$history_file" expect ":) " # Make a query diff --git a/tests/queries/0_stateless/02417_repeat_input_commands.expect b/tests/queries/0_stateless/02417_repeat_input_commands.expect index 85be0a02881..3658d5d8494 100755 --- a/tests/queries/0_stateless/02417_repeat_input_commands.expect +++ b/tests/queries/0_stateless/02417_repeat_input_commands.expect @@ -3,6 +3,7 @@ set basedir [file dirname $argv0] set basename [file tail $argv0] exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 +set history_file $env(CLICKHOUSE_TMP)/$basename.history log_user 0 set timeout 60 @@ -15,7 +16,7 @@ expect_after { timeout { exit 1 } } -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=/dev/null" +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --history_file=$history_file" expect ":) " # ----------------------------------------- From 361678ad7327d1c41b78c6eb5b744c178410812a Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 20 Feb 2023 13:55:27 +0100 Subject: [PATCH 125/253] rabbitmq-test-fix --- src/Storages/RabbitMQ/RabbitMQHandler.cpp | 4 ++-- src/Storages/RabbitMQ/RabbitMQHandler.h | 2 +- src/Storages/RabbitMQ/RabbitMQProducer.cpp | 15 ++++++++++++++- tests/integration/test_storage_rabbitmq/test.py | 14 +++++++------- 4 files changed, 24 insertions(+), 11 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.cpp b/src/Storages/RabbitMQ/RabbitMQHandler.cpp index 934753257b4..745af0d20e3 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.cpp +++ b/src/Storages/RabbitMQ/RabbitMQHandler.cpp @@ -56,10 +56,10 @@ int RabbitMQHandler::iterateLoop() /// Do not need synchronization as in iterateLoop(), because this method is used only for /// initial RabbitMQ setup - at this point there is no background loop thread. -void RabbitMQHandler::startBlockingLoop() +int RabbitMQHandler::startBlockingLoop() { LOG_DEBUG(log, "Started blocking loop."); - uv_run(loop, UV_RUN_DEFAULT); + return uv_run(loop, UV_RUN_DEFAULT); } void RabbitMQHandler::stopLoop() diff --git a/src/Storages/RabbitMQ/RabbitMQHandler.h b/src/Storages/RabbitMQ/RabbitMQHandler.h index 948d56416fd..4223732a4a0 100644 --- a/src/Storages/RabbitMQ/RabbitMQHandler.h +++ b/src/Storages/RabbitMQ/RabbitMQHandler.h @@ -38,7 +38,7 @@ public: /// Loop to wait for small tasks in a blocking mode. /// No synchronization is done with the main loop thread. - void startBlockingLoop(); + int startBlockingLoop(); void stopLoop(); diff --git a/src/Storages/RabbitMQ/RabbitMQProducer.cpp b/src/Storages/RabbitMQ/RabbitMQProducer.cpp index 5d639b77f53..246569060d0 100644 --- a/src/Storages/RabbitMQ/RabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/RabbitMQProducer.cpp @@ -262,7 +262,20 @@ void RabbitMQProducer::startProducingTaskLoop() LOG_TEST(log, "Waiting for pending callbacks to finish (count: {}, try: {})", res, try_num); } - LOG_DEBUG(log, "Producer on channel {} completed", channel_id); + producer_channel->close() + .onSuccess([&]() + { + LOG_TRACE(log, "Successfully closed producer channel"); + connection.getHandler().stopLoop(); + }) + .onError([&](const char * message) + { + LOG_ERROR(log, "Failed to close producer channel: {}", message); + connection.getHandler().stopLoop(); + }); + + int active = connection.getHandler().startBlockingLoop(); + LOG_DEBUG(log, "Producer on channel completed (not finished events: {})", active); } diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 030d9507d4f..5ca6f2acedf 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -1086,6 +1086,9 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): time.sleep(random.uniform(0, 1)) thread.start() + for thread in threads: + thread.join() + while True: result = instance.query("SELECT count() FROM test.view_overload") expected = messages_num * threads_num @@ -1096,16 +1099,13 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): instance.query( """ - DROP TABLE test.consumer_overload; - DROP TABLE test.view_overload; - DROP TABLE test.rabbitmq_consume; - DROP TABLE test.rabbitmq_overload; + DROP TABLE test.consumer_overload NO DELAY; + DROP TABLE test.view_overload NO DELAY; + DROP TABLE test.rabbitmq_consume NO DELAY; + DROP TABLE test.rabbitmq_overload NO DELAY; """ ) - for thread in threads: - thread.join() - assert ( int(result) == messages_num * threads_num ), "ClickHouse lost some messages: {}".format(result) From 514e3cbf90487a304f718290aea4d2ca45e3b99c Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 20 Feb 2023 14:29:04 +0100 Subject: [PATCH 126/253] Update test.py --- .../test_storage_meilisearch/test.py | 24 ++++++++++++++++--- 1 file changed, 21 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_meilisearch/test.py b/tests/integration/test_storage_meilisearch/test.py index 32da7f20155..66c32459251 100644 --- a/tests/integration/test_storage_meilisearch/test.py +++ b/tests/integration/test_storage_meilisearch/test.py @@ -58,6 +58,7 @@ def test_simple_select(started_cluster): push_data(client, table, data) node = started_cluster.instances["meili"] + node.query("DROP TABLE IF EXISTS simple_meili_table") node.query( "CREATE TABLE simple_meili_table(id UInt64, data String) ENGINE = MeiliSearch('http://meili1:7700', 'new_table', '')" ) @@ -83,6 +84,7 @@ def test_insert(started_cluster): big_table = client.index("big_table") node = started_cluster.instances["meili"] + node.query("DROP TABLE IF EXISTS new_table") node.query( "CREATE TABLE new_table(id UInt64, data String) ENGINE = MeiliSearch('http://meili1:7700', 'new_table', '')" ) @@ -90,9 +92,10 @@ def test_insert(started_cluster): node.query( "INSERT INTO new_table (id, data) VALUES (1, '1') (2, '2') (3, '3') (4, '4') (5, '5') (6, '6') (7, '7')" ) - sleep(1) + sleep(5) assert len(new_table.get_documents()) == 7 + node.query("DROP TABLE IF EXISTS big_table") node.query( "CREATE TABLE big_table(id UInt64, data String) ENGINE = MeiliSearch('http://meili1:7700', 'big_table', '')" ) @@ -124,6 +127,7 @@ def test_meilimatch(started_cluster): push_movies(client) node = started_cluster.instances["meili"] + node.query("DROP TABLE IF EXISTS movies_table") node.query( "CREATE TABLE movies_table(id String, title String, release_date Int64) ENGINE = MeiliSearch('http://meili1:7700', 'movies', '')" ) @@ -208,6 +212,7 @@ def test_incorrect_data_type(started_cluster): push_data(client, table, data) node = started_cluster.instances["meili"] + node.query("DROP TABLE IF EXISTS strange_meili_table") node.query( "CREATE TABLE strange_meili_table(id UInt64, data String, bbbb String) ENGINE = MeiliSearch('http://meili1:7700', 'new_table', '')" ) @@ -230,10 +235,12 @@ def test_simple_select_secure(started_cluster): push_data(client, table, data) node = started_cluster.instances["meili"] + node.query("DROP TABLE IF EXISTS simple_meili_table") node.query( "CREATE TABLE simple_meili_table(id UInt64, data String) ENGINE = MeiliSearch('http://meili_secure:7700', 'new_table', 'password')" ) + node.query("DROP TABLE IF EXISTS wrong_meili_table") node.query( "CREATE TABLE wrong_meili_table(id UInt64, data String) ENGINE = MeiliSearch('http://meili_secure:7700', 'new_table', 'wrong_password')" ) @@ -272,6 +279,7 @@ def test_meilimatch_secure(started_cluster): push_movies(client) node = started_cluster.instances["meili"] + node.query("DROP TABLE IF EXISTS movies_table") node.query( "CREATE TABLE movies_table(id String, title String, release_date Int64) ENGINE = MeiliSearch('http://meili_secure:7700', 'movies', 'password')" ) @@ -356,6 +364,7 @@ def test_incorrect_data_type_secure(started_cluster): push_data(client, table, data) node = started_cluster.instances["meili"] + node.query("DROP TABLE IF EXISTS strange_meili_table") node.query( "CREATE TABLE strange_meili_table(id UInt64, data String, bbbb String) ENGINE = MeiliSearch('http://meili_secure:7700', 'new_table', 'password')" ) @@ -374,6 +383,7 @@ def test_insert_secure(started_cluster): big_table = client.index("big_table") node = started_cluster.instances["meili"] + node.query("DROP TABLE IF EXISTS new_table") node.query( "CREATE TABLE new_table(id UInt64, data String) ENGINE = MeiliSearch('http://meili_secure:7700', 'new_table', 'password')" ) @@ -381,9 +391,10 @@ def test_insert_secure(started_cluster): node.query( "INSERT INTO new_table (id, data) VALUES (1, '1') (2, '2') (3, '3') (4, '4') (5, '5') (6, '6') (7, '7')" ) - sleep(1) + sleep(5) assert len(new_table.get_documents()) == 7 + node.query("DROP TABLE IF EXISTS big_table") node.query( "CREATE TABLE big_table(id UInt64, data String) ENGINE = MeiliSearch('http://meili_secure:7700', 'big_table', 'password')" ) @@ -417,9 +428,11 @@ def test_security_levels(started_cluster): values += "(" + str(i) + ", " + "'" + str(i) + "'" + ") " node = started_cluster.instances["meili"] + node.query("DROP TABLE IF EXISTS read_table") node.query( f"CREATE TABLE read_table(id UInt64, data String) ENGINE = MeiliSearch('http://meili_secure:7700', 'new_table', '{search_key}')" ) + node.query("DROP TABLE IF EXISTS write_table") node.query( f"CREATE TABLE write_table(id UInt64, data String) ENGINE = MeiliSearch('http://meili_secure:7700', 'new_table', '{admin_key}')" ) @@ -430,7 +443,7 @@ def test_security_levels(started_cluster): assert "MEILISEARCH_EXCEPTION" in error node.query("INSERT INTO write_table (id, data) VALUES " + values) - sleep(1) + sleep(5) assert len(new_table.get_documents({"limit": 40010})) == 100 ans1 = ( @@ -493,6 +506,7 @@ def test_types(started_cluster): push_data(client, table, data) node = started_cluster.instances["meili"] + node.query("DROP TABLE IF EXISTS types_table") node.query( "CREATE TABLE types_table(\ id UInt64,\ @@ -556,6 +570,7 @@ def test_named_collection(started_cluster): push_data(client, table, data) node = started_cluster.instances["meili"] + node.query("DROP TABLE IF EXISTS simple_meili_table") node.query( "CREATE TABLE simple_meili_table(id UInt64, data String) ENGINE = MeiliSearch( named_collection_for_meili )" ) @@ -589,14 +604,17 @@ def test_named_collection_secure(started_cluster): push_data(client_free, table_free, data) node = started_cluster.instances["meili"] + node.query("DROP TABLE IF EXISTS simple_meili_table") node.query( "CREATE TABLE simple_meili_table(id UInt64, data String) ENGINE = MeiliSearch( named_collection_for_meili_secure )" ) + node.query("DROP TABLE IF EXISTS wrong_meili_table") node.query( "CREATE TABLE wrong_meili_table(id UInt64, data String) ENGINE = MeiliSearch( named_collection_for_meili_secure_no_password )" ) + node.query("DROP TABLE IF EXISTS combine_meili_table") node.query( 'CREATE TABLE combine_meili_table(id UInt64, data String) ENGINE = MeiliSearch( named_collection_for_meili_secure_no_password, key="password" )' ) From 1539f812b30657b4f96d63c562d1584ca8bfa219 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 20 Feb 2023 13:48:52 +0000 Subject: [PATCH 127/253] Fix processing Const(LowCardinality) in arrayMap --- src/Functions/array/FunctionArrayMapped.h | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/Functions/array/FunctionArrayMapped.h b/src/Functions/array/FunctionArrayMapped.h index 89599edd9d1..8d427bcf691 100644 --- a/src/Functions/array/FunctionArrayMapped.h +++ b/src/Functions/array/FunctionArrayMapped.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -341,9 +342,15 @@ public: replicated_column_function->appendArguments(arrays); auto lambda_result = replicated_column_function->reduce(); + + /// Convert LowCardinality(T) -> T and Const(LowCardinality(T)) -> Const(T), + /// because we removed LowCardinality from return type of lambda expression. if (lambda_result.column->lowCardinality()) lambda_result.column = lambda_result.column->convertToFullColumnIfLowCardinality(); + if (const auto * const_column = checkAndGetColumnConst(lambda_result.column.get())) + lambda_result.column = const_column->removeLowCardinality(); + if (Impl::needBoolean()) { /// If result column is Nothing or Nullable(Nothing), just create const UInt8 column with 0 value. From a80ce0219996bff88c8685087be5c8d03babe687 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 20 Feb 2023 13:50:20 +0000 Subject: [PATCH 128/253] Update test --- .../0_stateless/02667_array_map_const_low_cardinality.reference | 1 + ...g_structure.sql => 02667_array_map_const_low_cardinality.sql} | 0 tests/queries/0_stateless/02667_wrong_structure.reference | 0 3 files changed, 1 insertion(+) create mode 100644 tests/queries/0_stateless/02667_array_map_const_low_cardinality.reference rename tests/queries/0_stateless/{02667_wrong_structure.sql => 02667_array_map_const_low_cardinality.sql} (100%) delete mode 100644 tests/queries/0_stateless/02667_wrong_structure.reference diff --git a/tests/queries/0_stateless/02667_array_map_const_low_cardinality.reference b/tests/queries/0_stateless/02667_array_map_const_low_cardinality.reference new file mode 100644 index 00000000000..22fd87520ef --- /dev/null +++ b/tests/queries/0_stateless/02667_array_map_const_low_cardinality.reference @@ -0,0 +1 @@ +[2] diff --git a/tests/queries/0_stateless/02667_wrong_structure.sql b/tests/queries/0_stateless/02667_array_map_const_low_cardinality.sql similarity index 100% rename from tests/queries/0_stateless/02667_wrong_structure.sql rename to tests/queries/0_stateless/02667_array_map_const_low_cardinality.sql diff --git a/tests/queries/0_stateless/02667_wrong_structure.reference b/tests/queries/0_stateless/02667_wrong_structure.reference deleted file mode 100644 index e69de29bb2d..00000000000 From 669c4e94d50a4d4886ff4ffe6b9c3ee66dc921e0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 20 Feb 2023 17:35:30 +0300 Subject: [PATCH 129/253] Update compare.sh --- docker/test/performance-comparison/compare.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 338a0c02a55..725dcbd7157 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -538,7 +538,7 @@ unset IFS numactl --show numactl --cpunodebind=all --membind=all numactl --show # Use less jobs to avoid OOM. Some queries can consume 8+ GB of memory. -jobs_count=$(($(grep -c ^processor /proc/cpuinfo) / 3)) +jobs_count=$(($(grep -c ^processor /proc/cpuinfo) / 4)) numactl --cpunodebind=all --membind=all parallel --jobs $jobs_count --joblog analyze/parallel-log.txt --null < analyze/commands.txt 2>> analyze/errors.log clickhouse-local --query " From 4715e2c172fc06728a18b32a3a6444b614ea6c77 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 20 Feb 2023 15:36:45 +0100 Subject: [PATCH 130/253] update llvm-project to fix gwp-asan --- contrib/llvm-project | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/llvm-project b/contrib/llvm-project index e61a81aa6fc..a8bf69e9cd3 160000 --- a/contrib/llvm-project +++ b/contrib/llvm-project @@ -1 +1 @@ -Subproject commit e61a81aa6fc529b469e2a54b7ce788606e138b5d +Subproject commit a8bf69e9cd39a23140a2b633c172d201484172da From 32de43753375f99f883beda91efca15204afa109 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Mon, 20 Feb 2023 09:49:14 -0500 Subject: [PATCH 131/253] Update changelog README --- utils/changelog/README.md | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/utils/changelog/README.md b/utils/changelog/README.md index 739229b49c9..ccc235c4990 100644 --- a/utils/changelog/README.md +++ b/utils/changelog/README.md @@ -12,8 +12,13 @@ python3 changelog.py -h Usage example: -``` -git fetch --tags # changelog.py depends on having the tags available, this will fetch them +Note: The working directory is ClickHouse/utils/changelog + +```bash +export GITHUB_TOKEN="" + +git fetch --tags # changelog.py depends on having the tags available, this will fetch them. + # If you are working from a branch in your personal fork, then you may need `git fetch --all` python3 changelog.py --output=changelog-v22.4.1.2305-prestable.md --gh-user-or-token="$GITHUB_TOKEN" v21.6.2.7-prestable python3 changelog.py --output=changelog-v22.4.1.2305-prestable.md --gh-user-or-token="$USER" --gh-password="$PASSWORD" v21.6.2.7-prestable From 9d16205c8a532f5b2600fe9049c99372535a6dc9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 20 Feb 2023 15:29:09 +0100 Subject: [PATCH 132/253] Load named collections on first access --- programs/local/LocalServer.cpp | 5 -- programs/server/Server.cpp | 4 -- .../NamedCollections/NamedCollectionUtils.cpp | 51 ++++++++++++++++++- .../NamedCollections/NamedCollectionUtils.h | 2 + src/Storages/NamedCollectionsHelpers.cpp | 2 + .../System/StorageSystemNamedCollections.cpp | 2 + 6 files changed, 55 insertions(+), 11 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 133d629bbb1..23ba65fed44 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -37,7 +37,6 @@ #include #include #include -#include #include #include #include @@ -131,8 +130,6 @@ void LocalServer::initialize(Poco::Util::Application & self) config().getUInt("max_io_thread_pool_size", 100), config().getUInt("max_io_thread_pool_free_size", 0), config().getUInt("io_thread_pool_queue_size", 10000)); - - NamedCollectionUtils::loadFromConfig(config()); } @@ -224,8 +221,6 @@ void LocalServer::tryInitPath() global_context->setUserFilesPath(""); // user's files are everywhere - NamedCollectionUtils::loadFromSQL(global_context); - /// top_level_domains_lists const std::string & top_level_domains_path = config().getString("top_level_domains_path", path + "top_level_domains/"); if (!top_level_domains_path.empty()) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b97b48d9c68..10710d61b84 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -770,8 +770,6 @@ try config().getUInt("max_io_thread_pool_free_size", 0), config().getUInt("io_thread_pool_queue_size", 10000)); - NamedCollectionUtils::loadFromConfig(config()); - /// Initialize global local cache for remote filesystem. if (config().has("local_cache_for_remote_fs")) { @@ -1177,8 +1175,6 @@ try SensitiveDataMasker::setInstance(std::make_unique(config(), "query_masking_rules")); } - NamedCollectionUtils::loadFromSQL(global_context); - auto main_config_reloader = std::make_unique( config_path, include_from_path, diff --git a/src/Common/NamedCollections/NamedCollectionUtils.cpp b/src/Common/NamedCollections/NamedCollectionUtils.cpp index 8beaa38210e..6ec09fb8a77 100644 --- a/src/Common/NamedCollections/NamedCollectionUtils.cpp +++ b/src/Common/NamedCollections/NamedCollectionUtils.cpp @@ -32,6 +32,9 @@ namespace ErrorCodes namespace NamedCollectionUtils { +static std::atomic is_loaded_from_config = false; +static std::atomic is_loaded_from_sql = false; + class LoadFromConfig { private: @@ -329,10 +332,21 @@ std::unique_lock lockNamedCollectionsTransaction() return std::unique_lock(transaction_lock); } +void loadFromConfigUnlocked(const Poco::Util::AbstractConfiguration & config, std::unique_lock &) +{ + auto named_collections = LoadFromConfig(config).getAll(); + LOG_TRACE( + &Poco::Logger::get("NamedCollectionsUtils"), + "Loaded {} collections from config", named_collections.size()); + + NamedCollectionFactory::instance().add(std::move(named_collections)); + is_loaded_from_config = true; +} + void loadFromConfig(const Poco::Util::AbstractConfiguration & config) { auto lock = lockNamedCollectionsTransaction(); - NamedCollectionFactory::instance().add(LoadFromConfig(config).getAll()); + loadFromConfigUnlocked(config, lock); } void reloadFromConfig(const Poco::Util::AbstractConfiguration & config) @@ -342,17 +356,47 @@ void reloadFromConfig(const Poco::Util::AbstractConfiguration & config) auto & instance = NamedCollectionFactory::instance(); instance.removeById(SourceId::CONFIG); instance.add(collections); + is_loaded_from_config = true; +} + +void loadFromSQLUnlocked(ContextPtr context, std::unique_lock &) +{ + auto named_collections = LoadFromSQL(context).getAll(); + LOG_TRACE( + &Poco::Logger::get("NamedCollectionsUtils"), + "Loaded {} collections from SQL", named_collections.size()); + + NamedCollectionFactory::instance().add(std::move(named_collections)); + is_loaded_from_sql = true; } void loadFromSQL(ContextPtr context) { auto lock = lockNamedCollectionsTransaction(); - NamedCollectionFactory::instance().add(LoadFromSQL(context).getAll()); + loadFromSQLUnlocked(context, lock); +} + +void loadIfNotUnlocked(std::unique_lock & lock) +{ + auto global_context = Context::getGlobalContextInstance(); + if (!is_loaded_from_config) + loadFromConfigUnlocked(global_context->getConfigRef(), lock); + if (!is_loaded_from_sql) + loadFromSQLUnlocked(global_context, lock); +} + +void loadIfNot() +{ + if (is_loaded_from_sql && is_loaded_from_config) + return; + auto lock = lockNamedCollectionsTransaction(); + return loadIfNotUnlocked(lock); } void removeFromSQL(const std::string & collection_name, ContextPtr context) { auto lock = lockNamedCollectionsTransaction(); + loadIfNotUnlocked(lock); LoadFromSQL(context).remove(collection_name); NamedCollectionFactory::instance().remove(collection_name); } @@ -360,6 +404,7 @@ void removeFromSQL(const std::string & collection_name, ContextPtr context) void removeIfExistsFromSQL(const std::string & collection_name, ContextPtr context) { auto lock = lockNamedCollectionsTransaction(); + loadIfNotUnlocked(lock); LoadFromSQL(context).removeIfExists(collection_name); NamedCollectionFactory::instance().removeIfExists(collection_name); } @@ -367,12 +412,14 @@ void removeIfExistsFromSQL(const std::string & collection_name, ContextPtr conte void createFromSQL(const ASTCreateNamedCollectionQuery & query, ContextPtr context) { auto lock = lockNamedCollectionsTransaction(); + loadIfNotUnlocked(lock); NamedCollectionFactory::instance().add(query.collection_name, LoadFromSQL(context).create(query)); } void updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr context) { auto lock = lockNamedCollectionsTransaction(); + loadIfNotUnlocked(lock); LoadFromSQL(context).update(query); auto collection = NamedCollectionFactory::instance().getMutable(query.collection_name); diff --git a/src/Common/NamedCollections/NamedCollectionUtils.h b/src/Common/NamedCollections/NamedCollectionUtils.h index 8befc9cac3c..c929abb5d74 100644 --- a/src/Common/NamedCollections/NamedCollectionUtils.h +++ b/src/Common/NamedCollections/NamedCollectionUtils.h @@ -35,6 +35,8 @@ void createFromSQL(const ASTCreateNamedCollectionQuery & query, ContextPtr conte /// Update definition of already existing collection from AST and update result in `context->getPath() / named_collections /`. void updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr context); +void loadIfNot(); + } } diff --git a/src/Storages/NamedCollectionsHelpers.cpp b/src/Storages/NamedCollectionsHelpers.cpp index cefed555781..6c783beaecb 100644 --- a/src/Storages/NamedCollectionsHelpers.cpp +++ b/src/Storages/NamedCollectionsHelpers.cpp @@ -58,6 +58,8 @@ NamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts) if (asts.empty()) return nullptr; + NamedCollectionUtils::loadIfNot(); + auto collection = tryGetNamedCollectionFromASTs(asts); if (!collection) return nullptr; diff --git a/src/Storages/System/StorageSystemNamedCollections.cpp b/src/Storages/System/StorageSystemNamedCollections.cpp index bc1e3a45e6b..aa095f48179 100644 --- a/src/Storages/System/StorageSystemNamedCollections.cpp +++ b/src/Storages/System/StorageSystemNamedCollections.cpp @@ -31,6 +31,8 @@ void StorageSystemNamedCollections::fillData(MutableColumns & res_columns, Conte { context->checkAccess(AccessType::SHOW_NAMED_COLLECTIONS); + NamedCollectionUtils::loadIfNot(); + auto collections = NamedCollectionFactory::instance().getAll(); for (const auto & [name, collection] : collections) { From 0a3639884746a6bb1c057bb5480369d026a96854 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 20 Feb 2023 16:53:06 +0100 Subject: [PATCH 133/253] Update test --- tests/integration/test_storage_rabbitmq/test.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 5ca6f2acedf..c3e1843a417 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -1033,8 +1033,7 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): rabbitmq_exchange_type = 'direct', rabbitmq_num_consumers = 2, rabbitmq_flush_interval_ms=1000, - rabbitmq_max_block_size = 1000, - rabbitmq_num_queues = 2, + rabbitmq_max_block_size = 100, rabbitmq_routing_key_list = 'over', rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; @@ -1044,8 +1043,6 @@ def test_rabbitmq_overloaded_insert(rabbitmq_cluster): rabbitmq_exchange_name = 'over', rabbitmq_exchange_type = 'direct', rabbitmq_routing_key_list = 'over', - rabbitmq_flush_interval_ms=1000, - rabbitmq_max_block_size = 1000, rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view_overload (key UInt64, value UInt64) From c84a64d22fe287e0b6a5caa4624791fb0dd9d4b8 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 20 Feb 2023 17:15:03 +0100 Subject: [PATCH 134/253] Return chunks with 0 rows from MergeTreeSource to report progress when rows are filtered in PREWHERE --- .../MergeTree/MergeTreeBaseSelectProcessor.cpp | 8 ++++---- .../MergeTree/MergeTreeBaseSelectProcessor.h | 3 +++ src/Storages/MergeTree/MergeTreeSource.cpp | 13 +++++++------ src/Storages/MergeTree/MergeTreeSource.h | 2 +- 4 files changed, 15 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 49458be4232..531f48377a9 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -170,16 +170,16 @@ ChunkAndProgress IMergeTreeSelectAlgorithm::read() return ChunkAndProgress{ .chunk = Chunk(ordered_columns, res.row_count), .num_read_rows = res.num_read_rows, - .num_read_bytes = res.num_read_bytes}; + .num_read_bytes = res.num_read_bytes, + .is_finished = false}; } else { - num_read_rows += res.num_read_rows; - num_read_bytes += res.num_read_bytes; + return {Chunk(), res.num_read_rows, res.num_read_bytes, false}; } } - return {Chunk(), num_read_rows, num_read_bytes}; + return {Chunk(), num_read_rows, num_read_bytes, true}; } void IMergeTreeSelectAlgorithm::initializeMergeTreeReadersForCurrentTask( diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index c6680676ce9..22c15635529 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -20,6 +20,9 @@ struct ChunkAndProgress Chunk chunk; size_t num_read_rows = 0; size_t num_read_bytes = 0; + /// Explicitly indicate that we have read all data. + /// This is needed to occasionally return empty chunk to indicate the progress while the rows are filtered out in PREWHERE. + bool is_finished = false; }; struct ParallelReadingExtension diff --git a/src/Storages/MergeTree/MergeTreeSource.cpp b/src/Storages/MergeTree/MergeTreeSource.cpp index ae1679cebfd..a37d1d3ec2c 100644 --- a/src/Storages/MergeTree/MergeTreeSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSource.cpp @@ -176,15 +176,16 @@ ISource::Status MergeTreeSource::prepare() } -std::optional MergeTreeSource::reportProgress(ChunkAndProgress chunk) +Chunk MergeTreeSource::processReadResult(ChunkAndProgress chunk) { if (chunk.num_read_rows || chunk.num_read_bytes) progress(chunk.num_read_rows, chunk.num_read_bytes); - if (chunk.chunk.hasRows()) - return std::move(chunk.chunk); + finished = chunk.is_finished; - return {}; + /// We can return a chunk with no rows even if are not finished. + /// This allows to report progress when all the rows are filtered out inside MergeTreeBaseSelectProcessor by PREWHERE logic. + return std::move(chunk.chunk); } @@ -194,7 +195,7 @@ std::optional MergeTreeSource::tryGenerate() if (async_reading_state) { if (async_reading_state->getStage() == AsyncReadingState::Stage::IsFinished) - return reportProgress(async_reading_state->getResult()); + return processReadResult(async_reading_state->getResult()); chassert(async_reading_state->getStage() == AsyncReadingState::Stage::NotStarted); @@ -220,7 +221,7 @@ std::optional MergeTreeSource::tryGenerate() } #endif - return reportProgress(algorithm->read()); + return processReadResult(algorithm->read()); } #if defined(OS_LINUX) diff --git a/src/Storages/MergeTree/MergeTreeSource.h b/src/Storages/MergeTree/MergeTreeSource.h index bba0c0af80e..463faad0fab 100644 --- a/src/Storages/MergeTree/MergeTreeSource.h +++ b/src/Storages/MergeTree/MergeTreeSource.h @@ -36,7 +36,7 @@ private: std::unique_ptr async_reading_state; #endif - std::optional reportProgress(ChunkAndProgress chunk); + Chunk processReadResult(ChunkAndProgress chunk); }; } From 6ed71120fbb6d0b214e96ca75b759427774d1d1f Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 20 Feb 2023 14:46:45 -0300 Subject: [PATCH 135/253] possibly fix explain syntax test --- tests/queries/0_stateless/02420_final_setting.reference | 8 ++++---- .../0_stateless/02420_final_setting_analyzer.reference | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/02420_final_setting.reference b/tests/queries/0_stateless/02420_final_setting.reference index 5b9780e27cd..42acc78c57e 100644 --- a/tests/queries/0_stateless/02420_final_setting.reference +++ b/tests/queries/0_stateless/02420_final_setting.reference @@ -82,16 +82,16 @@ FROM ALL INNER JOIN ( SELECT - val_middle, - id + id, + val_middle FROM middle_table ) AS middle_table ON `--left_table.id` = `--middle_table.id` ) AS `--.s` ALL INNER JOIN ( SELECT - val_right, - id + id, + val_right FROM right_table FINAL ) AS right_table ON `--middle_table.id` = id diff --git a/tests/queries/0_stateless/02420_final_setting_analyzer.reference b/tests/queries/0_stateless/02420_final_setting_analyzer.reference index 2f8bc48fc65..ee7c2541bcf 100644 --- a/tests/queries/0_stateless/02420_final_setting_analyzer.reference +++ b/tests/queries/0_stateless/02420_final_setting_analyzer.reference @@ -83,16 +83,16 @@ FROM ALL INNER JOIN ( SELECT - val_middle, - id + id, + val_middle FROM middle_table ) AS middle_table ON `--left_table.id` = `--middle_table.id` ) AS `--.s` ALL INNER JOIN ( SELECT - val_right, - id + id, + val_right FROM right_table FINAL ) AS right_table ON `--middle_table.id` = id From 3c4c527bce41b7418de94f5c730478c1ed715938 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 20 Feb 2023 18:40:57 +0000 Subject: [PATCH 136/253] Fix MemoryTracker counters for async inserts --- src/Common/CurrentThread.cpp | 22 +++++++++++ src/Common/CurrentThread.h | 6 +++ src/Interpreters/AsynchronousInsertQueue.cpp | 6 ++- src/Interpreters/AsynchronousInsertQueue.h | 39 +++++++++++++++++++- 4 files changed, 70 insertions(+), 3 deletions(-) diff --git a/src/Common/CurrentThread.cpp b/src/Common/CurrentThread.cpp index e54b2c8abe4..95b316671a4 100644 --- a/src/Common/CurrentThread.cpp +++ b/src/Common/CurrentThread.cpp @@ -110,4 +110,26 @@ ThreadGroupStatusPtr CurrentThread::getGroup() return current_thread->getThreadGroup(); } +MemoryTracker * CurrentThread::getUserMemoryTracker() +{ + if (unlikely(!current_thread)) + return nullptr; + + if (auto group = current_thread->getThreadGroup()) + return group->memory_tracker.getParent(); + + return nullptr; +} + +void CurrentThread::flushUntrackedMemory() +{ + if (unlikely(!current_thread)) + return; + if (current_thread->untracked_memory == 0) + return; + + current_thread->memory_tracker.adjustWithUntrackedMemory(current_thread->untracked_memory); + current_thread->untracked_memory = 0; +} + } diff --git a/src/Common/CurrentThread.h b/src/Common/CurrentThread.h index cbe60365798..382ae5f6e77 100644 --- a/src/Common/CurrentThread.h +++ b/src/Common/CurrentThread.h @@ -40,6 +40,12 @@ public: /// Group to which belongs current thread static ThreadGroupStatusPtr getGroup(); + /// MemoryTracker for user that owns current thread if any + static MemoryTracker * getUserMemoryTracker(); + + /// Adjust counters in MemoryTracker hierarchy if untracked_memory is not 0. + static void flushUntrackedMemory(); + /// A logs queue used by TCPHandler to pass logs to a client static void attachInternalTextLogsQueue(const std::shared_ptr & logs_queue, LogsLevel client_logs_level); diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index fa3e9915e8f..bfd9988fd10 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -102,9 +103,10 @@ bool AsynchronousInsertQueue::InsertQuery::operator==(const InsertQuery & other) return query_str == other.query_str && settings == other.settings; } -AsynchronousInsertQueue::InsertData::Entry::Entry(String && bytes_, String && query_id_) +AsynchronousInsertQueue::InsertData::Entry::Entry(String && bytes_, String && query_id_, MemoryTracker * user_memory_tracker_) : bytes(std::move(bytes_)) , query_id(std::move(query_id_)) + , user_memory_tracker(user_memory_tracker_) , create_time(std::chrono::system_clock::now()) { } @@ -209,7 +211,7 @@ std::future AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_c if (auto quota = query_context->getQuota()) quota->used(QuotaType::WRITTEN_BYTES, bytes.size()); - auto entry = std::make_shared(std::move(bytes), query_context->getCurrentQueryId()); + auto entry = std::make_shared(std::move(bytes), query_context->getCurrentQueryId(), CurrentThread::getUserMemoryTracker()); InsertQuery key{query, settings}; InsertDataPtr data_to_process; diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index ee1265673a6..b8c7d9d285b 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -41,6 +42,31 @@ private: UInt128 calculateHash() const; }; + struct UserMemoryTrackerSwitcher + { + explicit UserMemoryTrackerSwitcher(MemoryTracker * new_tracker) + { + auto * thread_tracker = CurrentThread::getMemoryTracker(); + prev_untracked_memory = current_thread->untracked_memory; + prev_memory_tracker_parent = thread_tracker->getParent(); + + current_thread->untracked_memory = 0; + thread_tracker->setParent(new_tracker); + } + + ~UserMemoryTrackerSwitcher() + { + CurrentThread::flushUntrackedMemory(); + auto * thread_tracker = CurrentThread::getMemoryTracker(); + + current_thread->untracked_memory = prev_untracked_memory; + thread_tracker->setParent(prev_memory_tracker_parent); + } + + MemoryTracker * prev_memory_tracker_parent; + Int64 prev_untracked_memory; + }; + struct InsertData { struct Entry @@ -48,9 +74,10 @@ private: public: const String bytes; const String query_id; + MemoryTracker * const user_memory_tracker; const std::chrono::time_point create_time; - Entry(String && bytes_, String && query_id_); + Entry(String && bytes_, String && query_id_, MemoryTracker * user_memory_tracker_); void finish(std::exception_ptr exception_ = nullptr); std::future getFuture() { return promise.get_future(); } @@ -61,6 +88,16 @@ private: std::atomic_bool finished = false; }; + ~InsertData() + { + auto it = entries.begin(); + while (it != entries.end()) + { + UserMemoryTrackerSwitcher switcher((*it)->user_memory_tracker); + it = entries.erase(it); + } + } + using EntryPtr = std::shared_ptr; std::list entries; From 186a29a2aad4e05be56c4462e55ee3eb5a0ec360 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 20 Feb 2023 19:46:39 +0100 Subject: [PATCH 137/253] Resurrect processors_profile_log docs. --- docs/en/operations/settings/settings.md | 9 +++ .../system-tables/processors_profile_log.md | 74 +++++++++++++++++++ 2 files changed, 83 insertions(+) create mode 100644 docs/en/operations/system-tables/processors_profile_log.md diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 1d1d8b32d1d..9b16188ad6d 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -851,6 +851,15 @@ Result: └─────────────┴───────────┘ ``` +## log_processors_profiles {#settings-log_processors_profiles} + +Write time that processor spent during execution/waiting for data to `system.processors_profile_log` table. + +See also: + +- [`system.processors_profile_log`](../../operations/system-tables/processors_profile_log.md#system-processors_profile_log) +- [`EXPLAIN PIPELINE`](../../sql-reference/statements/explain.md#explain-pipeline) + ## max_insert_block_size {#settings-max_insert_block_size} The size of blocks (in a count of rows) to form for insertion into a table. diff --git a/docs/en/operations/system-tables/processors_profile_log.md b/docs/en/operations/system-tables/processors_profile_log.md new file mode 100644 index 00000000000..269385deab6 --- /dev/null +++ b/docs/en/operations/system-tables/processors_profile_log.md @@ -0,0 +1,74 @@ +# system.processors_profile_log {#system-processors_profile_log} + +This table contains profiling on processors level (that you can find in [`EXPLAIN PIPELINE`](../../sql-reference/statements/explain.md#explain-pipeline)). + +Columns: + +- `event_date` ([Date](../../sql-reference/data-types/date.md)) — The date when the event happened. +- `event_time` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — The date and time when the event happened. +- `id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — ID of processor +- `parent_ids` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — Parent processors IDs +- `query_id` ([String](../../sql-reference/data-types/string.md)) — ID of the query +- `name` ([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md)) — Name of the processor. +- `elapsed_us` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of microseconds this processor was executed. +- `input_wait_elapsed_us` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of microseconds this processor was waiting for data (from other processor). +- `output_wait_elapsed_us` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of microseconds this processor was waiting because output port was full. +- `plan_step` ([UInt64](../../sql-reference/data-types/int-uint.md)) — ID of the query plan step which created this processor. The value is zero if the processor was not added from any step. +- `plan_group` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Group of the processor if it was created by query plan step. A group is a logical partitioning of processors added from the same query plan step. Group is used only for beautifying the result of EXPLAIN PIPELINE result. +- `input_rows` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The number of rows consumed by processor. +- `input_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The number of bytes consumed by processor. +- `output_rows` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The number of rows generated by processor. +- `output_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The number of bytes generated by processor. +**Example** + +Query: + +``` sql +EXPLAIN PIPELINE +SELECT sleep(1) +┌─explain─────────────────────────┐ +│ (Expression) │ +│ ExpressionTransform │ +│ (SettingQuotaAndLimits) │ +│ (ReadFromStorage) │ +│ SourceFromSingleChunk 0 → 1 │ +└─────────────────────────────────┘ +SELECT sleep(1) +SETTINGS log_processors_profiles = 1 +Query id: feb5ed16-1c24-4227-aa54-78c02b3b27d4 +┌─sleep(1)─┐ +│ 0 │ +└──────────┘ +1 rows in set. Elapsed: 1.018 sec. +SELECT + name, + elapsed_us, + input_wait_elapsed_us, + output_wait_elapsed_us +FROM system.processors_profile_log +WHERE query_id = 'feb5ed16-1c24-4227-aa54-78c02b3b27d4' +ORDER BY name ASC +``` + +Result: + +``` text +┌─name────────────────────┬─elapsed_us─┬─input_wait_elapsed_us─┬─output_wait_elapsed_us─┐ +│ ExpressionTransform │ 1000497 │ 2823 │ 197 │ +│ LazyOutputFormat │ 36 │ 1002188 │ 0 │ +│ LimitsCheckingTransform │ 10 │ 1002994 │ 106 │ +│ NullSource │ 5 │ 1002074 │ 0 │ +│ NullSource │ 1 │ 1002084 │ 0 │ +│ SourceFromSingleChunk │ 45 │ 4736 │ 1000819 │ +└─────────────────────────┴────────────┴───────────────────────┴────────────────────────┘ +``` + +Here you can see: + +- `ExpressionTransform` was executing `sleep(1)` function, so it `work` will takes 1e6, and so `elapsed_us` > 1e6. +- `SourceFromSingleChunk` need to wait, because `ExpressionTransform` does not accept any data during execution of `sleep(1)`, so it will be in `PortFull` state for 1e6 us, and so `output_wait_elapsed_us` > 1e6. +- `LimitsCheckingTransform`/`NullSource`/`LazyOutputFormat` need to wait until `ExpressionTransform` will execute `sleep(1)` to process the result, so `input_wait_elapsed_us` > 1e6. + +**See Also** + +- [`EXPLAIN PIPELINE`](../../sql-reference/statements/explain.md#explain-pipeline) \ No newline at end of file From 7f487f03e75a769722c6c1115f8120e1176c4033 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 20 Feb 2023 19:24:15 +0000 Subject: [PATCH 138/253] Automatic style fix --- tests/ci/workflow_approve_rerun_lambda/app.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index b563a9786c4..fb14dfd2258 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -123,7 +123,7 @@ TRUSTED_CONTRIBUTORS = { "BoloniniD", # Seasoned contributor, HSE "tonickkozlov", # Cloudflare "tylerhannan", # ClickHouse Employee - "myrrc", # Mike Kot, DoubleCloud + "myrrc", # Mike Kot, DoubleCloud ] } From a568704d63a5402de9af2852a06f5419afc3da38 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 20 Feb 2023 20:43:28 +0000 Subject: [PATCH 139/253] Fix avro --- src/Processors/Formats/Impl/AvroRowInputFormat.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index bedf3cc932e..c3ea1b5e23b 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -778,15 +778,14 @@ AvroDeserializer::AvroDeserializer(const Block & header, avro::ValidSchema schem void AvroDeserializer::deserializeRow(MutableColumns & columns, avro::Decoder & decoder, RowReadExtension & ext) const { + size_t row = columns[0]->size() + 1; ext.read_columns.assign(columns.size(), false); row_action.execute(columns, decoder, ext); for (size_t i = 0; i < ext.read_columns.size(); ++i) { /// Insert default in missing columns. - if (!column_found[i]) - { + if (columns[i]->size() != row) columns[i]->insertDefault(); - } } } From b1e0e587e61dcf38c0647dfb196fcb206b380507 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Feb 2023 23:43:11 +0100 Subject: [PATCH 140/253] Inhibit index_granularity_bytes randomization in some tests --- .../00755_avg_value_size_hint_passing.sql | 2 +- .../00804_test_alter_compression_codecs.sql | 2 +- .../00804_test_custom_compression_codecs.sql | 2 +- .../queries/0_stateless/00837_minmax_index.sh | 4 ++-- ...minmax_index_replicated_zookeeper_long.sql | 4 ++-- .../queries/0_stateless/00838_unique_index.sh | 2 +- .../0_stateless/00907_set_index_max_rows.sh | 4 ++-- .../0_stateless/00908_bloom_filter_index.sh | 6 +++--- ...tom_compression_codecs_replicated_long.sql | 2 +- .../queries/0_stateless/00942_mutate_index.sh | 2 +- .../0_stateless/00943_materialize_index.sh | 2 +- .../00944_clear_index_in_partition.sh | 2 +- .../0_stateless/00945_bloom_filter_index.sql | 18 ++++++++--------- ...hecksums_in_system_parts_columns_table.sql | 3 +-- .../00964_bloom_index_string_functions.sh | 2 +- .../00965_set_index_string_functions.sh | 2 +- ...ices_mutation_replicated_zookeeper_long.sh | 4 ++-- .../0_stateless/00980_alter_settings_race.sh | 2 +- .../00980_merge_alter_settings.reference | 20 +++++++++---------- .../00980_merge_alter_settings.sql | 4 ++-- .../00990_hasToken_and_tokenbf.sql | 4 ++-- .../01030_final_mark_empty_primary_key.sql | 2 +- .../01055_minmax_index_compact_parts.sh | 4 ++-- ...tal_streaming_from_2_src_with_feedback.sql | 2 +- ..._materialize_clear_index_compact_parts.sql | 2 +- .../01201_read_single_thread_in_order.sql | 2 +- ...oom_filter_index_string_multi_granulas.sql | 2 +- ...with_constant_string_in_index_analysis.sql | 2 +- ...515_mv_and_array_join_optimisation_bag.sql | 2 +- .../01605_skip_idx_compact_parts.sql | 2 +- .../01681_bloom_filter_nullable_column.sql | 2 +- .../0_stateless/01780_column_sparse_full.sql | 2 +- .../01781_token_extractor_buffer_overflow.sql | 2 +- .../0_stateless/01926_order_by_desc_limit.sql | 2 +- .../0_stateless/02030_tuple_filter.sql | 2 +- .../02149_read_in_order_fixed_prefix.sql | 2 +- .../02155_read_in_order_max_rows_to_read.sql | 2 +- .../02267_empty_arrays_read_reverse.sql | 2 +- .../0_stateless/02346_full_text_search.sql | 13 ++++++------ tests/queries/0_stateless/02354_annoy.sh | 20 +++++++++---------- .../0_stateless/02374_in_tuple_index.sql | 2 +- .../02465_limit_trivial_max_rows_to_read.sql | 2 +- .../02467_set_with_lowcardinality_type.sql | 4 ++-- ...default_value_used_in_row_level_filter.sql | 2 +- ...ewhere_filtered_rows_div_by_zero.reference | 2 +- ...481_prewhere_filtered_rows_div_by_zero.sql | 2 +- 46 files changed, 87 insertions(+), 89 deletions(-) diff --git a/tests/queries/0_stateless/00755_avg_value_size_hint_passing.sql b/tests/queries/0_stateless/00755_avg_value_size_hint_passing.sql index 8ce6716e03c..1d033b7aa47 100644 --- a/tests/queries/0_stateless/00755_avg_value_size_hint_passing.sql +++ b/tests/queries/0_stateless/00755_avg_value_size_hint_passing.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS size_hint; -CREATE TABLE size_hint (s Array(String)) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 1000; +CREATE TABLE size_hint (s Array(String)) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 1000, index_granularity_bytes = '10Mi'; SET max_block_size = 1000; SET max_memory_usage = 1000000000; diff --git a/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql b/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql index 9e911262ed7..1c83d2ae36d 100644 --- a/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql +++ b/tests/queries/0_stateless/00804_test_alter_compression_codecs.sql @@ -65,7 +65,7 @@ CREATE TABLE large_alter_table_00804 ( somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12)), id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC), data String CODEC(ZSTD(2), LZ4HC, NONE, LZ4, LZ4) -) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2, min_bytes_for_wide_part = 0; +) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi', min_bytes_for_wide_part = 0; INSERT INTO large_alter_table_00804 SELECT toDate('2019-01-01'), number, toString(number + rand()) FROM system.numbers LIMIT 300000; diff --git a/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index c3573588ae9..593d349caf9 100644 --- a/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -114,7 +114,7 @@ CREATE TABLE compression_codec_multiple_with_key ( somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12), Delta, Delta), id UInt64 CODEC(LZ4, ZSTD, Delta, NONE, LZ4HC, Delta), data String CODEC(ZSTD(2), Delta(1), LZ4HC, NONE, LZ4, LZ4) -) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2; +) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi'; INSERT INTO compression_codec_multiple_with_key VALUES(toDate('2018-10-12'), 100000, 'hello'), (toDate('2018-10-12'), 100002, 'world'), (toDate('2018-10-12'), 1111, '!'); diff --git a/tests/queries/0_stateless/00837_minmax_index.sh b/tests/queries/0_stateless/00837_minmax_index.sh index 39e39f9d628..e4de0b9ebfc 100755 --- a/tests/queries/0_stateless/00837_minmax_index.sh +++ b/tests/queries/0_stateless/00837_minmax_index.sh @@ -23,7 +23,7 @@ CREATE TABLE minmax_idx INDEX idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE minmax GRANULARITY 3 ) ENGINE = MergeTree() ORDER BY u64 -SETTINGS index_granularity = 2;" +SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';" $CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES @@ -48,4 +48,4 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM minmax_idx WHERE i32 = 5 AND i32 + f64 $CLICKHOUSE_CLIENT --query="SELECT * FROM minmax_idx WHERE (u64 < 2 OR u64 > 10) AND e != 'b' ORDER BY dt" $CLICKHOUSE_CLIENT --query="SELECT * FROM minmax_idx WHERE (u64 < 2 OR u64 > 10) AND e != 'b' ORDER BY dt FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="DROP TABLE minmax_idx" \ No newline at end of file +$CLICKHOUSE_CLIENT --query="DROP TABLE minmax_idx" diff --git a/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper_long.sql b/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper_long.sql index a2fe1ebd63a..ba080949548 100644 --- a/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper_long.sql +++ b/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper_long.sql @@ -19,7 +19,7 @@ CREATE TABLE minmax_idx1 idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE minmax GRANULARITY 3 ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00837/minmax', 'r1') ORDER BY u64 -SETTINGS index_granularity = 2; +SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi'; CREATE TABLE minmax_idx2 ( @@ -36,7 +36,7 @@ CREATE TABLE minmax_idx2 idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE minmax GRANULARITY 3 ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00837/minmax', 'r2') ORDER BY u64 -SETTINGS index_granularity = 2; +SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi'; /* many small inserts => table will make merges */ diff --git a/tests/queries/0_stateless/00838_unique_index.sh b/tests/queries/0_stateless/00838_unique_index.sh index 36504b754a7..b267b6a8eb3 100755 --- a/tests/queries/0_stateless/00838_unique_index.sh +++ b/tests/queries/0_stateless/00838_unique_index.sh @@ -22,7 +22,7 @@ CREATE TABLE set_idx INDEX idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE set(6) GRANULARITY 3 ) ENGINE = MergeTree() ORDER BY u64 -SETTINGS index_granularity = 2;" +SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';" $CLICKHOUSE_CLIENT --query="INSERT INTO set_idx VALUES (0, 5, 4.7, 6.5, 'cba', 'b', '2014-01-04'), diff --git a/tests/queries/0_stateless/00907_set_index_max_rows.sh b/tests/queries/0_stateless/00907_set_index_max_rows.sh index f780517934d..3707aaf2ca6 100755 --- a/tests/queries/0_stateless/00907_set_index_max_rows.sh +++ b/tests/queries/0_stateless/00907_set_index_max_rows.sh @@ -14,7 +14,7 @@ CREATE TABLE set_idx INDEX idx (i32) TYPE set(2) GRANULARITY 1 ) ENGINE = MergeTree() ORDER BY u64 -SETTINGS index_granularity = 6;" +SETTINGS index_granularity = 6, index_granularity_bytes = '10Mi';" $CLICKHOUSE_CLIENT --query=" INSERT INTO set_idx @@ -24,4 +24,4 @@ SELECT number, number FROM system.numbers LIMIT 100" $CLICKHOUSE_CLIENT --query="SELECT * FROM set_idx WHERE i32 > 0 FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="DROP TABLE set_idx;" \ No newline at end of file +$CLICKHOUSE_CLIENT --query="DROP TABLE set_idx;" diff --git a/tests/queries/0_stateless/00908_bloom_filter_index.sh b/tests/queries/0_stateless/00908_bloom_filter_index.sh index f1f07ca360b..92b5634c1db 100755 --- a/tests/queries/0_stateless/00908_bloom_filter_index.sh +++ b/tests/queries/0_stateless/00908_bloom_filter_index.sh @@ -20,7 +20,7 @@ CREATE TABLE bloom_filter_idx INDEX bf (s, lower(s)) TYPE ngrambf_v1(3, 512, 2, 0) GRANULARITY 1 ) ENGINE = MergeTree() ORDER BY k -SETTINGS index_granularity = 2;" +SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';" $CLICKHOUSE_CLIENT -n --query=" CREATE TABLE bloom_filter_idx2 @@ -30,7 +30,7 @@ CREATE TABLE bloom_filter_idx2 INDEX bf (s, lower(s)) TYPE ngrambf_v1(3, 512, 2, 0) GRANULARITY 1 ) ENGINE = MergeTree() ORDER BY k -SETTINGS index_granularity = 2;" +SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';" $CLICKHOUSE_CLIENT --query="INSERT INTO bloom_filter_idx VALUES @@ -113,7 +113,7 @@ CREATE TABLE bloom_filter_idx3 INDEX bf (s, lower(s)) TYPE tokenbf_v1(512, 3, 0) GRANULARITY 1 ) ENGINE = MergeTree() ORDER BY k -SETTINGS index_granularity = 2;" +SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';" $CLICKHOUSE_CLIENT --query="INSERT INTO bloom_filter_idx3 VALUES (0, 'ClickHouse is a column-oriented database management system (DBMS) for online analytical processing of queries (OLAP).'), diff --git a/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated_long.sql b/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated_long.sql index 369d897fafd..ed4d9dd4934 100644 --- a/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated_long.sql +++ b/tests/queries/0_stateless/00910_zookeeper_custom_compression_codecs_replicated_long.sql @@ -126,7 +126,7 @@ CREATE TABLE compression_codec_multiple_with_key_replicated ( somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12), Delta, Delta), id UInt64 CODEC(LZ4, ZSTD, Delta, NONE, LZ4HC, Delta), data String CODEC(ZSTD(2), Delta(1), LZ4HC, NONE, LZ4, LZ4) -) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/compression_codec_multiple_with_key_replicated', '1') PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2; +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_00910/compression_codec_multiple_with_key_replicated', '1') PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi'; INSERT INTO compression_codec_multiple_with_key_replicated VALUES(toDate('2018-10-12'), 100000, 'hello'), (toDate('2018-10-12'), 100002, 'world'), (toDate('2018-10-12'), 1111, '!'); diff --git a/tests/queries/0_stateless/00942_mutate_index.sh b/tests/queries/0_stateless/00942_mutate_index.sh index 4eebdd1147f..6ebb30c25b9 100755 --- a/tests/queries/0_stateless/00942_mutate_index.sh +++ b/tests/queries/0_stateless/00942_mutate_index.sh @@ -15,7 +15,7 @@ CREATE TABLE minmax_idx INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1 ) ENGINE = MergeTree() ORDER BY u64 -SETTINGS index_granularity = 2;" +SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';" $CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES (0, 1, 1), diff --git a/tests/queries/0_stateless/00943_materialize_index.sh b/tests/queries/0_stateless/00943_materialize_index.sh index 43c9af84672..30ef46e5cb0 100755 --- a/tests/queries/0_stateless/00943_materialize_index.sh +++ b/tests/queries/0_stateless/00943_materialize_index.sh @@ -16,7 +16,7 @@ CREATE TABLE minmax_idx ) ENGINE = MergeTree() PARTITION BY i32 ORDER BY u64 -SETTINGS index_granularity = 2;" +SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';" $CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES diff --git a/tests/queries/0_stateless/00944_clear_index_in_partition.sh b/tests/queries/0_stateless/00944_clear_index_in_partition.sh index 1f349cf5946..8b74bd94f2c 100755 --- a/tests/queries/0_stateless/00944_clear_index_in_partition.sh +++ b/tests/queries/0_stateless/00944_clear_index_in_partition.sh @@ -17,7 +17,7 @@ CREATE TABLE minmax_idx ) ENGINE = MergeTree() PARTITION BY i32 ORDER BY u64 -SETTINGS index_granularity = 2;" +SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';" $CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES diff --git a/tests/queries/0_stateless/00945_bloom_filter_index.sql b/tests/queries/0_stateless/00945_bloom_filter_index.sql index fc18a4a4dc5..dc47e858c4d 100644 --- a/tests/queries/0_stateless/00945_bloom_filter_index.sql +++ b/tests/queries/0_stateless/00945_bloom_filter_index.sql @@ -1,7 +1,7 @@ DROP TABLE IF EXISTS single_column_bloom_filter; -CREATE TABLE single_column_bloom_filter (u64 UInt64, i32 Int32, i64 UInt64, INDEX idx (i32) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 6; +CREATE TABLE single_column_bloom_filter (u64 UInt64, i32 Int32, i64 UInt64, INDEX idx (i32) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 6, index_granularity_bytes = '10Mi'; INSERT INTO single_column_bloom_filter SELECT number AS u64, number AS i32, number AS i64 FROM system.numbers LIMIT 100; @@ -28,7 +28,7 @@ DROP TABLE IF EXISTS single_column_bloom_filter; DROP TABLE IF EXISTS bloom_filter_types_test; -CREATE TABLE bloom_filter_types_test (order_key UInt64, i8 Int8, i16 Int16, i32 Int32, i64 Int64, u8 UInt8, u16 UInt16, u32 UInt32, u64 UInt64, f32 Float32, f64 Float64, date Date, date_time DateTime('Asia/Istanbul'), str String, fixed_string FixedString(5), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6; +CREATE TABLE bloom_filter_types_test (order_key UInt64, i8 Int8, i16 Int16, i32 Int32, i64 Int64, u8 UInt8, u16 UInt16, u32 UInt32, u64 UInt64, f32 Float32, f64 Float64, date Date, date_time DateTime('Asia/Istanbul'), str String, fixed_string FixedString(5), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6, index_granularity_bytes = '10Mi'; INSERT INTO bloom_filter_types_test SELECT number AS order_key, toInt8(number) AS i8, toInt16(number) AS i16, toInt32(number) AS i32, toInt64(number) AS i64, toUInt8(number) AS u8, toUInt16(number) AS u16, toUInt32(number) AS u32, toUInt64(number) AS u64, toFloat32(number) AS f32, toFloat64(number) AS f64, toDate(number, 'Asia/Istanbul') AS date, toDateTime(number, 'Asia/Istanbul') AS date_time, toString(number) AS str, toFixedString(toString(number), 5) AS fixed_string FROM system.numbers LIMIT 100; SELECT COUNT() FROM bloom_filter_types_test WHERE i8 = 1 SETTINGS max_rows_to_read = 6; @@ -52,7 +52,7 @@ DROP TABLE IF EXISTS bloom_filter_types_test; DROP TABLE IF EXISTS bloom_filter_array_types_test; -CREATE TABLE bloom_filter_array_types_test (order_key Array(UInt64), i8 Array(Int8), i16 Array(Int16), i32 Array(Int32), i64 Array(Int64), u8 Array(UInt8), u16 Array(UInt16), u32 Array(UInt32), u64 Array(UInt64), f32 Array(Float32), f64 Array(Float64), date Array(Date), date_time Array(DateTime('Asia/Istanbul')), str Array(String), fixed_string Array(FixedString(5)), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6; +CREATE TABLE bloom_filter_array_types_test (order_key Array(UInt64), i8 Array(Int8), i16 Array(Int16), i32 Array(Int32), i64 Array(Int64), u8 Array(UInt8), u16 Array(UInt16), u32 Array(UInt32), u64 Array(UInt64), f32 Array(Float32), f64 Array(Float64), date Array(Date), date_time Array(DateTime('Asia/Istanbul')), str Array(String), fixed_string Array(FixedString(5)), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6, index_granularity_bytes = '10Mi'; INSERT INTO bloom_filter_array_types_test SELECT groupArray(number) AS order_key, groupArray(toInt8(number)) AS i8, groupArray(toInt16(number)) AS i16, groupArray(toInt32(number)) AS i32, groupArray(toInt64(number)) AS i64, groupArray(toUInt8(number)) AS u8, groupArray(toUInt16(number)) AS u16, groupArray(toUInt32(number)) AS u32, groupArray(toUInt64(number)) AS u64, groupArray(toFloat32(number)) AS f32, groupArray(toFloat64(number)) AS f64, groupArray(toDate(number, 'Asia/Istanbul')) AS date, groupArray(toDateTime(number, 'Asia/Istanbul')) AS date_time, groupArray(toString(number)) AS str, groupArray(toFixedString(toString(number), 5)) AS fixed_string FROM (SELECT number FROM system.numbers LIMIT 15); INSERT INTO bloom_filter_array_types_test SELECT groupArray(number) AS order_key, groupArray(toInt8(number)) AS i8, groupArray(toInt16(number)) AS i16, groupArray(toInt32(number)) AS i32, groupArray(toInt64(number)) AS i64, groupArray(toUInt8(number)) AS u8, groupArray(toUInt16(number)) AS u16, groupArray(toUInt32(number)) AS u32, groupArray(toUInt64(number)) AS u64, groupArray(toFloat32(number)) AS f32, groupArray(toFloat64(number)) AS f64, groupArray(toDate(number, 'Asia/Istanbul')) AS date, groupArray(toDateTime(number, 'Asia/Istanbul')) AS date_time, groupArray(toString(number)) AS str, groupArray(toFixedString(toString(number), 5)) AS fixed_string FROM (SELECT number FROM system.numbers WHERE number >= 5 LIMIT 15); INSERT INTO bloom_filter_array_types_test SELECT groupArray(number) AS order_key, groupArray(toInt8(number)) AS i8, groupArray(toInt16(number)) AS i16, groupArray(toInt32(number)) AS i32, groupArray(toInt64(number)) AS i64, groupArray(toUInt8(number)) AS u8, groupArray(toUInt16(number)) AS u16, groupArray(toUInt32(number)) AS u32, groupArray(toUInt64(number)) AS u64, groupArray(toFloat32(number)) AS f32, groupArray(toFloat64(number)) AS f64, groupArray(toDate(number, 'Asia/Istanbul')) AS date, groupArray(toDateTime(number, 'Asia/Istanbul')) AS date_time, groupArray(toString(number)) AS str, groupArray(toFixedString(toString(number), 5)) AS fixed_string FROM (SELECT number FROM system.numbers WHERE number >= 10 LIMIT 15); @@ -106,7 +106,7 @@ DROP TABLE IF EXISTS bloom_filter_array_types_test; DROP TABLE IF EXISTS bloom_filter_null_types_test; -CREATE TABLE bloom_filter_null_types_test (order_key UInt64, i8 Nullable(Int8), i16 Nullable(Int16), i32 Nullable(Int32), i64 Nullable(Int64), u8 Nullable(UInt8), u16 Nullable(UInt16), u32 Nullable(UInt32), u64 Nullable(UInt64), f32 Nullable(Float32), f64 Nullable(Float64), date Nullable(Date), date_time Nullable(DateTime('Asia/Istanbul')), str Nullable(String), fixed_string Nullable(FixedString(5)), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6; +CREATE TABLE bloom_filter_null_types_test (order_key UInt64, i8 Nullable(Int8), i16 Nullable(Int16), i32 Nullable(Int32), i64 Nullable(Int64), u8 Nullable(UInt8), u16 Nullable(UInt16), u32 Nullable(UInt32), u64 Nullable(UInt64), f32 Nullable(Float32), f64 Nullable(Float64), date Nullable(Date), date_time Nullable(DateTime('Asia/Istanbul')), str Nullable(String), fixed_string Nullable(FixedString(5)), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6, index_granularity_bytes = '10Mi'; INSERT INTO bloom_filter_null_types_test SELECT number AS order_key, toInt8(number) AS i8, toInt16(number) AS i16, toInt32(number) AS i32, toInt64(number) AS i64, toUInt8(number) AS u8, toUInt16(number) AS u16, toUInt32(number) AS u32, toUInt64(number) AS u64, toFloat32(number) AS f32, toFloat64(number) AS f64, toDate(number, 'Asia/Istanbul') AS date, toDateTime(number, 'Asia/Istanbul') AS date_time, toString(number) AS str, toFixedString(toString(number), 5) AS fixed_string FROM system.numbers LIMIT 100; INSERT INTO bloom_filter_null_types_test SELECT 0 AS order_key, NULL AS i8, NULL AS i16, NULL AS i32, NULL AS i64, NULL AS u8, NULL AS u16, NULL AS u32, NULL AS u64, NULL AS f32, NULL AS f64, NULL AS date, NULL AS date_time, NULL AS str, NULL AS fixed_string; @@ -146,7 +146,7 @@ DROP TABLE IF EXISTS bloom_filter_null_types_test; DROP TABLE IF EXISTS bloom_filter_lc_null_types_test; -CREATE TABLE bloom_filter_lc_null_types_test (order_key UInt64, str LowCardinality(Nullable(String)), fixed_string LowCardinality(Nullable(FixedString(5))), INDEX idx (str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6; +CREATE TABLE bloom_filter_lc_null_types_test (order_key UInt64, str LowCardinality(Nullable(String)), fixed_string LowCardinality(Nullable(FixedString(5))), INDEX idx (str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6, index_granularity_bytes = '10Mi'; INSERT INTO bloom_filter_lc_null_types_test SELECT number AS order_key, toString(number) AS str, toFixedString(toString(number), 5) AS fixed_string FROM system.numbers LIMIT 100; INSERT INTO bloom_filter_lc_null_types_test SELECT 0 AS order_key, NULL AS str, NULL AS fixed_string; @@ -183,7 +183,7 @@ CREATE TABLE bloom_filter_array_lc_null_types_test ( fixed_string Array(LowCardinality(Nullable(FixedString(5)))), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) -ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6, allow_nullable_key = 1; +ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6, index_granularity_bytes = '10Mi', allow_nullable_key = 1; INSERT INTO bloom_filter_array_lc_null_types_test SELECT groupArray(number) AS order_key, @@ -286,19 +286,19 @@ SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(fixed_string DROP TABLE IF EXISTS bloom_filter_array_lc_null_types_test; DROP TABLE IF EXISTS bloom_filter_array_offsets_lc_str; -CREATE TABLE bloom_filter_array_offsets_lc_str (order_key int, str Array(LowCardinality(String)), INDEX idx str TYPE bloom_filter(1.) GRANULARITY 1024) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 1024; +CREATE TABLE bloom_filter_array_offsets_lc_str (order_key int, str Array(LowCardinality(String)), INDEX idx str TYPE bloom_filter(1.) GRANULARITY 1024) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 1024, index_granularity_bytes = '10Mi'; INSERT INTO bloom_filter_array_offsets_lc_str SELECT number AS i, if(i%2, ['value'], []) FROM system.numbers LIMIT 10000; SELECT count() FROM bloom_filter_array_offsets_lc_str WHERE has(str, 'value'); DROP TABLE IF EXISTS bloom_filter_array_offsets_lc_str; DROP TABLE IF EXISTS bloom_filter_array_offsets_str; -CREATE TABLE bloom_filter_array_offsets_str (order_key int, str Array(String), INDEX idx str TYPE bloom_filter(1.) GRANULARITY 1024) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 1024; +CREATE TABLE bloom_filter_array_offsets_str (order_key int, str Array(String), INDEX idx str TYPE bloom_filter(1.) GRANULARITY 1024) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 1024, index_granularity_bytes = '10Mi'; INSERT INTO bloom_filter_array_offsets_str SELECT number AS i, if(i%2, ['value'], []) FROM system.numbers LIMIT 10000; SELECT count() FROM bloom_filter_array_offsets_str WHERE has(str, 'value'); DROP TABLE IF EXISTS bloom_filter_array_offsets_str; DROP TABLE IF EXISTS bloom_filter_array_offsets_i; -CREATE TABLE bloom_filter_array_offsets_i (order_key int, i Array(int), INDEX idx i TYPE bloom_filter(1.) GRANULARITY 1024) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 1024; +CREATE TABLE bloom_filter_array_offsets_i (order_key int, i Array(int), INDEX idx i TYPE bloom_filter(1.) GRANULARITY 1024) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 1024, index_granularity_bytes = '10Mi'; INSERT INTO bloom_filter_array_offsets_i SELECT number AS i, if(i%2, [99999], []) FROM system.numbers LIMIT 10000; SELECT count() FROM bloom_filter_array_offsets_i WHERE has(i, 99999); DROP TABLE IF EXISTS bloom_filter_array_offsets_i; diff --git a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql index 40da12baddc..43b7775e816 100644 --- a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql +++ b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS test_00961; CREATE TABLE test_00961 (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = MergeTree PARTITION BY d ORDER BY (a, b) - SETTINGS index_granularity = 111, min_bytes_for_wide_part = 0, compress_marks = 0, compress_primary_key = 0; + SETTINGS index_granularity = 111, min_bytes_for_wide_part = 0, compress_marks = 0, compress_primary_key = 0, index_granularity_bytes = '10Mi'; INSERT INTO test_00961 VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789); @@ -18,4 +18,3 @@ FROM system.parts WHERE table = 'test_00961' and database = currentDatabase(); DROP TABLE test_00961; - diff --git a/tests/queries/0_stateless/00964_bloom_index_string_functions.sh b/tests/queries/0_stateless/00964_bloom_index_string_functions.sh index 7697578ea66..e2ec7fd42e4 100755 --- a/tests/queries/0_stateless/00964_bloom_index_string_functions.sh +++ b/tests/queries/0_stateless/00964_bloom_index_string_functions.sh @@ -15,7 +15,7 @@ CREATE TABLE bloom_filter_idx INDEX bf (s, lower(s)) TYPE ngrambf_v1(3, 512, 2, 0) GRANULARITY 1 ) ENGINE = MergeTree() ORDER BY k -SETTINGS index_granularity = 2;" +SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';" $CLICKHOUSE_CLIENT --query="INSERT INTO bloom_filter_idx VALUES (0, 'ClickHouse - столбцовая система управления базами данных (СУБД)'), diff --git a/tests/queries/0_stateless/00965_set_index_string_functions.sh b/tests/queries/0_stateless/00965_set_index_string_functions.sh index dba33d9abcf..8892fb11752 100755 --- a/tests/queries/0_stateless/00965_set_index_string_functions.sh +++ b/tests/queries/0_stateless/00965_set_index_string_functions.sh @@ -14,7 +14,7 @@ CREATE TABLE set_idx INDEX idx (s) TYPE set(2) GRANULARITY 1 ) ENGINE = MergeTree() ORDER BY k -SETTINGS index_granularity = 2;" +SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';" $CLICKHOUSE_CLIENT --query="INSERT INTO set_idx VALUES (0, 'ClickHouse - столбцовая система управления базами данных (СУБД)'), diff --git a/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper_long.sh b/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper_long.sh index 5ad4007b873..89b17ffe2bf 100755 --- a/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper_long.sh +++ b/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper_long.sh @@ -21,7 +21,7 @@ CREATE TABLE indices_mutaions1 ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/indices_mutaions', 'r1') PARTITION BY i32 ORDER BY u64 -SETTINGS index_granularity = 2; +SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi'; CREATE TABLE indices_mutaions2 ( @@ -32,7 +32,7 @@ CREATE TABLE indices_mutaions2 ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/indices_mutaions', 'r2') PARTITION BY i32 ORDER BY u64 -SETTINGS index_granularity = 2;" +SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi';" $CLICKHOUSE_CLIENT --query="INSERT INTO indices_mutaions1 VALUES diff --git a/tests/queries/0_stateless/00980_alter_settings_race.sh b/tests/queries/0_stateless/00980_alter_settings_race.sh index 6eb1df8964c..2fad7b79301 100755 --- a/tests/queries/0_stateless/00980_alter_settings_race.sh +++ b/tests/queries/0_stateless/00980_alter_settings_race.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS table_for_concurrent_alter" -$CLICKHOUSE_CLIENT --query="CREATE TABLE table_for_concurrent_alter (id UInt64, Data String) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity=4096;"; +$CLICKHOUSE_CLIENT --query="CREATE TABLE table_for_concurrent_alter (id UInt64, Data String) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity=4096, index_granularity_bytes = '10Mi';"; n=0 while [ "$n" -lt 50 ]; diff --git a/tests/queries/0_stateless/00980_merge_alter_settings.reference b/tests/queries/0_stateless/00980_merge_alter_settings.reference index 7a958c40651..706b64184ca 100644 --- a/tests/queries/0_stateless/00980_merge_alter_settings.reference +++ b/tests/queries/0_stateless/00980_merge_alter_settings.reference @@ -1,12 +1,12 @@ -CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096 -CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, parts_to_throw_insert = 1, parts_to_delay_insert = 1 -CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, parts_to_throw_insert = 100, parts_to_delay_insert = 100 +CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi' +CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi', parts_to_throw_insert = 1, parts_to_delay_insert = 1 +CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi', parts_to_throw_insert = 100, parts_to_delay_insert = 100 2 -CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, parts_to_throw_insert = 100, parts_to_delay_insert = 100, check_delay_period = 30 -CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String,\n `Data2` UInt64\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, parts_to_throw_insert = 100, parts_to_delay_insert = 100, check_delay_period = 15 -CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096 +CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi', parts_to_throw_insert = 100, parts_to_delay_insert = 100, check_delay_period = 30 +CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String,\n `Data2` UInt64\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi', parts_to_throw_insert = 100, parts_to_delay_insert = 100, check_delay_period = 15 +CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi' CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, parts_to_throw_insert = 1, parts_to_delay_insert = 1 -CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096 -CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096 -CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, merge_with_ttl_timeout = 300, max_concurrent_queries = 1 -CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096 +CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi' +CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi' +CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi', merge_with_ttl_timeout = 300, max_concurrent_queries = 1 +CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi' diff --git a/tests/queries/0_stateless/00980_merge_alter_settings.sql b/tests/queries/0_stateless/00980_merge_alter_settings.sql index f595a09970d..02728a6ba8b 100644 --- a/tests/queries/0_stateless/00980_merge_alter_settings.sql +++ b/tests/queries/0_stateless/00980_merge_alter_settings.sql @@ -17,7 +17,7 @@ DROP TABLE IF EXISTS table_for_alter; CREATE TABLE table_for_alter ( id UInt64, Data String -) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity=4096; +) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity=4096, index_granularity_bytes = '10Mi'; ALTER TABLE table_for_alter MODIFY SETTING index_granularity=555; -- { serverError 472 } @@ -62,7 +62,7 @@ DROP TABLE IF EXISTS table_for_reset_setting; CREATE TABLE table_for_reset_setting ( id UInt64, Data String -) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity=4096; +) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity=4096, index_granularity_bytes = '10Mi'; ALTER TABLE table_for_reset_setting MODIFY SETTING index_granularity=555; -- { serverError 472 } diff --git a/tests/queries/0_stateless/00990_hasToken_and_tokenbf.sql b/tests/queries/0_stateless/00990_hasToken_and_tokenbf.sql index d3b36cda0d8..41676905771 100644 --- a/tests/queries/0_stateless/00990_hasToken_and_tokenbf.sql +++ b/tests/queries/0_stateless/00990_hasToken_and_tokenbf.sql @@ -5,7 +5,7 @@ CREATE TABLE bloom_filter id UInt64, s String, INDEX tok_bf (s, lower(s)) TYPE tokenbf_v1(512, 3, 0) GRANULARITY 1 -) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 8; +) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 8, index_granularity_bytes = '10Mi'; insert into bloom_filter select number, 'yyy,uuu' from numbers(1024); insert into bloom_filter select number+2000, 'abc,def,zzz' from numbers(8); @@ -38,4 +38,4 @@ SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'yyy'); -- { serverError 158 -- this syntax is not supported by tokenbf SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'zzz') == 1; -- { serverError 158 } -DROP TABLE bloom_filter; \ No newline at end of file +DROP TABLE bloom_filter; diff --git a/tests/queries/0_stateless/01030_final_mark_empty_primary_key.sql b/tests/queries/0_stateless/01030_final_mark_empty_primary_key.sql index c64b40dfd1d..7bf2e3e737b 100644 --- a/tests/queries/0_stateless/01030_final_mark_empty_primary_key.sql +++ b/tests/queries/0_stateless/01030_final_mark_empty_primary_key.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS empty_pk; -CREATE TABLE empty_pk (x UInt64) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 256; +CREATE TABLE empty_pk (x UInt64) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 256, index_granularity_bytes = '10Mi'; INSERT INTO empty_pk SELECT number FROM numbers(100000); diff --git a/tests/queries/0_stateless/01055_minmax_index_compact_parts.sh b/tests/queries/0_stateless/01055_minmax_index_compact_parts.sh index 030a32dcdcb..0b14ef8f6fa 100755 --- a/tests/queries/0_stateless/01055_minmax_index_compact_parts.sh +++ b/tests/queries/0_stateless/01055_minmax_index_compact_parts.sh @@ -25,7 +25,7 @@ CREATE TABLE minmax_idx INDEX idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE minmax GRANULARITY 3 ) ENGINE = MergeTree() ORDER BY u64 -SETTINGS index_granularity = 2, min_rows_for_wide_part = 1000000" +SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi', min_rows_for_wide_part = 1000000" $CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES @@ -50,4 +50,4 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM minmax_idx WHERE i32 = 5 AND i32 + f64 $CLICKHOUSE_CLIENT --query="SELECT * FROM minmax_idx WHERE (u64 < 2 OR u64 > 10) AND e != 'b' ORDER BY dt" $CLICKHOUSE_CLIENT --query="SELECT * FROM minmax_idx WHERE (u64 < 2 OR u64 > 10) AND e != 'b' ORDER BY dt FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="DROP TABLE minmax_idx" \ No newline at end of file +$CLICKHOUSE_CLIENT --query="DROP TABLE minmax_idx" diff --git a/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql b/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql index 9a439180265..3d75fb0ccc9 100644 --- a/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql +++ b/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql @@ -13,7 +13,7 @@ DROP TABLE IF EXISTS mv_checkouts2target; -- that is the final table, which is filled incrementally from 2 different sources CREATE TABLE target_table Engine=SummingMergeTree() ORDER BY id -SETTINGS index_granularity=128 +SETTINGS index_granularity=128, index_granularity_bytes = '10Mi' AS SELECT number as id, diff --git a/tests/queries/0_stateless/01114_materialize_clear_index_compact_parts.sql b/tests/queries/0_stateless/01114_materialize_clear_index_compact_parts.sql index 831cb25d967..767ca0e4073 100644 --- a/tests/queries/0_stateless/01114_materialize_clear_index_compact_parts.sql +++ b/tests/queries/0_stateless/01114_materialize_clear_index_compact_parts.sql @@ -8,7 +8,7 @@ CREATE TABLE minmax_compact ) ENGINE = MergeTree() PARTITION BY i32 ORDER BY u64 -SETTINGS index_granularity = 2, min_rows_for_wide_part = 1000000; +SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi', min_rows_for_wide_part = 1000000; INSERT INTO minmax_compact VALUES (0, 2, 1), (1, 1, 1), (2, 1, 1), (3, 1, 1), (4, 1, 1), (5, 2, 1), (6, 1, 2), (7, 1, 2), (8, 1, 2), (9, 1, 2); diff --git a/tests/queries/0_stateless/01201_read_single_thread_in_order.sql b/tests/queries/0_stateless/01201_read_single_thread_in_order.sql index bfe03192891..33ccbbbe84d 100644 --- a/tests/queries/0_stateless/01201_read_single_thread_in_order.sql +++ b/tests/queries/0_stateless/01201_read_single_thread_in_order.sql @@ -6,7 +6,7 @@ CREATE TABLE t ) ENGINE = MergeTree ORDER BY number -SETTINGS index_granularity = 128; +SETTINGS index_granularity = 128, index_granularity_bytes = '10Mi'; SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0; INSERT INTO t SELECT number FROM numbers(10000000); diff --git a/tests/queries/0_stateless/01307_bloom_filter_index_string_multi_granulas.sql b/tests/queries/0_stateless/01307_bloom_filter_index_string_multi_granulas.sql index e96c70bef7f..cfb1f45c19a 100644 --- a/tests/queries/0_stateless/01307_bloom_filter_index_string_multi_granulas.sql +++ b/tests/queries/0_stateless/01307_bloom_filter_index_string_multi_granulas.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS test_01307; -CREATE TABLE test_01307 (id UInt64, val String, INDEX ind val TYPE bloom_filter() GRANULARITY 1) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 2; +CREATE TABLE test_01307 (id UInt64, val String, INDEX ind val TYPE bloom_filter() GRANULARITY 1) ENGINE = MergeTree() ORDER BY id SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi'; INSERT INTO test_01307 (id, val) select number as id, toString(number) as val from numbers(4); SELECT count() FROM test_01307 WHERE identity(val) = '2'; SELECT count() FROM test_01307 WHERE val = '2'; diff --git a/tests/queries/0_stateless/01312_comparison_with_constant_string_in_index_analysis.sql b/tests/queries/0_stateless/01312_comparison_with_constant_string_in_index_analysis.sql index e37f647e81f..50aa434a28c 100644 --- a/tests/queries/0_stateless/01312_comparison_with_constant_string_in_index_analysis.sql +++ b/tests/queries/0_stateless/01312_comparison_with_constant_string_in_index_analysis.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS test; -CREATE TABLE test (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS index_granularity = 1000; +CREATE TABLE test (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS index_granularity = 1000, index_granularity_bytes = '10Mi'; INSERT INTO test SELECT * FROM numbers(1000000); OPTIMIZE TABLE test; diff --git a/tests/queries/0_stateless/01515_mv_and_array_join_optimisation_bag.sql b/tests/queries/0_stateless/01515_mv_and_array_join_optimisation_bag.sql index ad762ea65fb..56dabc2a7cf 100644 --- a/tests/queries/0_stateless/01515_mv_and_array_join_optimisation_bag.sql +++ b/tests/queries/0_stateless/01515_mv_and_array_join_optimisation_bag.sql @@ -43,7 +43,7 @@ CREATE TABLE goal `GoalID` UInt32, `Visits` AggregateFunction(sumIf, Int8, UInt8), `GoalReaches` AggregateFunction(sum, Int8) -) ENGINE = AggregatingMergeTree PARTITION BY toStartOfMonth(StartDate) ORDER BY (CounterID, StartDate, GoalID) SETTINGS index_granularity = 256; +) ENGINE = AggregatingMergeTree PARTITION BY toStartOfMonth(StartDate) ORDER BY (CounterID, StartDate, GoalID) SETTINGS index_granularity = 256, index_granularity_bytes = '10Mi'; INSERT INTO visits (`CounterID`,`StartDate`,`StartTime`,`Sign`,`GoalsID`) VALUES (1, toDate('2000-01-01'), toDateTime(toDate('2000-01-01')), 1, [1]); diff --git a/tests/queries/0_stateless/01605_skip_idx_compact_parts.sql b/tests/queries/0_stateless/01605_skip_idx_compact_parts.sql index d57ccdcf9fc..9d44550c0ba 100644 --- a/tests/queries/0_stateless/01605_skip_idx_compact_parts.sql +++ b/tests/queries/0_stateless/01605_skip_idx_compact_parts.sql @@ -1,7 +1,7 @@ DROP TABLE IF EXISTS skip_idx_comp_parts; CREATE TABLE skip_idx_comp_parts (a Int, b Int, index b_idx b TYPE minmax GRANULARITY 4) ENGINE = MergeTree ORDER BY a - SETTINGS index_granularity=256, merge_max_block_size=100; + SETTINGS index_granularity=256, index_granularity_bytes = '10Mi', merge_max_block_size=100; SYSTEM STOP MERGES skip_idx_comp_parts; diff --git a/tests/queries/0_stateless/01681_bloom_filter_nullable_column.sql b/tests/queries/0_stateless/01681_bloom_filter_nullable_column.sql index 4af1f74fca6..50663654b10 100644 --- a/tests/queries/0_stateless/01681_bloom_filter_nullable_column.sql +++ b/tests/queries/0_stateless/01681_bloom_filter_nullable_column.sql @@ -7,7 +7,7 @@ CREATE TABLE bloom_filter_nullable_index INDEX idx (str) TYPE bloom_filter GRANULARITY 1 ) ENGINE = MergeTree() - ORDER BY order_key SETTINGS index_granularity = 6; + ORDER BY order_key SETTINGS index_granularity = 6, index_granularity_bytes = '10Mi'; INSERT INTO bloom_filter_nullable_index VALUES (1, 'test'); INSERT INTO bloom_filter_nullable_index VALUES (2, 'test2'); diff --git a/tests/queries/0_stateless/01780_column_sparse_full.sql b/tests/queries/0_stateless/01780_column_sparse_full.sql index 08a1c0699a4..57da1974373 100644 --- a/tests/queries/0_stateless/01780_column_sparse_full.sql +++ b/tests/queries/0_stateless/01780_column_sparse_full.sql @@ -5,7 +5,7 @@ DROP TABLE IF EXISTS t_sparse_full; CREATE TABLE t_sparse_full (id UInt64, u UInt64, s String) ENGINE = MergeTree ORDER BY id -SETTINGS index_granularity = 32, +SETTINGS index_granularity = 32, index_granularity_bytes = '10Mi', ratio_of_defaults_for_sparse_serialization = 0.1; SYSTEM STOP MERGES t_sparse_full; diff --git a/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.sql b/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.sql index 4cc216955b3..400792df8be 100644 --- a/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.sql +++ b/tests/queries/0_stateless/01781_token_extractor_buffer_overflow.sql @@ -1,7 +1,7 @@ SET max_block_size = 10, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0, max_threads = 20; DROP TABLE IF EXISTS bloom_filter; -CREATE TABLE bloom_filter (`id` UInt64, `s` String, INDEX tok_bf (s, lower(s)) TYPE tokenbf_v1(512, 3, 0) GRANULARITY 1) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8; +CREATE TABLE bloom_filter (`id` UInt64, `s` String, INDEX tok_bf (s, lower(s)) TYPE tokenbf_v1(512, 3, 0) GRANULARITY 1) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8, index_granularity_bytes = '10Mi'; INSERT INTO bloom_filter SELECT number, 'yyy,uuu' FROM numbers(1024); SELECT max(id) FROM bloom_filter WHERE hasToken(s, 'abc'); 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 92c7a27bc9a..a0047a2925a 100644 --- a/tests/queries/0_stateless/01926_order_by_desc_limit.sql +++ b/tests/queries/0_stateless/01926_order_by_desc_limit.sql @@ -6,7 +6,7 @@ SET enable_filesystem_cache=0; CREATE TABLE order_by_desc (u UInt32, s String) ENGINE MergeTree ORDER BY u PARTITION BY u % 100 -SETTINGS index_granularity = 1024; +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; diff --git a/tests/queries/0_stateless/02030_tuple_filter.sql b/tests/queries/0_stateless/02030_tuple_filter.sql index d2a114a89f9..f2fc3a30aa6 100644 --- a/tests/queries/0_stateless/02030_tuple_filter.sql +++ b/tests/queries/0_stateless/02030_tuple_filter.sql @@ -2,7 +2,7 @@ SET allow_experimental_analyzer = 1; DROP TABLE IF EXISTS test_tuple_filter; -CREATE TABLE test_tuple_filter (id UInt32, value String, log_date Date) Engine=MergeTree() ORDER BY id PARTITION BY log_date SETTINGS index_granularity = 3; +CREATE TABLE test_tuple_filter (id UInt32, value String, log_date Date) Engine=MergeTree() ORDER BY id PARTITION BY log_date SETTINGS index_granularity = 3, index_granularity_bytes = '10Mi'; INSERT INTO test_tuple_filter VALUES (1,'A','2021-01-01'),(2,'B','2021-01-01'),(3,'C','2021-01-01'),(4,'D','2021-01-02'),(5,'E','2021-01-02'); diff --git a/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.sql b/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.sql index 65a8cadb996..f50aab67d77 100644 --- a/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.sql +++ b/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.sql @@ -38,7 +38,7 @@ DROP TABLE IF EXISTS t_read_in_order; CREATE TABLE t_read_in_order(a UInt32, b UInt32) ENGINE = MergeTree ORDER BY (a, b) -SETTINGS index_granularity = 3; +SETTINGS index_granularity = 3, index_granularity_bytes = '10Mi'; SYSTEM STOP MERGES t_read_in_order; diff --git a/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.sql b/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.sql index 314d0610d12..b387582296d 100644 --- a/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.sql +++ b/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS t_max_rows_to_read; CREATE TABLE t_max_rows_to_read (a UInt64) ENGINE = MergeTree ORDER BY a -SETTINGS index_granularity = 4; +SETTINGS index_granularity = 4, index_granularity_bytes = '10Mi'; INSERT INTO t_max_rows_to_read SELECT number FROM numbers(100); diff --git a/tests/queries/0_stateless/02267_empty_arrays_read_reverse.sql b/tests/queries/0_stateless/02267_empty_arrays_read_reverse.sql index 7c1cf47c540..0c6c1a46ee1 100644 --- a/tests/queries/0_stateless/02267_empty_arrays_read_reverse.sql +++ b/tests/queries/0_stateless/02267_empty_arrays_read_reverse.sql @@ -8,7 +8,7 @@ CREATE TABLE t_02267 ) ENGINE = MergeTree ORDER BY b -SETTINGS index_granularity = 500; +SETTINGS index_granularity = 500, index_granularity_bytes = '10Mi'; INSERT INTO t_02267 (b, a, c) SELECT 0, ['x'], ['1','2','3','4','5','6'] FROM numbers(1) ; INSERT INTO t_02267 (b, a, c) SELECT 1, [], ['1','2','3','4','5','6'] FROM numbers(300000); diff --git a/tests/queries/0_stateless/02346_full_text_search.sql b/tests/queries/0_stateless/02346_full_text_search.sql index cead2ce0666..af49c5d52c2 100644 --- a/tests/queries/0_stateless/02346_full_text_search.sql +++ b/tests/queries/0_stateless/02346_full_text_search.sql @@ -8,7 +8,7 @@ DROP TABLE IF EXISTS tab; CREATE TABLE tab(k UInt64, s String, INDEX af(s) TYPE inverted(2)) ENGINE = MergeTree() ORDER BY k - SETTINGS index_granularity = 2; + SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi'; INSERT INTO tab VALUES (101, 'Alick a01'), (102, 'Blick a02'), (103, 'Click a03'), (104, 'Dlick a04'), (105, 'Elick a05'), (106, 'Alick a06'), (107, 'Blick a07'), (108, 'Click a08'), (109, 'Dlick a09'), (110, 'Elick a10'), (111, 'Alick b01'), (112, 'Blick b02'), (113, 'Click b03'), (114, 'Dlick b04'), (115, 'Elick b05'), (116, 'Alick b06'), (117, 'Blick b07'), (118, 'Click b08'), (119, 'Dlick b09'), (120, 'Elick b10'); @@ -61,7 +61,7 @@ DROP TABLE IF EXISTS tab_x; CREATE TABLE tab_x(k UInt64, s String, INDEX af(s) TYPE inverted()) ENGINE = MergeTree() ORDER BY k - SETTINGS index_granularity = 2; + SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi'; INSERT INTO tab_x VALUES (101, 'Alick a01'), (102, 'Blick a02'), (103, 'Click a03'), (104, 'Dlick a04'), (105, 'Elick a05'), (106, 'Alick a06'), (107, 'Blick a07'), (108, 'Click a08'), (109, 'Dlick a09'), (110, 'Elick a10'), (111, 'Alick b01'), (112, 'Blick b02'), (113, 'Click b03'), (114, 'Dlick b04'), (115, 'Elick b05'), (116, 'Alick b06'), (117, 'Blick b07'), (118, 'Click b08'), (119, 'Dlick b09'), (120, 'Elick b10'); @@ -114,7 +114,7 @@ DROP TABLE IF EXISTS tab; create table tab (k UInt64, s Array(String), INDEX af(s) TYPE inverted(2)) ENGINE = MergeTree() ORDER BY k - SETTINGS index_granularity = 2; + SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi'; INSERT INTO tab SELECT rowNumberInBlock(), groupArray(s) FROM tab_x GROUP BY k%10; @@ -141,7 +141,7 @@ DROP TABLE IF EXISTS tab; CREATE TABLE tab (k UInt64, s Map(String,String), INDEX af(mapKeys(s)) TYPE inverted(2)) ENGINE = MergeTree() ORDER BY k - SETTINGS index_granularity = 2; + SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi'; INSERT INTO tab VALUES (101, {'Alick':'Alick a01'}), (102, {'Blick':'Blick a02'}), (103, {'Click':'Click a03'}), (104, {'Dlick':'Dlick a04'}), (105, {'Elick':'Elick a05'}), (106, {'Alick':'Alick a06'}), (107, {'Blick':'Blick a07'}), (108, {'Click':'Click a08'}), (109, {'Dlick':'Dlick a09'}), (110, {'Elick':'Elick a10'}), (111, {'Alick':'Alick b01'}), (112, {'Blick':'Blick b02'}), (113, {'Click':'Click b03'}), (114, {'Dlick':'Dlick b04'}), (115, {'Elick':'Elick b05'}), (116, {'Alick':'Alick b06'}), (117, {'Blick':'Blick b07'}), (118, {'Click':'Click b08'}), (119, {'Dlick':'Dlick b09'}), (120, {'Elick':'Elick b10'}); @@ -181,7 +181,7 @@ DROP TABLE IF EXISTS tab; CREATE TABLE tab(k UInt64, s String, INDEX af(s) TYPE inverted(2)) ENGINE = MergeTree() ORDER BY k - SETTINGS index_granularity = 2; + SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi'; INSERT INTO tab VALUES (101, 'Alick a01'), (102, 'Blick a02'), (103, 'Click a03'), (104, 'Dlick a04'), (105, 'Elick a05'), (106, 'Alick a06'), (107, 'Blick a07'), (108, 'Click a08'), (109, 'Dlick a09'), (110, 'Elick b10'), (111, 'Alick b01'), (112, 'Blick b02'), (113, 'Click b03'), (114, 'Dlick b04'), (115, 'Elick b05'), (116, 'Alick b06'), (117, 'Blick b07'), (118, 'Click b08'), (119, 'Dlick b09'), (120, 'Elick b10'); INSERT INTO tab VALUES (201, 'rick c01'), (202, 'mick c02'), (203, 'nick c03'); @@ -210,7 +210,7 @@ DROP TABLE IF EXISTS tab; CREATE TABLE tab(k UInt64, s String, INDEX af(s) TYPE inverted(2)) ENGINE = MergeTree() ORDER BY k - SETTINGS index_granularity = 2; + SETTINGS index_granularity = 2, index_granularity_bytes = '10Mi'; INSERT INTO tab VALUES (101, 'Alick 好'), (102, 'clickhouse你好'), (103, 'Click 你'), (104, 'Dlick 你a好'), (105, 'Elick 好好你你'), (106, 'Alick 好a好a你a你'); @@ -332,4 +332,3 @@ SELECT read_rows==512 from system.query_log AND type='QueryFinish' AND result_rows==1 LIMIT 1; - diff --git a/tests/queries/0_stateless/02354_annoy.sh b/tests/queries/0_stateless/02354_annoy.sh index 526886ec68d..7b49a338955 100755 --- a/tests/queries/0_stateless/02354_annoy.sh +++ b/tests/queries/0_stateless/02354_annoy.sh @@ -17,7 +17,7 @@ CREATE TABLE 02354_annoy_l2 ) ENGINE = MergeTree ORDER BY id -SETTINGS index_granularity=5; +SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; INSERT INTO 02354_annoy_l2 VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); @@ -64,7 +64,7 @@ CREATE TABLE 02354_annoy_l2 ) ENGINE = MergeTree ORDER BY id -SETTINGS index_granularity=5; +SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; INSERT INTO 02354_annoy_l2 VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); @@ -95,7 +95,7 @@ CREATE TABLE 02354_annoy_cosine ) ENGINE = MergeTree ORDER BY id -SETTINGS index_granularity=5; +SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; INSERT INTO 02354_annoy_cosine VALUES (1, [0.0, 0.0, 10.0]), (2, [0.2, 0.0, 10.0]), (3, [-0.3, 0.0, 10.0]), (4, [0.5, 0.0, 10.1]), (5, [0.8, 0.0, 10.0]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); @@ -124,7 +124,7 @@ CREATE TABLE 02354_annoy_cosine ) ENGINE = MergeTree ORDER BY id -SETTINGS index_granularity=5; +SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; INSERT INTO 02354_annoy_cosine VALUES (1, [0.0, 0.0, 10.0]), (2, [0.2, 0.0, 10.0]), (3, [-0.3, 0.0, 10.0]), (4, [0.5, 0.0, 10.1]), (5, [0.8, 0.0, 10.0]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); @@ -156,7 +156,7 @@ CREATE TABLE 02354_annoy ) ENGINE = MergeTree ORDER BY id -SETTINGS index_granularity=5; -- {serverError 7 } +SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 7 } -- Index must be created on Array(Float32) or Tuple(Float32) @@ -168,7 +168,7 @@ CREATE TABLE 02354_annoy ) ENGINE = MergeTree ORDER BY id -SETTINGS index_granularity=5; -- {serverError 44 } +SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 44 } CREATE TABLE 02354_annoy @@ -179,7 +179,7 @@ CREATE TABLE 02354_annoy ) ENGINE = MergeTree ORDER BY id -SETTINGS index_granularity=5; -- {serverError 44 } +SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 44 } CREATE TABLE 02354_annoy ( @@ -189,7 +189,7 @@ CREATE TABLE 02354_annoy ) ENGINE = MergeTree ORDER BY id -SETTINGS index_granularity=5; -- {serverError 44 } +SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 44 } CREATE TABLE 02354_annoy ( @@ -199,7 +199,7 @@ CREATE TABLE 02354_annoy ) ENGINE = MergeTree ORDER BY id -SETTINGS index_granularity=5; -- {serverError 44 } +SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 44 } CREATE TABLE 02354_annoy ( @@ -209,4 +209,4 @@ CREATE TABLE 02354_annoy ) ENGINE = MergeTree ORDER BY id -SETTINGS index_granularity=5; -- {serverError 44 }" +SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 44 }" diff --git a/tests/queries/0_stateless/02374_in_tuple_index.sql b/tests/queries/0_stateless/02374_in_tuple_index.sql index 7f9b7b5470e..4f489f74eda 100644 --- a/tests/queries/0_stateless/02374_in_tuple_index.sql +++ b/tests/queries/0_stateless/02374_in_tuple_index.sql @@ -8,7 +8,7 @@ CREATE TABLE t_in_tuple_index ) ENGINE = MergeTree() ORDER BY (PLATFORM, USER_ID, ID) -SETTINGS index_granularity = 2048; +SETTINGS index_granularity = 2048, index_granularity_bytes = '10Mi'; INSERT INTO t_in_tuple_index VALUES ('1', 33, 'insta'), ('2', 33, 'insta'); diff --git a/tests/queries/0_stateless/02465_limit_trivial_max_rows_to_read.sql b/tests/queries/0_stateless/02465_limit_trivial_max_rows_to_read.sql index ee7a4e6b6b5..c2e97c8c704 100644 --- a/tests/queries/0_stateless/02465_limit_trivial_max_rows_to_read.sql +++ b/tests/queries/0_stateless/02465_limit_trivial_max_rows_to_read.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS t_max_rows_to_read; CREATE TABLE t_max_rows_to_read (a UInt64) ENGINE = MergeTree ORDER BY a -SETTINGS index_granularity = 4; +SETTINGS index_granularity = 4, index_granularity_bytes = '10Mi'; INSERT INTO t_max_rows_to_read SELECT number FROM numbers(100); diff --git a/tests/queries/0_stateless/02467_set_with_lowcardinality_type.sql b/tests/queries/0_stateless/02467_set_with_lowcardinality_type.sql index dee6f7de74a..1607d96977b 100644 --- a/tests/queries/0_stateless/02467_set_with_lowcardinality_type.sql +++ b/tests/queries/0_stateless/02467_set_with_lowcardinality_type.sql @@ -6,7 +6,7 @@ CREATE TABLE bloom_filter_nullable_index__fuzz_0 `str` Nullable(String), INDEX idx str TYPE bloom_filter GRANULARITY 1 ) -ENGINE = MergeTree ORDER BY order_key SETTINGS index_granularity = 6; +ENGINE = MergeTree ORDER BY order_key SETTINGS index_granularity = 6, index_granularity_bytes = '10Mi'; INSERT INTO bloom_filter_nullable_index__fuzz_0 VALUES (1, 'test'); INSERT INTO bloom_filter_nullable_index__fuzz_0 VALUES (2, 'test2'); @@ -18,7 +18,7 @@ CREATE TABLE bloom_filter_nullable_index__fuzz_1 `str` String, INDEX idx str TYPE bloom_filter GRANULARITY 1 ) -ENGINE = MergeTree ORDER BY order_key SETTINGS index_granularity = 6; +ENGINE = MergeTree ORDER BY order_key SETTINGS index_granularity = 6, index_granularity_bytes = '10Mi'; INSERT INTO bloom_filter_nullable_index__fuzz_0 VALUES (1, 'test'); INSERT INTO bloom_filter_nullable_index__fuzz_0 VALUES (2, 'test2'); diff --git a/tests/queries/0_stateless/02481_default_value_used_in_row_level_filter.sql b/tests/queries/0_stateless/02481_default_value_used_in_row_level_filter.sql index 6835a3a57ea..ce1662699a1 100644 --- a/tests/queries/0_stateless/02481_default_value_used_in_row_level_filter.sql +++ b/tests/queries/0_stateless/02481_default_value_used_in_row_level_filter.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS test_rlp; -CREATE TABLE test_rlp (a Int32, b Int32) ENGINE=MergeTree() ORDER BY a SETTINGS index_granularity=5; +CREATE TABLE test_rlp (a Int32, b Int32) ENGINE=MergeTree() ORDER BY a SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; INSERT INTO test_rlp SELECT number, number FROM numbers(15); diff --git a/tests/queries/0_stateless/02481_prewhere_filtered_rows_div_by_zero.reference b/tests/queries/0_stateless/02481_prewhere_filtered_rows_div_by_zero.reference index bb8ce4a8396..f9c1c174a7d 100644 --- a/tests/queries/0_stateless/02481_prewhere_filtered_rows_div_by_zero.reference +++ b/tests/queries/0_stateless/02481_prewhere_filtered_rows_div_by_zero.reference @@ -1,5 +1,5 @@ -- { echoOn } -CREATE TABLE test_filter(a Int32, b Int32, c Int32) ENGINE = MergeTree() ORDER BY a SETTINGS index_granularity = 3; +CREATE TABLE test_filter(a Int32, b Int32, c Int32) ENGINE = MergeTree() ORDER BY a SETTINGS index_granularity = 3, index_granularity_bytes = '10Mi'; INSERT INTO test_filter SELECT number, number+1, (number/2 + 1) % 2 FROM numbers(15); SELECT _part_offset, intDiv(_part_offset, 3) as granule, * FROM test_filter ORDER BY _part_offset; 0 0 0 1 1 diff --git a/tests/queries/0_stateless/02481_prewhere_filtered_rows_div_by_zero.sql b/tests/queries/0_stateless/02481_prewhere_filtered_rows_div_by_zero.sql index 94ffb1b8730..ab675df75cf 100644 --- a/tests/queries/0_stateless/02481_prewhere_filtered_rows_div_by_zero.sql +++ b/tests/queries/0_stateless/02481_prewhere_filtered_rows_div_by_zero.sql @@ -1,7 +1,7 @@ DROP TABLE IF EXISTS test_filter; -- { echoOn } -CREATE TABLE test_filter(a Int32, b Int32, c Int32) ENGINE = MergeTree() ORDER BY a SETTINGS index_granularity = 3; +CREATE TABLE test_filter(a Int32, b Int32, c Int32) ENGINE = MergeTree() ORDER BY a SETTINGS index_granularity = 3, index_granularity_bytes = '10Mi'; INSERT INTO test_filter SELECT number, number+1, (number/2 + 1) % 2 FROM numbers(15); From b1d9ba9da19c2f23f21d01c0b3d998b80936b201 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 21 Feb 2023 01:46:10 +0300 Subject: [PATCH 141/253] Fix #46620 --- docs/en/development/continuous-integration.md | 15 +-------------- 1 file changed, 1 insertion(+), 14 deletions(-) diff --git a/docs/en/development/continuous-integration.md b/docs/en/development/continuous-integration.md index 9784c0863b4..232eee5b3cf 100644 --- a/docs/en/development/continuous-integration.md +++ b/docs/en/development/continuous-integration.md @@ -43,11 +43,6 @@ Tries to build the ClickHouse documentation website. It can fail if you changed something in the documentation. Most probable reason is that some cross-link in the documentation is wrong. Go to the check report and look for `ERROR` and `WARNING` messages. -### Report Details - -- [Status page example](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check.html) -- `docs_output.txt` contains the building log. [Successful result example](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check/docs_output.txt) - ## Description Check @@ -72,10 +67,6 @@ This check means that the CI system started to process the pull request. When it Performs some simple regex-based checks of code style, using the [`utils/check-style/check-style`](https://github.com/ClickHouse/ClickHouse/blob/master/utils/check-style/check-style) binary (note that it can be run locally). If it fails, fix the style errors following the [code style guide](style.md). -### Report Details -- [Status page example](https://clickhouse-test-reports.s3.yandex.net/12550/659c78c7abb56141723af6a81bfae39335aa8cb2/style_check.html) -- `output.txt` contains the check resulting errors (invalid tabulation etc), blank page means no errors. [Successful result example](https://clickhouse-test-reports.s3.yandex.net/12550/659c78c7abb56141723af6a81bfae39335aa8cb2/style_check/output.txt). - ## Fast Test Normally this is the first check that is ran for a PR. It builds ClickHouse and @@ -84,8 +75,6 @@ some. If it fails, further checks are not started until it is fixed. Look at the report to see which tests fail, then reproduce the failure locally as described [here](tests.md#functional-test-locally). -### Report Details -[Status page example](https://clickhouse-test-reports.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/fast_test.html) #### Status Page Files - `runlog.out.log` is the general log that includes all other logs. @@ -113,9 +102,7 @@ Builds ClickHouse in various configurations for use in further steps. You have t ### Report Details -[Status page example](https://clickhouse-builds.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/clickhouse_build_check/report.html). - -- **Compiler**: `gcc-9` or `clang-10` (or `clang-10-xx` for other architectures e.g. `clang-10-freebsd`). +- **Compiler**: `clang-15`, optionally with the name of a target platform - **Build type**: `Debug` or `RelWithDebInfo` (cmake). - **Sanitizer**: `none` (without sanitizers), `address` (ASan), `memory` (MSan), `undefined` (UBSan), or `thread` (TSan). - **Status**: `success` or `fail` From 4e467d842dda9a7dddd45adc2186f8ff8e1c1ca3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 20 Feb 2023 23:48:46 +0100 Subject: [PATCH 142/253] Inhibit index_granularity_bytes randomization in some tests --- tests/queries/0_stateless/01710_projection_vertical_merges.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01710_projection_vertical_merges.sql b/tests/queries/0_stateless/01710_projection_vertical_merges.sql index d54fef7e71d..e8a4a384017 100644 --- a/tests/queries/0_stateless/01710_projection_vertical_merges.sql +++ b/tests/queries/0_stateless/01710_projection_vertical_merges.sql @@ -2,7 +2,7 @@ drop table if exists t; -create table t (c1 Int64, c2 String, c3 DateTime, c4 Int8, c5 String, c6 String, c7 String, c8 String, c9 String, c10 String, c11 String, c12 String, c13 Int8, c14 Int64, c15 String, c16 String, c17 String, c18 Int64, c19 Int64, c20 Int64) engine MergeTree order by c18; +create table t (c1 Int64, c2 String, c3 DateTime, c4 Int8, c5 String, c6 String, c7 String, c8 String, c9 String, c10 String, c11 String, c12 String, c13 Int8, c14 Int64, c15 String, c16 String, c17 String, c18 Int64, c19 Int64, c20 Int64) engine MergeTree order by c18 SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into t (c1, c18) select number, -number from numbers(2000000); From 67d808f49ada57940d257ff9b9a457c1628f4e72 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 21 Feb 2023 00:30:38 +0100 Subject: [PATCH 143/253] Add settings changes history --- src/Core/SettingsChangesHistory.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 5684e4f3114..48dcded1868 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -83,7 +83,8 @@ static std::map sett {"23.2", {{"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, {"query_plan_remove_redundant_distinct", false, true, "Remove redundant Distinct step in query plan"}, - {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}}}, + {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}, + {"insert_keeper_max_retries", 0, 20, "Enable reconnections to Keeper on INSERT, improve reliability"}}}, {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, From 40d11ce422ef62161463df7abf32619c6fa8e236 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 21 Feb 2023 00:39:32 +0100 Subject: [PATCH 144/253] Adapt a test --- .../test.py | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_keeper_multinode_blocade_leader/test.py b/tests/integration/test_keeper_multinode_blocade_leader/test.py index a7a80d90a58..b11b0a0b64c 100644 --- a/tests/integration/test_keeper_multinode_blocade_leader/test.py +++ b/tests/integration/test_keeper_multinode_blocade_leader/test.py @@ -89,7 +89,7 @@ def test_blocade_leader(started_cluster): print("Got exception from node", smaller_exception(ex)) time.sleep(0.1) - node2.query("INSERT INTO ordinary.t1 SELECT number FROM numbers(10)") + node2.query("INSERT INTO ordinary.t1 SELECT number FROM numbers(10) SETTINGS insert_keeper_max_retries = 0") node1.query("SYSTEM SYNC REPLICA ordinary.t1", timeout=10) node3.query("SYSTEM SYNC REPLICA ordinary.t1", timeout=10) @@ -107,7 +107,7 @@ def test_blocade_leader(started_cluster): restart_replica_for_sure( node2, "ordinary.t1", "/clickhouse/t1/replicas/2" ) - node2.query("INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100)") + node2.query("INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0") break except Exception as ex: try: @@ -128,7 +128,7 @@ def test_blocade_leader(started_cluster): restart_replica_for_sure( node3, "ordinary.t1", "/clickhouse/t1/replicas/3" ) - node3.query("INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100)") + node3.query("INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0") break except Exception as ex: try: @@ -167,7 +167,7 @@ def test_blocade_leader(started_cluster): for i in range(100): try: - node1.query("INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100)") + node1.query("INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0") break except Exception as ex: print("Got exception node1", smaller_exception(ex)) @@ -293,7 +293,7 @@ def test_blocade_leader_twice(started_cluster): print("Got exception from node", smaller_exception(ex)) time.sleep(0.1) - node2.query("INSERT INTO ordinary.t2 SELECT number FROM numbers(10)") + node2.query("INSERT INTO ordinary.t2 SELECT number FROM numbers(10) SETTINGS insert_keeper_max_retries = 0") node1.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) node3.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) @@ -311,7 +311,7 @@ def test_blocade_leader_twice(started_cluster): restart_replica_for_sure( node2, "ordinary.t2", "/clickhouse/t2/replicas/2" ) - node2.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100)") + node2.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0") break except Exception as ex: try: @@ -333,7 +333,7 @@ def test_blocade_leader_twice(started_cluster): node3, "ordinary.t2", "/clickhouse/t2/replicas/3" ) node3.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) - node3.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100)") + node3.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0") break except Exception as ex: try: @@ -359,14 +359,14 @@ def test_blocade_leader_twice(started_cluster): for i in range(10): try: - node3.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100)") + node3.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0") assert False, "Node3 became leader?" except Exception as ex: time.sleep(0.5) for i in range(10): try: - node2.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100)") + node2.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0") assert False, "Node2 became leader?" except Exception as ex: time.sleep(0.5) @@ -399,7 +399,7 @@ def test_blocade_leader_twice(started_cluster): for n, node in enumerate([node1, node2, node3]): for i in range(100): try: - node.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100)") + node.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0") break except Exception as ex: print("Got exception node{}".format(n + 1), smaller_exception(ex)) From dc4b52e54ebc61b378e78f8f3377a6ccb94b7a7d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 21 Feb 2023 00:41:40 +0100 Subject: [PATCH 145/253] Adapt a test --- tests/integration/test_storage_kafka/configs/users.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_storage_kafka/configs/users.xml b/tests/integration/test_storage_kafka/configs/users.xml index 2cef0a6de3c..0f2f6d7c424 100644 --- a/tests/integration/test_storage_kafka/configs/users.xml +++ b/tests/integration/test_storage_kafka/configs/users.xml @@ -2,6 +2,8 @@ 1 + + 0 From 4a2bd6e0aaf55e5fc35840c0c865d83c27fcc375 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 21 Feb 2023 00:43:10 +0100 Subject: [PATCH 146/253] Adapt a test --- tests/integration/test_storage_kafka/configs/users.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_kafka/configs/users.xml b/tests/integration/test_storage_kafka/configs/users.xml index 0f2f6d7c424..992464a0ac2 100644 --- a/tests/integration/test_storage_kafka/configs/users.xml +++ b/tests/integration/test_storage_kafka/configs/users.xml @@ -3,7 +3,7 @@ 1 - 0 + 0 From d0d63061024b83f0123b962a376c7551d64cf3d8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 21 Feb 2023 00:46:27 +0100 Subject: [PATCH 147/253] Fix coverity --- .github/workflows/nightly.yml | 2 +- docker/packager/binary/build.sh | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index b4ef794c4ca..27c4f5811da 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -107,7 +107,7 @@ jobs: run: | curl --form token="${COVERITY_TOKEN}" \ --form email='security+coverity@clickhouse.com' \ - --form file="@$TEMP_PATH/$BUILD_NAME/coverity-scan.tar.zst" \ + --form file="@$TEMP_PATH/$BUILD_NAME/coverity-scan.tar.gz" \ --form version="${GITHUB_REF#refs/heads/}-${GITHUB_SHA::6}" \ --form description="Nighly Scan: $(date +'%Y-%m-%dT%H:%M:%S')" \ https://scan.coverity.com/builds?project=ClickHouse%2FClickHouse diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 7499aceae2e..24dca72e946 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -174,8 +174,9 @@ fi if [ "coverity" == "$COMBINED_OUTPUT" ] then - tar -cv --zstd -f "coverity-scan.tar.zst" cov-int - mv "coverity-scan.tar.zst" /output + # Coverity does not understand ZSTD. + tar -cvz -f "coverity-scan.tar.gz" cov-int + mv "coverity-scan.tar.gz" /output fi ccache_status From abeb1c5ba8834cec406dc42a3c287f37a9b792af Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 20 Feb 2023 23:49:25 +0000 Subject: [PATCH 148/253] Automatic style fix --- .../test.py | 40 ++++++++++++++----- 1 file changed, 30 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_keeper_multinode_blocade_leader/test.py b/tests/integration/test_keeper_multinode_blocade_leader/test.py index b11b0a0b64c..3af0751b0fd 100644 --- a/tests/integration/test_keeper_multinode_blocade_leader/test.py +++ b/tests/integration/test_keeper_multinode_blocade_leader/test.py @@ -89,7 +89,9 @@ def test_blocade_leader(started_cluster): print("Got exception from node", smaller_exception(ex)) time.sleep(0.1) - node2.query("INSERT INTO ordinary.t1 SELECT number FROM numbers(10) SETTINGS insert_keeper_max_retries = 0") + node2.query( + "INSERT INTO ordinary.t1 SELECT number FROM numbers(10) SETTINGS insert_keeper_max_retries = 0" + ) node1.query("SYSTEM SYNC REPLICA ordinary.t1", timeout=10) node3.query("SYSTEM SYNC REPLICA ordinary.t1", timeout=10) @@ -107,7 +109,9 @@ def test_blocade_leader(started_cluster): restart_replica_for_sure( node2, "ordinary.t1", "/clickhouse/t1/replicas/2" ) - node2.query("INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0") + node2.query( + "INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" + ) break except Exception as ex: try: @@ -128,7 +132,9 @@ def test_blocade_leader(started_cluster): restart_replica_for_sure( node3, "ordinary.t1", "/clickhouse/t1/replicas/3" ) - node3.query("INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0") + node3.query( + "INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" + ) break except Exception as ex: try: @@ -167,7 +173,9 @@ def test_blocade_leader(started_cluster): for i in range(100): try: - node1.query("INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0") + node1.query( + "INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" + ) break except Exception as ex: print("Got exception node1", smaller_exception(ex)) @@ -293,7 +301,9 @@ def test_blocade_leader_twice(started_cluster): print("Got exception from node", smaller_exception(ex)) time.sleep(0.1) - node2.query("INSERT INTO ordinary.t2 SELECT number FROM numbers(10) SETTINGS insert_keeper_max_retries = 0") + node2.query( + "INSERT INTO ordinary.t2 SELECT number FROM numbers(10) SETTINGS insert_keeper_max_retries = 0" + ) node1.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) node3.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) @@ -311,7 +321,9 @@ def test_blocade_leader_twice(started_cluster): restart_replica_for_sure( node2, "ordinary.t2", "/clickhouse/t2/replicas/2" ) - node2.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0") + node2.query( + "INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" + ) break except Exception as ex: try: @@ -333,7 +345,9 @@ def test_blocade_leader_twice(started_cluster): node3, "ordinary.t2", "/clickhouse/t2/replicas/3" ) node3.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) - node3.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0") + node3.query( + "INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" + ) break except Exception as ex: try: @@ -359,14 +373,18 @@ def test_blocade_leader_twice(started_cluster): for i in range(10): try: - node3.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0") + node3.query( + "INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" + ) assert False, "Node3 became leader?" except Exception as ex: time.sleep(0.5) for i in range(10): try: - node2.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0") + node2.query( + "INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" + ) assert False, "Node2 became leader?" except Exception as ex: time.sleep(0.5) @@ -399,7 +417,9 @@ def test_blocade_leader_twice(started_cluster): for n, node in enumerate([node1, node2, node3]): for i in range(100): try: - node.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0") + node.query( + "INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100) SETTINGS insert_keeper_max_retries = 0" + ) break except Exception as ex: print("Got exception node{}".format(n + 1), smaller_exception(ex)) From c0b50c981abad16b2c344c8eb7a5c5d974916f6f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 21 Feb 2023 02:53:36 +0300 Subject: [PATCH 149/253] Installation instructions in README --- README.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/README.md b/README.md index dee60cafb33..5b5dc4ae528 100644 --- a/README.md +++ b/README.md @@ -2,6 +2,11 @@ ClickHouse® is an open-source column-oriented database management system that allows generating analytical data reports in real-time. +## How To Install +``` +curl https://clickhouse.com/ | sh +``` + ## Useful Links * [Official website](https://clickhouse.com/) has a quick high-level overview of ClickHouse on the main page. From 16a9ac9118cfd8eb8ed579fd12becb2eff8858f0 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 21 Feb 2023 00:51:39 +0000 Subject: [PATCH 150/253] Try to add test --- .../test_async_insert_memory/__init__.py | 0 .../test_async_insert_memory/test.py | 35 +++++++++++++++++++ 2 files changed, 35 insertions(+) create mode 100644 tests/integration/test_async_insert_memory/__init__.py create mode 100644 tests/integration/test_async_insert_memory/test.py diff --git a/tests/integration/test_async_insert_memory/__init__.py b/tests/integration/test_async_insert_memory/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_async_insert_memory/test.py b/tests/integration/test_async_insert_memory/test.py new file mode 100644 index 00000000000..cf7b59c8d4a --- /dev/null +++ b/tests/integration/test_async_insert_memory/test.py @@ -0,0 +1,35 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance("node") + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_memory_usage(): + node.query( + "CREATE TABLE async_table(data Array(UInt64)) ENGINE=MergeTree() ORDER BY data" + ) + + response = node.get_query_request( + "SELECT groupArray(number + sleepEachRow(0.0001)) FROM numbers(1000000) SETTINGS max_memory_usage_for_user={}".format(30 * (2 ** 23)) + ) + + INSERT_QUERY = "INSERT INTO async_table SETTINGS async_insert=1, wait_for_async_insert=1 VALUES ({})" + for i in range(10): + node.query(INSERT_QUERY.format([i in range(i * 5000000, (i + 1) * 5000000)])) + + _, err = response.get_answer_and_error() + assert err == "", "Query failed" + + node.query("DROP TABLE async_table") From 573ce5040315ebd8dd82ef90796498280052dc57 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 21 Feb 2023 01:43:01 +0000 Subject: [PATCH 151/253] Fix test --- .../integration/test_async_insert_memory/test.py | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_async_insert_memory/test.py b/tests/integration/test_async_insert_memory/test.py index cf7b59c8d4a..93e3e771c71 100644 --- a/tests/integration/test_async_insert_memory/test.py +++ b/tests/integration/test_async_insert_memory/test.py @@ -21,15 +21,20 @@ def test_memory_usage(): "CREATE TABLE async_table(data Array(UInt64)) ENGINE=MergeTree() ORDER BY data" ) - response = node.get_query_request( - "SELECT groupArray(number + sleepEachRow(0.0001)) FROM numbers(1000000) SETTINGS max_memory_usage_for_user={}".format(30 * (2 ** 23)) + node.get_query_request( + "SELECT count() FROM system.numbers" ) INSERT_QUERY = "INSERT INTO async_table SETTINGS async_insert=1, wait_for_async_insert=1 VALUES ({})" - for i in range(10): - node.query(INSERT_QUERY.format([i in range(i * 5000000, (i + 1) * 5000000)])) + for iter in range(10): + values = list(range(iter * 5000000, (iter + 1) * 5000000)) + node.query(INSERT_QUERY.format(values)) + + response = node.get_query_request( + "SELECT groupArray(number) FROM numbers(1000000) SETTINGS max_memory_usage_for_user={}".format(30 * (2 ** 23)) + ) _, err = response.get_answer_and_error() - assert err == "", "Query failed" + assert err == "", "Query failed with error {}".format(err) node.query("DROP TABLE async_table") From 93aabf8c66006d892583d4a3f5478897b166ce4c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 21 Feb 2023 02:06:16 +0000 Subject: [PATCH 152/253] Automatic style fix --- tests/integration/test_async_insert_memory/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_async_insert_memory/test.py b/tests/integration/test_async_insert_memory/test.py index 93e3e771c71..279542f087c 100644 --- a/tests/integration/test_async_insert_memory/test.py +++ b/tests/integration/test_async_insert_memory/test.py @@ -21,9 +21,7 @@ def test_memory_usage(): "CREATE TABLE async_table(data Array(UInt64)) ENGINE=MergeTree() ORDER BY data" ) - node.get_query_request( - "SELECT count() FROM system.numbers" - ) + node.get_query_request("SELECT count() FROM system.numbers") INSERT_QUERY = "INSERT INTO async_table SETTINGS async_insert=1, wait_for_async_insert=1 VALUES ({})" for iter in range(10): @@ -31,7 +29,9 @@ def test_memory_usage(): node.query(INSERT_QUERY.format(values)) response = node.get_query_request( - "SELECT groupArray(number) FROM numbers(1000000) SETTINGS max_memory_usage_for_user={}".format(30 * (2 ** 23)) + "SELECT groupArray(number) FROM numbers(1000000) SETTINGS max_memory_usage_for_user={}".format( + 30 * (2**23) + ) ) _, err = response.get_answer_and_error() From fcd3e442971c427b6c7557013f0481668d811699 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 21 Feb 2023 10:55:13 +0800 Subject: [PATCH 153/253] finish dev --- src/Functions/JSONArrayLength.cpp | 110 ++++++++++++++++++++++++++++++ 1 file changed, 110 insertions(+) create mode 100644 src/Functions/JSONArrayLength.cpp diff --git a/src/Functions/JSONArrayLength.cpp b/src/Functions/JSONArrayLength.cpp new file mode 100644 index 00000000000..4f784ce140d --- /dev/null +++ b/src/Functions/JSONArrayLength.cpp @@ -0,0 +1,110 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "config.h" + +#if USE_SIMDJSON +# include +#elif USE_RAPIDJSON +# include +#else +# include +#endif + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + +namespace +{ + /// JSONArrayLength(json) + class FunctionJSONArrayLength : public IFunction + { + public: + static constexpr auto name = "JSONArrayLength"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + bool isVariadic() const override { return false; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + size_t getNumberOfArguments() const override { return 1; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + auto args = FunctionArgumentDescriptors{ + {"json", &isString, nullptr, "String"}, + }; + + validateFunctionArgumentTypes(*this, arguments, args); + return std::make_shared(std::make_shared()); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const ColumnPtr column = arguments[0].column; + const ColumnString * col = typeid_cast(column.get()); + if (!col) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "First argument of function {} must be string", getName()); + + auto null_map = ColumnUInt8::create(); + auto data = ColumnUInt64::create(); + null_map->reserve(input_rows_count); + data->reserve(input_rows_count); + +#if USE_SIMDJSON + SimdJSONParser parser; + SimdJSONParser::Element element; +#elif USE_RAPIDJSON + RapidJSONParser parser; + RapidJSONParser::Element element; +#else + DummyJSONParser parser; + DummyJSONParser::Element element; +#endif + + for (size_t i = 0; i < input_rows_count; ++i) + { + auto str_ref = col->getDataAt(i); + std::string_view str_view(str_ref.data, str_ref.size); + bool ok = parser.parse(std::move(str_view), element); + if (!ok || !element.isArray()) + { + null_map->insertValue(1); + data->insertDefault(); + } + else + { + auto array = element.getArray(); + null_map->insertValue(0); + data->insertValue(array.size()); + } + } + return ColumnNullable::create(std::move(data), std::move(null_map)); + } + }; + +} + +REGISTER_FUNCTION(JSONArrayLength) +{ + factory.registerFunction(Documentation{ + "Returns the number of elements in the outermost JSON array. The function returns NULL if input JSON string is invalid."}); + + /// For Spark compatibility. + factory.registerAlias("JSON_ARRAY_LENGTH", "JSONArrayLength", FunctionFactory::CaseInsensitive); +} + +} From 6b5a102a5b12835208007a7f7824dc39dd26dba6 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 21 Feb 2023 11:08:13 +0800 Subject: [PATCH 154/253] add tests --- .../02667_json_array_length.reference | 21 +++++++++++++++++++ .../0_stateless/02667_json_array_length.sql | 13 ++++++++++++ 2 files changed, 34 insertions(+) create mode 100644 tests/queries/0_stateless/02667_json_array_length.reference create mode 100644 tests/queries/0_stateless/02667_json_array_length.sql diff --git a/tests/queries/0_stateless/02667_json_array_length.reference b/tests/queries/0_stateless/02667_json_array_length.reference new file mode 100644 index 00000000000..b86f094d0fb --- /dev/null +++ b/tests/queries/0_stateless/02667_json_array_length.reference @@ -0,0 +1,21 @@ +-- { echoOn } +select json_array_length(null); +\N +select json_array_length(''); +\N +select json_array_length('[]'); +0 +select json_array_length('[1,2,3]'); +3 +select json_array_length('[[1,2],[5,6,7]]'); +2 +select json_array_length('[{"a":123},{"b":"hello"}]'); +2 +select json_array_length('[1,2,3,[33,44],{"key":[2,3,4]}]'); +5 +select json_array_length('{"key":"not a json array"}'); +\N +select json_array_length('[1,2,3,4,5'); +\N +select json_array_length(2); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select json_array_length(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } diff --git a/tests/queries/0_stateless/02667_json_array_length.sql b/tests/queries/0_stateless/02667_json_array_length.sql new file mode 100644 index 00000000000..7ea3dffbca6 --- /dev/null +++ b/tests/queries/0_stateless/02667_json_array_length.sql @@ -0,0 +1,13 @@ +-- { echoOn } +select json_array_length(null); +select json_array_length(''); +select json_array_length('[]'); +select json_array_length('[1,2,3]'); +select json_array_length('[[1,2],[5,6,7]]'); +select json_array_length('[{"a":123},{"b":"hello"}]'); +select json_array_length('[1,2,3,[33,44],{"key":[2,3,4]}]'); +select json_array_length('{"key":"not a json array"}'); +select json_array_length('[1,2,3,4,5'); + +select json_array_length(2); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select json_array_length(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } From eafac0f9a35937cb8715bc8d54809721d2f8a967 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 21 Feb 2023 11:17:44 +0800 Subject: [PATCH 155/253] add docs --- .../sql-reference/functions/json-functions.md | 35 +++++++++++++++++++ .../0_stateless/02667_json_array_length.sql | 25 +++++++------ 2 files changed, 49 insertions(+), 11 deletions(-) diff --git a/docs/en/sql-reference/functions/json-functions.md b/docs/en/sql-reference/functions/json-functions.md index ddea4b5c396..bfe2a541647 100644 --- a/docs/en/sql-reference/functions/json-functions.md +++ b/docs/en/sql-reference/functions/json-functions.md @@ -471,3 +471,38 @@ Result: - [output_format_json_quote_64bit_integers](../../operations/settings/settings.md#session_settings-output_format_json_quote_64bit_integers) - [output_format_json_quote_denormals](../../operations/settings/settings.md#settings-output_format_json_quote_denormals) + + +## JSONArrayLength + +Returns the number of elements in the outermost JSON array. The function returns NULL if input JSON string is invalid. + +**Syntax** + +``` sql +JSONArrayLength(json) +``` + +Alias: `JSON_ARRAY_LENGTH(json)`. + +**Arguments** + +- `json` — [String](../../sql-reference/data-types/string.md) with valid JSON. + +**Returned value** + +- If `json` is a valid JSON array string, returns the number of array elements, otherwise returns NULL. + +Type: [Nullable(UInt64)](../../sql-reference/data-types/int-uint.md). + +**Example** + +``` sql +SELECT + JSONArrayLength(''), + JSONArrayLength('[1,2,3]') + +┌─JSONArrayLength('')─┬─JSONArrayLength('[1,2,3]')─┐ +│ ᴺᵁᴸᴸ │ 3 │ +└─────────────────────┴────────────────────────────┘ +``` diff --git a/tests/queries/0_stateless/02667_json_array_length.sql b/tests/queries/0_stateless/02667_json_array_length.sql index 7ea3dffbca6..4f2127b9c81 100644 --- a/tests/queries/0_stateless/02667_json_array_length.sql +++ b/tests/queries/0_stateless/02667_json_array_length.sql @@ -1,13 +1,16 @@ -- { echoOn } -select json_array_length(null); -select json_array_length(''); -select json_array_length('[]'); -select json_array_length('[1,2,3]'); -select json_array_length('[[1,2],[5,6,7]]'); -select json_array_length('[{"a":123},{"b":"hello"}]'); -select json_array_length('[1,2,3,[33,44],{"key":[2,3,4]}]'); -select json_array_length('{"key":"not a json array"}'); -select json_array_length('[1,2,3,4,5'); +select JSONArrayLength(null); +select JSONArrayLength(''); +select JSONArrayLength('[]'); +select JSONArrayLength('[1,2,3]'); +select JSONArrayLength('[[1,2],[5,6,7]]'); +select JSONArrayLength('[{"a":123},{"b":"hello"}]'); +select JSONArrayLength('[1,2,3,[33,44],{"key":[2,3,4]}]'); +select JSONArrayLength('{"key":"not a json array"}'); +select JSONArrayLength('[1,2,3,4,5'); -select json_array_length(2); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -select json_array_length(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select JSON_ARRAY_LENGTH('[1,2,3,4,5'); +select JSON_ARRAY_LENGTH('[1,2,3,4,5]'); + +select JSONArrayLength(2); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select JSONArrayLength(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } From 3eaad70b16a0e442e1e2e4d8dc8cf495d12ac715 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 21 Feb 2023 12:31:02 +0800 Subject: [PATCH 156/253] fix fast test --- .../02667_json_array_length.reference | 26 +++++++++++-------- 1 file changed, 15 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/02667_json_array_length.reference b/tests/queries/0_stateless/02667_json_array_length.reference index b86f094d0fb..dbe66902f35 100644 --- a/tests/queries/0_stateless/02667_json_array_length.reference +++ b/tests/queries/0_stateless/02667_json_array_length.reference @@ -1,21 +1,25 @@ -- { echoOn } -select json_array_length(null); +select JSONArrayLength(null); \N -select json_array_length(''); +select JSONArrayLength(''); \N -select json_array_length('[]'); +select JSONArrayLength('[]'); 0 -select json_array_length('[1,2,3]'); +select JSONArrayLength('[1,2,3]'); 3 -select json_array_length('[[1,2],[5,6,7]]'); +select JSONArrayLength('[[1,2],[5,6,7]]'); 2 -select json_array_length('[{"a":123},{"b":"hello"}]'); +select JSONArrayLength('[{"a":123},{"b":"hello"}]'); 2 -select json_array_length('[1,2,3,[33,44],{"key":[2,3,4]}]'); +select JSONArrayLength('[1,2,3,[33,44],{"key":[2,3,4]}]'); 5 -select json_array_length('{"key":"not a json array"}'); +select JSONArrayLength('{"key":"not a json array"}'); \N -select json_array_length('[1,2,3,4,5'); +select JSONArrayLength('[1,2,3,4,5'); \N -select json_array_length(2); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -select json_array_length(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select JSON_ARRAY_LENGTH('[1,2,3,4,5'); +\N +select JSON_ARRAY_LENGTH('[1,2,3,4,5]'); +5 +select JSONArrayLength(2); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select JSONArrayLength(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } From 2945125a16a63d07f91c2ff55e057cf273240210 Mon Sep 17 00:00:00 2001 From: Jus <40656180+jus1096@users.noreply.github.com> Date: Tue, 21 Feb 2023 10:00:47 +0400 Subject: [PATCH 157/253] Update docs/ru/sql-reference/functions/other-functions.md Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- docs/ru/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 6e0c4ba146d..f457b54ae28 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -672,7 +672,7 @@ formatReadableTimeDelta(column[, maximum_unit]) **Аргументы** - `column` — Столбец с числовой дельтой времени. -- `maximum_unit` — Опицонально. Максимальная единица измерения для отображения. Допустимые значения: секунды, минуты, часы, дни, месяцы, годы. +- `maximum_unit` — Опциональный параметр. Максимальная единица измерения для отображения. Допустимые значения: секунды, минуты, часы, дни, месяцы, годы. Пример: From d905969006bd0619fc319dab638167185c574fdb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 20 Feb 2023 16:14:35 +0100 Subject: [PATCH 158/253] Fix 01565_reconnect_after_client_error test (wrong expect code for multi-line mode) Signed-off-by: Azat Khuzhin --- .../01565_reconnect_after_client_error.expect | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/tests/queries/0_stateless/01565_reconnect_after_client_error.expect b/tests/queries/0_stateless/01565_reconnect_after_client_error.expect index 255248ba61a..4e710fa8262 100755 --- a/tests/queries/0_stateless/01565_reconnect_after_client_error.expect +++ b/tests/queries/0_stateless/01565_reconnect_after_client_error.expect @@ -24,18 +24,27 @@ spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \ expect "\n:) " send -- "DROP TABLE IF EXISTS t01565;\n" +# NOTE: this is important for -mn mode, you should send "\r" only after reading echoed command +expect "DROP" +send -- "\r" expect "\nOk." expect "\n:)" send -- "CREATE TABLE t01565 (c0 String, c1 Int32) ENGINE = Memory() ;\n" +expect "CREATE" +send -- "\r" expect "\nOk." expect "\n:) " send -- "INSERT INTO t01565(c0, c1) VALUES (\"1\",1) ;\n" +expect "INSERT" +send -- "\r" expect "\nConnected" expect "\n:) " send -- "INSERT INTO t01565(c0, c1) VALUES ('1', 1) ;\n" +expect "INSERT" +send -- "\r" expect "\nOk." expect "\n:) " From 2a0cebfc493ae521cc88c06ca235bc8a0c9b6433 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 20 Feb 2023 19:27:11 +0100 Subject: [PATCH 159/253] Fix timeouts in 01565_reconnect_after_client_error Signed-off-by: Azat Khuzhin --- .../0_stateless/01565_reconnect_after_client_error.expect | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01565_reconnect_after_client_error.expect b/tests/queries/0_stateless/01565_reconnect_after_client_error.expect index 4e710fa8262..143e94ae1f3 100755 --- a/tests/queries/0_stateless/01565_reconnect_after_client_error.expect +++ b/tests/queries/0_stateless/01565_reconnect_after_client_error.expect @@ -15,9 +15,9 @@ match_max 100000 expect_after { # Do not ignore eof from expect - eof { exp_continue } + -i $any_spawn_id eof { exp_continue } # A default timeout action is to do nothing, change it to fail - timeout { exit 1 } + -i $any_spawn_id timeout { exit 1 } } spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion -mn --history_file=$history_file" From e169b36b8891840595085593bd77ef8bc885bdb3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 21 Feb 2023 08:22:23 +0100 Subject: [PATCH 160/253] Replace 01565_reconnect_after_client_error with 01565_query_loop_after_client_error There is no need in explicit reconnect in case of client errors (#19353), so just rewrite the test to ensure that everything works. Signed-off-by: Azat Khuzhin --- ...t_error.expect => 01565_query_loop_after_client_error.expect} | 1 - ...r.reference => 01565_query_loop_after_client_error.reference} | 0 2 files changed, 1 deletion(-) rename tests/queries/0_stateless/{01565_reconnect_after_client_error.expect => 01565_query_loop_after_client_error.expect} (98%) rename tests/queries/0_stateless/{01565_reconnect_after_client_error.reference => 01565_query_loop_after_client_error.reference} (100%) diff --git a/tests/queries/0_stateless/01565_reconnect_after_client_error.expect b/tests/queries/0_stateless/01565_query_loop_after_client_error.expect similarity index 98% rename from tests/queries/0_stateless/01565_reconnect_after_client_error.expect rename to tests/queries/0_stateless/01565_query_loop_after_client_error.expect index 143e94ae1f3..bf701225605 100755 --- a/tests/queries/0_stateless/01565_reconnect_after_client_error.expect +++ b/tests/queries/0_stateless/01565_query_loop_after_client_error.expect @@ -39,7 +39,6 @@ expect "\n:) " send -- "INSERT INTO t01565(c0, c1) VALUES (\"1\",1) ;\n" expect "INSERT" send -- "\r" -expect "\nConnected" expect "\n:) " send -- "INSERT INTO t01565(c0, c1) VALUES ('1', 1) ;\n" diff --git a/tests/queries/0_stateless/01565_reconnect_after_client_error.reference b/tests/queries/0_stateless/01565_query_loop_after_client_error.reference similarity index 100% rename from tests/queries/0_stateless/01565_reconnect_after_client_error.reference rename to tests/queries/0_stateless/01565_query_loop_after_client_error.reference From 367cf9e77cefccbf40b9489ac59c3e64d6449a5b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 20 Feb 2023 16:37:59 +0100 Subject: [PATCH 161/253] Fix 01179_insert_values_semicolon test Back in #19925 a check for reading data after semicolon had been added, but after #40474 it does not work, the test does not show the problem because of timeout does not work without stdin before (a more generic fix for timeouts in expect tests I will submit later). To make this test works, the only type that I can found that will work right now is DateTime64, other types does use peeking, or even if they do, they will fail while parsing the query as SQL expression. Signed-off-by: Azat Khuzhin --- .../0_stateless/01179_insert_values_semicolon.expect | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01179_insert_values_semicolon.expect b/tests/queries/0_stateless/01179_insert_values_semicolon.expect index 9d35941ae40..c0b67de5302 100755 --- a/tests/queries/0_stateless/01179_insert_values_semicolon.expect +++ b/tests/queries/0_stateless/01179_insert_values_semicolon.expect @@ -21,7 +21,7 @@ expect ":) " send -- "DROP TABLE IF EXISTS test_01179\r" expect "Ok." -send -- "CREATE TABLE test_01179 (date DateTime) ENGINE=Memory()\r" +send -- "CREATE TABLE test_01179 (date DateTime64(3)) ENGINE=Memory()\r" expect "Ok." send -- "INSERT INTO test_01179 values ('2020-01-01')\r" @@ -30,11 +30,11 @@ expect "Ok." send -- "INSERT INTO test_01179 values ('2020-01-01'); \r" expect "Ok." -send -- "INSERT INTO test_01179 values ('2020-01-01'); (1) \r" +send -- "INSERT INTO test_01179 values ('2020-01-01 0'); (1) \r" expect "Cannot read data after semicolon" send -- "SELECT date, count() FROM test_01179 GROUP BY date FORMAT TSV\r" -expect "2020-01-01 00:00:00\t3" +expect "2020-01-01 00:00:00.000\t2" send -- "DROP TABLE test_01179\r" expect "Ok." From 5cc183ac39836aca060590eb96a572e46b653058 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 20 Feb 2023 19:27:09 +0100 Subject: [PATCH 162/253] Fix timeouts in 01179_insert_values_semicolon Signed-off-by: Azat Khuzhin --- .../queries/0_stateless/01179_insert_values_semicolon.expect | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01179_insert_values_semicolon.expect b/tests/queries/0_stateless/01179_insert_values_semicolon.expect index c0b67de5302..16c62443856 100755 --- a/tests/queries/0_stateless/01179_insert_values_semicolon.expect +++ b/tests/queries/0_stateless/01179_insert_values_semicolon.expect @@ -10,9 +10,9 @@ set timeout 60 match_max 100000 expect_after { # Do not ignore eof from expect - eof { exp_continue } + -i $any_spawn_id eof { exp_continue } # A default timeout action is to do nothing, change it to fail - timeout { exit 1 } + -i $any_spawn_id timeout { exit 1 } } spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion" From 19d0c929d66d53b6f1ded3fdc7d3b67d7f02511c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 20 Feb 2023 17:26:49 +0100 Subject: [PATCH 163/253] Disable timeout logic for starting clickhouse-server from systemd service After ClickHouse became systemd aware (#43400), it waits not more then TimeoutStartSec (1m30sec by default), while before it simply ensures that the process is there. And likely 1m30sec can be not enough for some cluster, and this will lead to endless restarts. At first I've increased it to 10min, but there was a comment about that this is not enough, and I agree with this. But I'm not sure that using "inifinity" is a good option, but I cannot think of any downsides of this. Signed-off-by: Azat Khuzhin --- packages/clickhouse-server.service | 2 ++ 1 file changed, 2 insertions(+) diff --git a/packages/clickhouse-server.service b/packages/clickhouse-server.service index ace304e0c5e..037be826b97 100644 --- a/packages/clickhouse-server.service +++ b/packages/clickhouse-server.service @@ -17,6 +17,8 @@ User=clickhouse Group=clickhouse Restart=always RestartSec=30 +# Since ClickHouse is systemd aware default 1m30sec may not be enough +TimeoutStartSec=inifinity # %p is resolved to the systemd unit name RuntimeDirectory=%p ExecStart=/usr/bin/clickhouse-server --config=/etc/clickhouse-server/config.xml --pid-file=%t/%p/%p.pid From 593de3416bc7ef61fcbe5d103a65c4a2c614dc6d Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 21 Feb 2023 12:27:08 +0100 Subject: [PATCH 164/253] Fix typo in read prefetch --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index f638bd803f9..f90506b95a2 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -350,7 +350,7 @@ Pipe ReadFromMergeTree::readFromPool( && MergeTreePrefetchedReadPool::checkReadMethodAllowed(reader_settings.read_settings.remote_fs_method)) || (!all_parts_are_local && settings.allow_prefetched_read_pool_for_local_filesystem - && MergeTreePrefetchedReadPool::checkReadMethodAllowed(reader_settings.read_settings.remote_fs_method))) + && MergeTreePrefetchedReadPool::checkReadMethodAllowed(reader_settings.read_settings.local_fs_method))) { pool = std::make_shared( max_streams, From 07e96aafdac867e573184a1c0a73d7cdbd8f38c8 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 20 Feb 2023 18:42:47 +0100 Subject: [PATCH 165/253] Test progress bar while filtering all rows in PREWHERE --- ...gress_when_no_rows_from_prewhere.reference | 0 ...666_progress_when_no_rows_from_prewhere.sh | 23 +++++++++++++++++++ 2 files changed, 23 insertions(+) create mode 100644 tests/queries/0_stateless/02666_progress_when_no_rows_from_prewhere.reference create mode 100755 tests/queries/0_stateless/02666_progress_when_no_rows_from_prewhere.sh diff --git a/tests/queries/0_stateless/02666_progress_when_no_rows_from_prewhere.reference b/tests/queries/0_stateless/02666_progress_when_no_rows_from_prewhere.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02666_progress_when_no_rows_from_prewhere.sh b/tests/queries/0_stateless/02666_progress_when_no_rows_from_prewhere.sh new file mode 100755 index 00000000000..a25d72f357d --- /dev/null +++ b/tests/queries/0_stateless/02666_progress_when_no_rows_from_prewhere.sh @@ -0,0 +1,23 @@ +#!/usr/bin/expect -f +# Tags: no-tsan, no-asan, no-ubsan, no-msan, no-debug, no-fasttest + +log_user 0 +set timeout 60 +match_max 10000000 + +# Run query that filters all rows in PREWHERE +spawn clickhouse-local --progress -m -n --query "CREATE TABLE test_progress(n UInt64) ENGINE=MergeTree ORDER BY tuple() SETTINGS index_granularity=10 AS SELECT number FROM numbers(10000); SELECT count() FROM test_progress PREWHERE sleepEachRow(0.01) OR n > 1000000 SETTINGS max_block_size=10;" + +# Expect that progress is updated +expect { + "10.00 rows," { exit 0 } + "20.00 rows," { exit 0 } + "30.00 rows," { exit 0 } + "40.00 rows," { exit 0 } + "50.00 rows," { exit 0 } + "60.00 rows," { exit 0 } + "70.00 rows," { exit 0 } + "80.00 rows," { exit 0 } + "90.00 rows," { exit 0 } + timeout { exit 1 } +} From 2dfc0d008b5f120a468f91ef66d9fb8b93743d9a Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 21 Feb 2023 11:48:28 +0000 Subject: [PATCH 166/253] Fix: remove redundant sorting optimization + incorrect sorting step removal in case of parent step has more than 1 children --- .../QueryPlan/Optimizations/removeRedundantSorting.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp b/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp index 20d964dcb4f..c7b945b755c 100644 --- a/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp +++ b/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp @@ -188,7 +188,15 @@ private: return false; /// remove sorting - parent_node->children.front() = sorting_node->children.front(); + // parent_node->children.front() = sorting_node->children.front(); + for (auto & child : parent_node->children) + { + if (child == sorting_node) + { + child = sorting_node->children.front(); + break; + } + } /// sorting removed, so need to update sorting traits for upstream steps const DataStream * input_stream = &parent_node->children.front()->step->getOutputStream(); From 1f3be929f7a61d7c74f185e7bbc419a3ab5340af Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 21 Feb 2023 12:50:26 +0100 Subject: [PATCH 167/253] Update ReadFromMergeTree.cpp --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index f90506b95a2..fd3645bf2f0 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -348,7 +348,7 @@ Pipe ReadFromMergeTree::readFromPool( if ((all_parts_are_remote && settings.allow_prefetched_read_pool_for_remote_filesystem && MergeTreePrefetchedReadPool::checkReadMethodAllowed(reader_settings.read_settings.remote_fs_method)) - || (!all_parts_are_local + || (all_parts_are_local && settings.allow_prefetched_read_pool_for_local_filesystem && MergeTreePrefetchedReadPool::checkReadMethodAllowed(reader_settings.read_settings.local_fs_method))) { From b5e6d74d48910212de98b05108ecc8cf95cfc44e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 21 Feb 2023 15:36:23 +0300 Subject: [PATCH 168/253] More interesting settings for Stress Tests (#41534) * Update stress * fix --------- Co-authored-by: Alexey Milovidov --- docker/test/stress/stress | 6 +++++- src/Interpreters/IInterpreter.cpp | 9 ++++++++- src/Interpreters/IInterpreter.h | 2 +- src/Interpreters/InterpreterDropQuery.cpp | 4 ++-- src/Interpreters/InterpreterSelectQuery.cpp | 4 ++-- src/Interpreters/Session.cpp | 2 +- src/Server/HTTPHandler.cpp | 5 +++++ tests/queries/0_stateless/transactions.lib | 4 ++-- 8 files changed, 26 insertions(+), 10 deletions(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 667b5be90a5..310f8609d5a 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -40,12 +40,16 @@ def get_options(i, backward_compatibility_check): client_options.append("join_algorithm='auto'") client_options.append('max_rows_in_join=1000') - if i == 13: + if i % 5 == 1: client_options.append("memory_tracker_fault_probability=0.001") if i % 2 == 1 and not backward_compatibility_check: client_options.append("group_by_use_nulls=1") + if i == 12: # 12 % 3 == 0, so it's Atomic database + client_options.append("implicit_transaction=1") + client_options.append("throw_on_unsupported_query_inside_transaction=0") + if client_options: options.append(" --client-option " + " ".join(client_options)) diff --git a/src/Interpreters/IInterpreter.cpp b/src/Interpreters/IInterpreter.cpp index aff703f79af..148e73e43ce 100644 --- a/src/Interpreters/IInterpreter.cpp +++ b/src/Interpreters/IInterpreter.cpp @@ -30,7 +30,7 @@ void IInterpreter::extendQueryLogElem( extendQueryLogElemImpl(elem, ast, context); } -void IInterpreter::checkStorageSupportsTransactionsIfNeeded(const StoragePtr & storage, ContextPtr context) +void IInterpreter::checkStorageSupportsTransactionsIfNeeded(const StoragePtr & storage, ContextPtr context, bool is_readonly_query) { if (!context->getCurrentTransaction()) return; @@ -41,6 +41,13 @@ void IInterpreter::checkStorageSupportsTransactionsIfNeeded(const StoragePtr & s if (context->getSettingsRef().throw_on_unsupported_query_inside_transaction) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Storage {} (table {}) does not support transactions", storage->getName(), storage->getStorageID().getNameForLogs()); + + /// Do not allow transactions with ReplicatedMergeTree anyway (unless it's a readonly SELECT query) + /// because it may try to process transaction on MergeTreeData-level, + /// but then fail with a logical error or something on StorageReplicatedMergeTree-level. + if (!is_readonly_query && storage->supportsReplication()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "ReplicatedMergeTree (table {}) does not support transactions", + storage->getStorageID().getNameForLogs()); } } diff --git a/src/Interpreters/IInterpreter.h b/src/Interpreters/IInterpreter.h index 74a568c5cba..5290c64387f 100644 --- a/src/Interpreters/IInterpreter.h +++ b/src/Interpreters/IInterpreter.h @@ -39,7 +39,7 @@ public: virtual bool supportsTransactions() const { return false; } /// Helper function for some Interpreters. - static void checkStorageSupportsTransactionsIfNeeded(const StoragePtr & storage, ContextPtr context); + static void checkStorageSupportsTransactionsIfNeeded(const StoragePtr & storage, ContextPtr context, bool is_readonly_query = false); virtual ~IInterpreter() = default; }; diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index e2484a48da4..f4507de5ac7 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -120,10 +120,10 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue auto [database, table] = query.if_exists ? DatabaseCatalog::instance().tryGetDatabaseAndTable(table_id, context_) : DatabaseCatalog::instance().getDatabaseAndTable(table_id, context_); - checkStorageSupportsTransactionsIfNeeded(table, context_); - if (database && table) { + checkStorageSupportsTransactionsIfNeeded(table, context_); + auto & ast_drop_query = query.as(); if (ast_drop_query.is_view && !table->isView()) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 2209635cebf..358bbc83599 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -436,7 +436,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (context->getCurrentTransaction() && context->getSettingsRef().throw_on_unsupported_query_inside_transaction) { if (storage) - checkStorageSupportsTransactionsIfNeeded(storage, context); + checkStorageSupportsTransactionsIfNeeded(storage, context, /* is_readonly_query */ true); for (const auto & table : joined_tables.tablesWithColumns()) { if (table.table.table.empty()) @@ -444,7 +444,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( auto maybe_storage = DatabaseCatalog::instance().tryGetTable({table.table.database, table.table.table}, context); if (!maybe_storage) continue; - checkStorageSupportsTransactionsIfNeeded(storage, context); + checkStorageSupportsTransactionsIfNeeded(storage, context, /* is_readonly_query */ true); } } diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 5c72e24c577..7411050aa2d 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -453,7 +453,7 @@ std::shared_ptr Session::getSessionLog() const ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_to_copy, ClientInfo * client_info_to_move) const { if (!user_id && getClientInfo().interface != ClientInfo::Interface::TCP_INTERSERVER) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Session context must be created after authentication"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Query context must be created after authentication"); /// We can create a query context either from a session context or from a global context. bool from_session_context = static_cast(session_context); diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index ee8612974f0..bea2fe87e6d 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -554,6 +554,11 @@ void HTTPHandler::processQuery( std::string session_check = params.get("session_check", ""); session->makeSessionContext(session_id, session_timeout, session_check == "1"); } + else + { + /// We should create it even if we don't have a session_id + session->makeSessionContext(); + } auto client_info = session->getClientInfo(); auto context = session->makeQueryContext(std::move(client_info)); diff --git a/tests/queries/0_stateless/transactions.lib b/tests/queries/0_stateless/transactions.lib index 521c56754bc..6305caa4db1 100755 --- a/tests/queries/0_stateless/transactions.lib +++ b/tests/queries/0_stateless/transactions.lib @@ -13,7 +13,7 @@ function tx() url_without_session="https://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_HTTPS}/?" url="${url_without_session}session_id=$session&query_id=$query_id&database=$CLICKHOUSE_DATABASE" - ${CLICKHOUSE_CURL} -m 60 -sSk "$url" --data "$query" | sed "s/^/tx$tx_num\t/" + ${CLICKHOUSE_CURL} -m 90 -sSk "$url" --data "$query" | sed "s/^/tx$tx_num\t/" } # Waits for the last query in session to finish @@ -63,7 +63,7 @@ function tx_async() tmp_file_name="${CLICKHOUSE_TMP}/tmp_tx_${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}" # run query asynchronously - ${CLICKHOUSE_CURL} -m 60 -sSk "$url" --data "$query" | sed "s/^/tx$tx_num\t/" & + ${CLICKHOUSE_CURL} -m 90 -sSk "$url" --data "$query" | sed "s/^/tx$tx_num\t/" & query_pid=$! echo -e "$query_id\t$query_pid" >> "$tmp_file_name" } From 6a1621fcad2f7810dcba9765cd4de7382d2e9ff0 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 21 Feb 2023 14:15:01 +0100 Subject: [PATCH 169/253] Fix flaky test 01710_normal_projections --- tests/queries/0_stateless/01710_normal_projections.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01710_normal_projections.sh b/tests/queries/0_stateless/01710_normal_projections.sh index 70e38b3722a..8ee3f41ea28 100755 --- a/tests/queries/0_stateless/01710_normal_projections.sh +++ b/tests/queries/0_stateless/01710_normal_projections.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "CREATE TABLE test_sort_proj (x UInt32, y UInt32, PROJECTION p (SELECT x, y ORDER BY y)) ENGINE = MergeTree ORDER BY x" +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_sort_proj (x UInt32, y UInt32, PROJECTION p (SELECT x, y ORDER BY y)) ENGINE = MergeTree ORDER BY x SETTINGS index_granularity=8192" $CLICKHOUSE_CLIENT -q "insert into test_sort_proj select number, toUInt32(-number - 1) from numbers(100)" echo "select where x < 10" From 254cb1119b4d15a1ba2bb2e6d33a674de133c5b5 Mon Sep 17 00:00:00 2001 From: chen Date: Tue, 21 Feb 2023 21:52:17 +0800 Subject: [PATCH 170/253] Update s3Cluster.md --- docs/en/sql-reference/table-functions/s3Cluster.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/table-functions/s3Cluster.md b/docs/en/sql-reference/table-functions/s3Cluster.md index e77806a3665..9c1f7a66845 100644 --- a/docs/en/sql-reference/table-functions/s3Cluster.md +++ b/docs/en/sql-reference/table-functions/s3Cluster.md @@ -31,13 +31,13 @@ Select the data from all the files in the `/root/data/clickhouse` and `/root/dat ``` sql SELECT * FROM s3Cluster( - 'cluster_simple', - 'http://minio1:9001/root/data/{clickhouse,database}/*', - 'minio', - 'minio123', - 'CSV', - 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') ORDER BY (name, value, polygon -); + 'cluster_simple', + 'http://minio1:9001/root/data/{clickhouse,database}/*', + 'minio', + 'minio123', + 'CSV', + 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))' +) ORDER BY (name, value, polygon); ``` Count the total amount of rows in all files in the cluster `cluster_simple`: From c1e611334a383fceaa70ab0267c9b1c9c8ce6274 Mon Sep 17 00:00:00 2001 From: chen Date: Tue, 21 Feb 2023 21:53:44 +0800 Subject: [PATCH 171/253] Update s3Cluster.md --- docs/en/sql-reference/table-functions/s3Cluster.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/s3Cluster.md b/docs/en/sql-reference/table-functions/s3Cluster.md index 9c1f7a66845..f420a69596c 100644 --- a/docs/en/sql-reference/table-functions/s3Cluster.md +++ b/docs/en/sql-reference/table-functions/s3Cluster.md @@ -32,7 +32,7 @@ Select the data from all the files in the `/root/data/clickhouse` and `/root/dat ``` sql SELECT * FROM s3Cluster( 'cluster_simple', - 'http://minio1:9001/root/data/{clickhouse,database}/*', + 'http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV', From 3df7a10ac7e13c8d7836583e9e0896a797183939 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 21 Feb 2023 16:25:11 +0100 Subject: [PATCH 172/253] Update postgres_utility.py --- tests/integration/helpers/postgres_utility.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index 978b9a98fb4..838c22c8a7c 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -320,11 +320,11 @@ def check_tables_are_synchronized( ) result = instance.query(result_query) - for _ in range(30): + for _ in range(50): if result == expected: break else: - time.sleep(0.5) + time.sleep(1) result = instance.query(result_query) assert result == expected From 1d4352d82af2cc354890018d1e5d715d2968f7a9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 21 Feb 2023 17:01:19 +0100 Subject: [PATCH 173/253] Fix integration test: terminate old version without wait --- .../test_backup_with_other_granularity/test.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_backup_with_other_granularity/test.py b/tests/integration/test_backup_with_other_granularity/test.py index d30c45c3691..f456fae23a8 100644 --- a/tests/integration/test_backup_with_other_granularity/test.py +++ b/tests/integration/test_backup_with_other_granularity/test.py @@ -54,7 +54,8 @@ def test_backup_from_old_version(started_cluster): node1.query("ALTER TABLE source_table FREEZE PARTITION tuple();") - node1.restart_with_latest_version(fix_metadata=True) + # We don't want to wait old outdated version to finish properly, just terminate it + node1.restart_with_latest_version(fix_metadata=True, signal=9) node1.query( "CREATE TABLE dest_table (A Int64, B String, Y String) ENGINE = ReplicatedMergeTree('/test/dest_table1', '1') ORDER BY tuple()" @@ -107,7 +108,8 @@ def test_backup_from_old_version_setting(started_cluster): node2.query("ALTER TABLE source_table FREEZE PARTITION tuple();") - node2.restart_with_latest_version(fix_metadata=True) + # We don't want to wait old outdated version to finish properly, just terminate it + node2.restart_with_latest_version(fix_metadata=True, signal=9) node2.query( "CREATE TABLE dest_table (A Int64, B String, Y String) ENGINE = ReplicatedMergeTree('/test/dest_table2', '1') ORDER BY tuple() SETTINGS enable_mixed_granularity_parts = 1" @@ -163,7 +165,8 @@ def test_backup_from_old_version_config(started_cluster): "1", ) - node3.restart_with_latest_version(callback_onstop=callback, fix_metadata=True) + # We don't want to wait old outdated version to finish properly, just terminate it + node3.restart_with_latest_version(callback_onstop=callback, fix_metadata=True, signal=9) node3.query( "CREATE TABLE dest_table (A Int64, B String, Y String) ENGINE = ReplicatedMergeTree('/test/dest_table3', '1') ORDER BY tuple() SETTINGS enable_mixed_granularity_parts = 1" From af677c7dcd6663cc89fa5756c2bff9c59bbb0d8d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 21 Feb 2023 16:08:13 +0000 Subject: [PATCH 174/253] Automatic style fix --- tests/integration/test_backup_with_other_granularity/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_backup_with_other_granularity/test.py b/tests/integration/test_backup_with_other_granularity/test.py index f456fae23a8..2a82fc71951 100644 --- a/tests/integration/test_backup_with_other_granularity/test.py +++ b/tests/integration/test_backup_with_other_granularity/test.py @@ -166,7 +166,9 @@ def test_backup_from_old_version_config(started_cluster): ) # We don't want to wait old outdated version to finish properly, just terminate it - node3.restart_with_latest_version(callback_onstop=callback, fix_metadata=True, signal=9) + node3.restart_with_latest_version( + callback_onstop=callback, fix_metadata=True, signal=9 + ) node3.query( "CREATE TABLE dest_table (A Int64, B String, Y String) ENGINE = ReplicatedMergeTree('/test/dest_table3', '1') ORDER BY tuple() SETTINGS enable_mixed_granularity_parts = 1" From 97e9df01488b68139d79a58e21a6f12c78cc3335 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 21 Feb 2023 19:10:44 +0300 Subject: [PATCH 175/253] Update stress --- docker/test/stress/stress | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 310f8609d5a..e23d5988918 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -78,7 +78,7 @@ def run_func_test( pipes = [] for i in range(0, len(output_paths)): f = open(output_paths[i], "w") - full_command = "{} {} {} {} {} --stress".format( + full_command = "{} {} {} {} {}".format( cmd, get_options(i, backward_compatibility_check), global_time_limit_option, From b6612d2c18a7a45d2d4a36aa15e56a9933b3ff49 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Tue, 21 Feb 2023 11:24:39 -0500 Subject: [PATCH 176/253] fix anchor link --- docs/en/interfaces/formats.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index e94c6377ae9..992eaa8a49f 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -13,7 +13,7 @@ The supported formats are: | Format | Input | Output | |-------------------------------------------------------------------------------------------|------|--------| | [TabSeparated](#tabseparated) | ✔ | ✔ | -| [TabSeparatedRaw](#tabseparatedraw) | ✔ | ✔ | +| [TabSeparatedRaw](#tabseparatedraw) | ✔ | ✔ | | [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ | | [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ | | [TabSeparatedRawWithNames](#tabseparatedrawwithnames) | ✔ | ✔ | @@ -33,7 +33,7 @@ The supported formats are: | [JSONAsString](#jsonasstring) | ✔ | ✗ | | [JSONStrings](#jsonstrings) | ✔ | ✔ | | [JSONColumns](#jsoncolumns) | ✔ | ✔ | -| [JSONColumnsWithMetadata](#jsoncolumnswithmetadata) | ✔ | ✔ | +| [JSONColumnsWithMetadata](#jsoncolumnsmonoblock)) | ✔ | ✔ | | [JSONCompact](#jsoncompact) | ✔ | ✔ | | [JSONCompactStrings](#jsoncompactstrings) | ✗ | ✔ | | [JSONCompactColumns](#jsoncompactcolumns) | ✔ | ✔ | From cd2ab02f22aefafa2e203230018bda12429a4827 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 21 Feb 2023 17:34:16 +0100 Subject: [PATCH 177/253] Get rid of legacy DocsReleaseChecks --- .github/workflows/docs_release.yml | 118 ---------------- docker/docs/release/Dockerfile | 44 ------ docker/docs/release/run.sh | 12 -- docker/images.json | 4 - docs/tools/.gitignore | 2 - docs/tools/README.md | 1 - docs/tools/build.py | 108 --------------- docs/tools/release.sh | 42 ------ docs/tools/requirements.txt | 1 - .../cancel_and_rerun_workflow_lambda/app.py | 1 - tests/ci/docs_release.py | 126 ------------------ tests/ci/workflow_approve_rerun_lambda/app.py | 1 - 12 files changed, 460 deletions(-) delete mode 100644 .github/workflows/docs_release.yml delete mode 100644 docker/docs/release/Dockerfile delete mode 100644 docker/docs/release/run.sh delete mode 100644 docs/tools/.gitignore delete mode 100644 docs/tools/README.md delete mode 100755 docs/tools/build.py delete mode 100755 docs/tools/release.sh delete mode 100644 docs/tools/requirements.txt delete mode 100644 tests/ci/docs_release.py diff --git a/.github/workflows/docs_release.yml b/.github/workflows/docs_release.yml deleted file mode 100644 index fc4b9d88c3e..00000000000 --- a/.github/workflows/docs_release.yml +++ /dev/null @@ -1,118 +0,0 @@ -name: DocsReleaseChecks - -env: - # Force the stdout and stderr streams to be unbuffered - PYTHONUNBUFFERED: 1 - -concurrency: - group: master-release - cancel-in-progress: true -'on': - push: - branches: - - master - paths: - - '.github/**' - - 'docker/docs/release/**' - - 'docs/**' - - 'utils/list-versions/version_date.tsv' - - 'website/**' - - 'utils/check-style/aspell-ignore/**' - workflow_dispatch: -jobs: - DockerHubPushAarch64: - runs-on: [self-hosted, style-checker-aarch64] - steps: - - name: Check out repository code - uses: ClickHouse/checkout@v1 - with: - clear-repository: true - - name: Images check - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 docker_images_check.py --suffix aarch64 - - name: Upload images files to artifacts - uses: actions/upload-artifact@v3 - with: - name: changed_images_aarch64 - path: ${{ runner.temp }}/docker_images_check/changed_images_aarch64.json - DockerHubPushAmd64: - runs-on: [self-hosted, style-checker] - steps: - - name: Check out repository code - uses: ClickHouse/checkout@v1 - with: - clear-repository: true - - name: Images check - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 docker_images_check.py --suffix amd64 - - name: Upload images files to artifacts - uses: actions/upload-artifact@v3 - with: - name: changed_images_amd64 - path: ${{ runner.temp }}/docker_images_check/changed_images_amd64.json - DockerHubPush: - needs: [DockerHubPushAmd64, DockerHubPushAarch64] - runs-on: [self-hosted, style-checker] - steps: - - name: Check out repository code - uses: ClickHouse/checkout@v1 - with: - clear-repository: true - - name: Download changed aarch64 images - uses: actions/download-artifact@v3 - with: - name: changed_images_aarch64 - path: ${{ runner.temp }} - - name: Download changed amd64 images - uses: actions/download-artifact@v3 - with: - name: changed_images_amd64 - path: ${{ runner.temp }} - - name: Images check - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 docker_manifests_merge.py --suffix amd64 --suffix aarch64 - - name: Upload images files to artifacts - uses: actions/upload-artifact@v3 - with: - name: changed_images - path: ${{ runner.temp }}/changed_images.json - DocsRelease: - needs: DockerHubPush - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - # https://docs.github.com/en/actions/learn-github-actions/workflow-commands-for-github-actions#multiline-strings - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/docs_release - REPO_COPY=${{runner.temp}}/docs_release/ClickHouse - CLOUDFLARE_TOKEN=${{secrets.CLOUDFLARE}} - ROBOT_CLICKHOUSE_SSH_KEY<> /etc/ssh/ssh_known_hosts - -COPY run.sh / - -ENV REPO_PATH=/repo_path -ENV OUTPUT_PATH=/output_path - -CMD ["/bin/bash", "/run.sh"] diff --git a/docker/docs/release/run.sh b/docker/docs/release/run.sh deleted file mode 100644 index e5a9f2101aa..00000000000 --- a/docker/docs/release/run.sh +++ /dev/null @@ -1,12 +0,0 @@ -#!/usr/bin/env bash -set -euo pipefail - -cd "$REPO_PATH/docs/tools" -if ! [ -d venv ]; then - mkdir -p venv - virtualenv -p "$(which python3)" venv - source venv/bin/activate - python3 -m pip install --ignore-installed -r requirements.txt -fi -source venv/bin/activate -./release.sh 2>&1 | tee "$OUTPUT_PATH/output.log" diff --git a/docker/images.json b/docker/images.json index bd63aea24ba..78e7729671e 100644 --- a/docker/images.json +++ b/docker/images.json @@ -146,9 +146,5 @@ "name": "clickhouse/docs-builder", "dependent": [ ] - }, - "docker/docs/release": { - "name": "clickhouse/docs-release", - "dependent": [] } } diff --git a/docs/tools/.gitignore b/docs/tools/.gitignore deleted file mode 100644 index 8d35cb3277f..00000000000 --- a/docs/tools/.gitignore +++ /dev/null @@ -1,2 +0,0 @@ -__pycache__ -*.pyc diff --git a/docs/tools/README.md b/docs/tools/README.md deleted file mode 100644 index c7147a0c850..00000000000 --- a/docs/tools/README.md +++ /dev/null @@ -1 +0,0 @@ -See https://github.com/ClickHouse/clickhouse-docs/blob/main/contrib-writing-guide.md diff --git a/docs/tools/build.py b/docs/tools/build.py deleted file mode 100755 index 5653a9b949d..00000000000 --- a/docs/tools/build.py +++ /dev/null @@ -1,108 +0,0 @@ -#!/usr/bin/env python3 - -from pathlib import Path -import argparse -import logging -import shutil -import sys - -import livereload - - -def write_redirect_html(output_path: Path, to_url: str) -> None: - output_dir = output_path.parent - output_dir.mkdir(parents=True, exist_ok=True) - output_path.write_text( - f""" - - - - - - - Page Redirection - - - If you are not redirected automatically, follow this link. - -""" - ) - - -def build_static_redirects(output_dir: Path): - for static_redirect in [ - ("benchmark.html", "/benchmark/dbms/"), - ("benchmark_hardware.html", "/benchmark/hardware/"), - ( - "tutorial.html", - "/docs/en/getting_started/tutorial/", - ), - ( - "reference_en.html", - "/docs/en/single/", - ), - ( - "reference_ru.html", - "/docs/ru/single/", - ), - ( - "docs/index.html", - "/docs/en/", - ), - ]: - write_redirect_html(output_dir / static_redirect[0], static_redirect[1]) - - -def build(root_dir: Path, output_dir: Path): - if output_dir.exists(): - shutil.rmtree(args.output_dir) - - (output_dir / "data").mkdir(parents=True) - - logging.info("Building website") - - # This file can be requested to check for available ClickHouse releases. - shutil.copy2( - root_dir / "utils" / "list-versions" / "version_date.tsv", - output_dir / "data" / "version_date.tsv", - ) - - # This file can be requested to install ClickHouse. - shutil.copy2( - root_dir / "docs" / "_includes" / "install" / "universal.sh", - output_dir / "data" / "install.sh", - ) - - build_static_redirects(output_dir) - - -if __name__ == "__main__": - root_dir = Path(__file__).parent.parent.parent - docs_dir = root_dir / "docs" - - arg_parser = argparse.ArgumentParser( - formatter_class=argparse.ArgumentDefaultsHelpFormatter, - ) - arg_parser.add_argument( - "--output-dir", - type=Path, - default=docs_dir / "build", - help="path to the output dir", - ) - arg_parser.add_argument("--livereload", type=int, default="0") - arg_parser.add_argument("--verbose", action="store_true") - - args = arg_parser.parse_args() - - logging.basicConfig( - level=logging.DEBUG if args.verbose else logging.INFO, stream=sys.stderr - ) - - build(root_dir, args.output_dir) - - if args.livereload: - server = livereload.Server() - server.serve(root=args.output_dir, host="0.0.0.0", port=args.livereload) - sys.exit(0) diff --git a/docs/tools/release.sh b/docs/tools/release.sh deleted file mode 100755 index c198f488822..00000000000 --- a/docs/tools/release.sh +++ /dev/null @@ -1,42 +0,0 @@ -#!/usr/bin/env bash -set -ex - -BASE_DIR=$(dirname "$(readlink -f "$0")") -BUILD_DIR="${BASE_DIR}/../build" -PUBLISH_DIR="${BASE_DIR}/../publish" -BASE_DOMAIN="${BASE_DOMAIN:-content.clickhouse.com}" -GIT_PROD_URI="${GIT_PROD_URI:-git@github.com:ClickHouse/clickhouse-com-content.git}" -EXTRA_BUILD_ARGS="${EXTRA_BUILD_ARGS:---verbose}" - -if [[ -z "$1" ]] -then - source "${BASE_DIR}/venv/bin/activate" - # shellcheck disable=2086 - python3 "${BASE_DIR}/build.py" ${EXTRA_BUILD_ARGS} - rm -rf "${PUBLISH_DIR}" - mkdir "${PUBLISH_DIR}" && cd "${PUBLISH_DIR}" - - # Will make a repository with website content as the only commit. - git init - git remote add origin "${GIT_PROD_URI}" - git config user.email "robot-clickhouse@users.noreply.github.com" - git config user.name "robot-clickhouse" - - # Add files. - cp -R "${BUILD_DIR}"/* . - echo -n "${BASE_DOMAIN}" > CNAME - cat > README.md << 'EOF' -## This repo is the source for https://content.clickhouse.com -It's built in [the action](https://github.com/ClickHouse/ClickHouse/blob/master/.github/workflows/docs_release.yml) in the DocsRelease job. -EOF - echo -n "" > ".nojekyll" - cp "${BASE_DIR}/../../LICENSE" . - git add ./* - git add ".nojekyll" - - git commit --quiet -m "Add new release at $(date)" - - # Push to GitHub rewriting the existing contents. - # Sometimes it does not work with error message "! [remote rejected] master -> master (cannot lock ref 'refs/heads/master': is at 42a0f6b6b6c7be56a469441b4bf29685c1cebac3 but expected 520e9b02c0d4678a2a5f41d2f561e6532fb98cc1)" - for _ in {1..10}; do git push --force origin master && break; sleep 5; done -fi diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt deleted file mode 100644 index 0e0f7c6d044..00000000000 --- a/docs/tools/requirements.txt +++ /dev/null @@ -1 +0,0 @@ -livereload==2.6.3 diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/app.py b/tests/ci/cancel_and_rerun_workflow_lambda/app.py index 550ab45da55..047b630e241 100644 --- a/tests/ci/cancel_and_rerun_workflow_lambda/app.py +++ b/tests/ci/cancel_and_rerun_workflow_lambda/app.py @@ -20,7 +20,6 @@ NEED_RERUN_ON_EDITED = { } NEED_RERUN_OR_CANCELL_WORKFLOWS = { - "DocsReleaseChecks", "BackportPR", }.union(NEED_RERUN_ON_EDITED) diff --git a/tests/ci/docs_release.py b/tests/ci/docs_release.py deleted file mode 100644 index 1b93aba99ba..00000000000 --- a/tests/ci/docs_release.py +++ /dev/null @@ -1,126 +0,0 @@ -#!/usr/bin/env python3 -import argparse -import logging -import subprocess -import os -import sys - -from github import Github - -from commit_status_helper import get_commit -from docker_pull_helper import get_image_with_version -from env_helper import TEMP_PATH, REPO_COPY, CLOUDFLARE_TOKEN -from get_robot_token import get_best_robot_token -from pr_info import PRInfo -from report import TestResults, TestResult -from rerun_helper import RerunHelper -from s3_helper import S3Helper -from ssh import SSHKey -from tee_popen import TeePopen -from upload_result_helper import upload_results - -NAME = "Docs Release" - - -def parse_args() -> argparse.Namespace: - parser = argparse.ArgumentParser( - formatter_class=argparse.ArgumentDefaultsHelpFormatter, - description="ClickHouse building script using prebuilt Docker image", - ) - parser.add_argument( - "--as-root", action="store_true", help="if the container should run as root" - ) - return parser.parse_args() - - -def main(): - logging.basicConfig(level=logging.INFO) - args = parse_args() - - temp_path = TEMP_PATH - repo_path = REPO_COPY - - gh = Github(get_best_robot_token(), per_page=100) - pr_info = PRInfo() - rerun_helper = RerunHelper(gh, pr_info, NAME) - if rerun_helper.is_already_finished_by_status(): - logging.info("Check is already finished according to github status, exiting") - sys.exit(0) - - if not os.path.exists(temp_path): - os.makedirs(temp_path) - - docker_image = get_image_with_version(temp_path, "clickhouse/docs-release") - - test_output = os.path.join(temp_path, "docs_release_log") - if not os.path.exists(test_output): - os.makedirs(test_output) - - if args.as_root: - user = "0:0" - else: - user = f"{os.geteuid()}:{os.getegid()}" - - run_log_path = os.path.join(test_output, "run.log") - - with SSHKey("ROBOT_CLICKHOUSE_SSH_KEY"): - cmd = ( - f"docker run --cap-add=SYS_PTRACE --user={user} " - f"--volume='{os.getenv('SSH_AUTH_SOCK', '')}:/ssh-agent' " - f"--volume={repo_path}:/repo_path --volume={test_output}:/output_path " - f"-e SSH_AUTH_SOCK=/ssh-agent -e EXTRA_BUILD_ARGS='--verbose' " - f"-e CLOUDFLARE_TOKEN={CLOUDFLARE_TOKEN} {docker_image}" - ) - logging.info("Running command: %s", cmd) - with TeePopen(cmd, run_log_path) as process: - retcode = process.wait() - if retcode == 0: - logging.info("Run successfully") - status = "success" - description = "Released successfuly" - else: - description = "Release failed (non zero exit code)" - status = "failure" - logging.info("Run failed") - - subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) - files = os.listdir(test_output) - test_results = [] # type: TestResults - additional_files = [] - if not files: - logging.error("No output files after docs release") - description = "No output files after docs release" - status = "failure" - else: - for f in files: - path = os.path.join(test_output, f) - additional_files.append(path) - with open(path, "r", encoding="utf-8") as check_file: - for line in check_file: - if "ERROR" in line: - test_results.append(TestResult(line.split(":")[-1], "FAIL")) - if test_results: - status = "failure" - description = "Found errors in docs" - elif status != "failure": - test_results.append(TestResult("No errors found", "OK")) - else: - test_results.append(TestResult("Non zero exit code", "FAIL")) - - s3_helper = S3Helper() - - report_url = upload_results( - s3_helper, pr_info.number, pr_info.sha, test_results, additional_files, NAME - ) - print("::notice ::Report url: {report_url}") - commit = get_commit(gh, pr_info.sha) - commit.create_status( - context=NAME, description=description, state=status, target_url=report_url - ) - - if status == "failure": - sys.exit(1) - - -if __name__ == "__main__": - main() diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index fb14dfd2258..a4a5a013c36 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -62,7 +62,6 @@ TRUSTED_WORKFLOW_IDS = { NEED_RERUN_WORKFLOWS = { "BackportPR", "DocsCheck", - "DocsReleaseChecks", "MasterCI", "NightlyBuilds", "PullRequestCI", From eeac1abaa5532f12e04e3c5cfb15149fb5318663 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 21 Feb 2023 17:49:52 +0100 Subject: [PATCH 178/253] add docs for setting async_insert_max_query_number --- docs/en/operations/settings/settings.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 1d1d8b32d1d..a983970de34 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1561,6 +1561,17 @@ Possible values: Default value: `100000`. +### async_insert_max_query_number {#async-insert-max-query-number} + +The maximum number of insert query in per block before being inserted. It takes effect only if [async_insert_deduplicate](#settings-async-insert-deduplicate) is enabled. + +Possible values: + +- Positive integer. +- 0 — Asynchronous insertions are disabled. + +Default value: `450`. + ### async_insert_busy_timeout_ms {#async-insert-busy-timeout-ms} The maximum timeout in milliseconds since the first `INSERT` query before inserting collected data. From 45b1b66fd8d420db449eedb87b0271dcd53e0b08 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 21 Feb 2023 16:58:45 +0000 Subject: [PATCH 179/253] Remove unnecessary comment --- .../QueryPlan/Optimizations/removeRedundantSorting.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp b/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp index c7b945b755c..41e30dee83e 100644 --- a/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp +++ b/src/Processors/QueryPlan/Optimizations/removeRedundantSorting.cpp @@ -188,7 +188,6 @@ private: return false; /// remove sorting - // parent_node->children.front() = sorting_node->children.front(); for (auto & child : parent_node->children) { if (child == sorting_node) From 6822ad67e2346f6eb21e9dadfaab56299edbab4c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 21 Feb 2023 20:02:12 +0300 Subject: [PATCH 180/253] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 5b5dc4ae528..d2809c1b141 100644 --- a/README.md +++ b/README.md @@ -2,7 +2,7 @@ ClickHouse® is an open-source column-oriented database management system that allows generating analytical data reports in real-time. -## How To Install +## How To Install (Linux, macOS, FreeBSD) ``` curl https://clickhouse.com/ | sh ``` From 9a7c71b78e24e0f3b38a2a3d49a1f446b4a74e7e Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 21 Feb 2023 18:07:57 +0100 Subject: [PATCH 181/253] Allow to hide only values from system.named_collections --- src/Access/Common/AccessType.h | 1 + src/Access/UsersConfigAccessStorage.cpp | 6 ++++++ .../System/StorageSystemNamedCollections.cpp | 7 ++++++- .../configs/users.d/users.xml | 1 + .../integration/test_named_collections/test.py | 18 +++++++++++++++++- 5 files changed, 31 insertions(+), 2 deletions(-) diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 497327c1bad..f57cc2886e3 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -135,6 +135,7 @@ enum class AccessType M(SHOW_SETTINGS_PROFILES, "SHOW PROFILES, SHOW CREATE SETTINGS PROFILE, SHOW CREATE PROFILE", GLOBAL, SHOW_ACCESS) \ M(SHOW_ACCESS, "", GROUP, ACCESS_MANAGEMENT) \ M(SHOW_NAMED_COLLECTIONS, "SHOW NAMED COLLECTIONS", GLOBAL, ACCESS_MANAGEMENT) \ + M(SHOW_NAMED_COLLECTIONS_SECRETS, "SHOW NAMED COLLECTIONS SECRETS", GLOBAL, ACCESS_MANAGEMENT) \ M(ACCESS_MANAGEMENT, "", GROUP, ALL) \ \ M(SYSTEM_SHUTDOWN, "SYSTEM KILL, SHUTDOWN", GLOBAL, SYSTEM) \ diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index 58edff039ca..b893554cb8a 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -239,6 +239,12 @@ namespace user->access.revoke(AccessType::SHOW_NAMED_COLLECTIONS); } + bool show_named_collections_secrets = config.getBool(user_config + ".show_named_collections_secrets", false); + if (!show_named_collections_secrets) + { + user->access.revoke(AccessType::SHOW_NAMED_COLLECTIONS_SECRETS); + } + String default_database = config.getString(user_config + ".default_database", ""); user->default_database = default_database; diff --git a/src/Storages/System/StorageSystemNamedCollections.cpp b/src/Storages/System/StorageSystemNamedCollections.cpp index bc1e3a45e6b..621799c37f2 100644 --- a/src/Storages/System/StorageSystemNamedCollections.cpp +++ b/src/Storages/System/StorageSystemNamedCollections.cpp @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB @@ -30,6 +31,7 @@ StorageSystemNamedCollections::StorageSystemNamedCollections(const StorageID & t void StorageSystemNamedCollections::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { context->checkAccess(AccessType::SHOW_NAMED_COLLECTIONS); + const auto & access = context->getAccess(); auto collections = NamedCollectionFactory::instance().getAll(); for (const auto & [name, collection] : collections) @@ -47,7 +49,10 @@ void StorageSystemNamedCollections::fillData(MutableColumns & res_columns, Conte for (const auto & key : collection->getKeys()) { key_column.insertData(key.data(), key.size()); - value_column.insert(collection->get(key)); + if (access->isGranted(AccessType::SHOW_NAMED_COLLECTIONS_SECRETS)) + value_column.insert(collection->get(key)); + else + value_column.insert("[HIDDEN]"); size++; } diff --git a/tests/integration/test_named_collections/configs/users.d/users.xml b/tests/integration/test_named_collections/configs/users.d/users.xml index fb5e2028d6e..8556e73c82f 100644 --- a/tests/integration/test_named_collections/configs/users.d/users.xml +++ b/tests/integration/test_named_collections/configs/users.d/users.xml @@ -5,6 +5,7 @@ default default 1 + 1 diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index 3b102f1aa70..612b894461b 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -102,7 +102,23 @@ def test_access(cluster): ["bash", "-c", f"cat /etc/clickhouse-server/users.d/users.xml"] ) node.restart_clickhouse() - assert int(node.query("select count() from system.named_collections")) > 0 + assert node.query("select collection['key1'] from system.named_collections").strip() == "value1" + replace_in_users_config( + node, "show_named_collections_secrets>1", "show_named_collections_secrets>0" + ) + assert "show_named_collections_secrets>0" in node.exec_in_container( + ["bash", "-c", f"cat /etc/clickhouse-server/users.d/users.xml"] + ) + node.restart_clickhouse() + assert node.query("select collection['key1'] from system.named_collections").strip() == "[HIDDEN]" + replace_in_users_config( + node, "show_named_collections_secrets>0", "show_named_collections_secrets>1" + ) + assert "show_named_collections_secrets>1" in node.exec_in_container( + ["bash", "-c", f"cat /etc/clickhouse-server/users.d/users.xml"] + ) + node.restart_clickhouse() + assert node.query("select collection['key1'] from system.named_collections").strip() == "value1" def test_config_reload(cluster): From efea3cbc5c6f37bc269f70fb35fb1ce4843ac6a6 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Tue, 21 Feb 2023 12:19:56 -0500 Subject: [PATCH 182/253] Update docs/en/operations/settings/settings.md --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index a983970de34..1060eae1b0e 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1563,7 +1563,7 @@ Default value: `100000`. ### async_insert_max_query_number {#async-insert-max-query-number} -The maximum number of insert query in per block before being inserted. It takes effect only if [async_insert_deduplicate](#settings-async-insert-deduplicate) is enabled. +The maximum number of insert queries per block before being inserted. This setting takes effect only if [async_insert_deduplicate](#settings-async-insert-deduplicate) is enabled. Possible values: From 08b0e3c6309f520cdf1dcc97fd205ba5d4ffbd19 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 21 Feb 2023 18:27:37 +0100 Subject: [PATCH 183/253] Fix style check --- tests/integration/test_named_collections/test.py | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index 612b894461b..ba62880e9de 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -102,7 +102,10 @@ def test_access(cluster): ["bash", "-c", f"cat /etc/clickhouse-server/users.d/users.xml"] ) node.restart_clickhouse() - assert node.query("select collection['key1'] from system.named_collections").strip() == "value1" + assert ( + node.query("select collection['key1'] from system.named_collections").strip() + == "value1" + ) replace_in_users_config( node, "show_named_collections_secrets>1", "show_named_collections_secrets>0" ) @@ -110,7 +113,10 @@ def test_access(cluster): ["bash", "-c", f"cat /etc/clickhouse-server/users.d/users.xml"] ) node.restart_clickhouse() - assert node.query("select collection['key1'] from system.named_collections").strip() == "[HIDDEN]" + assert ( + node.query("select collection['key1'] from system.named_collections").strip() + == "[HIDDEN]" + ) replace_in_users_config( node, "show_named_collections_secrets>0", "show_named_collections_secrets>1" ) @@ -118,7 +124,10 @@ def test_access(cluster): ["bash", "-c", f"cat /etc/clickhouse-server/users.d/users.xml"] ) node.restart_clickhouse() - assert node.query("select collection['key1'] from system.named_collections").strip() == "value1" + assert ( + node.query("select collection['key1'] from system.named_collections").strip() + == "value1" + ) def test_config_reload(cluster): From bf9f1663bb73bd969fcb35242b060c7e55a61024 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 21 Feb 2023 18:15:16 +0000 Subject: [PATCH 184/253] Fix totals and extremes with constants in clickhouse-local --- src/Client/ClientBase.cpp | 4 ++-- .../02556_local_with_totals_and_extremes.reference | 6 ++++++ .../0_stateless/02556_local_with_totals_and_extremes.sh | 8 ++++++++ 3 files changed, 16 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02556_local_with_totals_and_extremes.reference create mode 100755 tests/queries/0_stateless/02556_local_with_totals_and_extremes.sh diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index bc8c43af8c6..96aff9aa304 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -481,14 +481,14 @@ void ClientBase::onLogData(Block & block) void ClientBase::onTotals(Block & block, ASTPtr parsed_query) { initOutputFormat(block, parsed_query); - output_format->setTotals(block); + output_format->setTotals(materializeBlock(block)); } void ClientBase::onExtremes(Block & block, ASTPtr parsed_query) { initOutputFormat(block, parsed_query); - output_format->setExtremes(block); + output_format->setExtremes(materializeBlock(block)); } diff --git a/tests/queries/0_stateless/02556_local_with_totals_and_extremes.reference b/tests/queries/0_stateless/02556_local_with_totals_and_extremes.reference new file mode 100644 index 00000000000..0b9836e530b --- /dev/null +++ b/tests/queries/0_stateless/02556_local_with_totals_and_extremes.reference @@ -0,0 +1,6 @@ +1,1 + +1,1 + +1,1 +1,1 diff --git a/tests/queries/0_stateless/02556_local_with_totals_and_extremes.sh b/tests/queries/0_stateless/02556_local_with_totals_and_extremes.sh new file mode 100755 index 00000000000..ef31b3855cd --- /dev/null +++ b/tests/queries/0_stateless/02556_local_with_totals_and_extremes.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "SELECT 1, sum(1) with totals format CSV settings extremes=1" + From 1f0ab8d427b7819baf13176234ae94acbd9addd7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 21 Feb 2023 19:24:01 +0100 Subject: [PATCH 185/253] Hide disk setting arguemtns --- src/Core/Field.h | 6 +- src/Parsers/ASTSetQuery.cpp | 6 +- src/Parsers/FieldFromAST.cpp | 55 +++++++++++++++++++ src/Parsers/FieldFromAST.h | 4 +- ...54_create_table_with_custom_disk.reference | 2 +- 5 files changed, 67 insertions(+), 6 deletions(-) diff --git a/src/Core/Field.h b/src/Core/Field.h index 95ce43ccd44..2e772a64afc 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -108,7 +108,8 @@ struct CustomType { virtual ~CustomTypeImpl() = default; virtual const char * getTypeName() const = 0; - virtual String toString() const = 0; + virtual String toString(bool show_secrets) const = 0; + virtual bool isSecret() const = 0; virtual bool operator < (const CustomTypeImpl &) const = 0; virtual bool operator <= (const CustomTypeImpl &) const = 0; @@ -120,8 +121,9 @@ struct CustomType CustomType() = default; explicit CustomType(std::shared_ptr impl_) : impl(impl_) {} + bool isSecret() const { return impl->isSecret(); } const char * getTypeName() const { return impl->getTypeName(); } - String toString() const { return impl->toString(); } + String toString(bool show_secrets = true) const { return impl->toString(show_secrets); } const CustomTypeImpl & getImpl() { return *impl; } bool operator < (const CustomType & rhs) const { return *impl < *rhs.impl; } diff --git a/src/Parsers/ASTSetQuery.cpp b/src/Parsers/ASTSetQuery.cpp index 26420f4988c..0b8d76dbb89 100644 --- a/src/Parsers/ASTSetQuery.cpp +++ b/src/Parsers/ASTSetQuery.cpp @@ -34,7 +34,11 @@ void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState &, Forma first = false; formatSettingName(change.name, format.ostr); - format.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value); + CustomType custom; + if (!format.show_secrets && change.value.tryGet(custom) && custom.isSecret()) + format.ostr << " = " << custom.toString(false); + else + format.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value); } for (const auto & setting_name : default_settings) diff --git a/src/Parsers/FieldFromAST.cpp b/src/Parsers/FieldFromAST.cpp index 7b7302696ed..5889699c081 100644 --- a/src/Parsers/FieldFromAST.cpp +++ b/src/Parsers/FieldFromAST.cpp @@ -1,10 +1,18 @@ #include +#include +#include +#include +#include +#include +#include + namespace DB { namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; } Field createFieldFromAST(ASTPtr ast) @@ -17,4 +25,51 @@ Field createFieldFromAST(ASTPtr ast) throw Exception(ErrorCodes::LOGICAL_ERROR, "Method {} not implemented for {}", method, getTypeName()); } +bool FieldFromASTImpl::isSecret() const +{ + return isDiskFunction(ast); +} + +String FieldFromASTImpl::toString(bool show_secrets) const +{ + if (!show_secrets && isDiskFunction(ast)) + { + auto hidden = ast->clone(); + auto & disk_function = assert_cast(*hidden); + auto * disk_function_args_expr = assert_cast(disk_function.arguments.get()); + auto & disk_function_args = disk_function_args_expr->children; + + auto is_secret_arg = [](const std::string & arg_name) + { + return arg_name != "type"; + }; + + for (auto & arg : disk_function_args) + { + auto * setting_function = arg->as(); + if (!setting_function || setting_function->name != "equals") + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad format: expected equals function"); + + auto * function_args_expr = assert_cast(setting_function->arguments.get()); + if (!function_args_expr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad format: expected arguments"); + + auto & function_args = function_args_expr->children; + if (function_args.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad format: expected non zero number of arguments"); + + auto * key_identifier = function_args[0]->as(); + if (!key_identifier) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad format: expected Identifier"); + + const std::string & key = key_identifier->name(); + if (is_secret_arg(key)) + function_args[1] = std::make_shared("[HIDDEN]"); + } + return serializeAST(*hidden); + } + + return serializeAST(*ast); +} + } diff --git a/src/Parsers/FieldFromAST.h b/src/Parsers/FieldFromAST.h index 132f7e3e705..a69c086a170 100644 --- a/src/Parsers/FieldFromAST.h +++ b/src/Parsers/FieldFromAST.h @@ -1,7 +1,6 @@ #pragma once #include #include -#include namespace DB { @@ -13,7 +12,8 @@ struct FieldFromASTImpl : public CustomType::CustomTypeImpl explicit FieldFromASTImpl(ASTPtr ast_) : ast(ast_) {} const char * getTypeName() const override { return name; } - String toString() const override { return serializeAST(*ast); } + String toString(bool show_secrets) const override; + bool isSecret() const override; [[noreturn]] void throwNotImplemented(std::string_view method) const; diff --git a/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference b/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference index 378722b5166..1d8610c59c9 100644 --- a/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference +++ b/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference @@ -6,6 +6,6 @@ ENGINE = MergeTree ORDER BY tuple() SETTINGS disk = disk(type = local, path = \'/var/lib/clickhouse/disks/local/\') 100 -CREATE TABLE default.test\n(\n `a` Int32\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS disk = disk(type = local, path = \'/var/lib/clickhouse/disks/local/\'), index_granularity = 8192 +CREATE TABLE default.test\n(\n `a` Int32\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS disk = disk(type = local, path = \'[HIDDEN]\'), index_granularity = 8192 a Int32 200 From 3ab54ac0adb6c77f0ca6e04dc7f4f0f746b0c337 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 21 Feb 2023 19:24:10 +0100 Subject: [PATCH 186/253] Fix test output --- .../00980_merge_alter_settings.reference | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/00980_merge_alter_settings.reference b/tests/queries/0_stateless/00980_merge_alter_settings.reference index 706b64184ca..a29577f362c 100644 --- a/tests/queries/0_stateless/00980_merge_alter_settings.reference +++ b/tests/queries/0_stateless/00980_merge_alter_settings.reference @@ -1,12 +1,12 @@ -CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi' -CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi', parts_to_throw_insert = 1, parts_to_delay_insert = 1 -CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi', parts_to_throw_insert = 100, parts_to_delay_insert = 100 +CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = \'10Mi\' +CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = \'10Mi\', parts_to_throw_insert = 1, parts_to_delay_insert = 1 +CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = \'10Mi\', parts_to_throw_insert = 100, parts_to_delay_insert = 100 2 -CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi', parts_to_throw_insert = 100, parts_to_delay_insert = 100, check_delay_period = 30 -CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String,\n `Data2` UInt64\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi', parts_to_throw_insert = 100, parts_to_delay_insert = 100, check_delay_period = 15 -CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi' -CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, parts_to_throw_insert = 1, parts_to_delay_insert = 1 -CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi' -CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi' -CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi', merge_with_ttl_timeout = 300, max_concurrent_queries = 1 -CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = '10Mi' +CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = \'10Mi\', parts_to_throw_insert = 100, parts_to_delay_insert = 100, check_delay_period = 30 +CREATE TABLE default.table_for_alter\n(\n `id` UInt64,\n `Data` String,\n `Data2` UInt64\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = \'10Mi\', parts_to_throw_insert = 100, parts_to_delay_insert = 100, check_delay_period = 15 +CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = \'10Mi\' +CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = \'10Mi\', parts_to_throw_insert = 1, parts_to_delay_insert = 1 +CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = \'10Mi\' +CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = \'10Mi\' +CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = \'10Mi\', merge_with_ttl_timeout = 300, max_concurrent_queries = 1 +CREATE TABLE default.table_for_reset_setting\n(\n `id` UInt64,\n `Data` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 4096, index_granularity_bytes = \'10Mi\' From d9cff3a5e8f19b58379830c435ceb04916fd15e4 Mon Sep 17 00:00:00 2001 From: Kevin Zhang Date: Tue, 21 Feb 2023 13:52:28 -0500 Subject: [PATCH 187/253] fix layout issues in dashboard.html --- programs/server/dashboard.html | 22 ++++++++++++++++------ 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index a91040b2701..fa940e01ad5 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -116,21 +116,29 @@ width: 50%; display: flex; - flex-flow: row wrap; + flex-flow: column nowrap; } .unconnected #url { width: 100%; } - .unconnected #user { + #user { + margin-right: 0.25rem; width: 50%; } - .unconnected #password { + #password { width: 49.5%; } .unconnected input { margin-bottom: 5px; } + #username-password { + width: 100%; + + display: flex; + flex-flow: row nowrap; + } + .inputs #chart-params { display: block; } @@ -142,7 +150,7 @@ #connection-params { margin-bottom: 0.5rem; display: grid; - grid-template-columns: auto 15% 15%; + grid-template-columns: 69.77% 30%; column-gap: 0.25rem; } @@ -339,8 +347,10 @@
- - +
+ + +
From dcf8aeab8fa221484836fa017b9d93e30277544b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 21 Feb 2023 22:16:05 +0300 Subject: [PATCH 188/253] Update stress --- docker/test/stress/stress | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index e23d5988918..c142da1be9e 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -46,7 +46,8 @@ def get_options(i, backward_compatibility_check): if i % 2 == 1 and not backward_compatibility_check: client_options.append("group_by_use_nulls=1") - if i == 12: # 12 % 3 == 0, so it's Atomic database + # 12 % 3 == 0, so it's Atomic database + if i == 12 and not backward_compatibility_check: client_options.append("implicit_transaction=1") client_options.append("throw_on_unsupported_query_inside_transaction=0") From cfef911f0d1be82e9a36edce0234e78438fd33c0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 22 Feb 2023 00:32:55 +0300 Subject: [PATCH 189/253] Update 01710_normal_projections.sh --- tests/queries/0_stateless/01710_normal_projections.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01710_normal_projections.sh b/tests/queries/0_stateless/01710_normal_projections.sh index 8ee3f41ea28..3f2114b9a2b 100755 --- a/tests/queries/0_stateless/01710_normal_projections.sh +++ b/tests/queries/0_stateless/01710_normal_projections.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "CREATE TABLE test_sort_proj (x UInt32, y UInt32, PROJECTION p (SELECT x, y ORDER BY y)) ENGINE = MergeTree ORDER BY x SETTINGS index_granularity=8192" +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_sort_proj (x UInt32, y UInt32, PROJECTION p (SELECT x, y ORDER BY y)) ENGINE = MergeTree ORDER BY x SETTINGS index_granularity=8192, index_granularity_bytes='10Mi'" $CLICKHOUSE_CLIENT -q "insert into test_sort_proj select number, toUInt32(-number - 1) from numbers(100)" echo "select where x < 10" From c009c2f4cbf7333407734f92780d16c8a0aed908 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 22 Feb 2023 00:50:21 +0300 Subject: [PATCH 190/253] Update test_ttl_move_memory_usage.py --- .../test_s3_zero_copy_ttl/test_ttl_move_memory_usage.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/integration/test_s3_zero_copy_ttl/test_ttl_move_memory_usage.py b/tests/integration/test_s3_zero_copy_ttl/test_ttl_move_memory_usage.py index 29177b6a67b..5fbe426074f 100644 --- a/tests/integration/test_s3_zero_copy_ttl/test_ttl_move_memory_usage.py +++ b/tests/integration/test_s3_zero_copy_ttl/test_ttl_move_memory_usage.py @@ -2,6 +2,12 @@ import time import pytest + +# FIXME This test is too flaky +# https://github.com/ClickHouse/ClickHouse/issues/45887 + +pytestmark = pytest.mark.skip + from helpers.cluster import ClickHouseCluster From 8232966b9e923f12e711ac4bb036949c61eb87db Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 21 Feb 2023 22:02:23 +0000 Subject: [PATCH 191/253] Add a comment --- src/Interpreters/AsynchronousInsertQueue.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index b8c7d9d285b..4cda5078801 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -91,6 +91,9 @@ private: ~InsertData() { auto it = entries.begin(); + // Entries must be destroyed in context of user who runs async insert. + // Each entry in the list may correspond to a different user, + // so we need to switch current thread's MemoryTracker parent on each iteration. while (it != entries.end()) { UserMemoryTrackerSwitcher switcher((*it)->user_memory_tracker); From ad1e5f391888cc5b1ac02b89fbdb19ecc201859d Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 21 Feb 2023 22:06:17 +0000 Subject: [PATCH 192/253] Review fixes --- src/Common/CurrentThread.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Common/CurrentThread.cpp b/src/Common/CurrentThread.cpp index 95b316671a4..f5cc9658f7a 100644 --- a/src/Common/CurrentThread.cpp +++ b/src/Common/CurrentThread.cpp @@ -115,10 +115,11 @@ MemoryTracker * CurrentThread::getUserMemoryTracker() if (unlikely(!current_thread)) return nullptr; - if (auto group = current_thread->getThreadGroup()) - return group->memory_tracker.getParent(); + auto * tracker = current_thread->memory_tracker.getParent(); + while (tracker && tracker->level != VariableContext::User) + tracker = tracker->getParent(); - return nullptr; + return tracker; } void CurrentThread::flushUntrackedMemory() From 0bf0fe488eb4b6637e863c95c2ed8f07c4509ec8 Mon Sep 17 00:00:00 2001 From: AVMusorin Date: Sun, 19 Feb 2023 21:09:40 +0100 Subject: [PATCH 193/253] added last_exception_time column into distribution_queue table --- src/Storages/Distributed/DirectoryMonitor.cpp | 1 + src/Storages/Distributed/DirectoryMonitor.h | 1 + src/Storages/System/StorageSystemDistributionQueue.cpp | 2 ++ .../0_stateless/01555_system_distribution_queue_mask.reference | 2 +- .../0_stateless/01555_system_distribution_queue_mask.sql | 2 +- .../0_stateless/02117_show_create_table_system.reference | 3 ++- 6 files changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 7aa7aac2ef3..cb6659e59ce 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -465,6 +465,7 @@ void StorageDistributedDirectoryMonitor::run() tryLogCurrentException(getLoggerName().data()); status.last_exception = std::current_exception(); + status.last_exception_time = std::chrono::system_clock::now(); } } else diff --git a/src/Storages/Distributed/DirectoryMonitor.h b/src/Storages/Distributed/DirectoryMonitor.h index 7015fca0311..030d6acf6e2 100644 --- a/src/Storages/Distributed/DirectoryMonitor.h +++ b/src/Storages/Distributed/DirectoryMonitor.h @@ -58,6 +58,7 @@ public: struct InternalStatus { std::exception_ptr last_exception; + std::chrono::system_clock::time_point last_exception_time; size_t error_count = 0; diff --git a/src/Storages/System/StorageSystemDistributionQueue.cpp b/src/Storages/System/StorageSystemDistributionQueue.cpp index 5297c4eb93c..34cff7df65d 100644 --- a/src/Storages/System/StorageSystemDistributionQueue.cpp +++ b/src/Storages/System/StorageSystemDistributionQueue.cpp @@ -101,6 +101,7 @@ NamesAndTypesList StorageSystemDistributionQueue::getNamesAndTypes() { "broken_data_files", std::make_shared() }, { "broken_data_compressed_bytes", std::make_shared() }, { "last_exception", std::make_shared() }, + { "last_exception_time", std::make_shared() }, }; } @@ -190,6 +191,7 @@ void StorageSystemDistributionQueue::fillData(MutableColumns & res_columns, Cont res_columns[col_num++]->insert(getExceptionMessage(status.last_exception, false)); else res_columns[col_num++]->insertDefault(); + res_columns[col_num++]->insert(static_cast(std::chrono::system_clock::to_time_t(status.last_exception_time))); } } } diff --git a/tests/queries/0_stateless/01555_system_distribution_queue_mask.reference b/tests/queries/0_stateless/01555_system_distribution_queue_mask.reference index bd0eac10816..745160a517e 100644 --- a/tests/queries/0_stateless/01555_system_distribution_queue_mask.reference +++ b/tests/queries/0_stateless/01555_system_distribution_queue_mask.reference @@ -1,4 +1,4 @@ masked -3,"default:*@127%2E0%2E0%2E1:9000,default:*@127%2E0%2E0%2E2:9000" +3,"default:*@127%2E0%2E0%2E1:9000,default:*@127%2E0%2E0%2E2:9000","AUTHENTICATION_FAILED",1 no masking 1,"default@localhost:9000" diff --git a/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql b/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql index bdcde1adbad..285e93a4f90 100644 --- a/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql +++ b/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql @@ -18,7 +18,7 @@ create table dist_01555 (key Int) Engine=Distributed(test_cluster_with_incorrect insert into dist_01555 values (1)(2); -- since test_cluster_with_incorrect_pw contains incorrect password ignore error system flush distributed dist_01555; -- { serverError 516; } -select length(splitByChar('*', data_path)), replaceRegexpOne(data_path, '^.*/([^/]*)/' , '\\1') from system.distribution_queue where database = currentDatabase() and table = 'dist_01555' format CSV; +select length(splitByChar('*', data_path)), replaceRegexpOne(data_path, '^.*/([^/]*)/' , '\\1'), extract(last_exception, 'AUTHENTICATION_FAILED'), dateDiff('s', last_exception_time, now()) < 5 from system.distribution_queue where database = currentDatabase() and table = 'dist_01555' format CSV; drop table dist_01555; diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index aabe05ea5e2..1840c5aa5a3 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -229,7 +229,8 @@ CREATE TABLE system.distribution_queue `data_compressed_bytes` UInt64, `broken_data_files` UInt64, `broken_data_compressed_bytes` UInt64, - `last_exception` String + `last_exception` String, + `last_exception_time` DateTime ) ENGINE = SystemDistributionQueue COMMENT 'SYSTEM TABLE is built on the fly.' From ef33d11e3fbc9c8eebf6ed1b2dfd40eac9a32a75 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Tue, 21 Feb 2023 18:40:11 -0800 Subject: [PATCH 194/253] Refactor code according to code review --- src/Functions/FunctionsHashing.h | 24 ++++++++---------------- 1 file changed, 8 insertions(+), 16 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 6bf1a2db3ac..59d573df3d1 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1025,27 +1025,19 @@ private: if constexpr (Impl::use_int_hash_for_pods) { - if constexpr (std::endian::native == std::endian::little) + if constexpr (std::is_same_v) { - if constexpr (std::is_same_v) - h = IntHash64Impl::apply(bit_cast(vec_from[i])); - else - h = IntHash32Impl::apply(bit_cast(vec_from[i])); + UInt64 v = bit_cast(vec_from[i]); + if constexpr (std::endian::native == std::endian::big) + v = __builtin_bswap64(v); + h = IntHash64Impl::apply(v); } else { - if constexpr (std::is_same_v) - { - UInt64 v = bit_cast(vec_from[i]); - v = __builtin_bswap64(v); - h = IntHash64Impl::apply(v); - } - else - { - UInt32 v = bit_cast(vec_from[i]); + UInt32 v = bit_cast(vec_from[i]); + if constexpr (std::endian::native == std::endian::big) v = __builtin_bswap32(v); - h = IntHash32Impl::apply(v); - } + h = IntHash32Impl::apply(v); } } else From 2ca47a6eb60ba886f689122ab6e8b22b2d2bbb84 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 22 Feb 2023 10:41:57 +0100 Subject: [PATCH 195/253] BackgroundSchedulePool should not have any query context BackgroundSchedulePool is used for some peridic jobs, not from the query context, i.e. flush of Buffer table. And for such jobs there cannot be any query context, and more importantly it will not work correctly since that query_context will eventually expires. And this is the reason of this failures [1]. [1]: https://s3.amazonaws.com/clickhouse-test-reports/46668/015991bc5e20c787851050c2eaa13f0fef3aac00/stateless_tests_flaky_check__asan_.html Signed-off-by: Azat Khuzhin --- src/Core/BackgroundSchedulePool.cpp | 28 ------------------- src/Core/BackgroundSchedulePool.h | 5 ---- src/Interpreters/ConcurrentHashJoin.h | 1 - .../MergeTree/MergeTreePrefetchedReadPool.h | 1 - 4 files changed, 35 deletions(-) diff --git a/src/Core/BackgroundSchedulePool.cpp b/src/Core/BackgroundSchedulePool.cpp index 165d8902e85..993cfb6ef04 100644 --- a/src/Core/BackgroundSchedulePool.cpp +++ b/src/Core/BackgroundSchedulePool.cpp @@ -252,36 +252,10 @@ void BackgroundSchedulePool::cancelDelayedTask(const TaskInfoPtr & task, std::lo } -scope_guard BackgroundSchedulePool::attachToThreadGroup() -{ - scope_guard guard = [&]() - { - if (thread_group) - CurrentThread::detachQueryIfNotDetached(); - }; - - std::lock_guard lock(delayed_tasks_mutex); - - if (thread_group) - { - /// Put all threads to one thread pool - CurrentThread::attachTo(thread_group); - } - else - { - CurrentThread::initializeQuery(); - thread_group = CurrentThread::getGroup(); - } - return guard; -} - - void BackgroundSchedulePool::threadFunction() { setThreadName(thread_name.c_str()); - auto detach_thread_guard = attachToThreadGroup(); - while (!shutdown) { TaskInfoPtr task; @@ -311,8 +285,6 @@ void BackgroundSchedulePool::delayExecutionThreadFunction() { setThreadName((thread_name + "/D").c_str()); - auto detach_thread_guard = attachToThreadGroup(); - while (!shutdown) { TaskInfoPtr task; diff --git a/src/Core/BackgroundSchedulePool.h b/src/Core/BackgroundSchedulePool.h index ba1be312f27..0fb70b1f715 100644 --- a/src/Core/BackgroundSchedulePool.h +++ b/src/Core/BackgroundSchedulePool.h @@ -90,13 +90,8 @@ private: /// Tasks ordered by scheduled time. DelayedTasks delayed_tasks; - /// Thread group used for profiling purposes - ThreadGroupStatusPtr thread_group; - CurrentMetrics::Metric tasks_metric; std::string thread_name; - - [[nodiscard]] scope_guard attachToThreadGroup(); }; diff --git a/src/Interpreters/ConcurrentHashJoin.h b/src/Interpreters/ConcurrentHashJoin.h index a00c3ed1326..5e53f9845aa 100644 --- a/src/Interpreters/ConcurrentHashJoin.h +++ b/src/Interpreters/ConcurrentHashJoin.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h index bad158cd7a7..98cfe28c563 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h @@ -4,7 +4,6 @@ #include #include #include -#include #include #include From 16d61832fbb032d42ebbd869488ae2ea32fec9a0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 22 Feb 2023 10:03:08 +0000 Subject: [PATCH 196/253] Bump minimum required Clang from 12 to 15 Needed due to https://github.com/ClickHouse/ClickHouse/pull/46247#discussion_r1109855435 --- cmake/tools.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/tools.cmake b/cmake/tools.cmake index 84376d13d9b..4d4d741cc3a 100644 --- a/cmake/tools.cmake +++ b/cmake/tools.cmake @@ -15,7 +15,7 @@ execute_process(COMMAND ${CMAKE_CXX_COMPILER} --version OUTPUT_VARIABLE COMPILER message (STATUS "Using compiler:\n${COMPILER_SELF_IDENTIFICATION}") # Require minimum compiler versions -set (CLANG_MINIMUM_VERSION 12) +set (CLANG_MINIMUM_VERSION 15) set (XCODE_MINIMUM_VERSION 12.0) set (APPLE_CLANG_MINIMUM_VERSION 12.0.0) set (GCC_MINIMUM_VERSION 11) From a4919ce3a20f93404512561e7e8655ff412df346 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 22 Feb 2023 10:13:39 +0000 Subject: [PATCH 197/253] Add doc for temporary_data_in_cache --- .../settings.md | 90 ++++++++++++++++--- 1 file changed, 80 insertions(+), 10 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index da42b31b78a..75ae6f3d2bc 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1539,33 +1539,103 @@ Example 9005 ``` + ## tmp_path {#tmp-path} -Path to temporary data for processing large queries. +Path on the local filesystem to store temporary data for processing large queries. :::note -The trailing slash is mandatory. +- Only one option can be used to configure temporary data storage: `tmp_path` ,`tmp_policy`, `temporary_data_in_cache`. +- The trailing slash is mandatory. ::: **Example** -``` xml +```xml /var/lib/clickhouse/tmp/ ``` ## tmp_policy {#tmp-policy} -Policy from [storage_configuration](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) to store temporary files. - -If not set, [tmp_path](#tmp-path) is used, otherwise it is ignored. +Alternatively, a policy from [storage_configuration](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) can be used to store temporary files. :::note -- `move_factor` is ignored. -- `keep_free_space_bytes` is ignored. -- `max_data_part_size_bytes` is ignored. -- Policy should have exactly one volume with local disks. +- Only one option can be used to configure temporary data storage: `tmp_path` ,`tmp_policy`, `temporary_data_in_cache`. +- `move_factor`, `keep_free_space_bytes`,`max_data_part_size_bytes` and are ignored. +- Policy should have exactly *one volume* with *local* disks. ::: +**Example** + +```xml + + + + /disk1/ + + + /disk2/ + + + + + + +
+ disk1 + disk2 +
+
+
+
+
+ + tmp_two_disks +
+ +``` + +When `/disk1` is full, temporary data will be stored on `/disk2`. + +## temporary_data_in_cache {#temporary-data-in-cache} + +With this option, temporary data will be stored in the cache for the particular disk. +In this section, you should specify the disk name with the type `cache`. +In that case, the cache and temporary data will share the same space, and the disk cache can be evicted to create temporary data. + +:::note +- Only one option can be used to configure temporary data storage: `tmp_path` ,`tmp_policy`, `temporary_data_in_cache`. +::: + +**Example** + +```xml + + + + + local + /local_disk/ + + + + cache + local_disk + /tiny_local_cache/ + 10M + 1M + 1 + 0 + + + + + tiny_local_cache + +``` + +Cache for `local_disk` and temporary data will be stored in `/tiny_local_cache` on the filesystem, managed by `tiny_local_cache`. + ## max_temporary_data_on_disk_size {#max_temporary_data_on_disk_size} Limit the amount of disk space consumed by temporary files in `tmp_path` for the server. From 678e4250cd40c8849a0c01e1de70545455cf0c06 Mon Sep 17 00:00:00 2001 From: flynn Date: Wed, 22 Feb 2023 18:54:19 +0800 Subject: [PATCH 198/253] Fix incorrect predicate push down with grouping sets (#46151) --- .../PredicateExpressionsOptimizer.cpp | 3 +- src/Processors/QueryPlan/AggregatingStep.h | 2 + .../Optimizations/filterPushDown.cpp | 61 ++++++++++ ...rouping_sets_predicate_push_down.reference | 62 ++++++++++ ..._fix_grouping_sets_predicate_push_down.sql | 109 ++++++++++++++++++ 5 files changed, 236 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.reference create mode 100644 tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.sql diff --git a/src/Interpreters/PredicateExpressionsOptimizer.cpp b/src/Interpreters/PredicateExpressionsOptimizer.cpp index d9ea29fe1d8..6606e64f689 100644 --- a/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -35,7 +35,8 @@ bool PredicateExpressionsOptimizer::optimize(ASTSelectQuery & select_query) if (!enable_optimize_predicate_expression) return false; - if (select_query.having() && (!select_query.group_by_with_cube && !select_query.group_by_with_rollup && !select_query.group_by_with_totals)) + const bool has_incompatible_constructs = select_query.group_by_with_cube || select_query.group_by_with_rollup || select_query.group_by_with_totals || select_query.group_by_with_grouping_sets; + if (select_query.having() && !has_incompatible_constructs) tryMovePredicatesFromHavingToWhere(select_query); if (!select_query.tables() || select_query.tables()->children.empty()) diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index d395e94c58b..5f5557fb204 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -56,6 +56,8 @@ public: const Aggregator::Params & getParams() const { return params; } + const auto & getGroupingSetsParamsList() const { return grouping_sets_params; } + bool inOrder() const { return !sort_description_for_merging.empty(); } bool explicitSortingRequired() const { return explicit_sorting_required_for_aggregation_in_order; } bool isGroupingSets() const { return !grouping_sets_params.empty(); } diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 46fe3055e32..d466c52725f 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -53,6 +53,53 @@ static void checkChildrenSize(QueryPlan::Node * node, size_t child_num) child_num, child->getInputStreams().size(), node->children.size()); } +static bool identifiersIsAmongAllGroupingSets(const GroupingSetsParamsList & grouping_sets_params, const NameSet & identifiers_in_predicate) +{ + for (const auto & grouping_set : grouping_sets_params) + { + for (const auto & identifier : identifiers_in_predicate) + { + if (std::find(grouping_set.used_keys.begin(), grouping_set.used_keys.end(), identifier) == grouping_set.used_keys.end()) + return false; + } + } + return true; +} + +static NameSet findIdentifiersOfNode(const ActionsDAG::Node * node) +{ + NameSet res; + + /// We treat all INPUT as identifier + if (node->type == ActionsDAG::ActionType::INPUT) + { + res.emplace(node->result_name); + return res; + } + + std::queue queue; + queue.push(node); + + while (!queue.empty()) + { + const auto * top = queue.front(); + for (const auto * child : top->children) + { + if (child->type == ActionsDAG::ActionType::INPUT) + { + res.emplace(child->result_name); + } + else + { + /// Only push non INPUT child into the queue + queue.push(child); + } + } + queue.pop(); + } + return res; +} + static ActionsDAGPtr splitFilter(QueryPlan::Node * parent_node, const Names & allowed_inputs, size_t child_idx = 0) { QueryPlan::Node * child_node = parent_node->children.front(); @@ -176,6 +223,20 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (auto * aggregating = typeid_cast(child.get())) { + /// If aggregating is GROUPING SETS, and not all the identifiers exist in all + /// of the grouping sets, we could not push the filter down. + if (aggregating->isGroupingSets()) + { + + const auto & actions = filter->getExpression(); + const auto & filter_node = actions->findInOutputs(filter->getFilterColumnName()); + + auto identifiers_in_predicate = findIdentifiersOfNode(&filter_node); + + if (!identifiersIsAmongAllGroupingSets(aggregating->getGroupingSetsParamsList(), identifiers_in_predicate)) + return 0; + } + const auto & params = aggregating->getParams(); const auto & keys = params.keys; diff --git a/tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.reference b/tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.reference new file mode 100644 index 00000000000..440f668c614 --- /dev/null +++ b/tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.reference @@ -0,0 +1,62 @@ +---Explain Syntax--- +SELECT + day_, + type_1 +FROM +( + SELECT + day_, + if(type_1 = \'\', \'all\', type_1) AS type_1 + FROM + ( + SELECT + day_, + type_1 + FROM test_grouping_sets_predicate + PREWHERE day_ = \'2023-01-05\' + GROUP BY + GROUPING SETS ( + (day_, type_1), + (day_)) + HAVING if(type_1 = \'\', \'all\', type_1) = \'all\' + ) AS t + WHERE type_1 = \'all\' +) +WHERE type_1 = \'all\' + +---Explain Pipeline--- +(Expression) +ExpressionTransform × 2 + (Filter) + FilterTransform × 2 + (Filter) + FilterTransform × 2 + (Filter) + FilterTransform × 2 + (Aggregating) + ExpressionTransform × 2 + AggregatingTransform × 2 + Copy 1 → 2 + (Expression) + ExpressionTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 + +---Result--- +2023-01-05 all + +---Explain Pipeline--- +(Expression) +ExpressionTransform × 2 + (Aggregating) + ExpressionTransform × 2 + AggregatingTransform × 2 + Copy 1 → 2 + (Filter) + FilterTransform + (Filter) + FilterTransform + (Filter) + FilterTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 diff --git a/tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.sql b/tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.sql new file mode 100644 index 00000000000..9a970674890 --- /dev/null +++ b/tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.sql @@ -0,0 +1,109 @@ +DROP TABLE IF EXISTS test_grouping_sets_predicate; + +CREATE TABLE test_grouping_sets_predicate +( + day_ Date, + type_1 String +) +ENGINE=MergeTree +ORDER BY day_; + +INSERT INTO test_grouping_sets_predicate SELECT + toDate('2023-01-05') AS day_, + 'hello, world' +FROM numbers (10); + +SELECT '---Explain Syntax---'; +EXPLAIN SYNTAX +SELECT * +FROM +( + SELECT + day_, + if(type_1 = '', 'all', type_1) AS type_1 + FROM + ( + SELECT + day_, + type_1 + FROM test_grouping_sets_predicate + WHERE day_ = '2023-01-05' + GROUP BY + GROUPING SETS ( + (day_, type_1), + (day_)) + ) AS t +) +WHERE type_1 = 'all'; + +SELECT ''; +SELECT '---Explain Pipeline---'; +EXPLAIN PIPELINE +SELECT * +FROM +( + SELECT + day_, + if(type_1 = '', 'all', type_1) AS type_1 + FROM + ( + SELECT + day_, + type_1 + FROM test_grouping_sets_predicate + WHERE day_ = '2023-01-05' + GROUP BY + GROUPING SETS ( + (day_, type_1), + (day_)) + ) AS t +) +WHERE type_1 = 'all'; + +SELECT ''; +SELECT '---Result---'; +SELECT * +FROM +( + SELECT + day_, + if(type_1 = '', 'all', type_1) AS type_1 + FROM + ( + SELECT + day_, + type_1 + FROM test_grouping_sets_predicate + WHERE day_ = '2023-01-05' + GROUP BY + GROUPING SETS ( + (day_, type_1), + (day_)) + ) AS t +) +WHERE type_1 = 'all'; + +SELECT ''; +SELECT '---Explain Pipeline---'; +EXPLAIN PIPELINE +SELECT * +FROM +( + SELECT + day_, + if(type_1 = '', 'all', type_1) AS type_1 + FROM + ( + SELECT + day_, + type_1 + FROM test_grouping_sets_predicate + GROUP BY + GROUPING SETS ( + (day_, type_1), + (day_)) + ) AS t +) +WHERE day_ = '2023-01-05'; + +DROP TABLE test_grouping_sets_predicate; From 21fcc3b69c6355945617e1a5d77aa57de4694e91 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 22 Feb 2023 12:01:18 +0100 Subject: [PATCH 199/253] Add iceberg doc --- .../table-engines/integrations/deltalake.md | 23 +++++++- .../table-engines/integrations/hudi.md | 23 +++++++- .../table-engines/integrations/iceberg.md | 52 +++++++++++++++++ .../sql-reference/table-functions/iceberg.md | 58 +++++++++++++++++++ 4 files changed, 152 insertions(+), 4 deletions(-) create mode 100644 docs/en/engines/table-engines/integrations/iceberg.md create mode 100644 docs/en/sql-reference/table-functions/iceberg.md diff --git a/docs/en/engines/table-engines/integrations/deltalake.md b/docs/en/engines/table-engines/integrations/deltalake.md index 83526ac944d..64ef7ec4dfc 100644 --- a/docs/en/engines/table-engines/integrations/deltalake.md +++ b/docs/en/engines/table-engines/integrations/deltalake.md @@ -19,7 +19,9 @@ CREATE TABLE deltalake **Engine parameters** - `url` — Bucket url with path to the existing Delta Lake table. -- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3). +- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. + +Engine parameters can be specified using [Named Collections](../operations/settings/named-collections.md) **Example** @@ -27,7 +29,24 @@ CREATE TABLE deltalake CREATE TABLE deltalake ENGINE=DeltaLake('http://mars-doc-test.s3.amazonaws.com/clickhouse-bucket-3/test_table/', 'ABC123', 'Abc+123') ``` +Using named collections: + +``` xml + + + + http://mars-doc-test.s3.amazonaws.com/clickhouse-bucket-3/ + ABC123 + Abc+123 + + + +``` + +```sql +CREATE TABLE iceberg_table ENGINE=DeltaLake(deltalake_conf, filename = 'test_table') +``` + ## See also - [deltaLake table function](../../../sql-reference/table-functions/deltalake.md) - diff --git a/docs/en/engines/table-engines/integrations/hudi.md b/docs/en/engines/table-engines/integrations/hudi.md index 4e335e6c075..eb916b17cf9 100644 --- a/docs/en/engines/table-engines/integrations/hudi.md +++ b/docs/en/engines/table-engines/integrations/hudi.md @@ -19,7 +19,9 @@ CREATE TABLE hudi_table **Engine parameters** - `url` — Bucket url with the path to an existing Hudi table. -- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3). +- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. + +Engine parameters can be specified using [Named Collections](../operations/settings/named-collections.md) **Example** @@ -27,7 +29,24 @@ CREATE TABLE hudi_table CREATE TABLE hudi_table ENGINE=Hudi('http://mars-doc-test.s3.amazonaws.com/clickhouse-bucket-3/test_table/', 'ABC123', 'Abc+123') ``` +Using named collections: + +``` xml + + + + http://mars-doc-test.s3.amazonaws.com/clickhouse-bucket-3/ + ABC123 + Abc+123 + + + +``` + +```sql +CREATE TABLE iceberg_table ENGINE=Hudi(hudi_conf, filename = 'test_table') +``` + ## See also - [hudi table function](/docs/en/sql-reference/table-functions/hudi.md) - diff --git a/docs/en/engines/table-engines/integrations/iceberg.md b/docs/en/engines/table-engines/integrations/iceberg.md new file mode 100644 index 00000000000..33ec5f877bf --- /dev/null +++ b/docs/en/engines/table-engines/integrations/iceberg.md @@ -0,0 +1,52 @@ +--- +slug: /en/engines/table-engines/integrations/iceberg +sidebar_label: Iceberg +--- + +# Iceberg Table Engine + +This engine provides a read-only integration with existing Apache [Iceberg](https://iceberg.apache.org/) tables in Amazon S3. + +## Create Table + +Note that the Iceberg table must already exist in S3, this command does not take DDL parameters to create a new table. + +``` sql +CREATE TABLE iceberg_table + ENGINE = Iceberg(url, [aws_access_key_id, aws_secret_access_key,]) +``` + +**Engine parameters** + +- `url` — url with the path to an existing Iceberg table. +- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. + +Engine parameters can be specified using [Named Collections](../operations/settings/named-collections.md) + +**Example** + +```sql +CREATE TABLE iceberg_table ENGINE=Iceberg('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test') +``` + +Using named collections: + +``` xml + + + + http://test.s3.amazonaws.com/clickhouse-bucket/ + test + test + + + +``` + +```sql +CREATE TABLE iceberg_table ENGINE=Iceberg(iceberg_conf, filename = 'test_table') +``` + +## See also + +- [iceberg table function](/docs/en/sql-reference/table-functions/iceberg.md) diff --git a/docs/en/sql-reference/table-functions/iceberg.md b/docs/en/sql-reference/table-functions/iceberg.md new file mode 100644 index 00000000000..036c1379847 --- /dev/null +++ b/docs/en/sql-reference/table-functions/iceberg.md @@ -0,0 +1,58 @@ +--- +slug: /en/sql-reference/table-functions/iceberg +sidebar_label: Iceberg +--- + +# iceberg Table Function + +Provides a read-only table-like interface to Apache [Iceberg](https://iceberg.apache.org/) tables in Amazon S3. + +## Syntax + +``` sql +iceberg(url [,aws_access_key_id, aws_secret_access_key] [,format] [,structure]) +``` + +## Arguments + +- `url` — Bucket url with the path to an existing Iceberg table in S3. +- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. These parameters are optional. If credentials are not specified, they are used from the ClickHouse configuration. For more information see [Using S3 for Data Storage](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-s3). +- `format` — The [format](/docs/en/interfaces/formats.md/#formats) of the file. By default `Parquet` is used. +- `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. + +Engine parameters can be specified using [Named Collections](../operations/settings/named-collections.md) + +**Returned value** + +A table with the specified structure for reading data in the specified Iceberg table in S3. + +**Example** + +```sql +SELECT * FROM iceberg('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test') +``` + +Using named collections: + +```xml + + + + http://test.s3.amazonaws.com/clickhouse-bucket/ + test + test + auto + auto + + + +``` + +```sql +SELECT * FROM iceberg(iceberg_conf, filename = 'test_table') +DESCRIBE iceberg(iceberg_conf, filename = 'test_table') +``` + +**See Also** + +- [Iceberg engine](/docs/en/engines/table-engines/integrations/iceberg.md) From ef15d6489565a8486815cfd43500ebf6fa0729ed Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 22 Feb 2023 12:11:23 +0100 Subject: [PATCH 200/253] Update docs/en/engines/table-engines/integrations/deltalake.md Co-authored-by: flynn --- docs/en/engines/table-engines/integrations/deltalake.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/deltalake.md b/docs/en/engines/table-engines/integrations/deltalake.md index 64ef7ec4dfc..a2816c7ff57 100644 --- a/docs/en/engines/table-engines/integrations/deltalake.md +++ b/docs/en/engines/table-engines/integrations/deltalake.md @@ -44,7 +44,7 @@ Using named collections: ``` ```sql -CREATE TABLE iceberg_table ENGINE=DeltaLake(deltalake_conf, filename = 'test_table') +CREATE TABLE deltalake ENGINE=DeltaLake(deltalake_conf, filename = 'test_table') ``` ## See also From c242fe3e5e80c2bcc3259c47b8e896be3aa13952 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 22 Feb 2023 12:11:42 +0100 Subject: [PATCH 201/253] Update docs/en/engines/table-engines/integrations/hudi.md Co-authored-by: flynn --- docs/en/engines/table-engines/integrations/hudi.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/hudi.md b/docs/en/engines/table-engines/integrations/hudi.md index eb916b17cf9..6ff998d86d9 100644 --- a/docs/en/engines/table-engines/integrations/hudi.md +++ b/docs/en/engines/table-engines/integrations/hudi.md @@ -44,7 +44,7 @@ Using named collections: ``` ```sql -CREATE TABLE iceberg_table ENGINE=Hudi(hudi_conf, filename = 'test_table') +CREATE TABLE hudi_table ENGINE=Hudi(hudi_conf, filename = 'test_table') ``` ## See also From 98c10ff6e5751404c48106293befb212b126e7d1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 22 Feb 2023 12:16:09 +0100 Subject: [PATCH 202/253] Update docs/en/operations/system-tables/processors_profile_log.md Co-authored-by: Nikita Taranov --- docs/en/operations/system-tables/processors_profile_log.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/operations/system-tables/processors_profile_log.md b/docs/en/operations/system-tables/processors_profile_log.md index 269385deab6..cc917138741 100644 --- a/docs/en/operations/system-tables/processors_profile_log.md +++ b/docs/en/operations/system-tables/processors_profile_log.md @@ -33,6 +33,7 @@ SELECT sleep(1) │ (ReadFromStorage) │ │ SourceFromSingleChunk 0 → 1 │ └─────────────────────────────────┘ + SELECT sleep(1) SETTINGS log_processors_profiles = 1 Query id: feb5ed16-1c24-4227-aa54-78c02b3b27d4 From ab94d6dc1831de374f1530c9c5e78bcc06227133 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 22 Feb 2023 12:16:19 +0100 Subject: [PATCH 203/253] Update docs/en/operations/system-tables/processors_profile_log.md Co-authored-by: Nikita Taranov --- docs/en/operations/system-tables/processors_profile_log.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/operations/system-tables/processors_profile_log.md b/docs/en/operations/system-tables/processors_profile_log.md index cc917138741..a2e7a9ebabd 100644 --- a/docs/en/operations/system-tables/processors_profile_log.md +++ b/docs/en/operations/system-tables/processors_profile_log.md @@ -41,6 +41,7 @@ Query id: feb5ed16-1c24-4227-aa54-78c02b3b27d4 │ 0 │ └──────────┘ 1 rows in set. Elapsed: 1.018 sec. + SELECT name, elapsed_us, From ceff5f41d14653a3c95208bcef963a18902db64d Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 22 Feb 2023 12:23:14 +0100 Subject: [PATCH 204/253] Fix tests --- tests/config/users.d/access_management.xml | 1 + .../configs/users.d/users.xml | 1 + .../test_create_query_constraints/configs/users.xml | 1 + .../test_global_overcommit_tracker/configs/users.xml | 1 + .../test_grant_and_revoke/configs/users.d/users.xml | 1 + .../test_overcommit_tracker/configs/users.d/users.xml | 1 + .../configs/users.d/users.xml | 1 + tests/queries/0_stateless/01271_show_privileges.reference | 1 + .../0_stateless/02117_show_create_table_system.reference | 6 +++--- 9 files changed, 11 insertions(+), 3 deletions(-) diff --git a/tests/config/users.d/access_management.xml b/tests/config/users.d/access_management.xml index 8f4d82805be..f7963cdb7f2 100644 --- a/tests/config/users.d/access_management.xml +++ b/tests/config/users.d/access_management.xml @@ -3,6 +3,7 @@ 1 1 + 1 diff --git a/tests/integration/test_access_control_on_cluster/configs/users.d/users.xml b/tests/integration/test_access_control_on_cluster/configs/users.d/users.xml index fb5e2028d6e..8556e73c82f 100644 --- a/tests/integration/test_access_control_on_cluster/configs/users.d/users.xml +++ b/tests/integration/test_access_control_on_cluster/configs/users.d/users.xml @@ -5,6 +5,7 @@ default default 1 + 1 diff --git a/tests/integration/test_create_query_constraints/configs/users.xml b/tests/integration/test_create_query_constraints/configs/users.xml index fb5e2028d6e..8556e73c82f 100644 --- a/tests/integration/test_create_query_constraints/configs/users.xml +++ b/tests/integration/test_create_query_constraints/configs/users.xml @@ -5,6 +5,7 @@ default default 1 + 1 diff --git a/tests/integration/test_global_overcommit_tracker/configs/users.xml b/tests/integration/test_global_overcommit_tracker/configs/users.xml index fb5e2028d6e..8556e73c82f 100644 --- a/tests/integration/test_global_overcommit_tracker/configs/users.xml +++ b/tests/integration/test_global_overcommit_tracker/configs/users.xml @@ -5,6 +5,7 @@ default default 1 + 1 diff --git a/tests/integration/test_grant_and_revoke/configs/users.d/users.xml b/tests/integration/test_grant_and_revoke/configs/users.d/users.xml index fb5e2028d6e..8556e73c82f 100644 --- a/tests/integration/test_grant_and_revoke/configs/users.d/users.xml +++ b/tests/integration/test_grant_and_revoke/configs/users.d/users.xml @@ -5,6 +5,7 @@ default default 1 + 1 diff --git a/tests/integration/test_overcommit_tracker/configs/users.d/users.xml b/tests/integration/test_overcommit_tracker/configs/users.d/users.xml index fb5e2028d6e..8556e73c82f 100644 --- a/tests/integration/test_overcommit_tracker/configs/users.d/users.xml +++ b/tests/integration/test_overcommit_tracker/configs/users.d/users.xml @@ -5,6 +5,7 @@ default default 1 + 1 diff --git a/tests/integration/test_settings_constraints_distributed/configs/users.d/users.xml b/tests/integration/test_settings_constraints_distributed/configs/users.d/users.xml index fb5e2028d6e..8556e73c82f 100644 --- a/tests/integration/test_settings_constraints_distributed/configs/users.d/users.xml +++ b/tests/integration/test_settings_constraints_distributed/configs/users.d/users.xml @@ -5,6 +5,7 @@ default default 1 + 1 diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 58b1cab6e20..c061eb95a65 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -90,6 +90,7 @@ SHOW QUOTAS ['SHOW CREATE QUOTA'] GLOBAL SHOW ACCESS SHOW SETTINGS PROFILES ['SHOW PROFILES','SHOW CREATE SETTINGS PROFILE','SHOW CREATE PROFILE'] GLOBAL SHOW ACCESS SHOW ACCESS [] \N ACCESS MANAGEMENT SHOW NAMED COLLECTIONS ['SHOW NAMED COLLECTIONS'] GLOBAL ACCESS MANAGEMENT +SHOW NAMED COLLECTIONS SECRETS ['SHOW NAMED COLLECTIONS SECRETS'] GLOBAL ACCESS MANAGEMENT ACCESS MANAGEMENT [] \N ALL SYSTEM SHUTDOWN ['SYSTEM KILL','SHUTDOWN'] GLOBAL SYSTEM SYSTEM DROP DNS CACHE ['SYSTEM DROP DNS','DROP DNS CACHE','DROP DNS'] GLOBAL SYSTEM DROP CACHE diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index aabe05ea5e2..fe93418aa6d 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -288,7 +288,7 @@ CREATE TABLE system.grants ( `user_name` Nullable(String), `role_name` Nullable(String), - `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP QUERY CACHE' = 98, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 99, 'SYSTEM DROP FILESYSTEM CACHE' = 100, 'SYSTEM DROP SCHEMA CACHE' = 101, 'SYSTEM DROP S3 CLIENT CACHE' = 102, 'SYSTEM DROP CACHE' = 103, 'SYSTEM RELOAD CONFIG' = 104, 'SYSTEM RELOAD USERS' = 105, 'SYSTEM RELOAD SYMBOLS' = 106, 'SYSTEM RELOAD DICTIONARY' = 107, 'SYSTEM RELOAD MODEL' = 108, 'SYSTEM RELOAD FUNCTION' = 109, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 110, 'SYSTEM RELOAD' = 111, 'SYSTEM RESTART DISK' = 112, 'SYSTEM MERGES' = 113, 'SYSTEM TTL MERGES' = 114, 'SYSTEM FETCHES' = 115, 'SYSTEM MOVES' = 116, 'SYSTEM DISTRIBUTED SENDS' = 117, 'SYSTEM REPLICATED SENDS' = 118, 'SYSTEM SENDS' = 119, 'SYSTEM REPLICATION QUEUES' = 120, 'SYSTEM DROP REPLICA' = 121, 'SYSTEM SYNC REPLICA' = 122, 'SYSTEM RESTART REPLICA' = 123, 'SYSTEM RESTORE REPLICA' = 124, 'SYSTEM WAIT LOADING PARTS' = 125, 'SYSTEM SYNC DATABASE REPLICA' = 126, 'SYSTEM SYNC TRANSACTION LOG' = 127, 'SYSTEM SYNC FILE CACHE' = 128, 'SYSTEM FLUSH DISTRIBUTED' = 129, 'SYSTEM FLUSH LOGS' = 130, 'SYSTEM FLUSH' = 131, 'SYSTEM THREAD FUZZER' = 132, 'SYSTEM UNFREEZE' = 133, 'SYSTEM' = 134, 'dictGet' = 135, 'addressToLine' = 136, 'addressToLineWithInlines' = 137, 'addressToSymbol' = 138, 'demangle' = 139, 'INTROSPECTION' = 140, 'FILE' = 141, 'URL' = 142, 'REMOTE' = 143, 'MONGO' = 144, 'MEILISEARCH' = 145, 'MYSQL' = 146, 'POSTGRES' = 147, 'SQLITE' = 148, 'ODBC' = 149, 'JDBC' = 150, 'HDFS' = 151, 'S3' = 152, 'HIVE' = 153, 'SOURCES' = 154, 'CLUSTER' = 155, 'ALL' = 156, 'NONE' = 157), + `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'SHOW NAMED COLLECTIONS SECRETS' = 92, 'ACCESS MANAGEMENT' = 93, 'SYSTEM SHUTDOWN' = 94, 'SYSTEM DROP DNS CACHE' = 95, 'SYSTEM DROP MARK CACHE' = 96, 'SYSTEM DROP UNCOMPRESSED CACHE' = 97, 'SYSTEM DROP MMAP CACHE' = 98, 'SYSTEM DROP QUERY CACHE' = 99, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 100, 'SYSTEM DROP FILESYSTEM CACHE' = 101, 'SYSTEM DROP SCHEMA CACHE' = 102, 'SYSTEM DROP S3 CLIENT CACHE' = 103, 'SYSTEM DROP CACHE' = 104, 'SYSTEM RELOAD CONFIG' = 105, 'SYSTEM RELOAD USERS' = 106, 'SYSTEM RELOAD SYMBOLS' = 107, 'SYSTEM RELOAD DICTIONARY' = 108, 'SYSTEM RELOAD MODEL' = 109, 'SYSTEM RELOAD FUNCTION' = 110, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 111, 'SYSTEM RELOAD' = 112, 'SYSTEM RESTART DISK' = 113, 'SYSTEM MERGES' = 114, 'SYSTEM TTL MERGES' = 115, 'SYSTEM FETCHES' = 116, 'SYSTEM MOVES' = 117, 'SYSTEM DISTRIBUTED SENDS' = 118, 'SYSTEM REPLICATED SENDS' = 119, 'SYSTEM SENDS' = 120, 'SYSTEM REPLICATION QUEUES' = 121, 'SYSTEM DROP REPLICA' = 122, 'SYSTEM SYNC REPLICA' = 123, 'SYSTEM RESTART REPLICA' = 124, 'SYSTEM RESTORE REPLICA' = 125, 'SYSTEM WAIT LOADING PARTS' = 126, 'SYSTEM SYNC DATABASE REPLICA' = 127, 'SYSTEM SYNC TRANSACTION LOG' = 128, 'SYSTEM SYNC FILE CACHE' = 129, 'SYSTEM FLUSH DISTRIBUTED' = 130, 'SYSTEM FLUSH LOGS' = 131, 'SYSTEM FLUSH' = 132, 'SYSTEM THREAD FUZZER' = 133, 'SYSTEM UNFREEZE' = 134, 'SYSTEM' = 135, 'dictGet' = 136, 'addressToLine' = 137, 'addressToLineWithInlines' = 138, 'addressToSymbol' = 139, 'demangle' = 140, 'INTROSPECTION' = 141, 'FILE' = 142, 'URL' = 143, 'REMOTE' = 144, 'MONGO' = 145, 'MEILISEARCH' = 146, 'MYSQL' = 147, 'POSTGRES' = 148, 'SQLITE' = 149, 'ODBC' = 150, 'JDBC' = 151, 'HDFS' = 152, 'S3' = 153, 'HIVE' = 154, 'SOURCES' = 155, 'CLUSTER' = 156, 'ALL' = 157, 'NONE' = 158), `database` Nullable(String), `table` Nullable(String), `column` Nullable(String), @@ -569,10 +569,10 @@ ENGINE = SystemPartsColumns COMMENT 'SYSTEM TABLE is built on the fly.' CREATE TABLE system.privileges ( - `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP QUERY CACHE' = 98, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 99, 'SYSTEM DROP FILESYSTEM CACHE' = 100, 'SYSTEM DROP SCHEMA CACHE' = 101, 'SYSTEM DROP S3 CLIENT CACHE' = 102, 'SYSTEM DROP CACHE' = 103, 'SYSTEM RELOAD CONFIG' = 104, 'SYSTEM RELOAD USERS' = 105, 'SYSTEM RELOAD SYMBOLS' = 106, 'SYSTEM RELOAD DICTIONARY' = 107, 'SYSTEM RELOAD MODEL' = 108, 'SYSTEM RELOAD FUNCTION' = 109, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 110, 'SYSTEM RELOAD' = 111, 'SYSTEM RESTART DISK' = 112, 'SYSTEM MERGES' = 113, 'SYSTEM TTL MERGES' = 114, 'SYSTEM FETCHES' = 115, 'SYSTEM MOVES' = 116, 'SYSTEM DISTRIBUTED SENDS' = 117, 'SYSTEM REPLICATED SENDS' = 118, 'SYSTEM SENDS' = 119, 'SYSTEM REPLICATION QUEUES' = 120, 'SYSTEM DROP REPLICA' = 121, 'SYSTEM SYNC REPLICA' = 122, 'SYSTEM RESTART REPLICA' = 123, 'SYSTEM RESTORE REPLICA' = 124, 'SYSTEM WAIT LOADING PARTS' = 125, 'SYSTEM SYNC DATABASE REPLICA' = 126, 'SYSTEM SYNC TRANSACTION LOG' = 127, 'SYSTEM SYNC FILE CACHE' = 128, 'SYSTEM FLUSH DISTRIBUTED' = 129, 'SYSTEM FLUSH LOGS' = 130, 'SYSTEM FLUSH' = 131, 'SYSTEM THREAD FUZZER' = 132, 'SYSTEM UNFREEZE' = 133, 'SYSTEM' = 134, 'dictGet' = 135, 'addressToLine' = 136, 'addressToLineWithInlines' = 137, 'addressToSymbol' = 138, 'demangle' = 139, 'INTROSPECTION' = 140, 'FILE' = 141, 'URL' = 142, 'REMOTE' = 143, 'MONGO' = 144, 'MEILISEARCH' = 145, 'MYSQL' = 146, 'POSTGRES' = 147, 'SQLITE' = 148, 'ODBC' = 149, 'JDBC' = 150, 'HDFS' = 151, 'S3' = 152, 'HIVE' = 153, 'SOURCES' = 154, 'CLUSTER' = 155, 'ALL' = 156, 'NONE' = 157), + `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'SHOW NAMED COLLECTIONS SECRETS' = 92, 'ACCESS MANAGEMENT' = 93, 'SYSTEM SHUTDOWN' = 94, 'SYSTEM DROP DNS CACHE' = 95, 'SYSTEM DROP MARK CACHE' = 96, 'SYSTEM DROP UNCOMPRESSED CACHE' = 97, 'SYSTEM DROP MMAP CACHE' = 98, 'SYSTEM DROP QUERY CACHE' = 99, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 100, 'SYSTEM DROP FILESYSTEM CACHE' = 101, 'SYSTEM DROP SCHEMA CACHE' = 102, 'SYSTEM DROP S3 CLIENT CACHE' = 103, 'SYSTEM DROP CACHE' = 104, 'SYSTEM RELOAD CONFIG' = 105, 'SYSTEM RELOAD USERS' = 106, 'SYSTEM RELOAD SYMBOLS' = 107, 'SYSTEM RELOAD DICTIONARY' = 108, 'SYSTEM RELOAD MODEL' = 109, 'SYSTEM RELOAD FUNCTION' = 110, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 111, 'SYSTEM RELOAD' = 112, 'SYSTEM RESTART DISK' = 113, 'SYSTEM MERGES' = 114, 'SYSTEM TTL MERGES' = 115, 'SYSTEM FETCHES' = 116, 'SYSTEM MOVES' = 117, 'SYSTEM DISTRIBUTED SENDS' = 118, 'SYSTEM REPLICATED SENDS' = 119, 'SYSTEM SENDS' = 120, 'SYSTEM REPLICATION QUEUES' = 121, 'SYSTEM DROP REPLICA' = 122, 'SYSTEM SYNC REPLICA' = 123, 'SYSTEM RESTART REPLICA' = 124, 'SYSTEM RESTORE REPLICA' = 125, 'SYSTEM WAIT LOADING PARTS' = 126, 'SYSTEM SYNC DATABASE REPLICA' = 127, 'SYSTEM SYNC TRANSACTION LOG' = 128, 'SYSTEM SYNC FILE CACHE' = 129, 'SYSTEM FLUSH DISTRIBUTED' = 130, 'SYSTEM FLUSH LOGS' = 131, 'SYSTEM FLUSH' = 132, 'SYSTEM THREAD FUZZER' = 133, 'SYSTEM UNFREEZE' = 134, 'SYSTEM' = 135, 'dictGet' = 136, 'addressToLine' = 137, 'addressToLineWithInlines' = 138, 'addressToSymbol' = 139, 'demangle' = 140, 'INTROSPECTION' = 141, 'FILE' = 142, 'URL' = 143, 'REMOTE' = 144, 'MONGO' = 145, 'MEILISEARCH' = 146, 'MYSQL' = 147, 'POSTGRES' = 148, 'SQLITE' = 149, 'ODBC' = 150, 'JDBC' = 151, 'HDFS' = 152, 'S3' = 153, 'HIVE' = 154, 'SOURCES' = 155, 'CLUSTER' = 156, 'ALL' = 157, 'NONE' = 158), `aliases` Array(String), `level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5)), - `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'ACCESS MANAGEMENT' = 92, 'SYSTEM SHUTDOWN' = 93, 'SYSTEM DROP DNS CACHE' = 94, 'SYSTEM DROP MARK CACHE' = 95, 'SYSTEM DROP UNCOMPRESSED CACHE' = 96, 'SYSTEM DROP MMAP CACHE' = 97, 'SYSTEM DROP QUERY CACHE' = 98, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 99, 'SYSTEM DROP FILESYSTEM CACHE' = 100, 'SYSTEM DROP SCHEMA CACHE' = 101, 'SYSTEM DROP S3 CLIENT CACHE' = 102, 'SYSTEM DROP CACHE' = 103, 'SYSTEM RELOAD CONFIG' = 104, 'SYSTEM RELOAD USERS' = 105, 'SYSTEM RELOAD SYMBOLS' = 106, 'SYSTEM RELOAD DICTIONARY' = 107, 'SYSTEM RELOAD MODEL' = 108, 'SYSTEM RELOAD FUNCTION' = 109, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 110, 'SYSTEM RELOAD' = 111, 'SYSTEM RESTART DISK' = 112, 'SYSTEM MERGES' = 113, 'SYSTEM TTL MERGES' = 114, 'SYSTEM FETCHES' = 115, 'SYSTEM MOVES' = 116, 'SYSTEM DISTRIBUTED SENDS' = 117, 'SYSTEM REPLICATED SENDS' = 118, 'SYSTEM SENDS' = 119, 'SYSTEM REPLICATION QUEUES' = 120, 'SYSTEM DROP REPLICA' = 121, 'SYSTEM SYNC REPLICA' = 122, 'SYSTEM RESTART REPLICA' = 123, 'SYSTEM RESTORE REPLICA' = 124, 'SYSTEM WAIT LOADING PARTS' = 125, 'SYSTEM SYNC DATABASE REPLICA' = 126, 'SYSTEM SYNC TRANSACTION LOG' = 127, 'SYSTEM SYNC FILE CACHE' = 128, 'SYSTEM FLUSH DISTRIBUTED' = 129, 'SYSTEM FLUSH LOGS' = 130, 'SYSTEM FLUSH' = 131, 'SYSTEM THREAD FUZZER' = 132, 'SYSTEM UNFREEZE' = 133, 'SYSTEM' = 134, 'dictGet' = 135, 'addressToLine' = 136, 'addressToLineWithInlines' = 137, 'addressToSymbol' = 138, 'demangle' = 139, 'INTROSPECTION' = 140, 'FILE' = 141, 'URL' = 142, 'REMOTE' = 143, 'MONGO' = 144, 'MEILISEARCH' = 145, 'MYSQL' = 146, 'POSTGRES' = 147, 'SQLITE' = 148, 'ODBC' = 149, 'JDBC' = 150, 'HDFS' = 151, 'S3' = 152, 'HIVE' = 153, 'SOURCES' = 154, 'CLUSTER' = 155, 'ALL' = 156, 'NONE' = 157)) + `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE FUNCTION' = 53, 'CREATE NAMED COLLECTION' = 54, 'CREATE' = 55, 'DROP DATABASE' = 56, 'DROP TABLE' = 57, 'DROP VIEW' = 58, 'DROP DICTIONARY' = 59, 'DROP FUNCTION' = 60, 'DROP NAMED COLLECTION' = 61, 'DROP' = 62, 'TRUNCATE' = 63, 'OPTIMIZE' = 64, 'BACKUP' = 65, 'KILL QUERY' = 66, 'KILL TRANSACTION' = 67, 'MOVE PARTITION BETWEEN SHARDS' = 68, 'CREATE USER' = 69, 'ALTER USER' = 70, 'DROP USER' = 71, 'CREATE ROLE' = 72, 'ALTER ROLE' = 73, 'DROP ROLE' = 74, 'ROLE ADMIN' = 75, 'CREATE ROW POLICY' = 76, 'ALTER ROW POLICY' = 77, 'DROP ROW POLICY' = 78, 'CREATE QUOTA' = 79, 'ALTER QUOTA' = 80, 'DROP QUOTA' = 81, 'CREATE SETTINGS PROFILE' = 82, 'ALTER SETTINGS PROFILE' = 83, 'DROP SETTINGS PROFILE' = 84, 'SHOW USERS' = 85, 'SHOW ROLES' = 86, 'SHOW ROW POLICIES' = 87, 'SHOW QUOTAS' = 88, 'SHOW SETTINGS PROFILES' = 89, 'SHOW ACCESS' = 90, 'SHOW NAMED COLLECTIONS' = 91, 'SHOW NAMED COLLECTIONS SECRETS' = 92, 'ACCESS MANAGEMENT' = 93, 'SYSTEM SHUTDOWN' = 94, 'SYSTEM DROP DNS CACHE' = 95, 'SYSTEM DROP MARK CACHE' = 96, 'SYSTEM DROP UNCOMPRESSED CACHE' = 97, 'SYSTEM DROP MMAP CACHE' = 98, 'SYSTEM DROP QUERY CACHE' = 99, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 100, 'SYSTEM DROP FILESYSTEM CACHE' = 101, 'SYSTEM DROP SCHEMA CACHE' = 102, 'SYSTEM DROP S3 CLIENT CACHE' = 103, 'SYSTEM DROP CACHE' = 104, 'SYSTEM RELOAD CONFIG' = 105, 'SYSTEM RELOAD USERS' = 106, 'SYSTEM RELOAD SYMBOLS' = 107, 'SYSTEM RELOAD DICTIONARY' = 108, 'SYSTEM RELOAD MODEL' = 109, 'SYSTEM RELOAD FUNCTION' = 110, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 111, 'SYSTEM RELOAD' = 112, 'SYSTEM RESTART DISK' = 113, 'SYSTEM MERGES' = 114, 'SYSTEM TTL MERGES' = 115, 'SYSTEM FETCHES' = 116, 'SYSTEM MOVES' = 117, 'SYSTEM DISTRIBUTED SENDS' = 118, 'SYSTEM REPLICATED SENDS' = 119, 'SYSTEM SENDS' = 120, 'SYSTEM REPLICATION QUEUES' = 121, 'SYSTEM DROP REPLICA' = 122, 'SYSTEM SYNC REPLICA' = 123, 'SYSTEM RESTART REPLICA' = 124, 'SYSTEM RESTORE REPLICA' = 125, 'SYSTEM WAIT LOADING PARTS' = 126, 'SYSTEM SYNC DATABASE REPLICA' = 127, 'SYSTEM SYNC TRANSACTION LOG' = 128, 'SYSTEM SYNC FILE CACHE' = 129, 'SYSTEM FLUSH DISTRIBUTED' = 130, 'SYSTEM FLUSH LOGS' = 131, 'SYSTEM FLUSH' = 132, 'SYSTEM THREAD FUZZER' = 133, 'SYSTEM UNFREEZE' = 134, 'SYSTEM' = 135, 'dictGet' = 136, 'addressToLine' = 137, 'addressToLineWithInlines' = 138, 'addressToSymbol' = 139, 'demangle' = 140, 'INTROSPECTION' = 141, 'FILE' = 142, 'URL' = 143, 'REMOTE' = 144, 'MONGO' = 145, 'MEILISEARCH' = 146, 'MYSQL' = 147, 'POSTGRES' = 148, 'SQLITE' = 149, 'ODBC' = 150, 'JDBC' = 151, 'HDFS' = 152, 'S3' = 153, 'HIVE' = 154, 'SOURCES' = 155, 'CLUSTER' = 156, 'ALL' = 157, 'NONE' = 158)) ) ENGINE = SystemPrivileges COMMENT 'SYSTEM TABLE is built on the fly.' From ec8b6c5590ff64b6ab1045a385b61ab04736861b Mon Sep 17 00:00:00 2001 From: lzydmxy <13126752315@163.com> Date: Wed, 22 Feb 2023 19:57:56 +0800 Subject: [PATCH 205/253] add __init__.py for integration test test_move_partition_to_disk_on_cluster --- .../test_move_partition_to_disk_on_cluster/__init__.py | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/integration/test_move_partition_to_disk_on_cluster/__init__.py diff --git a/tests/integration/test_move_partition_to_disk_on_cluster/__init__.py b/tests/integration/test_move_partition_to_disk_on_cluster/__init__.py new file mode 100644 index 00000000000..e69de29bb2d From fba2ec30a2a3d117e95f591fbfc635887e27ed6a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 22 Feb 2023 13:53:43 +0100 Subject: [PATCH 206/253] fix style check --- .../test.py | 34 +++++++++---------- 1 file changed, 16 insertions(+), 18 deletions(-) diff --git a/tests/integration/test_move_partition_to_disk_on_cluster/test.py b/tests/integration/test_move_partition_to_disk_on_cluster/test.py index fe8606bd549..90753fc8ce3 100644 --- a/tests/integration/test_move_partition_to_disk_on_cluster/test.py +++ b/tests/integration/test_move_partition_to_disk_on_cluster/test.py @@ -43,10 +43,10 @@ def test_move_partition_to_disk_on_cluster(start_cluster): for node in [node1, node2]: node.query( sql="CREATE TABLE test_local_table" - "(x UInt64) " - "ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_local_table', '{replica}') " - "ORDER BY tuple()" - "SETTINGS storage_policy = 'jbod_with_external';", + "(x UInt64) " + "ENGINE=ReplicatedMergeTree('/clickhouse/tables/test_local_table', '{replica}') " + "ORDER BY tuple()" + "SETTINGS storage_policy = 'jbod_with_external';", ) node1.query("INSERT INTO test_local_table VALUES (0)") @@ -61,10 +61,10 @@ def test_move_partition_to_disk_on_cluster(start_cluster): for node in [node1, node2]: assert ( - node.query( - "SELECT partition_id, disk_name FROM system.parts WHERE table = 'test_local_table' FORMAT Values" - ) - == "('all','jbod1')" + node.query( + "SELECT partition_id, disk_name FROM system.parts WHERE table = 'test_local_table' FORMAT Values" + ) + == "('all','jbod1')" ) node1.query( @@ -73,10 +73,10 @@ def test_move_partition_to_disk_on_cluster(start_cluster): for node in [node1, node2]: assert ( - node.query( - "SELECT partition_id, disk_name FROM system.parts WHERE table = 'test_local_table' FORMAT Values" - ) - == "('all','external')" + node.query( + "SELECT partition_id, disk_name FROM system.parts WHERE table = 'test_local_table' FORMAT Values" + ) + == "('all','external')" ) node1.query( @@ -85,10 +85,8 @@ def test_move_partition_to_disk_on_cluster(start_cluster): for node in [node1, node2]: assert ( - node.query( - "SELECT partition_id, disk_name FROM system.parts WHERE table = 'test_local_table' FORMAT Values" - ) - == "('all','jbod1')" + node.query( + "SELECT partition_id, disk_name FROM system.parts WHERE table = 'test_local_table' FORMAT Values" + ) + == "('all','jbod1')" ) - - From bac464f89b3fdfe612e721a2fc976146e17dd696 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 22 Feb 2023 13:57:30 +0100 Subject: [PATCH 207/253] Fix --- docs/en/engines/table-engines/integrations/deltalake.md | 2 +- docs/en/engines/table-engines/integrations/hudi.md | 2 +- docs/en/engines/table-engines/integrations/iceberg.md | 2 +- docs/en/sql-reference/table-functions/iceberg.md | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/deltalake.md b/docs/en/engines/table-engines/integrations/deltalake.md index a2816c7ff57..99183ac7308 100644 --- a/docs/en/engines/table-engines/integrations/deltalake.md +++ b/docs/en/engines/table-engines/integrations/deltalake.md @@ -21,7 +21,7 @@ CREATE TABLE deltalake - `url` — Bucket url with path to the existing Delta Lake table. - `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. -Engine parameters can be specified using [Named Collections](../operations/settings/named-collections.md) +Engine parameters can be specified using [Named Collections](../../../operations/named-collections.md) **Example** diff --git a/docs/en/engines/table-engines/integrations/hudi.md b/docs/en/engines/table-engines/integrations/hudi.md index 6ff998d86d9..a14134ecdfa 100644 --- a/docs/en/engines/table-engines/integrations/hudi.md +++ b/docs/en/engines/table-engines/integrations/hudi.md @@ -21,7 +21,7 @@ CREATE TABLE hudi_table - `url` — Bucket url with the path to an existing Hudi table. - `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. -Engine parameters can be specified using [Named Collections](../operations/settings/named-collections.md) +Engine parameters can be specified using [Named Collections](../../../operations/named-collections.md) **Example** diff --git a/docs/en/engines/table-engines/integrations/iceberg.md b/docs/en/engines/table-engines/integrations/iceberg.md index 33ec5f877bf..4322fc6b773 100644 --- a/docs/en/engines/table-engines/integrations/iceberg.md +++ b/docs/en/engines/table-engines/integrations/iceberg.md @@ -21,7 +21,7 @@ CREATE TABLE iceberg_table - `url` — url with the path to an existing Iceberg table. - `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. -Engine parameters can be specified using [Named Collections](../operations/settings/named-collections.md) +Engine parameters can be specified using [Named Collections](../../../operations/named-collections.md) **Example** diff --git a/docs/en/sql-reference/table-functions/iceberg.md b/docs/en/sql-reference/table-functions/iceberg.md index 036c1379847..fda4d274005 100644 --- a/docs/en/sql-reference/table-functions/iceberg.md +++ b/docs/en/sql-reference/table-functions/iceberg.md @@ -20,7 +20,7 @@ iceberg(url [,aws_access_key_id, aws_secret_access_key] [,format] [,structure]) - `format` — The [format](/docs/en/interfaces/formats.md/#formats) of the file. By default `Parquet` is used. - `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. -Engine parameters can be specified using [Named Collections](../operations/settings/named-collections.md) +Engine parameters can be specified using [Named Collections](../../operations/named-collections.md) **Returned value** From c4761d6cc688733124081b87a5fefffd1d7541ce Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 22 Feb 2023 12:14:59 +0100 Subject: [PATCH 208/253] Fix checks --- src/Disks/getOrCreateDiskFromAST.cpp | 10 +--------- src/Disks/getOrCreateDiskFromAST.h | 5 ----- src/Parsers/FieldFromAST.cpp | 1 + src/Parsers/isDiskFunction.cpp | 16 ++++++++++++++++ src/Parsers/isDiskFunction.h | 9 +++++++++ src/Storages/MergeTree/MergeTreeSettings.cpp | 1 + .../integration/test_disk_configuration/test.py | 2 +- 7 files changed, 29 insertions(+), 15 deletions(-) create mode 100644 src/Parsers/isDiskFunction.cpp create mode 100644 src/Parsers/isDiskFunction.h diff --git a/src/Disks/getOrCreateDiskFromAST.cpp b/src/Disks/getOrCreateDiskFromAST.cpp index fc9cd7edbee..997bd2c853f 100644 --- a/src/Disks/getOrCreateDiskFromAST.cpp +++ b/src/Disks/getOrCreateDiskFromAST.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include namespace DB @@ -17,15 +18,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -bool isDiskFunction(ASTPtr ast) -{ - if (!ast) - return false; - - const auto * function = ast->as(); - return function && function->name == "disk" && function->arguments->as(); -} - std::string getOrCreateDiskFromDiskAST(const ASTFunction & function, ContextPtr context) { /// We need a unique name for a created custom disk, but it needs to be the same diff --git a/src/Disks/getOrCreateDiskFromAST.h b/src/Disks/getOrCreateDiskFromAST.h index c1d4bda1a49..7c64707b0bd 100644 --- a/src/Disks/getOrCreateDiskFromAST.h +++ b/src/Disks/getOrCreateDiskFromAST.h @@ -15,9 +15,4 @@ class ASTFunction; */ std::string getOrCreateDiskFromDiskAST(const ASTFunction & function, ContextPtr context); -/* - * Is given ast has form of a disk() function. - */ -bool isDiskFunction(ASTPtr ast); - } diff --git a/src/Parsers/FieldFromAST.cpp b/src/Parsers/FieldFromAST.cpp index 5889699c081..c46a9a08e68 100644 --- a/src/Parsers/FieldFromAST.cpp +++ b/src/Parsers/FieldFromAST.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include diff --git a/src/Parsers/isDiskFunction.cpp b/src/Parsers/isDiskFunction.cpp new file mode 100644 index 00000000000..e60229cb3f7 --- /dev/null +++ b/src/Parsers/isDiskFunction.cpp @@ -0,0 +1,16 @@ +#include +#include + +namespace DB +{ + +bool isDiskFunction(ASTPtr ast) +{ + if (!ast) + return false; + + const auto * function = ast->as(); + return function && function->name == "disk" && function->arguments->as(); +} + +} diff --git a/src/Parsers/isDiskFunction.h b/src/Parsers/isDiskFunction.h new file mode 100644 index 00000000000..97b3c58fa17 --- /dev/null +++ b/src/Parsers/isDiskFunction.h @@ -0,0 +1,9 @@ +#pragma once +#include + +namespace DB +{ + +bool isDiskFunction(ASTPtr ast); + +} diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index e5af0c772ba..e951b8f54cf 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include diff --git a/tests/integration/test_disk_configuration/test.py b/tests/integration/test_disk_configuration/test.py index 60d75e4dac1..34f8bea219f 100644 --- a/tests/integration/test_disk_configuration/test.py +++ b/tests/integration/test_disk_configuration/test.py @@ -262,7 +262,7 @@ def test_merge_tree_custom_disk_setting(start_cluster): ) expected = """ - SETTINGS disk = disk(type = s3, endpoint = \\'http://minio1:9001/root/data2/\\', access_key_id = \\'minio\\', secret_access_key = \\'minio123\\'), index_granularity = 8192 + SETTINGS disk = disk(type = s3, endpoint = \\'[HIDDEN]\\', access_key_id = \\'[HIDDEN]\\', secret_access_key = \\'[HIDDEN]\\'), index_granularity = 8192 """ assert expected.strip() in node1.query(f"SHOW CREATE TABLE {TABLE_NAME}_4").strip() From e433ecc18f896bb61193ea059ed217502d0f2101 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 22 Feb 2023 14:37:55 +0100 Subject: [PATCH 209/253] Better exception message during Tuple JSON deserialization --- src/DataTypes/Serializations/SerializationTuple.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index ce15e099222..2b703a15a9b 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -231,7 +231,15 @@ void SerializationTuple::deserializeTextJSON(IColumn & column, ReadBuffer & istr seen_elements[element_pos] = 1; auto & element_column = extractElementColumn(column, element_pos); - elems[element_pos]->deserializeTextJSON(element_column, istr, settings); + try + { + elems[element_pos]->deserializeTextJSON(element_column, istr, settings); + } + catch (Exception & e) + { + e.addMessage("(while reading the value of nested key " + name + ")"); + throw; + } skipWhitespaceIfAny(istr); ++processed; From 4fd4e77737b4f1aa29898849baf289c7e05b8710 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 22 Feb 2023 13:48:29 +0000 Subject: [PATCH 210/253] Poco: POCO_HAVE_INT64 is always defined --- .../Foundation/include/Poco/BinaryReader.h | 4 +-- .../Foundation/include/Poco/BinaryWriter.h | 4 +-- base/poco/Foundation/include/Poco/ByteOrder.h | 29 ------------------- .../Foundation/include/Poco/NumberFormatter.h | 6 ---- .../Foundation/include/Poco/NumberParser.h | 2 -- .../Foundation/include/Poco/StreamCopier.h | 6 ---- base/poco/Foundation/include/Poco/Token.h | 2 -- base/poco/Foundation/include/Poco/Types.h | 1 - base/poco/Foundation/src/BinaryReader.cpp | 4 +-- base/poco/Foundation/src/BinaryWriter.cpp | 4 +-- base/poco/Foundation/src/NumberFormatter.cpp | 2 -- base/poco/Foundation/src/NumberParser.cpp | 2 -- base/poco/Foundation/src/StreamCopier.cpp | 6 ---- base/poco/Foundation/src/Token.cpp | 2 -- base/poco/JSON/include/Poco/JSON/Handler.h | 2 -- .../JSON/include/Poco/JSON/ParseHandler.h | 4 --- .../JSON/include/Poco/JSON/PrintHandler.h | 2 -- base/poco/JSON/src/PrintHandler.cpp | 2 -- .../include/Poco/Net/HTTPFixedLengthStream.h | 4 --- base/poco/Net/include/Poco/Net/HTTPMessage.h | 2 -- base/poco/Net/src/HTTPClientSession.cpp | 8 ----- base/poco/Net/src/HTTPMessage.cpp | 2 -- base/poco/Net/src/HTTPServerRequestImpl.cpp | 4 --- base/poco/Net/src/HTTPServerResponseImpl.cpp | 8 ----- .../include/Poco/Util/AbstractConfiguration.h | 6 ---- .../Util/include/Poco/Util/WinRegistryKey.h | 2 -- base/poco/Util/src/AbstractConfiguration.cpp | 4 --- 27 files changed, 4 insertions(+), 120 deletions(-) diff --git a/base/poco/Foundation/include/Poco/BinaryReader.h b/base/poco/Foundation/include/Poco/BinaryReader.h index 280724a8a47..4042b507a2f 100644 --- a/base/poco/Foundation/include/Poco/BinaryReader.h +++ b/base/poco/Foundation/include/Poco/BinaryReader.h @@ -76,7 +76,7 @@ public: BinaryReader & operator>>(float & value); BinaryReader & operator>>(double & value); -#if defined(POCO_HAVE_INT64) && !defined(POCO_LONG_IS_64_BIT) +#if !defined(POCO_LONG_IS_64_BIT) BinaryReader & operator>>(Int64 & value); BinaryReader & operator>>(UInt64 & value); #endif @@ -106,12 +106,10 @@ public: /// See BinaryWriter::write7BitEncoded() for a description /// of the compression algorithm. -#if defined(POCO_HAVE_INT64) void read7BitEncoded(UInt64 & value); /// Reads a 64-bit unsigned integer in compressed format. /// See BinaryWriter::write7BitEncoded() for a description /// of the compression algorithm. -#endif void readRaw(std::streamsize length, std::string & value); /// Reads length bytes of raw data into value. diff --git a/base/poco/Foundation/include/Poco/BinaryWriter.h b/base/poco/Foundation/include/Poco/BinaryWriter.h index 30a353a8ff7..aa280d4ccab 100644 --- a/base/poco/Foundation/include/Poco/BinaryWriter.h +++ b/base/poco/Foundation/include/Poco/BinaryWriter.h @@ -81,7 +81,7 @@ public: BinaryWriter & operator<<(float value); BinaryWriter & operator<<(double value); -#if defined(POCO_HAVE_INT64) && !defined(POCO_LONG_IS_64_BIT) +#if !defined(POCO_LONG_IS_64_BIT) BinaryWriter & operator<<(Int64 value); BinaryWriter & operator<<(UInt64 value); #endif @@ -114,7 +114,6 @@ public: /// written out. value is then shifted by seven bits and the next byte is written. /// This process is repeated until the entire integer has been written. -#if defined(POCO_HAVE_INT64) void write7BitEncoded(UInt64 value); /// Writes a 64-bit unsigned integer in a compressed format. /// The value written out seven bits at a time, starting @@ -125,7 +124,6 @@ public: /// If value will not fit in seven bits, the high bit is set on the first byte and /// written out. value is then shifted by seven bits and the next byte is written. /// This process is repeated until the entire integer has been written. -#endif void writeRaw(const std::string & rawData); /// Writes the string as-is to the stream. diff --git a/base/poco/Foundation/include/Poco/ByteOrder.h b/base/poco/Foundation/include/Poco/ByteOrder.h index 4f2644ddf4e..09f673c2718 100644 --- a/base/poco/Foundation/include/Poco/ByteOrder.h +++ b/base/poco/Foundation/include/Poco/ByteOrder.h @@ -34,64 +34,50 @@ public: static UInt16 flipBytes(UInt16 value); static Int32 flipBytes(Int32 value); static UInt32 flipBytes(UInt32 value); -#if defined(POCO_HAVE_INT64) static Int64 flipBytes(Int64 value); static UInt64 flipBytes(UInt64 value); -#endif static Int16 toBigEndian(Int16 value); static UInt16 toBigEndian(UInt16 value); static Int32 toBigEndian(Int32 value); static UInt32 toBigEndian(UInt32 value); -#if defined(POCO_HAVE_INT64) static Int64 toBigEndian(Int64 value); static UInt64 toBigEndian(UInt64 value); -#endif static Int16 fromBigEndian(Int16 value); static UInt16 fromBigEndian(UInt16 value); static Int32 fromBigEndian(Int32 value); static UInt32 fromBigEndian(UInt32 value); -#if defined(POCO_HAVE_INT64) static Int64 fromBigEndian(Int64 value); static UInt64 fromBigEndian(UInt64 value); -#endif static Int16 toLittleEndian(Int16 value); static UInt16 toLittleEndian(UInt16 value); static Int32 toLittleEndian(Int32 value); static UInt32 toLittleEndian(UInt32 value); -#if defined(POCO_HAVE_INT64) static Int64 toLittleEndian(Int64 value); static UInt64 toLittleEndian(UInt64 value); -#endif static Int16 fromLittleEndian(Int16 value); static UInt16 fromLittleEndian(UInt16 value); static Int32 fromLittleEndian(Int32 value); static UInt32 fromLittleEndian(UInt32 value); -#if defined(POCO_HAVE_INT64) static Int64 fromLittleEndian(Int64 value); static UInt64 fromLittleEndian(UInt64 value); -#endif static Int16 toNetwork(Int16 value); static UInt16 toNetwork(UInt16 value); static Int32 toNetwork(Int32 value); static UInt32 toNetwork(UInt32 value); -#if defined(POCO_HAVE_INT64) static Int64 toNetwork(Int64 value); static UInt64 toNetwork(UInt64 value); -#endif static Int16 fromNetwork(Int16 value); static UInt16 fromNetwork(UInt16 value); static Int32 fromNetwork(Int32 value); static UInt32 fromNetwork(UInt32 value); -#if defined(POCO_HAVE_INT64) static Int64 fromNetwork(Int64 value); static UInt64 fromNetwork(UInt64 value); -#endif }; @@ -143,7 +129,6 @@ inline Int32 ByteOrder::flipBytes(Int32 value) } -#if defined(POCO_HAVE_INT64) inline UInt64 ByteOrder::flipBytes(UInt64 value) { # if defined(POCO_HAVE_MSC_BYTESWAP) @@ -162,7 +147,6 @@ inline Int64 ByteOrder::flipBytes(Int64 value) { return Int64(flipBytes(UInt64(value))); } -#endif // POCO_HAVE_INT64 // @@ -180,7 +164,6 @@ inline Int64 ByteOrder::flipBytes(Int64 value) } -#if defined(POCO_HAVE_INT64) # define POCO_IMPLEMENT_BYTEORDER_NOOP(op) \ POCO_IMPLEMENT_BYTEORDER_NOOP_(op, Int16) \ POCO_IMPLEMENT_BYTEORDER_NOOP_(op, UInt16) \ @@ -195,18 +178,6 @@ inline Int64 ByteOrder::flipBytes(Int64 value) POCO_IMPLEMENT_BYTEORDER_FLIP_(op, UInt32) \ POCO_IMPLEMENT_BYTEORDER_FLIP_(op, Int64) \ POCO_IMPLEMENT_BYTEORDER_FLIP_(op, UInt64) -#else -# define POCO_IMPLEMENT_BYTEORDER_NOOP(op) \ - POCO_IMPLEMENT_BYTEORDER_NOOP_(op, Int16) \ - POCO_IMPLEMENT_BYTEORDER_NOOP_(op, UInt16) \ - POCO_IMPLEMENT_BYTEORDER_NOOP_(op, Int32) \ - POCO_IMPLEMENT_BYTEORDER_NOOP_(op, UInt32) -# define POCO_IMPLEMENT_BYTEORDER_FLIP(op) \ - POCO_IMPLEMENT_BYTEORDER_FLIP_(op, Int16) \ - POCO_IMPLEMENT_BYTEORDER_FLIP_(op, UInt16) \ - POCO_IMPLEMENT_BYTEORDER_FLIP_(op, Int32) \ - POCO_IMPLEMENT_BYTEORDER_FLIP_(op, UInt32) -#endif #if defined(POCO_ARCH_BIG_ENDIAN) diff --git a/base/poco/Foundation/include/Poco/NumberFormatter.h b/base/poco/Foundation/include/Poco/NumberFormatter.h index e246ca16ec3..a320b576083 100644 --- a/base/poco/Foundation/include/Poco/NumberFormatter.h +++ b/base/poco/Foundation/include/Poco/NumberFormatter.h @@ -151,7 +151,6 @@ public: /// If prefix is true, "0x" prefix is prepended to the /// resulting string. -#ifdef POCO_HAVE_INT64 # ifdef POCO_LONG_IS_64_BIT @@ -255,7 +254,6 @@ public: # endif // ifdef POCO_LONG_IS_64_BIT -#endif // ifdef POCO_HAVE_INT64 static std::string format(float value); /// Formats a float value in decimal floating-point notation, @@ -380,7 +378,6 @@ public: /// right justified and zero-padded in a field having at least the /// specified width. -#ifdef POCO_HAVE_INT64 # ifdef POCO_LONG_IS_64_BIT @@ -472,7 +469,6 @@ public: # endif // ifdef POCO_LONG_IS_64_BIT -#endif // ifdef POCO_HAVE_INT64 static void append(std::string & str, float value); /// Formats a float value in decimal floating-point notation, @@ -673,7 +669,6 @@ inline std::string NumberFormatter::formatHex(unsigned long value, int width, bo } -#ifdef POCO_HAVE_INT64 # ifdef POCO_LONG_IS_64_BIT @@ -843,7 +838,6 @@ inline std::string NumberFormatter::formatHex(UInt64 value, int width, bool pref # endif // ifdef POCO_LONG_IS_64_BIT -#endif // ifdef POCO_HAVE_INT64 inline std::string NumberFormatter::format(float value) diff --git a/base/poco/Foundation/include/Poco/NumberParser.h b/base/poco/Foundation/include/Poco/NumberParser.h index de813e37dae..32f8c0dc989 100644 --- a/base/poco/Foundation/include/Poco/NumberParser.h +++ b/base/poco/Foundation/include/Poco/NumberParser.h @@ -80,7 +80,6 @@ public: /// Returns true if a valid integer has been found, false otherwise. /// If parsing was not successful, value is undefined. -#if defined(POCO_HAVE_INT64) static Int64 parse64(const std::string & s, char thousandSeparator = ','); /// Parses a 64-bit integer value in decimal notation from the given string. @@ -118,7 +117,6 @@ public: /// Returns true if a valid integer has been found, false otherwise. /// If parsing was not successful, value is undefined. -#endif // defined(POCO_HAVE_INT64) static double parseFloat(const std::string & s, char decimalSeparator = '.', char thousandSeparator = ','); /// Parses a double value in decimal floating point notation diff --git a/base/poco/Foundation/include/Poco/StreamCopier.h b/base/poco/Foundation/include/Poco/StreamCopier.h index 72b19306388..c24e73d88dd 100644 --- a/base/poco/Foundation/include/Poco/StreamCopier.h +++ b/base/poco/Foundation/include/Poco/StreamCopier.h @@ -38,7 +38,6 @@ public: /// /// Returns the number of bytes copied. -#if defined(POCO_HAVE_INT64) static Poco::UInt64 copyStream64(std::istream & istr, std::ostream & ostr, std::size_t bufferSize = 8192); /// Writes all bytes readable from istr to ostr, using an internal buffer. /// @@ -46,14 +45,12 @@ public: /// /// Note: the only difference to copyStream() is that a 64-bit unsigned /// integer is used to count the number of bytes copied. -#endif static std::streamsize copyStreamUnbuffered(std::istream & istr, std::ostream & ostr); /// Writes all bytes readable from istr to ostr. /// /// Returns the number of bytes copied. -#if defined(POCO_HAVE_INT64) static Poco::UInt64 copyStreamUnbuffered64(std::istream & istr, std::ostream & ostr); /// Writes all bytes readable from istr to ostr. /// @@ -61,14 +58,12 @@ public: /// /// Note: the only difference to copyStreamUnbuffered() is that a 64-bit unsigned /// integer is used to count the number of bytes copied. -#endif static std::streamsize copyToString(std::istream & istr, std::string & str, std::size_t bufferSize = 8192); /// Appends all bytes readable from istr to the given string, using an internal buffer. /// /// Returns the number of bytes copied. -#if defined(POCO_HAVE_INT64) static Poco::UInt64 copyToString64(std::istream & istr, std::string & str, std::size_t bufferSize = 8192); /// Appends all bytes readable from istr to the given string, using an internal buffer. /// @@ -76,7 +71,6 @@ public: /// /// Note: the only difference to copyToString() is that a 64-bit unsigned /// integer is used to count the number of bytes copied. -#endif }; diff --git a/base/poco/Foundation/include/Poco/Token.h b/base/poco/Foundation/include/Poco/Token.h index 2d62ed87de6..1aec9e620fe 100644 --- a/base/poco/Foundation/include/Poco/Token.h +++ b/base/poco/Foundation/include/Poco/Token.h @@ -84,13 +84,11 @@ public: virtual std::string asString() const; /// Returns a string representation of the token. -#if defined(POCO_HAVE_INT64) virtual Int64 asInteger64() const; /// Returns a 64-bit integer representation of the token. virtual UInt64 asUnsignedInteger64() const; /// Returns an unsigned 64-bit integer representation of the token. -#endif virtual int asInteger() const; /// Returns an integer representation of the token. diff --git a/base/poco/Foundation/include/Poco/Types.h b/base/poco/Foundation/include/Poco/Types.h index 156b3584d15..d10047344f6 100644 --- a/base/poco/Foundation/include/Poco/Types.h +++ b/base/poco/Foundation/include/Poco/Types.h @@ -46,7 +46,6 @@ typedef unsigned long UInt64; typedef signed long long Int64; typedef unsigned long long UInt64; # endif -# define POCO_HAVE_INT64 1 #endif diff --git a/base/poco/Foundation/src/BinaryReader.cpp b/base/poco/Foundation/src/BinaryReader.cpp index fb57371fbc3..f2961e03966 100644 --- a/base/poco/Foundation/src/BinaryReader.cpp +++ b/base/poco/Foundation/src/BinaryReader.cpp @@ -170,7 +170,7 @@ BinaryReader& BinaryReader::operator >> (double& value) } -#if defined(POCO_HAVE_INT64) && !defined(POCO_LONG_IS_64_BIT) +#if !defined(POCO_LONG_IS_64_BIT) BinaryReader& BinaryReader::operator >> (Int64& value) @@ -233,7 +233,6 @@ void BinaryReader::read7BitEncoded(UInt32& value) } -#if defined(POCO_HAVE_INT64) void BinaryReader::read7BitEncoded(UInt64& value) @@ -254,7 +253,6 @@ void BinaryReader::read7BitEncoded(UInt64& value) } -#endif void BinaryReader::readRaw(std::streamsize length, std::string& value) diff --git a/base/poco/Foundation/src/BinaryWriter.cpp b/base/poco/Foundation/src/BinaryWriter.cpp index 62e1adfe373..6db5ab7cb90 100644 --- a/base/poco/Foundation/src/BinaryWriter.cpp +++ b/base/poco/Foundation/src/BinaryWriter.cpp @@ -212,7 +212,7 @@ BinaryWriter& BinaryWriter::operator << (double value) } -#if defined(POCO_HAVE_INT64) && !defined(POCO_LONG_IS_64_BIT) +#if !defined(POCO_LONG_IS_64_BIT) BinaryWriter& BinaryWriter::operator << (Int64 value) @@ -303,7 +303,6 @@ void BinaryWriter::write7BitEncoded(UInt32 value) } -#if defined(POCO_HAVE_INT64) void BinaryWriter::write7BitEncoded(UInt64 value) @@ -319,7 +318,6 @@ void BinaryWriter::write7BitEncoded(UInt64 value) } -#endif void BinaryWriter::writeRaw(const std::string& rawData) diff --git a/base/poco/Foundation/src/NumberFormatter.cpp b/base/poco/Foundation/src/NumberFormatter.cpp index 5c8126e9b0a..0a9334059a9 100644 --- a/base/poco/Foundation/src/NumberFormatter.cpp +++ b/base/poco/Foundation/src/NumberFormatter.cpp @@ -234,7 +234,6 @@ void NumberFormatter::appendHex(std::string& str, unsigned long value, int width } -#ifdef POCO_HAVE_INT64 #ifdef POCO_LONG_IS_64_BIT @@ -424,7 +423,6 @@ void NumberFormatter::appendHex(std::string& str, UInt64 value, int width) #endif // ifdef POCO_LONG_IS_64_BIT -#endif // ifdef POCO_HAVE_INT64 void NumberFormatter::append(std::string& str, float value) diff --git a/base/poco/Foundation/src/NumberParser.cpp b/base/poco/Foundation/src/NumberParser.cpp index 56eeb167595..4081f3b2663 100644 --- a/base/poco/Foundation/src/NumberParser.cpp +++ b/base/poco/Foundation/src/NumberParser.cpp @@ -104,7 +104,6 @@ bool NumberParser::tryParseOct(const std::string& s, unsigned& value) } -#if defined(POCO_HAVE_INT64) Int64 NumberParser::parse64(const std::string& s, char thSep) @@ -173,7 +172,6 @@ bool NumberParser::tryParseOct64(const std::string& s, UInt64& value) } -#endif // defined(POCO_HAVE_INT64) double NumberParser::parseFloat(const std::string& s, char decSep, char thSep) diff --git a/base/poco/Foundation/src/StreamCopier.cpp b/base/poco/Foundation/src/StreamCopier.cpp index 6f34cc233a2..508d1e7b2ae 100644 --- a/base/poco/Foundation/src/StreamCopier.cpp +++ b/base/poco/Foundation/src/StreamCopier.cpp @@ -42,7 +42,6 @@ std::streamsize StreamCopier::copyStream(std::istream& istr, std::ostream& ostr, } -#if defined(POCO_HAVE_INT64) Poco::UInt64 StreamCopier::copyStream64(std::istream& istr, std::ostream& ostr, std::size_t bufferSize) { poco_assert (bufferSize > 0); @@ -64,7 +63,6 @@ Poco::UInt64 StreamCopier::copyStream64(std::istream& istr, std::ostream& ostr, } return len; } -#endif std::streamsize StreamCopier::copyToString(std::istream& istr, std::string& str, std::size_t bufferSize) @@ -90,7 +88,6 @@ std::streamsize StreamCopier::copyToString(std::istream& istr, std::string& str, } -#if defined(POCO_HAVE_INT64) Poco::UInt64 StreamCopier::copyToString64(std::istream& istr, std::string& str, std::size_t bufferSize) { poco_assert (bufferSize > 0); @@ -112,7 +109,6 @@ Poco::UInt64 StreamCopier::copyToString64(std::istream& istr, std::string& str, } return len; } -#endif std::streamsize StreamCopier::copyStreamUnbuffered(std::istream& istr, std::ostream& ostr) @@ -130,7 +126,6 @@ std::streamsize StreamCopier::copyStreamUnbuffered(std::istream& istr, std::ostr } -#if defined(POCO_HAVE_INT64) Poco::UInt64 StreamCopier::copyStreamUnbuffered64(std::istream& istr, std::ostream& ostr) { char c = 0; @@ -144,7 +139,6 @@ Poco::UInt64 StreamCopier::copyStreamUnbuffered64(std::istream& istr, std::ostre } return len; } -#endif } // namespace Poco diff --git a/base/poco/Foundation/src/Token.cpp b/base/poco/Foundation/src/Token.cpp index 98e8bb25e93..4e81c6ef885 100644 --- a/base/poco/Foundation/src/Token.cpp +++ b/base/poco/Foundation/src/Token.cpp @@ -54,7 +54,6 @@ std::string Token::asString() const } -#if defined(POCO_HAVE_INT64) Int64 Token::asInteger64() const { return NumberParser::parse64(_value); @@ -65,7 +64,6 @@ UInt64 Token::asUnsignedInteger64() const { return NumberParser::parseUnsigned64(_value); } -#endif int Token::asInteger() const diff --git a/base/poco/JSON/include/Poco/JSON/Handler.h b/base/poco/JSON/include/Poco/JSON/Handler.h index f9114a59221..c412a05003f 100644 --- a/base/poco/JSON/include/Poco/JSON/Handler.h +++ b/base/poco/JSON/include/Poco/JSON/Handler.h @@ -74,14 +74,12 @@ namespace JSON /// An unsigned value is read. This will only be triggered if the /// value cannot fit into a signed int. -#if defined(POCO_HAVE_INT64) virtual void value(Int64 v) = 0; /// A 64-bit integer value is read. virtual void value(UInt64 v) = 0; /// An unsigned 64-bit integer value is read. This will only be /// triggered if the value cannot fit into a signed 64-bit integer. -#endif virtual void value(const std::string & value) = 0; /// A string value is read. diff --git a/base/poco/JSON/include/Poco/JSON/ParseHandler.h b/base/poco/JSON/include/Poco/JSON/ParseHandler.h index 4669dc8638f..1b8ac3066d2 100644 --- a/base/poco/JSON/include/Poco/JSON/ParseHandler.h +++ b/base/poco/JSON/include/Poco/JSON/ParseHandler.h @@ -73,14 +73,12 @@ namespace JSON /// An unsigned value is read. This will only be triggered if the /// value cannot fit into a signed int. -#if defined(POCO_HAVE_INT64) virtual void value(Int64 v); /// A 64-bit integer value is read virtual void value(UInt64 v); /// An unsigned 64-bit integer value is read. This will only be /// triggered if the value cannot fit into a signed 64-bit integer. -#endif virtual void value(const std::string & s); /// A string value is read. @@ -126,7 +124,6 @@ namespace JSON } -#if defined(POCO_HAVE_INT64) inline void ParseHandler::value(Int64 v) { setValue(v); @@ -137,7 +134,6 @@ namespace JSON { setValue(v); } -#endif inline void ParseHandler::value(const std::string & s) diff --git a/base/poco/JSON/include/Poco/JSON/PrintHandler.h b/base/poco/JSON/include/Poco/JSON/PrintHandler.h index 34a991653ba..390f4d8bba9 100644 --- a/base/poco/JSON/include/Poco/JSON/PrintHandler.h +++ b/base/poco/JSON/include/Poco/JSON/PrintHandler.h @@ -81,13 +81,11 @@ namespace JSON /// An unsigned value is read. This will only be triggered if the /// value cannot fit into a signed int. -#if defined(POCO_HAVE_INT64) void value(Int64 v); /// A 64-bit integer value is read; it will be written to the output. void value(UInt64 v); /// An unsigned 64-bit integer value is read; it will be written to the output. -#endif void value(const std::string & value); /// A string value is read; it will be formatted and written to the output. diff --git a/base/poco/JSON/src/PrintHandler.cpp b/base/poco/JSON/src/PrintHandler.cpp index bf735d0869c..ea81cbdd1c0 100644 --- a/base/poco/JSON/src/PrintHandler.cpp +++ b/base/poco/JSON/src/PrintHandler.cpp @@ -154,7 +154,6 @@ void PrintHandler::value(unsigned v) } -#if defined(POCO_HAVE_INT64) void PrintHandler::value(Int64 v) { arrayValue(); @@ -169,7 +168,6 @@ void PrintHandler::value(UInt64 v) _out << v; _objStart = false; } -#endif void PrintHandler::value(const std::string& value) diff --git a/base/poco/Net/include/Poco/Net/HTTPFixedLengthStream.h b/base/poco/Net/include/Poco/Net/HTTPFixedLengthStream.h index dcdd1cfcaf8..4de211fdb92 100644 --- a/base/poco/Net/include/Poco/Net/HTTPFixedLengthStream.h +++ b/base/poco/Net/include/Poco/Net/HTTPFixedLengthStream.h @@ -43,11 +43,7 @@ namespace Net public: typedef HTTPBasicStreamBuf::openmode openmode; -#if defined(POCO_HAVE_INT64) typedef Poco::Int64 ContentLength; -#else - typedef std::streamsize ContentLength; -#endif HTTPFixedLengthStreamBuf(HTTPSession & session, ContentLength length, openmode mode); ~HTTPFixedLengthStreamBuf(); diff --git a/base/poco/Net/include/Poco/Net/HTTPMessage.h b/base/poco/Net/include/Poco/Net/HTTPMessage.h index 5c54bf7306b..0bef50803a8 100644 --- a/base/poco/Net/include/Poco/Net/HTTPMessage.h +++ b/base/poco/Net/include/Poco/Net/HTTPMessage.h @@ -56,7 +56,6 @@ namespace Net /// which may be UNKNOWN_CONTENT_LENGTH if /// no Content-Length header is present. -#if defined(POCO_HAVE_INT64) void setContentLength64(Poco::Int64 length); /// Sets the Content-Length header. /// @@ -73,7 +72,6 @@ namespace Net /// /// In contrast to getContentLength(), this method /// always returns a 64-bit integer for content length. -#endif // defined(POCO_HAVE_INT64) bool hasContentLength() const; /// Returns true iff a Content-Length header is present. diff --git a/base/poco/Net/src/HTTPClientSession.cpp b/base/poco/Net/src/HTTPClientSession.cpp index 323e9526df5..c5697b556d1 100644 --- a/base/poco/Net/src/HTTPClientSession.cpp +++ b/base/poco/Net/src/HTTPClientSession.cpp @@ -264,11 +264,7 @@ std::ostream& HTTPClientSession::sendRequest(HTTPRequest& request) { Poco::CountingOutputStream cs; request.write(cs); -#if POCO_HAVE_INT64 _pRequestStream = new HTTPFixedLengthOutputStream(*this, request.getContentLength64() + cs.chars()); -#else - _pRequestStream = new HTTPFixedLengthOutputStream(*this, request.getContentLength() + cs.chars()); -#endif request.write(*_pRequestStream); } else if ((method != HTTPRequest::HTTP_PUT && method != HTTPRequest::HTTP_POST && method != HTTPRequest::HTTP_PATCH) || request.has(HTTPRequest::UPGRADE)) @@ -334,11 +330,7 @@ std::istream& HTTPClientSession::receiveResponse(HTTPResponse& response) else if (response.getChunkedTransferEncoding()) _pResponseStream = new HTTPChunkedInputStream(*this); else if (response.hasContentLength()) -#if defined(POCO_HAVE_INT64) _pResponseStream = new HTTPFixedLengthInputStream(*this, response.getContentLength64()); -#else - _pResponseStream = new HTTPFixedLengthInputStream(*this, response.getContentLength()); -#endif else _pResponseStream = new HTTPInputStream(*this); diff --git a/base/poco/Net/src/HTTPMessage.cpp b/base/poco/Net/src/HTTPMessage.cpp index debda04c3b3..0cd234ee9cb 100644 --- a/base/poco/Net/src/HTTPMessage.cpp +++ b/base/poco/Net/src/HTTPMessage.cpp @@ -89,7 +89,6 @@ std::streamsize HTTPMessage::getContentLength() const } -#if defined(POCO_HAVE_INT64) void HTTPMessage::setContentLength64(Poco::Int64 length) { if (length != UNKNOWN_CONTENT_LENGTH) @@ -108,7 +107,6 @@ Poco::Int64 HTTPMessage::getContentLength64() const } else return UNKNOWN_CONTENT_LENGTH; } -#endif // defined(POCO_HAVE_INT64) void HTTPMessage::setTransferEncoding(const std::string& transferEncoding) diff --git a/base/poco/Net/src/HTTPServerRequestImpl.cpp b/base/poco/Net/src/HTTPServerRequestImpl.cpp index d8ea7398c9b..d893e49aafb 100644 --- a/base/poco/Net/src/HTTPServerRequestImpl.cpp +++ b/base/poco/Net/src/HTTPServerRequestImpl.cpp @@ -49,11 +49,7 @@ HTTPServerRequestImpl::HTTPServerRequestImpl(HTTPServerResponseImpl& response, H if (getChunkedTransferEncoding()) _pStream = new HTTPChunkedInputStream(session); else if (hasContentLength()) -#if defined(POCO_HAVE_INT64) _pStream = new HTTPFixedLengthInputStream(session, getContentLength64()); -#else - _pStream = new HTTPFixedLengthInputStream(session, getContentLength()); -#endif else if (getMethod() == HTTPRequest::HTTP_GET || getMethod() == HTTPRequest::HTTP_HEAD || getMethod() == HTTPRequest::HTTP_DELETE) _pStream = new HTTPFixedLengthInputStream(session, 0); else diff --git a/base/poco/Net/src/HTTPServerResponseImpl.cpp b/base/poco/Net/src/HTTPServerResponseImpl.cpp index fb6783c633e..55de22c876c 100644 --- a/base/poco/Net/src/HTTPServerResponseImpl.cpp +++ b/base/poco/Net/src/HTTPServerResponseImpl.cpp @@ -92,11 +92,7 @@ std::ostream& HTTPServerResponseImpl::send() { Poco::CountingOutputStream cs; write(cs); -#if defined(POCO_HAVE_INT64) _pStream = new HTTPFixedLengthOutputStream(_session, getContentLength64() + cs.chars()); -#else - _pStream = new HTTPFixedLengthOutputStream(_session, getContentLength() + cs.chars()); -#endif write(*_pStream); } else @@ -153,11 +149,7 @@ void HTTPServerResponseImpl::sendFile(const std::string& path, const std::string Timestamp dateTime = f.getLastModified(); File::FileSize length = f.getSize(); set("Last-Modified", DateTimeFormatter::format(dateTime, DateTimeFormat::HTTP_FORMAT)); -#if defined(POCO_HAVE_INT64) setContentLength64(length); -#else - setContentLength(static_cast(length)); -#endif setContentType(mediaType); setChunkedTransferEncoding(false); diff --git a/base/poco/Util/include/Poco/Util/AbstractConfiguration.h b/base/poco/Util/include/Poco/Util/AbstractConfiguration.h index a0e5e2c50dd..926ac3ba8a9 100644 --- a/base/poco/Util/include/Poco/Util/AbstractConfiguration.h +++ b/base/poco/Util/include/Poco/Util/AbstractConfiguration.h @@ -167,7 +167,6 @@ namespace Util /// If the value contains references to other properties (${}), these /// are expanded. -#if defined(POCO_HAVE_INT64) Int64 getInt64(const std::string & key) const; /// Returns the Int64 value of the property with the given name. @@ -205,7 +204,6 @@ namespace Util /// If the value contains references to other properties (${}), these /// are expanded. -#endif // defined(POCO_HAVE_INT64) double getDouble(const std::string & key) const; /// Returns the double value of the property with the given name. @@ -255,7 +253,6 @@ namespace Util /// Sets the property with the given key to the given value. /// An already existing value for the key is overwritten. -#if defined(POCO_HAVE_INT64) virtual void setInt64(const std::string & key, Int64 value); /// Sets the property with the given key to the given value. @@ -265,7 +262,6 @@ namespace Util /// Sets the property with the given key to the given value. /// An already existing value for the key is overwritten. -#endif // defined(POCO_HAVE_INT64) virtual void setDouble(const std::string & key, double value); /// Sets the property with the given key to the given value. @@ -335,12 +331,10 @@ namespace Util static int parseInt(const std::string & value); static unsigned parseUInt(const std::string & value); -#if defined(POCO_HAVE_INT64) static Int64 parseInt64(const std::string & value); static UInt64 parseUInt64(const std::string & value); -#endif // defined(POCO_HAVE_INT64) static bool parseBool(const std::string & value); void setRawWithEvent(const std::string & key, std::string value); diff --git a/base/poco/Util/include/Poco/Util/WinRegistryKey.h b/base/poco/Util/include/Poco/Util/WinRegistryKey.h index b28f6aefb37..9aa5e35ed8a 100644 --- a/base/poco/Util/include/Poco/Util/WinRegistryKey.h +++ b/base/poco/Util/include/Poco/Util/WinRegistryKey.h @@ -123,7 +123,6 @@ namespace Util /// /// Throws a NotFoundException if the value does not exist. -#if defined(POCO_HAVE_INT64) void setInt64(const std::string & name, Poco::Int64 value); /// Sets the numeric (REG_QWORD) value with the given name. @@ -135,7 +134,6 @@ namespace Util /// /// Throws a NotFoundException if the value does not exist. -#endif // POCO_HAVE_INT64 void deleteValue(const std::string & name); /// Deletes the value with the given name. diff --git a/base/poco/Util/src/AbstractConfiguration.cpp b/base/poco/Util/src/AbstractConfiguration.cpp index 95e8da68a57..2c892decd9a 100644 --- a/base/poco/Util/src/AbstractConfiguration.cpp +++ b/base/poco/Util/src/AbstractConfiguration.cpp @@ -163,7 +163,6 @@ unsigned AbstractConfiguration::getUInt(const std::string& key, unsigned default } -#if defined(POCO_HAVE_INT64) Int64 AbstractConfiguration::getInt64(const std::string& key) const @@ -214,7 +213,6 @@ UInt64 AbstractConfiguration::getUInt64(const std::string& key, UInt64 defaultVa } -#endif // defined(POCO_HAVE_INT64) double AbstractConfiguration::getDouble(const std::string& key) const @@ -283,7 +281,6 @@ void AbstractConfiguration::setUInt(const std::string& key, unsigned int value) } -#if defined(POCO_HAVE_INT64) void AbstractConfiguration::setInt64(const std::string& key, Int64 value) @@ -302,7 +299,6 @@ void AbstractConfiguration::setUInt64(const std::string& key, UInt64 value) } -#endif // defined(POCO_HAVE_INT64) void AbstractConfiguration::setDouble(const std::string& key, double value) From 7f5fb77ed51e6dc8beb842dded898b891972e51d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 22 Feb 2023 15:09:48 +0100 Subject: [PATCH 211/253] Increase table retries in cluster copier tests (#46590) --- programs/copier/ClusterCopier.cpp | 4 ++-- tests/integration/test_cluster_copier/test.py | 16 ++++++++++++++-- 2 files changed, 16 insertions(+), 4 deletions(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 48a3578dd7b..bc882719a08 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -908,7 +908,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab /// Exit if success if (task_status != TaskStatus::Finished) { - LOG_WARNING(log, "Create destination Tale Failed "); + LOG_WARNING(log, "Create destination table failed "); return false; } @@ -1473,7 +1473,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( if (count != 0) { - LOG_INFO(log, "Partition {} piece {}is not empty. In contains {} rows.", task_partition.name, current_piece_number, count); + LOG_INFO(log, "Partition {} piece {} is not empty. In contains {} rows.", task_partition.name, current_piece_number, count); Coordination::Stat stat_shards{}; zookeeper->get(partition_piece.getPartitionPieceShardsPath(), &stat_shards); diff --git a/tests/integration/test_cluster_copier/test.py b/tests/integration/test_cluster_copier/test.py index 0aadcadc064..b261f7e3a39 100644 --- a/tests/integration/test_cluster_copier/test.py +++ b/tests/integration/test_cluster_copier/test.py @@ -565,13 +565,20 @@ def test_copy_with_recovering(started_cluster, use_sample_offset): str(COPYING_FAIL_PROBABILITY), "--experimental-use-sample-offset", "1", + "--max-table-tries", + "10", ], ) else: execute_task( started_cluster, Task1(started_cluster), - ["--copy-fault-probability", str(COPYING_FAIL_PROBABILITY)], + [ + "--copy-fault-probability", + str(COPYING_FAIL_PROBABILITY), + "--max-table-tries", + "10", + ], ) @@ -606,7 +613,12 @@ def test_copy_month_to_week_partition_with_recovering(started_cluster): execute_task( started_cluster, Task2(started_cluster, "test2"), - ["--copy-fault-probability", str(COPYING_FAIL_PROBABILITY)], + [ + "--copy-fault-probability", + str(COPYING_FAIL_PROBABILITY), + "--max-table-tries", + "10", + ], ) From 9ab4944b9ed0d0d126688d153477e5bbfd0d0fdd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 22 Feb 2023 13:55:55 +0100 Subject: [PATCH 212/253] Handle input_format_null_as_default for nested types Signed-off-by: Azat Khuzhin --- .../Serializations/SerializationArray.cpp | 6 ++- .../Serializations/SerializationMap.cpp | 6 ++- .../Serializations/SerializationNullable.cpp | 6 +-- .../Serializations/SerializationTuple.cpp | 6 ++- ...nore_unknown_keys_in_named_tuple.reference | 8 ++-- ...json_ignore_unknown_keys_in_named_tuple.sh | 23 ++++------ ..._as_default_null_as_empty_nested.reference | 42 +++++++++++++++++++ ...t_null_as_default_null_as_empty_nested.sql | 25 +++++++++++ 8 files changed, 94 insertions(+), 28 deletions(-) create mode 100644 tests/queries/0_stateless/02573_insert_null_as_default_null_as_empty_nested.reference create mode 100644 tests/queries/0_stateless/02573_insert_null_as_default_null_as_empty_nested.sql diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index 24aa9e8320d..73b232690c7 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -510,7 +511,10 @@ void SerializationArray::deserializeTextJSON(IColumn & column, ReadBuffer & istr deserializeTextImpl(column, istr, [&](IColumn & nested_column) { - nested->deserializeTextJSON(nested_column, istr, settings); + if (settings.null_as_default) + SerializationNullable::deserializeTextJSONImpl(nested_column, istr, settings, nested); + else + nested->deserializeTextJSON(nested_column, istr, settings); }, false); } diff --git a/src/DataTypes/Serializations/SerializationMap.cpp b/src/DataTypes/Serializations/SerializationMap.cpp index 98067077178..34da0f11cae 100644 --- a/src/DataTypes/Serializations/SerializationMap.cpp +++ b/src/DataTypes/Serializations/SerializationMap.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -211,7 +212,10 @@ void SerializationMap::deserializeTextJSON(IColumn & column, ReadBuffer & istr, deserializeTextImpl(column, istr, [&settings](ReadBuffer & buf, const SerializationPtr & subcolumn_serialization, IColumn & subcolumn) { - subcolumn_serialization->deserializeTextJSON(subcolumn, buf, settings); + if (settings.null_as_default) + SerializationNullable::deserializeTextJSONImpl(subcolumn, buf, settings, subcolumn_serialization); + else + subcolumn_serialization->deserializeTextJSON(subcolumn, buf, settings); }); } diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index 8b0bdc05d00..20188f7cec5 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -219,13 +219,9 @@ static ReturnType safeDeserialize( /// Deserialize value into non-nullable column. In case of NULL, insert default value and return false. template , ReturnType>* = nullptr> static ReturnType safeDeserialize( - IColumn & column, const ISerialization & nested, + IColumn & column, const ISerialization &, CheckForNull && check_for_null, DeserializeNested && deserialize_nested) { - assert(!dynamic_cast(&column)); - assert(!dynamic_cast(&nested)); - UNUSED(nested); - bool insert_default = check_for_null(); if (insert_default) column.insertDefault(); diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index ce15e099222..ef11b3e4660 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -231,7 +232,10 @@ void SerializationTuple::deserializeTextJSON(IColumn & column, ReadBuffer & istr seen_elements[element_pos] = 1; auto & element_column = extractElementColumn(column, element_pos); - elems[element_pos]->deserializeTextJSON(element_column, istr, settings); + if (settings.null_as_default) + SerializationNullable::deserializeTextJSONImpl(element_column, istr, settings, elems[element_pos]); + else + elems[element_pos]->deserializeTextJSON(element_column, istr, settings); skipWhitespaceIfAny(istr); ++processed; diff --git a/tests/queries/0_stateless/02540_input_format_json_ignore_unknown_keys_in_named_tuple.reference b/tests/queries/0_stateless/02540_input_format_json_ignore_unknown_keys_in_named_tuple.reference index a1b4e2b5a83..b7edddf46e0 100644 --- a/tests/queries/0_stateless/02540_input_format_json_ignore_unknown_keys_in_named_tuple.reference +++ b/tests/queries/0_stateless/02540_input_format_json_ignore_unknown_keys_in_named_tuple.reference @@ -3,11 +3,11 @@ INCORRECT_DATA NOT_FOUND_COLUMN_IN_BLOCK (1) { - "row_1": {"type":"CreateEvent","actor":{"login":"foobar"},"repo":{"name":"ClickHouse\/ClickHouse"},"created_at":"2023-01-26 10:48:02","payload":{"updated_at":"1970-01-01 00:00:00","action":"","comment":{"id":"0","path":"","position":0,"line":0,"user":{"login":""},"diff_hunk":"","original_position":0,"commit_id":"","original_commit_id":""},"review":{"body":"","author_association":"","state":""},"ref":"backport","ref_type":"branch","issue":{"number":0,"title":"","labels":[],"state":"","locked":0,"assignee":{"login":""},"assignees":[],"comment":"","closed_at":"1970-01-01 00:00:00"},"pull_request":{"merged_at":null,"merge_commit_sha":"","requested_reviewers":[],"requested_teams":[],"head":{"ref":"","sha":""},"base":{"ref":"","sha":""},"merged":0,"mergeable":0,"rebaseable":0,"mergeable_state":"","merged_by":null,"review_comments":0,"maintainer_can_modify":0,"commits":0,"additions":0,"deletions":0,"changed_files":0},"size":0,"distinct_size":0,"member":{"login":""},"release":{"tag_name":"","name":""}}} + "row_1": {"type":"CreateEvent","actor":{"login":"foobar"},"repo":{"name":"ClickHouse\/ClickHouse"},"created_at":"2023-01-26 10:48:02","payload":{"updated_at":"1970-01-01 00:00:00","action":"","comment":{"id":"0","path":"","position":0,"line":0,"user":{"login":""},"diff_hunk":"","original_position":0,"commit_id":"","original_commit_id":""},"review":{"body":"","author_association":"","state":""},"ref":"backport","ref_type":"branch","issue":{"number":0,"title":"","labels":[],"state":"","locked":0,"assignee":{"login":""},"assignees":[],"comment":"","closed_at":"1970-01-01 00:00:00"},"pull_request":{"merged_at":"1970-01-01 00:00:00","merge_commit_sha":"","requested_reviewers":[],"requested_teams":[],"head":{"ref":"","sha":""},"base":{"ref":"","sha":""},"merged":0,"mergeable":0,"rebaseable":0,"mergeable_state":"","merged_by":{"login":""},"review_comments":0,"maintainer_can_modify":0,"commits":0,"additions":0,"deletions":0,"changed_files":0},"size":0,"distinct_size":0,"member":{"login":""},"release":{"tag_name":"","name":""}}} } { - "row_1": {"labels":[],"merged_by":""}, + "row_1": {"labels":[],"merged_by":""}, "row_2": {"labels":[],"merged_by":"foobar"}, - "row_3": {"labels":[],"merged_by":""}, - "row_4": {"labels":["backport"],"merged_by":""} + "row_3": {"labels":[],"merged_by":""}, + "row_4": {"labels":["backport"],"merged_by":""} } diff --git a/tests/queries/0_stateless/02540_input_format_json_ignore_unknown_keys_in_named_tuple.sh b/tests/queries/0_stateless/02540_input_format_json_ignore_unknown_keys_in_named_tuple.sh index f37a36fa192..eccac543215 100755 --- a/tests/queries/0_stateless/02540_input_format_json_ignore_unknown_keys_in_named_tuple.sh +++ b/tests/queries/0_stateless/02540_input_format_json_ignore_unknown_keys_in_named_tuple.sh @@ -60,7 +60,7 @@ gharchive_structure=( closed_at DateTime('UTC') ), pull_request Tuple( - merged_at Nullable(DateTime('UTC')), + merged_at DateTime('UTC'), merge_commit_sha String, requested_reviewers Nested( login String @@ -80,16 +80,9 @@ gharchive_structure=( mergeable UInt8, rebaseable UInt8, mergeable_state String, - merged_by Nullable(String), - /* NOTE: correct type is Tuple, however Tuple cannot be Nullable, - * so you still have to use Nullable(String) and rely on - * input_format_json_read_objects_as_strings, but see also - * https://github.com/ClickHouse/ClickHouse/issues/36464 - */ - /* merged_by Tuple( - * login String - * ), - */ + merged_by Tuple( + login String + ), review_comments UInt32, maintainer_can_modify UInt8, commits UInt32, @@ -122,12 +115,10 @@ EOL # NOTE: due to [1] we cannot use dot.dot notation, only tupleElement() # # [1]: https://github.com/ClickHouse/ClickHouse/issues/24607 -$CLICKHOUSE_LOCAL "${gharchive_settings[@]}" --structure="${gharchive_structure[*]}" -q " - WITH - tupleElement(tupleElement(payload, 'pull_request'), 'merged_by') AS merged_by_ +$CLICKHOUSE_LOCAL --allow_experimental_analyzer=1 "${gharchive_settings[@]}" --structure="${gharchive_structure[*]}" -q " SELECT - tupleElement(tupleElement(tupleElement(payload, 'issue'), 'labels'), 'name') AS labels, - if(merged_by_ IS NULL, '', JSONExtractString(merged_by_, 'login')) AS merged_by + payload.issue.labels.name AS labels, + payload.pull_request.merged_by.login AS merged_by FROM table " < Date: Wed, 22 Feb 2023 15:51:13 +0100 Subject: [PATCH 213/253] Analyzer AutoFinalOnQueryPass fix --- src/Analyzer/Passes/AutoFinalOnQueryPass.cpp | 87 ++++++++++++-------- src/Analyzer/Passes/AutoFinalOnQueryPass.h | 16 +++- src/Analyzer/TableExpressionModifiers.h | 6 ++ src/Analyzer/TableFunctionNode.h | 6 ++ src/Analyzer/TableNode.h | 6 ++ 5 files changed, 82 insertions(+), 39 deletions(-) diff --git a/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp b/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp index 10efebe0731..fdf818681d7 100644 --- a/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp +++ b/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp @@ -1,8 +1,11 @@ #include "AutoFinalOnQueryPass.h" -#include -#include #include + +#include +#include +#include +#include #include namespace DB @@ -10,52 +13,64 @@ namespace DB namespace { - class AutoFinalOnQueryPassVisitor : public InDepthQueryTreeVisitorWithContext + +class AutoFinalOnQueryPassVisitor : public InDepthQueryTreeVisitorWithContext +{ +public: + using Base = InDepthQueryTreeVisitorWithContext; + using Base::Base; + + void visitImpl(QueryTreeNodePtr & node) { - public: - using Base = InDepthQueryTreeVisitorWithContext; - using Base::Base; + const auto & context = getContext(); + if (!context->getSettingsRef().final) + return; - void visitImpl(QueryTreeNodePtr & node) + const auto * query_node = node->as(); + if (!query_node) + return; + + auto table_expressions = extractTableExpressions(query_node->getJoinTree()); + for (auto & table_expression : table_expressions) + applyFinalIfNeeded(table_expression); + } +private: + static void applyFinalIfNeeded(QueryTreeNodePtr & node) + { + auto * table_node = node->as(); + auto * table_function_node = node->as(); + if (!table_node && !table_function_node) + return; + + const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage(); + bool is_final_supported = storage && storage->supportsFinal() && !storage->isRemote(); + if (!is_final_supported) + return; + + TableExpressionModifiers table_expression_modifiers_with_final(true /*has_final*/, {}, {}); + + if (table_node) { - if (auto * table_node = node->as()) - { - if (autoFinalOnQuery(*table_node, table_node->getStorage(), getContext())) - { - auto modifier = TableExpressionModifiers(true, std::nullopt, std::nullopt); - table_node->setTableExpressionModifiers(modifier); - } - } + if (table_node->hasTableExpressionModifiers()) + table_node->getTableExpressionModifiers()->setHasFinal(true); + else + table_node->setTableExpressionModifiers(table_expression_modifiers_with_final); } - - private: - static bool autoFinalOnQuery(TableNode & table_node, StoragePtr storage, ContextPtr context) + else if (table_function_node) { - bool is_auto_final_setting_on = context->getSettingsRef().final; - bool is_final_supported = storage && storage->supportsFinal() && !storage->isRemote(); - bool is_query_already_final = table_node.hasTableExpressionModifiers() ? table_node.getTableExpressionModifiers().has_value() : false; - - return is_auto_final_setting_on && !is_query_already_final && is_final_supported; + if (table_function_node->hasTableExpressionModifiers()) + table_function_node->getTableExpressionModifiers()->setHasFinal(true); + else + table_function_node->setTableExpressionModifiers(table_expression_modifiers_with_final); } + } +}; - }; - -} - -String AutoFinalOnQueryPass::getName() -{ - return "AutoFinalOnQueryPass"; -} - -String AutoFinalOnQueryPass::getDescription() -{ - return "Automatically applies final modifier to queries if it is supported and if user level final setting is set."; } void AutoFinalOnQueryPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) { auto visitor = AutoFinalOnQueryPassVisitor(std::move(context)); - visitor.visit(query_tree_node); } diff --git a/src/Analyzer/Passes/AutoFinalOnQueryPass.h b/src/Analyzer/Passes/AutoFinalOnQueryPass.h index eacbe0f8235..3489597108c 100644 --- a/src/Analyzer/Passes/AutoFinalOnQueryPass.h +++ b/src/Analyzer/Passes/AutoFinalOnQueryPass.h @@ -7,13 +7,23 @@ namespace DB { - +/** Automatically applies final modifier to table expressions in queries if it is supported and if user level final setting is set. + * + * Example: SELECT id, value FROM test_table; + * Result: SELECT id, value FROM test_table FINAL; + */ class AutoFinalOnQueryPass final : public IQueryTreePass { public: - String getName() override; + String getName() override + { + return "AutoFinalOnQueryPass"; + } - String getDescription() override; + String getDescription() override + { + return "Automatically applies final modifier to table expressions in queries if it is supported and if user level final setting is set"; + } void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; }; diff --git a/src/Analyzer/TableExpressionModifiers.h b/src/Analyzer/TableExpressionModifiers.h index f61c2a61610..9b76c9bc0fd 100644 --- a/src/Analyzer/TableExpressionModifiers.h +++ b/src/Analyzer/TableExpressionModifiers.h @@ -28,6 +28,12 @@ public: return has_final; } + /// Set has final value + void setHasFinal(bool value) + { + has_final = value; + } + /// Returns true if sample size ratio is specified, false otherwise bool hasSampleSizeRatio() const { diff --git a/src/Analyzer/TableFunctionNode.h b/src/Analyzer/TableFunctionNode.h index 292ab740c5b..a88630ffd00 100644 --- a/src/Analyzer/TableFunctionNode.h +++ b/src/Analyzer/TableFunctionNode.h @@ -116,6 +116,12 @@ public: return table_expression_modifiers; } + /// Get table expression modifiers + std::optional & getTableExpressionModifiers() + { + return table_expression_modifiers; + } + /// Set table expression modifiers void setTableExpressionModifiers(TableExpressionModifiers table_expression_modifiers_value) { diff --git a/src/Analyzer/TableNode.h b/src/Analyzer/TableNode.h index 4965de535df..6d47f87c78b 100644 --- a/src/Analyzer/TableNode.h +++ b/src/Analyzer/TableNode.h @@ -68,6 +68,12 @@ public: return table_expression_modifiers; } + /// Get table expression modifiers + std::optional & getTableExpressionModifiers() + { + return table_expression_modifiers; + } + /// Set table expression modifiers void setTableExpressionModifiers(TableExpressionModifiers table_expression_modifiers_value) { From cdbff57e6c91930b7b4eb9535f6b4d17e17641be Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Wed, 22 Feb 2023 15:58:06 +0100 Subject: [PATCH 214/253] Ask for password interactively --- programs/client/Client.cpp | 24 +++++++++++++++++++++++- 1 file changed, 23 insertions(+), 1 deletion(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 3be96a4b0a0..af66a4ac61d 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -327,7 +327,29 @@ try showClientVersion(); } - connect(); + try + { + connect(); + } + catch (const Exception & e) + { + if (e.code() == DB::ErrorCodes::AUTHENTICATION_FAILED) + { + if (!config().getString("password", "").empty()) + throw; + + if (!is_interactive) + throw; + + String prompt = fmt::format("Password for user ({}): ", config().getString("user", "")); + String password; + if (auto * result = readpassphrase(prompt, buf, sizeof(buf), 0)) + password = result; + + config().setString("password", password); + connect(); + } + } /// Show warnings at the beginning of connection. if (is_interactive && !config().has("no-warnings")) From 4f31e59dcd966a52b32ddf87ed75dd4804f2fa55 Mon Sep 17 00:00:00 2001 From: Julio Jimenez Date: Wed, 22 Feb 2023 10:34:27 -0500 Subject: [PATCH 215/253] Fix SonarCloud Job Signed-off-by: Julio Jimenez --- .github/workflows/nightly.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 27c4f5811da..f6d6d192f48 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -154,7 +154,7 @@ jobs: - name: Set Up Build Tools run: | sudo apt-get update - sudo apt-get install -yq git cmake ccache python3 ninja-build + sudo apt-get install -yq git cmake ccache ninja-build python3 yasm sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)" - name: Run build-wrapper run: | From af992ca2db35b0d20f703a44d8ccae5c5f955060 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Wed, 22 Feb 2023 16:51:36 +0100 Subject: [PATCH 216/253] Better --- programs/client/Client.cpp | 22 +++++++--------------- 1 file changed, 7 insertions(+), 15 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index af66a4ac61d..660b8d7c00a 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -333,22 +333,14 @@ try } catch (const Exception & e) { - if (e.code() == DB::ErrorCodes::AUTHENTICATION_FAILED) - { - if (!config().getString("password", "").empty()) - throw; + if (e.code() != DB::ErrorCodes::AUTHENTICATION_FAILED || + config().has("password") || + config().getBool("ask-password", false) || + !is_interactive) + throw; - if (!is_interactive) - throw; - - String prompt = fmt::format("Password for user ({}): ", config().getString("user", "")); - String password; - if (auto * result = readpassphrase(prompt, buf, sizeof(buf), 0)) - password = result; - - config().setString("password", password); - connect(); - } + config().setBool("ask-password", true); + connect(); } /// Show warnings at the beginning of connection. From 67dcd9694c67b5dcf00e2a408c77f330aa51b232 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 22 Feb 2023 17:49:22 +0100 Subject: [PATCH 217/253] Remove unused MergeTreeReadTask::remove_prewhere_column --- src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp | 2 -- src/Storages/MergeTree/MergeTreeBlockReadUtils.h | 3 --- src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.cpp | 1 - src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp | 2 +- src/Storages/MergeTree/MergeTreeReadPool.cpp | 3 +-- src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp | 6 ++---- 6 files changed, 4 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index edb36cf6e55..4c3d4bc8aa0 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -141,7 +141,6 @@ MergeTreeReadTask::MergeTreeReadTask( size_t part_index_in_query_, const NameSet & column_name_set_, const MergeTreeReadTaskColumns & task_columns_, - bool remove_prewhere_column_, MergeTreeBlockSizePredictorPtr size_predictor_, int64_t priority_, std::future reader_, @@ -151,7 +150,6 @@ MergeTreeReadTask::MergeTreeReadTask( , part_index_in_query{part_index_in_query_} , column_name_set{column_name_set_} , task_columns{task_columns_} - , remove_prewhere_column{remove_prewhere_column_} , size_predictor{size_predictor_} , reader(std::move(reader_)) , pre_reader_for_step(std::move(pre_reader_for_step_)) diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h index 66127c675a1..e7dad5173ff 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h @@ -59,8 +59,6 @@ struct MergeTreeReadTask const NameSet & column_name_set; /// column names to read during PREWHERE and WHERE const MergeTreeReadTaskColumns & task_columns; - /// should PREWHERE column be returned to requesting side? - const bool remove_prewhere_column; /// Used to satistfy preferred_block_size_bytes limitation MergeTreeBlockSizePredictorPtr size_predictor; /// Used to save current range processing status @@ -87,7 +85,6 @@ struct MergeTreeReadTask size_t part_index_in_query_, const NameSet & column_name_set_, const MergeTreeReadTaskColumns & task_columns_, - bool remove_prewhere_column_, MergeTreeBlockSizePredictorPtr size_predictor_, int64_t priority_ = 0, std::future reader_ = {}, diff --git a/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.cpp index bd7aa34ec0e..813f144ee98 100644 --- a/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.cpp @@ -56,7 +56,6 @@ try task = std::make_unique( data_part, mark_ranges_for_task, part_index_in_query, column_name_set, task_columns, - prewhere_info && prewhere_info->remove_prewhere_column, std::move(size_predictor)); return true; diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index d5d55277149..d76b8522f42 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -509,7 +509,7 @@ MergeTreePrefetchedReadPool::ThreadsTasks MergeTreePrefetchedReadPool::createThr auto read_task = std::make_unique( part.data_part, ranges_to_get_from_part, part.part_index_in_query, - part.column_name_set, part.task_columns, prewhere_info && prewhere_info->remove_prewhere_column, + part.column_name_set, part.task_columns, std::move(curr_task_size_predictor)); read_task->priority = priority; diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 023afa5cc93..54a040724fc 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -208,7 +208,7 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(size_t thread) return std::make_unique( part.data_part, ranges_to_get_from_part, part.part_index_in_query, per_part.column_name_set, per_part.task_columns, - prewhere_info && prewhere_info->remove_prewhere_column, std::move(curr_task_size_predictor)); + std::move(curr_task_size_predictor)); } Block MergeTreeReadPool::getHeader() const @@ -459,7 +459,6 @@ MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicas::getTask(size_t thread) part.part_index_in_query, per_part.column_name_set, per_part.task_columns, - prewhere_info && prewhere_info->remove_prewhere_column, std::move(curr_task_size_predictor)); } diff --git a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp index 367818c7af1..da2d0b0ae4a 100644 --- a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp @@ -49,8 +49,7 @@ bool MergeTreeReverseSelectAlgorithm::getNewTaskOrdinaryReading() task = std::make_unique( data_part, mark_ranges_for_task, part_index_in_query, column_name_set, - task_columns, prewhere_info && prewhere_info->remove_prewhere_column, - std::move(size_predictor)); + task_columns, std::move(size_predictor)); return true; @@ -88,8 +87,7 @@ bool MergeTreeReverseSelectAlgorithm::getNewTaskParallelReplicas() task = std::make_unique( data_part, mark_ranges_for_task, part_index_in_query, column_name_set, - task_columns, prewhere_info && prewhere_info->remove_prewhere_column, - std::move(size_predictor)); + task_columns, std::move(size_predictor)); return true; } From 0e7143070e6081131b51c646fb8d2ba587392da8 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 22 Feb 2023 17:51:29 +0100 Subject: [PATCH 218/253] Update docs about format table function --- .../sql-reference/table-functions/format.md | 32 +++++++++++++++++-- 1 file changed, 29 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/table-functions/format.md b/docs/en/sql-reference/table-functions/format.md index 3af48249e3c..811eae12942 100644 --- a/docs/en/sql-reference/table-functions/format.md +++ b/docs/en/sql-reference/table-functions/format.md @@ -6,25 +6,28 @@ sidebar_label: format # format -Extracts table structure from data and parses it according to specified input format. +Parses data from arguments according to specified input format. If structure argument is not specified, it's extracted from the data. **Syntax** ``` sql -format(format_name, data) +format(format_name, [structure], data) ``` **Parameters** - `format_name` — The [format](../../interfaces/formats.md#formats) of the data. +- `structure` - Structure of the table. Optional. Format 'column1_name column1_type, column2_name column2_type, ...'. - `data` — String literal or constant expression that returns a string containing data in specified format **Returned value** -A table with data parsed from `data` argument according specified format and extracted schema. +A table with data parsed from `data` argument according to specified format and specified or extracted structure. **Examples** +Without `structure` argument: + **Query:** ``` sql SELECT * FROM format(JSONEachRow, @@ -67,6 +70,29 @@ $$) └──────┴───────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` +With `structure` argument: + +**Query:** +```sql +SELECT * FROM format(JSONEachRow, 'a String, b UInt32', +$$ +{"a": "Hello", "b": 111} +{"a": "World", "b": 123} +{"a": "Hello", "b": 112} +{"a": "World", "b": 124} +$$) +``` + +**Result:** +```response +┌─a─────┬───b─┐ +│ Hello │ 111 │ +│ World │ 123 │ +│ Hello │ 112 │ +│ World │ 124 │ +└───────┴─────┘ +``` + **See Also** - [Formats](../../interfaces/formats.md) From f5a6a86dec084e6176719b5d440e5439eaf4fa35 Mon Sep 17 00:00:00 2001 From: Jiebin Sun Date: Fri, 10 Feb 2023 09:48:52 -0500 Subject: [PATCH 219/253] Optimize the SIMD StringSearcher by searching first two chars This patch offers the optimized SIMD StringSearcher by searching the first and second chars together rather than only the first char, which will result in big performance gain. The patch also provides a quick path when the needle size is 1. With this patch, I have tested the 43 queries in clickbench on ICX server. Query 20 has got 35% performance gain. Other StringSearcher related queries have got around 10% performance improvement. And the overall geomean of all the queries has got 4.1% performance gain. Signed-off-by: Jiebin Sun --- src/Common/StringSearcher.h | 67 ++++++++++++++++++++++++++++++------- src/Common/Volnitsky.h | 8 +++++ 2 files changed, 63 insertions(+), 12 deletions(-) diff --git a/src/Common/StringSearcher.h b/src/Common/StringSearcher.h index 3afc7635a9d..a2acea1ca39 100644 --- a/src/Common/StringSearcher.h +++ b/src/Common/StringSearcher.h @@ -569,8 +569,10 @@ private: uint8_t first{}; #ifdef __SSE4_1__ - /// vector filled `first` for determining leftmost position of the first symbol - __m128i pattern; + uint8_t second{}; + /// vector filled `first` or `second` for determining leftmost position of the first and second symbols + __m128i first_pattern; + __m128i second_pattern; /// vector of first 16 characters of `needle` __m128i cache = _mm_setzero_si128(); int cachemask{}; @@ -587,9 +589,14 @@ public: first = *needle; -#ifdef __SSE4_1__ - pattern = _mm_set1_epi8(first); +#ifdef __SSE4_1__ + first_pattern = _mm_set1_epi8(first); + if (needle + 1 < needle_end) + { + second = *(needle + 1); + second_pattern = _mm_set1_epi8(second); + } const auto * needle_pos = needle; for (const auto i : collections::range(0, n)) @@ -660,18 +667,54 @@ public: if (needle == needle_end) return haystack; +#ifdef __SSE4_1__ + /// Here is the quick path when needle_size is 1. Compare the first and second characters if the needle_size >= 2. + if (needle + 1 == needle_end) + { + while (haystack < haystack_end) + { + if (haystack + n <= haystack_end && pageSafe(haystack)) + { + const auto v_haystack = _mm_loadu_si128(reinterpret_cast(haystack)); + const auto v_against_pattern = _mm_cmpeq_epi8(v_haystack, first_pattern); + const auto mask = _mm_movemask_epi8(v_against_pattern); + if (mask == 0) + { + haystack += n; + continue; + } + + const auto offset = __builtin_ctz(mask); + haystack += offset; + + return haystack; + } + + if (haystack == haystack_end) + return haystack_end; + + if (*haystack == first) + return haystack; + + ++haystack; + } + + return haystack_end; + } +#endif + while (haystack < haystack_end) { #ifdef __SSE4_1__ - if (haystack + n <= haystack_end && pageSafe(haystack)) + /// find first and second characters + if ((haystack + 1 + n) <= haystack_end && pageSafe(haystack)) { - /// find first character - const auto v_haystack = _mm_loadu_si128(reinterpret_cast(haystack)); - const auto v_against_pattern = _mm_cmpeq_epi8(v_haystack, pattern); - - const auto mask = _mm_movemask_epi8(v_against_pattern); - - /// first character not present in 16 octets starting at `haystack` + const auto v_haystack_block_first = _mm_loadu_si128(reinterpret_cast(haystack)); + const auto v_haystack_block_second = _mm_loadu_si128(reinterpret_cast(haystack + 1)); + const auto v_against_pattern_first = _mm_cmpeq_epi8(v_haystack_block_first, first_pattern); + const auto v_against_pattern_second = _mm_cmpeq_epi8(v_haystack_block_second, second_pattern); + const auto mask = _mm_movemask_epi8(_mm_and_si128(v_against_pattern_first, v_against_pattern_second)); + /// first and second characters not present in 16 octets starting at `haystack` if (mask == 0) { haystack += n; diff --git a/src/Common/Volnitsky.h b/src/Common/Volnitsky.h index a27fd36f704..881bce7cc57 100644 --- a/src/Common/Volnitsky.h +++ b/src/Common/Volnitsky.h @@ -11,6 +11,10 @@ #include #include +#ifdef __SSE4_1__ + #include +#endif + /** Search for a substring in a string by Volnitsky's algorithm * http://volnitsky.com/project/str_search/ * @@ -429,6 +433,10 @@ public: const auto * haystack_end = haystack + haystack_size; +#ifdef __SSE4_1__ + return fallback_searcher.search(haystack, haystack_end); +#endif + if (fallback || haystack_size <= needle_size || fallback_searcher.force_fallback) return fallback_searcher.search(haystack, haystack_end); From d220e7f4fca7fe2401befe5c07362a101de120e7 Mon Sep 17 00:00:00 2001 From: Jiebin Sun Date: Wed, 22 Feb 2023 11:57:34 -0500 Subject: [PATCH 220/253] Optimize the SIMD StringSearcher if needle_size is large This patch offers an additional optimization when the needle_size is large. If the needle_size is larger than the haystack_size, there is no need to search any more. The optimized SIMD StringSearcher has led at most 41.7% than Volnitsky algorithm when the needle_size is less than 21, and fallen behind only about 1% even when the needle_size is bigger than 50, which is not considered as a common case. Test platform: ICX server Test query: SELECT COUNT(*) FROM hits WHERE URL LIKE '%{Needle}%'; Needle_size opt/baseline 5 141.7% 6 129.4% 8 118.5% 9 112.3% 10 107.4% 14 103.4% 20 100.2% 21 100.7% 51 99.0% Signed-off-by: Jiebin Sun --- src/Common/StringSearcher.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/StringSearcher.h b/src/Common/StringSearcher.h index a2acea1ca39..451b8cae0a1 100644 --- a/src/Common/StringSearcher.h +++ b/src/Common/StringSearcher.h @@ -703,7 +703,7 @@ public: } #endif - while (haystack < haystack_end) + while (haystack < haystack_end && haystack_end - haystack >= needle_end - needle) { #ifdef __SSE4_1__ /// find first and second characters From 1f62135ba712a2498f034501b07b2af74fb0e469 Mon Sep 17 00:00:00 2001 From: Jiebin Sun Date: Wed, 22 Feb 2023 12:18:13 -0500 Subject: [PATCH 221/253] Make the optimized SIMD StringSearcher clean This patch has revised the name of value and added comments to make the SIMD StringSearcher clean and easy to understand based on pull request 46289. Signed-off-by: Jiebin Sun --- src/Common/StringSearcher.h | 88 +++++++++++++++++++------------------ 1 file changed, 46 insertions(+), 42 deletions(-) diff --git a/src/Common/StringSearcher.h b/src/Common/StringSearcher.h index 451b8cae0a1..75ef7b9e7ca 100644 --- a/src/Common/StringSearcher.h +++ b/src/Common/StringSearcher.h @@ -566,16 +566,17 @@ private: const uint8_t * const needle; const uint8_t * const needle_end; /// first character in `needle` - uint8_t first{}; + uint8_t first_needle_character = 0; -#ifdef __SSE4_1__ - uint8_t second{}; - /// vector filled `first` or `second` for determining leftmost position of the first and second symbols - __m128i first_pattern; - __m128i second_pattern; +#ifdef __SSE2__ + /// second character of "needle" (if its length is > 1) + uint8_t second_needle_character = 0; + /// The first/second needle character broadcasted into a 16 bytes vector + __m128i first_needle_character_vec; + __m128i second_needle_character_vec; /// vector of first 16 characters of `needle` __m128i cache = _mm_setzero_si128(); - int cachemask{}; + uint16_t cachemask = 0; #endif public: @@ -587,19 +588,18 @@ public: if (0 == needle_size) return; - first = *needle; - + first_needle_character = *needle; #ifdef __SSE4_1__ - first_pattern = _mm_set1_epi8(first); - if (needle + 1 < needle_end) + first_needle_character_vec = _mm_set1_epi8(first_needle_character); + if (needle_size > 1) { - second = *(needle + 1); - second_pattern = _mm_set1_epi8(second); + second_needle_character = *(needle + 1); + second_needle_character_vec = _mm_set1_epi8(second_needle_character); } const auto * needle_pos = needle; - for (const auto i : collections::range(0, n)) + for (uint8_t i = 0; i < n; ++i) { cache = _mm_srli_si128(cache, 1); @@ -620,13 +620,13 @@ public: #ifdef __SSE4_1__ if (pageSafe(pos)) { - const auto v_haystack = _mm_loadu_si128(reinterpret_cast(pos)); - const auto v_against_cache = _mm_cmpeq_epi8(v_haystack, cache); - const auto mask = _mm_movemask_epi8(v_against_cache); + const __m128i haystack_characters = _mm_loadu_si128(reinterpret_cast(pos)); + const __m128i comparison_result = _mm_cmpeq_epi8(haystack_characters, cache); + const uint16_t comparison_result_mask = _mm_movemask_epi8(comparison_result); if (0xffff == cachemask) { - if (mask == cachemask) + if (comparison_result_mask == cachemask) { pos += n; const auto * needle_pos = needle + n; @@ -638,14 +638,14 @@ public: return true; } } - else if ((mask & cachemask) == cachemask) + else if ((comparison_result_mask & cachemask) == cachemask) return true; return false; } #endif - if (*pos == first) + if (*pos == first_needle_character) { ++pos; const auto * needle_pos = needle + 1; @@ -668,23 +668,23 @@ public: return haystack; #ifdef __SSE4_1__ - /// Here is the quick path when needle_size is 1. Compare the first and second characters if the needle_size >= 2. + /// Fast path for single-character needles. Compare 16 characters of the haystack against the needle character at once. if (needle + 1 == needle_end) { while (haystack < haystack_end) { if (haystack + n <= haystack_end && pageSafe(haystack)) { - const auto v_haystack = _mm_loadu_si128(reinterpret_cast(haystack)); - const auto v_against_pattern = _mm_cmpeq_epi8(v_haystack, first_pattern); - const auto mask = _mm_movemask_epi8(v_against_pattern); - if (mask == 0) + const __m128i haystack_characters = _mm_loadu_si128(reinterpret_cast(haystack)); + const __m128i comparison_result = _mm_cmpeq_epi8(haystack_characters, first_needle_character_vec); + const uint16_t comparison_result_mask = _mm_movemask_epi8(comparison_result); + if (comparison_result_mask == 0) { haystack += n; continue; } - const auto offset = __builtin_ctz(mask); + const auto offset = std::countr_zero(comparison_result_mask); haystack += offset; return haystack; @@ -693,7 +693,7 @@ public: if (haystack == haystack_end) return haystack_end; - if (*haystack == first) + if (*haystack == first_needle_character) return haystack; ++haystack; @@ -706,30 +706,34 @@ public: while (haystack < haystack_end && haystack_end - haystack >= needle_end - needle) { #ifdef __SSE4_1__ - /// find first and second characters - if ((haystack + 1 + n) <= haystack_end && pageSafe(haystack)) + /// Compare the [0:15] bytes from haystack and broadcasted 16 bytes vector from first character of needle. + /// Compare the [1:16] bytes from haystack and broadcasted 16 bytes vector from second character of needle. + /// Bit AND the results of above two comparisons and get the mask. + if ((haystack + 1 + n) <= haystack_end && pageSafe(haystack + 1)) { - const auto v_haystack_block_first = _mm_loadu_si128(reinterpret_cast(haystack)); - const auto v_haystack_block_second = _mm_loadu_si128(reinterpret_cast(haystack + 1)); - const auto v_against_pattern_first = _mm_cmpeq_epi8(v_haystack_block_first, first_pattern); - const auto v_against_pattern_second = _mm_cmpeq_epi8(v_haystack_block_second, second_pattern); - const auto mask = _mm_movemask_epi8(_mm_and_si128(v_against_pattern_first, v_against_pattern_second)); - /// first and second characters not present in 16 octets starting at `haystack` - if (mask == 0) + const __m128i haystack_characters_from_1st = _mm_loadu_si128(reinterpret_cast(haystack)); + const __m128i haystack_characters_from_2nd = _mm_loadu_si128(reinterpret_cast(haystack + 1)); + const __m128i comparison_result_1st = _mm_cmpeq_epi8(haystack_characters_from_1st, first_needle_character_vec); + const __m128i comparison_result_2nd = _mm_cmpeq_epi8(haystack_characters_from_2nd, second_needle_character_vec); + const __m128i comparison_result_combined = _mm_and_si128(comparison_result_1st, comparison_result_2nd); + const uint16_t comparison_result_mask = _mm_movemask_epi8(comparison_result_combined); + /// If the mask = 0, then first two characters [0:1] from needle are not in the [0:17] bytes of haystack. + if (comparison_result_mask == 0) { haystack += n; continue; } - const auto offset = __builtin_ctz(mask); + const auto offset = std::countr_zero(comparison_result_mask); haystack += offset; if (haystack + n <= haystack_end && pageSafe(haystack)) { - /// check for first 16 octets - const auto v_haystack_offset = _mm_loadu_si128(reinterpret_cast(haystack)); - const auto v_against_cache = _mm_cmpeq_epi8(v_haystack_offset, cache); - const auto mask_offset = _mm_movemask_epi8(v_against_cache); + /// Already find the haystack position where the [pos:pos + 1] two characters exactly match the first two characters of needle. + /// Compare the 16 bytes from needle (cache) and the first 16 bytes from haystack at once if the haystack size >= 16 bytes. + const __m128i haystack_characters = _mm_loadu_si128(reinterpret_cast(haystack)); + const __m128i comparison_result_cache = _mm_cmpeq_epi8(haystack_characters, cache); + const uint16_t mask_offset = _mm_movemask_epi8(comparison_result_cache); if (0xffff == cachemask) { @@ -758,7 +762,7 @@ public: if (haystack == haystack_end) return haystack_end; - if (*haystack == first) + if (*haystack == first_needle_character) { const auto * haystack_pos = haystack + 1; const auto * needle_pos = needle + 1; From fadbeb8ebd20310d6b1ad258180c7205c2b1118b Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 22 Feb 2023 19:25:48 +0000 Subject: [PATCH 222/253] T64 codec support for IPv4 --- src/Compression/CompressionCodecT64.cpp | 5 +++++ tests/queries/0_stateless/25338_ipv4_codec_t64.reference | 0 tests/queries/0_stateless/25338_ipv4_codec_t64.sql | 3 +++ 3 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/25338_ipv4_codec_t64.reference create mode 100644 tests/queries/0_stateless/25338_ipv4_codec_t64.sql diff --git a/src/Compression/CompressionCodecT64.cpp b/src/Compression/CompressionCodecT64.cpp index e718b24e8df..e7f1615128a 100644 --- a/src/Compression/CompressionCodecT64.cpp +++ b/src/Compression/CompressionCodecT64.cpp @@ -88,6 +88,7 @@ enum class MagicNumber : uint8_t Enum16 = 18, Decimal32 = 19, Decimal64 = 20, + IPv4 = 21, }; MagicNumber serializeTypeId(TypeIndex type_id) @@ -109,6 +110,7 @@ MagicNumber serializeTypeId(TypeIndex type_id) case TypeIndex::Enum16: return MagicNumber::Enum16; case TypeIndex::Decimal32: return MagicNumber::Decimal32; case TypeIndex::Decimal64: return MagicNumber::Decimal64; + case TypeIndex::IPv4: return MagicNumber::IPv4; default: break; } @@ -136,6 +138,7 @@ TypeIndex deserializeTypeId(uint8_t serialized_type_id) case MagicNumber::Enum16: return TypeIndex::Enum16; case MagicNumber::Decimal32: return TypeIndex::Decimal32; case MagicNumber::Decimal64: return TypeIndex::Decimal64; + case MagicNumber::IPv4: return TypeIndex::IPv4; } throw Exception(ErrorCodes::LOGICAL_ERROR, "Bad magic number in T64 codec: {}", static_cast(serialized_type_id)); @@ -171,6 +174,7 @@ TypeIndex baseType(TypeIndex type_idx) return TypeIndex::UInt16; case TypeIndex::UInt32: case TypeIndex::DateTime: + case TypeIndex::IPv4: return TypeIndex::UInt32; case TypeIndex::UInt64: return TypeIndex::UInt64; @@ -198,6 +202,7 @@ TypeIndex typeIdx(const IDataType * data_type) case TypeIndex::Date: case TypeIndex::Int32: case TypeIndex::UInt32: + case TypeIndex::IPv4: case TypeIndex::DateTime: case TypeIndex::DateTime64: case TypeIndex::Decimal32: diff --git a/tests/queries/0_stateless/25338_ipv4_codec_t64.reference b/tests/queries/0_stateless/25338_ipv4_codec_t64.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/25338_ipv4_codec_t64.sql b/tests/queries/0_stateless/25338_ipv4_codec_t64.sql new file mode 100644 index 00000000000..63a19cba5a2 --- /dev/null +++ b/tests/queries/0_stateless/25338_ipv4_codec_t64.sql @@ -0,0 +1,3 @@ +DROP TABLE IF EXISTS ipv4_t64; +CREATE TABLE ipv4_t64 (uid Int16, ip IPv4 CODEC(T64), INDEX ip_idx ip TYPE bloom_filter GRANULARITY 4) ENGINE=MergeTree ORDER BY uid; +DROP TABLE IF EXISTS ipv4_t64; From 18b93fc21276aa25ef723c23f5d482223823ab0e Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Wed, 22 Feb 2023 12:41:00 -0800 Subject: [PATCH 223/253] More refactoring for better style --- src/Functions/FunctionsHashing.h | 57 ++++++++++++-------------------- 1 file changed, 21 insertions(+), 36 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 59d573df3d1..1cefa566918 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1042,31 +1042,23 @@ private: } else { - if constexpr (std::endian::native == std::endian::little) - { - if (std::is_same_v) - h = JavaHashImpl::apply(vec_from[i]); - else - h = apply(key, reinterpret_cast(&vec_from[i]), sizeof(vec_from[i])); - } + if (std::is_same_v) + h = JavaHashImpl::apply(vec_from[i]); else { - if (std::is_same_v) - h = JavaHashImpl::apply(vec_from[i]); + if constexpr (std::is_same_v) + { + UInt64 v = bit_cast(vec_from[i]); + if constexpr (std::endian::native == std::endian::big) + v = __builtin_bswap64(v); + h = apply(key, reinterpret_cast(&v), sizeof(vec_from[i])); + } else { - if constexpr (std::is_same_v) - { - UInt64 v = bit_cast(vec_from[i]); - v = __builtin_bswap64(v); - h = apply(key, reinterpret_cast(&v), sizeof(vec_from[i])); - } - else - { - UInt32 v = bit_cast(vec_from[i]); + UInt32 v = bit_cast(vec_from[i]); + if constexpr (std::endian::native == std::endian::big) v = __builtin_bswap32(v); - h = apply(key, reinterpret_cast(&v), sizeof(vec_from[i])); - } + h = apply(key, reinterpret_cast(&v), sizeof(vec_from[i])); } } } @@ -1081,28 +1073,21 @@ private: { auto value = col_from_const->template getValue(); ToType hash; - if constexpr (std::endian::native == std::endian::little) + if constexpr (std::is_same_v) { - if constexpr (std::is_same_v) - hash = IntHash64Impl::apply(bit_cast(value)); - else - hash = IntHash32Impl::apply(bit_cast(value)); + UInt64 v = bit_cast(value); + if constexpr (std::endian::native == std::endian::big) + v = __builtin_bswap64(v); + hash = IntHash64Impl::apply(v); } else { - if constexpr (std::is_same_v) - { - UInt64 v = bit_cast(value); - v = __builtin_bswap64(v); - hash = IntHash64Impl::apply(v); - } - else - { - UInt32 v = bit_cast(value); + UInt32 v = bit_cast(value); + if constexpr (std::endian::native == std::endian::big) v = __builtin_bswap32(v); - hash = IntHash32Impl::apply(bit_cast(v)); - } + hash = IntHash32Impl::apply(bit_cast(v)); } + size_t size = vec_to.size(); if constexpr (first) { From e979a8660417c50139e659ed7de09a888497d3eb Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Wed, 22 Feb 2023 12:50:46 -0800 Subject: [PATCH 224/253] Minor change for adding constexpr --- src/Functions/FunctionsHashing.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 1cefa566918..25e2ac5207a 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1042,7 +1042,7 @@ private: } else { - if (std::is_same_v) + if constexpr (std::is_same_v) h = JavaHashImpl::apply(vec_from[i]); else { From d6710d9b34ed223cd8306c738b9b05186b6e9172 Mon Sep 17 00:00:00 2001 From: Jiebin Sun Date: Wed, 22 Feb 2023 16:15:16 -0500 Subject: [PATCH 225/253] Align all the SSE4.1 requirement and use needle_size Align all the SSE4.1 requirement from StringSearcher. Use needle_size in while loop to make the code clean. Signed-off-by: Jiebin Sun --- src/Common/StringSearcher.h | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Common/StringSearcher.h b/src/Common/StringSearcher.h index 75ef7b9e7ca..a80c8c73292 100644 --- a/src/Common/StringSearcher.h +++ b/src/Common/StringSearcher.h @@ -568,7 +568,7 @@ private: /// first character in `needle` uint8_t first_needle_character = 0; -#ifdef __SSE2__ +#ifdef __SSE4_1__ /// second character of "needle" (if its length is > 1) uint8_t second_needle_character = 0; /// The first/second needle character broadcasted into a 16 bytes vector @@ -664,12 +664,14 @@ public: requires (sizeof(CharT) == 1) const CharT * search(const CharT * haystack, const CharT * const haystack_end) const { + const auto needle_size = needle_end - needle; + if (needle == needle_end) return haystack; #ifdef __SSE4_1__ /// Fast path for single-character needles. Compare 16 characters of the haystack against the needle character at once. - if (needle + 1 == needle_end) + if (needle_size == 1) { while (haystack < haystack_end) { @@ -703,7 +705,7 @@ public: } #endif - while (haystack < haystack_end && haystack_end - haystack >= needle_end - needle) + while (haystack < haystack_end && haystack_end - haystack >= needle_size) { #ifdef __SSE4_1__ /// Compare the [0:15] bytes from haystack and broadcasted 16 bytes vector from first character of needle. From d5864fa88e9e57da22305aef516900fc078e08e4 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 22 Feb 2023 21:52:49 +0000 Subject: [PATCH 226/253] allow to fallback from async insert in case of large amount of data --- src/Interpreters/AsynchronousInsertQueue.cpp | 35 ++++++++++- src/Interpreters/AsynchronousInsertQueue.h | 20 +++++- src/Interpreters/InterpreterInsertQuery.cpp | 6 +- src/Interpreters/InterpreterInsertQuery.h | 4 ++ src/Interpreters/executeQuery.cpp | 63 +++++++++++++------ .../getSourceFromASTInsertQuery.cpp | 21 +++++-- .../Transforms/getSourceFromASTInsertQuery.h | 28 ++++++--- ...02668_fallback_from_async_insert.reference | 23 +++++++ .../02668_fallback_from_async_insert.sh | 52 +++++++++++++++ 9 files changed, 212 insertions(+), 40 deletions(-) create mode 100644 tests/queries/0_stateless/02668_fallback_from_async_insert.reference create mode 100755 tests/queries/0_stateless/02668_fallback_from_async_insert.sh diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 293849b88b7..992705f1c3c 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include #include @@ -181,7 +182,8 @@ void AsynchronousInsertQueue::scheduleDataProcessingJob(const InsertQuery & key, }); } -std::future AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context) +AsynchronousInsertQueue::PushResult +AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context) { query = query->clone(); const auto & settings = query_context->getSettingsRef(); @@ -201,9 +203,32 @@ std::future AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_c String bytes; { + /// Read at most 'async_insert_max_data_size' bytes of data. + /// If limit is exceeded we will fallback to synchronous insert + /// to avoid buffering of huge amount of data in memory. + auto read_buf = getReadBufferFromASTInsertQuery(query); + LimitReadBuffer limit_buf(*read_buf, settings.async_insert_max_data_size, false); + WriteBufferFromString write_buf(bytes); - copyData(*read_buf, write_buf); + copyData(limit_buf, write_buf); + + if (!read_buf->eof()) + { + write_buf.finalize(); + + /// Concat read buffer with already extracted from insert + /// query data and with the rest data from insert query. + std::vector> buffers; + buffers.emplace_back(std::make_unique(bytes)); + buffers.emplace_back(std::move(read_buf)); + + return PushResult + { + .status = PushResult::TOO_MUCH_DATA, + .insert_data_buffer = std::make_unique(std::move(buffers)), + }; + } } if (auto quota = query_context->getQuota()) @@ -263,7 +288,11 @@ std::future AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_c else shard.are_tasks_available.notify_one(); - return insert_future; + return PushResult + { + .status = PushResult::OK, + .future = std::move(insert_future), + }; } void AsynchronousInsertQueue::processBatchDeadlines(size_t shard_num) diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index ee1265673a6..23a2860364d 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -19,7 +19,25 @@ public: AsynchronousInsertQueue(ContextPtr context_, size_t pool_size_); ~AsynchronousInsertQueue(); - std::future push(ASTPtr query, ContextPtr query_context); + struct PushResult + { + enum Status + { + OK, + TOO_MUCH_DATA, + }; + + Status status; + + /// Future that allows to wait until the query is flushed. + std::future future; + + /// Read buffer that contains extracted + /// from query data in case of too much data. + std::unique_ptr insert_data_buffer; + }; + + PushResult push(ASTPtr query, ContextPtr query_context); size_t getPoolSize() const { return pool_size; } private: diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 2569e6ddc33..b4a19ea7403 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -540,7 +540,11 @@ BlockIO InterpreterInsertQuery::execute() if (query.hasInlinedData() && !async_insert) { /// can execute without additional data - auto pipe = getSourceFromASTInsertQuery(query_ptr, true, query_sample_block, getContext(), nullptr); + auto format = getInputFormatFromASTInsertQuery(query_ptr, true, query_sample_block, getContext(), nullptr); + for (auto && buffer : owned_buffers) + format->addBuffer(std::move(buffer)); + + auto pipe = getSourceFromInputFormat(query_ptr, std::move(format), getContext(), nullptr); res.pipeline.complete(std::move(pipe)); } } diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index 9b3f617e4b3..bb6509a9102 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -52,6 +52,8 @@ public: bool supportsTransactions() const override { return true; } + void addBuffer(std::unique_ptr buffer) { owned_buffers.push_back(std::move(buffer)); } + private: Block getSampleBlock(const Names & names, const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot) const; @@ -61,6 +63,8 @@ private: const bool no_destination; const bool async_insert; + std::vector> owned_buffers; + Chain buildChainImpl( const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index ca7544df4b9..c233060e646 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -590,6 +590,7 @@ static std::tuple executeQueryImpl( bool async_insert = false; auto * queue = context->getAsynchronousInsertQueue(); + auto * logger = &Poco::Logger::get("executeQuery"); if (insert_query && settings.async_insert) { @@ -605,41 +606,62 @@ static std::tuple executeQueryImpl( async_insert = true; if (!async_insert) - { - LOG_DEBUG(&Poco::Logger::get("executeQuery"), - "Setting async_insert=1, but INSERT query will be executed synchronously (reason: {})", reason); - } + LOG_DEBUG(logger, "Setting async_insert=1, but INSERT query will be executed synchronously (reason: {})", reason); } + bool quota_checked = false; + std::unique_ptr insert_data_buffer_holder; + if (async_insert) { + if (context->getCurrentTransaction() && settings.throw_on_unsupported_query_inside_transaction) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Async inserts inside transactions are not supported"); + if (settings.implicit_transaction && settings.throw_on_unsupported_query_inside_transaction) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Async inserts with 'implicit_transaction' are not supported"); + quota = context->getQuota(); if (quota) { + quota_checked = true; quota->used(QuotaType::QUERY_INSERTS, 1); quota->used(QuotaType::QUERIES, 1); quota->checkExceeded(QuotaType::ERRORS); } - auto insert_future = queue->push(ast, context); + auto result = queue->push(ast, context); - if (settings.wait_for_async_insert) + if (result.status == AsynchronousInsertQueue::PushResult::OK) { - auto timeout = settings.wait_for_async_insert_timeout.totalMilliseconds(); - auto source = std::make_shared(std::move(insert_future), timeout); - res.pipeline = QueryPipeline(Pipe(std::move(source))); + if (settings.wait_for_async_insert) + { + auto timeout = settings.wait_for_async_insert_timeout.totalMilliseconds(); + auto source = std::make_shared(std::move(result.future), timeout); + res.pipeline = QueryPipeline(Pipe(std::move(source))); + } + + const auto & table_id = insert_query->table_id; + if (!table_id.empty()) + context->setInsertionTable(table_id); } + else if (result.status == AsynchronousInsertQueue::PushResult::TOO_MUCH_DATA) + { + async_insert = false; + insert_data_buffer_holder = std::move(result.insert_data_buffer); - const auto & table_id = insert_query->table_id; - if (!table_id.empty()) - context->setInsertionTable(table_id); + if (insert_query->data) + { + /// Reset inlined data because it will be + /// available from tail read buffer. + insert_query->end = insert_query->data; + insert_query->data = nullptr; + } - if (context->getCurrentTransaction() && settings.throw_on_unsupported_query_inside_transaction) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Async inserts inside transactions are not supported"); - if (settings.implicit_transaction && settings.throw_on_unsupported_query_inside_transaction) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Async inserts with 'implicit_transaction' are not supported"); + insert_query->tail = insert_data_buffer_holder.get(); + LOG_DEBUG(logger, "Setting async_insert=1, but INSERT query will be executed synchronously because it has too much data"); + } } - else + + if (!async_insert) { /// We need to start the (implicit) transaction before getting the interpreter as this will get links to the latest snapshots if (!context->getCurrentTransaction() && settings.implicit_transaction && !ast->as()) @@ -671,7 +693,7 @@ static std::tuple executeQueryImpl( context->getSettingsRef().throw_on_unsupported_query_inside_transaction) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Transactions are not supported for this type of query ({})", ast->getID()); - if (!interpreter->ignoreQuota()) + if (!interpreter->ignoreQuota() && !quota_checked) { quota = context->getQuota(); if (quota) @@ -695,12 +717,15 @@ static std::tuple executeQueryImpl( limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode); } - if (const auto * insert_interpreter = typeid_cast(&*interpreter)) + if (auto * insert_interpreter = typeid_cast(&*interpreter)) { /// Save insertion table (not table function). TODO: support remote() table function. auto table_id = insert_interpreter->getDatabaseTable(); if (!table_id.empty()) context->setInsertionTable(std::move(table_id)); + + if (insert_data_buffer_holder) + insert_interpreter->addBuffer(std::move(insert_data_buffer_holder)); } { diff --git a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp index ab7cfca3de2..6c7c7447070 100644 --- a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp +++ b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp @@ -64,15 +64,13 @@ InputFormatPtr getInputFormatFromASTInsertQuery( return source; } -Pipe getSourceFromASTInsertQuery( +Pipe getSourceFromInputFormat( const ASTPtr & ast, - bool with_buffers, - const Block & header, + InputFormatPtr format, ContextPtr context, const ASTPtr & input_function) { - auto source = getInputFormatFromASTInsertQuery(ast, with_buffers, header, context, input_function); - Pipe pipe(source); + Pipe pipe(format); const auto * ast_insert_query = ast->as(); if (context->getSettingsRef().input_format_defaults_for_omitted_fields && ast_insert_query->table_id && !input_function) @@ -84,7 +82,7 @@ Pipe getSourceFromASTInsertQuery( { pipe.addSimpleTransform([&](const Block & cur_header) { - return std::make_shared(cur_header, columns, *source, context); + return std::make_shared(cur_header, columns, *format, context); }); } } @@ -92,6 +90,17 @@ Pipe getSourceFromASTInsertQuery( return pipe; } +Pipe getSourceFromASTInsertQuery( + const ASTPtr & ast, + bool with_buffers, + const Block & header, + ContextPtr context, + const ASTPtr & input_function) +{ + auto format = getInputFormatFromASTInsertQuery(ast, with_buffers, header, context, input_function); + return getSourceFromInputFormat(ast, std::move(format), std::move(context), input_function); +} + std::unique_ptr getReadBufferFromASTInsertQuery(const ASTPtr & ast) { const auto * insert_query = ast->as(); diff --git a/src/Processors/Transforms/getSourceFromASTInsertQuery.h b/src/Processors/Transforms/getSourceFromASTInsertQuery.h index 4a5ed952efc..dc541873972 100644 --- a/src/Processors/Transforms/getSourceFromASTInsertQuery.h +++ b/src/Processors/Transforms/getSourceFromASTInsertQuery.h @@ -14,19 +14,27 @@ class Pipe; /// Prepares a input format, which produce data containing in INSERT query. InputFormatPtr getInputFormatFromASTInsertQuery( - const ASTPtr & ast, - bool with_buffers, - const Block & header, - ContextPtr context, - const ASTPtr & input_function); + const ASTPtr & ast, + bool with_buffers, + const Block & header, + ContextPtr context, + const ASTPtr & input_function); + +/// Prepares a pipe from input format got from ASTInsertQuery, +/// which produce data containing in INSERT query. +Pipe getSourceFromInputFormat( + const ASTPtr & ast, + InputFormatPtr format, + ContextPtr context, + const ASTPtr & input_function); /// Prepares a pipe which produce data containing in INSERT query. Pipe getSourceFromASTInsertQuery( - const ASTPtr & ast, - bool with_buffers, - const Block & header, - ContextPtr context, - const ASTPtr & input_function); + const ASTPtr & ast, + bool with_buffers, + const Block & header, + ContextPtr context, + const ASTPtr & input_function); class ReadBuffer; diff --git a/tests/queries/0_stateless/02668_fallback_from_async_insert.reference b/tests/queries/0_stateless/02668_fallback_from_async_insert.reference new file mode 100644 index 00000000000..7aa58724b9e --- /dev/null +++ b/tests/queries/0_stateless/02668_fallback_from_async_insert.reference @@ -0,0 +1,23 @@ +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +16 +17 +18 +id_0 +id_1 +id_2 +id_3 +id_4 diff --git a/tests/queries/0_stateless/02668_fallback_from_async_insert.sh b/tests/queries/0_stateless/02668_fallback_from_async_insert.sh new file mode 100755 index 00000000000..9c158d6241b --- /dev/null +++ b/tests/queries/0_stateless/02668_fallback_from_async_insert.sh @@ -0,0 +1,52 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +message="INSERT query will be executed synchronously because it has too much data" + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS t_async_insert_fallback" +$CLICKHOUSE_CLIENT --query "CREATE TABLE t_async_insert_fallback (a UInt64) ENGINE = Memory" + +query_id_suffix="${CLICKHOUSE_DATABASE}_${RANDOM}" + +# inlined data via native protocol +$CLICKHOUSE_CLIENT \ + --query_id "0_$query_id_suffix" \ + --async_insert 1 \ + --async_insert_max_data_size 5 \ + --query "INSERT INTO t_async_insert_fallback VALUES (1) (2) (3)" + +# inlined data via http +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query_id=1_$query_id_suffix&async_insert=1&async_insert_max_data_size=3" \ + -d "INSERT INTO t_async_insert_fallback VALUES (4) (5) (6)" + +# partially inlined partially sent via post data +${CLICKHOUSE_CURL} -sS -X POST \ + "${CLICKHOUSE_URL}&query_id=2_$query_id_suffix&async_insert=1&async_insert_max_data_size=5&query=INSERT+INTO+t_async_insert_fallback+VALUES+(7)" \ + --data-binary @- <<< "(8) (9)" + +# partially inlined partially sent via post data +${CLICKHOUSE_CURL} -sS -X POST \ + "${CLICKHOUSE_URL}&query_id=3_$query_id_suffix&async_insert=1&async_insert_max_data_size=5&query=INSERT+INTO+t_async_insert_fallback+VALUES+(10)+(11)" \ + --data-binary @- <<< "(12)" + +# sent via post data +${CLICKHOUSE_CURL} -sS -X POST \ + "${CLICKHOUSE_URL}&query_id=4_$query_id_suffix&async_insert=1&async_insert_max_data_size=5&query=INSERT+INTO+t_async_insert_fallback+FORMAT+Values" \ + --data-binary @- <<< "(13) (14) (15)" + +# no limit for async insert size +${CLICKHOUSE_CURL} -sS -X POST \ + "${CLICKHOUSE_URL}&query_id=5_$query_id_suffix&async_insert=1&query=INSERT+INTO+t_async_insert_fallback+FORMAT+Values" \ + --data-binary @- <<< "(16) (17) (18)" + +$CLICKHOUSE_CLIENT --query "SELECT * FROM t_async_insert_fallback ORDER BY a" +$CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS" +$CLICKHOUSE_CLIENT --query " + SELECT 'id_' || splitByChar('_', query_id)[1] AS id FROM system.text_log + WHERE query_id LIKE '%$query_id_suffix' AND message LIKE '%$message%' +" + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS t_async_insert_fallback" From 776d4d9bb439022ed6d0356866b352a5a9f343db Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Wed, 22 Feb 2023 15:19:25 -0800 Subject: [PATCH 227/253] More refactoring --- src/Functions/FunctionsHashing.h | 24 ++++++++++-------------- 1 file changed, 10 insertions(+), 14 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 25e2ac5207a..7525153bd48 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1028,6 +1028,8 @@ private: if constexpr (std::is_same_v) { UInt64 v = bit_cast(vec_from[i]); + + /// Consider using std::byteswap(c++23) in the future if constexpr (std::endian::native == std::endian::big) v = __builtin_bswap64(v); h = IntHash64Impl::apply(v); @@ -1046,21 +1048,15 @@ private: h = JavaHashImpl::apply(vec_from[i]); else { - if constexpr (std::is_same_v) + FromType v = vec_from[i]; + if constexpr (std::endian::native == std::endian::big) { - UInt64 v = bit_cast(vec_from[i]); - if constexpr (std::endian::native == std::endian::big) - v = __builtin_bswap64(v); - h = apply(key, reinterpret_cast(&v), sizeof(vec_from[i])); + FromType tmp_v; + reverseMemcpy(&tmp_v, &v, sizeof(v)); + v = tmp_v; } - else - { - UInt32 v = bit_cast(vec_from[i]); - if constexpr (std::endian::native == std::endian::big) - v = __builtin_bswap32(v); - h = apply(key, reinterpret_cast(&v), sizeof(vec_from[i])); - } - } + h = apply(key, reinterpret_cast(&v), sizeof(v)); + } } if constexpr (first) @@ -1085,7 +1081,7 @@ private: UInt32 v = bit_cast(value); if constexpr (std::endian::native == std::endian::big) v = __builtin_bswap32(v); - hash = IntHash32Impl::apply(bit_cast(v)); + hash = IntHash32Impl::apply(v); } size_t size = vec_to.size(); From 904e6ae7690c1c8ff9ef59a77905cf4b23bcbc9b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 23 Feb 2023 00:45:51 +0100 Subject: [PATCH 228/253] On out-of-space `at` returns error, we must terminate still --- tests/ci/worker/init_runner.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh index 64f11b41777..c8b11bc6e37 100644 --- a/tests/ci/worker/init_runner.sh +++ b/tests/ci/worker/init_runner.sh @@ -66,7 +66,8 @@ terminate-and-exit() { INSTANCE_ID=$(ec2metadata --instance-id) # We execute it with at to not have it as an orphan process # GH Runners kill all remain processes - echo "sleep 10; aws ec2 terminate-instances --instance-ids $INSTANCE_ID" | at now + echo "sleep 10; aws ec2 terminate-instances --instance-ids $INSTANCE_ID" | at now || \ + aws ec2 terminate-instances --instance-ids "$INSTANCE_ID" # workaround for complete out of space exit 0 } From 7d837eea865bdc1cb25187e6a8342dd1c262ef62 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Feb 2023 08:15:14 +0300 Subject: [PATCH 229/253] Update FieldFromAST.cpp --- src/Parsers/FieldFromAST.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Parsers/FieldFromAST.cpp b/src/Parsers/FieldFromAST.cpp index c46a9a08e68..3cd10c1cf80 100644 --- a/src/Parsers/FieldFromAST.cpp +++ b/src/Parsers/FieldFromAST.cpp @@ -36,16 +36,16 @@ String FieldFromASTImpl::toString(bool show_secrets) const if (!show_secrets && isDiskFunction(ast)) { auto hidden = ast->clone(); - auto & disk_function = assert_cast(*hidden); - auto * disk_function_args_expr = assert_cast(disk_function.arguments.get()); - auto & disk_function_args = disk_function_args_expr->children; + const auto & disk_function = assert_cast(*hidden); + const auto * disk_function_args_expr = assert_cast(disk_function.arguments.get()); + const auto & disk_function_args = disk_function_args_expr->children; auto is_secret_arg = [](const std::string & arg_name) { return arg_name != "type"; }; - for (auto & arg : disk_function_args) + for (const auto & arg : disk_function_args) { auto * setting_function = arg->as(); if (!setting_function || setting_function->name != "equals") From f2509ba9c644d0447b13d83e9e85f0946e55a852 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Feb 2023 08:00:17 +0100 Subject: [PATCH 230/253] Add changelog for 23.2 --- CHANGELOG.md | 183 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 183 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index a89619aa7ca..0588e439b12 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,9 +1,192 @@ ### Table of Contents +**[ClickHouse release v23.2, 2023-02-23](#232)**
**[ClickHouse release v23.1, 2023-01-25](#231)**
**[Changelog for 2022](https://clickhouse.com/docs/en/whats-new/changelog/2022/)**
# 2023 Changelog +### ClickHouse release 23.2, 2023-02-23 + +#### Backward Incompatible Change +* Extend function "toDayOfWeek()" (alias: "DAYOFWEEK") with a mode argument that encodes whether the week starts on Monday or Sunday and whether counting starts at 0 or 1. For consistency with other date time functions, the mode argument was inserted between the time and the time zone arguments. This breaks existing usage of the (previously undocumented) 2-argument syntax "toDayOfWeek(time, time_zone)". A fix is to rewrite the function into "toDayOfWeek(time, 0, time_zone)". [#45233](https://github.com/ClickHouse/ClickHouse/pull/45233) ([Robert Schulze](https://github.com/rschu1ze)). +* Rename setting `max_query_cache_size` to `filesystem_cache_max_download_size`. [#45614](https://github.com/ClickHouse/ClickHouse/pull/45614) ([Kseniia Sumarokova](https://github.com/kssenii)). +* The default user will not have permissions for access type `SHOW NAMED COLLECTION` by default (e.g. by default, default user will not longer be able to do grant ALL to other users as it was before, therefore this PR is backward incompatible). [#46010](https://github.com/ClickHouse/ClickHouse/pull/46010) ([Kseniia Sumarokova](https://github.com/kssenii)). +* If the SETTINGS clause is specified before the FORMAT clause, the settings will be applied to formatting as well. [#46003](https://github.com/ClickHouse/ClickHouse/pull/46003) ([Azat Khuzhin](https://github.com/azat)). +* Remove support for setting `materialized_postgresql_allow_automatic_update` (which was by default turned off). Fix integration tests. [#46106](https://github.com/ClickHouse/ClickHouse/pull/46106) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Slightly improve performance of `countDigits` on realistic datasets. This closed [#44518](https://github.com/ClickHouse/ClickHouse/issues/44518). In previous versions, `countDigits(0)` returned `0`; now it returns `1`, which is more correct, and follows the existing documentation. [#46187](https://github.com/ClickHouse/ClickHouse/pull/46187) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Disallow creation of new columns compressed by a combination of codecs "Delta" or "DoubleDelta" followed by codecs "Gorilla" or "FPC". This can be bypassed using setting "allow_suspicious_codecs = true". [#45652](https://github.com/ClickHouse/ClickHouse/pull/45652) ([Robert Schulze](https://github.com/rschu1ze)). + +#### New Feature +* Add `StorageIceberg` and table function `iceberg` to access iceberg table store on S3. [#45384](https://github.com/ClickHouse/ClickHouse/pull/45384) ([flynn](https://github.com/ucasfl)). +* Allow configuring storage as `SETTINGS disk = ''` (instead of `storage_policy`) and with explicit disk creation `SETTINGS disk = disk(type=s3, ...)`. [#41976](https://github.com/ClickHouse/ClickHouse/pull/41976) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Expose `ProfileEvents counters in `system.part_log`. [#38614](https://github.com/ClickHouse/ClickHouse/pull/38614) ([Bharat Nallan](https://github.com/bharatnc)). +* Enrichment of the existing `ReplacingMergeTree` engine to allow duplicate the insertion. It leverages the power of both `ReplacingMergeTree` and `CollapsingMergeTree` in one MergeTree engine. Deleted data are not returned when queried, but not removed from disk neither. [#41005](https://github.com/ClickHouse/ClickHouse/pull/41005) ([youennL-cs](https://github.com/youennL-cs)). +* Add `generateULID` function. Closes [#36536](https://github.com/ClickHouse/ClickHouse/issues/36536). [#44662](https://github.com/ClickHouse/ClickHouse/pull/44662) ([Nikolay Degterinsky](https://github.com/evillique)). +* Add `corrMatrix` aggregate function, calculating each two columns. In addition, since Aggregatefunctions `covarSamp` and `covarPop` are similar to `corr`, I add `covarSampMatrix`, `covarPopMatrix` by the way. @alexey-milovidov closes [#44587](https://github.com/ClickHouse/ClickHouse/issues/44587). [#44680](https://github.com/ClickHouse/ClickHouse/pull/44680) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* Introduce arrayShuffle function for random array permutations. [#45271](https://github.com/ClickHouse/ClickHouse/pull/45271) ([Joanna Hulboj](https://github.com/jh0x)). +* Support types `FIXED_SIZE_BINARY` type in Arrow, `FIXED_LENGTH_BYTE_ARRAY` in `Parquet` and match them to `FixedString`. Add settings `output_format_parquet_fixed_string_as_fixed_byte_array/output_format_arrow_fixed_string_as_fixed_byte_array` to control default output type for FixedString. Closes [#45326](https://github.com/ClickHouse/ClickHouse/issues/45326). [#45340](https://github.com/ClickHouse/ClickHouse/pull/45340) ([Kruglov Pavel](https://github.com/Avogar)). +* Add a new column `last_exception_time` to system.replication_queue. [#45457](https://github.com/ClickHouse/ClickHouse/pull/45457) ([Frank Chen](https://github.com/FrankChen021)). +* Add two new functions which allow for user-defined keys/seeds with SipHash{64,128}. [#45513](https://github.com/ClickHouse/ClickHouse/pull/45513) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Allow a three-argument version for table function `format`. close [#45808](https://github.com/ClickHouse/ClickHouse/issues/45808). [#45873](https://github.com/ClickHouse/ClickHouse/pull/45873) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* add `JodaTime` format support for 'x','w','S'. Refer to https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html. [#46073](https://github.com/ClickHouse/ClickHouse/pull/46073) ([zk_kiger](https://github.com/zk-kiger)). +* Support window function `ntile`. +* Add setting `final` to implicitly apply the `FINAL` modifier to every table. [#40945](https://github.com/ClickHouse/ClickHouse/pull/40945) ([Arthur Passos](https://github.com/arthurpassos)). +* Added `arrayPartialSort` and `arrayPartialReverseSort` functions. [#46296](https://github.com/ClickHouse/ClickHouse/pull/46296) ([Joanna Hulboj](https://github.com/jh0x)). +* The new http parameter `client_protocol_version` allows setting a client protocol version for HTTP responses using the Native format. [#40397](https://github.com/ClickHouse/ClickHouse/issues/40397). [#46360](https://github.com/ClickHouse/ClickHouse/pull/46360) ([Geoff Genz](https://github.com/genzgd)). +* Add new function `regexpExtract`, like spark function `REGEXP_EXTRACT` for compatibility. It is similar to the existing function `extract`. [#46469](https://github.com/ClickHouse/ClickHouse/pull/46469) ([李扬](https://github.com/taiyang-li)). +* Add new function `JSONArrayLength`, which returns the number of elements in the outermost JSON array. The function returns NULL if the input JSON string is invalid. [#46631](https://github.com/ClickHouse/ClickHouse/pull/46631) ([李扬](https://github.com/taiyang-li)). + +#### Performance Improvement +* The introduced logic works if PREWHERE condition is a conjunction of multiple conditions (cond1 AND cond2 AND ... ). It groups those conditions that require reading the same columns into steps. After each step the corresponding part of the full condition is computed and the result rows might be filtered. This allows to read fewer rows in the next steps thus saving IO bandwidth and doing less computation. This logic is disabled by default for now. It will be enabled by default in one of the future releases once it is known to not have any regressions, so it is highly encouraged to be used for testing. It can be controlled by 2 settings: "enable_multiple_prewhere_read_steps" and "move_all_conditions_to_prewhere". [#46140](https://github.com/ClickHouse/ClickHouse/pull/46140) ([Alexander Gololobov](https://github.com/davenger)). +* An option added to aggregate partitions independently if table partition key and group by key are compatible. Controlled by the setting `allow_aggregate_partitions_independently`. Disabled by default because of limited applicability (please refer to the docs). [#45364](https://github.com/ClickHouse/ClickHouse/pull/45364) ([Nikita Taranov](https://github.com/nickitat)). +* Allow using Vertical merge algorithm with parts in Compact format. This will allow ClickHouse server to use much less memory for background operations. This closes [#46084](https://github.com/ClickHouse/ClickHouse/issues/46084). [#45681](https://github.com/ClickHouse/ClickHouse/pull/45681) [#46282](https://github.com/ClickHouse/ClickHouse/pull/46282) ([Anton Popov](https://github.com/CurtizJ)). +* Optimize `Parquet` reader by using batch reader. [#45878](https://github.com/ClickHouse/ClickHouse/pull/45878) ([LiuNeng](https://github.com/liuneng1994)). +* Add new `local_filesystem_read_method` method `io_uring` based on the asynchronous Linux [io_uring](https://kernel.dk/io_uring.pdf) subsystem, improving read performance almost universally compared to the default `pread` method. [#38456](https://github.com/ClickHouse/ClickHouse/pull/38456) ([Saulius Valatka](https://github.com/sauliusvl)). +* Rewrite aggregate functions with `if` expression as argument when logically equivalent. For example, `avg(if(cond, col, null))` can be rewritten to avgIf(cond, col). It is helpful in performance. [#44730](https://github.com/ClickHouse/ClickHouse/pull/44730) ([李扬](https://github.com/taiyang-li)). +* Improve lower/upper function performance with avx512 instructions. [#37894](https://github.com/ClickHouse/ClickHouse/pull/37894) ([yaqi-zhao](https://github.com/yaqi-zhao)). +* Remove the limitation that on systems with >=32 cores and SMT disabled ClickHouse uses only half of the cores (the case when you disable Hyper Threading in BIOS). [#44973](https://github.com/ClickHouse/ClickHouse/pull/44973) ([Robert Schulze](https://github.com/rschu1ze)). +* Improve performance of function `multiIf` by columnar executing, speed up by 2.3x. [#45296](https://github.com/ClickHouse/ClickHouse/pull/45296) ([李扬](https://github.com/taiyang-li)). +* Add fast path for function `position` when the needle is empty. [#45382](https://github.com/ClickHouse/ClickHouse/pull/45382) ([李扬](https://github.com/taiyang-li)). +* Enable `query_plan_remove_redundant_sorting` optimization by default. Optimization implemented in [#45420](https://github.com/ClickHouse/ClickHouse/issues/45420). [#45567](https://github.com/ClickHouse/ClickHouse/pull/45567) ([Igor Nikonov](https://github.com/devcrafter)). +* Increased HTTP Transfer Encoding chunk size to improve performance of large queries using the HTTP interface. [#45593](https://github.com/ClickHouse/ClickHouse/pull/45593) ([Geoff Genz](https://github.com/genzgd)). +* Fixed performance of short `SELECT` queries that read from tables with large number of `Array`/`Map`/`Nested` columns. [#45630](https://github.com/ClickHouse/ClickHouse/pull/45630) ([Anton Popov](https://github.com/CurtizJ)). +* Improve performance of filtering for big integers and decimal types. [#45949](https://github.com/ClickHouse/ClickHouse/pull/45949) ([李扬](https://github.com/taiyang-li)). +* This change could effectively reduce the overhead of obtaining the filter from ColumnNullable(UInt8) and improve the overall query performance. To evaluate the impact of this change, we adopted TPC-H benchmark but revised the column types from non-nullable to nullable, and we measured the QPS of its queries as the performance indicator. [#45962](https://github.com/ClickHouse/ClickHouse/pull/45962) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). +* Make the `_part` and `_partition_id` virtual column be `LowCardinality(String)` type. Closes [#45964](https://github.com/ClickHouse/ClickHouse/issues/45964). [#45975](https://github.com/ClickHouse/ClickHouse/pull/45975) ([flynn](https://github.com/ucasfl)). +* Improve the performance of Decimal conversion when the scale does not change. [#46095](https://github.com/ClickHouse/ClickHouse/pull/46095) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Allow to increase prefetching for read data. [#46168](https://github.com/ClickHouse/ClickHouse/pull/46168) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Rewrite `arrayExists(x -> x = 1, arr)` -> `has(arr, 1)`, which improve performance by 1.34x. [#46188](https://github.com/ClickHouse/ClickHouse/pull/46188) ([李扬](https://github.com/taiyang-li)). +* Fix too big memory usage for vertical merges on non-remote disk. Respect `max_insert_delayed_streams_for_parallel_write` for the remote disk. [#46275](https://github.com/ClickHouse/ClickHouse/pull/46275) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Update zstd to v1.5.4. It has some minor improvements in performance and compression ratio. If you run replicas with different versions of ClickHouse you may see reasonable error messages `Data after merge/mutation is not byte-identical to data on another replicas.` with explanation. These messages are Ok and you should not worry. [#46280](https://github.com/ClickHouse/ClickHouse/pull/46280) ([Raúl Marín](https://github.com/Algunenano)). +* Fix performance degradation caused by [#39737](https://github.com/ClickHouse/ClickHouse/issues/39737). [#46309](https://github.com/ClickHouse/ClickHouse/pull/46309) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The `replicas_status` handle will answer quickly even in case of a large replication queue. [#46310](https://github.com/ClickHouse/ClickHouse/pull/46310) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add avx512 support for aggregate function `sum`, function unary arithmetic, function comparison. [#37870](https://github.com/ClickHouse/ClickHouse/pull/37870) ([zhao zhou](https://github.com/zzachimed)). +* Rewrote the code around marks distribution and the overall coordination of the reading in order to achieve the maximum performance improvement. This closes [#34527](https://github.com/ClickHouse/ClickHouse/issues/34527). [#43772](https://github.com/ClickHouse/ClickHouse/pull/43772) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Remove redundant DISTINCT clauses in query (subqueries). Implemented on top of query plan. It does similar optimization as `optimize_duplicate_order_by_and_distinct` regarding DISTINCT clauses. Can be enabled via `query_plan_remove_redundant_distinct` setting. Related to [#42648](https://github.com/ClickHouse/ClickHouse/issues/42648). [#44176](https://github.com/ClickHouse/ClickHouse/pull/44176) ([Igor Nikonov](https://github.com/devcrafter)). +* A few query rewrite optimizations: `sumIf(123, cond) -> 123 * countIf(1, cond)`, `sum(if(cond, 123, 0)) -> 123 * countIf(cond)`, `sum(if(cond, 0, 123)) -> 123 * countIf(not(cond))` [#44728](https://github.com/ClickHouse/ClickHouse/pull/44728) ([李扬](https://github.com/taiyang-li)). +* Improved how memory bound merging and aggregation in order on top query plan interact. Previously we fell back to explicit sorting for AIO in some cases when it wasn't actually needed. [#45892](https://github.com/ClickHouse/ClickHouse/pull/45892) ([Nikita Taranov](https://github.com/nickitat)). +* Concurrent merges are scheduled using round-robin by default to ensure fair and starvation-free operation. Previously in heavily overloaded shards, big merges could possibly be starved by smaller merges due to the use of strict priority scheduling. Added `background_merges_mutations_scheduling_policy` server config option to select scheduling algorithm (`round_robin` or `shortest_task_first`). [#46247](https://github.com/ClickHouse/ClickHouse/pull/46247) ([Sergei Trifonov](https://github.com/serxa)). + +#### Improvement +* Enable retries for INSERT by default in case of ZooKeeper session loss. We already use it in production. [#46308](https://github.com/ClickHouse/ClickHouse/pull/46308) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add ability to ignore unknown keys in JSON object for named tuples (`input_format_json_ignore_unknown_keys_in_named_tuple`). [#45678](https://github.com/ClickHouse/ClickHouse/pull/45678) ([Azat Khuzhin](https://github.com/azat)). +* Support optimizing the `where` clause with sorting key expression move to `prewhere` for query with `final`. [#38893](https://github.com/ClickHouse/ClickHouse/issues/38893). [#38950](https://github.com/ClickHouse/ClickHouse/pull/38950) ([hexiaoting](https://github.com/hexiaoting)). +* Add new metrics for backups: num_processed_files and processed_files_size described actual number of processed files. [#42244](https://github.com/ClickHouse/ClickHouse/pull/42244) ([Aleksandr](https://github.com/AVMusorin)). +* Added retries on interserver DNS errors. [#43179](https://github.com/ClickHouse/ClickHouse/pull/43179) ([Anton Kozlov](https://github.com/tonickkozlov)). +* Keeper improvement: try preallocating space on the disk to avoid undefined out-of-space issues. Introduce setting `max_log_file_size` for the maximum size of Keeper's Raft log files. [#44370](https://github.com/ClickHouse/ClickHouse/pull/44370) ([Antonio Andelic](https://github.com/antonio2368)). +* Optimize behavior for a replica delay api logic in case the replica is read-only. [#45148](https://github.com/ClickHouse/ClickHouse/pull/45148) ([mateng915](https://github.com/mateng0915)). +* Ask for the password in clickhouse-client interactively in a case when the empty password is wrong. Closes [#46702](https://github.com/ClickHouse/ClickHouse/issues/46702). [#46730](https://github.com/ClickHouse/ClickHouse/pull/46730) ([Nikolay Degterinsky](https://github.com/evillique)). +* Mark `Gorilla` compression on columns of non-Float* type as suspicious. [#45376](https://github.com/ClickHouse/ClickHouse/pull/45376) ([Robert Schulze](https://github.com/rschu1ze)). +* Show replica name that is executing a merge in the `postpone_reason` column. [#45458](https://github.com/ClickHouse/ClickHouse/pull/45458) ([Frank Chen](https://github.com/FrankChen021)). +* Save exception stack trace in part_log. [#45459](https://github.com/ClickHouse/ClickHouse/pull/45459) ([Frank Chen](https://github.com/FrankChen021)). +* The `regexp_tree` dictionary is polished and now it is compatible with https://github.com/ua-parser/uap-core. [#45631](https://github.com/ClickHouse/ClickHouse/pull/45631) ([Han Fei](https://github.com/hanfei1991)). +* Updated checking of `SYSTEM SYNC REPLICA`, resolves [#45508](https://github.com/ClickHouse/ClickHouse/issues/45508) [#45648](https://github.com/ClickHouse/ClickHouse/pull/45648) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Rename setting `replication_alter_partitions_sync` to `alter_sync`. [#45659](https://github.com/ClickHouse/ClickHouse/pull/45659) ([Antonio Andelic](https://github.com/antonio2368)). +* The `generateRandom` table function and the engine now support `LowCardinality` data types. This is useful for testing, for example you can write `INSERT INTO table SELECT * FROM generateRandom() LIMIT 1000`. This is needed to debug [#45590](https://github.com/ClickHouse/ClickHouse/issues/45590). [#45661](https://github.com/ClickHouse/ClickHouse/pull/45661) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The experimental query result cache now provides more modular configuration settings. [#45679](https://github.com/ClickHouse/ClickHouse/pull/45679) ([Robert Schulze](https://github.com/rschu1ze)). +* Renamed "query result cache" to "query cache". [#45682](https://github.com/ClickHouse/ClickHouse/pull/45682) ([Robert Schulze](https://github.com/rschu1ze)). +* add `SYSTEM SYNC FILE CACHE` command. It will do the `sync` syscall. [#8921](https://github.com/ClickHouse/ClickHouse/issues/8921). [#45685](https://github.com/ClickHouse/ClickHouse/pull/45685) ([DR](https://github.com/freedomDR)). +* Add a new S3 setting `allow_head_object_request`. This PR makes usage of `GetObjectAttributes` request instead of `HeadObject` introduced in https://github.com/ClickHouse/ClickHouse/pull/45288 optional (and disabled by default). [#45701](https://github.com/ClickHouse/ClickHouse/pull/45701) ([Vitaly Baranov](https://github.com/vitlibar)). +* Add ability to override connection settings based on connection names (that said that now you can forget about storing password for each connection, you can simply put everything into `~/.clickhouse-client/config.xml` and even use different history files for them, which can be also useful). [#45715](https://github.com/ClickHouse/ClickHouse/pull/45715) ([Azat Khuzhin](https://github.com/azat)). +* Arrow format: support the duration type. Closes [#45669](https://github.com/ClickHouse/ClickHouse/issues/45669). [#45750](https://github.com/ClickHouse/ClickHouse/pull/45750) ([flynn](https://github.com/ucasfl)). +* Extend the logging in the Query Cache to improve investigations of the caching behavior. [#45751](https://github.com/ClickHouse/ClickHouse/pull/45751) ([Robert Schulze](https://github.com/rschu1ze)). +* The query cache's server-level settings are now reconfigurable at runtime. [#45758](https://github.com/ClickHouse/ClickHouse/pull/45758) ([Robert Schulze](https://github.com/rschu1ze)). +* Hide password in logs when a table function's arguments are specified with a named collection. [#45774](https://github.com/ClickHouse/ClickHouse/pull/45774) ([Vitaly Baranov](https://github.com/vitlibar)). +* Improve internal S3 client to correctly deduce regions and redirections for different types of URLs. [#45783](https://github.com/ClickHouse/ClickHouse/pull/45783) ([Antonio Andelic](https://github.com/antonio2368)). +* Add support for Map, IPv4 and IPv6 types in generateRandom. Mostly useful for testing. [#45785](https://github.com/ClickHouse/ClickHouse/pull/45785) ([Raúl Marín](https://github.com/Algunenano)). +* Support empty/notEmpty for IP types. [#45799](https://github.com/ClickHouse/ClickHouse/pull/45799) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* The column `num_processed_files` was split into two columns: `num_files` (for BACKUP) and `files_read` (for RESTORE). The column `processed_files_size` was split into two columns: `total_size` (for BACKUP) and `bytes_read` (for RESTORE). [#45800](https://github.com/ClickHouse/ClickHouse/pull/45800) ([Vitaly Baranov](https://github.com/vitlibar)). +* Add support for `SHOW ENGINES` query for MySQL compatibility. [#45859](https://github.com/ClickHouse/ClickHouse/pull/45859) ([Filatenkov Artur](https://github.com/FArthur-cmd)). +* Improved how the obfuscator deals with queries. [#45867](https://github.com/ClickHouse/ClickHouse/pull/45867) ([Raúl Marín](https://github.com/Algunenano)). +* Improve behaviour of conversion into Date for boundary value 65535 (2149-06-06). [#46042](https://github.com/ClickHouse/ClickHouse/pull/46042) [#45914](https://github.com/ClickHouse/ClickHouse/pull/45914) ([Joanna Hulboj](https://github.com/jh0x)). +* Add setting `check_referential_table_dependencies` to check referential dependencies on `DROP TABLE`. This PR solves [#38326](https://github.com/ClickHouse/ClickHouse/issues/38326). [#45936](https://github.com/ClickHouse/ClickHouse/pull/45936) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix `tupleElement` to return `Null` when having `Null` argument. Closes [#45894](https://github.com/ClickHouse/ClickHouse/issues/45894). [#45952](https://github.com/ClickHouse/ClickHouse/pull/45952) ([flynn](https://github.com/ucasfl)). +* Throw an error on no files satisfying the S3 wildcard. Closes [#45587](https://github.com/ClickHouse/ClickHouse/issues/45587). [#45957](https://github.com/ClickHouse/ClickHouse/pull/45957) ([chen](https://github.com/xiedeyantu)). +* Use cluster state data to check concurrent backup/restore. [#45982](https://github.com/ClickHouse/ClickHouse/pull/45982) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* ClickHouse Client: Use "exact" matching for fuzzy search, which has correct case ignorance and more appropriate algorithm for matching SQL queries. [#46000](https://github.com/ClickHouse/ClickHouse/pull/46000) ([Azat Khuzhin](https://github.com/azat)). +* Forbid wrong create View syntax `CREATE View X TO Y AS SELECT`. Closes [#4331](https://github.com/ClickHouse/ClickHouse/issues/4331). [#46043](https://github.com/ClickHouse/ClickHouse/pull/46043) ([flynn](https://github.com/ucasfl)). +* Storage `Log` family support setting the `storage_policy`. Closes [#43421](https://github.com/ClickHouse/ClickHouse/issues/43421). [#46044](https://github.com/ClickHouse/ClickHouse/pull/46044) ([flynn](https://github.com/ucasfl)). +* Improve `JSONColumns` format when the result is empty. Closes [#46024](https://github.com/ClickHouse/ClickHouse/issues/46024). [#46053](https://github.com/ClickHouse/ClickHouse/pull/46053) ([flynn](https://github.com/ucasfl)). +* Add reference implementation for SipHash128. [#46065](https://github.com/ClickHouse/ClickHouse/pull/46065) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Add a new metric to record allocations times and bytes using mmap. [#46068](https://github.com/ClickHouse/ClickHouse/pull/46068) ([李扬](https://github.com/taiyang-li)). +* Currently for functions like `leftPad`, `rightPad`, `leftPadUTF8`, `rightPadUTF8`, the second argument `length` must be UInt8|16|32|64|128|256. Which is too strict for clickhouse users, besides, it is not consistent with other similar functions like `arrayResize`, `substring` and so on. [#46103](https://github.com/ClickHouse/ClickHouse/pull/46103) ([李扬](https://github.com/taiyang-li)). +* Fix assertion in the `welchTTest` function in debug build when the resulting statistics is NaN. Unified the behavior with other similar functions. Change the behavior of `studentTTest` to return NaN instead of throwing an exception because the previous behavior was inconvenient. This closes [#41176](https://github.com/ClickHouse/ClickHouse/issues/41176) This closes [#42162](https://github.com/ClickHouse/ClickHouse/issues/42162). [#46141](https://github.com/ClickHouse/ClickHouse/pull/46141) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* More convenient usage of big integers and ORDER BY WITH FILL. Allow using plain integers for start and end points in WITH FILL when ORDER BY big (128-bit and 256-bit) integers. Fix the wrong result for big integers with negative start or end points. This closes [#16733](https://github.com/ClickHouse/ClickHouse/issues/16733). [#46152](https://github.com/ClickHouse/ClickHouse/pull/46152) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add `parts`, `active_parts` and `total_marks` columns to `system.tables` on [issue](https://github.com/ClickHouse/ClickHouse/issues/44336). [#46161](https://github.com/ClickHouse/ClickHouse/pull/46161) ([attack204](https://github.com/attack204)). +* Functions "multi[Fuzzy]Match(Any|AnyIndex|AllIndices}" now reject regexes which will likely evaluate very slowly in vectorscan. [#46167](https://github.com/ClickHouse/ClickHouse/pull/46167) ([Robert Schulze](https://github.com/rschu1ze)). +* When `insert_null_as_default` is enabled and column doesn't have defined default value, the default of column type will be used. Also this PR fixes using default values on nulls in case of LowCardinality columns. [#46171](https://github.com/ClickHouse/ClickHouse/pull/46171) ([Kruglov Pavel](https://github.com/Avogar)). +* Prefer explicitly defined access keys for S3 clients. If `use_environment_credentials` is set to `true`, and the user has provided the access key through query or config, they will be used instead of the ones from the environment variable. [#46191](https://github.com/ClickHouse/ClickHouse/pull/46191) ([Antonio Andelic](https://github.com/antonio2368)). +* Add an alias "DATE_FORMAT()" for function "formatDateTime()" to improve compatibility with MySQL's SQL dialect, extend function `formatDateTime` with substitutions "a", "b", "c", "h", "i", "k", "l" "r", "s", "W". ### Documentation entry for user-facing changes User-readable short description: `DATE_FORMAT` is an alias of `formatDateTime`. Formats a Time according to the given Format string. Format is a constant expression, so you cannot have multiple formats for a single result column. (Provide link to [formatDateTime](https://clickhouse.com/docs/en/sql-reference/functions/date-time-functions/#formatdatetime)). [#46302](https://github.com/ClickHouse/ClickHouse/pull/46302) ([Jake Bamrah](https://github.com/JakeBamrah)). +* Add `ProfileEvents` and `CurrentMetrics` about the callback tasks for parallel replicas (`s3Cluster` and `MergeTree` tables). [#46313](https://github.com/ClickHouse/ClickHouse/pull/46313) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add support for `DELETE` and `UPDATE` for tables using `KeeperMap` storage engine. [#46330](https://github.com/ClickHouse/ClickHouse/pull/46330) ([Antonio Andelic](https://github.com/antonio2368)). +* Allow writing RENAME queries with query parameters. Resolves [#45778](https://github.com/ClickHouse/ClickHouse/issues/45778). [#46407](https://github.com/ClickHouse/ClickHouse/pull/46407) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix parameterized SELECT queries with REPLACE transformer. Resolves [#33002](https://github.com/ClickHouse/ClickHouse/issues/33002). [#46420](https://github.com/ClickHouse/ClickHouse/pull/46420) ([Nikolay Degterinsky](https://github.com/evillique)). +* Exclude the internal database used for temporary/external tables from the calculation of asynchronous metric "NumberOfDatabases". This makes the behavior consistent with system table "system.databases". [#46435](https://github.com/ClickHouse/ClickHouse/pull/46435) ([Robert Schulze](https://github.com/rschu1ze)). +* Added `last_exception_time` column into distribution_queue table. [#46564](https://github.com/ClickHouse/ClickHouse/pull/46564) ([Aleksandr](https://github.com/AVMusorin)). +* Support for IN clause with parameter in parameterized views. [#46583](https://github.com/ClickHouse/ClickHouse/pull/46583) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Do not load named collections on server startup (load them on first access instead). [#46607](https://github.com/ClickHouse/ClickHouse/pull/46607) ([Kseniia Sumarokova](https://github.com/kssenii)). + + +#### Build/Testing/Packaging Improvement +* Introduce GWP-ASan implemented by the LLVM runtime. This closes [#27039](https://github.com/ClickHouse/ClickHouse/issues/27039). [#45226](https://github.com/ClickHouse/ClickHouse/pull/45226) ([Han Fei](https://github.com/hanfei1991)). +* We want to make our tests less stable and more flaky: add randomization for merge tree settings in tests. [#38983](https://github.com/ClickHouse/ClickHouse/pull/38983) ([Anton Popov](https://github.com/CurtizJ)). +* Enable the HDFS support in PowerPC and which helps to fixes the following functional tests 02113_hdfs_assert.sh, 02244_hdfs_cluster.sql and 02368_cancel_write_into_hdfs.sh. [#44949](https://github.com/ClickHouse/ClickHouse/pull/44949) ([MeenaRenganathan22](https://github.com/MeenaRenganathan22)). +* Add systemd.service file for clickhouse-keeper. Fixes [#44293](https://github.com/ClickHouse/ClickHouse/issues/44293). [#45568](https://github.com/ClickHouse/ClickHouse/pull/45568) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* ClickHouse's fork of poco was moved from "contrib/" to "base/poco/". [#46075](https://github.com/ClickHouse/ClickHouse/pull/46075) ([Robert Schulze](https://github.com/rschu1ze)). +* Add an option for `clickhouse-watchdog` to restart the child process. This does not make a lot of use. [#46312](https://github.com/ClickHouse/ClickHouse/pull/46312) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* If the environment variable `CLICKHOUSE_DOCKER_RESTART_ON_EXIT` is set to 1, the Docker container will run `clickhouse-server` as a child instead of the first process, and restart it when it exited. [#46391](https://github.com/ClickHouse/ClickHouse/pull/46391) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix Systemd service file. [#46461](https://github.com/ClickHouse/ClickHouse/pull/46461) ([SuperDJY](https://github.com/cmsxbc)). +* Raised the minimum Clang version needed to build ClickHouse from 12 to 15. [#46710](https://github.com/ClickHouse/ClickHouse/pull/46710) ([Robert Schulze](https://github.com/rschu1ze)). +* Upgrade Intel QPL from v0.3.0 to v1.0.0 2. Build libaccel-config and link it statically to QPL library instead of dynamically. [#45809](https://github.com/ClickHouse/ClickHouse/pull/45809) ([jasperzhu](https://github.com/jinjunzh)). + + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Flush data exactly by `rabbitmq_flush_interval_ms` or by `rabbitmq_max_block_size` in `StorageRabbitMQ`. Closes [#42389](https://github.com/ClickHouse/ClickHouse/issues/42389). Closes [#45160](https://github.com/ClickHouse/ClickHouse/issues/45160). [#44404](https://github.com/ClickHouse/ClickHouse/pull/44404) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Use PODArray to render in sparkBar function, so we can control the memory usage. Close [#44467](https://github.com/ClickHouse/ClickHouse/issues/44467). [#44489](https://github.com/ClickHouse/ClickHouse/pull/44489) ([Duc Canh Le](https://github.com/canhld94)). +* Fix functions (quantilesExactExclusive, quantilesExactInclusive) return unsorted array element. [#45379](https://github.com/ClickHouse/ClickHouse/pull/45379) ([wujunfu](https://github.com/wujunfu)). +* Fix uncaught exception in HTTPHandler when open telemetry is enabled. [#45456](https://github.com/ClickHouse/ClickHouse/pull/45456) ([Frank Chen](https://github.com/FrankChen021)). +* Don't infer Dates from 8 digit numbers. It could lead to wrong data to be read. [#45581](https://github.com/ClickHouse/ClickHouse/pull/45581) ([Kruglov Pavel](https://github.com/Avogar)). +* Fixes to correctly use `odbc_bridge_use_connection_pooling` setting. [#45591](https://github.com/ClickHouse/ClickHouse/pull/45591) ([Bharat Nallan](https://github.com/bharatnc)). +* When the callback in the cache is called, it is possible that this cache is destructed. To keep it safe, we capture members by value. It's also safe for task schedule because it will be deactivated before storage is destroyed. Resolve [#45548](https://github.com/ClickHouse/ClickHouse/issues/45548). [#45601](https://github.com/ClickHouse/ClickHouse/pull/45601) ([Han Fei](https://github.com/hanfei1991)). +* Fix data corruption when codecs Delta or DoubleDelta are combined with codec Gorilla. [#45615](https://github.com/ClickHouse/ClickHouse/pull/45615) ([Robert Schulze](https://github.com/rschu1ze)). +* Correctly check types when using N-gram bloom filter index to avoid invalid reads. [#45617](https://github.com/ClickHouse/ClickHouse/pull/45617) ([Antonio Andelic](https://github.com/antonio2368)). +* A couple of segfaults have been reported around `c-ares`. They were introduced in my previous pull requests. I have fixed them with the help of Alexander Tokmakov. [#45629](https://github.com/ClickHouse/ClickHouse/pull/45629) ([Arthur Passos](https://github.com/arthurpassos)). +* Fix key description when encountering duplicate primary keys. This can happen in projections. See [#45590](https://github.com/ClickHouse/ClickHouse/issues/45590) for details. [#45686](https://github.com/ClickHouse/ClickHouse/pull/45686) ([Amos Bird](https://github.com/amosbird)). +* Set compression method and level for backup Closes [#45690](https://github.com/ClickHouse/ClickHouse/issues/45690). [#45737](https://github.com/ClickHouse/ClickHouse/pull/45737) ([Pradeep Chhetri](https://github.com/chhetripradeep)). +* Should use `select_query_typed.limitByOffset()` instead of `select_query_typed.limitOffset()`. [#45817](https://github.com/ClickHouse/ClickHouse/pull/45817) ([刘陶峰](https://github.com/taofengliu)). +* When use experimental analyzer, queries like `SELECT number FROM numbers(100) LIMIT 10 OFFSET 10;` get wrong results (empty result for this sql). That is caused by an unnecessary offset step added by planner. [#45822](https://github.com/ClickHouse/ClickHouse/pull/45822) ([刘陶峰](https://github.com/taofengliu)). +* Backward compatibility - allow implicit narrowing conversion from UInt64 to IPv4 - required for "INSERT ... VALUES ..." expression. [#45865](https://github.com/ClickHouse/ClickHouse/pull/45865) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Bugfix IPv6 parser for mixed ip4 address with missed first octet (like `::.1.2.3`). [#45871](https://github.com/ClickHouse/ClickHouse/pull/45871) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Add the `query_kind` column to the `system.processes` table and the `SHOW PROCESSLIST` query. Remove duplicate code. It fixes a bug: the global configuration parameter `max_concurrent_select_queries` was not respected to queries with `INTERSECT` or `EXCEPT` chains. [#45872](https://github.com/ClickHouse/ClickHouse/pull/45872) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix crash in a function `stochasticLinearRegression`. Found by WingFuzz. [#45985](https://github.com/ClickHouse/ClickHouse/pull/45985) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix crash in `SELECT` queries with `INTERSECT` and `EXCEPT` modifiers that read data from tables with enabled sparse columns (controlled by setting `ratio_of_defaults_for_sparse_serialization). [#45987](https://github.com/ClickHouse/ClickHouse/pull/45987) ([Anton Popov](https://github.com/CurtizJ)). +* Fix read in order optimization for DESC sorting with FINAL, close [#45815](https://github.com/ClickHouse/ClickHouse/issues/45815). [#46009](https://github.com/ClickHouse/ClickHouse/pull/46009) ([Vladimir C](https://github.com/vdimir)). +* Fix reading of non existing nested columns with multiple level in compact parts. [#46045](https://github.com/ClickHouse/ClickHouse/pull/46045) ([Azat Khuzhin](https://github.com/azat)). +* Fix elapsed column in system.processes (10x error). [#46047](https://github.com/ClickHouse/ClickHouse/pull/46047) ([Azat Khuzhin](https://github.com/azat)). +* Follow-up fix for Replace domain IP types (IPv4, IPv6) with native https://github.com/ClickHouse/ClickHouse/pull/43221. [#46087](https://github.com/ClickHouse/ClickHouse/pull/46087) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix environment variable substitution in the configuration when a parameter already has a value. This closes [#46131](https://github.com/ClickHouse/ClickHouse/issues/46131). This closes [#9547](https://github.com/ClickHouse/ClickHouse/issues/9547). [#46144](https://github.com/ClickHouse/ClickHouse/pull/46144) ([pufit](https://github.com/pufit)). +* Fix incorrect predicate push down with grouping sets. Closes [#45947](https://github.com/ClickHouse/ClickHouse/issues/45947). [#46151](https://github.com/ClickHouse/ClickHouse/pull/46151) ([flynn](https://github.com/ucasfl)). +* Fix possible pipeline stuck error on `fulls_sorting_join` with constant keys. [#46175](https://github.com/ClickHouse/ClickHouse/pull/46175) ([Vladimir C](https://github.com/vdimir)). +* Never rewrite tuple functions as literals during formatting to avoid incorrect results. [#46232](https://github.com/ClickHouse/ClickHouse/pull/46232) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Fix possible out of bounds error while reading LowCardinality(Nullable) in Arrow format. [#46270](https://github.com/ClickHouse/ClickHouse/pull/46270) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix `SYSTEM UNFREEZE` queries failing with the exception `CANNOT_PARSE_INPUT_ASSERTION_FAILED`. [#46325](https://github.com/ClickHouse/ClickHouse/pull/46325) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Fix possible crash which can be caused by an integer overflow while deserializing aggregating state of a function that stores HashTable. [#46349](https://github.com/ClickHouse/ClickHouse/pull/46349) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix possible `LOGICAL_ERROR` in asynchronous inserts with invalid data sent in format `VALUES`. [#46350](https://github.com/ClickHouse/ClickHouse/pull/46350) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed a LOGICAL_ERROR on an attempt to execute `ALTER ... MOVE PART ... TO TABLE`. This type of query was never actually supported. [#46359](https://github.com/ClickHouse/ClickHouse/pull/46359) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix s3Cluster schema inference in parallel distributed insert select when `parallel_distributed_insert_select` is enabled. [#46381](https://github.com/ClickHouse/ClickHouse/pull/46381) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix queries like `ALTER TABLE ... UPDATE nested.arr1 = nested.arr2 ...`, where `arr1` and `arr2` are fields of the same `Nested` column. [#46387](https://github.com/ClickHouse/ClickHouse/pull/46387) ([Anton Popov](https://github.com/CurtizJ)). +* Scheduler may fail to schedule a task. If it happens, the whole MulityPartUpload should be aborted and `UploadHelper` must wait for already scheduled tasks. [#46451](https://github.com/ClickHouse/ClickHouse/pull/46451) ([Dmitry Novik](https://github.com/novikd)). +* Fix PREWHERE for Merge with different default types (fixes some `NOT_FOUND_COLUMN_IN_BLOCK` when the default type for the column differs, also allow `PREWHERE` when the type of column is the same across tables, and prohibit it, only if it differs). [#46454](https://github.com/ClickHouse/ClickHouse/pull/46454) ([Azat Khuzhin](https://github.com/azat)). +* Fix a crash that could happen when constant values are used in `ORDER BY`. Fixes [#46466](https://github.com/ClickHouse/ClickHouse/issues/46466). [#46493](https://github.com/ClickHouse/ClickHouse/pull/46493) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Do not throw exception if `disk` setting was specified on query level, but `storage_policy` was specified in config merge tree settings section. `disk` will override setting from config. [#46533](https://github.com/ClickHouse/ClickHouse/pull/46533) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix an invalid processing of constant `LowCardinality` argument in function `arrayMap`. This bug could lead to a segfault in release, and logical error `Bad cast` in debug build. [#46569](https://github.com/ClickHouse/ClickHouse/pull/46569) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* fixes [#46557](https://github.com/ClickHouse/ClickHouse/issues/46557). [#46611](https://github.com/ClickHouse/ClickHouse/pull/46611) ([Alexander Gololobov](https://github.com/davenger)). +* Fix endless restarts of clickhouse-server systemd unit if server cannot start within 1m30sec (Disable timeout logic for starting clickhouse-server from systemd service). [#46613](https://github.com/ClickHouse/ClickHouse/pull/46613) ([Azat Khuzhin](https://github.com/azat)). +* Allocated during asynchronous inserts memory buffers were deallocated in the global context and MemoryTracker counters for corresponding user and query were not updated correctly. That led to false positive OOM exceptions. [#46622](https://github.com/ClickHouse/ClickHouse/pull/46622) ([Dmitry Novik](https://github.com/novikd)). +* Updated to not clear on_expression from table_join as its used by future analyze runs resolves [#45185](https://github.com/ClickHouse/ClickHouse/issues/45185). [#46487](https://github.com/ClickHouse/ClickHouse/pull/46487) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). + + ### ClickHouse release 23.1, 2023-01-26 ### ClickHouse release 23.1 From cb2ec24ea9136f62792e2c0fe3de8f3dbd99f0cf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Feb 2023 08:02:14 +0100 Subject: [PATCH 231/253] Add changelog for 23.2 --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 0588e439b12..cf8645384ba 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -19,7 +19,7 @@ #### New Feature * Add `StorageIceberg` and table function `iceberg` to access iceberg table store on S3. [#45384](https://github.com/ClickHouse/ClickHouse/pull/45384) ([flynn](https://github.com/ucasfl)). * Allow configuring storage as `SETTINGS disk = ''` (instead of `storage_policy`) and with explicit disk creation `SETTINGS disk = disk(type=s3, ...)`. [#41976](https://github.com/ClickHouse/ClickHouse/pull/41976) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Expose `ProfileEvents counters in `system.part_log`. [#38614](https://github.com/ClickHouse/ClickHouse/pull/38614) ([Bharat Nallan](https://github.com/bharatnc)). +* Expose `ProfileEvents` counters in `system.part_log`. [#38614](https://github.com/ClickHouse/ClickHouse/pull/38614) ([Bharat Nallan](https://github.com/bharatnc)). * Enrichment of the existing `ReplacingMergeTree` engine to allow duplicate the insertion. It leverages the power of both `ReplacingMergeTree` and `CollapsingMergeTree` in one MergeTree engine. Deleted data are not returned when queried, but not removed from disk neither. [#41005](https://github.com/ClickHouse/ClickHouse/pull/41005) ([youennL-cs](https://github.com/youennL-cs)). * Add `generateULID` function. Closes [#36536](https://github.com/ClickHouse/ClickHouse/issues/36536). [#44662](https://github.com/ClickHouse/ClickHouse/pull/44662) ([Nikolay Degterinsky](https://github.com/evillique)). * Add `corrMatrix` aggregate function, calculating each two columns. In addition, since Aggregatefunctions `covarSamp` and `covarPop` are similar to `corr`, I add `covarSampMatrix`, `covarPopMatrix` by the way. @alexey-milovidov closes [#44587](https://github.com/ClickHouse/ClickHouse/issues/44587). [#44680](https://github.com/ClickHouse/ClickHouse/pull/44680) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). From cabb8fb81b47b7a79bdf0b9d883c8b8b5cbb0eef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 23 Feb 2023 08:02:37 +0100 Subject: [PATCH 232/253] Add changelog for 23.2 --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index cf8645384ba..7c516faa6a0 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -160,7 +160,7 @@ * Bugfix IPv6 parser for mixed ip4 address with missed first octet (like `::.1.2.3`). [#45871](https://github.com/ClickHouse/ClickHouse/pull/45871) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). * Add the `query_kind` column to the `system.processes` table and the `SHOW PROCESSLIST` query. Remove duplicate code. It fixes a bug: the global configuration parameter `max_concurrent_select_queries` was not respected to queries with `INTERSECT` or `EXCEPT` chains. [#45872](https://github.com/ClickHouse/ClickHouse/pull/45872) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Fix crash in a function `stochasticLinearRegression`. Found by WingFuzz. [#45985](https://github.com/ClickHouse/ClickHouse/pull/45985) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix crash in `SELECT` queries with `INTERSECT` and `EXCEPT` modifiers that read data from tables with enabled sparse columns (controlled by setting `ratio_of_defaults_for_sparse_serialization). [#45987](https://github.com/ClickHouse/ClickHouse/pull/45987) ([Anton Popov](https://github.com/CurtizJ)). +* Fix crash in `SELECT` queries with `INTERSECT` and `EXCEPT` modifiers that read data from tables with enabled sparse columns (controlled by setting `ratio_of_defaults_for_sparse_serialization`). [#45987](https://github.com/ClickHouse/ClickHouse/pull/45987) ([Anton Popov](https://github.com/CurtizJ)). * Fix read in order optimization for DESC sorting with FINAL, close [#45815](https://github.com/ClickHouse/ClickHouse/issues/45815). [#46009](https://github.com/ClickHouse/ClickHouse/pull/46009) ([Vladimir C](https://github.com/vdimir)). * Fix reading of non existing nested columns with multiple level in compact parts. [#46045](https://github.com/ClickHouse/ClickHouse/pull/46045) ([Azat Khuzhin](https://github.com/azat)). * Fix elapsed column in system.processes (10x error). [#46047](https://github.com/ClickHouse/ClickHouse/pull/46047) ([Azat Khuzhin](https://github.com/azat)). From 9054eb49ec11d5a707cac86abad5bdef7fd190c0 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Thu, 23 Feb 2023 10:06:18 +0100 Subject: [PATCH 233/253] removing dated event --- README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/README.md b/README.md index d2809c1b141..1974de3ce94 100644 --- a/README.md +++ b/README.md @@ -21,7 +21,6 @@ curl https://clickhouse.com/ | sh * [Contacts](https://clickhouse.com/company/contact) can help to get your questions answered if there are any. ## Upcoming Events -* [**ClickHouse Workshop**](https://clickhouse.com/company/events/2023-02-15-clickhouse-workshop?utm_source=github&utm_medium=social&utm_campaign=workshop) - Feb 15 & 16 - In this 2-day (3 hrs per day) free training, topics range from introductory content to a deep dive on interacting with and understanding your data. There will be both live training and hands-on labs. * [**v23.2 Release Webinar**](https://clickhouse.com/company/events/v23-2-release-webinar?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-02) - Feb 23 - 23.2 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. * [**ClickHouse Meetup in Amsterdam**](https://www.meetup.com/clickhouse-netherlands-user-group/events/291485868/) - Mar 9 - The first ClickHouse Amsterdam Meetup of 2023 is here! 🎉 Join us for short lightning talks and long discussions. Food, drinks & good times on us. * [**ClickHouse Meetup in SF Bay Area**](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/291490121/) - Mar 14 - A night to meet with ClickHouse team in the San Francisco area! Food and drink are a given...but networking is the primary focus. From f64caafaf90007ae7f187963e7462c585477d536 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Thu, 23 Feb 2023 10:34:23 +0100 Subject: [PATCH 234/253] Update CHANGELOG.md --- CHANGELOG.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7c516faa6a0..4e4785892c3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -10,9 +10,9 @@ #### Backward Incompatible Change * Extend function "toDayOfWeek()" (alias: "DAYOFWEEK") with a mode argument that encodes whether the week starts on Monday or Sunday and whether counting starts at 0 or 1. For consistency with other date time functions, the mode argument was inserted between the time and the time zone arguments. This breaks existing usage of the (previously undocumented) 2-argument syntax "toDayOfWeek(time, time_zone)". A fix is to rewrite the function into "toDayOfWeek(time, 0, time_zone)". [#45233](https://github.com/ClickHouse/ClickHouse/pull/45233) ([Robert Schulze](https://github.com/rschu1ze)). * Rename setting `max_query_cache_size` to `filesystem_cache_max_download_size`. [#45614](https://github.com/ClickHouse/ClickHouse/pull/45614) ([Kseniia Sumarokova](https://github.com/kssenii)). -* The default user will not have permissions for access type `SHOW NAMED COLLECTION` by default (e.g. by default, default user will not longer be able to do grant ALL to other users as it was before, therefore this PR is backward incompatible). [#46010](https://github.com/ClickHouse/ClickHouse/pull/46010) ([Kseniia Sumarokova](https://github.com/kssenii)). +* The `default` user will not have permissions for access type `SHOW NAMED COLLECTION` by default (e.g. `default` user will no longer be able to grant ALL to other users as it was before, therefore this PR is backward incompatible). [#46010](https://github.com/ClickHouse/ClickHouse/pull/46010) ([Kseniia Sumarokova](https://github.com/kssenii)). * If the SETTINGS clause is specified before the FORMAT clause, the settings will be applied to formatting as well. [#46003](https://github.com/ClickHouse/ClickHouse/pull/46003) ([Azat Khuzhin](https://github.com/azat)). -* Remove support for setting `materialized_postgresql_allow_automatic_update` (which was by default turned off). Fix integration tests. [#46106](https://github.com/ClickHouse/ClickHouse/pull/46106) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Remove support for setting `materialized_postgresql_allow_automatic_update` (which was by default turned off). [#46106](https://github.com/ClickHouse/ClickHouse/pull/46106) ([Kseniia Sumarokova](https://github.com/kssenii)). * Slightly improve performance of `countDigits` on realistic datasets. This closed [#44518](https://github.com/ClickHouse/ClickHouse/issues/44518). In previous versions, `countDigits(0)` returned `0`; now it returns `1`, which is more correct, and follows the existing documentation. [#46187](https://github.com/ClickHouse/ClickHouse/pull/46187) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Disallow creation of new columns compressed by a combination of codecs "Delta" or "DoubleDelta" followed by codecs "Gorilla" or "FPC". This can be bypassed using setting "allow_suspicious_codecs = true". [#45652](https://github.com/ClickHouse/ClickHouse/pull/45652) ([Robert Schulze](https://github.com/rschu1ze)). @@ -28,7 +28,7 @@ * Add a new column `last_exception_time` to system.replication_queue. [#45457](https://github.com/ClickHouse/ClickHouse/pull/45457) ([Frank Chen](https://github.com/FrankChen021)). * Add two new functions which allow for user-defined keys/seeds with SipHash{64,128}. [#45513](https://github.com/ClickHouse/ClickHouse/pull/45513) ([Salvatore Mesoraca](https://github.com/aiven-sal)). * Allow a three-argument version for table function `format`. close [#45808](https://github.com/ClickHouse/ClickHouse/issues/45808). [#45873](https://github.com/ClickHouse/ClickHouse/pull/45873) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). -* add `JodaTime` format support for 'x','w','S'. Refer to https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html. [#46073](https://github.com/ClickHouse/ClickHouse/pull/46073) ([zk_kiger](https://github.com/zk-kiger)). +* Add `JodaTime` format support for 'x','w','S'. Refer to https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html. [#46073](https://github.com/ClickHouse/ClickHouse/pull/46073) ([zk_kiger](https://github.com/zk-kiger)). * Support window function `ntile`. * Add setting `final` to implicitly apply the `FINAL` modifier to every table. [#40945](https://github.com/ClickHouse/ClickHouse/pull/40945) ([Arthur Passos](https://github.com/arthurpassos)). * Added `arrayPartialSort` and `arrayPartialReverseSort` functions. [#46296](https://github.com/ClickHouse/ClickHouse/pull/46296) ([Joanna Hulboj](https://github.com/jh0x)). From 052010592c420c82a65e3e0cb7b4679f8755a7fc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 Feb 2023 10:12:13 +0000 Subject: [PATCH 235/253] Poco: Remove some dead code --- base/poco/Foundation/include/Poco/Alignment.h | 12 ------------ base/poco/Foundation/include/Poco/ByteOrder.h | 6 +----- .../poco/Foundation/include/Poco/Foundation.h | 4 +--- base/poco/Foundation/include/Poco/Platform.h | 19 ------------------- .../Foundation/include/Poco/Platform_POSIX.h | 6 ++---- base/poco/Foundation/include/Poco/Types.h | 2 -- base/poco/Net/include/Poco/Net/HTTPRequest.h | 4 ---- base/poco/Net/include/Poco/Net/HTTPResponse.h | 4 ---- 8 files changed, 4 insertions(+), 53 deletions(-) diff --git a/base/poco/Foundation/include/Poco/Alignment.h b/base/poco/Foundation/include/Poco/Alignment.h index b1d48ffd62d..300c55ee8ef 100644 --- a/base/poco/Foundation/include/Poco/Alignment.h +++ b/base/poco/Foundation/include/Poco/Alignment.h @@ -136,7 +136,6 @@ struct AlignedCharArrayImpl; // MSVC requires special handling here. -# ifdef POCO_COMPILER_CLANG # if __has_feature(cxx_alignas) # define POCO_ALIGNEDCHARARRAY_TEMPLATE_ALIGNMENT(x) \ @@ -148,17 +147,6 @@ struct AlignedCharArrayImpl; # define POCO_HAVE_ALIGNMENT # endif -# elif defined(__GNUC__) || defined(__IBM_ATTRIBUTES) - -# define POCO_ALIGNEDCHARARRAY_TEMPLATE_ALIGNMENT(x) \ - template <> \ - struct AlignedCharArrayImpl \ - { \ - char aligned __attribute__((aligned(x))); \ - } -# define POCO_HAVE_ALIGNMENT - -# endif # ifdef POCO_HAVE_ALIGNMENT POCO_ALIGNEDCHARARRAY_TEMPLATE_ALIGNMENT(1); diff --git a/base/poco/Foundation/include/Poco/ByteOrder.h b/base/poco/Foundation/include/Poco/ByteOrder.h index 09f673c2718..a8abf09f93b 100644 --- a/base/poco/Foundation/include/Poco/ByteOrder.h +++ b/base/poco/Foundation/include/Poco/ByteOrder.h @@ -82,11 +82,7 @@ public: #if !defined(POCO_NO_BYTESWAP_BUILTINS) -# if defined(__clang__) -# if __has_builtin(__builtin_bswap32) -# define POCO_HAVE_GCC_BYTESWAP 1 -# endif -# elif defined(__GNUC__) && (__GNUC__ > 4 || (__GNUC__ == 4 && __GNUC_MINOR__ >= 3)) +# if __has_builtin(__builtin_bswap32) # define POCO_HAVE_GCC_BYTESWAP 1 # endif #endif diff --git a/base/poco/Foundation/include/Poco/Foundation.h b/base/poco/Foundation/include/Poco/Foundation.h index 34493041720..bee9d9d90a9 100644 --- a/base/poco/Foundation/include/Poco/Foundation.h +++ b/base/poco/Foundation/include/Poco/Foundation.h @@ -98,10 +98,8 @@ # define POCO_DEPRECATED #elif defined(_GNUC_) # define POCO_DEPRECATED __attribute__((deprecated)) -#elif defined(__clang__) -# define POCO_DEPRECATED __attribute__((deprecated)) #else -# define POCO_DEPRECATED +# define POCO_DEPRECATED __attribute__((deprecated)) #endif diff --git a/base/poco/Foundation/include/Poco/Platform.h b/base/poco/Foundation/include/Poco/Platform.h index eb8f80a0d25..fe45833aea6 100644 --- a/base/poco/Foundation/include/Poco/Platform.h +++ b/base/poco/Foundation/include/Poco/Platform.h @@ -212,25 +212,6 @@ #endif -#if defined(__clang__) -# define POCO_COMPILER_CLANG -#elif defined(__GNUC__) -# define POCO_COMPILER_GCC -#elif defined(__MINGW32__) || defined(__MINGW64__) -# define POCO_COMPILER_MINGW -#elif defined(__INTEL_COMPILER) || defined(__ICC) || defined(__ECC) || defined(__ICL) -# define POCO_COMPILER_INTEL -#elif defined(__MWERKS__) || defined(__CWCC__) -# define POCO_COMPILER_CODEWARRIOR -#elif defined(__sgi) || defined(sgi) -# define POCO_COMPILER_SGI -#elif defined(__BORLANDC__) || defined(__CODEGEARC__) -# define POCO_COMPILER_CBUILDER -#elif defined(__DMC__) -# define POCO_COMPILER_DMARS -#endif - - #ifdef __GNUC__ # define POCO_UNUSED __attribute__((unused)) #else diff --git a/base/poco/Foundation/include/Poco/Platform_POSIX.h b/base/poco/Foundation/include/Poco/Platform_POSIX.h index 96f0c32cb9e..b23c6d68b90 100644 --- a/base/poco/Foundation/include/Poco/Platform_POSIX.h +++ b/base/poco/Foundation/include/Poco/Platform_POSIX.h @@ -32,10 +32,8 @@ // // Thread-safety of local static initialization // -#if __cplusplus >= 201103L || __GNUC__ >= 4 || defined(__clang__) -# ifndef POCO_LOCAL_STATIC_INIT_IS_THREADSAFE -# define POCO_LOCAL_STATIC_INIT_IS_THREADSAFE 1 -# endif +#ifndef POCO_LOCAL_STATIC_INIT_IS_THREADSAFE +# define POCO_LOCAL_STATIC_INIT_IS_THREADSAFE 1 #endif diff --git a/base/poco/Foundation/include/Poco/Types.h b/base/poco/Foundation/include/Poco/Types.h index d10047344f6..4f4924a2542 100644 --- a/base/poco/Foundation/include/Poco/Types.h +++ b/base/poco/Foundation/include/Poco/Types.h @@ -25,7 +25,6 @@ namespace Poco { -#if defined(__GNUC__) || defined(__clang__) // // Unix/GCC/Clang // @@ -46,7 +45,6 @@ typedef unsigned long UInt64; typedef signed long long Int64; typedef unsigned long long UInt64; # endif -#endif } // namespace Poco diff --git a/base/poco/Net/include/Poco/Net/HTTPRequest.h b/base/poco/Net/include/Poco/Net/HTTPRequest.h index 7f17342b22d..269167feb83 100644 --- a/base/poco/Net/include/Poco/Net/HTTPRequest.h +++ b/base/poco/Net/include/Poco/Net/HTTPRequest.h @@ -132,14 +132,10 @@ namespace Net /// Writes the HTTP request to the given /// output stream. -#if __clang__ # pragma clang diagnostic push # pragma clang diagnostic ignored "-Woverloaded-virtual" -#endif void read(std::istream & istr); -#if __clang__ # pragma clang diagnostic pop -#endif /// Reads the HTTP request from the /// given input stream. diff --git a/base/poco/Net/include/Poco/Net/HTTPResponse.h b/base/poco/Net/include/Poco/Net/HTTPResponse.h index b889f0b30fb..3c444c3d38c 100644 --- a/base/poco/Net/include/Poco/Net/HTTPResponse.h +++ b/base/poco/Net/include/Poco/Net/HTTPResponse.h @@ -188,14 +188,10 @@ namespace Net /// Writes the HTTP response to the given /// output stream, but do not finish with \r\n delimiter. -#if __clang__ # pragma clang diagnostic push # pragma clang diagnostic ignored "-Woverloaded-virtual" -#endif void read(std::istream & istr); -#if __clang__ # pragma clang diagnostic pop -#endif /// Reads the HTTP response from the /// given input stream. /// From cbcb1a78527fbfca9afd44c686deca27afae9697 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 23 Feb 2023 13:31:56 +0100 Subject: [PATCH 236/253] More concise logging at trace level --- .../MergeTree/MergeTreeBaseSelectProcessor.cpp | 8 +++++++- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 10 ++++++++++ src/Storages/MergeTree/MergeTreeRangeReader.h | 2 ++ .../MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp | 6 ------ 4 files changed, 19 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 8bbd15c6fe4..0ad91d84d29 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -79,7 +79,13 @@ IMergeTreeSelectAlgorithm::IMergeTreeSelectAlgorithm( result_header = header_without_const_virtual_columns; injectPartConstVirtualColumns(0, result_header, nullptr, partition_value_type, virt_column_names); - LOG_TEST(log, "PREWHERE actions: {}", (prewhere_actions ? prewhere_actions->dump() : std::string(""))); + if (prewhere_actions) + LOG_TRACE(log, "PREWHERE condition was split into {} steps: {}", prewhere_actions->steps.size(), prewhere_actions->dumpConditions()); + + if (prewhere_info) + LOG_TEST(log, "Original PREWHERE DAG:\n{}\nPREWHERE actions:\n{}", + (prewhere_info->prewhere_actions ? prewhere_info->prewhere_actions->dumpDAG(): std::string("")), + (prewhere_actions ? prewhere_actions->dump() : std::string(""))); } bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, PrewhereExprInfo & prewhere); diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 44f6cf9f70d..5ff5d255372 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -1430,4 +1430,14 @@ std::string PrewhereExprInfo::dump() const return s.str(); } +std::string PrewhereExprInfo::dumpConditions() const +{ + WriteBufferFromOwnString s; + + for (size_t i = 0; i < steps.size(); ++i) + s << (i == 0 ? "\"" : ", \"") << steps[i].column_name << "\""; + + return s.str(); +} + } diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index 039a499e9c1..5ffd464cfe2 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -35,6 +35,8 @@ struct PrewhereExprInfo std::vector steps; std::string dump() const; + + std::string dumpConditions() const; }; class FilterWithCachedCount diff --git a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp index b42900d239d..533875d80cd 100644 --- a/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp +++ b/src/Storages/MergeTree/MergeTreeSplitPrewhereIntoReadSteps.cpp @@ -197,10 +197,6 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction if (!prewhere_info || !prewhere_info->prewhere_actions) return true; - Poco::Logger * log = &Poco::Logger::get("tryBuildPrewhereSteps"); - - LOG_TRACE(log, "Original PREWHERE DAG:\n{}", prewhere_info->prewhere_actions->dumpDAG()); - /// 1. List all condition nodes that are combined with AND into PREWHERE condition const auto & condition_root = prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name); const bool is_conjunction = (condition_root.type == ActionsDAG::ActionType::FUNCTION && condition_root.function_base->getName() == "and"); @@ -339,8 +335,6 @@ bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionAction prewhere.steps.back().need_filter = prewhere_info->need_filter; } - LOG_TRACE(log, "Resulting PREWHERE:\n{}", prewhere.dump()); - return true; } From b8b6d597aec37bbc2ec1d052516f8a4cd16c0bd4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 23 Feb 2023 14:05:18 +0100 Subject: [PATCH 237/253] Avoid OOM in perf tests (#46641) * Avoid OOM in perf tests At some point perf tests started to fail for one setup on CI [1]: /home/ubuntu/actions-runner/_work/_temp/f8fce7b1-8bc4-49c8-a203-c96867f4420a.sh: line 5: 1882659 Killed python3 performance_comparison_check.py "$CHECK_NAME" Error: Process completed with exit code 137. [1]: https://github.com/ClickHouse/ClickHouse/actions/runs/4230936986/jobs/7349818625 Signed-off-by: Azat Khuzhin * Switch perf tests to ubuntu 22.04 for parallel with --memsuspend Signed-off-by: Azat Khuzhin --------- Signed-off-by: Azat Khuzhin --- docker/test/performance-comparison/Dockerfile | 7 ++++++- docker/test/performance-comparison/compare.sh | 17 ++++++++++++++--- 2 files changed, 20 insertions(+), 4 deletions(-) diff --git a/docker/test/performance-comparison/Dockerfile b/docker/test/performance-comparison/Dockerfile index 66d535bc94a..ab9f1f8a2e3 100644 --- a/docker/test/performance-comparison/Dockerfile +++ b/docker/test/performance-comparison/Dockerfile @@ -1,5 +1,10 @@ # docker build -t clickhouse/performance-comparison . -FROM ubuntu:20.04 + +# Using ubuntu:22.04 over 20.04 as all other images, since: +# a) ubuntu 20.04 has too old parallel, and does not support --memsuspend +# b) anyway for perf tests it should not be important (backward compatiblity +# with older ubuntu had been checked lots of times in various tests) +FROM ubuntu:22.04 # ARG for quick switch to a given ubuntu mirror ARG apt_archive="http://archive.ubuntu.com" diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 725dcbd7157..293ad9ac411 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -537,9 +537,20 @@ unset IFS # all nodes. numactl --show numactl --cpunodebind=all --membind=all numactl --show -# Use less jobs to avoid OOM. Some queries can consume 8+ GB of memory. -jobs_count=$(($(grep -c ^processor /proc/cpuinfo) / 4)) -numactl --cpunodebind=all --membind=all parallel --jobs $jobs_count --joblog analyze/parallel-log.txt --null < analyze/commands.txt 2>> analyze/errors.log + +# Notes for parallel: +# +# Some queries can consume 8+ GB of memory, so it worth to limit amount of jobs +# that can be run in parallel. +# +# --memfree: +# +# will kill jobs, which is not good (and retried until --retries exceeded) +# +# --memsuspend: +# +# If the available memory falls below 2 * size, GNU parallel will suspend some of the running jobs. +numactl --cpunodebind=all --membind=all parallel -v --joblog analyze/parallel-log.txt --memsuspend 15G --null < analyze/commands.txt 2>> analyze/errors.log clickhouse-local --query " -- Join the metric names back to the metric statistics we've calculated, and make From ce1176a744e76b19d25d3cf8d9cc54db6c982b62 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 Feb 2023 13:10:36 +0000 Subject: [PATCH 238/253] CI: don't run builds/tests when CHANGELOG.md or README.md were modified --- .github/workflows/docs_check.yml | 4 +++- .github/workflows/pull_request.yml | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/.github/workflows/docs_check.yml b/.github/workflows/docs_check.yml index d69020d810e..d55fc45332d 100644 --- a/.github/workflows/docs_check.yml +++ b/.github/workflows/docs_check.yml @@ -13,10 +13,12 @@ on: # yamllint disable-line rule:truthy branches: - master paths: + - 'CHANGELOG.md' + - 'README.md' - 'docker/docs/**' - 'docs/**' - - 'website/**' - 'utils/check-style/aspell-ignore/**' + - 'website/**' jobs: CheckLabels: runs-on: [self-hosted, style-checker] diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index bdf9c0615a8..bb28538834c 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -13,10 +13,12 @@ on: # yamllint disable-line rule:truthy branches: - master paths-ignore: + - 'CHANGELOG.md' + - 'README.md' - 'docker/docs/**' - 'docs/**' - - 'website/**' - 'utils/check-style/aspell-ignore/**' + - 'website/**' ########################################################################################## ##################################### SMALL CHECKS ####################################### ########################################################################################## From bf50b3ab2f76ac4d1151f5c6ad417feb340dbaf7 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 23 Feb 2023 14:30:13 +0100 Subject: [PATCH 239/253] Update test --- tests/queries/0_stateless/02532_send_logs_level_test.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02532_send_logs_level_test.reference b/tests/queries/0_stateless/02532_send_logs_level_test.reference index e3378ac8c3d..dbd49cfc0a4 100644 --- a/tests/queries/0_stateless/02532_send_logs_level_test.reference +++ b/tests/queries/0_stateless/02532_send_logs_level_test.reference @@ -1,3 +1,2 @@ - MergeTreeBaseSelectProcessor: PREWHERE actions: MergeTreeRangeReader: First reader returned: num_rows: 1, columns: 1, total_rows_per_granule: 1, no filter, column[0]: Int32(size = 1), requested columns: key MergeTreeRangeReader: read() returned num_rows: 1, columns: 1, total_rows_per_granule: 1, no filter, column[0]: Int32(size = 1), sample block key From 9914d4c59139c47004e21a6b1247692d779f34f3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 Feb 2023 10:32:24 +0000 Subject: [PATCH 240/253] Cosmetics --- src/Functions/FunctionsHashing.h | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 7525153bd48..75505a173cd 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1029,7 +1029,7 @@ private: { UInt64 v = bit_cast(vec_from[i]); - /// Consider using std::byteswap(c++23) in the future + /// Consider using std::byteswap() once ClickHouse builds with C++23 if constexpr (std::endian::native == std::endian::big) v = __builtin_bswap64(v); h = IntHash64Impl::apply(v); @@ -1072,6 +1072,7 @@ private: if constexpr (std::is_same_v) { UInt64 v = bit_cast(value); + /// Consider using std::byteswap() once ClickHouse builds with C++23 if constexpr (std::endian::native == std::endian::big) v = __builtin_bswap64(v); hash = IntHash64Impl::apply(v); @@ -1086,9 +1087,7 @@ private: size_t size = vec_to.size(); if constexpr (first) - { vec_to.assign(size, hash); - } else { for (size_t i = 0; i < size; ++i) @@ -1113,9 +1112,7 @@ private: { ToType h; if constexpr (std::endian::native == std::endian::little) - { h = apply(key, reinterpret_cast(&vec_from[i]), sizeof(vec_from[i])); - } else { char tmp_buffer[sizeof(vec_from[i])]; @@ -1134,9 +1131,7 @@ private: ToType h; if constexpr (std::endian::native == std::endian::little) - { h = apply(key, reinterpret_cast(&value), sizeof(value)); - } else { char tmp_buffer[sizeof(value)]; @@ -1145,9 +1140,7 @@ private: } size_t size = vec_to.size(); if constexpr (first) - { vec_to.assign(size, h); - } else { for (size_t i = 0; i < size; ++i) From 114986b0a6a7cf71acdf5e1d74539e8c4b6b80c0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 23 Feb 2023 16:40:46 +0300 Subject: [PATCH 241/253] Update DDLWorker.cpp --- src/Interpreters/DDLWorker.cpp | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 0f91212e6a9..7a0309fe055 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -54,6 +54,7 @@ namespace ErrorCodes extern const int CANNOT_ASSIGN_ALTER; extern const int CANNOT_ALLOCATE_MEMORY; extern const int MEMORY_LIMIT_EXCEEDED; + extern const int NOT_IMPLEMENTED; } constexpr const char * TASK_PROCESSED_OUT_REASON = "Task has been already processed"; @@ -456,6 +457,15 @@ bool DDLWorker::tryExecuteQuery(DDLTaskBase & task, const ZooKeeperPtr & zookeep try { auto query_context = task.makeQueryContext(context, zookeeper); + + chassert(!query_context->getCurrentTransaction()); + if (query_context->getSettingsRef().implicit_transaction) + { + if (query_context->getSettingsRef().throw_on_unsupported_query_inside_transaction) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot begin an implicit transaction inside distributed DDL query"); + query_context->setSetting("implicit_transaction", Field{0}); + } + if (!task.is_initial_query) query_scope.emplace(query_context); executeQuery(istr, ostr, !task.is_initial_query, query_context, {}); From de04760bcdfa7fa35d2ee10204bad5b12e1ca29d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 23 Feb 2023 15:57:11 +0100 Subject: [PATCH 242/253] add docs for #44543 --- docs/en/operations/system-tables/text_log.md | 2 ++ docs/ru/operations/system-tables/text_log.md | 2 ++ 2 files changed, 4 insertions(+) diff --git a/docs/en/operations/system-tables/text_log.md b/docs/en/operations/system-tables/text_log.md index 214f8157d48..c0ddacc719c 100644 --- a/docs/en/operations/system-tables/text_log.md +++ b/docs/en/operations/system-tables/text_log.md @@ -28,6 +28,7 @@ Columns: - `revision` (UInt32) — ClickHouse revision. - `source_file` (LowCardinality(String)) — Source file from which the logging was done. - `source_line` (UInt64) — Source line from which the logging was done. +- `message_format_string` (LowCardinality(String)) — A format string that was used to format the message. **Example** @@ -51,4 +52,5 @@ message: Update period 15 seconds revision: 54440 source_file: /ClickHouse/src/Interpreters/DNSCacheUpdater.cpp; void DB::DNSCacheUpdater::start() source_line: 45 +message_format_string: Update period {} seconds ``` diff --git a/docs/ru/operations/system-tables/text_log.md b/docs/ru/operations/system-tables/text_log.md index 69c52471834..59ae804d85f 100644 --- a/docs/ru/operations/system-tables/text_log.md +++ b/docs/ru/operations/system-tables/text_log.md @@ -28,6 +28,7 @@ slug: /ru/operations/system-tables/text_log - `revision` (UInt32) — ревизия ClickHouse. - `source_file` (LowCardinality(String)) — исходный файл, из которого была сделана запись. - `source_line` (UInt64) — исходная строка, из которой была сделана запись. +- `message_format_string` (LowCardinality(String)) — форматная строка, с помощью которой было отформатировано сообщение. **Пример** @@ -51,4 +52,5 @@ message: Update period 15 seconds revision: 54440 source_file: /ClickHouse/src/Interpreters/DNSCacheUpdater.cpp; void DB::DNSCacheUpdater::start() source_line: 45 +message_format_string: Update period {} seconds ``` From 2e3ff4690522640c82c7c378beb51c3fb1571fe4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 Feb 2023 16:33:03 +0100 Subject: [PATCH 243/253] Update .github/workflows/docs_check.yml Co-authored-by: Mikhail f. Shiryaev --- .github/workflows/docs_check.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.github/workflows/docs_check.yml b/.github/workflows/docs_check.yml index d55fc45332d..2354dc7425f 100644 --- a/.github/workflows/docs_check.yml +++ b/.github/workflows/docs_check.yml @@ -18,7 +18,6 @@ on: # yamllint disable-line rule:truthy - 'docker/docs/**' - 'docs/**' - 'utils/check-style/aspell-ignore/**' - - 'website/**' jobs: CheckLabels: runs-on: [self-hosted, style-checker] From 99e184c754f5e754ed2c439d35de08078d9fadb4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 Feb 2023 16:33:10 +0100 Subject: [PATCH 244/253] Update .github/workflows/pull_request.yml Co-authored-by: Mikhail f. Shiryaev --- .github/workflows/pull_request.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index bb28538834c..fb838991019 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -18,7 +18,6 @@ on: # yamllint disable-line rule:truthy - 'docker/docs/**' - 'docs/**' - 'utils/check-style/aspell-ignore/**' - - 'website/**' ########################################################################################## ##################################### SMALL CHECKS ####################################### ########################################################################################## From f9bf0b5a8eaf5ef34d3fb950eb5b2b24aae40a16 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 Feb 2023 16:33:22 +0100 Subject: [PATCH 245/253] Update .github/workflows/pull_request.yml Co-authored-by: Mikhail f. Shiryaev --- .github/workflows/pull_request.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index fb838991019..e73d97c8123 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -15,6 +15,7 @@ on: # yamllint disable-line rule:truthy paths-ignore: - 'CHANGELOG.md' - 'README.md' + - 'SECURITY.md' - 'docker/docs/**' - 'docs/**' - 'utils/check-style/aspell-ignore/**' From ab1571b70f1c64cc1448199e01fd81a260da7209 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 Feb 2023 16:33:36 +0100 Subject: [PATCH 246/253] Update .github/workflows/docs_check.yml Co-authored-by: Mikhail f. Shiryaev --- .github/workflows/docs_check.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/docs_check.yml b/.github/workflows/docs_check.yml index 2354dc7425f..a0d0e49b95b 100644 --- a/.github/workflows/docs_check.yml +++ b/.github/workflows/docs_check.yml @@ -15,6 +15,7 @@ on: # yamllint disable-line rule:truthy paths: - 'CHANGELOG.md' - 'README.md' + - 'SECURITY.md' - 'docker/docs/**' - 'docs/**' - 'utils/check-style/aspell-ignore/**' From d7cd4c6fcad7290ff98c3cee845971acf670bf57 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 23 Feb 2023 20:20:29 +0300 Subject: [PATCH 247/253] Update clickhouse-test --- tests/clickhouse-test | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index e47ba8e10ba..f5d1099c3f4 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2175,6 +2175,7 @@ def main(args): print(json.dumps(processlist, indent=4)) print(get_transactions_list(args)) + print_stacktraces() exit_code.value = 1 else: print(colored("\nNo queries hung.", args, "green", attrs=["bold"])) From 04d3d01bbebf707937b46dca1499def90e5da44c Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 23 Feb 2023 19:23:02 +0100 Subject: [PATCH 248/253] Update prefetch_settings.xml --- tests/config/users.d/prefetch_settings.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/users.d/prefetch_settings.xml b/tests/config/users.d/prefetch_settings.xml index 33ac0a4eb01..d2b6d8a9386 100644 --- a/tests/config/users.d/prefetch_settings.xml +++ b/tests/config/users.d/prefetch_settings.xml @@ -1,7 +1,7 @@ - 1 + 0 0 1Gi From 1efba8995d47dfbb09e39ff7c90c4f85a3b56353 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 23 Feb 2023 19:35:57 +0000 Subject: [PATCH 249/253] Update version to 23.3.1.2537 --- cmake/autogenerated_versions.txt | 10 +++---- .../StorageSystemContributors.generated.cpp | 30 +++++++++++++++++++ 2 files changed, 35 insertions(+), 5 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index 812a0d9e64b..b52b2eda992 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -2,11 +2,11 @@ # NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION, # only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. -SET(VERSION_REVISION 54471) +SET(VERSION_REVISION 54472) SET(VERSION_MAJOR 23) -SET(VERSION_MINOR 2) +SET(VERSION_MINOR 3) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH dcaac47702510cc87ddf266bc524f6b7ce0a8e6e) -SET(VERSION_DESCRIBE v23.2.1.1-testing) -SET(VERSION_STRING 23.2.1.1) +SET(VERSION_GITHASH 52bf836e03a6ba7cf2d654eaaf73231701abc3a2) +SET(VERSION_DESCRIBE v23.3.1.2537-testing) +SET(VERSION_STRING 23.3.1.2537) # end of autochange diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index 6ca6a9db046..ca19687918c 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -9,9 +9,11 @@ const char * auto_contributors[] { "3ldar-nasyrov", "546", "7", + "7vikpeculiar", "821008736@qq.com", "94rain", "ANDREI STAROVEROV", + "AVMusorin", "Aaron Katz", "Adam Rutkowski", "Adri Fernandez", @@ -83,6 +85,7 @@ const char * auto_contributors[] { "Alexey Ilyukhov", "Alexey Ivanov", "Alexey Milovidov", + "Alexey Perevyshin", "Alexey Tronov", "Alexey Vasiliev", "Alexey Zatelepin", @@ -261,6 +264,7 @@ const char * auto_contributors[] { "Denys Golotiuk", "Derek Chia", "Derek Perkins", + "Diego Nieto", "Diego Nieto (lesandie)", "DimaAmega", "Ding Xiang Fei", @@ -294,6 +298,7 @@ const char * auto_contributors[] { "DuckSoft", "Egor O'Sten", "Egor Savin", + "Eirik", "Ekaterina", "Eldar Zaitov", "Elena", @@ -322,6 +327,7 @@ const char * auto_contributors[] { "Evgeny Markov", "Ewout", "FArthur-cmd", + "FFFFFFFHHHHHHH", "Fabian Stäber", "Fabiano Francesconi", "Fadi Hadzh", @@ -432,6 +438,7 @@ const char * auto_contributors[] { "JackyWoo", "Jacob Hayes", "Jacob Herrington", + "Jake Bamrah", "Jake Liu", "Jakub Kuklis", "James Maidment", @@ -462,6 +469,7 @@ const char * auto_contributors[] { "Julian Gilyadov", "Julian Zhou", "Julio Jimenez", + "Junfu Wu", "Jus", "Justin Hilliard", "Kang Liu", @@ -472,12 +480,14 @@ const char * auto_contributors[] { "Kerry Clendinning", "Kevin Chiang", "Kevin Michel", + "Kevin Zhang", "KinderRiven", "Kiran", "Kirill Danshin", "Kirill Ershov", "Kirill Malev", "Kirill Shvakov", + "KitKatKKK", "Koblikov Mihail", "KochetovNicolai", "Konstantin Bogdanov", @@ -495,6 +505,7 @@ const char * auto_contributors[] { "Kruglov Pavel", "Krzysztof Góralski", "Kseniia Sumarokova", + "Kunal Gurnani", "Kuz Le", "Ky Li", "LAL2211", @@ -748,6 +759,7 @@ const char * auto_contributors[] { "Roman", "Roman Bug", "Roman Chyrva", + "Roman Heinrich", "Roman Lipovsky", "Roman Nikolaev", "Roman Nikonov", @@ -767,6 +779,7 @@ const char * auto_contributors[] { "Saad Ur Rahman", "Sabyanin Maxim", "Sachin", + "SadiHassan", "Safronov Michail", "SaltTan", "Salvatore Mesoraca", @@ -832,6 +845,7 @@ const char * auto_contributors[] { "SuperBot", "SuperDJY", "Suzy Wang", + "SuzyWangIBMer", "Sébastien", "Sébastien Launay", "TABLUM.IO", @@ -986,6 +1000,7 @@ const char * auto_contributors[] { "aaapetrenko", "abdrakhmanov", "abel-wang", + "abidino", "abyss7", "achimbab", "achulkov2", @@ -1023,6 +1038,7 @@ const char * auto_contributors[] { "artpaul", "asiana21", "atereh", + "attack204", "avasiliev", "avogar", "avsharapov", @@ -1120,6 +1136,7 @@ const char * auto_contributors[] { "feng lv", "fenglv", "fessmage", + "fhbai", "fibersel", "filimonov", "filipe", @@ -1176,6 +1193,7 @@ const char * auto_contributors[] { "imgbot[bot]", "ip", "it1804", + "ivan-klass", "ivan-kush", "ivanzhukov", "ivoleg", @@ -1202,6 +1220,7 @@ const char * auto_contributors[] { "kevin wan", "kgurjev", "khamadiev", + "kigerzhang", "kirillikoff", "kmeaw", "koloshmet", @@ -1243,6 +1262,7 @@ const char * auto_contributors[] { "litao91", "liu-bov", "liumaojing", + "liuneng", "liuneng1994", "liuyangkuan", "liuyimin", @@ -1271,6 +1291,7 @@ const char * auto_contributors[] { "martincholuj", "mastertheknife", "mateng0915", + "mateng915", "maxim", "maxim-babenko", "maxkuzn", @@ -1358,10 +1379,14 @@ const char * auto_contributors[] { "ritaank", "rnbondarenko", "robert", + "robot-ch-test-poll", "robot-ch-test-poll1", + "robot-ch-test-poll2", + "robot-ch-test-poll3", "robot-ch-test-poll4", "robot-clickhouse", "robot-clickhouse-ci-1", + "robot-clickhouse-ci-2", "robot-metrika-test", "rodrigargar", "roman", @@ -1371,6 +1396,7 @@ const char * auto_contributors[] { "ruct", "ryzuo", "s-kat", + "sanjam", "santaux", "santrancisco", "satanson", @@ -1474,6 +1500,7 @@ const char * auto_contributors[] { "ylchou", "yonesko", "youenn lebras", + "youennL-cs", "young scott", "yuanyimeng", "yuchuansun", @@ -1486,6 +1513,7 @@ const char * auto_contributors[] { "zhang2014", "zhanghuajie", "zhanglistar", + "zhangnew", "zhangshengyu", "zhangxiao018", "zhangxiao871", @@ -1499,6 +1527,7 @@ const char * auto_contributors[] { "zhoubintao", "zhukai", "zimv", + "zk_kiger", "zkun", "zlx19950903", "zombee0", @@ -1523,6 +1552,7 @@ const char * auto_contributors[] { "万康", "何李夫", "凌涛", + "刘陶峰", "吴健", "小蝌蚪", "小路", From f2e71bc3b7d9683cf0dbae7c4f9ce3efe667d577 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 23 Feb 2023 19:46:46 +0000 Subject: [PATCH 250/253] Update version_date.tsv and changelogs after v23.2.1.2537-stable --- SECURITY.md | 3 +- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.2.1.2537-stable.md | 473 +++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 6 files changed, 479 insertions(+), 4 deletions(-) create mode 100644 docs/changelogs/v23.2.1.2537-stable.md diff --git a/SECURITY.md b/SECURITY.md index 0fd72971d30..7c6648c70eb 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -13,9 +13,10 @@ The following versions of ClickHouse server are currently being supported with s | Version | Supported | |:-|:-| +| 23.2 | ✔️ | | 23.1 | ✔️ | | 22.12 | ✔️ | -| 22.11 | ✔️ | +| 22.11 | ❌ | | 22.10 | ❌ | | 22.9 | ❌ | | 22.8 | ✔️ | diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index eda8274edac..09395befdad 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -29,7 +29,7 @@ RUN arch=${TARGETARCH:-amd64} \ esac ARG REPOSITORY="https://s3.amazonaws.com/clickhouse-builds/22.4/31c367d3cd3aefd316778601ff6565119fe36682/package_release" -ARG VERSION="23.1.3.5" +ARG VERSION="23.2.1.2537" 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 8a73d72b3a5..472f25eed2d 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -33,7 +33,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.1.3.5" +ARG VERSION="23.2.1.2537" 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 ba2d7430e06..961c528f19c 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -21,7 +21,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="23.1.3.5" +ARG VERSION="23.2.1.2537" 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.2.1.2537-stable.md b/docs/changelogs/v23.2.1.2537-stable.md new file mode 100644 index 00000000000..3fdcf6d6571 --- /dev/null +++ b/docs/changelogs/v23.2.1.2537-stable.md @@ -0,0 +1,473 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.2.1.2537-stable (52bf836e03a) FIXME as compared to v23.1.1.3077-stable (dcaac477025) + +#### Backward Incompatible Change +* Extend function "toDayOfWeek()" (alias: "DAYOFWEEK") with a mode argument that encodes whether the week starts on Monday or Sunday and whether counting starts at 0 or 1. For consistency with other date time functions, the mode argument was inserted between the time and the time zone arguments. This breaks existing usage of the (previously undocumented) 2-argument syntax "toDayOfWeek(time, time_zone)". A fix is to rewrite the function into "toDayOfWeek(time, 0, time_zone)". [#45233](https://github.com/ClickHouse/ClickHouse/pull/45233) ([Robert Schulze](https://github.com/rschu1ze)). +* Rename setting `max_query_cache_size` to `filesystem_cache_max_download_size`. [#45614](https://github.com/ClickHouse/ClickHouse/pull/45614) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix applying settings for FORMAT on the client. [#46003](https://github.com/ClickHouse/ClickHouse/pull/46003) ([Azat Khuzhin](https://github.com/azat)). +* Default user will not have permissions for access type `SHOW NAMED COLLECTION` by default (e.g. by default, default user will not longer be able to do grant ALL to other users as it was before, therefore this PR is backward incompatible). [#46010](https://github.com/ClickHouse/ClickHouse/pull/46010) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Remove support for setting `materialized_postgresql_allow_automatic_update` (which was by default turned off). Fix integration tests. [#46106](https://github.com/ClickHouse/ClickHouse/pull/46106) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Slightly improve performance of `countDigits` on realistic datasets. This closed [#44518](https://github.com/ClickHouse/ClickHouse/issues/44518). In previous versions, `countDigits(0)` returned `0`; now it returns `1`, which is more correct, and follows the existing documentation. [#46187](https://github.com/ClickHouse/ClickHouse/pull/46187) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### New Feature +* Expose ProfileEvents counters in system.part_log. [#38614](https://github.com/ClickHouse/ClickHouse/pull/38614) ([Bharat Nallan](https://github.com/bharatnc)). +* Enrichment of the existing ReplacingMergeTree engine to allow duplicates insertion. It leverages the power of both ReplacingMergeTree and CollapsingMergeTree in one mergeTree engine. Deleted data are not returned when queried, but not removed from disk neither. [#41005](https://github.com/ClickHouse/ClickHouse/pull/41005) ([youennL-cs](https://github.com/youennL-cs)). +* Add `generateULID()` function. Closes [#36536](https://github.com/ClickHouse/ClickHouse/issues/36536). [#44662](https://github.com/ClickHouse/ClickHouse/pull/44662) ([Nikolay Degterinsky](https://github.com/evillique)). +* Add `corrMatrix` Aggregatefunction, calculating each two columns. In addition, since Aggregatefunctions `covarSamp` and `covarPop` are similar to `corr`, I add `covarSampMatrix`, `covarPopMatrix` by the way. @alexey-milovidov closes [#44587](https://github.com/ClickHouse/ClickHouse/issues/44587). [#44680](https://github.com/ClickHouse/ClickHouse/pull/44680) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* Rewrite aggregate functions with if expression as argument when logically equivalent. For example, avg(if(cond, col, null)) can be rewritten to avgIf(cond, col). It is helpful in performance. [#44730](https://github.com/ClickHouse/ClickHouse/pull/44730) ([李扬](https://github.com/taiyang-li)). +* Introduce arrayShuffle function for random array permutations. [#45271](https://github.com/ClickHouse/ClickHouse/pull/45271) ([Joanna Hulboj](https://github.com/jh0x)). +* Support types FIXED_SIZE_BINARY type in Arrow, FIXED_LENGTH_BYTE_ARRAY in Parquet and match them to FixedString. Add settings `output_format_parquet_fixed_string_as_fixed_byte_array/output_format_arrow_fixed_string_as_fixed_byte_array` to control default output type for FixedString. Closes [#45326](https://github.com/ClickHouse/ClickHouse/issues/45326). [#45340](https://github.com/ClickHouse/ClickHouse/pull/45340) ([Kruglov Pavel](https://github.com/Avogar)). +* Add `StorageIceberg` and table function `iceberg` to access iceberg table store on S3. [#45384](https://github.com/ClickHouse/ClickHouse/pull/45384) ([flynn](https://github.com/ucasfl)). +* Add a new column `last_exception_time` to system.replication_queue. [#45457](https://github.com/ClickHouse/ClickHouse/pull/45457) ([Frank Chen](https://github.com/FrankChen021)). +* Add two new functions which allow for user-defined keys/seeds with SipHash{64,128}. [#45513](https://github.com/ClickHouse/ClickHouse/pull/45513) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Allow a three-argument version for table function `format`. close [#45808](https://github.com/ClickHouse/ClickHouse/issues/45808). [#45873](https://github.com/ClickHouse/ClickHouse/pull/45873) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* add joda format support for 'x','w','S'.Refer to https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html. [#46073](https://github.com/ClickHouse/ClickHouse/pull/46073) ([zk_kiger](https://github.com/zk-kiger)). +* ... Support window function `ntile`. ``` insert into test_data values(1,2), (1,3), (1,4), (2,5),(2,6); select a, b, ntile(2) over (partition by a order by b rows between unbounded preceding and unbounded following ) from test_data;. [#46256](https://github.com/ClickHouse/ClickHouse/pull/46256) ([lgbo](https://github.com/lgbo-ustc)). +* Added arrayPartialSort and arrayPartialReverseSort functions. [#46296](https://github.com/ClickHouse/ClickHouse/pull/46296) ([Joanna Hulboj](https://github.com/jh0x)). +* The new http parameter `client_protocol_version` allows setting a client protocol version for HTTP responses using the Native format. [#40397](https://github.com/ClickHouse/ClickHouse/issues/40397). [#46360](https://github.com/ClickHouse/ClickHouse/pull/46360) ([Geoff Genz](https://github.com/genzgd)). +* Add new function regexpExtract, like spark function REGEXP_EXTRACT. [#46469](https://github.com/ClickHouse/ClickHouse/pull/46469) ([李扬](https://github.com/taiyang-li)). +* Author: [taiyang-li](https://github.com/taiyang-li) Add new function regexpExtract, like spark function REGEXP_EXTRACT. [#46529](https://github.com/ClickHouse/ClickHouse/pull/46529) ([Alexander Gololobov](https://github.com/davenger)). +* Add new function JSONArrayLength, which returns the number of elements in the outermost JSON array. The function returns NULL if input JSON string is invalid. [#46631](https://github.com/ClickHouse/ClickHouse/pull/46631) ([李扬](https://github.com/taiyang-li)). + +#### Performance Improvement +* Improve lower/upper function performance with avx512 instructions. [#37894](https://github.com/ClickHouse/ClickHouse/pull/37894) ([yaqi-zhao](https://github.com/yaqi-zhao)). +* Add new `local_filesystem_read_method` method `io_uring` based on the asynchronous Linux [io_uring](https://kernel.dk/io_uring.pdf) subsystem, improving read performance almost universally compared to the default `pread` method. [#38456](https://github.com/ClickHouse/ClickHouse/pull/38456) ([Saulius Valatka](https://github.com/sauliusvl)). +* Remove the limitation that on systems with >=32 cores and SMT disabled ClickHouse uses only half of the cores. [#44973](https://github.com/ClickHouse/ClickHouse/pull/44973) ([Robert Schulze](https://github.com/rschu1ze)). +* Improve performance of function multiIf by columnar executing, speed up by 2.3x. [#45296](https://github.com/ClickHouse/ClickHouse/pull/45296) ([李扬](https://github.com/taiyang-li)). +* An option added to aggregate partitions independently if table partition key and group by key are compatible. Controlled by the setting `allow_aggregate_partitions_independently`. Disabled by default because of limited applicability (please refer to the docs). [#45364](https://github.com/ClickHouse/ClickHouse/pull/45364) ([Nikita Taranov](https://github.com/nickitat)). +* Add fastpath for function position when needle is empty. [#45382](https://github.com/ClickHouse/ClickHouse/pull/45382) ([李扬](https://github.com/taiyang-li)). +* Enable `query_plan_remove_redundant_sorting` optimization by default. Optimization implemented in [#45420](https://github.com/ClickHouse/ClickHouse/issues/45420). [#45567](https://github.com/ClickHouse/ClickHouse/pull/45567) ([Igor Nikonov](https://github.com/devcrafter)). +* Increased HTTP Transfer Encoding chunk size to improve performance of large queries using the HTTP interface. [#45593](https://github.com/ClickHouse/ClickHouse/pull/45593) ([Geoff Genz](https://github.com/genzgd)). +* Fixed performance of short `SELECT` queries that read from tables with large number of`Array`/`Map`/`Nested` columns. [#45630](https://github.com/ClickHouse/ClickHouse/pull/45630) ([Anton Popov](https://github.com/CurtizJ)). +* Allow using Vertical merge algorithm with parts in Compact format. This will allow ClickHouse server to use much less memory for background operations. This closes [#46084](https://github.com/ClickHouse/ClickHouse/issues/46084). [#45681](https://github.com/ClickHouse/ClickHouse/pull/45681) ([Anton Popov](https://github.com/CurtizJ)). +* Optimize Parquet reader by using batch reader. [#45878](https://github.com/ClickHouse/ClickHouse/pull/45878) ([LiuNeng](https://github.com/liuneng1994)). +* Improve performance of ColumnArray::filter for big int and decimal. [#45949](https://github.com/ClickHouse/ClickHouse/pull/45949) ([李扬](https://github.com/taiyang-li)). +* This change could effectively reduce the overhead of obtaining the filter from ColumnNullable(UInt8) and improve the overall query performance. To evaluate the impact of this change, we adopted TPC-H benchmark but revised the column types from non-nullable to nullable, and we measured the QPS of its queries as the performance indicator. [#45962](https://github.com/ClickHouse/ClickHouse/pull/45962) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). +* Make the `_part` and `_partition_id` virtual column be `LowCardinality(String)` type. Closes [#45964](https://github.com/ClickHouse/ClickHouse/issues/45964). [#45975](https://github.com/ClickHouse/ClickHouse/pull/45975) ([flynn](https://github.com/ucasfl)). +* Improve the performance of Decimal conversion when the scale does not change. [#46095](https://github.com/ClickHouse/ClickHouse/pull/46095) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The introduced logic works if PREWHERE condition is a conjunction of multiple conditions (cond1 AND cond2 AND ... ). It groups those conditions that require reading the same columns into steps. After each step the corresponding part of the full condition is computed and the result rows might be filtered. This allows to read fewer rows in the next steps thus saving IO bandwidth and doing less computation. This logic is disabled by default for now. It will be enabled by default in one of the future releases once it is known to not have any regressions, so it is highly encouraged to be used for testing. It can be controlled by 2 settings: "enable_multiple_prewhere_read_steps" and "move_all_conditions_to_prewhere". [#46140](https://github.com/ClickHouse/ClickHouse/pull/46140) ([Alexander Gololobov](https://github.com/davenger)). +* Allow to increase prefetching for read data. [#46168](https://github.com/ClickHouse/ClickHouse/pull/46168) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Rewrite arrayExists(x -> x = 1, arr) -> has(arr, 1), which improve performance by 1.34x. [#46188](https://github.com/ClickHouse/ClickHouse/pull/46188) ([李扬](https://github.com/taiyang-li)). +* Fix too big memory usage for vertical merges on non-remote disk. Respect `max_insert_delayed_streams_for_parallel_write` for the remote disk. [#46275](https://github.com/ClickHouse/ClickHouse/pull/46275) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Update zstd to v1.5.4. It has some minor improvements in performance and compression ratio. If you run replicas with different versions of ClickHouse you may see reasonable error messages `Data after merge/mutation is not byte-identical to data on another replicas.` with explanation. These messages are Ok and you should not worry. [#46280](https://github.com/ClickHouse/ClickHouse/pull/46280) ([Raúl Marín](https://github.com/Algunenano)). +* Allow using Vertical merge algorithm with parts in Compact format. This will allow ClickHouse server to use much less memory for background operations. This closes [#46084](https://github.com/ClickHouse/ClickHouse/issues/46084). [#46282](https://github.com/ClickHouse/ClickHouse/pull/46282) ([Anton Popov](https://github.com/CurtizJ)). +* Fix performance degradation caused by [#39737](https://github.com/ClickHouse/ClickHouse/issues/39737). [#46309](https://github.com/ClickHouse/ClickHouse/pull/46309) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The `replicas_status` handle will answer quickly even in case of a large replication queue. [#46310](https://github.com/ClickHouse/ClickHouse/pull/46310) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Improvement +* Add avx512 support for Aggregate Sum, function unary arithmetic, function comparison. [#37870](https://github.com/ClickHouse/ClickHouse/pull/37870) ([zhao zhou](https://github.com/zzachimed)). +* close issue: [#38893](https://github.com/ClickHouse/ClickHouse/issues/38893). [#38950](https://github.com/ClickHouse/ClickHouse/pull/38950) ([hexiaoting](https://github.com/hexiaoting)). +* Migration from other databases and updates/deletes are mimicked by Collapsing/Replacing. Want to use the same SELECT queries without adding FINAL to all the existing queries. [#40945](https://github.com/ClickHouse/ClickHouse/pull/40945) ([Arthur Passos](https://github.com/arthurpassos)). +* Allow configuring storage as `SETTINGS disk=''` (instead of `storage_policy`) and with explicit disk creation `SETTINGS disk=disk(type=s3, ...)`. [#41976](https://github.com/ClickHouse/ClickHouse/pull/41976) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add new metrics for backups: num_processed_files and processed_files_size described actual number of processed files. [#42244](https://github.com/ClickHouse/ClickHouse/pull/42244) ([Aleksandr](https://github.com/AVMusorin)). +* Added retries on interserver DNS errors. [#43179](https://github.com/ClickHouse/ClickHouse/pull/43179) ([Anton Kozlov](https://github.com/tonickkozlov)). +* Rewrote the code around marks distribution and the overall coordination of the reading in order to achieve the maximum performance improvement. This closes [#34527](https://github.com/ClickHouse/ClickHouse/issues/34527). [#43772](https://github.com/ClickHouse/ClickHouse/pull/43772) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Remove redundant DISTINCT clauses in query (subqueries). Implemented on top of query plan. It does similar optimization as `optimize_duplicate_order_by_and_distinct` regarding DISTINCT clauses. Can be enabled via `query_plan_remove_redundant_distinct` setting. Related to [#42648](https://github.com/ClickHouse/ClickHouse/issues/42648). [#44176](https://github.com/ClickHouse/ClickHouse/pull/44176) ([Igor Nikonov](https://github.com/devcrafter)). +* Keeper improvement: try preallocating space on the disk to avoid undefined out-of-space issues. Introduce setting `max_log_file_size` for the maximum size of Keeper's Raft log files. [#44370](https://github.com/ClickHouse/ClickHouse/pull/44370) ([Antonio Andelic](https://github.com/antonio2368)). +* ``` sumIf(123, cond) -> 123 * countIf(1, cond) sum(if(cond, 123, 0)) -> 123 * countIf(cond) sum(if(cond, 0, 123)) -> 123 * countIf(not(cond)) ```. [#44728](https://github.com/ClickHouse/ClickHouse/pull/44728) ([李扬](https://github.com/taiyang-li)). +* Optimize behavior for a replica delay api logic in case the replica is read-only. [#45148](https://github.com/ClickHouse/ClickHouse/pull/45148) ([mateng915](https://github.com/mateng0915)). +* Introduce gwp-asan implemented by llvm runtime. This closes [#27039](https://github.com/ClickHouse/ClickHouse/issues/27039). [#45226](https://github.com/ClickHouse/ClickHouse/pull/45226) ([Han Fei](https://github.com/hanfei1991)). +* ... in the case key casted from uint64 to uint32, small impact for little endian platform but key value becomes zero in big endian case. ### Documentation entry for user-facing changes. [#45375](https://github.com/ClickHouse/ClickHouse/pull/45375) ([Suzy Wang](https://github.com/SuzyWangIBMer)). +* Mark Gorilla compression on columns of non-Float* type as suspicious. [#45376](https://github.com/ClickHouse/ClickHouse/pull/45376) ([Robert Schulze](https://github.com/rschu1ze)). +* Allow removing redundant aggregation keys with constants (e.g., simplify `GROUP BY a, a + 1` to `GROUP BY a`). [#45415](https://github.com/ClickHouse/ClickHouse/pull/45415) ([Dmitry Novik](https://github.com/novikd)). +* Show replica name that is executing a merge in the postpone_reason. [#45458](https://github.com/ClickHouse/ClickHouse/pull/45458) ([Frank Chen](https://github.com/FrankChen021)). +* Save exception stack trace in part_log. [#45459](https://github.com/ClickHouse/ClickHouse/pull/45459) ([Frank Chen](https://github.com/FrankChen021)). +* Make RegExpTreeDictionary a ua parser which is compatible with https://github.com/ua-parser/uap-core. [#45631](https://github.com/ClickHouse/ClickHouse/pull/45631) ([Han Fei](https://github.com/hanfei1991)). +* Enable ICU data support on s390x platform. [#45632](https://github.com/ClickHouse/ClickHouse/pull/45632) ([Suzy Wang](https://github.com/SuzyWangIBMer)). +* Updated checking of SYSTEM SYNC REPLICA resolves [#45508](https://github.com/ClickHouse/ClickHouse/issues/45508) Implementation: * Updated to wait for current last entry to be processed (after pulling shared log) instead of queue size becoming 0. * Updated Subscriber to notify both queue size and removed log_entry_id. [#45648](https://github.com/ClickHouse/ClickHouse/pull/45648) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Disallow creation of new columns compressed by a combination of codecs "Delta" or "DoubleDelta" followed by codecs "Gorilla" or "FPC". This can be bypassed using setting "allow_suspicious_codecs = true". [#45652](https://github.com/ClickHouse/ClickHouse/pull/45652) ([Robert Schulze](https://github.com/rschu1ze)). +* Rename setting `replication_alter_partitions_sync` to `alter_sync`. [#45659](https://github.com/ClickHouse/ClickHouse/pull/45659) ([Antonio Andelic](https://github.com/antonio2368)). +* The `generateRandom` table function and the engine now support `LowCardinality` data types. This is useful for testing, for example you can write `INSERT INTO table SELECT * FROM generateRandom() LIMIT 1000`. This is needed to debug [#45590](https://github.com/ClickHouse/ClickHouse/issues/45590). [#45661](https://github.com/ClickHouse/ClickHouse/pull/45661) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add ability to ignore unknown keys in JSON object for named tuples (`input_format_json_ignore_unknown_keys_in_named_tuple`). [#45678](https://github.com/ClickHouse/ClickHouse/pull/45678) ([Azat Khuzhin](https://github.com/azat)). +* - The experimental query result cache now provides more modular configuration settings. [#45679](https://github.com/ClickHouse/ClickHouse/pull/45679) ([Robert Schulze](https://github.com/rschu1ze)). +* Renamed "query result cache" to "query cache". [#45682](https://github.com/ClickHouse/ClickHouse/pull/45682) ([Robert Schulze](https://github.com/rschu1ze)). +* add **SYSTEM SYNC FILE CACHE** command. It will call sync syscall. It achieve [#8921](https://github.com/ClickHouse/ClickHouse/issues/8921). [#45685](https://github.com/ClickHouse/ClickHouse/pull/45685) ([DR](https://github.com/freedomDR)). +* Add new S3 setting `allow_head_object_request`. This PR makes usage of `GetObjectAttributes` request instead of `HeadObject` introduced in https://github.com/ClickHouse/ClickHouse/pull/45288 optional (and disabled by default). [#45701](https://github.com/ClickHouse/ClickHouse/pull/45701) ([Vitaly Baranov](https://github.com/vitlibar)). +* Add ability to override connection settings based on connection names (that said that now you can forget about storing password for each connection, you can simply put everything into `~/.clickhouse-client/config.xml` and even use different history files for them, which can be also useful). [#45715](https://github.com/ClickHouse/ClickHouse/pull/45715) ([Azat Khuzhin](https://github.com/azat)). +* Arrow format support duration type. Closes [#45669](https://github.com/ClickHouse/ClickHouse/issues/45669). [#45750](https://github.com/ClickHouse/ClickHouse/pull/45750) ([flynn](https://github.com/ucasfl)). +* Extend the logging in the Query Cache to improve investigations of the caching behavior. [#45751](https://github.com/ClickHouse/ClickHouse/pull/45751) ([Robert Schulze](https://github.com/rschu1ze)). +* The query cache's server-level settings are now reconfigurable at runtime. [#45758](https://github.com/ClickHouse/ClickHouse/pull/45758) ([Robert Schulze](https://github.com/rschu1ze)). +* Hide password in logs when a table function's arguments are specified with a named collection:. [#45774](https://github.com/ClickHouse/ClickHouse/pull/45774) ([Vitaly Baranov](https://github.com/vitlibar)). +* Improve internal S3 client to correctly deduce regions and redirections for different types of URLs. [#45783](https://github.com/ClickHouse/ClickHouse/pull/45783) ([Antonio Andelic](https://github.com/antonio2368)). +* - Add support for Map, IPv4 and IPv6 types in generateRandom. Mostly useful for testing. [#45785](https://github.com/ClickHouse/ClickHouse/pull/45785) ([Raúl Marín](https://github.com/Algunenano)). +* Support empty/notEmpty for IP types. [#45799](https://github.com/ClickHouse/ClickHouse/pull/45799) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* The column `num_processed_files` was splitted into two columns: `num_files` (for BACKUP) and `files_read` (for RESTORE). The column `processed_files_size` was splitted into two columns: `total_size` (for BACKUP) and `bytes_read` (for RESTORE). [#45800](https://github.com/ClickHouse/ClickHouse/pull/45800) ([Vitaly Baranov](https://github.com/vitlibar)). +* 1. Upgrade Intel QPL from v0.3.0 to v1.0.0 2. Build libaccel-config and link it statically to QPL library instead of dynamically. [#45809](https://github.com/ClickHouse/ClickHouse/pull/45809) ([jasperzhu](https://github.com/jinjunzh)). +* Add support for `SHOW ENGINES` query. [#45859](https://github.com/ClickHouse/ClickHouse/pull/45859) ([Filatenkov Artur](https://github.com/FArthur-cmd)). +* - Improved how the obfuscator deals with queries. [#45867](https://github.com/ClickHouse/ClickHouse/pull/45867) ([Raúl Marín](https://github.com/Algunenano)). +* Improved how memory bound merging and aggregation in order on top query plan interact. Previously we fell back to explicit sorting for AIO in some cases when it wasn't actually needed. So it is a perf issue, not a correctness one. [#45892](https://github.com/ClickHouse/ClickHouse/pull/45892) ([Nikita Taranov](https://github.com/nickitat)). +* Improve behaviour of conversion into Date for boundary value 65535 (2149-06-06). [#45914](https://github.com/ClickHouse/ClickHouse/pull/45914) ([Joanna Hulboj](https://github.com/jh0x)). +* Add setting `check_referential_table_dependencies` to check referential dependencies on `DROP TABLE`. This PR solves [#38326](https://github.com/ClickHouse/ClickHouse/issues/38326). [#45936](https://github.com/ClickHouse/ClickHouse/pull/45936) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix `tupleElement` return `Null` when having `Null` argument. Closes [#45894](https://github.com/ClickHouse/ClickHouse/issues/45894). [#45952](https://github.com/ClickHouse/ClickHouse/pull/45952) ([flynn](https://github.com/ucasfl)). +* Throw an error on no files satisfying S3 wildcard. Closes [#45587](https://github.com/ClickHouse/ClickHouse/issues/45587). [#45957](https://github.com/ClickHouse/ClickHouse/pull/45957) ([chen](https://github.com/xiedeyantu)). +* Use cluster state data to check concurrent backup/restore. [#45982](https://github.com/ClickHouse/ClickHouse/pull/45982) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Use "exact" matching for fuzzy search, which has correct case ignorance and more appropriate algorithm for matching SQL queries. [#46000](https://github.com/ClickHouse/ClickHouse/pull/46000) ([Azat Khuzhin](https://github.com/azat)). +* Improve behaviour of conversion into Date for boundary value 65535 (2149-06-06). [#46042](https://github.com/ClickHouse/ClickHouse/pull/46042) ([Joanna Hulboj](https://github.com/jh0x)). +* Forbid wrong create View syntax `CREATE View X TO Y AS SELECT`. Closes [#4331](https://github.com/ClickHouse/ClickHouse/issues/4331). [#46043](https://github.com/ClickHouse/ClickHouse/pull/46043) ([flynn](https://github.com/ucasfl)). +* Storage Log family support settings `storage_policy`. Closes [#43421](https://github.com/ClickHouse/ClickHouse/issues/43421). [#46044](https://github.com/ClickHouse/ClickHouse/pull/46044) ([flynn](https://github.com/ucasfl)). +* Improve format `JSONColumns` when result is empty. Closes [#46024](https://github.com/ClickHouse/ClickHouse/issues/46024). [#46053](https://github.com/ClickHouse/ClickHouse/pull/46053) ([flynn](https://github.com/ucasfl)). +* - MultiVersion: replace lock_guard to atomic op. [#46057](https://github.com/ClickHouse/ClickHouse/pull/46057) ([Konstantin Morozov](https://github.com/k-morozov)). +* Add reference implementation for SipHash128. [#46065](https://github.com/ClickHouse/ClickHouse/pull/46065) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Add new metric to record allocations times and bytes using mmap. [#46068](https://github.com/ClickHouse/ClickHouse/pull/46068) ([李扬](https://github.com/taiyang-li)). +* Currently for functions like `leftPad`, `rightPad`, `leftPadUTF8`, `rightPadUTF8`, the second argument `length` must be UInt8|16|32|64|128|256. Which is too strict for clickhouse users, besides, it is not consistent with other similar functions like `arrayResize`, `substring` and so on. [#46103](https://github.com/ClickHouse/ClickHouse/pull/46103) ([李扬](https://github.com/taiyang-li)). +* Update CapnProto to v0.10.3 to avoid CVE-2022-46149 ### Documentation entry for user-facing changes. [#46139](https://github.com/ClickHouse/ClickHouse/pull/46139) ([Mallik Hassan](https://github.com/SadiHassan)). +* Fix assertion in the `welchTTest` function in debug build when the resulting statistics is NaN. Unified the behavior with other similar functions. Change the behavior of `studentTTest` to return NaN instead of throwing an exception because the previous behavior was inconvenient. This closes [#41176](https://github.com/ClickHouse/ClickHouse/issues/41176) This closes [#42162](https://github.com/ClickHouse/ClickHouse/issues/42162). [#46141](https://github.com/ClickHouse/ClickHouse/pull/46141) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* More convenient usage of big integers and ORDER BY WITH FILL. Allow using plain integers for start and end points in WITH FILL when ORDER BY big (128-bit and 256-bit) integers. Fix the wrong result for big integers with negative start or end points. This closes [#16733](https://github.com/ClickHouse/ClickHouse/issues/16733). [#46152](https://github.com/ClickHouse/ClickHouse/pull/46152) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add parts, active_parts and total_marks columns to system.tables on [issue](https://github.com/ClickHouse/ClickHouse/issues/44336). [#46161](https://github.com/ClickHouse/ClickHouse/pull/46161) ([attack204](https://github.com/attack204)). +* Functions "multi[Fuzzy]Match(Any|AnyIndex|AllIndices}" now reject regexes which will likely evaluate very slowly in vectorscan. [#46167](https://github.com/ClickHouse/ClickHouse/pull/46167) ([Robert Schulze](https://github.com/rschu1ze)). +* When `insert_null_as_default` is enabled and column doesn't have defined default value, the default of column type will be used. Also this PR fixes using default values on nulls in case of LowCardinality columns. [#46171](https://github.com/ClickHouse/ClickHouse/pull/46171) ([Kruglov Pavel](https://github.com/Avogar)). +* Prefer explicitly defined access keys for S3 clients. If `use_environment_credentials` is set to `true`, and the user has provided the access key through query or config, they will be used instead of the ones from the environment variable. [#46191](https://github.com/ClickHouse/ClickHouse/pull/46191) ([Antonio Andelic](https://github.com/antonio2368)). +* Concurrent merges are scheduled using round-robin by default to ensure fair and starvation-free operation. Previously in heavily overloaded shards, big merges could possibly be starved by smaller merges due to the use of strict priority scheduling. Added `background_merges_mutations_scheduling_policy` server config option to select scheduling algorithm (`round_robin` or `shortest_task_first`). [#46247](https://github.com/ClickHouse/ClickHouse/pull/46247) ([Sergei Trifonov](https://github.com/serxa)). +* Extend setting `input_format_null_as_default` for more formats. Fix setting `input_format_defaults_for_omitted_fields` with Native and TSKV formats. [#46284](https://github.com/ClickHouse/ClickHouse/pull/46284) ([Kruglov Pavel](https://github.com/Avogar)). +* - Add an alias "DATE_FORMAT()" for function "formatDateTime()" to improve compatibility with MySQL's SQL dialect, extend function`formatDateTime()` with substitutions "a", "b", "c", "h", "i", "k", "l" "r", "s", "W". ### Documentation entry for user-facing changes User-readable short description: `DATE_FORMAT` is an alias of `formatDateTime`. Formats a Time according to the given Format string. Format is a constant expression, so you cannot have multiple formats for a single result column. (Provide link to [formatDateTime](https://clickhouse.com/docs/en/sql-reference/functions/date-time-functions/#formatdatetime)). [#46302](https://github.com/ClickHouse/ClickHouse/pull/46302) ([Jake Bamrah](https://github.com/JakeBamrah)). +* not for changelog - part of [#42648](https://github.com/ClickHouse/ClickHouse/issues/42648). [#46306](https://github.com/ClickHouse/ClickHouse/pull/46306) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Enable retries for INSERT by default in case of ZooKeeper session loss. We already use it in production. [#46308](https://github.com/ClickHouse/ClickHouse/pull/46308) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add `ProfileEvents` and `CurrentMetrics` about the callback tasks for parallel replicas (`s3Cluster` and `MergeTree` tables). [#46313](https://github.com/ClickHouse/ClickHouse/pull/46313) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add support for `DELETE` and `UPDATE` for tables using `KeeperMap` storage engine. [#46330](https://github.com/ClickHouse/ClickHouse/pull/46330) ([Antonio Andelic](https://github.com/antonio2368)). +* Update unixodbc to v2.3.11 to mitigate CVE-2011-1145 ### Documentation entry for user-facing changes. [#46363](https://github.com/ClickHouse/ClickHouse/pull/46363) ([Mallik Hassan](https://github.com/SadiHassan)). +* - Apply `ALTER TABLE table_name ON CLUSTER cluster MOVE PARTITION|PART partition_expr TO DISK|VOLUME 'disk_name'` to all replicas. Because `ALTER TABLE t MOVE` is not replicated. [#46402](https://github.com/ClickHouse/ClickHouse/pull/46402) ([lizhuoyu5](https://github.com/lzydmxy)). +* Allow writing RENAME queries with query parameters. Resolves [#45778](https://github.com/ClickHouse/ClickHouse/issues/45778). [#46407](https://github.com/ClickHouse/ClickHouse/pull/46407) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix parameterized SELECT queries with REPLACE transformer. Resolves [#33002](https://github.com/ClickHouse/ClickHouse/issues/33002). [#46420](https://github.com/ClickHouse/ClickHouse/pull/46420) ([Nikolay Degterinsky](https://github.com/evillique)). +* Exclude the internal database used for temporary/external tables from the calculation of asynchronous metric "NumberOfDatabases". This makes the behavior consistent with system table "system.databases". [#46435](https://github.com/ClickHouse/ClickHouse/pull/46435) ([Robert Schulze](https://github.com/rschu1ze)). +* Added `last_exception_time` column into distribution_queue table. [#46564](https://github.com/ClickHouse/ClickHouse/pull/46564) ([Aleksandr](https://github.com/AVMusorin)). +* Support for IN clause in parameterized views Implementation: * In case of parameterized views, the IN clause cannot be evaluated as constant expression during CREATE VIEW, added a check to ignore this step in case of parameterized view. * If parmeters are not in IN clause, we continue to evaluate it as constant expression. [#46583](https://github.com/ClickHouse/ClickHouse/pull/46583) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Do not load named collections on server startup (load them on first access instead). [#46607](https://github.com/ClickHouse/ClickHouse/pull/46607) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add separate access type `SHOW_NAMED_COLLECTIONS_SECRETS` to allow to see named collections and their keys, but making values hidden. Nevertheless, access type `SHOW NAMED COLLECTIONS` is still required. [#46667](https://github.com/ClickHouse/ClickHouse/pull/46667) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Hide arguments of custom disk merge tree setting. [#46670](https://github.com/ClickHouse/ClickHouse/pull/46670) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Ask for the password in clickhouse-client interactively in a case when the empty password is wrong. Closes [#46702](https://github.com/ClickHouse/ClickHouse/issues/46702). [#46730](https://github.com/ClickHouse/ClickHouse/pull/46730) ([Nikolay Degterinsky](https://github.com/evillique)). +* Backward compatibility for T64 codec support for IPv4. [#46747](https://github.com/ClickHouse/ClickHouse/pull/46747) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Allow to fallback from asynchronous insert to synchronous in case of large amount of data (more than `async_insert_max_data_size` bytes in single insert). [#46753](https://github.com/ClickHouse/ClickHouse/pull/46753) ([Anton Popov](https://github.com/CurtizJ)). + +#### Bug Fix +* Fix wiping sensitive info in logs. [#45603](https://github.com/ClickHouse/ClickHouse/pull/45603) ([Vitaly Baranov](https://github.com/vitlibar)). +* There is a check in format "time_check() || ptr ? ptr->finished() : data->is_finished()". Operator "||" will be executed before operator "?", but expected that there should be separated time and ptr checks. Also it's unexpected to run "ptr->finished()" in case of nullptr, but with current expression it's possible. [#46054](https://github.com/ClickHouse/ClickHouse/pull/46054) ([Alexey Perevyshin](https://github.com/alexX512)). + +#### Build/Testing/Packaging Improvement +* Allow to randomize merge tree settings in tests. [#38983](https://github.com/ClickHouse/ClickHouse/pull/38983) ([Anton Popov](https://github.com/CurtizJ)). +* Enable the HDFS support in PowerPC and which helps to fixes the following functional tests 02113_hdfs_assert.sh, 02244_hdfs_cluster.sql and 02368_cancel_write_into_hdfs.sh. [#44949](https://github.com/ClickHouse/ClickHouse/pull/44949) ([MeenaRenganathan22](https://github.com/MeenaRenganathan22)). +* Add systemd.service file for clickhouse-keeper. Fixes [#44293](https://github.com/ClickHouse/ClickHouse/issues/44293). [#45568](https://github.com/ClickHouse/ClickHouse/pull/45568) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* ClickHouse's fork of poco was moved from "contrib/" to "base/poco/". [#46075](https://github.com/ClickHouse/ClickHouse/pull/46075) ([Robert Schulze](https://github.com/rschu1ze)). +* Remove excessive license notices from preciseExp10.cpp. [#46163](https://github.com/ClickHouse/ClickHouse/pull/46163) ([DimasKovas](https://github.com/DimasKovas)). +* Add an option for `clickhouse-watchdog` to restart the child process. This does not make a lot of use. [#46312](https://github.com/ClickHouse/ClickHouse/pull/46312) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Get rid of unnecessary build for standalone clickhouse-keeper. [#46367](https://github.com/ClickHouse/ClickHouse/pull/46367) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* If the environment variable `CLICKHOUSE_DOCKER_RESTART_ON_EXIT` is set to 1, the Docker container will run `clickhouse-server` as a child instead of the first process, and restart it when it exited. [#46391](https://github.com/ClickHouse/ClickHouse/pull/46391) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* - Fix Systemd service file. [#46461](https://github.com/ClickHouse/ClickHouse/pull/46461) ([SuperDJY](https://github.com/cmsxbc)). +* Some time ago the ccache compression was changed to `zst`, but `gz` archives are downloaded by default. It fixes it by prioritizing zst archive. [#46490](https://github.com/ClickHouse/ClickHouse/pull/46490) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Raised the minimum Clang version needed to build ClickHouse from 12 to 15. [#46710](https://github.com/ClickHouse/ClickHouse/pull/46710) ([Robert Schulze](https://github.com/rschu1ze)). + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Flush data exactly by `rabbitmq_flush_interval_ms` or by `rabbitmq_max_block_size` in `StorageRabbitMQ`. Closes [#42389](https://github.com/ClickHouse/ClickHouse/issues/42389). Closes [#45160](https://github.com/ClickHouse/ClickHouse/issues/45160). [#44404](https://github.com/ClickHouse/ClickHouse/pull/44404) ([Kseniia Sumarokova](https://github.com/kssenii)). +* - Use PODArray to render in sparkBar function, so we can control the memory usage. Close [#44467](https://github.com/ClickHouse/ClickHouse/issues/44467). [#44489](https://github.com/ClickHouse/ClickHouse/pull/44489) ([Duc Canh Le](https://github.com/canhld94)). +* Fix functions (quantilesExactExclusive, quantilesExactInclusive) return unsorted array element. [#45379](https://github.com/ClickHouse/ClickHouse/pull/45379) ([wujunfu](https://github.com/wujunfu)). +* Fix uncaught exception in HTTPHandler when open telemetry is enabled. [#45456](https://github.com/ClickHouse/ClickHouse/pull/45456) ([Frank Chen](https://github.com/FrankChen021)). +* Don't infer Dates from 8 digit numbers. It could lead to wrong data to be read. [#45581](https://github.com/ClickHouse/ClickHouse/pull/45581) ([Kruglov Pavel](https://github.com/Avogar)). +* Fixes to correctly use `odbc_bridge_use_connection_pooling` setting. [#45591](https://github.com/ClickHouse/ClickHouse/pull/45591) ([Bharat Nallan](https://github.com/bharatnc)). +* when the callback in the cache is called, it is possible that this cache is destructed. To keep it safe, we capture members by value. It's also safe for task schedule because it will be deactivated before storage is destroyed. Resolve [#45548](https://github.com/ClickHouse/ClickHouse/issues/45548). [#45601](https://github.com/ClickHouse/ClickHouse/pull/45601) ([Han Fei](https://github.com/hanfei1991)). +* - Fix data corruption when codecs Delta or DoubleDelta are combined with codec Gorilla. [#45615](https://github.com/ClickHouse/ClickHouse/pull/45615) ([Robert Schulze](https://github.com/rschu1ze)). +* Correctly check types when using N-gram bloom filter index to avoid invalid reads. [#45617](https://github.com/ClickHouse/ClickHouse/pull/45617) ([Antonio Andelic](https://github.com/antonio2368)). +* A couple of seg faults have been reported around `c-ares`. All of the recent stack traces observed fail on inserting into `std::unodered_set<>`. I believe I have found the root cause of this, it seems to be unprocessed queries. Prior to this PR, CH calls `poll` to wait on the file descriptors in the `c-ares` channel. According to the [poll docs](https://man7.org/linux/man-pages/man2/poll.2.html), a negative return value means an error has ocurred. Because of this, we would abort the execution and return failure. The problem is that `poll` will also return a negative value if a system interrupt occurs. A system interrupt does not mean the processing has failed or ended, but we would abort it anyways because we were checking for negative values. Once the execution is aborted, the whole stack is destroyed, which includes the `std::unordered_set` passed to the `void *` parameter of the c-ares callback. Once c-ares completed the request, the callback would be invoked and would access an invalid memory address causing a segfault. [#45629](https://github.com/ClickHouse/ClickHouse/pull/45629) ([Arthur Passos](https://github.com/arthurpassos)). +* Fix key description when encountering duplicate primary keys. This can happen in projections. See [#45590](https://github.com/ClickHouse/ClickHouse/issues/45590) for details. [#45686](https://github.com/ClickHouse/ClickHouse/pull/45686) ([Amos Bird](https://github.com/amosbird)). +* Set compression method and level for backup Closes [#45690](https://github.com/ClickHouse/ClickHouse/issues/45690). [#45737](https://github.com/ClickHouse/ClickHouse/pull/45737) ([Pradeep Chhetri](https://github.com/chhetripradeep)). +* Should use `select_query_typed.limitByOffset()` instead of `select_query_typed.limitOffset()`. [#45817](https://github.com/ClickHouse/ClickHouse/pull/45817) ([刘陶峰](https://github.com/taofengliu)). +* When use experimental analyzer, queries like `SELECT number FROM numbers(100) LIMIT 10 OFFSET 10;` get wrong results (empty result for this sql). That is caused by an unnecessary offset step added by planner. [#45822](https://github.com/ClickHouse/ClickHouse/pull/45822) ([刘陶峰](https://github.com/taofengliu)). +* Backward compatibility - allow implicit narrowing conversion from UInt64 to IPv4 - required for "INSERT ... VALUES ..." expression. [#45865](https://github.com/ClickHouse/ClickHouse/pull/45865) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Bugfix IPv6 parser for mixed ip4 address with missed first octet (like `::.1.2.3`). [#45871](https://github.com/ClickHouse/ClickHouse/pull/45871) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Add the `query_kind` column to the `system.processes` table and the `SHOW PROCESSLIST` query. Remove duplicate code. It fixes a bug: the global configuration parameter `max_concurrent_select_queries` was not respected to queries with `INTERSECT` or `EXCEPT` chains. [#45872](https://github.com/ClickHouse/ClickHouse/pull/45872) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix crash in a function `stochasticLinearRegression`. Found by WingFuzz. [#45985](https://github.com/ClickHouse/ClickHouse/pull/45985) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix crash in `SELECT` queries with `INTERSECT` and `EXCEPT` modifiers that read data from tables with enabled sparse columns (controlled by setting `ratio_of_defaults_for_sparse_serialization). [#45987](https://github.com/ClickHouse/ClickHouse/pull/45987) ([Anton Popov](https://github.com/CurtizJ)). +* * Fix read in order optimization for DESC sorting with FINAL, close [#45815](https://github.com/ClickHouse/ClickHouse/issues/45815). [#46009](https://github.com/ClickHouse/ClickHouse/pull/46009) ([Vladimir C](https://github.com/vdimir)). +* Fix reading of non existing nested columns with multiple level in compact parts. [#46045](https://github.com/ClickHouse/ClickHouse/pull/46045) ([Azat Khuzhin](https://github.com/azat)). +* Fix elapsed column in system.processes (10x error). [#46047](https://github.com/ClickHouse/ClickHouse/pull/46047) ([Azat Khuzhin](https://github.com/azat)). +* Follow-up fix for Replace domain IP types (IPv4, IPv6) with native https://github.com/ClickHouse/ClickHouse/pull/43221. [#46087](https://github.com/ClickHouse/ClickHouse/pull/46087) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix environment variable substitution in the configuration when a parameter already has a value. This closes [#46131](https://github.com/ClickHouse/ClickHouse/issues/46131). This closes [#9547](https://github.com/ClickHouse/ClickHouse/issues/9547). [#46144](https://github.com/ClickHouse/ClickHouse/pull/46144) ([pufit](https://github.com/pufit)). +* Fix incorrect predicate push down with grouping sets. Closes [#45947](https://github.com/ClickHouse/ClickHouse/issues/45947). [#46151](https://github.com/ClickHouse/ClickHouse/pull/46151) ([flynn](https://github.com/ucasfl)). +* Fix possible pipeline stuck error on `fulls_sorting_join` with constant keys. [#46175](https://github.com/ClickHouse/ClickHouse/pull/46175) ([Vladimir C](https://github.com/vdimir)). +* Never rewrite tuple functions as literals during formatting to avoid incorrect results. [#46232](https://github.com/ClickHouse/ClickHouse/pull/46232) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Fix possible out of bounds error while reading LowCardinality(Nullable) in Arrow format. [#46270](https://github.com/ClickHouse/ClickHouse/pull/46270) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix `SYSTEM UNFREEZE` queries failing with the exception `CANNOT_PARSE_INPUT_ASSERTION_FAILED`. [#46325](https://github.com/ClickHouse/ClickHouse/pull/46325) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Fix possible crash which can be caused by an integer overflow while deserializing aggregating state of a function that stores HashTable. [#46349](https://github.com/ClickHouse/ClickHouse/pull/46349) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix possible `LOGICAL_ERROR` in asynchronous inserts with invalid data sent in format `VALUES`. [#46350](https://github.com/ClickHouse/ClickHouse/pull/46350) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed a LOGICAL_ERROR on an attempt to execute `ALTER ... MOVE PART ... TO TABLE`. This type of query was never actually supported. [#46359](https://github.com/ClickHouse/ClickHouse/pull/46359) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix s3Cluster schema inference in parallel distributed insert select when `parallel_distributed_insert_select` is enabled. [#46381](https://github.com/ClickHouse/ClickHouse/pull/46381) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix queries like `ALTER TABLE ... UPDATE nested.arr1 = nested.arr2 ...`, where `arr1` and `arr2` are fields of the same `Nested` column. [#46387](https://github.com/ClickHouse/ClickHouse/pull/46387) ([Anton Popov](https://github.com/CurtizJ)). +* Scheduler may fail to schedule a task. If it happens, the whole MulityPartUpload should be aborted and `UploadHelper` must wait for already scheduled tasks. [#46451](https://github.com/ClickHouse/ClickHouse/pull/46451) ([Dmitry Novik](https://github.com/novikd)). +* Fix PREWHERE for Merge with different default types (fixes some `NOT_FOUND_COLUMN_IN_BLOCK` when the default type for the column differs, also allow `PREWHERE` when the type of column is the same across tables, and prohibit it, only if it differs). [#46454](https://github.com/ClickHouse/ClickHouse/pull/46454) ([Azat Khuzhin](https://github.com/azat)). +* Fix a crash that could happen when constant values are used in `ORDER BY`. Fixes [#46466](https://github.com/ClickHouse/ClickHouse/issues/46466). [#46493](https://github.com/ClickHouse/ClickHouse/pull/46493) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Do not throw exception if `disk` setting was specified on query level, but `storage_policy` was specified in config merge tree settings section. `disk` will override setting from config. [#46533](https://github.com/ClickHouse/ClickHouse/pull/46533) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix an invalid processing of constant `LowCardinality` argument in function `arrayMap`. This bug could lead to a segfault in release, and logical error `Bad cast` in debug build. [#46569](https://github.com/ClickHouse/ClickHouse/pull/46569) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* fixes [#46557](https://github.com/ClickHouse/ClickHouse/issues/46557). [#46611](https://github.com/ClickHouse/ClickHouse/pull/46611) ([Alexander Gololobov](https://github.com/davenger)). +* Fix endless restarts of clickhouse-server systemd unit if server cannot start within 1m30sec (Disable timeout logic for starting clickhouse-server from systemd service). [#46613](https://github.com/ClickHouse/ClickHouse/pull/46613) ([Azat Khuzhin](https://github.com/azat)). +* Allocated during asynchronous inserts memory buffers were deallocated in the global context and MemoryTracker counters for corresponding user and query were not updated correctly. That led to false positive OOM exceptions. [#46622](https://github.com/ClickHouse/ClickHouse/pull/46622) ([Dmitry Novik](https://github.com/novikd)). +* Fix totals and extremes with constants in clickhouse-local. Closes [#43831](https://github.com/ClickHouse/ClickHouse/issues/43831). [#46669](https://github.com/ClickHouse/ClickHouse/pull/46669) ([Kruglov Pavel](https://github.com/Avogar)). +* Handle `input_format_null_as_default` for nested types. [#46725](https://github.com/ClickHouse/ClickHouse/pull/46725) ([Azat Khuzhin](https://github.com/azat)). + +#### Bug-fix + +* Updated to not clear on_expression from table_join as its used by future analyze runs resolves [#45185](https://github.com/ClickHouse/ClickHouse/issues/45185). [#46487](https://github.com/ClickHouse/ClickHouse/pull/46487) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). + +#### Build Improvement + +* Fixed endian issue in snappy library for s390x. [#45670](https://github.com/ClickHouse/ClickHouse/pull/45670) ([Harry Lee](https://github.com/HarryLeeIBM)). +* Fixed endian issue in CityHash for s390x. [#46096](https://github.com/ClickHouse/ClickHouse/pull/46096) ([Harry Lee](https://github.com/HarryLeeIBM)). +* Fixed Functional Test 00900_long_parquet for S390x. [#46181](https://github.com/ClickHouse/ClickHouse/pull/46181) ([Sanjam Panda](https://github.com/saitama951)). +* Fixed endian issues in SQL hash functions on s390x architectures. [#46495](https://github.com/ClickHouse/ClickHouse/pull/46495) ([Harry Lee](https://github.com/HarryLeeIBM)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Add check for running workflows to merge_pr.py"'. [#45802](https://github.com/ClickHouse/ClickHouse/pull/45802) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* NO CL ENTRY: 'Revert "Improve behaviour of conversion into Date for boundary value 65535"'. [#46007](https://github.com/ClickHouse/ClickHouse/pull/46007) ([Antonio Andelic](https://github.com/antonio2368)). +* NO CL ENTRY: 'Revert "Allow vertical merges from compact to wide parts"'. [#46236](https://github.com/ClickHouse/ClickHouse/pull/46236) ([Anton Popov](https://github.com/CurtizJ)). +* NO CL ENTRY: 'Revert "Beter diagnostics from http in clickhouse-test"'. [#46301](https://github.com/ClickHouse/ClickHouse/pull/46301) ([Alexander Tokmakov](https://github.com/tavplubix)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Revert "Merge pull request [#38212](https://github.com/ClickHouse/ClickHouse/issues/38212) from azat/no-stress" [#38750](https://github.com/ClickHouse/ClickHouse/pull/38750) ([Azat Khuzhin](https://github.com/azat)). +* More interesting settings for Stress Tests [#41534](https://github.com/ClickHouse/ClickHouse/pull/41534) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Attempt to fix 'Local: No offset stored message' from Kafka [#42391](https://github.com/ClickHouse/ClickHouse/pull/42391) ([filimonov](https://github.com/filimonov)). +* Analyzer SETTINGS push down [#42976](https://github.com/ClickHouse/ClickHouse/pull/42976) ([Maksim Kita](https://github.com/kitaisreal)). +* Simply filesystem helpers to check is-readable/writable/executable [#43405](https://github.com/ClickHouse/ClickHouse/pull/43405) ([Azat Khuzhin](https://github.com/azat)). +* Add CPU flamegraphs for perf tests [#43529](https://github.com/ClickHouse/ClickHouse/pull/43529) ([Azat Khuzhin](https://github.com/azat)). +* More robust CI parsers [#44226](https://github.com/ClickHouse/ClickHouse/pull/44226) ([Azat Khuzhin](https://github.com/azat)). +* Fix error message for a broken distributed batches ("While sending batch") [#44907](https://github.com/ClickHouse/ClickHouse/pull/44907) ([Azat Khuzhin](https://github.com/azat)). +* Catch exceptions in BackgroundSchedulePool [#44923](https://github.com/ClickHouse/ClickHouse/pull/44923) ([Azat Khuzhin](https://github.com/azat)). +* Add encryption support to OpenSSL [#45258](https://github.com/ClickHouse/ClickHouse/pull/45258) ([Boris Kuschel](https://github.com/bkuschel)). +* Revert code in TreeRewriter for proper column order for UNION [#45282](https://github.com/ClickHouse/ClickHouse/pull/45282) ([Azat Khuzhin](https://github.com/azat)). +* Fix no shared id during drop for the fourth time [#45363](https://github.com/ClickHouse/ClickHouse/pull/45363) ([alesapin](https://github.com/alesapin)). +* HashedDictionary sharded fix nullable values [#45396](https://github.com/ClickHouse/ClickHouse/pull/45396) ([Maksim Kita](https://github.com/kitaisreal)). +* Another attempt to fix automerge, or at least to have debug footprint [#45476](https://github.com/ClickHouse/ClickHouse/pull/45476) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Simplify binary locating in clickhouse-test [#45484](https://github.com/ClickHouse/ClickHouse/pull/45484) ([Azat Khuzhin](https://github.com/azat)). +* Fix race in NuRaft's asio listener [#45511](https://github.com/ClickHouse/ClickHouse/pull/45511) ([Antonio Andelic](https://github.com/antonio2368)). +* Make ColumnNode::isEqualImpl more strict [#45518](https://github.com/ClickHouse/ClickHouse/pull/45518) ([Dmitry Novik](https://github.com/novikd)). +* Fix krb5 for OpenSSL [#45519](https://github.com/ClickHouse/ClickHouse/pull/45519) ([Boris Kuschel](https://github.com/bkuschel)). +* s390x build support [#45520](https://github.com/ClickHouse/ClickHouse/pull/45520) ([Suzy Wang](https://github.com/SuzyWangIBMer)). +* Better formatting for exception messages 2 [#45527](https://github.com/ClickHouse/ClickHouse/pull/45527) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Try to fix test `test_storage_s3/test.py::test_wrong_s3_syntax` (race in `StorageS3`) [#45529](https://github.com/ClickHouse/ClickHouse/pull/45529) ([Anton Popov](https://github.com/CurtizJ)). +* Analyzer add test for CREATE TABLE AS SELECT [#45533](https://github.com/ClickHouse/ClickHouse/pull/45533) ([Maksim Kita](https://github.com/kitaisreal)). +* LowCardinality insert fix [#45585](https://github.com/ClickHouse/ClickHouse/pull/45585) ([Maksim Kita](https://github.com/kitaisreal)). +* Update 02482_load_parts_refcounts.sh [#45604](https://github.com/ClickHouse/ClickHouse/pull/45604) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Extend assertion in buildPushingToViewsChain() to respect is_detached [#45610](https://github.com/ClickHouse/ClickHouse/pull/45610) ([Azat Khuzhin](https://github.com/azat)). +* Remove useless code [#45612](https://github.com/ClickHouse/ClickHouse/pull/45612) ([Anton Popov](https://github.com/CurtizJ)). +* Improve "at least part X is missing" error message [#45613](https://github.com/ClickHouse/ClickHouse/pull/45613) ([Azat Khuzhin](https://github.com/azat)). +* Refactoring of code near merge tree parts [#45619](https://github.com/ClickHouse/ClickHouse/pull/45619) ([Anton Popov](https://github.com/CurtizJ)). +* Update version after release [#45634](https://github.com/ClickHouse/ClickHouse/pull/45634) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update version_date.tsv and changelogs after v23.1.1.3077-stable [#45635](https://github.com/ClickHouse/ClickHouse/pull/45635) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Trim refs/tags/ from GITHUB_TAG in release workflow [#45636](https://github.com/ClickHouse/ClickHouse/pull/45636) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update version_date.tsv and changelogs after v22.10.7.13-stable [#45637](https://github.com/ClickHouse/ClickHouse/pull/45637) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Improve release script [#45657](https://github.com/ClickHouse/ClickHouse/pull/45657) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Suppress TOO_MANY_PARTS in BC check [#45691](https://github.com/ClickHouse/ClickHouse/pull/45691) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix build [#45692](https://github.com/ClickHouse/ClickHouse/pull/45692) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add recordings for 23.1 and Tel Aviv [#45695](https://github.com/ClickHouse/ClickHouse/pull/45695) ([Tyler Hannan](https://github.com/tylerhannan)). +* Integrate IO scheduler with buffers for remote reads and writes [#45711](https://github.com/ClickHouse/ClickHouse/pull/45711) ([Sergei Trifonov](https://github.com/serxa)). +* Add missing SYSTEM FLUSH LOGS for clickhouse-test [#45713](https://github.com/ClickHouse/ClickHouse/pull/45713) ([Azat Khuzhin](https://github.com/azat)). +* tests: add missing allow_suspicious_codecs in 02536_delta_gorilla_corruption (fixes fasttest) [#45735](https://github.com/ClickHouse/ClickHouse/pull/45735) ([Azat Khuzhin](https://github.com/azat)). +* Improve MEMERY_LIMIT_EXCEEDED exception message [#45743](https://github.com/ClickHouse/ClickHouse/pull/45743) ([Dmitry Novik](https://github.com/novikd)). +* Fix style and typo [#45744](https://github.com/ClickHouse/ClickHouse/pull/45744) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update version_date.tsv and changelogs after v22.8.13.20-lts [#45749](https://github.com/ClickHouse/ClickHouse/pull/45749) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v22.11.5.15-stable [#45754](https://github.com/ClickHouse/ClickHouse/pull/45754) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v23.1.2.9-stable [#45755](https://github.com/ClickHouse/ClickHouse/pull/45755) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Docs: Fix formatting [#45756](https://github.com/ClickHouse/ClickHouse/pull/45756) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix typo + add boringssl comment [#45757](https://github.com/ClickHouse/ClickHouse/pull/45757) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix flaky test, @alesapin please help! [#45759](https://github.com/ClickHouse/ClickHouse/pull/45759) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove trash [#45760](https://github.com/ClickHouse/ClickHouse/pull/45760) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix Flaky Check [#45765](https://github.com/ClickHouse/ClickHouse/pull/45765) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update dictionary.md [#45775](https://github.com/ClickHouse/ClickHouse/pull/45775) ([Derek Chia](https://github.com/DerekChia)). +* Added a test for multiple ignore subqueries with nested select [#45784](https://github.com/ClickHouse/ClickHouse/pull/45784) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Support DELETE ON CLUSTER [#45786](https://github.com/ClickHouse/ClickHouse/pull/45786) ([Alexander Gololobov](https://github.com/davenger)). +* outdated parts are loading async, need to wait them after attach [#45787](https://github.com/ClickHouse/ClickHouse/pull/45787) ([Sema Checherinda](https://github.com/CheSema)). +* Fix bug in tables drop which can lead to potential query hung [#45791](https://github.com/ClickHouse/ClickHouse/pull/45791) ([alesapin](https://github.com/alesapin)). +* Fix race condition on a part check cancellation [#45793](https://github.com/ClickHouse/ClickHouse/pull/45793) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Do not restrict count() query to 1 thread in isStorageTouchedByMutations() [#45794](https://github.com/ClickHouse/ClickHouse/pull/45794) ([Alexander Gololobov](https://github.com/davenger)). +* Fix test `test_azure_blob_storage_zero_copy_replication ` (memory leak in azure sdk) [#45796](https://github.com/ClickHouse/ClickHouse/pull/45796) ([Anton Popov](https://github.com/CurtizJ)). +* Add check for running workflows to merge_pr.py [#45801](https://github.com/ClickHouse/ClickHouse/pull/45801) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add check for running workflows to merge_pr.py [#45803](https://github.com/ClickHouse/ClickHouse/pull/45803) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix flaky test `02531_two_level_aggregation_bug.sh` [#45806](https://github.com/ClickHouse/ClickHouse/pull/45806) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Minor doc follow-up to [#45382](https://github.com/ClickHouse/ClickHouse/issues/45382) [#45816](https://github.com/ClickHouse/ClickHouse/pull/45816) ([Robert Schulze](https://github.com/rschu1ze)). +* Get rid of progress timestamps in release publishing [#45818](https://github.com/ClickHouse/ClickHouse/pull/45818) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Make separate DROP_PART log entry type [#45821](https://github.com/ClickHouse/ClickHouse/pull/45821) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Do not cancel created prs [#45823](https://github.com/ClickHouse/ClickHouse/pull/45823) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix ASTQualifiedAsterisk cloning [#45829](https://github.com/ClickHouse/ClickHouse/pull/45829) ([Raúl Marín](https://github.com/Algunenano)). +* Update 02540_duplicate_primary_key.sql [#45846](https://github.com/ClickHouse/ClickHouse/pull/45846) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Proper includes for ConnectionTimeoutsContext.h [#45848](https://github.com/ClickHouse/ClickHouse/pull/45848) ([Raúl Marín](https://github.com/Algunenano)). +* Fix minor mistake after refactoring [#45857](https://github.com/ClickHouse/ClickHouse/pull/45857) ([Anton Popov](https://github.com/CurtizJ)). +* Fix flaky ttl_replicated test (remove sleep) [#45858](https://github.com/ClickHouse/ClickHouse/pull/45858) ([alesapin](https://github.com/alesapin)). +* Add some context to stress test failures [#45869](https://github.com/ClickHouse/ClickHouse/pull/45869) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix clang-tidy error in Keeper `Changelog` [#45888](https://github.com/ClickHouse/ClickHouse/pull/45888) ([Antonio Andelic](https://github.com/antonio2368)). +* do not block merges when old parts are droping in drop queries [#45889](https://github.com/ClickHouse/ClickHouse/pull/45889) ([Sema Checherinda](https://github.com/CheSema)). +* do not run wal on remote disks [#45907](https://github.com/ClickHouse/ClickHouse/pull/45907) ([Sema Checherinda](https://github.com/CheSema)). +* Dashboard improvements [#45935](https://github.com/ClickHouse/ClickHouse/pull/45935) ([Kevin Zhang](https://github.com/Kinzeng)). +* Better context for stress tests failures [#45937](https://github.com/ClickHouse/ClickHouse/pull/45937) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix IO URing [#45940](https://github.com/ClickHouse/ClickHouse/pull/45940) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Docs: Remove obsolete query result cache page [#45958](https://github.com/ClickHouse/ClickHouse/pull/45958) ([Robert Schulze](https://github.com/rschu1ze)). +* Add necessary dependency for sanitizers [#45959](https://github.com/ClickHouse/ClickHouse/pull/45959) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update AggregateFunctionSparkbar [#45961](https://github.com/ClickHouse/ClickHouse/pull/45961) ([Vladimir C](https://github.com/vdimir)). +* Update cherrypick_pr to get mergeable state [#45972](https://github.com/ClickHouse/ClickHouse/pull/45972) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add "final" specifier to some classes [#45973](https://github.com/ClickHouse/ClickHouse/pull/45973) ([Robert Schulze](https://github.com/rschu1ze)). +* Improve local running of cherry_pick.py [#45980](https://github.com/ClickHouse/ClickHouse/pull/45980) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Properly detect changes in Rust code and recompile Rust libraries [#45981](https://github.com/ClickHouse/ClickHouse/pull/45981) ([Azat Khuzhin](https://github.com/azat)). +* Avoid leaving symbols leftovers for query fuzzy search [#45983](https://github.com/ClickHouse/ClickHouse/pull/45983) ([Azat Khuzhin](https://github.com/azat)). +* Fix basic functionality with type `Object` and new analyzer [#45992](https://github.com/ClickHouse/ClickHouse/pull/45992) ([Anton Popov](https://github.com/CurtizJ)). +* Check dynamic columns of part before its commit [#45995](https://github.com/ClickHouse/ClickHouse/pull/45995) ([Anton Popov](https://github.com/CurtizJ)). +* Minor doc fixes for inverted index [#46004](https://github.com/ClickHouse/ClickHouse/pull/46004) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix terribly broken, fragile and potentially cyclic linking [#46006](https://github.com/ClickHouse/ClickHouse/pull/46006) ([Robert Schulze](https://github.com/rschu1ze)). +* Docs: Mention time zone randomization [#46008](https://github.com/ClickHouse/ClickHouse/pull/46008) ([Robert Schulze](https://github.com/rschu1ze)). +* Analyzer limit offset test rename [#46011](https://github.com/ClickHouse/ClickHouse/pull/46011) ([Maksim Kita](https://github.com/kitaisreal)). +* Update version_date.tsv and changelogs after v23.1.3.5-stable [#46012](https://github.com/ClickHouse/ClickHouse/pull/46012) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update sorting properties after reading in order applied [#46014](https://github.com/ClickHouse/ClickHouse/pull/46014) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix disabled by mistake hung check [#46020](https://github.com/ClickHouse/ClickHouse/pull/46020) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix memory leak at creation of curl connection in azure sdk [#46025](https://github.com/ClickHouse/ClickHouse/pull/46025) ([Anton Popov](https://github.com/CurtizJ)). +* Add checks for installable packages to workflows [#46036](https://github.com/ClickHouse/ClickHouse/pull/46036) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix data race in BACKUP [#46040](https://github.com/ClickHouse/ClickHouse/pull/46040) ([Azat Khuzhin](https://github.com/azat)). +* Dump sanitizer errors in the integration tests logs [#46041](https://github.com/ClickHouse/ClickHouse/pull/46041) ([Azat Khuzhin](https://github.com/azat)). +* Temporarily disable one rabbitmq flaky test [#46052](https://github.com/ClickHouse/ClickHouse/pull/46052) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Remove unnecessary execute() while evaluating a constant expression. [#46058](https://github.com/ClickHouse/ClickHouse/pull/46058) ([Vitaly Baranov](https://github.com/vitlibar)). +* Polish S3 client [#46070](https://github.com/ClickHouse/ClickHouse/pull/46070) ([Antonio Andelic](https://github.com/antonio2368)). +* Smallish follow-up to [#46057](https://github.com/ClickHouse/ClickHouse/issues/46057) [#46072](https://github.com/ClickHouse/ClickHouse/pull/46072) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix 00002_log_and_exception_messages_formatting [#46077](https://github.com/ClickHouse/ClickHouse/pull/46077) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Disable temporarily rabbitmq tests which use channel.startConsuming() [#46078](https://github.com/ClickHouse/ClickHouse/pull/46078) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Update yarn packages for dev branches [#46079](https://github.com/ClickHouse/ClickHouse/pull/46079) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add helping logging to auto-merge script [#46080](https://github.com/ClickHouse/ClickHouse/pull/46080) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Simplify code around storages s3/hudi/delta-lake [#46083](https://github.com/ClickHouse/ClickHouse/pull/46083) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix build with `-DENABLE_LIBURING=0` (or `-DENABLE_LIBRARIES=0`) [#46088](https://github.com/ClickHouse/ClickHouse/pull/46088) ([Robert Schulze](https://github.com/rschu1ze)). +* Add also last messages from stdout/stderr/debuglog in clickhouse-test [#46090](https://github.com/ClickHouse/ClickHouse/pull/46090) ([Azat Khuzhin](https://github.com/azat)). +* Sanity assertions for closing file descriptors [#46091](https://github.com/ClickHouse/ClickHouse/pull/46091) ([Azat Khuzhin](https://github.com/azat)). +* Fix flaky rabbitmq test [#46107](https://github.com/ClickHouse/ClickHouse/pull/46107) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix test_merge_tree_azure_blob_storage::test_zero_copy_replication test [#46108](https://github.com/ClickHouse/ClickHouse/pull/46108) ([Azat Khuzhin](https://github.com/azat)). +* allow_drop_detached requires an argument [#46110](https://github.com/ClickHouse/ClickHouse/pull/46110) ([Sema Checherinda](https://github.com/CheSema)). +* Fix fault injection in copier and test_cluster_copier flakiness [#46120](https://github.com/ClickHouse/ClickHouse/pull/46120) ([Azat Khuzhin](https://github.com/azat)). +* Update liburing CMakeLists.txt [#46127](https://github.com/ClickHouse/ClickHouse/pull/46127) ([Nikolay Degterinsky](https://github.com/evillique)). +* Use BAD_ARGUMENTS over LOGICAL_ERROR for schema inference error file() over fd [#46132](https://github.com/ClickHouse/ClickHouse/pull/46132) ([Azat Khuzhin](https://github.com/azat)). +* Stricter warnings + fix whitespaces in poco [#46133](https://github.com/ClickHouse/ClickHouse/pull/46133) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix dependency checks [#46138](https://github.com/ClickHouse/ClickHouse/pull/46138) ([Vitaly Baranov](https://github.com/vitlibar)). +* Interpret `cluster_name` identifier in `s3Cluster` function as literal [#46143](https://github.com/ClickHouse/ClickHouse/pull/46143) ([Nikolay Degterinsky](https://github.com/evillique)). +* Remove flaky test [#46149](https://github.com/ClickHouse/ClickHouse/pull/46149) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix spelling + duplicate includes in poco [#46155](https://github.com/ClickHouse/ClickHouse/pull/46155) ([Robert Schulze](https://github.com/rschu1ze)). +* Add 00002_log_and_exception_messages_formatting back [#46156](https://github.com/ClickHouse/ClickHouse/pull/46156) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix clickhouse/clickhouse-server description to make it in sync [#46159](https://github.com/ClickHouse/ClickHouse/pull/46159) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Complain about missing Yasm at configure time at of build time [#46162](https://github.com/ClickHouse/ClickHouse/pull/46162) ([Robert Schulze](https://github.com/rschu1ze)). +* Update Dockerfile.ubuntu [#46173](https://github.com/ClickHouse/ClickHouse/pull/46173) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Cleanup disk unittest [#46179](https://github.com/ClickHouse/ClickHouse/pull/46179) ([Sergei Trifonov](https://github.com/serxa)). +* Update 01513_optimize_aggregation_in_order_memory_long.sql [#46180](https://github.com/ClickHouse/ClickHouse/pull/46180) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Make a bug in HTTP interface less annoying [#46183](https://github.com/ClickHouse/ClickHouse/pull/46183) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix write buffer destruction order for vertical merge. [#46205](https://github.com/ClickHouse/ClickHouse/pull/46205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* fix typo [#46207](https://github.com/ClickHouse/ClickHouse/pull/46207) ([Sergei Trifonov](https://github.com/serxa)). +* increase a time gap between insert and ttl move [#46233](https://github.com/ClickHouse/ClickHouse/pull/46233) ([Sema Checherinda](https://github.com/CheSema)). +* Make `test_replicated_merge_tree_s3_restore` less flaky [#46242](https://github.com/ClickHouse/ClickHouse/pull/46242) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix test_distributed_ddl_parallel [#46243](https://github.com/ClickHouse/ClickHouse/pull/46243) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Update 00564_versioned_collapsing_merge_tree.sql [#46245](https://github.com/ClickHouse/ClickHouse/pull/46245) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Optimize docker binary-builder [#46246](https://github.com/ClickHouse/ClickHouse/pull/46246) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update Curl to 7.87.0 [#46248](https://github.com/ClickHouse/ClickHouse/pull/46248) ([Boris Kuschel](https://github.com/bkuschel)). +* Upgrade libxml2 to address CVE-2022-40303 CVE-2022-40304 [#46249](https://github.com/ClickHouse/ClickHouse/pull/46249) ([larryluogit](https://github.com/larryluogit)). +* Run clang-format over poco [#46259](https://github.com/ClickHouse/ClickHouse/pull/46259) ([Robert Schulze](https://github.com/rschu1ze)). +* Suppress "Container already exists" in BC check [#46260](https://github.com/ClickHouse/ClickHouse/pull/46260) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix failure description for hung check [#46267](https://github.com/ClickHouse/ClickHouse/pull/46267) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add upcoming Events [#46271](https://github.com/ClickHouse/ClickHouse/pull/46271) ([Tyler Hannan](https://github.com/tylerhannan)). +* coordination: do not allow election_timeout_lower_bound_ms > election_timeout_upper_bound_ms [#46274](https://github.com/ClickHouse/ClickHouse/pull/46274) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* fix data race between check table request and background checker [#46278](https://github.com/ClickHouse/ClickHouse/pull/46278) ([Sema Checherinda](https://github.com/CheSema)). +* Try to make 02346_full_text_search less flaky [#46279](https://github.com/ClickHouse/ClickHouse/pull/46279) ([Robert Schulze](https://github.com/rschu1ze)). +* Beter diagnostics from http in clickhouse-test [#46281](https://github.com/ClickHouse/ClickHouse/pull/46281) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add more logging to RabbitMQ (to help debug tests) [#46283](https://github.com/ClickHouse/ClickHouse/pull/46283) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix window view test [#46285](https://github.com/ClickHouse/ClickHouse/pull/46285) ([Kseniia Sumarokova](https://github.com/kssenii)). +* suppressing test inaccuracy 00738_lock_for_inner_table [#46287](https://github.com/ClickHouse/ClickHouse/pull/46287) ([Sema Checherinda](https://github.com/CheSema)). +* Simplify ATTACH MergeTree table FROM S3 in tests [#46288](https://github.com/ClickHouse/ClickHouse/pull/46288) ([Azat Khuzhin](https://github.com/azat)). +* Update RabbitMQProducer.cpp [#46295](https://github.com/ClickHouse/ClickHouse/pull/46295) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix macOs compilation due to sprintf [#46298](https://github.com/ClickHouse/ClickHouse/pull/46298) ([Jordi Villar](https://github.com/jrdi)). +* Slightly improve error message for required Yasm assembler [#46328](https://github.com/ClickHouse/ClickHouse/pull/46328) ([Robert Schulze](https://github.com/rschu1ze)). +* Unifdef unused parts of poco [#46329](https://github.com/ClickHouse/ClickHouse/pull/46329) ([Robert Schulze](https://github.com/rschu1ze)). +* Trigger automerge on approved PRs [#46332](https://github.com/ClickHouse/ClickHouse/pull/46332) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Wait for background tasks in ~UploadHelper [#46334](https://github.com/ClickHouse/ClickHouse/pull/46334) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix flaky test_storage_rabbitmq::test_rabbitmq_address [#46337](https://github.com/ClickHouse/ClickHouse/pull/46337) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Extract common logic for S3 [#46339](https://github.com/ClickHouse/ClickHouse/pull/46339) ([Antonio Andelic](https://github.com/antonio2368)). +* Update cluster.py [#46340](https://github.com/ClickHouse/ClickHouse/pull/46340) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Try to stabilize test 02346_full_text_search.sql [#46344](https://github.com/ClickHouse/ClickHouse/pull/46344) ([Robert Schulze](https://github.com/rschu1ze)). +* Remove an unused argument [#46346](https://github.com/ClickHouse/ClickHouse/pull/46346) ([Alexander Tokmakov](https://github.com/tavplubix)). +* fix candidate selection [#46347](https://github.com/ClickHouse/ClickHouse/pull/46347) ([Sema Checherinda](https://github.com/CheSema)). +* Do not pollute logs in clickhouse-test [#46361](https://github.com/ClickHouse/ClickHouse/pull/46361) ([Azat Khuzhin](https://github.com/azat)). +* Do not continue perf tests in case of exception in create_query/fill_query [#46362](https://github.com/ClickHouse/ClickHouse/pull/46362) ([Azat Khuzhin](https://github.com/azat)). +* Minor fix in files locating for Bugfix validate check [#46368](https://github.com/ClickHouse/ClickHouse/pull/46368) ([Vladimir C](https://github.com/vdimir)). +* Temporarily disable test_rabbitmq_overloaded_insert [#46403](https://github.com/ClickHouse/ClickHouse/pull/46403) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix test test_rabbitmq_overloaded_insert [#46404](https://github.com/ClickHouse/ClickHouse/pull/46404) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix stress test [#46405](https://github.com/ClickHouse/ClickHouse/pull/46405) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix stress tests statuses [#46406](https://github.com/ClickHouse/ClickHouse/pull/46406) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Follow-up to [#46168](https://github.com/ClickHouse/ClickHouse/issues/46168) [#46409](https://github.com/ClickHouse/ClickHouse/pull/46409) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix noisy log messages [#46410](https://github.com/ClickHouse/ClickHouse/pull/46410) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Docs: Clarify query parameters [#46419](https://github.com/ClickHouse/ClickHouse/pull/46419) ([Robert Schulze](https://github.com/rschu1ze)). +* Make tests with window view less bad [#46421](https://github.com/ClickHouse/ClickHouse/pull/46421) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Move MongoDB and PostgreSQL sources to Sources folder [#46422](https://github.com/ClickHouse/ClickHouse/pull/46422) ([Nikolay Degterinsky](https://github.com/evillique)). +* Another fix for cluster copier [#46433](https://github.com/ClickHouse/ClickHouse/pull/46433) ([Antonio Andelic](https://github.com/antonio2368)). +* Update version_date.tsv and changelogs after v22.3.18.37-lts [#46436](https://github.com/ClickHouse/ClickHouse/pull/46436) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix a backup test [#46449](https://github.com/ClickHouse/ClickHouse/pull/46449) ([Vitaly Baranov](https://github.com/vitlibar)). +* Do not fetch submodules in release.py [#46450](https://github.com/ClickHouse/ClickHouse/pull/46450) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* resolve race in getCSNAndAssert [#46452](https://github.com/ClickHouse/ClickHouse/pull/46452) ([Sema Checherinda](https://github.com/CheSema)). +* move database credential inputs to the center on initial load [#46455](https://github.com/ClickHouse/ClickHouse/pull/46455) ([Kevin Zhang](https://github.com/Kinzeng)). +* Improve install_check.py [#46458](https://github.com/ClickHouse/ClickHouse/pull/46458) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Change logging level of a verbose message to Trace [#46459](https://github.com/ClickHouse/ClickHouse/pull/46459) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Analyzer planner fixes before enable by default [#46471](https://github.com/ClickHouse/ClickHouse/pull/46471) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix some flaky integration tests [#46478](https://github.com/ClickHouse/ClickHouse/pull/46478) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Allow to override host for client connection credentials [#46480](https://github.com/ClickHouse/ClickHouse/pull/46480) ([Azat Khuzhin](https://github.com/azat)). +* Try fix flaky test test_parallel_distributed_insert_select_with_schema_inference [#46488](https://github.com/ClickHouse/ClickHouse/pull/46488) ([Kruglov Pavel](https://github.com/Avogar)). +* Planner filter push down optimization fix [#46494](https://github.com/ClickHouse/ClickHouse/pull/46494) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix 01161_all_system_tables test flakiness [#46499](https://github.com/ClickHouse/ClickHouse/pull/46499) ([Azat Khuzhin](https://github.com/azat)). +* Compress tar archives with zstd in intergration tests [#46516](https://github.com/ClickHouse/ClickHouse/pull/46516) ([Azat Khuzhin](https://github.com/azat)). +* chore: bump testcontainers-go to 0.18.0 [#46518](https://github.com/ClickHouse/ClickHouse/pull/46518) ([Manuel de la Peña](https://github.com/mdelapenya)). +* Rollback unnecessary sync because of checking exit code [#46520](https://github.com/ClickHouse/ClickHouse/pull/46520) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix stress test [#46521](https://github.com/ClickHouse/ClickHouse/pull/46521) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add myrrc to trusted contributors [#46526](https://github.com/ClickHouse/ClickHouse/pull/46526) ([Anton Popov](https://github.com/CurtizJ)). +* fix style [#46530](https://github.com/ClickHouse/ClickHouse/pull/46530) ([flynn](https://github.com/ucasfl)). +* Autoupdate keeper dockerfile [#46535](https://github.com/ClickHouse/ClickHouse/pull/46535) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fixes for OpenSSL and s390x [#46546](https://github.com/ClickHouse/ClickHouse/pull/46546) ([Boris Kuschel](https://github.com/bkuschel)). +* enable async-insert-max-query-number only if async_insert_deduplicate [#46549](https://github.com/ClickHouse/ClickHouse/pull/46549) ([Han Fei](https://github.com/hanfei1991)). +* Remove extra try/catch for QueryState/LocalQueryState reset [#46552](https://github.com/ClickHouse/ClickHouse/pull/46552) ([Azat Khuzhin](https://github.com/azat)). +* Whitespaces [#46553](https://github.com/ClickHouse/ClickHouse/pull/46553) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* fix build without avro [#46554](https://github.com/ClickHouse/ClickHouse/pull/46554) ([flynn](https://github.com/ucasfl)). +* Inhibit randomization in test `01551_mergetree_read_in_order_spread.sql` [#46562](https://github.com/ClickHouse/ClickHouse/pull/46562) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove PVS-Studio [#46565](https://github.com/ClickHouse/ClickHouse/pull/46565) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Inhibit settings randomization in `01304_direct_io_long.sh` [#46566](https://github.com/ClickHouse/ClickHouse/pull/46566) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix double whitespace in comment in test [#46567](https://github.com/ClickHouse/ClickHouse/pull/46567) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Rename test [#46568](https://github.com/ClickHouse/ClickHouse/pull/46568) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix ASTAsterisk::clone() [#46570](https://github.com/ClickHouse/ClickHouse/pull/46570) ([Nikolay Degterinsky](https://github.com/evillique)). +* Small update of sparkbar docs [#46579](https://github.com/ClickHouse/ClickHouse/pull/46579) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix flakiness of expect tests for clickhouse-client by avoiding history overlap [#46582](https://github.com/ClickHouse/ClickHouse/pull/46582) ([Azat Khuzhin](https://github.com/azat)). +* Always log rollback for release.py [#46586](https://github.com/ClickHouse/ClickHouse/pull/46586) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Increase table retries in `test_cluster_copier` [#46590](https://github.com/ClickHouse/ClickHouse/pull/46590) ([Antonio Andelic](https://github.com/antonio2368)). +* Update 00170_s3_cache.sql [#46593](https://github.com/ClickHouse/ClickHouse/pull/46593) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix rabbitmq test [#46595](https://github.com/ClickHouse/ClickHouse/pull/46595) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix meilisearch test flakyness [#46596](https://github.com/ClickHouse/ClickHouse/pull/46596) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix dependencies for InstallPackagesTestAarch64 [#46597](https://github.com/ClickHouse/ClickHouse/pull/46597) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update compare.sh [#46599](https://github.com/ClickHouse/ClickHouse/pull/46599) ([Alexander Tokmakov](https://github.com/tavplubix)). +* update llvm-project to fix gwp-asan [#46600](https://github.com/ClickHouse/ClickHouse/pull/46600) ([Han Fei](https://github.com/hanfei1991)). +* Temporarily disable test_rabbitmq_overloaded_insert [#46608](https://github.com/ClickHouse/ClickHouse/pull/46608) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Update 01565_reconnect_after_client_error to not expect explicit reconnect [#46619](https://github.com/ClickHouse/ClickHouse/pull/46619) ([Azat Khuzhin](https://github.com/azat)). +* Inhibit `index_granularity_bytes` randomization in some tests [#46626](https://github.com/ClickHouse/ClickHouse/pull/46626) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix coverity [#46629](https://github.com/ClickHouse/ClickHouse/pull/46629) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix 01179_insert_values_semicolon test [#46636](https://github.com/ClickHouse/ClickHouse/pull/46636) ([Azat Khuzhin](https://github.com/azat)). +* Fix typo in read prefetch [#46640](https://github.com/ClickHouse/ClickHouse/pull/46640) ([Nikita Taranov](https://github.com/nickitat)). +* Avoid OOM in perf tests [#46641](https://github.com/ClickHouse/ClickHouse/pull/46641) ([Azat Khuzhin](https://github.com/azat)). +* Fix: remove redundant sorting optimization [#46642](https://github.com/ClickHouse/ClickHouse/pull/46642) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix flaky test 01710_normal_projections [#46645](https://github.com/ClickHouse/ClickHouse/pull/46645) ([Kruglov Pavel](https://github.com/Avogar)). +* Update postgres_utility.py [#46656](https://github.com/ClickHouse/ClickHouse/pull/46656) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix integration test: terminate old version without wait [#46660](https://github.com/ClickHouse/ClickHouse/pull/46660) ([alesapin](https://github.com/alesapin)). +* Break Stress tests [#46663](https://github.com/ClickHouse/ClickHouse/pull/46663) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Get rid of legacy DocsReleaseChecks [#46665](https://github.com/ClickHouse/ClickHouse/pull/46665) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* fix layout issues in dashboard.html [#46671](https://github.com/ClickHouse/ClickHouse/pull/46671) ([Kevin Zhang](https://github.com/Kinzeng)). +* Fix Stress tests [#46683](https://github.com/ClickHouse/ClickHouse/pull/46683) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Disable flaky test_ttl_move_memory_usage.py [#46687](https://github.com/ClickHouse/ClickHouse/pull/46687) ([Alexander Tokmakov](https://github.com/tavplubix)). +* BackgroundSchedulePool should not have any query context [#46709](https://github.com/ClickHouse/ClickHouse/pull/46709) ([Azat Khuzhin](https://github.com/azat)). +* Better exception message during Tuple JSON deserialization [#46727](https://github.com/ClickHouse/ClickHouse/pull/46727) ([Kruglov Pavel](https://github.com/Avogar)). +* Poco: POCO_HAVE_INT64 is always defined [#46728](https://github.com/ClickHouse/ClickHouse/pull/46728) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix SonarCloud Job [#46732](https://github.com/ClickHouse/ClickHouse/pull/46732) ([Julio Jimenez](https://github.com/juliojimenez)). +* Remove unused MergeTreeReadTask::remove_prewhere_column [#46744](https://github.com/ClickHouse/ClickHouse/pull/46744) ([Alexander Gololobov](https://github.com/davenger)). +* On out-of-space `at` returns error, we must terminate still [#46754](https://github.com/ClickHouse/ClickHouse/pull/46754) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* CI: don't run builds/tests when CHANGELOG.md or README.md were modified [#46773](https://github.com/ClickHouse/ClickHouse/pull/46773) ([Robert Schulze](https://github.com/rschu1ze)). +* Cosmetics in hashing code [#46780](https://github.com/ClickHouse/ClickHouse/pull/46780) ([Robert Schulze](https://github.com/rschu1ze)). + +#### Testing Improvement + +* Fixed functional test 00304_http_external_data for s390x. [#45807](https://github.com/ClickHouse/ClickHouse/pull/45807) ([Harry Lee](https://github.com/HarryLeeIBM)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index d313c4bfb78..3c0f5e7e72a 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v23.2.1.2537-stable 2023-02-23 v23.1.3.5-stable 2023-02-03 v23.1.2.9-stable 2023-01-29 v23.1.1.3077-stable 2023-01-25 From 25a759d8d156be980bd6584e00602234e04a8e35 Mon Sep 17 00:00:00 2001 From: Rich Raposa Date: Thu, 23 Feb 2023 14:41:45 -0700 Subject: [PATCH 251/253] Update insert-into.md Fix the formatting --- docs/en/sql-reference/statements/insert-into.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/insert-into.md b/docs/en/sql-reference/statements/insert-into.md index 609740dec56..03a4ab3453c 100644 --- a/docs/en/sql-reference/statements/insert-into.md +++ b/docs/en/sql-reference/statements/insert-into.md @@ -95,7 +95,7 @@ You can insert data separately from the query by using the command-line client o If table has [constraints](../../sql-reference/statements/create/table.md#constraints), their expressions will be checked for each row of inserted data. If any of those constraints is not satisfied — server will raise an exception containing constraint name and expression, the query will be stopped. -## Inserting the Results of `SELECT` +## Inserting the Results of SELECT **Syntax** From dd3591cb9c28230da59927e8213fe894c7050381 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 23 Feb 2023 22:07:08 +0000 Subject: [PATCH 252/253] Update version_date.tsv and changelogs after v22.11.6.44-stable --- docs/changelogs/v22.11.6.44-stable.md | 37 +++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 38 insertions(+) create mode 100644 docs/changelogs/v22.11.6.44-stable.md diff --git a/docs/changelogs/v22.11.6.44-stable.md b/docs/changelogs/v22.11.6.44-stable.md new file mode 100644 index 00000000000..6e628b85150 --- /dev/null +++ b/docs/changelogs/v22.11.6.44-stable.md @@ -0,0 +1,37 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v22.11.6.44-stable (73ddf91298f) FIXME as compared to v22.11.5.15-stable (d763e5a9239) + +#### Performance Improvement +* Backported in [#45703](https://github.com/ClickHouse/ClickHouse/issues/45703): Fixed performance of short `SELECT` queries that read from tables with large number of`Array`/`Map`/`Nested` columns. [#45630](https://github.com/ClickHouse/ClickHouse/pull/45630) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#46376](https://github.com/ClickHouse/ClickHouse/issues/46376): Fix too big memory usage for vertical merges on non-remote disk. Respect `max_insert_delayed_streams_for_parallel_write` for the remote disk. [#46275](https://github.com/ClickHouse/ClickHouse/pull/46275) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### Build/Testing/Packaging Improvement +* Backported in [#45977](https://github.com/ClickHouse/ClickHouse/issues/45977): Fix zookeeper downloading, update the version, and optimize the image size. [#44853](https://github.com/ClickHouse/ClickHouse/pull/44853) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#46114](https://github.com/ClickHouse/ClickHouse/issues/46114): Remove the dependency on the `adduser` tool from the packages, because we don't use it. This fixes [#44934](https://github.com/ClickHouse/ClickHouse/issues/44934). [#45011](https://github.com/ClickHouse/ClickHouse/pull/45011) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#46483](https://github.com/ClickHouse/ClickHouse/issues/46483): Get rid of unnecessary build for standalone clickhouse-keeper. [#46367](https://github.com/ClickHouse/ClickHouse/pull/46367) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#46507](https://github.com/ClickHouse/ClickHouse/issues/46507): Some time ago the ccache compression was changed to `zst`, but `gz` archives are downloaded by default. It fixes it by prioritizing zst archive. [#46490](https://github.com/ClickHouse/ClickHouse/pull/46490) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Backported in [#45903](https://github.com/ClickHouse/ClickHouse/issues/45903): Fixed bug with non-parsable default value for EPHEMERAL column in table metadata. [#44026](https://github.com/ClickHouse/ClickHouse/pull/44026) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Backported in [#46239](https://github.com/ClickHouse/ClickHouse/issues/46239): A couple of seg faults have been reported around `c-ares`. All of the recent stack traces observed fail on inserting into `std::unodered_set<>`. I believe I have found the root cause of this, it seems to be unprocessed queries. Prior to this PR, CH calls `poll` to wait on the file descriptors in the `c-ares` channel. According to the [poll docs](https://man7.org/linux/man-pages/man2/poll.2.html), a negative return value means an error has ocurred. Because of this, we would abort the execution and return failure. The problem is that `poll` will also return a negative value if a system interrupt occurs. A system interrupt does not mean the processing has failed or ended, but we would abort it anyways because we were checking for negative values. Once the execution is aborted, the whole stack is destroyed, which includes the `std::unordered_set` passed to the `void *` parameter of the c-ares callback. Once c-ares completed the request, the callback would be invoked and would access an invalid memory address causing a segfault. [#45629](https://github.com/ClickHouse/ClickHouse/pull/45629) ([Arthur Passos](https://github.com/arthurpassos)). +* Backported in [#46216](https://github.com/ClickHouse/ClickHouse/issues/46216): Fix reading of non existing nested columns with multiple level in compact parts. [#46045](https://github.com/ClickHouse/ClickHouse/pull/46045) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#46444](https://github.com/ClickHouse/ClickHouse/issues/46444): Fix possible `LOGICAL_ERROR` in asynchronous inserts with invalid data sent in format `VALUES`. [#46350](https://github.com/ClickHouse/ClickHouse/pull/46350) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#46676](https://github.com/ClickHouse/ClickHouse/issues/46676): Fix an invalid processing of constant `LowCardinality` argument in function `arrayMap`. This bug could lead to a segfault in release, and logical error `Bad cast` in debug build. [#46569](https://github.com/ClickHouse/ClickHouse/pull/46569) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Another attempt to fix automerge, or at least to have debug footprint [#45476](https://github.com/ClickHouse/ClickHouse/pull/45476) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add check for running workflows to merge_pr.py [#45803](https://github.com/ClickHouse/ClickHouse/pull/45803) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Get rid of progress timestamps in release publishing [#45818](https://github.com/ClickHouse/ClickHouse/pull/45818) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add necessary dependency for sanitizers [#45959](https://github.com/ClickHouse/ClickHouse/pull/45959) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add helping logging to auto-merge script [#46080](https://github.com/ClickHouse/ClickHouse/pull/46080) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix write buffer destruction order for vertical merge. [#46205](https://github.com/ClickHouse/ClickHouse/pull/46205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Get rid of legacy DocsReleaseChecks [#46665](https://github.com/ClickHouse/ClickHouse/pull/46665) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 3c0f5e7e72a..d4abbdebee1 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -5,6 +5,7 @@ v23.1.1.3077-stable 2023-01-25 v22.12.3.5-stable 2023-01-10 v22.12.2.25-stable 2023-01-06 v22.12.1.1752-stable 2022-12-15 +v22.11.6.44-stable 2023-02-23 v22.11.5.15-stable 2023-01-29 v22.11.4.3-stable 2023-01-10 v22.11.3.47-stable 2023-01-09 From 9fb24341e2b1d5622d793b4889bef04ef7b887ac Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Fri, 24 Feb 2023 12:12:45 +0100 Subject: [PATCH 253/253] Update settings.md --- docs/en/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 75ae6f3d2bc..17d03dfa4ec 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -467,7 +467,7 @@ Port for exchanging data between ClickHouse servers. The hostname that can be used by other servers to access this server. -If omitted, it is defined in the same way as the `hostname-f` command. +If omitted, it is defined in the same way as the `hostname -f` command. Useful for breaking away from a specific network interface.