From d4fe0fae8fb3b0b7688f6516b5a0189ad9f14c40 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 8 Sep 2022 00:30:44 +0000 Subject: [PATCH 01/98] Add 'FROM table SELECT column' syntax --- src/Parsers/ParserSelectQuery.cpp | 9 ++++++++- .../0_stateless/02417_from_select_syntax.reference | 2 ++ tests/queries/0_stateless/02417_from_select_syntax.sql | 2 ++ 3 files changed, 12 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02417_from_select_syntax.reference create mode 100644 tests/queries/0_stateless/02417_from_select_syntax.sql diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index 66428b144bf..936d7758f46 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -108,6 +108,13 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } } + /// FROM database.table or FROM table or FROM (subquery) or FROM tableFunction(...) + if (s_from.ignore(pos, expected)) + { + if (!ParserTablesInSelectQuery().parse(pos, tables, expected)) + return false; + } + /// SELECT [ALL/DISTINCT [ON (expr_list)]] [TOP N [WITH TIES]] expr_list { bool has_all = false; @@ -166,7 +173,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } /// FROM database.table or FROM table or FROM (subquery) or FROM tableFunction(...) - if (s_from.ignore(pos, expected)) + if (!tables && s_from.ignore(pos, expected)) { if (!ParserTablesInSelectQuery().parse(pos, tables, expected)) return false; diff --git a/tests/queries/0_stateless/02417_from_select_syntax.reference b/tests/queries/0_stateless/02417_from_select_syntax.reference new file mode 100644 index 00000000000..aa47d0d46d4 --- /dev/null +++ b/tests/queries/0_stateless/02417_from_select_syntax.reference @@ -0,0 +1,2 @@ +0 +0 diff --git a/tests/queries/0_stateless/02417_from_select_syntax.sql b/tests/queries/0_stateless/02417_from_select_syntax.sql new file mode 100644 index 00000000000..ab3efa4292e --- /dev/null +++ b/tests/queries/0_stateless/02417_from_select_syntax.sql @@ -0,0 +1,2 @@ +FROM numbers(1) SELECT number; +WITH 1 as n FROM numbers(1) SELECT number * n; From fc6ec8474ec31e8443da1dad45ecc748ea844624 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 8 Sep 2022 15:20:28 +0000 Subject: [PATCH 02/98] Better --- src/Parsers/ParserSelectQuery.cpp | 2 +- src/Parsers/ParserTablesInSelectQuery.cpp | 14 +++++++------- src/Parsers/ParserTablesInSelectQuery.h | 18 +++++++++++++++++- .../02417_from_select_syntax.reference | 2 ++ .../0_stateless/02417_from_select_syntax.sql | 2 ++ 5 files changed, 29 insertions(+), 9 deletions(-) diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index 936d7758f46..049c4ab4b44 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -111,7 +111,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) /// FROM database.table or FROM table or FROM (subquery) or FROM tableFunction(...) if (s_from.ignore(pos, expected)) { - if (!ParserTablesInSelectQuery().parse(pos, tables, expected)) + if (!ParserTablesInSelectQuery(false).parse(pos, tables, expected)) return false; } diff --git a/src/Parsers/ParserTablesInSelectQuery.cpp b/src/Parsers/ParserTablesInSelectQuery.cpp index 8137093b990..67da9d77bee 100644 --- a/src/Parsers/ParserTablesInSelectQuery.cpp +++ b/src/Parsers/ParserTablesInSelectQuery.cpp @@ -21,9 +21,9 @@ bool ParserTableExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expec { auto res = std::make_shared(); - if (!ParserWithOptionalAlias(std::make_unique(), true).parse(pos, res->subquery, expected) - && !ParserWithOptionalAlias(std::make_unique(true, true), true).parse(pos, res->table_function, expected) - && !ParserWithOptionalAlias(std::make_unique(true, true), true) + if (!ParserWithOptionalAlias(std::make_unique(), allow_alias_without_as_keyword).parse(pos, res->subquery, expected) + && !ParserWithOptionalAlias(std::make_unique(true, true), allow_alias_without_as_keyword).parse(pos, res->table_function, expected) + && !ParserWithOptionalAlias(std::make_unique(true, true), allow_alias_without_as_keyword) .parse(pos, res->database_and_table_name, expected)) return false; @@ -126,7 +126,7 @@ bool ParserTablesInSelectQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expec if (is_first) { - if (!ParserTableExpression().parse(pos, res->table_expression, expected)) + if (!ParserTableExpression(allow_alias_without_as_keyword).parse(pos, res->table_expression, expected)) return false; } else if (ParserArrayJoin().parse(pos, res->array_join, expected)) @@ -200,7 +200,7 @@ bool ParserTablesInSelectQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expec return false; } - if (!ParserTableExpression().parse(pos, res->table_expression, expected)) + if (!ParserTableExpression(allow_alias_without_as_keyword).parse(pos, res->table_expression, expected)) return false; if (table_join->kind != JoinKind::Comma @@ -261,12 +261,12 @@ bool ParserTablesInSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e ASTPtr child; - if (ParserTablesInSelectQueryElement(true).parse(pos, child, expected)) + if (ParserTablesInSelectQueryElement(true, allow_alias_without_as_keyword).parse(pos, child, expected)) res->children.emplace_back(child); else return false; - while (ParserTablesInSelectQueryElement(false).parse(pos, child, expected)) + while (ParserTablesInSelectQueryElement(false, allow_alias_without_as_keyword).parse(pos, child, expected)) res->children.emplace_back(child); node = res; diff --git a/src/Parsers/ParserTablesInSelectQuery.h b/src/Parsers/ParserTablesInSelectQuery.h index 772f1992f4d..633860920b4 100644 --- a/src/Parsers/ParserTablesInSelectQuery.h +++ b/src/Parsers/ParserTablesInSelectQuery.h @@ -12,16 +12,24 @@ struct ASTTableJoin; */ class ParserTablesInSelectQuery : public IParserBase { +public: + ParserTablesInSelectQuery(bool allow_alias_without_as_keyword_ = true) + : allow_alias_without_as_keyword(allow_alias_without_as_keyword_) {} + protected: const char * getName() const override { return "table, table function, subquery or list of joined tables"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + +private: + bool allow_alias_without_as_keyword; }; class ParserTablesInSelectQueryElement : public IParserBase { public: - explicit ParserTablesInSelectQueryElement(bool is_first_) : is_first(is_first_) {} + explicit ParserTablesInSelectQueryElement(bool is_first_, bool allow_alias_without_as_keyword_ = true) + : is_first(is_first_), allow_alias_without_as_keyword(allow_alias_without_as_keyword_) {} protected: const char * getName() const override { return "table, table function, subquery or list of joined tables"; } @@ -29,6 +37,7 @@ protected: private: bool is_first; + bool allow_alias_without_as_keyword; static void parseJoinStrictness(Pos & pos, ASTTableJoin & table_join); }; @@ -36,9 +45,16 @@ private: class ParserTableExpression : public IParserBase { +public: + explicit ParserTableExpression(bool allow_alias_without_as_keyword_ = true) + : allow_alias_without_as_keyword(allow_alias_without_as_keyword_) {} + protected: const char * getName() const override { return "table or subquery or table function"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + +private: + bool allow_alias_without_as_keyword; }; diff --git a/tests/queries/0_stateless/02417_from_select_syntax.reference b/tests/queries/0_stateless/02417_from_select_syntax.reference index aa47d0d46d4..44e0be8e356 100644 --- a/tests/queries/0_stateless/02417_from_select_syntax.reference +++ b/tests/queries/0_stateless/02417_from_select_syntax.reference @@ -1,2 +1,4 @@ 0 0 +0 +0 diff --git a/tests/queries/0_stateless/02417_from_select_syntax.sql b/tests/queries/0_stateless/02417_from_select_syntax.sql index ab3efa4292e..ce6cb3a14da 100644 --- a/tests/queries/0_stateless/02417_from_select_syntax.sql +++ b/tests/queries/0_stateless/02417_from_select_syntax.sql @@ -1,2 +1,4 @@ FROM numbers(1) SELECT number; WITH 1 as n FROM numbers(1) SELECT number * n; +FROM (FROM numbers(1) SELECT *) SELECT number; +FROM (FROM numbers(1) SELECT *) AS select SELECT number; From 2ce0f1b678b90f7015b1f59962b04bc16abfb301 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Sat, 17 Sep 2022 01:08:36 +0200 Subject: [PATCH 03/98] Update src/Parsers/ParserTablesInSelectQuery.h Co-authored-by: Sergei Trifonov --- src/Parsers/ParserTablesInSelectQuery.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ParserTablesInSelectQuery.h b/src/Parsers/ParserTablesInSelectQuery.h index 633860920b4..428b1482663 100644 --- a/src/Parsers/ParserTablesInSelectQuery.h +++ b/src/Parsers/ParserTablesInSelectQuery.h @@ -13,7 +13,7 @@ struct ASTTableJoin; class ParserTablesInSelectQuery : public IParserBase { public: - ParserTablesInSelectQuery(bool allow_alias_without_as_keyword_ = true) + explicit ParserTablesInSelectQuery(bool allow_alias_without_as_keyword_ = true) : allow_alias_without_as_keyword(allow_alias_without_as_keyword_) {} protected: From d08dce5ec7b7bd9810998f293ca26fdf525e1f73 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 14 Nov 2022 11:36:00 +0000 Subject: [PATCH 04/98] Test parallel append --- src/Coordination/Changelog.cpp | 68 +++++++++++++++++++++---- src/Coordination/Changelog.h | 29 ++++++++++- src/Coordination/KeeperLogStore.cpp | 12 +++-- src/Coordination/KeeperLogStore.h | 4 +- src/Coordination/KeeperServer.cpp | 5 +- src/Coordination/KeeperServer.h | 1 + src/Coordination/KeeperStateManager.cpp | 6 ++- src/Coordination/KeeperStateManager.h | 3 +- 8 files changed, 108 insertions(+), 20 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index cc5f292eae2..489d28d7270 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -273,13 +273,16 @@ Changelog::Changelog( uint64_t rotate_interval_, bool force_sync_, Poco::Logger * log_, - bool compress_logs_) + bool compress_logs_, + nuraft::ptr * raft_server_) : changelogs_dir(changelogs_dir_) , changelogs_detached_dir(changelogs_dir / "detached") , rotate_interval(rotate_interval_) , force_sync(force_sync_) , log(log_) , compress_logs(compress_logs_) + , write_operations(std::numeric_limits::max()) + , raft_server(raft_server_) { /// Load all files in changelog directory namespace fs = std::filesystem; @@ -299,6 +302,8 @@ Changelog::Changelog( LOG_WARNING(log, "No logs exists in {}. It's Ok if it's the first run of clickhouse-keeper.", changelogs_dir.generic_string()); clean_log_thread = ThreadFromGlobalPool([this] { cleanLogThread(); }); + + write_thread = ThreadFromGlobalPool([this] { writeThread(); }); } void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uint64_t logs_to_keep) @@ -503,7 +508,8 @@ void Changelog::removeAllLogs() void Changelog::rotate(uint64_t new_start_log_index) { /// Flush previous log - flush(); + if (current_writer) + current_writer->flush(force_sync); /// Start new one ChangelogFileDescription new_description; @@ -540,6 +546,44 @@ ChangelogRecord Changelog::buildRecord(uint64_t index, const LogEntryPtr & log_e return record; } +void Changelog::writeThread() +{ + size_t last_appended = 0; + while (true) + { + WriteOperation write_operation; + if (!write_operations.pop(write_operation)) + break; + + std::visit([&, this](const WriteOperationType & operation) -> void + { + if constexpr (std::same_as) + { + const auto & current_changelog_description = existing_changelogs[current_writer->getStartIndex()]; + const bool log_is_complete = operation.index - current_writer->getStartIndex() == current_changelog_description.expectedEntriesCountInLog(); + + if (log_is_complete) + rotate(operation.index); + + current_writer->appendRecord(buildRecord(operation.index, operation.log_entry)); + + last_appended = operation.index; + } + else + { + if (current_writer) + current_writer->flush(force_sync); + + last_durable_idx = last_appended; + + if (*raft_server) + (*raft_server)->notify_log_append_completion(true); + } + }, write_operation); + } +} + + void Changelog::appendEntry(uint64_t index, const LogEntryPtr & log_entry) { if (!current_writer) @@ -548,15 +592,11 @@ void Changelog::appendEntry(uint64_t index, const LogEntryPtr & log_entry) if (logs.empty()) min_log_id = index; - const auto & current_changelog_description = existing_changelogs[current_writer->getStartIndex()]; - const bool log_is_complete = index - current_writer->getStartIndex() == current_changelog_description.expectedEntriesCountInLog(); - - if (log_is_complete) - rotate(index); - - current_writer->appendRecord(buildRecord(index, log_entry)); logs[index] = log_entry; max_log_id = index; + + if (!write_operations.push(AppendLog{index, log_entry})) + std::abort(); } void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry) @@ -747,8 +787,14 @@ void Changelog::applyEntriesFromBuffer(uint64_t index, nuraft::buffer & buffer) void Changelog::flush() { - if (current_writer) - current_writer->flush(force_sync); + flushAsync(); + last_durable_idx.wait(max_log_id); +} + +void Changelog::flushAsync() +{ + if (!write_operations.push(Flush{})) + std::abort(); } void Changelog::shutdown() diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 9f90f72d9f3..e6b57ba8ca3 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -75,7 +76,7 @@ class Changelog public: Changelog(const std::string & changelogs_dir_, uint64_t rotate_interval_, - bool force_sync_, Poco::Logger * log_, bool compress_logs_ = true); + bool force_sync_, Poco::Logger * log_, bool compress_logs_ = true, nuraft::ptr * raft_server_ = nullptr); /// Read changelog from files on changelogs_dir_ skipping all entries before from_log_index /// Truncate broken entries, remove files after broken entries. @@ -121,6 +122,8 @@ public: /// Fsync latest log to disk and flush buffer void flush(); + void flushAsync(); + void shutdown(); uint64_t size() const @@ -128,6 +131,11 @@ public: return logs.size(); } + uint64_t lastDurableIndex() const + { + return last_durable_idx; + } + /// Fsync log to disk ~Changelog(); @@ -175,6 +183,25 @@ private: /// 128 is enough, even if log is not removed, it's not a problem ConcurrentBoundedQueue log_files_to_delete_queue{128}; ThreadFromGlobalPool clean_log_thread; + + struct AppendLog + { + uint64_t index; + nuraft::ptr log_entry; + }; + + struct Flush + {}; + + using WriteOperation = std::variant; + + void writeThread(); + + ThreadFromGlobalPool write_thread; + ConcurrentBoundedQueue write_operations; + + std::atomic last_durable_idx{0}; + nuraft::ptr * raft_server{nullptr}; }; } diff --git a/src/Coordination/KeeperLogStore.cpp b/src/Coordination/KeeperLogStore.cpp index 3787f30626b..905cd4dfe15 100644 --- a/src/Coordination/KeeperLogStore.cpp +++ b/src/Coordination/KeeperLogStore.cpp @@ -4,9 +4,9 @@ namespace DB { -KeeperLogStore::KeeperLogStore(const std::string & changelogs_path, uint64_t rotate_interval_, bool force_sync_, bool compress_logs_) +KeeperLogStore::KeeperLogStore(const std::string & changelogs_path, uint64_t rotate_interval_, bool force_sync_, bool compress_logs_, nuraft::ptr * raft_server) : log(&Poco::Logger::get("KeeperLogStore")) - , changelog(changelogs_path, rotate_interval_, force_sync_, log, compress_logs_) + , changelog(changelogs_path, rotate_interval_, force_sync_, log, compress_logs_, raft_server) { if (force_sync_) LOG_INFO(log, "force_sync enabled"); @@ -109,7 +109,7 @@ uint64_t KeeperLogStore::size() const void KeeperLogStore::end_of_append_batch(uint64_t /*start_index*/, uint64_t /*count*/) { std::lock_guard lock(changelog_lock); - changelog.flush(); + changelog.flushAsync(); } nuraft::ptr KeeperLogStore::getLatestConfigChange() const @@ -132,4 +132,10 @@ bool KeeperLogStore::flushChangelogAndShutdown() return true; } +uint64_t KeeperLogStore::last_durable_index() +{ + std::lock_guard lock(changelog_lock); + return changelog.lastDurableIndex(); +} + } diff --git a/src/Coordination/KeeperLogStore.h b/src/Coordination/KeeperLogStore.h index e1c66599e0a..6d46989900c 100644 --- a/src/Coordination/KeeperLogStore.h +++ b/src/Coordination/KeeperLogStore.h @@ -14,7 +14,7 @@ namespace DB class KeeperLogStore : public nuraft::log_store { public: - KeeperLogStore(const std::string & changelogs_path, uint64_t rotate_interval_, bool force_sync_, bool compress_logs_); + KeeperLogStore(const std::string & changelogs_path, uint64_t rotate_interval_, bool force_sync_, bool compress_logs_, nuraft::ptr * raft_server = nullptr); /// Read log storage from filesystem starting from last_commited_log_index void init(uint64_t last_commited_log_index, uint64_t logs_to_keep); @@ -62,6 +62,8 @@ public: /// Current log storage size uint64_t size() const; + uint64_t last_durable_index() override; + /// Flush batch of appended entries void end_of_append_batch(uint64_t start_index, uint64_t count) override; diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 487d0dc4cc3..9b962be7738 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -136,7 +136,8 @@ KeeperServer::KeeperServer( configuration_and_settings_->log_storage_path, configuration_and_settings_->state_file_path, config, - coordination_settings); + coordination_settings, + &raft_instance_raw); } /** @@ -266,6 +267,7 @@ void KeeperServer::forceRecovery() void KeeperServer::launchRaftServer(const Poco::Util::AbstractConfiguration & config, bool enable_ipv6) { nuraft::raft_params params; + params.parallel_log_appending_ = true; params.heart_beat_interval_ = getValueOrMaxInt32AndLogWarning(coordination_settings->heart_beat_interval_ms.totalMilliseconds(), "heart_beat_interval_ms", log); params.election_timeout_lower_bound_ = getValueOrMaxInt32AndLogWarning( @@ -348,6 +350,7 @@ void KeeperServer::launchRaftServer(const Poco::Util::AbstractConfiguration & co = new nuraft::context(casted_state_manager, casted_state_machine, asio_listeners, logger, rpc_cli_factory, scheduler, params); raft_instance = nuraft::cs_new(ctx, init_options); + raft_instance_raw = raft_instance; if (!raft_instance) throw Exception(ErrorCodes::RAFT_ERROR, "Cannot allocate RAFT instance"); diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 192c8f470b1..39ad27f1b6b 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -29,6 +29,7 @@ private: struct KeeperRaftServer; nuraft::ptr raft_instance; + nuraft::ptr raft_instance_raw; nuraft::ptr asio_service; std::vector> asio_listeners; // because some actions can be applied diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 9b6aab5533e..829d13bc876 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -231,7 +231,8 @@ KeeperStateManager::KeeperStateManager( const std::string & log_storage_path, const std::string & state_file_path, const Poco::Util::AbstractConfiguration & config, - const CoordinationSettingsPtr & coordination_settings) + const CoordinationSettingsPtr & coordination_settings, + nuraft::ptr * raft_server) : my_server_id(my_server_id_) , secure(config.getBool(config_prefix_ + ".raft_configuration.secure", false)) , config_prefix(config_prefix_) @@ -240,7 +241,8 @@ KeeperStateManager::KeeperStateManager( log_storage_path, coordination_settings->rotate_log_storage_interval, coordination_settings->force_sync, - coordination_settings->compress_logs)) + coordination_settings->compress_logs, + raft_server)) , server_state_path(state_file_path) , logger(&Poco::Logger::get("KeeperStateManager")) { diff --git a/src/Coordination/KeeperStateManager.h b/src/Coordination/KeeperStateManager.h index 5d210f8c0ea..b1bb9d65bd6 100644 --- a/src/Coordination/KeeperStateManager.h +++ b/src/Coordination/KeeperStateManager.h @@ -42,7 +42,8 @@ public: const std::string & log_storage_path, const std::string & state_file_path, const Poco::Util::AbstractConfiguration & config, - const CoordinationSettingsPtr & coordination_settings); + const CoordinationSettingsPtr & coordination_settings, + nuraft::ptr * raft_server); /// Constructor for tests KeeperStateManager( From 84aea78d0f269610865242791bbc7ea5fb59b3af Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 16 Nov 2022 13:11:05 +0800 Subject: [PATCH 05/98] fix array first / last or null with nullable column --- src/Functions/array/arrayFirstLast.cpp | 18 +++++++++++++++--- .../02241_array_first_last_or_null.reference | 6 ++++++ .../02241_array_first_last_or_null.sql | 8 ++++++++ 3 files changed, 29 insertions(+), 3 deletions(-) diff --git a/src/Functions/array/arrayFirstLast.cpp b/src/Functions/array/arrayFirstLast.cpp index 8160234a6b0..9b193756f4a 100644 --- a/src/Functions/array/arrayFirstLast.cpp +++ b/src/Functions/array/arrayFirstLast.cpp @@ -3,6 +3,8 @@ #include #include +#include "Columns/IColumn.h" +#include "Core/Types.h" #include "FunctionArrayMapped.h" @@ -43,6 +45,16 @@ struct ArrayFirstLastImpl return array_element; } + static ColumnPtr createNullableColumn(MutableColumnPtr && column, ColumnUInt8::MutablePtr && null_map) + { + if (auto * nullable_column = typeid_cast(column.get())) + { + nullable_column->applyNullMap(*null_map); + return std::move(column); + } + return ColumnNullable::create(std::move(column), std::move(null_map)); + } + static ColumnPtr execute(const ColumnArray & array, ColumnPtr mapped) { const auto * column_filter = typeid_cast(&*mapped); @@ -94,7 +106,7 @@ struct ArrayFirstLastImpl } if constexpr (element_not_exists_strategy == ArrayFirstLastElementNotExistsStrategy::Null) - return ColumnNullable::create(std::move(out), std::move(col_null_map_to)); + return createNullableColumn(std::move(out), std::move(col_null_map_to)); return out; } @@ -106,7 +118,7 @@ struct ArrayFirstLastImpl if constexpr (element_not_exists_strategy == ArrayFirstLastElementNotExistsStrategy::Null) { auto col_null_map_to = ColumnUInt8::create(out->size(), true); - return ColumnNullable::create(std::move(out), std::move(col_null_map_to)); + return createNullableColumn(std::move(out), std::move(col_null_map_to)); } return out; @@ -172,7 +184,7 @@ struct ArrayFirstLastImpl } if constexpr (element_not_exists_strategy == ArrayFirstLastElementNotExistsStrategy::Null) - return ColumnNullable::create(std::move(out), std::move(col_null_map_to)); + return createNullableColumn(std::move(out), std::move(col_null_map_to)); return out; } diff --git a/tests/queries/0_stateless/02241_array_first_last_or_null.reference b/tests/queries/0_stateless/02241_array_first_last_or_null.reference index 2906b04ecd0..fc4a5ff8af5 100644 --- a/tests/queries/0_stateless/02241_array_first_last_or_null.reference +++ b/tests/queries/0_stateless/02241_array_first_last_or_null.reference @@ -7,6 +7,9 @@ ArrayFirst non constant predicate \N 2 2 +ArrayFirst with Null +2 +\N ArrayLast constant predicate \N \N @@ -16,3 +19,6 @@ ArrayLast non constant predicate \N 3 3 +ArrayLast with Null +2 +\N diff --git a/tests/queries/0_stateless/02241_array_first_last_or_null.sql b/tests/queries/0_stateless/02241_array_first_last_or_null.sql index 3230e4d483a..aa8f0cdbf92 100644 --- a/tests/queries/0_stateless/02241_array_first_last_or_null.sql +++ b/tests/queries/0_stateless/02241_array_first_last_or_null.sql @@ -9,6 +9,10 @@ SELECT arrayFirstOrNull(x -> x >= 2, emptyArrayUInt8()); SELECT arrayFirstOrNull(x -> x >= 2, [1, 2, 3]); SELECT arrayFirstOrNull(x -> x >= 2, materialize([1, 2, 3])); +SELECT 'ArrayFirst with Null'; +SELECT arrayFirstOrNull((x,f) -> f, [1,2,3,NULL], [0,1,0,0]); +SELECT arrayFirstOrNull((x,f) -> f, [1,2,3,NULL], [0,0,0,1]); + SELECT 'ArrayLast constant predicate'; SELECT arrayLastOrNull(x -> 1, emptyArrayUInt8()); SELECT arrayLastOrNull(x -> 0, emptyArrayUInt8()); @@ -19,3 +23,7 @@ SELECT 'ArrayLast non constant predicate'; SELECT arrayLastOrNull(x -> x >= 2, emptyArrayUInt8()); SELECT arrayLastOrNull(x -> x >= 2, [1, 2, 3]); SELECT arrayLastOrNull(x -> x >= 2, materialize([1, 2, 3])); + +SELECT 'ArrayLast with Null'; +SELECT arrayLastOrNull((x,f) -> f, [1,2,3,NULL], [0,1,0,0]); +SELECT arrayLastOrNull((x,f) -> f, [1,2,3,NULL], [0,1,0,1]); \ No newline at end of file From 318064cccee9602362b17b5c02c3c57248b51d75 Mon Sep 17 00:00:00 2001 From: Raevsky Rudolf Date: Thu, 17 Nov 2022 00:25:57 +0100 Subject: [PATCH 06/98] Fixed: Typo --- src/Functions/translate.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/translate.cpp b/src/Functions/translate.cpp index b3f1d5ae460..7471fdacbb5 100644 --- a/src/Functions/translate.cpp +++ b/src/Functions/translate.cpp @@ -27,14 +27,14 @@ struct TranslateImpl const std::string & map_to) { if (map_from.size() != map_to.size()) - throw Exception("Second and trird arguments must be the same length", ErrorCodes::BAD_ARGUMENTS); + throw Exception("Second and third arguments must be the same length", ErrorCodes::BAD_ARGUMENTS); std::iota(map.begin(), map.end(), 0); for (size_t i = 0; i < map_from.size(); ++i) { if (!isASCII(map_from[i]) || !isASCII(map_to[i])) - throw Exception("Second and trird arguments must be ASCII strings", ErrorCodes::BAD_ARGUMENTS); + throw Exception("Second and third arguments must be ASCII strings", ErrorCodes::BAD_ARGUMENTS); map[map_from[i]] = map_to[i]; } @@ -125,7 +125,7 @@ struct TranslateUTF8Impl auto map_to_size = UTF8::countCodePoints(reinterpret_cast(map_to.data()), map_to.size()); if (map_from_size != map_to_size) - throw Exception("Second and trird arguments must be the same length", ErrorCodes::BAD_ARGUMENTS); + throw Exception("Second and third arguments must be the same length", ErrorCodes::BAD_ARGUMENTS); std::iota(map_ascii.begin(), map_ascii.end(), 0); From fa70e335dbd4dcffd60bb9d433b4771c53d6ceb6 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 21 Nov 2022 10:53:54 +0800 Subject: [PATCH 07/98] minor fix --- src/Functions/array/arrayFirstLast.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Functions/array/arrayFirstLast.cpp b/src/Functions/array/arrayFirstLast.cpp index 9b193756f4a..fa72ecba161 100644 --- a/src/Functions/array/arrayFirstLast.cpp +++ b/src/Functions/array/arrayFirstLast.cpp @@ -3,8 +3,6 @@ #include #include -#include "Columns/IColumn.h" -#include "Core/Types.h" #include "FunctionArrayMapped.h" @@ -47,7 +45,7 @@ struct ArrayFirstLastImpl static ColumnPtr createNullableColumn(MutableColumnPtr && column, ColumnUInt8::MutablePtr && null_map) { - if (auto * nullable_column = typeid_cast(column.get())) + if (auto * nullable_column = typeid_cast(column.get())) { nullable_column->applyNullMap(*null_map); return std::move(column); From c9472d67b636c4503e1b6fc99db12bbaab20193d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 21 Nov 2022 12:22:36 +0000 Subject: [PATCH 08/98] Refactor changelog --- src/Coordination/Changelog.cpp | 18 +++++++++++++----- src/Coordination/Changelog.h | 7 ++++--- src/Coordination/KeeperLogStore.cpp | 10 ++++++++-- src/Coordination/KeeperLogStore.h | 4 +++- src/Coordination/KeeperServer.cpp | 6 +++--- src/Coordination/KeeperServer.h | 1 - src/Coordination/KeeperStateManager.cpp | 6 ++---- src/Coordination/KeeperStateManager.h | 3 +-- 8 files changed, 34 insertions(+), 21 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 489d28d7270..3d760792c32 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -273,8 +273,7 @@ Changelog::Changelog( uint64_t rotate_interval_, bool force_sync_, Poco::Logger * log_, - bool compress_logs_, - nuraft::ptr * raft_server_) + bool compress_logs_) : changelogs_dir(changelogs_dir_) , changelogs_detached_dir(changelogs_dir / "detached") , rotate_interval(rotate_interval_) @@ -282,7 +281,6 @@ Changelog::Changelog( , log(log_) , compress_logs(compress_logs_) , write_operations(std::numeric_limits::max()) - , raft_server(raft_server_) { /// Load all files in changelog directory namespace fs = std::filesystem; @@ -548,6 +546,9 @@ ChangelogRecord Changelog::buildRecord(uint64_t index, const LogEntryPtr & log_e void Changelog::writeThread() { + // we don't protect current_writer because rotate at the same time can be called from compact only + // when the node is applying snapshot from leader + // in that case, no entry write should happen, i.e. this thread will be inactive size_t last_appended = 0; while (true) { @@ -576,8 +577,9 @@ void Changelog::writeThread() last_durable_idx = last_appended; - if (*raft_server) - (*raft_server)->notify_log_append_completion(true); + // we shouldn't start the raft_server before sending it here + assert(raft_server); + raft_server->notify_log_append_completion(true); } }, write_operation); } @@ -835,4 +837,10 @@ void Changelog::cleanLogThread() } } +void Changelog::setRaftServer(nuraft::ptr raft_server_) +{ + assert(raft_server_); + raft_server = std::move(raft_server_); +} + } diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index e6b57ba8ca3..9f693382c75 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -76,7 +76,7 @@ class Changelog public: Changelog(const std::string & changelogs_dir_, uint64_t rotate_interval_, - bool force_sync_, Poco::Logger * log_, bool compress_logs_ = true, nuraft::ptr * raft_server_ = nullptr); + bool force_sync_, Poco::Logger * log_, bool compress_logs_ = true); /// Read changelog from files on changelogs_dir_ skipping all entries before from_log_index /// Truncate broken entries, remove files after broken entries. @@ -136,6 +136,8 @@ public: return last_durable_idx; } + void setRaftServer(nuraft::ptr raft_server_); + /// Fsync log to disk ~Changelog(); @@ -170,7 +172,6 @@ private: Poco::Logger * log; bool compress_logs; - /// Current writer for changelog file std::unique_ptr current_writer; /// Mapping log_id -> log_entry @@ -201,7 +202,7 @@ private: ConcurrentBoundedQueue write_operations; std::atomic last_durable_idx{0}; - nuraft::ptr * raft_server{nullptr}; + nuraft::ptr raft_server{nullptr}; }; } diff --git a/src/Coordination/KeeperLogStore.cpp b/src/Coordination/KeeperLogStore.cpp index 905cd4dfe15..448f1f8532e 100644 --- a/src/Coordination/KeeperLogStore.cpp +++ b/src/Coordination/KeeperLogStore.cpp @@ -4,9 +4,9 @@ namespace DB { -KeeperLogStore::KeeperLogStore(const std::string & changelogs_path, uint64_t rotate_interval_, bool force_sync_, bool compress_logs_, nuraft::ptr * raft_server) +KeeperLogStore::KeeperLogStore(const std::string & changelogs_path, uint64_t rotate_interval_, bool force_sync_, bool compress_logs_) : log(&Poco::Logger::get("KeeperLogStore")) - , changelog(changelogs_path, rotate_interval_, force_sync_, log, compress_logs_, raft_server) + , changelog(changelogs_path, rotate_interval_, force_sync_, log, compress_logs_) { if (force_sync_) LOG_INFO(log, "force_sync enabled"); @@ -138,4 +138,10 @@ uint64_t KeeperLogStore::last_durable_index() return changelog.lastDurableIndex(); } +void KeeperLogStore::setRaftServer(nuraft::ptr raft_server) +{ + std::lock_guard lock(changelog_lock); + return changelog.setRaftServer(raft_server); +} + } diff --git a/src/Coordination/KeeperLogStore.h b/src/Coordination/KeeperLogStore.h index 6d46989900c..2addbe61c01 100644 --- a/src/Coordination/KeeperLogStore.h +++ b/src/Coordination/KeeperLogStore.h @@ -14,7 +14,7 @@ namespace DB class KeeperLogStore : public nuraft::log_store { public: - KeeperLogStore(const std::string & changelogs_path, uint64_t rotate_interval_, bool force_sync_, bool compress_logs_, nuraft::ptr * raft_server = nullptr); + KeeperLogStore(const std::string & changelogs_path, uint64_t rotate_interval_, bool force_sync_, bool compress_logs_); /// Read log storage from filesystem starting from last_commited_log_index void init(uint64_t last_commited_log_index, uint64_t logs_to_keep); @@ -70,6 +70,8 @@ public: /// Get entry with latest config in logstore nuraft::ptr getLatestConfigChange() const; + void setRaftServer(nuraft::ptr raft_server); + private: mutable std::mutex changelog_lock; Poco::Logger * log; diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 9b962be7738..6151aa47ef7 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -136,8 +136,7 @@ KeeperServer::KeeperServer( configuration_and_settings_->log_storage_path, configuration_and_settings_->state_file_path, config, - coordination_settings, - &raft_instance_raw); + coordination_settings); } /** @@ -350,11 +349,12 @@ void KeeperServer::launchRaftServer(const Poco::Util::AbstractConfiguration & co = new nuraft::context(casted_state_manager, casted_state_machine, asio_listeners, logger, rpc_cli_factory, scheduler, params); raft_instance = nuraft::cs_new(ctx, init_options); - raft_instance_raw = raft_instance; if (!raft_instance) throw Exception(ErrorCodes::RAFT_ERROR, "Cannot allocate RAFT instance"); + state_manager->getLogStore()->setRaftServer(raft_instance); + raft_instance->start_server(init_options.skip_initial_election_timeout_); nuraft::ptr casted_raft_server = raft_instance; diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 39ad27f1b6b..192c8f470b1 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -29,7 +29,6 @@ private: struct KeeperRaftServer; nuraft::ptr raft_instance; - nuraft::ptr raft_instance_raw; nuraft::ptr asio_service; std::vector> asio_listeners; // because some actions can be applied diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 43f43fca400..9a3b423d4ac 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -231,8 +231,7 @@ KeeperStateManager::KeeperStateManager( const std::string & log_storage_path, const std::string & state_file_path, const Poco::Util::AbstractConfiguration & config, - const CoordinationSettingsPtr & coordination_settings, - nuraft::ptr * raft_server) + const CoordinationSettingsPtr & coordination_settings) : my_server_id(my_server_id_) , secure(config.getBool(config_prefix_ + ".raft_configuration.secure", false)) , config_prefix(config_prefix_) @@ -241,8 +240,7 @@ KeeperStateManager::KeeperStateManager( log_storage_path, coordination_settings->rotate_log_storage_interval, coordination_settings->force_sync, - coordination_settings->compress_logs, - raft_server)) + coordination_settings->compress_logs)) , server_state_path(state_file_path) , logger(&Poco::Logger::get("KeeperStateManager")) { diff --git a/src/Coordination/KeeperStateManager.h b/src/Coordination/KeeperStateManager.h index b1bb9d65bd6..5d210f8c0ea 100644 --- a/src/Coordination/KeeperStateManager.h +++ b/src/Coordination/KeeperStateManager.h @@ -42,8 +42,7 @@ public: const std::string & log_storage_path, const std::string & state_file_path, const Poco::Util::AbstractConfiguration & config, - const CoordinationSettingsPtr & coordination_settings, - nuraft::ptr * raft_server); + const CoordinationSettingsPtr & coordination_settings); /// Constructor for tests KeeperStateManager( From aeb0d39b501f7b0926b4f48ca9476ccce379d80a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 21 Nov 2022 15:01:18 +0000 Subject: [PATCH 09/98] Fix shutdown --- src/Coordination/Changelog.cpp | 42 +++++++++++++++++++------------ src/Coordination/Changelog.h | 14 ++++++++--- src/Coordination/KeeperServer.cpp | 2 +- 3 files changed, 38 insertions(+), 20 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 3d760792c32..49eb644719d 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -549,13 +549,9 @@ void Changelog::writeThread() // we don't protect current_writer because rotate at the same time can be called from compact only // when the node is applying snapshot from leader // in that case, no entry write should happen, i.e. this thread will be inactive - size_t last_appended = 0; - while (true) + WriteOperation write_operation; + while (write_operations.pop(write_operation)) { - WriteOperation write_operation; - if (!write_operations.pop(write_operation)) - break; - std::visit([&, this](const WriteOperationType & operation) -> void { if constexpr (std::same_as) @@ -567,15 +563,18 @@ void Changelog::writeThread() rotate(operation.index); current_writer->appendRecord(buildRecord(operation.index, operation.log_entry)); - - last_appended = operation.index; } else { if (current_writer) current_writer->flush(force_sync); - last_durable_idx = last_appended; + { + std::lock_guard lock{durable_idx_mutex}; + last_durable_idx = operation.index; + } + + durable_idx_cv.notify_all(); // we shouldn't start the raft_server before sending it here assert(raft_server); @@ -597,8 +596,8 @@ void Changelog::appendEntry(uint64_t index, const LogEntryPtr & log_entry) logs[index] = log_entry; max_log_id = index; - if (!write_operations.push(AppendLog{index, log_entry})) - std::abort(); + if (!write_operations.tryPush(AppendLog{index, log_entry})) + LOG_WARNING(log, "Changelog is shut down"); } void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry) @@ -789,14 +788,19 @@ void Changelog::applyEntriesFromBuffer(uint64_t index, nuraft::buffer & buffer) void Changelog::flush() { - flushAsync(); - last_durable_idx.wait(max_log_id); + if (flushAsync()) + { + std::unique_lock lock{durable_idx_mutex}; + durable_idx_cv.wait(lock, [&] { return last_durable_idx == max_log_id; }); + } } -void Changelog::flushAsync() +bool Changelog::flushAsync() { - if (!write_operations.push(Flush{})) - std::abort(); + bool pushed = write_operations.push(Flush{max_log_id}); + if (!pushed) + LOG_WARNING(log, "Changelog is shut down"); + return pushed; } void Changelog::shutdown() @@ -806,6 +810,12 @@ void Changelog::shutdown() if (clean_log_thread.joinable()) clean_log_thread.join(); + + if (!write_operations.isFinished()) + write_operations.finish(); + + if (write_thread.joinable()) + write_thread.join(); } Changelog::~Changelog() diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 9f693382c75..01ce7cdd145 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -122,7 +123,7 @@ public: /// Fsync latest log to disk and flush buffer void flush(); - void flushAsync(); + bool flushAsync(); void shutdown(); @@ -133,6 +134,7 @@ public: uint64_t lastDurableIndex() const { + std::lock_guard lock{durable_idx_mutex}; return last_durable_idx; } @@ -192,7 +194,9 @@ private: }; struct Flush - {}; + { + uint64_t index; + }; using WriteOperation = std::variant; @@ -201,7 +205,11 @@ private: ThreadFromGlobalPool write_thread; ConcurrentBoundedQueue write_operations; - std::atomic last_durable_idx{0}; + + mutable std::mutex durable_idx_mutex; + std::condition_variable durable_idx_cv; + uint64_t last_durable_idx{0}; + nuraft::ptr raft_server{nullptr}; }; diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 6151aa47ef7..9ca3f219928 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -449,8 +449,8 @@ void KeeperServer::shutdownRaftServer() void KeeperServer::shutdown() { - state_manager->flushAndShutDownLogStore(); shutdownRaftServer(); + state_manager->flushAndShutDownLogStore(); state_machine->shutdownStorage(); } From 2d2c6eee301aced5645a5e65e7bbf976c77c3264 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 21 Nov 2022 15:03:05 +0000 Subject: [PATCH 10/98] Update NuRaft --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index e4e746a24eb..2bb4a8b6465 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit e4e746a24eb56861a86f3672771e3308d8c40722 +Subproject commit 2bb4a8b6465f8556f19db3cb1529ed8e9dfc5236 From 55ef96dc11a98ecc4c48be050f55464046b84f01 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 22 Nov 2022 11:37:59 +0000 Subject: [PATCH 11/98] Fix unit tests --- src/Coordination/Changelog.cpp | 12 +- src/Coordination/KeeperDispatcher.cpp | 1 - src/Coordination/tests/gtest_coordination.cpp | 127 ++++++++++++++---- 3 files changed, 112 insertions(+), 28 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 49eb644719d..5880d71e909 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -117,7 +117,7 @@ public: WriteBuffer * working_buf = compressed_buffer ? compressed_buffer->getNestedBuffer() : file_buf.get(); - /// Flush working buffer to file system + /// Flush working buffer to file system working_buf->next(); /// Fsync file system if needed @@ -577,8 +577,14 @@ void Changelog::writeThread() durable_idx_cv.notify_all(); // we shouldn't start the raft_server before sending it here - assert(raft_server); - raft_server->notify_log_append_completion(true); + if (raft_server) + { + raft_server->notify_log_append_completion(true); + } + else + { + LOG_WARNING(log, "Raft server is not set in LogStore."); + } } }, write_operation); } diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 6e9116d4b75..102ff25f692 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -100,7 +100,6 @@ void KeeperDispatcher::requestThread() } else { - current_batch.emplace_back(request); } } diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index fa4c42dd82a..d354a902e9b 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -129,10 +129,13 @@ struct SimpliestRaftServer params.snapshot_distance_ = 1; /// forcefully send snapshots params.client_req_timeout_ = 3000; params.return_method_ = nuraft::raft_params::blocking; + params.parallel_log_appending_ = true; + nuraft::raft_server::init_options opts; + opts.start_server_in_constructor_ = false; raft_instance = launcher.init( state_machine, state_manager, nuraft::cs_new("ToyRaftLogger", DB::LogsLevel::trace), port, - nuraft::asio_service::options{}, params); + nuraft::asio_service::options{}, params, opts); if (!raft_instance) { @@ -140,6 +143,10 @@ struct SimpliestRaftServer _exit(1); } + state_manager->getLogStore()->setRaftServer(raft_instance); + + raft_instance->start_server(false); + std::cout << "init Raft instance " << server_id; for (size_t ii = 0; ii < 20; ++ii) { @@ -240,6 +247,15 @@ TEST_P(CoordinationTest, ChangelogTestSimple) EXPECT_EQ(changelog.log_entries(1, 2)->size(), 1); } +namespace +{ +void waitDurableIndex(nuraft::log_store & log_store, uint64_t index) +{ + while (log_store.last_durable_index() != index) + std::this_thread::sleep_for(std::chrono::milliseconds(200)); +} + +} TEST_P(CoordinationTest, ChangelogTestFile) { @@ -247,9 +263,13 @@ TEST_P(CoordinationTest, ChangelogTestFile) ChangelogDirTest test("./logs"); DB::KeeperLogStore changelog("./logs", 5, true, params.enable_compression); changelog.init(1, 0); + uint64_t last_entry_idx = 0; auto entry = getLogEntry("hello world", 77); - changelog.append(entry); + last_entry_idx = changelog.append(entry); changelog.end_of_append_batch(0, 0); + + waitDurableIndex(changelog, last_entry_idx); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); for (const auto & p : fs::directory_iterator("./logs")) EXPECT_EQ(p.path(), "./logs/changelog_1_5.bin" + params.extension); @@ -258,9 +278,11 @@ TEST_P(CoordinationTest, ChangelogTestFile) changelog.append(entry); changelog.append(entry); changelog.append(entry); - changelog.append(entry); + last_entry_idx = changelog.append(entry); changelog.end_of_append_batch(0, 0); + waitDurableIndex(changelog, last_entry_idx); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); } @@ -315,6 +337,8 @@ TEST_P(CoordinationTest, ChangelogWriteAt) changelog.write_at(7, entry); changelog.end_of_append_batch(0, 0); + waitDurableIndex(changelog, 7); + EXPECT_EQ(changelog.size(), 7); EXPECT_EQ(changelog.last_entry()->get_term(), 77); EXPECT_EQ(changelog.entry_at(7)->get_term(), 77); @@ -336,14 +360,18 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead) ChangelogDirTest test("./logs"); DB::KeeperLogStore changelog("./logs", 5, true, params.enable_compression); changelog.init(1, 0); + uint64_t last_entry_idx = 0; for (size_t i = 0; i < 7; ++i) { auto entry = getLogEntry("hello world", i * 10); - changelog.append(entry); + last_entry_idx = changelog.append(entry); } changelog.end_of_append_batch(0, 0); EXPECT_EQ(changelog.size(), 7); + + waitDurableIndex(changelog, last_entry_idx); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); @@ -354,10 +382,12 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead) for (size_t i = 7; i < 10; ++i) { auto entry = getLogEntry("hello world", i * 10); - changelog_reader.append(entry); + last_entry_idx = changelog_reader.append(entry); } changelog_reader.end_of_append_batch(0, 0); EXPECT_EQ(changelog_reader.size(), 10); + + waitDurableIndex(changelog_reader, last_entry_idx); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); @@ -368,9 +398,11 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead) EXPECT_EQ(logs_count, 2); auto entry = getLogEntry("someentry", 77); - changelog_reader.append(entry); + last_entry_idx = changelog_reader.append(entry); changelog_reader.end_of_append_batch(0, 0); EXPECT_EQ(changelog_reader.size(), 11); + + waitDurableIndex(changelog_reader, last_entry_idx); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); @@ -389,13 +421,16 @@ TEST_P(CoordinationTest, ChangelogTestCompaction) DB::KeeperLogStore changelog("./logs", 5, true, params.enable_compression); changelog.init(1, 0); + uint64_t last_entry_idx = 0; for (size_t i = 0; i < 3; ++i) { auto entry = getLogEntry("hello world", i * 10); - changelog.append(entry); + last_entry_idx = changelog.append(entry); } changelog.end_of_append_batch(0, 0); + waitDurableIndex(changelog, last_entry_idx); + EXPECT_EQ(changelog.size(), 3); changelog.compact(2); @@ -413,9 +448,11 @@ TEST_P(CoordinationTest, ChangelogTestCompaction) auto e3 = getLogEntry("hello world", 50); changelog.append(e3); auto e4 = getLogEntry("hello world", 60); - changelog.append(e4); + last_entry_idx = changelog.append(e4); changelog.end_of_append_batch(0, 0); + waitDurableIndex(changelog, last_entry_idx); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); @@ -445,15 +482,18 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperations) ChangelogDirTest test("./logs"); DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression); changelog.init(1, 0); + uint64_t last_entry_idx = 0; for (size_t i = 0; i < 10; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); - changelog.append(entry); + last_entry_idx = changelog.append(entry); } changelog.end_of_append_batch(0, 0); EXPECT_EQ(changelog.size(), 10); + waitDurableIndex(changelog, last_entry_idx); + auto entries = changelog.pack(1, 5); DB::KeeperLogStore apply_changelog("./logs", 100, true, params.enable_compression); @@ -490,15 +530,18 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperationsEmpty) ChangelogDirTest test("./logs"); DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression); changelog.init(1, 0); + uint64_t last_entry_idx = 0; for (size_t i = 0; i < 10; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); - changelog.append(entry); + last_entry_idx = changelog.append(entry); } changelog.end_of_append_batch(0, 0); EXPECT_EQ(changelog.size(), 10); + waitDurableIndex(changelog, last_entry_idx); + auto entries = changelog.pack(5, 5); ChangelogDirTest test1("./logs1"); @@ -536,13 +579,16 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtPreviousFile) DB::KeeperLogStore changelog("./logs", 5, true, params.enable_compression); changelog.init(1, 0); + uint64_t last_entry_idx = 0; for (size_t i = 0; i < 33; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); - changelog.append(entry); + last_entry_idx = changelog.append(entry); } changelog.end_of_append_batch(0, 0); + waitDurableIndex(changelog, last_entry_idx); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); @@ -561,6 +607,8 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtPreviousFile) EXPECT_EQ(changelog.next_slot(), 8); EXPECT_EQ(changelog.last_entry()->get_term(), 5555); + waitDurableIndex(changelog, 7); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); @@ -585,13 +633,16 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtFileBorder) DB::KeeperLogStore changelog("./logs", 5, true, params.enable_compression); changelog.init(1, 0); + uint64_t last_entry_idx = 0; for (size_t i = 0; i < 33; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); - changelog.append(entry); + last_entry_idx = changelog.append(entry); } changelog.end_of_append_batch(0, 0); + waitDurableIndex(changelog, last_entry_idx); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); @@ -610,6 +661,8 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtFileBorder) EXPECT_EQ(changelog.next_slot(), 12); EXPECT_EQ(changelog.last_entry()->get_term(), 5555); + waitDurableIndex(changelog, 11); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); @@ -633,14 +686,16 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtAllFiles) ChangelogDirTest test("./logs"); DB::KeeperLogStore changelog("./logs", 5, true, params.enable_compression); changelog.init(1, 0); - + uint64_t last_entry_idx = 0; for (size_t i = 0; i < 33; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); - changelog.append(entry); + last_entry_idx = changelog.append(entry); } changelog.end_of_append_batch(0, 0); + waitDurableIndex(changelog, last_entry_idx); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); @@ -659,6 +714,8 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtAllFiles) EXPECT_EQ(changelog.next_slot(), 2); EXPECT_EQ(changelog.last_entry()->get_term(), 5555); + waitDurableIndex(changelog, 1); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_FALSE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); @@ -676,13 +733,16 @@ TEST_P(CoordinationTest, ChangelogTestStartNewLogAfterRead) DB::KeeperLogStore changelog("./logs", 5, true, params.enable_compression); changelog.init(1, 0); + uint64_t last_entry_idx = 0; for (size_t i = 0; i < 35; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); - changelog.append(entry); + last_entry_idx = changelog.append(entry); } changelog.end_of_append_batch(0, 0); EXPECT_EQ(changelog.size(), 35); + + waitDurableIndex(changelog, last_entry_idx); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); @@ -692,15 +752,16 @@ TEST_P(CoordinationTest, ChangelogTestStartNewLogAfterRead) EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin" + params.extension)); EXPECT_FALSE(fs::exists("./logs/changelog_36_40.bin" + params.extension)); - DB::KeeperLogStore changelog_reader("./logs", 5, true, params.enable_compression); changelog_reader.init(1, 0); auto entry = getLogEntry("36_hello_world", 360); - changelog_reader.append(entry); + last_entry_idx = changelog_reader.append(entry); changelog_reader.end_of_append_batch(0, 0); EXPECT_EQ(changelog_reader.size(), 36); + + waitDurableIndex(changelog_reader, last_entry_idx); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); @@ -739,13 +800,16 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) DB::KeeperLogStore changelog(log_folder, 5, true, params.enable_compression); changelog.init(1, 0); + uint64_t last_entry_idx = 0; for (size_t i = 0; i < 35; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); - changelog.append(entry); + last_entry_idx = changelog.append(entry); } changelog.end_of_append_batch(0, 0); EXPECT_EQ(changelog.size(), 35); + + waitDurableIndex(changelog, last_entry_idx); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); @@ -774,11 +838,13 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) assertBrokenLogRemoved(log_folder, "changelog_31_35.bin" + params.extension); auto entry = getLogEntry("h", 7777); - changelog_reader.append(entry); + last_entry_idx = changelog_reader.append(entry); changelog_reader.end_of_append_batch(0, 0); EXPECT_EQ(changelog_reader.size(), 11); EXPECT_EQ(changelog_reader.last_entry()->get_term(), 7777); + waitDurableIndex(changelog_reader, last_entry_idx); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); @@ -802,13 +868,15 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) DB::KeeperLogStore changelog("./logs", 20, true, params.enable_compression); changelog.init(1, 0); + uint64_t last_entry_idx = 0; for (size_t i = 0; i < 35; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", (i + 44) * 10); - changelog.append(entry); + last_entry_idx = changelog.append(entry); } changelog.end_of_append_batch(0, 0); + waitDurableIndex(changelog, last_entry_idx); EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_21_40.bin" + params.extension)); @@ -822,8 +890,11 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin" + params.extension)); assertBrokenLogRemoved("./logs", "changelog_21_40.bin" + params.extension); auto entry = getLogEntry("hello_world", 7777); - changelog_reader.append(entry); + last_entry_idx = changelog_reader.append(entry); changelog_reader.end_of_append_batch(0, 0); + + waitDurableIndex(changelog_reader, last_entry_idx); + EXPECT_EQ(changelog_reader.size(), 1); EXPECT_EQ(changelog_reader.last_entry()->get_term(), 7777); @@ -841,13 +912,15 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles) DB::KeeperLogStore changelog("./logs", 20, true, params.enable_compression); changelog.init(1, 0); + uint64_t last_entry_idx = 0; for (size_t i = 0; i < 35; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", (i + 44) * 10); - changelog.append(entry); + last_entry_idx = changelog.append(entry); } changelog.end_of_append_batch(0, 0); + waitDurableIndex(changelog, last_entry_idx); EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_21_40.bin" + params.extension)); @@ -867,13 +940,16 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles2) DB::KeeperLogStore changelog("./logs", 10, true, params.enable_compression); changelog.init(1, 0); + uint64_t last_entry_idx = 0; for (size_t i = 0; i < 35; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", (i + 44) * 10); - changelog.append(entry); + last_entry_idx = changelog.append(entry); } changelog.end_of_append_batch(0, 0); + waitDurableIndex(changelog, last_entry_idx); + EXPECT_TRUE(fs::exists("./logs/changelog_1_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_20.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_21_30.bin" + params.extension)); @@ -1322,14 +1398,17 @@ void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, uint state_machine->init(); DB::KeeperLogStore changelog("./logs", settings->rotate_log_storage_interval, true, enable_compression); changelog.init(state_machine->last_commit_index() + 1, settings->reserved_log_items); + uint64_t last_entry_idx = 0; for (size_t i = 1; i < total_logs + 1; ++i) { std::shared_ptr request = std::make_shared(); request->path = "/hello_" + std::to_string(i); auto entry = getLogEntryFromZKRequest(0, 1, i, request); - changelog.append(entry); + last_entry_idx = changelog.append(entry); changelog.end_of_append_batch(0, 0); + waitDurableIndex(changelog, last_entry_idx); + state_machine->pre_commit(i, changelog.entry_at(i)->get_buf()); state_machine->commit(i, changelog.entry_at(i)->get_buf()); bool snapshot_created = false; From e910648c5dae22617a8fae41049f6e6218408aea Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 23 Nov 2022 13:15:35 +0100 Subject: [PATCH 12/98] Add benchmark for query interpretation with JOINs --- .../performance/query_interpretation_join.xml | 393 ++++++++++++++++++ 1 file changed, 393 insertions(+) create mode 100644 tests/performance/query_interpretation_join.xml diff --git a/tests/performance/query_interpretation_join.xml b/tests/performance/query_interpretation_join.xml new file mode 100644 index 00000000000..5bbb0baf842 --- /dev/null +++ b/tests/performance/query_interpretation_join.xml @@ -0,0 +1,393 @@ + + + CREATE TABLE IF NOT EXISTS interpret_table_01 + ( + `idColumnU64` UInt64, + `dateColumn` DateTime, + `aggCount` AggregateFunction(count), + `aggArgMaxFloat32_1` AggregateFunction(argMax, Float32, DateTime), + `aggArgMaxString` AggregateFunction(argMax, String, DateTime), + `aggArgMaxFloat32_2` AggregateFunction(argMax, Float32, DateTime), + `nDateTime_02_date` SimpleAggregateFunction(max, DateTime), + `nDateTime_02_date_292929292` SimpleAggregateFunction(max, DateTime), + `agg_topk_uint32` AggregateFunction(topKWeighted(2), UInt32, UInt32), + `agg_argmax_string_datetime_01` AggregateFunction(argMax, String, DateTime), + `agg_argmax_string_u8_01` AggregateFunction(argMax, UInt8, DateTime), + `agg_argmax_string_datetime_02` AggregateFunction(argMax, String, DateTime), + `agg_argmax_string_datetime_03` AggregateFunction(argMax, String, DateTime), + `agg_argmax_string_datetime_04` AggregateFunction(argMax, String, DateTime), + `agg_argmax_string_datetime_05` AggregateFunction(argMax, String, DateTime), + `agg_argmax_string_datetime_06` AggregateFunction(argMax, String, DateTime), + `agg_argmax_string_datetime_07` AggregateFunction(argMax, String, DateTime), + `agg_argmax_string_u8_02` AggregateFunction(argMax, UInt8, DateTime), + `agg_argmax_string_u8_03` AggregateFunction(argMax, UInt8, DateTime), + `agg_argmax_string_u8_04` AggregateFunction(argMax, UInt8, DateTime), + `agg_count_01` AggregateFunction(count), + `agg_count_02` AggregateFunction(count), + `agg_count_03` AggregateFunction(count), + `agg_count_04` AggregateFunction(count), + `agg_count_05` AggregateFunction(count), + `agg_count_06` AggregateFunction(count), + `agg_count_07` AggregateFunction(count), + `agg_count_08` AggregateFunction(count), + `agg_count_09` AggregateFunction(count), + `agg_count_10` AggregateFunction(count), + `agg_count_11` AggregateFunction(count), + `agg_count_12` AggregateFunction(count), + `agg_count_13` AggregateFunction(count), + `agg_count_14` AggregateFunction(count), + `agg_count_15` AggregateFunction(count), + `agg_count_16` AggregateFunction(count), + `agg_argmax_string_datetime_08` AggregateFunction(argMax, String, DateTime), + `agg_argmax_f32_datetime_01` AggregateFunction(argMax, Float32, DateTime), + `agg_argmax_string_datetime_09` AggregateFunction(argMax, String, DateTime), + `agg_argmax_f32_datetime_02` AggregateFunction(argMax, Float32, DateTime), + `agg_argmax_date_datetime_01` AggregateFunction(argMax, Date, DateTime), + `agg_argmax_date_datetime_02` AggregateFunction(argMax, Date, DateTime), + `agg_argmax_u8_other_01` AggregateFunction(argMax, UInt8, DateTime), + `agg_argmax_u8_other_02` AggregateFunction(argMax, UInt8, DateTime), + `agg_argmax_u8_other_03` AggregateFunction(argMax, UInt8, DateTime), + `agg_argmax_u8_other_04` AggregateFunction(argMax, UInt8, DateTime), + `agg_argmax_string_datetime_10` AggregateFunction(argMax, String, DateTime), + `agg_argmax_u8_other_05` AggregateFunction(argMax, UInt8, DateTime), + `agg_argmax_u8_other_06` AggregateFunction(argMax, UInt8, DateTime), + `agg_argmax_u8_other_07` AggregateFunction(argMax, UInt8, DateTime), + `agg_argmax_string_datetime_11` AggregateFunction(argMax, String, DateTime), + `other_max_datetime_01` SimpleAggregateFunction(max, DateTime), + `other_max_datetime_02` SimpleAggregateFunction(max, DateTime), + `nDateTime_03_date` SimpleAggregateFunction(max, DateTime), + `nDateTime_03_shown_date` SimpleAggregateFunction(max, DateTime), + `nDateTime_04_date` SimpleAggregateFunction(max, DateTime), + `nDateTime_04_shown_date` SimpleAggregateFunction(max, DateTime), + `aggCount_3` AggregateFunction(count), + `uniq_date_agg` AggregateFunction(uniq, Date), + `aggCount_4` AggregateFunction(count), + `agg_argmax_u128_datetime_01` AggregateFunction(argMax, UInt128, DateTime), + `topk_u128_01` AggregateFunction(topKWeighted(5), UInt128, UInt32), + `agg_argmax_string_datetime_12` AggregateFunction(argMax, String, DateTime), + `agg_argmax_string_datetime_13` AggregateFunction(argMax, String, DateTime), + `agg_argmax_string_datetime_14` AggregateFunction(argMax, String, DateTime), + `agg_argmax_string_datetime_15` AggregateFunction(argMax, String, DateTime), + `agg_argmax_u32_datetime_01` AggregateFunction(argMax, UInt32, DateTime), + `agg_argmax_string_datetime_16` AggregateFunction(argMax, String, DateTime), + `agg_argmax_string_u8_100` AggregateFunction(argMax, String, UInt8), + `agg_argmax_string_datetime_18` AggregateFunction(argMax, String, DateTime), + `other_max_datetime_05` SimpleAggregateFunction(max, DateTime), + `topk_Datetime_u32_u32` AggregateFunction(topKWeighted(5), UInt32, UInt32), + `agg_argmax_string_datetime_17` AggregateFunction(argMax, String, DateTime), + `other_max_datetime_09` SimpleAggregateFunction(max, DateTime), + `agg_count_17` AggregateFunction(count), + `agg_count_18` AggregateFunction(count), + `agg_count_19` AggregateFunction(count), + `agg_count_20` AggregateFunction(count) + ) + ENGINE = AggregatingMergeTree() + PARTITION BY toYYYYMM(dateColumn) + ORDER BY idColumnU64 + TTL dateColumn + toIntervalMonth(6) + SETTINGS index_granularity = 1024, ttl_only_drop_parts = 1, min_rows_for_wide_part = 1000000000; + + DROP TABLE interpret_table_01 + + + CREATE TABLE IF NOT EXISTS interpret_table_02 + ( + `idColumnU64` UInt64, + `dateColumn` DateTime, + `agg_uniq_u128_01` AggregateFunction(uniq, UInt128), + `agg_uniq_u128_02` AggregateFunction(uniq, UInt128), + `aggCount` AggregateFunction(count), + `agg_uniq_u128_03` AggregateFunction(uniq, UInt128), + `agg_uniq_u128_04` AggregateFunction(uniq, UInt128), + `aggCount_3` AggregateFunction(count), + `aggCount_4` AggregateFunction(count), + `agg_topk_01` AggregateFunction(topKWeighted(2), UInt128, UInt64) + ) + ENGINE = AggregatingMergeTree() + PARTITION BY toYYYYMM(dateColumn) + ORDER BY idColumnU64 + TTL dateColumn + toIntervalMonth(6) + SETTINGS index_granularity = 1024, ttl_only_drop_parts = 1; + + DROP TABLE interpret_table_02 + + + CREATE TABLE IF NOT EXISTS interpret_table_03 + ( + `idColumnU64` UInt64, + `dateColumn` Date, + `aggCount` AggregateFunction(count), + `aggCount_2` AggregateFunction(count), + `aggCount_2_shown` AggregateFunction(count), + `minDate` SimpleAggregateFunction(min, Date), + `maxDate` SimpleAggregateFunction(max, Date), + `maxInt16` SimpleAggregateFunction(max, Int16), + `minUInt16` SimpleAggregateFunction(min, UInt16), + `minUInt16_2` SimpleAggregateFunction(min, UInt16), + `aggCount_3` AggregateFunction(count), + `aggCount_4` AggregateFunction(count) + ) + ENGINE = AggregatingMergeTree() + PARTITION BY toYYYYMM(dateColumn) + ORDER BY (idColumnU64, dateColumn) + TTL dateColumn + toIntervalDay(30) + SETTINGS index_granularity = 1024, ttl_only_drop_parts = 1; + + DROP TABLE interpret_table_03 + + + CREATE TABLE IF NOT EXISTS interpret_table_04 + ( + `idColumnU64` UInt64, + `dateColumn` DateTime, + `u128_id_02` UInt128, + `ls_01` LowCardinality(String), + `agg_count_01` AggregateFunction(count), + `agg_count_02` AggregateFunction(count), + `agg_smax_datetime_01` SimpleAggregateFunction(max, DateTime), + `agg_smax_datetime_02` SimpleAggregateFunction(max, DateTime), + `agg_count_03` AggregateFunction(count), + `agg_count_04` AggregateFunction(count) + ) + ENGINE = AggregatingMergeTree() + PARTITION BY toYYYYMM(dateColumn) + ORDER BY (idColumnU64, u128_id_02, ls_01) + TTL dateColumn + toIntervalMonth(6) + SETTINGS index_granularity = 1024, ttl_only_drop_parts = 1; + + DROP TABLE interpret_table_04 + + + CREATE TABLE IF NOT EXISTS interpret_table_05 + ( + `idColumnU64` UInt64, + `dateColumn` Date, + `agg_uniq_u128_01` AggregateFunction(uniq, UInt128), + `agg_uniq_u128_02` AggregateFunction(uniq, UInt128), + `agg_uniq_u128_03` AggregateFunction(uniq, UInt128), + `agg_uniq_u128_04` AggregateFunction(uniq, UInt128), + `aggCount_3` AggregateFunction(count), + `aggCount_4` AggregateFunction(count), + `aggCount` AggregateFunction(count) + ) + ENGINE = AggregatingMergeTree() + PARTITION BY toYYYYMM(dateColumn) + ORDER BY (idColumnU64, dateColumn) + TTL dateColumn + toIntervalDay(30) + SETTINGS index_granularity = 1024, ttl_only_drop_parts = 1; + + DROP TABLE interpret_table_05 + + + CREATE TABLE IF NOT EXISTS interpret_table_06 + ( + `idColumnU64` UInt64, + `dateColumn` DateTime, + `aggCount_3` AggregateFunction(count), + `aggCount` AggregateFunction(count), + `sagg_max_date` SimpleAggregateFunction(max, DateTime) + ) + ENGINE = AggregatingMergeTree() + PARTITION BY toYYYYMM(dateColumn) + ORDER BY idColumnU64 + TTL dateColumn + toIntervalMonth(6) + SETTINGS index_granularity = 1024, ttl_only_drop_parts = 1; + + DROP TABLE interpret_table_06 + + + CREATE TABLE IF NOT EXISTS interpret_table_07 + ( + `idU128` UInt128, + `idU128_2` UInt128, + `idU128_3` UInt128, + `nI16` Nullable(Int16) DEFAULT CAST(NULL, 'Nullable(Int16)'), + `idColumnI64` Nullable(Int64) DEFAULT CAST(NULL, 'Nullable(Int64)'), + `nStr` Nullable(String) DEFAULT CAST(NULL, 'Nullable(String)'), + `nStr_2` Nullable(String) DEFAULT CAST(NULL, 'Nullable(String)'), + `nI16_02` Nullable(Int16) DEFAULT CAST(NULL, 'Nullable(Int16)'), + `nStr_3` Nullable(String) DEFAULT CAST(NULL, 'Nullable(String)'), + `nStr_4` Nullable(String) DEFAULT CAST(NULL, 'Nullable(String)'), + `nStr_5` Nullable(String) DEFAULT CAST(NULL, 'Nullable(String)'), + `nI8_01` Nullable(Int8) DEFAULT CAST(NULL, 'Nullable(Int8)'), + `nI8_02` Nullable(Int8) DEFAULT CAST(NULL, 'Nullable(Int8)'), + `nI8_03` Nullable(Int8) DEFAULT CAST(NULL, 'Nullable(Int8)'), + `nI8_04` Nullable(Int8) DEFAULT CAST(NULL, 'Nullable(Int8)'), + `nI8_05` Nullable(Int8) DEFAULT CAST(NULL, 'Nullable(Int8)'), + `nI8_06` Nullable(Int8) DEFAULT CAST(NULL, 'Nullable(Int8)'), + `nDate_01` Nullable(Date) DEFAULT CAST(NULL, 'Nullable(Date)'), + `nStr_6` Nullable(String) DEFAULT CAST(NULL, 'Nullable(String)'), + `nStr_7` Nullable(String) DEFAULT CAST(NULL, 'Nullable(String)'), + `nStr_8` Nullable(String) DEFAULT CAST(NULL, 'Nullable(String)'), + `nStr_9` Nullable(String) DEFAULT CAST(NULL, 'Nullable(String)'), + `nStr_10` Nullable(String) DEFAULT CAST(NULL, 'Nullable(String)'), + `nStr_11` Nullable(String) DEFAULT CAST(NULL, 'Nullable(String)'), + `nI8_07` Nullable(UInt8) DEFAULT CAST(NULL, 'Nullable(UInt8)'), + `nI8_08` Nullable(UInt8) DEFAULT CAST(NULL, 'Nullable(UInt8)'), + `Str_01` String, + `nI32_01` Nullable(Int32) DEFAULT CAST(NULL, 'Nullable(Int32)'), + `nI8_19` Nullable(Int8) DEFAULT CAST(NULL, 'Nullable(Int8)'), + `nI8_09` Nullable(Int8) DEFAULT CAST(NULL, 'Nullable(Int8)'), + `nI8_10` Nullable(Int8) DEFAULT CAST(NULL, 'Nullable(Int8)'), + `nI8_11` Nullable(Int8) DEFAULT CAST(NULL, 'Nullable(Int8)'), + `nI8_12` Nullable(Int8) DEFAULT CAST(NULL, 'Nullable(Int8)'), + `nI8_13` Nullable(Int8) DEFAULT CAST(NULL, 'Nullable(Int8)'), + `nI8_14` Nullable(Int8) DEFAULT CAST(NULL, 'Nullable(Int8)'), + `nStr_12` Nullable(String) DEFAULT CAST(NULL, 'Nullable(String)'), + `nStr_13` Nullable(String) DEFAULT CAST(NULL, 'Nullable(String)'), + `nI8_15` Nullable(Int8) DEFAULT CAST(NULL, 'Nullable(Int8)'), + `nI8_16` Nullable(Int8) DEFAULT CAST(NULL, 'Nullable(Int8)'), + `nDateTime_01` Nullable(DateTime) DEFAULT CAST(NULL, 'Nullable(DateTime)'), + `nDateTime_02` Nullable(DateTime) DEFAULT CAST(NULL, 'Nullable(DateTime)'), + `nDateTime_03` Nullable(DateTime) DEFAULT CAST(NULL, 'Nullable(DateTime)'), + `nDateTime_04` Nullable(DateTime) DEFAULT CAST(NULL, 'Nullable(DateTime)'), + `nDateTime_05` Nullable(DateTime) DEFAULT CAST(NULL, 'Nullable(DateTime)'), + `nDateTime_06` Nullable(DateTime) DEFAULT CAST(NULL, 'Nullable(DateTime)'), + `nDateTime_07` Nullable(DateTime) DEFAULT CAST(NULL, 'Nullable(DateTime)'), + `nDateTime_08` Nullable(DateTime) DEFAULT CAST(NULL, 'Nullable(DateTime)'), + `nDateTime_09` Nullable(DateTime) DEFAULT CAST(NULL, 'Nullable(DateTime)'), + `nDateTime_10` Nullable(DateTime) DEFAULT CAST(NULL, 'Nullable(DateTime)'), + `nDateTime_11` Nullable(DateTime) DEFAULT CAST(NULL, 'Nullable(DateTime)'), + `nDateTime_12` Nullable(DateTime) DEFAULT CAST(NULL, 'Nullable(DateTime)'), + `nF64_01` Nullable(Float64) DEFAULT CAST(NULL, 'Nullable(Float64)'), + `nStr_14` Nullable(String) DEFAULT CAST(NULL, 'Nullable(String)'), + `nDate_02` Nullable(Date) DEFAULT CAST(NULL, 'Nullable(Date)'), + `nDateTime_13` Nullable(DateTime) DEFAULT CAST(NULL, 'Nullable(DateTime)'), + `nF64_02` Nullable(Float64) DEFAULT CAST(NULL, 'Nullable(Float64)'), + `nF64_03` Nullable(Float64) DEFAULT CAST(NULL, 'Nullable(Float64)'), + `nF64_04` Nullable(Float64) DEFAULT CAST(NULL, 'Nullable(Float64)'), + `nF64_05` Nullable(Float64) DEFAULT CAST(NULL, 'Nullable(Float64)'), + `nI8_18` Nullable(Int8) DEFAULT CAST(NULL, 'Nullable(Int8)'), + `nI8_17` Nullable(Int8) DEFAULT CAST(NULL, 'Nullable(Int8)') + ) + ENGINE = Join(ANY, LEFT, idU128); + + DROP TABLE interpret_table_07 + + + CREATE TABLE IF NOT EXISTS interpret_table_08 + ( + `idColumnU64` UInt64, + `dateColumn` Date, + `aggCount_3` AggregateFunction(count), + `aggCount_4` AggregateFunction(count) + ) + ENGINE = AggregatingMergeTree() + PARTITION BY toYYYYMM(dateColumn) + ORDER BY (idColumnU64, dateColumn) + TTL dateColumn + toIntervalDay(30) + SETTINGS index_granularity = 1024, ttl_only_drop_parts = 1; + + DROP TABLE interpret_table_08 + + + + SELECT * + FROM + ( + SELECT + cityHash64('0321352416546546546546546546546', lower('BU'), lower('random2'), toUInt128(toUInt128('1015029'))) AS idColumnU64, + * + FROM + ( + SELECT + if(max(nDateTime_02_date_292929292) > '2020-10-31 00:00:00', max(nDateTime_02_date_292929292), NULL) AS o1, + if(max(other_max_datetime_05) > '2020-10-31 00:00:00', max(other_max_datetime_05), NULL) AS o2, + if(max(nDateTime_03_date) > '2020-10-31 00:00:00', max(nDateTime_03_date), NULL) AS o3, + if(max(nDateTime_04_date) > '2020-10-31 00:00:00', max(nDateTime_04_date), NULL) AS o4, + if(max(nDateTime_02_date) > '2020-10-31 00:00:00', max(nDateTime_02_date), NULL) AS o5, + if(max(other_max_datetime_01) > '2020-10-31 00:00:00', max(other_max_datetime_01), NULL) AS o6, + if(max(other_max_datetime_02) > '2020-10-31 00:00:00', max(other_max_datetime_02), NULL) AS o7, + argMaxMerge(agg_argmax_string_datetime_13) AS o8, + argMaxMerge(agg_argmax_string_datetime_05) AS o9, + argMaxMerge(agg_argmax_string_datetime_06) AS o10, + argMaxMerge(agg_argmax_string_datetime_02) AS o11, + argMaxMerge(agg_argmax_string_datetime_04) AS o12, + argMaxMerge(agg_argmax_string_datetime_15) AS o13, + argMaxMerge(agg_argmax_string_datetime_01) AS o14, + argMaxMerge(agg_argmax_string_u8_01) AS o15, + argMaxMerge(agg_argmax_f32_datetime_02) AS o16, + if(argMaxMerge(agg_argmax_string_datetime_09) != '', argMaxMerge(agg_argmax_string_datetime_09), NULL) AS o17, + if(argMaxMerge(agg_argmax_date_datetime_01) > '2020-10-31', argMaxMerge(agg_argmax_date_datetime_01), NULL) AS o18, + if(argMaxMerge(agg_argmax_date_datetime_02) > '2020-10-31', argMaxMerge(agg_argmax_date_datetime_02), NULL) AS o19, + argMaxMerge(agg_argmax_u8_other_02) AS o20, + argMaxMerge(agg_argmax_u8_other_03) AS o21, + argMaxMerge(agg_argmax_u8_other_04) AS o22, + argMaxMerge(agg_argmax_u8_other_01) AS o23, + argMaxMerge(agg_argmax_string_datetime_10) AS o24, + argMaxMerge(agg_argmax_string_datetime_11) AS o25, + countMerge(aggCount_3) AS o26, + countMerge(aggCount_4) AS o27 + FROM interpret_table_01 AS c + PREWHERE cityHash64('0321352416546546546546546546546', lower('BU'), lower('random2'), toUInt128(toUInt128('1015029'))) = c.idColumnU64 + ) AS s01, + ( + WITH ( + SELECT coalesce(if((topKWeightedMerge(2)(agg_topk_01)[1]) != toUInt128(toUInt128('1015029')), topKWeightedMerge(2)(agg_topk_01)[1], topKWeightedMerge(2)(agg_topk_01)[2]), 0) + FROM interpret_table_02 AS c + PREWHERE cityHash64('0321352416546546546546546546546', lower('BU'), lower('random2')) = c.idColumnU64 + ) AS other_idU128 + SELECT + if(max(other_max_datetime_05) > '2020-10-31 00:00:00', max(other_max_datetime_05), NULL) AS o28, + if(max(other_max_datetime_01) > '2020-10-31 00:00:00', max(other_max_datetime_01), NULL) AS o29, + if(max(nDateTime_02_date) > '2020-10-31 00:00:00', max(nDateTime_02_date), NULL) AS o30, + other_idU128 + FROM interpret_table_01 AS c + PREWHERE cityHash64('0321352416546546546546546546546', lower('BU'), lower('random2'), toUInt128(other_idU128)) = c.idColumnU64 + ) AS s02, + ( + SELECT + minIf(minDate, dateColumn > (now() - toIntervalDay(7))) AS o31, + maxIf(maxDate, dateColumn > (now() - toIntervalDay(7))) AS o32, + maxIf(maxInt16, dateColumn > (now() - toIntervalDay(28))) AS o33, + countMergeIf(aggCount_3, dateColumn > (now() - toIntervalHour(24))) AS o34, + countMergeIf(aggCount_3, dateColumn > (now() - toIntervalDay(14))) AS o35, + countMergeIf(aggCount_3, dateColumn > (now() - toIntervalDay(28))) AS o36, + countMergeIf(aggCount_4, dateColumn > (now() - toIntervalHour(24))) AS o37, + countMergeIf(aggCount_4, dateColumn > (now() - toIntervalDay(7))) AS o38, + countMergeIf(aggCount_4, dateColumn > (now() - toIntervalDay(28))) AS o27_month, + countMergeIf(aggCount_2_shown, dateColumn > (now() - toIntervalDay(14))) AS o40 + FROM interpret_table_03 AS c + PREWHERE cityHash64('0321352416546546546546546546546', lower('BU'), lower('random2'), toUInt128(toUInt128('1015029'))) = c.idColumnU64 + ) AS s03, + ( + SELECT + countMerge(agg_count_03) AS o41, + countMerge(agg_count_04) AS o42 + FROM interpret_table_04 AS c + PREWHERE (cityHash64('0321352416546546546546546546546', lower('BU'), lower('random2'), toUInt128(toUInt128('1015029'))) = c.idColumnU64) AND (ls_01 = 'exit') + ) AS s04, + ( + SELECT + countMerge(aggCount_3) AS o43, + countMerge(aggCount_4) AS o44, + countMerge(aggCount) AS o45 + FROM interpret_table_02 AS c + PREWHERE cityHash64('0321352416546546546546546546546', lower('BU'), lower('random2')) = c.idColumnU64 + ) AS s05, + ( + SELECT + countMergeIf(aggCount_3, dateColumn > (now() - toIntervalDay(14))) AS o46, + uniqMergeIf(agg_uniq_u128_03, dateColumn > (now() - toIntervalHour(24))) AS o47, + uniqMergeIf(agg_uniq_u128_03, dateColumn > (now() - toIntervalDay(14))) AS o48, + countMergeIf(aggCount_4, dateColumn > (now() - toIntervalDay(14))) AS o49, + countMergeIf(aggCount_4, dateColumn > (now() - toIntervalDay(28))) AS o50 + FROM interpret_table_05 AS c + PREWHERE cityHash64('0321352416546546546546546546546', lower('BU'), lower('random2')) = c.idColumnU64 + ) AS s06, + ( + SELECT countMerge(aggCount_3) AS o51 + FROM interpret_table_06 AS c + PREWHERE cityHash64('0321352416546546546546546546546', lower('BU'), lower('random2'), toUInt128(joinGet(interpret_table_07, 'idColumnI64', toUInt128('1015029')))) = c.idColumnU64 + ) AS s07, + ( + SELECT + countMergeIf(aggCount_3, dateColumn > (now() - toIntervalDay(28))) AS s52, + countMergeIf(aggCount_4, dateColumn > (now() - toIntervalDay(28))) AS s53 + FROM interpret_table_08 AS c + PREWHERE cityHash64('0321352416546546546546546546546', lower('BU'), lower('random2'), toUInt128(joinGet(interpret_table_07, 'idColumnI64', toUInt128('1015029')))) = c.idColumnU64 + ) AS s08 + ) AS final_s01 + FORMAT JSONEachRow; + + From ce532cd12dff5a0b3945b49fefe74d1998ab42b4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 24 Nov 2022 13:43:05 +0000 Subject: [PATCH 13/98] Fix circular deps --- src/Coordination/Changelog.cpp | 12 ++++-------- src/Coordination/Changelog.h | 5 ++--- src/Coordination/KeeperLogStore.cpp | 2 +- src/Coordination/KeeperLogStore.h | 2 +- src/Coordination/tests/gtest_coordination.cpp | 6 +++++- 5 files changed, 13 insertions(+), 14 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 5880d71e909..3d01f98f5b1 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -577,14 +577,10 @@ void Changelog::writeThread() durable_idx_cv.notify_all(); // we shouldn't start the raft_server before sending it here - if (raft_server) - { - raft_server->notify_log_append_completion(true); - } + if (auto raft_server_locked = raft_server.lock()) + raft_server_locked->notify_log_append_completion(true); else - { LOG_WARNING(log, "Raft server is not set in LogStore."); - } } }, write_operation); } @@ -853,10 +849,10 @@ void Changelog::cleanLogThread() } } -void Changelog::setRaftServer(nuraft::ptr raft_server_) +void Changelog::setRaftServer(const nuraft::ptr & raft_server_) { assert(raft_server_); - raft_server = std::move(raft_server_); + raft_server = raft_server_; } } diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 01ce7cdd145..60ebaf5994e 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -138,7 +138,7 @@ public: return last_durable_idx; } - void setRaftServer(nuraft::ptr raft_server_); + void setRaftServer(const nuraft::ptr & raft_server_); /// Fsync log to disk ~Changelog(); @@ -205,12 +205,11 @@ private: ThreadFromGlobalPool write_thread; ConcurrentBoundedQueue write_operations; - mutable std::mutex durable_idx_mutex; std::condition_variable durable_idx_cv; uint64_t last_durable_idx{0}; - nuraft::ptr raft_server{nullptr}; + nuraft::wptr raft_server; }; } diff --git a/src/Coordination/KeeperLogStore.cpp b/src/Coordination/KeeperLogStore.cpp index 448f1f8532e..ea72022af09 100644 --- a/src/Coordination/KeeperLogStore.cpp +++ b/src/Coordination/KeeperLogStore.cpp @@ -138,7 +138,7 @@ uint64_t KeeperLogStore::last_durable_index() return changelog.lastDurableIndex(); } -void KeeperLogStore::setRaftServer(nuraft::ptr raft_server) +void KeeperLogStore::setRaftServer(const nuraft::ptr & raft_server) { std::lock_guard lock(changelog_lock); return changelog.setRaftServer(raft_server); diff --git a/src/Coordination/KeeperLogStore.h b/src/Coordination/KeeperLogStore.h index 2addbe61c01..260a6b29320 100644 --- a/src/Coordination/KeeperLogStore.h +++ b/src/Coordination/KeeperLogStore.h @@ -70,7 +70,7 @@ public: /// Get entry with latest config in logstore nuraft::ptr getLatestConfigChange() const; - void setRaftServer(nuraft::ptr raft_server); + void setRaftServer(const nuraft::ptr & raft_server); private: mutable std::mutex changelog_lock; diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index d354a902e9b..11b2fd33367 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -293,15 +293,19 @@ TEST_P(CoordinationTest, ChangelogReadWrite) ChangelogDirTest test("./logs"); DB::KeeperLogStore changelog("./logs", 1000, true, params.enable_compression); changelog.init(1, 0); + + uint64_t last_entry_idx = 0; for (size_t i = 0; i < 10; ++i) { auto entry = getLogEntry("hello world", i * 10); - changelog.append(entry); + last_entry_idx = changelog.append(entry); } changelog.end_of_append_batch(0, 0); EXPECT_EQ(changelog.size(), 10); + waitDurableIndex(changelog, last_entry_idx); + DB::KeeperLogStore changelog_reader("./logs", 1000, true, params.enable_compression); changelog_reader.init(1, 0); EXPECT_EQ(changelog_reader.size(), 10); From 312dd725b4bf2200b05a4774eddb967770fe1d09 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 24 Nov 2022 19:54:39 +0000 Subject: [PATCH 14/98] allow to send profile events to trace_log --- src/Common/MemoryTracker.cpp | 10 ++-- src/Common/ProfileEvents.cpp | 19 +++++++- src/Common/ProfileEvents.h | 17 ++++--- src/Common/QueryProfiler.cpp | 4 +- src/Common/TraceSender.cpp | 12 +++-- src/Common/TraceSender.h | 14 +++++- src/Core/Settings.h | 1 + src/Interpreters/ProcessList.cpp | 1 + src/Interpreters/TraceCollector.cpp | 9 +++- src/Interpreters/TraceCollector.h | 7 --- src/Interpreters/TraceLog.cpp | 11 +++++ src/Interpreters/TraceLog.h | 10 +++- .../02494_trace_log_profile_events.reference | 3 ++ .../02494_trace_log_profile_events.sh | 48 +++++++++++++++++++ 14 files changed, 132 insertions(+), 34 deletions(-) create mode 100644 tests/queries/0_stateless/02494_trace_log_profile_events.reference create mode 100755 tests/queries/0_stateless/02494_trace_log_profile_events.sh diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index f556b255fc2..27d0adcf24f 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include @@ -178,7 +178,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT if (unlikely(current_profiler_limit && will_be > current_profiler_limit)) { MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); - DB::TraceCollector::collect(DB::TraceType::Memory, StackTrace(), size); + DB::TraceSender::send(DB::TraceType::Memory, StackTrace(), {.size = size}); setOrRaiseProfilerLimit((will_be + profiler_step - 1) / profiler_step * profiler_step); allocation_traced = true; } @@ -187,7 +187,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT if (unlikely(sample_probability > 0.0 && sample(thread_local_rng))) { MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); - DB::TraceCollector::collect(DB::TraceType::MemorySample, StackTrace(), size); + DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = size}); allocation_traced = true; } @@ -305,7 +305,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT if (peak_updated && allocation_traced) { MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); - DB::TraceCollector::collect(DB::TraceType::MemoryPeak, StackTrace(), will_be); + DB::TraceSender::send(DB::TraceType::MemoryPeak, StackTrace(), {.size = will_be}); } if (auto * loaded_next = parent.load(std::memory_order_relaxed)) @@ -361,7 +361,7 @@ void MemoryTracker::free(Int64 size) if (unlikely(sample_probability > 0.0 && sample(thread_local_rng))) { MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); - DB::TraceCollector::collect(DB::TraceType::MemorySample, StackTrace(), -size); + DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = -size}); } Int64 accounted_size = size; diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 90d24ec027e..e793b334eea 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -1,5 +1,6 @@ #include #include +#include /// Available events. Add something here as you wish. @@ -514,15 +515,29 @@ const char * getDocumentation(Event event) return strings[event]; } - Event end() { return END; } - void increment(Event event, Count amount) { DB::CurrentThread::getProfileEvents().increment(event, amount); } +void Counters::increment(Event event, Count amount) +{ + Counters * current = this; + bool send_to_trace_log = false; + + do + { + send_to_trace_log |= current->trace_profile_events; + current->counters[event].fetch_add(amount, std::memory_order_relaxed); + current = current->parent; + } while (current != nullptr); + + if (unlikely(send_to_trace_log)) + DB::TraceSender::send(DB::TraceType::ProfileEvent, StackTrace(), {.event = event, .increment = amount}); +} + CountersIncrement::CountersIncrement(Counters::Snapshot const & snapshot) { init(); diff --git a/src/Common/ProfileEvents.h b/src/Common/ProfileEvents.h index 6eebb75c5ca..256a17cc080 100644 --- a/src/Common/ProfileEvents.h +++ b/src/Common/ProfileEvents.h @@ -25,10 +25,12 @@ namespace ProfileEvents class Counters { + private: Counter * counters = nullptr; std::unique_ptr counters_holder; /// Used to propagate increments Counters * parent = nullptr; + bool trace_profile_events = false; public: @@ -51,15 +53,7 @@ namespace ProfileEvents return counters[event]; } - inline void increment(Event event, Count amount = 1) - { - Counters * current = this; - do - { - current->counters[event].fetch_add(amount, std::memory_order_relaxed); - current = current->parent; - } while (current != nullptr); - } + void increment(Event event, Count amount = 1); struct Snapshot { @@ -97,6 +91,11 @@ namespace ProfileEvents parent = parent_; } + void setTraceProfileEvents(bool value) + { + trace_profile_events = value; + } + /// Set all counters to zero void resetCounters(); diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index b50e0c0ab49..14a6a06088c 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -1,7 +1,7 @@ #include "QueryProfiler.h" #include -#include +#include #include #include #include @@ -66,7 +66,7 @@ namespace const auto signal_context = *reinterpret_cast(context); const StackTrace stack_trace(signal_context); - TraceCollector::collect(trace_type, stack_trace, 0); + TraceSender::send(trace_type, stack_trace, {}); ProfileEvents::increment(ProfileEvents::QueryProfilerRuns); errno = saved_errno; diff --git a/src/Common/TraceSender.cpp b/src/Common/TraceSender.cpp index ad88e508d06..2ff75cdd82c 100644 --- a/src/Common/TraceSender.cpp +++ b/src/Common/TraceSender.cpp @@ -23,7 +23,7 @@ namespace DB LazyPipeFDs TraceSender::pipe; -void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Int64 size) +void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Extras extras) { constexpr size_t buf_size = sizeof(char) /// TraceCollector stop flag + sizeof(UInt8) /// String size @@ -32,12 +32,14 @@ void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Int + sizeof(StackTrace::FramePointers) /// Collected stack trace, maximum capacity + sizeof(TraceType) /// trace type + sizeof(UInt64) /// thread_id - + sizeof(Int64); /// size + + sizeof(Int64) /// size + + sizeof(ProfileEvents::Event) /// event + + sizeof(ProfileEvents::Count); /// increment /// Write should be atomic to avoid overlaps /// (since recursive collect() is possible) static_assert(PIPE_BUF >= 512); - static_assert(buf_size <= 512, "Only write of PIPE_BUF to pipe is atomic and the minimal known PIPE_BUF across supported platforms is 512"); + static_assert(buf_size <= PIPE_BUF, "Only write of PIPE_BUF to pipe is atomic and the minimal known PIPE_BUF across supported platforms is 512"); char buffer[buf_size]; WriteBufferFromFileDescriptorDiscardOnFailure out(pipe.fds_rw[1], buf_size, buffer); @@ -71,7 +73,9 @@ void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Int writePODBinary(trace_type, out); writePODBinary(thread_id, out); - writePODBinary(size, out); + writePODBinary(extras.size, out); + writePODBinary(extras.event, out); + writePODBinary(extras.increment, out); out.next(); } diff --git a/src/Common/TraceSender.h b/src/Common/TraceSender.h index a93e605a6e7..21b44b651dd 100644 --- a/src/Common/TraceSender.h +++ b/src/Common/TraceSender.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include class StackTrace; @@ -17,6 +18,7 @@ enum class TraceType : uint8_t Memory, MemorySample, MemoryPeak, + ProfileEvent, }; /// This is the second part of TraceCollector, that sends stacktrace to the pipe. @@ -24,10 +26,18 @@ enum class TraceType : uint8_t class TraceSender { public: + struct Extras + { + /// size - for memory tracing is the amount of memory allocated; for other trace types it is 0. + Int64 size{}; + /// Event type and increment for 'ProfileEvent' trace type; for other trace types defaults. + ProfileEvents::Event event{ProfileEvents::end()}; + ProfileEvents::Count increment{}; + }; + /// Collect a stack trace. This method is signal safe. /// Precondition: the TraceCollector object must be created. - /// size - for memory tracing is the amount of memory allocated; for other trace types it is 0. - static void send(TraceType trace_type, const StackTrace & stack_trace, Int64 size); + static void send(TraceType trace_type, const StackTrace & stack_trace, Extras extras); private: friend class TraceCollector; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index cfc7df6c853..7ccf1b4f717 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -397,6 +397,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \ M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \ M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ + M(Bool, trace_profile_events, false, "Send to system.trace_log profile event and value of increment on each increment with 'ProfileEvent' trace_type", 0) \ \ M(UInt64, memory_usage_overcommit_max_wait_microseconds, 5'000'000, "Maximum time thread will wait for memory to be freed in the case of memory overcommit. If timeout is reached and memory is not freed, exception is thrown.", 0) \ \ diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 84f5570349b..cc22ca6597e 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -230,6 +230,7 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as /// Set up memory profiling thread_group->memory_tracker.setProfilerStep(settings.memory_profiler_step); thread_group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability); + thread_group->performance_counters.setTraceProfileEvents(settings.trace_profile_events); } thread_group->memory_tracker.setDescription("(for query)"); diff --git a/src/Interpreters/TraceCollector.cpp b/src/Interpreters/TraceCollector.cpp index 41a7fcf8389..050dea02717 100644 --- a/src/Interpreters/TraceCollector.cpp +++ b/src/Interpreters/TraceCollector.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -96,6 +97,12 @@ void TraceCollector::run() Int64 size; readPODBinary(size, in); + ProfileEvents::Event event; + readPODBinary(event, in); + + ProfileEvents::Count increment; + readPODBinary(increment, in); + if (trace_log) { // time and time_in_microseconds are both being constructed from the same timespec so that the @@ -105,7 +112,7 @@ void TraceCollector::run() UInt64 time = static_cast(ts.tv_sec * 1000000000LL + ts.tv_nsec); UInt64 time_in_microseconds = static_cast((ts.tv_sec * 1000000LL) + (ts.tv_nsec / 1000)); - TraceLogElement element{time_t(time / 1000000000), time_in_microseconds, time, trace_type, thread_id, query_id, trace, size}; + TraceLogElement element{time_t(time / 1000000000), time_in_microseconds, time, trace_type, thread_id, query_id, trace, size, event, increment}; trace_log->add(element); } } diff --git a/src/Interpreters/TraceCollector.h b/src/Interpreters/TraceCollector.h index b3f11ca5756..40fa854b791 100644 --- a/src/Interpreters/TraceCollector.h +++ b/src/Interpreters/TraceCollector.h @@ -1,7 +1,5 @@ #pragma once - #include -#include class StackTrace; @@ -21,11 +19,6 @@ public: explicit TraceCollector(std::shared_ptr trace_log_); ~TraceCollector(); - static inline void collect(TraceType trace_type, const StackTrace & stack_trace, Int64 size) - { - return TraceSender::send(trace_type, stack_trace, size); - } - private: std::shared_ptr trace_log; ThreadFromGlobalPool thread; diff --git a/src/Interpreters/TraceLog.cpp b/src/Interpreters/TraceLog.cpp index c16a73e75dc..0408ebe504b 100644 --- a/src/Interpreters/TraceLog.cpp +++ b/src/Interpreters/TraceLog.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -20,6 +21,7 @@ const TraceDataType::Values TraceLogElement::trace_values = {"Memory", static_cast(TraceType::Memory)}, {"MemorySample", static_cast(TraceType::MemorySample)}, {"MemoryPeak", static_cast(TraceType::MemoryPeak)}, + {"ProfileEvent", static_cast(TraceType::ProfileEvent)}, }; NamesAndTypesList TraceLogElement::getNamesAndTypes() @@ -36,6 +38,8 @@ NamesAndTypesList TraceLogElement::getNamesAndTypes() {"query_id", std::make_shared()}, {"trace", std::make_shared(std::make_shared())}, {"size", std::make_shared()}, + {"event", std::make_shared(std::make_shared())}, + {"increment", std::make_shared()}, }; } @@ -53,6 +57,13 @@ void TraceLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insertData(query_id.data(), query_id.size()); columns[i++]->insert(trace); columns[i++]->insert(size); + + String event_name; + if (event != ProfileEvents::end()) + event_name = ProfileEvents::getName(event); + + columns[i++]->insert(event_name); + columns[i++]->insert(increment); } } diff --git a/src/Interpreters/TraceLog.h b/src/Interpreters/TraceLog.h index 43d7861327f..c481f033a72 100644 --- a/src/Interpreters/TraceLog.h +++ b/src/Interpreters/TraceLog.h @@ -3,8 +3,9 @@ #include #include #include -#include #include +#include +#include #include #include @@ -26,7 +27,12 @@ struct TraceLogElement UInt64 thread_id{}; String query_id{}; Array trace{}; - Int64 size{}; /// Allocation size in bytes for TraceType::Memory + /// Allocation size in bytes for TraceType::Memory. + Int64 size{}; + /// ProfileEvent for TraceType::ProfileEvent. + ProfileEvents::Event event{ProfileEvents::end()}; + /// Increment of profile event for TraceType::ProfileEvent. + ProfileEvents::Count increment{}; static std::string name() { return "TraceLog"; } static NamesAndTypesList getNamesAndTypes(); diff --git a/tests/queries/0_stateless/02494_trace_log_profile_events.reference b/tests/queries/0_stateless/02494_trace_log_profile_events.reference new file mode 100644 index 00000000000..5923753dbc7 --- /dev/null +++ b/tests/queries/0_stateless/02494_trace_log_profile_events.reference @@ -0,0 +1,3 @@ +1 +1 1 +0 diff --git a/tests/queries/0_stateless/02494_trace_log_profile_events.sh b/tests/queries/0_stateless/02494_trace_log_profile_events.sh new file mode 100755 index 00000000000..58549efd381 --- /dev/null +++ b/tests/queries/0_stateless/02494_trace_log_profile_events.sh @@ -0,0 +1,48 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +query_id=$RANDOM +${CLICKHOUSE_CLIENT} --query_id $query_id --query "SELECT 1 FORMAT Null SETTINGS trace_profile_events = 0" + +${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} --query "SELECT count() = 0 FROM system.trace_log WHERE query_id = '$query_id' AND trace_type = 'ProfileEvent'" + +query_id=$RANDOM +${CLICKHOUSE_CLIENT} --query_id $query_id --query "SELECT 1 FORMAT Null SETTINGS trace_profile_events = 1" + +${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} --query "SELECT count() > 0, sum(empty(trace)) = 0 FROM system.trace_log WHERE query_id = '$query_id' AND trace_type = 'ProfileEvent'" + +query_id=$RANDOM +${CLICKHOUSE_CLIENT} --query_id $query_id --query "SELECT count() FROM numbers_mt(1000000) FORMAT Null SETTINGS trace_profile_events = 1" + +${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} --query " +SELECT count() +FROM +( + ( + SELECT + Events.1 AS event, + Events.2 AS value + FROM system.query_log + ARRAY JOIN CAST(ProfileEvents, 'Array(Tuple(String, Int64))') AS Events + WHERE query_id = '$query_id' AND type = 'QueryFinish' + ORDER BY event + ) + EXCEPT + ( + SELECT + event, + sum(increment) AS value + FROM system.trace_log + WHERE (trace_type = 'ProfileEvent') AND (query_id = '$query_id') AND increment != 0 + GROUP BY event + ORDER BY event ASC + ) +) +WHERE event NOT IN ('ContextLock', 'NetworkSendBytes', 'NetworkSendElapsedMicroseconds'); +" From b94bee3aa4126c9d54474beedcf2be200aa94865 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 25 Nov 2022 03:06:55 +0000 Subject: [PATCH 15/98] fix test --- tests/queries/0_stateless/02494_trace_log_profile_events.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02494_trace_log_profile_events.sh b/tests/queries/0_stateless/02494_trace_log_profile_events.sh index 58549efd381..37fc9ad85bf 100755 --- a/tests/queries/0_stateless/02494_trace_log_profile_events.sh +++ b/tests/queries/0_stateless/02494_trace_log_profile_events.sh @@ -4,19 +4,19 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -query_id=$RANDOM +query_id="$RANDOM-$CLICKHOUSE_DATABASE" ${CLICKHOUSE_CLIENT} --query_id $query_id --query "SELECT 1 FORMAT Null SETTINGS trace_profile_events = 0" ${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" ${CLICKHOUSE_CLIENT} --query "SELECT count() = 0 FROM system.trace_log WHERE query_id = '$query_id' AND trace_type = 'ProfileEvent'" -query_id=$RANDOM +query_id="$RANDOM-$CLICKHOUSE_DATABASE" ${CLICKHOUSE_CLIENT} --query_id $query_id --query "SELECT 1 FORMAT Null SETTINGS trace_profile_events = 1" ${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" ${CLICKHOUSE_CLIENT} --query "SELECT count() > 0, sum(empty(trace)) = 0 FROM system.trace_log WHERE query_id = '$query_id' AND trace_type = 'ProfileEvent'" -query_id=$RANDOM +query_id="$RANDOM-$CLICKHOUSE_DATABASE" ${CLICKHOUSE_CLIENT} --query_id $query_id --query "SELECT count() FROM numbers_mt(1000000) FORMAT Null SETTINGS trace_profile_events = 1" ${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" From 396ecf826becad8a3ab04d141dfa8159590df799 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 25 Nov 2022 12:59:50 +0000 Subject: [PATCH 16/98] fix tests --- src/Common/TraceSender.cpp | 2 +- tests/queries/0_stateless/02494_trace_log_profile_events.sh | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Common/TraceSender.cpp b/src/Common/TraceSender.cpp index 2ff75cdd82c..64d7b2b0eaf 100644 --- a/src/Common/TraceSender.cpp +++ b/src/Common/TraceSender.cpp @@ -14,7 +14,7 @@ namespace /// The performance test query ids can be surprisingly long like /// `aggregating_merge_tree_simple_aggregate_function_string.query100.profile100`, /// so make some allowance for them as well. - constexpr size_t QUERY_ID_MAX_LEN = 128; + constexpr size_t QUERY_ID_MAX_LEN = 100; static_assert(QUERY_ID_MAX_LEN <= std::numeric_limits::max()); } diff --git a/tests/queries/0_stateless/02494_trace_log_profile_events.sh b/tests/queries/0_stateless/02494_trace_log_profile_events.sh index 37fc9ad85bf..58e1c739bc0 100755 --- a/tests/queries/0_stateless/02494_trace_log_profile_events.sh +++ b/tests/queries/0_stateless/02494_trace_log_profile_events.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-tsan CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -44,5 +45,5 @@ FROM ORDER BY event ASC ) ) -WHERE event NOT IN ('ContextLock', 'NetworkSendBytes', 'NetworkSendElapsedMicroseconds'); +WHERE event NOT IN ('ContextLock', 'NetworkSendBytes', 'NetworkSendElapsedMicroseconds', 'QueryProfilerRuns'); " From 38d99c241ff4a09864a577d77f0daa58039c75ac Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 25 Nov 2022 13:53:03 +0000 Subject: [PATCH 17/98] Fix tests --- src/Coordination/tests/gtest_coordination.cpp | 133 +++++++++--------- 1 file changed, 68 insertions(+), 65 deletions(-) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 11b2fd33367..1fc02e83866 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -249,9 +249,9 @@ TEST_P(CoordinationTest, ChangelogTestSimple) namespace { -void waitDurableIndex(nuraft::log_store & log_store, uint64_t index) +void waitDurableLogs(nuraft::log_store & log_store) { - while (log_store.last_durable_index() != index) + while (log_store.last_durable_index() != log_store.next_slot() - 1) std::this_thread::sleep_for(std::chrono::milliseconds(200)); } @@ -263,12 +263,11 @@ TEST_P(CoordinationTest, ChangelogTestFile) ChangelogDirTest test("./logs"); DB::KeeperLogStore changelog("./logs", 5, true, params.enable_compression); changelog.init(1, 0); - uint64_t last_entry_idx = 0; auto entry = getLogEntry("hello world", 77); - last_entry_idx = changelog.append(entry); + changelog.append(entry); changelog.end_of_append_batch(0, 0); - waitDurableIndex(changelog, last_entry_idx); + waitDurableLogs(changelog); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); for (const auto & p : fs::directory_iterator("./logs")) @@ -278,10 +277,10 @@ TEST_P(CoordinationTest, ChangelogTestFile) changelog.append(entry); changelog.append(entry); changelog.append(entry); - last_entry_idx = changelog.append(entry); + changelog.append(entry); changelog.end_of_append_batch(0, 0); - waitDurableIndex(changelog, last_entry_idx); + waitDurableLogs(changelog); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); @@ -294,17 +293,16 @@ TEST_P(CoordinationTest, ChangelogReadWrite) DB::KeeperLogStore changelog("./logs", 1000, true, params.enable_compression); changelog.init(1, 0); - uint64_t last_entry_idx = 0; for (size_t i = 0; i < 10; ++i) { auto entry = getLogEntry("hello world", i * 10); - last_entry_idx = changelog.append(entry); + changelog.append(entry); } changelog.end_of_append_batch(0, 0); EXPECT_EQ(changelog.size(), 10); - waitDurableIndex(changelog, last_entry_idx); + waitDurableLogs(changelog); DB::KeeperLogStore changelog_reader("./logs", 1000, true, params.enable_compression); changelog_reader.init(1, 0); @@ -341,7 +339,7 @@ TEST_P(CoordinationTest, ChangelogWriteAt) changelog.write_at(7, entry); changelog.end_of_append_batch(0, 0); - waitDurableIndex(changelog, 7); + waitDurableLogs(changelog); EXPECT_EQ(changelog.size(), 7); EXPECT_EQ(changelog.last_entry()->get_term(), 77); @@ -364,17 +362,16 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead) ChangelogDirTest test("./logs"); DB::KeeperLogStore changelog("./logs", 5, true, params.enable_compression); changelog.init(1, 0); - uint64_t last_entry_idx = 0; for (size_t i = 0; i < 7; ++i) { auto entry = getLogEntry("hello world", i * 10); - last_entry_idx = changelog.append(entry); + changelog.append(entry); } changelog.end_of_append_batch(0, 0); EXPECT_EQ(changelog.size(), 7); - waitDurableIndex(changelog, last_entry_idx); + waitDurableLogs(changelog); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); @@ -386,12 +383,12 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead) for (size_t i = 7; i < 10; ++i) { auto entry = getLogEntry("hello world", i * 10); - last_entry_idx = changelog_reader.append(entry); + changelog_reader.append(entry); } changelog_reader.end_of_append_batch(0, 0); EXPECT_EQ(changelog_reader.size(), 10); - waitDurableIndex(changelog_reader, last_entry_idx); + waitDurableLogs(changelog_reader); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); @@ -402,11 +399,11 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead) EXPECT_EQ(logs_count, 2); auto entry = getLogEntry("someentry", 77); - last_entry_idx = changelog_reader.append(entry); + changelog_reader.append(entry); changelog_reader.end_of_append_batch(0, 0); EXPECT_EQ(changelog_reader.size(), 11); - waitDurableIndex(changelog_reader, last_entry_idx); + waitDurableLogs(changelog_reader); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); @@ -425,15 +422,14 @@ TEST_P(CoordinationTest, ChangelogTestCompaction) DB::KeeperLogStore changelog("./logs", 5, true, params.enable_compression); changelog.init(1, 0); - uint64_t last_entry_idx = 0; for (size_t i = 0; i < 3; ++i) { auto entry = getLogEntry("hello world", i * 10); - last_entry_idx = changelog.append(entry); + changelog.append(entry); } changelog.end_of_append_batch(0, 0); - waitDurableIndex(changelog, last_entry_idx); + waitDurableLogs(changelog); EXPECT_EQ(changelog.size(), 3); @@ -452,10 +448,10 @@ TEST_P(CoordinationTest, ChangelogTestCompaction) auto e3 = getLogEntry("hello world", 50); changelog.append(e3); auto e4 = getLogEntry("hello world", 60); - last_entry_idx = changelog.append(e4); + changelog.append(e4); changelog.end_of_append_batch(0, 0); - waitDurableIndex(changelog, last_entry_idx); + waitDurableLogs(changelog); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); @@ -486,17 +482,16 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperations) ChangelogDirTest test("./logs"); DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression); changelog.init(1, 0); - uint64_t last_entry_idx = 0; for (size_t i = 0; i < 10; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); - last_entry_idx = changelog.append(entry); + changelog.append(entry); } changelog.end_of_append_batch(0, 0); EXPECT_EQ(changelog.size(), 10); - waitDurableIndex(changelog, last_entry_idx); + waitDurableLogs(changelog); auto entries = changelog.pack(1, 5); @@ -534,17 +529,16 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperationsEmpty) ChangelogDirTest test("./logs"); DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression); changelog.init(1, 0); - uint64_t last_entry_idx = 0; for (size_t i = 0; i < 10; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); - last_entry_idx = changelog.append(entry); + changelog.append(entry); } changelog.end_of_append_batch(0, 0); EXPECT_EQ(changelog.size(), 10); - waitDurableIndex(changelog, last_entry_idx); + waitDurableLogs(changelog); auto entries = changelog.pack(5, 5); @@ -583,15 +577,14 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtPreviousFile) DB::KeeperLogStore changelog("./logs", 5, true, params.enable_compression); changelog.init(1, 0); - uint64_t last_entry_idx = 0; for (size_t i = 0; i < 33; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); - last_entry_idx = changelog.append(entry); + changelog.append(entry); } changelog.end_of_append_batch(0, 0); - waitDurableIndex(changelog, last_entry_idx); + waitDurableLogs(changelog); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); @@ -611,7 +604,7 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtPreviousFile) EXPECT_EQ(changelog.next_slot(), 8); EXPECT_EQ(changelog.last_entry()->get_term(), 5555); - waitDurableIndex(changelog, 7); + waitDurableLogs(changelog); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); @@ -637,15 +630,14 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtFileBorder) DB::KeeperLogStore changelog("./logs", 5, true, params.enable_compression); changelog.init(1, 0); - uint64_t last_entry_idx = 0; for (size_t i = 0; i < 33; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); - last_entry_idx = changelog.append(entry); + changelog.append(entry); } changelog.end_of_append_batch(0, 0); - waitDurableIndex(changelog, last_entry_idx); + waitDurableLogs(changelog); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); @@ -665,7 +657,7 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtFileBorder) EXPECT_EQ(changelog.next_slot(), 12); EXPECT_EQ(changelog.last_entry()->get_term(), 5555); - waitDurableIndex(changelog, 11); + waitDurableLogs(changelog); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); @@ -690,15 +682,14 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtAllFiles) ChangelogDirTest test("./logs"); DB::KeeperLogStore changelog("./logs", 5, true, params.enable_compression); changelog.init(1, 0); - uint64_t last_entry_idx = 0; for (size_t i = 0; i < 33; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); - last_entry_idx = changelog.append(entry); + changelog.append(entry); } changelog.end_of_append_batch(0, 0); - waitDurableIndex(changelog, last_entry_idx); + waitDurableLogs(changelog); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); @@ -718,7 +709,7 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtAllFiles) EXPECT_EQ(changelog.next_slot(), 2); EXPECT_EQ(changelog.last_entry()->get_term(), 5555); - waitDurableIndex(changelog, 1); + waitDurableLogs(changelog); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); @@ -737,16 +728,15 @@ TEST_P(CoordinationTest, ChangelogTestStartNewLogAfterRead) DB::KeeperLogStore changelog("./logs", 5, true, params.enable_compression); changelog.init(1, 0); - uint64_t last_entry_idx = 0; for (size_t i = 0; i < 35; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); - last_entry_idx = changelog.append(entry); + changelog.append(entry); } changelog.end_of_append_batch(0, 0); EXPECT_EQ(changelog.size(), 35); - waitDurableIndex(changelog, last_entry_idx); + waitDurableLogs(changelog); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); @@ -760,12 +750,12 @@ TEST_P(CoordinationTest, ChangelogTestStartNewLogAfterRead) changelog_reader.init(1, 0); auto entry = getLogEntry("36_hello_world", 360); - last_entry_idx = changelog_reader.append(entry); + changelog_reader.append(entry); changelog_reader.end_of_append_batch(0, 0); EXPECT_EQ(changelog_reader.size(), 36); - waitDurableIndex(changelog_reader, last_entry_idx); + waitDurableLogs(changelog_reader); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); @@ -804,16 +794,15 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) DB::KeeperLogStore changelog(log_folder, 5, true, params.enable_compression); changelog.init(1, 0); - uint64_t last_entry_idx = 0; for (size_t i = 0; i < 35; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); - last_entry_idx = changelog.append(entry); + changelog.append(entry); } changelog.end_of_append_batch(0, 0); EXPECT_EQ(changelog.size(), 35); - waitDurableIndex(changelog, last_entry_idx); + waitDurableLogs(changelog); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); @@ -842,12 +831,12 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) assertBrokenLogRemoved(log_folder, "changelog_31_35.bin" + params.extension); auto entry = getLogEntry("h", 7777); - last_entry_idx = changelog_reader.append(entry); + changelog_reader.append(entry); changelog_reader.end_of_append_batch(0, 0); EXPECT_EQ(changelog_reader.size(), 11); EXPECT_EQ(changelog_reader.last_entry()->get_term(), 7777); - waitDurableIndex(changelog_reader, last_entry_idx); + waitDurableLogs(changelog_reader); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); @@ -872,15 +861,14 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) DB::KeeperLogStore changelog("./logs", 20, true, params.enable_compression); changelog.init(1, 0); - uint64_t last_entry_idx = 0; for (size_t i = 0; i < 35; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", (i + 44) * 10); - last_entry_idx = changelog.append(entry); + changelog.append(entry); } changelog.end_of_append_batch(0, 0); - waitDurableIndex(changelog, last_entry_idx); + waitDurableLogs(changelog); EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_21_40.bin" + params.extension)); @@ -894,10 +882,10 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin" + params.extension)); assertBrokenLogRemoved("./logs", "changelog_21_40.bin" + params.extension); auto entry = getLogEntry("hello_world", 7777); - last_entry_idx = changelog_reader.append(entry); + changelog_reader.append(entry); changelog_reader.end_of_append_batch(0, 0); - waitDurableIndex(changelog_reader, last_entry_idx); + waitDurableLogs(changelog_reader); EXPECT_EQ(changelog_reader.size(), 1); EXPECT_EQ(changelog_reader.last_entry()->get_term(), 7777); @@ -916,15 +904,14 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles) DB::KeeperLogStore changelog("./logs", 20, true, params.enable_compression); changelog.init(1, 0); - uint64_t last_entry_idx = 0; for (size_t i = 0; i < 35; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", (i + 44) * 10); - last_entry_idx = changelog.append(entry); + changelog.append(entry); } changelog.end_of_append_batch(0, 0); - waitDurableIndex(changelog, last_entry_idx); + waitDurableLogs(changelog); EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_21_40.bin" + params.extension)); @@ -944,15 +931,14 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles2) DB::KeeperLogStore changelog("./logs", 10, true, params.enable_compression); changelog.init(1, 0); - uint64_t last_entry_idx = 0; for (size_t i = 0; i < 35; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", (i + 44) * 10); - last_entry_idx = changelog.append(entry); + changelog.append(entry); } changelog.end_of_append_batch(0, 0); - waitDurableIndex(changelog, last_entry_idx); + waitDurableLogs(changelog); EXPECT_TRUE(fs::exists("./logs/changelog_1_10.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_11_20.bin" + params.extension)); @@ -1402,16 +1388,15 @@ void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, uint state_machine->init(); DB::KeeperLogStore changelog("./logs", settings->rotate_log_storage_interval, true, enable_compression); changelog.init(state_machine->last_commit_index() + 1, settings->reserved_log_items); - uint64_t last_entry_idx = 0; for (size_t i = 1; i < total_logs + 1; ++i) { std::shared_ptr request = std::make_shared(); request->path = "/hello_" + std::to_string(i); auto entry = getLogEntryFromZKRequest(0, 1, i, request); - last_entry_idx = changelog.append(entry); + changelog.append(entry); changelog.end_of_append_batch(0, 0); - waitDurableIndex(changelog, last_entry_idx); + waitDurableLogs(changelog); state_machine->pre_commit(i, changelog.entry_at(i)->get_buf()); state_machine->commit(i, changelog.entry_at(i)->get_buf()); @@ -1594,6 +1579,8 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) changelog.append(entry); changelog.end_of_append_batch(0, 0); } + + waitDurableLogs(changelog); } @@ -1610,6 +1597,8 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) changelog_1.end_of_append_batch(0, 0); } + waitDurableLogs(changelog_1); + EXPECT_TRUE(fs::exists("./logs/changelog_1_100.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_101_110.bin" + params.extension)); @@ -1625,6 +1614,8 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) changelog_2.end_of_append_batch(0, 0); } + waitDurableLogs(changelog_2); + changelog_2.compact(105); std::this_thread::sleep_for(std::chrono::microseconds(1000)); @@ -1645,6 +1636,8 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) changelog_3.end_of_append_batch(0, 0); } + waitDurableLogs(changelog_3); + changelog_3.compact(125); std::this_thread::sleep_for(std::chrono::microseconds(1000)); EXPECT_FALSE(fs::exists("./logs/changelog_101_110.bin" + params.extension)); @@ -1692,6 +1685,7 @@ TEST_P(CoordinationTest, TestCompressedLogsMultipleRewrite) changelog.end_of_append_batch(0, 0); } + waitDurableLogs(changelog); DB::KeeperLogStore changelog1("./logs", 100, true, test_params.enable_compression); changelog1.init(0, 3); @@ -1773,6 +1767,7 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) changelog.append(entry); changelog.end_of_append_batch(0, 0); EXPECT_EQ(changelog.next_slot(), 2); + waitDurableLogs(changelog); } { @@ -1783,6 +1778,7 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) changelog.append(entry); changelog.end_of_append_batch(0, 0); EXPECT_EQ(changelog.next_slot(), 3); + waitDurableLogs(changelog); } { @@ -1793,6 +1789,7 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) changelog.append(entry); changelog.end_of_append_batch(0, 0); EXPECT_EQ(changelog.next_slot(), 4); + waitDurableLogs(changelog); } { @@ -1803,6 +1800,7 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) changelog.append(entry); changelog.end_of_append_batch(0, 0); EXPECT_EQ(changelog.next_slot(), 5); + waitDurableLogs(changelog); } } @@ -1822,6 +1820,7 @@ TEST_P(CoordinationTest, ChangelogInsertMultipleTimesSmooth) changelog.append(entry); } changelog.end_of_append_batch(0, 0); + waitDurableLogs(changelog); } DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression); @@ -1840,6 +1839,7 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard) changelog1.append(entry); changelog1.end_of_append_batch(0, 0); EXPECT_EQ(changelog1.next_slot(), 2); + waitDurableLogs(changelog1); std::cerr << "================Second time=====================\n"; DB::KeeperLogStore changelog2("./logs", 100, true, params.enable_compression); @@ -1848,6 +1848,7 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard) changelog2.append(entry); changelog2.end_of_append_batch(0, 0); EXPECT_EQ(changelog2.next_slot(), 3); + waitDurableLogs(changelog2); std::cerr << "================Third time=====================\n"; DB::KeeperLogStore changelog3("./logs", 100, true, params.enable_compression); @@ -1856,6 +1857,7 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard) changelog3.append(entry); changelog3.end_of_append_batch(0, 0); EXPECT_EQ(changelog3.next_slot(), 4); + waitDurableLogs(changelog3); std::cerr << "================Fourth time=====================\n"; DB::KeeperLogStore changelog4("./logs", 100, true, params.enable_compression); @@ -1864,6 +1866,7 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard) changelog4.append(entry); changelog4.end_of_append_batch(0, 0); EXPECT_EQ(changelog4.next_slot(), 5); + waitDurableLogs(changelog4); } TEST_P(CoordinationTest, TestStorageSnapshotEqual) From e91314d57f037a21221ab7b7d0b436e5e7f759fb Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 28 Nov 2022 08:44:46 +0000 Subject: [PATCH 18/98] Use initialized flag --- src/Coordination/Changelog.cpp | 4 +++- src/Coordination/Changelog.h | 2 ++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 3d01f98f5b1..68c45326b15 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -431,6 +431,8 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin /// Start new log if we don't initialize writer from previous log. All logs can be "complete". if (!current_writer) rotate(max_log_id + 1); + + initialized = true; } @@ -589,7 +591,7 @@ void Changelog::writeThread() void Changelog::appendEntry(uint64_t index, const LogEntryPtr & log_entry) { - if (!current_writer) + if (!initialized) throw Exception(ErrorCodes::LOGICAL_ERROR, "Changelog must be initialized before appending records"); if (logs.empty()) diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 60ebaf5994e..fc85971ee94 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -210,6 +210,8 @@ private: uint64_t last_durable_idx{0}; nuraft::wptr raft_server; + + bool initialized = false; }; } From 4ea833afa1f2abae7dc91e52b9637d39ad5c6e31 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 28 Nov 2022 17:29:03 +0100 Subject: [PATCH 19/98] Fix bug with projections and zero-copy replication --- .../MergeTree/DataPartStorageOnDisk.cpp | 11 ++- src/Storages/MergeTree/IDataPartStorage.h | 3 + src/Storages/MergeTree/MergeTreeData.cpp | 18 +++++ ...ction_and_mutation_work_together.reference | 4 + ..._projection_and_mutation_work_together.sql | 79 +++++++++++++++++++ 5 files changed, 114 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.reference create mode 100644 tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.sql diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index e68a04fd9bc..347ea16950e 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -101,6 +101,7 @@ public: bool isValid() const override { return it->isValid(); } bool isFile() const override { return isValid() && disk->isFile(it->path()); } std::string name() const override { return it->name(); } + std::string path() const override { return it->path(); } private: DiskPtr disk; @@ -259,9 +260,17 @@ void DataPartStorageOnDisk::remove( std::string proj_dir_name = projection.name + proj_suffix; projection_directories.emplace(proj_dir_name); + NameSet files_not_to_remove_for_projection; + for (const auto & file_name : can_remove_description->files_not_to_remove) + { + if (file_name.starts_with(proj_dir_name)) + files_not_to_remove_for_projection.emplace(fs::path(file_name).filename()); + } + LOG_DEBUG(log, "Will not remove files [{}] for projection {}", fmt::join(files_not_to_remove_for_projection, ", "), projection.name); + clearDirectory( fs::path(to) / proj_dir_name, - can_remove_description->can_remove_anything, can_remove_description->files_not_to_remove, projection.checksums, {}, is_temp, state, log, true); + can_remove_description->can_remove_anything, files_not_to_remove_for_projection, projection.checksums, {}, is_temp, state, log, true); } /// It is possible that we are removing the part which have a written but not loaded projection. diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index c6669908db4..53ee2738fc6 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -39,6 +39,9 @@ public: /// Name of the file that the iterator currently points to. virtual std::string name() const = 0; + /// Path of the file that the iterator currently points to. + virtual std::string path() const = 0; + virtual ~IDataPartStorageIterator() = default; }; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 088821c6188..534ad9bd972 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6412,7 +6412,25 @@ std::pair MergeTreeData::cloneAn if (!files_to_copy_instead_of_hardlinks.contains(it->name()) && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) + { hardlinked_files->hardlinks_from_source_part.insert(it->name()); + } + } + + auto projections = src_part->getProjectionParts(); + for (const auto & [name, projection_part] : projections) + { + const auto & projection_storage = projection_part->getDataPartStorage(); + for (auto it = projection_storage.iterate(); it->isValid(); it->next()) + { + auto file_name_with_projection_prefix = fs::path(projection_storage.getPartDirectory()) / it->name(); + if (!files_to_copy_instead_of_hardlinks.contains(file_name_with_projection_prefix) + && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME + && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) + { + hardlinked_files->hardlinks_from_source_part.insert(file_name_with_projection_prefix); + } + } } } diff --git a/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.reference b/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.reference new file mode 100644 index 00000000000..726e74146fc --- /dev/null +++ b/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.reference @@ -0,0 +1,4 @@ +199 +199 +1990 199 +1990 199 diff --git a/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.sql b/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.sql new file mode 100644 index 00000000000..7a51d86dd30 --- /dev/null +++ b/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.sql @@ -0,0 +1,79 @@ +DROP TABLE IF EXISTS wikistat1; +DROP TABLE IF EXISTS wikistat2; + +CREATE TABLE wikistat1 +( + time DateTime, + project LowCardinality(String), + subproject LowCardinality(String), + path String, + hits UInt64, + PROJECTION total + ( + SELECT + project, + subproject, + path, + sum(hits), + count() + GROUP BY + project, + subproject, + path + ) +) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/02494_zero_copy_and_projection', '1') +ORDER BY (path, time) +SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0, allow_remote_fs_zero_copy_replication=1, min_bytes_for_wide_part=0; + +CREATE TABLE wikistat2 +( + time DateTime, + project LowCardinality(String), + subproject LowCardinality(String), + path String, + hits UInt64, + PROJECTION total + ( + SELECT + project, + subproject, + path, + sum(hits), + count() + GROUP BY + project, + subproject, + path + ) +) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/02494_zero_copy_and_projection', '2') +ORDER BY (path, time) +SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0, allow_remote_fs_zero_copy_replication=1, min_bytes_for_wide_part=0; + +INSERT INTO wikistat1 SELECT toDateTime('2020-10-01 00:00:00'), 'hello', 'world', '/data/path', 10 from numbers(100); + +INSERT INTO wikistat1 SELECT toDateTime('2020-10-01 00:00:00'), 'hello', 'world', '/data/path', 10 from numbers(99, 99); + +SYSTEM SYNC REPLICA wikistat2; + +SELECT COUNT() from wikistat1 WHERE NOT ignore(*); +SELECT COUNT() from wikistat2 WHERE NOT ignore(*); + +SYSTEM STOP REPLICATION QUEUES wikistat2; + +ALTER TABLE wikistat1 DELETE where time = toDateTime('2022-12-20 00:00:00') SETTINGS mutations_sync = 1; + +SYSTEM START REPLICATION QUEUES wikistat2; + +SYSTEM SYNC REPLICA wikistat2; + +-- it doesn't make test flaky, rarely we will not delete the parts because of cleanup thread was slow. +-- Such condition will lead to successful queries. +SELECT 0 FROM numbers(5) WHERE sleepEachRow(1) = 1; + +select sum(hits), count() from wikistat1 GROUP BY project, subproject, path settings allow_experimental_projection_optimization = 1, force_optimize_projection = 1; +select sum(hits), count() from wikistat2 GROUP BY project, subproject, path settings allow_experimental_projection_optimization = 1, force_optimize_projection = 1; + +DROP TABLE wikistat1; +DROP TABLE wikistat2; From a5c396ade9a78c287991a78f360681efa9a6d71c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 29 Nov 2022 09:33:42 +0000 Subject: [PATCH 20/98] Make better --- .../UserDefined/UserDefinedSQLFunctionVisitor.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp b/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp index 0a5aa657a89..d78a8623a18 100644 --- a/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp @@ -31,7 +31,17 @@ void UserDefinedSQLFunctionVisitor::visit(ASTPtr & ast) auto * old_value = child.get(); visit(child); - ast->setOrReplace(old_value, child); + + // child did not change + if (old_value == child.get()) + return; + + // child changed, we need to modify it in the list of children of the parent also + for (auto & current_child : ast->children) + { + if (current_child.get() == old_value) + current_child = child; + } }; if (auto * col_decl = ast->as()) From 7914e2674470bb3aa85efa3b91d9ee93f023d8cd Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 29 Nov 2022 12:50:20 +0100 Subject: [PATCH 21/98] Fix another part of code --- src/Storages/MergeTree/MutateTask.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index e5ba771a198..916befdf78f 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1322,9 +1322,11 @@ private: for (auto p_it = projection_data_part_storage_src->iterate(); p_it->isValid(); p_it->next()) { + auto file_name_with_projection_prefix = fs::path(projection_data_part_storage_src->getPartDirectory()) / it->name(); projection_data_part_storage_dst->createHardLinkFrom( *projection_data_part_storage_src, p_it->name(), p_it->name()); - hardlinked_files.insert(p_it->name()); + + hardlinked_files.insert(file_name_with_projection_prefix); } } } From c058f07fabb37d379ffc02f88e75029e0b8ba715 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 10 Nov 2022 16:40:52 +0100 Subject: [PATCH 22/98] Add mypy config, fix git_helper.py --- tests/ci/.mypy.ini | 16 ++++++++++++++++ tests/ci/git_helper.py | 29 ++++++++++++++++------------- 2 files changed, 32 insertions(+), 13 deletions(-) create mode 100644 tests/ci/.mypy.ini diff --git a/tests/ci/.mypy.ini b/tests/ci/.mypy.ini new file mode 100644 index 00000000000..7326675067c --- /dev/null +++ b/tests/ci/.mypy.ini @@ -0,0 +1,16 @@ +[mypy] +warn_no_return = False +warn_unused_configs = True +disallow_subclassing_any = True +disallow_untyped_calls = False +disallow_untyped_defs = False +disallow_incomplete_defs = True +check_untyped_defs = True +disallow_untyped_decorators = True +no_implicit_optional = True +warn_redundant_casts = True +warn_unused_ignores = True +warn_return_any = True +no_implicit_reexport = True +strict_equality = True +strict_concatenate = True diff --git a/tests/ci/git_helper.py b/tests/ci/git_helper.py index 77c2fc9cf05..eb5e835eab3 100644 --- a/tests/ci/git_helper.py +++ b/tests/ci/git_helper.py @@ -4,7 +4,7 @@ import logging import os.path as p import re import subprocess -from typing import List, Optional +from typing import Any, List, Optional logger = logging.getLogger(__name__) @@ -21,19 +21,19 @@ TWEAK = 1 # Py 3.8 removeprefix and removesuffix -def removeprefix(string: str, prefix: str): +def removeprefix(string: str, prefix: str) -> str: if string.startswith(prefix): return string[len(prefix) :] # noqa: ignore E203, false positive return string -def removesuffix(string: str, suffix: str): +def removesuffix(string: str, suffix: str) -> str: if string.endswith(suffix): return string[: -len(suffix)] return string -def commit(name: str): +def commit(name: str) -> str: r = re.compile(SHA_REGEXP) if not r.match(name): raise argparse.ArgumentTypeError( @@ -42,7 +42,7 @@ def commit(name: str): return name -def release_branch(name: str): +def release_branch(name: str) -> str: r = re.compile(RELEASE_BRANCH_REGEXP) if not r.match(name): raise argparse.ArgumentTypeError("release branch should be as 12.1") @@ -55,20 +55,23 @@ class Runner: def __init__(self, cwd: str = CWD): self._cwd = cwd - def run(self, cmd: str, cwd: Optional[str] = None, **kwargs) -> str: + def run(self, cmd: str, cwd: Optional[str] = None, **kwargs: Any) -> str: if cwd is None: cwd = self.cwd logger.debug("Running command: %s", cmd) - return subprocess.check_output( - cmd, shell=True, cwd=cwd, encoding="utf-8", **kwargs - ).strip() + output = str( + subprocess.check_output( + cmd, shell=True, cwd=cwd, encoding="utf-8", **kwargs + ).strip() + ) + return output @property def cwd(self) -> str: return self._cwd @cwd.setter - def cwd(self, value: str): + def cwd(self, value: str) -> None: # Set _cwd only once, then set it to readonly if self._cwd != CWD: return @@ -139,7 +142,7 @@ class Git: ) @staticmethod - def check_tag(value: str): + def check_tag(value: str) -> None: if value == "": return if not Git._tag_pattern.match(value): @@ -150,7 +153,7 @@ class Git: return self._latest_tag @latest_tag.setter - def latest_tag(self, value: str): + def latest_tag(self, value: str) -> None: self.check_tag(value) self._latest_tag = value @@ -159,7 +162,7 @@ class Git: return self._new_tag @new_tag.setter - def new_tag(self, value: str): + def new_tag(self, value: str) -> None: self.check_tag(value) self._new_tag = value From 9d173ca3696be5e60ec77732d890c326e99bf154 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 10 Nov 2022 16:47:44 +0100 Subject: [PATCH 23/98] Fix version_helper.py --- tests/ci/version_helper.py | 27 ++++++++++++++++----------- 1 file changed, 16 insertions(+), 11 deletions(-) diff --git a/tests/ci/version_helper.py b/tests/ci/version_helper.py index 162bab6a50a..dc4674b3699 100755 --- a/tests/ci/version_helper.py +++ b/tests/ci/version_helper.py @@ -2,9 +2,9 @@ import logging import os.path as p from argparse import ArgumentParser, ArgumentDefaultsHelpFormatter, ArgumentTypeError -from typing import Dict, List, Optional, Tuple, Union +from typing import Any, Dict, List, Literal, Optional, Tuple, Union -from git_helper import TWEAK, Git, get_tags, git_runner, removeprefix +from git_helper import TWEAK, Git as Git, get_tags, git_runner, removeprefix FILE_WITH_VERSION_PATH = "cmake/autogenerated_versions.txt" CHANGELOG_IN_PATH = "debian/changelog.in" @@ -45,7 +45,7 @@ class ClickHouseVersion: patch: Union[int, str], revision: Union[int, str], git: Optional[Git], - tweak: str = None, + tweak: Optional[str] = None, ): self._major = int(major) self._minor = int(minor) @@ -59,10 +59,15 @@ class ClickHouseVersion: self._tweak = self._git.tweak self._describe = "" - def update(self, part: str) -> "ClickHouseVersion": + def update(self, part: Literal["major", "minor", "patch"]) -> "ClickHouseVersion": """If part is valid, returns a new version""" - method = getattr(self, f"{part}_update") - return method() + if part == "major": + return self.major_update() + if part == "minor": + return self.minor_update() + if part == "patch": + return self.patch_update() + raise KeyError(f"wrong part {part} is used") def major_update(self) -> "ClickHouseVersion": if self._git is not None: @@ -139,10 +144,10 @@ class ClickHouseVersion: raise ValueError(f"version type {version_type} not in {VersionType.VALID}") self._describe = f"v{self.string}-{version_type}" - def __eq__(self, other) -> bool: + def __eq__(self, other: Any) -> bool: if not isinstance(self, type(other)): return NotImplemented - return ( + return ( # type: ignore self.major == other.major and self.minor == other.minor and self.patch == other.patch @@ -170,7 +175,7 @@ class VersionType: VALID = (TESTING, PRESTABLE, STABLE, LTS) -def validate_version(version: str): +def validate_version(version: str) -> None: parts = version.split(".") if len(parts) != 4: raise ValueError(f"{version} does not contain 4 parts") @@ -259,7 +264,7 @@ def get_tagged_versions() -> List[ClickHouseVersion]: def update_cmake_version( version: ClickHouseVersion, versions_path: str = FILE_WITH_VERSION_PATH, -): +) -> None: path_to_file = get_abs_path(versions_path) with open(path_to_file, "w", encoding="utf-8") as f: f.write(VERSIONS_TEMPLATE.format_map(version.as_dict())) @@ -269,7 +274,7 @@ def update_contributors( relative_contributors_path: str = GENERATED_CONTRIBUTORS, force: bool = False, raise_error: bool = False, -): +) -> None: # Check if we have shallow checkout by comparing number of lines # '--is-shallow-repository' is in git since 2.15, 2017-10-30 if git_runner.run("git rev-parse --is-shallow-repository") == "true" and not force: From 78e560f78e83c49c82ecae25a2e6338903db340f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 10 Nov 2022 16:57:01 +0100 Subject: [PATCH 24/98] Fix docker_image_check and docker_test --- tests/ci/docker_images_check.py | 6 +++--- tests/ci/docker_test.py | 20 ++++++++++---------- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 873aee9aabf..0618969f94c 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -8,7 +8,7 @@ import shutil import subprocess import time import sys -from typing import Dict, List, Optional, Set, Tuple, Union +from typing import Any, Dict, List, Optional, Set, Tuple, Union from github import Github @@ -52,7 +52,7 @@ class DockerImage: and self.only_amd64 == other.only_amd64 ) - def __lt__(self, other) -> bool: + def __lt__(self, other: Any) -> bool: if not isinstance(other, DockerImage): return False if self.parent and not other.parent: @@ -270,7 +270,7 @@ def build_and_push_one_image( def process_single_image( image: DockerImage, versions: List[str], - additional_cache, + additional_cache: str, push: bool, child: bool, ) -> List[Tuple[str, str, str]]: diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index 1848300e2f6..8b18a580ed7 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -43,55 +43,55 @@ class TestDockerImageCheck(unittest.TestCase): "docker/test/stateless", "clickhouse/stateless-test", False, - "clickhouse/test-base", + "clickhouse/test-base", # type: ignore ), di.DockerImage( "docker/test/integration/base", "clickhouse/integration-test", False, - "clickhouse/test-base", + "clickhouse/test-base", # type: ignore ), di.DockerImage( "docker/test/fuzzer", "clickhouse/fuzzer", False, - "clickhouse/test-base", + "clickhouse/test-base", # type: ignore ), di.DockerImage( "docker/test/keeper-jepsen", "clickhouse/keeper-jepsen-test", False, - "clickhouse/test-base", + "clickhouse/test-base", # type: ignore ), di.DockerImage( "docker/docs/check", "clickhouse/docs-check", False, - "clickhouse/docs-builder", + "clickhouse/docs-builder", # type: ignore ), di.DockerImage( "docker/docs/release", "clickhouse/docs-release", False, - "clickhouse/docs-builder", + "clickhouse/docs-builder", # type: ignore ), di.DockerImage( "docker/test/stateful", "clickhouse/stateful-test", False, - "clickhouse/stateless-test", + "clickhouse/stateless-test", # type: ignore ), di.DockerImage( "docker/test/unit", "clickhouse/unit-test", False, - "clickhouse/stateless-test", + "clickhouse/stateless-test", # type: ignore ), di.DockerImage( "docker/test/stress", "clickhouse/stress-test", False, - "clickhouse/stateful-test", + "clickhouse/stateful-test", # type: ignore ), ] ) @@ -277,7 +277,7 @@ class TestDockerServer(unittest.TestCase): ds.gen_tags(version, "auto") @patch("docker_server.get_tagged_versions") - def test_auto_release_type(self, mock_tagged_versions: MagicMock): + def test_auto_release_type(self, mock_tagged_versions: MagicMock) -> None: mock_tagged_versions.return_value = [ get_version_from_string("1.1.1.1"), get_version_from_string("1.2.1.1"), From 9dff6a80ab5303a7b33a93b1dbb0100c71591a12 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 10 Nov 2022 17:11:23 +0100 Subject: [PATCH 25/98] Fix run_check.py and dependencies --- tests/ci/build_download_helper.py | 12 ++++----- tests/ci/commit_status_helper.py | 16 ++++++------ tests/ci/get_robot_token.py | 26 ++++++++++++++----- tests/ci/pr_info.py | 6 ++--- tests/ci/run_check.py | 2 +- tests/ci/workflow_approve_rerun_lambda/app.py | 4 +-- 6 files changed, 39 insertions(+), 27 deletions(-) diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index 58997bed253..1a2fdedefed 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -5,7 +5,7 @@ import logging import os import sys import time -from typing import List, Optional +from typing import Any, List, Optional import requests # type: ignore @@ -18,7 +18,7 @@ def get_with_retries( url: str, retries: int = DOWNLOAD_RETRIES_COUNT, sleep: int = 3, - **kwargs, + **kwargs: Any, ) -> requests.Response: logging.info( "Getting URL with %i tries and sleep %i in between: %s", retries, sleep, url @@ -41,18 +41,18 @@ def get_with_retries( return response -def get_build_name_for_check(check_name) -> str: - return CI_CONFIG["tests_config"][check_name]["required_build"] +def get_build_name_for_check(check_name: str) -> str: + return CI_CONFIG["tests_config"][check_name]["required_build"] # type: ignore -def read_build_urls(build_name, reports_path) -> List[str]: +def read_build_urls(build_name: str, reports_path: str) -> List[str]: for root, _, files in os.walk(reports_path): for f in files: if build_name in f: logging.info("Found build report json %s", f) with open(os.path.join(root, f), "r", encoding="utf-8") as file_handler: build_report = json.load(file_handler) - return build_report["build_urls"] + return build_report["build_urls"] # type: ignore return [] diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 185dc64daa9..785250c3904 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -17,7 +17,7 @@ RETRY = 5 CommitStatuses = List[CommitStatus] -def override_status(status: str, check_name: str, invert=False) -> str: +def override_status(status: str, check_name: str, invert: bool = False) -> str: if CI_CONFIG["tests_config"].get(check_name, {}).get("force_tests", False): return "success" @@ -45,7 +45,7 @@ def get_commit(gh: Github, commit_sha: str, retry_count: int = RETRY) -> Commit: def post_commit_status( gh: Github, sha: str, check_name: str, description: str, state: str, report_url: str -): +) -> None: for i in range(RETRY): try: commit = get_commit(gh, sha, 1) @@ -64,7 +64,7 @@ def post_commit_status( def post_commit_status_to_file( file_path: str, description: str, state: str, report_url: str -): +) -> None: if os.path.exists(file_path): raise Exception(f'File "{file_path}" already exists!') with open(file_path, "w", encoding="utf-8") as f: @@ -88,21 +88,21 @@ def get_commit_filtered_statuses(commit: Commit) -> CommitStatuses: return list(filtered.values()) -def remove_labels(gh: Github, pr_info: PRInfo, labels_names: List[str]): +def remove_labels(gh: Github, pr_info: PRInfo, labels_names: List[str]) -> None: repo = gh.get_repo(GITHUB_REPOSITORY) pull_request = repo.get_pull(pr_info.number) for label in labels_names: pull_request.remove_from_labels(label) -def post_labels(gh: Github, pr_info: PRInfo, labels_names: List[str]): +def post_labels(gh: Github, pr_info: PRInfo, labels_names: List[str]) -> None: repo = gh.get_repo(GITHUB_REPOSITORY) pull_request = repo.get_pull(pr_info.number) for label in labels_names: pull_request.add_to_labels(label) -def fail_mergeable_check(commit: Commit, description: str): +def fail_mergeable_check(commit: Commit, description: str) -> None: commit.create_status( context="Mergeable Check", description=description, @@ -111,7 +111,7 @@ def fail_mergeable_check(commit: Commit, description: str): ) -def reset_mergeable_check(commit: Commit, description: str = ""): +def reset_mergeable_check(commit: Commit, description: str = "") -> None: commit.create_status( context="Mergeable Check", description=description, @@ -120,7 +120,7 @@ def reset_mergeable_check(commit: Commit, description: str = ""): ) -def update_mergeable_check(gh: Github, pr_info: PRInfo, check_name: str): +def update_mergeable_check(gh: Github, pr_info: PRInfo, check_name: str) -> None: if SKIP_MERGEABLE_CHECK_LABEL in pr_info.labels: return diff --git a/tests/ci/get_robot_token.py b/tests/ci/get_robot_token.py index 163e1ce071e..6ecaf468ed1 100644 --- a/tests/ci/get_robot_token.py +++ b/tests/ci/get_robot_token.py @@ -1,8 +1,17 @@ #!/usr/bin/env python3 import logging +from dataclasses import dataclass import boto3 # type: ignore -from github import Github # type: ignore +from github import Github +from github.AuthenticatedUser import AuthenticatedUser + + +@dataclass +class Token: + user: AuthenticatedUser + value: str + rest: int def get_parameter_from_ssm(name, decrypt=True, client=None): @@ -19,7 +28,7 @@ def get_best_robot_token(token_prefix_env_name="github_robot_token_"): ] )["Parameters"] assert parameters - token = {"login": "", "value": "", "rest": 0} + token = None for token_name in [p["Name"] for p in parameters]: value = get_parameter_from_ssm(token_name, True, client) @@ -29,12 +38,15 @@ def get_best_robot_token(token_prefix_env_name="github_robot_token_"): user = gh.get_user() rest, _ = gh.rate_limiting logging.info("Get token with %s remaining requests", rest) - if token["rest"] < rest: - token = {"user": user, "value": value, "rest": rest} + if token is None: + token = Token(user, value, rest) + continue + if token.rest < rest: + token.user, token.value, token.rest = user, value, rest - assert token["value"] + assert token logging.info( - "User %s with %s remaining requests is used", token["user"].login, token["rest"] + "User %s with %s remaining requests is used", token.user.login, token.rest ) - return token["value"] + return token.value diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 6a2fac0a291..f0a371c488f 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -146,7 +146,7 @@ class PRInfo: self.body = github_event["pull_request"]["body"] self.labels = { label["name"] for label in github_event["pull_request"]["labels"] - } + } # type: Set[str] self.user_login = github_event["pull_request"]["user"]["login"] self.user_orgs = set([]) @@ -178,7 +178,7 @@ class PRInfo: if pull_request is None or pull_request["state"] == "closed": # it's merged PR to master self.number = 0 - self.labels = {} + self.labels = set() self.pr_html_url = f"{repo_prefix}/commits/{ref}" self.base_ref = ref self.base_name = self.repo_full_name @@ -228,7 +228,7 @@ class PRInfo: print(json.dumps(github_event, sort_keys=True, indent=4)) self.sha = os.getenv("GITHUB_SHA") self.number = 0 - self.labels = {} + self.labels = set() repo_prefix = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}" self.task_url = GITHUB_RUN_URL self.commit_html_url = f"{repo_prefix}/commits/{self.sha}" diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 39dbc938c8f..7119f443719 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -112,7 +112,7 @@ def should_run_checks_for_pr(pr_info: PRInfo) -> Tuple[bool, str, str]: return True, "No special conditions apply", "pending" -def check_pr_description(pr_info) -> Tuple[str, str]: +def check_pr_description(pr_info: PRInfo) -> Tuple[str, str]: lines = list( map(lambda x: x.strip(), pr_info.body.split("\n") if pr_info.body else []) ) diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index 23e808b0861..d285e29943d 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -313,7 +313,7 @@ def check_suspicious_changed_files(changed_files): return False -def approve_run(workflow_description: WorkflowDescription, token): +def approve_run(workflow_description: WorkflowDescription, token: str) -> None: url = f"{workflow_description.api_url}/approve" _exec_post_with_retry(url, token) @@ -391,7 +391,7 @@ def rerun_workflow(workflow_description, token): def check_workflow_completed( - event_data, workflow_description: WorkflowDescription, token: str + event_data: dict, workflow_description: WorkflowDescription, token: str ) -> bool: if workflow_description.action == "completed": attempt = 0 From 080ff97627da25d88db072aa0ddab7f472ffc4ca Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 14 Nov 2022 15:47:09 +0100 Subject: [PATCH 26/98] Fix mypy errors in report.py --- tests/ci/report.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index a6700f50dfc..6a29aed9913 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -229,8 +229,8 @@ def create_test_html_report( if has_test_logs and not with_raw_logs: headers.append("Logs") - headers = "".join(["" + h + "" for h in headers]) - test_part = HTML_TEST_PART.format(headers=headers, rows=rows_part) + headers_html = "".join(["" + h + "" for h in headers]) + test_part = HTML_TEST_PART.format(headers=headers_html, rows=rows_part) else: test_part = "" @@ -341,7 +341,7 @@ def create_build_html_report( if build_result.elapsed_seconds: delta = datetime.timedelta(seconds=build_result.elapsed_seconds) else: - delta = "unknown" + delta = "unknown" # type: ignore row += "{}".format(str(delta)) From 324c8e4b127a63d03e835d2387e5811b7fb00154 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 14 Nov 2022 15:48:51 +0100 Subject: [PATCH 27/98] Use f-strings in report.py --- tests/ci/report.py | 36 ++++++++++++++---------------------- 1 file changed, 14 insertions(+), 22 deletions(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index 6a29aed9913..2904a5519a9 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -101,7 +101,7 @@ def _format_header(header, branch_name, branch_url=None): result = "ClickHouse " + result result += " for " if branch_url: - result += '{name}'.format(url=branch_url, name=branch_name) + result += f'{branch_name}' else: result += branch_name return result @@ -140,9 +140,7 @@ def _get_html_url(url): if isinstance(url, tuple): href, name = url[0], _get_html_url_name(url) if href and name: - return '{name}'.format( - href=href, name=_get_html_url_name(url) - ) + return f'{_get_html_url_name(url)}' return "" @@ -199,13 +197,7 @@ def create_test_html_report( num_fails = num_fails + 1 is_fail_id = 'id="fail' + str(num_fails) + '" ' - row += ( - "'.format(style) - + test_status - + "" - ) + row += f'{test_status}' if test_time is not None: row += "" + test_time + "" @@ -317,33 +309,33 @@ def create_build_html_report( build_results, build_logs_urls, artifact_urls_list ): row = "" - row += "{}".format(build_result.compiler) + row += f"{build_result.compiler}" if build_result.build_type: - row += "{}".format(build_result.build_type) + row += f"{build_result.build_type}" else: - row += "{}".format("relwithdebuginfo") + row += "relwithdebuginfo" if build_result.sanitizer: - row += "{}".format(build_result.sanitizer) + row += f"{build_result.sanitizer}" else: - row += "{}".format("none") + row += "none" - row += "{}".format(build_result.libraries) + row += f"{build_result.libraries}" if build_result.status: style = _get_status_style(build_result.status) - row += '{}'.format(style, build_result.status) + row += f'{build_result.status}' else: style = _get_status_style("error") - row += '{}'.format(style, "error") + row += f'error' - row += 'link'.format(build_log_url) + row += f'link' if build_result.elapsed_seconds: delta = datetime.timedelta(seconds=build_result.elapsed_seconds) else: delta = "unknown" # type: ignore - row += "{}".format(str(delta)) + row += f"{delta}" links = "" link_separator = "
" @@ -355,7 +347,7 @@ def create_build_html_report( links += link_separator if links: links = links[: -len(link_separator)] - row += "{}".format(links) + row += f"{links}" row += "" rows += row From 1fb090bfe20bd76345ed20ca2b50a9852fa46dbc Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 14 Nov 2022 18:50:31 +0100 Subject: [PATCH 28/98] Add typing to upload_result_helper.py, remove stale code --- tests/ci/s3_helper.py | 6 ++-- tests/ci/upload_result_helper.py | 52 +++++++++----------------------- 2 files changed, 18 insertions(+), 40 deletions(-) diff --git a/tests/ci/s3_helper.py b/tests/ci/s3_helper.py index 24ff013d69a..03e855a0057 100644 --- a/tests/ci/s3_helper.py +++ b/tests/ci/s3_helper.py @@ -46,7 +46,7 @@ class S3Helper: self.host = host self.download_host = download_host - def _upload_file_to_s3(self, bucket_name, file_path, s3_path): + def _upload_file_to_s3(self, bucket_name: str, file_path: str, s3_path: str) -> str: logging.debug( "Start uploading %s to bucket=%s path=%s", file_path, bucket_name, s3_path ) @@ -110,7 +110,7 @@ class S3Helper: url = f"{self.download_host}/{bucket_name}/{s3_path}" return url.replace("+", "%2B").replace(" ", "%20") - def upload_test_report_to_s3(self, file_path, s3_path): + def upload_test_report_to_s3(self, file_path: str, s3_path: str) -> str: if CI: return self._upload_file_to_s3(S3_TEST_REPORTS_BUCKET, file_path, s3_path) else: @@ -296,7 +296,7 @@ class S3Helper: return False @staticmethod - def copy_file_to_local(bucket_name, file_path, s3_path): + def copy_file_to_local(bucket_name: str, file_path: str, s3_path: str) -> str: local_path = os.path.abspath( os.path.join(RUNNER_TEMP, "s3", bucket_name, s3_path) ) diff --git a/tests/ci/upload_result_helper.py b/tests/ci/upload_result_helper.py index e145df02f80..745633a9e4d 100644 --- a/tests/ci/upload_result_helper.py +++ b/tests/ci/upload_result_helper.py @@ -1,6 +1,7 @@ import os import logging -import ast + +from typing import List, Tuple from env_helper import ( GITHUB_JOB_URL, @@ -9,35 +10,15 @@ from env_helper import ( GITHUB_SERVER_URL, ) from report import ReportColorTheme, create_test_html_report +from s3_helper import S3Helper def process_logs( - s3_client, additional_logs, s3_path_prefix, test_results, with_raw_logs -): + s3_client: S3Helper, additional_logs: List[str], s3_path_prefix: str +) -> List[str]: logging.info("Upload files to s3 %s", additional_logs) - processed_logs = {} - # Firstly convert paths of logs from test_results to urls to s3. - for test_result in test_results: - if len(test_result) <= 3 or with_raw_logs: - continue - - # Convert from string repr of list to list. - test_log_paths = ast.literal_eval(test_result[3]) - test_log_urls = [] - for log_path in test_log_paths: - if log_path in processed_logs: - test_log_urls.append(processed_logs[log_path]) - elif log_path: - url = s3_client.upload_test_report_to_s3( - log_path, s3_path_prefix + "/" + os.path.basename(log_path) - ) - test_log_urls.append(url) - processed_logs[log_path] = url - - test_result[3] = test_log_urls - - additional_urls = [] + additional_urls = [] # type: List[str] for log_path in additional_logs: if log_path: additional_urls.append( @@ -50,21 +31,18 @@ def process_logs( def upload_results( - s3_client, - pr_number, - commit_sha, - test_results, - additional_files, - check_name, - with_raw_logs=True, - statuscolors=None, -): + s3_client: S3Helper, + pr_number: int, + commit_sha: str, + test_results: List[Tuple[str, str]], + additional_files: List[str], + check_name: str, + with_raw_logs: bool = True, +) -> str: s3_path_prefix = f"{pr_number}/{commit_sha}/" + check_name.lower().replace( " ", "_" ).replace("(", "_").replace(")", "_").replace(",", "_") - additional_urls = process_logs( - s3_client, additional_files, s3_path_prefix, test_results, with_raw_logs - ) + additional_urls = process_logs(s3_client, additional_files, s3_path_prefix) branch_url = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/commits/master" branch_name = "master" From 88381026576eecc3d01e70a71d201d1a956487f9 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 14 Nov 2022 18:52:38 +0100 Subject: [PATCH 29/98] Fix issue in docker_manifests_merge.py --- tests/ci/docker_manifests_merge.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/docker_manifests_merge.py b/tests/ci/docker_manifests_merge.py index 09b7a99da78..2ba5a99de0a 100644 --- a/tests/ci/docker_manifests_merge.py +++ b/tests/ci/docker_manifests_merge.py @@ -70,7 +70,7 @@ def parse_args() -> argparse.Namespace: def load_images(path: str, suffix: str) -> Images: with open(os.path.join(path, CHANGED_IMAGES.format(suffix)), "rb") as images: - return json.load(images) + return json.load(images) # type: ignore def strip_suffix(suffix: str, images: Images) -> Images: From 02b8da2a0ffb3e5e89135ea33c2c1c3df4888fac Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 14 Nov 2022 18:59:01 +0100 Subject: [PATCH 30/98] Add typing for TeePopen --- tests/ci/tee_popen.py | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/tests/ci/tee_popen.py b/tests/ci/tee_popen.py index 7270cd6fb03..61404847bff 100644 --- a/tests/ci/tee_popen.py +++ b/tests/ci/tee_popen.py @@ -3,6 +3,7 @@ from subprocess import Popen, PIPE, STDOUT from threading import Thread from time import sleep +from typing import Optional import logging import os import sys @@ -18,7 +19,7 @@ class TeePopen: self.command = command self.log_file = log_file self.env = env - self.process = None + self._process = None # type: Optional[Popen] self.timeout = timeout def _check_timeout(self): @@ -51,7 +52,7 @@ class TeePopen: return self def __exit__(self, t, value, traceback): - for line in self.process.stdout: + for line in self.process.stdout: # type: ignore sys.stdout.write(line) self.log_file.write(line) @@ -59,8 +60,18 @@ class TeePopen: self.log_file.close() def wait(self): - for line in self.process.stdout: + for line in self.process.stdout: # type: ignore sys.stdout.write(line) self.log_file.write(line) return self.process.wait() + + @property + def process(self) -> Popen: + if self._process is not None: + return self._process + raise AttributeError("process is not created yet") + + @process.setter + def process(self, process: Popen) -> None: + self._process = process From 6e00bb2e744f6d38f930442f08b3809b69d0ef4f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 14 Nov 2022 20:02:33 +0100 Subject: [PATCH 31/98] Add and fix typing for docker_pull_helper, github_helper and style_check --- tests/ci/docker_pull_helper.py | 18 ++++++++++++------ tests/ci/github_helper.py | 11 ++++++----- tests/ci/style_check.py | 16 ++++++++++------ 3 files changed, 28 insertions(+), 17 deletions(-) diff --git a/tests/ci/docker_pull_helper.py b/tests/ci/docker_pull_helper.py index 04817ed7de3..5336966b3eb 100644 --- a/tests/ci/docker_pull_helper.py +++ b/tests/ci/docker_pull_helper.py @@ -6,11 +6,11 @@ import time import subprocess import logging -from typing import Optional +from typing import List, Optional class DockerImage: - def __init__(self, name, version: Optional[str] = None): + def __init__(self, name: str, version: Optional[str] = None): self.name = name if version is None: self.version = "latest" @@ -22,8 +22,11 @@ class DockerImage: def get_images_with_versions( - reports_path, required_image, pull=True, version: Optional[str] = None -): + reports_path: str, + required_images: List[str], + pull: bool = True, + version: Optional[str] = None, +) -> List[DockerImage]: images_path = None for root, _, files in os.walk(reports_path): for f in files: @@ -45,12 +48,13 @@ def get_images_with_versions( images = {} docker_images = [] - for image_name in required_image: + for image_name in required_images: docker_image = DockerImage(image_name, version) if image_name in images: docker_image.version = images[image_name] docker_images.append(docker_image) + latest_error = Exception("predefined to avoid access before created") if pull: for docker_image in docker_images: for i in range(10): @@ -75,6 +79,8 @@ def get_images_with_versions( return docker_images -def get_image_with_version(reports_path, image, pull=True, version=None): +def get_image_with_version( + reports_path: str, image: str, pull: bool = True, version: Optional[str] = None +) -> DockerImage: logging.info("Looking for images file in %s", reports_path) return get_images_with_versions(reports_path, [image], pull, version=version)[0] diff --git a/tests/ci/github_helper.py b/tests/ci/github_helper.py index 685d9f2c841..5a862a9fdca 100644 --- a/tests/ci/github_helper.py +++ b/tests/ci/github_helper.py @@ -90,7 +90,7 @@ class GitHub(github.Github): raise exception # pylint: enable=signature-differs - def get_pulls_from_search(self, *args, **kwargs) -> PullRequests: + def get_pulls_from_search(self, *args, **kwargs) -> PullRequests: # type: ignore """The search api returns actually issues, so we need to fetch PullRequests""" issues = self.search_issues(*args, **kwargs) repos = {} @@ -168,7 +168,7 @@ class GitHub(github.Github): self.dump(user, prfd) # type: ignore return user - def _get_cached(self, path: Path): + def _get_cached(self, path: Path): # type: ignore with open(path, "rb") as ob_fd: return self.load(ob_fd) # type: ignore @@ -190,11 +190,11 @@ class GitHub(github.Github): return False, cached_obj @property - def cache_path(self): + def cache_path(self) -> Path: return self._cache_path @cache_path.setter - def cache_path(self, value: str): + def cache_path(self, value: str) -> None: self._cache_path = Path(value) if self._cache_path.exists(): assert self._cache_path.is_dir() @@ -208,5 +208,6 @@ class GitHub(github.Github): return self._retries @retries.setter - def retries(self, value: int): + def retries(self, value: int) -> None: + assert isinstance(value, int) self._retries = value diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 23a1dd467d7..58d06f890af 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -1,11 +1,13 @@ #!/usr/bin/env python3 import argparse +import atexit import csv import logging import os import subprocess import sys -import atexit + +from typing import List, Tuple from clickhouse_helper import ( @@ -29,8 +31,10 @@ from upload_result_helper import upload_results NAME = "Style Check" -def process_result(result_folder): - test_results = [] +def process_result( + result_folder: str, +) -> Tuple[str, str, List[Tuple[str, str]], List[str]]: + test_results = [] # type: List[Tuple[str, str]] additional_files = [] # Just upload all files from result_folder. # If task provides processed results, then it's responsible @@ -57,7 +61,7 @@ def process_result(result_folder): try: results_path = os.path.join(result_folder, "test_results.tsv") with open(results_path, "r", encoding="utf-8") as fd: - test_results = list(csv.reader(fd, delimiter="\t")) + test_results = list(csv.reader(fd, delimiter="\t")) # type: ignore if len(test_results) == 0: raise Exception("Empty results") @@ -81,7 +85,7 @@ def parse_args(): return parser.parse_args() -def checkout_head(pr_info: PRInfo): +def checkout_head(pr_info: PRInfo) -> None: # It works ONLY for PRs, and only over ssh, so either # ROBOT_CLICKHOUSE_SSH_KEY should be set or ssh-agent should work assert pr_info.number @@ -107,7 +111,7 @@ def checkout_head(pr_info: PRInfo): git_runner(f"git checkout -f head-{pr_info.head_ref}") -def commit_push_staged(pr_info: PRInfo): +def commit_push_staged(pr_info: PRInfo) -> None: # It works ONLY for PRs, and only over ssh, so either # ROBOT_CLICKHOUSE_SSH_KEY should be set or ssh-agent should work assert pr_info.number From da71dbe040f90fa6e67c4101f24d1cb5be5001b5 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 15 Nov 2022 13:01:27 +0100 Subject: [PATCH 32/98] Fix issues in build_report_check.py --- tests/ci/build_report_check.py | 26 +++++++++++++------------- tests/ci/pr_info.py | 3 ++- 2 files changed, 15 insertions(+), 14 deletions(-) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 673b0204864..03e18d7766e 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -19,7 +19,7 @@ from env_helper import ( from report import create_build_html_report from s3_helper import S3Helper from get_robot_token import get_best_robot_token -from pr_info import PRInfo +from pr_info import NeedsDataType, PRInfo from commit_status_helper import ( get_commit, update_mergeable_check, @@ -28,7 +28,7 @@ from ci_config import CI_CONFIG from rerun_helper import RerunHelper -NEEDS_DATA_PATH = os.getenv("NEEDS_DATA_PATH") +NEEDS_DATA_PATH = os.getenv("NEEDS_DATA_PATH", "") class BuildResult: @@ -98,7 +98,7 @@ def get_failed_report( def process_report( - build_report, + build_report: dict, ) -> Tuple[List[BuildResult], List[List[str]], List[str]]: build_config = build_report["build_config"] build_result = BuildResult( @@ -144,16 +144,14 @@ def main(): os.makedirs(temp_path) build_check_name = sys.argv[1] - needs_data = None + needs_data = {} # type: NeedsDataType required_builds = 0 if os.path.exists(NEEDS_DATA_PATH): with open(NEEDS_DATA_PATH, "rb") as file_handler: needs_data = json.load(file_handler) required_builds = len(needs_data) - if needs_data is not None and all( - i["result"] == "skipped" for i in needs_data.values() - ): + if needs_data and all(i["result"] == "skipped" for i in needs_data.values()): logging.info("All builds are skipped, exiting") sys.exit(0) @@ -218,19 +216,21 @@ def main(): build_logs = [] for build_report in build_reports: - build_result, build_artifacts_url, build_logs_url = process_report(build_report) - logging.info( - "Got %s artifact groups for build report report", len(build_result) + _build_results, build_artifacts_url, build_logs_url = process_report( + build_report ) - build_results.extend(build_result) + logging.info( + "Got %s artifact groups for build report report", len(_build_results) + ) + build_results.extend(_build_results) build_artifacts.extend(build_artifacts_url) build_logs.extend(build_logs_url) for failed_job in missing_build_names: - build_result, build_artifacts_url, build_logs_url = get_failed_report( + _build_results, build_artifacts_url, build_logs_url = get_failed_report( failed_job ) - build_results.extend(build_result) + build_results.extend(_build_results) build_artifacts.extend(build_artifacts_url) build_logs.extend(build_logs_url) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index f0a371c488f..942edeaa81c 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -2,7 +2,7 @@ import json import logging import os -from typing import Set +from typing import Dict, Set, Union from unidiff import PatchSet # type: ignore @@ -16,6 +16,7 @@ from env_helper import ( FORCE_TESTS_LABEL = "force tests" SKIP_MERGEABLE_CHECK_LABEL = "skip mergeable check" +NeedsDataType = Dict[str, Dict[str, Union[str, Dict[str, str]]]] DIFF_IN_DOCUMENTATION_EXT = [ ".html", From 4fc5bb7a639eeefc99fb23a94d3b0407288caf09 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 15 Nov 2022 13:10:13 +0100 Subject: [PATCH 33/98] Fix release.py typing issue --- tests/ci/release.py | 38 ++++++++++++++++++++++++-------------- 1 file changed, 24 insertions(+), 14 deletions(-) diff --git a/tests/ci/release.py b/tests/ci/release.py index 8024091e300..dec97f2a54f 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -11,7 +11,7 @@ On another hand, PyGithub is used for convenient getting commit's status from AP from contextlib import contextmanager -from typing import List, Optional +from typing import Any, Iterator, List, Literal, Optional import argparse import logging import subprocess @@ -48,7 +48,7 @@ class Repo: return self._url @url.setter - def url(self, protocol: str): + def url(self, protocol: str) -> None: if protocol == "ssh": self._url = f"git@github.com:{self}.git" elif protocol == "https": @@ -68,17 +68,23 @@ class Release: CMAKE_PATH = get_abs_path(FILE_WITH_VERSION_PATH) CONTRIBUTORS_PATH = get_abs_path(GENERATED_CONTRIBUTORS) - def __init__(self, repo: Repo, release_commit: str, release_type: str): + def __init__( + self, + repo: Repo, + release_commit: str, + release_type: Literal["major", "minor", "patch"], + ): self.repo = repo self._release_commit = "" self.release_commit = release_commit + assert release_type in self.BIG + self.SMALL self.release_type = release_type self._git = git self._version = get_version_from_repo(git=self._git) self._release_branch = "" self._rollback_stack = [] # type: List[str] - def run(self, cmd: str, cwd: Optional[str] = None, **kwargs) -> str: + def run(self, cmd: str, cwd: Optional[str] = None, **kwargs: Any) -> str: cwd_text = "" if cwd: cwd_text = f" (CWD='{cwd}')" @@ -153,7 +159,9 @@ class Release: self.check_commit_release_ready() - def do(self, check_dirty: bool, check_branch: bool, with_release_branch: bool): + def do( + self, check_dirty: bool, check_branch: bool, with_release_branch: bool + ) -> None: self.check_prerequisites() if check_dirty: @@ -310,7 +318,7 @@ class Release: return self._version @version.setter - def version(self, version: ClickHouseVersion): + def version(self, version: ClickHouseVersion) -> None: if not isinstance(version, ClickHouseVersion): raise ValueError(f"version must be ClickHouseVersion, not {type(version)}") self._version = version @@ -320,7 +328,7 @@ class Release: return self._release_branch @release_branch.setter - def release_branch(self, branch: str): + def release_branch(self, branch: str) -> None: self._release_branch = release_branch(branch) @property @@ -328,7 +336,7 @@ class Release: return self._release_commit @release_commit.setter - def release_commit(self, release_commit: str): + def release_commit(self, release_commit: str) -> None: self._release_commit = commit(release_commit) @contextmanager @@ -367,7 +375,7 @@ class Release: yield @contextmanager - def _bump_testing_version(self, helper_branch: str): + def _bump_testing_version(self, helper_branch: str) -> Iterator[None]: self.read_version() self.version = self.version.update(self.release_type) self.version.with_description(VersionType.TESTING) @@ -387,7 +395,7 @@ class Release: yield @contextmanager - def _checkout(self, ref: str, with_checkout_back: bool = False): + def _checkout(self, ref: str, with_checkout_back: bool = False) -> Iterator[None]: orig_ref = self._git.branch or self._git.sha need_rollback = False if ref not in (self._git.branch, self._git.sha): @@ -406,7 +414,7 @@ class Release: self.run(rollback_cmd) @contextmanager - def _create_branch(self, name: str, start_point: str = ""): + def _create_branch(self, name: str, start_point: str = "") -> Iterator[None]: self.run(f"git branch {name} {start_point}") rollback_cmd = f"git branch -D {name}" self._rollback_stack.append(rollback_cmd) @@ -418,7 +426,7 @@ class Release: raise @contextmanager - def _create_gh_label(self, label: str, color_hex: str): + def _create_gh_label(self, label: str, color_hex: str) -> Iterator[None]: # API call, https://docs.github.com/en/rest/reference/issues#create-a-label self.run( f"gh api repos/{self.repo}/labels -f name={label} -f color={color_hex}" @@ -433,7 +441,7 @@ class Release: raise @contextmanager - def _create_gh_release(self, as_prerelease: bool): + def _create_gh_release(self, as_prerelease: bool) -> Iterator[None]: with self._create_tag(): # Preserve tag if version is changed tag = self.version.describe @@ -468,7 +476,9 @@ class Release: raise @contextmanager - def _push(self, ref: str, with_rollback_on_fail: bool = True, remote_ref: str = ""): + def _push( + self, ref: str, with_rollback_on_fail: bool = True, remote_ref: str = "" + ) -> Iterator[None]: if remote_ref == "": remote_ref = ref From ec0684dfce09ed94b87f27c1fabb0f7e5fa80a4a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 15 Nov 2022 14:03:16 +0100 Subject: [PATCH 34/98] Fix typing issues in build_check.py --- tests/ci/build_check.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index d668dbe0498..c9e8dac2c00 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -121,7 +121,7 @@ def check_for_success_run( s3_prefix: str, build_name: str, build_config: BuildConfig, -): +) -> None: logged_prefix = os.path.join(S3_BUILDS_BUCKET, s3_prefix) logging.info("Checking for artifacts in %s", logged_prefix) try: @@ -174,7 +174,7 @@ def create_json_artifact( build_config: BuildConfig, elapsed: int, success: bool, -): +) -> None: subprocess.check_call( f"echo 'BUILD_URLS=build_urls_{build_name}' >> $GITHUB_ENV", shell=True ) @@ -218,7 +218,7 @@ def upload_master_static_binaries( build_config: BuildConfig, s3_helper: S3Helper, build_output_path: str, -): +) -> None: """Upload binary artifacts to a static S3 links""" static_binary_name = build_config.get("static_binary_name", False) if pr_info.number != 0: From ba9e51d6865844f79fc380e083e06078535d5821 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 15 Nov 2022 14:46:48 +0100 Subject: [PATCH 35/98] Fix cherry-pick.py typing issues --- tests/ci/cherry_pick.py | 17 ++++++++++------- tests/ci/github_helper.py | 17 ++++++++++++----- 2 files changed, 22 insertions(+), 12 deletions(-) diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index d1c9d3d394c..20d2d780484 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -92,7 +92,8 @@ Merge it only if you intend to backport changes to the target branch, otherwise if branch_updated: self._backported = True - def pop_prs(self, prs: PullRequests): + def pop_prs(self, prs: PullRequests) -> None: + """the method processes all prs and pops the ReleaseBranch related prs""" to_pop = [] # type: List[int] for i, pr in enumerate(prs): if self.name not in pr.head.ref: @@ -105,14 +106,14 @@ Merge it only if you intend to backport changes to the target branch, otherwise to_pop.append(i) else: logging.error( - "PR #%s doesn't head ref starting with known suffix", + "head ref of PR #%s isn't starting with known suffix", pr.number, ) for i in reversed(to_pop): # Going from the tail to keep the order and pop greater index first prs.pop(i) - def process(self, dry_run: bool): + def process(self, dry_run: bool) -> None: if self.backported: return if not self.cherrypick_pr: @@ -209,6 +210,7 @@ Merge it only if you intend to backport changes to the target branch, otherwise self._assign_new_pr(self.cherrypick_pr) def create_backport(self): + assert self.cherrypick_pr is not None # Checkout the backport branch from the remote and make all changes to # apply like they are only one cherry-pick commit on top of release git_runner(f"{self.git_prefix} checkout -f {self.backport_branch}") @@ -239,7 +241,7 @@ Merge it only if you intend to backport changes to the target branch, otherwise self.backport_pr.add_to_labels(Labels.BACKPORT) self._assign_new_pr(self.backport_pr) - def _assign_new_pr(self, new_pr: PullRequest): + def _assign_new_pr(self, new_pr: PullRequest) -> None: """Assign `new_pr` to author, merger and assignees of an original PR""" # It looks there some race when multiple .add_to_assignees are executed, # so we'll add all at once @@ -340,7 +342,7 @@ class Backport: ) self.error = e - def process_pr(self, pr: PullRequest): + def process_pr(self, pr: PullRequest) -> None: pr_labels = [label.name for label in pr.labels] if Labels.MUST_BACKPORT in pr_labels: branches = [ @@ -403,7 +405,7 @@ class Backport: # And check it after the running self.mark_pr_backported(pr) - def mark_pr_backported(self, pr: PullRequest): + def mark_pr_backported(self, pr: PullRequest) -> None: if self.dry_run: logging.info("DRY RUN: would mark PR #%s as done", pr.number) return @@ -488,7 +490,8 @@ def main(): gh = GitHub(token, per_page=100) bp = Backport(gh, args.repo, args.dry_run) - bp.gh.cache_path = str(f"{TEMP_PATH}/gh_cache") + # https://github.com/python/mypy/issues/3004 + bp.gh.cache_path = f"{TEMP_PATH}/gh_cache" # type: ignore bp.receive_release_prs() bp.receive_prs_for_backport() bp.process_backports() diff --git a/tests/ci/github_helper.py b/tests/ci/github_helper.py index 5a862a9fdca..bd740827b34 100644 --- a/tests/ci/github_helper.py +++ b/tests/ci/github_helper.py @@ -8,11 +8,18 @@ from time import sleep from typing import List, Optional, Tuple import github -from github.GithubException import RateLimitExceededException -from github.Issue import Issue -from github.NamedUser import NamedUser -from github.PullRequest import PullRequest -from github.Repository import Repository + +# explicit reimport +# pylint: disable=useless-import-alias +from github.GithubException import ( + RateLimitExceededException as RateLimitExceededException, +) +from github.Issue import Issue as Issue +from github.NamedUser import NamedUser as NamedUser +from github.PullRequest import PullRequest as PullRequest +from github.Repository import Repository as Repository + +# pylint: enable=useless-import-alias CACHE_PATH = p.join(p.dirname(p.realpath(__file__)), "gh_cache") From 459dde989bb6166f6b78532872ea73b8d9b5a344 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 15 Nov 2022 14:52:40 +0100 Subject: [PATCH 36/98] A tiny fix for missing types --- tests/ci/get_previous_release_tag.py | 2 +- tests/ci/jepsen_check.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/get_previous_release_tag.py b/tests/ci/get_previous_release_tag.py index bfce69a17d9..b9ad51379d2 100755 --- a/tests/ci/get_previous_release_tag.py +++ b/tests/ci/get_previous_release_tag.py @@ -3,7 +3,7 @@ import re import logging -import requests +import requests # type: ignore CLICKHOUSE_TAGS_URL = "https://api.github.com/repos/ClickHouse/ClickHouse/tags" VERSION_PATTERN = r"(v(?:\d+\.)?(?:\d+\.)?(?:\d+\.)?\d+-[a-zA-Z]*)" diff --git a/tests/ci/jepsen_check.py b/tests/ci/jepsen_check.py index 4116d15bba6..69964c0a0bc 100644 --- a/tests/ci/jepsen_check.py +++ b/tests/ci/jepsen_check.py @@ -7,9 +7,9 @@ import sys import argparse -import boto3 +import boto3 # type: ignore +import requests # type: ignore from github import Github -import requests from env_helper import REPO_COPY, TEMP_PATH, S3_BUILDS_BUCKET, S3_DOWNLOAD from stopwatch import Stopwatch From d4859275f4156b9881944ca7ad25c9041034ab5e Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 15 Nov 2022 17:35:15 +0100 Subject: [PATCH 37/98] Fix typing in version_test.py --- tests/ci/version_test.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/ci/version_test.py b/tests/ci/version_test.py index 86a2d58c3c8..abd0f9349f4 100644 --- a/tests/ci/version_test.py +++ b/tests/ci/version_test.py @@ -17,9 +17,9 @@ class TestFunctions(unittest.TestCase): ("v1.1.1.2-testing", vh.get_version_from_string("1.1.1.2")), ("refs/tags/v1.1.1.2-testing", vh.get_version_from_string("1.1.1.2")), ) - for case in cases: - version = vh.version_arg(case[0]) - self.assertEqual(case[1], version) + for test_case in cases: + version = vh.version_arg(test_case[0]) + self.assertEqual(test_case[1], version) error_cases = ( "0.0.0", "1.1.1.a", @@ -28,6 +28,6 @@ class TestFunctions(unittest.TestCase): "v1.1.1.2-testin", "refs/tags/v1.1.1.2-testin", ) - for case in error_cases: + for error_case in error_cases: with self.assertRaises(ArgumentTypeError): - version = vh.version_arg(case[0]) + version = vh.version_arg(error_case[0]) From 40b8c8559f44f5c63da8e7cc260cf2850ab219b5 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 15 Nov 2022 17:49:35 +0100 Subject: [PATCH 38/98] Simplify finish_check.py --- tests/ci/finish_check.py | 27 ++++++++------------------- 1 file changed, 8 insertions(+), 19 deletions(-) diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index a0b7f14ecfb..ea2f5eb3136 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -5,27 +5,11 @@ from github import Github from env_helper import GITHUB_RUN_URL from pr_info import PRInfo from get_robot_token import get_best_robot_token -from commit_status_helper import get_commit +from commit_status_helper import get_commit, get_commit_filtered_statuses NAME = "Run Check" -def filter_statuses(statuses): - """ - Squash statuses to latest state - 1. context="first", state="success", update_time=1 - 2. context="second", state="success", update_time=2 - 3. context="first", stat="failure", update_time=3 - =========> - 1. context="second", state="success" - 2. context="first", stat="failure" - """ - filt = {} - for status in sorted(statuses, key=lambda x: x.updated_at): - filt[status.context] = status - return filt - - if __name__ == "__main__": logging.basicConfig(level=logging.INFO) @@ -34,8 +18,13 @@ if __name__ == "__main__": commit = get_commit(gh, pr_info.sha) url = GITHUB_RUN_URL - statuses = filter_statuses(list(commit.get_statuses())) - if NAME in statuses and statuses[NAME].state == "pending": + statuses = get_commit_filtered_statuses(commit) + pending_status = any( # find NAME status in pending state + True + for status in statuses + if status.context == NAME and status.state == "pending" + ) + if pending_status: commit.create_status( context=NAME, description="All checks finished", From eb738fa13e68a38369c02fa782aacb11cc899f1d Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 15 Nov 2022 17:57:56 +0100 Subject: [PATCH 39/98] Use PRInfo and TEMP_PATH in codebrowser --- tests/ci/codebrowser_check.py | 38 ++++++++++++++++++----------------- 1 file changed, 20 insertions(+), 18 deletions(-) diff --git a/tests/ci/codebrowser_check.py b/tests/ci/codebrowser_check.py index 97036c6fc7b..412bcdf8818 100644 --- a/tests/ci/codebrowser_check.py +++ b/tests/ci/codebrowser_check.py @@ -7,14 +7,21 @@ import logging from github import Github -from env_helper import IMAGES_PATH, REPO_COPY, S3_TEST_REPORTS_BUCKET, S3_DOWNLOAD -from stopwatch import Stopwatch -from upload_result_helper import upload_results -from s3_helper import S3Helper -from get_robot_token import get_best_robot_token +from env_helper import ( + IMAGES_PATH, + REPO_COPY, + S3_DOWNLOAD, + S3_TEST_REPORTS_BUCKET, + TEMP_PATH, +) from commit_status_helper import post_commit_status from docker_pull_helper import get_image_with_version +from get_robot_token import get_best_robot_token +from pr_info import PRInfo +from s3_helper import S3Helper +from stopwatch import Stopwatch from tee_popen import TeePopen +from upload_result_helper import upload_results NAME = "Woboq Build" @@ -33,17 +40,16 @@ if __name__ == "__main__": stopwatch = Stopwatch() - temp_path = os.getenv("TEMP_PATH", os.path.abspath(".")) - gh = Github(get_best_robot_token(), per_page=100) + pr_info = PRInfo() - if not os.path.exists(temp_path): - os.makedirs(temp_path) + if not os.path.exists(TEMP_PATH): + os.makedirs(TEMP_PATH) docker_image = get_image_with_version(IMAGES_PATH, "clickhouse/codebrowser") s3_helper = S3Helper() - result_path = os.path.join(temp_path, "result_path") + result_path = os.path.join(TEMP_PATH, "result_path") if not os.path.exists(result_path): os.makedirs(result_path) @@ -51,7 +57,7 @@ if __name__ == "__main__": logging.info("Going to run codebrowser: %s", run_command) - run_log_path = os.path.join(temp_path, "runlog.log") + run_log_path = os.path.join(TEMP_PATH, "runlog.log") with TeePopen(run_command, run_log_path) as process: retcode = process.wait() @@ -60,7 +66,7 @@ if __name__ == "__main__": else: logging.info("Run failed") - subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) + subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {TEMP_PATH}", shell=True) report_path = os.path.join(result_path, "html_report") logging.info("Report path %s", report_path) @@ -76,12 +82,8 @@ if __name__ == "__main__": test_results = [(index_html, "Look at the report")] - report_url = upload_results( - s3_helper, 0, os.getenv("GITHUB_SHA"), test_results, [], NAME - ) + report_url = upload_results(s3_helper, 0, pr_info.sha, test_results, [], NAME) print(f"::notice ::Report url: {report_url}") - post_commit_status( - gh, os.getenv("GITHUB_SHA"), NAME, "Report built", "success", report_url - ) + post_commit_status(gh, pr_info.sha, NAME, "Report built", "success", report_url) From 33424bc411c593bb6caf9ebb667e9aee762a6fed Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 15 Nov 2022 18:02:42 +0100 Subject: [PATCH 40/98] Add typing to fast tests --- tests/ci/fast_test_check.py | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 03e42726808..2a6a0d5fa57 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -6,6 +6,7 @@ import os import csv import sys import atexit +from typing import List, Tuple from github import Github @@ -50,8 +51,10 @@ def get_fasttest_cmd( ) -def process_results(result_folder): - test_results = [] +def process_results( + result_folder: str, +) -> Tuple[str, str, List[Tuple[str, str]], List[str]]: + test_results = [] # type: List[Tuple[str, str]] additional_files = [] # Just upload all files from result_folder. # If task provides processed results, then it's responsible for content of @@ -78,7 +81,7 @@ def process_results(result_folder): results_path = os.path.join(result_folder, "test_results.tsv") if os.path.exists(results_path): with open(results_path, "r", encoding="utf-8") as results_file: - test_results = list(csv.reader(results_file, delimiter="\t")) + test_results = list(csv.reader(results_file, delimiter="\t")) # type: ignore if len(test_results) == 0: return "error", "Empty test_results.tsv", test_results, additional_files @@ -172,7 +175,7 @@ if __name__ == "__main__": "test_log.txt" in test_output_files or "test_result.txt" in test_output_files ) test_result_exists = "test_results.tsv" in test_output_files - test_results = [] + test_results = [] # type: List[Tuple[str, str]] if "submodule_log.txt" not in test_output_files: description = "Cannot clone repository" state = "failure" From 1e5fec1157ee1400749e731086454d9f591a24c7 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Nov 2022 10:01:17 +0100 Subject: [PATCH 41/98] Fix functional_test_check.py typing --- tests/ci/functional_test_check.py | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index f7d3288c316..87833d688af 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -7,6 +7,7 @@ import os import subprocess import sys import atexit +from typing import List, Tuple from github import Github @@ -122,8 +123,11 @@ def get_tests_to_run(pr_info): return list(result) -def process_results(result_folder, server_log_path): - test_results = [] +def process_results( + result_folder: str, + server_log_path: str, +) -> Tuple[str, str, List[Tuple[str, str]], List[str]]: + test_results = [] # type: List[Tuple[str, str]] additional_files = [] # Just upload all files from result_folder. # If task provides processed results, then it's responsible for content of result_folder. @@ -166,7 +170,7 @@ def process_results(result_folder, server_log_path): return "error", "Not found test_results.tsv", test_results, additional_files with open(results_path, "r", encoding="utf-8") as results_file: - test_results = list(csv.reader(results_file, delimiter="\t")) + test_results = list(csv.reader(results_file, delimiter="\t")) # type: ignore if len(test_results) == 0: return "error", "Empty test_results.tsv", test_results, additional_files @@ -232,8 +236,8 @@ if __name__ == "__main__": sys.exit(0) if "RUN_BY_HASH_NUM" in os.environ: - run_by_hash_num = int(os.getenv("RUN_BY_HASH_NUM")) - run_by_hash_total = int(os.getenv("RUN_BY_HASH_TOTAL")) + run_by_hash_num = int(os.getenv("RUN_BY_HASH_NUM", "0")) + run_by_hash_total = int(os.getenv("RUN_BY_HASH_TOTAL", "0")) check_name_with_group = ( check_name + f" [{run_by_hash_num + 1}/{run_by_hash_total}]" ) From d0d23af63e3d2b3de7d0a84fad56b6e9950229f2 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Nov 2022 10:08:53 +0100 Subject: [PATCH 42/98] Fix integration_test_check.py typing --- tests/ci/integration_test_check.py | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index cba428cbcf5..e61117a4b45 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -7,6 +7,7 @@ import logging import os import subprocess import sys +from typing import List, Tuple from github import Github @@ -87,8 +88,10 @@ def get_env_for_runner(build_path, repo_path, result_path, work_path): return my_env -def process_results(result_folder): - test_results = [] +def process_results( + result_folder: str, +) -> Tuple[str, str, List[Tuple[str, str]], List[str]]: + test_results = [] # type: List[Tuple[str, str]] additional_files = [] # Just upload all files from result_folder. # If task provides processed results, then it's responsible for content of result_folder. @@ -115,7 +118,7 @@ def process_results(result_folder): results_path = os.path.join(result_folder, "test_results.tsv") if os.path.exists(results_path): with open(results_path, "r", encoding="utf-8") as results_file: - test_results = list(csv.reader(results_file, delimiter="\t")) + test_results = list(csv.reader(results_file, delimiter="\t")) # type: ignore if len(test_results) == 0: return "error", "Empty test_results.tsv", test_results, additional_files @@ -153,8 +156,8 @@ if __name__ == "__main__": validate_bugix_check = args.validate_bugfix if "RUN_BY_HASH_NUM" in os.environ: - run_by_hash_num = int(os.getenv("RUN_BY_HASH_NUM")) - run_by_hash_total = int(os.getenv("RUN_BY_HASH_TOTAL")) + run_by_hash_num = int(os.getenv("RUN_BY_HASH_NUM", "0")) + run_by_hash_total = int(os.getenv("RUN_BY_HASH_TOTAL", "0")) check_name_with_group = ( check_name + f" [{run_by_hash_num + 1}/{run_by_hash_total}]" ) From 5af2bc6660fc876d0a098fbbf3b9f03fd207b24d Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Nov 2022 10:12:07 +0100 Subject: [PATCH 43/98] Fix unit_tests_check.py typing --- tests/ci/unit_tests_check.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/ci/unit_tests_check.py b/tests/ci/unit_tests_check.py index c2dfab9dddc..4777296da18 100644 --- a/tests/ci/unit_tests_check.py +++ b/tests/ci/unit_tests_check.py @@ -5,6 +5,7 @@ import os import sys import subprocess import atexit +from typing import List, Tuple from github import Github @@ -37,14 +38,16 @@ def get_test_name(line): raise Exception(f"No test name in line '{line}'") -def process_result(result_folder): +def process_results( + result_folder: str, +) -> Tuple[str, str, List[Tuple[str, str]], List[str]]: OK_SIGN = "OK ]" FAILED_SIGN = "FAILED ]" SEGFAULT = "Segmentation fault" SIGNAL = "received signal SIG" PASSED = "PASSED" - summary = [] + summary = [] # type: List[Tuple[str, str]] total_counter = 0 failed_counter = 0 result_log_path = f"{result_folder}/test_result.txt" @@ -151,7 +154,7 @@ if __name__ == "__main__": subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) s3_helper = S3Helper() - state, description, test_results, additional_logs = process_result(test_output) + state, description, test_results, additional_logs = process_results(test_output) ch_helper = ClickHouseHelper() mark_flaky_tests(ch_helper, check_name, test_results) From e1a9f2976a7d76d7012d34174dae102877acf1d0 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Nov 2022 10:24:10 +0100 Subject: [PATCH 44/98] Fix stress_check.py typing --- tests/ci/stress_check.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 8f310eaa99d..c02128d114f 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -5,6 +5,7 @@ import logging import subprocess import os import sys +from typing import List, Tuple from github import Github @@ -44,8 +45,10 @@ def get_run_command( return cmd -def process_results(result_folder, server_log_path, run_log_path): - test_results = [] +def process_results( + result_folder: str, server_log_path: str, run_log_path: str +) -> Tuple[str, str, List[Tuple[str, str]], List[str]]: + test_results = [] # type: List[Tuple[str, str]] additional_files = [] # Just upload all files from result_folder. # If task provides processed results, then it's responsible for content @@ -89,7 +92,7 @@ def process_results(result_folder, server_log_path, run_log_path): results_path = os.path.join(result_folder, "test_results.tsv") with open(results_path, "r", encoding="utf-8") as results_file: - test_results = list(csv.reader(results_file, delimiter="\t")) + test_results = list(csv.reader(results_file, delimiter="\t")) # type: ignore if len(test_results) == 0: raise Exception("Empty results") From db44e40989469f23ba472acabc84121466bf63c8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Nov 2022 10:24:32 +0100 Subject: [PATCH 45/98] Fix sqlancer_check.py typing --- tests/ci/sqlancer_check.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/ci/sqlancer_check.py b/tests/ci/sqlancer_check.py index 63c7d18fe46..5e94969d4b1 100644 --- a/tests/ci/sqlancer_check.py +++ b/tests/ci/sqlancer_check.py @@ -4,6 +4,7 @@ import logging import subprocess import os import sys +from typing import List, Tuple from github import Github @@ -137,7 +138,7 @@ if __name__ == "__main__": report_url = GITHUB_RUN_URL status = "success" - test_results = [] + test_results = [] # type: List[Tuple[str, str]] # Try to get status message saved by the SQLancer try: # with open( @@ -145,7 +146,7 @@ if __name__ == "__main__": # ) as status_f: # status = status_f.readline().rstrip("\n") if os.path.exists(os.path.join(workspace_path, "server_crashed.log")): - test_results.append("Server crashed", "FAIL") + test_results.append(("Server crashed", "FAIL")) with open( os.path.join(workspace_path, "summary.tsv"), "r", encoding="utf-8" ) as summary_f: From f4a7ff409e7057f6d980b56d377723f2a8fc9d01 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Nov 2022 13:52:31 +0100 Subject: [PATCH 46/98] Fix push_to_artifactory.py --- tests/ci/push_to_artifactory.py | 29 +++++++++++++++++------------ 1 file changed, 17 insertions(+), 12 deletions(-) diff --git a/tests/ci/push_to_artifactory.py b/tests/ci/push_to_artifactory.py index dd8081227bf..97971f207ce 100755 --- a/tests/ci/push_to_artifactory.py +++ b/tests/ci/push_to_artifactory.py @@ -5,7 +5,7 @@ import logging import os import re from collections import namedtuple -from typing import Dict, List, Tuple +from typing import Dict, List, Optional, Tuple from artifactory import ArtifactorySaaSPath # type: ignore from build_download_helper import download_build_with_progress @@ -14,7 +14,7 @@ from git_helper import TAG_REGEXP, commit, removeprefix, removesuffix # Necessary ENV variables -def getenv(name: str, default: str = None): +def getenv(name: str, default: Optional[str] = None) -> str: env = os.getenv(name, default) if env is not None: return env @@ -62,7 +62,7 @@ class Packages: raise ValueError(f"{deb_pkg} not in {self.deb}") return removesuffix(deb_pkg, ".deb").split("_")[-1] - def replace_with_fallback(self, name: str): + def replace_with_fallback(self, name: str) -> None: if name.endswith(".deb"): suffix = self.deb.pop(name) self.deb[self.fallback_to_all(name)] = self.fallback_to_all(suffix) @@ -80,7 +80,7 @@ class Packages: return os.path.join(TEMP_PATH, package_file) @staticmethod - def fallback_to_all(url_or_name: str): + def fallback_to_all(url_or_name: str) -> str: """Until July 2022 we had clickhouse-server and clickhouse-client with arch 'all'""" # deb @@ -111,7 +111,7 @@ class S3: self.force_download = force_download self.packages = Packages(version) - def download_package(self, package_file: str, s3_path_suffix: str): + def download_package(self, package_file: str, s3_path_suffix: str) -> None: path = Packages.path(package_file) fallback_path = Packages.fallback_to_all(path) if not self.force_download and ( @@ -186,7 +186,12 @@ class Release: class Artifactory: def __init__( - self, url: str, release: str, deb_repo="deb", rpm_repo="rpm", tgz_repo="tgz" + self, + url: str, + release: str, + deb_repo: str = "deb", + rpm_repo: str = "rpm", + tgz_repo: str = "tgz", ): self._url = url self._release = release @@ -196,7 +201,7 @@ class Artifactory: # check the credentials ENVs for early exit self.__path_helper("_deb", "") - def deploy_deb(self, packages: Packages): + def deploy_deb(self, packages: Packages) -> None: for package_file in packages.deb: path = packages.path(package_file) dist = self._release @@ -212,13 +217,13 @@ class Artifactory: ) self.deb_path(package_file).deploy_deb(path, dist, comp, arch) - def deploy_rpm(self, packages: Packages): + def deploy_rpm(self, packages: Packages) -> None: for package_file in packages.rpm: path = packages.path(package_file) logging.info("Deploy %s to artifactory", path) self.rpm_path(package_file).deploy_file(path) - def deploy_tgz(self, packages: Packages): + def deploy_tgz(self, packages: Packages) -> None: for package_file in packages.tgz: path = packages.path(package_file) logging.info("Deploy %s to artifactory", path) @@ -316,19 +321,19 @@ def parse_args() -> argparse.Namespace: return args -def process_deb(s3: S3, art_clients: List[Artifactory]): +def process_deb(s3: S3, art_clients: List[Artifactory]) -> None: s3.download_deb() for art_client in art_clients: art_client.deploy_deb(s3.packages) -def process_rpm(s3: S3, art_clients: List[Artifactory]): +def process_rpm(s3: S3, art_clients: List[Artifactory]) -> None: s3.download_rpm() for art_client in art_clients: art_client.deploy_rpm(s3.packages) -def process_tgz(s3: S3, art_clients: List[Artifactory]): +def process_tgz(s3: S3, art_clients: List[Artifactory]) -> None: s3.download_tgz() for art_client in art_clients: art_client.deploy_tgz(s3.packages) From 3bf3dacb4d6f2493824931641beba56c2d50f26f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Nov 2022 14:11:32 +0100 Subject: [PATCH 47/98] Add __init__.py to lambdas for mypy --- tests/ci/cancel_and_rerun_workflow_lambda/__init__.py | 0 tests/ci/runner_token_rotation_lambda/__init__.py | 0 tests/ci/team_keys_lambda/__init__.py | 0 tests/ci/terminate_runner_lambda/__init__.py | 0 4 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/ci/cancel_and_rerun_workflow_lambda/__init__.py create mode 100644 tests/ci/runner_token_rotation_lambda/__init__.py create mode 100644 tests/ci/team_keys_lambda/__init__.py create mode 100644 tests/ci/terminate_runner_lambda/__init__.py diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/__init__.py b/tests/ci/cancel_and_rerun_workflow_lambda/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/ci/runner_token_rotation_lambda/__init__.py b/tests/ci/runner_token_rotation_lambda/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/ci/team_keys_lambda/__init__.py b/tests/ci/team_keys_lambda/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/ci/terminate_runner_lambda/__init__.py b/tests/ci/terminate_runner_lambda/__init__.py new file mode 100644 index 00000000000..e69de29bb2d From eacf7e01a1c79a16ff2bc57fca8047aa27a08ab6 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 28 Nov 2022 17:17:45 +0100 Subject: [PATCH 48/98] Fix typing in team_keys_lambda --- tests/ci/team_keys_lambda/app.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/team_keys_lambda/app.py b/tests/ci/team_keys_lambda/app.py index 9e73a3f0993..870d41c441e 100644 --- a/tests/ci/team_keys_lambda/app.py +++ b/tests/ci/team_keys_lambda/app.py @@ -14,7 +14,7 @@ import boto3 # type: ignore class Keys(set): def __init__(self, *args, **kwargs): super().__init__(*args, **kwargs) - self.updated_at = 0 + self.updated_at = 0.0 def update_now(self): self.updated_at = datetime.now().timestamp() @@ -88,7 +88,7 @@ def get_token_from_aws() -> str: ) get_secret_value_response = client.get_secret_value(SecretId=secret_name) data = json.loads(get_secret_value_response["SecretString"]) - return data["clickhouse_robot_token"] + return data["clickhouse_robot_token"] # type: ignore def main(token: str, org: str, team_slug: str) -> str: From 3feb8e3657e11a1131b7e9a638c1d0a9c1f87bd9 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 28 Nov 2022 17:58:50 +0100 Subject: [PATCH 49/98] Add type hinting to ci_runners_metrics_lambda --- tests/ci/ci_runners_metrics_lambda/app.py | 58 ++++++++++++++--------- 1 file changed, 35 insertions(+), 23 deletions(-) diff --git a/tests/ci/ci_runners_metrics_lambda/app.py b/tests/ci/ci_runners_metrics_lambda/app.py index c1b20beb599..2bc568bb462 100644 --- a/tests/ci/ci_runners_metrics_lambda/app.py +++ b/tests/ci/ci_runners_metrics_lambda/app.py @@ -12,11 +12,12 @@ import json import time from collections import namedtuple from datetime import datetime +from typing import Dict, List, Tuple import jwt -import requests -import boto3 -from botocore.exceptions import ClientError +import requests # type: ignore +import boto3 # type: ignore +from botocore.exceptions import ClientError # type: ignore UNIVERSAL_LABEL = "universal" RUNNER_TYPE_LABELS = [ @@ -29,8 +30,13 @@ RUNNER_TYPE_LABELS = [ "style-checker-aarch64", ] +RunnerDescription = namedtuple( + "RunnerDescription", ["id", "name", "tags", "offline", "busy"] +) +RunnerDescriptions = List[RunnerDescription] -def get_dead_runners_in_ec2(runners): + +def get_dead_runners_in_ec2(runners: RunnerDescriptions) -> RunnerDescriptions: ids = { runner.name: runner for runner in runners @@ -92,7 +98,7 @@ def get_dead_runners_in_ec2(runners): return result_to_delete -def get_lost_ec2_instances(runners): +def get_lost_ec2_instances(runners: RunnerDescriptions) -> List[dict]: client = boto3.client("ec2") reservations = client.describe_instances( Filters=[{"Name": "tag-key", "Values": ["github:runner-type"]}] @@ -130,7 +136,7 @@ def get_lost_ec2_instances(runners): return lost_instances -def get_key_and_app_from_aws(): +def get_key_and_app_from_aws() -> Tuple[str, int]: secret_name = "clickhouse_github_secret_key" session = boto3.session.Session() client = session.client( @@ -146,7 +152,7 @@ def handler(event, context): main(private_key, app_id, True, True) -def get_installation_id(jwt_token): +def get_installation_id(jwt_token: str) -> int: headers = { "Authorization": f"Bearer {jwt_token}", "Accept": "application/vnd.github.v3+json", @@ -157,10 +163,12 @@ def get_installation_id(jwt_token): for installation in data: if installation["account"]["login"] == "ClickHouse": installation_id = installation["id"] - return installation_id + break + + return installation_id # type: ignore -def get_access_token(jwt_token, installation_id): +def get_access_token(jwt_token: str, installation_id: int) -> str: headers = { "Authorization": f"Bearer {jwt_token}", "Accept": "application/vnd.github.v3+json", @@ -171,15 +179,10 @@ def get_access_token(jwt_token, installation_id): ) response.raise_for_status() data = response.json() - return data["token"] + return data["token"] # type: ignore -RunnerDescription = namedtuple( - "RunnerDescription", ["id", "name", "tags", "offline", "busy"] -) - - -def list_runners(access_token): +def list_runners(access_token: str) -> RunnerDescriptions: headers = { "Authorization": f"token {access_token}", "Accept": "application/vnd.github.v3+json", @@ -225,8 +228,10 @@ def list_runners(access_token): return result -def group_runners_by_tag(listed_runners): - result = {} +def group_runners_by_tag( + listed_runners: RunnerDescriptions, +) -> Dict[str, RunnerDescriptions]: + result = {} # type: Dict[str, RunnerDescriptions] def add_to_result(tag, runner): if tag not in result: @@ -248,7 +253,9 @@ def group_runners_by_tag(listed_runners): return result -def push_metrics_to_cloudwatch(listed_runners, namespace): +def push_metrics_to_cloudwatch( + listed_runners: RunnerDescriptions, namespace: str +) -> None: client = boto3.client("cloudwatch") metrics_data = [] busy_runners = sum( @@ -278,7 +285,7 @@ def push_metrics_to_cloudwatch(listed_runners, namespace): } ) if total_active_runners == 0: - busy_ratio = 100 + busy_ratio = 100.0 else: busy_ratio = busy_runners / total_active_runners * 100 @@ -293,7 +300,7 @@ def push_metrics_to_cloudwatch(listed_runners, namespace): client.put_metric_data(Namespace=namespace, MetricData=metrics_data) -def delete_runner(access_token, runner): +def delete_runner(access_token: str, runner: RunnerDescription) -> bool: headers = { "Authorization": f"token {access_token}", "Accept": "application/vnd.github.v3+json", @@ -305,10 +312,15 @@ def delete_runner(access_token, runner): ) response.raise_for_status() print(f"Response code deleting {runner.name} is {response.status_code}") - return response.status_code == 204 + return bool(response.status_code == 204) -def main(github_secret_key, github_app_id, push_to_cloudwatch, delete_offline_runners): +def main( + github_secret_key: str, + github_app_id: int, + push_to_cloudwatch: bool, + delete_offline_runners: bool, +) -> None: payload = { "iat": int(time.time()) - 60, "exp": int(time.time()) + (10 * 60), From 23002bc808b6cfb8600d4fec77516bd6fc1385ab Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 29 Nov 2022 13:24:45 +0100 Subject: [PATCH 50/98] Add typing to terminate_runner_lambda --- tests/ci/terminate_runner_lambda/app.py | 52 ++++++++++++++----------- 1 file changed, 29 insertions(+), 23 deletions(-) diff --git a/tests/ci/terminate_runner_lambda/app.py b/tests/ci/terminate_runner_lambda/app.py index 4140690e891..223555ced74 100644 --- a/tests/ci/terminate_runner_lambda/app.py +++ b/tests/ci/terminate_runner_lambda/app.py @@ -1,17 +1,18 @@ #!/usr/bin/env python3 -import requests import argparse -import jwt import sys import json import time from collections import namedtuple +from typing import Any, Dict, List, Tuple + +import boto3 # type: ignore +import requests # type: ignore +import jwt -def get_key_and_app_from_aws(): - import boto3 - +def get_key_and_app_from_aws() -> Tuple[str, int]: secret_name = "clickhouse_github_secret_key" session = boto3.session.Session() client = session.client( @@ -22,7 +23,7 @@ def get_key_and_app_from_aws(): return data["clickhouse-app-key"], int(data["clickhouse-app-id"]) -def get_installation_id(jwt_token): +def get_installation_id(jwt_token: str) -> int: headers = { "Authorization": f"Bearer {jwt_token}", "Accept": "application/vnd.github.v3+json", @@ -33,10 +34,12 @@ def get_installation_id(jwt_token): for installation in data: if installation["account"]["login"] == "ClickHouse": installation_id = installation["id"] - return installation_id + break + + return installation_id # type: ignore -def get_access_token(jwt_token, installation_id): +def get_access_token(jwt_token: str, installation_id: int) -> str: headers = { "Authorization": f"Bearer {jwt_token}", "Accept": "application/vnd.github.v3+json", @@ -47,15 +50,16 @@ def get_access_token(jwt_token, installation_id): ) response.raise_for_status() data = response.json() - return data["token"] + return data["token"] # type: ignore RunnerDescription = namedtuple( "RunnerDescription", ["id", "name", "tags", "offline", "busy"] ) +RunnerDescriptions = List[RunnerDescription] -def list_runners(access_token): +def list_runners(access_token: str) -> RunnerDescriptions: headers = { "Authorization": f"token {access_token}", "Accept": "application/vnd.github.v3+json", @@ -94,9 +98,9 @@ def list_runners(access_token): return result -def how_many_instances_to_kill(event_data): +def how_many_instances_to_kill(event_data: dict) -> Dict[str, int]: data_array = event_data["CapacityToTerminate"] - to_kill_by_zone = {} + to_kill_by_zone = {} # type: Dict[str, int] for av_zone in data_array: zone_name = av_zone["AvailabilityZone"] to_kill = av_zone["Capacity"] @@ -104,15 +108,16 @@ def how_many_instances_to_kill(event_data): to_kill_by_zone[zone_name] = 0 to_kill_by_zone[zone_name] += to_kill + return to_kill_by_zone -def get_candidates_to_be_killed(event_data): +def get_candidates_to_be_killed(event_data: dict) -> Dict[str, List[str]]: data_array = event_data["Instances"] - instances_by_zone = {} + instances_by_zone = {} # type: Dict[str, List[str]] for instance in data_array: zone_name = instance["AvailabilityZone"] - instance_id = instance["InstanceId"] + instance_id = instance["InstanceId"] # type: str if zone_name not in instances_by_zone: instances_by_zone[zone_name] = [] instances_by_zone[zone_name].append(instance_id) @@ -120,7 +125,7 @@ def get_candidates_to_be_killed(event_data): return instances_by_zone -def delete_runner(access_token, runner): +def delete_runner(access_token: str, runner: RunnerDescription) -> bool: headers = { "Authorization": f"token {access_token}", "Accept": "application/vnd.github.v3+json", @@ -134,10 +139,12 @@ def delete_runner(access_token, runner): print( f"Response code deleting {runner.name} with id {runner.id} is {response.status_code}" ) - return response.status_code == 204 + return bool(response.status_code == 204) -def main(github_secret_key, github_app_id, event): +def main( + github_secret_key: str, github_app_id: int, event: dict +) -> Dict[str, List[str]]: print("Got event", json.dumps(event, sort_keys=True, indent=4)) to_kill_by_zone = how_many_instances_to_kill(event) instances_by_zone = get_candidates_to_be_killed(event) @@ -156,17 +163,16 @@ def main(github_secret_key, github_app_id, event): to_delete_runners = [] instances_to_kill = [] - for zone in to_kill_by_zone: - num_to_kill = to_kill_by_zone[zone] + for zone, num_to_kill in to_kill_by_zone.items(): candidates = instances_by_zone[zone] if num_to_kill > len(candidates): raise Exception( f"Required to kill {num_to_kill}, but have only {len(candidates)} candidates in AV {zone}" ) - delete_for_av = [] + delete_for_av = [] # type: RunnerDescriptions for candidate in candidates: - if candidate not in set([runner.name for runner in runners]): + if candidate not in set(runner.name for runner in runners): print( f"Candidate {candidate} was not in runners list, simply delete it" ) @@ -214,7 +220,7 @@ def main(github_secret_key, github_app_id, event): return response -def handler(event, context): +def handler(event: dict, context: Any) -> Dict[str, List[str]]: private_key, app_id = get_key_and_app_from_aws() return main(private_key, app_id, event) From 0f2704703ff245d4b70d787d59d0ff374cc804cf Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 29 Nov 2022 13:33:45 +0100 Subject: [PATCH 51/98] Fix typing and bug in cancel_and_rerun_workflow_lambda --- tests/ci/cancel_and_rerun_workflow_lambda/app.py | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/app.py b/tests/ci/cancel_and_rerun_workflow_lambda/app.py index ebdfe2fdb5b..d93a9062a3b 100644 --- a/tests/ci/cancel_and_rerun_workflow_lambda/app.py +++ b/tests/ci/cancel_and_rerun_workflow_lambda/app.py @@ -106,7 +106,7 @@ def _exec_get_with_retry(url: str, token: str) -> dict: try: response = requests.get(url, headers=headers) response.raise_for_status() - return response.json() + return response.json() # type: ignore except Exception as ex: print("Got exception executing request", ex) time.sleep(i + 1) @@ -130,8 +130,7 @@ WorkflowDescription = namedtuple( def get_workflows_description_for_pull_request( - pull_request_event, - token, + pull_request_event: dict, token: str ) -> List[WorkflowDescription]: head_repo = pull_request_event["head"]["repo"]["full_name"] head_branch = pull_request_event["head"]["ref"] @@ -193,7 +192,7 @@ def get_workflows_description_for_pull_request( def get_workflow_description_fallback( - pull_request_event, token + pull_request_event: dict, token: str ) -> List[WorkflowDescription]: head_repo = pull_request_event["head"]["repo"]["full_name"] head_branch = pull_request_event["head"]["ref"] @@ -241,7 +240,7 @@ def get_workflow_description_fallback( WorkflowDescription( url=wf["url"], run_id=wf["id"], - name=workflow["name"], + name=wf["name"], head_sha=wf["head_sha"], status=wf["status"], rerun_url=wf["rerun_url"], @@ -254,7 +253,7 @@ def get_workflow_description_fallback( return workflow_descriptions -def get_workflow_description(workflow_url, token) -> WorkflowDescription: +def get_workflow_description(workflow_url: str, token: str) -> WorkflowDescription: workflow = _exec_get_with_retry(workflow_url, token) return WorkflowDescription( url=workflow["url"], @@ -331,7 +330,7 @@ def main(event): workflow_descriptions or get_workflow_description_fallback(pull_request, token) ) - workflow_descriptions.sort(key=lambda x: x.run_id) + workflow_descriptions.sort(key=lambda x: x.run_id) # type: ignore most_recent_workflow = workflow_descriptions[-1] if ( most_recent_workflow.status == "completed" @@ -376,7 +375,7 @@ def main(event): print("Not found any workflows") return - workflow_descriptions.sort(key=lambda x: x.run_id) + workflow_descriptions.sort(key=lambda x: x.run_id) # type: ignore most_recent_workflow = workflow_descriptions[-1] print("Latest workflow", most_recent_workflow) if ( From e46f615176ab629d696cfd0c1d822374a92307a1 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 29 Nov 2022 14:08:04 +0100 Subject: [PATCH 52/98] Simplify checks list in style-check image --- .../test/style/process_style_check_result.py | 19 ++++++++++--------- docker/test/style/run.sh | 4 ++-- 2 files changed, 12 insertions(+), 11 deletions(-) diff --git a/docker/test/style/process_style_check_result.py b/docker/test/style/process_style_check_result.py index 8c2110d64e5..e07e6153cee 100755 --- a/docker/test/style/process_style_check_result.py +++ b/docker/test/style/process_style_check_result.py @@ -11,17 +11,18 @@ def process_result(result_folder): description = "" test_results = [] checks = ( - ("header duplicates", "duplicate_output.txt"), - ("shellcheck", "shellcheck_output.txt"), - ("style", "style_output.txt"), - ("black", "black_output.txt"), - ("typos", "typos_output.txt"), - ("whitespaces", "whitespaces_output.txt"), - ("workflows", "workflows_output.txt"), - ("doc typos", "doc_spell_output.txt"), + "header duplicates", + "shellcheck", + "style", + "black", + "typos", + "whitespaces", + "workflows", + "doc typos", ) - for name, out_file in checks: + for name in checks: + out_file = name.replace(" ", "_") + "_output.txt" full_path = os.path.join(result_folder, out_file) if not os.path.exists(full_path): logging.info("No %s check log on path %s", name, full_path) diff --git a/docker/test/style/run.sh b/docker/test/style/run.sh index 06ecadbfebf..9f0490af120 100755 --- a/docker/test/style/run.sh +++ b/docker/test/style/run.sh @@ -4,7 +4,7 @@ cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_output/check_status.tsv echo "Check duplicates" | ts -./check-duplicate-includes.sh |& tee /test_output/duplicate_output.txt +./check-duplicate-includes.sh |& tee /test_output/header_duplicates_output.txt echo "Check style" | ts ./check-style -n |& tee /test_output/style_output.txt echo "Check python formatting with black" | ts @@ -12,7 +12,7 @@ echo "Check python formatting with black" | ts echo "Check typos" | ts ./check-typos |& tee /test_output/typos_output.txt echo "Check docs spelling" | ts -./check-doc-aspell |& tee /test_output/doc_spell_output.txt +./check-doc-aspell |& tee /test_output/doc_typos_output.txt echo "Check whitespaces" | ts ./check-whitespaces -n |& tee /test_output/whitespaces_output.txt echo "Check workflows" | ts From 9cb2aa1c464420309c1a610bcfd213dee8ee93ad Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 29 Nov 2022 15:48:52 +0100 Subject: [PATCH 53/98] Add python mypy check to CI --- docker/test/style/Dockerfile | 2 +- .../test/style/process_style_check_result.py | 1 + docker/test/style/run.sh | 2 ++ utils/check-style/check-mypy | 23 +++++++++++++++++++ 4 files changed, 27 insertions(+), 1 deletion(-) create mode 100755 utils/check-style/check-mypy diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index cb8c914e53d..e8c5e17024c 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -17,7 +17,7 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ python3-pip \ shellcheck \ yamllint \ - && pip3 install black==22.8.0 boto3 codespell==2.2.1 dohq-artifactory PyGithub unidiff pylint==2.6.2 \ + && pip3 install black==22.8.0 boto3 codespell==2.2.1 dohq-artifactory mypy PyGithub unidiff pylint==2.6.2 \ && apt-get clean \ && rm -rf /root/.cache/pip diff --git a/docker/test/style/process_style_check_result.py b/docker/test/style/process_style_check_result.py index e07e6153cee..d914ceffb86 100755 --- a/docker/test/style/process_style_check_result.py +++ b/docker/test/style/process_style_check_result.py @@ -15,6 +15,7 @@ def process_result(result_folder): "shellcheck", "style", "black", + "mypy", "typos", "whitespaces", "workflows", diff --git a/docker/test/style/run.sh b/docker/test/style/run.sh index 9f0490af120..911536ed03b 100755 --- a/docker/test/style/run.sh +++ b/docker/test/style/run.sh @@ -9,6 +9,8 @@ echo "Check style" | ts ./check-style -n |& tee /test_output/style_output.txt echo "Check python formatting with black" | ts ./check-black -n |& tee /test_output/black_output.txt +echo "Check python type hinting with mypy" | ts +./check-mypy -n |& tee /test_output/mypy_output.txt echo "Check typos" | ts ./check-typos |& tee /test_output/typos_output.txt echo "Check docs spelling" | ts diff --git a/utils/check-style/check-mypy b/utils/check-style/check-mypy new file mode 100755 index 00000000000..42cb7fbbd15 --- /dev/null +++ b/utils/check-style/check-mypy @@ -0,0 +1,23 @@ +#!/usr/bin/env bash + +# The mypy supports pyproject.toml, but unfortunately it doesn't support it recursively +# https://github.com/python/mypy/issues/10613 +# +# Unless it's done, mypy only runs against tests/ci +# Let's leave here a room for improvement and redo it when mypy will test anything else + +GIT_ROOT=$(git rev-parse --show-cdup) +GIT_ROOT=${GIT_ROOT:-.} +CONFIG="$GIT_ROOT/tests/ci/.mypy.ini" +DIRS=("$GIT_ROOT/tests/ci/" "$GIT_ROOT/tests/ci/"*/) +tmp=$(mktemp) +for dir in "${DIRS[@]}"; do + if ! compgen -G "$dir"/*.py > /dev/null; then + continue + fi + if ! mypy --config-file="$CONFIG" --sqlite-cache "$dir"/*.py > "$tmp" 2>&1; then + echo "Errors while processing $dir": + cat "$tmp" + fi +done +rm -rf "$tmp" From fe8107e95f68f2571264e29eb755a2f979f1fa61 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 29 Nov 2022 17:18:29 +0100 Subject: [PATCH 54/98] Fix type ignorance in version_helper.py --- tests/ci/version_helper.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/version_helper.py b/tests/ci/version_helper.py index dc4674b3699..69cfba64be3 100755 --- a/tests/ci/version_helper.py +++ b/tests/ci/version_helper.py @@ -147,7 +147,7 @@ class ClickHouseVersion: def __eq__(self, other: Any) -> bool: if not isinstance(self, type(other)): return NotImplemented - return ( # type: ignore + return bool( self.major == other.major and self.minor == other.minor and self.patch == other.patch From 7b4025ba7ceabe48ea90d1c56d9caf15486e4439 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 30 Nov 2022 12:43:46 +0100 Subject: [PATCH 55/98] Fix bug --- src/Storages/MergeTree/MutateTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 916befdf78f..f6befe67fd4 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1322,7 +1322,7 @@ private: for (auto p_it = projection_data_part_storage_src->iterate(); p_it->isValid(); p_it->next()) { - auto file_name_with_projection_prefix = fs::path(projection_data_part_storage_src->getPartDirectory()) / it->name(); + auto file_name_with_projection_prefix = fs::path(projection_data_part_storage_src->getPartDirectory()) / p_it->name(); projection_data_part_storage_dst->createHardLinkFrom( *projection_data_part_storage_src, p_it->name(), p_it->name()); From fd262cb56b28e6d9022279b4de225c099b5b8e38 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 30 Nov 2022 14:48:46 +0000 Subject: [PATCH 56/98] update docs for trace_log --- docs/en/operations/settings/settings.md | 37 ++++++++++++++++++- docs/en/operations/system-tables/trace_log.md | 11 +++++- 2 files changed, 46 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 7494f3db71a..06b661453da 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -668,7 +668,7 @@ log_query_views=1 ## log_formatted_queries {#settings-log-formatted-queries} -Allows to log formatted queries to the [system.query_log](../../operations/system-tables/query_log.md) system table (populates `formatted_query` column in the [system.query_log](../../operations/system-tables/query_log.md)). +Allows to log formatted queries to the [system.query_log](../../operations/system-tables/query_log.md) system table (populates `formatted_query` column in the [system.query_log](../../operations/system-tables/query_log.md)). Possible values: @@ -1807,6 +1807,41 @@ See also: - System table [trace_log](../../operations/system-tables/trace_log.md/#system_tables-trace_log) +## memory_profiler_step {#memory_profiler_step} + +Sets the step of memory profiler. Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stacktrace and will write it into [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log). + +Possible values: + +- A positive integer number of bytes. + +- 0 for turning off the memory profiler. + +Default value: 4,194,304 bytes (4 MiB). + +## memory_profiler_sample_probability {#memory_profiler_sample_probability} + +Sets the probability of collecting stacktraces at random allocations and deallocations and writing them into [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log). + +Possible values: + +- A positive floating-point number in the range [0..1]. + +- 0.0 for turning off the memory sampling. + +Default value: 0.0. + +## trace_profile_events {#trace_profile_events} + +Enables or disables collecting stacktraces on each update of profile events along with the name of profile event and the value of increment and sending them into [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log). + +Possible values: + +- 1 — Tracing of profile events enabled. +- 0 — Tracing of profile events disabled. + +Default value: 0. + ## allow_introspection_functions {#settings-allow_introspection_functions} Enables or disables [introspections functions](../../sql-reference/functions/introspection.md) for query profiling. diff --git a/docs/en/operations/system-tables/trace_log.md b/docs/en/operations/system-tables/trace_log.md index 0effe085b80..6299aafcae2 100644 --- a/docs/en/operations/system-tables/trace_log.md +++ b/docs/en/operations/system-tables/trace_log.md @@ -5,7 +5,8 @@ slug: /en/operations/system-tables/trace_log Contains stack traces collected by the sampling query profiler. -ClickHouse creates this table when the [trace_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) server configuration section is set. Also the [query_profiler_real_time_period_ns](../../operations/settings/settings.md#query_profiler_real_time_period_ns) and [query_profiler_cpu_time_period_ns](../../operations/settings/settings.md#query_profiler_cpu_time_period_ns) settings should be set. +ClickHouse creates this table when the [trace_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-trace_log) server configuration section is set. Also see settings: [query_profiler_real_time_period_ns](../../operations/settings/settings.md#query_profiler_real_time_period_ns), [query_profiler_cpu_time_period_ns](../../operations/settings/settings.md#query_profiler_cpu_time_period_ns), [memory_profiler_step](../../operations/settings/settings.md#memory_profiler_step), +[memory_profiler_sample_probability](../../operations/settings/settings.md#memory_profiler_sample_probability), [trace_profile_events](../../operations/settings/settings.md#trace_profile_events). To analyze logs, use the `addressToLine`, `addressToLineWithInlines`, `addressToSymbol` and `demangle` introspection functions. @@ -29,6 +30,8 @@ Columns: - `CPU` represents collecting stack traces by CPU time. - `Memory` represents collecting allocations and deallocations when memory allocation exceeds the subsequent watermark. - `MemorySample` represents collecting random allocations and deallocations. + - `MemoryPeak` represents collecting updates of peak memory usage. + - `ProfileEvent` represents collecting of increments of profile events. - `thread_number` ([UInt32](../../sql-reference/data-types/int-uint.md)) — Thread identifier. @@ -36,6 +39,12 @@ Columns: - `trace` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — Stack trace at the moment of sampling. Each element is a virtual memory address inside ClickHouse server process. +- `size` ([Int64](../../sql-reference/data-types/int-uint.md)) - For trace types `Memory`, `MemorySample` or `MemoryPeak` is the amount of memory allocated, for other trace types is 0. + +- `event` ([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md)) - For trace type `ProfileEvent` is the name of updated profile event, for other trace types is an empty string. + +- `increment` ([UInt64](../../sql-reference/data-types/int-uint.md)) - For trace type `ProfileEvent` is the amount of incremnt of profile event, for other trace types is 0. + **Example** ``` sql From 46d1c92ba281a21d24debcd7a647b5e43f3d0b11 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 30 Nov 2022 17:48:52 +0300 Subject: [PATCH 57/98] Update DatabaseCatalog.cpp --- src/Interpreters/DatabaseCatalog.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 9015f2eeee2..2118c633a0f 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -869,6 +869,8 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr { chassert(hasUUIDMapping(table_id.uuid)); drop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + /// Do not postpone removal of in-memory tables + ignore_delay = ignore_delay || !table->storesDataOnDisk(); table->is_dropped = true; } else From 2a8774483e199fbf17374c960181f5a645b96cad Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 30 Nov 2022 17:31:35 +0000 Subject: [PATCH 58/98] mark test as no-parallel --- tests/queries/0_stateless/02494_trace_log_profile_events.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02494_trace_log_profile_events.sh b/tests/queries/0_stateless/02494_trace_log_profile_events.sh index 58e1c739bc0..62afc5ea8aa 100755 --- a/tests/queries/0_stateless/02494_trace_log_profile_events.sh +++ b/tests/queries/0_stateless/02494_trace_log_profile_events.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-tsan +# Tags: no-tsan, no-parallel CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 15f8030e04e6b01091e3289bdc573a74e4f4a4e9 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 30 Nov 2022 21:47:50 +0300 Subject: [PATCH 59/98] [docs] minor improvements to "Cache Types" article --- docs/en/operations/caches.md | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/docs/en/operations/caches.md b/docs/en/operations/caches.md index 3aeae7d1c9d..0001c383486 100644 --- a/docs/en/operations/caches.md +++ b/docs/en/operations/caches.md @@ -11,6 +11,7 @@ Main cache types: - `mark_cache` — Cache of marks used by table engines of the [MergeTree](../engines/table-engines/mergetree-family/mergetree.md) family. - `uncompressed_cache` — Cache of uncompressed data used by table engines of the [MergeTree](../engines/table-engines/mergetree-family/mergetree.md) family. +- Operating system page cache (used indirectly, for files with actual data). Additional cache types: @@ -22,10 +23,4 @@ Additional cache types: - Schema inference cache. - [Filesystem cache](storing-data.md) over S3, Azure, Local and other disks. -Indirectly used: - -- OS page cache. - -To drop cache, use [SYSTEM DROP ... CACHE](../sql-reference/statements/system.md) statements. - -[Original article](https://clickhouse.com/docs/en/operations/caches/) +To drop one of caches, use [SYSTEM DROP ... CACHE](../sql-reference/statements/system.md#drop-mark-cache) statements. From ec15d52642a19c4712630ed2f45f7500263373d6 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 30 Nov 2022 16:46:24 +0100 Subject: [PATCH 60/98] FuseFunctionsPass small fix --- src/Analyzer/Passes/FuseFunctionsPass.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Analyzer/Passes/FuseFunctionsPass.cpp b/src/Analyzer/Passes/FuseFunctionsPass.cpp index 6e26e3cd8d8..f7e703cdaa4 100644 --- a/src/Analyzer/Passes/FuseFunctionsPass.cpp +++ b/src/Analyzer/Passes/FuseFunctionsPass.cpp @@ -79,8 +79,6 @@ FunctionNodePtr createResolvedAggregateFunction(const String & name, const Query function_node->resolveAsAggregateFunction(aggregate_function, aggregate_function->getReturnType()); function_node->getArguments().getNodes() = { argument }; - function_node->getArguments().getNodes() = { argument }; - if (!parameters.empty()) { QueryTreeNodes parameter_nodes; From bbcd389fa4965efba8724b576fb4080c7a18fe87 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 30 Nov 2022 20:26:58 +0000 Subject: [PATCH 61/98] remove flaky test --- .../02494_trace_log_profile_events.reference | 1 - .../02494_trace_log_profile_events.sh | 31 ------------------- 2 files changed, 32 deletions(-) diff --git a/tests/queries/0_stateless/02494_trace_log_profile_events.reference b/tests/queries/0_stateless/02494_trace_log_profile_events.reference index 5923753dbc7..cd121fd3feb 100644 --- a/tests/queries/0_stateless/02494_trace_log_profile_events.reference +++ b/tests/queries/0_stateless/02494_trace_log_profile_events.reference @@ -1,3 +1,2 @@ 1 1 1 -0 diff --git a/tests/queries/0_stateless/02494_trace_log_profile_events.sh b/tests/queries/0_stateless/02494_trace_log_profile_events.sh index 62afc5ea8aa..4dd0a34d202 100755 --- a/tests/queries/0_stateless/02494_trace_log_profile_events.sh +++ b/tests/queries/0_stateless/02494_trace_log_profile_events.sh @@ -16,34 +16,3 @@ ${CLICKHOUSE_CLIENT} --query_id $query_id --query "SELECT 1 FORMAT Null SETTINGS ${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" ${CLICKHOUSE_CLIENT} --query "SELECT count() > 0, sum(empty(trace)) = 0 FROM system.trace_log WHERE query_id = '$query_id' AND trace_type = 'ProfileEvent'" - -query_id="$RANDOM-$CLICKHOUSE_DATABASE" -${CLICKHOUSE_CLIENT} --query_id $query_id --query "SELECT count() FROM numbers_mt(1000000) FORMAT Null SETTINGS trace_profile_events = 1" - -${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} --query " -SELECT count() -FROM -( - ( - SELECT - Events.1 AS event, - Events.2 AS value - FROM system.query_log - ARRAY JOIN CAST(ProfileEvents, 'Array(Tuple(String, Int64))') AS Events - WHERE query_id = '$query_id' AND type = 'QueryFinish' - ORDER BY event - ) - EXCEPT - ( - SELECT - event, - sum(increment) AS value - FROM system.trace_log - WHERE (trace_type = 'ProfileEvent') AND (query_id = '$query_id') AND increment != 0 - GROUP BY event - ORDER BY event ASC - ) -) -WHERE event NOT IN ('ContextLock', 'NetworkSendBytes', 'NetworkSendElapsedMicroseconds', 'QueryProfilerRuns'); -" From 0a35a072d998230a4f190a2b12effa3be5ccef8e Mon Sep 17 00:00:00 2001 From: Elena Date: Wed, 30 Nov 2022 19:42:31 +0100 Subject: [PATCH 62/98] Add RESET_SETTING to check if is a settings alter --- src/Parsers/ASTAlterQuery.cpp | 2 +- .../test_alter_settings_on_cluster/test.py | 11 +++++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 2d8193871b0..b7d7c2e8f2d 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -509,7 +509,7 @@ bool ASTAlterQuery::isOneCommandTypeOnly(const ASTAlterCommand::Type & type) con bool ASTAlterQuery::isSettingsAlter() const { - return isOneCommandTypeOnly(ASTAlterCommand::MODIFY_SETTING); + return isOneCommandTypeOnly(ASTAlterCommand::MODIFY_SETTING) || isOneCommandTypeOnly(ASTAlterCommand::RESET_SETTING); } bool ASTAlterQuery::isFreezeAlter() const diff --git a/tests/integration/test_alter_settings_on_cluster/test.py b/tests/integration/test_alter_settings_on_cluster/test.py index 6ab3d446b59..51911dec6eb 100644 --- a/tests/integration/test_alter_settings_on_cluster/test.py +++ b/tests/integration/test_alter_settings_on_cluster/test.py @@ -52,3 +52,14 @@ def test_default_database_on_cluster(started_cluster): database="test_default_database", sql="SHOW CREATE test_local_table FORMAT TSV", ).endswith("old_parts_lifetime = 100\n") + + ch1.query( + database="test_default_database", + sql="ALTER TABLE test_local_table ON CLUSTER 'cluster' RESET SETTING old_parts_lifetime;", + ) + + for node in [ch1, ch2]: + assert not node.query( + database="test_default_database", + sql="SHOW CREATE test_local_table FORMAT TSV", + ).endswith("old_parts_lifetime = 100\n") From 304e4a73d75f5f182130efb6bf0765b103b92d64 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Wed, 30 Nov 2022 13:22:04 -0800 Subject: [PATCH 63/98] Fix setting TCP_KEEPAIDLE for s390x --- src/Client/Connection.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 8ddd0334396..6d6ce2e006c 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -148,7 +148,8 @@ void Connection::connect(const ConnectionTimeouts & timeouts) socket->setReceiveTimeout(timeouts.receive_timeout); socket->setSendTimeout(timeouts.send_timeout); socket->setNoDelay(true); - if (timeouts.tcp_keep_alive_timeout.totalSeconds()) + int tcp_keep_alive_timeout_in_sec = timeouts.tcp_keep_alive_timeout.totalSeconds(); + if (tcp_keep_alive_timeout_in_sec) { socket->setKeepAlive(true); socket->setOption(IPPROTO_TCP, @@ -157,7 +158,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts) #else TCP_KEEPIDLE // __APPLE__ #endif - , timeouts.tcp_keep_alive_timeout); + , tcp_keep_alive_timeout_in_sec); } in = std::make_shared(*socket); From 25ae52111e1cefe4493518995b219bab02e0064f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Dec 2022 11:18:23 +0000 Subject: [PATCH 64/98] Protect writer with mutex --- src/Coordination/Changelog.cpp | 109 ++++++++++++++++++--------------- src/Coordination/Changelog.h | 4 +- 2 files changed, 61 insertions(+), 52 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 68c45326b15..2fcd7784796 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -306,6 +306,7 @@ Changelog::Changelog( void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uint64_t logs_to_keep) { + std::lock_guard writer_lock(writer_mutex); std::optional last_log_read_result; /// Last log has some free space to write @@ -339,7 +340,7 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin removeAllLogs(); min_log_id = last_commited_log_index; max_log_id = last_commited_log_index == 0 ? 0 : last_commited_log_index - 1; - rotate(max_log_id + 1); + rotate(max_log_id + 1, writer_lock); return; } else if (changelog_description.from_log_index > start_to_read_from) @@ -430,7 +431,7 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin /// Start new log if we don't initialize writer from previous log. All logs can be "complete". if (!current_writer) - rotate(max_log_id + 1); + rotate(max_log_id + 1, writer_lock); initialized = true; } @@ -505,7 +506,7 @@ void Changelog::removeAllLogs() logs.clear(); } -void Changelog::rotate(uint64_t new_start_log_index) +void Changelog::rotate(uint64_t new_start_log_index, std::lock_guard &) { /// Flush previous log if (current_writer) @@ -554,37 +555,39 @@ void Changelog::writeThread() WriteOperation write_operation; while (write_operations.pop(write_operation)) { - std::visit([&, this](const WriteOperationType & operation) -> void + std::lock_guard writer_lock(writer_mutex); + assert(initialized && current_writer); + + if (auto * append_log = std::get_if(&write_operation)) { - if constexpr (std::same_as) + const auto & current_changelog_description = existing_changelogs[current_writer->getStartIndex()]; + const bool log_is_complete = append_log->index - current_writer->getStartIndex() == current_changelog_description.expectedEntriesCountInLog(); + + if (log_is_complete) + rotate(append_log->index, writer_lock); + + current_writer->appendRecord(buildRecord(append_log->index, append_log->log_entry)); + } + else + { + const auto & flush = std::get(write_operation); + + if (current_writer) + current_writer->flush(force_sync); + { - const auto & current_changelog_description = existing_changelogs[current_writer->getStartIndex()]; - const bool log_is_complete = operation.index - current_writer->getStartIndex() == current_changelog_description.expectedEntriesCountInLog(); - - if (log_is_complete) - rotate(operation.index); - - current_writer->appendRecord(buildRecord(operation.index, operation.log_entry)); + std::lock_guard lock{durable_idx_mutex}; + last_durable_idx = flush.index; } + + durable_idx_cv.notify_all(); + + // we shouldn't start the raft_server before sending it here + if (auto raft_server_locked = raft_server.lock()) + raft_server_locked->notify_log_append_completion(true); else - { - if (current_writer) - current_writer->flush(force_sync); - - { - std::lock_guard lock{durable_idx_mutex}; - last_durable_idx = operation.index; - } - - durable_idx_cv.notify_all(); - - // we shouldn't start the raft_server before sending it here - if (auto raft_server_locked = raft_server.lock()) - raft_server_locked->notify_log_append_completion(true); - else - LOG_WARNING(log, "Raft server is not set in LogStore."); - } - }, write_operation); + LOG_WARNING(log, "Raft server is not set in LogStore."); + } } } @@ -606,29 +609,32 @@ void Changelog::appendEntry(uint64_t index, const LogEntryPtr & log_entry) void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry) { - /// This write_at require to overwrite everything in this file and also in previous file(s) - const bool go_to_previous_file = index < current_writer->getStartIndex(); - - if (go_to_previous_file) { - auto index_changelog = existing_changelogs.lower_bound(index); + std::lock_guard lock(writer_mutex); + /// This write_at require to overwrite everything in this file and also in previous file(s) + const bool go_to_previous_file = index < current_writer->getStartIndex(); - ChangelogFileDescription description; - - if (index_changelog->first == index) /// exactly this file starts from index - description = index_changelog->second; - else - description = std::prev(index_changelog)->second; - - /// Initialize writer from this log file - current_writer = std::make_unique(description.path, WriteMode::Append, index_changelog->first); - - /// Remove all subsequent files if overwritten something in previous one - auto to_remove_itr = existing_changelogs.upper_bound(index); - for (auto itr = to_remove_itr; itr != existing_changelogs.end();) + if (go_to_previous_file) { - std::filesystem::remove(itr->second.path); - itr = existing_changelogs.erase(itr); + auto index_changelog = existing_changelogs.lower_bound(index); + + ChangelogFileDescription description; + + if (index_changelog->first == index) /// exactly this file starts from index + description = index_changelog->second; + else + description = std::prev(index_changelog)->second; + + /// Initialize writer from this log file + current_writer = std::make_unique(description.path, WriteMode::Append, index_changelog->first); + + /// Remove all subsequent files if overwritten something in previous one + auto to_remove_itr = existing_changelogs.upper_bound(index); + for (auto itr = to_remove_itr; itr != existing_changelogs.end();) + { + std::filesystem::remove(itr->second.path); + itr = existing_changelogs.erase(itr); + } } } @@ -642,6 +648,7 @@ void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry) void Changelog::compact(uint64_t up_to_log_index) { + std::lock_guard lock(writer_mutex); LOG_INFO(log, "Compact logs up to log index {}, our max log id is {}", up_to_log_index, max_log_id); bool remove_all_logs = false; @@ -688,7 +695,7 @@ void Changelog::compact(uint64_t up_to_log_index) std::erase_if(logs, [up_to_log_index] (const auto & item) { return item.first <= up_to_log_index; }); if (need_rotate) - rotate(up_to_log_index + 1); + rotate(up_to_log_index + 1, lock); LOG_INFO(log, "Compaction up to {} finished new min index {}, new max index {}", up_to_log_index, min_log_id, max_log_id); } diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index fc85971ee94..a9464a59003 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -148,7 +148,7 @@ private: static ChangelogRecord buildRecord(uint64_t index, const LogEntryPtr & log_entry); /// Starts new file [new_start_log_index, new_start_log_index + rotate_interval] - void rotate(uint64_t new_start_log_index); + void rotate(uint64_t new_start_log_index, std::lock_guard & writer_lock); /// Currently existing changelogs std::map existing_changelogs; @@ -174,6 +174,7 @@ private: Poco::Logger * log; bool compress_logs; + std::mutex writer_mutex; /// Current writer for changelog file std::unique_ptr current_writer; /// Mapping log_id -> log_entry @@ -205,6 +206,7 @@ private: ThreadFromGlobalPool write_thread; ConcurrentBoundedQueue write_operations; + // last_durable_index needs to be exposed through const getter so we make mutex mutable mutable std::mutex durable_idx_mutex; std::condition_variable durable_idx_cv; uint64_t last_durable_idx{0}; From 940fdd4abd0f8e208f574129f1b09a37cbc7f807 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Dec 2022 12:24:27 +0100 Subject: [PATCH 65/98] Remove comment --- src/Coordination/Changelog.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 2fcd7784796..4f36e635eb9 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -549,9 +549,6 @@ ChangelogRecord Changelog::buildRecord(uint64_t index, const LogEntryPtr & log_e void Changelog::writeThread() { - // we don't protect current_writer because rotate at the same time can be called from compact only - // when the node is applying snapshot from leader - // in that case, no entry write should happen, i.e. this thread will be inactive WriteOperation write_operation; while (write_operations.pop(write_operation)) { From 98fe3c6c02b10c67f3d51c52bf2a9362cb792006 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 21 Nov 2022 17:45:59 +0000 Subject: [PATCH 66/98] Temporary files evict fs cache --- programs/local/LocalServer.cpp | 2 +- programs/server/Server.cpp | 58 +++------- src/Common/filesystemHelpers.cpp | 6 +- src/Common/filesystemHelpers.h | 2 +- .../IO/CachedOnDiskWriteBufferFromFile.cpp | 42 +++++-- .../IO/CachedOnDiskWriteBufferFromFile.h | 8 +- src/Disks/IO/FileCachePlaceholder.cpp | 7 ++ src/Disks/IO/FileCachePlaceholder.h | 104 ++++++++++++++++++ .../Cached/CachedObjectStorage.h | 2 + .../ObjectStorages/DiskObjectStorage.cpp | 8 ++ src/Disks/ObjectStorages/DiskObjectStorage.h | 1 + src/Disks/TemporaryFileInPath.cpp | 10 ++ src/Disks/TemporaryFileInPath.h | 27 +++++ src/Disks/TemporaryFileOnDisk.cpp | 2 +- src/Disks/TemporaryFileOnDisk.h | 15 ++- src/Formats/NativeWriter.cpp | 8 +- src/Formats/NativeWriter.h | 2 +- src/Interpreters/Cache/FileCache.cpp | 8 ++ src/Interpreters/Cache/FileCache.h | 2 + src/Interpreters/Cache/FileCacheFactory.cpp | 11 +- src/Interpreters/Cache/FileCacheFactory.h | 1 + src/Interpreters/Cache/FileSegment.cpp | 49 ++++++--- src/Interpreters/Cache/FileSegment.h | 15 +-- src/Interpreters/Context.cpp | 104 ++++++++++++++---- src/Interpreters/Context.h | 4 +- src/Interpreters/TemporaryDataOnDisk.cpp | 76 +++++++++---- src/Interpreters/TemporaryDataOnDisk.h | 22 +++- .../tests/gtest_lru_file_cache.cpp | 44 ++++++++ 28 files changed, 499 insertions(+), 141 deletions(-) create mode 100644 src/Disks/IO/FileCachePlaceholder.cpp create mode 100644 src/Disks/IO/FileCachePlaceholder.h create mode 100644 src/Disks/TemporaryFileInPath.cpp create mode 100644 src/Disks/TemporaryFileInPath.h diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index ce7e27026f1..22c94e01a51 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -207,7 +207,7 @@ void LocalServer::tryInitPath() global_context->setPath(path); - global_context->setTemporaryStorage(path + "tmp", "", 0); + global_context->setTemporaryStoragePath(path + "tmp/", 0); global_context->setFlagsPath(path + "flags"); global_context->setUserFilesPath(""); // user's files are everywhere diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b6ce358a5ef..e772208c30e 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -203,46 +203,6 @@ int mainEntryClickHouseServer(int argc, char ** argv) namespace { -void setupTmpPath(Poco::Logger * log, const std::string & path) -try -{ - LOG_DEBUG(log, "Setting up {} to store temporary data in it", path); - - fs::create_directories(path); - - /// Clearing old temporary files. - fs::directory_iterator dir_end; - size_t unknown_files = 0; - for (fs::directory_iterator it(path); it != dir_end; ++it) - { - if (it->is_regular_file() && startsWith(it->path().filename(), "tmp")) - { - LOG_DEBUG(log, "Removing old temporary file {}", it->path().string()); - fs::remove(it->path()); - } - else - { - unknown_files++; - if (unknown_files < 100) - LOG_DEBUG(log, "Found unknown {} {} in temporary path", - it->is_regular_file() ? "file" : (it->is_directory() ? "directory" : "element"), - it->path().string()); - } - } - - if (unknown_files) - LOG_DEBUG(log, "Found {} unknown files in temporary path", unknown_files); -} -catch (...) -{ - DB::tryLogCurrentException( - log, - fmt::format( - "Caught exception while setup temporary path: {}. It is ok to skip this exception as cleaning old temporary files is not " - "necessary", - path)); -} - size_t waitServersToFinish(std::vector & servers, size_t seconds_to_wait) { const size_t sleep_max_ms = 1000 * seconds_to_wait; @@ -1037,13 +997,21 @@ try LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::instance().getTimeZone()); /// Storage with temporary data for processing of heavy queries. + if (auto temporary_policy = config().getString("tmp_policy", ""); !temporary_policy.empty()) + { + size_t max_size = config().getUInt64("max_temporary_data_on_disk_size", 0); + global_context->setTemporaryStoragePolicy(temporary_policy, max_size); + } + else if (auto temporary_cache = config().getString("tmp_cache", ""); !temporary_cache.empty()) + { + size_t max_size = config().getUInt64("max_temporary_data_on_disk_size", 0); + global_context->setTemporaryStorageInCache(temporary_cache, max_size); + } + else { std::string temporary_path = config().getString("tmp_path", path / "tmp/"); - std::string temporary_policy = config().getString("tmp_policy", ""); size_t max_size = config().getUInt64("max_temporary_data_on_disk_size", 0); - const VolumePtr & volume = global_context->setTemporaryStorage(temporary_path, temporary_policy, max_size); - for (const DiskPtr & disk : volume->getDisks()) - setupTmpPath(log, disk->getPath()); + global_context->setTemporaryStoragePath(temporary_path, max_size); } /** Directory with 'flags': files indicating temporary settings for the server set by system administrator. @@ -1442,7 +1410,7 @@ try } catch (...) { - tryLogCurrentException(log); + tryLogCurrentException(log, "Caught exception while setting up access control."); throw; } diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 07a08dc7fbc..b259d1929ea 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -64,11 +64,11 @@ bool enoughSpaceInDirectory(const std::string & path, size_t data_size) return data_size <= free_space; } -std::unique_ptr createTemporaryFile(const std::string & path) +std::unique_ptr createTemporaryFile(const std::string & folder_path) { ProfileEvents::increment(ProfileEvents::ExternalProcessingFilesTotal); - fs::create_directories(path); - return std::make_unique(path); + fs::create_directories(folder_path); + return std::make_unique(folder_path); } #if !defined(OS_LINUX) diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index 0e6e16941bb..fd080771b52 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -17,7 +17,7 @@ namespace DB using TemporaryFile = Poco::TemporaryFile; bool enoughSpaceInDirectory(const std::string & path, size_t data_size); -std::unique_ptr createTemporaryFile(const std::string & path); +std::unique_ptr createTemporaryFile(const std::string & folder_path); // Determine what block device is responsible for specified path diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index 994bb743c5f..609ef2cdaa7 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -52,9 +52,16 @@ FileSegmentRangeWriter::FileSegmentRangeWriter( } bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset, bool is_persistent) +{ + size_t written_size = tryWrite(data, size, offset, is_persistent, true); + chassert(written_size == 0 || written_size == size); + return written_size == size; +} + +size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t offset, bool is_persistent, bool strict) { if (finalized) - return false; + return 0; auto & file_segments = file_segments_holder.file_segments; @@ -93,8 +100,8 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset file_segment->completePartAndResetDownloader(); }); - bool reserved = file_segment->reserve(size); - if (!reserved) + size_t reserved_size = file_segment->tryReserve(size, strict); + if (strict && reserved_size != size) { file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); appendFilesystemCacheLog(*file_segment); @@ -104,9 +111,15 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset "Unsuccessful space reservation attempt (size: {}, file segment info: {}", size, file_segment->getInfoForLog()); - return false; + return 0; } + if (reserved_size == 0) + return 0; + + /// shrink + size = reserved_size; + try { file_segment->write(data, size, offset); @@ -120,15 +133,26 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset file_segment->completePartAndResetDownloader(); current_file_segment_write_offset += size; - return true; + return size; } -void FileSegmentRangeWriter::finalize() +void FileSegmentRangeWriter::finalize(bool clear) { if (finalized) return; auto & file_segments = file_segments_holder.file_segments; + + /// Set all segments state to SKIP_CACHE to remove it from cache immediately on complete + /// Note: if segments are hold by someone else, it won't be removed + if (clear) + { + for (auto & file_segment : file_segments) + completeFileSegment(*file_segment, FileSegment::State::SKIP_CACHE); + finalized = true; + return; + } + if (file_segments.empty() || current_file_segment_it == file_segments.end()) return; @@ -196,12 +220,16 @@ void FileSegmentRangeWriter::appendFilesystemCacheLog(const FileSegment & file_s } } -void FileSegmentRangeWriter::completeFileSegment(FileSegment & file_segment) +void FileSegmentRangeWriter::completeFileSegment(FileSegment & file_segment, std::optional state) { /// File segment can be detached if space reservation failed. if (file_segment.isDetached()) return; + // if (state.has_value()) + // file_segment.setDownloadState(*state); + UNUSED(state); + file_segment.completeWithoutState(); appendFilesystemCacheLog(file_segment); } diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h index cec7305ab1b..4f5bebb92e8 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace Poco { @@ -33,8 +34,11 @@ public: * it until it is full and then allocate next file segment. */ bool write(const char * data, size_t size, size_t offset, bool is_persistent); + size_t tryWrite(const char * data, size_t size, size_t offset, bool is_persistent, bool strict = false); - void finalize(); + void finalize(bool clear = false); + + size_t currentOffset() const { return current_file_segment_write_offset; } ~FileSegmentRangeWriter(); @@ -43,7 +47,7 @@ private: void appendFilesystemCacheLog(const FileSegment & file_segment); - void completeFileSegment(FileSegment & file_segment); + void completeFileSegment(FileSegment & file_segment, std::optional state = {}); FileCache * cache; FileSegment::Key key; diff --git a/src/Disks/IO/FileCachePlaceholder.cpp b/src/Disks/IO/FileCachePlaceholder.cpp new file mode 100644 index 00000000000..a2640aaffdc --- /dev/null +++ b/src/Disks/IO/FileCachePlaceholder.cpp @@ -0,0 +1,7 @@ +#include + +namespace DB +{ + + +} diff --git a/src/Disks/IO/FileCachePlaceholder.h b/src/Disks/IO/FileCachePlaceholder.h new file mode 100644 index 00000000000..3c5c2bb1723 --- /dev/null +++ b/src/Disks/IO/FileCachePlaceholder.h @@ -0,0 +1,104 @@ +#pragma once + +#include +#include + +#include + +namespace fs = std::filesystem; + +namespace DB +{ + + +namespace ErrorCodes +{ + extern const int NOT_ENOUGH_SPACE; +} + + +class ISpacePlaceholder +{ + +public: + void reserveCapacity(size_t requested_capacity) + { + chassert(used_space <= capacity); + + size_t remaining_space = capacity - used_space; + if (requested_capacity <= remaining_space) + return; + + size_t capacity_to_reserve = requested_capacity - remaining_space; + reserveImpl(capacity_to_reserve); + capacity += capacity_to_reserve; + } + + void setUsed(size_t size) + { + if (used_space + size > capacity) + { + LOG_WARNING(&Poco::Logger::get("ISpacePlaceholder"), "Used space is greater than capacity. It may lead to not enough space error"); + reserveCapacity(used_space + size - capacity); + } + + used_space = used_space + size; + } + + virtual ~ISpacePlaceholder() = default; + +private: + virtual void reserveImpl(size_t size) = 0; + + size_t capacity = 0; + size_t used_space = 0; +}; + +class FileCachePlaceholder : public ISpacePlaceholder +{ +public: + FileCachePlaceholder(FileCache * cache, const String & name) + : key(cache->hash(name)) + , directory(cache->getPathInLocalCache(key)) + , cache_writer(cache, key, nullptr, "", name) + { + + } + + fs::path getDirectoryPath() const + { + return directory; + } + + void reserveImpl(size_t size) override + { + while (size > 0) + { + size_t current_offset = cache_writer.currentOffset(); + size_t written = cache_writer.tryWrite(nullptr, size, current_offset, /* is_persistent */ false); + if (written == 0) + throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Cannot reserve space in file cache ({} bytes required)", size); + size -= written; + } + } + + ~FileCachePlaceholder() override + { + try + { + cache_writer.finalize(/* clear */ true); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + +private: + FileSegment::Key key; + fs::path directory; + + FileSegmentRangeWriter cache_writer; +}; + +} diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 2d67203be0f..119dc25c66b 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -113,6 +113,8 @@ public: WriteSettings getAdjustedSettingsFromMetadataFile(const WriteSettings & settings, const std::string & path) const override; + FileCachePtr getCache() const { return cache; } + private: FileCache::Key getCacheKey(const std::string & path) const; diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 263a9a9d0e1..4230fb6254a 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -519,6 +519,14 @@ void DiskObjectStorage::wrapWithCache(FileCachePtr cache, const FileCacheSetting object_storage = std::make_shared(object_storage, cache, cache_settings, layer_name); } +FileCachePtr DiskObjectStorage::getCache() const +{ + const auto * cached_object_storage = typeid_cast(object_storage.get()); + if (!cached_object_storage) + return nullptr; + return cached_object_storage->getCache(); +} + NameSet DiskObjectStorage::getCacheLayersNames() const { NameSet cache_layers; diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index 00e3cf98142..a24acc270c0 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -186,6 +186,7 @@ public: /// There can be any number of cache layers: /// DiskObjectStorage(CachedObjectStorage(...CacheObjectStorage(S3ObjectStorage)...)) void wrapWithCache(FileCachePtr cache, const FileCacheSettings & cache_settings, const String & layer_name); + FileCachePtr getCache() const; /// Get structure of object storage this disk works with. Examples: /// DiskObjectStorage(S3ObjectStorage) diff --git a/src/Disks/TemporaryFileInPath.cpp b/src/Disks/TemporaryFileInPath.cpp new file mode 100644 index 00000000000..b7d5d7ed85e --- /dev/null +++ b/src/Disks/TemporaryFileInPath.cpp @@ -0,0 +1,10 @@ +#include + + + +namespace DB +{ + + + +} diff --git a/src/Disks/TemporaryFileInPath.h b/src/Disks/TemporaryFileInPath.h new file mode 100644 index 00000000000..540711e59c7 --- /dev/null +++ b/src/Disks/TemporaryFileInPath.h @@ -0,0 +1,27 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/// Wrapper around Poco::TemporaryFile to implement ITemporaryFile. +class TemporaryFileInPath : public ITemporaryFile +{ +public: + explicit TemporaryFileInPath(const String & folder_path) + : tmp_file(createTemporaryFile(folder_path)) + { + chassert(tmp_file); + } + + String getPath() const override { return tmp_file->path(); } + +private: + std::unique_ptr tmp_file; +}; + + + +} diff --git a/src/Disks/TemporaryFileOnDisk.cpp b/src/Disks/TemporaryFileOnDisk.cpp index 4f348519037..af1f3f87c71 100644 --- a/src/Disks/TemporaryFileOnDisk.cpp +++ b/src/Disks/TemporaryFileOnDisk.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include @@ -15,7 +16,6 @@ namespace CurrentMetrics extern const Metric TotalTemporaryFiles; } - namespace DB { diff --git a/src/Disks/TemporaryFileOnDisk.h b/src/Disks/TemporaryFileOnDisk.h index 9ba59c3eaf0..fa3a0383297 100644 --- a/src/Disks/TemporaryFileOnDisk.h +++ b/src/Disks/TemporaryFileOnDisk.h @@ -9,21 +9,30 @@ namespace DB { using DiskPtr = std::shared_ptr; +class ITemporaryFile +{ +public: + virtual String getPath() const = 0; + virtual ~ITemporaryFile() = default; +}; + +using TemporaryFileHolder = std::unique_ptr; + /// This class helps with the handling of temporary files or directories. /// A unique name for the temporary file or directory is automatically chosen based on a specified prefix. /// Create a directory in the constructor. /// The destructor always removes the temporary file or directory with all contained files. -class TemporaryFileOnDisk +class TemporaryFileOnDisk : public ITemporaryFile { public: explicit TemporaryFileOnDisk(const DiskPtr & disk_); explicit TemporaryFileOnDisk(const DiskPtr & disk_, CurrentMetrics::Value metric_scope); explicit TemporaryFileOnDisk(const DiskPtr & disk_, const String & prefix); - ~TemporaryFileOnDisk(); + ~TemporaryFileOnDisk() override; DiskPtr getDisk() const { return disk; } - String getPath() const; + String getPath() const override; private: DiskPtr disk; diff --git a/src/Formats/NativeWriter.cpp b/src/Formats/NativeWriter.cpp index c4dea371afd..e932bb88c2d 100644 --- a/src/Formats/NativeWriter.cpp +++ b/src/Formats/NativeWriter.cpp @@ -64,8 +64,10 @@ static void writeData(const ISerialization & serialization, const ColumnPtr & co } -void NativeWriter::write(const Block & block) +size_t NativeWriter::write(const Block & block) { + size_t written_before = ostr.count(); + /// Additional information about the block. if (client_revision > 0) block.info.write(ostr); @@ -161,6 +163,10 @@ void NativeWriter::write(const Block & block) if (index) index->blocks.emplace_back(std::move(index_block)); + + size_t written_after = ostr.count(); + size_t written_size = written_after - written_before; + return written_size; } } diff --git a/src/Formats/NativeWriter.h b/src/Formats/NativeWriter.h index 010a03ec722..eba7ac8443d 100644 --- a/src/Formats/NativeWriter.h +++ b/src/Formats/NativeWriter.h @@ -27,7 +27,7 @@ public: IndexForNativeFormat * index_ = nullptr, size_t initial_size_of_file_ = 0); Block getHeader() const { return header; } - void write(const Block & block); + size_t write(const Block & block); void flush(); static String getContentType() { return "application/octet-stream"; } diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 72fa1b3c324..56c3eef5015 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -12,6 +13,7 @@ #include #include + namespace fs = std::filesystem; namespace DB @@ -1009,6 +1011,12 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_lock fs::directory_iterator offset_it{key_it->path()}; for (; offset_it != fs::directory_iterator(); ++offset_it) { + if (offset_it->is_directory()) + { + LOG_DEBUG(log, "Unexpected directory: {}. Expected a file", offset_it->path().string()); + continue; + } + auto offset_with_suffix = offset_it->path().filename().string(); auto delim_pos = offset_with_suffix.find('_'); bool parsed; diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 706762b6915..1df634b48ed 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -221,6 +221,8 @@ private: FileSegmentCell * getCell(const Key & key, size_t offset, std::lock_guard & cache_lock); + /// Returns non-owened pointer to the cell stored in the `files` map. + /// Doesn't reserve any space. FileSegmentCell * addCell( const Key & key, size_t offset, diff --git a/src/Interpreters/Cache/FileCacheFactory.cpp b/src/Interpreters/Cache/FileCacheFactory.cpp index b276760c0dd..e120fe3fc27 100644 --- a/src/Interpreters/Cache/FileCacheFactory.cpp +++ b/src/Interpreters/Cache/FileCacheFactory.cpp @@ -31,14 +31,21 @@ const FileCacheSettings & FileCacheFactory::getSettings(const std::string & cach } -FileCachePtr FileCacheFactory::get(const std::string & cache_base_path) +FileCachePtr FileCacheFactory::tryGet(const std::string & cache_base_path) { std::lock_guard lock(mutex); auto it = caches_by_path.find(cache_base_path); if (it == caches_by_path.end()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "No cache found by path: {}", cache_base_path); + return nullptr; return it->second->cache; +} +FileCachePtr FileCacheFactory::get(const std::string & cache_base_path) +{ + auto file_cache_ptr = tryGet(cache_base_path); + if (!file_cache_ptr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No cache found by path: {}", cache_base_path); + return file_cache_ptr; } FileCachePtr FileCacheFactory::getOrCreate( diff --git a/src/Interpreters/Cache/FileCacheFactory.h b/src/Interpreters/Cache/FileCacheFactory.h index 82e0ec8f928..32ecd05f019 100644 --- a/src/Interpreters/Cache/FileCacheFactory.h +++ b/src/Interpreters/Cache/FileCacheFactory.h @@ -33,6 +33,7 @@ public: FileCachePtr getOrCreate(const std::string & cache_base_path, const FileCacheSettings & file_cache_settings, const std::string & name); + FileCachePtr tryGet(const std::string & cache_base_path); FileCachePtr get(const std::string & cache_base_path); CacheByBasePath getAll(); diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 418bcee05d9..73cbaaaca57 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -99,6 +99,7 @@ size_t FileSegment::getFirstNonDownloadedOffsetUnlocked(std::unique_lock 0) throw Exception( @@ -324,11 +325,13 @@ void FileSegment::write(const char * from, size_t size, size_t offset) try { - cache_writer->write(from, size); + if (cache_writer && from != nullptr) + cache_writer->write(from, size); std::unique_lock download_lock(download_mutex); - cache_writer->next(); + if (cache_writer && from != nullptr) + cache_writer->next(); downloaded_size += size; } @@ -379,6 +382,13 @@ FileSegment::State FileSegment::wait() } bool FileSegment::reserve(size_t size_to_reserve) +{ + size_t reserved = tryReserve(size_to_reserve, true); + assert(reserved == 0 || reserved == size_to_reserve); + return reserved == size_to_reserve; +} + +size_t FileSegment::tryReserve(size_t size_to_reserve, bool strict) { if (!size_to_reserve) throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Zero space reservation is not allowed"); @@ -394,10 +404,16 @@ bool FileSegment::reserve(size_t size_to_reserve) expected_downloaded_size = getDownloadedSizeUnlocked(segment_lock); if (expected_downloaded_size + size_to_reserve > range().size()) - throw Exception( - ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, - "Attempt to reserve space too much space ({}) for file segment with range: {} (downloaded size: {})", - size_to_reserve, range().toString(), downloaded_size); + { + if (strict) + { + throw Exception( + ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, + "Attempt to reserve space too much space ({}) for file segment with range: {} (downloaded size: {})", + size_to_reserve, range().toString(), downloaded_size); + } + size_to_reserve = range().size() - expected_downloaded_size; + } chassert(reserved_size >= expected_downloaded_size); } @@ -415,17 +431,16 @@ bool FileSegment::reserve(size_t size_to_reserve) { std::lock_guard cache_lock(cache->mutex); - size_to_reserve = size_to_reserve - already_reserved_size; - reserved = cache->tryReserve(key(), offset(), size_to_reserve, cache_lock); + size_t need_to_reserve = size_to_reserve - already_reserved_size; + reserved = cache->tryReserve(key(), offset(), need_to_reserve, cache_lock); - if (reserved) - { - std::lock_guard segment_lock(mutex); - reserved_size += size_to_reserve; - } + if (!reserved) + return 0; + + std::lock_guard segment_lock(mutex); + reserved_size += need_to_reserve; } - - return reserved; + return size_to_reserve; } void FileSegment::setDownloadedUnlocked([[maybe_unused]] std::unique_lock & segment_lock) @@ -783,6 +798,8 @@ FileSegmentsHolder::~FileSegmentsHolder() if (!cache) cache = file_segment->cache; + assert(cache == file_segment->cache); /// all segments should belong to the same cache + try { bool is_detached = false; diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 8f9c0097d77..02ae7cd5c8d 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -183,18 +183,13 @@ public: void assertCorrectness() const; - /** - * ========== Methods for _only_ file segment's `writer` ====================== - */ - - void synchronousWrite(const char * from, size_t size, size_t offset); - /** * ========== Methods for _only_ file segment's `downloader` ================== */ /// Try to reserve exactly `size` bytes. bool reserve(size_t size_to_reserve); + size_t tryReserve(size_t size_to_reserve, bool strict); /// Write data into reserved space. void write(const char * from, size_t size, size_t offset); @@ -247,9 +242,9 @@ private: void assertIsDownloaderUnlocked(const std::string & operation, std::unique_lock & segment_lock) const; void assertCorrectnessUnlocked(std::unique_lock & segment_lock) const; - /// complete() without any completion state is called from destructor of - /// FileSegmentsHolder. complete() might check if the caller of the method - /// is the last alive holder of the segment. Therefore, complete() and destruction + /// completeWithoutStateUnlocked() is called from destructor of FileSegmentsHolder. + /// Function might check if the caller of the method + /// is the last alive holder of the segment. Therefore, completion and destruction /// of the file segment pointer must be done under the same cache mutex. void completeWithoutStateUnlocked(std::lock_guard & cache_lock); void completeBasedOnCurrentState(std::lock_guard & cache_lock, std::unique_lock & segment_lock); @@ -295,7 +290,7 @@ private: /// In general case, all file segments are owned by cache. bool is_detached = false; - bool is_downloaded{false}; + bool is_downloaded = false; std::atomic hits_count = 0; /// cache hits. std::atomic ref_count = 0; /// Used for getting snapshot state diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 913b0535358..796b93998a9 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include #include #include @@ -102,6 +103,7 @@ #include #include #include +#include #include #include @@ -746,28 +748,65 @@ void Context::setPath(const String & path) shared->user_scripts_path = shared->path + "user_scripts/"; } -VolumePtr Context::setTemporaryStorage(const String & path, const String & policy_name, size_t max_size) +static void setupTmpPath(Poco::Logger * log, const std::string & path) +try +{ + LOG_DEBUG(log, "Setting up {} to store temporary data in it", path); + + fs::create_directories(path); + + /// Clearing old temporary files. + fs::directory_iterator dir_end; + for (fs::directory_iterator it(path); it != dir_end; ++it) + { + if (it->is_regular_file() && startsWith(it->path().filename(), "tmp")) + { + LOG_DEBUG(log, "Removing old temporary file {}", it->path().string()); + fs::remove(it->path()); + } + else + LOG_DEBUG(log, "Found unknown file in temporary path {}", it->path().string()); + } +} +catch (...) +{ + DB::tryLogCurrentException(log, fmt::format( + "Caught exception while setup temporary path: {}. " + "It is ok to skip this exception as cleaning old temporary files is not necessary", path)); +} + +static VolumePtr createLocalSingleDiskVolume(const std::string & path) +{ + auto disk = std::make_shared("_tmp_default", path, 0); + VolumePtr volume = std::make_shared("_tmp_default", disk, 0); + return volume; +} + +void Context::setTemporaryStoragePath(const String & path, size_t max_size) +{ + shared->tmp_path = path; + if (!shared->tmp_path.ends_with('/')) + shared->tmp_path += '/'; + + VolumePtr volume = createLocalSingleDiskVolume(shared->tmp_path); + + for (const auto & disk : volume->getDisks()) + { + setupTmpPath(shared->log, disk->getPath()); + } + + shared->temp_data_on_disk = std::make_shared(volume, nullptr, max_size); +} + +void Context::setTemporaryStoragePolicy(const String & policy_name, size_t max_size) { std::lock_guard lock(shared->storage_policies_mutex); - VolumePtr volume; - if (policy_name.empty()) - { - shared->tmp_path = path; - if (!shared->tmp_path.ends_with('/')) - shared->tmp_path += '/'; - - auto disk = std::make_shared("_tmp_default", shared->tmp_path, 0); - volume = std::make_shared("_tmp_default", disk, 0); - } - else - { - StoragePolicyPtr tmp_policy = getStoragePolicySelector(lock)->get(policy_name); - if (tmp_policy->getVolumes().size() != 1) - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, - "Policy '{}' is used temporary files, such policy should have exactly one volume", policy_name); - volume = tmp_policy->getVolume(0); - } + StoragePolicyPtr tmp_policy = getStoragePolicySelector(lock)->get(policy_name); + if (tmp_policy->getVolumes().size() != 1) + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, + "Policy '{}' is used temporary files, such policy should have exactly one volume", policy_name); + VolumePtr volume = tmp_policy->getVolume(0); if (volume->getDisks().empty()) throw Exception("No disks volume for temporary files", ErrorCodes::NO_ELEMENTS_IN_CONFIG); @@ -789,10 +828,33 @@ VolumePtr Context::setTemporaryStorage(const String & path, const String & polic "Disk '{}' ({}) is not local and can't be used for temporary files", disk_ptr->getName(), typeid(*disk_raw_ptr).name()); } + + setupTmpPath(shared->log, disk->getPath()); } - shared->temp_data_on_disk = std::make_shared(volume, max_size); - return volume; + shared->temp_data_on_disk = std::make_shared(volume, nullptr, max_size); +} + + +void Context::setTemporaryStorageInCache(const String & cache_disk_name, size_t max_size) +{ + auto disk_ptr = getDisk(cache_disk_name); + if (!disk_ptr) + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Disk '{}' is not found", cache_disk_name); + + const auto * disk_object_storage_ptr = dynamic_cast(disk_ptr.get()); + if (!disk_object_storage_ptr) + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Disk '{}' does not use cache", cache_disk_name); + + auto file_cache = disk_object_storage_ptr->getCache(); + if (!file_cache) + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Cache '{}' is not found", file_cache->getBasePath()); + + LOG_DEBUG(shared->log, "Using file cache ({}) for temporary files", file_cache->getBasePath()); + + shared->tmp_path = file_cache->getBasePath(); + VolumePtr volume = createLocalSingleDiskVolume(shared->tmp_path); + shared->temp_data_on_disk = std::make_shared(volume, file_cache.get(), max_size); } void Context::setFlagsPath(const String & path) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index bc89ce36edc..2c3122e4109 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -461,7 +461,9 @@ public: void addWarningMessage(const String & msg) const; - VolumePtr setTemporaryStorage(const String & path, const String & policy_name, size_t max_size); + void setTemporaryStorageInCache(const String & cache_disk_name, size_t max_size); + void setTemporaryStoragePolicy(const String & policy_name, size_t max_size); + void setTemporaryStoragePath(const String & path, size_t max_size); using ConfigurationPtr = Poco::AutoPtr; diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index c5ae6f6c885..9d07d51bad8 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include @@ -43,24 +44,14 @@ void TemporaryDataOnDiskScope::deltaAllocAndCheck(ssize_t compressed_delta, ssiz TemporaryFileStream & TemporaryDataOnDisk::createStream(const Block & header, size_t max_file_size) { - DiskPtr disk; - if (max_file_size > 0) - { - auto reservation = volume->reserve(max_file_size); - if (!reservation) - throw Exception("Not enough space on temporary disk", ErrorCodes::NOT_ENOUGH_SPACE); - disk = reservation->getDisk(); - } + TemporaryFileStreamPtr tmp_stream; + if (cache) + tmp_stream = TemporaryFileStream::create(cache, header, max_file_size, this); else - { - disk = volume->getDisk(); - } - - auto tmp_file = std::make_unique(disk, current_metric_scope); + tmp_stream = TemporaryFileStream::create(volume, header, max_file_size, this); std::lock_guard lock(mutex); - TemporaryFileStreamPtr & tmp_stream = streams.emplace_back(std::make_unique(std::move(tmp_file), header, this)); - return *tmp_stream; + return *streams.emplace_back(std::move(tmp_stream)); } @@ -89,12 +80,13 @@ struct TemporaryFileStream::OutputWriter { } - void write(const Block & block) + size_t write(const Block & block) { if (finalized) throw Exception("Cannot write to finalized stream", ErrorCodes::LOGICAL_ERROR); - out_writer.write(block); + size_t written_bytes = out_writer.write(block); num_rows += block.rows(); + return written_bytes; } void finalize() @@ -155,21 +147,65 @@ struct TemporaryFileStream::InputReader NativeReader in_reader; }; -TemporaryFileStream::TemporaryFileStream(TemporaryFileOnDiskHolder file_, const Block & header_, TemporaryDataOnDisk * parent_) +TemporaryFileStreamPtr TemporaryFileStream::create(const VolumePtr & volume, const Block & header, size_t max_file_size, TemporaryDataOnDisk * parent_) +{ + DiskPtr disk; + if (max_file_size > 0) + { + auto reservation = volume->reserve(max_file_size); + if (!reservation) + throw Exception("Not enough space on temporary disk", ErrorCodes::NOT_ENOUGH_SPACE); + disk = reservation->getDisk(); + } + else + { + disk = volume->getDisk(); + } + + auto tmp_file = std::make_unique(disk, parent_->getMetricScope()); + return std::make_unique(std::move(tmp_file), header, /* cache_placeholder */ nullptr, /* parent */ parent_); +} + +TemporaryFileStreamPtr TemporaryFileStream::create(FileCache * cache, const Block & header, size_t max_file_size, TemporaryDataOnDisk * parent_) +{ + auto tmp_file = std::make_unique(fs::path(cache->getBasePath()) / "tmp"); + + auto cache_placeholder = std::make_unique(cache, tmp_file->getPath()); + cache_placeholder->reserveCapacity(max_file_size); + + return std::make_unique(std::move(tmp_file), header, std::move(cache_placeholder), parent_); +} + +TemporaryFileStream::TemporaryFileStream( + TemporaryFileHolder file_, + const Block & header_, + std::unique_ptr space_holder_, + TemporaryDataOnDisk * parent_) : parent(parent_) , header(header_) , file(std::move(file_)) + , space_holder(std::move(space_holder_)) , out_writer(std::make_unique(file->getPath(), header)) { } -void TemporaryFileStream::write(const Block & block) +size_t TemporaryFileStream::write(const Block & block) { if (!out_writer) throw Exception("Writing has been finished", ErrorCodes::LOGICAL_ERROR); + size_t block_size_in_memory = block.allocatedBytes(); + + if (space_holder) + space_holder->reserveCapacity(block_size_in_memory); + updateAllocAndCheck(); - out_writer->write(block); + + size_t bytes_written = out_writer->write(block); + if (space_holder) + space_holder->setUsed(bytes_written); + + return bytes_written; } TemporaryFileStream::Stat TemporaryFileStream::finishWriting() diff --git a/src/Interpreters/TemporaryDataOnDisk.h b/src/Interpreters/TemporaryDataOnDisk.h index 11edc8700d2..33d1ba183b9 100644 --- a/src/Interpreters/TemporaryDataOnDisk.h +++ b/src/Interpreters/TemporaryDataOnDisk.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace CurrentMetrics @@ -40,12 +41,12 @@ public: std::atomic uncompressed_size; }; - explicit TemporaryDataOnDiskScope(VolumePtr volume_, size_t limit_) - : volume(std::move(volume_)), limit(limit_) + explicit TemporaryDataOnDiskScope(VolumePtr volume_, FileCache * cache_, size_t limit_) + : volume(std::move(volume_)), cache(cache_), limit(limit_) {} explicit TemporaryDataOnDiskScope(TemporaryDataOnDiskScopePtr parent_, size_t limit_) - : parent(std::move(parent_)), volume(parent->volume), limit(limit_) + : parent(std::move(parent_)), volume(parent->volume), cache(parent->cache), limit(limit_) {} /// TODO: remove @@ -56,7 +57,9 @@ protected: void deltaAllocAndCheck(ssize_t compressed_delta, ssize_t uncompressed_delta); TemporaryDataOnDiskScopePtr parent = nullptr; + VolumePtr volume; + FileCache * cache = nullptr; StatAtomic stat; size_t limit = 0; @@ -91,6 +94,7 @@ public: bool empty() const; const StatAtomic & getStat() const { return stat; } + CurrentMetrics::Value getMetricScope() const { return current_metric_scope; } private: mutable std::mutex mutex; @@ -116,9 +120,14 @@ public: size_t num_rows = 0; }; - TemporaryFileStream(TemporaryFileOnDiskHolder file_, const Block & header_, TemporaryDataOnDisk * parent_); + static TemporaryFileStreamPtr create(const VolumePtr & volume, const Block & header, size_t max_file_size, TemporaryDataOnDisk * parent_); + static TemporaryFileStreamPtr create(FileCache * cache, const Block & header, size_t max_file_size, TemporaryDataOnDisk * parent_); + + TemporaryFileStream(TemporaryFileHolder file_, const Block & header_, std::unique_ptr space_holder, TemporaryDataOnDisk * parent_); + + /// Returns number of written bytes + size_t write(const Block & block); - void write(const Block & block); Stat finishWriting(); bool isWriteFinished() const; @@ -142,7 +151,8 @@ private: Block header; - TemporaryFileOnDiskHolder file; + TemporaryFileHolder file; + std::unique_ptr space_holder; Stat stat; diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index 22150b9f656..71557a99897 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -15,6 +15,8 @@ #include #include +#include + namespace fs = std::filesystem; fs::path caches_dir = fs::current_path() / "lru_cache_test"; @@ -517,3 +519,45 @@ TEST(FileCache, get) } } + + +TEST(FileCache, rangeWriter) +{ + if (fs::exists(cache_base_path)) + fs::remove_all(cache_base_path); + fs::create_directories(cache_base_path); + + DB::FileCacheSettings settings; + settings.max_size = 25; + settings.max_elements = 5; + settings.max_file_segment_size = 10; + + auto cache = DB::FileCache(cache_base_path, settings); + cache.initialize(); + auto key = cache.hash("key1"); + + DB::FileSegmentRangeWriter writer(&cache, key, nullptr, "", "key1"); + + std::string data(100, '\xf0'); + + /// Write first segment + ASSERT_EQ(writer.tryWrite(data.data(), 5, 0, false, false), 5); + ASSERT_EQ(writer.tryWrite(data.data(), 3, 5, false, false), 3); + ASSERT_EQ(writer.tryWrite(data.data(), 1, 8, false, false), 1); + ASSERT_EQ(writer.tryWrite(data.data(), 1, 9, false, false), 1); + + /// Second segment starts + ASSERT_EQ(writer.tryWrite(data.data(), 1, 10, false, false), 1); + ASSERT_EQ(writer.tryWrite(data.data(), 1, 11, false, false), 1); + /// Can't write 10 bytes into the rest of current segment + ASSERT_EQ(writer.tryWrite(data.data(), 10, 12, false, false), 8); + + /// Rest can be written into the next segment + ASSERT_EQ(writer.tryWrite(data.data(), 2, 20, false, false), 2); + /// Only 3 bytes left, can't write 4 and nothing should be written + ASSERT_EQ(writer.tryWrite(data.data(), 4, 22, false, false), 0); + ASSERT_EQ(writer.tryWrite(data.data(), 4, 22, false, false), 0); + ASSERT_EQ(writer.tryWrite(data.data(), 3, 22, false, false), 3); + + writer.finalize(); +} From 7a814d285432c2c96b5c4932f5ff3669fffccc89 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 23 Nov 2022 14:48:51 +0000 Subject: [PATCH 67/98] wip: temporary files evict fs cache --- src/Common/filesystemHelpers.cpp | 4 +- src/Common/filesystemHelpers.h | 4 +- .../IO/CachedOnDiskReadBufferFromFile.cpp | 2 +- .../IO/CachedOnDiskWriteBufferFromFile.cpp | 17 +- src/Disks/IO/FileCachePlaceholder.cpp | 79 ++++++++ src/Disks/IO/FileCachePlaceholder.h | 75 ++----- src/Disks/TemporaryFileInPath.cpp | 14 +- src/Disks/TemporaryFileInPath.h | 16 +- src/IO/WriteBufferFromTemporaryFile.cpp | 6 +- src/IO/WriteBufferFromTemporaryFile.h | 4 +- src/Interpreters/Cache/FileCache.cpp | 25 ++- src/Interpreters/Cache/FileCache.h | 2 + src/Interpreters/Cache/FileSegment.h | 2 + src/Interpreters/TemporaryDataOnDisk.cpp | 10 +- src/Interpreters/TemporaryDataOnDisk.h | 7 +- .../tests/gtest_lru_file_cache.cpp | 189 +++++++++++++++--- src/Storages/MergeTree/MergeTask.h | 4 +- tests/integration/test_tmp_policy/test.py | 2 +- 18 files changed, 323 insertions(+), 139 deletions(-) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index b259d1929ea..43f88dd7faa 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -64,11 +64,11 @@ bool enoughSpaceInDirectory(const std::string & path, size_t data_size) return data_size <= free_space; } -std::unique_ptr createTemporaryFile(const std::string & folder_path) +std::unique_ptr createTemporaryFile(const std::string & folder_path) { ProfileEvents::increment(ProfileEvents::ExternalProcessingFilesTotal); fs::create_directories(folder_path); - return std::make_unique(folder_path); + return std::make_unique(folder_path); } #if !defined(OS_LINUX) diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index fd080771b52..14ee5f54322 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -14,10 +14,10 @@ namespace fs = std::filesystem; namespace DB { -using TemporaryFile = Poco::TemporaryFile; +using PocoTemporaryFile = Poco::TemporaryFile; bool enoughSpaceInDirectory(const std::string & path, size_t data_size); -std::unique_ptr createTemporaryFile(const std::string & folder_path); +std::unique_ptr createTemporaryFile(const std::string & folder_path); // Determine what block device is responsible for specified path diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 5b5d746ab55..6b51481e833 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -951,7 +951,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() } else { - LOG_TRACE(log, "No space left in cache, will continue without cache download"); + LOG_TRACE(log, "No space left in cache to reserve {} bytes, will continue without cache download", size); file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); } diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index 609ef2cdaa7..1eb7ef0090e 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -101,7 +101,7 @@ size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t o }); size_t reserved_size = file_segment->tryReserve(size, strict); - if (strict && reserved_size != size) + if (reserved_size == 0 || (strict && reserved_size != size)) { file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); appendFilesystemCacheLog(*file_segment); @@ -114,8 +114,6 @@ size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t o return 0; } - if (reserved_size == 0) - return 0; /// shrink size = reserved_size; @@ -147,10 +145,12 @@ void FileSegmentRangeWriter::finalize(bool clear) /// Note: if segments are hold by someone else, it won't be removed if (clear) { - for (auto & file_segment : file_segments) - completeFileSegment(*file_segment, FileSegment::State::SKIP_CACHE); + for (auto file_segment_it = file_segments.begin(); file_segment_it != file_segments.end(); ++file_segment_it) + { + completeFileSegment(**file_segment_it, FileSegment::State::SKIP_CACHE); + file_segments.erase(file_segment_it); + } finalized = true; - return; } if (file_segments.empty() || current_file_segment_it == file_segments.end()) @@ -226,9 +226,8 @@ void FileSegmentRangeWriter::completeFileSegment(FileSegment & file_segment, std if (file_segment.isDetached()) return; - // if (state.has_value()) - // file_segment.setDownloadState(*state); - UNUSED(state); + if (state.has_value()) + file_segment.setDownloadState(*state); file_segment.completeWithoutState(); appendFilesystemCacheLog(file_segment); diff --git a/src/Disks/IO/FileCachePlaceholder.cpp b/src/Disks/IO/FileCachePlaceholder.cpp index a2640aaffdc..33e8d13b648 100644 --- a/src/Disks/IO/FileCachePlaceholder.cpp +++ b/src/Disks/IO/FileCachePlaceholder.cpp @@ -3,5 +3,84 @@ namespace DB { +void ISpacePlaceholder::reserveCapacity(size_t requested_capacity) +{ + chassert(used_space <= capacity); + + size_t remaining_space = capacity - used_space; + /// TODO LOG_TEST + LOG_DEBUG(&Poco::Logger::get("ISpacePlaceholder"), "reserving {} bytes (used_space: {}, capacity: {})", requested_capacity, used_space, capacity); + + if (requested_capacity <= remaining_space) + return; + + size_t capacity_to_reserve = requested_capacity - remaining_space; + reserveImpl(capacity_to_reserve); + capacity += capacity_to_reserve; +} + +void ISpacePlaceholder::setUsed(size_t size) +{ + /// TODO LOG_TEST + LOG_DEBUG(&Poco::Logger::get("ISpacePlaceholder"), "using {} bytes ({} already used, {} capacity)", size, used_space, capacity); + + if (used_space + size > capacity) + { + LOG_WARNING(&Poco::Logger::get("ISpacePlaceholder"), "Used space is greater than capacity. It may lead to not enough space error"); + reserveCapacity(size); + } + + used_space = used_space + size; + +} + +FileCachePlaceholder::FileCachePlaceholder(FileCache * cache, const String & name) + : key_name(name) + , file_cache(cache) +{ +} + + +void FileCachePlaceholder::reserveImpl(size_t requested_size) +{ + String key = fmt::format("{}_{}", key_name, cache_writers.size()); + auto cache_writer = std::make_unique(file_cache, + file_cache->hash(key), + /* cache_log_ */ nullptr, + /* query_id_ */ "", + /* source_path_ */ key); + + while (requested_size > 0) + { + size_t current_offset = cache_writer->currentOffset(); + size_t written = cache_writer->tryWrite(nullptr, requested_size, current_offset, /* is_persistent */ false, /* strict */ false); + if (written == 0) + { + cache_writer->finalize(/* clear */ true); + throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, + "Cannot reserve space in file cache ({} bytes required, {} / {} bytes used, {} / {} elements used)", + requested_size, file_cache->getUsedCacheSize(), file_cache->getTotalMaxSize(), + file_cache->getFileSegmentsNum(), file_cache->getTotalMaxElements()); + } + requested_size -= written; + } + + cache_writers.push_back(std::move(cache_writer)); +} + +FileCachePlaceholder::~FileCachePlaceholder() +{ + try + { + for (auto & cache_writer : cache_writers) + { + cache_writer->finalize(/* clear */ true); + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} } diff --git a/src/Disks/IO/FileCachePlaceholder.h b/src/Disks/IO/FileCachePlaceholder.h index 3c5c2bb1723..26c0ddc2375 100644 --- a/src/Disks/IO/FileCachePlaceholder.h +++ b/src/Disks/IO/FileCachePlaceholder.h @@ -3,6 +3,9 @@ #include #include +#include +#include + #include namespace fs = std::filesystem; @@ -10,40 +13,16 @@ namespace fs = std::filesystem; namespace DB { - namespace ErrorCodes { extern const int NOT_ENOUGH_SPACE; } - class ISpacePlaceholder { - public: - void reserveCapacity(size_t requested_capacity) - { - chassert(used_space <= capacity); - - size_t remaining_space = capacity - used_space; - if (requested_capacity <= remaining_space) - return; - - size_t capacity_to_reserve = requested_capacity - remaining_space; - reserveImpl(capacity_to_reserve); - capacity += capacity_to_reserve; - } - - void setUsed(size_t size) - { - if (used_space + size > capacity) - { - LOG_WARNING(&Poco::Logger::get("ISpacePlaceholder"), "Used space is greater than capacity. It may lead to not enough space error"); - reserveCapacity(used_space + size - capacity); - } - - used_space = used_space + size; - } + void reserveCapacity(size_t requested_capacity); + void setUsed(size_t size); virtual ~ISpacePlaceholder() = default; @@ -54,51 +33,21 @@ private: size_t used_space = 0; }; + class FileCachePlaceholder : public ISpacePlaceholder { public: - FileCachePlaceholder(FileCache * cache, const String & name) - : key(cache->hash(name)) - , directory(cache->getPathInLocalCache(key)) - , cache_writer(cache, key, nullptr, "", name) - { + FileCachePlaceholder(FileCache * cache, const String & name); - } + void reserveImpl(size_t requested_size) override; - fs::path getDirectoryPath() const - { - return directory; - } - - void reserveImpl(size_t size) override - { - while (size > 0) - { - size_t current_offset = cache_writer.currentOffset(); - size_t written = cache_writer.tryWrite(nullptr, size, current_offset, /* is_persistent */ false); - if (written == 0) - throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Cannot reserve space in file cache ({} bytes required)", size); - size -= written; - } - } - - ~FileCachePlaceholder() override - { - try - { - cache_writer.finalize(/* clear */ true); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } + ~FileCachePlaceholder() override; private: - FileSegment::Key key; - fs::path directory; + std::string key_name; + FileCache * file_cache; - FileSegmentRangeWriter cache_writer; + std::vector> cache_writers; }; } diff --git a/src/Disks/TemporaryFileInPath.cpp b/src/Disks/TemporaryFileInPath.cpp index b7d5d7ed85e..eae7fa66855 100644 --- a/src/Disks/TemporaryFileInPath.cpp +++ b/src/Disks/TemporaryFileInPath.cpp @@ -1,10 +1,20 @@ #include - - +#include namespace DB { +TemporaryFileInPath::TemporaryFileInPath(const String & folder_path) + : tmp_file(createTemporaryFile(folder_path)) +{ + chassert(tmp_file); +} +String TemporaryFileInPath::getPath() const +{ + return tmp_file->path(); +} + +TemporaryFileInPath::~TemporaryFileInPath() = default; } diff --git a/src/Disks/TemporaryFileInPath.h b/src/Disks/TemporaryFileInPath.h index 540711e59c7..503247e3f89 100644 --- a/src/Disks/TemporaryFileInPath.h +++ b/src/Disks/TemporaryFileInPath.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { @@ -10,18 +10,12 @@ namespace DB class TemporaryFileInPath : public ITemporaryFile { public: - explicit TemporaryFileInPath(const String & folder_path) - : tmp_file(createTemporaryFile(folder_path)) - { - chassert(tmp_file); - } - - String getPath() const override { return tmp_file->path(); } + explicit TemporaryFileInPath(const String & folder_path); + String getPath() const override; + ~TemporaryFileInPath() override; private: - std::unique_ptr tmp_file; + std::unique_ptr tmp_file; }; - - } diff --git a/src/IO/WriteBufferFromTemporaryFile.cpp b/src/IO/WriteBufferFromTemporaryFile.cpp index f93c79ca587..4562ad512b3 100644 --- a/src/IO/WriteBufferFromTemporaryFile.cpp +++ b/src/IO/WriteBufferFromTemporaryFile.cpp @@ -13,7 +13,7 @@ namespace ErrorCodes } -WriteBufferFromTemporaryFile::WriteBufferFromTemporaryFile(std::unique_ptr && tmp_file_) +WriteBufferFromTemporaryFile::WriteBufferFromTemporaryFile(std::unique_ptr && tmp_file_) : WriteBufferFromFile(tmp_file_->path(), DBMS_DEFAULT_BUFFER_SIZE, O_RDWR | O_TRUNC | O_CREAT, 0600), tmp_file(std::move(tmp_file_)) {} @@ -40,11 +40,11 @@ public: return std::make_shared(fd, file_name, std::move(origin->tmp_file)); } - ReadBufferFromTemporaryWriteBuffer(int fd_, const std::string & file_name_, std::unique_ptr && tmp_file_) + ReadBufferFromTemporaryWriteBuffer(int fd_, const std::string & file_name_, std::unique_ptr && tmp_file_) : ReadBufferFromFile(fd_, file_name_), tmp_file(std::move(tmp_file_)) {} - std::unique_ptr tmp_file; + std::unique_ptr tmp_file; }; diff --git a/src/IO/WriteBufferFromTemporaryFile.h b/src/IO/WriteBufferFromTemporaryFile.h index 06e2911db26..a4e83b95ac6 100644 --- a/src/IO/WriteBufferFromTemporaryFile.h +++ b/src/IO/WriteBufferFromTemporaryFile.h @@ -20,11 +20,11 @@ public: ~WriteBufferFromTemporaryFile() override; private: - explicit WriteBufferFromTemporaryFile(std::unique_ptr && tmp_file); + explicit WriteBufferFromTemporaryFile(std::unique_ptr && tmp_file); std::shared_ptr getReadBufferImpl() override; - std::unique_ptr tmp_file; + std::unique_ptr tmp_file; friend class ReadBufferFromTemporaryWriteBuffer; }; diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 56c3eef5015..59a07df8439 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -542,9 +542,6 @@ FileSegmentPtr FileCache::createFileSegmentForDownload( assertCacheCorrectness(key, cache_lock); #endif - if (size > max_file_segment_size) - throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Requested size exceeds max file segment size"); - auto * cell = getCell(key, offset, cache_lock); if (cell) throw Exception( @@ -1001,9 +998,17 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_lock fs::directory_iterator key_it{key_prefix_it->path()}; for (; key_it != fs::directory_iterator(); ++key_it) { - if (!key_it->is_directory()) + if (key_it->is_regular_file()) { - LOG_DEBUG(log, "Unexpected file: {}. Expected a directory", key_it->path().string()); + if (key_prefix_it->path().filename() == "tmp" && startsWith(key_it->path().filename(), "tmp")) + { + LOG_DEBUG(log, "Found temporary file '{}', will remove it", key_it->path().string()); + fs::remove(key_it->path()); + } + else + { + LOG_DEBUG(log, "Unexpected file: {}. Expected a directory", key_it->path().string()); + } continue; } @@ -1181,6 +1186,16 @@ size_t FileCache::getAvailableCacheSizeUnlocked(std::lock_guard & ca return max_size - getUsedCacheSizeUnlocked(cache_lock); } +size_t FileCache::getTotalMaxSize() const +{ + return max_size; +} + +size_t FileCache::getTotalMaxElements() const +{ + return max_element_size; +} + size_t FileCache::getFileSegmentsNum() const { std::lock_guard cache_lock(mutex); diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 1df634b48ed..4c9c68569bd 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -89,8 +89,10 @@ public: size_t capacity() const { return max_size; } size_t getUsedCacheSize() const; + size_t getTotalMaxSize() const; size_t getFileSegmentsNum() const; + size_t getTotalMaxElements() const; static bool isReadOnly(); diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 02ae7cd5c8d..4c780eecdc3 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -308,6 +308,8 @@ struct FileSegmentsHolder : private boost::noncopyable FileSegmentsHolder(FileSegmentsHolder && other) noexcept : file_segments(std::move(other.file_segments)) {} + void reset() { file_segments.clear(); } + ~FileSegmentsHolder(); String toString(); diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index 9d07d51bad8..07fc3d22546 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -36,7 +36,8 @@ void TemporaryDataOnDiskScope::deltaAllocAndCheck(ssize_t compressed_delta, ssiz size_t new_consumprion = stat.compressed_size + compressed_delta; if (compressed_delta > 0 && limit && new_consumprion > limit) - throw Exception(ErrorCodes::TOO_MANY_ROWS_OR_BYTES, "Limit for temporary files size exceeded"); + throw Exception(ErrorCodes::TOO_MANY_ROWS_OR_BYTES, + "Limit for temporary files size exceeded (would consume {} / {} bytes)", new_consumprion, limit); stat.compressed_size += compressed_delta; stat.uncompressed_size += uncompressed_delta; @@ -149,6 +150,9 @@ struct TemporaryFileStream::InputReader TemporaryFileStreamPtr TemporaryFileStream::create(const VolumePtr & volume, const Block & header, size_t max_file_size, TemporaryDataOnDisk * parent_) { + if (!volume) + throw Exception("TemporaryDataOnDiskScope has no volume", ErrorCodes::LOGICAL_ERROR); + DiskPtr disk; if (max_file_size > 0) { @@ -173,7 +177,7 @@ TemporaryFileStreamPtr TemporaryFileStream::create(FileCache * cache, const Bloc auto cache_placeholder = std::make_unique(cache, tmp_file->getPath()); cache_placeholder->reserveCapacity(max_file_size); - return std::make_unique(std::move(tmp_file), header, std::move(cache_placeholder), parent_); + return std::make_unique(std::move(tmp_file), header, std::move(cache_placeholder), parent_); } TemporaryFileStream::TemporaryFileStream( @@ -194,7 +198,7 @@ size_t TemporaryFileStream::write(const Block & block) if (!out_writer) throw Exception("Writing has been finished", ErrorCodes::LOGICAL_ERROR); - size_t block_size_in_memory = block.allocatedBytes(); + size_t block_size_in_memory = block.bytes(); if (space_holder) space_holder->reserveCapacity(block_size_in_memory); diff --git a/src/Interpreters/TemporaryDataOnDisk.h b/src/Interpreters/TemporaryDataOnDisk.h index 33d1ba183b9..24e7e0506bc 100644 --- a/src/Interpreters/TemporaryDataOnDisk.h +++ b/src/Interpreters/TemporaryDataOnDisk.h @@ -51,7 +51,12 @@ public: /// TODO: remove /// Refactor all code that uses volume directly to use TemporaryDataOnDisk. - VolumePtr getVolume() const { return volume; } + VolumePtr getVolume() const + { + if (!volume) + throw Exception("TemporaryDataOnDiskScope has no volume", ErrorCodes::LOGICAL_ERROR); + return volume; + } protected: void deltaAllocAndCheck(ssize_t compressed_delta, ssize_t uncompressed_delta); diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index 71557a99897..a4b446bdd90 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -14,13 +15,14 @@ #include #include #include +#include #include namespace fs = std::filesystem; +using namespace DB; -fs::path caches_dir = fs::current_path() / "lru_cache_test"; -String cache_base_path = caches_dir / "cache1" / ""; +static constexpr auto TEST_LOG_LEVEL = "debug"; void assertRange( [[maybe_unused]] size_t assert_n, DB::FileSegmentPtr file_segment, @@ -55,7 +57,7 @@ String getFileSegmentPath(const String & base_path, const DB::FileCache::Key & k return fs::path(base_path) / key_str.substr(0, 3) / key_str / DB::toString(offset); } -void download(DB::FileSegmentPtr file_segment) +void download(const std::string & cache_base_path, DB::FileSegmentPtr file_segment) { const auto & key = file_segment->key(); size_t size = file_segment->range().size(); @@ -69,30 +71,54 @@ void download(DB::FileSegmentPtr file_segment) file_segment->write(data.data(), size, file_segment->getCurrentWriteOffset()); } -void prepareAndDownload(DB::FileSegmentPtr file_segment) +void prepareAndDownload(const std::string & cache_base_path, DB::FileSegmentPtr file_segment) { - // std::cerr << "Reserving: " << file_segment->range().size() << " for: " << file_segment->range().toString() << "\n"; ASSERT_TRUE(file_segment->reserve(file_segment->range().size())); - download(file_segment); + download(cache_base_path, file_segment); } -void complete(const DB::FileSegmentsHolder & holder) +void complete(const std::string & cache_base_path, const DB::FileSegmentsHolder & holder) { for (const auto & file_segment : holder.file_segments) { ASSERT_TRUE(file_segment->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(file_segment); + prepareAndDownload(cache_base_path, file_segment); file_segment->completeWithState(DB::FileSegment::State::DOWNLOADED); } } - -TEST(FileCache, get) +class FileCacheTest : public ::testing::Test { - if (fs::exists(cache_base_path)) - fs::remove_all(cache_base_path); - fs::create_directories(cache_base_path); +public: + static void setupLogs(const std::string & level) + { + Poco::AutoPtr channel(new Poco::ConsoleChannel(std::cerr)); + Poco::Logger::root().setChannel(channel); + Poco::Logger::root().setLevel(level); + } + + void SetUp() override + { + setupLogs(TEST_LOG_LEVEL); + + if (fs::exists(cache_base_path)) + fs::remove_all(cache_base_path); + fs::create_directories(cache_base_path); + } + + void TearDown() override + { + if (fs::exists(cache_base_path)) + fs::remove_all(cache_base_path); + } + + fs::path caches_dir = fs::current_path() / "lru_cache_test"; + std::string cache_base_path = caches_dir / "cache1" / ""; +}; + +TEST_F(FileCacheTest, get) +{ DB::ThreadStatus thread_status; /// To work with cache need query_id and query context. @@ -128,7 +154,7 @@ TEST(FileCache, get) ASSERT_TRUE(segments[0]->reserve(segments[0]->range().size())); assertRange(2, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADING); - download(segments[0]); + download(cache_base_path, segments[0]); segments[0]->completeWithState(DB::FileSegment::State::DOWNLOADED); assertRange(3, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED); } @@ -149,7 +175,7 @@ TEST(FileCache, get) assertRange(5, segments[1], DB::FileSegment::Range(10, 14), DB::FileSegment::State::EMPTY); ASSERT_TRUE(segments[1]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(segments[1]); + prepareAndDownload(cache_base_path, segments[1]); segments[1]->completeWithState(DB::FileSegment::State::DOWNLOADED); assertRange(6, segments[1], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED); } @@ -182,8 +208,8 @@ TEST(FileCache, get) assertRange(10, segments[0], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED); } - complete(cache.getOrSet(key, 17, 4, {})); /// Get [17, 20] - complete(cache.getOrSet(key, 24, 3, {})); /// Get [24, 26] + complete(cache_base_path, cache.getOrSet(key, 17, 4, {})); /// Get [17, 20] + complete(cache_base_path, cache.getOrSet(key, 24, 3, {})); /// Get [24, 26] /// completeWithState(cache.getOrSet(key, 27, 1, false)); /// Get [27, 27] /// Current cache: [__________][_____] [____] [___][] @@ -205,7 +231,7 @@ TEST(FileCache, get) assertRange(13, segments[2], DB::FileSegment::Range(15, 16), DB::FileSegment::State::EMPTY); ASSERT_TRUE(segments[2]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(segments[2]); + prepareAndDownload(cache_base_path, segments[2]); segments[2]->completeWithState(DB::FileSegment::State::DOWNLOADED); @@ -246,7 +272,7 @@ TEST(FileCache, get) assertRange(21, segments[3], DB::FileSegment::Range(21, 21), DB::FileSegment::State::EMPTY); ASSERT_TRUE(segments[3]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(segments[3]); + prepareAndDownload(cache_base_path, segments[3]); segments[3]->completeWithState(DB::FileSegment::State::DOWNLOADED); ASSERT_TRUE(segments[3]->state() == DB::FileSegment::State::DOWNLOADED); @@ -269,8 +295,8 @@ TEST(FileCache, get) ASSERT_TRUE(segments[0]->getOrSetDownloader() == DB::FileSegment::getCallerId()); ASSERT_TRUE(segments[2]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(segments[0]); - prepareAndDownload(segments[2]); + prepareAndDownload(cache_base_path, segments[0]); + prepareAndDownload(cache_base_path, segments[2]); segments[0]->completeWithState(DB::FileSegment::State::DOWNLOADED); segments[2]->completeWithState(DB::FileSegment::State::DOWNLOADED); } @@ -292,8 +318,8 @@ TEST(FileCache, get) ASSERT_TRUE(s5[0]->getOrSetDownloader() == DB::FileSegment::getCallerId()); ASSERT_TRUE(s1[0]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(s5[0]); - prepareAndDownload(s1[0]); + prepareAndDownload(cache_base_path, s5[0]); + prepareAndDownload(cache_base_path, s1[0]); s5[0]->completeWithState(DB::FileSegment::State::DOWNLOADED); s1[0]->completeWithState(DB::FileSegment::State::DOWNLOADED); @@ -396,7 +422,7 @@ TEST(FileCache, get) cv.wait(lock, [&]{ return lets_start_download; }); } - prepareAndDownload(segments[2]); + prepareAndDownload(cache_base_path, segments[2]); segments[2]->completeWithState(DB::FileSegment::State::DOWNLOADED); ASSERT_TRUE(segments[2]->state() == DB::FileSegment::State::DOWNLOADED); @@ -461,7 +487,7 @@ TEST(FileCache, get) ASSERT_TRUE(segments_2[1]->state() == DB::FileSegment::State::PARTIALLY_DOWNLOADED); ASSERT_TRUE(segments_2[1]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(segments_2[1]); + prepareAndDownload(cache_base_path, segments_2[1]); segments_2[1]->completeWithState(DB::FileSegment::State::DOWNLOADED); }); @@ -520,19 +546,14 @@ TEST(FileCache, get) } - -TEST(FileCache, rangeWriter) +TEST_F(FileCacheTest, rangeWriter) { - if (fs::exists(cache_base_path)) - fs::remove_all(cache_base_path); - fs::create_directories(cache_base_path); - DB::FileCacheSettings settings; settings.max_size = 25; settings.max_elements = 5; settings.max_file_segment_size = 10; - auto cache = DB::FileCache(cache_base_path, settings); + DB::FileCache cache(cache_base_path, settings); cache.initialize(); auto key = cache.hash("key1"); @@ -561,3 +582,107 @@ TEST(FileCache, rangeWriter) writer.finalize(); } + +static Block generateBlock(size_t size = 0) +{ + Block block; + ColumnWithTypeAndName column; + column.name = "x"; + column.type = std::make_shared(); + + { + MutableColumnPtr mut_col = column.type->createColumn(); + for (size_t i = 0; i < size; ++i) + mut_col->insert(i); + column.column = std::move(mut_col); + } + + block.insert(column); + + LOG_DEBUG(&Poco::Logger::get("FileCacheTest"), "generated block {} bytes", block.bytes()); + return block; +} + +static size_t readAllTemporaryData(TemporaryFileStream & stream) +{ + Block block; + size_t read_rows = 0; + do + { + block = stream.read(); + read_rows += block.rows(); + } while (block); + return read_rows; +} + +TEST_F(FileCacheTest, temporaryData) +{ + DB::FileCacheSettings settings; + settings.max_size = 10240; + settings.max_file_segment_size = 1024; + + DB::FileCache file_cache(cache_base_path, settings); + file_cache.initialize(); + + auto tmp_data_scope = std::make_shared(nullptr, &file_cache, 0); + + auto some_data_holder = file_cache.getOrSet(file_cache.hash("some_data"), 0, 1024 * 5, CreateFileSegmentSettings{}); + + { + auto segments = fromHolder(some_data_holder); + ASSERT_EQ(segments.size(), 5); + for (auto & segment : segments) + { + ASSERT_TRUE(segment->getOrSetDownloader() == DB::FileSegment::getCallerId()); + ASSERT_TRUE(segment->reserve(segment->range().size())); + download(cache_base_path, segment); + segment->completeWithState(DB::FileSegment::State::DOWNLOADED); + } + } + + size_t size_used_before_temporary_data = file_cache.getUsedCacheSize(); + size_t segments_used_before_temporary_data = file_cache.getFileSegmentsNum(); + ASSERT_GT(size_used_before_temporary_data, 0); + ASSERT_GT(segments_used_before_temporary_data, 0); + + size_t size_used_with_temporary_data; + size_t segments_used_with_temporary_data; + { + auto tmp_data = std::make_unique(tmp_data_scope); + + auto & stream = tmp_data->createStream(generateBlock()); + + ASSERT_GT(stream.write(generateBlock(100)), 0); + + EXPECT_GT(file_cache.getUsedCacheSize(), 0); + EXPECT_GT(file_cache.getFileSegmentsNum(), 0); + + size_t used_size_before_attempt = file_cache.getUsedCacheSize(); + /// data can't be evicted because it is still held by `some_data_holder` + ASSERT_THROW(stream.write(generateBlock(1000)), DB::Exception); + + ASSERT_EQ(file_cache.getUsedCacheSize(), used_size_before_attempt); + + some_data_holder.reset(); + + stream.write(generateBlock(1011)); + + auto stat = stream.finishWriting(); + + EXPECT_EQ(stat.num_rows, 1111); + EXPECT_EQ(readAllTemporaryData(stream), 1111); + + size_used_with_temporary_data = file_cache.getUsedCacheSize(); + segments_used_with_temporary_data = file_cache.getFileSegmentsNum(); + EXPECT_GT(size_used_with_temporary_data, 0); + EXPECT_GT(segments_used_with_temporary_data, 0); + } + + /// All temp data should be evicted after removing temporary files + EXPECT_LE(file_cache.getUsedCacheSize(), size_used_with_temporary_data); + EXPECT_LE(file_cache.getFileSegmentsNum(), segments_used_with_temporary_data); + + /// Some segments reserved by `some_data_holder` was eviced by temporary data + EXPECT_LE(file_cache.getUsedCacheSize(), size_used_before_temporary_data); + EXPECT_LE(file_cache.getFileSegmentsNum(), segments_used_before_temporary_data); +} diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 6a29cdbb5ca..2b8d9055cff 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -192,7 +192,7 @@ private: bool force_ttl{false}; CompressionCodecPtr compression_codec{nullptr}; size_t sum_input_rows_upper_bound{0}; - std::unique_ptr rows_sources_file{nullptr}; + std::unique_ptr rows_sources_file{nullptr}; std::unique_ptr rows_sources_uncompressed_write_buf{nullptr}; std::unique_ptr rows_sources_write_buf{nullptr}; std::optional column_sizes{}; @@ -257,7 +257,7 @@ private: /// Begin dependencies from previous stage std::unique_ptr rows_sources_write_buf{nullptr}; std::unique_ptr rows_sources_uncompressed_write_buf{nullptr}; - std::unique_ptr rows_sources_file; + std::unique_ptr rows_sources_file; std::optional column_sizes; CompressionCodecPtr compression_codec; DiskPtr tmp_disk{nullptr}; diff --git a/tests/integration/test_tmp_policy/test.py b/tests/integration/test_tmp_policy/test.py index c919d9a0c3d..870a70b127a 100644 --- a/tests/integration/test_tmp_policy/test.py +++ b/tests/integration/test_tmp_policy/test.py @@ -23,7 +23,7 @@ def start_cluster(): cluster.shutdown() -def test_different_versions(start_cluster): +def test_disk_selection(start_cluster): query = "SELECT count(ignore(*)) FROM (SELECT * FROM system.numbers LIMIT 1e7) GROUP BY number" settings = { "max_bytes_before_external_group_by": 1 << 20, From 08b8d335f06dd35eddee434a6e583bea86351d6e Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 23 Nov 2022 17:52:00 +0000 Subject: [PATCH 68/98] add integration/test_temporary_data_in_cache --- .../test_temporary_data_in_cache/__init__.py | 0 .../config.d/storage_configuration.xml | 39 +++++++++ .../test_temporary_data_in_cache/test.py | 81 +++++++++++++++++++ 3 files changed, 120 insertions(+) create mode 100644 tests/integration/test_temporary_data_in_cache/__init__.py create mode 100644 tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml create mode 100644 tests/integration/test_temporary_data_in_cache/test.py diff --git a/tests/integration/test_temporary_data_in_cache/__init__.py b/tests/integration/test_temporary_data_in_cache/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml b/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml new file mode 100644 index 00000000000..8ccd705c6f1 --- /dev/null +++ b/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml @@ -0,0 +1,39 @@ + + + + + local + /local_disk/ + + + + cache + local_disk + /tiny_local_cache/ + 10M + 1M + 1 + 0 + + + + + + local + /tiny_local_cache/ + + + + + + +
+ tiny_local_cache +
+
+
+
+
+ + tiny_local_cache +
diff --git a/tests/integration/test_temporary_data_in_cache/test.py b/tests/integration/test_temporary_data_in_cache/test.py new file mode 100644 index 00000000000..ba57348ee37 --- /dev/null +++ b/tests/integration/test_temporary_data_in_cache/test.py @@ -0,0 +1,81 @@ +# pylint: disable=unused-argument +# pylint: disable=redefined-outer-name + +import pytest + +from helpers.cluster import ClickHouseCluster +from helpers.client import QueryRuntimeException + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + main_configs=["configs/config.d/storage_configuration.xml"], + tmpfs=["/local_disk:size=50M", "/tiny_local_cache:size=12M"], +) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_cache_evicted_by_temporary_data(start_cluster): + q = node.query + qi = lambda query: int(node.query(query).strip()) + + cache_size_initial = qi("SELECT sum(size) FROM system.filesystem_cache") + assert cache_size_initial == 0 + + free_space_initial = qi( + "SELECT free_space FROM system.disks WHERE name = 'tiny_local_cache_local_disk'" + ) + assert free_space_initial > 8 * 1024 * 1024 + + q( + "CREATE TABLE t1 (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS storage_policy = 'tiny_local_cache'" + ) + q("INSERT INTO t1 SELECT number FROM numbers(1024 * 1024)") + + # To be sure that nothing is reading the cache and entries for t1 can be evited + q("OPTIMIZE TABLE t1 FINAL") + q("SYSTEM STOP MERGES t1") + + # Read some data to fill the cache + q("SELECT sum(x) FROM t1") + + cache_size_with_t1 = qi("SELECT sum(size) FROM system.filesystem_cache") + assert cache_size_with_t1 > 8 * 1024 * 1024 + + # Almost all disk space is occupied by t1 cache + free_space_with_t1 = qi( + "SELECT free_space FROM system.disks WHERE name = 'tiny_local_cache_local_disk'" + ) + assert free_space_with_t1 < 4 * 1024 * 1024 + + # Try to sort the table, but fail because of lack of disk space + with pytest.raises(QueryRuntimeException) as exc: + q( + "SELECT ignore(*) FROM numbers(10 * 1024 * 1024) ORDER BY sipHash64(number)", + settings={ + "max_bytes_before_external_group_by": "4M", + "max_bytes_before_external_sort": "4M", + }, + ) + assert "Cannot reserve space in file cache" in str(exc.value) + + # Some data evicted from cache by temporary data + cache_size_after_eviction = qi("SELECT sum(size) FROM system.filesystem_cache") + assert cache_size_after_eviction < cache_size_with_t1 + + # Disk space freed, at least 3 MB, because temporary data tried to write 4 MB + free_space_after_eviction = qi( + "SELECT free_space FROM system.disks WHERE name = 'tiny_local_cache_local_disk'" + ) + assert free_space_after_eviction > free_space_with_t1 + 3 * 1024 * 1024 + + q("DROP TABLE IF EXISTS t1") From 816af3dc1657dd9f12d727b6b69495375293d9ce Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 23 Nov 2022 19:14:36 +0000 Subject: [PATCH 69/98] wip: temporary files evict fs cache --- .../IO/CachedOnDiskWriteBufferFromFile.cpp | 6 +++++- .../IO/CachedOnDiskWriteBufferFromFile.h | 12 +++++++++++ src/Disks/IO/FileCachePlaceholder.cpp | 21 ++++++++++++------- src/Disks/IO/FileCachePlaceholder.h | 19 ++++++++++++----- src/Formats/NativeWriter.h | 2 ++ src/Interpreters/Cache/FileSegment.cpp | 1 + src/Interpreters/TemporaryDataOnDisk.cpp | 8 ++++++- src/Interpreters/TemporaryDataOnDisk.h | 7 +------ 8 files changed, 56 insertions(+), 20 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index 1eb7ef0090e..3283c9424d2 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -114,7 +114,6 @@ size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t o return 0; } - /// shrink size = reserved_size; @@ -134,6 +133,11 @@ size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t o return size; } +size_t FileSegmentRangeWriter::tryReserve(size_t size, size_t offset, bool is_persistent, bool strict) +{ + return tryWrite(nullptr, size, offset, is_persistent, strict); +} + void FileSegmentRangeWriter::finalize(bool clear) { if (finalized) diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h index 4f5bebb92e8..b8b7d519b78 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h @@ -34,8 +34,20 @@ public: * it until it is full and then allocate next file segment. */ bool write(const char * data, size_t size, size_t offset, bool is_persistent); + + /* Tries to write data to current file segment. + * Size of written data may be less than requested_size, because current file segment may not have enough space. + * In strict mode, if current file segment doesn't have enough space, then exception is thrown. + * + * Returns size of written data. + * If returned non zero value, then we can try to write again. + * If no space is available, returns zero. + */ size_t tryWrite(const char * data, size_t size, size_t offset, bool is_persistent, bool strict = false); + /// Same as tryWrite, but doesn't write anything, just reserves some space in cache + size_t tryReserve(size_t size, size_t offset, bool is_persistent, bool strict = false); + void finalize(bool clear = false); size_t currentOffset() const { return current_file_segment_write_offset; } diff --git a/src/Disks/IO/FileCachePlaceholder.cpp b/src/Disks/IO/FileCachePlaceholder.cpp index 33e8d13b648..4877d9532c2 100644 --- a/src/Disks/IO/FileCachePlaceholder.cpp +++ b/src/Disks/IO/FileCachePlaceholder.cpp @@ -3,13 +3,17 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_ENOUGH_SPACE; +} + void ISpacePlaceholder::reserveCapacity(size_t requested_capacity) { chassert(used_space <= capacity); size_t remaining_space = capacity - used_space; - /// TODO LOG_TEST - LOG_DEBUG(&Poco::Logger::get("ISpacePlaceholder"), "reserving {} bytes (used_space: {}, capacity: {})", requested_capacity, used_space, capacity); + LOG_TEST(&Poco::Logger::get("ISpacePlaceholder"), "Reserving {} bytes (used_space: {}, capacity: {})", requested_capacity, used_space, capacity); if (requested_capacity <= remaining_space) return; @@ -21,8 +25,7 @@ void ISpacePlaceholder::reserveCapacity(size_t requested_capacity) void ISpacePlaceholder::setUsed(size_t size) { - /// TODO LOG_TEST - LOG_DEBUG(&Poco::Logger::get("ISpacePlaceholder"), "using {} bytes ({} already used, {} capacity)", size, used_space, capacity); + LOG_TEST(&Poco::Logger::get("ISpacePlaceholder"), "Using {} bytes ({} already used, {} capacity)", size, used_space, capacity); if (used_space + size > capacity) { @@ -31,7 +34,6 @@ void ISpacePlaceholder::setUsed(size_t size) } used_space = used_space + size; - } FileCachePlaceholder::FileCachePlaceholder(FileCache * cache, const String & name) @@ -40,9 +42,9 @@ FileCachePlaceholder::FileCachePlaceholder(FileCache * cache, const String & nam { } - void FileCachePlaceholder::reserveImpl(size_t requested_size) { + /// We create new cache_writer and will try to reserve requested_size in it String key = fmt::format("{}_{}", key_name, cache_writers.size()); auto cache_writer = std::make_unique(file_cache, file_cache->hash(key), @@ -50,10 +52,15 @@ void FileCachePlaceholder::reserveImpl(size_t requested_size) /* query_id_ */ "", /* source_path_ */ key); + /* Sometimes several calls of tryReserve are required, + * because it allocates space in current segment and its size may be less than requested_size. + * If we can't write any data, we need to free all space occupied by current reservation. + * To do this we will just destroy current cache_writer. + */ while (requested_size > 0) { size_t current_offset = cache_writer->currentOffset(); - size_t written = cache_writer->tryWrite(nullptr, requested_size, current_offset, /* is_persistent */ false, /* strict */ false); + size_t written = cache_writer->tryReserve(requested_size, current_offset, /* is_persistent */ false, /* strict */ false); if (written == 0) { cache_writer->finalize(/* clear */ true); diff --git a/src/Disks/IO/FileCachePlaceholder.h b/src/Disks/IO/FileCachePlaceholder.h index 26c0ddc2375..589aec289fe 100644 --- a/src/Disks/IO/FileCachePlaceholder.h +++ b/src/Disks/IO/FileCachePlaceholder.h @@ -13,15 +13,21 @@ namespace fs = std::filesystem; namespace DB { -namespace ErrorCodes -{ - extern const int NOT_ENOUGH_SPACE; -} +/* ISpacePlaceholder is a base class for all classes that need to reserve space in some storage. + * You should resrve space with call reserveCapacity() before writing to it. + * After writing you should call setUsed() to let ISpacePlaceholder know how much space was used. + * It can be different because in some cases you don't know exact size of data you will write (because of compression, for example). + * It's better to reserve more space in advance not to overuse space. + */ class ISpacePlaceholder { public: + /// Reserve space in storage void reserveCapacity(size_t requested_capacity); + + /// Indicate that some space is used + /// It uses reserved space if it is possible, otherwise it reserves more space void setUsed(size_t size); virtual ~ISpacePlaceholder() = default; @@ -33,7 +39,9 @@ private: size_t used_space = 0; }; - +/* FileCachePlaceholder is a class that reserves space in FileCache. + * Data is written externally, and FileCachePlaceholder is only used to hold space in FileCache. + */ class FileCachePlaceholder : public ISpacePlaceholder { public: @@ -47,6 +55,7 @@ private: std::string key_name; FileCache * file_cache; + /// On each reserveImpl() call we create new FileSegmentRangeWriter that would be hold space std::vector> cache_writers; }; diff --git a/src/Formats/NativeWriter.h b/src/Formats/NativeWriter.h index eba7ac8443d..7bb377d2e4a 100644 --- a/src/Formats/NativeWriter.h +++ b/src/Formats/NativeWriter.h @@ -27,6 +27,8 @@ public: IndexForNativeFormat * index_ = nullptr, size_t initial_size_of_file_ = 0); Block getHeader() const { return header; } + + /// Returns the number of bytes written. size_t write(const Block & block); void flush(); diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 73cbaaaca57..36600f1fca4 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -325,6 +325,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset) try { + /// if `from` is nullptr, then we just allocate and hold space by current segment and it was (or would) be written outside if (cache_writer && from != nullptr) cache_writer->write(from, size); diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index 07fc3d22546..a039053b011 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -43,6 +43,13 @@ void TemporaryDataOnDiskScope::deltaAllocAndCheck(ssize_t compressed_delta, ssiz stat.uncompressed_size += uncompressed_delta; } +VolumePtr TemporaryDataOnDiskScope::getVolume() const +{ + if (!volume) + throw Exception("TemporaryDataOnDiskScope has no volume", ErrorCodes::LOGICAL_ERROR); + return volume; +} + TemporaryFileStream & TemporaryDataOnDisk::createStream(const Block & header, size_t max_file_size) { TemporaryFileStreamPtr tmp_stream; @@ -55,7 +62,6 @@ TemporaryFileStream & TemporaryDataOnDisk::createStream(const Block & header, si return *streams.emplace_back(std::move(tmp_stream)); } - std::vector TemporaryDataOnDisk::getStreams() const { std::vector res; diff --git a/src/Interpreters/TemporaryDataOnDisk.h b/src/Interpreters/TemporaryDataOnDisk.h index 24e7e0506bc..2790529754f 100644 --- a/src/Interpreters/TemporaryDataOnDisk.h +++ b/src/Interpreters/TemporaryDataOnDisk.h @@ -51,12 +51,7 @@ public: /// TODO: remove /// Refactor all code that uses volume directly to use TemporaryDataOnDisk. - VolumePtr getVolume() const - { - if (!volume) - throw Exception("TemporaryDataOnDiskScope has no volume", ErrorCodes::LOGICAL_ERROR); - return volume; - } + VolumePtr getVolume() const; protected: void deltaAllocAndCheck(ssize_t compressed_delta, ssize_t uncompressed_delta); From 35c4e8ac842cb229a3124a87965cf057dee21413 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 25 Nov 2022 20:46:56 +0000 Subject: [PATCH 70/98] wip: temporary files evict fs cache --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 5 +- .../IO/CachedOnDiskWriteBufferFromFile.cpp | 52 +++++++++++++------ .../IO/CachedOnDiskWriteBufferFromFile.h | 32 +++++++----- src/Disks/IO/FileCachePlaceholder.cpp | 25 +++------ src/Interpreters/Cache/FileCache.cpp | 29 ++++++++--- src/Interpreters/Cache/FileCache.h | 2 +- src/Interpreters/Cache/FileSegment.cpp | 29 +++++++++-- src/Interpreters/Cache/FileSegment.h | 43 +++++++++++++-- .../tests/gtest_lru_file_cache.cpp | 34 ++++++------ .../System/StorageSystemFilesystemCache.cpp | 7 +-- 10 files changed, 171 insertions(+), 87 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 6b51481e833..0dd40e7f153 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -118,10 +118,7 @@ void CachedOnDiskReadBufferFromFile::initialize(size_t offset, size_t size) } else { - CreateFileSegmentSettings create_settings{ - .is_persistent = is_persistent - }; - + CreateFileSegmentSettings create_settings(is_persistent ? FileSegmentKind::Persistent : FileSegmentKind::Regular); file_segments_holder.emplace(cache->getOrSet(cache_key, offset, size, create_settings)); } diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index 3283c9424d2..c9960e75c16 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -51,14 +51,31 @@ FileSegmentRangeWriter::FileSegmentRangeWriter( { } -bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset, bool is_persistent) +bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind) { - size_t written_size = tryWrite(data, size, offset, is_persistent, true); - chassert(written_size == 0 || written_size == size); + size_t written_size = tryWrite(data, size, offset, segment_kind, true); return written_size == size; } -size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t offset, bool is_persistent, bool strict) +size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind, bool strict) +{ + size_t total_written_size = 0; + while (size > 0) + { + size_t written_size = tryWriteImpl(data, size, offset, segment_kind, strict); + chassert(written_size <= size); + if (written_size == 0) + break; + + data += written_size; + size -= written_size; + offset += written_size; + total_written_size += written_size; + } + return total_written_size; +} + +size_t FileSegmentRangeWriter::tryWriteImpl(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind, bool strict) { if (finalized) return 0; @@ -67,7 +84,7 @@ size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t o if (current_file_segment_it == file_segments.end()) { - current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, is_persistent); + current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, segment_kind); } else { @@ -85,7 +102,7 @@ size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t o if (file_segment->range().size() == file_segment->getDownloadedSize()) { completeFileSegment(*file_segment); - current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, is_persistent); + current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, segment_kind); } } @@ -103,8 +120,11 @@ size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t o size_t reserved_size = file_segment->tryReserve(size, strict); if (reserved_size == 0 || (strict && reserved_size != size)) { - file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); - appendFilesystemCacheLog(*file_segment); + if (strict) + { + file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); + appendFilesystemCacheLog(*file_segment); + } LOG_DEBUG( &Poco::Logger::get("FileSegmentRangeWriter"), @@ -114,7 +134,7 @@ size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t o return 0; } - /// shrink + /// Shrink to reserved size, because we can't write more than reserved size = reserved_size; try @@ -133,9 +153,9 @@ size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t o return size; } -size_t FileSegmentRangeWriter::tryReserve(size_t size, size_t offset, bool is_persistent, bool strict) +bool FileSegmentRangeWriter::reserve(size_t size, size_t offset) { - return tryWrite(nullptr, size, offset, is_persistent, strict); + return write(nullptr, size, offset, FileSegmentKind::Temporary); } void FileSegmentRangeWriter::finalize(bool clear) @@ -177,7 +197,7 @@ FileSegmentRangeWriter::~FileSegmentRangeWriter() } } -FileSegments::iterator FileSegmentRangeWriter::allocateFileSegment(size_t offset, bool is_persistent) +FileSegments::iterator FileSegmentRangeWriter::allocateFileSegment(size_t offset, FileSegmentKind segment_kind) { /** * Allocate a new file segment starting `offset`. @@ -186,10 +206,7 @@ FileSegments::iterator FileSegmentRangeWriter::allocateFileSegment(size_t offset std::lock_guard cache_lock(cache->mutex); - CreateFileSegmentSettings create_settings - { - .is_persistent = is_persistent, - }; + CreateFileSegmentSettings create_settings(segment_kind); /// We set max_file_segment_size to be downloaded, /// if we have less size to write, file segment will be resized in complete() method. @@ -301,7 +318,8 @@ void CachedOnDiskWriteBufferFromFile::cacheData(char * data, size_t size) try { - if (!cache_writer->write(data, size, current_download_offset, is_persistent_cache_file)) + auto segment_kind = is_persistent_cache_file ? FileSegmentKind::Persistent : FileSegmentKind::Regular; + if (!cache_writer->write(data, size, current_download_offset, segment_kind)) { LOG_INFO(log, "Write-through cache is stopped as cache limit is reached and nothing can be evicted"); return; diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h index b8b7d519b78..bb63e727ec8 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h @@ -29,24 +29,23 @@ public: FileCache * cache_, const FileSegment::Key & key_, std::shared_ptr cache_log_, const String & query_id_, const String & source_path_); - /** - * Write a range of file segments. Allocate file segment of `max_file_segment_size` and write to - * it until it is full and then allocate next file segment. - */ - bool write(const char * data, size_t size, size_t offset, bool is_persistent); + /* Write a range of file segments. + * Allocate file segment of `max_file_segment_size` and write to it until it is full and then allocate next file segment. + * If it's impossible to allocate new file segment and reserve space to write all data, then returns false. + * + * Note: the data that was written to file segments before the error occurred is not rolled back. + */ + bool write(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind); /* Tries to write data to current file segment. - * Size of written data may be less than requested_size, because current file segment may not have enough space. - * In strict mode, if current file segment doesn't have enough space, then exception is thrown. + * Size of written data may be less than requested_size, because it may not be enough space. * * Returns size of written data. - * If returned non zero value, then we can try to write again. - * If no space is available, returns zero. */ - size_t tryWrite(const char * data, size_t size, size_t offset, bool is_persistent, bool strict = false); + size_t tryWrite(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind = FileSegmentKind::Regular, bool strict = false); - /// Same as tryWrite, but doesn't write anything, just reserves some space in cache - size_t tryReserve(size_t size, size_t offset, bool is_persistent, bool strict = false); + /// Same as `write`, but doesn't write anything, just reserves some space in cache + bool reserve(size_t size, size_t offset); void finalize(bool clear = false); @@ -55,12 +54,19 @@ public: ~FileSegmentRangeWriter(); private: - FileSegments::iterator allocateFileSegment(size_t offset, bool is_persistent); + FileSegments::iterator allocateFileSegment(size_t offset, FileSegmentKind segment_kind); void appendFilesystemCacheLog(const FileSegment & file_segment); void completeFileSegment(FileSegment & file_segment, std::optional state = {}); + /* Writes data to current file segment as much as possible and returns size of written data, do not allocate new file segments + * In `strict` mode it will write all data or nothing, otherwise it will write as much as possible + * If returned non zero value, then we can try to write again to next file segment. + * If no space is available, returns zero. + */ + size_t tryWriteImpl(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind, bool strict); + FileCache * cache; FileSegment::Key key; diff --git a/src/Disks/IO/FileCachePlaceholder.cpp b/src/Disks/IO/FileCachePlaceholder.cpp index 4877d9532c2..63dc4ffb1c2 100644 --- a/src/Disks/IO/FileCachePlaceholder.cpp +++ b/src/Disks/IO/FileCachePlaceholder.cpp @@ -52,24 +52,15 @@ void FileCachePlaceholder::reserveImpl(size_t requested_size) /* query_id_ */ "", /* source_path_ */ key); - /* Sometimes several calls of tryReserve are required, - * because it allocates space in current segment and its size may be less than requested_size. - * If we can't write any data, we need to free all space occupied by current reservation. - * To do this we will just destroy current cache_writer. - */ - while (requested_size > 0) + size_t current_offset = cache_writer->currentOffset(); + bool is_reserved = cache_writer->reserve(requested_size, current_offset); + if (!is_reserved) { - size_t current_offset = cache_writer->currentOffset(); - size_t written = cache_writer->tryReserve(requested_size, current_offset, /* is_persistent */ false, /* strict */ false); - if (written == 0) - { - cache_writer->finalize(/* clear */ true); - throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, - "Cannot reserve space in file cache ({} bytes required, {} / {} bytes used, {} / {} elements used)", - requested_size, file_cache->getUsedCacheSize(), file_cache->getTotalMaxSize(), - file_cache->getFileSegmentsNum(), file_cache->getTotalMaxElements()); - } - requested_size -= written; + cache_writer->finalize(/* clear */ true); + throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, + "Cannot reserve space in file cache ({} bytes required, {} / {} bytes used, {} / {} elements used)", + requested_size, file_cache->getUsedCacheSize(), file_cache->getTotalMaxSize(), + file_cache->getFileSegmentsNum(), file_cache->getTotalMaxElements()); } cache_writers.push_back(std::move(cache_writer)); diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 59a07df8439..e2a0e839f19 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -48,13 +48,27 @@ FileCache::Key FileCache::hash(const String & path) return Key(sipHash128(path.data(), path.size())); } -String FileCache::getPathInLocalCache(const Key & key, size_t offset, bool is_persistent) const +String FileCache::getPathInLocalCache(const Key & key, size_t offset, FileSegmentKind segment_kind) const { + String file_suffix; + switch (segment_kind) + { + case FileSegmentKind::Persistent: + file_suffix = "_persistent"; + break; + case FileSegmentKind::Temporary: + file_suffix = "_temporary"; + break; + case FileSegmentKind::Regular: + file_suffix = ""; + break; + } + auto key_str = key.toString(); return fs::path(cache_base_path) / key_str.substr(0, 3) / key_str - / (std::to_string(offset) + (is_persistent ? "_persistent" : "")); + / (std::to_string(offset) + file_suffix); } String FileCache::getPathInLocalCache(const Key & key) const @@ -1025,14 +1039,17 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_lock auto offset_with_suffix = offset_it->path().filename().string(); auto delim_pos = offset_with_suffix.find('_'); bool parsed; - bool is_persistent = false; + FileSegmentKind segment_kind = FileSegmentKind::Regular; if (delim_pos == std::string::npos) parsed = tryParse(offset, offset_with_suffix); else { parsed = tryParse(offset, offset_with_suffix.substr(0, delim_pos)); - is_persistent = offset_with_suffix.substr(delim_pos+1) == "persistent"; + if (offset_with_suffix.substr(delim_pos+1) == "persistent") + segment_kind = FileSegmentKind::Persistent; + if (offset_with_suffix.substr(delim_pos+1) == "temporary") + segment_kind = FileSegmentKind::Temporary; } if (!parsed) @@ -1052,7 +1069,7 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_lock { auto * cell = addCell( key, offset, size, FileSegment::State::DOWNLOADED, - CreateFileSegmentSettings{ .is_persistent = is_persistent }, cache_lock); + CreateFileSegmentSettings(segment_kind), cache_lock); if (cell) queue_entries.emplace_back(cell->queue_iterator, cell->file_segment); @@ -1164,7 +1181,7 @@ std::vector FileCache::tryGetCachePaths(const Key & key) for (const auto & [offset, cell] : cells_by_offset) { if (cell.file_segment->state() == FileSegment::State::DOWNLOADED) - cache_paths.push_back(getPathInLocalCache(key, offset, cell.file_segment->isPersistent())); + cache_paths.push_back(getPathInLocalCache(key, offset, cell.file_segment->getKind())); } return cache_paths; diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 4c9c68569bd..4a2610fd76b 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -80,7 +80,7 @@ public: static Key hash(const String & path); - String getPathInLocalCache(const Key & key, size_t offset, bool is_persistent) const; + String getPathInLocalCache(const Key & key, size_t offset, FileSegmentKind segment_kind) const; String getPathInLocalCache(const Key & key) const; diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 36600f1fca4..19ea6784c02 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -23,6 +23,19 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +String toString(FileSegmentKind type) +{ + switch (type) + { + case FileSegmentKind::Regular: + return "Regular"; + case FileSegmentKind::Persistent: + return "Persistent"; + case FileSegmentKind::Temporary: + return "Temporary"; + } +} + FileSegment::FileSegment( size_t offset_, size_t size_, @@ -39,7 +52,7 @@ FileSegment::FileSegment( #else , log(&Poco::Logger::get("FileSegment")) #endif - , is_persistent(settings.is_persistent) + , segment_kind(settings.type) { /// On creation, file segment state can be EMPTY, DOWNLOADED, DOWNLOADING. switch (download_state) @@ -73,7 +86,7 @@ FileSegment::FileSegment( String FileSegment::getPathInLocalCache() const { - return cache->getPathInLocalCache(key(), offset(), isPersistent()); + return cache->getPathInLocalCache(key(), offset(), segment_kind); } FileSegment::State FileSegment::state() const @@ -99,7 +112,6 @@ size_t FileSegment::getFirstNonDownloadedOffsetUnlocked(std::unique_lock & cach resetDownloaderUnlocked(segment_lock); } + if (segment_kind == FileSegmentKind::Temporary && is_last_holder) + { + cache->remove(key(), offset(), cache_lock, segment_lock); + LOG_TEST(log, "Removing temporary file segment: {}", getInfoForLogUnlocked(segment_lock)); + return; + } + switch (download_state) { case State::SKIP_CACHE: @@ -642,7 +661,7 @@ String FileSegment::getInfoForLogUnlocked(std::unique_lock & segment info << "first non-downloaded offset: " << getFirstNonDownloadedOffsetUnlocked(segment_lock) << ", "; info << "caller id: " << getCallerId() << ", "; info << "detached: " << is_detached << ", "; - info << "persistent: " << is_persistent; + info << "kind: " << toString(segment_kind); return info.str(); } @@ -737,7 +756,7 @@ FileSegmentPtr FileSegment::getSnapshot(const FileSegmentPtr & file_segment, std snapshot->ref_count = file_segment.use_count(); snapshot->downloaded_size = file_segment->getDownloadedSizeUnlocked(segment_lock); snapshot->download_state = file_segment->download_state; - snapshot->is_persistent = file_segment->isPersistent(); + snapshot->segment_kind = file_segment->getKind(); return snapshot; } diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 4c780eecdc3..8915b2f0a36 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -30,9 +30,38 @@ using FileSegmentPtr = std::shared_ptr; using FileSegments = std::list; +/* + * FileSegmentKind is used to specify the eviction policy for file segments. + */ +enum class FileSegmentKind +{ + /* `Regular` file segment is still in cache after usage, and can be evicted + * (unless there're some holders). + */ + Regular, + + /* `Persistent` file segment can't be evicted from cache, + * it should be removed manually. + */ + Persistent, + + /* `Temporary` file segment is removed right after relesing. + * Also corresponding files are removed during cache loading (if any). + */ + Temporary, +}; + +String toString(FileSegmentKind type); + struct CreateFileSegmentSettings { - bool is_persistent = false; + FileSegmentKind type = FileSegmentKind::Regular; + + CreateFileSegmentSettings() = default; + + explicit CreateFileSegmentSettings(FileSegmentKind type_) + : type(type_) + {} }; class FileSegment : private boost::noncopyable, public std::enable_shared_from_this @@ -127,7 +156,8 @@ public: size_t offset() const { return range().left; } - bool isPersistent() const { return is_persistent; } + FileSegmentKind getKind() const { return segment_kind; } + bool isPersistent() const { return segment_kind == FileSegmentKind::Persistent; } using UniqueId = std::pair; UniqueId getUniqueId() const { return std::pair(key(), offset()); } @@ -188,8 +218,13 @@ public: */ /// Try to reserve exactly `size` bytes. + /// Returns true if reservation was successful, false otherwise. bool reserve(size_t size_to_reserve); - size_t tryReserve(size_t size_to_reserve, bool strict); + + /// Try to reserve at max `size` bytes. + /// Returns actual size reserved. + /// In strict mode throws an error on attempt to reserve space too much space + size_t tryReserve(size_t size_to_reserve, bool strict = false); /// Write data into reserved space. void write(const char * from, size_t size, size_t offset); @@ -295,7 +330,7 @@ private: std::atomic hits_count = 0; /// cache hits. std::atomic ref_count = 0; /// Used for getting snapshot state - bool is_persistent; + FileSegmentKind segment_kind; CurrentMetrics::Increment metric_increment{CurrentMetrics::CacheFileSegments}; }; diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index a4b446bdd90..cb66aeafa0a 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -100,7 +100,10 @@ public: void SetUp() override { - setupLogs(TEST_LOG_LEVEL); + if(const char * test_log_level = std::getenv("TEST_LOG_LEVEL")) + setupLogs(test_log_level); + else + setupLogs(TEST_LOG_LEVEL); if (fs::exists(cache_base_path)) fs::remove_all(cache_base_path); @@ -561,24 +564,21 @@ TEST_F(FileCacheTest, rangeWriter) std::string data(100, '\xf0'); - /// Write first segment - ASSERT_EQ(writer.tryWrite(data.data(), 5, 0, false, false), 5); - ASSERT_EQ(writer.tryWrite(data.data(), 3, 5, false, false), 3); - ASSERT_EQ(writer.tryWrite(data.data(), 1, 8, false, false), 1); - ASSERT_EQ(writer.tryWrite(data.data(), 1, 9, false, false), 1); + size_t total_written = 0; + for (const size_t size : {3, 5, 8, 1, 1, 3}) + { + total_written += size; + ASSERT_EQ(writer.tryWrite(data.data(), size, writer.currentOffset()), size); + } + ASSERT_LT(total_written, settings.max_size); - /// Second segment starts - ASSERT_EQ(writer.tryWrite(data.data(), 1, 10, false, false), 1); - ASSERT_EQ(writer.tryWrite(data.data(), 1, 11, false, false), 1); - /// Can't write 10 bytes into the rest of current segment - ASSERT_EQ(writer.tryWrite(data.data(), 10, 12, false, false), 8); + size_t offset_before_unsuccessful_write = writer.currentOffset(); + size_t space_left = settings.max_size - total_written; + ASSERT_EQ(writer.tryWrite(data.data(), space_left + 1, writer.currentOffset()), 0); - /// Rest can be written into the next segment - ASSERT_EQ(writer.tryWrite(data.data(), 2, 20, false, false), 2); - /// Only 3 bytes left, can't write 4 and nothing should be written - ASSERT_EQ(writer.tryWrite(data.data(), 4, 22, false, false), 0); - ASSERT_EQ(writer.tryWrite(data.data(), 4, 22, false, false), 0); - ASSERT_EQ(writer.tryWrite(data.data(), 3, 22, false, false), 3); + ASSERT_EQ(writer.currentOffset(), offset_before_unsuccessful_write); + + ASSERT_EQ(writer.tryWrite(data.data(), space_left, writer.currentOffset()), space_left); writer.finalize(); } diff --git a/src/Storages/System/StorageSystemFilesystemCache.cpp b/src/Storages/System/StorageSystemFilesystemCache.cpp index cd9324b3253..5220a2a59e6 100644 --- a/src/Storages/System/StorageSystemFilesystemCache.cpp +++ b/src/Storages/System/StorageSystemFilesystemCache.cpp @@ -24,7 +24,8 @@ NamesAndTypesList StorageSystemFilesystemCache::getNamesAndTypes() {"cache_hits", std::make_shared()}, {"references", std::make_shared()}, {"downloaded_size", std::make_shared()}, - {"persistent", std::make_shared>()} + {"persistent", std::make_shared>()}, + {"kind", std::make_shared()}, }; } @@ -45,8 +46,7 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex for (const auto & file_segment : file_segments) { res_columns[0]->insert(cache_base_path); - res_columns[1]->insert( - cache->getPathInLocalCache(file_segment->key(), file_segment->offset(), file_segment->isPersistent())); + res_columns[1]->insert(file_segment->getPathInLocalCache()); const auto & range = file_segment->range(); res_columns[2]->insert(range.left); @@ -57,6 +57,7 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex res_columns[7]->insert(file_segment->getRefCount()); res_columns[8]->insert(file_segment->getDownloadedSize()); res_columns[9]->insert(file_segment->isPersistent()); + res_columns[10]->insert(toString(file_segment->getKind())); } } } From 1da1adc88475eb9e288e144143bb81dba8363477 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 29 Nov 2022 11:13:14 +0000 Subject: [PATCH 71/98] Better finalization in FileSegmentRangeWriter --- .../IO/CachedOnDiskWriteBufferFromFile.cpp | 23 ++++++-------- .../IO/CachedOnDiskWriteBufferFromFile.h | 5 +-- src/Disks/IO/FileCachePlaceholder.cpp | 31 ++++++------------- src/Disks/IO/FileCachePlaceholder.h | 3 +- src/Interpreters/Cache/FileSegment.cpp | 4 ++- 5 files changed, 26 insertions(+), 40 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index c9960e75c16..b11edd7e701 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -67,7 +67,9 @@ size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t o if (written_size == 0) break; - data += written_size; + if (data) + data += written_size; + size -= written_size; offset += written_size; total_written_size += written_size; @@ -158,25 +160,18 @@ bool FileSegmentRangeWriter::reserve(size_t size, size_t offset) return write(nullptr, size, offset, FileSegmentKind::Temporary); } -void FileSegmentRangeWriter::finalize(bool clear) +size_t FileSegmentRangeWriter::tryReserve(size_t size, size_t offset) +{ + return tryWrite(nullptr, size, offset, FileSegmentKind::Temporary); +} + +void FileSegmentRangeWriter::finalize() { if (finalized) return; auto & file_segments = file_segments_holder.file_segments; - /// Set all segments state to SKIP_CACHE to remove it from cache immediately on complete - /// Note: if segments are hold by someone else, it won't be removed - if (clear) - { - for (auto file_segment_it = file_segments.begin(); file_segment_it != file_segments.end(); ++file_segment_it) - { - completeFileSegment(**file_segment_it, FileSegment::State::SKIP_CACHE); - file_segments.erase(file_segment_it); - } - finalized = true; - } - if (file_segments.empty() || current_file_segment_it == file_segments.end()) return; diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h index bb63e727ec8..38ec2b46a5d 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h @@ -44,10 +44,11 @@ public: */ size_t tryWrite(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind = FileSegmentKind::Regular, bool strict = false); - /// Same as `write`, but doesn't write anything, just reserves some space in cache + /// Same as `write/tryWrite`, but doesn't write anything, just reserves some space in cache bool reserve(size_t size, size_t offset); + size_t tryReserve(size_t size, size_t offset); - void finalize(bool clear = false); + void finalize(); size_t currentOffset() const { return current_file_segment_write_offset; } diff --git a/src/Disks/IO/FileCachePlaceholder.cpp b/src/Disks/IO/FileCachePlaceholder.cpp index 63dc4ffb1c2..d45d7b2d83d 100644 --- a/src/Disks/IO/FileCachePlaceholder.cpp +++ b/src/Disks/IO/FileCachePlaceholder.cpp @@ -53,32 +53,21 @@ void FileCachePlaceholder::reserveImpl(size_t requested_size) /* source_path_ */ key); size_t current_offset = cache_writer->currentOffset(); - bool is_reserved = cache_writer->reserve(requested_size, current_offset); - if (!is_reserved) + size_t reserved_size = cache_writer->tryReserve(requested_size, current_offset); + if (reserved_size != requested_size) { - cache_writer->finalize(/* clear */ true); throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, - "Cannot reserve space in file cache ({} bytes required, {} / {} bytes used, {} / {} elements used)", - requested_size, file_cache->getUsedCacheSize(), file_cache->getTotalMaxSize(), - file_cache->getFileSegmentsNum(), file_cache->getTotalMaxElements()); + "Cannot reserve space in file cache " + "({} bytes required, got {} reserved " + "{} / {} bytes used, " + "{} / {} elements used)" + , requested_size, reserved_size + , file_cache->getUsedCacheSize(), file_cache->getTotalMaxSize() + , file_cache->getFileSegmentsNum(), file_cache->getTotalMaxElements()); } - + /// Add to cache_writers only if we successfully reserved space, otherwise free reserved_size back cache_writers.push_back(std::move(cache_writer)); } -FileCachePlaceholder::~FileCachePlaceholder() -{ - try - { - for (auto & cache_writer : cache_writers) - { - cache_writer->finalize(/* clear */ true); - } - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } -} } diff --git a/src/Disks/IO/FileCachePlaceholder.h b/src/Disks/IO/FileCachePlaceholder.h index 589aec289fe..6ddeb85286b 100644 --- a/src/Disks/IO/FileCachePlaceholder.h +++ b/src/Disks/IO/FileCachePlaceholder.h @@ -49,13 +49,12 @@ public: void reserveImpl(size_t requested_size) override; - ~FileCachePlaceholder() override; - private: std::string key_name; FileCache * file_cache; /// On each reserveImpl() call we create new FileSegmentRangeWriter that would be hold space + /// It's required to easily release already reserved space on unsuccessful attempt std::vector> cache_writers; }; diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 19ea6784c02..e098a994ba2 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -575,8 +575,10 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard & cach if (segment_kind == FileSegmentKind::Temporary && is_last_holder) { - cache->remove(key(), offset(), cache_lock, segment_lock); LOG_TEST(log, "Removing temporary file segment: {}", getInfoForLogUnlocked(segment_lock)); + detach(cache_lock, segment_lock); + setDownloadState(State::SKIP_CACHE); + cache->remove(key(), offset(), cache_lock, segment_lock); return; } From 9bdf01a55cd85bdc8db42f3eb0c44fc6902c67ae Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 30 Nov 2022 12:58:52 +0000 Subject: [PATCH 72/98] Do not use in FileSegment::getPathInLocalCache in StorageSystemFilesystemCache --- src/Interpreters/Cache/FileSegment.cpp | 1 + src/Interpreters/tests/gtest_lru_file_cache.cpp | 2 +- src/Storages/System/StorageSystemFilesystemCache.cpp | 6 +++++- 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index e098a994ba2..e070317e454 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -86,6 +86,7 @@ FileSegment::FileSegment( String FileSegment::getPathInLocalCache() const { + chassert(cache); return cache->getPathInLocalCache(key(), offset(), segment_kind); } diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index cb66aeafa0a..2d408bd9b34 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -100,7 +100,7 @@ public: void SetUp() override { - if(const char * test_log_level = std::getenv("TEST_LOG_LEVEL")) + if(const char * test_log_level = std::getenv("TEST_LOG_LEVEL")) // NOLINT(concurrency-mt-unsafe) setupLogs(test_log_level); else setupLogs(TEST_LOG_LEVEL); diff --git a/src/Storages/System/StorageSystemFilesystemCache.cpp b/src/Storages/System/StorageSystemFilesystemCache.cpp index 5220a2a59e6..bec92a60436 100644 --- a/src/Storages/System/StorageSystemFilesystemCache.cpp +++ b/src/Storages/System/StorageSystemFilesystemCache.cpp @@ -46,7 +46,11 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex for (const auto & file_segment : file_segments) { res_columns[0]->insert(cache_base_path); - res_columns[1]->insert(file_segment->getPathInLocalCache()); + + /// Do not use `file_segment->getPathInLocalCache` here because it will lead to nullptr dereference + /// (because file_segments in getSnapshot doesn't have `cache` field set) + res_columns[1]->insert( + cache->getPathInLocalCache(file_segment->key(), file_segment->offset(), file_segment->getKind())); const auto & range = file_segment->range(); res_columns[2]->insert(range.left); From 5d7cb6b85a75eb2e4db9305e113c8ed88a5ba753 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 1 Dec 2022 13:01:35 +0100 Subject: [PATCH 73/98] Change email for robot-clickhouse to immutable one --- docs/tools/release.sh | 2 +- tests/ci/cherry_pick.py | 2 +- tests/ci/style_check.py | 25 ++++++++++--------------- 3 files changed, 12 insertions(+), 17 deletions(-) diff --git a/docs/tools/release.sh b/docs/tools/release.sh index 1d344457bf1..67499631baa 100755 --- a/docs/tools/release.sh +++ b/docs/tools/release.sh @@ -19,7 +19,7 @@ then # 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@clickhouse.com" + git config user.email "robot-clickhouse@users.noreply.github.com" git config user.name "robot-clickhouse" # Add files. diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index d1c9d3d394c..9e7b250db75 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -79,7 +79,7 @@ Merge it only if you intend to backport changes to the target branch, otherwise self.backport_pr = None # type: Optional[PullRequest] self._backported = None # type: Optional[bool] self.git_prefix = ( # All commits to cherrypick are done as robot-clickhouse - "git -c user.email=robot-clickhouse@clickhouse.com " + "git -c user.email=robot-clickhouse@users.noreply.github.com " "-c user.name=robot-clickhouse -c commit.gpgsign=false" ) self.pre_check() diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 23a1dd467d7..a4bf5d54105 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -28,6 +28,13 @@ from upload_result_helper import upload_results NAME = "Style Check" +GIT_PREFIX = ( # All commits to remote are done as robot-clickhouse + "git -c user.email=robot-clickhouse@users.noreply.github.com " + "-c user.name=robot-clickhouse -c commit.gpgsign=false " + "-c core.sshCommand=" + "'ssh -o UserKnownHostsFile=/dev/null -o StrictHostKeyChecking=no'" +) + def process_result(result_folder): test_results = [] @@ -89,14 +96,8 @@ def checkout_head(pr_info: PRInfo): # We can't push to forks, sorry folks return remote_url = pr_info.event["pull_request"]["base"]["repo"]["ssh_url"] - git_prefix = ( # All commits to remote are done as robot-clickhouse - "git -c user.email=robot-clickhouse@clickhouse.com " - "-c user.name=robot-clickhouse -c commit.gpgsign=false " - "-c core.sshCommand=" - "'ssh -o UserKnownHostsFile=/dev/null -o StrictHostKeyChecking=no'" - ) fetch_cmd = ( - f"{git_prefix} fetch --depth=1 " + f"{GIT_PREFIX} fetch --depth=1 " f"{remote_url} {pr_info.head_ref}:head-{pr_info.head_ref}" ) if os.getenv("ROBOT_CLICKHOUSE_SSH_KEY", ""): @@ -118,15 +119,9 @@ def commit_push_staged(pr_info: PRInfo): if not git_staged: return remote_url = pr_info.event["pull_request"]["base"]["repo"]["ssh_url"] - git_prefix = ( # All commits to remote are done as robot-clickhouse - "git -c user.email=robot-clickhouse@clickhouse.com " - "-c user.name=robot-clickhouse -c commit.gpgsign=false " - "-c core.sshCommand=" - "'ssh -o UserKnownHostsFile=/dev/null -o StrictHostKeyChecking=no'" - ) - git_runner(f"{git_prefix} commit -m 'Automatic style fix'") + git_runner(f"{GIT_PREFIX} commit -m 'Automatic style fix'") push_cmd = ( - f"{git_prefix} push {remote_url} head-{pr_info.head_ref}:{pr_info.head_ref}" + f"{GIT_PREFIX} push {remote_url} head-{pr_info.head_ref}:{pr_info.head_ref}" ) if os.getenv("ROBOT_CLICKHOUSE_SSH_KEY", ""): with SSHKey("ROBOT_CLICKHOUSE_SSH_KEY"): From 6388bc5000043082f445119f4f43bd1e830542b4 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 1 Dec 2022 13:36:40 +0100 Subject: [PATCH 74/98] Followup fixes for XML helpers (#43845) --- src/Common/XMLUtils.cpp | 62 ++++++++++++++++++++++++++++++----------- src/Common/XMLUtils.h | 29 ++++++++++++------- 2 files changed, 65 insertions(+), 26 deletions(-) diff --git a/src/Common/XMLUtils.cpp b/src/Common/XMLUtils.cpp index 3d15400461e..db84e00adce 100644 --- a/src/Common/XMLUtils.cpp +++ b/src/Common/XMLUtils.cpp @@ -42,15 +42,15 @@ public: return s; } - template + template static ValueType getValue(const Node * node, const std::string & path, - const std::optional & default_value, const ParseFunction & parse_function) + const ValueType & default_value, const ParseFunction & parse_function) { const auto * value_node = node->getNodeByPath(path); if (!value_node) { - if (default_value) - return *default_value; + if constexpr (ReturnDefault) + return default_value; else throw Poco::NotFoundException(path); } @@ -59,34 +59,64 @@ public: }; -std::string getString(const Node * node, const std::string & path, const std::optional & default_value) +std::string getString(const Node * node, const std::string & path) { - return ParseHelper::getValue(node, path, default_value, ParseHelper::parseString); + return ParseHelper::getValue(node, path, {}, ParseHelper::parseString); } -Int64 getInt64(const Node * node, const std::string & path, const std::optional & default_value) +std::string getString(const Node * node, const std::string & path, const std::string & default_value) { - return ParseHelper::getValue(node, path, default_value, ParseHelper::parseInt64); + return ParseHelper::getValue(node, path, default_value, ParseHelper::parseString); } -UInt64 getUInt64(const Node * node, const std::string & path, const std::optional & default_value) +Int64 getInt64(const Node * node, const std::string & path) { - return ParseHelper::getValue(node, path, default_value, ParseHelper::parseUInt64); + return ParseHelper::getValue(node, path, {}, ParseHelper::parseInt64); } -int getInt(const Node * node, const std::string & path, const std::optional & default_value) +Int64 getInt64(const Node * node, const std::string & path, Int64 default_value) { - return ParseHelper::getValue(node, path, default_value, ParseHelper::parseInt); + return ParseHelper::getValue(node, path, default_value, ParseHelper::parseInt64); } -unsigned getUInt(const Node * node, const std::string & path, const std::optional & default_value) +UInt64 getUInt64(const Node * node, const std::string & path) { - return ParseHelper::getValue(node, path, default_value, ParseHelper::parseUInt); + return ParseHelper::getValue(node, path, {}, ParseHelper::parseUInt64); } -bool getBool(const Node * node, const std::string & path, const std::optional & default_value) +UInt64 getUInt64(const Node * node, const std::string & path, UInt64 default_value) { - return ParseHelper::getValue(node, path, default_value, ParseHelper::parseBool); + return ParseHelper::getValue(node, path, default_value, ParseHelper::parseUInt64); +} + +int getInt(const Node * node, const std::string & path) +{ + return ParseHelper::getValue(node, path, {}, ParseHelper::parseInt); +} + +int getInt(const Node * node, const std::string & path, int default_value) +{ + return ParseHelper::getValue(node, path, default_value, ParseHelper::parseInt); +} + +unsigned getUInt(const Node * node, const std::string & path) +{ + return ParseHelper::getValue(node, path, {}, ParseHelper::parseUInt); +} + +unsigned getUInt(const Node * node, const std::string & path, unsigned default_value) +{ + return ParseHelper::getValue(node, path, default_value, ParseHelper::parseUInt); +} + +bool getBool(const Node * node, const std::string & path) +{ + return ParseHelper::getValue(node, path, {}, ParseHelper::parseBool); +} + +bool getBool(const Node * node, const std::string & path, bool default_value) +{ + return ParseHelper::getValue(node, path, default_value, ParseHelper::parseBool); } } diff --git a/src/Common/XMLUtils.h b/src/Common/XMLUtils.h index 24efc691704..af9613b67ad 100644 --- a/src/Common/XMLUtils.h +++ b/src/Common/XMLUtils.h @@ -7,17 +7,26 @@ namespace DB:: XMLUtils { +/// Returns root element of the document. Poco::XML::Node * getRootNode(Poco::XML::Document * document); -std::string getString(const Poco::XML::Node * node, const std::string & path, const std::optional & default_value = std::nullopt); +/// Finds the element in the node's subtree by the specified path and returns its inner text +/// trying to parse it as the requested type. +/// Throws an exception if path is not found. +std::string getString(const Poco::XML::Node * node, const std::string & path); +Int64 getInt64(const Poco::XML::Node * node, const std::string & path); +UInt64 getUInt64(const Poco::XML::Node * node, const std::string & path); +int getInt(const Poco::XML::Node * node, const std::string & path); +unsigned getUInt(const Poco::XML::Node * node, const std::string & path); +bool getBool(const Poco::XML::Node * node, const std::string & path); -Int64 getInt64(const Poco::XML::Node * node, const std::string & path, const std::optional & default_value = std::nullopt); - -UInt64 getUInt64(const Poco::XML::Node * node, const std::string & path, const std::optional & default_value = std::nullopt); - -int getInt(const Poco::XML::Node * node, const std::string & path, const std::optional & default_value = std::nullopt); - -unsigned getUInt(const Poco::XML::Node * node, const std::string & path, const std::optional & default_value = std::nullopt); - -bool getBool(const Poco::XML::Node * node, const std::string & path, const std::optional & default_value = std::nullopt); +/// Finds the element in the node's subtree by the specified path and returns its inner text +/// trying to parse it as the requested type. +/// Returns the specified default value if path is not found. +std::string getString(const Poco::XML::Node * node, const std::string & path, const std::string & default_value); +Int64 getInt64(const Poco::XML::Node * node, const std::string & path, Int64 default_value); +UInt64 getUInt64(const Poco::XML::Node * node, const std::string & path, UInt64 default_value); +int getInt(const Poco::XML::Node * node, const std::string & path, int default_value); +unsigned getUInt(const Poco::XML::Node * node, const std::string & path, unsigned default_value); +bool getBool(const Poco::XML::Node * node, const std::string & path, bool default_value); } From 4c1fb543e1993d755e343b22af345be51e17075d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 1 Dec 2022 13:42:59 +0100 Subject: [PATCH 75/98] Sync everything to persistent storage to avoid writeback affects perf tests (#43530) Signed-off-by: Azat Khuzhin Signed-off-by: Azat Khuzhin --- docker/test/performance-comparison/perf.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 7a034c741eb..cb23372d31f 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -295,6 +295,9 @@ if not args.use_existing_tables: reportStageEnd("create") +# Let's sync the data to avoid writeback affects performance +os.system("sync") + # By default, test all queries. queries_to_run = range(0, len(test_queries)) From 9ef54c6ecc39eee4ae96ae6cb5a2ed12675e2730 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 1 Dec 2022 13:50:56 +0100 Subject: [PATCH 76/98] Try to use exchanges during database replicated replica recovery process (#43628) --- src/Databases/DatabaseReplicated.cpp | 48 ++++++-- .../test_replicated_database/test.py | 104 +++++++++++------- 2 files changed, 104 insertions(+), 48 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index c0bc9d3f3a2..2a9f06e77fc 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -702,7 +702,18 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep /// We will drop or move tables which exist only in local metadata Strings tables_to_detach; - std::vector> replicated_tables_to_rename; + + struct RenameEdge + { + String from; + String intermediate; + String to; + }; + + /// This is needed to generate intermediate name + String salt = toString(thread_local_rng()); + + std::vector replicated_tables_to_rename; size_t total_tables = 0; std::vector replicated_ids; for (auto existing_tables_it = getTablesIterator(getContext(), {}); existing_tables_it->isValid(); @@ -719,8 +730,15 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep { if (name != it->second) { + String intermediate_name; + /// Possibly we failed to rename it on previous iteration + /// And this table was already renamed to an intermediate name + if (startsWith(name, ".rename-") && !startsWith(it->second, ".rename-")) + intermediate_name = name; + else + intermediate_name = fmt::format(".rename-{}-{}", name, sipHash64(fmt::format("{}-{}", name, salt))); /// Need just update table name - replicated_tables_to_rename.emplace_back(name, it->second); + replicated_tables_to_rename.push_back({name, intermediate_name, it->second}); } continue; } @@ -840,13 +858,13 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep tables_to_detach.size(), dropped_dictionaries, dropped_tables.size() - dropped_dictionaries, moved_tables); /// Now database is cleared from outdated tables, let's rename ReplicatedMergeTree tables to actual names - for (const auto & old_to_new : replicated_tables_to_rename) + /// We have to take into account that tables names could be changed with two general queries + /// 1) RENAME TABLE. There could be multiple pairs of tables (e.g. RENAME b TO c, a TO b, c TO d) + /// But it is equal to multiple subsequent RENAMEs each of which operates only with two tables + /// 2) EXCHANGE TABLE. This query swaps two names atomically and could not be represented with two separate RENAMEs + auto rename_table = [&](String from, String to) { - const String & from = old_to_new.first; - const String & to = old_to_new.second; - LOG_DEBUG(log, "Will RENAME TABLE {} TO {}", backQuoteIfNeed(from), backQuoteIfNeed(to)); - /// TODO Maybe we should do it in two steps: rename all tables to temporary names and then rename them to actual names? DDLGuardPtr table_guard = DatabaseCatalog::instance().getDDLGuard(db_name, std::min(from, to)); DDLGuardPtr to_table_guard = DatabaseCatalog::instance().getDDLGuard(db_name, std::max(from, to)); @@ -858,7 +876,23 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep DatabaseAtomic::renameTable(make_query_context(), from, *this, to, false, false); tables_metadata_digest = new_digest; assert(checkDigestValid(getContext())); + }; + + LOG_DEBUG(log, "Starting first stage of renaming process. Will rename tables to intermediate names"); + for (auto & [from, intermediate, _] : replicated_tables_to_rename) + { + /// Due to some unknown failures there could be tables + /// which are already in an intermediate state + /// For them we skip the first stage + if (from == intermediate) + continue; + rename_table(from, intermediate); } + LOG_DEBUG(log, "Starting second stage of renaming process. Will rename tables from intermediate to desired names"); + for (auto & [_, intermediate, to] : replicated_tables_to_rename) + rename_table(intermediate, to); + + LOG_DEBUG(log, "Renames completed succesessfully"); for (const auto & id : dropped_tables) DatabaseCatalog::instance().waitTableFinallyDropped(id); diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index de5433d5beb..1e6a39ee1bd 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -592,60 +592,64 @@ def test_alters_from_different_replicas(started_cluster): def create_some_tables(db): settings = {"distributed_ddl_task_timeout": 0} - main_node.query( - "CREATE TABLE {}.t1 (n int) ENGINE=Memory".format(db), settings=settings - ) + main_node.query(f"CREATE TABLE {db}.t1 (n int) ENGINE=Memory", settings=settings) dummy_node.query( - "CREATE TABLE {}.t2 (s String) ENGINE=Memory".format(db), settings=settings + f"CREATE TABLE {db}.t2 (s String) ENGINE=Memory", settings=settings ) main_node.query( - "CREATE TABLE {}.mt1 (n int) ENGINE=MergeTree order by n".format(db), + f"CREATE TABLE {db}.mt1 (n int) ENGINE=MergeTree order by n", settings=settings, ) dummy_node.query( - "CREATE TABLE {}.mt2 (n int) ENGINE=MergeTree order by n".format(db), + f"CREATE TABLE {db}.mt2 (n int) ENGINE=MergeTree order by n", settings=settings, ) main_node.query( - "CREATE TABLE {}.rmt1 (n int) ENGINE=ReplicatedMergeTree order by n".format(db), + f"CREATE TABLE {db}.rmt1 (n int) ENGINE=ReplicatedMergeTree order by n", settings=settings, ) dummy_node.query( - "CREATE TABLE {}.rmt2 (n int) ENGINE=ReplicatedMergeTree order by n".format(db), + f"CREATE TABLE {db}.rmt2 (n int) ENGINE=ReplicatedMergeTree order by n", settings=settings, ) main_node.query( - "CREATE TABLE {}.rmt3 (n int) ENGINE=ReplicatedMergeTree order by n".format(db), + f"CREATE TABLE {db}.rmt3 (n int) ENGINE=ReplicatedMergeTree order by n", settings=settings, ) dummy_node.query( - "CREATE TABLE {}.rmt5 (n int) ENGINE=ReplicatedMergeTree order by n".format(db), + f"CREATE TABLE {db}.rmt5 (n int) ENGINE=ReplicatedMergeTree order by n", settings=settings, ) main_node.query( - "CREATE MATERIALIZED VIEW {}.mv1 (n int) ENGINE=ReplicatedMergeTree order by n AS SELECT n FROM recover.rmt1".format( - db - ), + f"CREATE MATERIALIZED VIEW {db}.mv1 (n int) ENGINE=ReplicatedMergeTree order by n AS SELECT n FROM recover.rmt1", settings=settings, ) dummy_node.query( - "CREATE MATERIALIZED VIEW {}.mv2 (n int) ENGINE=ReplicatedMergeTree order by n AS SELECT n FROM recover.rmt2".format( - db - ), + f"CREATE MATERIALIZED VIEW {db}.mv2 (n int) ENGINE=ReplicatedMergeTree order by n AS SELECT n FROM recover.rmt2", settings=settings, ) main_node.query( - "CREATE DICTIONARY {}.d1 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n " + f"CREATE DICTIONARY {db}.d1 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n " "SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'rmt1' PASSWORD '' DB 'recover')) " - "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT())".format(db) + "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT())" ) dummy_node.query( - "CREATE DICTIONARY {}.d2 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n " + f"CREATE DICTIONARY {db}.d2 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n " "SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'rmt2' PASSWORD '' DB 'recover')) " - "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT())".format(db) + "LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT())" ) +# These tables are used to check that DatabaseReplicated correctly renames all the tables in case when it restores from the lost state +def create_table_for_exchanges(db): + settings = {"distributed_ddl_task_timeout": 0} + for table in ["a1", "a2", "a3", "a4", "a5", "a6"]: + main_node.query( + f"CREATE TABLE {db}.{table} (s String) ENGINE=ReplicatedMergeTree order by s", + settings=settings, + ) + + def test_recover_staled_replica(started_cluster): main_node.query( "CREATE DATABASE recover ENGINE = Replicated('/clickhouse/databases/recover', 'shard1', 'replica1');" @@ -659,13 +663,20 @@ def test_recover_staled_replica(started_cluster): settings = {"distributed_ddl_task_timeout": 0} create_some_tables("recover") + create_table_for_exchanges("recover") for table in ["t1", "t2", "mt1", "mt2", "rmt1", "rmt2", "rmt3", "rmt5"]: - main_node.query("INSERT INTO recover.{} VALUES (42)".format(table)) + main_node.query(f"INSERT INTO recover.{table} VALUES (42)") for table in ["t1", "t2", "mt1", "mt2"]: - dummy_node.query("INSERT INTO recover.{} VALUES (42)".format(table)) + dummy_node.query(f"INSERT INTO recover.{table} VALUES (42)") + + for i, table in enumerate(["a1", "a2", "a3", "a4", "a5", "a6"]): + main_node.query(f"INSERT INTO recover.{table} VALUES ('{str(i + 1) * 10}')") + for table in ["rmt1", "rmt2", "rmt3", "rmt5"]: - main_node.query("SYSTEM SYNC REPLICA recover.{}".format(table)) + main_node.query(f"SYSTEM SYNC REPLICA recover.{table}") + for table in ["a1", "a2", "a3", "a4", "a5", "a6"]: + main_node.query(f"SYSTEM SYNC REPLICA recover.{table}") with PartitionManager() as pm: pm.drop_instance_zk_connections(dummy_node) @@ -699,19 +710,15 @@ def test_recover_staled_replica(started_cluster): ).strip() ) main_node.query_with_retry( - "ALTER TABLE recover.`{}` MODIFY COLUMN n int DEFAULT 42".format( - inner_table - ), + f"ALTER TABLE recover.`{inner_table}` MODIFY COLUMN n int DEFAULT 42", settings=settings, ) main_node.query_with_retry( - "ALTER TABLE recover.mv1 MODIFY QUERY SELECT m FROM recover.rmt1".format( - inner_table - ), + "ALTER TABLE recover.mv1 MODIFY QUERY SELECT m FROM recover.rmt1", settings=settings, ) main_node.query_with_retry( - "RENAME TABLE recover.mv2 TO recover.mv3".format(inner_table), + "RENAME TABLE recover.mv2 TO recover.mv3", settings=settings, ) @@ -727,11 +734,18 @@ def test_recover_staled_replica(started_cluster): "CREATE TABLE recover.tmp AS recover.m1", settings=settings ) + main_node.query("EXCHANGE TABLES recover.a1 AND recover.a2", settings=settings) + main_node.query("EXCHANGE TABLES recover.a3 AND recover.a4", settings=settings) + main_node.query("EXCHANGE TABLES recover.a5 AND recover.a4", settings=settings) + main_node.query("EXCHANGE TABLES recover.a6 AND recover.a3", settings=settings) + main_node.query("RENAME TABLE recover.a6 TO recover.a7", settings=settings) + main_node.query("RENAME TABLE recover.a1 TO recover.a8", settings=settings) + assert ( main_node.query( "SELECT name FROM system.tables WHERE database='recover' AND name NOT LIKE '.inner_id.%' ORDER BY name" ) - == "d1\nd2\nm1\nmt1\nmt2\nmv1\nmv3\nrmt1\nrmt2\nrmt4\nt2\ntmp\n" + == "a2\na3\na4\na5\na7\na8\nd1\nd2\nm1\nmt1\nmt2\nmv1\nmv3\nrmt1\nrmt2\nrmt4\nt2\ntmp\n" ) query = ( "SELECT name, uuid, create_table_query FROM system.tables WHERE database='recover' AND name NOT LIKE '.inner_id.%' " @@ -752,6 +766,12 @@ def test_recover_staled_replica(started_cluster): == "2\n" ) + # Check that Database Replicated renamed all the tables correctly + for i, table in enumerate(["a2", "a8", "a5", "a7", "a4", "a3"]): + assert ( + dummy_node.query(f"SELECT * FROM recover.{table}") == f"{str(i + 1) * 10}\n" + ) + for table in [ "m1", "t2", @@ -765,11 +785,11 @@ def test_recover_staled_replica(started_cluster): "mv1", "mv3", ]: - assert main_node.query("SELECT (*,).1 FROM recover.{}".format(table)) == "42\n" + assert main_node.query(f"SELECT (*,).1 FROM recover.{table}") == "42\n" for table in ["t2", "rmt1", "rmt2", "rmt4", "d1", "d2", "mt2", "mv1", "mv3"]: - assert dummy_node.query("SELECT (*,).1 FROM recover.{}".format(table)) == "42\n" + assert dummy_node.query(f"SELECT (*,).1 FROM recover.{table}") == "42\n" for table in ["m1", "mt1"]: - assert dummy_node.query("SELECT count() FROM recover.{}".format(table)) == "0\n" + assert dummy_node.query(f"SELECT count() FROM recover.{table}") == "0\n" global test_recover_staled_replica_run assert ( dummy_node.query( @@ -784,20 +804,22 @@ def test_recover_staled_replica(started_cluster): == f"{test_recover_staled_replica_run}\n" ) test_recover_staled_replica_run += 1 + + print(dummy_node.query("SHOW DATABASES")) + print(dummy_node.query("SHOW TABLES FROM recover_broken_tables")) + print(dummy_node.query("SHOW TABLES FROM recover_broken_replicated_tables")) + table = dummy_node.query( - "SHOW TABLES FROM recover_broken_tables LIKE 'mt1_29_%' LIMIT 1" + "SHOW TABLES FROM recover_broken_tables LIKE 'mt1_41_%' LIMIT 1" ).strip() assert ( - dummy_node.query("SELECT (*,).1 FROM recover_broken_tables.{}".format(table)) - == "42\n" + dummy_node.query(f"SELECT (*,).1 FROM recover_broken_tables.{table}") == "42\n" ) table = dummy_node.query( - "SHOW TABLES FROM recover_broken_replicated_tables LIKE 'rmt5_29_%' LIMIT 1" + "SHOW TABLES FROM recover_broken_replicated_tables LIKE 'rmt5_41_%' LIMIT 1" ).strip() assert ( - dummy_node.query( - "SELECT (*,).1 FROM recover_broken_replicated_tables.{}".format(table) - ) + dummy_node.query(f"SELECT (*,).1 FROM recover_broken_replicated_tables.{table}") == "42\n" ) From 554ff1db6f3b914e00544c766dd7556af073f038 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Dec 2022 12:54:29 +0000 Subject: [PATCH 77/98] Small changes --- src/Coordination/Changelog.cpp | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 4f36e635eb9..c85caa43f19 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -552,11 +552,13 @@ void Changelog::writeThread() WriteOperation write_operation; while (write_operations.pop(write_operation)) { - std::lock_guard writer_lock(writer_mutex); - assert(initialized && current_writer); + assert(initialized); if (auto * append_log = std::get_if(&write_operation)) { + std::lock_guard writer_lock(writer_mutex); + assert(current_writer); + const auto & current_changelog_description = existing_changelogs[current_writer->getStartIndex()]; const bool log_is_complete = append_log->index - current_writer->getStartIndex() == current_changelog_description.expectedEntriesCountInLog(); @@ -569,8 +571,11 @@ void Changelog::writeThread() { const auto & flush = std::get(write_operation); - if (current_writer) - current_writer->flush(force_sync); + { + std::lock_guard writer_lock(writer_mutex); + if (current_writer) + current_writer->flush(force_sync); + } { std::lock_guard lock{durable_idx_mutex}; From 1c5610bf65a28be927061f7a1c802682f28b3b64 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 1 Dec 2022 14:06:17 +0100 Subject: [PATCH 78/98] Changes during the review --- docker/test/style/process_style_check_result.py | 4 ++-- docker/test/style/run.sh | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docker/test/style/process_style_check_result.py b/docker/test/style/process_style_check_result.py index d914ceffb86..6dc3d05d051 100755 --- a/docker/test/style/process_style_check_result.py +++ b/docker/test/style/process_style_check_result.py @@ -11,7 +11,7 @@ def process_result(result_folder): description = "" test_results = [] checks = ( - "header duplicates", + "duplicate includes", "shellcheck", "style", "black", @@ -19,7 +19,7 @@ def process_result(result_folder): "typos", "whitespaces", "workflows", - "doc typos", + "docs spelling", ) for name in checks: diff --git a/docker/test/style/run.sh b/docker/test/style/run.sh index 911536ed03b..80911bf8627 100755 --- a/docker/test/style/run.sh +++ b/docker/test/style/run.sh @@ -4,7 +4,7 @@ cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_output/check_status.tsv echo "Check duplicates" | ts -./check-duplicate-includes.sh |& tee /test_output/header_duplicates_output.txt +./check-duplicate-includes.sh |& tee /test_output/duplicate_includes_output.txt echo "Check style" | ts ./check-style -n |& tee /test_output/style_output.txt echo "Check python formatting with black" | ts @@ -14,7 +14,7 @@ echo "Check python type hinting with mypy" | ts echo "Check typos" | ts ./check-typos |& tee /test_output/typos_output.txt echo "Check docs spelling" | ts -./check-doc-aspell |& tee /test_output/doc_typos_output.txt +./check-doc-aspell |& tee /test_output/docs_spelling_output.txt echo "Check whitespaces" | ts ./check-whitespaces -n |& tee /test_output/whitespaces_output.txt echo "Check workflows" | ts From fd2c7657bec8f1bddc8fb31537d7051cddfd2535 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 1 Dec 2022 16:03:39 +0100 Subject: [PATCH 79/98] do not read part status just for logging --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 8 -------- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 - src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 2 +- 4 files changed, 2 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 630fbda833e..e03e430b144 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -2041,14 +2041,6 @@ bool isCompressedFromIndexExtension(const String & index_extension) return index_extension == getIndexExtension(true); } -Strings getPartsNamesWithStates(const MergeTreeDataPartsVector & parts) -{ - Strings part_names; - for (const auto & p : parts) - part_names.push_back(p->getNameWithState()); - return part_names; -} - Strings getPartsNames(const MergeTreeDataPartsVector & parts) { Strings part_names; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 7801e5a60ba..e6c6f02b098 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -597,7 +597,6 @@ bool isCompressedFromIndexExtension(const String & index_extension); using MergeTreeDataPartsVector = std::vector; -Strings getPartsNamesWithStates(const MergeTreeDataPartsVector & parts); Strings getPartsNames(const MergeTreeDataPartsVector & parts); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6e14fec22af..730a66b4c32 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4691,7 +4691,7 @@ void MergeTreeData::filterVisibleDataParts(DataPartsVector & maybe_visible_parts [[maybe_unused]] size_t visible_size = maybe_visible_parts.size(); LOG_TEST(log, "Got {} parts (of {}) visible in snapshot {} (TID {}): {}", - visible_size, total_size, snapshot_version, current_tid, fmt::join(getPartsNamesWithStates(maybe_visible_parts), ", ")); + visible_size, total_size, snapshot_version, current_tid, fmt::join(getPartsNames(maybe_visible_parts), ", ")); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 030ccb5ace2..bd57579505a 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1541,7 +1541,7 @@ void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont LOG_TEST(log, "Made {} empty parts in order to cover {} parts. Empty parts: {}, covered parts: {}. With txn {}", future_parts.size(), parts.size(), - fmt::join(getPartsNames(future_parts), ", "), fmt::join(getPartsNamesWithStates(parts), ", "), + fmt::join(getPartsNames(future_parts), ", "), fmt::join(getPartsNames(parts), ", "), transaction.getTID()); captureTmpDirectoryHolders(*this, future_parts); From c2a19fc354e17a31d22d3baecfd7511ced6be1cd Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 1 Dec 2022 19:41:29 +0300 Subject: [PATCH 80/98] Update test.py --- .../integration/test_alter_settings_on_cluster/test.py | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/tests/integration/test_alter_settings_on_cluster/test.py b/tests/integration/test_alter_settings_on_cluster/test.py index 51911dec6eb..32f7f2efa30 100644 --- a/tests/integration/test_alter_settings_on_cluster/test.py +++ b/tests/integration/test_alter_settings_on_cluster/test.py @@ -53,6 +53,16 @@ def test_default_database_on_cluster(started_cluster): sql="SHOW CREATE test_local_table FORMAT TSV", ).endswith("old_parts_lifetime = 100\n") + ch1.query_and_get_error( + database="test_default_database", + sql="ALTER TABLE test_local_table MODIFY SETTING temporary_directories_lifetime = 1 RESET SETTING old_parts_lifetime;", + ) + + ch1.query_and_get_error( + database="test_default_database", + sql="ALTER TABLE test_local_table RESET SETTING old_parts_lifetime MODIFY SETTING temporary_directories_lifetime = 1;", + ) + ch1.query( database="test_default_database", sql="ALTER TABLE test_local_table ON CLUSTER 'cluster' RESET SETTING old_parts_lifetime;", From 933f2a93f351633c8b2f898c7c9cd3c8c2450cd3 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Thu, 1 Dec 2022 17:52:50 -0500 Subject: [PATCH 81/98] Update docs/en/operations/caches.md --- docs/en/operations/caches.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/caches.md b/docs/en/operations/caches.md index 0001c383486..86760ec245f 100644 --- a/docs/en/operations/caches.md +++ b/docs/en/operations/caches.md @@ -23,4 +23,4 @@ Additional cache types: - Schema inference cache. - [Filesystem cache](storing-data.md) over S3, Azure, Local and other disks. -To drop one of caches, use [SYSTEM DROP ... CACHE](../sql-reference/statements/system.md#drop-mark-cache) statements. +To drop one of the caches, use [SYSTEM DROP ... CACHE](../sql-reference/statements/system.md#drop-mark-cache) statements. From 200141da136d6d97b597c56c8c2ad32caf9ccbb7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Dec 2022 08:16:18 +0000 Subject: [PATCH 82/98] Use logger in tests --- src/Coordination/tests/gtest_coordination.cpp | 87 ++++++++++--------- 1 file changed, 48 insertions(+), 39 deletions(-) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 1fc02e83866..628fe408d01 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -67,6 +67,7 @@ class CoordinationTest : public ::testing::TestWithParam { protected: DB::KeeperContextPtr keeper_context = std::make_shared(); + Poco::Logger * log{&Poco::Logger::get("CoordinationTest")}; }; TEST_P(CoordinationTest, BuildTest) @@ -214,7 +215,7 @@ TEST_P(CoordinationTest, TestSummingRaft1) while (s1.state_machine->getValue() != 143) { - std::cout << "Waiting s1 to apply entry\n"; + LOG_INFO(log, "Waiting s1 to apply entry"); std::this_thread::sleep_for(std::chrono::milliseconds(100)); } @@ -1407,7 +1408,7 @@ void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, uint nuraft::async_result::handler_type when_done = [&snapshot_created] (bool & ret, nuraft::ptr &/*exception*/) { snapshot_created = ret; - std::cerr << "Snapshot finished\n"; + LOG_INFO(&Poco::Logger::get("CoordinationTest"), "Snapshot finished"); }; state_machine->create_snapshot(s, when_done); @@ -1760,7 +1761,7 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) auto params = GetParam(); ChangelogDirTest test("./logs"); { - std::cerr << "================First time=====================\n"; + LOG_INFO(log, "================First time====================="); DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression); changelog.init(1, 0); auto entry = getLogEntry("hello_world", 1000); @@ -1771,7 +1772,7 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) } { - std::cerr << "================Second time=====================\n"; + LOG_INFO(log, "================Second time====================="); DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression); changelog.init(1, 0); auto entry = getLogEntry("hello_world", 1000); @@ -1782,7 +1783,7 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) } { - std::cerr << "================Third time=====================\n"; + LOG_INFO(log, "================Third time====================="); DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression); changelog.init(1, 0); auto entry = getLogEntry("hello_world", 1000); @@ -1793,7 +1794,7 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) } { - std::cerr << "================Fourth time=====================\n"; + LOG_INFO(log, "================Fourth time====================="); DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression); changelog.init(1, 0); auto entry = getLogEntry("hello_world", 1000); @@ -1811,7 +1812,7 @@ TEST_P(CoordinationTest, ChangelogInsertMultipleTimesSmooth) ChangelogDirTest test("./logs"); for (size_t i = 0; i < 36; ++i) { - std::cerr << "================First time=====================\n"; + LOG_INFO(log, "================First time====================="); DB::KeeperLogStore changelog("./logs", 100, true, params.enable_compression); changelog.init(1, 0); for (size_t j = 0; j < 7; ++j) @@ -1832,41 +1833,49 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard) { auto params = GetParam(); ChangelogDirTest test("./logs"); - std::cerr << "================First time=====================\n"; - DB::KeeperLogStore changelog1("./logs", 100, true, params.enable_compression); - changelog1.init(1, 0); - auto entry = getLogEntry("hello_world", 1000); - changelog1.append(entry); - changelog1.end_of_append_batch(0, 0); - EXPECT_EQ(changelog1.next_slot(), 2); - waitDurableLogs(changelog1); + { + LOG_INFO(log, "================First time====================="); + DB::KeeperLogStore changelog1("./logs", 100, true, params.enable_compression); + changelog1.init(1, 0); + auto entry = getLogEntry("hello_world", 1000); + changelog1.append(entry); + changelog1.end_of_append_batch(0, 0); + EXPECT_EQ(changelog1.next_slot(), 2); + waitDurableLogs(changelog1); + } - std::cerr << "================Second time=====================\n"; - DB::KeeperLogStore changelog2("./logs", 100, true, params.enable_compression); - changelog2.init(1, 0); - entry = getLogEntry("hello_world", 1000); - changelog2.append(entry); - changelog2.end_of_append_batch(0, 0); - EXPECT_EQ(changelog2.next_slot(), 3); - waitDurableLogs(changelog2); + { + LOG_INFO(log, "================Second time====================="); + DB::KeeperLogStore changelog2("./logs", 100, true, params.enable_compression); + changelog2.init(1, 0); + auto entry = getLogEntry("hello_world", 1000); + changelog2.append(entry); + changelog2.end_of_append_batch(0, 0); + EXPECT_EQ(changelog2.next_slot(), 3); + waitDurableLogs(changelog2); + } - std::cerr << "================Third time=====================\n"; - DB::KeeperLogStore changelog3("./logs", 100, true, params.enable_compression); - changelog3.init(1, 0); - entry = getLogEntry("hello_world", 1000); - changelog3.append(entry); - changelog3.end_of_append_batch(0, 0); - EXPECT_EQ(changelog3.next_slot(), 4); - waitDurableLogs(changelog3); + { + LOG_INFO(log, "================Third time====================="); + DB::KeeperLogStore changelog3("./logs", 100, true, params.enable_compression); + changelog3.init(1, 0); + auto entry = getLogEntry("hello_world", 1000); + changelog3.append(entry); + changelog3.end_of_append_batch(0, 0); + EXPECT_EQ(changelog3.next_slot(), 4); + waitDurableLogs(changelog3); + } - std::cerr << "================Fourth time=====================\n"; - DB::KeeperLogStore changelog4("./logs", 100, true, params.enable_compression); - changelog4.init(1, 0); - entry = getLogEntry("hello_world", 1000); - changelog4.append(entry); - changelog4.end_of_append_batch(0, 0); - EXPECT_EQ(changelog4.next_slot(), 5); - waitDurableLogs(changelog4); + { + LOG_INFO(log, "================Fourth time====================="); + DB::KeeperLogStore changelog4("./logs", 100, true, params.enable_compression); + changelog4.init(1, 0); + auto entry = getLogEntry("hello_world", 1000); + changelog4.append(entry); + changelog4.end_of_append_batch(0, 0); + EXPECT_EQ(changelog4.next_slot(), 5); + waitDurableLogs(changelog4); + } } TEST_P(CoordinationTest, TestStorageSnapshotEqual) From 2623804dc50396ec79744c003260fda393407cb5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Dec 2022 08:17:48 +0000 Subject: [PATCH 83/98] Update NuRaft --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index 2bb4a8b6465..afc36dfa9b0 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 2bb4a8b6465f8556f19db3cb1529ed8e9dfc5236 +Subproject commit afc36dfa9b0beb45bc4cd935060631cc80ba04a5 From da0002c4c469e623a3e324d9df0f37f9af4f98a6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Dec 2022 10:52:01 +0000 Subject: [PATCH 84/98] Document how the library-bridge port can be changed Fixes #43605 --- docs/en/sql-reference/functions/other-functions.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index b9ec21bb59d..536249626e5 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -1865,6 +1865,17 @@ Next, specify the path to `libcatboostmodel.` in the clickhouse config ``` +For security and isolation reasons, the model evaluation does not run in the server process but in the clickhouse-library-bridge process. +At the first execution of `catboostEvaluate()`, the server starts the library bridge process if it is not running already. Both processes +communicate using a HTTP interface. By default, port `9012` is used. A different port can be specified as follows - this is useful if port +`9012` is already assigned to a different service. + +``` xml + + 9019 + +``` + 2. Train a catboost model using libcatboost See [Training and applying models](https://catboost.ai/docs/features/training.html#training) for how to train catboost models from a training data set. From c35b2a64956738ad6a4a004d15fcca2b16bbe598 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 2 Dec 2022 13:57:11 +0100 Subject: [PATCH 85/98] Add a limit for string size in RowBinary format (#43842) --- docs/en/interfaces/formats.md | 4 ++ docs/en/operations/settings/settings.md | 8 ++++ .../AggregateFunctionGroupArrayInsertAt.h | 4 +- .../AggregateFunctionMinMaxAny.h | 4 +- .../AggregateFunctionSumMap.h | 12 +++--- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 1 + src/DataTypes/Serializations/ISerialization.h | 8 ++-- .../SerializationAggregateFunction.cpp | 8 ++-- .../SerializationAggregateFunction.h | 8 ++-- .../Serializations/SerializationArray.cpp | 16 ++++---- .../Serializations/SerializationArray.h | 8 ++-- .../SerializationDecimalBase.cpp | 8 ++-- .../Serializations/SerializationDecimalBase.h | 8 ++-- .../SerializationFixedString.cpp | 8 ++-- .../Serializations/SerializationFixedString.h | 8 ++-- .../SerializationLowCardinality.cpp | 16 ++++---- .../SerializationLowCardinality.h | 8 ++-- .../Serializations/SerializationMap.cpp | 20 +++++----- .../Serializations/SerializationMap.h | 8 ++-- .../Serializations/SerializationNothing.h | 8 ++-- .../Serializations/SerializationNullable.cpp | 16 ++++---- .../Serializations/SerializationNullable.h | 8 ++-- .../Serializations/SerializationNumber.cpp | 8 ++-- .../Serializations/SerializationNumber.h | 8 ++-- .../Serializations/SerializationObject.cpp | 8 ++-- .../Serializations/SerializationObject.h | 8 ++-- .../Serializations/SerializationSparse.cpp | 14 +++---- .../Serializations/SerializationSparse.h | 8 ++-- .../Serializations/SerializationString.cpp | 40 +++++++++++++++++-- .../Serializations/SerializationString.h | 8 ++-- .../Serializations/SerializationTuple.cpp | 16 ++++---- .../Serializations/SerializationTuple.h | 8 ++-- .../Serializations/SerializationUUID.cpp | 8 ++-- .../Serializations/SerializationUUID.h | 8 ++-- .../Serializations/SerializationWrapper.cpp | 16 ++++---- .../Serializations/SerializationWrapper.h | 8 ++-- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 2 + .../Formats/Impl/BinaryRowInputFormat.cpp | 4 +- .../Formats/Impl/BinaryRowOutputFormat.cpp | 10 ++--- .../Formats/Impl/BinaryRowOutputFormat.h | 3 +- src/Storages/KVStorageUtils.cpp | 4 +- src/Storages/KVStorageUtils.h | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 10 ++--- .../MergeTreeDataPartWriterOnDisk.cpp | 4 +- .../MergeTree/MergeTreeIndexHypothesis.cpp | 4 +- .../MergeTree/MergeTreeIndexMinMax.cpp | 16 ++++---- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 6 +-- src/Storages/MergeTree/MergeTreePartition.cpp | 4 +- src/Storages/RocksDB/EmbeddedRocksDBSink.cpp | 2 +- .../RocksDB/StorageEmbeddedRocksDB.cpp | 2 +- src/Storages/StorageKeeperMap.cpp | 2 +- ...496_row_binary_large_string_size.reference | 1 + .../02496_row_binary_large_string_size.sh | 7 ++++ 55 files changed, 253 insertions(+), 195 deletions(-) create mode 100644 tests/queries/0_stateless/02496_row_binary_large_string_size.reference create mode 100755 tests/queries/0_stateless/02496_row_binary_large_string_size.sh diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 3221b1a06fa..731348abfe7 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1456,6 +1456,10 @@ If setting [input_format_with_types_use_header](../operations/settings/settings. the types from input data will be compared with the types of the corresponding columns from the table. Otherwise, the second row will be skipped. ::: +## RowBinary format settings {#row-binary-format-settings} + +- [format_binary_max_string_size](../operations/settings/settings.md#format_binary_max_string_size) - The maximum allowed size for String in RowBinary format. Default value - `1GiB`. + ## Values {#data-format-values} Prints every row in brackets. Rows are separated by commas. There is no comma after the last row. The values inside the brackets are also comma-separated. Numbers are output in a decimal format without quotes. Arrays are output in square brackets. Strings, dates, and dates with times are output in quotes. Escaping rules and parsing are similar to the [TabSeparated](#tabseparated) format. During formatting, extra spaces aren’t inserted, but during parsing, they are allowed and skipped (except for spaces inside array values, which are not allowed). [NULL](../sql-reference/syntax.md) is represented as `NULL`. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 7494f3db71a..1ba8c78ee1b 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4829,3 +4829,11 @@ Disabled by default. Allow skipping columns with unsupported types while schema inference for format BSONEachRow. Disabled by default. + +## RowBinary format settings {#row-binary-format-settings} + +### format_binary_max_string_size {#format_binary_max_string_size} + +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. + +Default value: `1GiB` diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h b/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h index aa3f78c8f0b..a1a2ce2669b 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h @@ -160,7 +160,7 @@ public: else { writeBinary(UInt8(0), buf); - serialization->serializeBinary(elem, buf); + serialization->serializeBinary(elem, buf, {}); } } } @@ -181,7 +181,7 @@ public: UInt8 is_null = 0; readBinary(is_null, buf); if (!is_null) - serialization->deserializeBinary(arr[i], buf); + serialization->deserializeBinary(arr[i], buf, {}); } } diff --git a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h index d6a8e895a11..8117daa4760 100644 --- a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h +++ b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h @@ -795,7 +795,7 @@ public: if (!value.isNull()) { writeBinary(true, buf); - serialization.serializeBinary(value, buf); + serialization.serializeBinary(value, buf, {}); } else writeBinary(false, buf); @@ -807,7 +807,7 @@ public: readBinary(is_not_null, buf); if (is_not_null) - serialization.deserializeBinary(value, buf); + serialization.deserializeBinary(value, buf, {}); } void change(const IColumn & column, size_t row_num, Arena *) diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index e9db1a71511..1e32be987ff 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -296,19 +296,19 @@ public: { case 0: { - serialize = [&](size_t col_idx, const Array & values){ values_serializations[col_idx]->serializeBinary(values[col_idx], buf); }; + serialize = [&](size_t col_idx, const Array & values){ values_serializations[col_idx]->serializeBinary(values[col_idx], buf, {}); }; break; } case 1: { - serialize = [&](size_t col_idx, const Array & values){ promoted_values_serializations[col_idx]->serializeBinary(values[col_idx], buf); }; + serialize = [&](size_t col_idx, const Array & values){ promoted_values_serializations[col_idx]->serializeBinary(values[col_idx], buf, {}); }; break; } } for (const auto & elem : merged_maps) { - keys_serialization->serializeBinary(elem.first, buf); + keys_serialization->serializeBinary(elem.first, buf, {}); for (size_t col = 0; col < values_types.size(); ++col) serialize(col, elem.second); } @@ -328,12 +328,12 @@ public: { case 0: { - deserialize = [&](size_t col_idx, Array & values){ values_serializations[col_idx]->deserializeBinary(values[col_idx], buf); }; + deserialize = [&](size_t col_idx, Array & values){ values_serializations[col_idx]->deserializeBinary(values[col_idx], buf, {}); }; break; } case 1: { - deserialize = [&](size_t col_idx, Array & values){ promoted_values_serializations[col_idx]->deserializeBinary(values[col_idx], buf); }; + deserialize = [&](size_t col_idx, Array & values){ promoted_values_serializations[col_idx]->deserializeBinary(values[col_idx], buf, {}); }; break; } } @@ -341,7 +341,7 @@ public: for (size_t i = 0; i < size; ++i) { Field key; - keys_serialization->deserializeBinary(key, buf); + keys_serialization->deserializeBinary(key, buf, {}); Array values; values.resize(values_types.size()); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 874a0bd1773..91647a5f165 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -782,6 +782,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) 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) \ 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) \ \ M(Bool, output_format_json_quote_64bit_integers, true, "Controls quoting of 64-bit integers in JSON output format.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 7635e121f8e..ee378b295fa 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -78,6 +78,7 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { + {"22.12", {{"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index d64b41253f5..ea86a91ac88 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -303,17 +303,17 @@ public: */ /// There is two variants for binary serde. First variant work with Field. - virtual void serializeBinary(const Field & field, WriteBuffer & ostr) const = 0; - virtual void deserializeBinary(Field & field, ReadBuffer & istr) const = 0; + virtual void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const = 0; + virtual void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const = 0; /// Other variants takes a column, to avoid creating temporary Field object. /// Column must be non-constant. /// Serialize one value of a column at specified row number. - virtual void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const = 0; + virtual void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0; /// Deserialize one value and insert into a column. /// If method will throw an exception, then column will be in same state as before call to method. - virtual void deserializeBinary(IColumn & column, ReadBuffer & istr) const = 0; + virtual void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0; /** Text serialization with escaping but without quoting. */ diff --git a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp index c8db1a56ed0..7e192595114 100644 --- a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp +++ b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp @@ -17,13 +17,13 @@ namespace DB { -void SerializationAggregateFunction::serializeBinary(const Field & field, WriteBuffer & ostr) const +void SerializationAggregateFunction::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const { const AggregateFunctionStateData & state = field.get(); writeBinary(state.data, ostr); } -void SerializationAggregateFunction::deserializeBinary(Field & field, ReadBuffer & istr) const +void SerializationAggregateFunction::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const { field = AggregateFunctionStateData(); AggregateFunctionStateData & s = field.get(); @@ -31,12 +31,12 @@ void SerializationAggregateFunction::deserializeBinary(Field & field, ReadBuffer s.name = type_name; } -void SerializationAggregateFunction::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const +void SerializationAggregateFunction::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { function->serialize(assert_cast(column).getData()[row_num], ostr, version); } -void SerializationAggregateFunction::deserializeBinary(IColumn & column, ReadBuffer & istr) const +void SerializationAggregateFunction::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { ColumnAggregateFunction & column_concrete = assert_cast(column); diff --git a/src/DataTypes/Serializations/SerializationAggregateFunction.h b/src/DataTypes/Serializations/SerializationAggregateFunction.h index 1e32ce5d6f3..4212298bbc1 100644 --- a/src/DataTypes/Serializations/SerializationAggregateFunction.h +++ b/src/DataTypes/Serializations/SerializationAggregateFunction.h @@ -22,11 +22,11 @@ public: : function(function_), type_name(std::move(type_name_)), version(version_) {} /// NOTE These two functions for serializing single values are incompatible with the functions below. - void serializeBinary(const Field & field, WriteBuffer & ostr) const override; - void deserializeBinary(Field & field, ReadBuffer & istr) const override; + void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const override; - void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; - void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; + void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index 143a3264381..f09589c50c3 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -27,18 +27,18 @@ static constexpr size_t MAX_ARRAY_SIZE = 1ULL << 30; static constexpr size_t MAX_ARRAYS_SIZE = 1ULL << 40; -void SerializationArray::serializeBinary(const Field & field, WriteBuffer & ostr) const +void SerializationArray::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const { const Array & a = field.get(); writeVarUInt(a.size(), ostr); for (size_t i = 0; i < a.size(); ++i) { - nested->serializeBinary(a[i], ostr); + nested->serializeBinary(a[i], ostr, settings); } } -void SerializationArray::deserializeBinary(Field & field, ReadBuffer & istr) const +void SerializationArray::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const { size_t size; readVarUInt(size, istr); @@ -46,11 +46,11 @@ void SerializationArray::deserializeBinary(Field & field, ReadBuffer & istr) con Array & arr = field.get(); arr.reserve(size); for (size_t i = 0; i < size; ++i) - nested->deserializeBinary(arr.emplace_back(), istr); + nested->deserializeBinary(arr.emplace_back(), istr, settings); } -void SerializationArray::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const +void SerializationArray::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { const ColumnArray & column_array = assert_cast(column); const ColumnArray::Offsets & offsets = column_array.getOffsets(); @@ -63,11 +63,11 @@ void SerializationArray::serializeBinary(const IColumn & column, size_t row_num, const IColumn & nested_column = column_array.getData(); for (size_t i = offset; i < next_offset; ++i) - nested->serializeBinary(nested_column, i, ostr); + nested->serializeBinary(nested_column, i, ostr, settings); } -void SerializationArray::deserializeBinary(IColumn & column, ReadBuffer & istr) const +void SerializationArray::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { ColumnArray & column_array = assert_cast(column); ColumnArray::Offsets & offsets = column_array.getOffsets(); @@ -81,7 +81,7 @@ void SerializationArray::deserializeBinary(IColumn & column, ReadBuffer & istr) try { for (; i < size; ++i) - nested->deserializeBinary(nested_column, istr); + nested->deserializeBinary(nested_column, istr, settings); } catch (...) { diff --git a/src/DataTypes/Serializations/SerializationArray.h b/src/DataTypes/Serializations/SerializationArray.h index 860461d667f..a5e10cd22fb 100644 --- a/src/DataTypes/Serializations/SerializationArray.h +++ b/src/DataTypes/Serializations/SerializationArray.h @@ -13,10 +13,10 @@ private: public: explicit SerializationArray(const SerializationPtr & nested_) : nested(nested_) {} - void serializeBinary(const Field & field, WriteBuffer & ostr) const override; - void deserializeBinary(Field & field, ReadBuffer & istr) const override; - void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; - void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; + void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const override; + void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override; diff --git a/src/DataTypes/Serializations/SerializationDecimalBase.cpp b/src/DataTypes/Serializations/SerializationDecimalBase.cpp index 00ffd607664..642ea1c7cd8 100644 --- a/src/DataTypes/Serializations/SerializationDecimalBase.cpp +++ b/src/DataTypes/Serializations/SerializationDecimalBase.cpp @@ -12,14 +12,14 @@ namespace DB { template -void SerializationDecimalBase::serializeBinary(const Field & field, WriteBuffer & ostr) const +void SerializationDecimalBase::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const { FieldType x = field.get>(); writeBinary(x, ostr); } template -void SerializationDecimalBase::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const +void SerializationDecimalBase::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { const FieldType & x = assert_cast(column).getElement(row_num); writeBinary(x, ostr); @@ -39,7 +39,7 @@ void SerializationDecimalBase::serializeBinaryBulk(const IColumn & column, Wr } template -void SerializationDecimalBase::deserializeBinary(Field & field, ReadBuffer & istr) const +void SerializationDecimalBase::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const { typename FieldType::NativeType x; readBinary(x, istr); @@ -47,7 +47,7 @@ void SerializationDecimalBase::deserializeBinary(Field & field, ReadBuffer & } template -void SerializationDecimalBase::deserializeBinary(IColumn & column, ReadBuffer & istr) const +void SerializationDecimalBase::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { typename FieldType::NativeType x; readBinary(x, istr); diff --git a/src/DataTypes/Serializations/SerializationDecimalBase.h b/src/DataTypes/Serializations/SerializationDecimalBase.h index fd3dcb17e35..08f963cedbb 100644 --- a/src/DataTypes/Serializations/SerializationDecimalBase.h +++ b/src/DataTypes/Serializations/SerializationDecimalBase.h @@ -20,12 +20,12 @@ public: SerializationDecimalBase(UInt32 precision_, UInt32 scale_) : precision(precision_), scale(scale_) {} - void serializeBinary(const Field & field, WriteBuffer & ostr) const override; - void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; + void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override; + void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; - void deserializeBinary(Field & field, ReadBuffer & istr) const override; - void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; + void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const override; + void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; }; diff --git a/src/DataTypes/Serializations/SerializationFixedString.cpp b/src/DataTypes/Serializations/SerializationFixedString.cpp index dd476103108..7f9ebe174fa 100644 --- a/src/DataTypes/Serializations/SerializationFixedString.cpp +++ b/src/DataTypes/Serializations/SerializationFixedString.cpp @@ -26,7 +26,7 @@ namespace ErrorCodes static constexpr size_t MAX_STRINGS_SIZE = 1ULL << 30; -void SerializationFixedString::serializeBinary(const Field & field, WriteBuffer & ostr) const +void SerializationFixedString::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const { const String & s = field.get(); ostr.write(s.data(), std::min(s.size(), n)); @@ -36,7 +36,7 @@ void SerializationFixedString::serializeBinary(const Field & field, WriteBuffer } -void SerializationFixedString::deserializeBinary(Field & field, ReadBuffer & istr) const +void SerializationFixedString::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const { field = String(); String & s = field.get(); @@ -45,13 +45,13 @@ void SerializationFixedString::deserializeBinary(Field & field, ReadBuffer & ist } -void SerializationFixedString::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const +void SerializationFixedString::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { ostr.write(reinterpret_cast(&assert_cast(column).getChars()[n * row_num]), n); } -void SerializationFixedString::deserializeBinary(IColumn & column, ReadBuffer & istr) const +void SerializationFixedString::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { ColumnFixedString::Chars & data = assert_cast(column).getChars(); size_t old_size = data.size(); diff --git a/src/DataTypes/Serializations/SerializationFixedString.h b/src/DataTypes/Serializations/SerializationFixedString.h index c3c08b20419..3db31ab02cb 100644 --- a/src/DataTypes/Serializations/SerializationFixedString.h +++ b/src/DataTypes/Serializations/SerializationFixedString.h @@ -15,10 +15,10 @@ public: explicit SerializationFixedString(size_t n_) : n(n_) {} size_t getN() const { return n; } - void serializeBinary(const Field & field, WriteBuffer & ostr) const override; - void deserializeBinary(Field & field, ReadBuffer & istr) const override; - void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; - void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; + void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const override; + void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.cpp b/src/DataTypes/Serializations/SerializationLowCardinality.cpp index c70bb1e1465..b3f91c0297b 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.cpp +++ b/src/DataTypes/Serializations/SerializationLowCardinality.cpp @@ -718,22 +718,22 @@ void SerializationLowCardinality::deserializeBinaryBulkWithMultipleStreams( column = std::move(mutable_column); } -void SerializationLowCardinality::serializeBinary(const Field & field, WriteBuffer & ostr) const +void SerializationLowCardinality::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const { - dictionary_type->getDefaultSerialization()->serializeBinary(field, ostr); + dictionary_type->getDefaultSerialization()->serializeBinary(field, ostr, settings); } -void SerializationLowCardinality::deserializeBinary(Field & field, ReadBuffer & istr) const +void SerializationLowCardinality::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const { - dictionary_type->getDefaultSerialization()->deserializeBinary(field, istr); + dictionary_type->getDefaultSerialization()->deserializeBinary(field, istr, settings); } -void SerializationLowCardinality::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const +void SerializationLowCardinality::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - serializeImpl(column, row_num, &ISerialization::serializeBinary, ostr); + serializeImpl(column, row_num, &ISerialization::serializeBinary, ostr, settings); } -void SerializationLowCardinality::deserializeBinary(IColumn & column, ReadBuffer & istr) const +void SerializationLowCardinality::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - deserializeImpl(column, &ISerialization::deserializeBinary, istr); + deserializeImpl(column, &ISerialization::deserializeBinary, istr, settings); } void SerializationLowCardinality::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.h b/src/DataTypes/Serializations/SerializationLowCardinality.h index 1d0c3226faf..5f56bcf8108 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.h +++ b/src/DataTypes/Serializations/SerializationLowCardinality.h @@ -49,10 +49,10 @@ public: DeserializeBinaryBulkStatePtr & state, SubstreamsCache * cache) const override; - void serializeBinary(const Field & field, WriteBuffer & ostr) const override; - void deserializeBinary(Field & field, ReadBuffer & istr) const override; - void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; - void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; + void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const override; + void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; diff --git a/src/DataTypes/Serializations/SerializationMap.cpp b/src/DataTypes/Serializations/SerializationMap.cpp index cd0a99c0c68..61b04e843b2 100644 --- a/src/DataTypes/Serializations/SerializationMap.cpp +++ b/src/DataTypes/Serializations/SerializationMap.cpp @@ -36,7 +36,7 @@ static IColumn & extractNestedColumn(IColumn & column) return assert_cast(column).getNestedColumn(); } -void SerializationMap::serializeBinary(const Field & field, WriteBuffer & ostr) const +void SerializationMap::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const { const auto & map = field.get(); writeVarUInt(map.size(), ostr); @@ -44,12 +44,12 @@ void SerializationMap::serializeBinary(const Field & field, WriteBuffer & ostr) { const auto & tuple = elem.safeGet(); assert(tuple.size() == 2); - key->serializeBinary(tuple[0], ostr); - value->serializeBinary(tuple[1], ostr); + key->serializeBinary(tuple[0], ostr, settings); + value->serializeBinary(tuple[1], ostr, settings); } } -void SerializationMap::deserializeBinary(Field & field, ReadBuffer & istr) const +void SerializationMap::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const { size_t size; readVarUInt(size, istr); @@ -59,20 +59,20 @@ void SerializationMap::deserializeBinary(Field & field, ReadBuffer & istr) const for (size_t i = 0; i < size; ++i) { Tuple tuple(2); - key->deserializeBinary(tuple[0], istr); - value->deserializeBinary(tuple[1], istr); + key->deserializeBinary(tuple[0], istr, settings); + value->deserializeBinary(tuple[1], istr, settings); map.push_back(std::move(tuple)); } } -void SerializationMap::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const +void SerializationMap::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - nested->serializeBinary(extractNestedColumn(column), row_num, ostr); + nested->serializeBinary(extractNestedColumn(column), row_num, ostr, settings); } -void SerializationMap::deserializeBinary(IColumn & column, ReadBuffer & istr) const +void SerializationMap::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - nested->deserializeBinary(extractNestedColumn(column), istr); + nested->deserializeBinary(extractNestedColumn(column), istr, settings); } diff --git a/src/DataTypes/Serializations/SerializationMap.h b/src/DataTypes/Serializations/SerializationMap.h index 864ac1f3a99..556a50fbbc1 100644 --- a/src/DataTypes/Serializations/SerializationMap.h +++ b/src/DataTypes/Serializations/SerializationMap.h @@ -18,10 +18,10 @@ private: public: SerializationMap(const SerializationPtr & key_type_, const SerializationPtr & value_type_, const SerializationPtr & nested_); - void serializeBinary(const Field & field, WriteBuffer & ostr) const override; - void deserializeBinary(Field & field, ReadBuffer & istr) const override; - void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; - void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; + void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const override; + void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; diff --git a/src/DataTypes/Serializations/SerializationNothing.h b/src/DataTypes/Serializations/SerializationNothing.h index e46a1e6ed30..06ce00eb636 100644 --- a/src/DataTypes/Serializations/SerializationNothing.h +++ b/src/DataTypes/Serializations/SerializationNothing.h @@ -19,10 +19,10 @@ private: throw Exception("Serialization is not implemented for type Nothing", ErrorCodes::NOT_IMPLEMENTED); } public: - void serializeBinary(const Field &, WriteBuffer &) const override { throwNoSerialization(); } - void deserializeBinary(Field &, ReadBuffer &) const override { throwNoSerialization(); } - void serializeBinary(const IColumn &, size_t, WriteBuffer &) const override { throwNoSerialization(); } - void deserializeBinary(IColumn &, ReadBuffer &) const override { throwNoSerialization(); } + void serializeBinary(const Field &, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); } + void deserializeBinary(Field &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); } + void serializeBinary(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); } + void deserializeBinary(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); } void serializeText(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); } void deserializeText(IColumn &, ReadBuffer &, const FormatSettings &, bool) const override { throwNoSerialization(); } diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index c46fde27ddb..8a57c4bc9a1 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -150,7 +150,7 @@ void SerializationNullable::deserializeBinaryBulkWithMultipleStreams( } -void SerializationNullable::serializeBinary(const Field & field, WriteBuffer & ostr) const +void SerializationNullable::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const { if (field.isNull()) { @@ -159,17 +159,17 @@ void SerializationNullable::serializeBinary(const Field & field, WriteBuffer & o else { writeBinary(false, ostr); - nested->serializeBinary(field, ostr); + nested->serializeBinary(field, ostr, settings); } } -void SerializationNullable::deserializeBinary(Field & field, ReadBuffer & istr) const +void SerializationNullable::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const { bool is_null = false; readBinary(is_null, istr); if (!is_null) { - nested->deserializeBinary(field, istr); + nested->deserializeBinary(field, istr, settings); } else { @@ -177,14 +177,14 @@ void SerializationNullable::deserializeBinary(Field & field, ReadBuffer & istr) } } -void SerializationNullable::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const +void SerializationNullable::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { const ColumnNullable & col = assert_cast(column); bool is_null = col.isNullAt(row_num); writeBinary(is_null, ostr); if (!is_null) - nested->serializeBinary(col.getNestedColumn(), row_num, ostr); + nested->serializeBinary(col.getNestedColumn(), row_num, ostr, settings); } /// Deserialize value into ColumnNullable. @@ -235,11 +235,11 @@ static ReturnType safeDeserialize( } -void SerializationNullable::deserializeBinary(IColumn & column, ReadBuffer & istr) const +void SerializationNullable::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { safeDeserialize(column, *nested, [&istr] { bool is_null = false; readBinary(is_null, istr); return is_null; }, - [this, &istr] (IColumn & nested_column) { nested->deserializeBinary(nested_column, istr); }); + [this, &istr, settings] (IColumn & nested_column) { nested->deserializeBinary(nested_column, istr, settings); }); } diff --git a/src/DataTypes/Serializations/SerializationNullable.h b/src/DataTypes/Serializations/SerializationNullable.h index 9aabbe299cc..3ec01b46de5 100644 --- a/src/DataTypes/Serializations/SerializationNullable.h +++ b/src/DataTypes/Serializations/SerializationNullable.h @@ -45,10 +45,10 @@ public: DeserializeBinaryBulkStatePtr & state, SubstreamsCache * cache) const override; - void serializeBinary(const Field & field, WriteBuffer & ostr) const override; - void deserializeBinary(Field & field, ReadBuffer & istr) const override; - void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; - void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; + void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const override; + void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; diff --git a/src/DataTypes/Serializations/SerializationNumber.cpp b/src/DataTypes/Serializations/SerializationNumber.cpp index 10e80d92b30..8cabaec753d 100644 --- a/src/DataTypes/Serializations/SerializationNumber.cpp +++ b/src/DataTypes/Serializations/SerializationNumber.cpp @@ -102,7 +102,7 @@ void SerializationNumber::deserializeTextCSV(IColumn & column, ReadBuffer & i } template -void SerializationNumber::serializeBinary(const Field & field, WriteBuffer & ostr) const +void SerializationNumber::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const { /// ColumnVector::ValueType is a narrower type. For example, UInt8, when the Field type is UInt64 typename ColumnVector::ValueType x = static_cast::ValueType>(field.get()); @@ -110,7 +110,7 @@ void SerializationNumber::serializeBinary(const Field & field, WriteBuffer & } template -void SerializationNumber::deserializeBinary(Field & field, ReadBuffer & istr) const +void SerializationNumber::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const { typename ColumnVector::ValueType x; readBinary(x, istr); @@ -118,13 +118,13 @@ void SerializationNumber::deserializeBinary(Field & field, ReadBuffer & istr) } template -void SerializationNumber::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const +void SerializationNumber::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { writeBinary(assert_cast &>(column).getData()[row_num], ostr); } template -void SerializationNumber::deserializeBinary(IColumn & column, ReadBuffer & istr) const +void SerializationNumber::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { typename ColumnVector::ValueType x; readBinary(x, istr); diff --git a/src/DataTypes/Serializations/SerializationNumber.h b/src/DataTypes/Serializations/SerializationNumber.h index 062453b1e19..67ed91848d4 100644 --- a/src/DataTypes/Serializations/SerializationNumber.h +++ b/src/DataTypes/Serializations/SerializationNumber.h @@ -22,10 +22,10 @@ public: void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; /** Format is platform-dependent. */ - void serializeBinary(const Field & field, WriteBuffer & ostr) const override; - void deserializeBinary(Field & field, ReadBuffer & istr) const override; - void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; - void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; + void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const override; + void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; }; diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 98a94886f67..3e5e1934614 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -376,25 +376,25 @@ void SerializationObject::deserializeBinaryBulkFromTuple( } template -void SerializationObject::serializeBinary(const Field &, WriteBuffer &) const +void SerializationObject::serializeBinary(const Field &, WriteBuffer &, const FormatSettings &) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for SerializationObject"); } template -void SerializationObject::deserializeBinary(Field &, ReadBuffer &) const +void SerializationObject::deserializeBinary(Field &, ReadBuffer &, const FormatSettings &) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for SerializationObject"); } template -void SerializationObject::serializeBinary(const IColumn &, size_t, WriteBuffer &) const +void SerializationObject::serializeBinary(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for SerializationObject"); } template -void SerializationObject::deserializeBinary(IColumn &, ReadBuffer &) const +void SerializationObject::deserializeBinary(IColumn &, ReadBuffer &, const FormatSettings &) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented for SerializationObject"); } diff --git a/src/DataTypes/Serializations/SerializationObject.h b/src/DataTypes/Serializations/SerializationObject.h index 47a7127cd1c..9cf56fcab96 100644 --- a/src/DataTypes/Serializations/SerializationObject.h +++ b/src/DataTypes/Serializations/SerializationObject.h @@ -57,10 +57,10 @@ public: DeserializeBinaryBulkStatePtr & state, SubstreamsCache * cache) const override; - void serializeBinary(const Field & field, WriteBuffer & ostr) const override; - void deserializeBinary(Field & field, ReadBuffer & istr) const override; - void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; - void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; + void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const override; + void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index cd09cd7be5a..4d7514271ad 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -302,23 +302,23 @@ void SerializationSparse::deserializeBinaryBulkWithMultipleStreams( /// All methods below just wrap nested serialization. -void SerializationSparse::serializeBinary(const Field & field, WriteBuffer & ostr) const +void SerializationSparse::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const { - nested->serializeBinary(field, ostr); + nested->serializeBinary(field, ostr, settings); } -void SerializationSparse::deserializeBinary(Field & field, ReadBuffer & istr) const +void SerializationSparse::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const { - nested->deserializeBinary(field, istr); + nested->deserializeBinary(field, istr, settings); } -void SerializationSparse::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const +void SerializationSparse::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { const auto & column_sparse = assert_cast(column); - nested->serializeBinary(column_sparse.getValuesColumn(), column_sparse.getValueIndex(row_num), ostr); + nested->serializeBinary(column_sparse.getValuesColumn(), column_sparse.getValueIndex(row_num), ostr, settings); } -void SerializationSparse::deserializeBinary(IColumn &, ReadBuffer &) const +void SerializationSparse::deserializeBinary(IColumn &, ReadBuffer &, const FormatSettings &) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'deserializeBinary' is not implemented for SerializationSparse"); } diff --git a/src/DataTypes/Serializations/SerializationSparse.h b/src/DataTypes/Serializations/SerializationSparse.h index c157fe7ce98..2d31fba2509 100644 --- a/src/DataTypes/Serializations/SerializationSparse.h +++ b/src/DataTypes/Serializations/SerializationSparse.h @@ -61,11 +61,11 @@ public: DeserializeBinaryBulkStatePtr & state, SubstreamsCache * cache) const override; - void serializeBinary(const Field & field, WriteBuffer & ostr) const override; - void deserializeBinary(Field & field, ReadBuffer & istr) const override; + void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const override; - void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; - void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; + void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index 46acaada72c..c314ed49973 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -25,20 +25,37 @@ namespace DB namespace ErrorCodes { extern const int INCORRECT_DATA; + extern const int TOO_LARGE_STRING_SIZE; } -void SerializationString::serializeBinary(const Field & field, WriteBuffer & ostr) const +void SerializationString::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const { const String & s = field.get(); + if (settings.max_binary_string_size && s.size() > settings.max_binary_string_size) + throw Exception( + ErrorCodes::TOO_LARGE_STRING_SIZE, + "Too large string size: {}. The maximum is: {}. To increase the maximum, use setting " + "format_binary_max_string_size", + s.size(), + settings.max_binary_string_size); + writeVarUInt(s.size(), ostr); writeString(s, ostr); } -void SerializationString::deserializeBinary(Field & field, ReadBuffer & istr) const +void SerializationString::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const { UInt64 size; readVarUInt(size, istr); + if (settings.max_binary_string_size && size > settings.max_binary_string_size) + throw Exception( + ErrorCodes::TOO_LARGE_STRING_SIZE, + "Too large string size: {}. The maximum is: {}. To increase the maximum, use setting " + "format_binary_max_string_size", + size, + settings.max_binary_string_size); + field = String(); String & s = field.get(); s.resize(size); @@ -46,15 +63,23 @@ void SerializationString::deserializeBinary(Field & field, ReadBuffer & istr) co } -void SerializationString::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const +void SerializationString::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { const StringRef & s = assert_cast(column).getDataAt(row_num); + if (settings.max_binary_string_size && s.size > settings.max_binary_string_size) + throw Exception( + ErrorCodes::TOO_LARGE_STRING_SIZE, + "Too large string size: {}. The maximum is: {}. To increase the maximum, use setting " + "format_binary_max_string_size", + s.size, + settings.max_binary_string_size); + writeVarUInt(s.size, ostr); writeString(s, ostr); } -void SerializationString::deserializeBinary(IColumn & column, ReadBuffer & istr) const +void SerializationString::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { ColumnString & column_string = assert_cast(column); ColumnString::Chars & data = column_string.getChars(); @@ -62,6 +87,13 @@ void SerializationString::deserializeBinary(IColumn & column, ReadBuffer & istr) UInt64 size; readVarUInt(size, istr); + if (settings.max_binary_string_size && size > settings.max_binary_string_size) + throw Exception( + ErrorCodes::TOO_LARGE_STRING_SIZE, + "Too large string size: {}. The maximum is: {}. To increase the maximum, use setting " + "format_binary_max_string_size", + size, + settings.max_binary_string_size); size_t old_chars_size = data.size(); size_t offset = old_chars_size + size + 1; diff --git a/src/DataTypes/Serializations/SerializationString.h b/src/DataTypes/Serializations/SerializationString.h index ee5de2c18f1..f27a5116c15 100644 --- a/src/DataTypes/Serializations/SerializationString.h +++ b/src/DataTypes/Serializations/SerializationString.h @@ -8,10 +8,10 @@ namespace DB class SerializationString final : public ISerialization { public: - void serializeBinary(const Field & field, WriteBuffer & ostr) const override; - void deserializeBinary(Field & field, ReadBuffer & istr) const override; - void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; - void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; + void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const override; + void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index 8ffb1fe86bc..0ed2b034985 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -29,17 +29,17 @@ static inline const IColumn & extractElementColumn(const IColumn & column, size_ return assert_cast(column).getColumn(idx); } -void SerializationTuple::serializeBinary(const Field & field, WriteBuffer & ostr) const +void SerializationTuple::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const { const auto & tuple = field.get(); for (size_t element_index = 0; element_index < elems.size(); ++element_index) { const auto & serialization = elems[element_index]; - serialization->serializeBinary(tuple[element_index], ostr); + serialization->serializeBinary(tuple[element_index], ostr, settings); } } -void SerializationTuple::deserializeBinary(Field & field, ReadBuffer & istr) const +void SerializationTuple::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const { const size_t size = elems.size(); @@ -47,15 +47,15 @@ void SerializationTuple::deserializeBinary(Field & field, ReadBuffer & istr) con Tuple & tuple = field.get(); tuple.reserve(size); for (size_t i = 0; i < size; ++i) - elems[i]->deserializeBinary(tuple.emplace_back(), istr); + elems[i]->deserializeBinary(tuple.emplace_back(), istr, settings); } -void SerializationTuple::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const +void SerializationTuple::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { for (size_t element_index = 0; element_index < elems.size(); ++element_index) { const auto & serialization = elems[element_index]; - serialization->serializeBinary(extractElementColumn(column, element_index), row_num, ostr); + serialization->serializeBinary(extractElementColumn(column, element_index), row_num, ostr, settings); } } @@ -97,12 +97,12 @@ static void addElementSafe(size_t num_elems, IColumn & column, F && impl) } } -void SerializationTuple::deserializeBinary(IColumn & column, ReadBuffer & istr) const +void SerializationTuple::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { addElementSafe(elems.size(), column, [&] { for (size_t i = 0; i < elems.size(); ++i) - elems[i]->deserializeBinary(extractElementColumn(column, i), istr); + elems[i]->deserializeBinary(extractElementColumn(column, i), istr, settings); }); } diff --git a/src/DataTypes/Serializations/SerializationTuple.h b/src/DataTypes/Serializations/SerializationTuple.h index db0339bc996..5c177e3f0e8 100644 --- a/src/DataTypes/Serializations/SerializationTuple.h +++ b/src/DataTypes/Serializations/SerializationTuple.h @@ -17,10 +17,10 @@ public: { } - void serializeBinary(const Field & field, WriteBuffer & ostr) const override; - void deserializeBinary(Field & field, ReadBuffer & istr) const override; - void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; - void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; + void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const override; + void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; diff --git a/src/DataTypes/Serializations/SerializationUUID.cpp b/src/DataTypes/Serializations/SerializationUUID.cpp index 1fa9219a643..7b184aeee96 100644 --- a/src/DataTypes/Serializations/SerializationUUID.cpp +++ b/src/DataTypes/Serializations/SerializationUUID.cpp @@ -82,25 +82,25 @@ void SerializationUUID::deserializeTextCSV(IColumn & column, ReadBuffer & istr, } -void SerializationUUID::serializeBinary(const Field & field, WriteBuffer & ostr) const +void SerializationUUID::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const { UUID x = field.get(); writeBinary(x, ostr); } -void SerializationUUID::deserializeBinary(Field & field, ReadBuffer & istr) const +void SerializationUUID::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const { UUID x; readBinary(x, istr); field = NearestFieldType(x); } -void SerializationUUID::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const +void SerializationUUID::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { writeBinary(assert_cast &>(column).getData()[row_num], ostr); } -void SerializationUUID::deserializeBinary(IColumn & column, ReadBuffer & istr) const +void SerializationUUID::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { UUID x; readBinary(x, istr); diff --git a/src/DataTypes/Serializations/SerializationUUID.h b/src/DataTypes/Serializations/SerializationUUID.h index 061e58f4670..da8c15f7279 100644 --- a/src/DataTypes/Serializations/SerializationUUID.h +++ b/src/DataTypes/Serializations/SerializationUUID.h @@ -19,10 +19,10 @@ public: void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; - void serializeBinary(const Field & field, WriteBuffer & ostr) const override; - void deserializeBinary(Field & field, ReadBuffer & istr) const override; - void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; - void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; + void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const override; + void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; }; diff --git a/src/DataTypes/Serializations/SerializationWrapper.cpp b/src/DataTypes/Serializations/SerializationWrapper.cpp index c83de614751..ce598142ab2 100644 --- a/src/DataTypes/Serializations/SerializationWrapper.cpp +++ b/src/DataTypes/Serializations/SerializationWrapper.cpp @@ -66,24 +66,24 @@ void SerializationWrapper::deserializeBinaryBulk(IColumn & column, ReadBuffer & nested_serialization->deserializeBinaryBulk(column, istr, limit, avg_value_size_hint); } -void SerializationWrapper::serializeBinary(const Field & field, WriteBuffer & ostr) const +void SerializationWrapper::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const { - nested_serialization->serializeBinary(field, ostr); + nested_serialization->serializeBinary(field, ostr, settings); } -void SerializationWrapper::deserializeBinary(Field & field, ReadBuffer & istr) const +void SerializationWrapper::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const { - nested_serialization->deserializeBinary(field, istr); + nested_serialization->deserializeBinary(field, istr, settings); } -void SerializationWrapper::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const +void SerializationWrapper::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - nested_serialization->serializeBinary(column, row_num, ostr); + nested_serialization->serializeBinary(column, row_num, ostr, settings); } -void SerializationWrapper::deserializeBinary(IColumn & column, ReadBuffer & istr) const +void SerializationWrapper::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - nested_serialization->deserializeBinary(column, istr); + nested_serialization->deserializeBinary(column, istr, settings); } void SerializationWrapper::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const diff --git a/src/DataTypes/Serializations/SerializationWrapper.h b/src/DataTypes/Serializations/SerializationWrapper.h index 46941f150e1..c141ff5e38d 100644 --- a/src/DataTypes/Serializations/SerializationWrapper.h +++ b/src/DataTypes/Serializations/SerializationWrapper.h @@ -55,11 +55,11 @@ public: void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; - void serializeBinary(const Field & field, WriteBuffer & ostr) const override; - void deserializeBinary(Field & field, ReadBuffer & istr) const override; + void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const override; - void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; - void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; + void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index a773368b231..fe84d780714 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -180,6 +180,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.try_infer_datetimes = settings.input_format_try_infer_datetimes; format_settings.bson.output_string_as_string = settings.output_format_bson_string_as_string; format_settings.bson.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_bson_skip_fields_with_unsupported_types_in_schema_inference; + format_settings.max_binary_string_size = settings.format_binary_max_string_size; /// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context if (format_settings.schema.is_server) diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index c7c9bfc816c..ad2f05a5819 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -79,6 +79,8 @@ struct FormatSettings UInt64 input_allow_errors_num = 0; Float32 input_allow_errors_ratio = 0; + UInt64 max_binary_string_size = 0; + struct { UInt64 row_group_size = 1000000; diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp index 047a55d3f90..a41cf687b39 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp @@ -59,7 +59,7 @@ std::vector BinaryFormatReader::readTypes() bool BinaryFormatReader::readField(IColumn & column, const DataTypePtr & /*type*/, const SerializationPtr & serialization, bool /*is_last_file_column*/, const String & /*column_name*/) { - serialization->deserializeBinary(column, *in); + serialization->deserializeBinary(column, *in, format_settings); return true; } @@ -92,7 +92,7 @@ void BinaryFormatReader::skipField(size_t file_column) if (file_column >= read_data_types.size()) throw Exception(ErrorCodes::CANNOT_SKIP_UNKNOWN_FIELD, "Cannot skip unknown field in RowBinaryWithNames format, because it's type is unknown"); Field field; - read_data_types[file_column]->getDefaultSerialization()->deserializeBinary(field, *in); + read_data_types[file_column]->getDefaultSerialization()->deserializeBinary(field, *in, format_settings); } BinaryWithNamesAndTypesSchemaReader::BinaryWithNamesAndTypesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) diff --git a/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp index 60b722569a2..c9ed8e03449 100644 --- a/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp @@ -10,8 +10,8 @@ namespace DB { -BinaryRowOutputFormat::BinaryRowOutputFormat(WriteBuffer & out_, const Block & header, bool with_names_, bool with_types_, const RowOutputFormatParams & params_) - : IRowOutputFormat(header, out_, params_), with_names(with_names_), with_types(with_types_) +BinaryRowOutputFormat::BinaryRowOutputFormat(WriteBuffer & out_, const Block & header, bool with_names_, bool with_types_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_) + : IRowOutputFormat(header, out_, params_), with_names(with_names_), with_types(with_types_), format_settings(format_settings_) { } @@ -44,7 +44,7 @@ void BinaryRowOutputFormat::writePrefix() void BinaryRowOutputFormat::writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) { - serialization.serializeBinary(column, row_num, out); + serialization.serializeBinary(column, row_num, out, format_settings); } @@ -56,9 +56,9 @@ void registerOutputFormatRowBinary(FormatFactory & factory) WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, - const FormatSettings &) + const FormatSettings & format_settings) { - return std::make_shared(buf, sample, with_names, with_types, params); + return std::make_shared(buf, sample, with_names, with_types, params, format_settings); }); factory.markOutputFormatSupportsParallelFormatting(format_name); }; diff --git a/src/Processors/Formats/Impl/BinaryRowOutputFormat.h b/src/Processors/Formats/Impl/BinaryRowOutputFormat.h index 40894608677..e8198cb6ee0 100644 --- a/src/Processors/Formats/Impl/BinaryRowOutputFormat.h +++ b/src/Processors/Formats/Impl/BinaryRowOutputFormat.h @@ -17,7 +17,7 @@ class WriteBuffer; class BinaryRowOutputFormat final: public IRowOutputFormat { public: - BinaryRowOutputFormat(WriteBuffer & out_, const Block & header, bool with_names_, bool with_types_, const RowOutputFormatParams & params_); + BinaryRowOutputFormat(WriteBuffer & out_, const Block & header, bool with_names_, bool with_types_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_); String getName() const override { return "BinaryRowOutputFormat"; } @@ -29,6 +29,7 @@ private: bool with_names; bool with_types; + const FormatSettings format_settings; }; } diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index 41aa91eef31..7ec1340e339 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -140,7 +140,7 @@ std::vector serializeKeysToRawString( { std::string & serialized_key = result.emplace_back(); WriteBufferFromString wb(serialized_key); - key_column_type->getDefaultSerialization()->serializeBinary(*it, wb); + key_column_type->getDefaultSerialization()->serializeBinary(*it, wb, {}); wb.finalize(); ++it; @@ -165,7 +165,7 @@ std::vector serializeKeysToRawString(const ColumnWithTypeAndName & Field field; keys.column->get(i, field); /// TODO(@vdimir): use serializeBinaryBulk - keys.type->getDefaultSerialization()->serializeBinary(field, wb); + keys.type->getDefaultSerialization()->serializeBinary(field, wb, {}); wb.finalize(); } return result; diff --git a/src/Storages/KVStorageUtils.h b/src/Storages/KVStorageUtils.h index e3216164869..0574539f4c7 100644 --- a/src/Storages/KVStorageUtils.h +++ b/src/Storages/KVStorageUtils.h @@ -30,7 +30,7 @@ void fillColumns(const K & key, const V & value, size_t key_pos, const Block & h for (size_t i = 0; i < header.columns(); ++i) { const auto & serialization = header.getByPosition(i).type->getDefaultSerialization(); - serialization->deserializeBinary(*columns[i], i == key_pos ? key_buffer : value_buffer); + serialization->deserializeBinary(*columns[i], i == key_pos ? key_buffer : value_buffer, {}); } } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 630fbda833e..46249f73ad5 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -86,9 +86,9 @@ void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const Par auto serialization = minmax_column_types[i]->getDefaultSerialization(); Field min_val; - serialization->deserializeBinary(min_val, *file); + serialization->deserializeBinary(min_val, *file, {}); Field max_val; - serialization->deserializeBinary(max_val, *file); + serialization->deserializeBinary(max_val, *file, {}); // NULL_LAST if (min_val.isNull()) @@ -134,8 +134,8 @@ IMergeTreeDataPart::MinMaxIndex::WrittenFiles IMergeTreeDataPart::MinMaxIndex::s auto out = part_storage.writeFile(file_name, DBMS_DEFAULT_BUFFER_SIZE, {}); HashingWriteBuffer out_hashing(*out); - serialization->serializeBinary(hyperrectangle[i].left, out_hashing); - serialization->serializeBinary(hyperrectangle[i].right, out_hashing); + serialization->serializeBinary(hyperrectangle[i].left, out_hashing, {}); + serialization->serializeBinary(hyperrectangle[i].right, out_hashing, {}); out_hashing.next(); out_checksums.files[file_name].file_size = out_hashing.count(); out_checksums.files[file_name].file_hash = out_hashing.getHash(); @@ -755,7 +755,7 @@ void IMergeTreeDataPart::loadIndex() for (size_t i = 0; i < marks_count; ++i) //-V756 for (size_t j = 0; j < key_size; ++j) - key_serializations[j]->deserializeBinary(*loaded_index[j], *index_file); + key_serializations[j]->deserializeBinary(*loaded_index[j], *index_file, {}); for (size_t i = 0; i < key_size; ++i) { diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index d085bb29b20..a887b0ee322 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -244,7 +244,7 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializePrimaryIndex(const Bloc const auto & primary_column = primary_index_block.getByPosition(j); index_columns[j]->insertFrom(*primary_column.column, granule.start_row); primary_column.type->getDefaultSerialization()->serializeBinary( - *primary_column.column, granule.start_row, compress_primary_key ? *index_source_hashing_stream : *index_file_hashing_stream); + *primary_column.column, granule.start_row, compress_primary_key ? *index_source_hashing_stream : *index_file_hashing_stream, {}); } } } @@ -312,7 +312,7 @@ void MergeTreeDataPartWriterOnDisk::fillPrimaryIndexChecksums(MergeTreeData::Dat size_t last_row_number = column.size() - 1; index_columns[j]->insertFrom(column, last_row_number); index_types[j]->getDefaultSerialization()->serializeBinary( - column, last_row_number, compress_primary_key ? *index_source_hashing_stream : *index_file_hashing_stream); + column, last_row_number, compress_primary_key ? *index_source_hashing_stream : *index_file_hashing_stream, {}); } last_block_index_columns.clear(); } diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp index 088029d9e8e..d8765ddb9bc 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp @@ -26,7 +26,7 @@ MergeTreeIndexGranuleHypothesis::MergeTreeIndexGranuleHypothesis(const String & void MergeTreeIndexGranuleHypothesis::serializeBinary(WriteBuffer & ostr) const { const auto & size_type = DataTypePtr(std::make_shared()); - size_type->getDefaultSerialization()->serializeBinary(static_cast(met), ostr); + size_type->getDefaultSerialization()->serializeBinary(static_cast(met), ostr, {}); } void MergeTreeIndexGranuleHypothesis::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) @@ -36,7 +36,7 @@ void MergeTreeIndexGranuleHypothesis::deserializeBinary(ReadBuffer & istr, Merge Field field_met; const auto & size_type = DataTypePtr(std::make_shared()); - size_type->getDefaultSerialization()->deserializeBinary(field_met, istr); + size_type->getDefaultSerialization()->deserializeBinary(field_met, istr, {}); met = field_met.get(); is_empty = false; } diff --git a/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp b/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp index 43e655a4ee5..fc19f819cf1 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp @@ -42,8 +42,8 @@ void MergeTreeIndexGranuleMinMax::serializeBinary(WriteBuffer & ostr) const const DataTypePtr & type = index_sample_block.getByPosition(i).type; auto serialization = type->getDefaultSerialization(); - serialization->serializeBinary(hyperrectangle[i].left, ostr); - serialization->serializeBinary(hyperrectangle[i].right, ostr); + serialization->serializeBinary(hyperrectangle[i].left, ostr, {}); + serialization->serializeBinary(hyperrectangle[i].right, ostr, {}); } } @@ -63,8 +63,8 @@ void MergeTreeIndexGranuleMinMax::deserializeBinary(ReadBuffer & istr, MergeTree case 1: if (!type->isNullable()) { - serialization->deserializeBinary(min_val, istr); - serialization->deserializeBinary(max_val, istr); + serialization->deserializeBinary(min_val, istr, {}); + serialization->deserializeBinary(max_val, istr, {}); } else { @@ -78,8 +78,8 @@ void MergeTreeIndexGranuleMinMax::deserializeBinary(ReadBuffer & istr, MergeTree readBinary(is_null, istr); if (!is_null) { - serialization->deserializeBinary(min_val, istr); - serialization->deserializeBinary(max_val, istr); + serialization->deserializeBinary(min_val, istr, {}); + serialization->deserializeBinary(max_val, istr, {}); } else { @@ -91,8 +91,8 @@ void MergeTreeIndexGranuleMinMax::deserializeBinary(ReadBuffer & istr, MergeTree /// New format with proper Nullable support for values that includes Null values case 2: - serialization->deserializeBinary(min_val, istr); - serialization->deserializeBinary(max_val, istr); + serialization->deserializeBinary(min_val, istr, {}); + serialization->deserializeBinary(max_val, istr, {}); // NULL_LAST if (min_val.isNull()) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 0e15f2c4cb6..a28394e943e 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -56,11 +56,11 @@ void MergeTreeIndexGranuleSet::serializeBinary(WriteBuffer & ostr) const if (max_rows != 0 && size() > max_rows) { - size_serialization->serializeBinary(0, ostr); + size_serialization->serializeBinary(0, ostr, {}); return; } - size_serialization->serializeBinary(size(), ostr); + size_serialization->serializeBinary(size(), ostr, {}); for (size_t i = 0; i < index_sample_block.columns(); ++i) { @@ -90,7 +90,7 @@ void MergeTreeIndexGranuleSet::deserializeBinary(ReadBuffer & istr, MergeTreeInd Field field_rows; const auto & size_type = DataTypePtr(std::make_shared()); - size_type->getDefaultSerialization()->deserializeBinary(field_rows, istr); + size_type->getDefaultSerialization()->deserializeBinary(field_rows, istr, {}); size_t rows_to_read = field_rows.get(); if (rows_to_read == 0) diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 10f5cc95baf..e7fdf1617f0 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -379,7 +379,7 @@ void MergeTreePartition::load(const MergeTreeData & storage, const PartMetadataM auto file = manager->read("partition.dat"); value.resize(partition_key_sample.columns()); for (size_t i = 0; i < partition_key_sample.columns(); ++i) - partition_key_sample.getByPosition(i).type->getDefaultSerialization()->deserializeBinary(value[i], *file); + partition_key_sample.getByPosition(i).type->getDefaultSerialization()->deserializeBinary(value[i], *file, {}); } std::unique_ptr MergeTreePartition::store(const MergeTreeData & storage, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const @@ -399,7 +399,7 @@ std::unique_ptr MergeTreePartition::store(const Block & HashingWriteBuffer out_hashing(*out); for (size_t i = 0; i < value.size(); ++i) { - partition_key_sample.getByPosition(i).type->getDefaultSerialization()->serializeBinary(value[i], out_hashing); + partition_key_sample.getByPosition(i).type->getDefaultSerialization()->serializeBinary(value[i], out_hashing, {}); } out_hashing.next(); diff --git a/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp index c39e70745fd..b1b158a2aa5 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp @@ -46,7 +46,7 @@ void EmbeddedRocksDBSink::consume(Chunk chunk) size_t idx = 0; for (const auto & elem : block) { - elem.type->getDefaultSerialization()->serializeBinary(*elem.column, i, idx == primary_key_pos ? wb_key : wb_value); + elem.type->getDefaultSerialization()->serializeBinary(*elem.column, i, idx == primary_key_pos ? wb_key : wb_value, {}); ++idx; } status = batch.Put(wb_key.str(), wb_value.str()); diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 46ddb650eee..2fcedf550e8 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -259,7 +259,7 @@ void StorageEmbeddedRocksDB::mutate(const MutationCommands & commands, ContextPt { wb_key.restart(); - column_it->type->getDefaultSerialization()->serializeBinary(*column, i, wb_key); + column_it->type->getDefaultSerialization()->serializeBinary(*column, i, wb_key, {}); auto status = batch.Delete(wb_key.str()); if (!status.ok()) throw Exception("RocksDB write error: " + status.ToString(), ErrorCodes::ROCKSDB_ERROR); diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 21be205c0f6..bd255a952dc 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -111,7 +111,7 @@ public: size_t idx = 0; for (const auto & elem : block) { - elem.type->getDefaultSerialization()->serializeBinary(*elem.column, i, idx == primary_key_pos ? wb_key : wb_value); + elem.type->getDefaultSerialization()->serializeBinary(*elem.column, i, idx == primary_key_pos ? wb_key : wb_value, {}); ++idx; } diff --git a/tests/queries/0_stateless/02496_row_binary_large_string_size.reference b/tests/queries/0_stateless/02496_row_binary_large_string_size.reference new file mode 100644 index 00000000000..d86bac9de59 --- /dev/null +++ b/tests/queries/0_stateless/02496_row_binary_large_string_size.reference @@ -0,0 +1 @@ +OK diff --git a/tests/queries/0_stateless/02496_row_binary_large_string_size.sh b/tests/queries/0_stateless/02496_row_binary_large_string_size.sh new file mode 100755 index 00000000000..39f83f6c2b8 --- /dev/null +++ b/tests/queries/0_stateless/02496_row_binary_large_string_size.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +printf '\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff' | $CLICKHOUSE_LOCAL --format_binary_max_string_size=100000 --input-format=RowBinary --structure='s String' -q "select * from table" 2>&1 | grep -q -F "TOO_LARGE_STRING_SIZE" && echo "OK" || echo FAIL"" From 7d6950d397c29f9a0f391746a76ead59f57503d4 Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Fri, 2 Dec 2022 14:50:56 +0100 Subject: [PATCH 86/98] Revert "Temporary files evict fs cache" --- programs/local/LocalServer.cpp | 2 +- programs/server/Server.cpp | 58 +++-- src/Common/filesystemHelpers.cpp | 6 +- src/Common/filesystemHelpers.h | 4 +- .../IO/CachedOnDiskReadBufferFromFile.cpp | 7 +- .../IO/CachedOnDiskWriteBufferFromFile.cpp | 78 ++----- .../IO/CachedOnDiskWriteBufferFromFile.h | 37 +-- src/Disks/IO/FileCachePlaceholder.cpp | 73 ------ src/Disks/IO/FileCachePlaceholder.h | 61 ----- .../Cached/CachedObjectStorage.h | 2 - .../ObjectStorages/DiskObjectStorage.cpp | 8 - src/Disks/ObjectStorages/DiskObjectStorage.h | 1 - src/Disks/TemporaryFileInPath.cpp | 20 -- src/Disks/TemporaryFileInPath.h | 21 -- src/Disks/TemporaryFileOnDisk.cpp | 2 +- src/Disks/TemporaryFileOnDisk.h | 15 +- src/Formats/NativeWriter.cpp | 8 +- src/Formats/NativeWriter.h | 4 +- src/IO/WriteBufferFromTemporaryFile.cpp | 6 +- src/IO/WriteBufferFromTemporaryFile.h | 4 +- src/Interpreters/Cache/FileCache.cpp | 62 +---- src/Interpreters/Cache/FileCache.h | 6 +- src/Interpreters/Cache/FileCacheFactory.cpp | 13 +- src/Interpreters/Cache/FileCacheFactory.h | 1 - src/Interpreters/Cache/FileSegment.cpp | 80 ++----- src/Interpreters/Cache/FileSegment.h | 58 ++--- src/Interpreters/Context.cpp | 104 ++------- src/Interpreters/Context.h | 4 +- src/Interpreters/TemporaryDataOnDisk.cpp | 90 ++----- src/Interpreters/TemporaryDataOnDisk.h | 24 +- .../tests/gtest_lru_file_cache.cpp | 219 ++---------------- src/Storages/MergeTree/MergeTask.h | 4 +- .../System/StorageSystemFilesystemCache.cpp | 9 +- .../test_temporary_data_in_cache/__init__.py | 0 .../config.d/storage_configuration.xml | 39 ---- .../test_temporary_data_in_cache/test.py | 81 ------- tests/integration/test_tmp_policy/test.py | 2 +- 37 files changed, 220 insertions(+), 993 deletions(-) delete mode 100644 src/Disks/IO/FileCachePlaceholder.cpp delete mode 100644 src/Disks/IO/FileCachePlaceholder.h delete mode 100644 src/Disks/TemporaryFileInPath.cpp delete mode 100644 src/Disks/TemporaryFileInPath.h delete mode 100644 tests/integration/test_temporary_data_in_cache/__init__.py delete mode 100644 tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml delete mode 100644 tests/integration/test_temporary_data_in_cache/test.py diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 22c94e01a51..ce7e27026f1 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -207,7 +207,7 @@ void LocalServer::tryInitPath() global_context->setPath(path); - global_context->setTemporaryStoragePath(path + "tmp/", 0); + global_context->setTemporaryStorage(path + "tmp", "", 0); global_context->setFlagsPath(path + "flags"); global_context->setUserFilesPath(""); // user's files are everywhere diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 40f156bc211..e7e359df21a 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -203,6 +203,46 @@ int mainEntryClickHouseServer(int argc, char ** argv) namespace { +void setupTmpPath(Poco::Logger * log, const std::string & path) +try +{ + LOG_DEBUG(log, "Setting up {} to store temporary data in it", path); + + fs::create_directories(path); + + /// Clearing old temporary files. + fs::directory_iterator dir_end; + size_t unknown_files = 0; + for (fs::directory_iterator it(path); it != dir_end; ++it) + { + if (it->is_regular_file() && startsWith(it->path().filename(), "tmp")) + { + LOG_DEBUG(log, "Removing old temporary file {}", it->path().string()); + fs::remove(it->path()); + } + else + { + unknown_files++; + if (unknown_files < 100) + LOG_DEBUG(log, "Found unknown {} {} in temporary path", + it->is_regular_file() ? "file" : (it->is_directory() ? "directory" : "element"), + it->path().string()); + } + } + + if (unknown_files) + LOG_DEBUG(log, "Found {} unknown files in temporary path", unknown_files); +} +catch (...) +{ + DB::tryLogCurrentException( + log, + fmt::format( + "Caught exception while setup temporary path: {}. It is ok to skip this exception as cleaning old temporary files is not " + "necessary", + path)); +} + size_t waitServersToFinish(std::vector & servers, size_t seconds_to_wait) { const size_t sleep_max_ms = 1000 * seconds_to_wait; @@ -997,21 +1037,13 @@ try LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::instance().getTimeZone()); /// Storage with temporary data for processing of heavy queries. - if (auto temporary_policy = config().getString("tmp_policy", ""); !temporary_policy.empty()) - { - size_t max_size = config().getUInt64("max_temporary_data_on_disk_size", 0); - global_context->setTemporaryStoragePolicy(temporary_policy, max_size); - } - else if (auto temporary_cache = config().getString("tmp_cache", ""); !temporary_cache.empty()) - { - size_t max_size = config().getUInt64("max_temporary_data_on_disk_size", 0); - global_context->setTemporaryStorageInCache(temporary_cache, max_size); - } - else { std::string temporary_path = config().getString("tmp_path", path / "tmp/"); + std::string temporary_policy = config().getString("tmp_policy", ""); size_t max_size = config().getUInt64("max_temporary_data_on_disk_size", 0); - global_context->setTemporaryStoragePath(temporary_path, max_size); + const VolumePtr & volume = global_context->setTemporaryStorage(temporary_path, temporary_policy, max_size); + for (const DiskPtr & disk : volume->getDisks()) + setupTmpPath(log, disk->getPath()); } /** Directory with 'flags': files indicating temporary settings for the server set by system administrator. @@ -1410,7 +1442,7 @@ try } catch (...) { - tryLogCurrentException(log, "Caught exception while setting up access control."); + tryLogCurrentException(log); throw; } diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 43f88dd7faa..07a08dc7fbc 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -64,11 +64,11 @@ bool enoughSpaceInDirectory(const std::string & path, size_t data_size) return data_size <= free_space; } -std::unique_ptr createTemporaryFile(const std::string & folder_path) +std::unique_ptr createTemporaryFile(const std::string & path) { ProfileEvents::increment(ProfileEvents::ExternalProcessingFilesTotal); - fs::create_directories(folder_path); - return std::make_unique(folder_path); + fs::create_directories(path); + return std::make_unique(path); } #if !defined(OS_LINUX) diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index 14ee5f54322..0e6e16941bb 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -14,10 +14,10 @@ namespace fs = std::filesystem; namespace DB { -using PocoTemporaryFile = Poco::TemporaryFile; +using TemporaryFile = Poco::TemporaryFile; bool enoughSpaceInDirectory(const std::string & path, size_t data_size); -std::unique_ptr createTemporaryFile(const std::string & folder_path); +std::unique_ptr createTemporaryFile(const std::string & path); // Determine what block device is responsible for specified path diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 0dd40e7f153..5b5d746ab55 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -118,7 +118,10 @@ void CachedOnDiskReadBufferFromFile::initialize(size_t offset, size_t size) } else { - CreateFileSegmentSettings create_settings(is_persistent ? FileSegmentKind::Persistent : FileSegmentKind::Regular); + CreateFileSegmentSettings create_settings{ + .is_persistent = is_persistent + }; + file_segments_holder.emplace(cache->getOrSet(cache_key, offset, size, create_settings)); } @@ -948,7 +951,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() } else { - LOG_TRACE(log, "No space left in cache to reserve {} bytes, will continue without cache download", size); + LOG_TRACE(log, "No space left in cache, will continue without cache download"); file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); } diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index b11edd7e701..994bb743c5f 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -51,42 +51,16 @@ FileSegmentRangeWriter::FileSegmentRangeWriter( { } -bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind) -{ - size_t written_size = tryWrite(data, size, offset, segment_kind, true); - return written_size == size; -} - -size_t FileSegmentRangeWriter::tryWrite(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind, bool strict) -{ - size_t total_written_size = 0; - while (size > 0) - { - size_t written_size = tryWriteImpl(data, size, offset, segment_kind, strict); - chassert(written_size <= size); - if (written_size == 0) - break; - - if (data) - data += written_size; - - size -= written_size; - offset += written_size; - total_written_size += written_size; - } - return total_written_size; -} - -size_t FileSegmentRangeWriter::tryWriteImpl(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind, bool strict) +bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset, bool is_persistent) { if (finalized) - return 0; + return false; auto & file_segments = file_segments_holder.file_segments; if (current_file_segment_it == file_segments.end()) { - current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, segment_kind); + current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, is_persistent); } else { @@ -104,7 +78,7 @@ size_t FileSegmentRangeWriter::tryWriteImpl(const char * data, size_t size, size if (file_segment->range().size() == file_segment->getDownloadedSize()) { completeFileSegment(*file_segment); - current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, segment_kind); + current_file_segment_it = allocateFileSegment(current_file_segment_write_offset, is_persistent); } } @@ -119,26 +93,20 @@ size_t FileSegmentRangeWriter::tryWriteImpl(const char * data, size_t size, size file_segment->completePartAndResetDownloader(); }); - size_t reserved_size = file_segment->tryReserve(size, strict); - if (reserved_size == 0 || (strict && reserved_size != size)) + bool reserved = file_segment->reserve(size); + if (!reserved) { - if (strict) - { - file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); - appendFilesystemCacheLog(*file_segment); - } + file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); + appendFilesystemCacheLog(*file_segment); LOG_DEBUG( &Poco::Logger::get("FileSegmentRangeWriter"), "Unsuccessful space reservation attempt (size: {}, file segment info: {}", size, file_segment->getInfoForLog()); - return 0; + return false; } - /// Shrink to reserved size, because we can't write more than reserved - size = reserved_size; - try { file_segment->write(data, size, offset); @@ -152,17 +120,7 @@ size_t FileSegmentRangeWriter::tryWriteImpl(const char * data, size_t size, size file_segment->completePartAndResetDownloader(); current_file_segment_write_offset += size; - return size; -} - -bool FileSegmentRangeWriter::reserve(size_t size, size_t offset) -{ - return write(nullptr, size, offset, FileSegmentKind::Temporary); -} - -size_t FileSegmentRangeWriter::tryReserve(size_t size, size_t offset) -{ - return tryWrite(nullptr, size, offset, FileSegmentKind::Temporary); + return true; } void FileSegmentRangeWriter::finalize() @@ -171,7 +129,6 @@ void FileSegmentRangeWriter::finalize() return; auto & file_segments = file_segments_holder.file_segments; - if (file_segments.empty() || current_file_segment_it == file_segments.end()) return; @@ -192,7 +149,7 @@ FileSegmentRangeWriter::~FileSegmentRangeWriter() } } -FileSegments::iterator FileSegmentRangeWriter::allocateFileSegment(size_t offset, FileSegmentKind segment_kind) +FileSegments::iterator FileSegmentRangeWriter::allocateFileSegment(size_t offset, bool is_persistent) { /** * Allocate a new file segment starting `offset`. @@ -201,7 +158,10 @@ FileSegments::iterator FileSegmentRangeWriter::allocateFileSegment(size_t offset std::lock_guard cache_lock(cache->mutex); - CreateFileSegmentSettings create_settings(segment_kind); + CreateFileSegmentSettings create_settings + { + .is_persistent = is_persistent, + }; /// We set max_file_segment_size to be downloaded, /// if we have less size to write, file segment will be resized in complete() method. @@ -236,15 +196,12 @@ void FileSegmentRangeWriter::appendFilesystemCacheLog(const FileSegment & file_s } } -void FileSegmentRangeWriter::completeFileSegment(FileSegment & file_segment, std::optional state) +void FileSegmentRangeWriter::completeFileSegment(FileSegment & file_segment) { /// File segment can be detached if space reservation failed. if (file_segment.isDetached()) return; - if (state.has_value()) - file_segment.setDownloadState(*state); - file_segment.completeWithoutState(); appendFilesystemCacheLog(file_segment); } @@ -313,8 +270,7 @@ void CachedOnDiskWriteBufferFromFile::cacheData(char * data, size_t size) try { - auto segment_kind = is_persistent_cache_file ? FileSegmentKind::Persistent : FileSegmentKind::Regular; - if (!cache_writer->write(data, size, current_download_offset, segment_kind)) + if (!cache_writer->write(data, size, current_download_offset, is_persistent_cache_file)) { LOG_INFO(log, "Write-through cache is stopped as cache limit is reached and nothing can be evicted"); return; diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h index 38ec2b46a5d..cec7305ab1b 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h @@ -4,7 +4,6 @@ #include #include #include -#include namespace Poco { @@ -29,44 +28,22 @@ public: FileCache * cache_, const FileSegment::Key & key_, std::shared_ptr cache_log_, const String & query_id_, const String & source_path_); - /* Write a range of file segments. - * Allocate file segment of `max_file_segment_size` and write to it until it is full and then allocate next file segment. - * If it's impossible to allocate new file segment and reserve space to write all data, then returns false. - * - * Note: the data that was written to file segments before the error occurred is not rolled back. - */ - bool write(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind); - - /* Tries to write data to current file segment. - * Size of written data may be less than requested_size, because it may not be enough space. - * - * Returns size of written data. - */ - size_t tryWrite(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind = FileSegmentKind::Regular, bool strict = false); - - /// Same as `write/tryWrite`, but doesn't write anything, just reserves some space in cache - bool reserve(size_t size, size_t offset); - size_t tryReserve(size_t size, size_t offset); + /** + * Write a range of file segments. Allocate file segment of `max_file_segment_size` and write to + * it until it is full and then allocate next file segment. + */ + bool write(const char * data, size_t size, size_t offset, bool is_persistent); void finalize(); - size_t currentOffset() const { return current_file_segment_write_offset; } - ~FileSegmentRangeWriter(); private: - FileSegments::iterator allocateFileSegment(size_t offset, FileSegmentKind segment_kind); + FileSegments::iterator allocateFileSegment(size_t offset, bool is_persistent); void appendFilesystemCacheLog(const FileSegment & file_segment); - void completeFileSegment(FileSegment & file_segment, std::optional state = {}); - - /* Writes data to current file segment as much as possible and returns size of written data, do not allocate new file segments - * In `strict` mode it will write all data or nothing, otherwise it will write as much as possible - * If returned non zero value, then we can try to write again to next file segment. - * If no space is available, returns zero. - */ - size_t tryWriteImpl(const char * data, size_t size, size_t offset, FileSegmentKind segment_kind, bool strict); + void completeFileSegment(FileSegment & file_segment); FileCache * cache; FileSegment::Key key; diff --git a/src/Disks/IO/FileCachePlaceholder.cpp b/src/Disks/IO/FileCachePlaceholder.cpp deleted file mode 100644 index d45d7b2d83d..00000000000 --- a/src/Disks/IO/FileCachePlaceholder.cpp +++ /dev/null @@ -1,73 +0,0 @@ -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NOT_ENOUGH_SPACE; -} - -void ISpacePlaceholder::reserveCapacity(size_t requested_capacity) -{ - chassert(used_space <= capacity); - - size_t remaining_space = capacity - used_space; - LOG_TEST(&Poco::Logger::get("ISpacePlaceholder"), "Reserving {} bytes (used_space: {}, capacity: {})", requested_capacity, used_space, capacity); - - if (requested_capacity <= remaining_space) - return; - - size_t capacity_to_reserve = requested_capacity - remaining_space; - reserveImpl(capacity_to_reserve); - capacity += capacity_to_reserve; -} - -void ISpacePlaceholder::setUsed(size_t size) -{ - LOG_TEST(&Poco::Logger::get("ISpacePlaceholder"), "Using {} bytes ({} already used, {} capacity)", size, used_space, capacity); - - if (used_space + size > capacity) - { - LOG_WARNING(&Poco::Logger::get("ISpacePlaceholder"), "Used space is greater than capacity. It may lead to not enough space error"); - reserveCapacity(size); - } - - used_space = used_space + size; -} - -FileCachePlaceholder::FileCachePlaceholder(FileCache * cache, const String & name) - : key_name(name) - , file_cache(cache) -{ -} - -void FileCachePlaceholder::reserveImpl(size_t requested_size) -{ - /// We create new cache_writer and will try to reserve requested_size in it - String key = fmt::format("{}_{}", key_name, cache_writers.size()); - auto cache_writer = std::make_unique(file_cache, - file_cache->hash(key), - /* cache_log_ */ nullptr, - /* query_id_ */ "", - /* source_path_ */ key); - - size_t current_offset = cache_writer->currentOffset(); - size_t reserved_size = cache_writer->tryReserve(requested_size, current_offset); - if (reserved_size != requested_size) - { - throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, - "Cannot reserve space in file cache " - "({} bytes required, got {} reserved " - "{} / {} bytes used, " - "{} / {} elements used)" - , requested_size, reserved_size - , file_cache->getUsedCacheSize(), file_cache->getTotalMaxSize() - , file_cache->getFileSegmentsNum(), file_cache->getTotalMaxElements()); - } - /// Add to cache_writers only if we successfully reserved space, otherwise free reserved_size back - cache_writers.push_back(std::move(cache_writer)); -} - - -} diff --git a/src/Disks/IO/FileCachePlaceholder.h b/src/Disks/IO/FileCachePlaceholder.h deleted file mode 100644 index 6ddeb85286b..00000000000 --- a/src/Disks/IO/FileCachePlaceholder.h +++ /dev/null @@ -1,61 +0,0 @@ -#pragma once - -#include -#include - -#include -#include - -#include - -namespace fs = std::filesystem; - -namespace DB -{ - - -/* ISpacePlaceholder is a base class for all classes that need to reserve space in some storage. - * You should resrve space with call reserveCapacity() before writing to it. - * After writing you should call setUsed() to let ISpacePlaceholder know how much space was used. - * It can be different because in some cases you don't know exact size of data you will write (because of compression, for example). - * It's better to reserve more space in advance not to overuse space. - */ -class ISpacePlaceholder -{ -public: - /// Reserve space in storage - void reserveCapacity(size_t requested_capacity); - - /// Indicate that some space is used - /// It uses reserved space if it is possible, otherwise it reserves more space - void setUsed(size_t size); - - virtual ~ISpacePlaceholder() = default; - -private: - virtual void reserveImpl(size_t size) = 0; - - size_t capacity = 0; - size_t used_space = 0; -}; - -/* FileCachePlaceholder is a class that reserves space in FileCache. - * Data is written externally, and FileCachePlaceholder is only used to hold space in FileCache. - */ -class FileCachePlaceholder : public ISpacePlaceholder -{ -public: - FileCachePlaceholder(FileCache * cache, const String & name); - - void reserveImpl(size_t requested_size) override; - -private: - std::string key_name; - FileCache * file_cache; - - /// On each reserveImpl() call we create new FileSegmentRangeWriter that would be hold space - /// It's required to easily release already reserved space on unsuccessful attempt - std::vector> cache_writers; -}; - -} diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 119dc25c66b..2d67203be0f 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -113,8 +113,6 @@ public: WriteSettings getAdjustedSettingsFromMetadataFile(const WriteSettings & settings, const std::string & path) const override; - FileCachePtr getCache() const { return cache; } - private: FileCache::Key getCacheKey(const std::string & path) const; diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 4230fb6254a..263a9a9d0e1 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -519,14 +519,6 @@ void DiskObjectStorage::wrapWithCache(FileCachePtr cache, const FileCacheSetting object_storage = std::make_shared(object_storage, cache, cache_settings, layer_name); } -FileCachePtr DiskObjectStorage::getCache() const -{ - const auto * cached_object_storage = typeid_cast(object_storage.get()); - if (!cached_object_storage) - return nullptr; - return cached_object_storage->getCache(); -} - NameSet DiskObjectStorage::getCacheLayersNames() const { NameSet cache_layers; diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index a24acc270c0..00e3cf98142 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -186,7 +186,6 @@ public: /// There can be any number of cache layers: /// DiskObjectStorage(CachedObjectStorage(...CacheObjectStorage(S3ObjectStorage)...)) void wrapWithCache(FileCachePtr cache, const FileCacheSettings & cache_settings, const String & layer_name); - FileCachePtr getCache() const; /// Get structure of object storage this disk works with. Examples: /// DiskObjectStorage(S3ObjectStorage) diff --git a/src/Disks/TemporaryFileInPath.cpp b/src/Disks/TemporaryFileInPath.cpp deleted file mode 100644 index eae7fa66855..00000000000 --- a/src/Disks/TemporaryFileInPath.cpp +++ /dev/null @@ -1,20 +0,0 @@ -#include -#include - -namespace DB -{ - -TemporaryFileInPath::TemporaryFileInPath(const String & folder_path) - : tmp_file(createTemporaryFile(folder_path)) -{ - chassert(tmp_file); -} - -String TemporaryFileInPath::getPath() const -{ - return tmp_file->path(); -} - -TemporaryFileInPath::~TemporaryFileInPath() = default; - -} diff --git a/src/Disks/TemporaryFileInPath.h b/src/Disks/TemporaryFileInPath.h deleted file mode 100644 index 503247e3f89..00000000000 --- a/src/Disks/TemporaryFileInPath.h +++ /dev/null @@ -1,21 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -/// Wrapper around Poco::TemporaryFile to implement ITemporaryFile. -class TemporaryFileInPath : public ITemporaryFile -{ -public: - explicit TemporaryFileInPath(const String & folder_path); - String getPath() const override; - - ~TemporaryFileInPath() override; -private: - std::unique_ptr tmp_file; -}; - -} diff --git a/src/Disks/TemporaryFileOnDisk.cpp b/src/Disks/TemporaryFileOnDisk.cpp index af1f3f87c71..4f348519037 100644 --- a/src/Disks/TemporaryFileOnDisk.cpp +++ b/src/Disks/TemporaryFileOnDisk.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include @@ -16,6 +15,7 @@ namespace CurrentMetrics extern const Metric TotalTemporaryFiles; } + namespace DB { diff --git a/src/Disks/TemporaryFileOnDisk.h b/src/Disks/TemporaryFileOnDisk.h index fa3a0383297..9ba59c3eaf0 100644 --- a/src/Disks/TemporaryFileOnDisk.h +++ b/src/Disks/TemporaryFileOnDisk.h @@ -9,30 +9,21 @@ namespace DB { using DiskPtr = std::shared_ptr; -class ITemporaryFile -{ -public: - virtual String getPath() const = 0; - virtual ~ITemporaryFile() = default; -}; - -using TemporaryFileHolder = std::unique_ptr; - /// This class helps with the handling of temporary files or directories. /// A unique name for the temporary file or directory is automatically chosen based on a specified prefix. /// Create a directory in the constructor. /// The destructor always removes the temporary file or directory with all contained files. -class TemporaryFileOnDisk : public ITemporaryFile +class TemporaryFileOnDisk { public: explicit TemporaryFileOnDisk(const DiskPtr & disk_); explicit TemporaryFileOnDisk(const DiskPtr & disk_, CurrentMetrics::Value metric_scope); explicit TemporaryFileOnDisk(const DiskPtr & disk_, const String & prefix); - ~TemporaryFileOnDisk() override; + ~TemporaryFileOnDisk(); DiskPtr getDisk() const { return disk; } - String getPath() const override; + String getPath() const; private: DiskPtr disk; diff --git a/src/Formats/NativeWriter.cpp b/src/Formats/NativeWriter.cpp index e932bb88c2d..c4dea371afd 100644 --- a/src/Formats/NativeWriter.cpp +++ b/src/Formats/NativeWriter.cpp @@ -64,10 +64,8 @@ static void writeData(const ISerialization & serialization, const ColumnPtr & co } -size_t NativeWriter::write(const Block & block) +void NativeWriter::write(const Block & block) { - size_t written_before = ostr.count(); - /// Additional information about the block. if (client_revision > 0) block.info.write(ostr); @@ -163,10 +161,6 @@ size_t NativeWriter::write(const Block & block) if (index) index->blocks.emplace_back(std::move(index_block)); - - size_t written_after = ostr.count(); - size_t written_size = written_after - written_before; - return written_size; } } diff --git a/src/Formats/NativeWriter.h b/src/Formats/NativeWriter.h index 7bb377d2e4a..010a03ec722 100644 --- a/src/Formats/NativeWriter.h +++ b/src/Formats/NativeWriter.h @@ -27,9 +27,7 @@ public: IndexForNativeFormat * index_ = nullptr, size_t initial_size_of_file_ = 0); Block getHeader() const { return header; } - - /// Returns the number of bytes written. - size_t write(const Block & block); + void write(const Block & block); void flush(); static String getContentType() { return "application/octet-stream"; } diff --git a/src/IO/WriteBufferFromTemporaryFile.cpp b/src/IO/WriteBufferFromTemporaryFile.cpp index 4562ad512b3..f93c79ca587 100644 --- a/src/IO/WriteBufferFromTemporaryFile.cpp +++ b/src/IO/WriteBufferFromTemporaryFile.cpp @@ -13,7 +13,7 @@ namespace ErrorCodes } -WriteBufferFromTemporaryFile::WriteBufferFromTemporaryFile(std::unique_ptr && tmp_file_) +WriteBufferFromTemporaryFile::WriteBufferFromTemporaryFile(std::unique_ptr && tmp_file_) : WriteBufferFromFile(tmp_file_->path(), DBMS_DEFAULT_BUFFER_SIZE, O_RDWR | O_TRUNC | O_CREAT, 0600), tmp_file(std::move(tmp_file_)) {} @@ -40,11 +40,11 @@ public: return std::make_shared(fd, file_name, std::move(origin->tmp_file)); } - ReadBufferFromTemporaryWriteBuffer(int fd_, const std::string & file_name_, std::unique_ptr && tmp_file_) + ReadBufferFromTemporaryWriteBuffer(int fd_, const std::string & file_name_, std::unique_ptr && tmp_file_) : ReadBufferFromFile(fd_, file_name_), tmp_file(std::move(tmp_file_)) {} - std::unique_ptr tmp_file; + std::unique_ptr tmp_file; }; diff --git a/src/IO/WriteBufferFromTemporaryFile.h b/src/IO/WriteBufferFromTemporaryFile.h index a4e83b95ac6..06e2911db26 100644 --- a/src/IO/WriteBufferFromTemporaryFile.h +++ b/src/IO/WriteBufferFromTemporaryFile.h @@ -20,11 +20,11 @@ public: ~WriteBufferFromTemporaryFile() override; private: - explicit WriteBufferFromTemporaryFile(std::unique_ptr && tmp_file); + explicit WriteBufferFromTemporaryFile(std::unique_ptr && tmp_file); std::shared_ptr getReadBufferImpl() override; - std::unique_ptr tmp_file; + std::unique_ptr tmp_file; friend class ReadBufferFromTemporaryWriteBuffer; }; diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index e2a0e839f19..72fa1b3c324 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -2,7 +2,6 @@ #include #include -#include #include #include #include @@ -13,7 +12,6 @@ #include #include - namespace fs = std::filesystem; namespace DB @@ -48,27 +46,13 @@ FileCache::Key FileCache::hash(const String & path) return Key(sipHash128(path.data(), path.size())); } -String FileCache::getPathInLocalCache(const Key & key, size_t offset, FileSegmentKind segment_kind) const +String FileCache::getPathInLocalCache(const Key & key, size_t offset, bool is_persistent) const { - String file_suffix; - switch (segment_kind) - { - case FileSegmentKind::Persistent: - file_suffix = "_persistent"; - break; - case FileSegmentKind::Temporary: - file_suffix = "_temporary"; - break; - case FileSegmentKind::Regular: - file_suffix = ""; - break; - } - auto key_str = key.toString(); return fs::path(cache_base_path) / key_str.substr(0, 3) / key_str - / (std::to_string(offset) + file_suffix); + / (std::to_string(offset) + (is_persistent ? "_persistent" : "")); } String FileCache::getPathInLocalCache(const Key & key) const @@ -556,6 +540,9 @@ FileSegmentPtr FileCache::createFileSegmentForDownload( assertCacheCorrectness(key, cache_lock); #endif + if (size > max_file_segment_size) + throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Requested size exceeds max file segment size"); + auto * cell = getCell(key, offset, cache_lock); if (cell) throw Exception( @@ -1012,17 +999,9 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_lock fs::directory_iterator key_it{key_prefix_it->path()}; for (; key_it != fs::directory_iterator(); ++key_it) { - if (key_it->is_regular_file()) + if (!key_it->is_directory()) { - if (key_prefix_it->path().filename() == "tmp" && startsWith(key_it->path().filename(), "tmp")) - { - LOG_DEBUG(log, "Found temporary file '{}', will remove it", key_it->path().string()); - fs::remove(key_it->path()); - } - else - { - LOG_DEBUG(log, "Unexpected file: {}. Expected a directory", key_it->path().string()); - } + LOG_DEBUG(log, "Unexpected file: {}. Expected a directory", key_it->path().string()); continue; } @@ -1030,26 +1009,17 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_lock fs::directory_iterator offset_it{key_it->path()}; for (; offset_it != fs::directory_iterator(); ++offset_it) { - if (offset_it->is_directory()) - { - LOG_DEBUG(log, "Unexpected directory: {}. Expected a file", offset_it->path().string()); - continue; - } - auto offset_with_suffix = offset_it->path().filename().string(); auto delim_pos = offset_with_suffix.find('_'); bool parsed; - FileSegmentKind segment_kind = FileSegmentKind::Regular; + bool is_persistent = false; if (delim_pos == std::string::npos) parsed = tryParse(offset, offset_with_suffix); else { parsed = tryParse(offset, offset_with_suffix.substr(0, delim_pos)); - if (offset_with_suffix.substr(delim_pos+1) == "persistent") - segment_kind = FileSegmentKind::Persistent; - if (offset_with_suffix.substr(delim_pos+1) == "temporary") - segment_kind = FileSegmentKind::Temporary; + is_persistent = offset_with_suffix.substr(delim_pos+1) == "persistent"; } if (!parsed) @@ -1069,7 +1039,7 @@ void FileCache::loadCacheInfoIntoMemory(std::lock_guard & cache_lock { auto * cell = addCell( key, offset, size, FileSegment::State::DOWNLOADED, - CreateFileSegmentSettings(segment_kind), cache_lock); + CreateFileSegmentSettings{ .is_persistent = is_persistent }, cache_lock); if (cell) queue_entries.emplace_back(cell->queue_iterator, cell->file_segment); @@ -1181,7 +1151,7 @@ std::vector FileCache::tryGetCachePaths(const Key & key) for (const auto & [offset, cell] : cells_by_offset) { if (cell.file_segment->state() == FileSegment::State::DOWNLOADED) - cache_paths.push_back(getPathInLocalCache(key, offset, cell.file_segment->getKind())); + cache_paths.push_back(getPathInLocalCache(key, offset, cell.file_segment->isPersistent())); } return cache_paths; @@ -1203,16 +1173,6 @@ size_t FileCache::getAvailableCacheSizeUnlocked(std::lock_guard & ca return max_size - getUsedCacheSizeUnlocked(cache_lock); } -size_t FileCache::getTotalMaxSize() const -{ - return max_size; -} - -size_t FileCache::getTotalMaxElements() const -{ - return max_element_size; -} - size_t FileCache::getFileSegmentsNum() const { std::lock_guard cache_lock(mutex); diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 4a2610fd76b..706762b6915 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -80,7 +80,7 @@ public: static Key hash(const String & path); - String getPathInLocalCache(const Key & key, size_t offset, FileSegmentKind segment_kind) const; + String getPathInLocalCache(const Key & key, size_t offset, bool is_persistent) const; String getPathInLocalCache(const Key & key) const; @@ -89,10 +89,8 @@ public: size_t capacity() const { return max_size; } size_t getUsedCacheSize() const; - size_t getTotalMaxSize() const; size_t getFileSegmentsNum() const; - size_t getTotalMaxElements() const; static bool isReadOnly(); @@ -223,8 +221,6 @@ private: FileSegmentCell * getCell(const Key & key, size_t offset, std::lock_guard & cache_lock); - /// Returns non-owened pointer to the cell stored in the `files` map. - /// Doesn't reserve any space. FileSegmentCell * addCell( const Key & key, size_t offset, diff --git a/src/Interpreters/Cache/FileCacheFactory.cpp b/src/Interpreters/Cache/FileCacheFactory.cpp index e120fe3fc27..b276760c0dd 100644 --- a/src/Interpreters/Cache/FileCacheFactory.cpp +++ b/src/Interpreters/Cache/FileCacheFactory.cpp @@ -31,21 +31,14 @@ const FileCacheSettings & FileCacheFactory::getSettings(const std::string & cach } -FileCachePtr FileCacheFactory::tryGet(const std::string & cache_base_path) +FileCachePtr FileCacheFactory::get(const std::string & cache_base_path) { std::lock_guard lock(mutex); auto it = caches_by_path.find(cache_base_path); if (it == caches_by_path.end()) - return nullptr; - return it->second->cache; -} - -FileCachePtr FileCacheFactory::get(const std::string & cache_base_path) -{ - auto file_cache_ptr = tryGet(cache_base_path); - if (!file_cache_ptr) throw Exception(ErrorCodes::BAD_ARGUMENTS, "No cache found by path: {}", cache_base_path); - return file_cache_ptr; + return it->second->cache; + } FileCachePtr FileCacheFactory::getOrCreate( diff --git a/src/Interpreters/Cache/FileCacheFactory.h b/src/Interpreters/Cache/FileCacheFactory.h index 32ecd05f019..82e0ec8f928 100644 --- a/src/Interpreters/Cache/FileCacheFactory.h +++ b/src/Interpreters/Cache/FileCacheFactory.h @@ -33,7 +33,6 @@ public: FileCachePtr getOrCreate(const std::string & cache_base_path, const FileCacheSettings & file_cache_settings, const std::string & name); - FileCachePtr tryGet(const std::string & cache_base_path); FileCachePtr get(const std::string & cache_base_path); CacheByBasePath getAll(); diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index e070317e454..418bcee05d9 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -23,19 +23,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -String toString(FileSegmentKind type) -{ - switch (type) - { - case FileSegmentKind::Regular: - return "Regular"; - case FileSegmentKind::Persistent: - return "Persistent"; - case FileSegmentKind::Temporary: - return "Temporary"; - } -} - FileSegment::FileSegment( size_t offset_, size_t size_, @@ -52,7 +39,7 @@ FileSegment::FileSegment( #else , log(&Poco::Logger::get("FileSegment")) #endif - , segment_kind(settings.type) + , is_persistent(settings.is_persistent) { /// On creation, file segment state can be EMPTY, DOWNLOADED, DOWNLOADING. switch (download_state) @@ -86,8 +73,7 @@ FileSegment::FileSegment( String FileSegment::getPathInLocalCache() const { - chassert(cache); - return cache->getPathInLocalCache(key(), offset(), segment_kind); + return cache->getPathInLocalCache(key(), offset(), isPersistent()); } FileSegment::State FileSegment::state() const @@ -323,7 +309,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset) if (current_downloaded_size == range().size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "File segment is already fully downloaded"); - if (!cache_writer && from != nullptr) + if (!cache_writer) { if (current_downloaded_size > 0) throw Exception( @@ -338,14 +324,11 @@ void FileSegment::write(const char * from, size_t size, size_t offset) try { - /// if `from` is nullptr, then we just allocate and hold space by current segment and it was (or would) be written outside - if (cache_writer && from != nullptr) - cache_writer->write(from, size); + cache_writer->write(from, size); std::unique_lock download_lock(download_mutex); - if (cache_writer && from != nullptr) - cache_writer->next(); + cache_writer->next(); downloaded_size += size; } @@ -396,13 +379,6 @@ FileSegment::State FileSegment::wait() } bool FileSegment::reserve(size_t size_to_reserve) -{ - size_t reserved = tryReserve(size_to_reserve, true); - assert(reserved == 0 || reserved == size_to_reserve); - return reserved == size_to_reserve; -} - -size_t FileSegment::tryReserve(size_t size_to_reserve, bool strict) { if (!size_to_reserve) throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Zero space reservation is not allowed"); @@ -418,16 +394,10 @@ size_t FileSegment::tryReserve(size_t size_to_reserve, bool strict) expected_downloaded_size = getDownloadedSizeUnlocked(segment_lock); if (expected_downloaded_size + size_to_reserve > range().size()) - { - if (strict) - { - throw Exception( - ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, - "Attempt to reserve space too much space ({}) for file segment with range: {} (downloaded size: {})", - size_to_reserve, range().toString(), downloaded_size); - } - size_to_reserve = range().size() - expected_downloaded_size; - } + throw Exception( + ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, + "Attempt to reserve space too much space ({}) for file segment with range: {} (downloaded size: {})", + size_to_reserve, range().toString(), downloaded_size); chassert(reserved_size >= expected_downloaded_size); } @@ -445,16 +415,17 @@ size_t FileSegment::tryReserve(size_t size_to_reserve, bool strict) { std::lock_guard cache_lock(cache->mutex); - size_t need_to_reserve = size_to_reserve - already_reserved_size; - reserved = cache->tryReserve(key(), offset(), need_to_reserve, cache_lock); + size_to_reserve = size_to_reserve - already_reserved_size; + reserved = cache->tryReserve(key(), offset(), size_to_reserve, cache_lock); - if (!reserved) - return 0; - - std::lock_guard segment_lock(mutex); - reserved_size += need_to_reserve; + if (reserved) + { + std::lock_guard segment_lock(mutex); + reserved_size += size_to_reserve; + } } - return size_to_reserve; + + return reserved; } void FileSegment::setDownloadedUnlocked([[maybe_unused]] std::unique_lock & segment_lock) @@ -574,15 +545,6 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard & cach resetDownloaderUnlocked(segment_lock); } - if (segment_kind == FileSegmentKind::Temporary && is_last_holder) - { - LOG_TEST(log, "Removing temporary file segment: {}", getInfoForLogUnlocked(segment_lock)); - detach(cache_lock, segment_lock); - setDownloadState(State::SKIP_CACHE); - cache->remove(key(), offset(), cache_lock, segment_lock); - return; - } - switch (download_state) { case State::SKIP_CACHE: @@ -664,7 +626,7 @@ String FileSegment::getInfoForLogUnlocked(std::unique_lock & segment info << "first non-downloaded offset: " << getFirstNonDownloadedOffsetUnlocked(segment_lock) << ", "; info << "caller id: " << getCallerId() << ", "; info << "detached: " << is_detached << ", "; - info << "kind: " << toString(segment_kind); + info << "persistent: " << is_persistent; return info.str(); } @@ -759,7 +721,7 @@ FileSegmentPtr FileSegment::getSnapshot(const FileSegmentPtr & file_segment, std snapshot->ref_count = file_segment.use_count(); snapshot->downloaded_size = file_segment->getDownloadedSizeUnlocked(segment_lock); snapshot->download_state = file_segment->download_state; - snapshot->segment_kind = file_segment->getKind(); + snapshot->is_persistent = file_segment->isPersistent(); return snapshot; } @@ -821,8 +783,6 @@ FileSegmentsHolder::~FileSegmentsHolder() if (!cache) cache = file_segment->cache; - assert(cache == file_segment->cache); /// all segments should belong to the same cache - try { bool is_detached = false; diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 8915b2f0a36..8f9c0097d77 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -30,38 +30,9 @@ using FileSegmentPtr = std::shared_ptr; using FileSegments = std::list; -/* - * FileSegmentKind is used to specify the eviction policy for file segments. - */ -enum class FileSegmentKind -{ - /* `Regular` file segment is still in cache after usage, and can be evicted - * (unless there're some holders). - */ - Regular, - - /* `Persistent` file segment can't be evicted from cache, - * it should be removed manually. - */ - Persistent, - - /* `Temporary` file segment is removed right after relesing. - * Also corresponding files are removed during cache loading (if any). - */ - Temporary, -}; - -String toString(FileSegmentKind type); - struct CreateFileSegmentSettings { - FileSegmentKind type = FileSegmentKind::Regular; - - CreateFileSegmentSettings() = default; - - explicit CreateFileSegmentSettings(FileSegmentKind type_) - : type(type_) - {} + bool is_persistent = false; }; class FileSegment : private boost::noncopyable, public std::enable_shared_from_this @@ -156,8 +127,7 @@ public: size_t offset() const { return range().left; } - FileSegmentKind getKind() const { return segment_kind; } - bool isPersistent() const { return segment_kind == FileSegmentKind::Persistent; } + bool isPersistent() const { return is_persistent; } using UniqueId = std::pair; UniqueId getUniqueId() const { return std::pair(key(), offset()); } @@ -213,19 +183,19 @@ public: void assertCorrectness() const; + /** + * ========== Methods for _only_ file segment's `writer` ====================== + */ + + void synchronousWrite(const char * from, size_t size, size_t offset); + /** * ========== Methods for _only_ file segment's `downloader` ================== */ /// Try to reserve exactly `size` bytes. - /// Returns true if reservation was successful, false otherwise. bool reserve(size_t size_to_reserve); - /// Try to reserve at max `size` bytes. - /// Returns actual size reserved. - /// In strict mode throws an error on attempt to reserve space too much space - size_t tryReserve(size_t size_to_reserve, bool strict = false); - /// Write data into reserved space. void write(const char * from, size_t size, size_t offset); @@ -277,9 +247,9 @@ private: void assertIsDownloaderUnlocked(const std::string & operation, std::unique_lock & segment_lock) const; void assertCorrectnessUnlocked(std::unique_lock & segment_lock) const; - /// completeWithoutStateUnlocked() is called from destructor of FileSegmentsHolder. - /// Function might check if the caller of the method - /// is the last alive holder of the segment. Therefore, completion and destruction + /// complete() without any completion state is called from destructor of + /// FileSegmentsHolder. complete() might check if the caller of the method + /// is the last alive holder of the segment. Therefore, complete() and destruction /// of the file segment pointer must be done under the same cache mutex. void completeWithoutStateUnlocked(std::lock_guard & cache_lock); void completeBasedOnCurrentState(std::lock_guard & cache_lock, std::unique_lock & segment_lock); @@ -325,12 +295,12 @@ private: /// In general case, all file segments are owned by cache. bool is_detached = false; - bool is_downloaded = false; + bool is_downloaded{false}; std::atomic hits_count = 0; /// cache hits. std::atomic ref_count = 0; /// Used for getting snapshot state - FileSegmentKind segment_kind; + bool is_persistent; CurrentMetrics::Increment metric_increment{CurrentMetrics::CacheFileSegments}; }; @@ -343,8 +313,6 @@ struct FileSegmentsHolder : private boost::noncopyable FileSegmentsHolder(FileSegmentsHolder && other) noexcept : file_segments(std::move(other.file_segments)) {} - void reset() { file_segments.clear(); } - ~FileSegmentsHolder(); String toString(); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 796b93998a9..913b0535358 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -32,7 +32,6 @@ #include #include #include -#include #include #include #include @@ -103,7 +102,6 @@ #include #include #include -#include #include #include @@ -748,65 +746,28 @@ void Context::setPath(const String & path) shared->user_scripts_path = shared->path + "user_scripts/"; } -static void setupTmpPath(Poco::Logger * log, const std::string & path) -try -{ - LOG_DEBUG(log, "Setting up {} to store temporary data in it", path); - - fs::create_directories(path); - - /// Clearing old temporary files. - fs::directory_iterator dir_end; - for (fs::directory_iterator it(path); it != dir_end; ++it) - { - if (it->is_regular_file() && startsWith(it->path().filename(), "tmp")) - { - LOG_DEBUG(log, "Removing old temporary file {}", it->path().string()); - fs::remove(it->path()); - } - else - LOG_DEBUG(log, "Found unknown file in temporary path {}", it->path().string()); - } -} -catch (...) -{ - DB::tryLogCurrentException(log, fmt::format( - "Caught exception while setup temporary path: {}. " - "It is ok to skip this exception as cleaning old temporary files is not necessary", path)); -} - -static VolumePtr createLocalSingleDiskVolume(const std::string & path) -{ - auto disk = std::make_shared("_tmp_default", path, 0); - VolumePtr volume = std::make_shared("_tmp_default", disk, 0); - return volume; -} - -void Context::setTemporaryStoragePath(const String & path, size_t max_size) -{ - shared->tmp_path = path; - if (!shared->tmp_path.ends_with('/')) - shared->tmp_path += '/'; - - VolumePtr volume = createLocalSingleDiskVolume(shared->tmp_path); - - for (const auto & disk : volume->getDisks()) - { - setupTmpPath(shared->log, disk->getPath()); - } - - shared->temp_data_on_disk = std::make_shared(volume, nullptr, max_size); -} - -void Context::setTemporaryStoragePolicy(const String & policy_name, size_t max_size) +VolumePtr Context::setTemporaryStorage(const String & path, const String & policy_name, size_t max_size) { std::lock_guard lock(shared->storage_policies_mutex); + VolumePtr volume; - StoragePolicyPtr tmp_policy = getStoragePolicySelector(lock)->get(policy_name); - if (tmp_policy->getVolumes().size() != 1) - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, - "Policy '{}' is used temporary files, such policy should have exactly one volume", policy_name); - VolumePtr volume = tmp_policy->getVolume(0); + if (policy_name.empty()) + { + shared->tmp_path = path; + if (!shared->tmp_path.ends_with('/')) + shared->tmp_path += '/'; + + auto disk = std::make_shared("_tmp_default", shared->tmp_path, 0); + volume = std::make_shared("_tmp_default", disk, 0); + } + else + { + StoragePolicyPtr tmp_policy = getStoragePolicySelector(lock)->get(policy_name); + if (tmp_policy->getVolumes().size() != 1) + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, + "Policy '{}' is used temporary files, such policy should have exactly one volume", policy_name); + volume = tmp_policy->getVolume(0); + } if (volume->getDisks().empty()) throw Exception("No disks volume for temporary files", ErrorCodes::NO_ELEMENTS_IN_CONFIG); @@ -828,33 +789,10 @@ void Context::setTemporaryStoragePolicy(const String & policy_name, size_t max_s "Disk '{}' ({}) is not local and can't be used for temporary files", disk_ptr->getName(), typeid(*disk_raw_ptr).name()); } - - setupTmpPath(shared->log, disk->getPath()); } - shared->temp_data_on_disk = std::make_shared(volume, nullptr, max_size); -} - - -void Context::setTemporaryStorageInCache(const String & cache_disk_name, size_t max_size) -{ - auto disk_ptr = getDisk(cache_disk_name); - if (!disk_ptr) - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Disk '{}' is not found", cache_disk_name); - - const auto * disk_object_storage_ptr = dynamic_cast(disk_ptr.get()); - if (!disk_object_storage_ptr) - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Disk '{}' does not use cache", cache_disk_name); - - auto file_cache = disk_object_storage_ptr->getCache(); - if (!file_cache) - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Cache '{}' is not found", file_cache->getBasePath()); - - LOG_DEBUG(shared->log, "Using file cache ({}) for temporary files", file_cache->getBasePath()); - - shared->tmp_path = file_cache->getBasePath(); - VolumePtr volume = createLocalSingleDiskVolume(shared->tmp_path); - shared->temp_data_on_disk = std::make_shared(volume, file_cache.get(), max_size); + shared->temp_data_on_disk = std::make_shared(volume, max_size); + return volume; } void Context::setFlagsPath(const String & path) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 13bbf8a7ea0..63f321db993 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -463,9 +463,7 @@ public: void addWarningMessage(const String & msg) const; - void setTemporaryStorageInCache(const String & cache_disk_name, size_t max_size); - void setTemporaryStoragePolicy(const String & policy_name, size_t max_size); - void setTemporaryStoragePath(const String & path, size_t max_size); + VolumePtr setTemporaryStorage(const String & path, const String & policy_name, size_t max_size); using ConfigurationPtr = Poco::AutoPtr; diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index a039053b011..c5ae6f6c885 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include @@ -36,32 +35,35 @@ void TemporaryDataOnDiskScope::deltaAllocAndCheck(ssize_t compressed_delta, ssiz size_t new_consumprion = stat.compressed_size + compressed_delta; if (compressed_delta > 0 && limit && new_consumprion > limit) - throw Exception(ErrorCodes::TOO_MANY_ROWS_OR_BYTES, - "Limit for temporary files size exceeded (would consume {} / {} bytes)", new_consumprion, limit); + throw Exception(ErrorCodes::TOO_MANY_ROWS_OR_BYTES, "Limit for temporary files size exceeded"); stat.compressed_size += compressed_delta; stat.uncompressed_size += uncompressed_delta; } -VolumePtr TemporaryDataOnDiskScope::getVolume() const -{ - if (!volume) - throw Exception("TemporaryDataOnDiskScope has no volume", ErrorCodes::LOGICAL_ERROR); - return volume; -} - TemporaryFileStream & TemporaryDataOnDisk::createStream(const Block & header, size_t max_file_size) { - TemporaryFileStreamPtr tmp_stream; - if (cache) - tmp_stream = TemporaryFileStream::create(cache, header, max_file_size, this); + DiskPtr disk; + if (max_file_size > 0) + { + auto reservation = volume->reserve(max_file_size); + if (!reservation) + throw Exception("Not enough space on temporary disk", ErrorCodes::NOT_ENOUGH_SPACE); + disk = reservation->getDisk(); + } else - tmp_stream = TemporaryFileStream::create(volume, header, max_file_size, this); + { + disk = volume->getDisk(); + } + + auto tmp_file = std::make_unique(disk, current_metric_scope); std::lock_guard lock(mutex); - return *streams.emplace_back(std::move(tmp_stream)); + TemporaryFileStreamPtr & tmp_stream = streams.emplace_back(std::make_unique(std::move(tmp_file), header, this)); + return *tmp_stream; } + std::vector TemporaryDataOnDisk::getStreams() const { std::vector res; @@ -87,13 +89,12 @@ struct TemporaryFileStream::OutputWriter { } - size_t write(const Block & block) + void write(const Block & block) { if (finalized) throw Exception("Cannot write to finalized stream", ErrorCodes::LOGICAL_ERROR); - size_t written_bytes = out_writer.write(block); + out_writer.write(block); num_rows += block.rows(); - return written_bytes; } void finalize() @@ -154,68 +155,21 @@ struct TemporaryFileStream::InputReader NativeReader in_reader; }; -TemporaryFileStreamPtr TemporaryFileStream::create(const VolumePtr & volume, const Block & header, size_t max_file_size, TemporaryDataOnDisk * parent_) -{ - if (!volume) - throw Exception("TemporaryDataOnDiskScope has no volume", ErrorCodes::LOGICAL_ERROR); - - DiskPtr disk; - if (max_file_size > 0) - { - auto reservation = volume->reserve(max_file_size); - if (!reservation) - throw Exception("Not enough space on temporary disk", ErrorCodes::NOT_ENOUGH_SPACE); - disk = reservation->getDisk(); - } - else - { - disk = volume->getDisk(); - } - - auto tmp_file = std::make_unique(disk, parent_->getMetricScope()); - return std::make_unique(std::move(tmp_file), header, /* cache_placeholder */ nullptr, /* parent */ parent_); -} - -TemporaryFileStreamPtr TemporaryFileStream::create(FileCache * cache, const Block & header, size_t max_file_size, TemporaryDataOnDisk * parent_) -{ - auto tmp_file = std::make_unique(fs::path(cache->getBasePath()) / "tmp"); - - auto cache_placeholder = std::make_unique(cache, tmp_file->getPath()); - cache_placeholder->reserveCapacity(max_file_size); - - return std::make_unique(std::move(tmp_file), header, std::move(cache_placeholder), parent_); -} - -TemporaryFileStream::TemporaryFileStream( - TemporaryFileHolder file_, - const Block & header_, - std::unique_ptr space_holder_, - TemporaryDataOnDisk * parent_) +TemporaryFileStream::TemporaryFileStream(TemporaryFileOnDiskHolder file_, const Block & header_, TemporaryDataOnDisk * parent_) : parent(parent_) , header(header_) , file(std::move(file_)) - , space_holder(std::move(space_holder_)) , out_writer(std::make_unique(file->getPath(), header)) { } -size_t TemporaryFileStream::write(const Block & block) +void TemporaryFileStream::write(const Block & block) { if (!out_writer) throw Exception("Writing has been finished", ErrorCodes::LOGICAL_ERROR); - size_t block_size_in_memory = block.bytes(); - - if (space_holder) - space_holder->reserveCapacity(block_size_in_memory); - updateAllocAndCheck(); - - size_t bytes_written = out_writer->write(block); - if (space_holder) - space_holder->setUsed(bytes_written); - - return bytes_written; + out_writer->write(block); } TemporaryFileStream::Stat TemporaryFileStream::finishWriting() diff --git a/src/Interpreters/TemporaryDataOnDisk.h b/src/Interpreters/TemporaryDataOnDisk.h index 2790529754f..11edc8700d2 100644 --- a/src/Interpreters/TemporaryDataOnDisk.h +++ b/src/Interpreters/TemporaryDataOnDisk.h @@ -6,7 +6,6 @@ #include #include #include -#include namespace CurrentMetrics @@ -41,25 +40,23 @@ public: std::atomic uncompressed_size; }; - explicit TemporaryDataOnDiskScope(VolumePtr volume_, FileCache * cache_, size_t limit_) - : volume(std::move(volume_)), cache(cache_), limit(limit_) + explicit TemporaryDataOnDiskScope(VolumePtr volume_, size_t limit_) + : volume(std::move(volume_)), limit(limit_) {} explicit TemporaryDataOnDiskScope(TemporaryDataOnDiskScopePtr parent_, size_t limit_) - : parent(std::move(parent_)), volume(parent->volume), cache(parent->cache), limit(limit_) + : parent(std::move(parent_)), volume(parent->volume), limit(limit_) {} /// TODO: remove /// Refactor all code that uses volume directly to use TemporaryDataOnDisk. - VolumePtr getVolume() const; + VolumePtr getVolume() const { return volume; } protected: void deltaAllocAndCheck(ssize_t compressed_delta, ssize_t uncompressed_delta); TemporaryDataOnDiskScopePtr parent = nullptr; - VolumePtr volume; - FileCache * cache = nullptr; StatAtomic stat; size_t limit = 0; @@ -94,7 +91,6 @@ public: bool empty() const; const StatAtomic & getStat() const { return stat; } - CurrentMetrics::Value getMetricScope() const { return current_metric_scope; } private: mutable std::mutex mutex; @@ -120,14 +116,9 @@ public: size_t num_rows = 0; }; - static TemporaryFileStreamPtr create(const VolumePtr & volume, const Block & header, size_t max_file_size, TemporaryDataOnDisk * parent_); - static TemporaryFileStreamPtr create(FileCache * cache, const Block & header, size_t max_file_size, TemporaryDataOnDisk * parent_); - - TemporaryFileStream(TemporaryFileHolder file_, const Block & header_, std::unique_ptr space_holder, TemporaryDataOnDisk * parent_); - - /// Returns number of written bytes - size_t write(const Block & block); + TemporaryFileStream(TemporaryFileOnDiskHolder file_, const Block & header_, TemporaryDataOnDisk * parent_); + void write(const Block & block); Stat finishWriting(); bool isWriteFinished() const; @@ -151,8 +142,7 @@ private: Block header; - TemporaryFileHolder file; - std::unique_ptr space_holder; + TemporaryFileOnDiskHolder file; Stat stat; diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index 2d408bd9b34..22150b9f656 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include @@ -15,14 +14,11 @@ #include #include #include -#include - -#include namespace fs = std::filesystem; -using namespace DB; -static constexpr auto TEST_LOG_LEVEL = "debug"; +fs::path caches_dir = fs::current_path() / "lru_cache_test"; +String cache_base_path = caches_dir / "cache1" / ""; void assertRange( [[maybe_unused]] size_t assert_n, DB::FileSegmentPtr file_segment, @@ -57,7 +53,7 @@ String getFileSegmentPath(const String & base_path, const DB::FileCache::Key & k return fs::path(base_path) / key_str.substr(0, 3) / key_str / DB::toString(offset); } -void download(const std::string & cache_base_path, DB::FileSegmentPtr file_segment) +void download(DB::FileSegmentPtr file_segment) { const auto & key = file_segment->key(); size_t size = file_segment->range().size(); @@ -71,57 +67,30 @@ void download(const std::string & cache_base_path, DB::FileSegmentPtr file_segme file_segment->write(data.data(), size, file_segment->getCurrentWriteOffset()); } -void prepareAndDownload(const std::string & cache_base_path, DB::FileSegmentPtr file_segment) +void prepareAndDownload(DB::FileSegmentPtr file_segment) { + // std::cerr << "Reserving: " << file_segment->range().size() << " for: " << file_segment->range().toString() << "\n"; ASSERT_TRUE(file_segment->reserve(file_segment->range().size())); - download(cache_base_path, file_segment); + download(file_segment); } -void complete(const std::string & cache_base_path, const DB::FileSegmentsHolder & holder) +void complete(const DB::FileSegmentsHolder & holder) { for (const auto & file_segment : holder.file_segments) { ASSERT_TRUE(file_segment->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(cache_base_path, file_segment); + prepareAndDownload(file_segment); file_segment->completeWithState(DB::FileSegment::State::DOWNLOADED); } } -class FileCacheTest : public ::testing::Test + +TEST(FileCache, get) { -public: + if (fs::exists(cache_base_path)) + fs::remove_all(cache_base_path); + fs::create_directories(cache_base_path); - static void setupLogs(const std::string & level) - { - Poco::AutoPtr channel(new Poco::ConsoleChannel(std::cerr)); - Poco::Logger::root().setChannel(channel); - Poco::Logger::root().setLevel(level); - } - - void SetUp() override - { - if(const char * test_log_level = std::getenv("TEST_LOG_LEVEL")) // NOLINT(concurrency-mt-unsafe) - setupLogs(test_log_level); - else - setupLogs(TEST_LOG_LEVEL); - - if (fs::exists(cache_base_path)) - fs::remove_all(cache_base_path); - fs::create_directories(cache_base_path); - } - - void TearDown() override - { - if (fs::exists(cache_base_path)) - fs::remove_all(cache_base_path); - } - - fs::path caches_dir = fs::current_path() / "lru_cache_test"; - std::string cache_base_path = caches_dir / "cache1" / ""; -}; - -TEST_F(FileCacheTest, get) -{ DB::ThreadStatus thread_status; /// To work with cache need query_id and query context. @@ -157,7 +126,7 @@ TEST_F(FileCacheTest, get) ASSERT_TRUE(segments[0]->reserve(segments[0]->range().size())); assertRange(2, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADING); - download(cache_base_path, segments[0]); + download(segments[0]); segments[0]->completeWithState(DB::FileSegment::State::DOWNLOADED); assertRange(3, segments[0], DB::FileSegment::Range(0, 9), DB::FileSegment::State::DOWNLOADED); } @@ -178,7 +147,7 @@ TEST_F(FileCacheTest, get) assertRange(5, segments[1], DB::FileSegment::Range(10, 14), DB::FileSegment::State::EMPTY); ASSERT_TRUE(segments[1]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(cache_base_path, segments[1]); + prepareAndDownload(segments[1]); segments[1]->completeWithState(DB::FileSegment::State::DOWNLOADED); assertRange(6, segments[1], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED); } @@ -211,8 +180,8 @@ TEST_F(FileCacheTest, get) assertRange(10, segments[0], DB::FileSegment::Range(10, 14), DB::FileSegment::State::DOWNLOADED); } - complete(cache_base_path, cache.getOrSet(key, 17, 4, {})); /// Get [17, 20] - complete(cache_base_path, cache.getOrSet(key, 24, 3, {})); /// Get [24, 26] + complete(cache.getOrSet(key, 17, 4, {})); /// Get [17, 20] + complete(cache.getOrSet(key, 24, 3, {})); /// Get [24, 26] /// completeWithState(cache.getOrSet(key, 27, 1, false)); /// Get [27, 27] /// Current cache: [__________][_____] [____] [___][] @@ -234,7 +203,7 @@ TEST_F(FileCacheTest, get) assertRange(13, segments[2], DB::FileSegment::Range(15, 16), DB::FileSegment::State::EMPTY); ASSERT_TRUE(segments[2]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(cache_base_path, segments[2]); + prepareAndDownload(segments[2]); segments[2]->completeWithState(DB::FileSegment::State::DOWNLOADED); @@ -275,7 +244,7 @@ TEST_F(FileCacheTest, get) assertRange(21, segments[3], DB::FileSegment::Range(21, 21), DB::FileSegment::State::EMPTY); ASSERT_TRUE(segments[3]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(cache_base_path, segments[3]); + prepareAndDownload(segments[3]); segments[3]->completeWithState(DB::FileSegment::State::DOWNLOADED); ASSERT_TRUE(segments[3]->state() == DB::FileSegment::State::DOWNLOADED); @@ -298,8 +267,8 @@ TEST_F(FileCacheTest, get) ASSERT_TRUE(segments[0]->getOrSetDownloader() == DB::FileSegment::getCallerId()); ASSERT_TRUE(segments[2]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(cache_base_path, segments[0]); - prepareAndDownload(cache_base_path, segments[2]); + prepareAndDownload(segments[0]); + prepareAndDownload(segments[2]); segments[0]->completeWithState(DB::FileSegment::State::DOWNLOADED); segments[2]->completeWithState(DB::FileSegment::State::DOWNLOADED); } @@ -321,8 +290,8 @@ TEST_F(FileCacheTest, get) ASSERT_TRUE(s5[0]->getOrSetDownloader() == DB::FileSegment::getCallerId()); ASSERT_TRUE(s1[0]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(cache_base_path, s5[0]); - prepareAndDownload(cache_base_path, s1[0]); + prepareAndDownload(s5[0]); + prepareAndDownload(s1[0]); s5[0]->completeWithState(DB::FileSegment::State::DOWNLOADED); s1[0]->completeWithState(DB::FileSegment::State::DOWNLOADED); @@ -425,7 +394,7 @@ TEST_F(FileCacheTest, get) cv.wait(lock, [&]{ return lets_start_download; }); } - prepareAndDownload(cache_base_path, segments[2]); + prepareAndDownload(segments[2]); segments[2]->completeWithState(DB::FileSegment::State::DOWNLOADED); ASSERT_TRUE(segments[2]->state() == DB::FileSegment::State::DOWNLOADED); @@ -490,7 +459,7 @@ TEST_F(FileCacheTest, get) ASSERT_TRUE(segments_2[1]->state() == DB::FileSegment::State::PARTIALLY_DOWNLOADED); ASSERT_TRUE(segments_2[1]->getOrSetDownloader() == DB::FileSegment::getCallerId()); - prepareAndDownload(cache_base_path, segments_2[1]); + prepareAndDownload(segments_2[1]); segments_2[1]->completeWithState(DB::FileSegment::State::DOWNLOADED); }); @@ -548,141 +517,3 @@ TEST_F(FileCacheTest, get) } } - -TEST_F(FileCacheTest, rangeWriter) -{ - DB::FileCacheSettings settings; - settings.max_size = 25; - settings.max_elements = 5; - settings.max_file_segment_size = 10; - - DB::FileCache cache(cache_base_path, settings); - cache.initialize(); - auto key = cache.hash("key1"); - - DB::FileSegmentRangeWriter writer(&cache, key, nullptr, "", "key1"); - - std::string data(100, '\xf0'); - - size_t total_written = 0; - for (const size_t size : {3, 5, 8, 1, 1, 3}) - { - total_written += size; - ASSERT_EQ(writer.tryWrite(data.data(), size, writer.currentOffset()), size); - } - ASSERT_LT(total_written, settings.max_size); - - size_t offset_before_unsuccessful_write = writer.currentOffset(); - size_t space_left = settings.max_size - total_written; - ASSERT_EQ(writer.tryWrite(data.data(), space_left + 1, writer.currentOffset()), 0); - - ASSERT_EQ(writer.currentOffset(), offset_before_unsuccessful_write); - - ASSERT_EQ(writer.tryWrite(data.data(), space_left, writer.currentOffset()), space_left); - - writer.finalize(); -} - -static Block generateBlock(size_t size = 0) -{ - Block block; - ColumnWithTypeAndName column; - column.name = "x"; - column.type = std::make_shared(); - - { - MutableColumnPtr mut_col = column.type->createColumn(); - for (size_t i = 0; i < size; ++i) - mut_col->insert(i); - column.column = std::move(mut_col); - } - - block.insert(column); - - LOG_DEBUG(&Poco::Logger::get("FileCacheTest"), "generated block {} bytes", block.bytes()); - return block; -} - -static size_t readAllTemporaryData(TemporaryFileStream & stream) -{ - Block block; - size_t read_rows = 0; - do - { - block = stream.read(); - read_rows += block.rows(); - } while (block); - return read_rows; -} - -TEST_F(FileCacheTest, temporaryData) -{ - DB::FileCacheSettings settings; - settings.max_size = 10240; - settings.max_file_segment_size = 1024; - - DB::FileCache file_cache(cache_base_path, settings); - file_cache.initialize(); - - auto tmp_data_scope = std::make_shared(nullptr, &file_cache, 0); - - auto some_data_holder = file_cache.getOrSet(file_cache.hash("some_data"), 0, 1024 * 5, CreateFileSegmentSettings{}); - - { - auto segments = fromHolder(some_data_holder); - ASSERT_EQ(segments.size(), 5); - for (auto & segment : segments) - { - ASSERT_TRUE(segment->getOrSetDownloader() == DB::FileSegment::getCallerId()); - ASSERT_TRUE(segment->reserve(segment->range().size())); - download(cache_base_path, segment); - segment->completeWithState(DB::FileSegment::State::DOWNLOADED); - } - } - - size_t size_used_before_temporary_data = file_cache.getUsedCacheSize(); - size_t segments_used_before_temporary_data = file_cache.getFileSegmentsNum(); - ASSERT_GT(size_used_before_temporary_data, 0); - ASSERT_GT(segments_used_before_temporary_data, 0); - - size_t size_used_with_temporary_data; - size_t segments_used_with_temporary_data; - { - auto tmp_data = std::make_unique(tmp_data_scope); - - auto & stream = tmp_data->createStream(generateBlock()); - - ASSERT_GT(stream.write(generateBlock(100)), 0); - - EXPECT_GT(file_cache.getUsedCacheSize(), 0); - EXPECT_GT(file_cache.getFileSegmentsNum(), 0); - - size_t used_size_before_attempt = file_cache.getUsedCacheSize(); - /// data can't be evicted because it is still held by `some_data_holder` - ASSERT_THROW(stream.write(generateBlock(1000)), DB::Exception); - - ASSERT_EQ(file_cache.getUsedCacheSize(), used_size_before_attempt); - - some_data_holder.reset(); - - stream.write(generateBlock(1011)); - - auto stat = stream.finishWriting(); - - EXPECT_EQ(stat.num_rows, 1111); - EXPECT_EQ(readAllTemporaryData(stream), 1111); - - size_used_with_temporary_data = file_cache.getUsedCacheSize(); - segments_used_with_temporary_data = file_cache.getFileSegmentsNum(); - EXPECT_GT(size_used_with_temporary_data, 0); - EXPECT_GT(segments_used_with_temporary_data, 0); - } - - /// All temp data should be evicted after removing temporary files - EXPECT_LE(file_cache.getUsedCacheSize(), size_used_with_temporary_data); - EXPECT_LE(file_cache.getFileSegmentsNum(), segments_used_with_temporary_data); - - /// Some segments reserved by `some_data_holder` was eviced by temporary data - EXPECT_LE(file_cache.getUsedCacheSize(), size_used_before_temporary_data); - EXPECT_LE(file_cache.getFileSegmentsNum(), segments_used_before_temporary_data); -} diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 2b8d9055cff..6a29cdbb5ca 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -192,7 +192,7 @@ private: bool force_ttl{false}; CompressionCodecPtr compression_codec{nullptr}; size_t sum_input_rows_upper_bound{0}; - std::unique_ptr rows_sources_file{nullptr}; + std::unique_ptr rows_sources_file{nullptr}; std::unique_ptr rows_sources_uncompressed_write_buf{nullptr}; std::unique_ptr rows_sources_write_buf{nullptr}; std::optional column_sizes{}; @@ -257,7 +257,7 @@ private: /// Begin dependencies from previous stage std::unique_ptr rows_sources_write_buf{nullptr}; std::unique_ptr rows_sources_uncompressed_write_buf{nullptr}; - std::unique_ptr rows_sources_file; + std::unique_ptr rows_sources_file; std::optional column_sizes; CompressionCodecPtr compression_codec; DiskPtr tmp_disk{nullptr}; diff --git a/src/Storages/System/StorageSystemFilesystemCache.cpp b/src/Storages/System/StorageSystemFilesystemCache.cpp index bec92a60436..cd9324b3253 100644 --- a/src/Storages/System/StorageSystemFilesystemCache.cpp +++ b/src/Storages/System/StorageSystemFilesystemCache.cpp @@ -24,8 +24,7 @@ NamesAndTypesList StorageSystemFilesystemCache::getNamesAndTypes() {"cache_hits", std::make_shared()}, {"references", std::make_shared()}, {"downloaded_size", std::make_shared()}, - {"persistent", std::make_shared>()}, - {"kind", std::make_shared()}, + {"persistent", std::make_shared>()} }; } @@ -46,11 +45,8 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex for (const auto & file_segment : file_segments) { res_columns[0]->insert(cache_base_path); - - /// Do not use `file_segment->getPathInLocalCache` here because it will lead to nullptr dereference - /// (because file_segments in getSnapshot doesn't have `cache` field set) res_columns[1]->insert( - cache->getPathInLocalCache(file_segment->key(), file_segment->offset(), file_segment->getKind())); + cache->getPathInLocalCache(file_segment->key(), file_segment->offset(), file_segment->isPersistent())); const auto & range = file_segment->range(); res_columns[2]->insert(range.left); @@ -61,7 +57,6 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex res_columns[7]->insert(file_segment->getRefCount()); res_columns[8]->insert(file_segment->getDownloadedSize()); res_columns[9]->insert(file_segment->isPersistent()); - res_columns[10]->insert(toString(file_segment->getKind())); } } } diff --git a/tests/integration/test_temporary_data_in_cache/__init__.py b/tests/integration/test_temporary_data_in_cache/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml b/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml deleted file mode 100644 index 8ccd705c6f1..00000000000 --- a/tests/integration/test_temporary_data_in_cache/configs/config.d/storage_configuration.xml +++ /dev/null @@ -1,39 +0,0 @@ - - - - - local - /local_disk/ - - - - cache - local_disk - /tiny_local_cache/ - 10M - 1M - 1 - 0 - - - - - - local - /tiny_local_cache/ - - - - - - -
- tiny_local_cache -
-
-
-
-
- - tiny_local_cache -
diff --git a/tests/integration/test_temporary_data_in_cache/test.py b/tests/integration/test_temporary_data_in_cache/test.py deleted file mode 100644 index ba57348ee37..00000000000 --- a/tests/integration/test_temporary_data_in_cache/test.py +++ /dev/null @@ -1,81 +0,0 @@ -# pylint: disable=unused-argument -# pylint: disable=redefined-outer-name - -import pytest - -from helpers.cluster import ClickHouseCluster -from helpers.client import QueryRuntimeException - -cluster = ClickHouseCluster(__file__) - -node = cluster.add_instance( - "node", - main_configs=["configs/config.d/storage_configuration.xml"], - tmpfs=["/local_disk:size=50M", "/tiny_local_cache:size=12M"], -) - - -@pytest.fixture(scope="module") -def start_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def test_cache_evicted_by_temporary_data(start_cluster): - q = node.query - qi = lambda query: int(node.query(query).strip()) - - cache_size_initial = qi("SELECT sum(size) FROM system.filesystem_cache") - assert cache_size_initial == 0 - - free_space_initial = qi( - "SELECT free_space FROM system.disks WHERE name = 'tiny_local_cache_local_disk'" - ) - assert free_space_initial > 8 * 1024 * 1024 - - q( - "CREATE TABLE t1 (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS storage_policy = 'tiny_local_cache'" - ) - q("INSERT INTO t1 SELECT number FROM numbers(1024 * 1024)") - - # To be sure that nothing is reading the cache and entries for t1 can be evited - q("OPTIMIZE TABLE t1 FINAL") - q("SYSTEM STOP MERGES t1") - - # Read some data to fill the cache - q("SELECT sum(x) FROM t1") - - cache_size_with_t1 = qi("SELECT sum(size) FROM system.filesystem_cache") - assert cache_size_with_t1 > 8 * 1024 * 1024 - - # Almost all disk space is occupied by t1 cache - free_space_with_t1 = qi( - "SELECT free_space FROM system.disks WHERE name = 'tiny_local_cache_local_disk'" - ) - assert free_space_with_t1 < 4 * 1024 * 1024 - - # Try to sort the table, but fail because of lack of disk space - with pytest.raises(QueryRuntimeException) as exc: - q( - "SELECT ignore(*) FROM numbers(10 * 1024 * 1024) ORDER BY sipHash64(number)", - settings={ - "max_bytes_before_external_group_by": "4M", - "max_bytes_before_external_sort": "4M", - }, - ) - assert "Cannot reserve space in file cache" in str(exc.value) - - # Some data evicted from cache by temporary data - cache_size_after_eviction = qi("SELECT sum(size) FROM system.filesystem_cache") - assert cache_size_after_eviction < cache_size_with_t1 - - # Disk space freed, at least 3 MB, because temporary data tried to write 4 MB - free_space_after_eviction = qi( - "SELECT free_space FROM system.disks WHERE name = 'tiny_local_cache_local_disk'" - ) - assert free_space_after_eviction > free_space_with_t1 + 3 * 1024 * 1024 - - q("DROP TABLE IF EXISTS t1") diff --git a/tests/integration/test_tmp_policy/test.py b/tests/integration/test_tmp_policy/test.py index 870a70b127a..c919d9a0c3d 100644 --- a/tests/integration/test_tmp_policy/test.py +++ b/tests/integration/test_tmp_policy/test.py @@ -23,7 +23,7 @@ def start_cluster(): cluster.shutdown() -def test_disk_selection(start_cluster): +def test_different_versions(start_cluster): query = "SELECT count(ignore(*)) FROM (SELECT * FROM system.numbers LIMIT 1e7) GROUP BY number" settings = { "max_bytes_before_external_group_by": 1 << 20, From e1f7f047528f8af81273f75869707c66aa4d5ca2 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 2 Dec 2022 15:05:46 +0100 Subject: [PATCH 87/98] Referential dependencies for RESTORE (#43834) * Rename DDLDependencyVisitor -> DDLLoadingDependencyVisitor. * Move building a loading graph to TablesLoader. * Implement referential dependencies for tables and use them when restoring tables from a backup. * Remove StorageID::operator < (because of its inconsistency with ==). * Add new tests. * Fix test. * Fix memory leak. Co-authored-by: Nikita Mikhaylov --- src/Backups/RestorerFromBackup.cpp | 126 ++-- src/Backups/RestorerFromBackup.h | 6 +- src/Databases/DDLDependencyVisitor.cpp | 373 +++++----- src/Databases/DDLDependencyVisitor.h | 66 +- src/Databases/DDLLoadingDependencyVisitor.cpp | 152 ++++ src/Databases/DDLLoadingDependencyVisitor.h | 54 ++ src/Databases/DatabaseMemory.cpp | 7 +- src/Databases/DatabaseOrdinary.cpp | 18 +- .../NormalizeAndEvaluateConstantsVisitor.cpp | 56 ++ .../NormalizeAndEvaluateConstantsVisitor.h | 32 + src/Databases/TablesDependencyGraph.cpp | 656 ++++++++++++++++++ src/Databases/TablesDependencyGraph.h | 171 +++++ src/Databases/TablesLoader.cpp | 258 ++----- src/Databases/TablesLoader.h | 44 +- src/Interpreters/Context.h | 1 + src/Interpreters/DatabaseCatalog.cpp | 157 ++--- src/Interpreters/DatabaseCatalog.h | 48 +- src/Interpreters/InterpreterCreateQuery.cpp | 7 +- src/Interpreters/InterpreterDropQuery.cpp | 8 +- src/Interpreters/InterpreterRenameQuery.cpp | 8 +- src/Interpreters/StorageID.cpp | 17 +- src/Interpreters/StorageID.h | 42 +- src/Interpreters/loadMetadata.h | 1 - .../Transforms/buildPushingToViewsChain.cpp | 32 +- src/Storages/FileLog/StorageFileLog.cpp | 16 +- src/Storages/IStorage.cpp | 12 +- src/Storages/Kafka/StorageKafka.cpp | 20 +- src/Storages/LiveView/StorageLiveView.cpp | 14 +- src/Storages/NATS/StorageNATS.cpp | 20 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 20 +- src/Storages/StorageMaterializedView.cpp | 10 +- src/Storages/System/StorageSystemTables.cpp | 59 +- src/Storages/WindowView/StorageWindowView.cpp | 14 +- .../test_backup_restore_new/test.py | 69 ++ .../01155_rename_move_materialized_view.sql | 4 +- 35 files changed, 1809 insertions(+), 789 deletions(-) create mode 100644 src/Databases/DDLLoadingDependencyVisitor.cpp create mode 100644 src/Databases/DDLLoadingDependencyVisitor.h create mode 100644 src/Databases/NormalizeAndEvaluateConstantsVisitor.cpp create mode 100644 src/Databases/NormalizeAndEvaluateConstantsVisitor.h create mode 100644 src/Databases/TablesDependencyGraph.cpp create mode 100644 src/Databases/TablesDependencyGraph.h diff --git a/src/Backups/RestorerFromBackup.cpp b/src/Backups/RestorerFromBackup.cpp index 185c23a479e..244a51669a1 100644 --- a/src/Backups/RestorerFromBackup.cpp +++ b/src/Backups/RestorerFromBackup.cpp @@ -96,6 +96,7 @@ RestorerFromBackup::RestorerFromBackup( , on_cluster_first_sync_timeout(context->getConfigRef().getUInt64("backups.on_cluster_first_sync_timeout", 180000)) , create_table_timeout(context->getConfigRef().getUInt64("backups.create_table_timeout", 300000)) , log(&Poco::Logger::get("RestorerFromBackup")) + , tables_dependencies("RestorerFromBackup") { } @@ -133,6 +134,7 @@ RestorerFromBackup::DataRestoreTasks RestorerFromBackup::run(Mode mode) /// Create tables using the create queries read from the backup. setStage(Stage::CREATING_TABLES); + removeUnresolvedDependencies(); createTables(); /// All what's left is to insert data to tables. @@ -341,10 +343,11 @@ void RestorerFromBackup::findTableInBackup(const QualifiedTableName & table_name TableInfo & res_table_info = table_infos[table_name]; res_table_info.create_table_query = create_table_query; res_table_info.is_predefined_table = DatabaseCatalog::instance().isPredefinedTable(StorageID{table_name.database, table_name.table}); - res_table_info.dependencies = getDependenciesSetFromCreateQuery(context->getGlobalContext(), table_name, create_table_query); res_table_info.has_data = backup->hasFiles(data_path_in_backup); res_table_info.data_path_in_backup = data_path_in_backup; + tables_dependencies.addDependencies(table_name, getDependenciesFromCreateQuery(context->getGlobalContext(), table_name, create_table_query)); + if (partitions) { if (!res_table_info.partitions) @@ -622,21 +625,62 @@ void RestorerFromBackup::checkDatabase(const String & database_name) } } +void RestorerFromBackup::removeUnresolvedDependencies() +{ + auto need_exclude_dependency = [this](const StorageID & table_id) + { + /// Table will be restored. + if (table_infos.contains(table_id.getQualifiedName())) + return false; + + /// Table exists and it already exists + if (!DatabaseCatalog::instance().isTableExist(table_id, context)) + { + LOG_WARNING( + log, + "Tables {} in backup depend on {}, but seems like {} is not in the backup and does not exist. " + "Will try to ignore that and restore tables", + fmt::join(tables_dependencies.getDependents(table_id), ", "), + table_id, + table_id); + } + + size_t num_dependencies, num_dependents; + tables_dependencies.getNumberOfAdjacents(table_id, num_dependencies, num_dependents); + if (num_dependencies || !num_dependents) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Table {} in backup doesn't have dependencies and dependent tables as it expected to. It's a bug", + table_id); + + return true; /// Exclude this dependency. + }; + + tables_dependencies.removeTablesIf(need_exclude_dependency); + + if (tables_dependencies.getNumberOfTables() != table_infos.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Number of tables to be restored is not as expected. It's a bug"); + + if (tables_dependencies.hasCyclicDependencies()) + { + LOG_WARNING( + log, + "Tables {} in backup have cyclic dependencies: {}. Will try to ignore that and restore tables", + fmt::join(tables_dependencies.getTablesWithCyclicDependencies(), ", "), + tables_dependencies.describeCyclicDependencies()); + } +} + void RestorerFromBackup::createTables() { - while (true) + /// We need to create tables considering their dependencies. + auto tables_to_create = tables_dependencies.getTablesSortedByDependency(); + for (const auto & table_id : tables_to_create) { - /// We need to create tables considering their dependencies. - auto tables_to_create = findTablesWithoutDependencies(); - if (tables_to_create.empty()) - break; /// We've already created all the tables. - - for (const auto & table_name : tables_to_create) - { - createTable(table_name); - checkTable(table_name); - insertDataToTable(table_name); - } + auto table_name = table_id.getQualifiedName(); + createTable(table_name); + checkTable(table_name); + insertDataToTable(table_name); } } @@ -752,62 +796,6 @@ void RestorerFromBackup::insertDataToTable(const QualifiedTableName & table_name } } -/// Returns the list of tables without dependencies or those which dependencies have been created before. -std::vector RestorerFromBackup::findTablesWithoutDependencies() const -{ - std::vector tables_without_dependencies; - bool all_tables_created = true; - - for (const auto & [key, table_info] : table_infos) - { - if (table_info.storage) - continue; - - /// Found a table which is not created yet. - all_tables_created = false; - - /// Check if all dependencies have been created before. - bool all_dependencies_met = true; - for (const auto & dependency : table_info.dependencies) - { - auto it = table_infos.find(dependency); - if ((it != table_infos.end()) && !it->second.storage) - { - all_dependencies_met = false; - break; - } - } - - if (all_dependencies_met) - tables_without_dependencies.push_back(key); - } - - if (!tables_without_dependencies.empty()) - return tables_without_dependencies; - - if (all_tables_created) - return {}; - - /// Cyclic dependency? We'll try to create those tables anyway but probably it's going to fail. - std::vector tables_with_cyclic_dependencies; - for (const auto & [key, table_info] : table_infos) - { - if (!table_info.storage) - tables_with_cyclic_dependencies.push_back(key); - } - - /// Only show a warning here, proper exception will be thrown later on creating those tables. - LOG_WARNING( - log, - "Some tables have cyclic dependency from each other: {}", - boost::algorithm::join( - tables_with_cyclic_dependencies - | boost::adaptors::transformed([](const QualifiedTableName & table_name) -> String { return table_name.getFullName(); }), - ", ")); - - return tables_with_cyclic_dependencies; -} - void RestorerFromBackup::addDataRestoreTask(DataRestoreTask && new_task) { if (current_stage == Stage::INSERTING_DATA_TO_TABLES) diff --git a/src/Backups/RestorerFromBackup.h b/src/Backups/RestorerFromBackup.h index b081e16e2ce..93b5a6c7694 100644 --- a/src/Backups/RestorerFromBackup.h +++ b/src/Backups/RestorerFromBackup.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -94,6 +95,7 @@ private: void createDatabase(const String & database_name) const; void checkDatabase(const String & database_name); + void removeUnresolvedDependencies(); void createTables(); void createTable(const QualifiedTableName & table_name); void checkTable(const QualifiedTableName & table_name); @@ -114,7 +116,6 @@ private: { ASTPtr create_table_query; bool is_predefined_table = false; - std::unordered_set dependencies; bool has_data = false; std::filesystem::path data_path_in_backup; std::optional partitions; @@ -123,11 +124,10 @@ private: TableLockHolder table_lock; }; - std::vector findTablesWithoutDependencies() const; - String current_stage; std::unordered_map database_infos; std::map table_infos; + TablesDependencyGraph tables_dependencies; std::vector data_restore_tasks; std::unique_ptr access_restorer; bool access_restored = false; diff --git a/src/Databases/DDLDependencyVisitor.cpp b/src/Databases/DDLDependencyVisitor.cpp index bedaec75565..525f4fb7b12 100644 --- a/src/Databases/DDLDependencyVisitor.cpp +++ b/src/Databases/DDLDependencyVisitor.cpp @@ -1,198 +1,239 @@ #include #include #include -#include #include #include #include #include #include +#include #include + namespace DB { -using TableLoadingDependenciesVisitor = DDLDependencyVisitor::Visitor; +namespace +{ + /// CREATE TABLE or CREATE DICTIONARY or CREATE VIEW or CREATE TEMPORARY TABLE or CREATE DATABASE query. + void visitCreateQuery(const ASTCreateQuery & create, DDLDependencyVisitor::Data & data) + { + QualifiedTableName to_table{create.to_table_id.database_name, create.to_table_id.table_name}; + if (!to_table.table.empty()) + { + /// TO target_table (for materialized views) + if (to_table.database.empty()) + to_table.database = data.default_database; + data.dependencies.emplace(to_table); + } -TableNamesSet getDependenciesSetFromCreateQuery(ContextPtr global_context, const QualifiedTableName & table, const ASTPtr & ast) + QualifiedTableName as_table{create.as_database, create.as_table}; + if (!as_table.table.empty()) + { + /// AS table_name + if (as_table.database.empty()) + as_table.database = data.default_database; + data.dependencies.emplace(as_table); + } + } + + /// ASTTableExpression represents a reference to a table in SELECT query. + /// DDLDependencyVisitor should handle ASTTableExpression because some CREATE queries can contain SELECT queries after AS + /// (for example, CREATE VIEW). + void visitTableExpression(const ASTTableExpression & expr, DDLDependencyVisitor::Data & data) + { + if (!expr.database_and_table_name) + return; + + const ASTIdentifier * identifier = dynamic_cast(expr.database_and_table_name.get()); + if (!identifier) + return; + + auto table_identifier = identifier->createTable(); + if (!table_identifier) + return; + + QualifiedTableName qualified_name{table_identifier->getDatabaseName(), table_identifier->shortName()}; + if (qualified_name.table.empty()) + return; + + if (qualified_name.database.empty()) + { + /// It can be table/dictionary from default database or XML dictionary, but we cannot distinguish it here. + qualified_name.database = data.default_database; + } + + data.dependencies.emplace(qualified_name); + } + + /// Extracts a table name with optional database written in the form db_name.table_name (as identifier) or 'db_name.table_name' (as string). + void extractQualifiedTableNameFromArgument(const ASTFunction & function, DDLDependencyVisitor::Data & data, size_t arg_idx) + { + /// Just ignore incorrect arguments, proper exception will be thrown later + if (!function.arguments || function.arguments->children.size() <= arg_idx) + return; + + QualifiedTableName qualified_name; + + const auto * expr_list = function.arguments->as(); + if (!expr_list) + return; + + const auto * arg = expr_list->children[arg_idx].get(); + if (const auto * literal = arg->as()) + { + if (literal->value.getType() != Field::Types::String) + return; + + auto maybe_qualified_name = QualifiedTableName::tryParseFromString(literal->value.get()); + /// Just return if name if invalid + if (!maybe_qualified_name) + return; + + qualified_name = std::move(*maybe_qualified_name); + } + else if (const auto * identifier = dynamic_cast(arg)) + { + /// ASTIdentifier or ASTTableIdentifier + auto table_identifier = identifier->createTable(); + /// Just return if table identified is invalid + if (!table_identifier) + return; + + qualified_name.database = table_identifier->getDatabaseName(); + qualified_name.table = table_identifier->shortName(); + } + else + { + /// Just return because we don't validate AST in this function. + return; + } + + if (qualified_name.database.empty()) + { + /// It can be table/dictionary from default database or XML dictionary, but we cannot distinguish it here. + qualified_name.database = data.default_database; + } + data.dependencies.emplace(std::move(qualified_name)); + } + + /// Extracts a table name with database written in the form 'db_name', 'table_name' (two strings). + void extractDatabaseAndTableNameFromArguments(const ASTFunction & function, DDLDependencyVisitor::Data & data, size_t database_arg_idx, size_t table_arg_idx) + { + /// Just ignore incorrect arguments, proper exception will be thrown later + if (!function.arguments || (function.arguments->children.size() <= database_arg_idx) + || (function.arguments->children.size() <= table_arg_idx)) + return; + + const auto * expr_list = function.arguments->as(); + if (!expr_list) + return; + + const auto * database_literal = expr_list->children[database_arg_idx]->as(); + const auto * table_name_literal = expr_list->children[table_arg_idx]->as(); + + if (!database_literal || !table_name_literal || (database_literal->value.getType() != Field::Types::String) + || (table_name_literal->value.getType() != Field::Types::String)) + return; + + QualifiedTableName qualified_name{database_literal->value.get(), table_name_literal->value.get()}; + if (qualified_name.table.empty()) + return; + + if (qualified_name.database.empty()) + qualified_name.database = data.default_database; + + data.dependencies.emplace(qualified_name); + } + + void visitFunction(const ASTFunction & function, DDLDependencyVisitor::Data & data) + { + if (function.name == "joinGet" || function.name == "dictHas" || function.name == "dictIsIn" || function.name.starts_with("dictGet")) + { + /// dictGet('dict_name', attr_names, id_expr) + /// dictHas('dict_name', id_expr) + /// joinGet(join_storage_table_name, `value_column`, join_keys) + extractQualifiedTableNameFromArgument(function, data, 0); + } + else if (function.name == "in" || function.name == "notIn" || function.name == "globalIn" || function.name == "globalNotIn") + { + /// in(x, table_name) - function for evaluating (x IN table_name) + extractQualifiedTableNameFromArgument(function, data, 1); + } + else if (function.name == "dictionary") + { + /// dictionary(dict_name) + extractQualifiedTableNameFromArgument(function, data, 0); + } + } + + void visitTableEngine(const ASTFunction & table_engine, DDLDependencyVisitor::Data & data) + { + if (table_engine.name == "Dictionary") + extractQualifiedTableNameFromArgument(table_engine, data, 0); + + if (table_engine.name == "Buffer") + extractDatabaseAndTableNameFromArguments(table_engine, data, 0, 1); + } + + void visitDictionaryDef(const ASTDictionary & dictionary, DDLDependencyVisitor::Data & data) + { + if (!dictionary.source || dictionary.source->name != "clickhouse" || !dictionary.source->elements) + return; + + auto config = getDictionaryConfigurationFromAST(data.create_query->as(), data.global_context); + auto info = getInfoIfClickHouseDictionarySource(config, data.global_context); + + if (!info || !info->is_local) + return; + + if (info->table_name.database.empty()) + info->table_name.database = data.default_database; + data.dependencies.emplace(std::move(info->table_name)); + } +} + + +TableNamesSet getDependenciesFromCreateQuery(const ContextPtr & global_context, const QualifiedTableName & table_name, const ASTPtr & ast) { assert(global_context == global_context->getGlobalContext()); - TableLoadingDependenciesVisitor::Data data; + DDLDependencyVisitor::Data data; + data.table_name = table_name; data.default_database = global_context->getCurrentDatabase(); data.create_query = ast; data.global_context = global_context; - TableLoadingDependenciesVisitor visitor{data}; + DDLDependencyVisitor::Visitor visitor{data}; visitor.visit(ast); - data.dependencies.erase(table); + data.dependencies.erase(data.table_name); return data.dependencies; } void DDLDependencyVisitor::visit(const ASTPtr & ast, Data & data) { - /// Looking for functions in column default expressions and dictionary source definition - if (const auto * function = ast->as()) - visit(*function, data); - else if (const auto * dict_source = ast->as()) - visit(*dict_source, data); - else if (const auto * storage = ast->as()) - visit(*storage, data); + if (auto * create = ast->as()) + { + visitCreateQuery(*create, data); + } + else if (auto * dictionary = ast->as()) + { + visitDictionaryDef(*dictionary, data); + } + else if (auto * expr = ast->as()) + { + visitTableExpression(*expr, data); + } + else if (const auto * function = ast->as()) + { + if (function->kind == ASTFunction::Kind::TABLE_ENGINE) + visitTableEngine(*function, data); + else + visitFunction(*function, data); + } } -bool DDLMatcherBase::needChildVisit(const ASTPtr & node, const ASTPtr & child) +bool DDLDependencyVisitor::needChildVisit(const ASTPtr &, const ASTPtr &) { - if (node->as()) - return false; - - if (auto * create = node->as()) - { - if (child.get() == create->select) - return false; - } - return true; } -ssize_t DDLMatcherBase::getPositionOfTableNameArgument(const ASTFunction & function) -{ - if (function.name == "joinGet" || - function.name == "dictHas" || - function.name == "dictIsIn" || - function.name.starts_with("dictGet")) - return 0; - - if (Poco::toLower(function.name) == "in") - return 1; - - return -1; -} - -void DDLDependencyVisitor::visit(const ASTFunction & function, Data & data) -{ - ssize_t table_name_arg_idx = getPositionOfTableNameArgument(function); - if (table_name_arg_idx < 0) - return; - extractTableNameFromArgument(function, data, table_name_arg_idx); -} - -void DDLDependencyVisitor::visit(const ASTFunctionWithKeyValueArguments & dict_source, Data & data) -{ - if (dict_source.name != "clickhouse") - return; - if (!dict_source.elements) - return; - - auto config = getDictionaryConfigurationFromAST(data.create_query->as(), data.global_context); - auto info = getInfoIfClickHouseDictionarySource(config, data.global_context); - - if (!info || !info->is_local) - return; - - if (info->table_name.database.empty()) - info->table_name.database = data.default_database; - data.dependencies.emplace(std::move(info->table_name)); -} - -void DDLDependencyVisitor::visit(const ASTStorage & storage, Data & data) -{ - if (!storage.engine) - return; - if (storage.engine->name != "Dictionary") - return; - - extractTableNameFromArgument(*storage.engine, data, 0); -} - - -void DDLDependencyVisitor::extractTableNameFromArgument(const ASTFunction & function, Data & data, size_t arg_idx) -{ - /// Just ignore incorrect arguments, proper exception will be thrown later - if (!function.arguments || function.arguments->children.size() <= arg_idx) - return; - - QualifiedTableName qualified_name; - - const auto * arg = function.arguments->as()->children[arg_idx].get(); - if (const auto * literal = arg->as()) - { - if (literal->value.getType() != Field::Types::String) - return; - - auto maybe_qualified_name = QualifiedTableName::tryParseFromString(literal->value.get()); - /// Just return if name if invalid - if (!maybe_qualified_name) - return; - - qualified_name = std::move(*maybe_qualified_name); - } - else if (const auto * identifier = dynamic_cast(arg)) - { - /// ASTIdentifier or ASTTableIdentifier - auto table_identifier = identifier->createTable(); - /// Just return if table identified is invalid - if (!table_identifier) - return; - - qualified_name.database = table_identifier->getDatabaseName(); - qualified_name.table = table_identifier->shortName(); - } - else - { - assert(false); - return; - } - - if (qualified_name.database.empty()) - { - /// It can be table/dictionary from default database or XML dictionary, but we cannot distinguish it here. - qualified_name.database = data.default_database; - } - data.dependencies.emplace(std::move(qualified_name)); -} - - -void NormalizeAndEvaluateConstants::visit(const ASTPtr & ast, Data & data) -{ - assert(data.create_query_context->hasQueryContext()); - - /// Looking for functions in column default expressions and dictionary source definition - if (const auto * function = ast->as()) - visit(*function, data); - else if (const auto * dict_source = ast->as()) - visit(*dict_source, data); -} - -void NormalizeAndEvaluateConstants::visit(const ASTFunction & function, Data & data) -{ - /// Replace expressions like "dictGet(currentDatabase() || '.dict', 'value', toUInt32(1))" - /// with "dictGet('db_name.dict', 'value', toUInt32(1))" - ssize_t table_name_arg_idx = getPositionOfTableNameArgument(function); - if (table_name_arg_idx < 0) - return; - - if (!function.arguments || function.arguments->children.size() <= static_cast(table_name_arg_idx)) - return; - - auto & arg = function.arguments->as().children[table_name_arg_idx]; - if (arg->as()) - arg = evaluateConstantExpressionAsLiteral(arg, data.create_query_context); -} - - -void NormalizeAndEvaluateConstants::visit(const ASTFunctionWithKeyValueArguments & dict_source, Data & data) -{ - if (!dict_source.elements) - return; - - auto & expr_list = dict_source.elements->as(); - for (auto & child : expr_list.children) - { - ASTPair * pair = child->as(); - if (pair->second->as()) - { - auto ast_literal = evaluateConstantExpressionAsLiteral(pair->children[0], data.create_query_context); - pair->replace(pair->second, ast_literal); - } - } -} - } diff --git a/src/Databases/DDLDependencyVisitor.h b/src/Databases/DDLDependencyVisitor.h index d23a7a697a9..9709eeec9d3 100644 --- a/src/Databases/DDLDependencyVisitor.h +++ b/src/Databases/DDLDependencyVisitor.h @@ -1,72 +1,36 @@ #pragma once -#include + #include #include +#include + namespace DB { - -class ASTFunction; -class ASTFunctionWithKeyValueArguments; -class ASTStorage; - using TableNamesSet = std::unordered_set; -TableNamesSet getDependenciesSetFromCreateQuery(ContextPtr global_context, const QualifiedTableName & table, const ASTPtr & ast); - - -class DDLMatcherBase -{ -public: - static bool needChildVisit(const ASTPtr & node, const ASTPtr & child); - static ssize_t getPositionOfTableNameArgument(const ASTFunction & function); -}; - -/// Visits ASTCreateQuery and extracts names of table (or dictionary) dependencies -/// from column default expressions (joinGet, dictGet, etc) -/// or dictionary source (for dictionaries from local ClickHouse table). +/// Returns a list of all tables explicitly referenced in the create query of a specified table. +/// For example, a column default expression can use dictGet() and thus reference a dictionary. /// Does not validate AST, works a best-effort way. -class DDLDependencyVisitor : public DDLMatcherBase +TableNamesSet getDependenciesFromCreateQuery(const ContextPtr & global_context, const QualifiedTableName & table_name, const ASTPtr & ast); + +/// Visits ASTCreateQuery and extracts the names of all tables explicitly referenced in the create query. +class DDLDependencyVisitor { public: struct Data { - String default_database; - TableNamesSet dependencies; - ContextPtr global_context; ASTPtr create_query; + QualifiedTableName table_name; + String default_database; + ContextPtr global_context; + TableNamesSet dependencies; }; - using Visitor = ConstInDepthNodeVisitor; + using Visitor = ConstInDepthNodeVisitor; static void visit(const ASTPtr & ast, Data & data); - -private: - static void visit(const ASTFunction & function, Data & data); - static void visit(const ASTFunctionWithKeyValueArguments & dict_source, Data & data); - static void visit(const ASTStorage & storage, Data & data); - - static void extractTableNameFromArgument(const ASTFunction & function, Data & data, size_t arg_idx); + static bool needChildVisit(const ASTPtr & node, const ASTPtr & child); }; -class NormalizeAndEvaluateConstants : public DDLMatcherBase -{ -public: - struct Data - { - ContextPtr create_query_context; - }; - - using Visitor = ConstInDepthNodeVisitor; - - static void visit(const ASTPtr & ast, Data & data); - -private: - static void visit(const ASTFunction & function, Data & data); - static void visit(const ASTFunctionWithKeyValueArguments & dict_source, Data & data); - -}; - -using NormalizeAndEvaluateConstantsVisitor = NormalizeAndEvaluateConstants::Visitor; - } diff --git a/src/Databases/DDLLoadingDependencyVisitor.cpp b/src/Databases/DDLLoadingDependencyVisitor.cpp new file mode 100644 index 00000000000..8536d1c890d --- /dev/null +++ b/src/Databases/DDLLoadingDependencyVisitor.cpp @@ -0,0 +1,152 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +using TableLoadingDependenciesVisitor = DDLLoadingDependencyVisitor::Visitor; + +TableNamesSet getLoadingDependenciesFromCreateQuery(ContextPtr global_context, const QualifiedTableName & table, const ASTPtr & ast) +{ + assert(global_context == global_context->getGlobalContext()); + TableLoadingDependenciesVisitor::Data data; + data.default_database = global_context->getCurrentDatabase(); + data.create_query = ast; + data.global_context = global_context; + TableLoadingDependenciesVisitor visitor{data}; + visitor.visit(ast); + data.dependencies.erase(table); + return data.dependencies; +} + +void DDLLoadingDependencyVisitor::visit(const ASTPtr & ast, Data & data) +{ + /// Looking for functions in column default expressions and dictionary source definition + if (const auto * function = ast->as()) + visit(*function, data); + else if (const auto * dict_source = ast->as()) + visit(*dict_source, data); + else if (const auto * storage = ast->as()) + visit(*storage, data); +} + +bool DDLMatcherBase::needChildVisit(const ASTPtr & node, const ASTPtr & child) +{ + if (node->as()) + return false; + + if (auto * create = node->as()) + { + if (child.get() == create->select) + return false; + } + + return true; +} + +ssize_t DDLMatcherBase::getPositionOfTableNameArgument(const ASTFunction & function) +{ + if (function.name == "joinGet" || + function.name == "dictHas" || + function.name == "dictIsIn" || + function.name.starts_with("dictGet")) + return 0; + + if (Poco::toLower(function.name) == "in") + return 1; + + return -1; +} + +void DDLLoadingDependencyVisitor::visit(const ASTFunction & function, Data & data) +{ + ssize_t table_name_arg_idx = getPositionOfTableNameArgument(function); + if (table_name_arg_idx < 0) + return; + extractTableNameFromArgument(function, data, table_name_arg_idx); +} + +void DDLLoadingDependencyVisitor::visit(const ASTFunctionWithKeyValueArguments & dict_source, Data & data) +{ + if (dict_source.name != "clickhouse") + return; + if (!dict_source.elements) + return; + + auto config = getDictionaryConfigurationFromAST(data.create_query->as(), data.global_context); + auto info = getInfoIfClickHouseDictionarySource(config, data.global_context); + + if (!info || !info->is_local) + return; + + if (info->table_name.database.empty()) + info->table_name.database = data.default_database; + data.dependencies.emplace(std::move(info->table_name)); +} + +void DDLLoadingDependencyVisitor::visit(const ASTStorage & storage, Data & data) +{ + if (!storage.engine) + return; + if (storage.engine->name != "Dictionary") + return; + + extractTableNameFromArgument(*storage.engine, data, 0); +} + + +void DDLLoadingDependencyVisitor::extractTableNameFromArgument(const ASTFunction & function, Data & data, size_t arg_idx) +{ + /// Just ignore incorrect arguments, proper exception will be thrown later + if (!function.arguments || function.arguments->children.size() <= arg_idx) + return; + + QualifiedTableName qualified_name; + + const auto * arg = function.arguments->as()->children[arg_idx].get(); + if (const auto * literal = arg->as()) + { + if (literal->value.getType() != Field::Types::String) + return; + + auto maybe_qualified_name = QualifiedTableName::tryParseFromString(literal->value.get()); + /// Just return if name if invalid + if (!maybe_qualified_name) + return; + + qualified_name = std::move(*maybe_qualified_name); + } + else if (const auto * identifier = dynamic_cast(arg)) + { + /// ASTIdentifier or ASTTableIdentifier + auto table_identifier = identifier->createTable(); + /// Just return if table identified is invalid + if (!table_identifier) + return; + + qualified_name.database = table_identifier->getDatabaseName(); + qualified_name.table = table_identifier->shortName(); + } + else + { + assert(false); + return; + } + + if (qualified_name.database.empty()) + { + /// It can be table/dictionary from default database or XML dictionary, but we cannot distinguish it here. + qualified_name.database = data.default_database; + } + data.dependencies.emplace(std::move(qualified_name)); +} + +} diff --git a/src/Databases/DDLLoadingDependencyVisitor.h b/src/Databases/DDLLoadingDependencyVisitor.h new file mode 100644 index 00000000000..f987e885266 --- /dev/null +++ b/src/Databases/DDLLoadingDependencyVisitor.h @@ -0,0 +1,54 @@ +#pragma once +#include +#include +#include + + +namespace DB +{ + +class ASTFunction; +class ASTFunctionWithKeyValueArguments; +class ASTStorage; + +using TableNamesSet = std::unordered_set; + +/// Returns a list of all tables which should be loaded before a specified table. +/// For example, a local ClickHouse table should be loaded before a dictionary which uses that table as its source. +/// Does not validate AST, works a best-effort way. +TableNamesSet getLoadingDependenciesFromCreateQuery(ContextPtr global_context, const QualifiedTableName & table, const ASTPtr & ast); + + +class DDLMatcherBase +{ +public: + static bool needChildVisit(const ASTPtr & node, const ASTPtr & child); + static ssize_t getPositionOfTableNameArgument(const ASTFunction & function); +}; + +/// Visits ASTCreateQuery and extracts the names of all tables which should be loaded before a specified table. +/// TODO: Combine this class with DDLDependencyVisitor (because loading dependencies are a subset of referential dependencies). +class DDLLoadingDependencyVisitor : public DDLMatcherBase +{ +public: + struct Data + { + String default_database; + TableNamesSet dependencies; + ContextPtr global_context; + ASTPtr create_query; + }; + + using Visitor = ConstInDepthNodeVisitor; + + static void visit(const ASTPtr & ast, Data & data); + +private: + static void visit(const ASTFunction & function, Data & data); + static void visit(const ASTFunctionWithKeyValueArguments & dict_source, Data & data); + static void visit(const ASTStorage & storage, Data & data); + + static void extractTableNameFromArgument(const ASTFunction & function, Data & data, size_t arg_idx); +}; + +} diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index 99d88597385..39295bf499a 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include @@ -142,8 +142,9 @@ void DatabaseMemory::alterTable(ContextPtr local_context, const StorageID & tabl throw Exception(ErrorCodes::UNKNOWN_TABLE, "Cannot alter: There is no metadata of table {}", table_id.getNameForLogs()); applyMetadataChangesToCreateQuery(it->second, metadata); - TableNamesSet new_dependencies = getDependenciesSetFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), it->second); - DatabaseCatalog::instance().updateLoadingDependencies(table_id, std::move(new_dependencies)); + + auto new_dependencies = getLoadingDependenciesFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), it->second); + DatabaseCatalog::instance().updateDependencies(table_id, new_dependencies); } std::vector> DatabaseMemory::getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 017199fe44a..01c6e5c8d8c 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include @@ -205,21 +205,9 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables } QualifiedTableName qualified_name{TSA_SUPPRESS_WARNING_FOR_READ(database_name), create_query->getTable()}; - TableNamesSet loading_dependencies = getDependenciesSetFromCreateQuery(getContext(), qualified_name, ast); std::lock_guard lock{metadata.mutex}; metadata.parsed_tables[qualified_name] = ParsedTableMetadata{full_path.string(), ast}; - if (loading_dependencies.empty()) - { - metadata.independent_database_objects.emplace_back(std::move(qualified_name)); - } - else - { - for (const auto & dependency : loading_dependencies) - metadata.dependencies_info[dependency].dependent_database_objects.insert(qualified_name); - assert(metadata.dependencies_info[qualified_name].dependencies.empty()); - metadata.dependencies_info[qualified_name].dependencies = std::move(loading_dependencies); - } metadata.total_dictionaries += create_query->is_dictionary; } } @@ -321,8 +309,8 @@ void DatabaseOrdinary::alterTable(ContextPtr local_context, const StorageID & ta out.close(); } - TableNamesSet new_dependencies = getDependenciesSetFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), ast); - DatabaseCatalog::instance().updateLoadingDependencies(table_id, std::move(new_dependencies)); + auto new_dependencies = getLoadingDependenciesFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), ast); + DatabaseCatalog::instance().updateDependencies(table_id, new_dependencies); commitAlterTable(table_id, table_metadata_tmp_path, table_metadata_path, statement, local_context); } diff --git a/src/Databases/NormalizeAndEvaluateConstantsVisitor.cpp b/src/Databases/NormalizeAndEvaluateConstantsVisitor.cpp new file mode 100644 index 00000000000..d9e494e7c9a --- /dev/null +++ b/src/Databases/NormalizeAndEvaluateConstantsVisitor.cpp @@ -0,0 +1,56 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +void NormalizeAndEvaluateConstants::visit(const ASTPtr & ast, Data & data) +{ + assert(data.create_query_context->hasQueryContext()); + + /// Looking for functions in column default expressions and dictionary source definition + if (const auto * function = ast->as()) + visit(*function, data); + else if (const auto * dict_source = ast->as()) + visit(*dict_source, data); +} + +void NormalizeAndEvaluateConstants::visit(const ASTFunction & function, Data & data) +{ + /// Replace expressions like "dictGet(currentDatabase() || '.dict', 'value', toUInt32(1))" + /// with "dictGet('db_name.dict', 'value', toUInt32(1))" + ssize_t table_name_arg_idx = getPositionOfTableNameArgument(function); + if (table_name_arg_idx < 0) + return; + + if (!function.arguments || function.arguments->children.size() <= static_cast(table_name_arg_idx)) + return; + + auto & arg = function.arguments->as().children[table_name_arg_idx]; + if (arg->as()) + arg = evaluateConstantExpressionAsLiteral(arg, data.create_query_context); +} + + +void NormalizeAndEvaluateConstants::visit(const ASTFunctionWithKeyValueArguments & dict_source, Data & data) +{ + if (!dict_source.elements) + return; + + auto & expr_list = dict_source.elements->as(); + for (auto & child : expr_list.children) + { + ASTPair * pair = child->as(); + if (pair->second->as()) + { + auto ast_literal = evaluateConstantExpressionAsLiteral(pair->children[0], data.create_query_context); + pair->replace(pair->second, ast_literal); + } + } +} + +} diff --git a/src/Databases/NormalizeAndEvaluateConstantsVisitor.h b/src/Databases/NormalizeAndEvaluateConstantsVisitor.h new file mode 100644 index 00000000000..bc51ddb0601 --- /dev/null +++ b/src/Databases/NormalizeAndEvaluateConstantsVisitor.h @@ -0,0 +1,32 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +/// Evaluates constants in DDL query. +class NormalizeAndEvaluateConstants : public DDLMatcherBase +{ +public: + struct Data + { + ContextPtr create_query_context; + }; + + using Visitor = ConstInDepthNodeVisitor; + + static void visit(const ASTPtr & ast, Data & data); + +private: + static void visit(const ASTFunction & function, Data & data); + static void visit(const ASTFunctionWithKeyValueArguments & dict_source, Data & data); +}; + +using NormalizeAndEvaluateConstantsVisitor = NormalizeAndEvaluateConstants::Visitor; + +} diff --git a/src/Databases/TablesDependencyGraph.cpp b/src/Databases/TablesDependencyGraph.cpp new file mode 100644 index 00000000000..9461be7024b --- /dev/null +++ b/src/Databases/TablesDependencyGraph.cpp @@ -0,0 +1,656 @@ +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INFINITE_LOOP; +} + + +namespace +{ + constexpr const size_t CYCLIC_LEVEL = static_cast(-2); +} + + +TablesDependencyGraph::TablesDependencyGraph(const String & name_for_logging_) + : name_for_logging(name_for_logging_) +{ +} + + +TablesDependencyGraph::TablesDependencyGraph(const TablesDependencyGraph & src) + : TablesDependencyGraph(src.name_for_logging) +{ + *this = src; +} + + +TablesDependencyGraph::TablesDependencyGraph(TablesDependencyGraph && src) noexcept + : TablesDependencyGraph(src.name_for_logging) +{ + *this = std::move(src); +} + + +TablesDependencyGraph & TablesDependencyGraph::operator=(const TablesDependencyGraph & src) +{ + nodes = src.nodes; + nodes_by_database_and_table_names = src.nodes_by_database_and_table_names; + nodes_by_uuid = src.nodes_by_uuid; + levels_calculated = src.levels_calculated; + nodes_sorted_by_level_lazy = src.nodes_sorted_by_level_lazy; + return *this; +} + + +TablesDependencyGraph & TablesDependencyGraph::operator=(TablesDependencyGraph && src) noexcept +{ + nodes = std::exchange(src.nodes, decltype(nodes){}); + nodes_by_database_and_table_names = std::exchange(src.nodes_by_database_and_table_names, decltype(nodes_by_database_and_table_names){}); + nodes_by_uuid = std::exchange(src.nodes_by_uuid, decltype(nodes_by_uuid){}); + levels_calculated = std::exchange(src.levels_calculated, false); + nodes_sorted_by_level_lazy = std::exchange(src.nodes_sorted_by_level_lazy, decltype(nodes_sorted_by_level_lazy){}); + return *this; +} + + +void TablesDependencyGraph::clear() +{ + nodes.clear(); + nodes_by_database_and_table_names.clear(); + nodes_by_uuid.clear(); + setNeedRecalculateLevels(); +} + + +bool TablesDependencyGraph::empty() const +{ + return nodes.empty(); +} + + +size_t TablesDependencyGraph::getNumberOfTables() const +{ + return nodes.size(); +} + + +void TablesDependencyGraph::addDependency(const StorageID & table_id, const StorageID & dependency) +{ + auto * table_node = addOrUpdateNode(table_id); + auto * dependency_node = addOrUpdateNode(dependency); + + if (table_node->dependencies.contains(dependency_node)) + return; /// Already have this dependency. + + table_node->dependencies.insert(dependency_node); + dependency_node->dependents.insert(table_node); + + setNeedRecalculateLevels(); +} + + +void TablesDependencyGraph::addDependencies(const StorageID & table_id, const std::vector & dependencies) +{ + auto * table_node = addOrUpdateNode(table_id); + + std::unordered_set new_dependency_nodes; + for (const auto & dependency : dependencies) + new_dependency_nodes.emplace(addOrUpdateNode(dependency)); + + if (table_node->dependencies == new_dependency_nodes) + return; + + auto old_dependencies = getDependencies(*table_node); + auto old_dependency_nodes = std::move(table_node->dependencies); + + if (!old_dependencies.empty()) + { + LOG_WARNING( + getLogger(), + "Replacing outdated dependencies ({}) of {} with: {}", + fmt::join(old_dependencies, ", "), + table_id, + fmt::join(dependencies, ", ")); + } + + for (auto * dependency_node : old_dependency_nodes) + { + if (!new_dependency_nodes.contains(dependency_node)) + dependency_node->dependents.erase(table_node); + } + + for (auto * dependency_node : new_dependency_nodes) + { + if (!old_dependency_nodes.contains(dependency_node)) + dependency_node->dependents.insert(table_node); + } + + table_node->dependencies = std::move(new_dependency_nodes); + setNeedRecalculateLevels(); +} + + +void TablesDependencyGraph::addDependencies(const StorageID & table_id, const TableNamesSet & dependencies) +{ + std::vector converted_dependencies; + for (const auto & dependency : dependencies) + converted_dependencies.emplace_back(StorageID{dependency}); + addDependencies(table_id, converted_dependencies); +} + + +void TablesDependencyGraph::addDependencies(const QualifiedTableName & table_name, const TableNamesSet & dependencies) +{ + addDependencies(StorageID{table_name}, dependencies); +} + + +bool TablesDependencyGraph::removeDependency(const StorageID & table_id, const StorageID & dependency, bool remove_isolated_tables) +{ + auto * table_node = findNode(table_id); + if (!table_node) + return false; + + auto * dependency_node = findNode(dependency); + if (!dependency_node) + return false; + + auto dependency_it = table_node->dependencies.find(dependency_node); + if (dependency_it == table_node->dependencies.end()) + return false; + + table_node->dependencies.erase(dependency_it); + dependency_node->dependents.erase(table_node); + bool table_node_removed = false; + + if (remove_isolated_tables && dependency_node->dependencies.empty() && dependency_node->dependents.empty()) + { + removeNode(dependency_node); + if (table_node == dependency_node) + table_node_removed = true; + } + + if (remove_isolated_tables && !table_node_removed && table_node->dependencies.empty() && table_node->dependents.empty()) + removeNode(table_node); + + setNeedRecalculateLevels(); + return true; +} + + +std::vector TablesDependencyGraph::removeDependencies(const StorageID & table_id, bool remove_isolated_tables) +{ + auto * table_node = findNode(table_id); + if (!table_node) + return {}; + + auto dependency_nodes = std::move(table_node->dependencies); + table_node->dependencies.clear(); + bool table_node_removed = false; + + std::vector dependencies; + dependencies.reserve(dependency_nodes.size()); + + for (auto * dependency_node : dependency_nodes) + { + dependencies.emplace_back(dependency_node->storage_id); + dependency_node->dependents.erase(table_node); + + if (remove_isolated_tables && dependency_node->dependencies.empty() && dependency_node->dependents.empty()) + { + removeNode(dependency_node); + if (table_node == dependency_node) + table_node_removed = true; + } + } + + if (remove_isolated_tables && !table_node_removed && table_node->dependencies.empty() && table_node->dependents.empty()) + removeNode(table_node); + + setNeedRecalculateLevels(); + return dependencies; +} + + +bool TablesDependencyGraph::removeTable(const StorageID & table_id) +{ + auto * table_node = findNode(table_id); + if (!table_node) + return false; + + removeNode(table_node); + + setNeedRecalculateLevels(); + return true; +} + + +TablesDependencyGraph::Node * TablesDependencyGraph::findNode(const StorageID & table_id) const +{ + table_id.assertNotEmpty(); + if (table_id.hasUUID()) + { + auto it = nodes_by_uuid.find(table_id.uuid); + if (it != nodes_by_uuid.end()) + return it->second; /// Found by UUID. + } + if (!table_id.table_name.empty()) + { + auto it = nodes_by_database_and_table_names.find(table_id); + if (it != nodes_by_database_and_table_names.end()) + { + auto * node = it->second; + if (table_id.hasUUID() && node->storage_id.hasUUID() && (table_id.uuid != node->storage_id.uuid)) + return nullptr; /// UUID is different, it's not the node we're looking for. + return node; /// Found by table name. + } + } + return nullptr; /// Not found. +} + + +TablesDependencyGraph::Node * TablesDependencyGraph::addOrUpdateNode(const StorageID & table_id) +{ + auto * node = findNode(table_id); + if (node) + { + /// Node has been found, maybe we can update the information in the graph with new table_name or new UUID. + if (table_id.hasUUID() && !node->storage_id.hasUUID()) + { + node->storage_id.uuid = table_id.uuid; + nodes_by_uuid.emplace(node->storage_id.uuid, node); + } + + if (!table_id.table_name.empty() && ((table_id.table_name != node->storage_id.table_name) || (table_id.database_name != node->storage_id.database_name))) + { + auto it = nodes_by_database_and_table_names.find(table_id); + if (it != nodes_by_database_and_table_names.end()) + { + LOG_WARNING(getLogger(), "Name conflict in the graph having tables {} and {} while adding table {}. Will remove {} from the graph", + node->storage_id, it->second->storage_id, table_id, it->second->storage_id); + removeNode(it->second); + } + nodes_by_database_and_table_names.erase(node->storage_id); + node->storage_id.database_name = table_id.database_name; + node->storage_id.table_name = table_id.table_name; + nodes_by_database_and_table_names.emplace(node->storage_id, node); + } + } + else + { + /// Node has not been found by UUID or table name. + if (!table_id.table_name.empty()) + { + auto it = nodes_by_database_and_table_names.find(table_id); + if (it != nodes_by_database_and_table_names.end()) + { + LOG_WARNING(getLogger(), "Name conflict in the graph having table {} while adding table {}. Will remove {} from the graph", + it->second->storage_id, table_id, it->second->storage_id); + removeNode(it->second); + } + } + auto node_ptr = std::make_shared(table_id); + nodes.insert(node_ptr); + node = node_ptr.get(); + if (table_id.hasUUID()) + nodes_by_uuid.emplace(table_id.uuid, node); + if (!table_id.table_name.empty()) + nodes_by_database_and_table_names.emplace(table_id, node); + } + return node; +} + + +void TablesDependencyGraph::removeNode(Node * node) +{ + auto dependency_nodes = std::move(node->dependencies); + auto dependent_nodes = std::move(node->dependents); + + if (node->storage_id.hasUUID()) + nodes_by_uuid.erase(node->storage_id.uuid); + + if (!node->storage_id.table_name.empty()) + nodes_by_database_and_table_names.erase(node->storage_id); + + for (auto * dependency_node : dependency_nodes) + dependency_node->dependents.erase(node); + + for (auto * dependent_node : dependent_nodes) + dependent_node->dependencies.erase(node); + + nodes.erase(node->shared_from_this()); +} + + +size_t TablesDependencyGraph::removeTablesIf(const std::function & function) +{ + size_t num_removed = 0; + + auto it = nodes.begin(); + while (it != nodes.end()) + { + auto * current = (it++)->get(); + if (function(current->storage_id)) + { + StorageID storage_id = current->storage_id; + removeNode(current); + ++num_removed; + } + } + + if (num_removed) + setNeedRecalculateLevels(); + + return num_removed; +} + + +size_t TablesDependencyGraph::removeIsolatedTables() +{ + size_t num_removed = 0; + auto it = nodes.begin(); + while (it != nodes.end()) + { + auto current = (it++)->get(); + if (current->dependencies.empty() && current->dependents.empty()) + { + removeNode(current); + ++num_removed; + } + } + + if (num_removed) + setNeedRecalculateLevels(); + + return num_removed; +} + + +std::vector TablesDependencyGraph::getTables() const +{ + std::vector res; + res.reserve(nodes.size()); + for (const auto & node : nodes) + res.emplace_back(node->storage_id); + return res; +} + + +void TablesDependencyGraph::mergeWith(const TablesDependencyGraph & other) +{ + for (const auto & other_node : other.nodes) + addDependencies(other_node->storage_id, other.getDependencies(*other_node)); +} + + +std::vector TablesDependencyGraph::getDependencies(const StorageID & table_id) const +{ + const auto * node = findNode(table_id); + if (!node) + return {}; + return getDependencies(*node); +} + + +std::vector TablesDependencyGraph::getDependencies(const Node & node) +{ + std::vector res; + res.reserve(node.dependencies.size()); + for (const auto * dependency_node : node.dependencies) + res.emplace_back(dependency_node->storage_id); + return res; +} + +size_t TablesDependencyGraph::getNumberOfDependencies(const StorageID & table_id) const +{ + const auto * node = findNode(table_id); + if (!node) + return 0; + return node->dependencies.size(); +} + + +std::vector TablesDependencyGraph::getDependents(const StorageID & table_id) const +{ + const auto * node = findNode(table_id); + if (!node) + return {}; + return getDependents(*node); +} + + +std::vector TablesDependencyGraph::getDependents(const Node & node) +{ + std::vector res; + res.reserve(node.dependents.size()); + for (const auto * dependent_node : node.dependents) + res.emplace_back(dependent_node->storage_id); + return res; +} + + +size_t TablesDependencyGraph::getNumberOfDependents(const StorageID & table_id) const +{ + const auto * node = findNode(table_id); + if (!node) + return 0; + return node->dependents.size(); +} + + +void TablesDependencyGraph::getNumberOfAdjacents(const StorageID & table_id, size_t & num_dependencies, size_t & num_dependents) const +{ + num_dependencies = 0; + num_dependents = 0; + + const auto * node = findNode(table_id); + if (!node) + return; + + num_dependencies = node->dependencies.size(); + num_dependents = node->dependents.size(); +} + + +bool TablesDependencyGraph::isIsolatedTable(const StorageID & table_id) const +{ + const auto * node = findNode(table_id); + if (!node) + return false; + + return node->dependencies.empty() && node->dependents.empty(); +} + + +void TablesDependencyGraph::checkNoCyclicDependencies() const +{ + if (hasCyclicDependencies()) + { + throw Exception( + ErrorCodes::INFINITE_LOOP, + "{}: Tables {} have cyclic dependencies: {}", + name_for_logging, + fmt::join(getTablesWithCyclicDependencies(), ", "), + describeCyclicDependencies()); + } +} + + +bool TablesDependencyGraph::hasCyclicDependencies() const +{ + const auto & nodes_sorted_by_level = getNodesSortedByLevel(); + return !nodes_sorted_by_level.empty() && (nodes_sorted_by_level.back()->level == CYCLIC_LEVEL); +} + + +std::vector TablesDependencyGraph::getTablesWithCyclicDependencies() const +{ + std::vector res; + for (const auto * node : getNodesSortedByLevel() | boost::adaptors::reversed) + { + if (node->level != CYCLIC_LEVEL) + break; + res.emplace_back(node->storage_id); + } + return res; +} + + +String TablesDependencyGraph::describeCyclicDependencies() const +{ + String res; + for (const auto * node : getNodesSortedByLevel() | boost::adaptors::reversed) + { + if (node->level != CYCLIC_LEVEL) + break; + if (!res.empty()) + res += "; "; + res += node->storage_id.getNameForLogs(); + res += " -> ["; + bool need_comma = false; + for (const auto * dependency_node : node->dependencies) + { + if (dependency_node->level != CYCLIC_LEVEL) + continue; + if (need_comma) + res += ", "; + need_comma = true; + res += dependency_node->storage_id.getNameForLogs(); + } + res += "]"; + } + return res; +} + + +void TablesDependencyGraph::setNeedRecalculateLevels() +{ + levels_calculated = false; + nodes_sorted_by_level_lazy.clear(); +} + + +void TablesDependencyGraph::calculateLevels() const +{ + if (levels_calculated) + return; + levels_calculated = true; + + nodes_sorted_by_level_lazy.clear(); + nodes_sorted_by_level_lazy.reserve(nodes.size()); + + std::unordered_set nodes_to_process; + for (const auto & node_ptr : nodes) + nodes_to_process.emplace(node_ptr.get()); + + size_t current_level = 0; + + while (!nodes_to_process.empty()) + { + size_t old_num_sorted = nodes_sorted_by_level_lazy.size(); + + for (auto it = nodes_to_process.begin(); it != nodes_to_process.end();) + { + const auto * current_node = *(it++); + bool has_dependencies = false; + for (const auto * dependency : current_node->dependencies) + { + if (nodes_to_process.contains(dependency)) + has_dependencies = true; + } + + if (!has_dependencies) + { + current_node->level = current_level; + nodes_sorted_by_level_lazy.emplace_back(current_node); + } + } + + if (nodes_sorted_by_level_lazy.size() == old_num_sorted) + break; + + for (size_t i = old_num_sorted; i != nodes_sorted_by_level_lazy.size(); ++i) + nodes_to_process.erase(nodes_sorted_by_level_lazy[i]); + + ++current_level; + } + + for (const auto * node_with_cyclic_dependencies : nodes_to_process) + { + node_with_cyclic_dependencies->level = CYCLIC_LEVEL; + nodes_sorted_by_level_lazy.emplace_back(node_with_cyclic_dependencies); + } +} + + +const TablesDependencyGraph::NodesSortedByLevel & TablesDependencyGraph::getNodesSortedByLevel() const +{ + calculateLevels(); + return nodes_sorted_by_level_lazy; +} + + +std::vector TablesDependencyGraph::getTablesSortedByDependency() const +{ + std::vector res; + res.reserve(nodes.size()); + for (const auto * node : getNodesSortedByLevel()) + { + res.emplace_back(node->storage_id); + } + return res; +} + + +std::vector> TablesDependencyGraph::getTablesSortedByDependencyForParallel() const +{ + std::vector> res; + std::optional last_level; + for (const auto * node : getNodesSortedByLevel()) + { + if (node->level != last_level) + res.emplace_back(); + auto & table_ids = res.back(); + table_ids.emplace_back(node->storage_id); + last_level = node->level; + } + return res; +} + + +void TablesDependencyGraph::log() const +{ + if (empty()) + { + LOG_TEST(getLogger(), "No tables"); + return; + } + + for (const auto * node : getNodesSortedByLevel()) + { + String dependencies_desc = node->dependencies.empty() + ? "no dependencies" + : fmt::format("{} dependencies: {}", node->dependencies.size(), fmt::join(getDependencies(*node), ", ")); + + String level_desc = (node->level == CYCLIC_LEVEL) ? "cyclic" : fmt::format("level {}", node->level); + + LOG_TEST(getLogger(), "Table {} has {} ({})", node->storage_id, dependencies_desc, level_desc); + } +} + + +Poco::Logger * TablesDependencyGraph::getLogger() const +{ + if (!logger) + logger = &Poco::Logger::get(name_for_logging); + return logger; +} + +} diff --git a/src/Databases/TablesDependencyGraph.h b/src/Databases/TablesDependencyGraph.h new file mode 100644 index 00000000000..0d60857dea8 --- /dev/null +++ b/src/Databases/TablesDependencyGraph.h @@ -0,0 +1,171 @@ +#pragma once + +#include + +#include +#include + + +namespace DB +{ +using TableNamesSet = std::unordered_set; + +/// Represents dependencies of some tables on other tables or dictionaries. +/// +/// NOTES: A "dependent" depends on its "dependency". For example, if table "A" depends on table "B", then +/// "B" is a dependency for "A", and "A" is a dependent for "B". +/// +/// Dependencies can be added to the graph in any order. For example, if table "A" depends on "B", and "B" depends on "C", then +/// it's allowed to add first "A->B" and then "B->C", or first "B->C" and then "A->B", the resulting graph will be the same. +/// +/// This class is used to represent various types of table-table dependencies: +/// 1. View dependencies: "source_table -> materialized_view". +/// Data inserted to a source table is also inserted to corresponding materialized views. +/// 2. Loading dependencies: specify in which order tables must be loaded during startup. +/// For example a dictionary should be loaded after it's source table and it's written in the graph as "dictionary -> source_table". +/// 3. Referential dependencies: "table -> all tables mentioned in its definition". +/// Referential dependencies are checked to decide if it's safe to drop a table (it can be unsafe if the table is used by another table). +/// +/// WARNING: This class doesn't have an embedded mutex, so it must be synchronized outside. +class TablesDependencyGraph +{ +public: + explicit TablesDependencyGraph(const String & name_for_logging_); + + TablesDependencyGraph(const TablesDependencyGraph & src); + TablesDependencyGraph(TablesDependencyGraph && src) noexcept; + TablesDependencyGraph & operator=(const TablesDependencyGraph & src); + TablesDependencyGraph & operator=(TablesDependencyGraph && src) noexcept; + + /// The dependency graph is empty if doesn't contain any tables. + bool empty() const; + + /// Clears this dependency graph. + void clear(); + + /// Adds a single dependency "table_id" on "dependency". + void addDependency(const StorageID & table_id, const StorageID & dependency); + + /// Adds a table with specified dependencies if there are no dependencies of the table in the graph yet; + /// otherwise it replaces the dependencies of the table in the graph and shows a warning. + void addDependencies(const StorageID & table_id, const std::vector & dependencies); + void addDependencies(const StorageID & table_id, const TableNamesSet & dependencies); + void addDependencies(const QualifiedTableName & table_name, const TableNamesSet & dependencies); + + /// Removes a single dependency of "table_id" on "dependency". + /// If "remove_isolated_tables" is set the function will also remove tables with no dependencies and no dependents + /// from the graph. + bool removeDependency(const StorageID & table_id, const StorageID & dependency, bool remove_isolated_tables = false); + + /// Removes all dependencies of "table_id", returns those dependencies. + std::vector removeDependencies(const StorageID & table_id, bool remove_isolated_tables = false); + + /// Removes a table from the graph and removes all references to in from the graph (both from its dependencies and dependents). + bool removeTable(const StorageID & table_id); + + /// Removes tables from the graph by a specified filter. + size_t removeTablesIf(const std::function & function); + + /// Removes tables with no dependencies and no dependents from the graph. + size_t removeIsolatedTables(); + + /// Returns the number of tables in the graph. + size_t getNumberOfTables() const; + + /// Returns a list of all tables in the graph. + std::vector getTables() const; + + /// Adds tables and dependencies with another graph. + void mergeWith(const TablesDependencyGraph & other); + + /// Returns a list of dependencies of a specified table. + std::vector getDependencies(const StorageID & table_id) const; + size_t getNumberOfDependencies(const StorageID & table_id) const; + bool hasDependencies(const StorageID & table_id) const { return getNumberOfDependencies(table_id) != 0; } + + /// Returns a list of dependents of a specified table. + std::vector getDependents(const StorageID & table_id) const; + size_t getNumberOfDependents(const StorageID & table_id) const; + bool hasDependents(const StorageID & table_id) const { return getNumberOfDependents(table_id) != 0; } + + /// Returns the number of dependencies and the number of dependents of a specified table. + void getNumberOfAdjacents(const StorageID & table_id, size_t & num_dependencies, size_t & num_dependents) const; + + /// Returns true if a specified table has no dependencies and no dependents. + bool isIsolatedTable(const StorageID & table_id) const; + + /// Checks that there are no cyclic dependencies in the graph. + /// Cyclic dependencies are dependencies like "A->A" or "A->B->C->D->A". + void checkNoCyclicDependencies() const; + bool hasCyclicDependencies() const; + std::vector getTablesWithCyclicDependencies() const; + String describeCyclicDependencies() const; + + /// Returns a list of tables sorted by their dependencies: + /// tables without dependencies first, then + /// tables which depend on the tables without dependencies, then + /// tables which depend on the tables which depend on the tables without dependencies, and so on. + std::vector getTablesSortedByDependency() const; + + /// The same as getTablesSortedByDependency() but make a list for parallel processing. + std::vector> getTablesSortedByDependencyForParallel() const; + + /// Outputs information about this graph as a bunch of logging messages. + void log() const; + +private: + struct Node : public std::enable_shared_from_this + { + StorageID storage_id; + + /// If A depends on B then "A.dependencies" contains "B". + std::unordered_set dependencies; + + /// If A depends on B then "B.dependents" contains "A". + std::unordered_set dependents; + + /// Tables without dependencies have level == 0, tables which depend on the tables without dependencies have level == 1, and so on. + /// Calculated lazily. + mutable size_t level = 0; + + explicit Node(const StorageID & storage_id_) : storage_id(storage_id_) {} + }; + + using NodeSharedPtr = std::shared_ptr; + + struct LessByLevel + { + bool operator()(const Node * left, const Node * right) { return left->level < right->level; } + }; + + std::unordered_set nodes; + + /// Nodes can be found either by UUID or by database name & table name. That's why we need two maps here. + std::unordered_map nodes_by_database_and_table_names; + std::unordered_map nodes_by_uuid; + + /// This is set if both `level` inside each node and `nodes_sorted_by_level_lazy` are calculated. + mutable bool levels_calculated = false; + + /// Nodes sorted by their level. Calculated lazily. + using NodesSortedByLevel = std::vector; + mutable NodesSortedByLevel nodes_sorted_by_level_lazy; + + const String name_for_logging; + mutable Poco::Logger * logger = nullptr; + + Node * findNode(const StorageID & table_id) const; + Node * addOrUpdateNode(const StorageID & table_id); + void removeNode(Node * node); + + static std::vector getDependencies(const Node & node); + static std::vector getDependents(const Node & node); + + void setNeedRecalculateLevels(); + void calculateLevels() const; + const NodesSortedByLevel & getNodesSortedByLevel() const; + + Poco::Logger * getLogger() const; +}; + +} diff --git a/src/Databases/TablesLoader.cpp b/src/Databases/TablesLoader.cpp index 1114206d469..fbb5b1f17d9 100644 --- a/src/Databases/TablesLoader.cpp +++ b/src/Databases/TablesLoader.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -14,45 +15,12 @@ namespace DB namespace ErrorCodes { - extern const int INFINITE_LOOP; extern const int LOGICAL_ERROR; } static constexpr size_t PRINT_MESSAGE_EACH_N_OBJECTS = 256; static constexpr size_t PRINT_MESSAGE_EACH_N_SECONDS = 5; -void mergeDependenciesGraphs(DependenciesInfos & main_dependencies_info, const DependenciesInfos & additional_info) -{ - for (const auto & table_and_info : additional_info) - { - const QualifiedTableName & table = table_and_info.first; - const TableNamesSet & dependent_tables = table_and_info.second.dependent_database_objects; - const TableNamesSet & dependencies = table_and_info.second.dependencies; - - DependenciesInfo & maybe_existing_info = main_dependencies_info[table]; - maybe_existing_info.dependent_database_objects.insert(dependent_tables.begin(), dependent_tables.end()); - if (!dependencies.empty()) - { - if (maybe_existing_info.dependencies.empty()) - maybe_existing_info.dependencies = dependencies; - else if (maybe_existing_info.dependencies != dependencies) - { - /// Can happen on DatabaseReplicated recovery - LOG_WARNING(&Poco::Logger::get("TablesLoader"), "Replacing outdated dependencies ({}) of {} with: {}", - fmt::join(maybe_existing_info.dependencies, ", "), - table, - fmt::join(dependencies, ", ")); - for (const auto & old_dependency : maybe_existing_info.dependencies) - { - [[maybe_unused]] bool removed = main_dependencies_info[old_dependency].dependent_database_objects.erase(table); - assert(removed); - } - maybe_existing_info.dependencies = dependencies; - } - } - } -} - void logAboutProgress(Poco::Logger * log, size_t processed, size_t total, AtomicStopwatch & watch) { if (processed % PRINT_MESSAGE_EACH_N_OBJECTS == 0 || watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS)) @@ -66,6 +34,8 @@ TablesLoader::TablesLoader(ContextMutablePtr global_context_, Databases database : global_context(global_context_) , databases(std::move(databases_)) , strictness_mode(strictness_mode_) +, referential_dependencies("ReferentialDeps") +, loading_dependencies("LoadingDeps") { metadata.default_database = global_context->getCurrentDatabase(); log = &Poco::Logger::get("TablesLoader"); @@ -101,20 +71,18 @@ void TablesLoader::loadTables() stopwatch.restart(); - logDependencyGraph(); - - /// Remove tables that do not exist - removeUnresolvableDependencies(/* remove_loaded */ false); + buildDependencyGraph(); /// Update existing info (it's important for ATTACH DATABASE) - DatabaseCatalog::instance().addLoadingDependencies(metadata.dependencies_info); + DatabaseCatalog::instance().addDependencies(referential_dependencies); - /// Some tables were loaded by database with loadStoredObjects(...). Remove them from graph if necessary. - removeUnresolvableDependencies(/* remove_loaded */ true); + /// Remove tables that do not exist + removeUnresolvableDependencies(); loadTablesInTopologicalOrder(pool); } + void TablesLoader::startupTables() { /// Startup tables after all tables are loaded. Background tasks (merges, mutations, etc) may slow down data parts loading. @@ -123,52 +91,79 @@ void TablesLoader::startupTables() } -void TablesLoader::removeUnresolvableDependencies(bool remove_loaded) +void TablesLoader::buildDependencyGraph() { - auto need_exclude_dependency = [this, remove_loaded](const QualifiedTableName & dependency_name, const DependenciesInfo & info) + for (const auto & [table_name, table_metadata] : metadata.parsed_tables) + { + auto new_loading_dependencies = getLoadingDependenciesFromCreateQuery(global_context, table_name, table_metadata.ast); + + if (!new_loading_dependencies.empty()) + referential_dependencies.addDependencies(table_name, new_loading_dependencies); + + /// We're adding `new_loading_dependencies` to the graph here even if they're empty because + /// we need to have all tables from `metadata.parsed_tables` in the graph. + loading_dependencies.addDependencies(table_name, new_loading_dependencies); + } + + referential_dependencies.log(); + loading_dependencies.log(); +} + + +void TablesLoader::removeUnresolvableDependencies() +{ + auto need_exclude_dependency = [this](const StorageID & table_id) { /// Table exists and will be loaded - if (metadata.parsed_tables.contains(dependency_name)) + if (metadata.parsed_tables.contains(table_id.getQualifiedName())) return false; - /// Table exists and it's already loaded - if (DatabaseCatalog::instance().isTableExist(StorageID(dependency_name.database, dependency_name.table), global_context)) - return remove_loaded; - /// It's XML dictionary. - if (dependency_name.database == metadata.default_database && - global_context->getExternalDictionariesLoader().has(dependency_name.table)) + + if (DatabaseCatalog::instance().isTableExist(table_id, global_context)) { - LOG_WARNING(log, "Tables {} depend on XML dictionary {}, but XML dictionaries are loaded independently." - "Consider converting it to DDL dictionary.", fmt::join(info.dependent_database_objects, ", "), dependency_name); - return true; + /// Table exists and it's already loaded + } + else if (table_id.database_name == metadata.default_database && + global_context->getExternalDictionariesLoader().has(table_id.table_name)) + { + /// Tables depend on a XML dictionary. + LOG_WARNING( + log, + "Tables {} depend on XML dictionary {}, but XML dictionaries are loaded independently." + "Consider converting it to DDL dictionary.", + fmt::join(loading_dependencies.getDependents(table_id), ", "), + table_id); + } + else + { + /// Some tables depend on table "table_id", but there is no such table in DatabaseCatalog and we don't have its metadata. + /// We will ignore it and try to load dependent tables without "table_id" + /// (but most likely dependent tables will fail to load). + LOG_WARNING( + log, + "Tables {} depend on {}, but seems like that does not exist. Will ignore it and try to load existing tables", + fmt::join(loading_dependencies.getDependents(table_id), ", "), + table_id); } - /// Some tables depends on table "dependency_name", but there is no such table in DatabaseCatalog and we don't have its metadata. - /// We will ignore it and try to load dependent tables without "dependency_name" - /// (but most likely dependent tables will fail to load). - LOG_WARNING(log, "Tables {} depend on {}, but seems like the it does not exist. Will ignore it and try to load existing tables", - fmt::join(info.dependent_database_objects, ", "), dependency_name); - - if (!info.dependencies.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {} does not exist, but we have seen its AST and found {} dependencies." - "It's a bug", dependency_name, info.dependencies.size()); - if (info.dependent_database_objects.empty()) + size_t num_dependencies, num_dependents; + loading_dependencies.getNumberOfAdjacents(table_id, num_dependencies, num_dependents); + if (num_dependencies || !num_dependents) throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {} does not have dependencies and dependent tables as it expected to." - "It's a bug", dependency_name); + "It's a bug", table_id); - return true; + return true; /// Exclude this dependency. }; - auto table_it = metadata.dependencies_info.begin(); - while (table_it != metadata.dependencies_info.end()) - { - auto & info = table_it->second; - if (need_exclude_dependency(table_it->first, info)) - table_it = removeResolvedDependency(table_it, metadata.independent_database_objects); - else - ++table_it; - } + loading_dependencies.removeTablesIf(need_exclude_dependency); + + if (loading_dependencies.getNumberOfTables() != metadata.parsed_tables.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Number of tables to be loaded is not as expected. It's a bug"); + + /// Cannot load tables with cyclic dependencies. + loading_dependencies.checkNoCyclicDependencies(); } + void TablesLoader::loadTablesInTopologicalOrder(ThreadPool & pool) { /// Compatibility setting which should be enabled by default on attach @@ -176,81 +171,25 @@ void TablesLoader::loadTablesInTopologicalOrder(ThreadPool & pool) ContextMutablePtr load_context = Context::createCopy(global_context); load_context->setSetting("cast_ipv4_ipv6_default_on_conversion_error", 1); - /// Load independent tables in parallel. - /// Then remove loaded tables from dependency graph, find tables/dictionaries that do not have unresolved dependencies anymore, - /// move them to the list of independent tables and load. - /// Repeat until we have some tables to load. - /// If we do not, then either all objects are loaded or there is cyclic dependency. - /// Complexity: O(V + E) - size_t level = 0; - do + /// Load tables in parallel. + auto tables_to_load = loading_dependencies.getTablesSortedByDependencyForParallel(); + + for (size_t level = 0; level != tables_to_load.size(); ++level) { - assert(metadata.parsed_tables.size() == tables_processed + metadata.independent_database_objects.size() + getNumberOfTablesWithDependencies()); - logDependencyGraph(); - - startLoadingIndependentTables(pool, level, load_context); - - TableNames new_independent_database_objects; - for (const auto & table_name : metadata.independent_database_objects) - { - auto info_it = metadata.dependencies_info.find(table_name); - if (info_it == metadata.dependencies_info.end()) - { - /// No tables depend on table_name and it was not even added to dependencies_info - continue; - } - removeResolvedDependency(info_it, new_independent_database_objects); - } - + startLoadingTables(pool, load_context, tables_to_load[level], level); pool.wait(); - - metadata.independent_database_objects = std::move(new_independent_database_objects); - ++level; - } while (!metadata.independent_database_objects.empty()); - - checkCyclicDependencies(); -} - -DependenciesInfosIter TablesLoader::removeResolvedDependency(const DependenciesInfosIter & info_it, TableNames & independent_database_objects) -{ - const QualifiedTableName & table_name = info_it->first; - const DependenciesInfo & info = info_it->second; - if (!info.dependencies.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {} is in list of independent tables, but dependencies count is {}." - "It's a bug", table_name, info.dependencies.size()); - if (info.dependent_database_objects.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {} does not have dependent tables. It's a bug", table_name); - - /// Decrement number of dependencies for each dependent table - for (const auto & dependent_table : info.dependent_database_objects) - { - auto & dependent_info = metadata.dependencies_info[dependent_table]; - auto & dependencies_set = dependent_info.dependencies; - if (dependencies_set.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to decrement 0 dependencies counter for {}. It's a bug", dependent_table); - if (!dependencies_set.erase(table_name)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot remove {} from dependencies set of {}, it contains only {}", - table_name, dependent_table, fmt::join(dependencies_set, ", ")); - if (dependencies_set.empty()) - { - independent_database_objects.push_back(dependent_table); - if (dependent_info.dependent_database_objects.empty()) - metadata.dependencies_info.erase(dependent_table); - } } - - return metadata.dependencies_info.erase(info_it); } -void TablesLoader::startLoadingIndependentTables(ThreadPool & pool, size_t level, ContextMutablePtr load_context) +void TablesLoader::startLoadingTables(ThreadPool & pool, ContextMutablePtr load_context, const std::vector & tables_to_load, size_t level) { size_t total_tables = metadata.parsed_tables.size(); - LOG_INFO(log, "Loading {} tables with {} dependency level", metadata.independent_database_objects.size(), level); + LOG_INFO(log, "Loading {} tables with dependency level {}", tables_to_load.size(), level); - for (const auto & table_name : metadata.independent_database_objects) + for (const auto & table_id : tables_to_load) { - pool.scheduleOrThrowOnError([this, load_context, total_tables, &table_name]() + pool.scheduleOrThrowOnError([this, load_context, total_tables, table_name = table_id.getQualifiedName()]() { const auto & path_and_query = metadata.parsed_tables[table_name]; databases[table_name.database]->loadTableFromMetadata(load_context, path_and_query.path, table_name, path_and_query.ast, strictness_mode); @@ -259,47 +198,4 @@ void TablesLoader::startLoadingIndependentTables(ThreadPool & pool, size_t level } } -size_t TablesLoader::getNumberOfTablesWithDependencies() const -{ - size_t number_of_tables_with_dependencies = 0; - for (const auto & info : metadata.dependencies_info) - if (!info.second.dependencies.empty()) - ++number_of_tables_with_dependencies; - return number_of_tables_with_dependencies; -} - -void TablesLoader::checkCyclicDependencies() const -{ - /// Loading is finished if all dependencies are resolved - if (metadata.dependencies_info.empty()) - return; - - for (const auto & info : metadata.dependencies_info) - { - LOG_WARNING(log, "Cannot resolve dependencies: Table {} have {} dependencies and {} dependent tables. List of dependent tables: {}", - info.first, info.second.dependencies.size(), - info.second.dependent_database_objects.size(), fmt::join(info.second.dependent_database_objects, ", ")); - assert(info.second.dependencies.empty()); - } - - throw Exception(ErrorCodes::INFINITE_LOOP, "Cannot attach {} tables due to cyclic dependencies. " - "See server log for details.", metadata.dependencies_info.size()); -} - -void TablesLoader::logDependencyGraph() const -{ - LOG_TEST(log, "Have {} independent tables: {}", - metadata.independent_database_objects.size(), - fmt::join(metadata.independent_database_objects, ", ")); - for (const auto & dependencies : metadata.dependencies_info) - { - LOG_TEST(log, - "Table {} have {} dependencies and {} dependent tables. List of dependent tables: {}", - dependencies.first, - dependencies.second.dependencies.size(), - dependencies.second.dependent_database_objects.size(), - fmt::join(dependencies.second.dependent_database_objects, ", ")); - } -} - } diff --git a/src/Databases/TablesLoader.h b/src/Databases/TablesLoader.h index 7a29d0e3958..13d404b96ce 100644 --- a/src/Databases/TablesLoader.h +++ b/src/Databases/TablesLoader.h @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -34,21 +35,6 @@ struct ParsedTableMetadata }; using ParsedMetadata = std::map; -using TableNames = std::vector; -using TableNamesSet = std::unordered_set; - -struct DependenciesInfo -{ - /// Set of dependencies - TableNamesSet dependencies; - /// Set of tables/dictionaries which depend on this table/dictionary - TableNamesSet dependent_database_objects; -}; - -using DependenciesInfos = std::unordered_map; -using DependenciesInfosIter = std::unordered_map::iterator; - -void mergeDependenciesGraphs(DependenciesInfos & main_dependencies_info, const DependenciesInfos & additional_info); struct ParsedTablesMetadata { @@ -59,17 +45,6 @@ struct ParsedTablesMetadata /// For logging size_t total_dictionaries = 0; - - /// List of tables/dictionaries that do not have any dependencies and can be loaded - TableNames independent_database_objects; - - /// Adjacent list of dependency graph, contains two maps - /// 2. table/dictionary name -> dependent tables/dictionaries list (adjacency list of dependencies graph). - /// 1. table/dictionary name -> dependencies of table/dictionary (adjacency list of inverted dependencies graph) - /// If table A depends on table B, then there is an edge B --> A, i.e. dependencies_info[B].dependent_database_objects contains A - /// and dependencies_info[A].dependencies contain B. - /// We need inverted graph to effectively maintain it on DDL queries that can modify the graph. - DependenciesInfos dependencies_info; }; /// Loads tables (and dictionaries) from specified databases @@ -92,25 +67,18 @@ private: Strings databases_to_load; ParsedTablesMetadata metadata; + TablesDependencyGraph referential_dependencies; + TablesDependencyGraph loading_dependencies; Poco::Logger * log; std::atomic tables_processed{0}; AtomicStopwatch stopwatch; ThreadPool pool; - void removeUnresolvableDependencies(bool remove_loaded); - + void buildDependencyGraph(); + void removeUnresolvableDependencies(); void loadTablesInTopologicalOrder(ThreadPool & pool); - - DependenciesInfosIter removeResolvedDependency(const DependenciesInfosIter & info_it, TableNames & independent_database_objects); - - void startLoadingIndependentTables(ThreadPool & pool, size_t level, ContextMutablePtr load_context); - - void checkCyclicDependencies() const; - - size_t getNumberOfTablesWithDependencies() const; - - void logDependencyGraph() const; + void startLoadingTables(ThreadPool & pool, ContextMutablePtr load_context, const std::vector & tables_to_load, size_t level); }; } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 63f321db993..6d7213e0f31 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 2118c633a0f..a76b13e5dcf 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -223,6 +223,7 @@ void DatabaseCatalog::shutdownImpl() return it != elem.map.end(); }) == uuid_map.end()); databases.clear(); + referential_dependencies.clear(); view_dependencies.clear(); } @@ -473,13 +474,8 @@ void DatabaseCatalog::updateDatabaseName(const String & old_name, const String & for (const auto & table_name : tables_in_database) { - QualifiedTableName new_table_name{new_name, table_name}; - auto dependencies = tryRemoveLoadingDependenciesUnlocked(QualifiedTableName{old_name, table_name}, /* check_dependencies */ false); - DependenciesInfos new_info; - for (const auto & dependency : dependencies) - new_info[dependency].dependent_database_objects.insert(new_table_name); - new_info[new_table_name].dependencies = std::move(dependencies); - mergeDependenciesGraphs(loading_dependencies, new_info); + auto dependencies = referential_dependencies.removeDependencies(StorageID{old_name, table_name}, /* remove_isolated_tables= */ true); + referential_dependencies.addDependencies(StorageID{new_name, table_name}, dependencies); } } @@ -648,7 +644,10 @@ bool DatabaseCatalog::hasUUIDMapping(const UUID & uuid) std::unique_ptr DatabaseCatalog::database_catalog; DatabaseCatalog::DatabaseCatalog(ContextMutablePtr global_context_) - : WithMutableContext(global_context_), log(&Poco::Logger::get("DatabaseCatalog")) + : WithMutableContext(global_context_) + , referential_dependencies{"ReferentialDeps"} + , view_dependencies{"ViewDeps"} + , log(&Poco::Logger::get("DatabaseCatalog")) { } @@ -692,39 +691,33 @@ DatabasePtr DatabaseCatalog::getDatabase(const String & database_name, ContextPt return getDatabase(resolved_database); } -void DatabaseCatalog::addDependency(const StorageID & from, const StorageID & where) +void DatabaseCatalog::addViewDependency(const StorageID & source_table_id, const StorageID & view_id) { std::lock_guard lock{databases_mutex}; - // FIXME when loading metadata storage may not know UUIDs of it's dependencies, because they are not loaded yet, - // so UUID of `from` is not used here. (same for remove, get and update) - view_dependencies[{from.getDatabaseName(), from.getTableName()}].insert(where); + view_dependencies.addDependency(source_table_id, view_id); } -void DatabaseCatalog::removeDependency(const StorageID & from, const StorageID & where) +void DatabaseCatalog::removeViewDependency(const StorageID & source_table_id, const StorageID & view_id) { std::lock_guard lock{databases_mutex}; - view_dependencies[{from.getDatabaseName(), from.getTableName()}].erase(where); + view_dependencies.removeDependency(source_table_id, view_id, /* remove_isolated_tables= */ true); } -Dependencies DatabaseCatalog::getDependencies(const StorageID & from) const +std::vector DatabaseCatalog::getDependentViews(const StorageID & source_table_id) const { std::lock_guard lock{databases_mutex}; - auto iter = view_dependencies.find({from.getDatabaseName(), from.getTableName()}); - if (iter == view_dependencies.end()) - return {}; - return Dependencies(iter->second.begin(), iter->second.end()); + return view_dependencies.getDependencies(source_table_id); } -void -DatabaseCatalog::updateDependency(const StorageID & old_from, const StorageID & old_where, const StorageID & new_from, - const StorageID & new_where) +void DatabaseCatalog::updateViewDependency(const StorageID & old_source_table_id, const StorageID & old_view_id, + const StorageID & new_source_table_id, const StorageID & new_view_id) { std::lock_guard lock{databases_mutex}; - if (!old_from.empty()) - view_dependencies[{old_from.getDatabaseName(), old_from.getTableName()}].erase(old_where); - if (!new_from.empty()) - view_dependencies[{new_from.getDatabaseName(), new_from.getTableName()}].insert(new_where); + if (!old_source_table_id.empty()) + view_dependencies.removeDependency(old_source_table_id, old_view_id, /* remove_isolated_tables= */ true); + if (!new_source_table_id.empty()) + view_dependencies.addDependency(new_source_table_id, new_view_id); } DDLGuardPtr DatabaseCatalog::getDDLGuard(const String & database, const String & table) @@ -1050,121 +1043,79 @@ void DatabaseCatalog::waitTableFinallyDropped(const UUID & uuid) }); } -void DatabaseCatalog::addLoadingDependencies(const QualifiedTableName & table, TableNamesSet && dependencies) -{ - DependenciesInfos new_info; - for (const auto & dependency : dependencies) - new_info[dependency].dependent_database_objects.insert(table); - new_info[table].dependencies = std::move(dependencies); - addLoadingDependencies(new_info); -} - -void DatabaseCatalog::addLoadingDependencies(const DependenciesInfos & new_infos) +void DatabaseCatalog::addDependencies(const StorageID & table_id, const std::vector & dependencies) { std::lock_guard lock{databases_mutex}; - mergeDependenciesGraphs(loading_dependencies, new_infos); + referential_dependencies.addDependencies(table_id, dependencies); } -DependenciesInfo DatabaseCatalog::getLoadingDependenciesInfo(const StorageID & table_id) const +void DatabaseCatalog::addDependencies(const QualifiedTableName & table_name, const TableNamesSet & dependencies) { std::lock_guard lock{databases_mutex}; - auto it = loading_dependencies.find(table_id.getQualifiedName()); - if (it == loading_dependencies.end()) - return {}; - return it->second; + referential_dependencies.addDependencies(table_name, dependencies); } -TableNamesSet DatabaseCatalog::tryRemoveLoadingDependencies(const StorageID & table_id, bool check_dependencies, bool is_drop_database) +void DatabaseCatalog::addDependencies(const TablesDependencyGraph & extra_graph) { - QualifiedTableName removing_table = table_id.getQualifiedName(); std::lock_guard lock{databases_mutex}; - return tryRemoveLoadingDependenciesUnlocked(removing_table, check_dependencies, is_drop_database); + referential_dependencies.mergeWith(extra_graph); } -TableNamesSet DatabaseCatalog::tryRemoveLoadingDependenciesUnlocked(const QualifiedTableName & removing_table, bool check_dependencies, bool is_drop_database) +std::vector DatabaseCatalog::getDependencies(const StorageID & table_id) const { - auto it = loading_dependencies.find(removing_table); - if (it == loading_dependencies.end()) - return {}; + std::lock_guard lock{databases_mutex}; + return referential_dependencies.getDependencies(table_id); +} - TableNamesSet & dependent = it->second.dependent_database_objects; - if (!dependent.empty()) - { - if (check_dependencies) - checkTableCanBeRemovedOrRenamedImpl(dependent, removing_table, is_drop_database); +std::vector DatabaseCatalog::getDependents(const StorageID & table_id) const +{ + std::lock_guard lock{databases_mutex}; + return referential_dependencies.getDependents(table_id); +} - for (const auto & table : dependent) - { - [[maybe_unused]] bool removed = loading_dependencies[table].dependencies.erase(removing_table); - assert(removed); - } - dependent.clear(); - } - - TableNamesSet dependencies = it->second.dependencies; - for (const auto & table : dependencies) - { - [[maybe_unused]] bool removed = loading_dependencies[table].dependent_database_objects.erase(removing_table); - assert(removed); - } - - loading_dependencies.erase(it); - return dependencies; +std::vector DatabaseCatalog::removeDependencies(const StorageID & table_id, bool check_dependencies, bool is_drop_database) +{ + std::lock_guard lock{databases_mutex}; + if (check_dependencies) + checkTableCanBeRemovedOrRenamedUnlocked(table_id, is_drop_database); + return referential_dependencies.removeDependencies(table_id, /* remove_isolated_tables= */ true); } void DatabaseCatalog::checkTableCanBeRemovedOrRenamed(const StorageID & table_id, bool is_drop_database) const { - QualifiedTableName removing_table = table_id.getQualifiedName(); std::lock_guard lock{databases_mutex}; - auto it = loading_dependencies.find(removing_table); - if (it == loading_dependencies.end()) - return; - - const TableNamesSet & dependent = it->second.dependent_database_objects; - checkTableCanBeRemovedOrRenamedImpl(dependent, removing_table, is_drop_database); + return checkTableCanBeRemovedOrRenamedUnlocked(table_id, is_drop_database); } -void DatabaseCatalog::checkTableCanBeRemovedOrRenamedImpl(const TableNamesSet & dependent, const QualifiedTableName & removing_table, bool is_drop_database) +void DatabaseCatalog::checkTableCanBeRemovedOrRenamedUnlocked(const StorageID & removing_table, bool is_drop_database) const { + const auto & dependents = referential_dependencies.getDependents(removing_table); + if (!is_drop_database) { - if (!dependent.empty()) + if (!dependents.empty()) throw Exception(ErrorCodes::HAVE_DEPENDENT_OBJECTS, "Cannot drop or rename {}, because some tables depend on it: {}", - removing_table, fmt::join(dependent, ", ")); + removing_table, fmt::join(dependents, ", ")); + return; } /// For DROP DATABASE we should ignore dependent tables from the same database. /// TODO unload tables in reverse topological order and remove this code - TableNames from_other_databases; - for (const auto & table : dependent) - if (table.database != removing_table.database) - from_other_databases.push_back(table); + std::vector from_other_databases; + for (const auto & dependent : dependents) + if (dependent.database_name != removing_table.database_name) + from_other_databases.push_back(dependent); if (!from_other_databases.empty()) throw Exception(ErrorCodes::HAVE_DEPENDENT_OBJECTS, "Cannot drop or rename {}, because some tables depend on it: {}", removing_table, fmt::join(from_other_databases, ", ")); } -void DatabaseCatalog::updateLoadingDependencies(const StorageID & table_id, TableNamesSet && new_dependencies) +void DatabaseCatalog::updateDependencies(const StorageID & table_id, const TableNamesSet & new_dependencies) { - if (new_dependencies.empty()) - return; - QualifiedTableName table_name = table_id.getQualifiedName(); std::lock_guard lock{databases_mutex}; - auto it = loading_dependencies.find(table_name); - if (it == loading_dependencies.end()) - it = loading_dependencies.emplace(table_name, DependenciesInfo{}).first; - - auto & old_dependencies = it->second.dependencies; - for (const auto & dependency : old_dependencies) - if (!new_dependencies.contains(dependency)) - loading_dependencies[dependency].dependent_database_objects.erase(table_name); - - for (const auto & dependency : new_dependencies) - if (!old_dependencies.contains(dependency)) - loading_dependencies[dependency].dependent_database_objects.insert(table_name); - - old_dependencies = std::move(new_dependencies); + referential_dependencies.removeDependencies(table_id, /* remove_isolated_tables= */ true); + referential_dependencies.addDependencies(table_id, new_dependencies); } void DatabaseCatalog::cleanupStoreDirectoryTask() diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index a44099b9fdc..a3fa4515a69 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include @@ -37,11 +37,7 @@ using DatabasePtr = std::shared_ptr; using DatabaseAndTable = std::pair; using Databases = std::map>; using DiskPtr = std::shared_ptr; - -/// Table -> set of table-views that make SELECT from it. -using ViewDependencies = std::map>; -using Dependencies = std::vector; - +using TableNamesSet = std::unordered_set; /// Allows executing DDL query only in one thread. /// Puts an element into the map, locks tables's mutex, counts how much threads run parallel query on the table, @@ -188,12 +184,11 @@ public: /// Four views (tables, views, columns, schemata) in the "information_schema" database are predefined too. bool isPredefinedTable(const StorageID & table_id) const; - void addDependency(const StorageID & from, const StorageID & where); - void removeDependency(const StorageID & from, const StorageID & where); - Dependencies getDependencies(const StorageID & from) const; - - /// For Materialized and Live View - void updateDependency(const StorageID & old_from, const StorageID & old_where,const StorageID & new_from, const StorageID & new_where); + /// View dependencies between a source table and its view. + void addViewDependency(const StorageID & source_table_id, const StorageID & view_id); + void removeViewDependency(const StorageID & source_table_id, const StorageID & view_id); + std::vector getDependentViews(const StorageID & source_table_id) const; + void updateViewDependency(const StorageID & old_source_table_id, const StorageID & old_view_id, const StorageID & new_source_table_id, const StorageID & new_view_id); /// If table has UUID, addUUIDMapping(...) must be called when table attached to some database /// removeUUIDMapping(...) must be called when it detached, @@ -223,16 +218,20 @@ public: void waitTableFinallyDropped(const UUID & uuid); - void addLoadingDependencies(const QualifiedTableName & table, TableNamesSet && dependencies); - void addLoadingDependencies(const DependenciesInfos & new_infos); - DependenciesInfo getLoadingDependenciesInfo(const StorageID & table_id) const; + /// Referential dependencies between tables: table "A" depends on table "B" + /// if "B" is referenced in the definition of "A". + void addDependencies(const StorageID & table_id, const std::vector & dependencies); + void addDependencies(const QualifiedTableName & table_name, const TableNamesSet & dependencies); + void addDependencies(const TablesDependencyGraph & extra_graph); + std::vector removeDependencies(const StorageID & table_id, bool check_dependencies, bool is_drop_database = false); + + std::vector getDependencies(const StorageID & table_id) const; + std::vector getDependents(const StorageID & table_id) const; + + void updateDependencies(const StorageID & table_id, const TableNamesSet & new_dependencies); - TableNamesSet tryRemoveLoadingDependencies(const StorageID & table_id, bool check_dependencies, bool is_drop_database = false); - TableNamesSet tryRemoveLoadingDependenciesUnlocked(const QualifiedTableName & removing_table, bool check_dependencies, bool is_drop_database = false) TSA_REQUIRES(databases_mutex); void checkTableCanBeRemovedOrRenamed(const StorageID & table_id, bool is_drop_database = false) const; - void updateLoadingDependencies(const StorageID & table_id, TableNamesSet && new_dependencies); - private: // The global instance of database catalog. unique_ptr is to allow // deferred initialization. Thought I'd use std::optional, but I can't @@ -245,7 +244,7 @@ private: void shutdownImpl(); - static void checkTableCanBeRemovedOrRenamedImpl(const TableNamesSet & dependent, const QualifiedTableName & removing_table, bool is_drop_database); + void checkTableCanBeRemovedOrRenamedUnlocked(const StorageID & removing_table, bool is_drop_database) const TSA_REQUIRES(databases_mutex); struct UUIDToStorageMapPart { @@ -281,12 +280,15 @@ private: mutable std::mutex databases_mutex; - ViewDependencies view_dependencies TSA_GUARDED_BY(databases_mutex); - Databases databases TSA_GUARDED_BY(databases_mutex); UUIDToStorageMap uuid_map; - DependenciesInfos loading_dependencies TSA_GUARDED_BY(databases_mutex); + /// Referential dependencies between tables: table "A" depends on table "B" + /// if the table "B" is referenced in the definition of the table "A". + TablesDependencyGraph referential_dependencies TSA_GUARDED_BY(databases_mutex); + + /// View dependencies between a source table and its view. + TablesDependencyGraph view_dependencies TSA_GUARDED_BY(databases_mutex); Poco::Logger * log; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 5a25cbd3a50..50536b66185 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -58,6 +58,7 @@ #include #include #include +#include #include @@ -1234,9 +1235,9 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) /// If table has dependencies - add them to the graph QualifiedTableName qualified_name{database_name, create.getTable()}; - TableNamesSet loading_dependencies = getDependenciesSetFromCreateQuery(getContext()->getGlobalContext(), qualified_name, query_ptr); - if (!loading_dependencies.empty()) - DatabaseCatalog::instance().addLoadingDependencies(qualified_name, std::move(loading_dependencies)); + TableNamesSet dependencies = getLoadingDependenciesFromCreateQuery(getContext()->getGlobalContext(), qualified_name, query_ptr); + if (!dependencies.empty()) + DatabaseCatalog::instance().addDependencies(qualified_name, dependencies); return fillTableIfNeeded(create); } diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 2fc733f5608..f237814f879 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -188,8 +188,8 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue if (query.permanently) { /// Server may fail to restart of DETACH PERMANENTLY if table has dependent ones - DatabaseCatalog::instance().tryRemoveLoadingDependencies(table_id, getContext()->getSettingsRef().check_table_dependencies, - is_drop_or_detach_database); + DatabaseCatalog::instance().removeDependencies(table_id, getContext()->getSettingsRef().check_table_dependencies, + is_drop_or_detach_database); /// Drop table from memory, don't touch data, metadata file renamed and will be skipped during server restart database->detachTablePermanently(context_, table_id.table_name); } @@ -243,8 +243,8 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue if (database->getUUID() == UUIDHelpers::Nil) table_lock = table->lockExclusively(context_->getCurrentQueryId(), context_->getSettingsRef().lock_acquire_timeout); - DatabaseCatalog::instance().tryRemoveLoadingDependencies(table_id, getContext()->getSettingsRef().check_table_dependencies, - is_drop_or_detach_database); + DatabaseCatalog::instance().removeDependencies(table_id, getContext()->getSettingsRef().check_table_dependencies, + is_drop_or_detach_database); database->dropTable(context_, table_id.table_name, query.sync); /// We have to drop mmapio cache when dropping table from Ordinary database diff --git a/src/Interpreters/InterpreterRenameQuery.cpp b/src/Interpreters/InterpreterRenameQuery.cpp index 666a674b2c8..82c230ef8e2 100644 --- a/src/Interpreters/InterpreterRenameQuery.cpp +++ b/src/Interpreters/InterpreterRenameQuery.cpp @@ -124,10 +124,10 @@ BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, c } else { - TableNamesSet dependencies; + std::vector dependencies; if (!exchange_tables) - dependencies = database_catalog.tryRemoveLoadingDependencies(StorageID(elem.from_database_name, elem.from_table_name), - getContext()->getSettingsRef().check_table_dependencies); + dependencies = database_catalog.removeDependencies(StorageID(elem.from_database_name, elem.from_table_name), + getContext()->getSettingsRef().check_table_dependencies); database->renameTable( getContext(), @@ -138,7 +138,7 @@ BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, c rename.dictionary); if (!dependencies.empty()) - DatabaseCatalog::instance().addLoadingDependencies(QualifiedTableName{elem.to_database_name, elem.to_table_name}, std::move(dependencies)); + DatabaseCatalog::instance().addDependencies(StorageID(elem.to_database_name, elem.to_table_name), dependencies); } } diff --git a/src/Interpreters/StorageID.cpp b/src/Interpreters/StorageID.cpp index 8811adc087b..70dea02ccc5 100644 --- a/src/Interpreters/StorageID.cpp +++ b/src/Interpreters/StorageID.cpp @@ -64,21 +64,8 @@ String StorageID::getNameForLogs() const + (hasUUID() ? " (" + toString(uuid) + ")" : ""); } -bool StorageID::operator<(const StorageID & rhs) const -{ - assertNotEmpty(); - /// It's needed for ViewDependencies - if (!hasUUID() && !rhs.hasUUID()) - /// If both IDs don't have UUID, compare them like pair of strings - return std::tie(database_name, table_name) < std::tie(rhs.database_name, rhs.table_name); - else if (hasUUID() && rhs.hasUUID()) - /// If both IDs have UUID, compare UUIDs and ignore database and table name - return uuid < rhs.uuid; - else - /// All IDs without UUID are less, then all IDs with UUID - return !hasUUID(); -} - +/// NOTE: This implementation doesn't allow to implement a good "operator <". +/// Because "a != b" must be equivalent to "(a < b) || (b < a)", and we can't make "operator <" to meet that. bool StorageID::operator==(const StorageID & rhs) const { assertNotEmpty(); diff --git a/src/Interpreters/StorageID.h b/src/Interpreters/StorageID.h index 43710988243..68c83f753b5 100644 --- a/src/Interpreters/StorageID.h +++ b/src/Interpreters/StorageID.h @@ -45,6 +45,8 @@ struct StorageID StorageID(const ASTTableIdentifier & table_identifier_node); /// NOLINT StorageID(const ASTPtr & node); /// NOLINT + explicit StorageID(const QualifiedTableName & qualified_name) : StorageID(qualified_name.database, qualified_name.table) { } + String getDatabaseName() const; String getTableName() const; @@ -71,7 +73,6 @@ struct StorageID bool hasDatabase() const { return !database_name.empty(); } - bool operator<(const StorageID & rhs) const; bool operator==(const StorageID & rhs) const; void assertNotEmpty() const @@ -97,8 +98,47 @@ struct StorageID /// Get short, but unique, name. String getShortName() const; + /// Calculates hash using only the database and table name of a StorageID. + struct DatabaseAndTableNameHash + { + size_t operator()(const StorageID & storage_id) const + { + SipHash hash_state; + hash_state.update(storage_id.database_name.data(), storage_id.database_name.size()); + hash_state.update(storage_id.table_name.data(), storage_id.table_name.size()); + return hash_state.get64(); + } + }; + + /// Checks if the database and table name of two StorageIDs are equal. + struct DatabaseAndTableNameEqual + { + bool operator()(const StorageID & left, const StorageID & right) const + { + return (left.database_name == right.database_name) && (left.table_name == right.table_name); + } + }; + private: StorageID() = default; }; } + +namespace fmt +{ + template <> + struct formatter + { + static constexpr auto parse(format_parse_context & ctx) + { + return ctx.begin(); + } + + template + auto format(const DB::StorageID & storage_id, FormatContext & ctx) + { + return format_to(ctx.out(), "{}", storage_id.getNameForLogs()); + } + }; +} diff --git a/src/Interpreters/loadMetadata.h b/src/Interpreters/loadMetadata.h index b229a2b4c31..3553011fe4d 100644 --- a/src/Interpreters/loadMetadata.h +++ b/src/Interpreters/loadMetadata.h @@ -1,7 +1,6 @@ #pragma once #include -#include namespace DB diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index cc484855e76..085399e4941 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -225,13 +225,13 @@ Chain buildPushingToViewsChain( disable_deduplication_for_children = !no_destination && storage->supportsDeduplication(); auto table_id = storage->getStorageID(); - Dependencies dependencies = DatabaseCatalog::instance().getDependencies(table_id); + auto views = DatabaseCatalog::instance().getDependentViews(table_id); /// We need special context for materialized views insertions ContextMutablePtr select_context; ContextMutablePtr insert_context; ViewsDataPtr views_data; - if (!dependencies.empty()) + if (!views.empty()) { select_context = Context::createCopy(context); insert_context = Context::createCopy(context); @@ -253,10 +253,10 @@ Chain buildPushingToViewsChain( std::vector chains; - for (const auto & database_table : dependencies) + for (const auto & view_id : views) { - auto dependent_table = DatabaseCatalog::instance().getTable(database_table, context); - auto dependent_metadata_snapshot = dependent_table->getInMemoryMetadataPtr(); + auto view = DatabaseCatalog::instance().getTable(view_id, context); + auto view_metadata_snapshot = view->getInMemoryMetadataPtr(); ASTPtr query; Chain out; @@ -288,7 +288,7 @@ Chain buildPushingToViewsChain( views_data->thread_status_holder->thread_statuses.push_front(std::move(view_thread_status_ptr)); auto runtime_stats = std::make_unique(); - runtime_stats->target_name = database_table.getFullTableName(); + runtime_stats->target_name = view_id.getFullTableName(); runtime_stats->thread_status = view_thread_status; runtime_stats->event_time = std::chrono::system_clock::now(); runtime_stats->event_status = QueryViewsLogElement::ViewStatus::EXCEPTION_BEFORE_START; @@ -297,7 +297,7 @@ Chain buildPushingToViewsChain( auto & target_name = runtime_stats->target_name; auto * view_counter_ms = &runtime_stats->elapsed_ms; - if (auto * materialized_view = dynamic_cast(dependent_table.get())) + if (auto * materialized_view = dynamic_cast(view.get())) { type = QueryViewsLogElement::ViewType::MATERIALIZED; result_chain.addTableLock(materialized_view->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout)); @@ -305,7 +305,7 @@ Chain buildPushingToViewsChain( StoragePtr inner_table = materialized_view->getTargetTable(); auto inner_table_id = inner_table->getStorageID(); auto inner_metadata_snapshot = inner_table->getInMemoryMetadataPtr(); - query = dependent_metadata_snapshot->getSelectQuery().inner_query; + query = view_metadata_snapshot->getSelectQuery().inner_query; target_name = inner_table_id.getFullTableName(); /// Get list of columns we get from select query. @@ -324,31 +324,31 @@ Chain buildPushingToViewsChain( InterpreterInsertQuery interpreter(nullptr, insert_context, false, false, false); out = interpreter.buildChain(inner_table, inner_metadata_snapshot, insert_columns, thread_status_holder, view_counter_ms); - out.addStorageHolder(dependent_table); + out.addStorageHolder(view); out.addStorageHolder(inner_table); } - else if (auto * live_view = dynamic_cast(dependent_table.get())) + else if (auto * live_view = dynamic_cast(view.get())) { runtime_stats->type = QueryViewsLogElement::ViewType::LIVE; query = live_view->getInnerQuery(); // Used only to log in system.query_views_log out = buildPushingToViewsChain( - dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), true, thread_status_holder, view_counter_ms, storage_header); + view, view_metadata_snapshot, insert_context, ASTPtr(), true, thread_status_holder, view_counter_ms, storage_header); } - else if (auto * window_view = dynamic_cast(dependent_table.get())) + else if (auto * window_view = dynamic_cast(view.get())) { runtime_stats->type = QueryViewsLogElement::ViewType::WINDOW; query = window_view->getMergeableQuery(); // Used only to log in system.query_views_log out = buildPushingToViewsChain( - dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), true, thread_status_holder, view_counter_ms); + view, view_metadata_snapshot, insert_context, ASTPtr(), true, thread_status_holder, view_counter_ms); } else out = buildPushingToViewsChain( - dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), false, thread_status_holder, view_counter_ms); + view, view_metadata_snapshot, insert_context, ASTPtr(), false, thread_status_holder, view_counter_ms); views_data->views.emplace_back(ViewRuntimeData{ //-V614 std::move(query), out.getInputHeader(), - database_table, + view_id, nullptr, std::move(runtime_stats)}); @@ -367,7 +367,7 @@ Chain buildPushingToViewsChain( if (!no_destination) { context->getQueryContext()->addQueryAccessInfo( - backQuoteIfNeed(database_table.getDatabaseName()), views_data->views.back().runtime_stats->target_name, {}, "", database_table.getFullTableName()); + backQuoteIfNeed(view_id.getDatabaseName()), views_data->views.back().runtime_stats->target_name, {}, "", view_id.getFullTableName()); } } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 722843a7ab6..0f4563b6f35 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -547,23 +547,23 @@ size_t StorageFileLog::getPollTimeoutMillisecond() const bool StorageFileLog::checkDependencies(const StorageID & table_id) { // Check if all dependencies are attached - auto dependencies = DatabaseCatalog::instance().getDependencies(table_id); - if (dependencies.empty()) + auto view_ids = DatabaseCatalog::instance().getDependentViews(table_id); + if (view_ids.empty()) return true; - for (const auto & storage : dependencies) + for (const auto & view_id : view_ids) { - auto table = DatabaseCatalog::instance().tryGetTable(storage, getContext()); - if (!table) + auto view = DatabaseCatalog::instance().tryGetTable(view_id, getContext()); + if (!view) return false; // If it materialized view, check it's target table - auto * materialized_view = dynamic_cast(table.get()); + auto * materialized_view = dynamic_cast(view.get()); if (materialized_view && !materialized_view->tryGetTargetTable()) return false; // Check all its dependencies - if (!checkDependencies(storage)) + if (!checkDependencies(view_id)) return false; } @@ -574,7 +574,7 @@ size_t StorageFileLog::getTableDependentCount() const { auto table_id = getStorageID(); // Check if at least one direct dependency is attached - return DatabaseCatalog::instance().getDependencies(table_id).size(); + return DatabaseCatalog::instance().getDependentViews(table_id).size(); } void StorageFileLog::threadFunc() diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index bc2d38de215..76100624d51 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -232,16 +232,16 @@ Names IStorage::getAllRegisteredNames() const NameDependencies IStorage::getDependentViewsByColumn(ContextPtr context) const { NameDependencies name_deps; - auto dependencies = DatabaseCatalog::instance().getDependencies(storage_id); - for (const auto & depend_id : dependencies) + auto view_ids = DatabaseCatalog::instance().getDependentViews(storage_id); + for (const auto & view_id : view_ids) { - auto depend_table = DatabaseCatalog::instance().getTable(depend_id, context); - if (depend_table->getInMemoryMetadataPtr()->select.inner_query) + auto view = DatabaseCatalog::instance().getTable(view_id, context); + if (view->getInMemoryMetadataPtr()->select.inner_query) { - const auto & select_query = depend_table->getInMemoryMetadataPtr()->select.inner_query; + const auto & select_query = view->getInMemoryMetadataPtr()->select.inner_query; auto required_columns = InterpreterSelectQuery(select_query, context, SelectQueryOptions{}.noModify()).getRequiredColumns(); for (const auto & col_name : required_columns) - name_deps[col_name].push_back(depend_id.table_name); + name_deps[col_name].push_back(view_id.table_name); } } return name_deps; diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 8e4dd78379e..77afa7ba623 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -584,24 +584,24 @@ void StorageKafka::updateConfiguration(cppkafka::Configuration & conf) bool StorageKafka::checkDependencies(const StorageID & table_id) { // Check if all dependencies are attached - auto dependencies = DatabaseCatalog::instance().getDependencies(table_id); - if (dependencies.empty()) + auto view_ids = DatabaseCatalog::instance().getDependentViews(table_id); + if (view_ids.empty()) return true; // Check the dependencies are ready? - for (const auto & db_tab : dependencies) + for (const auto & view_id : view_ids) { - auto table = DatabaseCatalog::instance().tryGetTable(db_tab, getContext()); - if (!table) + auto view = DatabaseCatalog::instance().tryGetTable(view_id, getContext()); + if (!view) return false; // If it materialized view, check it's target table - auto * materialized_view = dynamic_cast(table.get()); + auto * materialized_view = dynamic_cast(view.get()); if (materialized_view && !materialized_view->tryGetTargetTable()) return false; // Check all its dependencies - if (!checkDependencies(db_tab)) + if (!checkDependencies(view_id)) return false; } @@ -616,8 +616,8 @@ void StorageKafka::threadFunc(size_t idx) { auto table_id = getStorageID(); // Check if at least one direct dependency is attached - size_t dependencies_count = DatabaseCatalog::instance().getDependencies(table_id).size(); - if (dependencies_count) + size_t num_views = DatabaseCatalog::instance().getDependentViews(table_id).size(); + if (num_views) { auto start_time = std::chrono::steady_clock::now(); @@ -629,7 +629,7 @@ void StorageKafka::threadFunc(size_t idx) if (!checkDependencies(table_id)) break; - LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); + LOG_DEBUG(log, "Started streaming to {} attached views", num_views); // Exit the loop & reschedule if some stream stalled auto some_stream_is_stalled = streamToViews(); diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 3d27205d638..c92968e4bcc 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -304,7 +304,7 @@ StorageLiveView::StorageLiveView( auto inner_query_tmp = inner_query->clone(); select_table_id = extractDependentTable(inner_query_tmp, getContext(), table_id_.table_name, inner_subquery); - DatabaseCatalog::instance().addDependency(select_table_id, table_id_); + DatabaseCatalog::instance().addViewDependency(select_table_id, table_id_); if (query.live_view_periodic_refresh) { @@ -434,11 +434,11 @@ bool StorageLiveView::getNewBlocks() void StorageLiveView::checkTableCanBeDropped() const { auto table_id = getStorageID(); - Dependencies dependencies = DatabaseCatalog::instance().getDependencies(table_id); - if (!dependencies.empty()) + auto view_ids = DatabaseCatalog::instance().getDependentViews(table_id); + if (!view_ids.empty()) { - StorageID dependent_table_id = dependencies.front(); - throw Exception("Table has dependency " + dependent_table_id.getNameForLogs(), ErrorCodes::TABLE_WAS_NOT_DROPPED); + StorageID view_id = *view_ids.begin(); + throw Exception(ErrorCodes::TABLE_WAS_NOT_DROPPED, "Table has dependency {}", view_id); } } @@ -455,7 +455,7 @@ void StorageLiveView::shutdown() if (is_periodically_refreshed) periodic_refresh_task->deactivate(); - DatabaseCatalog::instance().removeDependency(select_table_id, getStorageID()); + DatabaseCatalog::instance().removeViewDependency(select_table_id, getStorageID()); } StorageLiveView::~StorageLiveView() @@ -466,7 +466,7 @@ StorageLiveView::~StorageLiveView() void StorageLiveView::drop() { auto table_id = getStorageID(); - DatabaseCatalog::instance().removeDependency(select_table_id, table_id); + DatabaseCatalog::instance().removeViewDependency(select_table_id, table_id); std::lock_guard lock(mutex); is_dropped = true; diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index dea2553700b..5a8e250a972 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -535,24 +535,24 @@ bool StorageNATS::isSubjectInSubscriptions(const std::string & subject) bool StorageNATS::checkDependencies(const StorageID & table_id) { // Check if all dependencies are attached - auto dependencies = DatabaseCatalog::instance().getDependencies(table_id); - if (dependencies.empty()) + auto view_ids = DatabaseCatalog::instance().getDependentViews(table_id); + if (view_ids.empty()) return true; // Check the dependencies are ready? - for (const auto & db_tab : dependencies) + for (const auto & view_id : view_ids) { - auto table = DatabaseCatalog::instance().tryGetTable(db_tab, getContext()); - if (!table) + auto view = DatabaseCatalog::instance().tryGetTable(view_id, getContext()); + if (!view) return false; // If it materialized view, check it's target table - auto * materialized_view = dynamic_cast(table.get()); + auto * materialized_view = dynamic_cast(view.get()); if (materialized_view && !materialized_view->tryGetTargetTable()) return false; // Check all its dependencies - if (!checkDependencies(db_tab)) + if (!checkDependencies(view_id)) return false; } @@ -568,10 +568,10 @@ void StorageNATS::streamingToViewsFunc() auto table_id = getStorageID(); // Check if at least one direct dependency is attached - size_t dependencies_count = DatabaseCatalog::instance().getDependencies(table_id).size(); + size_t num_views = DatabaseCatalog::instance().getDependentViews(table_id).size(); bool nats_connected = connection->isConnected() || connection->reconnect(); - if (dependencies_count && nats_connected) + if (num_views && nats_connected) { auto start_time = std::chrono::steady_clock::now(); @@ -583,7 +583,7 @@ void StorageNATS::streamingToViewsFunc() if (!checkDependencies(table_id)) break; - LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); + LOG_DEBUG(log, "Started streaming to {} attached views", num_views); if (streamToViews()) { diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 57f5ddd86e6..bce3fee71f7 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -938,24 +938,24 @@ ProducerBufferPtr StorageRabbitMQ::createWriteBuffer() bool StorageRabbitMQ::checkDependencies(const StorageID & table_id) { // Check if all dependencies are attached - auto dependencies = DatabaseCatalog::instance().getDependencies(table_id); - if (dependencies.empty()) + auto view_ids = DatabaseCatalog::instance().getDependentViews(table_id); + if (view_ids.empty()) return true; // Check the dependencies are ready? - for (const auto & db_tab : dependencies) + for (const auto & view_id : view_ids) { - auto table = DatabaseCatalog::instance().tryGetTable(db_tab, getContext()); - if (!table) + auto view = DatabaseCatalog::instance().tryGetTable(view_id, getContext()); + if (!view) return false; // If it materialized view, check it's target table - auto * materialized_view = dynamic_cast(table.get()); + auto * materialized_view = dynamic_cast(view.get()); if (materialized_view && !materialized_view->tryGetTargetTable()) return false; // Check all its dependencies - if (!checkDependencies(db_tab)) + if (!checkDependencies(view_id)) return false; } @@ -984,10 +984,10 @@ void StorageRabbitMQ::streamingToViewsFunc() auto table_id = getStorageID(); // Check if at least one direct dependency is attached - size_t dependencies_count = DatabaseCatalog::instance().getDependencies(table_id).size(); + size_t num_views = DatabaseCatalog::instance().getDependentViews(table_id).size(); bool rabbit_connected = connection->isConnected() || connection->reconnect(); - if (dependencies_count && rabbit_connected) + if (num_views && rabbit_connected) { initializeBuffers(); auto start_time = std::chrono::steady_clock::now(); @@ -1000,7 +1000,7 @@ void StorageRabbitMQ::streamingToViewsFunc() if (!checkDependencies(table_id)) break; - LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); + LOG_DEBUG(log, "Started streaming to {} attached views", num_views); if (streamToViews()) { diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index e256e087728..ed01ca9cec4 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -210,7 +210,7 @@ void StorageMaterializedView::drop() auto table_id = getStorageID(); const auto & select_query = getInMemoryMetadataPtr()->getSelectQuery(); if (!select_query.select_table_id.empty()) - DatabaseCatalog::instance().removeDependency(select_query.select_table_id, table_id); + DatabaseCatalog::instance().removeViewDependency(select_query.select_table_id, table_id); dropInnerTableIfAny(true, getContext()); } @@ -266,7 +266,7 @@ void StorageMaterializedView::alter( const auto & new_select = new_metadata.select; const auto & old_select = old_metadata.getSelectQuery(); - DatabaseCatalog::instance().updateDependency(old_select.select_table_id, table_id, new_select.select_table_id, table_id); + DatabaseCatalog::instance().updateViewDependency(old_select.select_table_id, table_id, new_select.select_table_id, table_id); new_metadata.setSelectQuery(new_select); } @@ -364,7 +364,7 @@ void StorageMaterializedView::renameInMemory(const StorageID & new_table_id) } const auto & select_query = metadata_snapshot->getSelectQuery(); // TODO Actually we don't need to update dependency if MV has UUID, but then db and table name will be outdated - DatabaseCatalog::instance().updateDependency(select_query.select_table_id, old_table_id, select_query.select_table_id, getStorageID()); + DatabaseCatalog::instance().updateViewDependency(select_query.select_table_id, old_table_id, select_query.select_table_id, getStorageID()); } void StorageMaterializedView::startup() @@ -372,7 +372,7 @@ void StorageMaterializedView::startup() auto metadata_snapshot = getInMemoryMetadataPtr(); const auto & select_query = metadata_snapshot->getSelectQuery(); if (!select_query.select_table_id.empty()) - DatabaseCatalog::instance().addDependency(select_query.select_table_id, getStorageID()); + DatabaseCatalog::instance().addViewDependency(select_query.select_table_id, getStorageID()); } void StorageMaterializedView::shutdown() @@ -381,7 +381,7 @@ void StorageMaterializedView::shutdown() const auto & select_query = metadata_snapshot->getSelectQuery(); /// Make sure the dependency is removed after DETACH TABLE if (!select_query.select_table_id.empty()) - DatabaseCatalog::instance().removeDependency(select_query.select_table_id, getStorageID()); + DatabaseCatalog::instance().removeViewDependency(select_query.select_table_id, getStorageID()); } StoragePtr StorageMaterializedView::getTargetTable() const diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 72301a56d49..e1611f1ecfd 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -348,26 +348,26 @@ protected: res_columns[res_index++]->insert(static_cast(database->getObjectMetadataModificationTime(table_name))); { - Array dependencies_table_name_array; - Array dependencies_database_name_array; + Array views_table_name_array; + Array views_database_name_array; if (columns_mask[src_index] || columns_mask[src_index + 1]) { - const auto dependencies = DatabaseCatalog::instance().getDependencies(StorageID(database_name, table_name)); + const auto view_ids = DatabaseCatalog::instance().getDependentViews(StorageID(database_name, table_name)); - dependencies_table_name_array.reserve(dependencies.size()); - dependencies_database_name_array.reserve(dependencies.size()); - for (const auto & dependency : dependencies) + views_table_name_array.reserve(view_ids.size()); + views_database_name_array.reserve(view_ids.size()); + for (const auto & view_id : view_ids) { - dependencies_table_name_array.push_back(dependency.table_name); - dependencies_database_name_array.push_back(dependency.database_name); + views_table_name_array.push_back(view_id.table_name); + views_database_name_array.push_back(view_id.database_name); } } if (columns_mask[src_index++]) - res_columns[res_index++]->insert(dependencies_database_name_array); + res_columns[res_index++]->insert(views_database_name_array); if (columns_mask[src_index++]) - res_columns[res_index++]->insert(dependencies_table_name_array); + res_columns[res_index++]->insert(views_table_name_array); } if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2]) @@ -513,37 +513,38 @@ protected: if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2] || columns_mask[src_index + 3]) { - DependenciesInfo info = DatabaseCatalog::instance().getLoadingDependenciesInfo({database_name, table_name}); + auto dependencies = DatabaseCatalog::instance().getDependencies(StorageID{database_name, table_name}); + auto dependents = DatabaseCatalog::instance().getDependents(StorageID{database_name, table_name}); - Array loading_dependencies_databases; - Array loading_dependencies_tables; - loading_dependencies_databases.reserve(info.dependencies.size()); - loading_dependencies_tables.reserve(info.dependencies.size()); - for (auto && dependency : info.dependencies) + Array dependencies_databases; + Array dependencies_tables; + dependencies_databases.reserve(dependencies.size()); + dependencies_tables.reserve(dependencies.size()); + for (const auto & dependency : dependencies) { - loading_dependencies_databases.push_back(dependency.database); - loading_dependencies_tables.push_back(dependency.table); + dependencies_databases.push_back(dependency.database_name); + dependencies_tables.push_back(dependency.table_name); } - Array loading_dependent_databases; - Array loading_dependent_tables; - loading_dependent_databases.reserve(info.dependencies.size()); - loading_dependent_tables.reserve(info.dependencies.size()); - for (auto && dependent : info.dependent_database_objects) + Array dependents_databases; + Array dependents_tables; + dependents_databases.reserve(dependents.size()); + dependents_tables.reserve(dependents.size()); + for (const auto & dependent : dependents) { - loading_dependent_databases.push_back(dependent.database); - loading_dependent_tables.push_back(dependent.table); + dependents_databases.push_back(dependent.database_name); + dependents_tables.push_back(dependent.table_name); } if (columns_mask[src_index++]) - res_columns[res_index++]->insert(loading_dependencies_databases); + res_columns[res_index++]->insert(dependencies_databases); if (columns_mask[src_index++]) - res_columns[res_index++]->insert(loading_dependencies_tables); + res_columns[res_index++]->insert(dependencies_tables); if (columns_mask[src_index++]) - res_columns[res_index++]->insert(loading_dependent_databases); + res_columns[res_index++]->insert(dependents_databases); if (columns_mask[src_index++]) - res_columns[res_index++]->insert(loading_dependent_tables); + res_columns[res_index++]->insert(dependents_tables); } } diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index c0bc5ad8da9..442a7822e33 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -471,7 +471,7 @@ void StorageWindowView::alter( create_interpreter.setInternal(true); create_interpreter.execute(); - DatabaseCatalog::instance().addDependency(select_table_id, table_id); + DatabaseCatalog::instance().addViewDependency(select_table_id, table_id); shutdown_called = false; @@ -1566,7 +1566,7 @@ void StorageWindowView::writeIntoWindowView( void StorageWindowView::startup() { - DatabaseCatalog::instance().addDependency(select_table_id, getStorageID()); + DatabaseCatalog::instance().addViewDependency(select_table_id, getStorageID()); fire_task->activate(); clean_cache_task->activate(); @@ -1586,17 +1586,17 @@ void StorageWindowView::shutdown() fire_task->deactivate(); auto table_id = getStorageID(); - DatabaseCatalog::instance().removeDependency(select_table_id, table_id); + DatabaseCatalog::instance().removeViewDependency(select_table_id, table_id); } void StorageWindowView::checkTableCanBeDropped() const { auto table_id = getStorageID(); - Dependencies dependencies = DatabaseCatalog::instance().getDependencies(table_id); - if (!dependencies.empty()) + auto view_ids = DatabaseCatalog::instance().getDependentViews(table_id); + if (!view_ids.empty()) { - StorageID dependent_table_id = dependencies.front(); - throw Exception("Table has dependency " + dependent_table_id.getNameForLogs(), ErrorCodes::TABLE_WAS_NOT_DROPPED); + StorageID view_id = *view_ids.begin(); + throw Exception(ErrorCodes::TABLE_WAS_NOT_DROPPED, "Table has dependency {}", view_id); } } diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index c94dc6d4a87..7eeabde1380 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -1,6 +1,7 @@ import pytest import asyncio import re +import random import os.path from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry, TSV @@ -1158,3 +1159,71 @@ def test_mutation(): instance.query("DROP TABLE test.table") instance.query(f"RESTORE TABLE test.table FROM {backup_name}") + + +def test_tables_dependency(): + instance.query("CREATE DATABASE test") + instance.query("CREATE DATABASE test2") + + # For this test we use random names of tables to check they're created according to their dependency (not just in alphabetic order). + random_table_names = [f"{chr(ord('A')+i)}" for i in range(0, 10)] + random.shuffle(random_table_names) + random_table_names = [ + random.choice(["test", "test2"]) + "." + table_name + for table_name in random_table_names + ] + print(f"random_table_names={random_table_names}") + + t1 = random_table_names[0] + t2 = random_table_names[1] + t3 = random_table_names[2] + t4 = random_table_names[3] + t5 = random_table_names[4] + t6 = random_table_names[5] + + # Create a materialized view and a dictionary with a local table as source. + instance.query( + f"CREATE TABLE {t1} (x Int64, y String) ENGINE=MergeTree ORDER BY tuple()" + ) + + instance.query( + f"CREATE TABLE {t2} (x Int64, y String) ENGINE=MergeTree ORDER BY tuple()" + ) + + instance.query(f"CREATE MATERIALIZED VIEW {t3} TO {t2} AS SELECT x, y FROM {t1}") + + instance.query( + f"CREATE DICTIONARY {t4} (x Int64, y String) PRIMARY KEY x SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE '{t1.split('.')[1]}' DB '{t1.split('.')[0]}')) LAYOUT(FLAT()) LIFETIME(0)" + ) + + instance.query(f"CREATE TABLE {t5} AS dictionary({t4})") + + instance.query( + f"CREATE TABLE {t6}(x Int64, y String DEFAULT dictGet({t4}, 'y', x)) ENGINE=MergeTree ORDER BY tuple()" + ) + + # Make backup. + backup_name = new_backup_name() + instance.query(f"BACKUP DATABASE test, DATABASE test2 TO {backup_name}") + + # Drop everything in reversive order. + def drop(): + instance.query(f"DROP TABLE {t6} NO DELAY") + instance.query(f"DROP TABLE {t5} NO DELAY") + instance.query(f"DROP DICTIONARY {t4}") + instance.query(f"DROP TABLE {t3} NO DELAY") + instance.query(f"DROP TABLE {t2} NO DELAY") + instance.query(f"DROP TABLE {t1} NO DELAY") + instance.query("DROP DATABASE test NO DELAY") + instance.query("DROP DATABASE test2 NO DELAY") + + drop() + + # Restore everything and check. + instance.query(f"RESTORE ALL FROM {backup_name}") + + assert instance.query( + "SELECT concat(database, '.', name) AS c FROM system.tables WHERE database IN ['test', 'test2'] ORDER BY c" + ) == TSV(sorted([t1, t2, t3, t4, t5, t6])) + + drop() diff --git a/tests/queries/0_stateless/01155_rename_move_materialized_view.sql b/tests/queries/0_stateless/01155_rename_move_materialized_view.sql index b3234e03a8f..c3cc0bbb9eb 100644 --- a/tests/queries/0_stateless/01155_rename_move_materialized_view.sql +++ b/tests/queries/0_stateless/01155_rename_move_materialized_view.sql @@ -39,7 +39,7 @@ RENAME TABLE test_01155_ordinary.mv1 TO test_01155_atomic.mv1; RENAME TABLE test_01155_ordinary.mv2 TO test_01155_atomic.mv2; RENAME TABLE test_01155_ordinary.dst TO test_01155_atomic.dst; RENAME TABLE test_01155_ordinary.src TO test_01155_atomic.src; -SET check_table_dependencies=0; +SET check_table_dependencies=0; -- Otherwise we'll get error "test_01155_atomic.dict depends on test_01155_ordinary.dist" in the next line. RENAME TABLE test_01155_ordinary.dist TO test_01155_atomic.dist; SET check_table_dependencies=1; RENAME DICTIONARY test_01155_ordinary.dict TO test_01155_atomic.dict; @@ -65,7 +65,9 @@ SELECT dictGet('test_01155_ordinary.dict', 'x', 'after renaming database'); SELECT database, substr(name, 1, 10) FROM system.tables WHERE database like 'test_01155_%'; -- Move tables back +SET check_table_dependencies=0; -- Otherwise we'll get error "test_01155_atomic.dict depends on test_01155_ordinary.dist" in the next line. RENAME DATABASE test_01155_ordinary TO test_01155_atomic; +SET check_table_dependencies=1; set allow_deprecated_database_ordinary=1; CREATE DATABASE test_01155_ordinary ENGINE=Ordinary; From b6eddbac0dcd52e45701905b22cef69759171e76 Mon Sep 17 00:00:00 2001 From: chen Date: Fri, 2 Dec 2022 22:43:29 +0800 Subject: [PATCH 88/98] fix s3Cluster function returns NOT_FOUND_COLUMN_IN_BLOCK error (#43629) * fix s3Cluster function returns NOT_FOUND_COLUMN_IN_BLOCK error * Update StorageS3Cluster.cpp * Update 01801_s3_cluster_count.sql * fix --- src/Storages/StorageS3Cluster.cpp | 19 +++++++++++++++---- .../01801_s3_cluster_count.reference | 2 ++ .../0_stateless/01801_s3_cluster_count.sql | 5 +++++ 3 files changed, 22 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/01801_s3_cluster_count.reference create mode 100644 tests/queries/0_stateless/01801_s3_cluster_count.sql diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index ec970654b6e..b10f3c65ebf 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -14,6 +14,8 @@ #include #include #include +#include +#include #include #include #include @@ -103,8 +105,7 @@ Pipe StorageS3Cluster::read( auto callback = std::make_shared([iterator]() mutable -> String { return iterator->next(); }); /// Calculate the header. This is significant, because some columns could be thrown away in some cases like query with count(*) - Block header = - InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + auto interpreter = InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage).analyze()); const Scalars & scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{}; @@ -112,11 +113,21 @@ Pipe StorageS3Cluster::read( const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState; - ASTPtr query_to_send = query_info.original_query->clone(); + ASTPtr query_to_send = interpreter.getQueryInfo().query->clone(); if (add_columns_structure_to_query) addColumnsStructureToQueryWithClusterEngine( query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll()), 5, getName()); + RestoreQualifiedNamesVisitor::Data data; + data.distributed_table = DatabaseAndTableWithAlias(*getTableExpression(query_info.query->as(), 0)); + data.remote_table.database = context->getCurrentDatabase(); + data.remote_table.table = getName(); + RestoreQualifiedNamesVisitor(data).visit(query_to_send); + AddDefaultDatabaseVisitor visitor(context, context->getCurrentDatabase(), + /* only_replace_current_database_function_= */false, + /* only_replace_in_join_= */true); + visitor.visit(query_to_send); + const auto & current_settings = context->getSettingsRef(); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); for (const auto & shard_info : cluster->getShardsInfo()) @@ -128,7 +139,7 @@ Pipe StorageS3Cluster::read( shard_info.pool, std::vector{try_result}, queryToString(query_to_send), - header, + interpreter.getSampleBlock(), context, /*throttler=*/nullptr, scalars, diff --git a/tests/queries/0_stateless/01801_s3_cluster_count.reference b/tests/queries/0_stateless/01801_s3_cluster_count.reference new file mode 100644 index 00000000000..c094c553f81 --- /dev/null +++ b/tests/queries/0_stateless/01801_s3_cluster_count.reference @@ -0,0 +1,2 @@ +12 +12 diff --git a/tests/queries/0_stateless/01801_s3_cluster_count.sql b/tests/queries/0_stateless/01801_s3_cluster_count.sql new file mode 100644 index 00000000000..8a4fb804967 --- /dev/null +++ b/tests/queries/0_stateless/01801_s3_cluster_count.sql @@ -0,0 +1,5 @@ +-- Tags: no-fasttest +-- Tag no-fasttest: Depends on AWS + +select COUNT() from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv'); +select COUNT(*) from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv'); From ed4db129f45e05fa68239b277fc850ca331ce145 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 2 Dec 2022 18:27:36 +0100 Subject: [PATCH 89/98] Reduce dependencies from unidiff --- tests/ci/mark_release_ready.py | 3 +-- tests/ci/release.py | 5 +++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/ci/mark_release_ready.py b/tests/ci/mark_release_ready.py index be1771e62bd..57ddb166693 100644 --- a/tests/ci/mark_release_ready.py +++ b/tests/ci/mark_release_ready.py @@ -5,8 +5,7 @@ from env_helper import GITHUB_JOB_URL from get_robot_token import get_best_robot_token from github_helper import GitHub from pr_info import PRInfo - -RELEASE_READY_STATUS = "Ready for release" +from release import RELEASE_READY_STATUS def main(): diff --git a/tests/ci/release.py b/tests/ci/release.py index dec97f2a54f..4970d57ef7b 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 """ script to create releases for ClickHouse @@ -18,7 +18,6 @@ import subprocess from git_helper import commit, release_branch from github_helper import GitHub -from mark_release_ready import RELEASE_READY_STATUS from version_helper import ( FILE_WITH_VERSION_PATH, GENERATED_CONTRIBUTORS, @@ -31,6 +30,8 @@ from version_helper import ( update_contributors, ) +RELEASE_READY_STATUS = "Ready for release" + git = Git() From 7356d48f67f7fcbb119d48d07de870e96145ee3e Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 2 Dec 2022 18:55:22 +0100 Subject: [PATCH 90/98] Get rid of PyGithub dependency --- tests/ci/release.py | 46 ++++++++++++++++++++++----------------------- 1 file changed, 22 insertions(+), 24 deletions(-) diff --git a/tests/ci/release.py b/tests/ci/release.py index 4970d57ef7b..8e58413f91f 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -13,11 +13,11 @@ On another hand, PyGithub is used for convenient getting commit's status from AP from contextlib import contextmanager from typing import Any, Iterator, List, Literal, Optional import argparse +import json import logging import subprocess from git_helper import commit, release_branch -from github_helper import GitHub from version_helper import ( FILE_WITH_VERSION_PATH, GENERATED_CONTRIBUTORS, @@ -32,7 +32,6 @@ from version_helper import ( RELEASE_READY_STATUS = "Ready for release" - git = Git() @@ -113,31 +112,30 @@ class Release: return VersionType.STABLE def check_commit_release_ready(self): - # First, get the auth token from gh cli - auth_status = self.run( - "gh auth status -t", stderr=subprocess.STDOUT - ).splitlines() - token = "" - for line in auth_status: - if "✓ Token:" in line: - token = line.split()[-1] - if not token: - logging.error("Can not extract token from `gh auth`") - raise subprocess.SubprocessError("Can not extract token from `gh auth`") - gh = GitHub(token, per_page=100) - repo = gh.get_repo(str(self.repo)) + per_page = 100 + page = 1 + while True: + statuses = json.loads( + self.run( + f"gh api 'repos/{self.repo}/commits/{self.release_commit}" + f"/statuses?per_page={per_page}&page={page}'" + ) + ) + + if not statuses: + break + + for status in statuses: + if status["context"] == RELEASE_READY_STATUS: + if not status["state"] == "success": + raise Exception( + f"the status {RELEASE_READY_STATUS} is {status['state']}" + ", not success" + ) - # Statuses are ordered by descending updated_at, so the first necessary - # status in the list is the most recent - statuses = repo.get_commit(self.release_commit).get_statuses() - for status in statuses: - if status.context == RELEASE_READY_STATUS: - if status.state == "success": return - raise Exception( - f"the status {RELEASE_READY_STATUS} is {status.state}, not success" - ) + page += 1 raise Exception( f"the status {RELEASE_READY_STATUS} " From 8e8068afe478b66e18b7b689900e97139fe0b3a0 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 2 Dec 2022 19:17:16 +0100 Subject: [PATCH 91/98] Use only PRs to our repository in pr_info on push --- tests/ci/pr_info.py | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 942edeaa81c..ddeb070b2b9 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -2,7 +2,7 @@ import json import logging import os -from typing import Dict, Set, Union +from typing import Dict, List, Set, Union from unidiff import PatchSet # type: ignore @@ -47,15 +47,22 @@ def get_pr_for_commit(sha, ref): try: response = get_with_retries(try_get_pr_url, sleep=RETRY_SLEEP) data = response.json() + our_prs = [] # type: List[Dict] if len(data) > 1: print("Got more than one pr for commit", sha) for pr in data: + # We need to check if the PR is created in our repo, because + # https://github.com/kaynewu/ClickHouse/pull/2 + # has broke our PR search once in a while + if pr["base"]["repo"]["full_name"] != GITHUB_REPOSITORY: + continue # refs for pushes looks like refs/head/XX # refs for RPs looks like XX if pr["head"]["ref"] in ref: return pr + our_prs.append(pr) print("Cannot find PR with required ref", ref, "returning first one") - first_pr = data[0] + first_pr = our_prs[0] return first_pr except Exception as ex: print("Cannot fetch PR info from commit", ex) From dcdacea8e828829ed1302058e77c2ee9bf09d22b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 2 Dec 2022 21:33:08 +0100 Subject: [PATCH 92/98] Revert "Add typing to upload_result_helper.py, remove stale code" This reverts commit 1fb090bfe20bd76345ed20ca2b50a9852fa46dbc. --- tests/ci/s3_helper.py | 6 ++-- tests/ci/upload_result_helper.py | 52 +++++++++++++++++++++++--------- 2 files changed, 40 insertions(+), 18 deletions(-) diff --git a/tests/ci/s3_helper.py b/tests/ci/s3_helper.py index 03e855a0057..24ff013d69a 100644 --- a/tests/ci/s3_helper.py +++ b/tests/ci/s3_helper.py @@ -46,7 +46,7 @@ class S3Helper: self.host = host self.download_host = download_host - def _upload_file_to_s3(self, bucket_name: str, file_path: str, s3_path: str) -> str: + def _upload_file_to_s3(self, bucket_name, file_path, s3_path): logging.debug( "Start uploading %s to bucket=%s path=%s", file_path, bucket_name, s3_path ) @@ -110,7 +110,7 @@ class S3Helper: url = f"{self.download_host}/{bucket_name}/{s3_path}" return url.replace("+", "%2B").replace(" ", "%20") - def upload_test_report_to_s3(self, file_path: str, s3_path: str) -> str: + def upload_test_report_to_s3(self, file_path, s3_path): if CI: return self._upload_file_to_s3(S3_TEST_REPORTS_BUCKET, file_path, s3_path) else: @@ -296,7 +296,7 @@ class S3Helper: return False @staticmethod - def copy_file_to_local(bucket_name: str, file_path: str, s3_path: str) -> str: + def copy_file_to_local(bucket_name, file_path, s3_path): local_path = os.path.abspath( os.path.join(RUNNER_TEMP, "s3", bucket_name, s3_path) ) diff --git a/tests/ci/upload_result_helper.py b/tests/ci/upload_result_helper.py index 745633a9e4d..e145df02f80 100644 --- a/tests/ci/upload_result_helper.py +++ b/tests/ci/upload_result_helper.py @@ -1,7 +1,6 @@ import os import logging - -from typing import List, Tuple +import ast from env_helper import ( GITHUB_JOB_URL, @@ -10,15 +9,35 @@ from env_helper import ( GITHUB_SERVER_URL, ) from report import ReportColorTheme, create_test_html_report -from s3_helper import S3Helper def process_logs( - s3_client: S3Helper, additional_logs: List[str], s3_path_prefix: str -) -> List[str]: + s3_client, additional_logs, s3_path_prefix, test_results, with_raw_logs +): logging.info("Upload files to s3 %s", additional_logs) - additional_urls = [] # type: List[str] + processed_logs = {} + # Firstly convert paths of logs from test_results to urls to s3. + for test_result in test_results: + if len(test_result) <= 3 or with_raw_logs: + continue + + # Convert from string repr of list to list. + test_log_paths = ast.literal_eval(test_result[3]) + test_log_urls = [] + for log_path in test_log_paths: + if log_path in processed_logs: + test_log_urls.append(processed_logs[log_path]) + elif log_path: + url = s3_client.upload_test_report_to_s3( + log_path, s3_path_prefix + "/" + os.path.basename(log_path) + ) + test_log_urls.append(url) + processed_logs[log_path] = url + + test_result[3] = test_log_urls + + additional_urls = [] for log_path in additional_logs: if log_path: additional_urls.append( @@ -31,18 +50,21 @@ def process_logs( def upload_results( - s3_client: S3Helper, - pr_number: int, - commit_sha: str, - test_results: List[Tuple[str, str]], - additional_files: List[str], - check_name: str, - with_raw_logs: bool = True, -) -> str: + s3_client, + pr_number, + commit_sha, + test_results, + additional_files, + check_name, + with_raw_logs=True, + statuscolors=None, +): s3_path_prefix = f"{pr_number}/{commit_sha}/" + check_name.lower().replace( " ", "_" ).replace("(", "_").replace(")", "_").replace(",", "_") - additional_urls = process_logs(s3_client, additional_files, s3_path_prefix) + additional_urls = process_logs( + s3_client, additional_files, s3_path_prefix, test_results, with_raw_logs + ) branch_url = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/commits/master" branch_name = "master" From 8d9dbda7f2f9695dedd30375e10fce3d3891b523 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 2 Dec 2022 21:34:21 +0100 Subject: [PATCH 93/98] Experiment with old upload_result_helper.py --- tests/ci/upload_result_helper.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/upload_result_helper.py b/tests/ci/upload_result_helper.py index e145df02f80..9fcd3733acb 100644 --- a/tests/ci/upload_result_helper.py +++ b/tests/ci/upload_result_helper.py @@ -16,7 +16,7 @@ def process_logs( ): logging.info("Upload files to s3 %s", additional_logs) - processed_logs = {} + processed_logs = {} # type: ignore # Firstly convert paths of logs from test_results to urls to s3. for test_result in test_results: if len(test_result) <= 3 or with_raw_logs: From 8d53d558bde7d95849695ef00165641400352912 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 2 Dec 2022 22:49:26 +0100 Subject: [PATCH 94/98] Restore typing for S3Helper --- tests/ci/s3_helper.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/s3_helper.py b/tests/ci/s3_helper.py index 24ff013d69a..03e855a0057 100644 --- a/tests/ci/s3_helper.py +++ b/tests/ci/s3_helper.py @@ -46,7 +46,7 @@ class S3Helper: self.host = host self.download_host = download_host - def _upload_file_to_s3(self, bucket_name, file_path, s3_path): + def _upload_file_to_s3(self, bucket_name: str, file_path: str, s3_path: str) -> str: logging.debug( "Start uploading %s to bucket=%s path=%s", file_path, bucket_name, s3_path ) @@ -110,7 +110,7 @@ class S3Helper: url = f"{self.download_host}/{bucket_name}/{s3_path}" return url.replace("+", "%2B").replace(" ", "%20") - def upload_test_report_to_s3(self, file_path, s3_path): + def upload_test_report_to_s3(self, file_path: str, s3_path: str) -> str: if CI: return self._upload_file_to_s3(S3_TEST_REPORTS_BUCKET, file_path, s3_path) else: @@ -296,7 +296,7 @@ class S3Helper: return False @staticmethod - def copy_file_to_local(bucket_name, file_path, s3_path): + def copy_file_to_local(bucket_name: str, file_path: str, s3_path: str) -> str: local_path = os.path.abspath( os.path.join(RUNNER_TEMP, "s3", bucket_name, s3_path) ) From 34d420cb3f6648c3d768ce86960aff3416b9e458 Mon Sep 17 00:00:00 2001 From: Amila Welihinda Date: Fri, 2 Dec 2022 14:27:57 -0800 Subject: [PATCH 95/98] fix broken link --- .../sql-reference/aggregate-functions/reference/welchttest.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/welchttest.md b/docs/en/sql-reference/aggregate-functions/reference/welchttest.md index 34f875e2138..1e0b1d88c6e 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/welchttest.md +++ b/docs/en/sql-reference/aggregate-functions/reference/welchttest.md @@ -32,8 +32,8 @@ The null hypothesis is that means of populations are equal. Normal distribution - calculated t-statistic. [Float64](../../../sql-reference/data-types/float.md). - calculated p-value. [Float64](../../../sql-reference/data-types/float.md). -- [calculated confidence-interval-low.] [Float64](../../../sql-reference/data-types/float.md). -- [calculated confidence-interval-high.] [Float64](../../../sql-reference/data-types/float.md). +- calculated confidence-interval-low. [Float64](../../../sql-reference/data-types/float.md). +- calculated confidence-interval-high. [Float64](../../../sql-reference/data-types/float.md). **Example** From ff8f95515c0199ca30f3faf29a16ab430922d003 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Dec 2022 21:22:18 +0100 Subject: [PATCH 96/98] Make vim rc file compatible with neovim neovim reads only .exrc, while vim read both. Signed-off-by: Azat Khuzhin --- .vimrc => .exrc | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename .vimrc => .exrc (100%) diff --git a/.vimrc b/.exrc similarity index 100% rename from .vimrc rename to .exrc From 9c6ea123fe9cb080db517dce5d8129c6f3f252d9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Dec 2022 21:22:50 +0100 Subject: [PATCH 97/98] Fix vim settings (wrong group for autocmd) Signed-off-by: Azat Khuzhin --- .exrc | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.exrc b/.exrc index ba996eb8a42..162bd41ce4f 100644 --- a/.exrc +++ b/.exrc @@ -1,2 +1 @@ -au BufRead,BufNewFile ./* set tabstop=4 softtabstop=0 expandtab shiftwidth=4 smarttab tags=tags,../tags - +au BufRead,BufNewFile * set tabstop=4 softtabstop=0 expandtab shiftwidth=4 smarttab tags=tags,../tags From 408cea80ebfddc943304055529acc6522d4d58d8 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 3 Dec 2022 22:58:46 +0100 Subject: [PATCH 98/98] impl --- src/Databases/TablesDependencyGraph.cpp | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/src/Databases/TablesDependencyGraph.cpp b/src/Databases/TablesDependencyGraph.cpp index 9461be7024b..c4c361089ad 100644 --- a/src/Databases/TablesDependencyGraph.cpp +++ b/src/Databases/TablesDependencyGraph.cpp @@ -40,11 +40,14 @@ TablesDependencyGraph::TablesDependencyGraph(TablesDependencyGraph && src) noexc TablesDependencyGraph & TablesDependencyGraph::operator=(const TablesDependencyGraph & src) { - nodes = src.nodes; - nodes_by_database_and_table_names = src.nodes_by_database_and_table_names; - nodes_by_uuid = src.nodes_by_uuid; - levels_calculated = src.levels_calculated; - nodes_sorted_by_level_lazy = src.nodes_sorted_by_level_lazy; + if (&src != this) + { + nodes = src.nodes; + nodes_by_database_and_table_names = src.nodes_by_database_and_table_names; + nodes_by_uuid = src.nodes_by_uuid; + levels_calculated = src.levels_calculated; + nodes_sorted_by_level_lazy = src.nodes_sorted_by_level_lazy; + } return *this; } @@ -358,7 +361,7 @@ size_t TablesDependencyGraph::removeIsolatedTables() auto it = nodes.begin(); while (it != nodes.end()) { - auto current = (it++)->get(); + auto * current = (it++)->get(); if (current->dependencies.empty() && current->dependents.empty()) { removeNode(current);