From 125e1bd1821af0562a65f5ea1a0955345165982e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 8 Jun 2022 09:22:46 +0000 Subject: [PATCH 01/40] Initial version for linearizable reads --- src/Coordination/KeeperDispatcher.cpp | 96 ++++++++++++++++++++++++- src/Coordination/KeeperDispatcher.h | 5 ++ src/Coordination/KeeperServer.cpp | 29 +++++++- src/Coordination/KeeperServer.h | 29 +++++++- src/Coordination/KeeperStateMachine.cpp | 5 +- src/Coordination/KeeperStateMachine.h | 7 +- 6 files changed, 165 insertions(+), 6 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 9ad5fe9e8ed..299768e170d 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -244,6 +244,40 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ request_info.time = duration_cast(system_clock::now().time_since_epoch()).count(); request_info.session_id = session_id; + if (request->isReadRequest()) + { + auto leader_info_result = server->getLeaderInfo(); + + auto & leader_info_ctx = leader_info_result->get(); + + /// If we get some errors, than send them to clients + if (!leader_info_result->get_accepted() || leader_info_result->get_result_code() == nuraft::cmd_result_code::TIMEOUT) + { + addErrorResponses({ request_info }, Coordination::Error::ZOPERATIONTIMEOUT); + return false; + } + else if (leader_info_result->get_result_code() != nuraft::cmd_result_code::OK) + { + addErrorResponses({ request_info }, Coordination::Error::ZCONNECTIONLOSS); + return false; + } + + KeeperServer::LeaderInfo leader_info; + leader_info.term = leader_info_ctx->get_ulong(); + leader_info.last_committed_index = leader_info_ctx->get_ulong(); + LOG_INFO(log, "Leader is {}, idx {}", leader_info.term, leader_info.last_committed_index); + + auto current_status = server->getCurrentState(); + LOG_INFO(log, "Current is {}, idx {}", current_status.term, current_status.last_committed_index); + + if (current_status.term < leader_info.term || current_status.last_committed_index < leader_info.last_committed_index) + { + std::lock_guard lock(leader_waiter_mutex); + leader_waiters[leader_info].push_back(std::move(request_info)); + return true; + } + } + std::lock_guard lock(push_request_mutex); if (shutdown_called) @@ -273,7 +307,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf responses_thread = ThreadFromGlobalPool([this] { responseThread(); }); snapshot_thread = ThreadFromGlobalPool([this] { snapshotThread(); }); - server = std::make_unique(configuration_and_settings, config, responses_queue, snapshots_queue); + server = std::make_unique(configuration_and_settings, config, responses_queue, snapshots_queue, [this](KeeperStorage::RequestForSession & request_for_session, uint64_t log_idx) { onRequestCommit(request_for_session, log_idx); }); try { @@ -590,6 +624,66 @@ void KeeperDispatcher::updateConfigurationThread() } } +<<<<<<< HEAD +======= +void KeeperDispatcher::onRequestCommit(KeeperStorage::RequestForSession & request_for_session, uint64_t log_idx) +{ + const auto committed_zxid = request_for_session.zxid; + + const auto process_requests = [this](auto & request_queue) + { + for (auto & request_info : request_queue) + { + std::lock_guard lock(push_request_mutex); + + if (shutdown_called) + return; + + /// Put close requests without timeouts + if (request_info.request->getOpNum() == Coordination::OpNum::Close) + { + if (!requests_queue->push(std::move(request_info))) + throw Exception("Cannot push request to queue", ErrorCodes::SYSTEM_ERROR); + } + else if (!requests_queue->tryPush(std::move(request_info), configuration_and_settings->coordination_settings->operation_timeout_ms.totalMilliseconds())) + { + throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::TIMEOUT_EXCEEDED); + } + } + + request_queue.clear(); + }; + { + std::lock_guard lock(session_waiter_mutex); + + for (auto it = session_waiters.begin(); it != session_waiters.end();) + { + auto & [session_id, session_waiter] = *it; + + if (session_waiter.wait_for_zxid <= committed_zxid) + { + process_requests(session_waiter.request_queue); + it = session_waiters.erase(it); + } + else + ++it; + } + } + + { + std::lock_guard lock(leader_waiter_mutex); + auto current_status = server->getCurrentState(); + LOG_INFO(log, "Got term {}, idx {}", current_status.term, log_idx); + auto request_queue_it = leader_waiters.find(KeeperServer::LeaderInfo{.term = current_status.term, .last_committed_index = log_idx}); + if (request_queue_it != leader_waiters.end()) + { + process_requests(request_queue_it->second); + leader_waiters.erase(request_queue_it); + } + } +} + +>>>>>>> Initial version for linearizable reads bool KeeperDispatcher::isServerActive() const { return checkInit() && hasLeader() && !server->isRecovering(); diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index b632327a165..9668fb34653 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -77,6 +77,9 @@ private: /// Counter for new session_id requests. std::atomic internal_session_id_counter{0}; + std::unordered_map> leader_waiters; + std::mutex leader_waiter_mutex; + /// Thread put requests to raft void requestThread(); /// Thread put responses for subscribed sessions @@ -116,6 +119,8 @@ public: return server && server->checkInit(); } + void onRequestCommit(KeeperStorage::RequestForSession & request_for_session, uint64_t log_idx); + /// Is server accepting requests, i.e. connected to the cluster /// and achieved quorum bool isServerActive() const; diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index d4c188fe8d9..6b48221a0db 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -102,7 +102,8 @@ KeeperServer::KeeperServer( const KeeperConfigurationAndSettingsPtr & configuration_and_settings_, const Poco::Util::AbstractConfiguration & config, ResponsesQueue & responses_queue_, - SnapshotsQueue & snapshots_queue_) + SnapshotsQueue & snapshots_queue_, + KeeperStateMachine::CommitCallback commit_callback) : server_id(configuration_and_settings_->server_id) , coordination_settings(configuration_and_settings_->coordination_settings) , state_machine(nuraft::cs_new( @@ -111,7 +112,8 @@ KeeperServer::KeeperServer( configuration_and_settings_->snapshot_storage_path, coordination_settings, checkAndGetSuperdigest(configuration_and_settings_->super_digest), - config.getBool("keeper_server.digest_enabled", true))) + config.getBool("keeper_server.digest_enabled", true), + std::move(commit_callback))) , state_manager(nuraft::cs_new( server_id, "keeper_server", configuration_and_settings_->log_storage_path, config, coordination_settings)) , log(&Poco::Logger::get("KeeperServer")) @@ -161,6 +163,19 @@ struct KeeperServer::KeeperRaftServer : public nuraft::raft_server reconfigure(new_config); } + RaftAppendResult getLeaderInfo() + { + nuraft::ptr req = nuraft::cs_new + ( 0ull, nuraft::msg_type::leader_status_request, 0, 0, + 0ull, 0ull, 0ull ) ; + auto result = send_msg_to_leader(req); + + if (!result->has_result()) + result->get(); + + return result; + } + using nuraft::raft_server::raft_server; // peers are initially marked as responding because at least one cycle @@ -637,6 +652,16 @@ std::vector KeeperServer::getDeadSessions() return state_machine->getDeadSessions(); } +RaftAppendResult KeeperServer::getLeaderInfo() +{ + return raft_instance->getLeaderInfo(); +} + +KeeperServer::LeaderInfo KeeperServer::getCurrentState() +{ + return { .term = raft_instance->get_term(), .last_committed_index = state_machine->last_commit_index() }; +} + ConfigUpdateActions KeeperServer::getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) { auto diff = state_manager->getConfigurationDiff(config); diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 8c21cf47d94..a18e1a20c5c 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -65,7 +65,8 @@ public: const KeeperConfigurationAndSettingsPtr & settings_, const Poco::Util::AbstractConfiguration & config_, ResponsesQueue & responses_queue_, - SnapshotsQueue & snapshots_queue_); + SnapshotsQueue & snapshots_queue_, + KeeperStateMachine::CommitCallback commit_callback); /// Load state machine from the latest snapshot and load log storage. Start NuRaft with required settings. void startup(const Poco::Util::AbstractConfiguration & config, bool enable_ipv6 = true); @@ -111,6 +112,17 @@ public: int getServerID() const { return server_id; } + struct LeaderInfo + { + uint64_t term; + uint64_t last_committed_index; + + bool operator==(const LeaderInfo &) const = default; + }; + + RaftAppendResult getLeaderInfo(); + LeaderInfo getCurrentState(); + /// Get configuration diff between current configuration in RAFT and in XML file ConfigUpdateActions getConfigurationDiff(const Poco::Util::AbstractConfiguration & config); @@ -125,3 +137,18 @@ public: }; } +namespace std { + + template <> + struct hash + { + size_t operator()(const DB::KeeperServer::LeaderInfo & info) const + { + SipHash hash_state; + hash_state.update(info.term); + hash_state.update(info.last_committed_index); + return hash_state.get64(); + } + }; + +} diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 368b23f34d2..db0d0a09e3e 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -29,7 +29,8 @@ KeeperStateMachine::KeeperStateMachine( const std::string & snapshots_path_, const CoordinationSettingsPtr & coordination_settings_, const std::string & superdigest_, - const bool digest_enabled_) + const bool digest_enabled_, + CommitCallback commit_callback_) : coordination_settings(coordination_settings_) , snapshot_manager( snapshots_path_, @@ -44,6 +45,7 @@ KeeperStateMachine::KeeperStateMachine( , log(&Poco::Logger::get("KeeperStateMachine")) , superdigest(superdigest_) , digest_enabled(digest_enabled_) + , commit_callback(std::move(commit_callback_)) { } @@ -240,6 +242,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n } } + commit_callback(request_for_session, log_idx); last_committed_idx = log_idx; return nullptr; } diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index c80b35bb704..cfebc81cbf8 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -19,13 +19,16 @@ using SnapshotsQueue = ConcurrentBoundedQueue; class KeeperStateMachine : public nuraft::state_machine { public: + using CommitCallback = std::function; + KeeperStateMachine( ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, const std::string & snapshots_path_, const CoordinationSettingsPtr & coordination_settings_, const std::string & superdigest_ = "", - bool digest_enabled_ = true); + bool digest_enabled_ = true, + CommitCallback commit_callback_ = [](KeeperStorage::RequestForSession &, uint64_t){}); /// Read state from the latest snapshot void init(); @@ -141,6 +144,8 @@ private: const std::string superdigest; const bool digest_enabled; + + const CommitCallback commit_callback; }; } From fa8bfced1af63ca30963c7614ecb1aca944bfe2a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 8 Jun 2022 09:23:58 +0000 Subject: [PATCH 02/40] Update NuRaft --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index 1334b9ae725..6cf55ad8c34 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 1334b9ae72576821a698d657d08838861cf33007 +Subproject commit 6cf55ad8c3426cb8b65e4225baf8a182dd48fc7d From 9fe06019c0cb9184585307878187d3fec66366e1 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 17 Jun 2022 18:42:46 +0000 Subject: [PATCH 03/40] Some small fixes --- src/Coordination/KeeperDispatcher.cpp | 39 +++++++-------------------- src/Coordination/KeeperDispatcher.h | 2 +- src/Coordination/KeeperServer.cpp | 2 +- src/Coordination/KeeperServer.h | 11 ++++---- 4 files changed, 16 insertions(+), 38 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 299768e170d..e9d1c60de1d 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -250,7 +250,7 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ auto & leader_info_ctx = leader_info_result->get(); - /// If we get some errors, than send them to clients + /// If we get some errors, then send them to clients if (!leader_info_result->get_accepted() || leader_info_result->get_result_code() == nuraft::cmd_result_code::TIMEOUT) { addErrorResponses({ request_info }, Coordination::Error::ZOPERATIONTIMEOUT); @@ -262,18 +262,18 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ return false; } - KeeperServer::LeaderInfo leader_info; + KeeperServer::NodeInfo leader_info; leader_info.term = leader_info_ctx->get_ulong(); leader_info.last_committed_index = leader_info_ctx->get_ulong(); LOG_INFO(log, "Leader is {}, idx {}", leader_info.term, leader_info.last_committed_index); - auto current_status = server->getCurrentState(); - LOG_INFO(log, "Current is {}, idx {}", current_status.term, current_status.last_committed_index); + auto node_info = server->getNodeInfo(); + LOG_INFO(log, "Current is {}, idx {}", node_info.term, node_info.last_committed_index); - if (current_status.term < leader_info.term || current_status.last_committed_index < leader_info.last_committed_index) + if (node_info.term < leader_info.term || node_info.last_committed_index < leader_info.last_committed_index) { std::lock_guard lock(leader_waiter_mutex); - leader_waiters[leader_info].push_back(std::move(request_info)); + leader_waiters[node_info].push_back(std::move(request_info)); return true; } } @@ -624,12 +624,8 @@ void KeeperDispatcher::updateConfigurationThread() } } -<<<<<<< HEAD -======= -void KeeperDispatcher::onRequestCommit(KeeperStorage::RequestForSession & request_for_session, uint64_t log_idx) +void KeeperDispatcher::onRequestCommit(KeeperStorage::RequestForSession & /* request_for_session */, uint64_t log_idx) { - const auto committed_zxid = request_for_session.zxid; - const auto process_requests = [this](auto & request_queue) { for (auto & request_info : request_queue) @@ -653,28 +649,12 @@ void KeeperDispatcher::onRequestCommit(KeeperStorage::RequestForSession & reques request_queue.clear(); }; - { - std::lock_guard lock(session_waiter_mutex); - - for (auto it = session_waiters.begin(); it != session_waiters.end();) - { - auto & [session_id, session_waiter] = *it; - - if (session_waiter.wait_for_zxid <= committed_zxid) - { - process_requests(session_waiter.request_queue); - it = session_waiters.erase(it); - } - else - ++it; - } - } { std::lock_guard lock(leader_waiter_mutex); - auto current_status = server->getCurrentState(); + auto current_status = server->getNodeInfo(); LOG_INFO(log, "Got term {}, idx {}", current_status.term, log_idx); - auto request_queue_it = leader_waiters.find(KeeperServer::LeaderInfo{.term = current_status.term, .last_committed_index = log_idx}); + auto request_queue_it = leader_waiters.find(KeeperServer::NodeInfo{.term = current_status.term, .last_committed_index = log_idx}); if (request_queue_it != leader_waiters.end()) { process_requests(request_queue_it->second); @@ -683,7 +663,6 @@ void KeeperDispatcher::onRequestCommit(KeeperStorage::RequestForSession & reques } } ->>>>>>> Initial version for linearizable reads bool KeeperDispatcher::isServerActive() const { return checkInit() && hasLeader() && !server->isRecovering(); diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 9668fb34653..1140457d56b 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -77,7 +77,7 @@ private: /// Counter for new session_id requests. std::atomic internal_session_id_counter{0}; - std::unordered_map> leader_waiters; + std::unordered_map> leader_waiters; std::mutex leader_waiter_mutex; /// Thread put requests to raft diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 6b48221a0db..b1b8a0976d2 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -657,7 +657,7 @@ RaftAppendResult KeeperServer::getLeaderInfo() return raft_instance->getLeaderInfo(); } -KeeperServer::LeaderInfo KeeperServer::getCurrentState() +KeeperServer::NodeInfo KeeperServer::getNodeInfo() { return { .term = raft_instance->get_term(), .last_committed_index = state_machine->last_commit_index() }; } diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index a18e1a20c5c..472cb67361c 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -112,16 +112,16 @@ public: int getServerID() const { return server_id; } - struct LeaderInfo + struct NodeInfo { uint64_t term; uint64_t last_committed_index; - bool operator==(const LeaderInfo &) const = default; + bool operator==(const NodeInfo &) const = default; }; RaftAppendResult getLeaderInfo(); - LeaderInfo getCurrentState(); + NodeInfo getNodeInfo(); /// Get configuration diff between current configuration in RAFT and in XML file ConfigUpdateActions getConfigurationDiff(const Poco::Util::AbstractConfiguration & config); @@ -130,7 +130,6 @@ public: /// Synchronously check for update results with retries. void applyConfigurationUpdate(const ConfigUpdateAction & task); - /// Wait configuration update for action. Used by followers. /// Return true if update was successfully received. bool waitConfigurationUpdate(const ConfigUpdateAction & task); @@ -140,9 +139,9 @@ public: namespace std { template <> - struct hash + struct hash { - size_t operator()(const DB::KeeperServer::LeaderInfo & info) const + size_t operator()(const DB::KeeperServer::NodeInfo & info) const { SipHash hash_state; hash_state.update(info.term); From d2f3d581eb008ee08191b76a7d45e83324695ce2 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sat, 18 Jun 2022 18:43:02 +0000 Subject: [PATCH 04/40] Use fast lin reads in jepsen --- src/Coordination/CoordinationSettings.h | 1 + src/Coordination/KeeperDispatcher.cpp | 11 ++++------- src/Coordination/KeeperDispatcher.h | 2 +- src/Coordination/KeeperServer.cpp | 1 + src/Coordination/KeeperStateMachine.cpp | 10 +++++----- src/Coordination/KeeperStateMachine.h | 6 +++--- .../resources/keeper_config.xml | 3 ++- .../src/jepsen/clickhouse_keeper/register.clj | 3 ++- 8 files changed, 19 insertions(+), 18 deletions(-) diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index b702b553a03..c1e41f46670 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -26,6 +26,7 @@ struct Settings; M(Milliseconds, heart_beat_interval_ms, 500, "Heartbeat interval between quorum nodes", 0) \ M(Milliseconds, election_timeout_lower_bound_ms, 1000, "Lower bound of election timer (avoid too often leader elections)", 0) \ M(Milliseconds, election_timeout_upper_bound_ms, 2000, "Upper bound of election timer (avoid too often leader elections)", 0) \ + M(Milliseconds, leadership_expiry, 10000, "How often will leader node check if it still has majority. Set it lower or equal to election_timeout_lower_bound_ms to have linearizable reads.", 0) \ M(UInt64, reserved_log_items, 100000, "How many log items to store (don't remove during compaction)", 0) \ M(UInt64, snapshot_distance, 100000, "How many log items we have to collect to write new snapshot", 0) \ M(Bool, auto_forwarding, true, "Allow to forward write requests from followers to leader", 0) \ diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index e9d1c60de1d..a7b40e9f128 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -265,15 +265,14 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ KeeperServer::NodeInfo leader_info; leader_info.term = leader_info_ctx->get_ulong(); leader_info.last_committed_index = leader_info_ctx->get_ulong(); - LOG_INFO(log, "Leader is {}, idx {}", leader_info.term, leader_info.last_committed_index); auto node_info = server->getNodeInfo(); - LOG_INFO(log, "Current is {}, idx {}", node_info.term, node_info.last_committed_index); if (node_info.term < leader_info.term || node_info.last_committed_index < leader_info.last_committed_index) { std::lock_guard lock(leader_waiter_mutex); leader_waiters[node_info].push_back(std::move(request_info)); + LOG_INFO(log, "waiting for {}, idx {}", leader_info.term, leader_info.last_committed_index); return true; } } @@ -307,7 +306,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf responses_thread = ThreadFromGlobalPool([this] { responseThread(); }); snapshot_thread = ThreadFromGlobalPool([this] { snapshotThread(); }); - server = std::make_unique(configuration_and_settings, config, responses_queue, snapshots_queue, [this](KeeperStorage::RequestForSession & request_for_session, uint64_t log_idx) { onRequestCommit(request_for_session, log_idx); }); + server = std::make_unique(configuration_and_settings, config, responses_queue, snapshots_queue, [this](uint64_t log_term, uint64_t log_idx) { onRequestCommit(log_term, log_idx); }); try { @@ -624,7 +623,7 @@ void KeeperDispatcher::updateConfigurationThread() } } -void KeeperDispatcher::onRequestCommit(KeeperStorage::RequestForSession & /* request_for_session */, uint64_t log_idx) +void KeeperDispatcher::onRequestCommit(uint64_t log_term, uint64_t log_idx) { const auto process_requests = [this](auto & request_queue) { @@ -652,9 +651,7 @@ void KeeperDispatcher::onRequestCommit(KeeperStorage::RequestForSession & /* req { std::lock_guard lock(leader_waiter_mutex); - auto current_status = server->getNodeInfo(); - LOG_INFO(log, "Got term {}, idx {}", current_status.term, log_idx); - auto request_queue_it = leader_waiters.find(KeeperServer::NodeInfo{.term = current_status.term, .last_committed_index = log_idx}); + auto request_queue_it = leader_waiters.find(KeeperServer::NodeInfo{.term = log_term, .last_committed_index = log_idx}); if (request_queue_it != leader_waiters.end()) { process_requests(request_queue_it->second); diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 1140457d56b..1f5de4007f2 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -119,7 +119,7 @@ public: return server && server->checkInit(); } - void onRequestCommit(KeeperStorage::RequestForSession & request_for_session, uint64_t log_idx); + void onRequestCommit(uint64_t log_term, uint64_t log_idx); /// Is server accepting requests, i.e. connected to the cluster /// and achieved quorum diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index b1b8a0976d2..0a7b1948495 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -258,6 +258,7 @@ void KeeperServer::launchRaftServer(bool enable_ipv6) coordination_settings->election_timeout_lower_bound_ms.totalMilliseconds(), "election_timeout_lower_bound_ms", log); params.election_timeout_upper_bound_ = getValueOrMaxInt32AndLogWarning( coordination_settings->election_timeout_upper_bound_ms.totalMilliseconds(), "election_timeout_upper_bound_ms", log); + params.leadership_expiry_ = getValueOrMaxInt32AndLogWarning(coordination_settings->leadership_expiry.totalMilliseconds(), "leadership_expiry", log); params.reserved_log_items_ = getValueOrMaxInt32AndLogWarning(coordination_settings->reserved_log_items, "reserved_log_items", log); params.snapshot_distance_ = getValueOrMaxInt32AndLogWarning(coordination_settings->snapshot_distance, "snapshot_distance", log); diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index db0d0a09e3e..8080f373497 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -197,11 +197,11 @@ void KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req assertDigest(*request_for_session.digest, storage->getNodesDigest(false), *request_for_session.request, false); } -nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data) +nuraft::ptr KeeperStateMachine::commit_ext(const ext_op_params& params) { - auto request_for_session = parseRequest(data); + auto request_for_session = parseRequest(*params.data); if (!request_for_session.zxid) - request_for_session.zxid = log_idx; + request_for_session.zxid = params.log_idx; /// Special processing of session_id request if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) @@ -242,8 +242,8 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n } } - commit_callback(request_for_session, log_idx); - last_committed_idx = log_idx; + commit_callback(params.log_term, params.log_idx); + last_committed_idx = params.log_idx; return nullptr; } diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index cfebc81cbf8..f5a38228c86 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -19,7 +19,7 @@ using SnapshotsQueue = ConcurrentBoundedQueue; class KeeperStateMachine : public nuraft::state_machine { public: - using CommitCallback = std::function; + using CommitCallback = std::function; KeeperStateMachine( ResponsesQueue & responses_queue_, @@ -28,7 +28,7 @@ public: const CoordinationSettingsPtr & coordination_settings_, const std::string & superdigest_ = "", bool digest_enabled_ = true, - CommitCallback commit_callback_ = [](KeeperStorage::RequestForSession &, uint64_t){}); + CommitCallback commit_callback_ = [](uint64_t, uint64_t){}); /// Read state from the latest snapshot void init(); @@ -39,7 +39,7 @@ public: nuraft::ptr pre_commit(uint64_t log_idx, nuraft::buffer & data) override; - nuraft::ptr commit(const uint64_t log_idx, nuraft::buffer & data) override; /// NOLINT + nuraft::ptr commit_ext(const ext_op_params& params) override; /// NOLINT /// Save new cluster config to our snapshot (copy of the config stored in StateManager) void commit_config(const uint64_t log_idx, nuraft::ptr & new_conf) override; /// NOLINT diff --git a/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml b/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml index 2ab747fbd71..41bdcdbd2b4 100644 --- a/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml +++ b/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml @@ -1,5 +1,5 @@ - :: + 0.0.0.0 trace @@ -19,6 +19,7 @@ 120000 trace 1000 + 1900 2000 4000 {quorum_reads} diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/register.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/register.clj index b2f381168bd..fb2375e6c54 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/register.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/register.clj @@ -20,7 +20,8 @@ (assoc this :conn (zk-connect node 9181 30000))) (setup! [this test] - (zk-create-range conn 300)) ; 300 nodes to be sure + (exec-with-retries 30 (fn [] + (zk-create-range conn 300)))) (invoke! [_ test op] (let [[k v] (:value op) From d56894e04e69f8f0a36297491e6f8c8a6588ae57 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sat, 18 Jun 2022 19:16:17 +0000 Subject: [PATCH 05/40] Don't rollback session_id --- src/Coordination/KeeperStateMachine.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 8080f373497..c2bd7772b1b 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -286,6 +286,10 @@ void KeeperStateMachine::commit_config(const uint64_t /* log_idx */, nuraft::ptr void KeeperStateMachine::rollback(uint64_t log_idx, nuraft::buffer & data) { auto request_for_session = parseRequest(data); + + if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) + return; + // If we received a log from an older node, use the log_idx as the zxid // log_idx will always be larger or equal to the zxid so we can safely do this // (log_idx is increased for all logs, while zxid is only increased for requests) From e484dff9257b46494abf5a793151fd110f5b0641 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sat, 18 Jun 2022 19:19:26 +0000 Subject: [PATCH 06/40] Disable quorum reads --- tests/jepsen.clickhouse-keeper/resources/keeper_config.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml b/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml index 41bdcdbd2b4..db857693a7b 100644 --- a/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml +++ b/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml @@ -22,7 +22,7 @@ 1900 2000 4000 - {quorum_reads} + 0 {snapshot_distance} {stale_log_gap} {reserved_log_items} From fdef4cc44dde8af61c4ade92fceb79efd62ff9cf Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sat, 18 Jun 2022 19:25:43 +0000 Subject: [PATCH 07/40] Update NuRaft --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index 6cf55ad8c34..31d44b03092 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 6cf55ad8c3426cb8b65e4225baf8a182dd48fc7d +Subproject commit 31d44b030928258d14e7d6bdfb8682d11f2bd7bf From ecd545fe4f9b1742f59dac1512f85f6029e79927 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 24 Jun 2022 11:36:12 +0000 Subject: [PATCH 08/40] Batch read requests --- src/Coordination/KeeperDispatcher.cpp | 164 ++++++++++++-------------- 1 file changed, 75 insertions(+), 89 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index a7b40e9f128..b07997882e5 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -39,6 +39,13 @@ void KeeperDispatcher::requestThread() /// to send errors to the client. KeeperStorage::RequestsForSessions prev_batch; + std::optional previous_request; + bool collecting_quorum_requests = false; + const auto needs_quorum = [&](const auto & coordination_settings, const auto & request) + { + return coordination_settings->quorum_reads || !request.request->isReadRequest(); + }; + while (!shutdown_called) { KeeperStorage::RequestForSession request; @@ -55,18 +62,20 @@ void KeeperDispatcher::requestThread() /// read request. So reads are some kind of "separator" for writes. try { + KeeperStorage::RequestsForSessions current_batch; + + if (previous_request) + { + current_batch.push_back(std::move(*previous_request)); + previous_request.reset(); + } + if (requests_queue->tryPop(request, max_wait)) { if (shutdown_called) break; - KeeperStorage::RequestsForSessions current_batch; - - bool has_read_request = false; - - /// If new request is not read request or we must to process it through quorum. - /// Otherwise we will process it locally. - if (coordination_settings->quorum_reads || !request.request->isReadRequest()) + if (collecting_quorum_requests == needs_quorum(coordination_settings, request)) { current_batch.emplace_back(request); @@ -79,16 +88,14 @@ void KeeperDispatcher::requestThread() if (requests_queue->tryPop(request, 1)) { /// Don't append read request into batch, we have to process them separately - if (!coordination_settings->quorum_reads && request.request->isReadRequest()) + if (collecting_quorum_requests != needs_quorum(coordination_settings, request)) { - has_read_request = true; + previous_request.emplace(std::move(request)); + collecting_quorum_requests = !collecting_quorum_requests; break; } - else - { - current_batch.emplace_back(request); - } + current_batch.emplace_back(request); } if (shutdown_called) @@ -96,7 +103,10 @@ void KeeperDispatcher::requestThread() } } else - has_read_request = true; + { + previous_request.emplace(std::move(request)); + collecting_quorum_requests = !collecting_quorum_requests; + } if (shutdown_called) break; @@ -105,34 +115,59 @@ void KeeperDispatcher::requestThread() if (prev_result) forceWaitAndProcessResult(prev_result, prev_batch); - /// Process collected write requests batch + /// Process collected requests batch if (!current_batch.empty()) { - auto result = server->putRequestBatch(current_batch); - - if (result) + if (collecting_quorum_requests) { - if (has_read_request) /// If we will execute read request next, than we have to process result now - forceWaitAndProcessResult(result, current_batch); + auto result = server->putRequestBatch(current_batch); + prev_batch = std::move(current_batch); + prev_result = result; } else { - addErrorResponses(current_batch, Coordination::Error::ZCONNECTIONLOSS); - current_batch.clear(); + auto leader_info_result = server->getLeaderInfo(); + + auto & leader_info_ctx = leader_info_result->get(); + + /// If we get some errors, then send them to clients + if (!leader_info_result->get_accepted() || leader_info_result->get_result_code() == nuraft::cmd_result_code::TIMEOUT) + { + addErrorResponses(current_batch, Coordination::Error::ZOPERATIONTIMEOUT); + } + else if (leader_info_result->get_result_code() != nuraft::cmd_result_code::OK) + { + addErrorResponses(current_batch, Coordination::Error::ZCONNECTIONLOSS); + } + else + { + KeeperServer::NodeInfo leader_info; + leader_info.term = leader_info_ctx->get_ulong(); + leader_info.last_committed_index = leader_info_ctx->get_ulong(); + + std::lock_guard lock(leader_waiter_mutex); + auto node_info = server->getNodeInfo(); + + if (node_info.term < leader_info.term || node_info.last_committed_index < leader_info.last_committed_index) + { + auto & leader_waiter = leader_waiters[node_info]; + leader_waiter.insert(leader_waiter.end(), current_batch.begin(), current_batch.end()); + LOG_INFO(log, "waiting for {}, idx {}", leader_info.term, leader_info.last_committed_index); + } + else + { + for (const auto & read_request : current_batch) + { + if (server->isLeaderAlive()) + server->putLocalReadRequest(read_request); + else + addErrorResponses({read_request}, Coordination::Error::ZCONNECTIONLOSS); + } + } + } } - - prev_batch = std::move(current_batch); - prev_result = result; - } - - /// Read request always goes after write batch (last request) - if (has_read_request) - { - if (server->isLeaderAlive()) - server->putLocalReadRequest(request); - else - addErrorResponses({request}, Coordination::Error::ZCONNECTIONLOSS); } + current_batch.clear(); } } catch (...) @@ -244,39 +279,6 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ request_info.time = duration_cast(system_clock::now().time_since_epoch()).count(); request_info.session_id = session_id; - if (request->isReadRequest()) - { - auto leader_info_result = server->getLeaderInfo(); - - auto & leader_info_ctx = leader_info_result->get(); - - /// If we get some errors, then send them to clients - if (!leader_info_result->get_accepted() || leader_info_result->get_result_code() == nuraft::cmd_result_code::TIMEOUT) - { - addErrorResponses({ request_info }, Coordination::Error::ZOPERATIONTIMEOUT); - return false; - } - else if (leader_info_result->get_result_code() != nuraft::cmd_result_code::OK) - { - addErrorResponses({ request_info }, Coordination::Error::ZCONNECTIONLOSS); - return false; - } - - KeeperServer::NodeInfo leader_info; - leader_info.term = leader_info_ctx->get_ulong(); - leader_info.last_committed_index = leader_info_ctx->get_ulong(); - - auto node_info = server->getNodeInfo(); - - if (node_info.term < leader_info.term || node_info.last_committed_index < leader_info.last_committed_index) - { - std::lock_guard lock(leader_waiter_mutex); - leader_waiters[node_info].push_back(std::move(request_info)); - LOG_INFO(log, "waiting for {}, idx {}", leader_info.term, leader_info.last_committed_index); - return true; - } - } - std::lock_guard lock(push_request_mutex); if (shutdown_called) @@ -629,34 +631,18 @@ void KeeperDispatcher::onRequestCommit(uint64_t log_term, uint64_t log_idx) { for (auto & request_info : request_queue) { - std::lock_guard lock(push_request_mutex); - - if (shutdown_called) - return; - - /// Put close requests without timeouts - if (request_info.request->getOpNum() == Coordination::OpNum::Close) - { - if (!requests_queue->push(std::move(request_info))) - throw Exception("Cannot push request to queue", ErrorCodes::SYSTEM_ERROR); - } - else if (!requests_queue->tryPush(std::move(request_info), configuration_and_settings->coordination_settings->operation_timeout_ms.totalMilliseconds())) - { - throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::TIMEOUT_EXCEEDED); - } + server->putLocalReadRequest(request_info); } request_queue.clear(); }; + std::lock_guard lock(leader_waiter_mutex); + auto request_queue_it = leader_waiters.find(KeeperServer::NodeInfo{.term = log_term, .last_committed_index = log_idx}); + if (request_queue_it != leader_waiters.end()) { - std::lock_guard lock(leader_waiter_mutex); - auto request_queue_it = leader_waiters.find(KeeperServer::NodeInfo{.term = log_term, .last_committed_index = log_idx}); - if (request_queue_it != leader_waiters.end()) - { - process_requests(request_queue_it->second); - leader_waiters.erase(request_queue_it); - } + process_requests(request_queue_it->second); + leader_waiters.erase(request_queue_it); } } From 197ae25aa3ec53ff2affc4e61a42cff438e3f9e3 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 24 Jun 2022 11:38:09 +0000 Subject: [PATCH 09/40] Allow only jepsen test --- .github/workflows/pull_request.yml | 3348 +--------------------------- 1 file changed, 1 insertion(+), 3347 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 79d54d77f06..f021554dd7b 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -109,196 +109,11 @@ jobs: with: name: changed_images path: ${{ runner.temp }}/changed_images.json - StyleCheck: - needs: DockerHubPush - runs-on: [self-hosted, style-checker] - if: ${{ success() || failure() }} - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{ runner.temp }}/style_check - EOF - - name: Download changed images - # even if artifact does not exist, e.g. on `do not test` label or failed Docker job - continue-on-error: true - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.TEMP_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Style Check - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 style_check.py - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FastTest: - needs: DockerHubPush - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/fasttest - REPO_COPY=${{runner.temp}}/fasttest/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - EOF - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" - mkdir "$GITHUB_WORKSPACE" - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.TEMP_PATH }} - - name: Fast Test - run: | - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 fast_test_check.py - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - CompatibilityCheck: - needs: [BuilderDebRelease] - runs-on: [self-hosted, style-checker] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/compatibility_check - REPO_COPY=${{runner.temp}}/compatibility_check/ClickHouse - REPORTS_PATH=${{runner.temp}}/reports_dir - EOF - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: CompatibilityCheck - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - SplitBuildSmokeTest: - needs: [BuilderDebSplitted] - runs-on: [self-hosted, style-checker] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/split_build_check - REPO_COPY=${{runner.temp}}/split_build_check/ClickHouse - REPORTS_PATH=${{runner.temp}}/reports_dir - EOF - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Split build check - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 split_build_smoke_check.py - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" ######################################################################################### #################################### ORDINARY BUILDS #################################### ######################################################################################### - BuilderDebRelease: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=package_release - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - with: - fetch-depth: 0 # for performance artifact - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" BuilderBinRelease: - needs: [DockerHubPush, FastTest] + needs: [DockerHubPush] runs-on: [self-hosted, builder] steps: - name: Set envs @@ -342,3170 +157,9 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - # BuilderBinGCC: - # needs: [DockerHubPush, FastTest] - # runs-on: [self-hosted, builder] - # steps: - # - name: Set envs - # run: | - # cat >> "$GITHUB_ENV" << 'EOF' - # TEMP_PATH=${{runner.temp}}/build_check - # IMAGES_PATH=${{runner.temp}}/images_path - # REPO_COPY=${{runner.temp}}/build_check/ClickHouse - # CACHES_PATH=${{runner.temp}}/../ccaches - # BUILD_NAME=binary_gcc - # EOF - # - name: Download changed images - # uses: actions/download-artifact@v2 - # with: - # name: changed_images - # path: ${{ runner.temp }}/images_path - # - name: Clear repository - # run: | - # sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - # - name: Check out repository code - # uses: actions/checkout@v2 - # - name: Build - # run: | - # git -C "$GITHUB_WORKSPACE" submodule sync --recursive - # git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - # sudo rm -fr "$TEMP_PATH" - # mkdir -p "$TEMP_PATH" - # cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - # cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - # - name: Upload build URLs to artifacts - # if: ${{ success() || failure() }} - # uses: actions/upload-artifact@v2 - # with: - # name: ${{ env.BUILD_URLS }} - # path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - # - name: Cleanup - # if: always() - # run: | - # # shellcheck disable=SC2046 - # docker kill $(docker ps -q) ||: - # # shellcheck disable=SC2046 - # docker rm -f $(docker ps -a -q) ||: - # sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderDebAarch64: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=package_aarch64 - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ runner.temp }}/images_path - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - with: - fetch-depth: 0 # for performance artifact - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderDebAsan: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=package_asan - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderDebUBsan: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=package_ubsan - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderDebTsan: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=package_tsan - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderDebMsan: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=package_msan - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderDebDebug: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=package_debug - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" -########################################################################################## -##################################### SPECIAL BUILDS ##################################### -########################################################################################## - BuilderDebSplitted: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_splitted - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderBinTidy: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_tidy - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderBinDarwin: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_darwin - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderBinAarch64: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_aarch64 - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderBinFreeBSD: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_freebsd - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderBinDarwinAarch64: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_darwin_aarch64 - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderBinPPC64: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_ppc64le - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" -############################################################################################ -##################################### Docker images ####################################### -############################################################################################ - DockerServerImages: - needs: - - BuilderDebRelease - - BuilderDebAarch64 - runs-on: [self-hosted, style-checker] - steps: - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - with: - fetch-depth: 0 # It MUST BE THE SAME for all dependencies and the job itself - - name: Check docker clickhouse/clickhouse-server building - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 docker_server.py --release-type head --no-push - python3 docker_server.py --release-type head --no-push --no-ubuntu \ - --image-repo clickhouse/clickhouse-keeper --image-path docker/keeper - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################ -##################################### BUILD REPORTER ####################################### -############################################################################################ - BuilderReport: - needs: - - BuilderBinRelease - - BuilderDebAarch64 - - BuilderDebAsan - - BuilderDebDebug - - BuilderDebMsan - - BuilderDebRelease - - BuilderDebTsan - - BuilderDebUBsan - runs-on: [self-hosted, style-checker] - if: ${{ success() || failure() }} - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - CHECK_NAME=ClickHouse build check (actions) - REPORTS_PATH=${{runner.temp}}/reports_dir - TEMP_PATH=${{runner.temp}}/report_check - NEEDS_DATA_PATH=${{runner.temp}}/needs.json - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Report Builder - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cat > "$NEEDS_DATA_PATH" << 'EOF' - ${{ toJSON(needs) }} - EOF - cd "$GITHUB_WORKSPACE/tests/ci" - python3 build_report_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - BuilderSpecialReport: - needs: - - BuilderBinAarch64 - - BuilderBinDarwin - - BuilderBinDarwinAarch64 - - BuilderBinFreeBSD - # - BuilderBinGCC - - BuilderBinPPC64 - - BuilderBinTidy - - BuilderDebSplitted - runs-on: [self-hosted, style-checker] - if: ${{ success() || failure() }} - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/report_check - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=ClickHouse special build check (actions) - NEEDS_DATA_PATH=${{runner.temp}}/needs.json - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Report Builder - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cat > "$NEEDS_DATA_PATH" << 'EOF' - ${{ toJSON(needs) }} - EOF - cd "$GITHUB_WORKSPACE/tests/ci" - python3 build_report_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################## -########################### FUNCTIONAl STATELESS TESTS ####################################### -############################################################################################## - FunctionalStatelessTestRelease: - needs: [BuilderDebRelease] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_release - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, actions) - REPO_COPY=${{runner.temp}}/stateless_release/ClickHouse - KILL_TIMEOUT=10800 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestReleaseDatabaseReplicated0: - needs: [BuilderDebRelease] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_database_replicated - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, DatabaseReplicated, actions) - REPO_COPY=${{runner.temp}}/stateless_database_replicated/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=2 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestReleaseDatabaseReplicated1: - needs: [BuilderDebRelease] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_database_replicated - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, DatabaseReplicated, actions) - REPO_COPY=${{runner.temp}}/stateless_database_replicated/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=2 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestReleaseWideParts: - needs: [BuilderDebRelease] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_wide_parts - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, wide parts enabled, actions) - REPO_COPY=${{runner.temp}}/stateless_wide_parts/ClickHouse - KILL_TIMEOUT=10800 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestReleaseS3: - needs: [BuilderDebRelease] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_s3_storage - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, s3 storage, actions) - REPO_COPY=${{runner.temp}}/stateless_s3_storage/ClickHouse - KILL_TIMEOUT=10800 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestAarch64: - needs: [BuilderDebAarch64] - runs-on: [self-hosted, func-tester-aarch64] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_release - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (aarch64, actions) - REPO_COPY=${{runner.temp}}/stateless_release/ClickHouse - KILL_TIMEOUT=10800 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestAsan0: - needs: [BuilderDebAsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (address, actions) - REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=2 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestAsan1: - needs: [BuilderDebAsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (address, actions) - REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=2 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestTsan0: - needs: [BuilderDebTsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread, actions) - REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestTsan1: - needs: [BuilderDebTsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread, actions) - REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestTsan2: - needs: [BuilderDebTsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread, actions) - REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestUBsan: - needs: [BuilderDebUBsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_ubsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (ubsan, actions) - REPO_COPY=${{runner.temp}}/stateless_ubsan/ClickHouse - KILL_TIMEOUT=10800 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestMsan0: - needs: [BuilderDebMsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_memory - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory, actions) - REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestMsan1: - needs: [BuilderDebMsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_memory - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory, actions) - REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestMsan2: - needs: [BuilderDebMsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_memory - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory, actions) - REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestDebug0: - needs: [BuilderDebDebug] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug, actions) - REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestDebug1: - needs: [BuilderDebDebug] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug, actions) - REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestDebug2: - needs: [BuilderDebDebug] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug, actions) - REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestFlakyCheck: - needs: [BuilderDebAsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_flaky_asan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests flaky check (address, actions) - REPO_COPY=${{runner.temp}}/stateless_flaky_asan/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - TestsBugfixCheck: - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/tests_bugfix_check - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Tests bugfix validate check (actions) - KILL_TIMEOUT=3600 - REPO_COPY=${{runner.temp}}/tests_bugfix_check/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Bugfix test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - - TEMP_PATH="${TEMP_PATH}/integration" \ - REPORTS_PATH="${REPORTS_PATH}/integration" \ - python3 integration_test_check.py "Integration tests bugfix validate check" \ - --validate-bugfix --post-commit-status=file || echo 'ignore exit code' - - TEMP_PATH="${TEMP_PATH}/stateless" \ - REPORTS_PATH="${REPORTS_PATH}/stateless" \ - python3 functional_test_check.py "Stateless tests bugfix validate check" "$KILL_TIMEOUT" \ - --validate-bugfix --post-commit-status=file || echo 'ignore exit code' - - python3 bugfix_validate_check.py "${TEMP_PATH}/stateless/post_commit_status.tsv" "${TEMP_PATH}/integration/post_commit_status.tsv" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################## -############################ FUNCTIONAl STATEFUL TESTS ####################################### -############################################################################################## - FunctionalStatefulTestRelease: - needs: [BuilderDebRelease] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateful_release - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (release, actions) - REPO_COPY=${{runner.temp}}/stateful_release/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatefulTestAarch64: - needs: [BuilderDebAarch64] - runs-on: [self-hosted, func-tester-aarch64] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateful_release - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (aarch64, actions) - REPO_COPY=${{runner.temp}}/stateful_release/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatefulTestAsan: - needs: [BuilderDebAsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateful_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (address, actions) - REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatefulTestTsan: - needs: [BuilderDebTsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateful_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (thread, actions) - REPO_COPY=${{runner.temp}}/stateful_tsan/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatefulTestMsan: - needs: [BuilderDebMsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateful_msan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (memory, actions) - REPO_COPY=${{runner.temp}}/stateful_msan/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatefulTestUBsan: - needs: [BuilderDebUBsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateful_ubsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (ubsan, actions) - REPO_COPY=${{runner.temp}}/stateful_ubsan/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatefulTestDebug: - needs: [BuilderDebDebug] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateful_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (debug, actions) - REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################## -######################################### STRESS TESTS ####################################### -############################################################################################## - StressTestAsan: - needs: [BuilderDebAsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stress_thread - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (address, actions) - REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Stress test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 stress_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - StressTestTsan: - needs: [BuilderDebTsan] - # func testers have 16 cores + 128 GB memory - # while stress testers have 36 cores + 72 memory - # It would be better to have something like 32 + 128, - # but such servers almost unavailable as spot instances. - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stress_thread - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (thread, actions) - REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Stress test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 stress_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - StressTestMsan: - needs: [BuilderDebMsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stress_memory - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (memory, actions) - REPO_COPY=${{runner.temp}}/stress_memory/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Stress test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 stress_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - StressTestUBsan: - needs: [BuilderDebUBsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stress_undefined - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (undefined, actions) - REPO_COPY=${{runner.temp}}/stress_undefined/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Stress test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 stress_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - StressTestDebug: - needs: [BuilderDebDebug] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stress_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (debug, actions) - REPO_COPY=${{runner.temp}}/stress_debug/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Stress test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 stress_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################## -##################################### AST FUZZERS ############################################ -############################################################################################## - ASTFuzzerTestAsan: - needs: [BuilderDebAsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/ast_fuzzer_asan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (ASan, actions) - REPO_COPY=${{runner.temp}}/ast_fuzzer_asan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Fuzzer - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 ast_fuzzer_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - ASTFuzzerTestTsan: - needs: [BuilderDebTsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/ast_fuzzer_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (TSan, actions) - REPO_COPY=${{runner.temp}}/ast_fuzzer_tsan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Fuzzer - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 ast_fuzzer_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - ASTFuzzerTestUBSan: - needs: [BuilderDebUBsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/ast_fuzzer_ubsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (UBSan, actions) - REPO_COPY=${{runner.temp}}/ast_fuzzer_ubsan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Fuzzer - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 ast_fuzzer_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - ASTFuzzerTestMSan: - needs: [BuilderDebMsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/ast_fuzzer_msan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (MSan, actions) - REPO_COPY=${{runner.temp}}/ast_fuzzer_msan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Fuzzer - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 ast_fuzzer_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - ASTFuzzerTestDebug: - needs: [BuilderDebDebug] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/ast_fuzzer_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (debug, actions) - REPO_COPY=${{runner.temp}}/ast_fuzzer_debug/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Fuzzer - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 ast_fuzzer_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################# -############################# INTEGRATION TESTS ############################################# -############################################################################################# - IntegrationTestsAsan0: - needs: [BuilderDebAsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_asan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan, actions) - REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsAsan1: - needs: [BuilderDebAsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_asan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan, actions) - REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsAsan2: - needs: [BuilderDebAsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_asan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan, actions) - REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse - RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsTsan0: - needs: [BuilderDebTsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread, actions) - REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsTsan1: - needs: [BuilderDebTsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread, actions) - REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsTsan2: - needs: [BuilderDebTsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread, actions) - REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse - RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsTsan3: - needs: [BuilderDebTsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread, actions) - REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse - RUN_BY_HASH_NUM=3 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsRelease0: - needs: [BuilderDebRelease] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_release - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (release, actions) - REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=2 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsRelease1: - needs: [BuilderDebRelease] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_release - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (release, actions) - REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=2 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsFlakyCheck: - needs: [BuilderDebAsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_asan_flaky_check - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests flaky check (asan, actions) - REPO_COPY=${{runner.temp}}/integration_tests_asan_flaky_check/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################# -#################################### UNIT TESTS ############################################# -############################################################################################# - UnitTestsAsan: - needs: [BuilderDebAsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/unit_tests_asan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (asan, actions) - REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Unit test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 unit_tests_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - UnitTestsReleaseClang: - needs: [BuilderBinRelease] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/unit_tests_asan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (release-clang, actions) - REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Unit test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 unit_tests_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - # UnitTestsReleaseGCC: - # needs: [BuilderBinGCC] - # runs-on: [self-hosted, fuzzer-unit-tester] - # steps: - # - name: Set envs - # run: | - # cat >> "$GITHUB_ENV" << 'EOF' - # TEMP_PATH=${{runner.temp}}/unit_tests_asan - # REPORTS_PATH=${{runner.temp}}/reports_dir - # CHECK_NAME=Unit tests (release-gcc, actions) - # REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse - # EOF - # - name: Download json reports - # uses: actions/download-artifact@v2 - # with: - # path: ${{ env.REPORTS_PATH }} - # - name: Clear repository - # run: | - # sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - # - name: Check out repository code - # uses: actions/checkout@v2 - # - name: Unit test - # run: | - # sudo rm -fr "$TEMP_PATH" - # mkdir -p "$TEMP_PATH" - # cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - # cd "$REPO_COPY/tests/ci" - # python3 unit_tests_check.py "$CHECK_NAME" - # - name: Cleanup - # if: always() - # run: | - # # shellcheck disable=SC2046 - # docker kill $(docker ps -q) ||: - # # shellcheck disable=SC2046 - # docker rm -f $(docker ps -a -q) ||: - # sudo rm -fr "$TEMP_PATH" - UnitTestsTsan: - needs: [BuilderDebTsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/unit_tests_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (tsan, actions) - REPO_COPY=${{runner.temp}}/unit_tests_tsan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Unit test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 unit_tests_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - UnitTestsMsan: - needs: [BuilderDebMsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/unit_tests_msan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (msan, actions) - REPO_COPY=${{runner.temp}}/unit_tests_msan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Unit test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 unit_tests_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - UnitTestsUBsan: - needs: [BuilderDebUBsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/unit_tests_ubsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (ubsan, actions) - REPO_COPY=${{runner.temp}}/unit_tests_ubsan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Unit test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 unit_tests_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################# -#################################### PERFORMANCE TESTS ###################################### -############################################################################################# - PerformanceComparison0: - needs: [BuilderDebRelease] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/performance_comparison - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Performance Comparison - REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Performance Comparison - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 performance_comparison_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - PerformanceComparison1: - needs: [BuilderDebRelease] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/performance_comparison - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Performance Comparison - REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Performance Comparison - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 performance_comparison_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - PerformanceComparison2: - needs: [BuilderDebRelease] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/performance_comparison - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Performance Comparison - REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Performance Comparison - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 performance_comparison_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - PerformanceComparison3: - needs: [BuilderDebRelease] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/performance_comparison - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Performance Comparison - REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM=3 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Performance Comparison - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 performance_comparison_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - PerformanceComparisonAarch0: - needs: [BuilderDebAarch64] - runs-on: [self-hosted, func-tester-aarch64] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/performance_comparison - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Performance Comparison Aarch64 - REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Performance Comparison - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 performance_comparison_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - PerformanceComparisonAarch1: - needs: [BuilderDebAarch64] - runs-on: [self-hosted, func-tester-aarch64] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/performance_comparison - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Performance Comparison Aarch64 - REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Performance Comparison - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 performance_comparison_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - PerformanceComparisonAarch2: - needs: [BuilderDebAarch64] - runs-on: [self-hosted, func-tester-aarch64] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/performance_comparison - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Performance Comparison Aarch64 - REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Performance Comparison - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 performance_comparison_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - PerformanceComparisonAarch3: - needs: [BuilderDebAarch64] - runs-on: [self-hosted, func-tester-aarch64] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/performance_comparison - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Performance Comparison Aarch64 - REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM=3 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Performance Comparison - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 performance_comparison_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" ############################################################################################# ###################################### JEPSEN TESTS ######################################### ############################################################################################# Jepsen: needs: [BuilderBinRelease] uses: ./.github/workflows/jepsen.yml - - FinishCheck: - needs: - - StyleCheck - - DockerHubPush - - DockerServerImages - - CheckLabels - - BuilderReport - - FastTest - - FunctionalStatelessTestDebug0 - - FunctionalStatelessTestDebug1 - - FunctionalStatelessTestDebug2 - - FunctionalStatelessTestRelease - - FunctionalStatelessTestReleaseDatabaseReplicated0 - - FunctionalStatelessTestReleaseDatabaseReplicated1 - - FunctionalStatelessTestReleaseWideParts - - FunctionalStatelessTestAarch64 - - FunctionalStatelessTestAsan0 - - FunctionalStatelessTestAsan1 - - FunctionalStatelessTestTsan0 - - FunctionalStatelessTestTsan1 - - FunctionalStatelessTestTsan2 - - FunctionalStatelessTestMsan0 - - FunctionalStatelessTestMsan1 - - FunctionalStatelessTestMsan2 - - FunctionalStatelessTestUBsan - - FunctionalStatefulTestDebug - - FunctionalStatefulTestRelease - - FunctionalStatefulTestAarch64 - - FunctionalStatefulTestAsan - - FunctionalStatefulTestTsan - - FunctionalStatefulTestMsan - - FunctionalStatefulTestUBsan - - FunctionalStatelessTestReleaseS3 - - StressTestDebug - - StressTestAsan - - StressTestTsan - - StressTestMsan - - StressTestUBsan - - ASTFuzzerTestDebug - - ASTFuzzerTestAsan - - ASTFuzzerTestTsan - - ASTFuzzerTestMSan - - ASTFuzzerTestUBSan - - IntegrationTestsAsan0 - - IntegrationTestsAsan1 - - IntegrationTestsAsan2 - - IntegrationTestsRelease0 - - IntegrationTestsRelease1 - - IntegrationTestsTsan0 - - IntegrationTestsTsan1 - - IntegrationTestsTsan2 - - IntegrationTestsTsan3 - - PerformanceComparison0 - - PerformanceComparison1 - - PerformanceComparison2 - - PerformanceComparison3 - - PerformanceComparisonAarch0 - - PerformanceComparisonAarch1 - - PerformanceComparisonAarch2 - - PerformanceComparisonAarch3 - - UnitTestsAsan - - UnitTestsTsan - - UnitTestsMsan - - UnitTestsUBsan - - UnitTestsReleaseClang - - SplitBuildSmokeTest - - CompatibilityCheck - - IntegrationTestsFlakyCheck - - Jepsen - runs-on: [self-hosted, style-checker] - steps: - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Finish label - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 finish_check.py From 76c4fd9c8a27b0f4ea35a04a11ed278ee137f258 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 24 Jun 2022 14:42:24 +0000 Subject: [PATCH 10/40] Some fixes for batch reads --- src/Coordination/KeeperDispatcher.cpp | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index b07997882e5..46e9514e2a7 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -41,7 +41,7 @@ void KeeperDispatcher::requestThread() std::optional previous_request; bool collecting_quorum_requests = false; - const auto needs_quorum = [&](const auto & coordination_settings, const auto & request) + const auto needs_quorum = [](const auto & coordination_settings, const auto & request) { return coordination_settings->quorum_reads || !request.request->isReadRequest(); }; @@ -64,8 +64,11 @@ void KeeperDispatcher::requestThread() { KeeperStorage::RequestsForSessions current_batch; + auto next_collecting_quorum_requests = collecting_quorum_requests; + if (previous_request) { + assert(collecting_quorum_requests == needs_quorum(coordination_settings, *previous_request)); current_batch.push_back(std::move(*previous_request)); previous_request.reset(); } @@ -91,7 +94,7 @@ void KeeperDispatcher::requestThread() if (collecting_quorum_requests != needs_quorum(coordination_settings, request)) { previous_request.emplace(std::move(request)); - collecting_quorum_requests = !collecting_quorum_requests; + next_collecting_quorum_requests = !collecting_quorum_requests; break; } @@ -105,7 +108,7 @@ void KeeperDispatcher::requestThread() else { previous_request.emplace(std::move(request)); - collecting_quorum_requests = !collecting_quorum_requests; + next_collecting_quorum_requests = !collecting_quorum_requests; } if (shutdown_called) @@ -169,6 +172,8 @@ void KeeperDispatcher::requestThread() } current_batch.clear(); } + + collecting_quorum_requests = next_collecting_quorum_requests; } catch (...) { @@ -629,9 +634,12 @@ void KeeperDispatcher::onRequestCommit(uint64_t log_term, uint64_t log_idx) { const auto process_requests = [this](auto & request_queue) { - for (auto & request_info : request_queue) + for (const auto & request_info : request_queue) { - server->putLocalReadRequest(request_info); + if (server->isLeaderAlive()) + server->putLocalReadRequest(request_info); + else + addErrorResponses({request_info}, Coordination::Error::ZCONNECTIONLOSS); } request_queue.clear(); From 8a3531c2dbe53832a0d998a9439b1cf9101eef9c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 27 Jun 2022 07:30:59 +0000 Subject: [PATCH 11/40] Refactor batching requests logic --- src/Coordination/KeeperDispatcher.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 46e9514e2a7..ce8d1a0d576 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -85,7 +85,7 @@ void KeeperDispatcher::requestThread() /// Waiting until previous append will be successful, or batch is big enough /// has_result == false && get_result_code == OK means that our request still not processed. /// Sometimes NuRaft set errorcode without setting result, so we check both here. - while (prev_result && (!prev_result->has_result() && prev_result->get_result_code() == nuraft::cmd_result_code::OK) && current_batch.size() <= max_batch_size) + while (current_batch.size() <= max_batch_size) { /// Trying to get batch requests as fast as possible if (requests_queue->tryPop(request, 1)) @@ -100,6 +100,8 @@ void KeeperDispatcher::requestThread() current_batch.emplace_back(request); } + else if (!prev_result || (prev_result->has_result() || prev_result->get_result_code() != nuraft::cmd_result_code::OK)) + break; if (shutdown_called) break; @@ -169,8 +171,9 @@ void KeeperDispatcher::requestThread() } } } + + current_batch.clear(); } - current_batch.clear(); } collecting_quorum_requests = next_collecting_quorum_requests; From f2e15197c66454ae10fe8d3ae3e18062f09bc8b2 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 27 Jun 2022 07:31:12 +0000 Subject: [PATCH 12/40] Retry only on final read --- .../src/jepsen/clickhouse_keeper/counter.clj | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/counter.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/counter.clj index f82d3f4c348..54f53f9e74f 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/counter.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/counter.clj @@ -24,7 +24,12 @@ (invoke! [this test op] (case (:f op) - :read (exec-with-retries 30 (fn [] + :read (try + (assoc op + :type :ok + :value (count (zk-list conn "/"))) + (catch Exception _ (assoc op :type :info, :error :connect-error))) + :final-read (exec-with-retries 30 (fn [] (assoc op :type :ok :value (count (zk-list conn "/"))))) @@ -46,7 +51,5 @@ :checker (checker/compose {:counter (checker/counter) :perf (checker/perf)}) - :generator (->> (range) - (map (fn [x] - (->> (gen/mix [r add]))))) - :final-generator (gen/once {:type :invoke, :f :read, :value nil})}) + :generator (gen/mix [r add]) + :final-generator (gen/once {:type :invoke, :f :final-read, :value nil})}) From 53e906fdd5191e68d7059eafe12e6ee75a10f1f6 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 27 Jun 2022 12:05:27 +0000 Subject: [PATCH 13/40] Improve batching by processing single request from each session --- src/Coordination/KeeperDispatcher.cpp | 224 ++++++++++++++++-------- src/Coordination/KeeperDispatcher.h | 14 +- src/Coordination/KeeperStateMachine.cpp | 2 +- src/Coordination/KeeperStateMachine.h | 4 +- 4 files changed, 169 insertions(+), 75 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index ce8d1a0d576..7304a400d6a 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -35,17 +35,16 @@ void KeeperDispatcher::requestThread() /// Result of requests batch from previous iteration RaftAppendResult prev_result = nullptr; - /// Requests from previous iteration. We store them to be able - /// to send errors to the client. KeeperStorage::RequestsForSessions prev_batch; - std::optional previous_request; - bool collecting_quorum_requests = false; const auto needs_quorum = [](const auto & coordination_settings, const auto & request) { return coordination_settings->quorum_reads || !request.request->isReadRequest(); }; + KeeperStorage::RequestsForSessions write_requests; + KeeperStorage::RequestsForSessions read_requests; + while (!shutdown_called) { KeeperStorage::RequestForSession request; @@ -62,55 +61,66 @@ void KeeperDispatcher::requestThread() /// read request. So reads are some kind of "separator" for writes. try { - KeeperStorage::RequestsForSessions current_batch; - - auto next_collecting_quorum_requests = collecting_quorum_requests; - - if (previous_request) - { - assert(collecting_quorum_requests == needs_quorum(coordination_settings, *previous_request)); - current_batch.push_back(std::move(*previous_request)); - previous_request.reset(); - } + KeeperStorage::RequestsForSessions * current_batch{nullptr}; + bool is_current_read = false; if (requests_queue->tryPop(request, max_wait)) { if (shutdown_called) break; - if (collecting_quorum_requests == needs_quorum(coordination_settings, request)) + if (needs_quorum(coordination_settings, request)) { - current_batch.emplace_back(request); + write_requests.emplace_back(request); - /// Waiting until previous append will be successful, or batch is big enough - /// has_result == false && get_result_code == OK means that our request still not processed. - /// Sometimes NuRaft set errorcode without setting result, so we check both here. - while (current_batch.size() <= max_batch_size) + if (!current_batch) { - /// Trying to get batch requests as fast as possible - if (requests_queue->tryPop(request, 1)) - { - /// Don't append read request into batch, we have to process them separately - if (collecting_quorum_requests != needs_quorum(coordination_settings, request)) - { - previous_request.emplace(std::move(request)); - next_collecting_quorum_requests = !collecting_quorum_requests; - break; - } - - current_batch.emplace_back(request); - } - else if (!prev_result || (prev_result->has_result() || prev_result->get_result_code() != nuraft::cmd_result_code::OK)) - break; - - if (shutdown_called) - break; + current_batch = &write_requests; + is_current_read = false; } } else { - previous_request.emplace(std::move(request)); - next_collecting_quorum_requests = !collecting_quorum_requests; + read_requests.emplace_back(request); + if (!current_batch) + { + current_batch = &read_requests; + is_current_read = true; + } + } + + /// Waiting until previous append will be successful, or batch is big enough + /// has_result == false && get_result_code == OK means that our request still not processed. + /// Sometimes NuRaft set errorcode without setting result, so we check both here. + while (true) + { + if (write_requests.size() > max_batch_size) + { + current_batch = &write_requests; + is_current_read = false; + break; + } + + if (read_requests.size() > max_batch_size) + { + current_batch = &read_requests; + is_current_read = true; + break; + } + + /// Trying to get batch requests as fast as possible + if (requests_queue->tryPop(request, 1)) + { + if (needs_quorum(coordination_settings, request)) + write_requests.emplace_back(request); + else + read_requests.emplace_back(request); + } + else if (!prev_result || prev_result->has_result() || prev_result->get_result_code() != nuraft::cmd_result_code::OK) + break; + + if (shutdown_called) + break; } if (shutdown_called) @@ -120,32 +130,29 @@ void KeeperDispatcher::requestThread() if (prev_result) forceWaitAndProcessResult(prev_result, prev_batch); - /// Process collected requests batch - if (!current_batch.empty()) + if (current_batch && !current_batch->empty()) { - if (collecting_quorum_requests) + LOG_INFO(&Poco::Logger::get("BATCHING"), "Processing {} batch of {}", is_current_read, current_batch->size()); + if (!is_current_read) { - auto result = server->putRequestBatch(current_batch); - prev_batch = std::move(current_batch); - prev_result = result; + prev_result = server->putRequestBatch(*current_batch); + prev_batch = std::move(*current_batch); + + if (!read_requests.empty()) + { + current_batch = &read_requests; + is_current_read = true; + } + else + current_batch = nullptr; } else { - auto leader_info_result = server->getLeaderInfo(); + prev_result = server->getLeaderInfo(); + prev_result->when_ready([&, requests_for_sessions = *current_batch](nuraft::cmd_result> & result, nuraft::ptr &) + { + auto & leader_info_ctx = result.get(); - auto & leader_info_ctx = leader_info_result->get(); - - /// If we get some errors, then send them to clients - if (!leader_info_result->get_accepted() || leader_info_result->get_result_code() == nuraft::cmd_result_code::TIMEOUT) - { - addErrorResponses(current_batch, Coordination::Error::ZOPERATIONTIMEOUT); - } - else if (leader_info_result->get_result_code() != nuraft::cmd_result_code::OK) - { - addErrorResponses(current_batch, Coordination::Error::ZCONNECTIONLOSS); - } - else - { KeeperServer::NodeInfo leader_info; leader_info.term = leader_info_ctx->get_ulong(); leader_info.last_committed_index = leader_info_ctx->get_ulong(); @@ -156,27 +163,34 @@ void KeeperDispatcher::requestThread() if (node_info.term < leader_info.term || node_info.last_committed_index < leader_info.last_committed_index) { auto & leader_waiter = leader_waiters[node_info]; - leader_waiter.insert(leader_waiter.end(), current_batch.begin(), current_batch.end()); + leader_waiter.insert(leader_waiter.end(), requests_for_sessions.begin(), requests_for_sessions.end()); LOG_INFO(log, "waiting for {}, idx {}", leader_info.term, leader_info.last_committed_index); } else { - for (const auto & read_request : current_batch) + for (const auto & read_request : requests_for_sessions) { if (server->isLeaderAlive()) server->putLocalReadRequest(read_request); else addErrorResponses({read_request}, Coordination::Error::ZCONNECTIONLOSS); + finalizeRequest(read_request); } } - } - } + }); - current_batch.clear(); + prev_batch = std::move(*current_batch); + + if (!write_requests.empty()) + { + current_batch = &write_requests; + is_current_read = false; + } + else + current_batch = nullptr; + } } } - - collecting_quorum_requests = next_collecting_quorum_requests; } catch (...) { @@ -287,6 +301,29 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ request_info.time = duration_cast(system_clock::now().time_since_epoch()).count(); request_info.session_id = session_id; + { + std::lock_guard lock{unprocessed_request_mutex}; + auto unprocessed_requests_it = unprocessed_requests_for_session.find(session_id); + if (unprocessed_requests_it == unprocessed_requests_for_session.end()) + { + auto & unprocessed_requests = unprocessed_requests_for_session[session_id]; + unprocessed_requests.unprocessed_num = 1; + unprocessed_requests.is_read = request->isReadRequest(); + } + else + { + auto & unprocessed_requests = unprocessed_requests_it->second; + + if (!unprocessed_requests.request_queue.empty() || unprocessed_requests.is_read != request->isReadRequest()) + { + unprocessed_requests.request_queue.push_back(std::move(request_info)); + return true; + } + + ++unprocessed_requests.unprocessed_num; + } + } + std::lock_guard lock(push_request_mutex); if (shutdown_called) @@ -316,7 +353,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf responses_thread = ThreadFromGlobalPool([this] { responseThread(); }); snapshot_thread = ThreadFromGlobalPool([this] { snapshotThread(); }); - server = std::make_unique(configuration_and_settings, config, responses_queue, snapshots_queue, [this](uint64_t log_term, uint64_t log_idx) { onRequestCommit(log_term, log_idx); }); + server = std::make_unique(configuration_and_settings, config, responses_queue, snapshots_queue, [this](const KeeperStorage::RequestForSession & request_for_session, uint64_t log_term, uint64_t log_idx) { onRequestCommit(request_for_session, log_term, log_idx); }); try { @@ -506,14 +543,13 @@ void KeeperDispatcher::forceWaitAndProcessResult(RaftAppendResult & result, Keep if (!result->has_result()) result->get(); - /// If we get some errors, than send them to clients if (!result->get_accepted() || result->get_result_code() == nuraft::cmd_result_code::TIMEOUT) addErrorResponses(requests_for_sessions, Coordination::Error::ZOPERATIONTIMEOUT); else if (result->get_result_code() != nuraft::cmd_result_code::OK) addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); - result = nullptr; requests_for_sessions.clear(); + result = nullptr; } int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms) @@ -633,8 +669,52 @@ void KeeperDispatcher::updateConfigurationThread() } } -void KeeperDispatcher::onRequestCommit(uint64_t log_term, uint64_t log_idx) +void KeeperDispatcher::finalizeRequest(const KeeperStorage::RequestForSession & request_for_session) { + std::lock_guard lock{unprocessed_request_mutex}; + auto unprocessed_requests_it = unprocessed_requests_for_session.find(request_for_session.session_id); + if (unprocessed_requests_it == unprocessed_requests_for_session.end()) + return; + + auto & unprocessed_requests = unprocessed_requests_it->second; + --unprocessed_requests.unprocessed_num; + + if (unprocessed_requests.unprocessed_num == 0) + { + if (!unprocessed_requests.request_queue.empty()) + { + auto & request_queue = unprocessed_requests.request_queue; + unprocessed_requests.is_read = !unprocessed_requests.is_read; + while (!request_queue.empty() && request_queue.front().request->isReadRequest() == unprocessed_requests.is_read) + { + auto & front_request = request_queue.front(); + + /// Put close requests without timeouts + if (front_request.request->getOpNum() == Coordination::OpNum::Close) + { + if (!requests_queue->push(std::move(front_request))) + throw Exception("Cannot push request to queue", ErrorCodes::SYSTEM_ERROR); + } + else if (!requests_queue->tryPush(std::move(front_request), configuration_and_settings->coordination_settings->operation_timeout_ms.totalMilliseconds())) + { + throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::TIMEOUT_EXCEEDED); + } + + ++unprocessed_requests.unprocessed_num; + request_queue.pop_front(); + } + } + else + { + unprocessed_requests_for_session.erase(unprocessed_requests_it); + } + } +} + +void KeeperDispatcher::onRequestCommit(const KeeperStorage::RequestForSession & request_for_session, uint64_t log_term, uint64_t log_idx) +{ + finalizeRequest(request_for_session); + const auto process_requests = [this](auto & request_queue) { for (const auto & request_info : request_queue) @@ -643,6 +723,8 @@ void KeeperDispatcher::onRequestCommit(uint64_t log_term, uint64_t log_idx) server->putLocalReadRequest(request_info); else addErrorResponses({request_info}, Coordination::Error::ZCONNECTIONLOSS); + + finalizeRequest(request_info); } request_queue.clear(); diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 1f5de4007f2..f7f00f3c5f9 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -80,6 +80,18 @@ private: std::unordered_map> leader_waiters; std::mutex leader_waiter_mutex; + struct UnprocessedRequests + { + size_t unprocessed_num{0}; + bool is_read{false}; + std::list request_queue; + }; + + void finalizeRequest(const KeeperStorage::RequestForSession & request_for_session); + + std::unordered_map unprocessed_requests_for_session; + std::mutex unprocessed_request_mutex; + /// Thread put requests to raft void requestThread(); /// Thread put responses for subscribed sessions @@ -119,7 +131,7 @@ public: return server && server->checkInit(); } - void onRequestCommit(uint64_t log_term, uint64_t log_idx); + void onRequestCommit(const KeeperStorage::RequestForSession & request_for_session, uint64_t log_term, uint64_t log_idx); /// Is server accepting requests, i.e. connected to the cluster /// and achieved quorum diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index c2bd7772b1b..2aa176423d1 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -242,7 +242,7 @@ nuraft::ptr KeeperStateMachine::commit_ext(const ext_op_params& } } - commit_callback(params.log_term, params.log_idx); + commit_callback(request_for_session, params.log_term, params.log_idx); last_committed_idx = params.log_idx; return nullptr; } diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index f5a38228c86..ceda081735a 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -19,7 +19,7 @@ using SnapshotsQueue = ConcurrentBoundedQueue; class KeeperStateMachine : public nuraft::state_machine { public: - using CommitCallback = std::function; + using CommitCallback = std::function; KeeperStateMachine( ResponsesQueue & responses_queue_, @@ -28,7 +28,7 @@ public: const CoordinationSettingsPtr & coordination_settings_, const std::string & superdigest_ = "", bool digest_enabled_ = true, - CommitCallback commit_callback_ = [](uint64_t, uint64_t){}); + CommitCallback commit_callback_ = [](const KeeperStorage::RequestForSession &, uint64_t, uint64_t){}); /// Read state from the latest snapshot void init(); From dc5cc3376962af868090e0ff8f3bedc506fec868 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 27 Jun 2022 14:22:53 +0000 Subject: [PATCH 14/40] Process outside of lock --- src/Coordination/KeeperDispatcher.cpp | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 7304a400d6a..bfd98205413 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -726,17 +726,19 @@ void KeeperDispatcher::onRequestCommit(const KeeperStorage::RequestForSession & finalizeRequest(request_info); } - - request_queue.clear(); }; - std::lock_guard lock(leader_waiter_mutex); - auto request_queue_it = leader_waiters.find(KeeperServer::NodeInfo{.term = log_term, .last_committed_index = log_idx}); - if (request_queue_it != leader_waiters.end()) + KeeperStorage::RequestsForSessions requests; { - process_requests(request_queue_it->second); - leader_waiters.erase(request_queue_it); + std::lock_guard lock(leader_waiter_mutex); + auto request_queue_it = leader_waiters.find(KeeperServer::NodeInfo{.term = log_term, .last_committed_index = log_idx}); + if (request_queue_it != leader_waiters.end()) + { + requests = std::move(request_queue_it->second); + leader_waiters.erase(request_queue_it); + } } + process_requests(requests); } bool KeeperDispatcher::isServerActive() const From a6c628cab4635f07405380477ff09f42930fb720 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 27 Jun 2022 14:34:27 +0000 Subject: [PATCH 15/40] Small fixes for queues --- src/Coordination/KeeperDispatcher.cpp | 111 ++++++++++++++---------- src/Coordination/KeeperDispatcher.h | 4 +- src/Coordination/KeeperStateMachine.cpp | 2 +- 3 files changed, 69 insertions(+), 48 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index bfd98205413..33e6ff57253 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -137,6 +137,7 @@ void KeeperDispatcher::requestThread() { prev_result = server->putRequestBatch(*current_batch); prev_batch = std::move(*current_batch); + current_batch->clear(); if (!read_requests.empty()) { @@ -149,7 +150,7 @@ void KeeperDispatcher::requestThread() else { prev_result = server->getLeaderInfo(); - prev_result->when_ready([&, requests_for_sessions = *current_batch](nuraft::cmd_result> & result, nuraft::ptr &) + server->getLeaderInfo()->when_ready([&, requests_for_sessions = *current_batch](nuraft::cmd_result> & result, nuraft::ptr &) mutable { auto & leader_info_ctx = result.get(); @@ -168,18 +169,23 @@ void KeeperDispatcher::requestThread() } else { - for (const auto & read_request : requests_for_sessions) + bg_read_request.scheduleOrThrow([&, requests = std::move(requests_for_sessions)] { - if (server->isLeaderAlive()) - server->putLocalReadRequest(read_request); - else - addErrorResponses({read_request}, Coordination::Error::ZCONNECTIONLOSS); - finalizeRequest(read_request); - } + for (const auto & request_info : requests) + { + if (server->isLeaderAlive()) + server->putLocalReadRequest(request_info); + else + addErrorResponses({request_info}, Coordination::Error::ZCONNECTIONLOSS); + } + + finalizeRequests(requests); + }); } }); prev_batch = std::move(*current_batch); + current_batch->clear(); if (!write_requests.empty()) { @@ -669,63 +675,76 @@ void KeeperDispatcher::updateConfigurationThread() } } -void KeeperDispatcher::finalizeRequest(const KeeperStorage::RequestForSession & request_for_session) +void KeeperDispatcher::finalizeRequests(const KeeperStorage::RequestsForSessions & requests_for_sessions) { - std::lock_guard lock{unprocessed_request_mutex}; - auto unprocessed_requests_it = unprocessed_requests_for_session.find(request_for_session.session_id); - if (unprocessed_requests_it == unprocessed_requests_for_session.end()) - return; + std::unordered_map counts_for_session; - auto & unprocessed_requests = unprocessed_requests_it->second; - --unprocessed_requests.unprocessed_num; - - if (unprocessed_requests.unprocessed_num == 0) + for (const auto & request_for_session : requests_for_sessions) { - if (!unprocessed_requests.request_queue.empty()) + ++counts_for_session[request_for_session.session_id]; + } + + std::lock_guard lock{unprocessed_request_mutex}; + for (const auto [session_id, count] : counts_for_session) + { + auto unprocessed_requests_it = unprocessed_requests_for_session.find(session_id); + if (unprocessed_requests_it == unprocessed_requests_for_session.end()) + continue; + + auto & unprocessed_requests = unprocessed_requests_it->second; + unprocessed_requests.unprocessed_num -= count; + + if (unprocessed_requests.unprocessed_num == 0) { - auto & request_queue = unprocessed_requests.request_queue; - unprocessed_requests.is_read = !unprocessed_requests.is_read; - while (!request_queue.empty() && request_queue.front().request->isReadRequest() == unprocessed_requests.is_read) + if (!unprocessed_requests.request_queue.empty()) { - auto & front_request = request_queue.front(); - - /// Put close requests without timeouts - if (front_request.request->getOpNum() == Coordination::OpNum::Close) + auto & request_queue = unprocessed_requests.request_queue; + unprocessed_requests.is_read = !unprocessed_requests.is_read; + while (!request_queue.empty() && request_queue.front().request->isReadRequest() == unprocessed_requests.is_read) { - if (!requests_queue->push(std::move(front_request))) - throw Exception("Cannot push request to queue", ErrorCodes::SYSTEM_ERROR); - } - else if (!requests_queue->tryPush(std::move(front_request), configuration_and_settings->coordination_settings->operation_timeout_ms.totalMilliseconds())) - { - throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::TIMEOUT_EXCEEDED); - } + auto & front_request = request_queue.front(); - ++unprocessed_requests.unprocessed_num; - request_queue.pop_front(); + /// Put close requests without timeouts + if (front_request.request->getOpNum() == Coordination::OpNum::Close) + { + if (!requests_queue->push(std::move(front_request))) + throw Exception("Cannot push request to queue", ErrorCodes::SYSTEM_ERROR); + } + else if (!requests_queue->tryPush(std::move(front_request), configuration_and_settings->coordination_settings->operation_timeout_ms.totalMilliseconds())) + { + throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::TIMEOUT_EXCEEDED); + } + + ++unprocessed_requests.unprocessed_num; + request_queue.pop_front(); + } + } + else + { + unprocessed_requests_for_session.erase(unprocessed_requests_it); } - } - else - { - unprocessed_requests_for_session.erase(unprocessed_requests_it); } } } void KeeperDispatcher::onRequestCommit(const KeeperStorage::RequestForSession & request_for_session, uint64_t log_term, uint64_t log_idx) { - finalizeRequest(request_for_session); + finalizeRequests({request_for_session}); const auto process_requests = [this](auto & request_queue) { - for (const auto & request_info : request_queue) + bg_read_request.scheduleOrThrow([&, requests = std::move(request_queue)] { - if (server->isLeaderAlive()) - server->putLocalReadRequest(request_info); - else - addErrorResponses({request_info}, Coordination::Error::ZCONNECTIONLOSS); + for (const auto & request_info : requests) + { + if (server->isLeaderAlive()) + server->putLocalReadRequest(request_info); + else + addErrorResponses({request_info}, Coordination::Error::ZCONNECTIONLOSS); - finalizeRequest(request_info); - } + } + finalizeRequests(requests); + }); }; KeeperStorage::RequestsForSessions requests; diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index f7f00f3c5f9..b19a2362268 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -87,7 +87,7 @@ private: std::list request_queue; }; - void finalizeRequest(const KeeperStorage::RequestForSession & request_for_session); + void finalizeRequests(const KeeperStorage::RequestsForSessions & requests_for_sessions); std::unordered_map unprocessed_requests_for_session; std::mutex unprocessed_request_mutex; @@ -213,6 +213,8 @@ public: { keeper_stats.reset(); } + + ThreadPool bg_read_request{1}; }; } diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 2aa176423d1..2dc849eb721 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -242,8 +242,8 @@ nuraft::ptr KeeperStateMachine::commit_ext(const ext_op_params& } } - commit_callback(request_for_session, params.log_term, params.log_idx); last_committed_idx = params.log_idx; + commit_callback(request_for_session, params.log_term, params.log_idx); return nullptr; } From 23c705222f31c6cebc88ac646482399a64af2a23 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 28 Jun 2022 08:48:19 +0000 Subject: [PATCH 16/40] Process read requests in bg thread --- src/Coordination/KeeperDispatcher.cpp | 126 ++++++++++++++------------ src/Coordination/KeeperDispatcher.h | 8 +- src/Coordination/KeeperServer.cpp | 4 - 3 files changed, 74 insertions(+), 64 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 33e6ff57253..ae670ea0f68 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include namespace fs = std::filesystem; @@ -23,6 +24,7 @@ namespace ErrorCodes KeeperDispatcher::KeeperDispatcher() : responses_queue(std::numeric_limits::max()) + , read_requests_queue(std::numeric_limits::max()) , configuration_and_settings(std::make_shared()) , log(&Poco::Logger::get("KeeperDispatcher")) { @@ -35,7 +37,6 @@ void KeeperDispatcher::requestThread() /// Result of requests batch from previous iteration RaftAppendResult prev_result = nullptr; - KeeperStorage::RequestsForSessions prev_batch; const auto needs_quorum = [](const auto & coordination_settings, const auto & request) { @@ -53,12 +54,6 @@ void KeeperDispatcher::requestThread() uint64_t max_wait = coordination_settings->operation_timeout_ms.totalMilliseconds(); uint64_t max_batch_size = coordination_settings->max_requests_batch_size; - /// The code below do a very simple thing: batch all write (quorum) requests into vector until - /// previous write batch is not finished or max_batch size achieved. The main complexity goes from - /// the ability to process read requests without quorum (from local state). So when we are collecting - /// requests into a batch we must check that the new request is not read request. Otherwise we have to - /// process all already accumulated write requests, wait them synchronously and only after that process - /// read request. So reads are some kind of "separator" for writes. try { KeeperStorage::RequestsForSessions * current_batch{nullptr}; @@ -116,7 +111,7 @@ void KeeperDispatcher::requestThread() else read_requests.emplace_back(request); } - else if (!prev_result || prev_result->has_result() || prev_result->get_result_code() != nuraft::cmd_result_code::OK) + else if (is_current_read || !prev_result || prev_result->has_result() || prev_result->get_result_code() != nuraft::cmd_result_code::OK) break; if (shutdown_called) @@ -127,16 +122,23 @@ void KeeperDispatcher::requestThread() break; /// Forcefully process all previous pending requests - if (prev_result) - forceWaitAndProcessResult(prev_result, prev_batch); + if (!is_current_read && prev_result) + forceWaitAndProcessResult(prev_result); - if (current_batch && !current_batch->empty()) + if (current_batch) { LOG_INFO(&Poco::Logger::get("BATCHING"), "Processing {} batch of {}", is_current_read, current_batch->size()); if (!is_current_read) { prev_result = server->putRequestBatch(*current_batch); - prev_batch = std::move(*current_batch); + prev_result->when_ready([&, requests_for_sessions = std::move(*current_batch)](nuraft::cmd_result> & result, nuraft::ptr &) mutable + { + if (!result.get_accepted() || result.get_result_code() == nuraft::cmd_result_code::TIMEOUT) + addErrorResponses(requests_for_sessions, Coordination::Error::ZOPERATIONTIMEOUT); + else if (result.get_result_code() != nuraft::cmd_result_code::OK) + addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); + }); + current_batch->clear(); if (!read_requests.empty()) @@ -149,9 +151,13 @@ void KeeperDispatcher::requestThread() } else { - prev_result = server->getLeaderInfo(); - server->getLeaderInfo()->when_ready([&, requests_for_sessions = *current_batch](nuraft::cmd_result> & result, nuraft::ptr &) mutable + server->getLeaderInfo()->when_ready([&, requests_for_sessions = std::move(*current_batch)](nuraft::cmd_result> & result, nuraft::ptr &) mutable { + if (!result.get_accepted() || result.get_result_code() == nuraft::cmd_result_code::TIMEOUT) + addErrorResponses(requests_for_sessions, Coordination::Error::ZOPERATIONTIMEOUT); + else if (result.get_result_code() != nuraft::cmd_result_code::OK) + addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); + auto & leader_info_ctx = result.get(); KeeperServer::NodeInfo leader_info; @@ -163,28 +169,14 @@ void KeeperDispatcher::requestThread() if (node_info.term < leader_info.term || node_info.last_committed_index < leader_info.last_committed_index) { - auto & leader_waiter = leader_waiters[node_info]; + auto & leader_waiter = leader_waiters[leader_info]; leader_waiter.insert(leader_waiter.end(), requests_for_sessions.begin(), requests_for_sessions.end()); LOG_INFO(log, "waiting for {}, idx {}", leader_info.term, leader_info.last_committed_index); } - else - { - bg_read_request.scheduleOrThrow([&, requests = std::move(requests_for_sessions)] - { - for (const auto & request_info : requests) - { - if (server->isLeaderAlive()) - server->putLocalReadRequest(request_info); - else - addErrorResponses({request_info}, Coordination::Error::ZCONNECTIONLOSS); - } - - finalizeRequests(requests); - }); - } + else if (!read_requests_queue.push(std::move(requests_for_sessions))) + throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push read requests to queue"); }); - prev_batch = std::move(*current_batch); current_batch->clear(); if (!write_requests.empty()) @@ -254,6 +246,37 @@ void KeeperDispatcher::snapshotThread() } } +void KeeperDispatcher::readRequestThread() +{ + setThreadName("KeeperReadT"); + while (!shutdown_called) + { + KeeperStorage::RequestsForSessions requests; + if (!read_requests_queue.pop(requests)) + break; + + if (shutdown_called) + break; + + try + { + for (const auto & request_info : requests) + { + if (server->isLeaderAlive()) + server->putLocalReadRequest(request_info); + else + addErrorResponses({request_info}, Coordination::Error::ZCONNECTIONLOSS); + } + + finalizeRequests(requests); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } +} + void KeeperDispatcher::setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response) { std::lock_guard lock(session_to_response_callback_mutex); @@ -301,6 +324,7 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ return false; } + LOG_INFO(&Poco::Logger::get("BATCHING"), "Got request {}", request->getOpNum()); KeeperStorage::RequestForSession request_info; request_info.request = request; using namespace std::chrono; @@ -325,8 +349,10 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ unprocessed_requests.request_queue.push_back(std::move(request_info)); return true; } - - ++unprocessed_requests.unprocessed_num; + else + { + ++unprocessed_requests.unprocessed_num; + } } } @@ -358,6 +384,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf request_thread = ThreadFromGlobalPool([this] { requestThread(); }); responses_thread = ThreadFromGlobalPool([this] { responseThread(); }); snapshot_thread = ThreadFromGlobalPool([this] { snapshotThread(); }); + read_request_thread = ThreadFromGlobalPool([this] { readRequestThread(); }); server = std::make_unique(configuration_and_settings, config, responses_queue, snapshots_queue, [this](const KeeperStorage::RequestForSession & request_for_session, uint64_t log_term, uint64_t log_idx) { onRequestCommit(request_for_session, log_term, log_idx); }); @@ -423,6 +450,10 @@ void KeeperDispatcher::shutdown() if (snapshot_thread.joinable()) snapshot_thread.join(); + read_requests_queue.finish(); + if (read_request_thread.joinable()) + read_request_thread.join(); + update_configuration_queue.finish(); if (update_configuration_thread.joinable()) update_configuration_thread.join(); @@ -544,17 +575,13 @@ void KeeperDispatcher::addErrorResponses(const KeeperStorage::RequestsForSession } } -void KeeperDispatcher::forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorage::RequestsForSessions & requests_for_sessions) +void KeeperDispatcher::forceWaitAndProcessResult(RaftAppendResult & result) { + LOG_INFO(&Poco::Logger::get("TESTER"), "Waiting for result"); if (!result->has_result()) result->get(); - if (!result->get_accepted() || result->get_result_code() == nuraft::cmd_result_code::TIMEOUT) - addErrorResponses(requests_for_sessions, Coordination::Error::ZOPERATIONTIMEOUT); - else if (result->get_result_code() != nuraft::cmd_result_code::OK) - addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); - - requests_for_sessions.clear(); + LOG_INFO(&Poco::Logger::get("TESTER"), "Got result"); result = nullptr; } @@ -731,22 +758,6 @@ void KeeperDispatcher::onRequestCommit(const KeeperStorage::RequestForSession & { finalizeRequests({request_for_session}); - const auto process_requests = [this](auto & request_queue) - { - bg_read_request.scheduleOrThrow([&, requests = std::move(request_queue)] - { - for (const auto & request_info : requests) - { - if (server->isLeaderAlive()) - server->putLocalReadRequest(request_info); - else - addErrorResponses({request_info}, Coordination::Error::ZCONNECTIONLOSS); - - } - finalizeRequests(requests); - }); - }; - KeeperStorage::RequestsForSessions requests; { std::lock_guard lock(leader_waiter_mutex); @@ -757,7 +768,8 @@ void KeeperDispatcher::onRequestCommit(const KeeperStorage::RequestForSession & leader_waiters.erase(request_queue_it); } } - process_requests(requests); + if (!read_requests_queue.push(std::move(requests))) + throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push read requests to queue"); } bool KeeperDispatcher::isServerActive() const diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index b19a2362268..50b842d73b4 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -35,6 +35,7 @@ private: std::unique_ptr requests_queue; ResponsesQueue responses_queue; SnapshotsQueue snapshots_queue{1}; + ConcurrentBoundedQueue read_requests_queue; /// More than 1k updates is definitely misconfiguration. UpdateConfigurationQueue update_configuration_queue{1000}; @@ -64,6 +65,7 @@ private: ThreadFromGlobalPool snapshot_thread; /// Apply or wait for configuration changes ThreadFromGlobalPool update_configuration_thread; + ThreadFromGlobalPool read_request_thread; /// RAFT wrapper. std::unique_ptr server; @@ -103,6 +105,8 @@ private: /// Thread apply or wait configuration changes from leader void updateConfigurationThread(); + void readRequestThread(); + void setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response); /// Add error responses for requests to responses queue. @@ -111,7 +115,7 @@ private: /// Forcefully wait for result and sets errors if something when wrong. /// Clears both arguments - void forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorage::RequestsForSessions & requests_for_sessions); + static void forceWaitAndProcessResult(RaftAppendResult & result); public: /// Just allocate some objects, real initialization is done by `intialize method` @@ -213,8 +217,6 @@ public: { keeper_stats.reset(); } - - ThreadPool bg_read_request{1}; }; } diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 0a7b1948495..b3fc8a11516 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -169,10 +169,6 @@ struct KeeperServer::KeeperRaftServer : public nuraft::raft_server ( 0ull, nuraft::msg_type::leader_status_request, 0, 0, 0ull, 0ull, 0ull ) ; auto result = send_msg_to_leader(req); - - if (!result->has_result()) - result->get(); - return result; } From 6d7050cb42cbdf658cd5e4c5ec141d0cd8f5c25e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 28 Jun 2022 09:54:16 +0000 Subject: [PATCH 17/40] Additional refactoring for request processing --- src/Coordination/KeeperDispatcher.cpp | 159 ++++++++++---------------- src/Coordination/KeeperServer.cpp | 3 +- 2 files changed, 63 insertions(+), 99 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index ae670ea0f68..edc04c03d81 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -46,6 +46,53 @@ void KeeperDispatcher::requestThread() KeeperStorage::RequestsForSessions write_requests; KeeperStorage::RequestsForSessions read_requests; + auto process_read_requests = [&, this]() mutable + { + server->getLeaderInfo()->when_ready([&, requests_for_sessions = std::move(read_requests)](nuraft::cmd_result> & result, nuraft::ptr &) mutable + { + if (!result.get_accepted() || result.get_result_code() == nuraft::cmd_result_code::TIMEOUT) + addErrorResponses(requests_for_sessions, Coordination::Error::ZOPERATIONTIMEOUT); + else if (result.get_result_code() != nuraft::cmd_result_code::OK) + addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); + + auto & leader_info_ctx = result.get(); + + KeeperServer::NodeInfo leader_info; + leader_info.term = leader_info_ctx->get_ulong(); + leader_info.last_committed_index = leader_info_ctx->get_ulong(); + + std::lock_guard lock(leader_waiter_mutex); + auto node_info = server->getNodeInfo(); + + if (node_info.term < leader_info.term || node_info.last_committed_index < leader_info.last_committed_index) + { + auto & leader_waiter = leader_waiters[leader_info]; + leader_waiter.insert(leader_waiter.end(), requests_for_sessions.begin(), requests_for_sessions.end()); + LOG_INFO(log, "waiting for {}, idx {}", leader_info.term, leader_info.last_committed_index); + } + else if (!read_requests_queue.push(std::move(requests_for_sessions))) + throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push read requests to queue"); + }); + read_requests.clear(); + }; + + auto process_write_requests = [&, this]() mutable + { + /// Forcefully process all previous pending requests + if (prev_result) + forceWaitAndProcessResult(prev_result); + + prev_result = server->putRequestBatch(write_requests); + prev_result->when_ready([&, requests_for_sessions = std::move(write_requests)](nuraft::cmd_result> & result, nuraft::ptr &) mutable + { + if (!result.get_accepted() || result.get_result_code() == nuraft::cmd_result_code::TIMEOUT) + addErrorResponses(requests_for_sessions, Coordination::Error::ZOPERATIONTIMEOUT); + else if (result.get_result_code() != nuraft::cmd_result_code::OK) + addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); + }); + write_requests.clear(); + }; + while (!shutdown_called) { KeeperStorage::RequestForSession request; @@ -56,33 +103,16 @@ void KeeperDispatcher::requestThread() try { - KeeperStorage::RequestsForSessions * current_batch{nullptr}; - bool is_current_read = false; - if (requests_queue->tryPop(request, max_wait)) { + if (shutdown_called) break; if (needs_quorum(coordination_settings, request)) - { write_requests.emplace_back(request); - - if (!current_batch) - { - current_batch = &write_requests; - is_current_read = false; - } - } else - { read_requests.emplace_back(request); - if (!current_batch) - { - current_batch = &read_requests; - is_current_read = true; - } - } /// Waiting until previous append will be successful, or batch is big enough /// has_result == false && get_result_code == OK means that our request still not processed. @@ -91,15 +121,13 @@ void KeeperDispatcher::requestThread() { if (write_requests.size() > max_batch_size) { - current_batch = &write_requests; - is_current_read = false; + process_write_requests(); break; } if (read_requests.size() > max_batch_size) { - current_batch = &read_requests; - is_current_read = true; + process_read_requests(); break; } @@ -111,8 +139,14 @@ void KeeperDispatcher::requestThread() else read_requests.emplace_back(request); } - else if (is_current_read || !prev_result || prev_result->has_result() || prev_result->get_result_code() != nuraft::cmd_result_code::OK) - break; + else + { + if (!read_requests.empty()) + process_read_requests(); + + if (!prev_result || prev_result->has_result() || prev_result->get_result_code() != nuraft::cmd_result_code::OK) + break; + } if (shutdown_called) break; @@ -121,73 +155,9 @@ void KeeperDispatcher::requestThread() if (shutdown_called) break; - /// Forcefully process all previous pending requests - if (!is_current_read && prev_result) - forceWaitAndProcessResult(prev_result); + if (!write_requests.empty()) + process_write_requests(); - if (current_batch) - { - LOG_INFO(&Poco::Logger::get("BATCHING"), "Processing {} batch of {}", is_current_read, current_batch->size()); - if (!is_current_read) - { - prev_result = server->putRequestBatch(*current_batch); - prev_result->when_ready([&, requests_for_sessions = std::move(*current_batch)](nuraft::cmd_result> & result, nuraft::ptr &) mutable - { - if (!result.get_accepted() || result.get_result_code() == nuraft::cmd_result_code::TIMEOUT) - addErrorResponses(requests_for_sessions, Coordination::Error::ZOPERATIONTIMEOUT); - else if (result.get_result_code() != nuraft::cmd_result_code::OK) - addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); - }); - - current_batch->clear(); - - if (!read_requests.empty()) - { - current_batch = &read_requests; - is_current_read = true; - } - else - current_batch = nullptr; - } - else - { - server->getLeaderInfo()->when_ready([&, requests_for_sessions = std::move(*current_batch)](nuraft::cmd_result> & result, nuraft::ptr &) mutable - { - if (!result.get_accepted() || result.get_result_code() == nuraft::cmd_result_code::TIMEOUT) - addErrorResponses(requests_for_sessions, Coordination::Error::ZOPERATIONTIMEOUT); - else if (result.get_result_code() != nuraft::cmd_result_code::OK) - addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); - - auto & leader_info_ctx = result.get(); - - KeeperServer::NodeInfo leader_info; - leader_info.term = leader_info_ctx->get_ulong(); - leader_info.last_committed_index = leader_info_ctx->get_ulong(); - - std::lock_guard lock(leader_waiter_mutex); - auto node_info = server->getNodeInfo(); - - if (node_info.term < leader_info.term || node_info.last_committed_index < leader_info.last_committed_index) - { - auto & leader_waiter = leader_waiters[leader_info]; - leader_waiter.insert(leader_waiter.end(), requests_for_sessions.begin(), requests_for_sessions.end()); - LOG_INFO(log, "waiting for {}, idx {}", leader_info.term, leader_info.last_committed_index); - } - else if (!read_requests_queue.push(std::move(requests_for_sessions))) - throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push read requests to queue"); - }); - - current_batch->clear(); - - if (!write_requests.empty()) - { - current_batch = &write_requests; - is_current_read = false; - } - else - current_batch = nullptr; - } - } } } catch (...) @@ -324,7 +294,6 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ return false; } - LOG_INFO(&Poco::Logger::get("BATCHING"), "Got request {}", request->getOpNum()); KeeperStorage::RequestForSession request_info; request_info.request = request; using namespace std::chrono; @@ -349,10 +318,8 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ unprocessed_requests.request_queue.push_back(std::move(request_info)); return true; } - else - { - ++unprocessed_requests.unprocessed_num; - } + + ++unprocessed_requests.unprocessed_num; } } @@ -577,11 +544,9 @@ void KeeperDispatcher::addErrorResponses(const KeeperStorage::RequestsForSession void KeeperDispatcher::forceWaitAndProcessResult(RaftAppendResult & result) { - LOG_INFO(&Poco::Logger::get("TESTER"), "Waiting for result"); if (!result->has_result()) result->get(); - LOG_INFO(&Poco::Logger::get("TESTER"), "Got result"); result = nullptr; } diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index b3fc8a11516..ae1164a65e9 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -168,8 +168,7 @@ struct KeeperServer::KeeperRaftServer : public nuraft::raft_server nuraft::ptr req = nuraft::cs_new ( 0ull, nuraft::msg_type::leader_status_request, 0, 0, 0ull, 0ull, 0ull ) ; - auto result = send_msg_to_leader(req); - return result; + return send_msg_to_leader(req); } using nuraft::raft_server::raft_server; From 73000a042d641e8bf02183e8b292d3dad13cd115 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 28 Jun 2022 13:44:30 +0000 Subject: [PATCH 18/40] Don't reconnect manually --- src/Coordination/KeeperDispatcher.cpp | 6 ++++++ .../src/jepsen/clickhouse_keeper/utils.clj | 2 +- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index edc04c03d81..f8ff21328f7 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -51,9 +51,15 @@ void KeeperDispatcher::requestThread() server->getLeaderInfo()->when_ready([&, requests_for_sessions = std::move(read_requests)](nuraft::cmd_result> & result, nuraft::ptr &) mutable { if (!result.get_accepted() || result.get_result_code() == nuraft::cmd_result_code::TIMEOUT) + { addErrorResponses(requests_for_sessions, Coordination::Error::ZOPERATIONTIMEOUT); + return; + } else if (result.get_result_code() != nuraft::cmd_result_code::OK) + { addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); + return; + } auto & leader_info_ctx = result.get(); diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/utils.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/utils.clj index 0457ff6eae2..3e192f66fb2 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/utils.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/utils.clj @@ -45,7 +45,7 @@ (defn zk-connect [host port timeout] - (exec-with-retries 30 (fn [] (zk/connect (str host ":" port) :timeout-msec timeout)))) + (zk/connect (str host ":" port) :timeout-msec timeout)) (defn zk-create-range [conn n] From 77144322d7693b643ac2bd09d378ae223b6559c6 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 29 Jun 2022 07:39:39 +0000 Subject: [PATCH 19/40] Add extra checks for read result --- src/Coordination/KeeperDispatcher.cpp | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index f8ff21328f7..65f71d22a95 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -48,7 +48,7 @@ void KeeperDispatcher::requestThread() auto process_read_requests = [&, this]() mutable { - server->getLeaderInfo()->when_ready([&, requests_for_sessions = std::move(read_requests)](nuraft::cmd_result> & result, nuraft::ptr &) mutable + server->getLeaderInfo()->when_ready([&, requests_for_sessions = std::move(read_requests)](nuraft::cmd_result> & result, nuraft::ptr & exception) mutable { if (!result.get_accepted() || result.get_result_code() == nuraft::cmd_result_code::TIMEOUT) { @@ -60,9 +60,21 @@ void KeeperDispatcher::requestThread() addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); return; } + else if (exception) + { + LOG_INFO(&Poco::Logger::get("KeeperDispatcher"), "Got exception while waiting for read results {}", exception->what()); + addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); + return; + } auto & leader_info_ctx = result.get(); + if (!leader_info_ctx) + { + addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); + return; + } + KeeperServer::NodeInfo leader_info; leader_info.term = leader_info_ctx->get_ulong(); leader_info.last_committed_index = leader_info_ctx->get_ulong(); From 6fd24b58e764e495957179c47f5fd94c3bc5ea95 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 29 Jun 2022 13:59:25 +0000 Subject: [PATCH 20/40] Revert "Allow only jepsen test" This reverts commit 197ae25aa3ec53ff2affc4e61a42cff438e3f9e3. --- .github/workflows/pull_request.yml | 3348 +++++++++++++++++++++++++++- 1 file changed, 3347 insertions(+), 1 deletion(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index f021554dd7b..79d54d77f06 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -109,11 +109,196 @@ jobs: with: name: changed_images path: ${{ runner.temp }}/changed_images.json + StyleCheck: + needs: DockerHubPush + runs-on: [self-hosted, style-checker] + if: ${{ success() || failure() }} + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{ runner.temp }}/style_check + EOF + - name: Download changed images + # even if artifact does not exist, e.g. on `do not test` label or failed Docker job + continue-on-error: true + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.TEMP_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Style Check + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 style_check.py + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FastTest: + needs: DockerHubPush + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/fasttest + REPO_COPY=${{runner.temp}}/fasttest/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + EOF + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" + mkdir "$GITHUB_WORKSPACE" + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.TEMP_PATH }} + - name: Fast Test + run: | + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 fast_test_check.py + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + CompatibilityCheck: + needs: [BuilderDebRelease] + runs-on: [self-hosted, style-checker] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/compatibility_check + REPO_COPY=${{runner.temp}}/compatibility_check/ClickHouse + REPORTS_PATH=${{runner.temp}}/reports_dir + EOF + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: CompatibilityCheck + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + SplitBuildSmokeTest: + needs: [BuilderDebSplitted] + runs-on: [self-hosted, style-checker] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/split_build_check + REPO_COPY=${{runner.temp}}/split_build_check/ClickHouse + REPORTS_PATH=${{runner.temp}}/reports_dir + EOF + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Split build check + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 split_build_smoke_check.py + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" ######################################################################################### #################################### ORDINARY BUILDS #################################### ######################################################################################### + BuilderDebRelease: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=package_release + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + with: + fetch-depth: 0 # for performance artifact + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" BuilderBinRelease: - needs: [DockerHubPush] + needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] steps: - name: Set envs @@ -157,9 +342,3170 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + # BuilderBinGCC: + # needs: [DockerHubPush, FastTest] + # runs-on: [self-hosted, builder] + # steps: + # - name: Set envs + # run: | + # cat >> "$GITHUB_ENV" << 'EOF' + # TEMP_PATH=${{runner.temp}}/build_check + # IMAGES_PATH=${{runner.temp}}/images_path + # REPO_COPY=${{runner.temp}}/build_check/ClickHouse + # CACHES_PATH=${{runner.temp}}/../ccaches + # BUILD_NAME=binary_gcc + # EOF + # - name: Download changed images + # uses: actions/download-artifact@v2 + # with: + # name: changed_images + # path: ${{ runner.temp }}/images_path + # - name: Clear repository + # run: | + # sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + # - name: Check out repository code + # uses: actions/checkout@v2 + # - name: Build + # run: | + # git -C "$GITHUB_WORKSPACE" submodule sync --recursive + # git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + # sudo rm -fr "$TEMP_PATH" + # mkdir -p "$TEMP_PATH" + # cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + # cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + # - name: Upload build URLs to artifacts + # if: ${{ success() || failure() }} + # uses: actions/upload-artifact@v2 + # with: + # name: ${{ env.BUILD_URLS }} + # path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + # - name: Cleanup + # if: always() + # run: | + # # shellcheck disable=SC2046 + # docker kill $(docker ps -q) ||: + # # shellcheck disable=SC2046 + # docker rm -f $(docker ps -a -q) ||: + # sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderDebAarch64: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=package_aarch64 + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + with: + fetch-depth: 0 # for performance artifact + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderDebAsan: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=package_asan + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderDebUBsan: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=package_ubsan + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderDebTsan: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=package_tsan + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderDebMsan: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=package_msan + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderDebDebug: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=package_debug + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" +########################################################################################## +##################################### SPECIAL BUILDS ##################################### +########################################################################################## + BuilderDebSplitted: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_splitted + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinTidy: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_tidy + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinDarwin: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_darwin + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinAarch64: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_aarch64 + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinFreeBSD: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_freebsd + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinDarwinAarch64: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_darwin_aarch64 + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinPPC64: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_ppc64le + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" +############################################################################################ +##################################### Docker images ####################################### +############################################################################################ + DockerServerImages: + needs: + - BuilderDebRelease + - BuilderDebAarch64 + runs-on: [self-hosted, style-checker] + steps: + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + with: + fetch-depth: 0 # It MUST BE THE SAME for all dependencies and the job itself + - name: Check docker clickhouse/clickhouse-server building + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 docker_server.py --release-type head --no-push + python3 docker_server.py --release-type head --no-push --no-ubuntu \ + --image-repo clickhouse/clickhouse-keeper --image-path docker/keeper + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################ +##################################### BUILD REPORTER ####################################### +############################################################################################ + BuilderReport: + needs: + - BuilderBinRelease + - BuilderDebAarch64 + - BuilderDebAsan + - BuilderDebDebug + - BuilderDebMsan + - BuilderDebRelease + - BuilderDebTsan + - BuilderDebUBsan + runs-on: [self-hosted, style-checker] + if: ${{ success() || failure() }} + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + CHECK_NAME=ClickHouse build check (actions) + REPORTS_PATH=${{runner.temp}}/reports_dir + TEMP_PATH=${{runner.temp}}/report_check + NEEDS_DATA_PATH=${{runner.temp}}/needs.json + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Report Builder + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cat > "$NEEDS_DATA_PATH" << 'EOF' + ${{ toJSON(needs) }} + EOF + cd "$GITHUB_WORKSPACE/tests/ci" + python3 build_report_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + BuilderSpecialReport: + needs: + - BuilderBinAarch64 + - BuilderBinDarwin + - BuilderBinDarwinAarch64 + - BuilderBinFreeBSD + # - BuilderBinGCC + - BuilderBinPPC64 + - BuilderBinTidy + - BuilderDebSplitted + runs-on: [self-hosted, style-checker] + if: ${{ success() || failure() }} + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/report_check + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=ClickHouse special build check (actions) + NEEDS_DATA_PATH=${{runner.temp}}/needs.json + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Report Builder + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cat > "$NEEDS_DATA_PATH" << 'EOF' + ${{ toJSON(needs) }} + EOF + cd "$GITHUB_WORKSPACE/tests/ci" + python3 build_report_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################## +########################### FUNCTIONAl STATELESS TESTS ####################################### +############################################################################################## + FunctionalStatelessTestRelease: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, actions) + REPO_COPY=${{runner.temp}}/stateless_release/ClickHouse + KILL_TIMEOUT=10800 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestReleaseDatabaseReplicated0: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_database_replicated + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, DatabaseReplicated, actions) + REPO_COPY=${{runner.temp}}/stateless_database_replicated/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=2 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestReleaseDatabaseReplicated1: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_database_replicated + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, DatabaseReplicated, actions) + REPO_COPY=${{runner.temp}}/stateless_database_replicated/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=2 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestReleaseWideParts: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_wide_parts + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, wide parts enabled, actions) + REPO_COPY=${{runner.temp}}/stateless_wide_parts/ClickHouse + KILL_TIMEOUT=10800 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestReleaseS3: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_s3_storage + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, s3 storage, actions) + REPO_COPY=${{runner.temp}}/stateless_s3_storage/ClickHouse + KILL_TIMEOUT=10800 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestAarch64: + needs: [BuilderDebAarch64] + runs-on: [self-hosted, func-tester-aarch64] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (aarch64, actions) + REPO_COPY=${{runner.temp}}/stateless_release/ClickHouse + KILL_TIMEOUT=10800 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestAsan0: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (address, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=2 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestAsan1: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (address, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=2 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestTsan0: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (thread, actions) + REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestTsan1: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (thread, actions) + REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestTsan2: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (thread, actions) + REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (ubsan, actions) + REPO_COPY=${{runner.temp}}/stateless_ubsan/ClickHouse + KILL_TIMEOUT=10800 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestMsan0: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (memory, actions) + REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestMsan1: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (memory, actions) + REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestMsan2: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (memory, actions) + REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestDebug0: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestDebug1: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestDebug2: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestFlakyCheck: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_flaky_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests flaky check (address, actions) + REPO_COPY=${{runner.temp}}/stateless_flaky_asan/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + TestsBugfixCheck: + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/tests_bugfix_check + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Tests bugfix validate check (actions) + KILL_TIMEOUT=3600 + REPO_COPY=${{runner.temp}}/tests_bugfix_check/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Bugfix test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + + TEMP_PATH="${TEMP_PATH}/integration" \ + REPORTS_PATH="${REPORTS_PATH}/integration" \ + python3 integration_test_check.py "Integration tests bugfix validate check" \ + --validate-bugfix --post-commit-status=file || echo 'ignore exit code' + + TEMP_PATH="${TEMP_PATH}/stateless" \ + REPORTS_PATH="${REPORTS_PATH}/stateless" \ + python3 functional_test_check.py "Stateless tests bugfix validate check" "$KILL_TIMEOUT" \ + --validate-bugfix --post-commit-status=file || echo 'ignore exit code' + + python3 bugfix_validate_check.py "${TEMP_PATH}/stateless/post_commit_status.tsv" "${TEMP_PATH}/integration/post_commit_status.tsv" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################## +############################ FUNCTIONAl STATEFUL TESTS ####################################### +############################################################################################## + FunctionalStatefulTestRelease: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (release, actions) + REPO_COPY=${{runner.temp}}/stateful_release/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestAarch64: + needs: [BuilderDebAarch64] + runs-on: [self-hosted, func-tester-aarch64] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (aarch64, actions) + REPO_COPY=${{runner.temp}}/stateful_release/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (address, actions) + REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (thread, actions) + REPO_COPY=${{runner.temp}}/stateful_tsan/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestMsan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_msan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (memory, actions) + REPO_COPY=${{runner.temp}}/stateful_msan/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (ubsan, actions) + REPO_COPY=${{runner.temp}}/stateful_ubsan/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestDebug: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (debug, actions) + REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################## +######################################### STRESS TESTS ####################################### +############################################################################################## + StressTestAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_thread + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (address, actions) + REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + StressTestTsan: + needs: [BuilderDebTsan] + # func testers have 16 cores + 128 GB memory + # while stress testers have 36 cores + 72 memory + # It would be better to have something like 32 + 128, + # but such servers almost unavailable as spot instances. + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_thread + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (thread, actions) + REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + StressTestMsan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (memory, actions) + REPO_COPY=${{runner.temp}}/stress_memory/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + StressTestUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_undefined + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (undefined, actions) + REPO_COPY=${{runner.temp}}/stress_undefined/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + StressTestDebug: + needs: [BuilderDebDebug] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (debug, actions) + REPO_COPY=${{runner.temp}}/stress_debug/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################## +##################################### AST FUZZERS ############################################ +############################################################################################## + ASTFuzzerTestAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (ASan, actions) + REPO_COPY=${{runner.temp}}/ast_fuzzer_asan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + ASTFuzzerTestTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (TSan, actions) + REPO_COPY=${{runner.temp}}/ast_fuzzer_tsan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + ASTFuzzerTestUBSan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (UBSan, actions) + REPO_COPY=${{runner.temp}}/ast_fuzzer_ubsan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + ASTFuzzerTestMSan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_msan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (MSan, actions) + REPO_COPY=${{runner.temp}}/ast_fuzzer_msan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + ASTFuzzerTestDebug: + needs: [BuilderDebDebug] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (debug, actions) + REPO_COPY=${{runner.temp}}/ast_fuzzer_debug/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################# +############################# INTEGRATION TESTS ############################################# +############################################################################################# + IntegrationTestsAsan0: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, actions) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAsan1: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, actions) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAsan2: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, actions) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsTsan0: + needs: [BuilderDebTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (thread, actions) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsTsan1: + needs: [BuilderDebTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (thread, actions) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsTsan2: + needs: [BuilderDebTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (thread, actions) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsTsan3: + needs: [BuilderDebTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (thread, actions) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsRelease0: + needs: [BuilderDebRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (release, actions) + REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=2 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsRelease1: + needs: [BuilderDebRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (release, actions) + REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=2 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsFlakyCheck: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan_flaky_check + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests flaky check (asan, actions) + REPO_COPY=${{runner.temp}}/integration_tests_asan_flaky_check/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################# +#################################### UNIT TESTS ############################################# +############################################################################################# + UnitTestsAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (asan, actions) + REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + UnitTestsReleaseClang: + needs: [BuilderBinRelease] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (release-clang, actions) + REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + # UnitTestsReleaseGCC: + # needs: [BuilderBinGCC] + # runs-on: [self-hosted, fuzzer-unit-tester] + # steps: + # - name: Set envs + # run: | + # cat >> "$GITHUB_ENV" << 'EOF' + # TEMP_PATH=${{runner.temp}}/unit_tests_asan + # REPORTS_PATH=${{runner.temp}}/reports_dir + # CHECK_NAME=Unit tests (release-gcc, actions) + # REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse + # EOF + # - name: Download json reports + # uses: actions/download-artifact@v2 + # with: + # path: ${{ env.REPORTS_PATH }} + # - name: Clear repository + # run: | + # sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + # - name: Check out repository code + # uses: actions/checkout@v2 + # - name: Unit test + # run: | + # sudo rm -fr "$TEMP_PATH" + # mkdir -p "$TEMP_PATH" + # cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + # cd "$REPO_COPY/tests/ci" + # python3 unit_tests_check.py "$CHECK_NAME" + # - name: Cleanup + # if: always() + # run: | + # # shellcheck disable=SC2046 + # docker kill $(docker ps -q) ||: + # # shellcheck disable=SC2046 + # docker rm -f $(docker ps -a -q) ||: + # sudo rm -fr "$TEMP_PATH" + UnitTestsTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (tsan, actions) + REPO_COPY=${{runner.temp}}/unit_tests_tsan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + UnitTestsMsan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_msan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (msan, actions) + REPO_COPY=${{runner.temp}}/unit_tests_msan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + UnitTestsUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (ubsan, actions) + REPO_COPY=${{runner.temp}}/unit_tests_ubsan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################# +#################################### PERFORMANCE TESTS ###################################### +############################################################################################# + PerformanceComparison0: + needs: [BuilderDebRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Performance Comparison + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 performance_comparison_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + PerformanceComparison1: + needs: [BuilderDebRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Performance Comparison + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 performance_comparison_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + PerformanceComparison2: + needs: [BuilderDebRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Performance Comparison + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 performance_comparison_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + PerformanceComparison3: + needs: [BuilderDebRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Performance Comparison + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 performance_comparison_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + PerformanceComparisonAarch0: + needs: [BuilderDebAarch64] + runs-on: [self-hosted, func-tester-aarch64] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison Aarch64 + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Performance Comparison + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 performance_comparison_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + PerformanceComparisonAarch1: + needs: [BuilderDebAarch64] + runs-on: [self-hosted, func-tester-aarch64] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison Aarch64 + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Performance Comparison + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 performance_comparison_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + PerformanceComparisonAarch2: + needs: [BuilderDebAarch64] + runs-on: [self-hosted, func-tester-aarch64] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison Aarch64 + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Performance Comparison + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 performance_comparison_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + PerformanceComparisonAarch3: + needs: [BuilderDebAarch64] + runs-on: [self-hosted, func-tester-aarch64] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison Aarch64 + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Performance Comparison + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 performance_comparison_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" ############################################################################################# ###################################### JEPSEN TESTS ######################################### ############################################################################################# Jepsen: needs: [BuilderBinRelease] uses: ./.github/workflows/jepsen.yml + + FinishCheck: + needs: + - StyleCheck + - DockerHubPush + - DockerServerImages + - CheckLabels + - BuilderReport + - FastTest + - FunctionalStatelessTestDebug0 + - FunctionalStatelessTestDebug1 + - FunctionalStatelessTestDebug2 + - FunctionalStatelessTestRelease + - FunctionalStatelessTestReleaseDatabaseReplicated0 + - FunctionalStatelessTestReleaseDatabaseReplicated1 + - FunctionalStatelessTestReleaseWideParts + - FunctionalStatelessTestAarch64 + - FunctionalStatelessTestAsan0 + - FunctionalStatelessTestAsan1 + - FunctionalStatelessTestTsan0 + - FunctionalStatelessTestTsan1 + - FunctionalStatelessTestTsan2 + - FunctionalStatelessTestMsan0 + - FunctionalStatelessTestMsan1 + - FunctionalStatelessTestMsan2 + - FunctionalStatelessTestUBsan + - FunctionalStatefulTestDebug + - FunctionalStatefulTestRelease + - FunctionalStatefulTestAarch64 + - FunctionalStatefulTestAsan + - FunctionalStatefulTestTsan + - FunctionalStatefulTestMsan + - FunctionalStatefulTestUBsan + - FunctionalStatelessTestReleaseS3 + - StressTestDebug + - StressTestAsan + - StressTestTsan + - StressTestMsan + - StressTestUBsan + - ASTFuzzerTestDebug + - ASTFuzzerTestAsan + - ASTFuzzerTestTsan + - ASTFuzzerTestMSan + - ASTFuzzerTestUBSan + - IntegrationTestsAsan0 + - IntegrationTestsAsan1 + - IntegrationTestsAsan2 + - IntegrationTestsRelease0 + - IntegrationTestsRelease1 + - IntegrationTestsTsan0 + - IntegrationTestsTsan1 + - IntegrationTestsTsan2 + - IntegrationTestsTsan3 + - PerformanceComparison0 + - PerformanceComparison1 + - PerformanceComparison2 + - PerformanceComparison3 + - PerformanceComparisonAarch0 + - PerformanceComparisonAarch1 + - PerformanceComparisonAarch2 + - PerformanceComparisonAarch3 + - UnitTestsAsan + - UnitTestsTsan + - UnitTestsMsan + - UnitTestsUBsan + - UnitTestsReleaseClang + - SplitBuildSmokeTest + - CompatibilityCheck + - IntegrationTestsFlakyCheck + - Jepsen + runs-on: [self-hosted, style-checker] + steps: + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Finish label + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 finish_check.py From ae0fe1ac85426e95befbd4fcc050eafdedf90706 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 1 Jul 2022 09:15:25 +0000 Subject: [PATCH 21/40] Add setting for read mode --- src/Coordination/CoordinationSettings.h | 5 +- src/Coordination/KeeperDispatcher.cpp | 113 ++++++++++-------- src/Coordination/KeeperServer.cpp | 13 ++ .../resources/keeper_config.xml | 1 + 4 files changed, 78 insertions(+), 54 deletions(-) diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index c1e41f46670..81edd86f671 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -26,7 +26,7 @@ struct Settings; M(Milliseconds, heart_beat_interval_ms, 500, "Heartbeat interval between quorum nodes", 0) \ M(Milliseconds, election_timeout_lower_bound_ms, 1000, "Lower bound of election timer (avoid too often leader elections)", 0) \ M(Milliseconds, election_timeout_upper_bound_ms, 2000, "Upper bound of election timer (avoid too often leader elections)", 0) \ - M(Milliseconds, leadership_expiry, 10000, "How often will leader node check if it still has majority. Set it lower or equal to election_timeout_lower_bound_ms to have linearizable reads.", 0) \ + M(Milliseconds, leadership_expiry, 0, "How often will leader node check if it still has majority. Set it lower or equal to election_timeout_lower_bound_ms to have linearizable reads.", 0) \ M(UInt64, reserved_log_items, 100000, "How many log items to store (don't remove during compaction)", 0) \ M(UInt64, snapshot_distance, 100000, "How many log items we have to collect to write new snapshot", 0) \ M(Bool, auto_forwarding, true, "Allow to forward write requests from followers to leader", 0) \ @@ -42,7 +42,8 @@ struct Settings; M(Bool, force_sync, true, "Call fsync on each change in RAFT changelog", 0) \ M(Bool, compress_logs, true, "Write compressed coordination logs in ZSTD format", 0) \ M(Bool, compress_snapshots_with_zstd_format, true, "Write compressed snapshots in ZSTD format (instead of custom LZ4)", 0) \ - M(UInt64, configuration_change_tries_count, 20, "How many times we will try to apply configuration change (add/remove server) to the cluster", 0) + M(UInt64, configuration_change_tries_count, 20, "How many times we will try to apply configuration change (add/remove server) to the cluster", 0) \ + M(String, read_mode, "nonlinear", "How should reads be processed", 0) DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 65f71d22a95..a31eb9195a2 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -40,75 +40,84 @@ void KeeperDispatcher::requestThread() const auto needs_quorum = [](const auto & coordination_settings, const auto & request) { - return coordination_settings->quorum_reads || !request.request->isReadRequest(); + return coordination_settings->quorum_reads || coordination_settings->read_mode.toString() == "quorum" || !request.request->isReadRequest(); }; - KeeperStorage::RequestsForSessions write_requests; + KeeperStorage::RequestsForSessions quorum_requests; KeeperStorage::RequestsForSessions read_requests; - auto process_read_requests = [&, this]() mutable + auto process_read_requests = [&, this](const auto & coordination_settings) mutable { - server->getLeaderInfo()->when_ready([&, requests_for_sessions = std::move(read_requests)](nuraft::cmd_result> & result, nuraft::ptr & exception) mutable + if (coordination_settings->read_mode.toString() == "fastlinear") { - if (!result.get_accepted() || result.get_result_code() == nuraft::cmd_result_code::TIMEOUT) + server->getLeaderInfo()->when_ready([&, requests_for_sessions = std::move(read_requests)](nuraft::cmd_result> & result, nuraft::ptr & exception) mutable { - addErrorResponses(requests_for_sessions, Coordination::Error::ZOPERATIONTIMEOUT); - return; - } - else if (result.get_result_code() != nuraft::cmd_result_code::OK) - { - addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); - return; - } - else if (exception) - { - LOG_INFO(&Poco::Logger::get("KeeperDispatcher"), "Got exception while waiting for read results {}", exception->what()); - addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); - return; - } + if (!result.get_accepted() || result.get_result_code() == nuraft::cmd_result_code::TIMEOUT) + { + addErrorResponses(requests_for_sessions, Coordination::Error::ZOPERATIONTIMEOUT); + return; + } + else if (result.get_result_code() != nuraft::cmd_result_code::OK) + { + addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); + return; + } + else if (exception) + { + LOG_INFO(&Poco::Logger::get("KeeperDispatcher"), "Got exception while waiting for read results {}", exception->what()); + addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); + return; + } - auto & leader_info_ctx = result.get(); + auto & leader_info_ctx = result.get(); - if (!leader_info_ctx) - { - addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); - return; - } + if (!leader_info_ctx) + { + addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); + return; + } - KeeperServer::NodeInfo leader_info; - leader_info.term = leader_info_ctx->get_ulong(); - leader_info.last_committed_index = leader_info_ctx->get_ulong(); + KeeperServer::NodeInfo leader_info; + leader_info.term = leader_info_ctx->get_ulong(); + leader_info.last_committed_index = leader_info_ctx->get_ulong(); + std::lock_guard lock(leader_waiter_mutex); + auto node_info = server->getNodeInfo(); - std::lock_guard lock(leader_waiter_mutex); - auto node_info = server->getNodeInfo(); - - if (node_info.term < leader_info.term || node_info.last_committed_index < leader_info.last_committed_index) - { - auto & leader_waiter = leader_waiters[leader_info]; - leader_waiter.insert(leader_waiter.end(), requests_for_sessions.begin(), requests_for_sessions.end()); - LOG_INFO(log, "waiting for {}, idx {}", leader_info.term, leader_info.last_committed_index); - } - else if (!read_requests_queue.push(std::move(requests_for_sessions))) + if (node_info.term < leader_info.term || node_info.last_committed_index < leader_info.last_committed_index) + { + auto & leader_waiter = leader_waiters[leader_info]; + leader_waiter.insert(leader_waiter.end(), requests_for_sessions.begin(), requests_for_sessions.end()); + LOG_INFO(log, "waiting for {}, idx {}", leader_info.term, leader_info.last_committed_index); + } + else if (!read_requests_queue.push(std::move(requests_for_sessions))) + throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push read requests to queue"); + }); + } + else + { + assert(coordination_settings->read_mode.toString() == "nonlinear"); + if (!read_requests_queue.push(std::move(read_requests))) throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push read requests to queue"); - }); + } + read_requests.clear(); }; - auto process_write_requests = [&, this]() mutable + auto process_quorum_requests = [&, this]() mutable { /// Forcefully process all previous pending requests if (prev_result) forceWaitAndProcessResult(prev_result); - prev_result = server->putRequestBatch(write_requests); - prev_result->when_ready([&, requests_for_sessions = std::move(write_requests)](nuraft::cmd_result> & result, nuraft::ptr &) mutable + prev_result = server->putRequestBatch(quorum_requests); + prev_result->when_ready([&, requests_for_sessions = std::move(quorum_requests)](nuraft::cmd_result> & result, nuraft::ptr &) mutable { if (!result.get_accepted() || result.get_result_code() == nuraft::cmd_result_code::TIMEOUT) addErrorResponses(requests_for_sessions, Coordination::Error::ZOPERATIONTIMEOUT); else if (result.get_result_code() != nuraft::cmd_result_code::OK) addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); }); - write_requests.clear(); + quorum_requests.clear(); }; while (!shutdown_called) @@ -128,7 +137,7 @@ void KeeperDispatcher::requestThread() break; if (needs_quorum(coordination_settings, request)) - write_requests.emplace_back(request); + quorum_requests.emplace_back(request); else read_requests.emplace_back(request); @@ -137,15 +146,15 @@ void KeeperDispatcher::requestThread() /// Sometimes NuRaft set errorcode without setting result, so we check both here. while (true) { - if (write_requests.size() > max_batch_size) + if (quorum_requests.size() > max_batch_size) { - process_write_requests(); + process_quorum_requests(); break; } if (read_requests.size() > max_batch_size) { - process_read_requests(); + process_read_requests(coordination_settings); break; } @@ -153,14 +162,14 @@ void KeeperDispatcher::requestThread() if (requests_queue->tryPop(request, 1)) { if (needs_quorum(coordination_settings, request)) - write_requests.emplace_back(request); + quorum_requests.emplace_back(request); else read_requests.emplace_back(request); } - else + else { if (!read_requests.empty()) - process_read_requests(); + process_read_requests(coordination_settings); if (!prev_result || prev_result->has_result() || prev_result->get_result_code() != nuraft::cmd_result_code::OK) break; @@ -173,8 +182,8 @@ void KeeperDispatcher::requestThread() if (shutdown_called) break; - if (!write_requests.empty()) - process_write_requests(); + if (!quorum_requests.empty()) + process_quorum_requests(); } } diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index dfea73de67b..3facf9df9e5 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -253,7 +253,20 @@ void KeeperServer::launchRaftServer(bool enable_ipv6) coordination_settings->election_timeout_lower_bound_ms.totalMilliseconds(), "election_timeout_lower_bound_ms", log); params.election_timeout_upper_bound_ = getValueOrMaxInt32AndLogWarning( coordination_settings->election_timeout_upper_bound_ms.totalMilliseconds(), "election_timeout_upper_bound_ms", log); + params.leadership_expiry_ = getValueOrMaxInt32AndLogWarning(coordination_settings->leadership_expiry.totalMilliseconds(), "leadership_expiry", log); + + if (coordination_settings->read_mode.toString() == "fastlinear") + { + if (params.leadership_expiry_ == 0) + params.leadership_expiry_ = params.election_timeout_lower_bound_; + else if (params.leadership_expiry_ > params.election_timeout_lower_bound_) + { + LOG_WARNING(log, "To use fast linearizable reads, leadership_expiry should be set to a value thath is less or equal to the election_timeout_upper_bound_ms. " + "Based on current settings, there are no guarantees for linearizability of reads."); + } + } + params.reserved_log_items_ = getValueOrMaxInt32AndLogWarning(coordination_settings->reserved_log_items, "reserved_log_items", log); params.snapshot_distance_ = getValueOrMaxInt32AndLogWarning(coordination_settings->snapshot_distance, "snapshot_distance", log); diff --git a/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml b/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml index db857693a7b..86a3da08bf9 100644 --- a/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml +++ b/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml @@ -23,6 +23,7 @@ 2000 4000 0 + fastlinear {snapshot_distance} {stale_log_gap} {reserved_log_items} From 5a97c0fc8bad77547f43429fc08bd9c1d9d660aa Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 1 Jul 2022 13:19:28 +0000 Subject: [PATCH 22/40] Finalize requests in separate thread --- src/Coordination/KeeperDispatcher.cpp | 45 ++++++++++++++++++++++----- src/Coordination/KeeperDispatcher.h | 4 +++ 2 files changed, 42 insertions(+), 7 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index a31eb9195a2..aebd58cc83f 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -25,6 +26,7 @@ namespace ErrorCodes KeeperDispatcher::KeeperDispatcher() : responses_queue(std::numeric_limits::max()) , read_requests_queue(std::numeric_limits::max()) + , finalize_requests_queue(std::numeric_limits::max()) , configuration_and_settings(std::make_shared()) , log(&Poco::Logger::get("KeeperDispatcher")) { @@ -37,6 +39,7 @@ void KeeperDispatcher::requestThread() /// Result of requests batch from previous iteration RaftAppendResult prev_result = nullptr; + const auto previous_quorum_done = [&] { return !prev_result || prev_result->has_result() || prev_result->get_result_code() != nuraft::cmd_result_code::OK; }; const auto needs_quorum = [](const auto & coordination_settings, const auto & request) { @@ -132,7 +135,6 @@ void KeeperDispatcher::requestThread() { if (requests_queue->tryPop(request, max_wait)) { - if (shutdown_called) break; @@ -147,15 +149,14 @@ void KeeperDispatcher::requestThread() while (true) { if (quorum_requests.size() > max_batch_size) - { - process_quorum_requests(); break; - } if (read_requests.size() > max_batch_size) { process_read_requests(coordination_settings); - break; + + if (previous_quorum_done()) + break; } /// Trying to get batch requests as fast as possible @@ -171,7 +172,7 @@ void KeeperDispatcher::requestThread() if (!read_requests.empty()) process_read_requests(coordination_settings); - if (!prev_result || prev_result->has_result() || prev_result->get_result_code() != nuraft::cmd_result_code::OK) + if (previous_quorum_done()) break; } @@ -265,6 +266,30 @@ void KeeperDispatcher::readRequestThread() addErrorResponses({request_info}, Coordination::Error::ZCONNECTIONLOSS); } + if (!finalize_requests_queue.push(std::move(requests))) + throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push read requests to queue"); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } +} + +void KeeperDispatcher::finalizeRequestsThread() +{ + setThreadName("KeeperFinalT"); + while (!shutdown_called) + { + KeeperStorage::RequestsForSessions requests; + if (!finalize_requests_queue.pop(requests)) + break; + + if (shutdown_called) + break; + + try + { finalizeRequests(requests); } catch (...) @@ -379,6 +404,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf responses_thread = ThreadFromGlobalPool([this] { responseThread(); }); snapshot_thread = ThreadFromGlobalPool([this] { snapshotThread(); }); read_request_thread = ThreadFromGlobalPool([this] { readRequestThread(); }); + finalize_requests_thread= ThreadFromGlobalPool([this] { finalizeRequestsThread(); }); server = std::make_unique(configuration_and_settings, config, responses_queue, snapshots_queue, [this](const KeeperStorage::RequestForSession & request_for_session, uint64_t log_term, uint64_t log_idx) { onRequestCommit(request_for_session, log_term, log_idx); }); @@ -448,6 +474,10 @@ void KeeperDispatcher::shutdown() if (read_request_thread.joinable()) read_request_thread.join(); + finalize_requests_queue.finish(); + if (finalize_requests_thread.joinable()) + finalize_requests_thread.join(); + update_configuration_queue.finish(); if (update_configuration_thread.joinable()) update_configuration_thread.join(); @@ -748,7 +778,8 @@ void KeeperDispatcher::finalizeRequests(const KeeperStorage::RequestsForSessions void KeeperDispatcher::onRequestCommit(const KeeperStorage::RequestForSession & request_for_session, uint64_t log_term, uint64_t log_idx) { - finalizeRequests({request_for_session}); + if (!finalize_requests_queue.push({request_for_session})) + throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push read requests to queue"); KeeperStorage::RequestsForSessions requests; { diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 50b842d73b4..b2c88d7ef30 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -36,6 +36,7 @@ private: ResponsesQueue responses_queue; SnapshotsQueue snapshots_queue{1}; ConcurrentBoundedQueue read_requests_queue; + ConcurrentBoundedQueue finalize_requests_queue; /// More than 1k updates is definitely misconfiguration. UpdateConfigurationQueue update_configuration_queue{1000}; @@ -66,6 +67,7 @@ private: /// Apply or wait for configuration changes ThreadFromGlobalPool update_configuration_thread; ThreadFromGlobalPool read_request_thread; + ThreadFromGlobalPool finalize_requests_thread; /// RAFT wrapper. std::unique_ptr server; @@ -107,6 +109,8 @@ private: void readRequestThread(); + void finalizeRequestsThread(); + void setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response); /// Add error responses for requests to responses queue. From 97e32e6ad9ce874576845802d9de58cb4d9db5b5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 1 Jul 2022 14:55:26 +0000 Subject: [PATCH 23/40] Use commit_ext in unit tests --- src/Coordination/tests/gtest_coordination.cpp | 22 ++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index ee75f2a0860..bf054ac5de3 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1312,8 +1312,9 @@ void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, uint changelog.append(entry); changelog.end_of_append_batch(0, 0); - state_machine->pre_commit(i, changelog.entry_at(i)->get_buf()); - state_machine->commit(i, changelog.entry_at(i)->get_buf()); + auto entry_buf = changelog.entry_at(i)->get_buf_ptr(); + state_machine->pre_commit(i, *entry_buf); + state_machine->commit_ext(nuraft::state_machine::ext_op_params{i, entry_buf}); bool snapshot_created = false; if (i % settings->snapshot_distance == 0) { @@ -1357,8 +1358,9 @@ void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, uint for (size_t i = restore_machine->last_commit_index() + 1; i < restore_changelog.next_slot(); ++i) { - restore_machine->pre_commit(i, changelog.entry_at(i)->get_buf()); - restore_machine->commit(i, changelog.entry_at(i)->get_buf()); + auto entry = changelog.entry_at(i)->get_buf_ptr(); + restore_machine->pre_commit(i, *entry); + restore_machine->commit_ext(nuraft::state_machine::ext_op_params{i, entry}); } auto & source_storage = state_machine->getStorage(); @@ -1459,18 +1461,18 @@ TEST_P(CoordinationTest, TestEphemeralNodeRemove) std::shared_ptr request_c = std::make_shared(); request_c->path = "/hello"; request_c->is_ephemeral = true; - auto entry_c = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), request_c); - state_machine->pre_commit(1, entry_c->get_buf()); - state_machine->commit(1, entry_c->get_buf()); + auto entry_c = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), request_c)->get_buf_ptr(); + state_machine->pre_commit(1, *entry_c); + state_machine->commit_ext(nuraft::state_machine::ext_op_params{1, entry_c}); const auto & storage = state_machine->getStorage(); EXPECT_EQ(storage.ephemerals.size(), 1); std::shared_ptr request_d = std::make_shared(); request_d->path = "/hello"; /// Delete from other session - auto entry_d = getLogEntryFromZKRequest(0, 2, state_machine->getNextZxid(), request_d); - state_machine->pre_commit(2, entry_d->get_buf()); - state_machine->commit(2, entry_d->get_buf()); + auto entry_d = getLogEntryFromZKRequest(0, 2, state_machine->getNextZxid(), request_d)->get_buf_ptr(); + state_machine->pre_commit(2, *entry_d); + state_machine->commit_ext(nuraft::state_machine::ext_op_params{2, entry_d}); EXPECT_EQ(storage.ephemerals.size(), 0); } From 8c776328812cafe32b61caafd0ad66929024e5ba Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 1 Jul 2022 14:58:19 +0000 Subject: [PATCH 24/40] Fix style --- src/Coordination/KeeperServer.cpp | 5 ++--- src/Coordination/KeeperServer.h | 5 ++--- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 3facf9df9e5..d20b8c78f5e 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -165,9 +165,8 @@ struct KeeperServer::KeeperRaftServer : public nuraft::raft_server RaftAppendResult getLeaderInfo() { - nuraft::ptr req = nuraft::cs_new - ( 0ull, nuraft::msg_type::leader_status_request, 0, 0, - 0ull, 0ull, 0ull ) ; + nuraft::ptr req + = nuraft::cs_new(0ull, nuraft::msg_type::leader_status_request, 0, 0, 0ull, 0ull, 0ull); return send_msg_to_leader(req); } diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 472cb67361c..fa219f73aeb 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -136,8 +136,8 @@ public: }; } -namespace std { - +namespace std +{ template <> struct hash { @@ -149,5 +149,4 @@ namespace std { return hash_state.get64(); } }; - } From fd1e5d43a036f6e965635c80acb6002ca9ab7ead Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 12 Jul 2022 07:52:50 +0000 Subject: [PATCH 25/40] fix unit tests in debug mode --- src/Coordination/tests/gtest_coordination.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 66e0dfe704b..1674194c13c 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1064,6 +1064,13 @@ void addNode(DB::KeeperStorage & storage, const std::string & path, const std::s node.setData(data); node.stat.ephemeralOwner = ephemeral_owner; storage.container.insertOrReplace(path, node); + auto child_it = storage.container.find(path); + auto child_path = DB::getBaseName(child_it->key); + storage.container.updateValue(DB::parentPath(StringRef{path}), [&](auto & parent) + { + parent.addChild(child_path); + parent.stat.numChildren++; + }); } TEST_P(CoordinationTest, TestStorageSnapshotSimple) @@ -1221,7 +1228,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotMode) storage.container.erase("/hello_" + std::to_string(i)); } EXPECT_EQ(storage.container.size(), 26); - EXPECT_EQ(storage.container.snapshotSizeWithVersion().first, 101); + EXPECT_EQ(storage.container.snapshotSizeWithVersion().first, 102); EXPECT_EQ(storage.container.snapshotSizeWithVersion().second, 1); auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, 50); @@ -1778,6 +1785,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotEqual) DB::KeeperSnapshotManager manager("./snapshots", 3, params.enable_compression); DB::KeeperStorage storage(500, "", true); + addNode(storage, "/hello", ""); for (size_t j = 0; j < 5000; ++j) { addNode(storage, "/hello_" + std::to_string(j), "world", 1); From 8bd86c39f6fa54ae492848bdacc1e123c3313d96 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 12 Jul 2022 08:00:31 +0000 Subject: [PATCH 26/40] Add valid values for read_mode setting --- src/Coordination/CoordinationSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 81edd86f671..5597bb7f82f 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -43,7 +43,7 @@ struct Settings; M(Bool, compress_logs, true, "Write compressed coordination logs in ZSTD format", 0) \ M(Bool, compress_snapshots_with_zstd_format, true, "Write compressed snapshots in ZSTD format (instead of custom LZ4)", 0) \ M(UInt64, configuration_change_tries_count, 20, "How many times we will try to apply configuration change (add/remove server) to the cluster", 0) \ - M(String, read_mode, "nonlinear", "How should reads be processed", 0) + M(String, read_mode, "nonlinear", "How should reads be processed. Valid values: 'nonlinear', 'fastlinear', 'quorum'", 0) DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS) From 2634f80956bcea78d1577a65ae938097cd47e06b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 13 Jul 2022 11:06:58 +0000 Subject: [PATCH 27/40] Remove trailing whitespaces --- src/Coordination/KeeperServer.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 40620832b5a..870d56fff39 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -139,7 +139,7 @@ public: }; } -namespace std +namespace std { template <> struct hash From 838a11a85a9121c5ae9f96b497de51441b485432 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 13 Jul 2022 11:35:06 +0000 Subject: [PATCH 28/40] Rename RaftAppendResult --- src/Coordination/KeeperDispatcher.cpp | 97 +++++++++++++++------------ src/Coordination/KeeperDispatcher.h | 2 +- src/Coordination/KeeperServer.cpp | 10 ++- src/Coordination/KeeperServer.h | 26 +++---- 4 files changed, 74 insertions(+), 61 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 8162fe277b2..3a744b41a05 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -45,7 +45,7 @@ void KeeperDispatcher::requestThread() setThreadName("KeeperReqT"); /// Result of requests batch from previous iteration - RaftAppendResult prev_result = nullptr; + RaftResult prev_result = nullptr; const auto previous_quorum_done = [&] { return !prev_result || prev_result->has_result() || prev_result->get_result_code() != nuraft::cmd_result_code::OK; }; const auto needs_quorum = [](const auto & coordination_settings, const auto & request) @@ -60,48 +60,52 @@ void KeeperDispatcher::requestThread() { if (coordination_settings->read_mode.toString() == "fastlinear") { - server->getLeaderInfo()->when_ready([&, requests_for_sessions = std::move(read_requests)](nuraft::cmd_result> & result, nuraft::ptr & exception) mutable + auto leader_info_result = server->getLeaderInfo(); + if (leader_info_result) { - if (!result.get_accepted() || result.get_result_code() == nuraft::cmd_result_code::TIMEOUT) + leader_info_result->when_ready([&, requests_for_sessions = std::move(read_requests)](nuraft::cmd_result> & result, nuraft::ptr & exception) mutable { - addErrorResponses(requests_for_sessions, Coordination::Error::ZOPERATIONTIMEOUT); - return; - } - else if (result.get_result_code() != nuraft::cmd_result_code::OK) - { - addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); - return; - } - else if (exception) - { - LOG_INFO(&Poco::Logger::get("KeeperDispatcher"), "Got exception while waiting for read results {}", exception->what()); - addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); - return; - } + if (!result.get_accepted() || result.get_result_code() == nuraft::cmd_result_code::TIMEOUT) + { + addErrorResponses(requests_for_sessions, Coordination::Error::ZOPERATIONTIMEOUT); + return; + } + else if (result.get_result_code() != nuraft::cmd_result_code::OK) + { + addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); + return; + } + else if (exception) + { + LOG_INFO(&Poco::Logger::get("KeeperDispatcher"), "Got exception while waiting for read results {}", exception->what()); + addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); + return; + } - auto & leader_info_ctx = result.get(); + auto & leader_info_ctx = result.get(); - if (!leader_info_ctx) - { - addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); - return; - } + if (!leader_info_ctx) + { + addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); + return; + } - KeeperServer::NodeInfo leader_info; - leader_info.term = leader_info_ctx->get_ulong(); - leader_info.last_committed_index = leader_info_ctx->get_ulong(); - std::lock_guard lock(leader_waiter_mutex); - auto node_info = server->getNodeInfo(); + KeeperServer::NodeInfo leader_info; + leader_info.term = leader_info_ctx->get_ulong(); + leader_info.last_committed_index = leader_info_ctx->get_ulong(); + std::lock_guard lock(leader_waiter_mutex); + auto node_info = server->getNodeInfo(); - if (node_info.term < leader_info.term || node_info.last_committed_index < leader_info.last_committed_index) - { - auto & leader_waiter = leader_waiters[leader_info]; - leader_waiter.insert(leader_waiter.end(), requests_for_sessions.begin(), requests_for_sessions.end()); - LOG_INFO(log, "waiting for {}, idx {}", leader_info.term, leader_info.last_committed_index); - } - else if (!read_requests_queue.push(std::move(requests_for_sessions))) - throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push read requests to queue"); - }); + if (node_info.term < leader_info.term || node_info.last_committed_index < leader_info.last_committed_index) + { + auto & leader_waiter = leader_waiters[leader_info]; + leader_waiter.insert(leader_waiter.end(), requests_for_sessions.begin(), requests_for_sessions.end()); + LOG_INFO(log, "waiting for {}, idx {}", leader_info.term, leader_info.last_committed_index); + } + else if (!read_requests_queue.push(std::move(requests_for_sessions))) + throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push read requests to queue"); + }); + } } else { @@ -120,13 +124,18 @@ void KeeperDispatcher::requestThread() forceWaitAndProcessResult(prev_result); prev_result = server->putRequestBatch(quorum_requests); - prev_result->when_ready([&, requests_for_sessions = std::move(quorum_requests)](nuraft::cmd_result> & result, nuraft::ptr &) mutable + + if (prev_result) { - if (!result.get_accepted() || result.get_result_code() == nuraft::cmd_result_code::TIMEOUT) - addErrorResponses(requests_for_sessions, Coordination::Error::ZOPERATIONTIMEOUT); - else if (result.get_result_code() != nuraft::cmd_result_code::OK) - addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); - }); + prev_result->when_ready([&, requests_for_sessions = std::move(quorum_requests)](nuraft::cmd_result> & result, nuraft::ptr &) mutable + { + if (!result.get_accepted() || result.get_result_code() == nuraft::cmd_result_code::TIMEOUT) + addErrorResponses(requests_for_sessions, Coordination::Error::ZOPERATIONTIMEOUT); + else if (result.get_result_code() != nuraft::cmd_result_code::OK) + addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); + }); + } + quorum_requests.clear(); }; @@ -618,7 +627,7 @@ void KeeperDispatcher::addErrorResponses(const KeeperStorage::RequestsForSession } } -void KeeperDispatcher::forceWaitAndProcessResult(RaftAppendResult & result) +void KeeperDispatcher::forceWaitAndProcessResult(RaftResult & result) { if (!result->has_result()) result->get(); diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 11cfd5afd08..6148b57b1f4 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -119,7 +119,7 @@ private: /// Forcefully wait for result and sets errors if something when wrong. /// Clears both arguments - static void forceWaitAndProcessResult(RaftAppendResult & result); + static void forceWaitAndProcessResult(RaftResult & result); public: /// Just allocate some objects, real initialization is done by `intialize method` diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 60e8e3b107a..bfbea82ed0c 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -163,7 +163,7 @@ struct KeeperServer::KeeperRaftServer : public nuraft::raft_server reconfigure(new_config); } - RaftAppendResult getLeaderInfo() + RaftResult getLeaderInfo() { nuraft::ptr req = nuraft::cs_new(0ull, nuraft::msg_type::leader_status_request, 0, 0, 0ull, 0ull, 0ull); @@ -447,7 +447,7 @@ void KeeperServer::putLocalReadRequest(const KeeperStorage::RequestForSession & state_machine->processReadRequest(request_for_session); } -RaftAppendResult KeeperServer::putRequestBatch(const KeeperStorage::RequestsForSessions & requests_for_sessions) +RaftResult KeeperServer::putRequestBatch(const KeeperStorage::RequestsForSessions & requests_for_sessions) { std::vector> entries; for (const auto & request_for_session : requests_for_sessions) @@ -658,8 +658,12 @@ std::vector KeeperServer::getDeadSessions() return state_machine->getDeadSessions(); } -RaftAppendResult KeeperServer::getLeaderInfo() +RaftResult KeeperServer::getLeaderInfo() { + std::lock_guard lock{server_write_mutex}; + if (is_recovering) + return nullptr; + return raft_instance->getLeaderInfo(); } diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 870d56fff39..813ddd00b9d 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -13,7 +13,7 @@ namespace DB { -using RaftAppendResult = nuraft::ptr>>; +using RaftResult = nuraft::ptr>>; class KeeperServer { @@ -80,7 +80,7 @@ public: /// Put batch of requests into Raft and get result of put. Responses will be set separately into /// responses_queue. - RaftAppendResult putRequestBatch(const KeeperStorage::RequestsForSessions & requests); + RaftResult putRequestBatch(const KeeperStorage::RequestsForSessions & requests); /// Return set of the non-active sessions std::vector getDeadSessions(); @@ -123,7 +123,7 @@ public: bool operator==(const NodeInfo &) const = default; }; - RaftAppendResult getLeaderInfo(); + RaftResult getLeaderInfo(); NodeInfo getNodeInfo(); /// Get configuration diff between current configuration in RAFT and in XML file @@ -141,15 +141,15 @@ public: } namespace std { - template <> - struct hash - { - size_t operator()(const DB::KeeperServer::NodeInfo & info) const + template <> + struct hash { - SipHash hash_state; - hash_state.update(info.term); - hash_state.update(info.last_committed_index); - return hash_state.get64(); - } - }; + size_t operator()(const DB::KeeperServer::NodeInfo & info) const + { + SipHash hash_state; + hash_state.update(info.term); + hash_state.update(info.last_committed_index); + return hash_state.get64(); + } + }; } From 46882791cdabbc1a2540675146ef1fea9cd30856 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 23 Aug 2022 07:29:47 +0000 Subject: [PATCH 29/40] Fix build --- src/Coordination/KeeperServer.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 6a6b9e14233..c9f9d6ab31d 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -127,7 +127,6 @@ KeeperServer::KeeperServer( coordination_settings, keeper_context, checkAndGetSuperdigest(configuration_and_settings_->super_digest), - config.getBool("keeper_server.digest_enabled", true), std::move(commit_callback)); state_manager = nuraft::cs_new( From 5d45f2faa9789af41c319edc130cb443c4cb8d1b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 23 Aug 2022 08:22:03 +0000 Subject: [PATCH 30/40] Add comments --- src/Coordination/KeeperDispatcher.cpp | 19 ++++++++++++++++++- src/Coordination/KeeperDispatcher.h | 6 ++++++ 2 files changed, 24 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 3a744b41a05..2bf6100d33e 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -60,6 +60,7 @@ void KeeperDispatcher::requestThread() { if (coordination_settings->read_mode.toString() == "fastlinear") { + // we just want to know what's the current latest committed log on Leader node auto leader_info_result = server->getLeaderInfo(); if (leader_info_result) { @@ -96,12 +97,14 @@ void KeeperDispatcher::requestThread() std::lock_guard lock(leader_waiter_mutex); auto node_info = server->getNodeInfo(); + // we're behind, we need to wait if (node_info.term < leader_info.term || node_info.last_committed_index < leader_info.last_committed_index) { auto & leader_waiter = leader_waiters[leader_info]; leader_waiter.insert(leader_waiter.end(), requests_for_sessions.begin(), requests_for_sessions.end()); - LOG_INFO(log, "waiting for {}, idx {}", leader_info.term, leader_info.last_committed_index); + LOG_TRACE(log, "waiting for term {}, idx {}", leader_info.term, leader_info.last_committed_index); } + // process it in background thread else if (!read_requests_queue.push(std::move(requests_for_sessions))) throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push read requests to queue"); }); @@ -139,6 +142,10 @@ void KeeperDispatcher::requestThread() quorum_requests.clear(); }; + // ZooKeeper requires that the requests inside a single session are processed in a strict order + // (we cannot process later requests before all the previous once are processed) + // By making sure that at this point we can either have just read requests or just write requests + // from a single session, we can process them independently while (!shutdown_called) { KeeperStorage::RequestForSession request; @@ -187,9 +194,13 @@ void KeeperDispatcher::requestThread() } else { + // batch of read requests can send at most one request + // so we don't care if the previous batch hasn't received response if (!read_requests.empty()) process_read_requests(coordination_settings); + // if we still didn't process previous batch we can + // increase are current batch even more if (previous_quorum_done()) break; } @@ -262,6 +273,7 @@ void KeeperDispatcher::snapshotThread() } } +// Background thread for processing read requests void KeeperDispatcher::readRequestThread() { setThreadName("KeeperReadT"); @@ -294,6 +306,9 @@ void KeeperDispatcher::readRequestThread() } } +// We finalize requests every time we commit a single log with request +// or process a batch of read requests. +// Because it can get heavy, we do it in background thread. void KeeperDispatcher::finalizeRequestsThread() { setThreadName("KeeperFinalT"); @@ -385,6 +400,7 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ { auto & unprocessed_requests = unprocessed_requests_it->second; + // queue is not empty, or the request types don't match, put it in the waiting queue if (!unprocessed_requests.request_queue.empty() || unprocessed_requests.is_read != request->isReadRequest()) { unprocessed_requests.request_queue.push_back(std::move(request_info)); @@ -778,6 +794,7 @@ void KeeperDispatcher::finalizeRequests(const KeeperStorage::RequestsForSessions { auto & request_queue = unprocessed_requests.request_queue; unprocessed_requests.is_read = !unprocessed_requests.is_read; + // start adding next type of requests while (!request_queue.empty() && request_queue.front().request->isReadRequest() == unprocessed_requests.is_read) { auto & front_request = request_queue.front(); diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 6148b57b1f4..6531031f458 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -84,6 +84,12 @@ private: std::unordered_map> leader_waiters; std::mutex leader_waiter_mutex; + // We can be actively processing one type of requests (either read or write) from a single session. + // If we receive a request of a type that is not currently being processed, we put it in the waiting queue. + // Also, we want to process them in ariving order, so if we have a different type in the queue, we cannot process that request + // but wait for all the previous requests to finish. + // E.g. READ -> WRITE -> READ, the last READ will go to the waiting queue even though we are currently processing the first READ + // because we have WRITE request before it that needs to be processed. struct UnprocessedRequests { size_t unprocessed_num{0}; From 6c180d0c44bc134f3b57d126ff079b180b489bce Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 23 Aug 2022 13:18:17 +0000 Subject: [PATCH 31/40] Use default expiry for jepsen --- tests/jepsen.clickhouse-keeper/resources/keeper_config.xml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml b/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml index 86a3da08bf9..677de5f6769 100644 --- a/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml +++ b/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml @@ -1,5 +1,5 @@ - 0.0.0.0 + :: trace @@ -19,7 +19,6 @@ 120000 trace 1000 - 1900 2000 4000 0 From 073b168b31a9842ffa498295b90abba70e55cc1e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 Aug 2022 07:50:08 +0000 Subject: [PATCH 32/40] add msg type string for leader status --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index 29e7ffa4275..dc63ab3cdc1 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 29e7ffa42756d1fbb6530b5e723ddc7b42bbeee3 +Subproject commit dc63ab3cdc1233ae6d6f343c42e52a9fc50b9e39 From c20b9e554b006aa5c256fd7522bb08f080f078cc Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 Aug 2022 09:12:52 +0000 Subject: [PATCH 33/40] Fix counter test --- .../src/jepsen/clickhouse_keeper/counter.clj | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/counter.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/counter.clj index 8501d521a11..e6e94371501 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/counter.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/counter.clj @@ -30,7 +30,7 @@ :read (try (assoc op :type :ok - :value (count (zk-list conn "/"))) + :value (count (zk-list conn root-path))) (catch Exception _ (assoc op :type :info, :error :connect-error))) :final-read (exec-with-retries 30 (fn [] (assoc op From 50af0b193884609019c4385b876a63d2e2f799f8 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 29 Aug 2022 10:50:18 +0000 Subject: [PATCH 34/40] Add bunch of comments for request processing --- src/Coordination/CoordinationSettings.cpp | 3 + src/Coordination/CoordinationSettings.h | 4 +- src/Coordination/KeeperDispatcher.cpp | 105 ++++++++++++------ src/Coordination/KeeperDispatcher.h | 23 ++-- src/Coordination/KeeperServer.cpp | 4 +- src/Coordination/KeeperStateMachine.cpp | 2 +- src/Coordination/KeeperStateMachine.h | 3 +- .../src/jepsen/clickhouse_keeper/db.clj | 1 - .../src/jepsen/clickhouse_keeper/main.clj | 2 +- 9 files changed, 98 insertions(+), 49 deletions(-) diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index 3e03ee0d6f4..3bc611809f4 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -189,6 +189,9 @@ KeeperConfigurationAndSettings::loadFromConfig(const Poco::Util::AbstractConfigu ret->coordination_settings->loadFromConfig("keeper_server.coordination_settings", config); + if (ret->coordination_settings->quorum_reads) + LOG_WARNING(&Poco::Logger::get("KeeperConfigurationAndSettings"), "Setting 'quorum_reads' is depricated. Please use 'read_mode'"); + return ret; } diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 9636c0ffd32..1a7cb7b11fb 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -38,12 +38,12 @@ struct Settings; M(UInt64, stale_log_gap, 10000, "When node became stale and should receive snapshots from leader", 0) \ M(UInt64, fresh_log_gap, 200, "When node became fresh", 0) \ M(UInt64, max_requests_batch_size, 100, "Max size of batch in requests count before it will be sent to RAFT", 0) \ - M(Bool, quorum_reads, false, "Execute read requests as writes through whole RAFT consesus with similar speed", 0) \ + M(Bool, quorum_reads, false, "Depricated - use read_mode. Execute read requests as writes through whole RAFT consesus with similar speed", 0) \ M(Bool, force_sync, true, "Call fsync on each change in RAFT changelog", 0) \ M(Bool, compress_logs, true, "Write compressed coordination logs in ZSTD format", 0) \ M(Bool, compress_snapshots_with_zstd_format, true, "Write compressed snapshots in ZSTD format (instead of custom LZ4)", 0) \ M(UInt64, configuration_change_tries_count, 20, "How many times we will try to apply configuration change (add/remove server) to the cluster", 0) \ - M(String, read_mode, "nonlinear", "How should reads be processed. Valid values: 'nonlinear', 'fastlinear', 'quorum'", 0) + M(String, read_mode, "nonlinear", "How should reads be processed. Valid values: 'nonlinear', 'fastlinear', 'quorum'. 'nonlinear' is the fastest option because there are no consistency requirements", 0) DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 2bf6100d33e..987fcb274ce 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -40,6 +40,33 @@ KeeperDispatcher::KeeperDispatcher() } +/// ZooKeepers has 2 requirements: +/// - writes need to be linearizable +/// - all requests from single session need to be processed in the order of their arrival +/// +/// Because of that, we cannot process read and write requests from SAME session at the same time. +/// To be able to process read and write reqeusts in parallel we need to make sure that only 1 type +/// of request is being processed from a single session. +/// Multiple types from different sessions can be processed at the same time. +/// +/// We do some in-session housekeeping to make sure that the multithreaded request processing is correct. +/// When a request is received from a client, we check if there are requests being processed from that same +/// session, and if yes, of what type. If the types are the same, and there are no requests of different +/// type inbetetween, we can instanly add it to active request queue. Otherwise, we need to wait until +/// all requests of the other tpye are processed. +/// +/// There are multiple threads used for processing the request, each of them communicating with a queue. +/// Assumption: only one type of request is being processed from a same session at any point in time (read or write). +/// +/// requestThread -> requests currently being processed +/// readRequestThread -> thread for processing read requests +/// finalizeRequestThread -> thread for finalizing requests: +/// - in-session housekeeping, add requests to the active request queue if there are any +/// +/// If reads are linearizable without quorum, a request can possibly wait for a certain log to be committed. +/// In that case we add it to the waiting queue for that log. +/// When that log is committed, the committing thread will send that read request to readRequestThread so it can be processed. +/// void KeeperDispatcher::requestThread() { setThreadName("KeeperReqT"); @@ -78,7 +105,7 @@ void KeeperDispatcher::requestThread() } else if (exception) { - LOG_INFO(&Poco::Logger::get("KeeperDispatcher"), "Got exception while waiting for read results {}", exception->what()); + LOG_INFO(log, "Got exception while waiting for read results {}", exception->what()); addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); return; } @@ -97,14 +124,14 @@ void KeeperDispatcher::requestThread() std::lock_guard lock(leader_waiter_mutex); auto node_info = server->getNodeInfo(); - // we're behind, we need to wait + /// we're behind, we need to wait if (node_info.term < leader_info.term || node_info.last_committed_index < leader_info.last_committed_index) { auto & leader_waiter = leader_waiters[leader_info]; leader_waiter.insert(leader_waiter.end(), requests_for_sessions.begin(), requests_for_sessions.end()); LOG_TRACE(log, "waiting for term {}, idx {}", leader_info.term, leader_info.last_committed_index); } - // process it in background thread + /// process it in background thread else if (!read_requests_queue.push(std::move(requests_for_sessions))) throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push read requests to queue"); }); @@ -142,10 +169,10 @@ void KeeperDispatcher::requestThread() quorum_requests.clear(); }; - // ZooKeeper requires that the requests inside a single session are processed in a strict order - // (we cannot process later requests before all the previous once are processed) - // By making sure that at this point we can either have just read requests or just write requests - // from a single session, we can process them independently + /// ZooKeeper requires that the requests inside a single session are processed in a strict order + /// (we cannot process later requests before all the previous once are processed) + /// By making sure that at this point we can either have just read requests or just write requests + /// from a single session, we can process them independently while (!shutdown_called) { KeeperStorage::RequestForSession request; @@ -156,7 +183,7 @@ void KeeperDispatcher::requestThread() try { - if (requests_queue->tryPop(request, max_wait)) + if (active_requests_queue->tryPop(request, max_wait)) { CurrentMetrics::sub(CurrentMetrics::KeeperOutstandingRequets); if (shutdown_called) @@ -184,7 +211,7 @@ void KeeperDispatcher::requestThread() } /// Trying to get batch requests as fast as possible - if (requests_queue->tryPop(request, 1)) + if (active_requests_queue->tryPop(request, 1)) { CurrentMetrics::sub(CurrentMetrics::KeeperOutstandingRequets); if (needs_quorum(coordination_settings, request)) @@ -194,13 +221,13 @@ void KeeperDispatcher::requestThread() } else { - // batch of read requests can send at most one request - // so we don't care if the previous batch hasn't received response + /// batch of read requests can send at most one request + /// so we don't care if the previous batch hasn't received response if (!read_requests.empty()) process_read_requests(coordination_settings); - // if we still didn't process previous batch we can - // increase are current batch even more + /// if we still didn't process previous batch we can + /// increase are current batch even more if (previous_quorum_done()) break; } @@ -273,7 +300,7 @@ void KeeperDispatcher::snapshotThread() } } -// Background thread for processing read requests +/// Background thread for processing read requests void KeeperDispatcher::readRequestThread() { setThreadName("KeeperReadT"); @@ -306,9 +333,9 @@ void KeeperDispatcher::readRequestThread() } } -// We finalize requests every time we commit a single log with request -// or process a batch of read requests. -// Because it can get heavy, we do it in background thread. +/// We finalize requests every time we commit a single log with request +/// or process a batch of read requests. +/// Because it can get heavy, we do it in background thread. void KeeperDispatcher::finalizeRequestsThread() { setThreadName("KeeperFinalT"); @@ -400,7 +427,7 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ { auto & unprocessed_requests = unprocessed_requests_it->second; - // queue is not empty, or the request types don't match, put it in the waiting queue + /// queue is not empty, or the request types don't match, put it in the waiting queue if (!unprocessed_requests.request_queue.empty() || unprocessed_requests.is_read != request->isReadRequest()) { unprocessed_requests.request_queue.push_back(std::move(request_info)); @@ -419,10 +446,10 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ /// Put close requests without timeouts if (request->getOpNum() == Coordination::OpNum::Close) { - if (!requests_queue->push(std::move(request_info))) + if (!active_requests_queue->push(std::move(request_info))) throw Exception("Cannot push request to queue", ErrorCodes::SYSTEM_ERROR); } - else if (!requests_queue->tryPush(std::move(request_info), configuration_and_settings->coordination_settings->operation_timeout_ms.totalMilliseconds())) + else if (!active_requests_queue->tryPush(std::move(request_info), configuration_and_settings->coordination_settings->operation_timeout_ms.totalMilliseconds())) { throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::TIMEOUT_EXCEEDED); } @@ -435,13 +462,13 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf LOG_DEBUG(log, "Initializing storage dispatcher"); configuration_and_settings = KeeperConfigurationAndSettings::loadFromConfig(config, standalone_keeper); - requests_queue = std::make_unique(configuration_and_settings->coordination_settings->max_requests_batch_size); + active_requests_queue = std::make_unique(configuration_and_settings->coordination_settings->max_requests_batch_size); request_thread = ThreadFromGlobalPool([this] { requestThread(); }); responses_thread = ThreadFromGlobalPool([this] { responseThread(); }); snapshot_thread = ThreadFromGlobalPool([this] { snapshotThread(); }); read_request_thread = ThreadFromGlobalPool([this] { readRequestThread(); }); - finalize_requests_thread= ThreadFromGlobalPool([this] { finalizeRequestsThread(); }); + finalize_requests_thread = ThreadFromGlobalPool([this] { finalizeRequestsThread(); }); server = std::make_unique(configuration_and_settings, config, responses_queue, snapshots_queue, [this](const KeeperStorage::RequestForSession & request_for_session, uint64_t log_term, uint64_t log_idx) { onRequestCommit(request_for_session, log_term, log_idx); }); @@ -491,9 +518,9 @@ void KeeperDispatcher::shutdown() if (session_cleaner_thread.joinable()) session_cleaner_thread.join(); - if (requests_queue) + if (active_requests_queue) { - requests_queue->finish(); + active_requests_queue->finish(); if (request_thread.joinable()) request_thread.join(); @@ -526,7 +553,7 @@ void KeeperDispatcher::shutdown() KeeperStorage::RequestForSession request_for_session; /// Set session expired for all pending requests - while (requests_queue && requests_queue->tryPop(request_for_session)) + while (active_requests_queue && active_requests_queue->tryPop(request_for_session)) { CurrentMetrics::sub(CurrentMetrics::KeeperOutstandingRequets); auto response = request_for_session.request->makeResponse(); @@ -593,7 +620,7 @@ void KeeperDispatcher::sessionCleanerTask() request_info.session_id = dead_session; { std::lock_guard lock(push_request_mutex); - if (!requests_queue->push(std::move(request_info))) + if (!active_requests_queue->push(std::move(request_info))) LOG_INFO(log, "Cannot push close request to queue while cleaning outdated sessions"); CurrentMetrics::add(CurrentMetrics::KeeperOutstandingRequets); } @@ -696,7 +723,7 @@ int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms) /// Push new session request to queue { std::lock_guard lock(push_request_mutex); - if (!requests_queue->tryPush(std::move(request_info), session_timeout_ms)) + if (!active_requests_queue->tryPush(std::move(request_info), session_timeout_ms)) throw Exception("Cannot push session id request to queue within session timeout", ErrorCodes::TIMEOUT_EXCEEDED); CurrentMetrics::add(CurrentMetrics::KeeperOutstandingRequets); } @@ -769,6 +796,10 @@ void KeeperDispatcher::updateConfigurationThread() } } +// Used to update the state for a session based on the requests +// - update the number of current unprocessed requests for the session +// - if the number of unprocessed requests is 0, we can start adding next type of requests +// from unprocessed requests queue to the active queue void KeeperDispatcher::finalizeRequests(const KeeperStorage::RequestsForSessions & requests_for_sessions) { std::unordered_map counts_for_session; @@ -792,26 +823,26 @@ void KeeperDispatcher::finalizeRequests(const KeeperStorage::RequestsForSessions { if (!unprocessed_requests.request_queue.empty()) { - auto & request_queue = unprocessed_requests.request_queue; + auto & unprocessed_requests_queue = unprocessed_requests.request_queue; unprocessed_requests.is_read = !unprocessed_requests.is_read; // start adding next type of requests - while (!request_queue.empty() && request_queue.front().request->isReadRequest() == unprocessed_requests.is_read) + while (!unprocessed_requests_queue.empty() && unprocessed_requests_queue.front().request->isReadRequest() == unprocessed_requests.is_read) { - auto & front_request = request_queue.front(); + auto & front_request = unprocessed_requests_queue.front(); /// Put close requests without timeouts if (front_request.request->getOpNum() == Coordination::OpNum::Close) { - if (!requests_queue->push(std::move(front_request))) + if (!active_requests_queue->push(std::move(front_request))) throw Exception("Cannot push request to queue", ErrorCodes::SYSTEM_ERROR); } - else if (!requests_queue->tryPush(std::move(front_request), configuration_and_settings->coordination_settings->operation_timeout_ms.totalMilliseconds())) + else if (!active_requests_queue->tryPush(std::move(front_request), configuration_and_settings->coordination_settings->operation_timeout_ms.totalMilliseconds())) { throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::TIMEOUT_EXCEEDED); } ++unprocessed_requests.unprocessed_num; - request_queue.pop_front(); + unprocessed_requests_queue.pop_front(); } } else @@ -822,6 +853,8 @@ void KeeperDispatcher::finalizeRequests(const KeeperStorage::RequestsForSessions } } +// Finalize request +// Process read requests that were waiting for this commit void KeeperDispatcher::onRequestCommit(const KeeperStorage::RequestForSession & request_for_session, uint64_t log_term, uint64_t log_idx) { if (!finalize_requests_queue.push({request_for_session})) @@ -837,6 +870,10 @@ void KeeperDispatcher::onRequestCommit(const KeeperStorage::RequestForSession & leader_waiters.erase(request_queue_it); } } + + if (requests.empty()) + return; + if (!read_requests_queue.push(std::move(requests))) throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push read requests to queue"); } @@ -905,7 +942,7 @@ Keeper4LWInfo KeeperDispatcher::getKeeper4LWInfo() const Keeper4LWInfo result = server->getPartiallyFilled4LWInfo(); { std::lock_guard lock(push_request_mutex); - result.outstanding_requests_count = requests_queue->size(); + result.outstanding_requests_count = active_requests_queue->size(); } { std::lock_guard lock(session_to_response_callback_mutex); diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 6531031f458..851f0165ae3 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -32,7 +32,8 @@ private: using UpdateConfigurationQueue = ConcurrentBoundedQueue; /// Size depends on coordination settings - std::unique_ptr requests_queue; + /// Request currently being processed + std::unique_ptr active_requests_queue; ResponsesQueue responses_queue; SnapshotsQueue snapshots_queue{1}; ConcurrentBoundedQueue read_requests_queue; @@ -81,22 +82,29 @@ private: /// Counter for new session_id requests. std::atomic internal_session_id_counter{0}; + /// A read request needs to have at least the log it was the last committed log on the leader + /// at the time the request was being made. + /// If the node is stale, we need to wait to commit that log before doing local read requests to achieve + /// linearizability. std::unordered_map> leader_waiters; std::mutex leader_waiter_mutex; - // We can be actively processing one type of requests (either read or write) from a single session. - // If we receive a request of a type that is not currently being processed, we put it in the waiting queue. - // Also, we want to process them in ariving order, so if we have a different type in the queue, we cannot process that request - // but wait for all the previous requests to finish. - // E.g. READ -> WRITE -> READ, the last READ will go to the waiting queue even though we are currently processing the first READ - // because we have WRITE request before it that needs to be processed. + /// We can be actively processing one type of requests (either read or write) from a single session. + /// If we receive a request of a type that is not currently being processed, we put it in the waiting queue. + /// Also, we want to process them in ariving order, so if we have a different type in the queue, we cannot process that request + /// but wait for all the previous requests to finish. + /// E.g. READ -> WRITE -> READ, the last READ will go to the waiting queue even though we are currently processing the first READ + /// because we have WRITE request before it that needs to be processed. struct UnprocessedRequests { + /// how many requests are currently in the active request queue size_t unprocessed_num{0}; + /// is_read currently being processed bool is_read{false}; std::list request_queue; }; + // Called every time a batch of reqeusts are processed. void finalizeRequests(const KeeperStorage::RequestsForSessions & requests_for_sessions); std::unordered_map unprocessed_requests_for_session; @@ -145,6 +153,7 @@ public: return server && server->checkInit(); } + /// Called when a single log with request is committed. void onRequestCommit(const KeeperStorage::RequestForSession & request_for_session, uint64_t log_term, uint64_t log_idx); /// Is server accepting requests, i.e. connected to the cluster diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index c9f9d6ab31d..7601881b326 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -114,7 +114,7 @@ KeeperServer::KeeperServer( , keeper_context{std::make_shared()} , create_snapshot_on_exit(config.getBool("keeper_server.create_snapshot_on_exit", true)) { - if (coordination_settings->quorum_reads) + if (coordination_settings->quorum_reads || coordination_settings->read_mode.toString() == "quorum") LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower."); keeper_context->digest_enabled = config.getBool("keeper_server.digest_enabled", false); @@ -287,7 +287,7 @@ void KeeperServer::launchRaftServer(const Poco::Util::AbstractConfiguration & co params.leadership_expiry_ = params.election_timeout_lower_bound_; else if (params.leadership_expiry_ > params.election_timeout_lower_bound_) { - LOG_WARNING(log, "To use fast linearizable reads, leadership_expiry should be set to a value thath is less or equal to the election_timeout_upper_bound_ms. " + LOG_WARNING(log, "To use fast linearizable reads, leadership_expiry should be set to a value that is less or equal to the election_timeout_upper_bound_ms. " "Based on current settings, there are no guarantees for linearizability of reads."); } } diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index c4486918dd2..dc52ad019cd 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -219,7 +219,7 @@ void KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req assertDigest(*request_for_session.digest, storage->getNodesDigest(false), *request_for_session.request, false); } -nuraft::ptr KeeperStateMachine::commit_ext(const ext_op_params& params) +nuraft::ptr KeeperStateMachine::commit_ext(const ext_op_params & params) { auto request_for_session = parseRequest(*params.data); if (!request_for_session.zxid) diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 4a4c887ba1b..de8f30437c0 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -40,7 +40,7 @@ public: nuraft::ptr pre_commit(uint64_t log_idx, nuraft::buffer & data) override; - nuraft::ptr commit_ext(const ext_op_params& params) override; /// NOLINT + nuraft::ptr commit_ext(const ext_op_params & params) override; /// NOLINT /// Save new cluster config to our snapshot (copy of the config stored in StateManager) void commit_config(const uint64_t log_idx, nuraft::ptr & new_conf) override; /// NOLINT @@ -148,6 +148,7 @@ private: /// Special part of ACL system -- superdigest specified in server config. const std::string superdigest; + // call when a request is committed const CommitCallback commit_callback; KeeperContextPtr keeper_context; diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj index c354e36e430..9e85b37dd75 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj @@ -98,7 +98,6 @@ #"\{srv2\}" (get nodes 1) #"\{srv3\}" (get nodes 2) #"\{id\}" (str (inc (.indexOf nodes node))) - #"\{quorum_reads\}" (str (boolean (:quorum test))) #"\{snapshot_distance\}" (str (:snapshot-distance test)) #"\{stale_log_gap\}" (str (:stale-log-gap test)) #"\{reserved_log_items\}" (str (:reserved-log-items test))}] diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/main.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/main.clj index cd1aa540e45..1919c8ce3ec 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/main.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/main.clj @@ -103,7 +103,7 @@ current-nemesis (get custom-nemesis/custom-nemesises (:nemesis opts))] (merge tests/noop-test opts - {:name (str "clickhouse-keeper-quorum=" quorum "-" (name (:workload opts)) "-" (name (:nemesis opts))) + {:name (str "clickhouse-keeper-" (name (:workload opts)) "-" (name (:nemesis opts))) :os ubuntu/os :db (get-db opts) :pure-generators true From 9acb2533341c72a42041717528f414bef5a24ff8 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Sep 2022 08:44:20 +0000 Subject: [PATCH 35/40] Update NuRaft --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index dc63ab3cdc1..362a8442dad 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit dc63ab3cdc1233ae6d6f343c42e52a9fc50b9e39 +Subproject commit 362a8442dadcd230febfa7f19b86c41765c45575 From fa4c7259f9b991f983dc90ff5c752a9c83bd4272 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 29 Aug 2022 13:56:51 +0200 Subject: [PATCH 36/40] Fix typos --- src/Coordination/CoordinationSettings.cpp | 2 +- src/Coordination/CoordinationSettings.h | 2 +- src/Coordination/KeeperDispatcher.cpp | 2 +- src/Coordination/KeeperDispatcher.h | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index 3bc611809f4..f634bcbb281 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -190,7 +190,7 @@ KeeperConfigurationAndSettings::loadFromConfig(const Poco::Util::AbstractConfigu ret->coordination_settings->loadFromConfig("keeper_server.coordination_settings", config); if (ret->coordination_settings->quorum_reads) - LOG_WARNING(&Poco::Logger::get("KeeperConfigurationAndSettings"), "Setting 'quorum_reads' is depricated. Please use 'read_mode'"); + LOG_WARNING(&Poco::Logger::get("KeeperConfigurationAndSettings"), "Setting 'quorum_reads' is deprecated. Please use 'read_mode'"); return ret; } diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 1a7cb7b11fb..971a0344e49 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -38,7 +38,7 @@ struct Settings; M(UInt64, stale_log_gap, 10000, "When node became stale and should receive snapshots from leader", 0) \ M(UInt64, fresh_log_gap, 200, "When node became fresh", 0) \ M(UInt64, max_requests_batch_size, 100, "Max size of batch in requests count before it will be sent to RAFT", 0) \ - M(Bool, quorum_reads, false, "Depricated - use read_mode. Execute read requests as writes through whole RAFT consesus with similar speed", 0) \ + M(Bool, quorum_reads, false, "Deprecated - use read_mode. Execute read requests as writes through whole RAFT consesus with similar speed", 0) \ M(Bool, force_sync, true, "Call fsync on each change in RAFT changelog", 0) \ M(Bool, compress_logs, true, "Write compressed coordination logs in ZSTD format", 0) \ M(Bool, compress_snapshots_with_zstd_format, true, "Write compressed snapshots in ZSTD format (instead of custom LZ4)", 0) \ diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 987fcb274ce..9b2f4e5f74e 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -45,7 +45,7 @@ KeeperDispatcher::KeeperDispatcher() /// - all requests from single session need to be processed in the order of their arrival /// /// Because of that, we cannot process read and write requests from SAME session at the same time. -/// To be able to process read and write reqeusts in parallel we need to make sure that only 1 type +/// To be able to process read and write requests in parallel we need to make sure that only 1 type /// of request is being processed from a single session. /// Multiple types from different sessions can be processed at the same time. /// diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 851f0165ae3..ae3334f17ec 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -104,7 +104,7 @@ private: std::list request_queue; }; - // Called every time a batch of reqeusts are processed. + // Called every time a batch of requests are processed. void finalizeRequests(const KeeperStorage::RequestsForSessions & requests_for_sessions); std::unordered_map unprocessed_requests_for_session; From f0ba3ee92702ded969a2fe4b839b2ecdfde2c101 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 5 Sep 2022 09:51:02 +0200 Subject: [PATCH 37/40] Fix typo --- src/Coordination/KeeperDispatcher.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 9b2f4e5f74e..847aeb2e02e 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -53,7 +53,7 @@ KeeperDispatcher::KeeperDispatcher() /// When a request is received from a client, we check if there are requests being processed from that same /// session, and if yes, of what type. If the types are the same, and there are no requests of different /// type inbetetween, we can instanly add it to active request queue. Otherwise, we need to wait until -/// all requests of the other tpye are processed. +/// all requests of the other type are processed. /// /// There are multiple threads used for processing the request, each of them communicating with a queue. /// Assumption: only one type of request is being processed from a same session at any point in time (read or write). From 951ee4422904eda4c31b14194305495437aebb58 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 7 Sep 2022 11:51:56 +0000 Subject: [PATCH 38/40] Process read requests with snapshot install --- src/Coordination/KeeperDispatcher.cpp | 45 ++++++++++++++++++++++++- src/Coordination/KeeperDispatcher.h | 5 ++- src/Coordination/KeeperServer.cpp | 6 ++-- src/Coordination/KeeperServer.h | 3 +- src/Coordination/KeeperStateMachine.cpp | 5 ++- src/Coordination/KeeperStateMachine.h | 8 +++-- 6 files changed, 64 insertions(+), 8 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 847aeb2e02e..40f5f539b4e 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -470,7 +471,15 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf read_request_thread = ThreadFromGlobalPool([this] { readRequestThread(); }); finalize_requests_thread = ThreadFromGlobalPool([this] { finalizeRequestsThread(); }); - server = std::make_unique(configuration_and_settings, config, responses_queue, snapshots_queue, [this](const KeeperStorage::RequestForSession & request_for_session, uint64_t log_term, uint64_t log_idx) { onRequestCommit(request_for_session, log_term, log_idx); }); + server = std::make_unique( + configuration_and_settings, + config, + responses_queue, + snapshots_queue, + [this](const KeeperStorage::RequestForSession & request_for_session, uint64_t log_term, uint64_t log_idx) + { onRequestCommit(request_for_session, log_term, log_idx); }, + [this](uint64_t term, uint64_t last_idx) + { onApplySnapshot(term, last_idx); }); try { @@ -878,6 +887,40 @@ void KeeperDispatcher::onRequestCommit(const KeeperStorage::RequestForSession & throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push read requests to queue"); } +/// Process all read request that are waiting for lower or currently last processed log index +void KeeperDispatcher::onApplySnapshot(uint64_t term, uint64_t last_idx) +{ + KeeperServer::NodeInfo current_node_info{term, last_idx}; + KeeperStorage::RequestsForSessions requests; + { + std::lock_guard lock(leader_waiter_mutex); + for (auto leader_waiter_it = leader_waiters.begin(); leader_waiter_it != leader_waiters.end();) + { + auto waiting_node_info = leader_waiter_it->first; + if (waiting_node_info.term <= current_node_info.term + && waiting_node_info.last_committed_index <= current_node_info.last_committed_index) + { + for (auto & request : leader_waiter_it->second) + { + requests.push_back(std::move(request)); + } + + leader_waiter_it = leader_waiters.erase(leader_waiter_it); + } + else + { + ++leader_waiter_it; + } + } + } + + if (requests.empty()) + return; + + if (!read_requests_queue.push(std::move(requests))) + throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push read requests to queue"); +} + bool KeeperDispatcher::isServerActive() const { return checkInit() && hasLeader() && !server->isRecovering(); diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index ae3334f17ec..0ebe67a4f39 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -86,7 +86,7 @@ private: /// at the time the request was being made. /// If the node is stale, we need to wait to commit that log before doing local read requests to achieve /// linearizability. - std::unordered_map> leader_waiters; + std::unordered_map leader_waiters; std::mutex leader_waiter_mutex; /// We can be actively processing one type of requests (either read or write) from a single session. @@ -156,6 +156,9 @@ public: /// Called when a single log with request is committed. void onRequestCommit(const KeeperStorage::RequestForSession & request_for_session, uint64_t log_term, uint64_t log_idx); + /// Called when a snapshot is applied + void onApplySnapshot(uint64_t term, uint64_t last_idx); + /// Is server accepting requests, i.e. connected to the cluster /// and achieved quorum bool isServerActive() const; diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 4b6b3826834..4ff19f9df2e 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -106,7 +106,8 @@ KeeperServer::KeeperServer( const Poco::Util::AbstractConfiguration & config, ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, - KeeperStateMachine::CommitCallback commit_callback) + KeeperStateMachine::CommitCallback commit_callback, + KeeperStateMachine::ApplySnapshotCallback apply_snapshot_callback) : server_id(configuration_and_settings_->server_id) , coordination_settings(configuration_and_settings_->coordination_settings) , log(&Poco::Logger::get("KeeperServer")) @@ -127,7 +128,8 @@ KeeperServer::KeeperServer( coordination_settings, keeper_context, checkAndGetSuperdigest(configuration_and_settings_->super_digest), - std::move(commit_callback)); + std::move(commit_callback), + std::move(apply_snapshot_callback)); state_manager = nuraft::cs_new( server_id, diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 7207af3e4a2..02ab643044a 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -72,7 +72,8 @@ public: const Poco::Util::AbstractConfiguration & config_, ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, - KeeperStateMachine::CommitCallback commit_callback); + KeeperStateMachine::CommitCallback commit_callback, + KeeperStateMachine::ApplySnapshotCallback apply_snapshot_callback); /// Load state machine from the latest snapshot and load log storage. Start NuRaft with required settings. void startup(const Poco::Util::AbstractConfiguration & config, bool enable_ipv6 = true); diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index dc52ad019cd..8a002279096 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -45,7 +45,8 @@ KeeperStateMachine::KeeperStateMachine( const CoordinationSettingsPtr & coordination_settings_, const KeeperContextPtr & keeper_context_, const std::string & superdigest_, - CommitCallback commit_callback_) + CommitCallback commit_callback_, + ApplySnapshotCallback apply_snapshot_callback_) : coordination_settings(coordination_settings_) , snapshot_manager( snapshots_path_, @@ -60,6 +61,7 @@ KeeperStateMachine::KeeperStateMachine( , log(&Poco::Logger::get("KeeperStateMachine")) , superdigest(superdigest_) , commit_callback(std::move(commit_callback_)) + , apply_snapshot_callback(std::move(apply_snapshot_callback_)) , keeper_context(keeper_context_) { } @@ -303,6 +305,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) ProfileEvents::increment(ProfileEvents::KeeperSnapshotApplys); last_committed_idx = s.get_last_log_idx(); + apply_snapshot_callback(s.get_last_log_term(), s.get_last_log_idx()); return true; } diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index de8f30437c0..7310eadd19a 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -21,6 +21,7 @@ class KeeperStateMachine : public nuraft::state_machine { public: using CommitCallback = std::function; + using ApplySnapshotCallback = std::function; KeeperStateMachine( ResponsesQueue & responses_queue_, @@ -29,7 +30,8 @@ public: const CoordinationSettingsPtr & coordination_settings_, const KeeperContextPtr & keeper_context_, const std::string & superdigest_ = "", - CommitCallback commit_callback_ = [](const KeeperStorage::RequestForSession &, uint64_t, uint64_t){}); + CommitCallback commit_callback_ = [](const KeeperStorage::RequestForSession &, uint64_t, uint64_t){}, + ApplySnapshotCallback apply_snapshot_callback_ = [](uint64_t, uint64_t){}); /// Read state from the latest snapshot void init(); @@ -148,8 +150,10 @@ private: /// Special part of ACL system -- superdigest specified in server config. const std::string superdigest; - // call when a request is committed + /// call when a request is committed const CommitCallback commit_callback; + /// call when snapshot is applied + const ApplySnapshotCallback apply_snapshot_callback; KeeperContextPtr keeper_context; }; From 761d53b438be4b2031d0acd4559cacabe355ff3d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 15 Sep 2022 16:14:45 +0000 Subject: [PATCH 39/40] Use merged NuRaft --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index e7f834e87ec..e15858f8ad0 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit e7f834e87ec5c82bc111840bc1f934d5866c042d +Subproject commit e15858f8ad0ce8aba85cf74e3763874c76bf927c From 39307f9dba215e13c97b87768cfc58ca6914f4da Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 15 Sep 2022 16:14:53 +0000 Subject: [PATCH 40/40] Extract into function --- src/Coordination/KeeperDispatcher.cpp | 134 +++++++++++++------------- src/Coordination/KeeperDispatcher.h | 2 + 2 files changed, 70 insertions(+), 66 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index cf7b82eda13..3445ef5ea23 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -84,70 +84,6 @@ void KeeperDispatcher::requestThread() KeeperStorage::RequestsForSessions quorum_requests; KeeperStorage::RequestsForSessions read_requests; - auto process_read_requests = [&, this](const auto & coordination_settings) mutable - { - if (coordination_settings->read_mode.toString() == "fastlinear") - { - // we just want to know what's the current latest committed log on Leader node - auto leader_info_result = server->getLeaderInfo(); - if (leader_info_result) - { - leader_info_result->when_ready([&, requests_for_sessions = std::move(read_requests)](nuraft::cmd_result> & result, nuraft::ptr & exception) mutable - { - if (!result.get_accepted() || result.get_result_code() == nuraft::cmd_result_code::TIMEOUT) - { - addErrorResponses(requests_for_sessions, Coordination::Error::ZOPERATIONTIMEOUT); - return; - } - else if (result.get_result_code() != nuraft::cmd_result_code::OK) - { - addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); - return; - } - else if (exception) - { - LOG_INFO(log, "Got exception while waiting for read results {}", exception->what()); - addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); - return; - } - - auto & leader_info_ctx = result.get(); - - if (!leader_info_ctx) - { - addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); - return; - } - - KeeperServer::NodeInfo leader_info; - leader_info.term = leader_info_ctx->get_ulong(); - leader_info.last_committed_index = leader_info_ctx->get_ulong(); - std::lock_guard lock(leader_waiter_mutex); - auto node_info = server->getNodeInfo(); - - /// we're behind, we need to wait - if (node_info.term < leader_info.term || node_info.last_committed_index < leader_info.last_committed_index) - { - auto & leader_waiter = leader_waiters[leader_info]; - leader_waiter.insert(leader_waiter.end(), requests_for_sessions.begin(), requests_for_sessions.end()); - LOG_TRACE(log, "waiting for term {}, idx {}", leader_info.term, leader_info.last_committed_index); - } - /// process it in background thread - else if (!read_requests_queue.push(std::move(requests_for_sessions))) - throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push read requests to queue"); - }); - } - } - else - { - assert(coordination_settings->read_mode.toString() == "nonlinear"); - if (!read_requests_queue.push(std::move(read_requests))) - throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push read requests to queue"); - } - - read_requests.clear(); - }; - auto process_quorum_requests = [&, this]() mutable { /// Forcefully process all previous pending requests @@ -205,7 +141,7 @@ void KeeperDispatcher::requestThread() if (read_requests.size() > max_batch_size) { - process_read_requests(coordination_settings); + processReadRequests(coordination_settings, read_requests); if (previous_quorum_done()) break; @@ -225,7 +161,7 @@ void KeeperDispatcher::requestThread() /// batch of read requests can send at most one request /// so we don't care if the previous batch hasn't received response if (!read_requests.empty()) - process_read_requests(coordination_settings); + processReadRequests(coordination_settings, read_requests); /// if we still didn't process previous batch we can /// increase are current batch even more @@ -252,6 +188,72 @@ void KeeperDispatcher::requestThread() } } +void KeeperDispatcher::processReadRequests(const CoordinationSettingsPtr & coordination_settings, KeeperStorage::RequestsForSessions & read_requests) +{ + if (coordination_settings->read_mode.toString() == "fastlinear") + { + // we just want to know what's the current latest committed log on Leader node + auto leader_info_result = server->getLeaderInfo(); + if (leader_info_result) + { + leader_info_result->when_ready([&, requests_for_sessions = std::move(read_requests)](nuraft::cmd_result> & result, nuraft::ptr & exception) mutable + { + if (!result.get_accepted() || result.get_result_code() == nuraft::cmd_result_code::TIMEOUT) + { + addErrorResponses(requests_for_sessions, Coordination::Error::ZOPERATIONTIMEOUT); + return; + } + + if (result.get_result_code() != nuraft::cmd_result_code::OK) + { + addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); + return; + } + + if (exception) + { + LOG_INFO(log, "Got exception while waiting for read results {}", exception->what()); + addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); + return; + } + + auto & leader_info_ctx = result.get(); + + if (!leader_info_ctx) + { + addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); + return; + } + + KeeperServer::NodeInfo leader_info; + leader_info.term = leader_info_ctx->get_ulong(); + leader_info.last_committed_index = leader_info_ctx->get_ulong(); + std::lock_guard lock(leader_waiter_mutex); + auto node_info = server->getNodeInfo(); + + /// we're behind, we need to wait + if (node_info.term < leader_info.term || node_info.last_committed_index < leader_info.last_committed_index) + { + auto & leader_waiter = leader_waiters[leader_info]; + leader_waiter.insert(leader_waiter.end(), requests_for_sessions.begin(), requests_for_sessions.end()); + LOG_TRACE(log, "waiting for term {}, idx {}", leader_info.term, leader_info.last_committed_index); + } + /// process it in background thread + else if (!read_requests_queue.push(std::move(requests_for_sessions))) + throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push read requests to queue"); + }); + } + } + else + { + assert(coordination_settings->read_mode.toString() == "nonlinear"); + if (!read_requests_queue.push(std::move(read_requests))) + throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push read requests to queue"); + } + + read_requests.clear(); +} + void KeeperDispatcher::responseThread() { setThreadName("KeeperRspT"); diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 0ebe67a4f39..6421db87793 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -125,6 +125,8 @@ private: void finalizeRequestsThread(); + void processReadRequests(const CoordinationSettingsPtr & coordination_settings, KeeperStorage::RequestsForSessions & read_requests); + void setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response); /// Add error responses for requests to responses queue.