From d7e6c8393fe2d55c246cae55fafdcc1faf34c6f9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 13 Jan 2021 13:32:20 +0300 Subject: [PATCH 01/86] Some useless code --- src/CMakeLists.txt | 4 +- src/Coordination/CMakeLists.txt | 0 src/Coordination/InMemoryLogStore.cpp | 193 +++++++++++++++++++++ src/Coordination/InMemoryLogStore.h | 47 +++++ src/Coordination/InMemoryStateManager.cpp | 32 ++++ src/Coordination/InMemoryStateManager.h | 41 +++++ src/Coordination/tests/gtest_for_build.cpp | 11 ++ 7 files changed, 327 insertions(+), 1 deletion(-) create mode 100644 src/Coordination/CMakeLists.txt create mode 100644 src/Coordination/InMemoryLogStore.cpp create mode 100644 src/Coordination/InMemoryLogStore.h create mode 100644 src/Coordination/InMemoryStateManager.cpp create mode 100644 src/Coordination/InMemoryStateManager.h create mode 100644 src/Coordination/tests/gtest_for_build.cpp diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 4e04f5607df..2027f527bae 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -60,6 +60,7 @@ add_subdirectory (Processors) add_subdirectory (Formats) add_subdirectory (Compression) add_subdirectory (Server) +add_subdirectory (Coordination) set(dbms_headers) @@ -185,6 +186,7 @@ add_object_library(clickhouse_processors_sources Processors/Sources) add_object_library(clickhouse_processors_merges Processors/Merges) add_object_library(clickhouse_processors_merges_algorithms Processors/Merges/Algorithms) add_object_library(clickhouse_processors_queryplan Processors/QueryPlan) +add_object_library(clickhouse_coordination Coordination) set (DBMS_COMMON_LIBRARIES) # libgcc_s does not provide an implementation of an atomics library. Instead, @@ -308,7 +310,7 @@ if (USE_KRB5) endif() if (USE_NURAFT) - dbms_target_link_libraries(PRIVATE ${NURAFT_LIBRARY}) + dbms_target_link_libraries(PUBLIC ${NURAFT_LIBRARY}) endif() if(RE2_INCLUDE_DIR) diff --git a/src/Coordination/CMakeLists.txt b/src/Coordination/CMakeLists.txt new file mode 100644 index 00000000000..e69de29bb2d diff --git a/src/Coordination/InMemoryLogStore.cpp b/src/Coordination/InMemoryLogStore.cpp new file mode 100644 index 00000000000..3b9ad3fe18a --- /dev/null +++ b/src/Coordination/InMemoryLogStore.cpp @@ -0,0 +1,193 @@ +#include + +namespace DB +{ + +namespace +{ +using namespace nuraft; +ptr makeClone(const ptr& entry) { + ptr clone = cs_new + ( entry->get_term(), + buffer::clone( entry->get_buf() ), + entry->get_val_type() ); + return clone; +} +} + +InMemoryLogStore::InMemoryLogStore() + : start_idx(1) +{} + +size_t InMemoryLogStore::start_index() const +{ + return start_idx; +} + +size_t InMemoryLogStore::next_slot() const +{ + std::lock_guard l(logs_lock); + // Exclude the dummy entry. + return start_idx + logs.size() - 1; +} + +nuraft::ptr InMemoryLogStore::last_entry() const +{ + ulong next_idx = next_slot(); + std::lock_guard lock(logs_lock); + auto entry = logs.find(next_idx - 1); + if (entry == logs.end()) + entry = logs.find(0); + + return makeClone(entry->second); +} + +size_t InMemoryLogStore::append(nuraft::ptr & entry) +{ + ptr clone = makeClone(entry); + + std::lock_guard l(logs_lock); + size_t idx = start_idx + logs.size() - 1; + logs[idx] = clone; + return idx; +} + +void InMemoryLogStore::write_at(size_t index, nuraft::ptr & entry) +{ + nuraft::ptr clone = makeClone(entry); + + // Discard all logs equal to or greater than `index. + std::lock_guard l(logs_lock); + auto itr = logs.lower_bound(index); + while (itr != logs.end()) + itr = logs.erase(itr); + logs[index] = clone; +} + +nuraft::ptr>> InMemoryLogStore::log_entries(size_t start, size_t end) +{ + nuraft::ptr>> ret = + nuraft::cs_new>>(); + + ret->resize(end - start); + size_t cc = 0; + for (size_t ii = start; ii < end; ++ii) + { + nuraft::ptr src = nullptr; + { + std::lock_guard l(logs_lock); + auto entry = logs.find(ii); + if (entry == logs.end()) + { + entry = logs.find(0); + assert(0); + } + src = entry->second; + } + (*ret)[cc++] = makeClone(src); + } + return ret; +} + +nuraft::ptr InMemoryLogStore::entry_at(size_t index) +{ + nuraft::ptr src = nullptr; + { + std::lock_guard l(logs_lock); + auto entry = logs.find(index); + if (entry == logs.end()) + entry = logs.find(0); + src = entry->second; + } + return makeClone(src); +} + +size_t InMemoryLogStore::term_at(size_t index) +{ + ulong term = 0; + { + std::lock_guard l(logs_lock); + auto entry = logs.find(index); + if (entry == logs.end()) + entry = logs.find(0); + term = entry->second->get_term(); + } + return term; +} + +nuraft::ptr InMemoryLogStore::pack(size_t index, Int32 cnt) +{ + std::vector> returned_logs; + + size_t size_total = 0; + for (ulong ii = index; ii < index + cnt; ++ii) + { + ptr le = nullptr; + { + std::lock_guard l(logs_lock); + le = logs[ii]; + } + assert(le.get()); + nuraft::ptr buf = le->serialize(); + size_total += buf->size(); + returned_logs.push_back(buf); + } + + nuraft::ptr buf_out = nuraft::buffer::alloc(sizeof(int32) + cnt * sizeof(int32) + size_total); + buf_out->pos(0); + buf_out->put(static_cast(cnt)); + + for (auto & entry : returned_logs) + { + nuraft::ptr & bb = entry; + buf_out->put(static_cast(bb->size())); + buf_out->put(*bb); + } + return buf_out; +} + +void InMemoryLogStore::apply_pack(size_t index, nuraft::buffer & pack) +{ + pack.pos(0); + Int32 num_logs = pack.get_int(); + + for (Int32 ii = 0; ii < num_logs; ++ii) + { + size_t cur_idx = index + ii; + Int32 buf_size = pack.get_int(); + + nuraft::ptr buf_local = nuraft::buffer::alloc(buf_size); + pack.get(buf_local); + + nuraft::ptr le = nuraft::log_entry::deserialize(*buf_local); + { + std::lock_guard l(logs_lock); + logs[cur_idx] = le; + } + } + + { + std::lock_guard l(logs_lock); + auto entry = logs.upper_bound(0); + if (entry != logs.end()) + start_idx = entry->first; + else + start_idx = 1; + } +} + +bool InMemoryLogStore::compact(size_t last_log_index) +{ + std::lock_guard l(logs_lock); + for (ulong ii = start_idx; ii <= last_log_index; ++ii) + { + auto entry = logs.find(ii); + if (entry != logs.end()) + logs.erase(entry); + } + + start_idx = last_log_index + 1; + return true; +} + +} diff --git a/src/Coordination/InMemoryLogStore.h b/src/Coordination/InMemoryLogStore.h new file mode 100644 index 00000000000..e9c41b50cf6 --- /dev/null +++ b/src/Coordination/InMemoryLogStore.h @@ -0,0 +1,47 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +class InMemoryLogStore : public nuraft::log_store +{ +public: + InMemoryLogStore(); + + size_t start_index() const override; + + size_t next_slot() const override; + + nuraft::ptr last_entry() const override; + + size_t append(nuraft::ptr & entry) override; + + void write_at(size_t index, nuraft::ptr & entry) override; + + nuraft::ptr>> log_entries(size_t start, size_t end) override; + + nuraft::ptr entry_at(size_t index) override; + + size_t term_at(size_t index) override; + + nuraft::ptr pack(size_t index, Int32 cnt) override; + + void apply_pack(size_t index, nuraft::buffer & pack) override; + + bool compact(size_t last_log_index) override; + + bool flush() override { return true; } + +private: + std::map> logs; + mutable std::mutex logs_lock; + std::atomic start_idx; +}; + +} diff --git a/src/Coordination/InMemoryStateManager.cpp b/src/Coordination/InMemoryStateManager.cpp new file mode 100644 index 00000000000..15a1f7aa622 --- /dev/null +++ b/src/Coordination/InMemoryStateManager.cpp @@ -0,0 +1,32 @@ +#include + +namespace DB +{ + +InMemoryStateManager::InMemoryStateManager(int my_server_id_, const std::string & endpoint_) + : my_server_id(my_server_id_) + , endpoint(endpoint_) + , log_store(nuraft::cs_new()) + , server_config(nuraft::cs_new(my_server_id, endpoint)) + , cluster_config(nuraft::cs_new()) +{ + cluster_config->get_servers().push_back(server_config); +} + +void InMemoryStateManager::save_config(const nuraft::cluster_config & config) +{ + // Just keep in memory in this example. + // Need to write to disk here, if want to make it durable. + nuraft::ptr buf = config.serialize(); + cluster_config = nuraft::cluster_config::deserialize(*buf); +} + +void InMemoryStateManager::save_state(const nuraft::srv_state & state) +{ + // Just keep in memory in this example. + // Need to write to disk here, if want to make it durable. + nuraft::ptr buf = state.serialize(); + server_state = nuraft::srv_state::deserialize(*buf); + } + +} diff --git a/src/Coordination/InMemoryStateManager.h b/src/Coordination/InMemoryStateManager.h new file mode 100644 index 00000000000..32eea343465 --- /dev/null +++ b/src/Coordination/InMemoryStateManager.h @@ -0,0 +1,41 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class InMemoryStateManager : public nuraft::state_mgr +{ +public: + InMemoryStateManager(int server_id_, const std::string & endpoint_); + + nuraft::ptr load_config() override { return cluster_config; } + + void save_config(const nuraft::cluster_config & config) override; + + void save_state(const nuraft::srv_state & state) override; + + nuraft::ptr read_state() override { return server_state; } + + nuraft::ptr load_log_store() override { return log_store; } + + Int32 server_id() override { return my_server_id; } + + nuraft::ptr get_srv_config() const { return server_config; } + + void system_exit(const int /* exit_code */) override {} + +private: + int my_server_id; + std::string endpoint; + nuraft::ptr log_store; + nuraft::ptr server_config; + nuraft::ptr cluster_config; + nuraft::ptr server_state; +}; + +} diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp new file mode 100644 index 00000000000..1026b779cdf --- /dev/null +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -0,0 +1,11 @@ +#include + +#include +#include + +TEST(CoordinationTest, BuildTest) +{ + DB::InMemoryLogStore store; + DB::InMemoryStateManager state_manager(1, "localhost:12345"); + EXPECT_EQ(1, 1); +} From 294e8f095d7cec5ef825c9c22dcfb5f9261e3f39 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 13 Jan 2021 18:00:39 +0300 Subject: [PATCH 02/86] I was able to replicate single number at localhost --- src/Coordination/InMemoryLogStore.cpp | 12 +- src/Coordination/tests/gtest_for_build.cpp | 175 +++++++++++++++++++++ 2 files changed, 181 insertions(+), 6 deletions(-) diff --git a/src/Coordination/InMemoryLogStore.cpp b/src/Coordination/InMemoryLogStore.cpp index 3b9ad3fe18a..9f8d398a110 100644 --- a/src/Coordination/InMemoryLogStore.cpp +++ b/src/Coordination/InMemoryLogStore.cpp @@ -6,18 +6,18 @@ namespace DB namespace { using namespace nuraft; -ptr makeClone(const ptr& entry) { - ptr clone = cs_new - ( entry->get_term(), - buffer::clone( entry->get_buf() ), - entry->get_val_type() ); +ptr makeClone(const ptr & entry) { + ptr clone = cs_new(entry->get_term(), buffer::clone(entry->get_buf()), entry->get_val_type()); return clone; } } InMemoryLogStore::InMemoryLogStore() : start_idx(1) -{} +{ + nuraft::ptr buf = nuraft::buffer::alloc(sizeof(size_t)); + logs[0] = nuraft::cs_new(0, buf); +} size_t InMemoryLogStore::start_index() const { diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index 1026b779cdf..f9856eb275a 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -2,10 +2,185 @@ #include #include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +} TEST(CoordinationTest, BuildTest) { DB::InMemoryLogStore store; DB::InMemoryStateManager state_manager(1, "localhost:12345"); + DB::SummingStateMachine machine; EXPECT_EQ(1, 1); } + +struct SummingRaftServer +{ + SummingRaftServer(int server_id_, const std::string & hostname_, int port_) + : server_id(server_id_) + , hostname(hostname_) + , port(port_) + , endpoint(hostname + ":" + std::to_string(port)) + , state_machine(nuraft::cs_new()) + , state_manager(nuraft::cs_new(server_id, endpoint)) + { + nuraft::raft_params params; + params.heart_beat_interval_ = 100; + params.election_timeout_lower_bound_ = 200; + params.election_timeout_upper_bound_ = 400; + params.reserved_log_items_ = 5; + params.snapshot_distance_ = 5; + params.client_req_timeout_ = 3000; + params.return_method_ = nuraft::raft_params::blocking; + + raft_instance = launcher.init( + state_machine, state_manager, nuraft::cs_new(), port, + nuraft::asio_service::options{}, params); + + if (!raft_instance) + { + std::cerr << "Failed to initialize launcher (see the message " + "in the log file)." << std::endl; + exit(-1); + } + std::cout << "init Raft instance " << server_id; + for (size_t ii = 0; ii < 20; ++ii) + { + if (raft_instance->is_initialized()) + { + std::cout << " done" << std::endl; + break; + } + std::cout << "."; + fflush(stdout); + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + } + + // Server ID. + int server_id; + + // Server address. + std::string hostname; + + // Server port. + int port; + + std::string endpoint; + + // State machine. + nuraft::ptr state_machine; + + // State manager. + nuraft::ptr state_manager; + + // Raft launcher. + nuraft::raft_launcher launcher; + + // Raft server instance. + nuraft::ptr raft_instance; +}; + +nuraft::ptr getLogEntry(int64_t number) +{ + nuraft::ptr ret = nuraft::buffer::alloc(sizeof(number)); + nuraft::buffer_serializer bs(ret); + // WARNING: We don't consider endian-safety in this example. + bs.put_raw(&number, sizeof(number)); + return ret; +} + +TEST(CoordinationTest, TestSummingRaft) +{ + SummingRaftServer s1(1, "localhost", 44444); + SummingRaftServer s2(2, "localhost", 44445); + SummingRaftServer s3(3, "localhost", 44446); + + nuraft::srv_config first_config(1, "localhost:44444"); + auto ret1 = s2.raft_instance->add_srv(first_config); + if (!ret1->get_accepted()) + { + std::cout << "failed to add server: " + << ret1->get_result_str() << std::endl; + EXPECT_TRUE(false); + } + + while(s1.raft_instance->get_leader() != 2) + { + std::cout << "Waiting s1 to join to s2 quorum\n"; + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + + nuraft::srv_config third_config(3, "localhost:44446"); + auto ret3 = s2.raft_instance->add_srv(third_config); + if (!ret3->get_accepted()) + { + std::cout << "failed to add server: " + << ret3->get_result_str() << std::endl; + EXPECT_TRUE(false); + } + + while(s3.raft_instance->get_leader() != 2) + { + std::cout << "Waiting s3 to join to s2 quorum\n"; + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + + /// S2 is leader + EXPECT_EQ(s1.raft_instance->get_leader(), 2); + EXPECT_EQ(s2.raft_instance->get_leader(), 2); + EXPECT_EQ(s3.raft_instance->get_leader(), 2); + + std::cerr << "Starting to add entries\n"; + auto entry = getLogEntry(1); + auto ret = s2.raft_instance->append_entries({entry}); + if (!ret->get_accepted()) + { + // Log append rejected, usually because this node is not a leader. + std::cout << "failed to replicate: entry 1" << ret->get_result_code() << std::endl; + EXPECT_TRUE(false); + } + if (ret->get_result_code() != nuraft::cmd_result_code::OK) + { + // Something went wrong. + // This means committing this log failed, + // but the log itself is still in the log store. + std::cout << "failed to replicate: entry 1" << ret->get_result_code() << std::endl; + EXPECT_TRUE(false); + } + else + { + std::cout << "Append ok\n"; + } + + while (s1.state_machine->getValue() != 1) + { + std::cout << "Waiting s1 to apply entry\n"; + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + + while (s2.state_machine->getValue() != 1) + { + std::cout << "Waiting s2 to apply entry\n"; + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + + EXPECT_EQ(s1.state_machine->getValue(), 1); + EXPECT_EQ(s2.state_machine->getValue(), 1); + EXPECT_EQ(s3.state_machine->getValue(), 1); + + s1.launcher.shutdown(5); + s2.launcher.shutdown(5); + s3.launcher.shutdown(5); +} From a2070bf13010d57e5614749177c1e7da3160c0a7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 14 Jan 2021 19:20:33 +0300 Subject: [PATCH 03/86] Add some missed files --- src/Coordination/LoggerWrapper.h | 40 +++++ src/Coordination/SummingStateMachine.cpp | 163 +++++++++++++++++++++ src/Coordination/SummingStateMachine.h | 77 ++++++++++ src/Coordination/tests/gtest_for_build.cpp | 91 +++++++++--- 4 files changed, 351 insertions(+), 20 deletions(-) create mode 100644 src/Coordination/LoggerWrapper.h create mode 100644 src/Coordination/SummingStateMachine.cpp create mode 100644 src/Coordination/SummingStateMachine.h diff --git a/src/Coordination/LoggerWrapper.h b/src/Coordination/LoggerWrapper.h new file mode 100644 index 00000000000..51718eaee8b --- /dev/null +++ b/src/Coordination/LoggerWrapper.h @@ -0,0 +1,40 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class LoggerWrapper : public nuraft::logger +{ + LoggerWrapper(const std::string & name) + : log(&Poco::Logger::get(name)) + {} + + void put_details( + int level, + const char * /* source_file */, + const char * /* func_name */, + size_t /* line_number */, + const std::string & msg) override + { + LOG_IMPL(log, level, level, msg); + } + + void set_level(int level) override + { + level = std::max(6, std::min(1, level)); + log->setLevel(level); + } + + int get_level() override + { + return log->getLevel(); + } + +pivate: + Poco::Logger * log; +}; + +} diff --git a/src/Coordination/SummingStateMachine.cpp b/src/Coordination/SummingStateMachine.cpp new file mode 100644 index 00000000000..16154ca8cd4 --- /dev/null +++ b/src/Coordination/SummingStateMachine.cpp @@ -0,0 +1,163 @@ +#include +#include + +namespace DB +{ + +static int64_t deserializeValue(nuraft::buffer & buffer) +{ + nuraft::buffer_serializer bs(buffer); + int64_t result; + memcpy(&result, bs.get_raw(buffer.size()), sizeof(result)); + return result; +} + +SummingStateMachine::SummingStateMachine() + : value(0) + , last_committed_idx(0) +{ +} + +nuraft::ptr SummingStateMachine::commit(const size_t log_idx, nuraft::buffer & data) +{ + int64_t value_to_add = deserializeValue(data); + + value += value_to_add; + last_committed_idx = log_idx; + + // Return Raft log number as a return result. + nuraft::ptr ret = nuraft::buffer::alloc(sizeof(log_idx)); + nuraft::buffer_serializer bs(ret); + bs.put_u64(log_idx); + return ret; +} + +bool SummingStateMachine::apply_snapshot(nuraft::snapshot & s) +{ + std::lock_guard ll(snapshots_lock); + auto entry = snapshots.find(s.get_last_log_idx()); + if (entry == snapshots.end()) + return false; + + auto ctx = entry->second; + value = ctx->value; + return true; +} + +nuraft::ptr SummingStateMachine::last_snapshot() +{ + // Just return the latest snapshot. + std::lock_guard ll(snapshots_lock); + auto entry = snapshots.rbegin(); + if (entry == snapshots.rend()) return nullptr; + + auto ctx = entry->second; + return ctx->snapshot; +} + + +void SummingStateMachine::createSnapshotInternal(nuraft::snapshot & s) +{ + // Clone snapshot from `s`. + nuraft::ptr snp_buf = s.serialize(); + nuraft::ptr ss = nuraft::snapshot::deserialize(*snp_buf); + + // Put into snapshot map. + auto ctx = cs_new(ss, value); + snapshots[s.get_last_log_idx()] = ctx; + + // Maintain last 3 snapshots only. + const int MAX_SNAPSHOTS = 3; + int num = snapshots.size(); + auto entry = snapshots.begin(); + + for (int ii = 0; ii < num - MAX_SNAPSHOTS; ++ii) + { + if (entry == snapshots.end()) + break; + entry = snapshots.erase(entry); + } +} + +void SummingStateMachine::save_logical_snp_obj( + nuraft::snapshot & s, + size_t & obj_id, + nuraft::buffer & data, + bool /*is_first_obj*/, + bool /*is_last_obj*/) +{ + if (obj_id == 0) + { + // Object ID == 0: it contains dummy value, create snapshot context. + createSnapshotInternal(s); + } + else + { + // Object ID > 0: actual snapshot value. + nuraft::buffer_serializer bs(data); + int64_t local_value = static_cast(bs.get_u64()); + + std::lock_guard ll(snapshots_lock); + auto entry = snapshots.find(s.get_last_log_idx()); + assert(entry != snapshots.end()); + entry->second->value = local_value; + } + // Request next object. + obj_id++; +} + +int SummingStateMachine::read_logical_snp_obj( + nuraft::snapshot & s, + void* & /*user_snp_ctx*/, + ulong obj_id, + nuraft::ptr & data_out, + bool & is_last_obj) +{ + nuraft::ptr ctx = nullptr; + { + std::lock_guard ll(snapshots_lock); + auto entry = snapshots.find(s.get_last_log_idx()); + if (entry == snapshots.end()) { + // Snapshot doesn't exist. + data_out = nullptr; + is_last_obj = true; + return 0; + } + ctx = entry->second; + } + + if (obj_id == 0) + { + // Object ID == 0: first object, put dummy data. + data_out = nuraft::buffer::alloc(sizeof(Int32)); + nuraft::buffer_serializer bs(data_out); + bs.put_i32(0); + is_last_obj = false; + + } + else + { + // Object ID > 0: second object, put actual value. + data_out = nuraft::buffer::alloc(sizeof(size_t)); + nuraft::buffer_serializer bs(data_out); + bs.put_u64(ctx->value); + is_last_obj = true; + } + return 0; +} + +void SummingStateMachine::create_snapshot( + nuraft::snapshot & s, + nuraft::async_result::handler_type & when_done) +{ + { + std::lock_guard ll(snapshots_lock); + createSnapshotInternal(s); + } + nuraft::ptr except(nullptr); + bool ret = true; + when_done(ret, except); +} + + +} diff --git a/src/Coordination/SummingStateMachine.h b/src/Coordination/SummingStateMachine.h new file mode 100644 index 00000000000..df343378408 --- /dev/null +++ b/src/Coordination/SummingStateMachine.h @@ -0,0 +1,77 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +class SummingStateMachine : public nuraft::state_machine +{ +public: + SummingStateMachine(); + + nuraft::ptr pre_commit(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override { return nullptr; } + + nuraft::ptr commit(const size_t log_idx, nuraft::buffer & data) override; + + void rollback(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override {} + + size_t last_commit_index() override { return last_committed_idx; } + + bool apply_snapshot(nuraft::snapshot & s) override; + + nuraft::ptr last_snapshot() override; + + void create_snapshot( + nuraft::snapshot & s, + nuraft::async_result::handler_type & when_done) override; + + void save_logical_snp_obj( + nuraft::snapshot & s, + size_t & obj_id, + nuraft::buffer & data, + bool is_first_obj, + bool is_last_obj) override; + + int read_logical_snp_obj( + nuraft::snapshot & s, + void* & user_snp_ctx, + ulong obj_id, + nuraft::ptr & data_out, + bool & is_last_obj) override; + + int64_t getValue() const { return value; } + +private: + struct SingleValueSnapshotContext + { + SingleValueSnapshotContext(nuraft::ptr & s, int64_t v) + : snapshot(s) + , value(v) + {} + + nuraft::ptr snapshot; + int64_t value; + }; + + void createSnapshotInternal(nuraft::snapshot & s); + + // State machine's current value. + std::atomic value; + + // Last committed Raft log number. + std::atomic last_committed_idx; + + // Keeps the last 3 snapshots, by their Raft log numbers. + std::map> snapshots; + + // Mutex for `snapshots_`. + std::mutex snapshots_lock; + +}; + +} diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index f9856eb275a..5785c9adb27 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -45,7 +46,7 @@ struct SummingRaftServer params.return_method_ = nuraft::raft_params::blocking; raft_instance = launcher.init( - state_machine, state_manager, nuraft::cs_new(), port, + state_machine, state_manager, nuraft::cs_new(), port, nuraft::asio_service::options{}, params); if (!raft_instance) @@ -101,7 +102,31 @@ nuraft::ptr getLogEntry(int64_t number) return ret; } -TEST(CoordinationTest, TestSummingRaft) + +TEST(CoordinationTest, TestSummingRaft1) +{ + SummingRaftServer s1(1, "localhost", 44444); + + /// Single node is leader + EXPECT_EQ(s1.raft_instance->get_leader(), 1); + + auto entry1 = getLogEntry(143); + auto ret = s1.raft_instance->append_entries({entry}); + EXPECT_TRUE(ret->get_accepted()) << "failed to replicate: entry 1" << ret->get_result_code(); + EXPECT_EQ(ret->get_result_code(), nuraft::cmd_result_code::OK) << "failed to replicate: entry 1" << ret->get_result_code(); + + while (s1.state_machine->getValue() != 143) + { + std::cout << "Waiting s1 to apply entry\n"; + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + + EXPECT_EQ(s1.state_machine->getValue(), 143); + + s1.launcher.shutdown(5); +} + +TEST(CoordinationTest, TestSummingRaft3) { SummingRaftServer s1(1, "localhost", 44444); SummingRaftServer s2(2, "localhost", 44445); @@ -145,24 +170,8 @@ TEST(CoordinationTest, TestSummingRaft) std::cerr << "Starting to add entries\n"; auto entry = getLogEntry(1); auto ret = s2.raft_instance->append_entries({entry}); - if (!ret->get_accepted()) - { - // Log append rejected, usually because this node is not a leader. - std::cout << "failed to replicate: entry 1" << ret->get_result_code() << std::endl; - EXPECT_TRUE(false); - } - if (ret->get_result_code() != nuraft::cmd_result_code::OK) - { - // Something went wrong. - // This means committing this log failed, - // but the log itself is still in the log store. - std::cout << "failed to replicate: entry 1" << ret->get_result_code() << std::endl; - EXPECT_TRUE(false); - } - else - { - std::cout << "Append ok\n"; - } + EXPECT_TRUE(ret->get_accepted()) << "failed to replicate: entry 1" << ret->get_result_code(); + EXPECT_EQ(ret->get_result_code(), nuraft::cmd_result_code::OK) << "failed to replicate: entry 1" << ret->get_result_code(); while (s1.state_machine->getValue() != 1) { @@ -176,10 +185,52 @@ TEST(CoordinationTest, TestSummingRaft) std::this_thread::sleep_for(std::chrono::milliseconds(100)); } + while (s3.state_machine->getValue() != 1) + { + std::cout << "Waiting s3 to apply entry\n"; + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + EXPECT_EQ(s1.state_machine->getValue(), 1); EXPECT_EQ(s2.state_machine->getValue(), 1); EXPECT_EQ(s3.state_machine->getValue(), 1); + auto non_leader_entry = getLogEntry(3); + auto ret_non_leader1 = s1.raft_instance->append_entries({non_leader_entry}); + + EXPECT_FALSE(ret_non_leader1->get_accepted()); + + auto ret_non_leader3 = s3.raft_instance->append_entries({non_leader_entry}); + + EXPECT_FALSE(ret_non_leader3->get_accepted()); + + auto leader_entry = getLogEntry(77); + auto ret_leader = s2.raft_instance->append_entries({leader_entry}); + EXPECT_TRUE(ret_leader->get_accepted()) << "failed to replicate: entry 78" << ret_leader->get_result_code(); + EXPECT_EQ(ret_leader->get_result_code(), nuraft::cmd_result_code::OK) << "failed to replicate: entry 78" << ret_leader->get_result_code(); + + while (s1.state_machine->getValue() != 78) + { + std::cout << "Waiting s1 to apply entry\n"; + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + + while (s2.state_machine->getValue() != 78) + { + std::cout << "Waiting s2 to apply entry\n"; + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + + while (s3.state_machine->getValue() != 78) + { + std::cout << "Waiting s3 to apply entry\n"; + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + + EXPECT_EQ(s1.state_machine->getValue(), 78); + EXPECT_EQ(s2.state_machine->getValue(), 78); + EXPECT_EQ(s3.state_machine->getValue(), 78); + s1.launcher.shutdown(5); s2.launcher.shutdown(5); s3.launcher.shutdown(5); From 1cc5be3b68d725919d812756f47f880316f26c69 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 14 Jan 2021 23:43:52 +0300 Subject: [PATCH 04/86] Compileable code --- src/Coordination/LoggerWrapper.h | 5 +++-- src/Coordination/tests/gtest_for_build.cpp | 4 ++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Coordination/LoggerWrapper.h b/src/Coordination/LoggerWrapper.h index 51718eaee8b..37de7806e9d 100644 --- a/src/Coordination/LoggerWrapper.h +++ b/src/Coordination/LoggerWrapper.h @@ -8,6 +8,7 @@ namespace DB class LoggerWrapper : public nuraft::logger { +public: LoggerWrapper(const std::string & name) : log(&Poco::Logger::get(name)) {} @@ -19,7 +20,7 @@ class LoggerWrapper : public nuraft::logger size_t /* line_number */, const std::string & msg) override { - LOG_IMPL(log, level, level, msg); + LOG_IMPL(log, static_cast(level), static_cast(level), msg); } void set_level(int level) override @@ -33,7 +34,7 @@ class LoggerWrapper : public nuraft::logger return log->getLevel(); } -pivate: +private: Poco::Logger * log; }; diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index 5785c9adb27..c13c5799ff7 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -46,7 +46,7 @@ struct SummingRaftServer params.return_method_ = nuraft::raft_params::blocking; raft_instance = launcher.init( - state_machine, state_manager, nuraft::cs_new(), port, + state_machine, state_manager, nuraft::cs_new("ToyRaftLogger"), port, nuraft::asio_service::options{}, params); if (!raft_instance) @@ -111,7 +111,7 @@ TEST(CoordinationTest, TestSummingRaft1) EXPECT_EQ(s1.raft_instance->get_leader(), 1); auto entry1 = getLogEntry(143); - auto ret = s1.raft_instance->append_entries({entry}); + auto ret = s1.raft_instance->append_entries({entry1}); EXPECT_TRUE(ret->get_accepted()) << "failed to replicate: entry 1" << ret->get_result_code(); EXPECT_EQ(ret->get_result_code(), nuraft::cmd_result_code::OK) << "failed to replicate: entry 1" << ret->get_result_code(); From 689655842419acf79351d7f79b960e48a4c3af7c Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 18 Jan 2021 19:03:26 +0300 Subject: [PATCH 05/86] Some code movements --- .../ZooKeeper => Coordination}/TestKeeperStorage.cpp | 2 +- src/{Common/ZooKeeper => Coordination}/TestKeeperStorage.h | 0 src/Coordination/tests/gtest_for_build.cpp | 6 ------ src/Interpreters/Context.cpp | 2 +- src/Server/TestKeeperTCPHandler.h | 2 +- 5 files changed, 3 insertions(+), 9 deletions(-) rename src/{Common/ZooKeeper => Coordination}/TestKeeperStorage.cpp (99%) rename src/{Common/ZooKeeper => Coordination}/TestKeeperStorage.h (100%) diff --git a/src/Common/ZooKeeper/TestKeeperStorage.cpp b/src/Coordination/TestKeeperStorage.cpp similarity index 99% rename from src/Common/ZooKeeper/TestKeeperStorage.cpp rename to src/Coordination/TestKeeperStorage.cpp index daadba6519e..00ce884ae7f 100644 --- a/src/Common/ZooKeeper/TestKeeperStorage.cpp +++ b/src/Coordination/TestKeeperStorage.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/src/Common/ZooKeeper/TestKeeperStorage.h b/src/Coordination/TestKeeperStorage.h similarity index 100% rename from src/Common/ZooKeeper/TestKeeperStorage.h rename to src/Coordination/TestKeeperStorage.h diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index c13c5799ff7..188565de4ce 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -215,12 +215,6 @@ TEST(CoordinationTest, TestSummingRaft3) std::this_thread::sleep_for(std::chrono::milliseconds(100)); } - while (s2.state_machine->getValue() != 78) - { - std::cout << "Waiting s2 to apply entry\n"; - std::this_thread::sleep_for(std::chrono::milliseconds(100)); - } - while (s3.state_machine->getValue() != 78) { std::cout << "Waiting s3 to apply entry\n"; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 2a8fdce869b..d1fdcd2955b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Server/TestKeeperTCPHandler.h b/src/Server/TestKeeperTCPHandler.h index 14e38ae6bd5..03d5ba40ab4 100644 --- a/src/Server/TestKeeperTCPHandler.h +++ b/src/Server/TestKeeperTCPHandler.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include From 1063b22b4c62b498d232f8acc10017663debdf21 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 19 Jan 2021 12:40:25 +0300 Subject: [PATCH 06/86] Add write buffer from nuraft --- src/Coordination/ReadBufferFromNuraftBuffer.h | 17 +++++ .../WriteBufferFromNuraftBuffer.cpp | 66 +++++++++++++++++++ .../WriteBufferFromNuraftBuffer.h | 30 +++++++++ src/Coordination/tests/gtest_for_build.cpp | 37 +++++++++++ 4 files changed, 150 insertions(+) create mode 100644 src/Coordination/ReadBufferFromNuraftBuffer.h create mode 100644 src/Coordination/WriteBufferFromNuraftBuffer.cpp create mode 100644 src/Coordination/WriteBufferFromNuraftBuffer.h diff --git a/src/Coordination/ReadBufferFromNuraftBuffer.h b/src/Coordination/ReadBufferFromNuraftBuffer.h new file mode 100644 index 00000000000..392a97bdd8f --- /dev/null +++ b/src/Coordination/ReadBufferFromNuraftBuffer.h @@ -0,0 +1,17 @@ +#pragma once +#include + +#include + +namespace DB +{ + +class ReadBufferFromNuraftBuffer : public ReadBufferFromMemory +{ +public: + explicit ReadBufferFromNuraftBuffer(nuraft::ptr buffer) + : ReadBufferFromMemory(buffer->data_begin(), buffer->size()) + {} +}; + +} diff --git a/src/Coordination/WriteBufferFromNuraftBuffer.cpp b/src/Coordination/WriteBufferFromNuraftBuffer.cpp new file mode 100644 index 00000000000..09e1034ae8f --- /dev/null +++ b/src/Coordination/WriteBufferFromNuraftBuffer.cpp @@ -0,0 +1,66 @@ +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_WRITE_AFTER_END_OF_BUFFER; +} + +void WriteBufferFromNuraftBuffer::nextImpl() +{ + if (is_finished) + throw Exception("WriteBufferFromNuraftBuffer is finished", ErrorCodes::CANNOT_WRITE_AFTER_END_OF_BUFFER); + + size_t old_size = buffer->size(); + /// pos may not be equal to vector.data() + old_size, because WriteBuffer::next() can be used to flush data + size_t pos_offset = pos - reinterpret_cast(buffer->data_begin()); + nuraft::ptr new_buffer = nuraft::buffer::alloc(old_size * size_multiplier); + memcpy(new_buffer->data_begin(), buffer->data_begin(), buffer->size()); + buffer = new_buffer; + internal_buffer = Buffer(reinterpret_cast(buffer->data_begin() + pos_offset), reinterpret_cast(buffer->data_begin() + buffer->size())); + working_buffer = internal_buffer; +} + +WriteBufferFromNuraftBuffer::WriteBufferFromNuraftBuffer() + : WriteBuffer(nullptr, 0) +{ + buffer = nuraft::buffer::alloc(initial_size); + set(reinterpret_cast(buffer->data_begin()), buffer->size()); +} + +void WriteBufferFromNuraftBuffer::finalize() +{ + if (is_finished) + return; + + is_finished = true; + size_t real_size = position() - reinterpret_cast(buffer->data_begin()); + nuraft::ptr new_buffer = nuraft::buffer::alloc(real_size); + memcpy(new_buffer->data_begin(), buffer->data_begin(), real_size); + buffer = new_buffer; + + /// Prevent further writes. + set(nullptr, 0); +} + +nuraft::ptr WriteBufferFromNuraftBuffer::getBuffer() +{ + finalize(); + return buffer; +} + + WriteBufferFromNuraftBuffer::~WriteBufferFromNuraftBuffer() +{ + try + { + finalize(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + +} diff --git a/src/Coordination/WriteBufferFromNuraftBuffer.h b/src/Coordination/WriteBufferFromNuraftBuffer.h new file mode 100644 index 00000000000..47a01fbc2a4 --- /dev/null +++ b/src/Coordination/WriteBufferFromNuraftBuffer.h @@ -0,0 +1,30 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class WriteBufferFromNuraftBuffer : public WriteBuffer +{ +private: + nuraft::ptr buffer; + bool is_finished = false; + + static constexpr size_t initial_size = 32; + static constexpr size_t size_multiplier = 2; + + void nextImpl() override; + +public: + WriteBufferFromNuraftBuffer(); + + void finalize() override final; + nuraft::ptr getBuffer(); + bool isFinished() const { return is_finished; } + + ~WriteBufferFromNuraftBuffer() override; +}; + +} diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index 188565de4ce..38602e48fae 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -4,6 +4,10 @@ #include #include #include +#include +#include +#include +#include #include #include #include @@ -26,6 +30,39 @@ TEST(CoordinationTest, BuildTest) EXPECT_EQ(1, 1); } +TEST(CoordinationTest, BufferSerde) +{ + Coordination::ZooKeeperRequestPtr request = Coordination::ZooKeeperRequestFactory::instance().get(Coordination::OpNum::Get); + request->xid = 3; + dynamic_cast(request.get())->path = "/path/value"; + + DB::WriteBufferFromNuraftBuffer wbuf; + request->write(wbuf); + auto nuraft_buffer = wbuf.getBuffer(); + EXPECT_EQ(nuraft_buffer->size(), 28); + + DB::ReadBufferFromNuraftBuffer rbuf(nuraft_buffer); + + int32_t length; + Coordination::read(length, rbuf); + EXPECT_EQ(length + sizeof(length), nuraft_buffer->size()); + + int32_t xid; + Coordination::read(xid, rbuf); + EXPECT_EQ(xid, request->xid); + + Coordination::OpNum opnum; + Coordination::read(opnum, rbuf); + + Coordination::ZooKeeperRequestPtr request_read = Coordination::ZooKeeperRequestFactory::instance().get(opnum); + request_read->xid = xid; + request_read->readImpl(rbuf); + + EXPECT_EQ(request_read->getOpNum(), Coordination::OpNum::Get); + EXPECT_EQ(request_read->xid, 3); + EXPECT_EQ(dynamic_cast(request_read.get())->path, "/path/value"); +} + struct SummingRaftServer { SummingRaftServer(int server_id_, const std::string & hostname_, int port_) From d5a3adffbd5159845dd522c1d3df2070e6a840e4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 20 Jan 2021 19:25:30 +0300 Subject: [PATCH 07/86] Replicate something in test keeper storage with raft --- src/Common/ya.make | 2 - src/Coordination/InMemoryLogStore.cpp | 3 +- src/Coordination/ReadBufferFromNuraftBuffer.h | 3 + src/Coordination/SummingStateMachine.cpp | 6 +- .../TestKeeperStorageDispatcher.cpp | 2 +- .../TestKeeperStorageDispatcher.h | 2 +- .../WriteBufferFromNuraftBuffer.cpp | 2 +- src/Coordination/tests/gtest_for_build.cpp | 142 ++++++++++++++++-- 8 files changed, 139 insertions(+), 23 deletions(-) rename src/{Common/ZooKeeper => Coordination}/TestKeeperStorageDispatcher.cpp (98%) rename src/{Common/ZooKeeper => Coordination}/TestKeeperStorageDispatcher.h (96%) diff --git a/src/Common/ya.make b/src/Common/ya.make index 4f2f1892a88..a17b57ebb04 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -84,8 +84,6 @@ SRCS( WeakHash.cpp ZooKeeper/IKeeper.cpp ZooKeeper/TestKeeper.cpp - ZooKeeper/TestKeeperStorage.cpp - ZooKeeper/TestKeeperStorageDispatcher.cpp ZooKeeper/ZooKeeper.cpp ZooKeeper/ZooKeeperCommon.cpp ZooKeeper/ZooKeeperConstants.cpp diff --git a/src/Coordination/InMemoryLogStore.cpp b/src/Coordination/InMemoryLogStore.cpp index 9f8d398a110..b9e2e502fc7 100644 --- a/src/Coordination/InMemoryLogStore.cpp +++ b/src/Coordination/InMemoryLogStore.cpp @@ -6,7 +6,8 @@ namespace DB namespace { using namespace nuraft; -ptr makeClone(const ptr & entry) { +ptr makeClone(const ptr & entry) +{ ptr clone = cs_new(entry->get_term(), buffer::clone(entry->get_buf()), entry->get_val_type()); return clone; } diff --git a/src/Coordination/ReadBufferFromNuraftBuffer.h b/src/Coordination/ReadBufferFromNuraftBuffer.h index 392a97bdd8f..cc01d3c8f39 100644 --- a/src/Coordination/ReadBufferFromNuraftBuffer.h +++ b/src/Coordination/ReadBufferFromNuraftBuffer.h @@ -12,6 +12,9 @@ public: explicit ReadBufferFromNuraftBuffer(nuraft::ptr buffer) : ReadBufferFromMemory(buffer->data_begin(), buffer->size()) {} + explicit ReadBufferFromNuraftBuffer(nuraft::buffer & buffer) + : ReadBufferFromMemory(buffer.data_begin(), buffer.size()) + {} }; } diff --git a/src/Coordination/SummingStateMachine.cpp b/src/Coordination/SummingStateMachine.cpp index 16154ca8cd4..bf2a5bb818f 100644 --- a/src/Coordination/SummingStateMachine.cpp +++ b/src/Coordination/SummingStateMachine.cpp @@ -49,7 +49,8 @@ nuraft::ptr SummingStateMachine::last_snapshot() // Just return the latest snapshot. std::lock_guard ll(snapshots_lock); auto entry = snapshots.rbegin(); - if (entry == snapshots.rend()) return nullptr; + if (entry == snapshots.rend()) + return nullptr; auto ctx = entry->second; return ctx->snapshot; @@ -117,7 +118,8 @@ int SummingStateMachine::read_logical_snp_obj( { std::lock_guard ll(snapshots_lock); auto entry = snapshots.find(s.get_last_log_idx()); - if (entry == snapshots.end()) { + if (entry == snapshots.end()) + { // Snapshot doesn't exist. data_out = nullptr; is_last_obj = true; diff --git a/src/Common/ZooKeeper/TestKeeperStorageDispatcher.cpp b/src/Coordination/TestKeeperStorageDispatcher.cpp similarity index 98% rename from src/Common/ZooKeeper/TestKeeperStorageDispatcher.cpp rename to src/Coordination/TestKeeperStorageDispatcher.cpp index b1233fc47e3..1700fa76092 100644 --- a/src/Common/ZooKeeper/TestKeeperStorageDispatcher.cpp +++ b/src/Coordination/TestKeeperStorageDispatcher.cpp @@ -1,4 +1,4 @@ -#include +#include #include namespace DB diff --git a/src/Common/ZooKeeper/TestKeeperStorageDispatcher.h b/src/Coordination/TestKeeperStorageDispatcher.h similarity index 96% rename from src/Common/ZooKeeper/TestKeeperStorageDispatcher.h rename to src/Coordination/TestKeeperStorageDispatcher.h index 27abf17ac73..f8cb06c3ced 100644 --- a/src/Common/ZooKeeper/TestKeeperStorageDispatcher.h +++ b/src/Coordination/TestKeeperStorageDispatcher.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include namespace zkutil diff --git a/src/Coordination/WriteBufferFromNuraftBuffer.cpp b/src/Coordination/WriteBufferFromNuraftBuffer.cpp index 09e1034ae8f..7d0a1dbcbb1 100644 --- a/src/Coordination/WriteBufferFromNuraftBuffer.cpp +++ b/src/Coordination/WriteBufferFromNuraftBuffer.cpp @@ -51,7 +51,7 @@ nuraft::ptr WriteBufferFromNuraftBuffer::getBuffer() return buffer; } - WriteBufferFromNuraftBuffer::~WriteBufferFromNuraftBuffer() +WriteBufferFromNuraftBuffer::~WriteBufferFromNuraftBuffer() { try { diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index 38602e48fae..fa330903ae2 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -12,15 +13,6 @@ #include #include -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -} TEST(CoordinationTest, BuildTest) { @@ -63,14 +55,15 @@ TEST(CoordinationTest, BufferSerde) EXPECT_EQ(dynamic_cast(request_read.get())->path, "/path/value"); } -struct SummingRaftServer +template +struct SimpliestRaftServer { - SummingRaftServer(int server_id_, const std::string & hostname_, int port_) + SimpliestRaftServer(int server_id_, const std::string & hostname_, int port_) : server_id(server_id_) , hostname(hostname_) , port(port_) , endpoint(hostname + ":" + std::to_string(port)) - , state_machine(nuraft::cs_new()) + , state_machine(nuraft::cs_new()) , state_manager(nuraft::cs_new(server_id, endpoint)) { nuraft::raft_params params; @@ -118,7 +111,7 @@ struct SummingRaftServer std::string endpoint; // State machine. - nuraft::ptr state_machine; + nuraft::ptr state_machine; // State manager. nuraft::ptr state_manager; @@ -130,6 +123,8 @@ struct SummingRaftServer nuraft::ptr raft_instance; }; +using SummingRaftServer = SimpliestRaftServer; + nuraft::ptr getLogEntry(int64_t number) { nuraft::ptr ret = nuraft::buffer::alloc(sizeof(number)); @@ -178,7 +173,7 @@ TEST(CoordinationTest, TestSummingRaft3) EXPECT_TRUE(false); } - while(s1.raft_instance->get_leader() != 2) + while (s1.raft_instance->get_leader() != 2) { std::cout << "Waiting s1 to join to s2 quorum\n"; std::this_thread::sleep_for(std::chrono::milliseconds(100)); @@ -193,7 +188,7 @@ TEST(CoordinationTest, TestSummingRaft3) EXPECT_TRUE(false); } - while(s3.raft_instance->get_leader() != 2) + while (s3.raft_instance->get_leader() != 2) { std::cout << "Waiting s3 to join to s2 quorum\n"; std::this_thread::sleep_for(std::chrono::milliseconds(100)); @@ -266,3 +261,120 @@ TEST(CoordinationTest, TestSummingRaft3) s2.launcher.shutdown(5); s3.launcher.shutdown(5); } + +using NuKeeperRaftServer = SimpliestRaftServer; + + +nuraft::ptr getZooKeeperLogEntry(int64_t session_id, const Coordination::ZooKeeperRequestPtr & request) +{ + DB::WriteBufferFromNuraftBuffer buf; + DB::writeIntBinary(session_id, buf); + request->write(buf); + return buf.getBuffer(); +} + +zkutil::TestKeeperStorage::ResponsesForSessions getZooKeeperResponses(nuraft::ptr & buffer, const Coordination::ZooKeeperRequestPtr & request) +{ + zkutil::TestKeeperStorage::ResponsesForSessions results; + DB::ReadBufferFromNuraftBuffer buf(buffer); + while (!buf.eof()) + { + int64_t session_id; + DB::readIntBinary(session_id, buf); + + int32_t length; + Coordination::XID xid; + int64_t zxid; + Coordination::Error err; + + Coordination::read(length, buf); + Coordination::read(xid, buf); + Coordination::read(zxid, buf); + Coordination::read(err, buf); + auto response = request->makeResponse(); + response->readImpl(buf); + results.push_back(zkutil::TestKeeperStorage::ResponseForSession{session_id, response}); + } + return results; +} + +TEST(CoordinationTest, TestNuKeeperRaft) +{ + NuKeeperRaftServer s1(1, "localhost", 44447); + NuKeeperRaftServer s2(2, "localhost", 44448); + NuKeeperRaftServer s3(3, "localhost", 44449); + + nuraft::srv_config first_config(1, "localhost:44447"); + auto ret1 = s2.raft_instance->add_srv(first_config); + + EXPECT_TRUE(ret1->get_accepted()) << "failed to add server: " << ret1->get_result_str() << std::endl; + + while (s1.raft_instance->get_leader() != 2) + { + std::cout << "Waiting s1 to join to s2 quorum\n"; + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + + nuraft::srv_config third_config(3, "localhost:44449"); + auto ret3 = s2.raft_instance->add_srv(third_config); + + EXPECT_TRUE(ret3->get_accepted()) << "failed to add server: " << ret3->get_result_str() << std::endl; + + while (s3.raft_instance->get_leader() != 2) + { + std::cout << "Waiting s3 to join to s2 quorum\n"; + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + + /// S2 is leader + EXPECT_EQ(s1.raft_instance->get_leader(), 2); + EXPECT_EQ(s2.raft_instance->get_leader(), 2); + EXPECT_EQ(s3.raft_instance->get_leader(), 2); + + int64_t session_id = 34; + std::shared_ptr create_request = std::make_shared(); + create_request->path = "/hello"; + create_request->data = "world"; + + auto entry1 = getZooKeeperLogEntry(session_id, create_request); + auto ret_leader = s2.raft_instance->append_entries({entry1}); + + EXPECT_TRUE(ret_leader->get_accepted()) << "failed to replicate create entry:" << ret_leader->get_result_code(); + EXPECT_EQ(ret_leader->get_result_code(), nuraft::cmd_result_code::OK) << "failed to replicate create entry:" << ret_leader->get_result_code(); + + auto result = ret_leader.get(); + + auto responses = getZooKeeperResponses(result->get(), create_request); + + EXPECT_EQ(responses.size(), 1); + EXPECT_EQ(responses[0].session_id, 34); + EXPECT_EQ(responses[0].response->getOpNum(), Coordination::OpNum::Create); + EXPECT_EQ(dynamic_cast(responses[0].response.get())->path_created, "/hello"); + + + while (s1.state_machine->getStorage().container.count("/hello") == 0) + { + std::cout << "Waiting s1 to apply entry\n"; + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + + while (s2.state_machine->getStorage().container.count("/hello") == 0) + { + std::cout << "Waiting s2 to apply entry\n"; + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + + while (s3.state_machine->getStorage().container.count("/hello") == 0) + { + std::cout << "Waiting s3 to apply entry\n"; + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + + EXPECT_EQ(s1.state_machine->getStorage().container["/hello"].data, "world"); + EXPECT_EQ(s2.state_machine->getStorage().container["/hello"].data, "world"); + EXPECT_EQ(s3.state_machine->getStorage().container["/hello"].data, "world"); + + s1.launcher.shutdown(5); + s2.launcher.shutdown(5); + s3.launcher.shutdown(5); +} From f7175819d57df8185e05fddd28435fb1abb4e56c Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 21 Jan 2021 14:07:55 +0300 Subject: [PATCH 08/86] Add storage simpliest serialization --- src/Common/ZooKeeper/ZooKeeperIO.cpp | 13 ++ src/Common/ZooKeeper/ZooKeeperIO.h | 2 + src/Coordination/NuKeeperStateMachine.cpp | 190 ++++++++++++++++++ src/Coordination/NuKeeperStateMachine.h | 63 ++++++ src/Coordination/TestKeeperStorage.cpp | 5 +- .../TestKeeperStorageSerializer.cpp | 87 ++++++++ .../TestKeeperStorageSerializer.h | 17 ++ src/Coordination/tests/gtest_for_build.cpp | 18 +- 8 files changed, 391 insertions(+), 4 deletions(-) create mode 100644 src/Coordination/NuKeeperStateMachine.cpp create mode 100644 src/Coordination/NuKeeperStateMachine.h create mode 100644 src/Coordination/TestKeeperStorageSerializer.cpp create mode 100644 src/Coordination/TestKeeperStorageSerializer.h diff --git a/src/Common/ZooKeeper/ZooKeeperIO.cpp b/src/Common/ZooKeeper/ZooKeeperIO.cpp index a0e4161f111..3f0905ea186 100644 --- a/src/Common/ZooKeeper/ZooKeeperIO.cpp +++ b/src/Common/ZooKeeper/ZooKeeperIO.cpp @@ -3,6 +3,13 @@ namespace Coordination { + +void write(size_t x, WriteBuffer & out) +{ + x = __builtin_bswap64(x); + writeBinary(x, out); +} + void write(int64_t x, WriteBuffer & out) { x = __builtin_bswap64(x); @@ -57,6 +64,12 @@ void write(const Error & x, WriteBuffer & out) write(static_cast(x), out); } +void read(size_t & x, ReadBuffer & in) +{ + readBinary(x, in); + x = __builtin_bswap64(x); +} + void read(int64_t & x, ReadBuffer & in) { readBinary(x, in); diff --git a/src/Common/ZooKeeper/ZooKeeperIO.h b/src/Common/ZooKeeper/ZooKeeperIO.h index edeb995f27b..fd47e324664 100644 --- a/src/Common/ZooKeeper/ZooKeeperIO.h +++ b/src/Common/ZooKeeper/ZooKeeperIO.h @@ -13,6 +13,7 @@ namespace Coordination using namespace DB; +void write(size_t x, WriteBuffer & out); void write(int64_t x, WriteBuffer & out); void write(int32_t x, WriteBuffer & out); void write(OpNum x, WriteBuffer & out); @@ -37,6 +38,7 @@ void write(const std::vector & arr, WriteBuffer & out) write(elem, out); } +void read(size_t & x, ReadBuffer & in); void read(int64_t & x, ReadBuffer & in); void read(int32_t & x, ReadBuffer & in); void read(OpNum & x, ReadBuffer & in); diff --git a/src/Coordination/NuKeeperStateMachine.cpp b/src/Coordination/NuKeeperStateMachine.cpp new file mode 100644 index 00000000000..59830040e66 --- /dev/null +++ b/src/Coordination/NuKeeperStateMachine.cpp @@ -0,0 +1,190 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +zkutil::TestKeeperStorage::RequestForSession parseRequest(nuraft::buffer & data) +{ + ReadBufferFromNuraftBuffer buffer(data); + zkutil::TestKeeperStorage::RequestForSession request_for_session; + readIntBinary(request_for_session.session_id, buffer); + + int32_t length; + Coordination::read(length, buffer); + + int32_t xid; + Coordination::read(xid, buffer); + + Coordination::OpNum opnum; + Coordination::read(opnum, buffer); + + request_for_session.request = Coordination::ZooKeeperRequestFactory::instance().get(opnum); + request_for_session.request->xid = xid; + request_for_session.request->readImpl(buffer); + return request_for_session; +} + +nuraft::ptr writeResponses(zkutil::TestKeeperStorage::ResponsesForSessions & responses) +{ + WriteBufferFromNuraftBuffer buffer; + for (const auto & response_and_session : responses) + { + writeIntBinary(response_and_session.session_id, buffer); + response_and_session.response->write(buffer); + } + return buffer.getBuffer(); +} + + +NuKeeperStateMachine::NuKeeperStateMachine() + : last_committed_idx(0) + , log(&Poco::Logger::get("NuRaftStateMachine")) +{ + LOG_DEBUG(log, "Created nukeeper state machine"); +} + +nuraft::ptr NuKeeperStateMachine::commit(const size_t log_idx, nuraft::buffer & data) +{ + LOG_DEBUG(log, "Commiting logidx {}", log_idx); + auto request_for_session = parseRequest(data); + auto responses_with_sessions = storage.processRequest(request_for_session.request, request_for_session.session_id); + + last_committed_idx = log_idx; + return writeResponses(responses_with_sessions); +} + +bool NuKeeperStateMachine::apply_snapshot(nuraft::snapshot & s) +{ + LOG_DEBUG(log, "Applying snapshot {}", s.get_last_log_idx()); + std::lock_guard lock(snapshots_lock); + auto entry = snapshots.find(s.get_last_log_idx()); + if (entry == snapshots.end()) + { + return false; + } + + /// TODO + return true; +} + +nuraft::ptr NuKeeperStateMachine::last_snapshot() +{ + + LOG_DEBUG(log, "Trying to get last snapshot"); + // Just return the latest snapshot. + std::lock_guard lock(snapshots_lock); + auto entry = snapshots.rbegin(); + if (entry == snapshots.rend()) + return nullptr; + + return entry->second; +} + +void NuKeeperStateMachine::create_snapshot( + nuraft::snapshot & s, + nuraft::async_result::handler_type & when_done) +{ + + LOG_DEBUG(log, "Creating snapshot {}", s.get_last_log_idx()); + { + std::lock_guard lock(snapshots_lock); + nuraft::ptr snp_buf = s.serialize(); + nuraft::ptr ss = nuraft::snapshot::deserialize(*snp_buf); + snapshots[s.get_last_log_idx()] = ss; + const int MAX_SNAPSHOTS = 3; + int num = snapshots.size(); + auto entry = snapshots.begin(); + + for (int i = 0; i < num - MAX_SNAPSHOTS; ++i) + { + if (entry == snapshots.end()) + break; + entry = snapshots.erase(entry); + } + } + nuraft::ptr except(nullptr); + bool ret = true; + when_done(ret, except); +} + +void NuKeeperStateMachine::save_logical_snp_obj( + nuraft::snapshot & s, + size_t & obj_id, + nuraft::buffer & /*data*/, + bool /*is_first_obj*/, + bool /*is_last_obj*/) +{ + LOG_DEBUG(log, "Saving snapshot {} obj_id {}", s.get_last_log_idx(), obj_id); + if (obj_id == 0) + { + std::lock_guard lock(snapshots_lock); + nuraft::ptr snp_buf = s.serialize(); + nuraft::ptr ss = nuraft::snapshot::deserialize(*snp_buf); + snapshots[s.get_last_log_idx()] = ss; + const int MAX_SNAPSHOTS = 3; + int num = snapshots.size(); + auto entry = snapshots.begin(); + + for (int i = 0; i < num - MAX_SNAPSHOTS; ++i) + { + if (entry == snapshots.end()) + break; + entry = snapshots.erase(entry); + } + } + else + { + std::lock_guard lock(snapshots_lock); + auto entry = snapshots.find(s.get_last_log_idx()); + assert(entry != snapshots.end()); + } + + obj_id++; +} + +int NuKeeperStateMachine::read_logical_snp_obj( + nuraft::snapshot & s, + void* & /*user_snp_ctx*/, + ulong obj_id, + nuraft::ptr & data_out, + bool & is_last_obj) +{ + + LOG_DEBUG(log, "Reading snapshot {} obj_id {}", s.get_last_log_idx(), obj_id); + { + std::lock_guard ll(snapshots_lock); + auto entry = snapshots.find(s.get_last_log_idx()); + if (entry == snapshots.end()) + { + // Snapshot doesn't exist. + data_out = nullptr; + is_last_obj = true; + return 0; + } + } + + if (obj_id == 0) + { + // Object ID == 0: first object, put dummy data. + data_out = nuraft::buffer::alloc(sizeof(size_t)); + nuraft::buffer_serializer bs(data_out); + bs.put_i32(0); + is_last_obj = false; + + } + else + { + // Object ID > 0: second object, put actual value. + data_out = nuraft::buffer::alloc(sizeof(size_t)); + nuraft::buffer_serializer bs(data_out); + bs.put_u64(1); + is_last_obj = true; + } + return 0; +} + +} diff --git a/src/Coordination/NuKeeperStateMachine.h b/src/Coordination/NuKeeperStateMachine.h new file mode 100644 index 00000000000..42b90116a9b --- /dev/null +++ b/src/Coordination/NuKeeperStateMachine.h @@ -0,0 +1,63 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class NuKeeperStateMachine : public nuraft::state_machine +{ +public: + NuKeeperStateMachine(); + + nuraft::ptr pre_commit(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override { return nullptr; } + + nuraft::ptr commit(const size_t log_idx, nuraft::buffer & data) override; + + void rollback(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override {} + + size_t last_commit_index() override { return last_committed_idx; } + + bool apply_snapshot(nuraft::snapshot & s) override; + + nuraft::ptr last_snapshot() override; + + void create_snapshot( + nuraft::snapshot & s, + nuraft::async_result::handler_type & when_done) override; + + void save_logical_snp_obj( + nuraft::snapshot & s, + size_t & obj_id, + nuraft::buffer & data, + bool is_first_obj, + bool is_last_obj) override; + + int read_logical_snp_obj( + nuraft::snapshot & s, + void* & user_snp_ctx, + ulong obj_id, + nuraft::ptr & data_out, + bool & is_last_obj) override; + + zkutil::TestKeeperStorage & getStorage() + { + return storage; + } + +private: + zkutil::TestKeeperStorage storage; + // Mutex for `snapshots_`. + std::mutex snapshots_lock; + + /// Fake snapshot storage + std::map> snapshots; + + /// Last committed Raft log number. + std::atomic last_committed_idx; + Poco::Logger * log; +}; + +} diff --git a/src/Coordination/TestKeeperStorage.cpp b/src/Coordination/TestKeeperStorage.cpp index b5bf9facbf1..31dc4116dc8 100644 --- a/src/Coordination/TestKeeperStorage.cpp +++ b/src/Coordination/TestKeeperStorage.cpp @@ -46,7 +46,7 @@ static TestKeeperStorage::ResponsesForSessions processWatchesImpl(const String & { std::shared_ptr watch_response = std::make_shared(); watch_response->path = path; - watch_response->xid = -1; + watch_response->xid = Coordination::WATCH_XID; watch_response->zxid = -1; watch_response->type = event_type; watch_response->state = Coordination::State::CONNECTED; @@ -62,7 +62,7 @@ static TestKeeperStorage::ResponsesForSessions processWatchesImpl(const String & { std::shared_ptr watch_list_response = std::make_shared(); watch_list_response->path = parent_path; - watch_list_response->xid = -1; + watch_list_response->xid = Coordination::WATCH_XID; watch_list_response->zxid = -1; watch_list_response->type = Coordination::Event::CHILD; watch_list_response->state = Coordination::State::CONNECTED; @@ -103,7 +103,6 @@ struct TestKeeperStorageHeartbeatRequest final : public TestKeeperStorageRequest } }; - struct TestKeeperStorageCreateRequest final : public TestKeeperStorageRequest { using TestKeeperStorageRequest::TestKeeperStorageRequest; diff --git a/src/Coordination/TestKeeperStorageSerializer.cpp b/src/Coordination/TestKeeperStorageSerializer.cpp new file mode 100644 index 00000000000..bf7015374be --- /dev/null +++ b/src/Coordination/TestKeeperStorageSerializer.cpp @@ -0,0 +1,87 @@ +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ + void writeNode(const zkutil::TestKeeperStorage::Node & node, WriteBuffer & out) + { + Coordination::write(node.data, out); + Coordination::write(node.acls, out); + Coordination::write(node.is_ephemeral, out); + Coordination::write(node.is_sequental, out); + Coordination::write(node.stat, out); + Coordination::write(node.seq_num, out); + } + + void readNode(zkutil::TestKeeperStorage::Node & node, ReadBuffer & in) + { + Coordination::read(node.data, in); + Coordination::read(node.acls, in); + Coordination::read(node.is_ephemeral, in); + Coordination::read(node.is_sequental, in); + Coordination::read(node.stat, in); + Coordination::read(node.seq_num, in); + } +} + +void TestKeeperStorageSerializer::serialize(const zkutil::TestKeeperStorage & storage, WriteBuffer & out) const +{ + Coordination::write(storage.zxid, out); + Coordination::write(storage.session_id_counter, out); + Coordination::write(storage.container.size(), out); + for (const auto & [path, node] : storage.container) + { + Coordination::write(path, out); + writeNode(node, out); + } + Coordination::write(storage.ephemerals.size(), out); + for (const auto & [session_id, paths] : storage.ephemerals) + { + Coordination::write(session_id, out); + Coordination::write(paths.size(), out); + for (const auto & path : paths) + Coordination::write(path, out); + } +} + +void TestKeeperStorageSerializer::deserialize(zkutil::TestKeeperStorage & storage, ReadBuffer & in) const +{ + int64_t session_id_counter, zxid; + Coordination::read(zxid, in); + Coordination::read(session_id_counter, in); + storage.zxid = zxid; + storage.session_id_counter = session_id_counter; + + size_t container_size; + Coordination::read(container_size, in); + while (storage.container.size() < container_size) + { + std::string path; + Coordination::read(path, in); + zkutil::TestKeeperStorage::Node node; + readNode(node, in); + storage.container[path] = node; + } + size_t ephemerals_size; + Coordination::read(ephemerals_size, in); + while (storage.ephemerals.size() < ephemerals_size) + { + int64_t session_id; + size_t ephemerals_for_session; + Coordination::read(session_id, in); + Coordination::read(ephemerals_for_session, in); + while (storage.ephemerals[session_id].size() < ephemerals_for_session) + { + std::string ephemeral_path; + Coordination::read(ephemeral_path, in); + storage.ephemerals[session_id].emplace(ephemeral_path); + } + } +} + +} diff --git a/src/Coordination/TestKeeperStorageSerializer.h b/src/Coordination/TestKeeperStorageSerializer.h new file mode 100644 index 00000000000..b4453574cfd --- /dev/null +++ b/src/Coordination/TestKeeperStorageSerializer.h @@ -0,0 +1,17 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +class TestKeeperStorageSerializer +{ +public: + void serialize(const zkutil::TestKeeperStorage & storage, WriteBuffer & out) const; + + void deserialize(zkutil::TestKeeperStorage & storage, ReadBuffer & in) const; +}; + +} diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index fa330903ae2..635ac88f737 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -351,7 +351,6 @@ TEST(CoordinationTest, TestNuKeeperRaft) EXPECT_EQ(responses[0].response->getOpNum(), Coordination::OpNum::Create); EXPECT_EQ(dynamic_cast(responses[0].response.get())->path_created, "/hello"); - while (s1.state_machine->getStorage().container.count("/hello") == 0) { std::cout << "Waiting s1 to apply entry\n"; @@ -374,6 +373,23 @@ TEST(CoordinationTest, TestNuKeeperRaft) EXPECT_EQ(s2.state_machine->getStorage().container["/hello"].data, "world"); EXPECT_EQ(s3.state_machine->getStorage().container["/hello"].data, "world"); + std::shared_ptr get_request = std::make_shared(); + get_request->path = "/hello"; + auto entry2 = getZooKeeperLogEntry(session_id, get_request); + auto ret_leader_get = s2.raft_instance->append_entries({entry2}); + + EXPECT_TRUE(ret_leader_get->get_accepted()) << "failed to replicate create entry: " << ret_leader_get->get_result_code(); + EXPECT_EQ(ret_leader_get->get_result_code(), nuraft::cmd_result_code::OK) << "failed to replicate create entry: " << ret_leader_get->get_result_code(); + + auto result_get = ret_leader_get.get(); + + auto get_responses = getZooKeeperResponses(result_get->get(), get_request); + + EXPECT_EQ(get_responses.size(), 1); + EXPECT_EQ(get_responses[0].session_id, 34); + EXPECT_EQ(get_responses[0].response->getOpNum(), Coordination::OpNum::Get); + EXPECT_EQ(dynamic_cast(get_responses[0].response.get())->data, "world"); + s1.launcher.shutdown(5); s2.launcher.shutdown(5); s3.launcher.shutdown(5); From d6b8dd75252aa40c1392241be2af563103c8ef68 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 21 Jan 2021 16:53:10 +0300 Subject: [PATCH 09/86] Dumb snapshoting --- src/Coordination/NuKeeperStateMachine.cpp | 98 ++++++++++++------- src/Coordination/NuKeeperStateMachine.h | 26 ++++- src/Coordination/TestKeeperStorage.h | 13 +-- .../TestKeeperStorageDispatcher.h | 6 +- 4 files changed, 93 insertions(+), 50 deletions(-) diff --git a/src/Coordination/NuKeeperStateMachine.cpp b/src/Coordination/NuKeeperStateMachine.cpp index 59830040e66..c0deb403f20 100644 --- a/src/Coordination/NuKeeperStateMachine.cpp +++ b/src/Coordination/NuKeeperStateMachine.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -51,23 +52,30 @@ nuraft::ptr NuKeeperStateMachine::commit(const size_t log_idx, n { LOG_DEBUG(log, "Commiting logidx {}", log_idx); auto request_for_session = parseRequest(data); - auto responses_with_sessions = storage.processRequest(request_for_session.request, request_for_session.session_id); + zkutil::TestKeeperStorage::ResponsesForSessions responses_for_sessions; + { + std::lock_guard lock(storage_lock); + responses_for_sessions = storage.processRequest(request_for_session.request, request_for_session.session_id); + } last_committed_idx = log_idx; - return writeResponses(responses_with_sessions); + return writeResponses(responses_for_sessions); } bool NuKeeperStateMachine::apply_snapshot(nuraft::snapshot & s) { LOG_DEBUG(log, "Applying snapshot {}", s.get_last_log_idx()); - std::lock_guard lock(snapshots_lock); - auto entry = snapshots.find(s.get_last_log_idx()); - if (entry == snapshots.end()) + StorageSnapshotPtr snapshot; { - return false; + std::lock_guard lock(snapshots_lock); + auto entry = snapshots.find(s.get_last_log_idx()); + if (entry == snapshots.end()) + return false; + snapshot = entry->second; } - - /// TODO + std::lock_guard lock(storage_lock); + storage = snapshot->storage; + last_committed_idx = s.get_last_log_idx(); return true; } @@ -81,7 +89,37 @@ nuraft::ptr NuKeeperStateMachine::last_snapshot() if (entry == snapshots.rend()) return nullptr; - return entry->second; + return entry->second->snapshot; +} + +NuKeeperStateMachine::StorageSnapshotPtr NuKeeperStateMachine::createSnapshotInternal(nuraft::snapshot & s) +{ + nuraft::ptr snp_buf = s.serialize(); + nuraft::ptr ss = nuraft::snapshot::deserialize(*snp_buf); + std::lock_guard lock(storage_lock); + return std::make_shared(ss, storage); +} + +NuKeeperStateMachine::StorageSnapshotPtr NuKeeperStateMachine::readSnapshot(nuraft::snapshot & s, nuraft::buffer & in) const +{ + nuraft::ptr snp_buf = s.serialize(); + nuraft::ptr ss = nuraft::snapshot::deserialize(*snp_buf); + TestKeeperStorageSerializer serializer; + + ReadBufferFromNuraftBuffer reader(in); + zkutil::TestKeeperStorage new_storage; + serializer.deserialize(new_storage, reader); + return std::make_shared(ss, new_storage); +} + + +void NuKeeperStateMachine::writeSnapshot(const NuKeeperStateMachine::StorageSnapshotPtr & snapshot, nuraft::ptr & out) const +{ + TestKeeperStorageSerializer serializer; + + WriteBufferFromNuraftBuffer writer; + serializer.serialize(snapshot->storage, writer); + out = writer.getBuffer(); } void NuKeeperStateMachine::create_snapshot( @@ -90,11 +128,10 @@ void NuKeeperStateMachine::create_snapshot( { LOG_DEBUG(log, "Creating snapshot {}", s.get_last_log_idx()); + auto snapshot = createSnapshotInternal(s); { std::lock_guard lock(snapshots_lock); - nuraft::ptr snp_buf = s.serialize(); - nuraft::ptr ss = nuraft::snapshot::deserialize(*snp_buf); - snapshots[s.get_last_log_idx()] = ss; + snapshots[s.get_last_log_idx()] = snapshot; const int MAX_SNAPSHOTS = 3; int num = snapshots.size(); auto entry = snapshots.begin(); @@ -114,33 +151,22 @@ void NuKeeperStateMachine::create_snapshot( void NuKeeperStateMachine::save_logical_snp_obj( nuraft::snapshot & s, size_t & obj_id, - nuraft::buffer & /*data*/, + nuraft::buffer & data, bool /*is_first_obj*/, bool /*is_last_obj*/) { LOG_DEBUG(log, "Saving snapshot {} obj_id {}", s.get_last_log_idx(), obj_id); if (obj_id == 0) { + auto new_snapshot = createSnapshotInternal(s); std::lock_guard lock(snapshots_lock); - nuraft::ptr snp_buf = s.serialize(); - nuraft::ptr ss = nuraft::snapshot::deserialize(*snp_buf); - snapshots[s.get_last_log_idx()] = ss; - const int MAX_SNAPSHOTS = 3; - int num = snapshots.size(); - auto entry = snapshots.begin(); - - for (int i = 0; i < num - MAX_SNAPSHOTS; ++i) - { - if (entry == snapshots.end()) - break; - entry = snapshots.erase(entry); - } + snapshots.try_emplace(s.get_last_log_idx(), std::move(new_snapshot)); } else { + auto received_snapshot = readSnapshot(s, data); std::lock_guard lock(snapshots_lock); - auto entry = snapshots.find(s.get_last_log_idx()); - assert(entry != snapshots.end()); + snapshots.try_emplace(s.get_last_log_idx(), std::move(received_snapshot)); } obj_id++; @@ -155,8 +181,9 @@ int NuKeeperStateMachine::read_logical_snp_obj( { LOG_DEBUG(log, "Reading snapshot {} obj_id {}", s.get_last_log_idx(), obj_id); + StorageSnapshotPtr required_snapshot; { - std::lock_guard ll(snapshots_lock); + std::lock_guard lock(snapshots_lock); auto entry = snapshots.find(s.get_last_log_idx()); if (entry == snapshots.end()) { @@ -165,23 +192,18 @@ int NuKeeperStateMachine::read_logical_snp_obj( is_last_obj = true; return 0; } + required_snapshot = entry->second; } if (obj_id == 0) { - // Object ID == 0: first object, put dummy data. - data_out = nuraft::buffer::alloc(sizeof(size_t)); - nuraft::buffer_serializer bs(data_out); - bs.put_i32(0); + auto new_snapshot = createSnapshotInternal(s); + writeSnapshot(new_snapshot, data_out); is_last_obj = false; - } else { - // Object ID > 0: second object, put actual value. - data_out = nuraft::buffer::alloc(sizeof(size_t)); - nuraft::buffer_serializer bs(data_out); - bs.put_u64(1); + writeSnapshot(required_snapshot, data_out); is_last_obj = true; } return 0; diff --git a/src/Coordination/NuKeeperStateMachine.h b/src/Coordination/NuKeeperStateMachine.h index 42b90116a9b..c8dd9f8e570 100644 --- a/src/Coordination/NuKeeperStateMachine.h +++ b/src/Coordination/NuKeeperStateMachine.h @@ -48,12 +48,34 @@ public: } private: + struct StorageSnapshot + { + StorageSnapshot(const nuraft::ptr & s, const zkutil::TestKeeperStorage & storage_) + : snapshot(s) + , storage(storage_) + {} + + nuraft::ptr snapshot; + zkutil::TestKeeperStorage storage; + }; + + using StorageSnapshotPtr = std::shared_ptr; + + StorageSnapshotPtr createSnapshotInternal(nuraft::snapshot & s); + + StorageSnapshotPtr readSnapshot(nuraft::snapshot & s, nuraft::buffer & in) const; + + void writeSnapshot(const StorageSnapshotPtr & snapshot, nuraft::ptr & out) const; + zkutil::TestKeeperStorage storage; - // Mutex for `snapshots_`. + /// Mutex for snapshots std::mutex snapshots_lock; + /// Lock for storage + std::mutex storage_lock; + /// Fake snapshot storage - std::map> snapshots; + std::map snapshots; /// Last committed Raft log number. std::atomic last_committed_idx; diff --git a/src/Coordination/TestKeeperStorage.h b/src/Coordination/TestKeeperStorage.h index 21b1ce16c32..0bdec50625e 100644 --- a/src/Coordination/TestKeeperStorage.h +++ b/src/Coordination/TestKeeperStorage.h @@ -19,7 +19,7 @@ using ResponseCallback = std::function session_id_counter{0}; + int64_t session_id_counter{0}; struct Node { @@ -58,8 +58,8 @@ public: Ephemerals ephemerals; SessionAndWatcher sessions_and_watchers; - std::atomic zxid{0}; - std::atomic finalized{false}; + int64_t zxid{0}; + bool finalized{false}; Watches watches; Watches list_watches; /// Watches for 'list' request (watches on children). @@ -68,7 +68,7 @@ public: int64_t getZXID() { - return zxid.fetch_add(1); + return zxid++; } public: @@ -76,11 +76,6 @@ public: ResponsesForSessions processRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id); ResponsesForSessions finalize(const RequestsForSessions & expired_requests); - - int64_t getSessionID() - { - return session_id_counter.fetch_add(1); - } }; } diff --git a/src/Coordination/TestKeeperStorageDispatcher.h b/src/Coordination/TestKeeperStorageDispatcher.h index f6a81d4a88e..e460ba41f0a 100644 --- a/src/Coordination/TestKeeperStorageDispatcher.h +++ b/src/Coordination/TestKeeperStorageDispatcher.h @@ -13,6 +13,8 @@ using ZooKeeperResponseCallback = std::function session_id_counter{0}; Poco::Timespan operation_timeout{0, Coordination::DEFAULT_OPERATION_TIMEOUT_MS * 1000}; using clock = std::chrono::steady_clock; @@ -48,10 +50,12 @@ public: ~TestKeeperStorageDispatcher(); void putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id); + int64_t getSessionID() { - return storage.getSessionID(); + return session_id_counter.fetch_add(1); } + void registerSession(int64_t session_id, ZooKeeperResponseCallback callback); /// Call if we don't need any responses for this session no more (session was expired) void finishSession(int64_t session_id); From 61fe49194b933e5db1fc35050fa01a5d44b6b1b3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 21 Jan 2021 17:34:34 +0300 Subject: [PATCH 10/86] First working snapshots --- src/Coordination/NuKeeperStateMachine.cpp | 5 +- src/Coordination/TestKeeperStorage.h | 4 ++ .../TestKeeperStorageDispatcher.h | 6 +- src/Coordination/tests/gtest_for_build.cpp | 56 ++++++++++++++++++- 4 files changed, 65 insertions(+), 6 deletions(-) diff --git a/src/Coordination/NuKeeperStateMachine.cpp b/src/Coordination/NuKeeperStateMachine.cpp index c0deb403f20..02f3016be32 100644 --- a/src/Coordination/NuKeeperStateMachine.cpp +++ b/src/Coordination/NuKeeperStateMachine.cpp @@ -126,7 +126,6 @@ void NuKeeperStateMachine::create_snapshot( nuraft::snapshot & s, nuraft::async_result::handler_type & when_done) { - LOG_DEBUG(log, "Creating snapshot {}", s.get_last_log_idx()); auto snapshot = createSnapshotInternal(s); { @@ -156,6 +155,7 @@ void NuKeeperStateMachine::save_logical_snp_obj( bool /*is_last_obj*/) { LOG_DEBUG(log, "Saving snapshot {} obj_id {}", s.get_last_log_idx(), obj_id); + if (obj_id == 0) { auto new_snapshot = createSnapshotInternal(s); @@ -165,8 +165,9 @@ void NuKeeperStateMachine::save_logical_snp_obj( else { auto received_snapshot = readSnapshot(s, data); + std::lock_guard lock(snapshots_lock); - snapshots.try_emplace(s.get_last_log_idx(), std::move(received_snapshot)); + snapshots[s.get_last_log_idx()] = std::move(received_snapshot); } obj_id++; diff --git a/src/Coordination/TestKeeperStorage.h b/src/Coordination/TestKeeperStorage.h index 0bdec50625e..76111490c78 100644 --- a/src/Coordination/TestKeeperStorage.h +++ b/src/Coordination/TestKeeperStorage.h @@ -74,6 +74,10 @@ public: public: TestKeeperStorage(); + int64_t getSessionID() + { + return session_id_counter++; + } ResponsesForSessions processRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id); ResponsesForSessions finalize(const RequestsForSessions & expired_requests); }; diff --git a/src/Coordination/TestKeeperStorageDispatcher.h b/src/Coordination/TestKeeperStorageDispatcher.h index e460ba41f0a..df4ac2cf99d 100644 --- a/src/Coordination/TestKeeperStorageDispatcher.h +++ b/src/Coordination/TestKeeperStorageDispatcher.h @@ -13,8 +13,6 @@ using ZooKeeperResponseCallback = std::function session_id_counter{0}; Poco::Timespan operation_timeout{0, Coordination::DEFAULT_OPERATION_TIMEOUT_MS * 1000}; using clock = std::chrono::steady_clock; @@ -39,6 +37,7 @@ private: ThreadFromGlobalPool processing_thread; TestKeeperStorage storage; + std::mutex session_id_mutex; private: void processingThread(); @@ -53,7 +52,8 @@ public: int64_t getSessionID() { - return session_id_counter.fetch_add(1); + std::lock_guard lock(session_id_mutex); + return storage.getSessionID(); } void registerSession(int64_t session_id, ZooKeeperResponseCallback callback); diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index 635ac88f737..09c5db03514 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -2,11 +2,14 @@ #include #include +#include #include #include #include #include #include +#include +#include #include #include #include @@ -71,7 +74,7 @@ struct SimpliestRaftServer params.election_timeout_lower_bound_ = 200; params.election_timeout_upper_bound_ = 400; params.reserved_log_items_ = 5; - params.snapshot_distance_ = 5; + params.snapshot_distance_ = 1; /// forcefully send snapshots params.client_req_timeout_ = 3000; params.return_method_ = nuraft::raft_params::blocking; @@ -298,6 +301,35 @@ zkutil::TestKeeperStorage::ResponsesForSessions getZooKeeperResponses(nuraft::pt return results; } +TEST(CoordinationTest, TestStorageSerialization) +{ + zkutil::TestKeeperStorage storage; + storage.container["/hello"] = zkutil::TestKeeperStorage::Node{.data="world"}; + storage.container["/hello/somepath"] = zkutil::TestKeeperStorage::Node{.data="somedata"}; + storage.session_id_counter = 5; + storage.zxid = 156; + storage.ephemerals[3] = {"/hello", "/"}; + storage.ephemerals[1] = {"/hello/somepath"}; + + DB::WriteBufferFromOwnString buffer; + zkutil::TestKeeperStorageSerializer serializer; + serializer.serialize(storage, buffer); + std::string serialized = buffer.str(); + EXPECT_NE(serialized.size(), 0); + DB::ReadBufferFromString read(serialized); + zkutil::TestKeeperStorage new_storage; + serializer.deserialize(new_storage, read); + + EXPECT_EQ(new_storage.container.size(), 3); + EXPECT_EQ(new_storage.container["/hello"].data, "world"); + EXPECT_EQ(new_storage.container["/hello/somepath"].data, "somedata"); + EXPECT_EQ(new_storage.session_id_counter, 5); + EXPECT_EQ(new_storage.zxid, 156); + EXPECT_EQ(new_storage.ephemerals.size(), 2); + EXPECT_EQ(new_storage.ephemerals[3].size(), 2); + EXPECT_EQ(new_storage.ephemerals[1].size(), 1); +} + TEST(CoordinationTest, TestNuKeeperRaft) { NuKeeperRaftServer s1(1, "localhost", 44447); @@ -390,7 +422,29 @@ TEST(CoordinationTest, TestNuKeeperRaft) EXPECT_EQ(get_responses[0].response->getOpNum(), Coordination::OpNum::Get); EXPECT_EQ(dynamic_cast(get_responses[0].response.get())->data, "world"); + + NuKeeperRaftServer s4(4, "localhost", 44450); + nuraft::srv_config fourth_config(4, "localhost:44450"); + auto ret4 = s2.raft_instance->add_srv(fourth_config); + while (s4.raft_instance->get_leader() != 2) + { + std::cout << "Waiting s1 to join to s2 quorum\n"; + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + + /// Applied snapshot + EXPECT_EQ(s4.raft_instance->get_leader(), 2); + + while (s4.state_machine->getStorage().container.count("/hello") == 0) + { + std::cout << "Waiting s4 to apply entry\n"; + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + + EXPECT_EQ(s4.state_machine->getStorage().container["/hello"].data, "world"); + s1.launcher.shutdown(5); s2.launcher.shutdown(5); s3.launcher.shutdown(5); + s4.launcher.shutdown(5); } From 4aa11b3494417f43d939d53b02d8773c2cf2944c Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 21 Jan 2021 18:09:48 +0300 Subject: [PATCH 11/86] Remove zkutil namespace from TestKeeperStorage --- src/Coordination/NuKeeperStateMachine.cpp | 10 +++++----- src/Coordination/NuKeeperStateMachine.h | 8 ++++---- src/Coordination/TestKeeperStorage.cpp | 7 ------- src/Coordination/TestKeeperStorage.h | 2 +- src/Coordination/TestKeeperStorageDispatcher.cpp | 4 ---- src/Coordination/TestKeeperStorageDispatcher.h | 2 +- src/Coordination/TestKeeperStorageSerializer.cpp | 10 +++++----- src/Coordination/TestKeeperStorageSerializer.h | 4 ++-- src/Coordination/tests/gtest_for_build.cpp | 16 ++++++++-------- src/Coordination/ya.make | 0 src/Interpreters/Context.cpp | 6 +++--- src/Interpreters/Context.h | 4 ++-- src/Server/TestKeeperTCPHandler.h | 2 +- 13 files changed, 32 insertions(+), 43 deletions(-) create mode 100644 src/Coordination/ya.make diff --git a/src/Coordination/NuKeeperStateMachine.cpp b/src/Coordination/NuKeeperStateMachine.cpp index 02f3016be32..abd7ca6b167 100644 --- a/src/Coordination/NuKeeperStateMachine.cpp +++ b/src/Coordination/NuKeeperStateMachine.cpp @@ -8,10 +8,10 @@ namespace DB { -zkutil::TestKeeperStorage::RequestForSession parseRequest(nuraft::buffer & data) +TestKeeperStorage::RequestForSession parseRequest(nuraft::buffer & data) { ReadBufferFromNuraftBuffer buffer(data); - zkutil::TestKeeperStorage::RequestForSession request_for_session; + TestKeeperStorage::RequestForSession request_for_session; readIntBinary(request_for_session.session_id, buffer); int32_t length; @@ -29,7 +29,7 @@ zkutil::TestKeeperStorage::RequestForSession parseRequest(nuraft::buffer & data) return request_for_session; } -nuraft::ptr writeResponses(zkutil::TestKeeperStorage::ResponsesForSessions & responses) +nuraft::ptr writeResponses(TestKeeperStorage::ResponsesForSessions & responses) { WriteBufferFromNuraftBuffer buffer; for (const auto & response_and_session : responses) @@ -52,7 +52,7 @@ nuraft::ptr NuKeeperStateMachine::commit(const size_t log_idx, n { LOG_DEBUG(log, "Commiting logidx {}", log_idx); auto request_for_session = parseRequest(data); - zkutil::TestKeeperStorage::ResponsesForSessions responses_for_sessions; + TestKeeperStorage::ResponsesForSessions responses_for_sessions; { std::lock_guard lock(storage_lock); responses_for_sessions = storage.processRequest(request_for_session.request, request_for_session.session_id); @@ -107,7 +107,7 @@ NuKeeperStateMachine::StorageSnapshotPtr NuKeeperStateMachine::readSnapshot(nura TestKeeperStorageSerializer serializer; ReadBufferFromNuraftBuffer reader(in); - zkutil::TestKeeperStorage new_storage; + TestKeeperStorage new_storage; serializer.deserialize(new_storage, reader); return std::make_shared(ss, new_storage); } diff --git a/src/Coordination/NuKeeperStateMachine.h b/src/Coordination/NuKeeperStateMachine.h index c8dd9f8e570..4e5e8406039 100644 --- a/src/Coordination/NuKeeperStateMachine.h +++ b/src/Coordination/NuKeeperStateMachine.h @@ -42,7 +42,7 @@ public: nuraft::ptr & data_out, bool & is_last_obj) override; - zkutil::TestKeeperStorage & getStorage() + TestKeeperStorage & getStorage() { return storage; } @@ -50,13 +50,13 @@ public: private: struct StorageSnapshot { - StorageSnapshot(const nuraft::ptr & s, const zkutil::TestKeeperStorage & storage_) + StorageSnapshot(const nuraft::ptr & s, const TestKeeperStorage & storage_) : snapshot(s) , storage(storage_) {} nuraft::ptr snapshot; - zkutil::TestKeeperStorage storage; + TestKeeperStorage storage; }; using StorageSnapshotPtr = std::shared_ptr; @@ -67,7 +67,7 @@ private: void writeSnapshot(const StorageSnapshotPtr & snapshot, nuraft::ptr & out) const; - zkutil::TestKeeperStorage storage; + TestKeeperStorage storage; /// Mutex for snapshots std::mutex snapshots_lock; diff --git a/src/Coordination/TestKeeperStorage.cpp b/src/Coordination/TestKeeperStorage.cpp index 31dc4116dc8..ef3ae1dfd16 100644 --- a/src/Coordination/TestKeeperStorage.cpp +++ b/src/Coordination/TestKeeperStorage.cpp @@ -17,13 +17,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -} - -namespace zkutil -{ - -using namespace DB; - static String parentPath(const String & path) { auto rslash_pos = path.rfind('/'); diff --git a/src/Coordination/TestKeeperStorage.h b/src/Coordination/TestKeeperStorage.h index 76111490c78..cc2ac34e7aa 100644 --- a/src/Coordination/TestKeeperStorage.h +++ b/src/Coordination/TestKeeperStorage.h @@ -8,7 +8,7 @@ #include #include -namespace zkutil +namespace DB { using namespace DB; diff --git a/src/Coordination/TestKeeperStorageDispatcher.cpp b/src/Coordination/TestKeeperStorageDispatcher.cpp index 2f8fbbb8fb6..63cb5920f9b 100644 --- a/src/Coordination/TestKeeperStorageDispatcher.cpp +++ b/src/Coordination/TestKeeperStorageDispatcher.cpp @@ -11,10 +11,6 @@ namespace ErrorCodes extern const int TIMEOUT_EXCEEDED; } -} -namespace zkutil -{ - void TestKeeperStorageDispatcher::processingThread() { setThreadName("TestKeeperSProc"); diff --git a/src/Coordination/TestKeeperStorageDispatcher.h b/src/Coordination/TestKeeperStorageDispatcher.h index df4ac2cf99d..c1c739db87d 100644 --- a/src/Coordination/TestKeeperStorageDispatcher.h +++ b/src/Coordination/TestKeeperStorageDispatcher.h @@ -5,7 +5,7 @@ #include #include -namespace zkutil +namespace DB { using ZooKeeperResponseCallback = std::function; diff --git a/src/Coordination/TestKeeperStorageSerializer.cpp b/src/Coordination/TestKeeperStorageSerializer.cpp index bf7015374be..cb3a2643f68 100644 --- a/src/Coordination/TestKeeperStorageSerializer.cpp +++ b/src/Coordination/TestKeeperStorageSerializer.cpp @@ -8,7 +8,7 @@ namespace DB namespace { - void writeNode(const zkutil::TestKeeperStorage::Node & node, WriteBuffer & out) + void writeNode(const TestKeeperStorage::Node & node, WriteBuffer & out) { Coordination::write(node.data, out); Coordination::write(node.acls, out); @@ -18,7 +18,7 @@ namespace Coordination::write(node.seq_num, out); } - void readNode(zkutil::TestKeeperStorage::Node & node, ReadBuffer & in) + void readNode(TestKeeperStorage::Node & node, ReadBuffer & in) { Coordination::read(node.data, in); Coordination::read(node.acls, in); @@ -29,7 +29,7 @@ namespace } } -void TestKeeperStorageSerializer::serialize(const zkutil::TestKeeperStorage & storage, WriteBuffer & out) const +void TestKeeperStorageSerializer::serialize(const TestKeeperStorage & storage, WriteBuffer & out) const { Coordination::write(storage.zxid, out); Coordination::write(storage.session_id_counter, out); @@ -49,7 +49,7 @@ void TestKeeperStorageSerializer::serialize(const zkutil::TestKeeperStorage & st } } -void TestKeeperStorageSerializer::deserialize(zkutil::TestKeeperStorage & storage, ReadBuffer & in) const +void TestKeeperStorageSerializer::deserialize(TestKeeperStorage & storage, ReadBuffer & in) const { int64_t session_id_counter, zxid; Coordination::read(zxid, in); @@ -63,7 +63,7 @@ void TestKeeperStorageSerializer::deserialize(zkutil::TestKeeperStorage & storag { std::string path; Coordination::read(path, in); - zkutil::TestKeeperStorage::Node node; + TestKeeperStorage::Node node; readNode(node, in); storage.container[path] = node; } diff --git a/src/Coordination/TestKeeperStorageSerializer.h b/src/Coordination/TestKeeperStorageSerializer.h index b4453574cfd..5a6a0cea0a5 100644 --- a/src/Coordination/TestKeeperStorageSerializer.h +++ b/src/Coordination/TestKeeperStorageSerializer.h @@ -9,9 +9,9 @@ namespace DB class TestKeeperStorageSerializer { public: - void serialize(const zkutil::TestKeeperStorage & storage, WriteBuffer & out) const; + void serialize(const TestKeeperStorage & storage, WriteBuffer & out) const; - void deserialize(zkutil::TestKeeperStorage & storage, ReadBuffer & in) const; + void deserialize(TestKeeperStorage & storage, ReadBuffer & in) const; }; } diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index 09c5db03514..0c7ff8a579c 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -276,9 +276,9 @@ nuraft::ptr getZooKeeperLogEntry(int64_t session_id, const Coord return buf.getBuffer(); } -zkutil::TestKeeperStorage::ResponsesForSessions getZooKeeperResponses(nuraft::ptr & buffer, const Coordination::ZooKeeperRequestPtr & request) +DB::TestKeeperStorage::ResponsesForSessions getZooKeeperResponses(nuraft::ptr & buffer, const Coordination::ZooKeeperRequestPtr & request) { - zkutil::TestKeeperStorage::ResponsesForSessions results; + DB::TestKeeperStorage::ResponsesForSessions results; DB::ReadBufferFromNuraftBuffer buf(buffer); while (!buf.eof()) { @@ -296,28 +296,28 @@ zkutil::TestKeeperStorage::ResponsesForSessions getZooKeeperResponses(nuraft::pt Coordination::read(err, buf); auto response = request->makeResponse(); response->readImpl(buf); - results.push_back(zkutil::TestKeeperStorage::ResponseForSession{session_id, response}); + results.push_back(DB::TestKeeperStorage::ResponseForSession{session_id, response}); } return results; } TEST(CoordinationTest, TestStorageSerialization) { - zkutil::TestKeeperStorage storage; - storage.container["/hello"] = zkutil::TestKeeperStorage::Node{.data="world"}; - storage.container["/hello/somepath"] = zkutil::TestKeeperStorage::Node{.data="somedata"}; + DB::TestKeeperStorage storage; + storage.container["/hello"] = DB::TestKeeperStorage::Node{.data="world"}; + storage.container["/hello/somepath"] = DB::TestKeeperStorage::Node{.data="somedata"}; storage.session_id_counter = 5; storage.zxid = 156; storage.ephemerals[3] = {"/hello", "/"}; storage.ephemerals[1] = {"/hello/somepath"}; DB::WriteBufferFromOwnString buffer; - zkutil::TestKeeperStorageSerializer serializer; + DB::TestKeeperStorageSerializer serializer; serializer.serialize(storage, buffer); std::string serialized = buffer.str(); EXPECT_NE(serialized.size(), 0); DB::ReadBufferFromString read(serialized); - zkutil::TestKeeperStorage new_storage; + DB::TestKeeperStorage new_storage; serializer.deserialize(new_storage, read); EXPECT_EQ(new_storage.container.size(), 3); diff --git a/src/Coordination/ya.make b/src/Coordination/ya.make new file mode 100644 index 00000000000..e69de29bb2d diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index ad6b09b2d88..959b96722e0 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -306,7 +306,7 @@ struct ContextShared ConfigurationPtr zookeeper_config; /// Stores zookeeper configs mutable std::mutex test_keeper_storage_dispatcher_mutex; - mutable std::shared_ptr test_keeper_storage_dispatcher; + mutable std::shared_ptr test_keeper_storage_dispatcher; mutable std::mutex auxiliary_zookeepers_mutex; mutable std::map auxiliary_zookeepers; /// Map for auxiliary ZooKeeper clients. ConfigurationPtr auxiliary_zookeepers_config; /// Stores auxiliary zookeepers configs @@ -1531,11 +1531,11 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const return shared->zookeeper; } -std::shared_ptr & Context::getTestKeeperStorageDispatcher() const +std::shared_ptr & Context::getTestKeeperStorageDispatcher() const { std::lock_guard lock(shared->test_keeper_storage_dispatcher_mutex); if (!shared->test_keeper_storage_dispatcher) - shared->test_keeper_storage_dispatcher = std::make_shared(); + shared->test_keeper_storage_dispatcher = std::make_shared(); return shared->test_keeper_storage_dispatcher; } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 9c8d5252373..616d2d97de0 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -40,7 +40,6 @@ namespace Poco namespace zkutil { class ZooKeeper; - class TestKeeperStorageDispatcher; } @@ -107,6 +106,7 @@ using StoragePolicyPtr = std::shared_ptr; using StoragePoliciesMap = std::map; class StoragePolicySelector; using StoragePolicySelectorPtr = std::shared_ptr; +class TestKeeperStorageDispatcher; class IOutputFormat; using OutputFormatPtr = std::shared_ptr; @@ -513,7 +513,7 @@ public: std::shared_ptr getAuxiliaryZooKeeper(const String & name) const; - std::shared_ptr & getTestKeeperStorageDispatcher() const; + std::shared_ptr & getTestKeeperStorageDispatcher() const; /// Set auxiliary zookeepers configuration at server starting or configuration reloading. void reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & config); diff --git a/src/Server/TestKeeperTCPHandler.h b/src/Server/TestKeeperTCPHandler.h index 38f4db56c69..e7372e8dd82 100644 --- a/src/Server/TestKeeperTCPHandler.h +++ b/src/Server/TestKeeperTCPHandler.h @@ -28,7 +28,7 @@ private: IServer & server; Poco::Logger * log; Context global_context; - std::shared_ptr test_keeper_storage_dispatcher; + std::shared_ptr test_keeper_storage_dispatcher; Poco::Timespan operation_timeout; Poco::Timespan session_timeout; int64_t session_id; From c2e6d6cfe8007afb13dc77d474f6e31d063014af Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 21 Jan 2021 23:01:25 +0300 Subject: [PATCH 12/86] Starting nukeeper server --- src/Coordination/NuKeeperServer.cpp | 13 +++++++ src/Coordination/NuKeeperServer.h | 43 +++++++++++++++++++++++ src/Coordination/NuKeeperStateMachine.cpp | 1 - src/Coordination/TestKeeperStorage.h | 1 + 4 files changed, 57 insertions(+), 1 deletion(-) create mode 100644 src/Coordination/NuKeeperServer.cpp create mode 100644 src/Coordination/NuKeeperServer.h diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp new file mode 100644 index 00000000000..162e521f1c8 --- /dev/null +++ b/src/Coordination/NuKeeperServer.cpp @@ -0,0 +1,13 @@ +#include + +namespace DB +{ + +void NuKeeperServer::addServer(int server_id_, const std::string & server_uri) +{ + if (raft_instance->is_leader()) + { + nuraft::srv_config first_config(server_id, server_uri); + } + +} diff --git a/src/Coordination/NuKeeperServer.h b/src/Coordination/NuKeeperServer.h new file mode 100644 index 00000000000..0dc536b1593 --- /dev/null +++ b/src/Coordination/NuKeeperServer.h @@ -0,0 +1,43 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +class NuKeeperServer +{ +private: + int server_id; + + std::string hostname; + + int port; + + std::string endpoint; + + nuraft::ptr state_machine; + + nuraft::ptr state_manager; + + nuraft::raft_launcher launcher; + + nuraft::ptr raft_instance; + +public: + NuKeeperServer(int server_id, const std::string & hostname, int port); + + void startup(); + + TestKeeperStorage::ResponsesForSessions putRequests(const TestKeeperStorage::RequestsForSessions & requests); + + void addServer(int server_id_, const std::string & server_uri); + + void shutdown(); +}; + +} diff --git a/src/Coordination/NuKeeperStateMachine.cpp b/src/Coordination/NuKeeperStateMachine.cpp index abd7ca6b167..136ead44596 100644 --- a/src/Coordination/NuKeeperStateMachine.cpp +++ b/src/Coordination/NuKeeperStateMachine.cpp @@ -81,7 +81,6 @@ bool NuKeeperStateMachine::apply_snapshot(nuraft::snapshot & s) nuraft::ptr NuKeeperStateMachine::last_snapshot() { - LOG_DEBUG(log, "Trying to get last snapshot"); // Just return the latest snapshot. std::lock_guard lock(snapshots_lock); diff --git a/src/Coordination/TestKeeperStorage.h b/src/Coordination/TestKeeperStorage.h index cc2ac34e7aa..2c7c6bad4fa 100644 --- a/src/Coordination/TestKeeperStorage.h +++ b/src/Coordination/TestKeeperStorage.h @@ -78,6 +78,7 @@ public: { return session_id_counter++; } + ResponsesForSessions processRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id); ResponsesForSessions finalize(const RequestsForSessions & expired_requests); }; From c1e36cfe7063250d020c0d687ea77301e74c6516 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 22 Jan 2021 19:04:57 +0300 Subject: [PATCH 13/86] Something working --- programs/server/Server.cpp | 3 + src/Coordination/NuKeeperServer.cpp | 158 +++++++++++++++++- src/Coordination/NuKeeperServer.h | 29 +++- src/Coordination/NuKeeperStateMachine.cpp | 29 +++- src/Coordination/TestKeeperStorage.cpp | 1 + .../TestKeeperStorageDispatcher.cpp | 27 +-- .../TestKeeperStorageDispatcher.h | 17 +- utils/zookeeper-test/main.cpp | 5 + 8 files changed, 231 insertions(+), 38 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 94cd6854f78..df1513e6b65 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -830,6 +830,9 @@ int Server::main(const std::vector & /*args*/) listen_try = true; } + /// Initialize test keeper raft + global_context->getTestKeeperStorageDispatcher(); + for (const auto & listen_host : listen_hosts) { /// TCP TestKeeper diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index 162e521f1c8..2aefc215451 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -1,13 +1,165 @@ #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include namespace DB { -void NuKeeperServer::addServer(int server_id_, const std::string & server_uri) + +NuKeeperServer::NuKeeperServer(int server_id_, const std::string & hostname_, int port_) + : server_id(server_id_) + , hostname(hostname_) + , port(port_) + , endpoint(hostname + ":" + std::to_string(port)) + , state_machine(nuraft::cs_new()) + , state_manager(nuraft::cs_new(server_id, endpoint)) { - if (raft_instance->is_leader()) +} + +NuraftError NuKeeperServer::addServer(int server_id_, const std::string & server_uri_) +{ + nuraft::srv_config config(server_id_, server_uri_); + auto ret1 = raft_instance->add_srv(config); + return NuraftError{ret1->get_result_code(), ret1->get_result_str()}; +} + + +NuraftError NuKeeperServer::startup() +{ + nuraft::raft_params params; + params.heart_beat_interval_ = 100; + params.election_timeout_lower_bound_ = 200; + params.election_timeout_upper_bound_ = 400; + params.reserved_log_items_ = 5; + params.snapshot_distance_ = 5; + params.client_req_timeout_ = 3000; + params.return_method_ = nuraft::raft_params::blocking; + + raft_instance = launcher.init( + state_machine, state_manager, nuraft::cs_new("RaftInstance"), port, + nuraft::asio_service::options{}, params); + + if (!raft_instance) + return NuraftError{nuraft::cmd_result_code::TIMEOUT, "Cannot create RAFT instance"}; + + static constexpr auto MAX_RETRY = 30; + for (size_t i = 0; i < MAX_RETRY; ++i) { - nuraft::srv_config first_config(server_id, server_uri); + if (raft_instance->is_initialized()) + return NuraftError{nuraft::cmd_result_code::OK, ""}; + + std::this_thread::sleep_for(std::chrono::milliseconds(100)); } + return NuraftError{nuraft::cmd_result_code::TIMEOUT, "Cannot start RAFT instance"}; +} + +NuraftError NuKeeperServer::shutdown() +{ + if (!launcher.shutdown(5)) + return NuraftError{nuraft::cmd_result_code::TIMEOUT, "Temout waiting RAFT instance to shutdown"}; + return NuraftError{nuraft::cmd_result_code::OK, ""}; +} + +namespace +{ + +nuraft::ptr getZooKeeperLogEntry(int64_t session_id, const Coordination::ZooKeeperRequestPtr & request) +{ + DB::WriteBufferFromNuraftBuffer buf; + DB::writeIntBinary(session_id, buf); + request->write(buf); + return buf.getBuffer(); +} + +} + +TestKeeperStorage::ResponsesForSessions NuKeeperServer::readZooKeeperResponses(nuraft::ptr & buffer) +{ + DB::TestKeeperStorage::ResponsesForSessions results; + DB::ReadBufferFromNuraftBuffer buf(buffer); + + while (!buf.eof()) + { + int64_t session_id; + DB::readIntBinary(session_id, buf); + int32_t length; + Coordination::XID xid; + int64_t zxid; + Coordination::Error err; + + Coordination::read(length, buf); + Coordination::read(xid, buf); + Coordination::read(zxid, buf); + Coordination::read(err, buf); + Coordination::ZooKeeperResponsePtr response; + + if (xid == Coordination::WATCH_XID) + response = std::make_shared(); + else + { + response = ops_mapping[session_id][xid]; + ops_mapping[session_id].erase(xid); + if (ops_mapping[session_id].empty()) + ops_mapping.erase(session_id); + } + + if (err == Coordination::Error::ZOK && (xid == Coordination::WATCH_XID || response->getOpNum() != Coordination::OpNum::Close)) + response->readImpl(buf); + + response->xid = xid; + response->zxid = zxid; + response->error = err; + + results.push_back(DB::TestKeeperStorage::ResponseForSession{session_id, response}); + } + return results; +} + +TestKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const TestKeeperStorage::RequestsForSessions & requests) +{ + std::vector> entries; + for (auto & [session_id, request] : requests) + { + ops_mapping[session_id][request->xid] = request->makeResponse(); + entries.push_back(getZooKeeperLogEntry(session_id, request)); + } + + auto result = raft_instance->append_entries(entries); + if (!result->get_accepted()) + return {}; + + if (result->get_result_code() != nuraft::cmd_result_code::OK) + return {}; + + return readZooKeeperResponses(result->get()); +} + + +int64_t NuKeeperServer::getSessionID() +{ + auto entry = nuraft::buffer::alloc(sizeof(size_t)); + nuraft::buffer_serializer bs(entry); + bs.put_i64(0); + + auto result = raft_instance->append_entries({entry}); + if (!result->get_accepted()) + return -1; + + if (result->get_result_code() != nuraft::cmd_result_code::OK) + return -1; + + auto resp = result->get(); + nuraft::buffer_serializer bs_resp(resp); + return bs_resp.get_i64(); +} + } diff --git a/src/Coordination/NuKeeperServer.h b/src/Coordination/NuKeeperServer.h index 0dc536b1593..c77a7a8be0a 100644 --- a/src/Coordination/NuKeeperServer.h +++ b/src/Coordination/NuKeeperServer.h @@ -5,10 +5,17 @@ #include #include #include +#include namespace DB { +struct NuraftError +{ + nuraft::cmd_result_code code; + std::string message; +}; + class NuKeeperServer { private: @@ -20,7 +27,7 @@ private: std::string endpoint; - nuraft::ptr state_machine; + nuraft::ptr state_machine; nuraft::ptr state_manager; @@ -28,16 +35,26 @@ private: nuraft::ptr raft_instance; -public: - NuKeeperServer(int server_id, const std::string & hostname, int port); + using XIDToOp = std::unordered_map; - void startup(); + using SessionIDOps = std::unordered_map; + + SessionIDOps ops_mapping; + + TestKeeperStorage::ResponsesForSessions readZooKeeperResponses(nuraft::ptr & buffer); + +public: + NuKeeperServer(int server_id_, const std::string & hostname_, int port_); + + NuraftError startup(); TestKeeperStorage::ResponsesForSessions putRequests(const TestKeeperStorage::RequestsForSessions & requests); - void addServer(int server_id_, const std::string & server_uri); + int64_t getSessionID(); - void shutdown(); + NuraftError addServer(int server_id_, const std::string & server_uri); + + NuraftError shutdown(); }; } diff --git a/src/Coordination/NuKeeperStateMachine.cpp b/src/Coordination/NuKeeperStateMachine.cpp index 136ead44596..79324c91cd3 100644 --- a/src/Coordination/NuKeeperStateMachine.cpp +++ b/src/Coordination/NuKeeperStateMachine.cpp @@ -51,15 +51,32 @@ NuKeeperStateMachine::NuKeeperStateMachine() nuraft::ptr NuKeeperStateMachine::commit(const size_t log_idx, nuraft::buffer & data) { LOG_DEBUG(log, "Commiting logidx {}", log_idx); - auto request_for_session = parseRequest(data); - TestKeeperStorage::ResponsesForSessions responses_for_sessions; + if (data.size() == sizeof(size_t)) { - std::lock_guard lock(storage_lock); - responses_for_sessions = storage.processRequest(request_for_session.request, request_for_session.session_id); + LOG_DEBUG(log, "Session ID response {}", log_idx); + auto response = nuraft::buffer::alloc(sizeof(size_t)); + nuraft::buffer_serializer bs(response); + { + std::lock_guard lock(storage_lock); + bs.put_i64(storage.getSessionID()); + } + last_committed_idx = log_idx; + return response; } + else + { + auto request_for_session = parseRequest(data); + //LOG_DEBUG(log, "GOT REQUEST {}", Coordination::toString(request_for_session.request->getOpNum())); + TestKeeperStorage::ResponsesForSessions responses_for_sessions; + { + std::lock_guard lock(storage_lock); + responses_for_sessions = storage.processRequest(request_for_session.request, request_for_session.session_id); + } + //LOG_DEBUG(log, "TOTAL RESPONSES {} FIRST XID {}", responses_for_sessions.size(), responses_for_sessions[0].response->xid); - last_committed_idx = log_idx; - return writeResponses(responses_for_sessions); + last_committed_idx = log_idx; + return writeResponses(responses_for_sessions); + } } bool NuKeeperStateMachine::apply_snapshot(nuraft::snapshot & s) diff --git a/src/Coordination/TestKeeperStorage.cpp b/src/Coordination/TestKeeperStorage.cpp index ef3ae1dfd16..ef72f5d4eaa 100644 --- a/src/Coordination/TestKeeperStorage.cpp +++ b/src/Coordination/TestKeeperStorage.cpp @@ -519,6 +519,7 @@ TestKeeperStorage::ResponsesForSessions TestKeeperStorage::finalize(const Reques finalized = true; + /// TODO delete ephemerals ResponsesForSessions finalize_results; auto finish_watch = [] (const auto & watch_pair) -> ResponsesForSessions { diff --git a/src/Coordination/TestKeeperStorageDispatcher.cpp b/src/Coordination/TestKeeperStorageDispatcher.cpp index 63cb5920f9b..9cc40f6e5c3 100644 --- a/src/Coordination/TestKeeperStorageDispatcher.cpp +++ b/src/Coordination/TestKeeperStorageDispatcher.cpp @@ -18,16 +18,16 @@ void TestKeeperStorageDispatcher::processingThread() { while (!shutdown) { - RequestInfo info; + TestKeeperStorage::RequestForSession request; UInt64 max_wait = UInt64(operation_timeout.totalMilliseconds()); - if (requests_queue.tryPop(info, max_wait)) + if (requests_queue.tryPop(request, max_wait)) { if (shutdown) break; - auto responses = storage.processRequest(info.request, info.session_id); + auto responses = server.putRequests({request}); for (const auto & response_for_session : responses) setResponse(response_for_session.session_id, response_for_session.response); } @@ -67,15 +67,17 @@ void TestKeeperStorageDispatcher::finalize() processing_thread.join(); } - RequestInfo info; - TestKeeperStorage::RequestsForSessions expired_requests; - while (requests_queue.tryPop(info)) - expired_requests.push_back(TestKeeperStorage::RequestForSession{info.session_id, info.request}); + //TestKeeperStorage::RequestsForSessions expired_requests; + //TestKeeperStorage::RequestForSession request; + //while (requests_queue.tryPop(request)) + // expired_requests.push_back(TestKeeperStorage::RequestForSession{request}); - auto expired_responses = storage.finalize(expired_requests); + //auto expired_responses = storage.finalize(expired_requests); - for (const auto & response_for_session : expired_responses) - setResponse(response_for_session.session_id, response_for_session.response); + //for (const auto & response_for_session : expired_responses) + // setResponse(response_for_session.session_id, response_for_session.response); + /// TODO FIXME + server.shutdown(); } void TestKeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id) @@ -87,8 +89,7 @@ void TestKeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperReques throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unknown session id {}", session_id); } - RequestInfo request_info; - request_info.time = clock::now(); + TestKeeperStorage::RequestForSession request_info; request_info.request = request; request_info.session_id = session_id; @@ -101,7 +102,9 @@ void TestKeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperReques } TestKeeperStorageDispatcher::TestKeeperStorageDispatcher() + : server(1, "localhost", 44444) { + server.startup(); processing_thread = ThreadFromGlobalPool([this] { processingThread(); }); } diff --git a/src/Coordination/TestKeeperStorageDispatcher.h b/src/Coordination/TestKeeperStorageDispatcher.h index c1c739db87d..ef788a16369 100644 --- a/src/Coordination/TestKeeperStorageDispatcher.h +++ b/src/Coordination/TestKeeperStorageDispatcher.h @@ -2,8 +2,9 @@ #include #include -#include #include +#include +#include namespace DB { @@ -17,16 +18,9 @@ private: using clock = std::chrono::steady_clock; - struct RequestInfo - { - Coordination::ZooKeeperRequestPtr request; - clock::time_point time; - int64_t session_id; - }; - std::mutex push_request_mutex; - using RequestsQueue = ConcurrentBoundedQueue; + using RequestsQueue = ConcurrentBoundedQueue; RequestsQueue requests_queue{1}; std::atomic shutdown{false}; using SessionToResponseCallback = std::unordered_map; @@ -36,7 +30,7 @@ private: ThreadFromGlobalPool processing_thread; - TestKeeperStorage storage; + NuKeeperServer server; std::mutex session_id_mutex; private: @@ -46,6 +40,7 @@ private: public: TestKeeperStorageDispatcher(); + ~TestKeeperStorageDispatcher(); void putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id); @@ -53,7 +48,7 @@ public: int64_t getSessionID() { std::lock_guard lock(session_id_mutex); - return storage.getSessionID(); + return server.getSessionID(); } void registerSession(int64_t session_id, ZooKeeperResponseCallback callback); diff --git a/utils/zookeeper-test/main.cpp b/utils/zookeeper-test/main.cpp index 8f8aac00866..bfd7df26726 100644 --- a/utils/zookeeper-test/main.cpp +++ b/utils/zookeeper-test/main.cpp @@ -127,18 +127,22 @@ void testCreateListWatchEvent(zkutil::ZooKeeper & zk) void testMultiRequest(zkutil::ZooKeeper & zk) { + std::cerr << "Testing multi request\n"; Coordination::Requests requests; requests.push_back(zkutil::makeCreateRequest("/data/multirequest", "aaa", zkutil::CreateMode::Persistent)); requests.push_back(zkutil::makeSetRequest("/data/multirequest", "bbb", -1)); zk.multi(requests); + std::cerr << "Multi executed\n"; try { requests.clear(); + std::cerr << "Testing bad multi\n"; requests.push_back(zkutil::makeCreateRequest("/data/multirequest", "qweqwe", zkutil::CreateMode::Persistent)); requests.push_back(zkutil::makeSetRequest("/data/multirequest", "bbb", -1)); requests.push_back(zkutil::makeSetRequest("/data/multirequest", "ccc", -1)); zk.multi(requests); + std::cerr << "Bad multi executed\n"; std::terminate(); } catch (...) @@ -147,6 +151,7 @@ void testMultiRequest(zkutil::ZooKeeper & zk) } checkEq(zk, "/data/multirequest", "bbb"); + std::cerr << "Multi request finished\n"; } std::mutex elements_mutex; From 8b03329f4d1589ad0e2ae7dd00d15246a6f95c14 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 22 Jan 2021 23:04:47 +0300 Subject: [PATCH 14/86] Some logging --- src/Coordination/NuKeeperServer.cpp | 2 ++ src/Coordination/WriteBufferFromNuraftBuffer.cpp | 2 ++ 2 files changed, 4 insertions(+) diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index 2aefc215451..7fb7f25aef6 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -127,10 +127,12 @@ TestKeeperStorage::ResponsesForSessions NuKeeperServer::readZooKeeperResponses(n TestKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const TestKeeperStorage::RequestsForSessions & requests) { std::vector> entries; + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "REQUESTS SIZE {}", requests.size()); for (auto & [session_id, request] : requests) { ops_mapping[session_id][request->xid] = request->makeResponse(); entries.push_back(getZooKeeperLogEntry(session_id, request)); + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "ENTRY SIZE {}", entries.back()->size()); } auto result = raft_instance->append_entries(entries); diff --git a/src/Coordination/WriteBufferFromNuraftBuffer.cpp b/src/Coordination/WriteBufferFromNuraftBuffer.cpp index 7d0a1dbcbb1..2f451af6538 100644 --- a/src/Coordination/WriteBufferFromNuraftBuffer.cpp +++ b/src/Coordination/WriteBufferFromNuraftBuffer.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { @@ -16,6 +17,7 @@ void WriteBufferFromNuraftBuffer::nextImpl() size_t old_size = buffer->size(); /// pos may not be equal to vector.data() + old_size, because WriteBuffer::next() can be used to flush data size_t pos_offset = pos - reinterpret_cast(buffer->data_begin()); + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "BUFFER SIZE {}", old_size * size_multiplier); nuraft::ptr new_buffer = nuraft::buffer::alloc(old_size * size_multiplier); memcpy(new_buffer->data_begin(), buffer->data_begin(), buffer->size()); buffer = new_buffer; From 7ff04d7532a378315ca91334d8e98630ccef29a0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 25 Jan 2021 13:19:02 +0300 Subject: [PATCH 15/86] Some fixes --- src/Coordination/LoggerWrapper.h | 6 ++++-- src/Coordination/NuKeeperServer.cpp | 4 +--- src/Coordination/NuKeeperStateMachine.cpp | 4 ++-- src/Coordination/WriteBufferFromNuraftBuffer.cpp | 15 +++++++++------ 4 files changed, 16 insertions(+), 13 deletions(-) diff --git a/src/Coordination/LoggerWrapper.h b/src/Coordination/LoggerWrapper.h index 37de7806e9d..5895457441a 100644 --- a/src/Coordination/LoggerWrapper.h +++ b/src/Coordination/LoggerWrapper.h @@ -11,7 +11,9 @@ class LoggerWrapper : public nuraft::logger public: LoggerWrapper(const std::string & name) : log(&Poco::Logger::get(name)) - {} + { + set_level(4); + } void put_details( int level, @@ -25,7 +27,7 @@ public: void set_level(int level) override { - level = std::max(6, std::min(1, level)); + level = std::min(6, std::max(1, level)); log->setLevel(level); } diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index 7fb7f25aef6..16f69585af7 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -39,7 +39,7 @@ NuraftError NuKeeperServer::startup() params.election_timeout_lower_bound_ = 200; params.election_timeout_upper_bound_ = 400; params.reserved_log_items_ = 5; - params.snapshot_distance_ = 5; + params.snapshot_distance_ = 50; params.client_req_timeout_ = 3000; params.return_method_ = nuraft::raft_params::blocking; @@ -127,12 +127,10 @@ TestKeeperStorage::ResponsesForSessions NuKeeperServer::readZooKeeperResponses(n TestKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const TestKeeperStorage::RequestsForSessions & requests) { std::vector> entries; - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "REQUESTS SIZE {}", requests.size()); for (auto & [session_id, request] : requests) { ops_mapping[session_id][request->xid] = request->makeResponse(); entries.push_back(getZooKeeperLogEntry(session_id, request)); - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "ENTRY SIZE {}", entries.back()->size()); } auto result = raft_instance->append_entries(entries); diff --git a/src/Coordination/NuKeeperStateMachine.cpp b/src/Coordination/NuKeeperStateMachine.cpp index 79324c91cd3..69088d09472 100644 --- a/src/Coordination/NuKeeperStateMachine.cpp +++ b/src/Coordination/NuKeeperStateMachine.cpp @@ -50,7 +50,7 @@ NuKeeperStateMachine::NuKeeperStateMachine() nuraft::ptr NuKeeperStateMachine::commit(const size_t log_idx, nuraft::buffer & data) { - LOG_DEBUG(log, "Commiting logidx {}", log_idx); + //LOG_DEBUG(log, "Commiting logidx {}", log_idx); if (data.size() == sizeof(size_t)) { LOG_DEBUG(log, "Session ID response {}", log_idx); @@ -72,9 +72,9 @@ nuraft::ptr NuKeeperStateMachine::commit(const size_t log_idx, n std::lock_guard lock(storage_lock); responses_for_sessions = storage.processRequest(request_for_session.request, request_for_session.session_id); } - //LOG_DEBUG(log, "TOTAL RESPONSES {} FIRST XID {}", responses_for_sessions.size(), responses_for_sessions[0].response->xid); last_committed_idx = log_idx; + //LOG_DEBUG(log, "TOTAL RESPONSES {} FIRST XID {} FOR LOG IDX {}", responses_for_sessions.size(), responses_for_sessions[0].response->xid, log_idx); return writeResponses(responses_for_sessions); } } diff --git a/src/Coordination/WriteBufferFromNuraftBuffer.cpp b/src/Coordination/WriteBufferFromNuraftBuffer.cpp index 2f451af6538..1a16b7cef24 100644 --- a/src/Coordination/WriteBufferFromNuraftBuffer.cpp +++ b/src/Coordination/WriteBufferFromNuraftBuffer.cpp @@ -14,15 +14,18 @@ void WriteBufferFromNuraftBuffer::nextImpl() if (is_finished) throw Exception("WriteBufferFromNuraftBuffer is finished", ErrorCodes::CANNOT_WRITE_AFTER_END_OF_BUFFER); - size_t old_size = buffer->size(); /// pos may not be equal to vector.data() + old_size, because WriteBuffer::next() can be used to flush data size_t pos_offset = pos - reinterpret_cast(buffer->data_begin()); - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "BUFFER SIZE {}", old_size * size_multiplier); - nuraft::ptr new_buffer = nuraft::buffer::alloc(old_size * size_multiplier); - memcpy(new_buffer->data_begin(), buffer->data_begin(), buffer->size()); - buffer = new_buffer; + size_t old_size = buffer->size(); + if (pos_offset == old_size) + { + nuraft::ptr new_buffer = nuraft::buffer::alloc(old_size * size_multiplier); + memcpy(new_buffer->data_begin(), buffer->data_begin(), buffer->size()); + buffer = new_buffer; + } internal_buffer = Buffer(reinterpret_cast(buffer->data_begin() + pos_offset), reinterpret_cast(buffer->data_begin() + buffer->size())); working_buffer = internal_buffer; + } WriteBufferFromNuraftBuffer::WriteBufferFromNuraftBuffer() @@ -38,7 +41,7 @@ void WriteBufferFromNuraftBuffer::finalize() return; is_finished = true; - size_t real_size = position() - reinterpret_cast(buffer->data_begin()); + size_t real_size = pos - reinterpret_cast(buffer->data_begin()); nuraft::ptr new_buffer = nuraft::buffer::alloc(real_size); memcpy(new_buffer->data_begin(), buffer->data_begin(), real_size); buffer = new_buffer; From dea4b5009bb716e53f8b1b84548ad5e0497574c6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 25 Jan 2021 15:29:12 +0300 Subject: [PATCH 16/86] Some server initialization --- programs/server/Server.cpp | 4 +- src/Common/ErrorCodes.cpp | 1 + src/Coordination/InMemoryLogStore.cpp | 8 +-- src/Coordination/NuKeeperServer.cpp | 40 +++++++----- src/Coordination/NuKeeperServer.h | 12 +--- src/Coordination/NuKeeperStateMachine.cpp | 4 -- .../TestKeeperStorageDispatcher.cpp | 61 ++++++++++++++----- .../TestKeeperStorageDispatcher.h | 10 +-- src/Interpreters/Context.cpp | 17 +++++- src/Interpreters/Context.h | 1 + src/Server/TestKeeperTCPHandler.cpp | 4 +- tests/config/config.d/test_keeper_port.xml | 8 +++ 12 files changed, 114 insertions(+), 56 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 66a9b700e89..ddd72e97dde 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -842,8 +842,8 @@ int Server::main(const std::vector & /*args*/) listen_try = true; } - /// Initialize test keeper raft - global_context->getTestKeeperStorageDispatcher(); + /// Initialize test keeper RAFT. Do nothing if no test_keeper_server in config. + global_context->initializeTestKeeperStorageDispatcher(); for (const auto & listen_host : listen_hosts) { diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index a2cd65137c0..1c398a52666 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -533,6 +533,7 @@ M(564, INTERSERVER_SCHEME_DOESNT_MATCH) \ M(565, TOO_MANY_PARTITIONS) \ M(566, CANNOT_RMDIR) \ + M(567, RAFT_ERROR) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Coordination/InMemoryLogStore.cpp b/src/Coordination/InMemoryLogStore.cpp index b9e2e502fc7..101458891e7 100644 --- a/src/Coordination/InMemoryLogStore.cpp +++ b/src/Coordination/InMemoryLogStore.cpp @@ -34,7 +34,7 @@ size_t InMemoryLogStore::next_slot() const nuraft::ptr InMemoryLogStore::last_entry() const { - ulong next_idx = next_slot(); + size_t next_idx = next_slot(); std::lock_guard lock(logs_lock); auto entry = logs.find(next_idx - 1); if (entry == logs.end()) @@ -105,7 +105,7 @@ nuraft::ptr InMemoryLogStore::entry_at(size_t index) size_t InMemoryLogStore::term_at(size_t index) { - ulong term = 0; + size_t term = 0; { std::lock_guard l(logs_lock); auto entry = logs.find(index); @@ -121,7 +121,7 @@ nuraft::ptr InMemoryLogStore::pack(size_t index, Int32 cnt) std::vector> returned_logs; size_t size_total = 0; - for (ulong ii = index; ii < index + cnt; ++ii) + for (size_t ii = index; ii < index + cnt; ++ii) { ptr le = nullptr; { @@ -180,7 +180,7 @@ void InMemoryLogStore::apply_pack(size_t index, nuraft::buffer & pack) bool InMemoryLogStore::compact(size_t last_log_index) { std::lock_guard l(logs_lock); - for (ulong ii = start_idx; ii <= last_log_index; ++ii) + for (size_t ii = start_idx; ii <= last_log_index; ++ii) { auto entry = logs.find(ii); if (entry != logs.end()) diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index 16f69585af7..c79cdd64014 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -13,6 +13,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int TIMEOUT_EXCEEDED; + extern const int RAFT_ERROR; +} NuKeeperServer::NuKeeperServer(int server_id_, const std::string & hostname_, int port_) : server_id(server_id_) @@ -24,22 +29,22 @@ NuKeeperServer::NuKeeperServer(int server_id_, const std::string & hostname_, in { } -NuraftError NuKeeperServer::addServer(int server_id_, const std::string & server_uri_) +bool NuKeeperServer::addServer(int server_id_, const std::string & server_uri_) { nuraft::srv_config config(server_id_, server_uri_); auto ret1 = raft_instance->add_srv(config); - return NuraftError{ret1->get_result_code(), ret1->get_result_str()}; + return ret1->get_result_code() == nuraft::cmd_result_code::OK; } -NuraftError NuKeeperServer::startup() +void NuKeeperServer::startup() { nuraft::raft_params params; params.heart_beat_interval_ = 100; params.election_timeout_lower_bound_ = 200; params.election_timeout_upper_bound_ = 400; - params.reserved_log_items_ = 5; - params.snapshot_distance_ = 50; + params.reserved_log_items_ = 5000; + params.snapshot_distance_ = 5000; params.client_req_timeout_ = 3000; params.return_method_ = nuraft::raft_params::blocking; @@ -48,25 +53,26 @@ NuraftError NuKeeperServer::startup() nuraft::asio_service::options{}, params); if (!raft_instance) - return NuraftError{nuraft::cmd_result_code::TIMEOUT, "Cannot create RAFT instance"}; + throw Exception(ErrorCodes::RAFT_ERROR, "Cannot allocate RAFT instance"); static constexpr auto MAX_RETRY = 30; for (size_t i = 0; i < MAX_RETRY; ++i) { if (raft_instance->is_initialized()) - return NuraftError{nuraft::cmd_result_code::OK, ""}; + return; std::this_thread::sleep_for(std::chrono::milliseconds(100)); } - return NuraftError{nuraft::cmd_result_code::TIMEOUT, "Cannot start RAFT instance"}; + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Cannot start RAFT server within startup timeout"); } -NuraftError NuKeeperServer::shutdown() +TestKeeperStorage::ResponsesForSessions NuKeeperServer::shutdown(const TestKeeperStorage::RequestsForSessions & expired_requests) { + auto responses = putRequests(expired_requests); if (!launcher.shutdown(5)) - return NuraftError{nuraft::cmd_result_code::TIMEOUT, "Temout waiting RAFT instance to shutdown"}; - return NuraftError{nuraft::cmd_result_code::OK, ""}; + LOG_WARNING(&Poco::Logger::get("NuKeeperServer"), "Failed to shutdown RAFT server in {} seconds", 5); + return responses; } namespace @@ -96,6 +102,7 @@ TestKeeperStorage::ResponsesForSessions NuKeeperServer::readZooKeeperResponses(n int64_t zxid; Coordination::Error err; + /// FIXME (alesap) We don't need to parse responses here Coordination::read(length, buf); Coordination::read(xid, buf); Coordination::read(zxid, buf); @@ -135,10 +142,10 @@ TestKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const TestKe auto result = raft_instance->append_entries(entries); if (!result->get_accepted()) - return {}; + throw Exception(ErrorCodes::RAFT_ERROR, "Cannot send requests to RAFT, mostly because we are not leader"); if (result->get_result_code() != nuraft::cmd_result_code::OK) - return {}; + throw Exception(ErrorCodes::RAFT_ERROR, "Requests failed"); return readZooKeeperResponses(result->get()); } @@ -146,16 +153,17 @@ TestKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const TestKe int64_t NuKeeperServer::getSessionID() { - auto entry = nuraft::buffer::alloc(sizeof(size_t)); + auto entry = nuraft::buffer::alloc(sizeof(int64_t)); + /// Just special session request nuraft::buffer_serializer bs(entry); bs.put_i64(0); auto result = raft_instance->append_entries({entry}); if (!result->get_accepted()) - return -1; + throw Exception(ErrorCodes::RAFT_ERROR, "Cannot send session_id request to RAFT"); if (result->get_result_code() != nuraft::cmd_result_code::OK) - return -1; + throw Exception(ErrorCodes::RAFT_ERROR, "session_id request failed to RAFT"); auto resp = result->get(); nuraft::buffer_serializer bs_resp(resp); diff --git a/src/Coordination/NuKeeperServer.h b/src/Coordination/NuKeeperServer.h index c77a7a8be0a..6f2ca72eae5 100644 --- a/src/Coordination/NuKeeperServer.h +++ b/src/Coordination/NuKeeperServer.h @@ -10,12 +10,6 @@ namespace DB { -struct NuraftError -{ - nuraft::cmd_result_code code; - std::string message; -}; - class NuKeeperServer { private: @@ -46,15 +40,15 @@ private: public: NuKeeperServer(int server_id_, const std::string & hostname_, int port_); - NuraftError startup(); + void startup(); TestKeeperStorage::ResponsesForSessions putRequests(const TestKeeperStorage::RequestsForSessions & requests); int64_t getSessionID(); - NuraftError addServer(int server_id_, const std::string & server_uri); + bool addServer(int server_id_, const std::string & server_uri); - NuraftError shutdown(); + TestKeeperStorage::ResponsesForSessions shutdown(const TestKeeperStorage::RequestsForSessions & expired_requests); }; } diff --git a/src/Coordination/NuKeeperStateMachine.cpp b/src/Coordination/NuKeeperStateMachine.cpp index 69088d09472..13c0f92e604 100644 --- a/src/Coordination/NuKeeperStateMachine.cpp +++ b/src/Coordination/NuKeeperStateMachine.cpp @@ -50,7 +50,6 @@ NuKeeperStateMachine::NuKeeperStateMachine() nuraft::ptr NuKeeperStateMachine::commit(const size_t log_idx, nuraft::buffer & data) { - //LOG_DEBUG(log, "Commiting logidx {}", log_idx); if (data.size() == sizeof(size_t)) { LOG_DEBUG(log, "Session ID response {}", log_idx); @@ -66,7 +65,6 @@ nuraft::ptr NuKeeperStateMachine::commit(const size_t log_idx, n else { auto request_for_session = parseRequest(data); - //LOG_DEBUG(log, "GOT REQUEST {}", Coordination::toString(request_for_session.request->getOpNum())); TestKeeperStorage::ResponsesForSessions responses_for_sessions; { std::lock_guard lock(storage_lock); @@ -74,7 +72,6 @@ nuraft::ptr NuKeeperStateMachine::commit(const size_t log_idx, n } last_committed_idx = log_idx; - //LOG_DEBUG(log, "TOTAL RESPONSES {} FIRST XID {} FOR LOG IDX {}", responses_for_sessions.size(), responses_for_sessions[0].response->xid, log_idx); return writeResponses(responses_for_sessions); } } @@ -98,7 +95,6 @@ bool NuKeeperStateMachine::apply_snapshot(nuraft::snapshot & s) nuraft::ptr NuKeeperStateMachine::last_snapshot() { - LOG_DEBUG(log, "Trying to get last snapshot"); // Just return the latest snapshot. std::lock_guard lock(snapshots_lock); auto entry = snapshots.rbegin(); diff --git a/src/Coordination/TestKeeperStorageDispatcher.cpp b/src/Coordination/TestKeeperStorageDispatcher.cpp index 9cc40f6e5c3..120e3b2aae6 100644 --- a/src/Coordination/TestKeeperStorageDispatcher.cpp +++ b/src/Coordination/TestKeeperStorageDispatcher.cpp @@ -27,7 +27,7 @@ void TestKeeperStorageDispatcher::processingThread() if (shutdown) break; - auto responses = server.putRequests({request}); + auto responses = server->putRequests({request}); for (const auto & response_for_session : responses) setResponse(response_for_session.session_id, response_for_session.response); } @@ -67,26 +67,27 @@ void TestKeeperStorageDispatcher::finalize() processing_thread.join(); } - //TestKeeperStorage::RequestsForSessions expired_requests; - //TestKeeperStorage::RequestForSession request; - //while (requests_queue.tryPop(request)) - // expired_requests.push_back(TestKeeperStorage::RequestForSession{request}); + if (server) + { + TestKeeperStorage::RequestsForSessions expired_requests; + TestKeeperStorage::RequestForSession request; + while (requests_queue.tryPop(request)) + expired_requests.push_back(TestKeeperStorage::RequestForSession{request}); - //auto expired_responses = storage.finalize(expired_requests); + auto expired_responses = server->shutdown(expired_requests); - //for (const auto & response_for_session : expired_responses) - // setResponse(response_for_session.session_id, response_for_session.response); - /// TODO FIXME - server.shutdown(); + for (const auto & response_for_session : expired_responses) + setResponse(response_for_session.session_id, response_for_session.response); + } } -void TestKeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id) +bool TestKeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id) { { std::lock_guard lock(session_to_response_callback_mutex); if (session_to_response_callback.count(session_id) == 0) - throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unknown session id {}", session_id); + return false; } TestKeeperStorage::RequestForSession request_info; @@ -99,13 +100,43 @@ void TestKeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperReques requests_queue.push(std::move(request_info)); else if (!requests_queue.tryPush(std::move(request_info), operation_timeout.totalMilliseconds())) throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::TIMEOUT_EXCEEDED); + return true; } -TestKeeperStorageDispatcher::TestKeeperStorageDispatcher() - : server(1, "localhost", 44444) + +void TestKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfiguration & config) { - server.startup(); + int myid = config.getInt("test_keeper_server.server_id"); + std::string myhostname; + int myport; + + Poco::Util::AbstractConfiguration::Keys keys; + config.keys("test_keeper_server.raft_configuration", keys); + + std::vector> server_configs; + for (const auto & server_key : keys) + { + int server_id = config.getInt("test_keeper_server.raft_configuration." + server_key + ".id"); + std::string hostname = config.getString("test_keeper_server.raft_configuration." + server_key + ".hostname"); + int port = config.getInt("test_keeper_server.raft_configuration." + server_key + ".port"); + if (server_id == myid) + { + myhostname = hostname; + myport = port; + } + else + { + server_configs.emplace_back(server_id, hostname, port); + } + } + + server = std::make_unique(myid, myhostname, myport); + server->startup(); + for (const auto & [id, hostname, port] : server_configs) + server->addServer(id, hostname + ":" + std::to_string(port)); + processing_thread = ThreadFromGlobalPool([this] { processingThread(); }); + } TestKeeperStorageDispatcher::~TestKeeperStorageDispatcher() diff --git a/src/Coordination/TestKeeperStorageDispatcher.h b/src/Coordination/TestKeeperStorageDispatcher.h index ef788a16369..aa220beecf2 100644 --- a/src/Coordination/TestKeeperStorageDispatcher.h +++ b/src/Coordination/TestKeeperStorageDispatcher.h @@ -30,7 +30,7 @@ private: ThreadFromGlobalPool processing_thread; - NuKeeperServer server; + std::unique_ptr server; std::mutex session_id_mutex; private: @@ -39,16 +39,18 @@ private: void setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response); public: - TestKeeperStorageDispatcher(); + TestKeeperStorageDispatcher() = default; + + void initialize(const Poco::Util::AbstractConfiguration & config); ~TestKeeperStorageDispatcher(); - void putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id); + bool putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id); int64_t getSessionID() { std::lock_guard lock(session_id_mutex); - return server.getSessionID(); + return server->getSessionID(); } void registerSession(int64_t session_id, ZooKeeperResponseCallback callback); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5f49a85843c..ee5be5f6edb 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1568,11 +1568,26 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const return shared->zookeeper; } +void Context::initializeTestKeeperStorageDispatcher() const +{ + std::lock_guard lock(shared->test_keeper_storage_dispatcher_mutex); + + if (shared->test_keeper_storage_dispatcher) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to initialize TestKeeper multiple times"); + + auto & config = getConfigRef(); + if (config.has("test_keeper_server")) + { + shared->test_keeper_storage_dispatcher = std::make_shared(); + shared->test_keeper_storage_dispatcher->initialize(config); + } +} + std::shared_ptr & Context::getTestKeeperStorageDispatcher() const { std::lock_guard lock(shared->test_keeper_storage_dispatcher_mutex); if (!shared->test_keeper_storage_dispatcher) - shared->test_keeper_storage_dispatcher = std::make_shared(); + throw Exception(ErrorCodes::LOGICAL_ERROR, "TestKeeper must be initialized before requests"); return shared->test_keeper_storage_dispatcher; } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 5f3f6b25256..537ddcc0ec8 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -574,6 +574,7 @@ public: std::shared_ptr getAuxiliaryZooKeeper(const String & name) const; + void initializeTestKeeperStorageDispatcher() const; std::shared_ptr & getTestKeeperStorageDispatcher() const; /// Set auxiliary zookeepers configuration at server starting or configuration reloading. diff --git a/src/Server/TestKeeperTCPHandler.cpp b/src/Server/TestKeeperTCPHandler.cpp index 97999c2b1c1..3e88d543112 100644 --- a/src/Server/TestKeeperTCPHandler.cpp +++ b/src/Server/TestKeeperTCPHandler.cpp @@ -30,6 +30,7 @@ namespace ErrorCodes extern const int SYSTEM_ERROR; extern const int LOGICAL_ERROR; extern const int UNEXPECTED_PACKET_FROM_CLIENT; + extern const int TIMEOUT_EXCEEDED; } struct PollResult @@ -423,7 +424,8 @@ std::pair TestKeeperTCPHandler::receiveR request->xid = xid; request->readImpl(*in); - test_keeper_storage_dispatcher->putRequest(request, session_id); + if (!test_keeper_storage_dispatcher->putRequest(request, session_id)) + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Sesssion {} already disconnected", session_id); return std::make_pair(opnum, xid); } diff --git a/tests/config/config.d/test_keeper_port.xml b/tests/config/config.d/test_keeper_port.xml index 79e993b41f7..fff60d749f6 100644 --- a/tests/config/config.d/test_keeper_port.xml +++ b/tests/config/config.d/test_keeper_port.xml @@ -3,5 +3,13 @@ 9181 10000 30000 + 1 + + + 1 + localhost + 44444 + + From 97b9dba460529d254a8416a80ae82f80bda302ac Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 25 Jan 2021 17:10:18 +0300 Subject: [PATCH 17/86] Multinode config --- programs/server/Server.cpp | 7 +++- src/Coordination/NuKeeperServer.cpp | 11 ++++-- src/Coordination/NuKeeperServer.h | 6 ++- .../TestKeeperStorageDispatcher.cpp | 16 +++++--- .../configs/enable_test_keeper.xml | 8 ++++ .../test_testkeeper_multinode/__init__.py | 1 + .../configs/enable_test_keeper1.xml | 28 +++++++++++++ .../configs/enable_test_keeper2.xml | 28 +++++++++++++ .../configs/enable_test_keeper3.xml | 28 +++++++++++++ .../configs/log_conf.xml | 12 ++++++ .../configs/use_test_keeper.xml | 8 ++++ .../test_testkeeper_multinode/test.py | 39 +++++++++++++++++++ 12 files changed, 179 insertions(+), 13 deletions(-) create mode 100644 tests/integration/test_testkeeper_multinode/__init__.py create mode 100644 tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml create mode 100644 tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml create mode 100644 tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml create mode 100644 tests/integration/test_testkeeper_multinode/configs/log_conf.xml create mode 100644 tests/integration/test_testkeeper_multinode/configs/use_test_keeper.xml create mode 100644 tests/integration/test_testkeeper_multinode/test.py diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index ddd72e97dde..04919e8504c 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -842,8 +842,11 @@ int Server::main(const std::vector & /*args*/) listen_try = true; } - /// Initialize test keeper RAFT. Do nothing if no test_keeper_server in config. - global_context->initializeTestKeeperStorageDispatcher(); + if (config().has("test_keeper_server")) + { + /// Initialize test keeper RAFT. Do nothing if no test_keeper_server in config. + global_context->initializeTestKeeperStorageDispatcher(); + } for (const auto & listen_host : listen_hosts) { diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index c79cdd64014..a3786342e05 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -19,19 +19,20 @@ namespace ErrorCodes extern const int RAFT_ERROR; } -NuKeeperServer::NuKeeperServer(int server_id_, const std::string & hostname_, int port_) +NuKeeperServer::NuKeeperServer(int server_id_, const std::string & hostname_, int port_, bool can_become_leader_) : server_id(server_id_) , hostname(hostname_) , port(port_) , endpoint(hostname + ":" + std::to_string(port)) + , can_become_leader(can_become_leader_) , state_machine(nuraft::cs_new()) , state_manager(nuraft::cs_new(server_id, endpoint)) { } -bool NuKeeperServer::addServer(int server_id_, const std::string & server_uri_) +bool NuKeeperServer::addServer(int server_id_, const std::string & server_uri_, bool can_become_leader_) { - nuraft::srv_config config(server_id_, server_uri_); + nuraft::srv_config config(server_id_, 0, server_uri_, "", /*FIXME follower=*/ !can_become_leader_); auto ret1 = raft_instance->add_srv(config); return ret1->get_result_code() == nuraft::cmd_result_code::OK; } @@ -69,7 +70,9 @@ void NuKeeperServer::startup() TestKeeperStorage::ResponsesForSessions NuKeeperServer::shutdown(const TestKeeperStorage::RequestsForSessions & expired_requests) { - auto responses = putRequests(expired_requests); + TestKeeperStorage::ResponsesForSessions responses; + if (can_become_leader) + responses = putRequests(expired_requests); if (!launcher.shutdown(5)) LOG_WARNING(&Poco::Logger::get("NuKeeperServer"), "Failed to shutdown RAFT server in {} seconds", 5); return responses; diff --git a/src/Coordination/NuKeeperServer.h b/src/Coordination/NuKeeperServer.h index 6f2ca72eae5..4c10614cd5c 100644 --- a/src/Coordination/NuKeeperServer.h +++ b/src/Coordination/NuKeeperServer.h @@ -21,6 +21,8 @@ private: std::string endpoint; + bool can_become_leader; + nuraft::ptr state_machine; nuraft::ptr state_manager; @@ -38,7 +40,7 @@ private: TestKeeperStorage::ResponsesForSessions readZooKeeperResponses(nuraft::ptr & buffer); public: - NuKeeperServer(int server_id_, const std::string & hostname_, int port_); + NuKeeperServer(int server_id_, const std::string & hostname_, int port_, bool can_become_leader_); void startup(); @@ -46,7 +48,7 @@ public: int64_t getSessionID(); - bool addServer(int server_id_, const std::string & server_uri); + bool addServer(int server_id_, const std::string & server_uri, bool can_become_leader_); TestKeeperStorage::ResponsesForSessions shutdown(const TestKeeperStorage::RequestsForSessions & expired_requests); }; diff --git a/src/Coordination/TestKeeperStorageDispatcher.cpp b/src/Coordination/TestKeeperStorageDispatcher.cpp index 120e3b2aae6..7c78ca0e79f 100644 --- a/src/Coordination/TestKeeperStorageDispatcher.cpp +++ b/src/Coordination/TestKeeperStorageDispatcher.cpp @@ -112,28 +112,34 @@ void TestKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigura Poco::Util::AbstractConfiguration::Keys keys; config.keys("test_keeper_server.raft_configuration", keys); + bool my_can_become_leader = true; - std::vector> server_configs; + std::vector> server_configs; for (const auto & server_key : keys) { int server_id = config.getInt("test_keeper_server.raft_configuration." + server_key + ".id"); std::string hostname = config.getString("test_keeper_server.raft_configuration." + server_key + ".hostname"); int port = config.getInt("test_keeper_server.raft_configuration." + server_key + ".port"); + bool can_become_leader = config.getBool("test_keeper_server.raft_configuration." + server_key + ".can_become_leader", true); if (server_id == myid) { myhostname = hostname; myport = port; + my_can_become_leader = can_become_leader; } else { - server_configs.emplace_back(server_id, hostname, port); + server_configs.emplace_back(server_id, hostname, port, can_become_leader); } } - server = std::make_unique(myid, myhostname, myport); + server = std::make_unique(myid, myhostname, myport, my_can_become_leader); server->startup(); - for (const auto & [id, hostname, port] : server_configs) - server->addServer(id, hostname + ":" + std::to_string(port)); + if (my_can_become_leader) + { + for (const auto & [id, hostname, port, can_become_leader] : server_configs) + server->addServer(id, hostname + ":" + std::to_string(port), can_become_leader); + } processing_thread = ThreadFromGlobalPool([this] { processingThread(); }); diff --git a/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml b/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml index 79e993b41f7..fff60d749f6 100644 --- a/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml +++ b/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml @@ -3,5 +3,13 @@ 9181 10000 30000 + 1 + + + 1 + localhost + 44444 + + diff --git a/tests/integration/test_testkeeper_multinode/__init__.py b/tests/integration/test_testkeeper_multinode/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_testkeeper_multinode/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml new file mode 100644 index 00000000000..486942aec71 --- /dev/null +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml @@ -0,0 +1,28 @@ + + + 9181 + 10000 + 30000 + 1 + + + 1 + node1 + 44444 + true + + + 2 + node2 + 44444 + false + + + 3 + node3 + 44444 + false + + + + diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml new file mode 100644 index 00000000000..94873883943 --- /dev/null +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml @@ -0,0 +1,28 @@ + + + 9181 + 10000 + 30000 + 2 + + + 1 + node1 + 44444 + true + + + 2 + node2 + 44444 + false + + + 3 + node3 + 44444 + false + + + + diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml new file mode 100644 index 00000000000..0219a0e5763 --- /dev/null +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml @@ -0,0 +1,28 @@ + + + 9181 + 10000 + 30000 + 3 + + + 1 + node1 + 44444 + true + + + 2 + node2 + 44444 + false + + + 3 + node3 + 44444 + false + + + + diff --git a/tests/integration/test_testkeeper_multinode/configs/log_conf.xml b/tests/integration/test_testkeeper_multinode/configs/log_conf.xml new file mode 100644 index 00000000000..318a6bca95d --- /dev/null +++ b/tests/integration/test_testkeeper_multinode/configs/log_conf.xml @@ -0,0 +1,12 @@ + + 3 + + trace + /var/log/clickhouse-server/log.log + /var/log/clickhouse-server/log.err.log + 1000M + 10 + /var/log/clickhouse-server/stderr.log + /var/log/clickhouse-server/stdout.log + + diff --git a/tests/integration/test_testkeeper_multinode/configs/use_test_keeper.xml b/tests/integration/test_testkeeper_multinode/configs/use_test_keeper.xml new file mode 100644 index 00000000000..20d731b8553 --- /dev/null +++ b/tests/integration/test_testkeeper_multinode/configs/use_test_keeper.xml @@ -0,0 +1,8 @@ + + + + node1 + 9181 + + + diff --git a/tests/integration/test_testkeeper_multinode/test.py b/tests/integration/test_testkeeper_multinode/test.py new file mode 100644 index 00000000000..d76e72ee92e --- /dev/null +++ b/tests/integration/test_testkeeper_multinode/test.py @@ -0,0 +1,39 @@ +import pytest +from helpers.cluster import ClickHouseCluster +import random +import string +import os +import time +from multiprocessing.dummy import Pool + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', main_configs=['configs/enable_test_keeper1.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml']) +node2 = cluster.add_instance('node2', main_configs=['configs/enable_test_keeper2.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml']) +node3 = cluster.add_instance('node3', main_configs=['configs/enable_test_keeper3.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml']) + +from kazoo.client import KazooClient + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def test_simple_replicated_table(started_cluster): + + for i, node in enumerate([node1, node2, node3]): + node.query("CREATE TABLE t (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t', '{}') ORDER BY tuple()".format(i + 1)) + + node2.query("INSERT INTO t SELECT number FROM numbers(10)") + + node1.query("SYSTEM SYNC REPLICA t", timeout=10) + node3.query("SYSTEM SYNC REPLICA t", timeout=10) + + assert node1.query("SELECT COUNT() FROM t") == "10\n" + assert node2.query("SELECT COUNT() FROM t") == "10\n" + assert node3.query("SELECT COUNT() FROM t") == "10\n" From 1576800289f1fbb5d222b4192d625c670d93ebe1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 25 Jan 2021 17:36:06 +0300 Subject: [PATCH 18/86] Remove races --- src/Coordination/tests/gtest_for_build.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index 0c7ff8a579c..d74eaafba27 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -330,6 +330,11 @@ TEST(CoordinationTest, TestStorageSerialization) EXPECT_EQ(new_storage.ephemerals[1].size(), 1); } +/// Code with obvious races, but I don't want to make it +/// more complex to avoid races. +#if defined(__has_feature) +# if ! __has_feature(thread_sanitizer) + TEST(CoordinationTest, TestNuKeeperRaft) { NuKeeperRaftServer s1(1, "localhost", 44447); @@ -448,3 +453,6 @@ TEST(CoordinationTest, TestNuKeeperRaft) s3.launcher.shutdown(5); s4.launcher.shutdown(5); } + +# endif +#endif From d7e805ad99565a1f19d02f9d43ca7c2f2ca0f07f Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 25 Jan 2021 17:47:03 +0300 Subject: [PATCH 19/86] Comment --- src/Coordination/SummingStateMachine.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Coordination/SummingStateMachine.h b/src/Coordination/SummingStateMachine.h index df343378408..20d6258eb0b 100644 --- a/src/Coordination/SummingStateMachine.h +++ b/src/Coordination/SummingStateMachine.h @@ -9,6 +9,7 @@ namespace DB { +/// Example trivial state machine. class SummingStateMachine : public nuraft::state_machine { public: From 43a2aae3686718ed6d09be6d5659b9492d53755e Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 25 Jan 2021 17:59:10 +0300 Subject: [PATCH 20/86] Add non working cmake --- src/Coordination/ya.make | 25 +++++++++++++++++++++++++ src/ya.make | 1 + 2 files changed, 26 insertions(+) diff --git a/src/Coordination/ya.make b/src/Coordination/ya.make index e69de29bb2d..de2be9df7ac 100644 --- a/src/Coordination/ya.make +++ b/src/Coordination/ya.make @@ -0,0 +1,25 @@ +# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. +OWNER(g:clickhouse) + +LIBRARY() + +PEERDIR( + clickhouse/src/Common + contrib/libs/NuRaft +) + + +SRCS( + InMemoryLogStore.cpp + InMemoryStateManager.cpp + NuKeeperServer.cpp + NuKeeperStateMachine.cpp + SummingStateMachine.cpp + TestKeeperStorage.cpp + TestKeeperStorageDispatcher.cpp + TestKeeperStorageSerializer.cpp + WriteBufferFromNuraftBuffer.cpp + +) + +END() diff --git a/src/ya.make b/src/ya.make index c3e6b41b9b9..5361c8a5695 100644 --- a/src/ya.make +++ b/src/ya.make @@ -9,6 +9,7 @@ PEERDIR( clickhouse/src/Columns clickhouse/src/Common clickhouse/src/Compression + clickhouse/src/Coordination clickhouse/src/Core clickhouse/src/Databases clickhouse/src/DataStreams From eccd9a29de5498998d957697531ae37db8b8a39f Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 25 Jan 2021 18:32:59 +0300 Subject: [PATCH 21/86] Build NuRaft even in fast test --- docker/test/fasttest/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 7211ce31a87..cf4a5031f8b 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -163,6 +163,7 @@ function clone_submodules contrib/xz contrib/dragonbox contrib/fast_float + contrib/NuRaft ) git submodule sync From 46ca832aa1a75cb9d20f631169501cc4cf0f0b13 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 25 Jan 2021 18:53:13 +0300 Subject: [PATCH 22/86] Enable nuraft in fast test --- docker/test/fasttest/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index cf4a5031f8b..b1ebd97a78c 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -183,6 +183,7 @@ function run_cmake "-DENABLE_EMBEDDED_COMPILER=0" "-DENABLE_THINLTO=0" "-DUSE_UNWIND=1" + "-DENABLE_NURAFT=1" ) # TODO remove this? we don't use ccache anyway. An option would be to download it From 045935151f37e628f44b17ad0048d60e98827d9c Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 25 Jan 2021 19:09:22 +0300 Subject: [PATCH 23/86] Bump From 10cec45e53ebf4774ee299d339cf12fe91a17770 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 26 Jan 2021 10:47:04 +0300 Subject: [PATCH 24/86] Fix obvious deadlock --- src/Coordination/NuKeeperServer.cpp | 21 +++++++++++--- .../TestKeeperStorageDispatcher.cpp | 28 +++++++++---------- 2 files changed, 30 insertions(+), 19 deletions(-) diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index a3786342e05..c7f9012f287 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -46,7 +46,7 @@ void NuKeeperServer::startup() params.election_timeout_upper_bound_ = 400; params.reserved_log_items_ = 5000; params.snapshot_distance_ = 5000; - params.client_req_timeout_ = 3000; + params.client_req_timeout_ = 10000; params.return_method_ = nuraft::raft_params::blocking; raft_instance = launcher.init( @@ -145,10 +145,23 @@ TestKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const TestKe auto result = raft_instance->append_entries(entries); if (!result->get_accepted()) - throw Exception(ErrorCodes::RAFT_ERROR, "Cannot send requests to RAFT, mostly because we are not leader"); + throw Exception(ErrorCodes::RAFT_ERROR, "Cannot send requests to RAFT, mostly because we are not leader, code {}, message: '{}'", result->get_result_code(), result->get_result_str()); - if (result->get_result_code() != nuraft::cmd_result_code::OK) - throw Exception(ErrorCodes::RAFT_ERROR, "Requests failed"); + if (result->get_result_code() == nuraft::cmd_result_code::TIMEOUT) + { + TestKeeperStorage::ResponsesForSessions responses; + for (const auto & [session_id, request] : requests) + { + auto response = request->makeResponse(); + response->xid = request->xid; + response->zxid = 0; /// FIXME what we can do with it? + response->error = Coordination::Error::ZOPERATIONTIMEOUT; + responses.push_back(DB::TestKeeperStorage::ResponseForSession{session_id, response}); + } + return responses; + } + else if (result->get_result_code() != nuraft::cmd_result_code::OK) + throw Exception(ErrorCodes::RAFT_ERROR, "Requests result failed with code {} and message: '{}'", result->get_result_code(), result->get_result_str()); return readZooKeeperResponses(result->get()); } diff --git a/src/Coordination/TestKeeperStorageDispatcher.cpp b/src/Coordination/TestKeeperStorageDispatcher.cpp index 7c78ca0e79f..3aef5213adc 100644 --- a/src/Coordination/TestKeeperStorageDispatcher.cpp +++ b/src/Coordination/TestKeeperStorageDispatcher.cpp @@ -14,30 +14,28 @@ namespace ErrorCodes void TestKeeperStorageDispatcher::processingThread() { setThreadName("TestKeeperSProc"); - try + while (!shutdown) { - while (!shutdown) + TestKeeperStorage::RequestForSession request; + + UInt64 max_wait = UInt64(operation_timeout.totalMilliseconds()); + + if (requests_queue.tryPop(request, max_wait)) { - TestKeeperStorage::RequestForSession request; - - UInt64 max_wait = UInt64(operation_timeout.totalMilliseconds()); - - if (requests_queue.tryPop(request, max_wait)) + if (shutdown) + break; + try { - if (shutdown) - break; - auto responses = server->putRequests({request}); for (const auto & response_for_session : responses) setResponse(response_for_session.session_id, response_for_session.response); } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } } } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - finalize(); - } } void TestKeeperStorageDispatcher::setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response) From ddeb008bbb6ee7209fd8c862fb1dd00672001ef7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 26 Jan 2021 10:52:34 +0300 Subject: [PATCH 25/86] Replace ulong with size_t --- src/Coordination/SummingStateMachine.cpp | 2 +- src/Coordination/SummingStateMachine.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Coordination/SummingStateMachine.cpp b/src/Coordination/SummingStateMachine.cpp index bf2a5bb818f..59649850123 100644 --- a/src/Coordination/SummingStateMachine.cpp +++ b/src/Coordination/SummingStateMachine.cpp @@ -110,7 +110,7 @@ void SummingStateMachine::save_logical_snp_obj( int SummingStateMachine::read_logical_snp_obj( nuraft::snapshot & s, void* & /*user_snp_ctx*/, - ulong obj_id, + size_t obj_id, nuraft::ptr & data_out, bool & is_last_obj) { diff --git a/src/Coordination/SummingStateMachine.h b/src/Coordination/SummingStateMachine.h index 20d6258eb0b..9aca02c6bdc 100644 --- a/src/Coordination/SummingStateMachine.h +++ b/src/Coordination/SummingStateMachine.h @@ -41,7 +41,7 @@ public: int read_logical_snp_obj( nuraft::snapshot & s, void* & user_snp_ctx, - ulong obj_id, + size_t obj_id, nuraft::ptr & data_out, bool & is_last_obj) override; From 71dca6dc006f1042156ec4b6799da9e4dbc52e06 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 26 Jan 2021 11:17:19 +0300 Subject: [PATCH 26/86] Tidy fixes --- src/Coordination/LoggerWrapper.h | 17 ++++++++++------- src/Coordination/NuKeeperServer.cpp | 2 +- src/Coordination/NuKeeperStateMachine.cpp | 7 ++++--- src/Coordination/NuKeeperStateMachine.h | 4 ++-- src/Coordination/SummingStateMachine.cpp | 3 ++- .../TestKeeperStorageSerializer.cpp | 4 ++-- src/Coordination/TestKeeperStorageSerializer.h | 4 ++-- src/Coordination/tests/gtest_for_build.cpp | 4 ++-- src/Interpreters/Context.cpp | 2 +- 9 files changed, 26 insertions(+), 21 deletions(-) diff --git a/src/Coordination/LoggerWrapper.h b/src/Coordination/LoggerWrapper.h index 5895457441a..00d4c6544a5 100644 --- a/src/Coordination/LoggerWrapper.h +++ b/src/Coordination/LoggerWrapper.h @@ -11,33 +11,36 @@ class LoggerWrapper : public nuraft::logger public: LoggerWrapper(const std::string & name) : log(&Poco::Logger::get(name)) + , level(4) { - set_level(4); + log->setLevel(level); } void put_details( - int level, + int level_, const char * /* source_file */, const char * /* func_name */, size_t /* line_number */, const std::string & msg) override { - LOG_IMPL(log, static_cast(level), static_cast(level), msg); + LOG_IMPL(log, static_cast(level_), static_cast(level_), msg); } - void set_level(int level) override + void set_level(int level_) override { - level = std::min(6, std::max(1, level)); - log->setLevel(level); + level_ = std::min(6, std::max(1, level_)); + log->setLevel(level_); + level = level_; } int get_level() override { - return log->getLevel(); + return level; } private: Poco::Logger * log; + std::atomic level; }; } diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index c7f9012f287..5b5aeb206c4 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -137,7 +137,7 @@ TestKeeperStorage::ResponsesForSessions NuKeeperServer::readZooKeeperResponses(n TestKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const TestKeeperStorage::RequestsForSessions & requests) { std::vector> entries; - for (auto & [session_id, request] : requests) + for (const auto & [session_id, request] : requests) { ops_mapping[session_id][request->xid] = request->makeResponse(); entries.push_back(getZooKeeperLogEntry(session_id, request)); diff --git a/src/Coordination/NuKeeperStateMachine.cpp b/src/Coordination/NuKeeperStateMachine.cpp index 13c0f92e604..52c82f44784 100644 --- a/src/Coordination/NuKeeperStateMachine.cpp +++ b/src/Coordination/NuKeeperStateMachine.cpp @@ -8,6 +8,8 @@ namespace DB { +static constexpr int MAX_SNAPSHOTS = 3; + TestKeeperStorage::RequestForSession parseRequest(nuraft::buffer & data) { ReadBufferFromNuraftBuffer buffer(data); @@ -112,7 +114,7 @@ NuKeeperStateMachine::StorageSnapshotPtr NuKeeperStateMachine::createSnapshotInt return std::make_shared(ss, storage); } -NuKeeperStateMachine::StorageSnapshotPtr NuKeeperStateMachine::readSnapshot(nuraft::snapshot & s, nuraft::buffer & in) const +NuKeeperStateMachine::StorageSnapshotPtr NuKeeperStateMachine::readSnapshot(nuraft::snapshot & s, nuraft::buffer & in) { nuraft::ptr snp_buf = s.serialize(); nuraft::ptr ss = nuraft::snapshot::deserialize(*snp_buf); @@ -125,7 +127,7 @@ NuKeeperStateMachine::StorageSnapshotPtr NuKeeperStateMachine::readSnapshot(nura } -void NuKeeperStateMachine::writeSnapshot(const NuKeeperStateMachine::StorageSnapshotPtr & snapshot, nuraft::ptr & out) const +void NuKeeperStateMachine::writeSnapshot(const NuKeeperStateMachine::StorageSnapshotPtr & snapshot, nuraft::ptr & out) { TestKeeperStorageSerializer serializer; @@ -143,7 +145,6 @@ void NuKeeperStateMachine::create_snapshot( { std::lock_guard lock(snapshots_lock); snapshots[s.get_last_log_idx()] = snapshot; - const int MAX_SNAPSHOTS = 3; int num = snapshots.size(); auto entry = snapshots.begin(); diff --git a/src/Coordination/NuKeeperStateMachine.h b/src/Coordination/NuKeeperStateMachine.h index 4e5e8406039..a120e3f1cf6 100644 --- a/src/Coordination/NuKeeperStateMachine.h +++ b/src/Coordination/NuKeeperStateMachine.h @@ -63,9 +63,9 @@ private: StorageSnapshotPtr createSnapshotInternal(nuraft::snapshot & s); - StorageSnapshotPtr readSnapshot(nuraft::snapshot & s, nuraft::buffer & in) const; + static StorageSnapshotPtr readSnapshot(nuraft::snapshot & s, nuraft::buffer & in); - void writeSnapshot(const StorageSnapshotPtr & snapshot, nuraft::ptr & out) const; + static void writeSnapshot(const StorageSnapshotPtr & snapshot, nuraft::ptr & out); TestKeeperStorage storage; /// Mutex for snapshots diff --git a/src/Coordination/SummingStateMachine.cpp b/src/Coordination/SummingStateMachine.cpp index 59649850123..f9a3f4f9de2 100644 --- a/src/Coordination/SummingStateMachine.cpp +++ b/src/Coordination/SummingStateMachine.cpp @@ -4,6 +4,8 @@ namespace DB { +static constexpr int MAX_SNAPSHOTS = 3; + static int64_t deserializeValue(nuraft::buffer & buffer) { nuraft::buffer_serializer bs(buffer); @@ -68,7 +70,6 @@ void SummingStateMachine::createSnapshotInternal(nuraft::snapshot & s) snapshots[s.get_last_log_idx()] = ctx; // Maintain last 3 snapshots only. - const int MAX_SNAPSHOTS = 3; int num = snapshots.size(); auto entry = snapshots.begin(); diff --git a/src/Coordination/TestKeeperStorageSerializer.cpp b/src/Coordination/TestKeeperStorageSerializer.cpp index cb3a2643f68..f6116d29104 100644 --- a/src/Coordination/TestKeeperStorageSerializer.cpp +++ b/src/Coordination/TestKeeperStorageSerializer.cpp @@ -29,7 +29,7 @@ namespace } } -void TestKeeperStorageSerializer::serialize(const TestKeeperStorage & storage, WriteBuffer & out) const +void TestKeeperStorageSerializer::serialize(const TestKeeperStorage & storage, WriteBuffer & out) { Coordination::write(storage.zxid, out); Coordination::write(storage.session_id_counter, out); @@ -49,7 +49,7 @@ void TestKeeperStorageSerializer::serialize(const TestKeeperStorage & storage, W } } -void TestKeeperStorageSerializer::deserialize(TestKeeperStorage & storage, ReadBuffer & in) const +void TestKeeperStorageSerializer::deserialize(TestKeeperStorage & storage, ReadBuffer & in) { int64_t session_id_counter, zxid; Coordination::read(zxid, in); diff --git a/src/Coordination/TestKeeperStorageSerializer.h b/src/Coordination/TestKeeperStorageSerializer.h index 5a6a0cea0a5..a3909c24694 100644 --- a/src/Coordination/TestKeeperStorageSerializer.h +++ b/src/Coordination/TestKeeperStorageSerializer.h @@ -9,9 +9,9 @@ namespace DB class TestKeeperStorageSerializer { public: - void serialize(const TestKeeperStorage & storage, WriteBuffer & out) const; + static void serialize(const TestKeeperStorage & storage, WriteBuffer & out); - void deserialize(TestKeeperStorage & storage, ReadBuffer & in) const; + static void deserialize(TestKeeperStorage & storage, ReadBuffer & in); }; } diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index d74eaafba27..b0fcec7e10d 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -379,7 +379,7 @@ TEST(CoordinationTest, TestNuKeeperRaft) EXPECT_TRUE(ret_leader->get_accepted()) << "failed to replicate create entry:" << ret_leader->get_result_code(); EXPECT_EQ(ret_leader->get_result_code(), nuraft::cmd_result_code::OK) << "failed to replicate create entry:" << ret_leader->get_result_code(); - auto result = ret_leader.get(); + auto * result = ret_leader.get(); auto responses = getZooKeeperResponses(result->get(), create_request); @@ -418,7 +418,7 @@ TEST(CoordinationTest, TestNuKeeperRaft) EXPECT_TRUE(ret_leader_get->get_accepted()) << "failed to replicate create entry: " << ret_leader_get->get_result_code(); EXPECT_EQ(ret_leader_get->get_result_code(), nuraft::cmd_result_code::OK) << "failed to replicate create entry: " << ret_leader_get->get_result_code(); - auto result_get = ret_leader_get.get(); + auto * result_get = ret_leader_get.get(); auto get_responses = getZooKeeperResponses(result_get->get(), get_request); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index ee5be5f6edb..0b381cf3fae 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1575,7 +1575,7 @@ void Context::initializeTestKeeperStorageDispatcher() const if (shared->test_keeper_storage_dispatcher) throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to initialize TestKeeper multiple times"); - auto & config = getConfigRef(); + const auto & config = getConfigRef(); if (config.has("test_keeper_server")) { shared->test_keeper_storage_dispatcher = std::make_shared(); From 61d006cbab6609c2cbde732546d05ee98980f3c2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 26 Jan 2021 11:18:00 +0300 Subject: [PATCH 27/86] Fix typo --- src/Server/TestKeeperTCPHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/TestKeeperTCPHandler.cpp b/src/Server/TestKeeperTCPHandler.cpp index 3e88d543112..81eaee3382c 100644 --- a/src/Server/TestKeeperTCPHandler.cpp +++ b/src/Server/TestKeeperTCPHandler.cpp @@ -425,7 +425,7 @@ std::pair TestKeeperTCPHandler::receiveR request->readImpl(*in); if (!test_keeper_storage_dispatcher->putRequest(request, session_id)) - throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Sesssion {} already disconnected", session_id); + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Session {} already disconnected", session_id); return std::make_pair(opnum, xid); } From a65430fcee7f4e0f25bd91a3f554f78963e63bf8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 26 Jan 2021 11:33:16 +0300 Subject: [PATCH 28/86] Trying to fix fast test --- contrib/nuraft-cmake/CMakeLists.txt | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/contrib/nuraft-cmake/CMakeLists.txt b/contrib/nuraft-cmake/CMakeLists.txt index e5bb7f7d11b..83137fe73bf 100644 --- a/contrib/nuraft-cmake/CMakeLists.txt +++ b/contrib/nuraft-cmake/CMakeLists.txt @@ -30,7 +30,12 @@ set(SRCS add_library(nuraft ${SRCS}) -target_compile_definitions(nuraft PRIVATE USE_BOOST_ASIO=1 BOOST_ASIO_STANDALONE=1) + +if (NOT OPENSSL_SSL_LIBRARY OR NOT OPENSSL_CRYPTO_LIBRARY) + target_compile_definitions(nuraft PRIVATE USE_BOOST_ASIO=1 BOOST_ASIO_STANDALONE=1 SSL_LIBRARY_NOT_FOUND=1) +else() + target_compile_definitions(nuraft PRIVATE USE_BOOST_ASIO=1 BOOST_ASIO_STANDALONE=1) +endif() target_include_directories (nuraft SYSTEM PRIVATE ${LIBRARY_DIR}/include/libnuraft) # for some reason include "asio.h" directly without "boost/" prefix. From 45192a2ef2ec24a3dd2d7c34a68685e4378d0f21 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 26 Jan 2021 11:46:05 +0300 Subject: [PATCH 29/86] Fix epoll events in boost asio for msan --- contrib/boost | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/boost b/contrib/boost index 8e259cd2a6b..b2368f43f37 160000 --- a/contrib/boost +++ b/contrib/boost @@ -1 +1 @@ -Subproject commit 8e259cd2a6b60d75dd17e73432f11bb7b9351bb1 +Subproject commit b2368f43f37c4a592b17b1e9a474b93749c47319 From e8a320cfd0d449f9a1118c751c94b913ba257407 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 26 Jan 2021 14:10:44 +0300 Subject: [PATCH 30/86] Fix more warnings --- src/Coordination/InMemoryLogStore.h | 2 +- src/Coordination/SummingStateMachine.cpp | 1 + src/Coordination/TestKeeperStorage.h | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Coordination/InMemoryLogStore.h b/src/Coordination/InMemoryLogStore.h index e9c41b50cf6..37f76f056ba 100644 --- a/src/Coordination/InMemoryLogStore.h +++ b/src/Coordination/InMemoryLogStore.h @@ -39,7 +39,7 @@ public: bool flush() override { return true; } private: - std::map> logs; + std::map> logs; mutable std::mutex logs_lock; std::atomic start_idx; }; diff --git a/src/Coordination/SummingStateMachine.cpp b/src/Coordination/SummingStateMachine.cpp index f9a3f4f9de2..0cb7a7da6c3 100644 --- a/src/Coordination/SummingStateMachine.cpp +++ b/src/Coordination/SummingStateMachine.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB { diff --git a/src/Coordination/TestKeeperStorage.h b/src/Coordination/TestKeeperStorage.h index 2c7c6bad4fa..6f70ff1c584 100644 --- a/src/Coordination/TestKeeperStorage.h +++ b/src/Coordination/TestKeeperStorage.h @@ -24,7 +24,7 @@ public: struct Node { String data; - Coordination::ACLs acls; + Coordination::ACLs acls{}; bool is_ephemeral = false; bool is_sequental = false; Coordination::Stat stat{}; From 817eb100a186e1244f51247d7b83956152c6c8da Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 26 Jan 2021 17:08:31 +0300 Subject: [PATCH 31/86] Better shutdown --- src/Coordination/NuKeeperServer.cpp | 12 +++- .../TestKeeperStorageDispatcher.cpp | 65 ++++++++++--------- .../TestKeeperStorageDispatcher.h | 6 +- src/Interpreters/Context.cpp | 2 +- 4 files changed, 48 insertions(+), 37 deletions(-) diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index 5b5aeb206c4..6d70eff1121 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -72,7 +72,17 @@ TestKeeperStorage::ResponsesForSessions NuKeeperServer::shutdown(const TestKeepe { TestKeeperStorage::ResponsesForSessions responses; if (can_become_leader) - responses = putRequests(expired_requests); + { + try + { + responses = putRequests(expired_requests); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + if (!launcher.shutdown(5)) LOG_WARNING(&Poco::Logger::get("NuKeeperServer"), "Failed to shutdown RAFT server in {} seconds", 5); return responses; diff --git a/src/Coordination/TestKeeperStorageDispatcher.cpp b/src/Coordination/TestKeeperStorageDispatcher.cpp index 3aef5213adc..7ce81df0bfd 100644 --- a/src/Coordination/TestKeeperStorageDispatcher.cpp +++ b/src/Coordination/TestKeeperStorageDispatcher.cpp @@ -14,7 +14,7 @@ namespace ErrorCodes void TestKeeperStorageDispatcher::processingThread() { setThreadName("TestKeeperSProc"); - while (!shutdown) + while (!shutdown_called) { TestKeeperStorage::RequestForSession request; @@ -22,8 +22,9 @@ void TestKeeperStorageDispatcher::processingThread() if (requests_queue.tryPop(request, max_wait)) { - if (shutdown) + if (shutdown_called) break; + try { auto responses = server->putRequests({request}); @@ -51,34 +52,6 @@ void TestKeeperStorageDispatcher::setResponse(int64_t session_id, const Coordina session_to_response_callback.erase(session_writer); } -void TestKeeperStorageDispatcher::finalize() -{ - { - std::lock_guard lock(push_request_mutex); - - if (shutdown) - return; - - shutdown = true; - - if (processing_thread.joinable()) - processing_thread.join(); - } - - if (server) - { - TestKeeperStorage::RequestsForSessions expired_requests; - TestKeeperStorage::RequestForSession request; - while (requests_queue.tryPop(request)) - expired_requests.push_back(TestKeeperStorage::RequestForSession{request}); - - auto expired_responses = server->shutdown(expired_requests); - - for (const auto & response_for_session : expired_responses) - setResponse(response_for_session.session_id, response_for_session.response); - } -} - bool TestKeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id) { @@ -143,11 +116,34 @@ void TestKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigura } -TestKeeperStorageDispatcher::~TestKeeperStorageDispatcher() +void TestKeeperStorageDispatcher::shutdown() { try { - finalize(); + { + std::lock_guard lock(push_request_mutex); + + if (shutdown_called) + return; + + shutdown_called = true; + + if (processing_thread.joinable()) + processing_thread.join(); + } + + if (server) + { + TestKeeperStorage::RequestsForSessions expired_requests; + TestKeeperStorage::RequestForSession request; + while (requests_queue.tryPop(request)) + expired_requests.push_back(TestKeeperStorage::RequestForSession{request}); + + auto expired_responses = server->shutdown(expired_requests); + + for (const auto & response_for_session : expired_responses) + setResponse(response_for_session.session_id, response_for_session.response); + } } catch (...) { @@ -155,6 +151,11 @@ TestKeeperStorageDispatcher::~TestKeeperStorageDispatcher() } } +TestKeeperStorageDispatcher::~TestKeeperStorageDispatcher() +{ + shutdown(); +} + void TestKeeperStorageDispatcher::registerSession(int64_t session_id, ZooKeeperResponseCallback callback) { std::lock_guard lock(session_to_response_callback_mutex); diff --git a/src/Coordination/TestKeeperStorageDispatcher.h b/src/Coordination/TestKeeperStorageDispatcher.h index aa220beecf2..5107f2f9cba 100644 --- a/src/Coordination/TestKeeperStorageDispatcher.h +++ b/src/Coordination/TestKeeperStorageDispatcher.h @@ -16,13 +16,12 @@ class TestKeeperStorageDispatcher private: Poco::Timespan operation_timeout{0, Coordination::DEFAULT_OPERATION_TIMEOUT_MS * 1000}; - using clock = std::chrono::steady_clock; std::mutex push_request_mutex; using RequestsQueue = ConcurrentBoundedQueue; RequestsQueue requests_queue{1}; - std::atomic shutdown{false}; + std::atomic shutdown_called{false}; using SessionToResponseCallback = std::unordered_map; std::mutex session_to_response_callback_mutex; @@ -35,7 +34,6 @@ private: private: void processingThread(); - void finalize(); void setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response); public: @@ -43,6 +41,8 @@ public: void initialize(const Poco::Util::AbstractConfiguration & config); + void shutdown(); + ~TestKeeperStorageDispatcher(); bool putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 0b381cf3fae..033f4b54a64 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -447,7 +447,7 @@ struct ContextShared /// Stop zookeeper connection zookeeper.reset(); /// Stop test_keeper storage - test_keeper_storage_dispatcher.reset(); + test_keeper_storage_dispatcher->shutdown(); } bool hasTraceCollector() const From 3935d51b14813e6ad2563eaf72b1a17b7f15f7b4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 26 Jan 2021 17:23:10 +0300 Subject: [PATCH 32/86] Fix segfault --- src/Interpreters/Context.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 033f4b54a64..4c396bd29f4 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -447,7 +447,8 @@ struct ContextShared /// Stop zookeeper connection zookeeper.reset(); /// Stop test_keeper storage - test_keeper_storage_dispatcher->shutdown(); + if (test_keeper_storage_dispatcher) + test_keeper_storage_dispatcher->shutdown(); } bool hasTraceCollector() const From 241d3ec8c275029cbe150746745377b3af1ef703 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 27 Jan 2021 15:40:16 +0300 Subject: [PATCH 33/86] Merge with master --- .../ZooKeeper/TestKeeperStorageDispatcher.cpp | 139 ------------------ 1 file changed, 139 deletions(-) delete mode 100644 src/Common/ZooKeeper/TestKeeperStorageDispatcher.cpp diff --git a/src/Common/ZooKeeper/TestKeeperStorageDispatcher.cpp b/src/Common/ZooKeeper/TestKeeperStorageDispatcher.cpp deleted file mode 100644 index 434a6a2e747..00000000000 --- a/src/Common/ZooKeeper/TestKeeperStorageDispatcher.cpp +++ /dev/null @@ -1,139 +0,0 @@ -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - - extern const int LOGICAL_ERROR; - extern const int TIMEOUT_EXCEEDED; -} - -} -namespace zkutil -{ - -void TestKeeperStorageDispatcher::processingThread() -{ - setThreadName("TestKeeperSProc"); - - while (!shutdown) - { - RequestInfo info; - - UInt64 max_wait = UInt64(operation_timeout.totalMilliseconds()); - - if (requests_queue.tryPop(info, max_wait)) - { - if (shutdown) - break; - - try - { - auto responses = storage.processRequest(info.request, info.session_id); - for (const auto & response_for_session : responses) - setResponse(response_for_session.session_id, response_for_session.response); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - } -} - -void TestKeeperStorageDispatcher::setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response) -{ - std::lock_guard lock(session_to_response_callback_mutex); - auto session_writer = session_to_response_callback.find(session_id); - if (session_writer == session_to_response_callback.end()) - return; - - session_writer->second(response); - /// Session closed, no more writes - if (response->xid != Coordination::WATCH_XID && response->getOpNum() == Coordination::OpNum::Close) - session_to_response_callback.erase(session_writer); -} - -void TestKeeperStorageDispatcher::finalize() -{ - { - std::lock_guard lock(push_request_mutex); - - if (shutdown) - return; - - shutdown = true; - - if (processing_thread.joinable()) - processing_thread.join(); - } - - RequestInfo info; - TestKeeperStorage::RequestsForSessions expired_requests; - while (requests_queue.tryPop(info)) - expired_requests.push_back(TestKeeperStorage::RequestForSession{info.session_id, info.request}); - - auto expired_responses = storage.finalize(expired_requests); - - for (const auto & response_for_session : expired_responses) - setResponse(response_for_session.session_id, response_for_session.response); -} - -void TestKeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id) -{ - - { - std::lock_guard lock(session_to_response_callback_mutex); - if (session_to_response_callback.count(session_id) == 0) - throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unknown session id {}", session_id); - } - - RequestInfo request_info; - request_info.time = clock::now(); - request_info.request = request; - request_info.session_id = session_id; - - std::lock_guard lock(push_request_mutex); - /// Put close requests without timeouts - if (request->getOpNum() == Coordination::OpNum::Close) - requests_queue.push(std::move(request_info)); - else if (!requests_queue.tryPush(std::move(request_info), operation_timeout.totalMilliseconds())) - throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::TIMEOUT_EXCEEDED); -} - -TestKeeperStorageDispatcher::TestKeeperStorageDispatcher() -{ - processing_thread = ThreadFromGlobalPool([this] { processingThread(); }); -} - -TestKeeperStorageDispatcher::~TestKeeperStorageDispatcher() -{ - try - { - finalize(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } -} - -void TestKeeperStorageDispatcher::registerSession(int64_t session_id, ZooKeeperResponseCallback callback) -{ - std::lock_guard lock(session_to_response_callback_mutex); - if (!session_to_response_callback.try_emplace(session_id, callback).second) - throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Session with id {} already registered in dispatcher", session_id); -} - -void TestKeeperStorageDispatcher::finishSession(int64_t session_id) -{ - std::lock_guard lock(session_to_response_callback_mutex); - auto session_it = session_to_response_callback.find(session_id); - if (session_it != session_to_response_callback.end()) - session_to_response_callback.erase(session_it); -} - -} From 7af28e758a5bab37e540d7e9f32a0dea23168753 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 27 Jan 2021 16:37:58 +0300 Subject: [PATCH 34/86] Process read requests without raft --- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 6 +++ src/Common/ZooKeeper/ZooKeeperCommon.h | 12 ++++++ src/Coordination/NuKeeperServer.cpp | 49 +++++++++++++---------- src/Coordination/NuKeeperStateMachine.cpp | 6 +++ src/Coordination/NuKeeperStateMachine.h | 2 + 5 files changed, 54 insertions(+), 21 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index 9c699ee298a..278d36f9245 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -326,6 +326,12 @@ void ZooKeeperMultiRequest::readImpl(ReadBuffer & in) } } +bool ZooKeeperMultiRequest::isReadRequest() const +{ + /// Possibly we can do better + return false; +} + void ZooKeeperMultiResponse::readImpl(ReadBuffer & in) { for (auto & response : responses) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index 9adb0c06e4c..b2c18c31798 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -60,6 +60,7 @@ struct ZooKeeperRequest : virtual Request static std::shared_ptr read(ReadBuffer & in); virtual ZooKeeperResponsePtr makeResponse() const = 0; + virtual bool isReadRequest() const = 0; }; using ZooKeeperRequestPtr = std::shared_ptr; @@ -71,6 +72,7 @@ struct ZooKeeperHeartbeatRequest final : ZooKeeperRequest void writeImpl(WriteBuffer &) const override {} void readImpl(ReadBuffer &) override {} ZooKeeperResponsePtr makeResponse() const override; + bool isReadRequest() const override { return true; } }; struct ZooKeeperHeartbeatResponse final : ZooKeeperResponse @@ -104,6 +106,7 @@ struct ZooKeeperAuthRequest final : ZooKeeperRequest void readImpl(ReadBuffer & in) override; ZooKeeperResponsePtr makeResponse() const override; + bool isReadRequest() const override { return false; } }; struct ZooKeeperAuthResponse final : ZooKeeperResponse @@ -122,6 +125,7 @@ struct ZooKeeperCloseRequest final : ZooKeeperRequest void readImpl(ReadBuffer &) override {} ZooKeeperResponsePtr makeResponse() const override; + bool isReadRequest() const override { return false; } }; struct ZooKeeperCloseResponse final : ZooKeeperResponse @@ -146,6 +150,7 @@ struct ZooKeeperCreateRequest final : public CreateRequest, ZooKeeperRequest void readImpl(ReadBuffer & in) override; ZooKeeperResponsePtr makeResponse() const override; + bool isReadRequest() const override { return false; } }; struct ZooKeeperCreateResponse final : CreateResponse, ZooKeeperResponse @@ -167,6 +172,7 @@ struct ZooKeeperRemoveRequest final : RemoveRequest, ZooKeeperRequest void readImpl(ReadBuffer & in) override; ZooKeeperResponsePtr makeResponse() const override; + bool isReadRequest() const override { return false; } }; struct ZooKeeperRemoveResponse final : RemoveResponse, ZooKeeperResponse @@ -183,6 +189,7 @@ struct ZooKeeperExistsRequest final : ExistsRequest, ZooKeeperRequest void readImpl(ReadBuffer & in) override; ZooKeeperResponsePtr makeResponse() const override; + bool isReadRequest() const override { return !has_watch; } }; struct ZooKeeperExistsResponse final : ExistsResponse, ZooKeeperResponse @@ -199,6 +206,7 @@ struct ZooKeeperGetRequest final : GetRequest, ZooKeeperRequest void readImpl(ReadBuffer & in) override; ZooKeeperResponsePtr makeResponse() const override; + bool isReadRequest() const override { return !has_watch; } }; struct ZooKeeperGetResponse final : GetResponse, ZooKeeperResponse @@ -217,6 +225,7 @@ struct ZooKeeperSetRequest final : SetRequest, ZooKeeperRequest void writeImpl(WriteBuffer & out) const override; void readImpl(ReadBuffer & in) override; ZooKeeperResponsePtr makeResponse() const override; + bool isReadRequest() const override { return false; } }; struct ZooKeeperSetResponse final : SetResponse, ZooKeeperResponse @@ -232,6 +241,7 @@ struct ZooKeeperListRequest : ListRequest, ZooKeeperRequest void writeImpl(WriteBuffer & out) const override; void readImpl(ReadBuffer & in) override; ZooKeeperResponsePtr makeResponse() const override; + bool isReadRequest() const override { return !has_watch; } }; struct ZooKeeperSimpleListRequest final : ZooKeeperListRequest @@ -261,6 +271,7 @@ struct ZooKeeperCheckRequest final : CheckRequest, ZooKeeperRequest void readImpl(ReadBuffer & in) override; ZooKeeperResponsePtr makeResponse() const override; + bool isReadRequest() const override { return !has_watch; } }; struct ZooKeeperCheckResponse final : CheckResponse, ZooKeeperResponse @@ -290,6 +301,7 @@ struct ZooKeeperMultiRequest final : MultiRequest, ZooKeeperRequest void readImpl(ReadBuffer & in) override; ZooKeeperResponsePtr makeResponse() const override; + bool isReadRequest() const override; }; struct ZooKeeperMultiResponse final : MultiResponse, ZooKeeperResponse diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index 6d70eff1121..8b8288424d9 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -146,34 +146,41 @@ TestKeeperStorage::ResponsesForSessions NuKeeperServer::readZooKeeperResponses(n TestKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const TestKeeperStorage::RequestsForSessions & requests) { - std::vector> entries; - for (const auto & [session_id, request] : requests) + if (requests.size() == 1 && requests[0].request->isReadRequest()) { - ops_mapping[session_id][request->xid] = request->makeResponse(); - entries.push_back(getZooKeeperLogEntry(session_id, request)); + return state_machine->processReadRequest(requests[0]); } - - auto result = raft_instance->append_entries(entries); - if (!result->get_accepted()) - throw Exception(ErrorCodes::RAFT_ERROR, "Cannot send requests to RAFT, mostly because we are not leader, code {}, message: '{}'", result->get_result_code(), result->get_result_str()); - - if (result->get_result_code() == nuraft::cmd_result_code::TIMEOUT) + else { - TestKeeperStorage::ResponsesForSessions responses; + std::vector> entries; for (const auto & [session_id, request] : requests) { - auto response = request->makeResponse(); - response->xid = request->xid; - response->zxid = 0; /// FIXME what we can do with it? - response->error = Coordination::Error::ZOPERATIONTIMEOUT; - responses.push_back(DB::TestKeeperStorage::ResponseForSession{session_id, response}); + ops_mapping[session_id][request->xid] = request->makeResponse(); + entries.push_back(getZooKeeperLogEntry(session_id, request)); } - return responses; - } - else if (result->get_result_code() != nuraft::cmd_result_code::OK) - throw Exception(ErrorCodes::RAFT_ERROR, "Requests result failed with code {} and message: '{}'", result->get_result_code(), result->get_result_str()); - return readZooKeeperResponses(result->get()); + auto result = raft_instance->append_entries(entries); + if (!result->get_accepted()) + throw Exception(ErrorCodes::RAFT_ERROR, "Cannot send requests to RAFT, mostly because we are not leader, code {}, message: '{}'", result->get_result_code(), result->get_result_str()); + + if (result->get_result_code() == nuraft::cmd_result_code::TIMEOUT) + { + TestKeeperStorage::ResponsesForSessions responses; + for (const auto & [session_id, request] : requests) + { + auto response = request->makeResponse(); + response->xid = request->xid; + response->zxid = 0; /// FIXME what we can do with it? + response->error = Coordination::Error::ZOPERATIONTIMEOUT; + responses.push_back(DB::TestKeeperStorage::ResponseForSession{session_id, response}); + } + return responses; + } + else if (result->get_result_code() != nuraft::cmd_result_code::OK) + throw Exception(ErrorCodes::RAFT_ERROR, "Requests result failed with code {} and message: '{}'", result->get_result_code(), result->get_result_str()); + + return readZooKeeperResponses(result->get()); + } } diff --git a/src/Coordination/NuKeeperStateMachine.cpp b/src/Coordination/NuKeeperStateMachine.cpp index 52c82f44784..9f4572c02e0 100644 --- a/src/Coordination/NuKeeperStateMachine.cpp +++ b/src/Coordination/NuKeeperStateMachine.cpp @@ -223,4 +223,10 @@ int NuKeeperStateMachine::read_logical_snp_obj( return 0; } +TestKeeperStorage::ResponsesForSessions NuKeeperStateMachine::processReadRequest(const TestKeeperStorage::RequestForSession & request_for_session) +{ + std::lock_guard lock(storage_lock); + return storage.processRequest(request_for_session.request, request_for_session.session_id); +} + } diff --git a/src/Coordination/NuKeeperStateMachine.h b/src/Coordination/NuKeeperStateMachine.h index a120e3f1cf6..368e088a2f9 100644 --- a/src/Coordination/NuKeeperStateMachine.h +++ b/src/Coordination/NuKeeperStateMachine.h @@ -47,6 +47,8 @@ public: return storage; } + TestKeeperStorage::ResponsesForSessions processReadRequest(const TestKeeperStorage::RequestForSession & requests); + private: struct StorageSnapshot { From a33963e211d305edc80d453a75bff2c7347ec5c0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 27 Jan 2021 20:54:25 +0300 Subject: [PATCH 35/86] Better raft server startup --- programs/server/Server.cpp | 2 + src/Coordination/NuKeeperServer.cpp | 56 ++++++++++++++++--- src/Coordination/NuKeeperServer.h | 12 ++-- src/Coordination/NuKeeperStateMachine.h | 2 +- .../TestKeeperStorageDispatcher.cpp | 26 +++++++-- .../TestKeeperStorageDispatcher.h | 5 ++ src/Interpreters/Context.cpp | 14 ++++- src/Interpreters/Context.h | 1 + src/Server/TestKeeperTCPHandler.cpp | 21 +++++-- src/Server/TestKeeperTCPHandler.h | 2 +- .../configs/use_test_keeper.xml | 8 +++ 11 files changed, 124 insertions(+), 25 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 04919e8504c..fefabd8be71 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -904,6 +904,8 @@ int Server::main(const std::vector & /*args*/) LOG_INFO(log, "Closed connections to servers for tables. But {} remain. Probably some tables of other users cannot finish their connections after context shutdown.", current_connections); else LOG_INFO(log, "Closed connections to servers for tables."); + + global_context->shutdownTestKeeperStorageDispatcher(); } /** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available. diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index 8b8288424d9..a005febd67d 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -19,22 +19,22 @@ namespace ErrorCodes extern const int RAFT_ERROR; } -NuKeeperServer::NuKeeperServer(int server_id_, const std::string & hostname_, int port_, bool can_become_leader_) +NuKeeperServer::NuKeeperServer(int server_id_, const std::string & hostname_, int port_) : server_id(server_id_) , hostname(hostname_) , port(port_) , endpoint(hostname + ":" + std::to_string(port)) - , can_become_leader(can_become_leader_) , state_machine(nuraft::cs_new()) , state_manager(nuraft::cs_new(server_id, endpoint)) { } -bool NuKeeperServer::addServer(int server_id_, const std::string & server_uri_, bool can_become_leader_) +void NuKeeperServer::addServer(int server_id_, const std::string & server_uri_, bool can_become_leader_) { nuraft::srv_config config(server_id_, 0, server_uri_, "", /*FIXME follower=*/ !can_become_leader_); auto ret1 = raft_instance->add_srv(config); - return ret1->get_result_code() == nuraft::cmd_result_code::OK; + if (ret1->get_result_code() != nuraft::cmd_result_code::OK) + throw Exception(ErrorCodes::RAFT_ERROR, "Cannot add server to RAFT quorum with code {}, message '{}'", ret1->get_result_code(), ret1->get_result_str()); } @@ -71,7 +71,7 @@ void NuKeeperServer::startup() TestKeeperStorage::ResponsesForSessions NuKeeperServer::shutdown(const TestKeeperStorage::RequestsForSessions & expired_requests) { TestKeeperStorage::ResponsesForSessions responses; - if (can_become_leader) + if (isLeader()) { try { @@ -161,7 +161,18 @@ TestKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const TestKe auto result = raft_instance->append_entries(entries); if (!result->get_accepted()) - throw Exception(ErrorCodes::RAFT_ERROR, "Cannot send requests to RAFT, mostly because we are not leader, code {}, message: '{}'", result->get_result_code(), result->get_result_str()); + { + TestKeeperStorage::ResponsesForSessions responses; + for (const auto & [session_id, request] : requests) + { + auto response = request->makeResponse(); + response->xid = request->xid; + response->zxid = 0; /// FIXME what we can do with it? + response->error = Coordination::Error::ZSESSIONEXPIRED; + responses.push_back(DB::TestKeeperStorage::ResponseForSession{session_id, response}); + } + return responses; + } if (result->get_result_code() == nuraft::cmd_result_code::TIMEOUT) { @@ -183,7 +194,6 @@ TestKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const TestKe } } - int64_t NuKeeperServer::getSessionID() { auto entry = nuraft::buffer::alloc(sizeof(int64_t)); @@ -203,4 +213,36 @@ int64_t NuKeeperServer::getSessionID() return bs_resp.get_i64(); } +bool NuKeeperServer::isLeader() const +{ + return raft_instance->is_leader(); +} + +bool NuKeeperServer::waitForServer(int32_t id) const +{ + for (size_t i = 0; i < 10; ++i) + { + if (raft_instance->get_srv_config(id) != nullptr) + return true; + LOG_DEBUG(&Poco::Logger::get("NuRaftInit"), "Waiting for server {} to join the cluster", id); + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + return false; +} + +void NuKeeperServer::waitForServers(const std::vector & ids) const +{ + for (int32_t id : ids) + waitForServer(id); +} + +void NuKeeperServer::waitForCatchUp() const +{ + while (raft_instance->is_catching_up() || raft_instance->is_receiving_snapshot()) + { + LOG_DEBUG(&Poco::Logger::get("NuRaftInit"), "Waiting current RAFT instance to catch up"); + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } +} + } diff --git a/src/Coordination/NuKeeperServer.h b/src/Coordination/NuKeeperServer.h index 4c10614cd5c..b9488cafc69 100644 --- a/src/Coordination/NuKeeperServer.h +++ b/src/Coordination/NuKeeperServer.h @@ -21,8 +21,6 @@ private: std::string endpoint; - bool can_become_leader; - nuraft::ptr state_machine; nuraft::ptr state_manager; @@ -40,7 +38,7 @@ private: TestKeeperStorage::ResponsesForSessions readZooKeeperResponses(nuraft::ptr & buffer); public: - NuKeeperServer(int server_id_, const std::string & hostname_, int port_, bool can_become_leader_); + NuKeeperServer(int server_id_, const std::string & hostname_, int port_); void startup(); @@ -48,7 +46,13 @@ public: int64_t getSessionID(); - bool addServer(int server_id_, const std::string & server_uri, bool can_become_leader_); + void addServer(int server_id_, const std::string & server_uri, bool can_become_leader_); + + bool isLeader() const; + + bool waitForServer(int32_t server_id) const; + void waitForServers(const std::vector & ids) const; + void waitForCatchUp() const; TestKeeperStorage::ResponsesForSessions shutdown(const TestKeeperStorage::RequestsForSessions & expired_requests); }; diff --git a/src/Coordination/NuKeeperStateMachine.h b/src/Coordination/NuKeeperStateMachine.h index 368e088a2f9..7767f552cec 100644 --- a/src/Coordination/NuKeeperStateMachine.h +++ b/src/Coordination/NuKeeperStateMachine.h @@ -47,7 +47,7 @@ public: return storage; } - TestKeeperStorage::ResponsesForSessions processReadRequest(const TestKeeperStorage::RequestForSession & requests); + TestKeeperStorage::ResponsesForSessions processReadRequest(const TestKeeperStorage::RequestForSession & request_for_session); private: struct StorageSnapshot diff --git a/src/Coordination/TestKeeperStorageDispatcher.cpp b/src/Coordination/TestKeeperStorageDispatcher.cpp index 7ce81df0bfd..f6ca389f2cf 100644 --- a/src/Coordination/TestKeeperStorageDispatcher.cpp +++ b/src/Coordination/TestKeeperStorageDispatcher.cpp @@ -86,6 +86,7 @@ void TestKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigura bool my_can_become_leader = true; std::vector> server_configs; + std::vector ids; for (const auto & server_key : keys) { int server_id = config.getInt("test_keeper_server.raft_configuration." + server_key + ".id"); @@ -102,14 +103,26 @@ void TestKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigura { server_configs.emplace_back(server_id, hostname, port, can_become_leader); } + ids.push_back(server_id); } - server = std::make_unique(myid, myhostname, myport, my_can_become_leader); + server = std::make_unique(myid, myhostname, myport); server->startup(); if (my_can_become_leader) { for (const auto & [id, hostname, port, can_become_leader] : server_configs) - server->addServer(id, hostname + ":" + std::to_string(port), can_become_leader); + { + do + { + server->addServer(id, hostname + ":" + std::to_string(port), can_become_leader); + } + while (!server->waitForServer(id)); + } + } + else + { + server->waitForServers(ids); + server->waitForCatchUp(); } processing_thread = ThreadFromGlobalPool([this] { processingThread(); }); @@ -135,9 +148,12 @@ void TestKeeperStorageDispatcher::shutdown() if (server) { TestKeeperStorage::RequestsForSessions expired_requests; - TestKeeperStorage::RequestForSession request; - while (requests_queue.tryPop(request)) - expired_requests.push_back(TestKeeperStorage::RequestForSession{request}); + if (server->isLeader()) + { + TestKeeperStorage::RequestForSession request; + while (requests_queue.tryPop(request)) + expired_requests.push_back(TestKeeperStorage::RequestForSession{request}); + } auto expired_responses = server->shutdown(expired_requests); diff --git a/src/Coordination/TestKeeperStorageDispatcher.h b/src/Coordination/TestKeeperStorageDispatcher.h index 5107f2f9cba..a6c6118f9c4 100644 --- a/src/Coordination/TestKeeperStorageDispatcher.h +++ b/src/Coordination/TestKeeperStorageDispatcher.h @@ -47,6 +47,11 @@ public: bool putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id); + bool isLeader() const + { + return server->isLeader(); + } + int64_t getSessionID() { std::lock_guard lock(session_id_mutex); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 4c396bd29f4..fc8d8654573 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -446,9 +446,7 @@ struct ContextShared trace_collector.reset(); /// Stop zookeeper connection zookeeper.reset(); - /// Stop test_keeper storage - if (test_keeper_storage_dispatcher) - test_keeper_storage_dispatcher->shutdown(); + } bool hasTraceCollector() const @@ -1593,6 +1591,16 @@ std::shared_ptr & Context::getTestKeeperStorageDisp return shared->test_keeper_storage_dispatcher; } +void Context::shutdownTestKeeperStorageDispatcher() const +{ + std::lock_guard lock(shared->test_keeper_storage_dispatcher_mutex); + if (shared->test_keeper_storage_dispatcher) + { + shared->test_keeper_storage_dispatcher->shutdown(); + shared->test_keeper_storage_dispatcher.reset(); + } +} + zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const { std::lock_guard lock(shared->auxiliary_zookeepers_mutex); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 537ddcc0ec8..e643c80183c 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -576,6 +576,7 @@ public: void initializeTestKeeperStorageDispatcher() const; std::shared_ptr & getTestKeeperStorageDispatcher() const; + void shutdownTestKeeperStorageDispatcher() const; /// Set auxiliary zookeepers configuration at server starting or configuration reloading. void reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & config); diff --git a/src/Server/TestKeeperTCPHandler.cpp b/src/Server/TestKeeperTCPHandler.cpp index 81eaee3382c..04e5c6ece1d 100644 --- a/src/Server/TestKeeperTCPHandler.cpp +++ b/src/Server/TestKeeperTCPHandler.cpp @@ -227,16 +227,19 @@ TestKeeperTCPHandler::TestKeeperTCPHandler(IServer & server_, const Poco::Net::S , test_keeper_storage_dispatcher(global_context.getTestKeeperStorageDispatcher()) , operation_timeout(0, global_context.getConfigRef().getUInt("test_keeper_server.operation_timeout_ms", Coordination::DEFAULT_OPERATION_TIMEOUT_MS) * 1000) , session_timeout(0, global_context.getConfigRef().getUInt("test_keeper_server.session_timeout_ms", Coordination::DEFAULT_SESSION_TIMEOUT_MS) * 1000) - , session_id(test_keeper_storage_dispatcher->getSessionID()) , poll_wrapper(std::make_unique(socket_)) , responses(std::make_unique()) { } -void TestKeeperTCPHandler::sendHandshake() +void TestKeeperTCPHandler::sendHandshake(bool is_leader) { Coordination::write(Coordination::SERVER_HANDSHAKE_LENGTH, *out); - Coordination::write(Coordination::ZOOKEEPER_PROTOCOL_VERSION, *out); + if (is_leader) + Coordination::write(Coordination::ZOOKEEPER_PROTOCOL_VERSION, *out); + else /// Specially ignore connections if we are not leader, client will throw exception + Coordination::write(42, *out); + Coordination::write(Coordination::DEFAULT_SESSION_TIMEOUT_MS, *out); Coordination::write(session_id, *out); std::array passwd{}; @@ -316,7 +319,17 @@ void TestKeeperTCPHandler::runImpl() return; } - sendHandshake(); + if (test_keeper_storage_dispatcher->isLeader()) + { + session_id = test_keeper_storage_dispatcher->getSessionID(); + sendHandshake(true); + } + else + { + sendHandshake(false); + LOG_WARNING(log, "Ignoring connection because we are not leader"); + return; + } auto response_fd = poll_wrapper->getResponseFD(); auto response_callback = [this, response_fd] (const Coordination::ZooKeeperResponsePtr & response) diff --git a/src/Server/TestKeeperTCPHandler.h b/src/Server/TestKeeperTCPHandler.h index e7372e8dd82..bb74513afce 100644 --- a/src/Server/TestKeeperTCPHandler.h +++ b/src/Server/TestKeeperTCPHandler.h @@ -45,7 +45,7 @@ private: void runImpl(); - void sendHandshake(); + void sendHandshake(bool is_leader); void receiveHandshake(); std::pair receiveRequest(); diff --git a/tests/integration/test_testkeeper_multinode/configs/use_test_keeper.xml b/tests/integration/test_testkeeper_multinode/configs/use_test_keeper.xml index 20d731b8553..b6139005d2f 100644 --- a/tests/integration/test_testkeeper_multinode/configs/use_test_keeper.xml +++ b/tests/integration/test_testkeeper_multinode/configs/use_test_keeper.xml @@ -4,5 +4,13 @@ node1 9181 + + node2 + 9181 + + + node3 + 9181 + From 8d0d2ca8e00324975d6c743e794c4167a0e45c00 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 28 Jan 2021 15:07:26 +0300 Subject: [PATCH 36/86] Add some partition tests --- src/Coordination/NuKeeperServer.cpp | 8 +- src/Coordination/NuKeeperServer.h | 2 +- .../TestKeeperStorageDispatcher.cpp | 37 +++- .../configs/enable_test_keeper1.xml | 7 +- .../configs/enable_test_keeper2.xml | 7 +- .../configs/enable_test_keeper3.xml | 7 +- .../test_testkeeper_multinode/test.py | 172 ++++++++++++++++++ 7 files changed, 224 insertions(+), 16 deletions(-) diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index a005febd67d..8995b51a13b 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -29,9 +29,9 @@ NuKeeperServer::NuKeeperServer(int server_id_, const std::string & hostname_, in { } -void NuKeeperServer::addServer(int server_id_, const std::string & server_uri_, bool can_become_leader_) +void NuKeeperServer::addServer(int server_id_, const std::string & server_uri_, bool can_become_leader_, int32_t priority) { - nuraft::srv_config config(server_id_, 0, server_uri_, "", /*FIXME follower=*/ !can_become_leader_); + nuraft::srv_config config(server_id_, 0, server_uri_, "", /* follower= */ !can_become_leader_, priority); auto ret1 = raft_instance->add_srv(config); if (ret1->get_result_code() != nuraft::cmd_result_code::OK) throw Exception(ErrorCodes::RAFT_ERROR, "Cannot add server to RAFT quorum with code {}, message '{}'", ret1->get_result_code(), ret1->get_result_str()); @@ -146,7 +146,7 @@ TestKeeperStorage::ResponsesForSessions NuKeeperServer::readZooKeeperResponses(n TestKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const TestKeeperStorage::RequestsForSessions & requests) { - if (requests.size() == 1 && requests[0].request->isReadRequest()) + if (isLeader() && requests.size() == 1 && requests[0].request->isReadRequest()) { return state_machine->processReadRequest(requests[0]); } @@ -238,7 +238,7 @@ void NuKeeperServer::waitForServers(const std::vector & ids) const void NuKeeperServer::waitForCatchUp() const { - while (raft_instance->is_catching_up() || raft_instance->is_receiving_snapshot()) + while (raft_instance->is_catching_up() || raft_instance->is_receiving_snapshot() || raft_instance->is_leader()) { LOG_DEBUG(&Poco::Logger::get("NuRaftInit"), "Waiting current RAFT instance to catch up"); std::this_thread::sleep_for(std::chrono::milliseconds(100)); diff --git a/src/Coordination/NuKeeperServer.h b/src/Coordination/NuKeeperServer.h index b9488cafc69..7fd70ac26e2 100644 --- a/src/Coordination/NuKeeperServer.h +++ b/src/Coordination/NuKeeperServer.h @@ -46,7 +46,7 @@ public: int64_t getSessionID(); - void addServer(int server_id_, const std::string & server_uri, bool can_become_leader_); + void addServer(int server_id_, const std::string & server_uri, bool can_become_leader_, int32_t priority); bool isLeader() const; diff --git a/src/Coordination/TestKeeperStorageDispatcher.cpp b/src/Coordination/TestKeeperStorageDispatcher.cpp index f6ca389f2cf..685fa58f8ad 100644 --- a/src/Coordination/TestKeeperStorageDispatcher.cpp +++ b/src/Coordination/TestKeeperStorageDispatcher.cpp @@ -74,18 +74,43 @@ bool TestKeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperReques return true; } +namespace +{ + bool shouldBuildQuorum(int32_t myid, int32_t my_priority, bool my_can_become_leader, const std::vector> & server_configs) + { + if (!my_can_become_leader) + return false; + + int32_t minid = myid; + bool has_equal_priority = false; + for (const auto & [id, hostname, port, can_become_leader, priority] : server_configs) + { + if (my_priority < priority) + return false; + else if (my_priority == priority) + has_equal_priority = true; + minid = std::min(minid, id); + } + + if (has_equal_priority) + return minid == myid; + else + return true; + } +} void TestKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfiguration & config) { int myid = config.getInt("test_keeper_server.server_id"); std::string myhostname; int myport; + int32_t my_priority = 1; Poco::Util::AbstractConfiguration::Keys keys; config.keys("test_keeper_server.raft_configuration", keys); bool my_can_become_leader = true; - std::vector> server_configs; + std::vector> server_configs; std::vector ids; for (const auto & server_key : keys) { @@ -93,28 +118,30 @@ void TestKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigura std::string hostname = config.getString("test_keeper_server.raft_configuration." + server_key + ".hostname"); int port = config.getInt("test_keeper_server.raft_configuration." + server_key + ".port"); bool can_become_leader = config.getBool("test_keeper_server.raft_configuration." + server_key + ".can_become_leader", true); + int32_t priority = config.getInt("test_keeper_server.raft_configuration." + server_key + ".priority", 1); if (server_id == myid) { myhostname = hostname; myport = port; my_can_become_leader = can_become_leader; + my_priority = priority; } else { - server_configs.emplace_back(server_id, hostname, port, can_become_leader); + server_configs.emplace_back(server_id, hostname, port, can_become_leader, priority); } ids.push_back(server_id); } server = std::make_unique(myid, myhostname, myport); server->startup(); - if (my_can_become_leader) + if (shouldBuildQuorum(myid, my_priority, my_can_become_leader, server_configs)) { - for (const auto & [id, hostname, port, can_become_leader] : server_configs) + for (const auto & [id, hostname, port, can_become_leader, priority] : server_configs) { do { - server->addServer(id, hostname + ":" + std::to_string(port), can_become_leader); + server->addServer(id, hostname + ":" + std::to_string(port), can_become_leader, priority); } while (!server->waitForServer(id)); } diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml index 486942aec71..81f68f50c7c 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml @@ -10,18 +10,21 @@ node1 44444 true + 3 2 node2 44444 - false + true + 2 3 node3 44444 - false + true + 1 diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml index 94873883943..73340973367 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml @@ -10,18 +10,21 @@ node1 44444 true + 3 2 node2 44444 - false + true + 2 3 node3 44444 - false + true + 1 diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml index 0219a0e5763..fbc51489d11 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml @@ -10,18 +10,21 @@ node1 44444 true + 3 2 node2 44444 - false + true + 2 3 node3 44444 - false + true + 1 diff --git a/tests/integration/test_testkeeper_multinode/test.py b/tests/integration/test_testkeeper_multinode/test.py index d76e72ee92e..8d35e30400a 100644 --- a/tests/integration/test_testkeeper_multinode/test.py +++ b/tests/integration/test_testkeeper_multinode/test.py @@ -5,6 +5,7 @@ import string import os import time from multiprocessing.dummy import Pool +from helpers.network import PartitionManager cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', main_configs=['configs/enable_test_keeper1.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml']) @@ -23,6 +24,8 @@ def started_cluster(): finally: cluster.shutdown() +def smaller_exception(ex): + return '\n'.join(str(ex).split('\n')[0:2]) def test_simple_replicated_table(started_cluster): @@ -37,3 +40,172 @@ def test_simple_replicated_table(started_cluster): assert node1.query("SELECT COUNT() FROM t") == "10\n" assert node2.query("SELECT COUNT() FROM t") == "10\n" assert node3.query("SELECT COUNT() FROM t") == "10\n" + + + +def test_blocade_leader(started_cluster): + for i, node in enumerate([node1, node2, node3]): + node.query("CREATE TABLE t1 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t1', '{}') ORDER BY tuple()".format(i + 1)) + + node2.query("INSERT INTO t1 SELECT number FROM numbers(10)") + + node1.query("SYSTEM SYNC REPLICA t1", timeout=10) + node3.query("SYSTEM SYNC REPLICA t1", timeout=10) + + assert node1.query("SELECT COUNT() FROM t1") == "10\n" + assert node2.query("SELECT COUNT() FROM t1") == "10\n" + assert node3.query("SELECT COUNT() FROM t1") == "10\n" + + with PartitionManager() as pm: + pm.partition_instances(node2, node1) + pm.partition_instances(node3, node1) + + for i in range(100): + try: + node2.query("INSERT INTO t1 SELECT rand() FROM numbers(100)") + break + except Exception as ex: + print("Got exception node2", smaller_exception(ex)) + time.sleep(0.5) + else: + assert False, "Cannot insert anything node2" + + for i in range(100): + try: + node3.query("INSERT INTO t1 SELECT rand() FROM numbers(100)") + break + except Exception as ex: + print("Got exception node3", smaller_exception(ex)) + time.sleep(0.5) + else: + assert False, "Cannot insert anything node3" + + for n, node in enumerate([node1, node2, node3]): + for i in range(100): + try: + node.query("SYSTEM RESTART REPLICA t1") + break + except Exception as ex: + print("Got exception node{}".format(n + 1), smaller_exception(ex)) + time.sleep(0.5) + else: + assert False, "Cannot reconnect for node{}".format(n + 1) + + for i in range(100): + try: + node1.query("INSERT INTO t1 SELECT rand() FROM numbers(100)") + break + except Exception as ex: + print("Got exception node1", smaller_exception(ex)) + time.sleep(0.5) + else: + assert False, "Cannot insert anything node1" + + for n, node in enumerate([node1, node2, node3]): + for i in range(100): + try: + node.query("SYSTEM SYNC REPLICA t1", timeout=10) + break + except Exception as ex: + print("Got exception node{}".format(n + 1), smaller_exception(ex)) + time.sleep(0.5) + else: + assert False, "Cannot sync replica node{}".format(n+1) + + assert node1.query("SELECT COUNT() FROM t1") == "310\n" + assert node2.query("SELECT COUNT() FROM t1") == "310\n" + assert node3.query("SELECT COUNT() FROM t1") == "310\n" + + +def test_blocade_leader_twice(started_cluster): + for i, node in enumerate([node1, node2, node3]): + node.query("CREATE TABLE t2 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t2', '{}') ORDER BY tuple()".format(i + 1)) + + node2.query("INSERT INTO t2 SELECT number FROM numbers(10)") + + node1.query("SYSTEM SYNC REPLICA t2", timeout=10) + node3.query("SYSTEM SYNC REPLICA t2", timeout=10) + + assert node1.query("SELECT COUNT() FROM t2") == "10\n" + assert node2.query("SELECT COUNT() FROM t2") == "10\n" + assert node3.query("SELECT COUNT() FROM t2") == "10\n" + + with PartitionManager() as pm: + pm.partition_instances(node2, node1) + pm.partition_instances(node3, node1) + + for i in range(100): + try: + node2.query("INSERT INTO t2 SELECT rand() FROM numbers(100)") + break + except Exception as ex: + print("Got exception node2", smaller_exception(ex)) + time.sleep(0.5) + else: + assert False, "Cannot reconnect for node2" + + for i in range(100): + try: + node3.query("INSERT INTO t2 SELECT rand() FROM numbers(100)") + break + except Exception as ex: + print("Got exception node3", smaller_exception(ex)) + time.sleep(0.5) + else: + assert False, "Cannot reconnect for node3" + + + # Total network partition + pm.partition_instances(node3, node2) + + for i in range(30): + try: + node3.query("INSERT INTO t2 SELECT rand() FROM numbers(100)") + assert False, "Node3 became leader?" + except Exception as ex: + time.sleep(0.5) + + for i in range(30): + try: + node2.query("INSERT INTO t2 SELECT rand() FROM numbers(100)") + assert False, "Node2 became leader?" + except Exception as ex: + time.sleep(0.5) + + + for n, node in enumerate([node1, node2, node3]): + for i in range(100): + try: + node.query("SYSTEM RESTART REPLICA t2") + break + except Exception as ex: + print("Got exception node{}".format(n + 1), smaller_exception(ex)) + time.sleep(0.5) + else: + assert False, "Cannot reconnect for node{}".format(n + 1) + + for n, node in enumerate([node1, node2, node3]): + for i in range(100): + try: + node.query("INSERT INTO t2 SELECT rand() FROM numbers(100)") + break + except Exception as ex: + print("Got exception node{}".format(n + 1), smaller_exception(ex)) + time.sleep(0.5) + else: + assert False, "Cannot reconnect for node{}".format(n + 1) + + for node in [node1, node2, node3]: + for i in range(100): + try: + node.query("SYSTEM SYNC REPLICA t2", timeout=10) + break + except Exception as ex: + print("Got exception node{}".format(n + 1), smaller_exception(ex)) + time.sleep(0.5) + else: + assert False, "Cannot reconnect for node{}".format(n + 1) + + assert node1.query("SELECT COUNT() FROM t2") == "510\n" + assert node2.query("SELECT COUNT() FROM t2") == "510\n" + assert node3.query("SELECT COUNT() FROM t2") == "510\n" From 1510e3147df939f3b9a3bff8c874fd4648af91f3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 28 Jan 2021 16:08:07 +0300 Subject: [PATCH 37/86] Different ports for nodes --- .../test_testkeeper_multinode/configs/enable_test_keeper1.xml | 4 ++-- .../test_testkeeper_multinode/configs/enable_test_keeper2.xml | 4 ++-- .../test_testkeeper_multinode/configs/enable_test_keeper3.xml | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml index 81f68f50c7c..7fcd76ea57a 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml @@ -15,14 +15,14 @@ 2 node2 - 44444 + 44445 true 2 3 node3 - 44444 + 44446 true 1 diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml index 73340973367..f9d6dcad1d6 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml @@ -15,14 +15,14 @@ 2 node2 - 44444 + 44445 true 2 3 node3 - 44444 + 44446 true 1 diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml index fbc51489d11..7d71fd3a20d 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml @@ -15,14 +15,14 @@ 2 node2 - 44444 + 44445 true 2 3 node3 - 44444 + 44446 true 1 From b602f259f5e2c5e9c08de8ab02a677eecb3fa1cb Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Jan 2021 15:24:52 +0300 Subject: [PATCH 38/86] Fix race in NuRaft --- .gitmodules | 2 +- contrib/NuRaft | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.gitmodules b/.gitmodules index ecccf0633e2..ecefbc32ae6 100644 --- a/.gitmodules +++ b/.gitmodules @@ -220,4 +220,4 @@ url = https://github.com/ClickHouse-Extras/boringssl.git [submodule "contrib/NuRaft"] path = contrib/NuRaft - url = https://github.com/eBay/NuRaft.git + url = https://github.com/ClickHouse-Extras/NuRaft.git diff --git a/contrib/NuRaft b/contrib/NuRaft index 410bd149da8..6b6aedebcf1 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 410bd149da84cdde60b4436b02b738749f4e87e1 +Subproject commit 6b6aedebcf15ec362c4b6a1390c0b0802bb3e2c2 From 6781c9f61da6b601969bf059162e623b07324b09 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Jan 2021 15:34:53 +0300 Subject: [PATCH 39/86] One more fix --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index 6b6aedebcf1..644c264252a 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 6b6aedebcf15ec362c4b6a1390c0b0802bb3e2c2 +Subproject commit 644c264252aae91d9ad58366b086641bf8314008 From bac8cc55d2c48404a4b6b85ca09d15114620ef52 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Jan 2021 15:39:04 +0300 Subject: [PATCH 40/86] Now we answer from follower nodes --- src/Coordination/NuKeeperServer.cpp | 3 ++- .../TestKeeperStorageDispatcher.cpp | 9 +++------ src/Server/TestKeeperTCPHandler.cpp | 18 +++++++----------- src/Server/TestKeeperTCPHandler.h | 2 +- 4 files changed, 13 insertions(+), 19 deletions(-) diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index 8995b51a13b..bcc348d1be3 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -47,6 +47,7 @@ void NuKeeperServer::startup() params.reserved_log_items_ = 5000; params.snapshot_distance_ = 5000; params.client_req_timeout_ = 10000; + params.auto_forwarding_ = true; params.return_method_ = nuraft::raft_params::blocking; raft_instance = launcher.init( @@ -146,7 +147,7 @@ TestKeeperStorage::ResponsesForSessions NuKeeperServer::readZooKeeperResponses(n TestKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const TestKeeperStorage::RequestsForSessions & requests) { - if (isLeader() && requests.size() == 1 && requests[0].request->isReadRequest()) + if (raft_instance->is_leader_alive() && requests.size() == 1 && requests[0].request->isReadRequest()) { return state_machine->processReadRequest(requests[0]); } diff --git a/src/Coordination/TestKeeperStorageDispatcher.cpp b/src/Coordination/TestKeeperStorageDispatcher.cpp index 685fa58f8ad..d5682e1688b 100644 --- a/src/Coordination/TestKeeperStorageDispatcher.cpp +++ b/src/Coordination/TestKeeperStorageDispatcher.cpp @@ -175,12 +175,9 @@ void TestKeeperStorageDispatcher::shutdown() if (server) { TestKeeperStorage::RequestsForSessions expired_requests; - if (server->isLeader()) - { - TestKeeperStorage::RequestForSession request; - while (requests_queue.tryPop(request)) - expired_requests.push_back(TestKeeperStorage::RequestForSession{request}); - } + TestKeeperStorage::RequestForSession request; + while (requests_queue.tryPop(request)) + expired_requests.push_back(TestKeeperStorage::RequestForSession{request}); auto expired_responses = server->shutdown(expired_requests); diff --git a/src/Server/TestKeeperTCPHandler.cpp b/src/Server/TestKeeperTCPHandler.cpp index 04e5c6ece1d..5e5ba19f1a6 100644 --- a/src/Server/TestKeeperTCPHandler.cpp +++ b/src/Server/TestKeeperTCPHandler.cpp @@ -232,14 +232,10 @@ TestKeeperTCPHandler::TestKeeperTCPHandler(IServer & server_, const Poco::Net::S { } -void TestKeeperTCPHandler::sendHandshake(bool is_leader) +void TestKeeperTCPHandler::sendHandshake() { Coordination::write(Coordination::SERVER_HANDSHAKE_LENGTH, *out); - if (is_leader) - Coordination::write(Coordination::ZOOKEEPER_PROTOCOL_VERSION, *out); - else /// Specially ignore connections if we are not leader, client will throw exception - Coordination::write(42, *out); - + Coordination::write(Coordination::ZOOKEEPER_PROTOCOL_VERSION, *out); Coordination::write(Coordination::DEFAULT_SESSION_TIMEOUT_MS, *out); Coordination::write(session_id, *out); std::array passwd{}; @@ -319,18 +315,18 @@ void TestKeeperTCPHandler::runImpl() return; } - if (test_keeper_storage_dispatcher->isLeader()) + try { session_id = test_keeper_storage_dispatcher->getSessionID(); - sendHandshake(true); } - else + catch (const Exception & e) { - sendHandshake(false); - LOG_WARNING(log, "Ignoring connection because we are not leader"); + LOG_WARNING(log, "Cannot receive session id {}", e.displayText()); return; } + sendHandshake(); + auto response_fd = poll_wrapper->getResponseFD(); auto response_callback = [this, response_fd] (const Coordination::ZooKeeperResponsePtr & response) { diff --git a/src/Server/TestKeeperTCPHandler.h b/src/Server/TestKeeperTCPHandler.h index bb74513afce..e7372e8dd82 100644 --- a/src/Server/TestKeeperTCPHandler.h +++ b/src/Server/TestKeeperTCPHandler.h @@ -45,7 +45,7 @@ private: void runImpl(); - void sendHandshake(bool is_leader); + void sendHandshake(); void receiveHandshake(); std::pair receiveRequest(); From 67412bd5296453f45dfc050806cc5914fc76dbe8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 1 Feb 2021 10:51:10 +0300 Subject: [PATCH 41/86] Don't accept connections if we don't see leader + race fix from upstream --- contrib/NuRaft | 2 +- src/Coordination/NuKeeperServer.cpp | 33 +++++++++++++++---- src/Coordination/NuKeeperServer.h | 2 ++ .../TestKeeperStorageDispatcher.h | 5 +++ src/Server/TestKeeperTCPHandler.cpp | 29 +++++++++++----- src/Server/TestKeeperTCPHandler.h | 2 +- .../configs/enable_test_keeper1.xml | 4 +-- .../configs/enable_test_keeper2.xml | 4 +-- .../configs/enable_test_keeper3.xml | 4 +-- .../test_testkeeper_multinode/test.py | 30 +++++++++++++++++ 10 files changed, 93 insertions(+), 22 deletions(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index 644c264252a..9eb76db3ff1 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 644c264252aae91d9ad58366b086641bf8314008 +Subproject commit 9eb76db3ff1a78f672303b5b51dcbe0f9b22cf96 diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index bcc348d1be3..272632387d5 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -50,9 +50,11 @@ void NuKeeperServer::startup() params.auto_forwarding_ = true; params.return_method_ = nuraft::raft_params::blocking; + nuraft::asio_service::options asio_opts{}; + raft_instance = launcher.init( state_machine, state_manager, nuraft::cs_new("RaftInstance"), port, - nuraft::asio_service::options{}, params); + asio_opts, params); if (!raft_instance) throw Exception(ErrorCodes::RAFT_ERROR, "Cannot allocate RAFT instance"); @@ -127,10 +129,17 @@ TestKeeperStorage::ResponsesForSessions NuKeeperServer::readZooKeeperResponses(n response = std::make_shared(); else { - response = ops_mapping[session_id][xid]; - ops_mapping[session_id].erase(xid); + auto session_xids = ops_mapping.find(session_id); + if (session_xids == ops_mapping.end()) + throw Exception(ErrorCodes::RAFT_ERROR, "Unknown session id {}", session_id); + auto response_it = session_xids->second.find(xid); + if (response_it == session_xids->second.end()) + throw Exception(ErrorCodes::RAFT_ERROR, "Unknown xid {} for session id {}", xid, session_id); + + response = response_it->second; + ops_mapping[session_id].erase(response_it); if (ops_mapping[session_id].empty()) - ops_mapping.erase(session_id); + ops_mapping.erase(session_xids); } if (err == Coordination::Error::ZOK && (xid == Coordination::WATCH_XID || response->getOpNum() != Coordination::OpNum::Close)) @@ -147,7 +156,7 @@ TestKeeperStorage::ResponsesForSessions NuKeeperServer::readZooKeeperResponses(n TestKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const TestKeeperStorage::RequestsForSessions & requests) { - if (raft_instance->is_leader_alive() && requests.size() == 1 && requests[0].request->isReadRequest()) + if (isLeaderAlive() && requests.size() == 1 && requests[0].request->isReadRequest()) { return state_machine->processReadRequest(requests[0]); } @@ -191,7 +200,11 @@ TestKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const TestKe else if (result->get_result_code() != nuraft::cmd_result_code::OK) throw Exception(ErrorCodes::RAFT_ERROR, "Requests result failed with code {} and message: '{}'", result->get_result_code(), result->get_result_str()); - return readZooKeeperResponses(result->get()); + auto result_buf = result->get(); + if (result_buf == nullptr) + throw Exception(ErrorCodes::RAFT_ERROR, "Received nullptr from RAFT leader"); + + return readZooKeeperResponses(result_buf); } } @@ -210,6 +223,9 @@ int64_t NuKeeperServer::getSessionID() throw Exception(ErrorCodes::RAFT_ERROR, "session_id request failed to RAFT"); auto resp = result->get(); + if (resp == nullptr) + throw Exception(ErrorCodes::RAFT_ERROR, "Received nullptr as session_id"); + nuraft::buffer_serializer bs_resp(resp); return bs_resp.get_i64(); } @@ -219,6 +235,11 @@ bool NuKeeperServer::isLeader() const return raft_instance->is_leader(); } +bool NuKeeperServer::isLeaderAlive() const +{ + return raft_instance->is_leader_alive(); +} + bool NuKeeperServer::waitForServer(int32_t id) const { for (size_t i = 0; i < 10; ++i) diff --git a/src/Coordination/NuKeeperServer.h b/src/Coordination/NuKeeperServer.h index 7fd70ac26e2..f5f52802025 100644 --- a/src/Coordination/NuKeeperServer.h +++ b/src/Coordination/NuKeeperServer.h @@ -50,6 +50,8 @@ public: bool isLeader() const; + bool isLeaderAlive() const; + bool waitForServer(int32_t server_id) const; void waitForServers(const std::vector & ids) const; void waitForCatchUp() const; diff --git a/src/Coordination/TestKeeperStorageDispatcher.h b/src/Coordination/TestKeeperStorageDispatcher.h index a6c6118f9c4..a67a0c1fa3a 100644 --- a/src/Coordination/TestKeeperStorageDispatcher.h +++ b/src/Coordination/TestKeeperStorageDispatcher.h @@ -52,6 +52,11 @@ public: return server->isLeader(); } + bool hasLeader() const + { + return server->isLeaderAlive(); + } + int64_t getSessionID() { std::lock_guard lock(session_id_mutex); diff --git a/src/Server/TestKeeperTCPHandler.cpp b/src/Server/TestKeeperTCPHandler.cpp index 5e5ba19f1a6..532c0723e69 100644 --- a/src/Server/TestKeeperTCPHandler.cpp +++ b/src/Server/TestKeeperTCPHandler.cpp @@ -232,10 +232,14 @@ TestKeeperTCPHandler::TestKeeperTCPHandler(IServer & server_, const Poco::Net::S { } -void TestKeeperTCPHandler::sendHandshake() +void TestKeeperTCPHandler::sendHandshake(bool has_leader) { Coordination::write(Coordination::SERVER_HANDSHAKE_LENGTH, *out); - Coordination::write(Coordination::ZOOKEEPER_PROTOCOL_VERSION, *out); + if (has_leader) + Coordination::write(Coordination::ZOOKEEPER_PROTOCOL_VERSION, *out); + else /// Specially ignore connections if we are not leader, client will throw exception + Coordination::write(42, *out); + Coordination::write(Coordination::DEFAULT_SESSION_TIMEOUT_MS, *out); Coordination::write(session_id, *out); std::array passwd{}; @@ -315,18 +319,27 @@ void TestKeeperTCPHandler::runImpl() return; } - try + if (test_keeper_storage_dispatcher->hasLeader()) { - session_id = test_keeper_storage_dispatcher->getSessionID(); + try + { + session_id = test_keeper_storage_dispatcher->getSessionID(); + } + catch (const Exception & e) + { + LOG_WARNING(log, "Cannot receive session id {}", e.displayText()); + return; + + } + + sendHandshake(true); } - catch (const Exception & e) + else { - LOG_WARNING(log, "Cannot receive session id {}", e.displayText()); + sendHandshake(false); return; } - sendHandshake(); - auto response_fd = poll_wrapper->getResponseFD(); auto response_callback = [this, response_fd] (const Coordination::ZooKeeperResponsePtr & response) { diff --git a/src/Server/TestKeeperTCPHandler.h b/src/Server/TestKeeperTCPHandler.h index e7372e8dd82..53132a2b491 100644 --- a/src/Server/TestKeeperTCPHandler.h +++ b/src/Server/TestKeeperTCPHandler.h @@ -45,7 +45,7 @@ private: void runImpl(); - void sendHandshake(); + void sendHandshake(bool has_leader); void receiveHandshake(); std::pair receiveRequest(); diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml index 7fcd76ea57a..81f68f50c7c 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml @@ -15,14 +15,14 @@ 2 node2 - 44445 + 44444 true 2 3 node3 - 44446 + 44444 true 1 diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml index f9d6dcad1d6..73340973367 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml @@ -15,14 +15,14 @@ 2 node2 - 44445 + 44444 true 2 3 node3 - 44446 + 44444 true 1 diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml index 7d71fd3a20d..fbc51489d11 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml @@ -15,14 +15,14 @@ 2 node2 - 44445 + 44444 true 2 3 node3 - 44446 + 44444 true 1 diff --git a/tests/integration/test_testkeeper_multinode/test.py b/tests/integration/test_testkeeper_multinode/test.py index 8d35e30400a..fe568e7252d 100644 --- a/tests/integration/test_testkeeper_multinode/test.py +++ b/tests/integration/test_testkeeper_multinode/test.py @@ -62,9 +62,14 @@ def test_blocade_leader(started_cluster): for i in range(100): try: + node2.query("SYSTEM RESTART REPLICA t1") node2.query("INSERT INTO t1 SELECT rand() FROM numbers(100)") break except Exception as ex: + try: + node2.query("ATTACH TABLE t1") + except Exception as attach_ex: + print("Got exception node2", smaller_exception(attach_ex)) print("Got exception node2", smaller_exception(ex)) time.sleep(0.5) else: @@ -72,9 +77,14 @@ def test_blocade_leader(started_cluster): for i in range(100): try: + node3.query("SYSTEM RESTART REPLICA t1") node3.query("INSERT INTO t1 SELECT rand() FROM numbers(100)") break except Exception as ex: + try: + node3.query("ATTACH TABLE t1") + except Exception as attach_ex: + print("Got exception node3", smaller_exception(attach_ex)) print("Got exception node3", smaller_exception(ex)) time.sleep(0.5) else: @@ -86,6 +96,11 @@ def test_blocade_leader(started_cluster): node.query("SYSTEM RESTART REPLICA t1") break except Exception as ex: + try: + node.query("ATTACH TABLE t1") + except Exception as attach_ex: + print("Got exception node{}".format(n + 1), smaller_exception(attach_ex)) + print("Got exception node{}".format(n + 1), smaller_exception(ex)) time.sleep(0.5) else: @@ -136,9 +151,14 @@ def test_blocade_leader_twice(started_cluster): for i in range(100): try: + node2.query("SYSTEM RESTART REPLICA t2") node2.query("INSERT INTO t2 SELECT rand() FROM numbers(100)") break except Exception as ex: + try: + node2.query("ATTACH TABLE t2") + except Exception as attach_ex: + print("Got exception node2", smaller_exception(attach_ex)) print("Got exception node2", smaller_exception(ex)) time.sleep(0.5) else: @@ -146,9 +166,14 @@ def test_blocade_leader_twice(started_cluster): for i in range(100): try: + node3.query("SYSTEM RESTART REPLICA t2") node3.query("INSERT INTO t2 SELECT rand() FROM numbers(100)") break except Exception as ex: + try: + node3.query("ATTACH TABLE t2") + except Exception as attach_ex: + print("Got exception node3", smaller_exception(attach_ex)) print("Got exception node3", smaller_exception(ex)) time.sleep(0.5) else: @@ -179,6 +204,11 @@ def test_blocade_leader_twice(started_cluster): node.query("SYSTEM RESTART REPLICA t2") break except Exception as ex: + try: + node.query("ATTACH TABLE t2") + except Exception as attach_ex: + print("Got exception node{}".format(n + 1), smaller_exception(attach_ex)) + print("Got exception node{}".format(n + 1), smaller_exception(ex)) time.sleep(0.5) else: From eb5c77f558bb823ec62410ed81f5763c3fc71d21 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 1 Feb 2021 14:27:26 +0300 Subject: [PATCH 42/86] Fix some races and better reaction to leader change --- contrib/NuRaft | 2 +- contrib/boost | 2 +- src/Coordination/NuKeeperServer.cpp | 18 ++++++--- src/Coordination/NuKeeperServer.h | 2 + .../TestKeeperStorageDispatcher.cpp | 40 ++++++++++++++----- .../TestKeeperStorageDispatcher.h | 7 ++-- src/Server/TestKeeperTCPHandler.cpp | 2 + 7 files changed, 54 insertions(+), 19 deletions(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index 9eb76db3ff1..c6f8528ead6 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 9eb76db3ff1a78f672303b5b51dcbe0f9b22cf96 +Subproject commit c6f8528ead61f7e4565164c6f15afef221235aa8 diff --git a/contrib/boost b/contrib/boost index b2368f43f37..48f40ebb539 160000 --- a/contrib/boost +++ b/contrib/boost @@ -1 +1 @@ -Subproject commit b2368f43f37c4a592b17b1e9a474b93749c47319 +Subproject commit 48f40ebb539220d328958f8823b094c0b07a4e79 diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index 272632387d5..014b2761f37 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -33,7 +33,11 @@ void NuKeeperServer::addServer(int server_id_, const std::string & server_uri_, { nuraft::srv_config config(server_id_, 0, server_uri_, "", /* follower= */ !can_become_leader_, priority); auto ret1 = raft_instance->add_srv(config); - if (ret1->get_result_code() != nuraft::cmd_result_code::OK) + auto code = ret1->get_result_code(); + if (code == nuraft::cmd_result_code::TIMEOUT + || code == nuraft::cmd_result_code::BAD_REQUEST + || code == nuraft::cmd_result_code::NOT_LEADER + || code == nuraft::cmd_result_code::FAILED) throw Exception(ErrorCodes::RAFT_ERROR, "Cannot add server to RAFT quorum with code {}, message '{}'", ret1->get_result_code(), ret1->get_result_str()); } @@ -41,9 +45,9 @@ void NuKeeperServer::addServer(int server_id_, const std::string & server_uri_, void NuKeeperServer::startup() { nuraft::raft_params params; - params.heart_beat_interval_ = 100; - params.election_timeout_lower_bound_ = 200; - params.election_timeout_upper_bound_ = 400; + params.heart_beat_interval_ = 1000; + params.election_timeout_lower_bound_ = 3000; + params.election_timeout_upper_bound_ = 6000; params.reserved_log_items_ = 5000; params.snapshot_distance_ = 5000; params.client_req_timeout_ = 10000; @@ -59,7 +63,7 @@ void NuKeeperServer::startup() if (!raft_instance) throw Exception(ErrorCodes::RAFT_ERROR, "Cannot allocate RAFT instance"); - static constexpr auto MAX_RETRY = 30; + static constexpr auto MAX_RETRY = 100; for (size_t i = 0; i < MAX_RETRY; ++i) { if (raft_instance->is_initialized()) @@ -169,6 +173,8 @@ TestKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const TestKe entries.push_back(getZooKeeperLogEntry(session_id, request)); } + std::lock_guard lock(append_entries_mutex); + auto result = raft_instance->append_entries(entries); if (!result->get_accepted()) { @@ -215,6 +221,8 @@ int64_t NuKeeperServer::getSessionID() nuraft::buffer_serializer bs(entry); bs.put_i64(0); + std::lock_guard lock(append_entries_mutex); + auto result = raft_instance->append_entries({entry}); if (!result->get_accepted()) throw Exception(ErrorCodes::RAFT_ERROR, "Cannot send session_id request to RAFT"); diff --git a/src/Coordination/NuKeeperServer.h b/src/Coordination/NuKeeperServer.h index f5f52802025..c1f32c67166 100644 --- a/src/Coordination/NuKeeperServer.h +++ b/src/Coordination/NuKeeperServer.h @@ -37,6 +37,8 @@ private: TestKeeperStorage::ResponsesForSessions readZooKeeperResponses(nuraft::ptr & buffer); + std::mutex append_entries_mutex; + public: NuKeeperServer(int server_id_, const std::string & hostname_, int port_); diff --git a/src/Coordination/TestKeeperStorageDispatcher.cpp b/src/Coordination/TestKeeperStorageDispatcher.cpp index d5682e1688b..d9f9dfd30eb 100644 --- a/src/Coordination/TestKeeperStorageDispatcher.cpp +++ b/src/Coordination/TestKeeperStorageDispatcher.cpp @@ -11,6 +11,11 @@ namespace ErrorCodes extern const int TIMEOUT_EXCEEDED; } +TestKeeperStorageDispatcher::TestKeeperStorageDispatcher() + : log(&Poco::Logger::get("TestKeeperDispatcher")) +{ +} + void TestKeeperStorageDispatcher::processingThread() { setThreadName("TestKeeperSProc"); @@ -101,6 +106,7 @@ namespace void TestKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfiguration & config) { + LOG_DEBUG(log, "Initializing storage dispatcher"); int myid = config.getInt("test_keeper_server.server_id"); std::string myhostname; int myport; @@ -134,26 +140,39 @@ void TestKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigura } server = std::make_unique(myid, myhostname, myport); - server->startup(); - if (shouldBuildQuorum(myid, my_priority, my_can_become_leader, server_configs)) + try { - for (const auto & [id, hostname, port, can_become_leader, priority] : server_configs) + server->startup(); + if (shouldBuildQuorum(myid, my_priority, my_can_become_leader, server_configs)) { - do + for (const auto & [id, hostname, port, can_become_leader, priority] : server_configs) { - server->addServer(id, hostname + ":" + std::to_string(port), can_become_leader, priority); + LOG_DEBUG(log, "Adding server with id {} ({}:{})", id, hostname, port); + do + { + server->addServer(id, hostname + ":" + std::to_string(port), can_become_leader, priority); + } + while (!server->waitForServer(id)); + + LOG_DEBUG(log, "Server with id {} ({}:{}) added to cluster", id, hostname, port); } - while (!server->waitForServer(id)); + } + else + { + LOG_DEBUG(log, "Waiting for {} servers to build cluster", ids.size()); + server->waitForServers(ids); + server->waitForCatchUp(); } } - else + catch (...) { - server->waitForServers(ids); - server->waitForCatchUp(); + tryLogCurrentException(__PRETTY_FUNCTION__); + throw; } processing_thread = ThreadFromGlobalPool([this] { processingThread(); }); + LOG_DEBUG(log, "Dispatcher initialized"); } void TestKeeperStorageDispatcher::shutdown() @@ -166,6 +185,7 @@ void TestKeeperStorageDispatcher::shutdown() if (shutdown_called) return; + LOG_DEBUG(log, "Shutting down storage dispatcher"); shutdown_called = true; if (processing_thread.joinable()) @@ -189,6 +209,8 @@ void TestKeeperStorageDispatcher::shutdown() { tryLogCurrentException(__PRETTY_FUNCTION__); } + + LOG_DEBUG(log, "Dispatcher shut down"); } TestKeeperStorageDispatcher::~TestKeeperStorageDispatcher() diff --git a/src/Coordination/TestKeeperStorageDispatcher.h b/src/Coordination/TestKeeperStorageDispatcher.h index a67a0c1fa3a..7ee88c8e940 100644 --- a/src/Coordination/TestKeeperStorageDispatcher.h +++ b/src/Coordination/TestKeeperStorageDispatcher.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -30,14 +31,15 @@ private: ThreadFromGlobalPool processing_thread; std::unique_ptr server; - std::mutex session_id_mutex; + + Poco::Logger * log; private: void processingThread(); void setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response); public: - TestKeeperStorageDispatcher() = default; + TestKeeperStorageDispatcher(); void initialize(const Poco::Util::AbstractConfiguration & config); @@ -59,7 +61,6 @@ public: int64_t getSessionID() { - std::lock_guard lock(session_id_mutex); return server->getSessionID(); } diff --git a/src/Server/TestKeeperTCPHandler.cpp b/src/Server/TestKeeperTCPHandler.cpp index 532c0723e69..bf7cb39d747 100644 --- a/src/Server/TestKeeperTCPHandler.cpp +++ b/src/Server/TestKeeperTCPHandler.cpp @@ -328,6 +328,7 @@ void TestKeeperTCPHandler::runImpl() catch (const Exception & e) { LOG_WARNING(log, "Cannot receive session id {}", e.displayText()); + sendHandshake(false); return; } @@ -336,6 +337,7 @@ void TestKeeperTCPHandler::runImpl() } else { + LOG_WARNING(log, "Ignoring user request, because no alive leader exist"); sendHandshake(false); return; } From 0fb7bc6cbe9cb73fb1aa99adc254f5611884d33f Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 1 Feb 2021 15:10:55 +0300 Subject: [PATCH 43/86] clang-tidy is very smart --- src/Coordination/NuKeeperServer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index 014b2761f37..99af40154ca 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -31,7 +31,7 @@ NuKeeperServer::NuKeeperServer(int server_id_, const std::string & hostname_, in void NuKeeperServer::addServer(int server_id_, const std::string & server_uri_, bool can_become_leader_, int32_t priority) { - nuraft::srv_config config(server_id_, 0, server_uri_, "", /* follower= */ !can_become_leader_, priority); + nuraft::srv_config config(server_id_, 0, server_uri_, "", /* learner = */ !can_become_leader_, priority); auto ret1 = raft_instance->add_srv(config); auto code = ret1->get_result_code(); if (code == nuraft::cmd_result_code::TIMEOUT From 57c9b6c864d10909e74cd02e9ba4c90cd4487339 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 1 Feb 2021 16:18:17 +0300 Subject: [PATCH 44/86] Fix build without nuraft --- cmake/find/nuraft.cmake | 6 +-- programs/server/Server.cpp | 42 +++++++++++-------- src/CMakeLists.txt | 5 ++- .../TestKeeperStorageDispatcher.h | 17 +++++++- src/Coordination/tests/gtest_for_build.cpp | 12 +++++- src/Core/config_core.h.in | 1 + src/Interpreters/Context.cpp | 10 +++++ src/Interpreters/Context.h | 5 ++- src/Server/TestKeeperTCPHandler.cpp | 7 ++++ src/Server/TestKeeperTCPHandler.h | 8 ++++ src/Server/TestKeeperTCPHandlerFactory.h | 1 + 11 files changed, 87 insertions(+), 27 deletions(-) diff --git a/cmake/find/nuraft.cmake b/cmake/find/nuraft.cmake index d31fe9c1de8..bcc656de129 100644 --- a/cmake/find/nuraft.cmake +++ b/cmake/find/nuraft.cmake @@ -1,6 +1,6 @@ option(ENABLE_NURAFT "Enable NuRaft" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_NURAFT) + if (NOT ENABLE_NURAFT) return() endif() @@ -11,7 +11,7 @@ if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/NuRaft/CMakeLists.txt") return() endif () -if (NOT OS_FREEBSD) +if (NOT OS_FREEBSD AND NOT OS_DARWIN) set (USE_NURAFT 1) set (NURAFT_LIBRARY nuraft) @@ -20,5 +20,5 @@ if (NOT OS_FREEBSD) message (STATUS "Using NuRaft=${USE_NURAFT}: ${NURAFT_INCLUDE_DIR} : ${NURAFT_LIBRARY}") else() set (USE_NURAFT 0) - message (STATUS "Using internal NuRaft library on FreeBSD is not supported") + message (STATUS "Using internal NuRaft library on FreeBSD and Darwin is not supported") endif() diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index fefabd8be71..801e8f2122b 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -59,7 +59,6 @@ #include #include #include -#include #include "MetricsTransmitter.h" #include #include @@ -94,6 +93,9 @@ # include #endif +#if USE_NURAFT +# include +#endif namespace CurrentMetrics { @@ -844,27 +846,31 @@ int Server::main(const std::vector & /*args*/) if (config().has("test_keeper_server")) { +#if USE_NURAFT /// Initialize test keeper RAFT. Do nothing if no test_keeper_server in config. global_context->initializeTestKeeperStorageDispatcher(); - } - - for (const auto & listen_host : listen_hosts) - { - /// TCP TestKeeper - const char * port_name = "test_keeper_server.tcp_port"; - createServer(listen_host, port_name, listen_try, [&](UInt16 port) + for (const auto & listen_host : listen_hosts) { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(socket, listen_host, port); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - servers_to_start_before_tables->emplace_back( - port_name, - std::make_unique( - new TestKeeperTCPHandlerFactory(*this), server_pool, socket, new Poco::Net::TCPServerParams)); + /// TCP TestKeeper + const char * port_name = "test_keeper_server.tcp_port"; + createServer(listen_host, port_name, listen_try, [&](UInt16 port) + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(socket, listen_host, port); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + servers_to_start_before_tables->emplace_back( + port_name, + std::make_unique( + new TestKeeperTCPHandlerFactory(*this), server_pool, socket, new Poco::Net::TCPServerParams)); + + LOG_INFO(log, "Listening for connections to fake zookeeper (tcp): {}", address.toString()); + }); + } +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "ClickHouse server built without NuRaft library. Cannot use internal coordination."); +#endif - LOG_INFO(log, "Listening for connections to fake zookeeper (tcp): {}", address.toString()); - }); } for (auto & server : *servers_to_start_before_tables) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 34c437ebde6..13703a5cd55 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -192,7 +192,10 @@ add_object_library(clickhouse_processors_merges Processors/Merges) add_object_library(clickhouse_processors_merges_algorithms Processors/Merges/Algorithms) add_object_library(clickhouse_processors_queryplan Processors/QueryPlan) add_object_library(clickhouse_processors_queryplan_optimizations Processors/QueryPlan/Optimizations) -add_object_library(clickhouse_coordination Coordination) + +if (USE_NURAFT) + add_object_library(clickhouse_coordination Coordination) +endif() set (DBMS_COMMON_LIBRARIES) # libgcc_s does not provide an implementation of an atomics library. Instead, diff --git a/src/Coordination/TestKeeperStorageDispatcher.h b/src/Coordination/TestKeeperStorageDispatcher.h index 7ee88c8e940..ddb90abb88a 100644 --- a/src/Coordination/TestKeeperStorageDispatcher.h +++ b/src/Coordination/TestKeeperStorageDispatcher.h @@ -1,11 +1,20 @@ #pragma once +#if !defined(ARCADIA_BUILD) +# include +# include "config_core.h" +#endif + +#if USE_NURAFT + #include #include +#include +#include +#include #include #include -#include -#include + namespace DB { @@ -14,6 +23,7 @@ using ZooKeeperResponseCallback = std::function +#if !defined(ARCADIA_BUILD) +# include +# include "config_core.h" +#endif + +#if USE_NURAFT + #include #include #include @@ -454,5 +461,8 @@ TEST(CoordinationTest, TestNuKeeperRaft) s4.launcher.shutdown(5); } -# endif +# endif + +#endif + #endif diff --git a/src/Core/config_core.h.in b/src/Core/config_core.h.in index 6c7a35abd7c..666ef32efdf 100644 --- a/src/Core/config_core.h.in +++ b/src/Core/config_core.h.in @@ -13,3 +13,4 @@ #cmakedefine01 USE_LDAP #cmakedefine01 USE_ROCKSDB #cmakedefine01 USE_LIBPQXX +#cmakedefine01 USE_NURAFT diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index adeb9b5862c..fe1b6a8a32e 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -304,8 +304,10 @@ struct ContextShared mutable zkutil::ZooKeeperPtr zookeeper; /// Client for ZooKeeper. ConfigurationPtr zookeeper_config; /// Stores zookeeper configs +#if USE_NURAFT mutable std::mutex test_keeper_storage_dispatcher_mutex; mutable std::shared_ptr test_keeper_storage_dispatcher; +#endif mutable std::mutex auxiliary_zookeepers_mutex; mutable std::map auxiliary_zookeepers; /// Map for auxiliary ZooKeeper clients. ConfigurationPtr auxiliary_zookeepers_config; /// Stores auxiliary zookeepers configs @@ -1579,8 +1581,10 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const return shared->zookeeper; } + void Context::initializeTestKeeperStorageDispatcher() const { +#if USE_NURAFT std::lock_guard lock(shared->test_keeper_storage_dispatcher_mutex); if (shared->test_keeper_storage_dispatcher) @@ -1592,8 +1596,10 @@ void Context::initializeTestKeeperStorageDispatcher() const shared->test_keeper_storage_dispatcher = std::make_shared(); shared->test_keeper_storage_dispatcher->initialize(config); } +#endif } +#if USE_NURAFT std::shared_ptr & Context::getTestKeeperStorageDispatcher() const { std::lock_guard lock(shared->test_keeper_storage_dispatcher_mutex); @@ -1602,17 +1608,21 @@ std::shared_ptr & Context::getTestKeeperStorageDisp return shared->test_keeper_storage_dispatcher; } +#endif void Context::shutdownTestKeeperStorageDispatcher() const { +#if USE_NURAFT std::lock_guard lock(shared->test_keeper_storage_dispatcher_mutex); if (shared->test_keeper_storage_dispatcher) { shared->test_keeper_storage_dispatcher->shutdown(); shared->test_keeper_storage_dispatcher.reset(); } +#endif } + zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const { std::lock_guard lock(shared->auxiliary_zookeepers_mutex); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index c8a71244164..3c78973b21a 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -573,9 +573,10 @@ public: /// Same as above but return a zookeeper connection from auxiliary_zookeepers configuration entry. std::shared_ptr getAuxiliaryZooKeeper(const String & name) const; - - void initializeTestKeeperStorageDispatcher() const; +#if USE_NURAFT std::shared_ptr & getTestKeeperStorageDispatcher() const; +#endif + void initializeTestKeeperStorageDispatcher() const; void shutdownTestKeeperStorageDispatcher() const; /// Set auxiliary zookeepers configuration at server starting or configuration reloading. diff --git a/src/Server/TestKeeperTCPHandler.cpp b/src/Server/TestKeeperTCPHandler.cpp index bf7cb39d747..17f2ec547c9 100644 --- a/src/Server/TestKeeperTCPHandler.cpp +++ b/src/Server/TestKeeperTCPHandler.cpp @@ -1,4 +1,7 @@ #include + +#if USE_NURAFT + #include #include #include @@ -22,9 +25,11 @@ #include #endif + namespace DB { + namespace ErrorCodes { extern const int SYSTEM_ERROR; @@ -454,3 +459,5 @@ std::pair TestKeeperTCPHandler::receiveR } } + +#endif diff --git a/src/Server/TestKeeperTCPHandler.h b/src/Server/TestKeeperTCPHandler.h index 53132a2b491..09543b5a888 100644 --- a/src/Server/TestKeeperTCPHandler.h +++ b/src/Server/TestKeeperTCPHandler.h @@ -1,5 +1,12 @@ #pragma once +#if !defined(ARCADIA_BUILD) +# include +# include "config_core.h" +#endif + +#if USE_NURAFT + #include #include "IServer.h" #include @@ -53,3 +60,4 @@ private: }; } +#endif diff --git a/src/Server/TestKeeperTCPHandlerFactory.h b/src/Server/TestKeeperTCPHandlerFactory.h index ebf91aa31d4..a5bf6be8c8a 100644 --- a/src/Server/TestKeeperTCPHandlerFactory.h +++ b/src/Server/TestKeeperTCPHandlerFactory.h @@ -1,4 +1,5 @@ #pragma once + #include #include #include From 365bf65f5a8223dba319c86182ecb20236b611a0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 1 Feb 2021 17:14:59 +0300 Subject: [PATCH 45/86] Fix install script --- programs/server/Server.cpp | 18 +- programs/server/config.d/test_keeper_port.xml | 1 - src/Coordination/NuKeeperServer.cpp | 24 +-- src/Coordination/NuKeeperServer.h | 8 +- src/Coordination/NuKeeperStateMachine.cpp | 18 +- src/Coordination/NuKeeperStateMachine.h | 12 +- ...tKeeperStorage.cpp => NuKeeperStorage.cpp} | 156 +++++++++--------- ...{TestKeeperStorage.h => NuKeeperStorage.h} | 8 +- ...cher.cpp => NuKeeperStorageDispatcher.cpp} | 48 +++--- ...spatcher.h => NuKeeperStorageDispatcher.h} | 8 +- ...izer.cpp => NuKeeperStorageSerializer.cpp} | 12 +- src/Coordination/NuKeeperStorageSerializer.h | 17 ++ .../TestKeeperStorageSerializer.h | 17 -- src/Coordination/tests/gtest_for_build.cpp | 18 +- src/Coordination/ya.make | 6 +- src/Interpreters/Context.cpp | 40 ++--- src/Interpreters/Context.h | 8 +- ...rTCPHandler.cpp => NuKeeperTCPHandler.cpp} | 36 ++-- ...eeperTCPHandler.h => NuKeeperTCPHandler.h} | 8 +- ...rFactory.h => NuKeeperTCPHandlerFactory.h} | 12 +- src/Server/ya.make | 2 +- ...est_keeper_port.xml => nu_keeper_port.xml} | 4 +- tests/config/install.sh | 2 +- .../configs/enable_test_keeper.xml | 4 +- .../configs/enable_test_keeper1.xml | 4 +- .../configs/enable_test_keeper2.xml | 4 +- .../configs/enable_test_keeper3.xml | 4 +- 27 files changed, 249 insertions(+), 250 deletions(-) delete mode 120000 programs/server/config.d/test_keeper_port.xml rename src/Coordination/{TestKeeperStorage.cpp => NuKeeperStorage.cpp} (75%) rename src/Coordination/{TestKeeperStorage.h => NuKeeperStorage.h} (92%) rename src/Coordination/{TestKeeperStorageDispatcher.cpp => NuKeeperStorageDispatcher.cpp} (76%) rename src/Coordination/{TestKeeperStorageDispatcher.h => NuKeeperStorageDispatcher.h} (90%) rename src/Coordination/{TestKeeperStorageSerializer.cpp => NuKeeperStorageSerializer.cpp} (84%) create mode 100644 src/Coordination/NuKeeperStorageSerializer.h delete mode 100644 src/Coordination/TestKeeperStorageSerializer.h rename src/Server/{TestKeeperTCPHandler.cpp => NuKeeperTCPHandler.cpp} (92%) rename src/Server/{TestKeeperTCPHandler.h => NuKeeperTCPHandler.h} (83%) rename src/Server/{TestKeeperTCPHandlerFactory.h => NuKeeperTCPHandlerFactory.h} (68%) rename tests/config/config.d/{test_keeper_port.xml => nu_keeper_port.xml} (88%) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 801e8f2122b..fb58e85d813 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -94,7 +94,7 @@ #endif #if USE_NURAFT -# include +# include #endif namespace CurrentMetrics @@ -844,15 +844,15 @@ int Server::main(const std::vector & /*args*/) listen_try = true; } - if (config().has("test_keeper_server")) + if (config().has("nu_keeper_server")) { #if USE_NURAFT - /// Initialize test keeper RAFT. Do nothing if no test_keeper_server in config. - global_context->initializeTestKeeperStorageDispatcher(); + /// Initialize test keeper RAFT. Do nothing if no nu_keeper_server in config. + global_context->initializeNuKeeperStorageDispatcher(); for (const auto & listen_host : listen_hosts) { - /// TCP TestKeeper - const char * port_name = "test_keeper_server.tcp_port"; + /// TCP NuKeeper + const char * port_name = "nu_keeper_server.tcp_port"; createServer(listen_host, port_name, listen_try, [&](UInt16 port) { Poco::Net::ServerSocket socket; @@ -862,9 +862,9 @@ int Server::main(const std::vector & /*args*/) servers_to_start_before_tables->emplace_back( port_name, std::make_unique( - new TestKeeperTCPHandlerFactory(*this), server_pool, socket, new Poco::Net::TCPServerParams)); + new NuKeeperTCPHandlerFactory(*this), server_pool, socket, new Poco::Net::TCPServerParams)); - LOG_INFO(log, "Listening for connections to fake zookeeper (tcp): {}", address.toString()); + LOG_INFO(log, "Listening for connections to NuKeeper (tcp): {}", address.toString()); }); } #else @@ -911,7 +911,7 @@ int Server::main(const std::vector & /*args*/) else LOG_INFO(log, "Closed connections to servers for tables."); - global_context->shutdownTestKeeperStorageDispatcher(); + global_context->shutdownNuKeeperStorageDispatcher(); } /** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available. diff --git a/programs/server/config.d/test_keeper_port.xml b/programs/server/config.d/test_keeper_port.xml deleted file mode 120000 index f3f721caae0..00000000000 --- a/programs/server/config.d/test_keeper_port.xml +++ /dev/null @@ -1 +0,0 @@ -../../../tests/config/config.d/test_keeper_port.xml \ No newline at end of file diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index 99af40154ca..bb74ea19aa7 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -46,8 +46,8 @@ void NuKeeperServer::startup() { nuraft::raft_params params; params.heart_beat_interval_ = 1000; - params.election_timeout_lower_bound_ = 3000; - params.election_timeout_upper_bound_ = 6000; + params.election_timeout_lower_bound_ = 500; + params.election_timeout_upper_bound_ = 1000; params.reserved_log_items_ = 5000; params.snapshot_distance_ = 5000; params.client_req_timeout_ = 10000; @@ -75,9 +75,9 @@ void NuKeeperServer::startup() throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Cannot start RAFT server within startup timeout"); } -TestKeeperStorage::ResponsesForSessions NuKeeperServer::shutdown(const TestKeeperStorage::RequestsForSessions & expired_requests) +NuKeeperStorage::ResponsesForSessions NuKeeperServer::shutdown(const NuKeeperStorage::RequestsForSessions & expired_requests) { - TestKeeperStorage::ResponsesForSessions responses; + NuKeeperStorage::ResponsesForSessions responses; if (isLeader()) { try @@ -108,9 +108,9 @@ nuraft::ptr getZooKeeperLogEntry(int64_t session_id, const Coord } -TestKeeperStorage::ResponsesForSessions NuKeeperServer::readZooKeeperResponses(nuraft::ptr & buffer) +NuKeeperStorage::ResponsesForSessions NuKeeperServer::readZooKeeperResponses(nuraft::ptr & buffer) { - DB::TestKeeperStorage::ResponsesForSessions results; + DB::NuKeeperStorage::ResponsesForSessions results; DB::ReadBufferFromNuraftBuffer buf(buffer); while (!buf.eof()) @@ -153,12 +153,12 @@ TestKeeperStorage::ResponsesForSessions NuKeeperServer::readZooKeeperResponses(n response->zxid = zxid; response->error = err; - results.push_back(DB::TestKeeperStorage::ResponseForSession{session_id, response}); + results.push_back(DB::NuKeeperStorage::ResponseForSession{session_id, response}); } return results; } -TestKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const TestKeeperStorage::RequestsForSessions & requests) +NuKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const NuKeeperStorage::RequestsForSessions & requests) { if (isLeaderAlive() && requests.size() == 1 && requests[0].request->isReadRequest()) { @@ -178,28 +178,28 @@ TestKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const TestKe auto result = raft_instance->append_entries(entries); if (!result->get_accepted()) { - TestKeeperStorage::ResponsesForSessions responses; + NuKeeperStorage::ResponsesForSessions responses; for (const auto & [session_id, request] : requests) { auto response = request->makeResponse(); response->xid = request->xid; response->zxid = 0; /// FIXME what we can do with it? response->error = Coordination::Error::ZSESSIONEXPIRED; - responses.push_back(DB::TestKeeperStorage::ResponseForSession{session_id, response}); + responses.push_back(DB::NuKeeperStorage::ResponseForSession{session_id, response}); } return responses; } if (result->get_result_code() == nuraft::cmd_result_code::TIMEOUT) { - TestKeeperStorage::ResponsesForSessions responses; + NuKeeperStorage::ResponsesForSessions responses; for (const auto & [session_id, request] : requests) { auto response = request->makeResponse(); response->xid = request->xid; response->zxid = 0; /// FIXME what we can do with it? response->error = Coordination::Error::ZOPERATIONTIMEOUT; - responses.push_back(DB::TestKeeperStorage::ResponseForSession{session_id, response}); + responses.push_back(DB::NuKeeperStorage::ResponseForSession{session_id, response}); } return responses; } diff --git a/src/Coordination/NuKeeperServer.h b/src/Coordination/NuKeeperServer.h index c1f32c67166..352836dfc27 100644 --- a/src/Coordination/NuKeeperServer.h +++ b/src/Coordination/NuKeeperServer.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include namespace DB @@ -35,7 +35,7 @@ private: SessionIDOps ops_mapping; - TestKeeperStorage::ResponsesForSessions readZooKeeperResponses(nuraft::ptr & buffer); + NuKeeperStorage::ResponsesForSessions readZooKeeperResponses(nuraft::ptr & buffer); std::mutex append_entries_mutex; @@ -44,7 +44,7 @@ public: void startup(); - TestKeeperStorage::ResponsesForSessions putRequests(const TestKeeperStorage::RequestsForSessions & requests); + NuKeeperStorage::ResponsesForSessions putRequests(const NuKeeperStorage::RequestsForSessions & requests); int64_t getSessionID(); @@ -58,7 +58,7 @@ public: void waitForServers(const std::vector & ids) const; void waitForCatchUp() const; - TestKeeperStorage::ResponsesForSessions shutdown(const TestKeeperStorage::RequestsForSessions & expired_requests); + NuKeeperStorage::ResponsesForSessions shutdown(const NuKeeperStorage::RequestsForSessions & expired_requests); }; } diff --git a/src/Coordination/NuKeeperStateMachine.cpp b/src/Coordination/NuKeeperStateMachine.cpp index 9f4572c02e0..b6521e1d648 100644 --- a/src/Coordination/NuKeeperStateMachine.cpp +++ b/src/Coordination/NuKeeperStateMachine.cpp @@ -3,17 +3,17 @@ #include #include #include -#include +#include namespace DB { static constexpr int MAX_SNAPSHOTS = 3; -TestKeeperStorage::RequestForSession parseRequest(nuraft::buffer & data) +NuKeeperStorage::RequestForSession parseRequest(nuraft::buffer & data) { ReadBufferFromNuraftBuffer buffer(data); - TestKeeperStorage::RequestForSession request_for_session; + NuKeeperStorage::RequestForSession request_for_session; readIntBinary(request_for_session.session_id, buffer); int32_t length; @@ -31,7 +31,7 @@ TestKeeperStorage::RequestForSession parseRequest(nuraft::buffer & data) return request_for_session; } -nuraft::ptr writeResponses(TestKeeperStorage::ResponsesForSessions & responses) +nuraft::ptr writeResponses(NuKeeperStorage::ResponsesForSessions & responses) { WriteBufferFromNuraftBuffer buffer; for (const auto & response_and_session : responses) @@ -67,7 +67,7 @@ nuraft::ptr NuKeeperStateMachine::commit(const size_t log_idx, n else { auto request_for_session = parseRequest(data); - TestKeeperStorage::ResponsesForSessions responses_for_sessions; + NuKeeperStorage::ResponsesForSessions responses_for_sessions; { std::lock_guard lock(storage_lock); responses_for_sessions = storage.processRequest(request_for_session.request, request_for_session.session_id); @@ -118,10 +118,10 @@ NuKeeperStateMachine::StorageSnapshotPtr NuKeeperStateMachine::readSnapshot(nura { nuraft::ptr snp_buf = s.serialize(); nuraft::ptr ss = nuraft::snapshot::deserialize(*snp_buf); - TestKeeperStorageSerializer serializer; + NuKeeperStorageSerializer serializer; ReadBufferFromNuraftBuffer reader(in); - TestKeeperStorage new_storage; + NuKeeperStorage new_storage; serializer.deserialize(new_storage, reader); return std::make_shared(ss, new_storage); } @@ -129,7 +129,7 @@ NuKeeperStateMachine::StorageSnapshotPtr NuKeeperStateMachine::readSnapshot(nura void NuKeeperStateMachine::writeSnapshot(const NuKeeperStateMachine::StorageSnapshotPtr & snapshot, nuraft::ptr & out) { - TestKeeperStorageSerializer serializer; + NuKeeperStorageSerializer serializer; WriteBufferFromNuraftBuffer writer; serializer.serialize(snapshot->storage, writer); @@ -223,7 +223,7 @@ int NuKeeperStateMachine::read_logical_snp_obj( return 0; } -TestKeeperStorage::ResponsesForSessions NuKeeperStateMachine::processReadRequest(const TestKeeperStorage::RequestForSession & request_for_session) +NuKeeperStorage::ResponsesForSessions NuKeeperStateMachine::processReadRequest(const NuKeeperStorage::RequestForSession & request_for_session) { std::lock_guard lock(storage_lock); return storage.processRequest(request_for_session.request, request_for_session.session_id); diff --git a/src/Coordination/NuKeeperStateMachine.h b/src/Coordination/NuKeeperStateMachine.h index 7767f552cec..41c28caa76c 100644 --- a/src/Coordination/NuKeeperStateMachine.h +++ b/src/Coordination/NuKeeperStateMachine.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -42,23 +42,23 @@ public: nuraft::ptr & data_out, bool & is_last_obj) override; - TestKeeperStorage & getStorage() + NuKeeperStorage & getStorage() { return storage; } - TestKeeperStorage::ResponsesForSessions processReadRequest(const TestKeeperStorage::RequestForSession & request_for_session); + NuKeeperStorage::ResponsesForSessions processReadRequest(const NuKeeperStorage::RequestForSession & request_for_session); private: struct StorageSnapshot { - StorageSnapshot(const nuraft::ptr & s, const TestKeeperStorage & storage_) + StorageSnapshot(const nuraft::ptr & s, const NuKeeperStorage & storage_) : snapshot(s) , storage(storage_) {} nuraft::ptr snapshot; - TestKeeperStorage storage; + NuKeeperStorage storage; }; using StorageSnapshotPtr = std::shared_ptr; @@ -69,7 +69,7 @@ private: static void writeSnapshot(const StorageSnapshotPtr & snapshot, nuraft::ptr & out); - TestKeeperStorage storage; + NuKeeperStorage storage; /// Mutex for snapshots std::mutex snapshots_lock; diff --git a/src/Coordination/TestKeeperStorage.cpp b/src/Coordination/NuKeeperStorage.cpp similarity index 75% rename from src/Coordination/TestKeeperStorage.cpp rename to src/Coordination/NuKeeperStorage.cpp index ef72f5d4eaa..9a8b96d63a3 100644 --- a/src/Coordination/TestKeeperStorage.cpp +++ b/src/Coordination/NuKeeperStorage.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -31,9 +31,9 @@ static String baseName(const String & path) return path.substr(rslash_pos + 1); } -static TestKeeperStorage::ResponsesForSessions processWatchesImpl(const String & path, TestKeeperStorage::Watches & watches, TestKeeperStorage::Watches & list_watches, Coordination::Event event_type) +static NuKeeperStorage::ResponsesForSessions processWatchesImpl(const String & path, NuKeeperStorage::Watches & watches, NuKeeperStorage::Watches & list_watches, Coordination::Event event_type) { - TestKeeperStorage::ResponsesForSessions result; + NuKeeperStorage::ResponsesForSessions result; auto it = watches.find(path); if (it != watches.end()) { @@ -44,7 +44,7 @@ static TestKeeperStorage::ResponsesForSessions processWatchesImpl(const String & watch_response->type = event_type; watch_response->state = Coordination::State::CONNECTED; for (auto watcher_session : it->second) - result.push_back(TestKeeperStorage::ResponseForSession{watcher_session, watch_response}); + result.push_back(NuKeeperStorage::ResponseForSession{watcher_session, watch_response}); watches.erase(it); } @@ -60,52 +60,52 @@ static TestKeeperStorage::ResponsesForSessions processWatchesImpl(const String & watch_list_response->type = Coordination::Event::CHILD; watch_list_response->state = Coordination::State::CONNECTED; for (auto watcher_session : it->second) - result.push_back(TestKeeperStorage::ResponseForSession{watcher_session, watch_list_response}); + result.push_back(NuKeeperStorage::ResponseForSession{watcher_session, watch_list_response}); list_watches.erase(it); } return result; } -TestKeeperStorage::TestKeeperStorage() +NuKeeperStorage::NuKeeperStorage() { container.emplace("/", Node()); } using Undo = std::function; -struct TestKeeperStorageRequest +struct NuKeeperStorageRequest { Coordination::ZooKeeperRequestPtr zk_request; - explicit TestKeeperStorageRequest(const Coordination::ZooKeeperRequestPtr & zk_request_) + explicit NuKeeperStorageRequest(const Coordination::ZooKeeperRequestPtr & zk_request_) : zk_request(zk_request_) {} - virtual std::pair process(TestKeeperStorage::Container & container, TestKeeperStorage::Ephemerals & ephemerals, int64_t zxid, int64_t session_id) const = 0; - virtual TestKeeperStorage::ResponsesForSessions processWatches(TestKeeperStorage::Watches & /*watches*/, TestKeeperStorage::Watches & /*list_watches*/) const { return {}; } + virtual std::pair process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & ephemerals, int64_t zxid, int64_t session_id) const = 0; + virtual NuKeeperStorage::ResponsesForSessions processWatches(NuKeeperStorage::Watches & /*watches*/, NuKeeperStorage::Watches & /*list_watches*/) const { return {}; } - virtual ~TestKeeperStorageRequest() = default; + virtual ~NuKeeperStorageRequest() = default; }; -struct TestKeeperStorageHeartbeatRequest final : public TestKeeperStorageRequest +struct NuKeeperStorageHeartbeatRequest final : public NuKeeperStorageRequest { - using TestKeeperStorageRequest::TestKeeperStorageRequest; - std::pair process(TestKeeperStorage::Container & /* container */, TestKeeperStorage::Ephemerals & /* ephemerals */, int64_t /* zxid */, int64_t /* session_id */) const override + using NuKeeperStorageRequest::NuKeeperStorageRequest; + std::pair process(NuKeeperStorage::Container & /* container */, NuKeeperStorage::Ephemerals & /* ephemerals */, int64_t /* zxid */, int64_t /* session_id */) const override { return {zk_request->makeResponse(), {}}; } }; -struct TestKeeperStorageCreateRequest final : public TestKeeperStorageRequest +struct NuKeeperStorageCreateRequest final : public NuKeeperStorageRequest { - using TestKeeperStorageRequest::TestKeeperStorageRequest; + using NuKeeperStorageRequest::NuKeeperStorageRequest; - TestKeeperStorage::ResponsesForSessions processWatches(TestKeeperStorage::Watches & watches, TestKeeperStorage::Watches & list_watches) const override + NuKeeperStorage::ResponsesForSessions processWatches(NuKeeperStorage::Watches & watches, NuKeeperStorage::Watches & list_watches) const override { return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::CREATED); } - std::pair process(TestKeeperStorage::Container & container, TestKeeperStorage::Ephemerals & ephemerals, int64_t zxid, int64_t session_id) const override + std::pair process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & ephemerals, int64_t zxid, int64_t session_id) const override { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Undo undo; @@ -130,7 +130,7 @@ struct TestKeeperStorageCreateRequest final : public TestKeeperStorageRequest } else { - TestKeeperStorage::Node created_node; + NuKeeperStorage::Node created_node; created_node.seq_num = 0; created_node.stat.czxid = zxid; created_node.stat.mzxid = zxid; @@ -185,10 +185,10 @@ struct TestKeeperStorageCreateRequest final : public TestKeeperStorageRequest } }; -struct TestKeeperStorageGetRequest final : public TestKeeperStorageRequest +struct NuKeeperStorageGetRequest final : public NuKeeperStorageRequest { - using TestKeeperStorageRequest::TestKeeperStorageRequest; - std::pair process(TestKeeperStorage::Container & container, TestKeeperStorage::Ephemerals & /* ephemerals */, int64_t /* zxid */, int64_t /* session_id */) const override + using NuKeeperStorageRequest::NuKeeperStorageRequest; + std::pair process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & /* ephemerals */, int64_t /* zxid */, int64_t /* session_id */) const override { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperGetResponse & response = dynamic_cast(*response_ptr); @@ -210,10 +210,10 @@ struct TestKeeperStorageGetRequest final : public TestKeeperStorageRequest } }; -struct TestKeeperStorageRemoveRequest final : public TestKeeperStorageRequest +struct NuKeeperStorageRemoveRequest final : public NuKeeperStorageRequest { - using TestKeeperStorageRequest::TestKeeperStorageRequest; - std::pair process(TestKeeperStorage::Container & container, TestKeeperStorage::Ephemerals & ephemerals, int64_t /*zxid*/, int64_t session_id) const override + using NuKeeperStorageRequest::NuKeeperStorageRequest; + std::pair process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & ephemerals, int64_t /*zxid*/, int64_t session_id) const override { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperRemoveResponse & response = dynamic_cast(*response_ptr); @@ -260,16 +260,16 @@ struct TestKeeperStorageRemoveRequest final : public TestKeeperStorageRequest return { response_ptr, undo }; } - TestKeeperStorage::ResponsesForSessions processWatches(TestKeeperStorage::Watches & watches, TestKeeperStorage::Watches & list_watches) const override + NuKeeperStorage::ResponsesForSessions processWatches(NuKeeperStorage::Watches & watches, NuKeeperStorage::Watches & list_watches) const override { return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::DELETED); } }; -struct TestKeeperStorageExistsRequest final : public TestKeeperStorageRequest +struct NuKeeperStorageExistsRequest final : public NuKeeperStorageRequest { - using TestKeeperStorageRequest::TestKeeperStorageRequest; - std::pair process(TestKeeperStorage::Container & container, TestKeeperStorage::Ephemerals & /* ephemerals */, int64_t /*zxid*/, int64_t /* session_id */) const override + using NuKeeperStorageRequest::NuKeeperStorageRequest; + std::pair process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & /* ephemerals */, int64_t /*zxid*/, int64_t /* session_id */) const override { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperExistsResponse & response = dynamic_cast(*response_ptr); @@ -290,10 +290,10 @@ struct TestKeeperStorageExistsRequest final : public TestKeeperStorageRequest } }; -struct TestKeeperStorageSetRequest final : public TestKeeperStorageRequest +struct NuKeeperStorageSetRequest final : public NuKeeperStorageRequest { - using TestKeeperStorageRequest::TestKeeperStorageRequest; - std::pair process(TestKeeperStorage::Container & container, TestKeeperStorage::Ephemerals & /* ephemerals */, int64_t zxid, int64_t /* session_id */) const override + using NuKeeperStorageRequest::NuKeeperStorageRequest; + std::pair process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & /* ephemerals */, int64_t zxid, int64_t /* session_id */) const override { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperSetResponse & response = dynamic_cast(*response_ptr); @@ -333,17 +333,17 @@ struct TestKeeperStorageSetRequest final : public TestKeeperStorageRequest return { response_ptr, undo }; } - TestKeeperStorage::ResponsesForSessions processWatches(TestKeeperStorage::Watches & watches, TestKeeperStorage::Watches & list_watches) const override + NuKeeperStorage::ResponsesForSessions processWatches(NuKeeperStorage::Watches & watches, NuKeeperStorage::Watches & list_watches) const override { return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::CHANGED); } }; -struct TestKeeperStorageListRequest final : public TestKeeperStorageRequest +struct NuKeeperStorageListRequest final : public NuKeeperStorageRequest { - using TestKeeperStorageRequest::TestKeeperStorageRequest; - std::pair process(TestKeeperStorage::Container & container, TestKeeperStorage::Ephemerals & /* ephemerals */, int64_t /*zxid*/, int64_t /*session_id*/) const override + using NuKeeperStorageRequest::NuKeeperStorageRequest; + std::pair process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & /* ephemerals */, int64_t /*zxid*/, int64_t /*session_id*/) const override { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperListResponse & response = dynamic_cast(*response_ptr); @@ -379,10 +379,10 @@ struct TestKeeperStorageListRequest final : public TestKeeperStorageRequest } }; -struct TestKeeperStorageCheckRequest final : public TestKeeperStorageRequest +struct NuKeeperStorageCheckRequest final : public NuKeeperStorageRequest { - using TestKeeperStorageRequest::TestKeeperStorageRequest; - std::pair process(TestKeeperStorage::Container & container, TestKeeperStorage::Ephemerals & /* ephemerals */, int64_t /*zxid*/, int64_t /*session_id*/) const override + using NuKeeperStorageRequest::NuKeeperStorageRequest; + std::pair process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & /* ephemerals */, int64_t /*zxid*/, int64_t /*session_id*/) const override { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperCheckResponse & response = dynamic_cast(*response_ptr); @@ -405,11 +405,11 @@ struct TestKeeperStorageCheckRequest final : public TestKeeperStorageRequest } }; -struct TestKeeperStorageMultiRequest final : public TestKeeperStorageRequest +struct NuKeeperStorageMultiRequest final : public NuKeeperStorageRequest { - std::vector concrete_requests; - explicit TestKeeperStorageMultiRequest(const Coordination::ZooKeeperRequestPtr & zk_request_) - : TestKeeperStorageRequest(zk_request_) + std::vector concrete_requests; + explicit NuKeeperStorageMultiRequest(const Coordination::ZooKeeperRequestPtr & zk_request_) + : NuKeeperStorageRequest(zk_request_) { Coordination::ZooKeeperMultiRequest & request = dynamic_cast(*zk_request); concrete_requests.reserve(request.requests.size()); @@ -419,26 +419,26 @@ struct TestKeeperStorageMultiRequest final : public TestKeeperStorageRequest auto sub_zk_request = std::dynamic_pointer_cast(sub_request); if (sub_zk_request->getOpNum() == Coordination::OpNum::Create) { - concrete_requests.push_back(std::make_shared(sub_zk_request)); + concrete_requests.push_back(std::make_shared(sub_zk_request)); } else if (sub_zk_request->getOpNum() == Coordination::OpNum::Remove) { - concrete_requests.push_back(std::make_shared(sub_zk_request)); + concrete_requests.push_back(std::make_shared(sub_zk_request)); } else if (sub_zk_request->getOpNum() == Coordination::OpNum::Set) { - concrete_requests.push_back(std::make_shared(sub_zk_request)); + concrete_requests.push_back(std::make_shared(sub_zk_request)); } else if (sub_zk_request->getOpNum() == Coordination::OpNum::Check) { - concrete_requests.push_back(std::make_shared(sub_zk_request)); + concrete_requests.push_back(std::make_shared(sub_zk_request)); } else throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal command as part of multi ZooKeeper request {}", sub_zk_request->getOpNum()); } } - std::pair process(TestKeeperStorage::Container & container, TestKeeperStorage::Ephemerals & ephemerals, int64_t zxid, int64_t session_id) const override + std::pair process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & ephemerals, int64_t zxid, int64_t session_id) const override { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperMultiResponse & response = dynamic_cast(*response_ptr); @@ -491,9 +491,9 @@ struct TestKeeperStorageMultiRequest final : public TestKeeperStorageRequest } } - TestKeeperStorage::ResponsesForSessions processWatches(TestKeeperStorage::Watches & watches, TestKeeperStorage::Watches & list_watches) const override + NuKeeperStorage::ResponsesForSessions processWatches(NuKeeperStorage::Watches & watches, NuKeeperStorage::Watches & list_watches) const override { - TestKeeperStorage::ResponsesForSessions result; + NuKeeperStorage::ResponsesForSessions result; for (const auto & generic_request : concrete_requests) { auto responses = generic_request->processWatches(watches, list_watches); @@ -503,16 +503,16 @@ struct TestKeeperStorageMultiRequest final : public TestKeeperStorageRequest } }; -struct TestKeeperStorageCloseRequest final : public TestKeeperStorageRequest +struct NuKeeperStorageCloseRequest final : public NuKeeperStorageRequest { - using TestKeeperStorageRequest::TestKeeperStorageRequest; - std::pair process(TestKeeperStorage::Container &, TestKeeperStorage::Ephemerals &, int64_t, int64_t) const override + using NuKeeperStorageRequest::NuKeeperStorageRequest; + std::pair process(NuKeeperStorage::Container &, NuKeeperStorage::Ephemerals &, int64_t, int64_t) const override { throw DB::Exception("Called process on close request", ErrorCodes::LOGICAL_ERROR); } }; -TestKeeperStorage::ResponsesForSessions TestKeeperStorage::finalize(const RequestsForSessions & expired_requests) +NuKeeperStorage::ResponsesForSessions NuKeeperStorage::finalize(const RequestsForSessions & expired_requests) { if (finalized) throw DB::Exception("Testkeeper storage already finalized", ErrorCodes::LOGICAL_ERROR); @@ -559,20 +559,20 @@ TestKeeperStorage::ResponsesForSessions TestKeeperStorage::finalize(const Reques } -class TestKeeperWrapperFactory final : private boost::noncopyable +class NuKeeperWrapperFactory final : private boost::noncopyable { public: - using Creator = std::function; + using Creator = std::function; using OpNumToRequest = std::unordered_map; - static TestKeeperWrapperFactory & instance() + static NuKeeperWrapperFactory & instance() { - static TestKeeperWrapperFactory factory; + static NuKeeperWrapperFactory factory; return factory; } - TestKeeperStorageRequestPtr get(const Coordination::ZooKeeperRequestPtr & zk_request) const + NuKeeperStorageRequestPtr get(const Coordination::ZooKeeperRequestPtr & zk_request) const { auto it = op_num_to_request.find(zk_request->getOpNum()); if (it == op_num_to_request.end()) @@ -589,36 +589,36 @@ public: private: OpNumToRequest op_num_to_request; - TestKeeperWrapperFactory(); + NuKeeperWrapperFactory(); }; template -void registerTestKeeperRequestWrapper(TestKeeperWrapperFactory & factory) +void registerNuKeeperRequestWrapper(NuKeeperWrapperFactory & factory) { factory.registerRequest(num, [] (const Coordination::ZooKeeperRequestPtr & zk_request) { return std::make_shared(zk_request); }); } -TestKeeperWrapperFactory::TestKeeperWrapperFactory() +NuKeeperWrapperFactory::NuKeeperWrapperFactory() { - registerTestKeeperRequestWrapper(*this); - //registerTestKeeperRequestWrapper(*this); - registerTestKeeperRequestWrapper(*this); - registerTestKeeperRequestWrapper(*this); - registerTestKeeperRequestWrapper(*this); - registerTestKeeperRequestWrapper(*this); - registerTestKeeperRequestWrapper(*this); - registerTestKeeperRequestWrapper(*this); - registerTestKeeperRequestWrapper(*this); - registerTestKeeperRequestWrapper(*this); - registerTestKeeperRequestWrapper(*this); - registerTestKeeperRequestWrapper(*this); + registerNuKeeperRequestWrapper(*this); + //registerNuKeeperRequestWrapper(*this); + registerNuKeeperRequestWrapper(*this); + registerNuKeeperRequestWrapper(*this); + registerNuKeeperRequestWrapper(*this); + registerNuKeeperRequestWrapper(*this); + registerNuKeeperRequestWrapper(*this); + registerNuKeeperRequestWrapper(*this); + registerNuKeeperRequestWrapper(*this); + registerNuKeeperRequestWrapper(*this); + registerNuKeeperRequestWrapper(*this); + registerNuKeeperRequestWrapper(*this); } -TestKeeperStorage::ResponsesForSessions TestKeeperStorage::processRequest(const Coordination::ZooKeeperRequestPtr & zk_request, int64_t session_id) +NuKeeperStorage::ResponsesForSessions NuKeeperStorage::processRequest(const Coordination::ZooKeeperRequestPtr & zk_request, int64_t session_id) { - TestKeeperStorage::ResponsesForSessions results; + NuKeeperStorage::ResponsesForSessions results; if (zk_request->getOpNum() == Coordination::OpNum::Close) { auto it = ephemerals.find(session_id); @@ -643,7 +643,7 @@ TestKeeperStorage::ResponsesForSessions TestKeeperStorage::processRequest(const else { - TestKeeperStorageRequestPtr storage_request = TestKeeperWrapperFactory::instance().get(zk_request); + NuKeeperStorageRequestPtr storage_request = NuKeeperWrapperFactory::instance().get(zk_request); auto [response, _] = storage_request->process(container, ephemerals, zxid, session_id); if (zk_request->has_watch) @@ -689,7 +689,7 @@ TestKeeperStorage::ResponsesForSessions TestKeeperStorage::processRequest(const } -void TestKeeperStorage::clearDeadWatches(int64_t session_id) +void NuKeeperStorage::clearDeadWatches(int64_t session_id) { auto watches_it = sessions_and_watchers.find(session_id); if (watches_it != sessions_and_watchers.end()) diff --git a/src/Coordination/TestKeeperStorage.h b/src/Coordination/NuKeeperStorage.h similarity index 92% rename from src/Coordination/TestKeeperStorage.h rename to src/Coordination/NuKeeperStorage.h index 6f70ff1c584..dce00391bce 100644 --- a/src/Coordination/TestKeeperStorage.h +++ b/src/Coordination/NuKeeperStorage.h @@ -12,11 +12,11 @@ namespace DB { using namespace DB; -struct TestKeeperStorageRequest; -using TestKeeperStorageRequestPtr = std::shared_ptr; +struct NuKeeperStorageRequest; +using NuKeeperStorageRequestPtr = std::shared_ptr; using ResponseCallback = std::function; -class TestKeeperStorage +class NuKeeperStorage { public: int64_t session_id_counter{0}; @@ -72,7 +72,7 @@ public: } public: - TestKeeperStorage(); + NuKeeperStorage(); int64_t getSessionID() { diff --git a/src/Coordination/TestKeeperStorageDispatcher.cpp b/src/Coordination/NuKeeperStorageDispatcher.cpp similarity index 76% rename from src/Coordination/TestKeeperStorageDispatcher.cpp rename to src/Coordination/NuKeeperStorageDispatcher.cpp index d9f9dfd30eb..c531939d6ee 100644 --- a/src/Coordination/TestKeeperStorageDispatcher.cpp +++ b/src/Coordination/NuKeeperStorageDispatcher.cpp @@ -1,4 +1,4 @@ -#include +#include #include namespace DB @@ -11,17 +11,17 @@ namespace ErrorCodes extern const int TIMEOUT_EXCEEDED; } -TestKeeperStorageDispatcher::TestKeeperStorageDispatcher() - : log(&Poco::Logger::get("TestKeeperDispatcher")) +NuKeeperStorageDispatcher::NuKeeperStorageDispatcher() + : log(&Poco::Logger::get("NuKeeperDispatcher")) { } -void TestKeeperStorageDispatcher::processingThread() +void NuKeeperStorageDispatcher::processingThread() { - setThreadName("TestKeeperSProc"); + setThreadName("NuKeeperSProc"); while (!shutdown_called) { - TestKeeperStorage::RequestForSession request; + NuKeeperStorage::RequestForSession request; UInt64 max_wait = UInt64(operation_timeout.totalMilliseconds()); @@ -44,7 +44,7 @@ void TestKeeperStorageDispatcher::processingThread() } } -void TestKeeperStorageDispatcher::setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response) +void NuKeeperStorageDispatcher::setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response) { std::lock_guard lock(session_to_response_callback_mutex); auto session_writer = session_to_response_callback.find(session_id); @@ -57,7 +57,7 @@ void TestKeeperStorageDispatcher::setResponse(int64_t session_id, const Coordina session_to_response_callback.erase(session_writer); } -bool TestKeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id) +bool NuKeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id) { { @@ -66,7 +66,7 @@ bool TestKeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperReques return false; } - TestKeeperStorage::RequestForSession request_info; + NuKeeperStorage::RequestForSession request_info; request_info.request = request; request_info.session_id = session_id; @@ -104,27 +104,27 @@ namespace } } -void TestKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfiguration & config) +void NuKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfiguration & config) { LOG_DEBUG(log, "Initializing storage dispatcher"); - int myid = config.getInt("test_keeper_server.server_id"); + int myid = config.getInt("nu_keeper_server.server_id"); std::string myhostname; int myport; int32_t my_priority = 1; Poco::Util::AbstractConfiguration::Keys keys; - config.keys("test_keeper_server.raft_configuration", keys); + config.keys("nu_keeper_server.raft_configuration", keys); bool my_can_become_leader = true; std::vector> server_configs; std::vector ids; for (const auto & server_key : keys) { - int server_id = config.getInt("test_keeper_server.raft_configuration." + server_key + ".id"); - std::string hostname = config.getString("test_keeper_server.raft_configuration." + server_key + ".hostname"); - int port = config.getInt("test_keeper_server.raft_configuration." + server_key + ".port"); - bool can_become_leader = config.getBool("test_keeper_server.raft_configuration." + server_key + ".can_become_leader", true); - int32_t priority = config.getInt("test_keeper_server.raft_configuration." + server_key + ".priority", 1); + int server_id = config.getInt("nu_keeper_server.raft_configuration." + server_key + ".id"); + std::string hostname = config.getString("nu_keeper_server.raft_configuration." + server_key + ".hostname"); + int port = config.getInt("nu_keeper_server.raft_configuration." + server_key + ".port"); + bool can_become_leader = config.getBool("nu_keeper_server.raft_configuration." + server_key + ".can_become_leader", true); + int32_t priority = config.getInt("nu_keeper_server.raft_configuration." + server_key + ".priority", 1); if (server_id == myid) { myhostname = hostname; @@ -175,7 +175,7 @@ void TestKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigura LOG_DEBUG(log, "Dispatcher initialized"); } -void TestKeeperStorageDispatcher::shutdown() +void NuKeeperStorageDispatcher::shutdown() { try { @@ -194,10 +194,10 @@ void TestKeeperStorageDispatcher::shutdown() if (server) { - TestKeeperStorage::RequestsForSessions expired_requests; - TestKeeperStorage::RequestForSession request; + NuKeeperStorage::RequestsForSessions expired_requests; + NuKeeperStorage::RequestForSession request; while (requests_queue.tryPop(request)) - expired_requests.push_back(TestKeeperStorage::RequestForSession{request}); + expired_requests.push_back(NuKeeperStorage::RequestForSession{request}); auto expired_responses = server->shutdown(expired_requests); @@ -213,19 +213,19 @@ void TestKeeperStorageDispatcher::shutdown() LOG_DEBUG(log, "Dispatcher shut down"); } -TestKeeperStorageDispatcher::~TestKeeperStorageDispatcher() +NuKeeperStorageDispatcher::~NuKeeperStorageDispatcher() { shutdown(); } -void TestKeeperStorageDispatcher::registerSession(int64_t session_id, ZooKeeperResponseCallback callback) +void NuKeeperStorageDispatcher::registerSession(int64_t session_id, ZooKeeperResponseCallback callback) { std::lock_guard lock(session_to_response_callback_mutex); if (!session_to_response_callback.try_emplace(session_id, callback).second) throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Session with id {} already registered in dispatcher", session_id); } -void TestKeeperStorageDispatcher::finishSession(int64_t session_id) +void NuKeeperStorageDispatcher::finishSession(int64_t session_id) { std::lock_guard lock(session_to_response_callback_mutex); auto session_it = session_to_response_callback.find(session_id); diff --git a/src/Coordination/TestKeeperStorageDispatcher.h b/src/Coordination/NuKeeperStorageDispatcher.h similarity index 90% rename from src/Coordination/TestKeeperStorageDispatcher.h rename to src/Coordination/NuKeeperStorageDispatcher.h index ddb90abb88a..c292cd99c4f 100644 --- a/src/Coordination/TestKeeperStorageDispatcher.h +++ b/src/Coordination/NuKeeperStorageDispatcher.h @@ -21,7 +21,7 @@ namespace DB using ZooKeeperResponseCallback = std::function; -class TestKeeperStorageDispatcher +class NuKeeperStorageDispatcher { private: @@ -30,7 +30,7 @@ private: std::mutex push_request_mutex; - using RequestsQueue = ConcurrentBoundedQueue; + using RequestsQueue = ConcurrentBoundedQueue; RequestsQueue requests_queue{1}; std::atomic shutdown_called{false}; using SessionToResponseCallback = std::unordered_map; @@ -49,13 +49,13 @@ private: void setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response); public: - TestKeeperStorageDispatcher(); + NuKeeperStorageDispatcher(); void initialize(const Poco::Util::AbstractConfiguration & config); void shutdown(); - ~TestKeeperStorageDispatcher(); + ~NuKeeperStorageDispatcher(); bool putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id); diff --git a/src/Coordination/TestKeeperStorageSerializer.cpp b/src/Coordination/NuKeeperStorageSerializer.cpp similarity index 84% rename from src/Coordination/TestKeeperStorageSerializer.cpp rename to src/Coordination/NuKeeperStorageSerializer.cpp index f6116d29104..298df45cde0 100644 --- a/src/Coordination/TestKeeperStorageSerializer.cpp +++ b/src/Coordination/NuKeeperStorageSerializer.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -8,7 +8,7 @@ namespace DB namespace { - void writeNode(const TestKeeperStorage::Node & node, WriteBuffer & out) + void writeNode(const NuKeeperStorage::Node & node, WriteBuffer & out) { Coordination::write(node.data, out); Coordination::write(node.acls, out); @@ -18,7 +18,7 @@ namespace Coordination::write(node.seq_num, out); } - void readNode(TestKeeperStorage::Node & node, ReadBuffer & in) + void readNode(NuKeeperStorage::Node & node, ReadBuffer & in) { Coordination::read(node.data, in); Coordination::read(node.acls, in); @@ -29,7 +29,7 @@ namespace } } -void TestKeeperStorageSerializer::serialize(const TestKeeperStorage & storage, WriteBuffer & out) +void NuKeeperStorageSerializer::serialize(const NuKeeperStorage & storage, WriteBuffer & out) { Coordination::write(storage.zxid, out); Coordination::write(storage.session_id_counter, out); @@ -49,7 +49,7 @@ void TestKeeperStorageSerializer::serialize(const TestKeeperStorage & storage, W } } -void TestKeeperStorageSerializer::deserialize(TestKeeperStorage & storage, ReadBuffer & in) +void NuKeeperStorageSerializer::deserialize(NuKeeperStorage & storage, ReadBuffer & in) { int64_t session_id_counter, zxid; Coordination::read(zxid, in); @@ -63,7 +63,7 @@ void TestKeeperStorageSerializer::deserialize(TestKeeperStorage & storage, ReadB { std::string path; Coordination::read(path, in); - TestKeeperStorage::Node node; + NuKeeperStorage::Node node; readNode(node, in); storage.container[path] = node; } diff --git a/src/Coordination/NuKeeperStorageSerializer.h b/src/Coordination/NuKeeperStorageSerializer.h new file mode 100644 index 00000000000..e54c65a739d --- /dev/null +++ b/src/Coordination/NuKeeperStorageSerializer.h @@ -0,0 +1,17 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +class NuKeeperStorageSerializer +{ +public: + static void serialize(const NuKeeperStorage & storage, WriteBuffer & out); + + static void deserialize(NuKeeperStorage & storage, ReadBuffer & in); +}; + +} diff --git a/src/Coordination/TestKeeperStorageSerializer.h b/src/Coordination/TestKeeperStorageSerializer.h deleted file mode 100644 index a3909c24694..00000000000 --- a/src/Coordination/TestKeeperStorageSerializer.h +++ /dev/null @@ -1,17 +0,0 @@ -#pragma once -#include -#include -#include - -namespace DB -{ - -class TestKeeperStorageSerializer -{ -public: - static void serialize(const TestKeeperStorage & storage, WriteBuffer & out); - - static void deserialize(TestKeeperStorage & storage, ReadBuffer & in); -}; - -} diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index d69f2c18bd4..d2f4938dfd3 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -9,7 +9,7 @@ #include #include -#include +#include #include #include #include @@ -283,9 +283,9 @@ nuraft::ptr getZooKeeperLogEntry(int64_t session_id, const Coord return buf.getBuffer(); } -DB::TestKeeperStorage::ResponsesForSessions getZooKeeperResponses(nuraft::ptr & buffer, const Coordination::ZooKeeperRequestPtr & request) +DB::NuKeeperStorage::ResponsesForSessions getZooKeeperResponses(nuraft::ptr & buffer, const Coordination::ZooKeeperRequestPtr & request) { - DB::TestKeeperStorage::ResponsesForSessions results; + DB::NuKeeperStorage::ResponsesForSessions results; DB::ReadBufferFromNuraftBuffer buf(buffer); while (!buf.eof()) { @@ -303,28 +303,28 @@ DB::TestKeeperStorage::ResponsesForSessions getZooKeeperResponses(nuraft::ptrmakeResponse(); response->readImpl(buf); - results.push_back(DB::TestKeeperStorage::ResponseForSession{session_id, response}); + results.push_back(DB::NuKeeperStorage::ResponseForSession{session_id, response}); } return results; } TEST(CoordinationTest, TestStorageSerialization) { - DB::TestKeeperStorage storage; - storage.container["/hello"] = DB::TestKeeperStorage::Node{.data="world"}; - storage.container["/hello/somepath"] = DB::TestKeeperStorage::Node{.data="somedata"}; + DB::NuKeeperStorage storage; + storage.container["/hello"] = DB::NuKeeperStorage::Node{.data="world"}; + storage.container["/hello/somepath"] = DB::NuKeeperStorage::Node{.data="somedata"}; storage.session_id_counter = 5; storage.zxid = 156; storage.ephemerals[3] = {"/hello", "/"}; storage.ephemerals[1] = {"/hello/somepath"}; DB::WriteBufferFromOwnString buffer; - DB::TestKeeperStorageSerializer serializer; + DB::NuKeeperStorageSerializer serializer; serializer.serialize(storage, buffer); std::string serialized = buffer.str(); EXPECT_NE(serialized.size(), 0); DB::ReadBufferFromString read(serialized); - DB::TestKeeperStorage new_storage; + DB::NuKeeperStorage new_storage; serializer.deserialize(new_storage, read); EXPECT_EQ(new_storage.container.size(), 3); diff --git a/src/Coordination/ya.make b/src/Coordination/ya.make index de2be9df7ac..833ca27f2f4 100644 --- a/src/Coordination/ya.make +++ b/src/Coordination/ya.make @@ -14,10 +14,10 @@ SRCS( InMemoryStateManager.cpp NuKeeperServer.cpp NuKeeperStateMachine.cpp + NuKeeperStorage.cpp + NuKeeperStorageDispatcher.cpp + NuKeeperStorageSerializer.cpp SummingStateMachine.cpp - TestKeeperStorage.cpp - TestKeeperStorageDispatcher.cpp - TestKeeperStorageSerializer.cpp WriteBufferFromNuraftBuffer.cpp ) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index fe1b6a8a32e..983ac733849 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include #include #include @@ -305,8 +305,8 @@ struct ContextShared ConfigurationPtr zookeeper_config; /// Stores zookeeper configs #if USE_NURAFT - mutable std::mutex test_keeper_storage_dispatcher_mutex; - mutable std::shared_ptr test_keeper_storage_dispatcher; + mutable std::mutex nu_keeper_storage_dispatcher_mutex; + mutable std::shared_ptr nu_keeper_storage_dispatcher; #endif mutable std::mutex auxiliary_zookeepers_mutex; mutable std::map auxiliary_zookeepers; /// Map for auxiliary ZooKeeper clients. @@ -1582,42 +1582,42 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const } -void Context::initializeTestKeeperStorageDispatcher() const +void Context::initializeNuKeeperStorageDispatcher() const { #if USE_NURAFT - std::lock_guard lock(shared->test_keeper_storage_dispatcher_mutex); + std::lock_guard lock(shared->nu_keeper_storage_dispatcher_mutex); - if (shared->test_keeper_storage_dispatcher) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to initialize TestKeeper multiple times"); + if (shared->nu_keeper_storage_dispatcher) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to initialize NuKeeper multiple times"); const auto & config = getConfigRef(); - if (config.has("test_keeper_server")) + if (config.has("nu_keeper_server")) { - shared->test_keeper_storage_dispatcher = std::make_shared(); - shared->test_keeper_storage_dispatcher->initialize(config); + shared->nu_keeper_storage_dispatcher = std::make_shared(); + shared->nu_keeper_storage_dispatcher->initialize(config); } #endif } #if USE_NURAFT -std::shared_ptr & Context::getTestKeeperStorageDispatcher() const +std::shared_ptr & Context::getNuKeeperStorageDispatcher() const { - std::lock_guard lock(shared->test_keeper_storage_dispatcher_mutex); - if (!shared->test_keeper_storage_dispatcher) - throw Exception(ErrorCodes::LOGICAL_ERROR, "TestKeeper must be initialized before requests"); + std::lock_guard lock(shared->nu_keeper_storage_dispatcher_mutex); + if (!shared->nu_keeper_storage_dispatcher) + throw Exception(ErrorCodes::LOGICAL_ERROR, "NuKeeper must be initialized before requests"); - return shared->test_keeper_storage_dispatcher; + return shared->nu_keeper_storage_dispatcher; } #endif -void Context::shutdownTestKeeperStorageDispatcher() const +void Context::shutdownNuKeeperStorageDispatcher() const { #if USE_NURAFT - std::lock_guard lock(shared->test_keeper_storage_dispatcher_mutex); - if (shared->test_keeper_storage_dispatcher) + std::lock_guard lock(shared->nu_keeper_storage_dispatcher_mutex); + if (shared->nu_keeper_storage_dispatcher) { - shared->test_keeper_storage_dispatcher->shutdown(); - shared->test_keeper_storage_dispatcher.reset(); + shared->nu_keeper_storage_dispatcher->shutdown(); + shared->nu_keeper_storage_dispatcher.reset(); } #endif } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 3c78973b21a..446c64f1bbd 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -106,7 +106,7 @@ using StoragePolicyPtr = std::shared_ptr; using StoragePoliciesMap = std::map; class StoragePolicySelector; using StoragePolicySelectorPtr = std::shared_ptr; -class TestKeeperStorageDispatcher; +class NuKeeperStorageDispatcher; class IOutputFormat; using OutputFormatPtr = std::shared_ptr; @@ -574,10 +574,10 @@ public: std::shared_ptr getAuxiliaryZooKeeper(const String & name) const; #if USE_NURAFT - std::shared_ptr & getTestKeeperStorageDispatcher() const; + std::shared_ptr & getNuKeeperStorageDispatcher() const; #endif - void initializeTestKeeperStorageDispatcher() const; - void shutdownTestKeeperStorageDispatcher() const; + void initializeNuKeeperStorageDispatcher() const; + void shutdownNuKeeperStorageDispatcher() const; /// Set auxiliary zookeepers configuration at server starting or configuration reloading. void reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & config); diff --git a/src/Server/TestKeeperTCPHandler.cpp b/src/Server/NuKeeperTCPHandler.cpp similarity index 92% rename from src/Server/TestKeeperTCPHandler.cpp rename to src/Server/NuKeeperTCPHandler.cpp index 17f2ec547c9..6deee5094ca 100644 --- a/src/Server/TestKeeperTCPHandler.cpp +++ b/src/Server/NuKeeperTCPHandler.cpp @@ -1,4 +1,4 @@ -#include +#include #if USE_NURAFT @@ -224,20 +224,20 @@ struct SocketInterruptablePollWrapper #endif }; -TestKeeperTCPHandler::TestKeeperTCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_) +NuKeeperTCPHandler::NuKeeperTCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_) : Poco::Net::TCPServerConnection(socket_) , server(server_) - , log(&Poco::Logger::get("TestKeeperTCPHandler")) + , log(&Poco::Logger::get("NuKeeperTCPHandler")) , global_context(server.context()) - , test_keeper_storage_dispatcher(global_context.getTestKeeperStorageDispatcher()) - , operation_timeout(0, global_context.getConfigRef().getUInt("test_keeper_server.operation_timeout_ms", Coordination::DEFAULT_OPERATION_TIMEOUT_MS) * 1000) - , session_timeout(0, global_context.getConfigRef().getUInt("test_keeper_server.session_timeout_ms", Coordination::DEFAULT_SESSION_TIMEOUT_MS) * 1000) + , nu_keeper_storage_dispatcher(global_context.getNuKeeperStorageDispatcher()) + , operation_timeout(0, global_context.getConfigRef().getUInt("nu_keeper_server.operation_timeout_ms", Coordination::DEFAULT_OPERATION_TIMEOUT_MS) * 1000) + , session_timeout(0, global_context.getConfigRef().getUInt("nu_keeper_server.session_timeout_ms", Coordination::DEFAULT_SESSION_TIMEOUT_MS) * 1000) , poll_wrapper(std::make_unique(socket_)) , responses(std::make_unique()) { } -void TestKeeperTCPHandler::sendHandshake(bool has_leader) +void NuKeeperTCPHandler::sendHandshake(bool has_leader) { Coordination::write(Coordination::SERVER_HANDSHAKE_LENGTH, *out); if (has_leader) @@ -252,12 +252,12 @@ void TestKeeperTCPHandler::sendHandshake(bool has_leader) out->next(); } -void TestKeeperTCPHandler::run() +void NuKeeperTCPHandler::run() { runImpl(); } -void TestKeeperTCPHandler::receiveHandshake() +void NuKeeperTCPHandler::receiveHandshake() { int32_t handshake_length; int32_t protocol_version; @@ -294,7 +294,7 @@ void TestKeeperTCPHandler::receiveHandshake() } -void TestKeeperTCPHandler::runImpl() +void NuKeeperTCPHandler::runImpl() { setThreadName("TstKprHandler"); ThreadStatus thread_status; @@ -324,11 +324,11 @@ void TestKeeperTCPHandler::runImpl() return; } - if (test_keeper_storage_dispatcher->hasLeader()) + if (nu_keeper_storage_dispatcher->hasLeader()) { try { - session_id = test_keeper_storage_dispatcher->getSessionID(); + session_id = nu_keeper_storage_dispatcher->getSessionID(); } catch (const Exception & e) { @@ -354,7 +354,7 @@ void TestKeeperTCPHandler::runImpl() UInt8 single_byte = 1; [[maybe_unused]] int result = write(response_fd, &single_byte, sizeof(single_byte)); }; - test_keeper_storage_dispatcher->registerSession(session_id, response_callback); + nu_keeper_storage_dispatcher->registerSession(session_id, response_callback); session_stopwatch.start(); bool close_received = false; @@ -428,18 +428,18 @@ void TestKeeperTCPHandler::runImpl() } } -void TestKeeperTCPHandler::finish() +void NuKeeperTCPHandler::finish() { Coordination::ZooKeeperRequestPtr request = Coordination::ZooKeeperRequestFactory::instance().get(Coordination::OpNum::Close); request->xid = close_xid; /// Put close request (so storage will remove all info about session) - test_keeper_storage_dispatcher->putRequest(request, session_id); + nu_keeper_storage_dispatcher->putRequest(request, session_id); /// We don't need any callbacks because session can be already dead and /// nobody wait for response - test_keeper_storage_dispatcher->finishSession(session_id); + nu_keeper_storage_dispatcher->finishSession(session_id); } -std::pair TestKeeperTCPHandler::receiveRequest() +std::pair NuKeeperTCPHandler::receiveRequest() { int32_t length; Coordination::read(length, *in); @@ -453,7 +453,7 @@ std::pair TestKeeperTCPHandler::receiveR request->xid = xid; request->readImpl(*in); - if (!test_keeper_storage_dispatcher->putRequest(request, session_id)) + if (!nu_keeper_storage_dispatcher->putRequest(request, session_id)) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Session {} already disconnected", session_id); return std::make_pair(opnum, xid); } diff --git a/src/Server/TestKeeperTCPHandler.h b/src/Server/NuKeeperTCPHandler.h similarity index 83% rename from src/Server/TestKeeperTCPHandler.h rename to src/Server/NuKeeperTCPHandler.h index 09543b5a888..1874b8cd309 100644 --- a/src/Server/TestKeeperTCPHandler.h +++ b/src/Server/NuKeeperTCPHandler.h @@ -13,7 +13,7 @@ #include #include #include -#include +#include #include #include #include @@ -26,16 +26,16 @@ using SocketInterruptablePollWrapperPtr = std::unique_ptr; -class TestKeeperTCPHandler : public Poco::Net::TCPServerConnection +class NuKeeperTCPHandler : public Poco::Net::TCPServerConnection { public: - TestKeeperTCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_); + NuKeeperTCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_); void run() override; private: IServer & server; Poco::Logger * log; Context global_context; - std::shared_ptr test_keeper_storage_dispatcher; + std::shared_ptr nu_keeper_storage_dispatcher; Poco::Timespan operation_timeout; Poco::Timespan session_timeout; int64_t session_id; diff --git a/src/Server/TestKeeperTCPHandlerFactory.h b/src/Server/NuKeeperTCPHandlerFactory.h similarity index 68% rename from src/Server/TestKeeperTCPHandlerFactory.h rename to src/Server/NuKeeperTCPHandlerFactory.h index a5bf6be8c8a..0fd86ebc21f 100644 --- a/src/Server/TestKeeperTCPHandlerFactory.h +++ b/src/Server/NuKeeperTCPHandlerFactory.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include @@ -9,7 +9,7 @@ namespace DB { -class TestKeeperTCPHandlerFactory : public Poco::Net::TCPServerConnectionFactory +class NuKeeperTCPHandlerFactory : public Poco::Net::TCPServerConnectionFactory { private: IServer & server; @@ -21,9 +21,9 @@ private: void run() override {} }; public: - TestKeeperTCPHandlerFactory(IServer & server_) + NuKeeperTCPHandlerFactory(IServer & server_) : server(server_) - , log(&Poco::Logger::get("TestKeeperTCPHandlerFactory")) + , log(&Poco::Logger::get("NuKeeperTCPHandlerFactory")) { } @@ -31,8 +31,8 @@ public: { try { - LOG_TRACE(log, "Test keeper request. Address: {}", socket.peerAddress().toString()); - return new TestKeeperTCPHandler(server, socket); + LOG_TRACE(log, "NuKeeper request. Address: {}", socket.peerAddress().toString()); + return new NuKeeperTCPHandler(server, socket); } catch (const Poco::Net::NetException &) { diff --git a/src/Server/ya.make b/src/Server/ya.make index 1e44577aea9..a0269e9ac84 100644 --- a/src/Server/ya.make +++ b/src/Server/ya.make @@ -17,6 +17,7 @@ SRCS( MySQLHandler.cpp MySQLHandlerFactory.cpp NotFoundHandler.cpp + NuKeeperTCPHandler.cpp PostgreSQLHandler.cpp PostgreSQLHandlerFactory.cpp PrometheusMetricsWriter.cpp @@ -25,7 +26,6 @@ SRCS( ReplicasStatusHandler.cpp StaticRequestHandler.cpp TCPHandler.cpp - TestKeeperTCPHandler.cpp WebUIRequestHandler.cpp ) diff --git a/tests/config/config.d/test_keeper_port.xml b/tests/config/config.d/nu_keeper_port.xml similarity index 88% rename from tests/config/config.d/test_keeper_port.xml rename to tests/config/config.d/nu_keeper_port.xml index fff60d749f6..afd22955a33 100644 --- a/tests/config/config.d/test_keeper_port.xml +++ b/tests/config/config.d/nu_keeper_port.xml @@ -1,5 +1,5 @@ - + 9181 10000 30000 @@ -11,5 +11,5 @@ 44444 - + diff --git a/tests/config/install.sh b/tests/config/install.sh index 9965e1fb1ad..6f620ef6404 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -29,7 +29,7 @@ ln -sf $SRC_PATH/config.d/graphite.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/database_atomic.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/max_concurrent_queries.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/test_cluster_with_incorrect_pw.xml $DEST_SERVER_PATH/config.d/ -ln -sf $SRC_PATH/config.d/test_keeper_port.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/nu_keeper_port.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/logging_no_rotate.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/tcp_with_proxy.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/top_level_domains_lists.xml $DEST_SERVER_PATH/config.d/ diff --git a/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml b/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml index fff60d749f6..afd22955a33 100644 --- a/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml +++ b/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml @@ -1,5 +1,5 @@ - + 9181 10000 30000 @@ -11,5 +11,5 @@ 44444 - + diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml index 81f68f50c7c..fde0d511886 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml @@ -1,5 +1,5 @@ - + 9181 10000 30000 @@ -27,5 +27,5 @@ 1 - + diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml index 73340973367..c6f4e7b5a22 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml @@ -1,5 +1,5 @@ - + 9181 10000 30000 @@ -27,5 +27,5 @@ 1 - + diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml index fbc51489d11..d1e8830c480 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml @@ -1,5 +1,5 @@ - + 9181 10000 30000 @@ -27,5 +27,5 @@ 1 - + From a8d30bedea4b2ccc00333c4d6621ab431985ae8d Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 1 Feb 2021 17:16:37 +0300 Subject: [PATCH 46/86] Missed config file --- programs/server/config.d/nu_keeper_port.xml | 1 + 1 file changed, 1 insertion(+) create mode 120000 programs/server/config.d/nu_keeper_port.xml diff --git a/programs/server/config.d/nu_keeper_port.xml b/programs/server/config.d/nu_keeper_port.xml new file mode 120000 index 00000000000..07f71c63435 --- /dev/null +++ b/programs/server/config.d/nu_keeper_port.xml @@ -0,0 +1 @@ +tests/config/config.d/nu_keeper_port.xml \ No newline at end of file From f6a8c90be269c9cdb5ed0cbb1c46838b2169dddd Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 1 Feb 2021 18:12:00 +0300 Subject: [PATCH 47/86] Fix config path --- programs/server/config.d/nu_keeper_port.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/config.d/nu_keeper_port.xml b/programs/server/config.d/nu_keeper_port.xml index 07f71c63435..8de0a309ff0 120000 --- a/programs/server/config.d/nu_keeper_port.xml +++ b/programs/server/config.d/nu_keeper_port.xml @@ -1 +1 @@ -tests/config/config.d/nu_keeper_port.xml \ No newline at end of file +../../../tests/config/config.d/nu_keeper_port.xml \ No newline at end of file From d265e3b4197a07235863f113f736d24de3e31fd9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 2 Feb 2021 11:02:25 +0300 Subject: [PATCH 48/86] Less timeouts --- programs/server/Server.cpp | 4 ++-- programs/server/config.d/nu_keeper_port.xml | 1 - programs/server/config.d/test_keeper_port.xml | 1 + src/Coordination/NuKeeperStorageDispatcher.cpp | 14 +++++++------- src/Coordination/ya.make | 10 ---------- src/Interpreters/Context.cpp | 2 +- .../{nu_keeper_port.xml => test_keeper_port.xml} | 4 ++-- .../configs/enable_test_keeper.xml | 4 ++-- .../configs/enable_test_keeper1.xml | 4 ++-- .../configs/enable_test_keeper2.xml | 4 ++-- .../configs/enable_test_keeper3.xml | 4 ++-- .../integration/test_testkeeper_multinode/test.py | 4 ++-- 12 files changed, 23 insertions(+), 33 deletions(-) delete mode 120000 programs/server/config.d/nu_keeper_port.xml create mode 120000 programs/server/config.d/test_keeper_port.xml rename tests/config/config.d/{nu_keeper_port.xml => test_keeper_port.xml} (88%) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index fb58e85d813..a96cb2b8973 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -844,7 +844,7 @@ int Server::main(const std::vector & /*args*/) listen_try = true; } - if (config().has("nu_keeper_server")) + if (config().has("test_keeper_server")) { #if USE_NURAFT /// Initialize test keeper RAFT. Do nothing if no nu_keeper_server in config. @@ -852,7 +852,7 @@ int Server::main(const std::vector & /*args*/) for (const auto & listen_host : listen_hosts) { /// TCP NuKeeper - const char * port_name = "nu_keeper_server.tcp_port"; + const char * port_name = "test_keeper_server.tcp_port"; createServer(listen_host, port_name, listen_try, [&](UInt16 port) { Poco::Net::ServerSocket socket; diff --git a/programs/server/config.d/nu_keeper_port.xml b/programs/server/config.d/nu_keeper_port.xml deleted file mode 120000 index 8de0a309ff0..00000000000 --- a/programs/server/config.d/nu_keeper_port.xml +++ /dev/null @@ -1 +0,0 @@ -../../../tests/config/config.d/nu_keeper_port.xml \ No newline at end of file diff --git a/programs/server/config.d/test_keeper_port.xml b/programs/server/config.d/test_keeper_port.xml new file mode 120000 index 00000000000..f3f721caae0 --- /dev/null +++ b/programs/server/config.d/test_keeper_port.xml @@ -0,0 +1 @@ +../../../tests/config/config.d/test_keeper_port.xml \ No newline at end of file diff --git a/src/Coordination/NuKeeperStorageDispatcher.cpp b/src/Coordination/NuKeeperStorageDispatcher.cpp index c531939d6ee..9988e0ac476 100644 --- a/src/Coordination/NuKeeperStorageDispatcher.cpp +++ b/src/Coordination/NuKeeperStorageDispatcher.cpp @@ -107,24 +107,24 @@ namespace void NuKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfiguration & config) { LOG_DEBUG(log, "Initializing storage dispatcher"); - int myid = config.getInt("nu_keeper_server.server_id"); + int myid = config.getInt("test_keeper_server.server_id"); std::string myhostname; int myport; int32_t my_priority = 1; Poco::Util::AbstractConfiguration::Keys keys; - config.keys("nu_keeper_server.raft_configuration", keys); + config.keys("test_keeper_server.raft_configuration", keys); bool my_can_become_leader = true; std::vector> server_configs; std::vector ids; for (const auto & server_key : keys) { - int server_id = config.getInt("nu_keeper_server.raft_configuration." + server_key + ".id"); - std::string hostname = config.getString("nu_keeper_server.raft_configuration." + server_key + ".hostname"); - int port = config.getInt("nu_keeper_server.raft_configuration." + server_key + ".port"); - bool can_become_leader = config.getBool("nu_keeper_server.raft_configuration." + server_key + ".can_become_leader", true); - int32_t priority = config.getInt("nu_keeper_server.raft_configuration." + server_key + ".priority", 1); + int server_id = config.getInt("test_keeper_server.raft_configuration." + server_key + ".id"); + std::string hostname = config.getString("test_keeper_server.raft_configuration." + server_key + ".hostname"); + int port = config.getInt("test_keeper_server.raft_configuration." + server_key + ".port"); + bool can_become_leader = config.getBool("test_keeper_server.raft_configuration." + server_key + ".can_become_leader", true); + int32_t priority = config.getInt("test_keeper_server.raft_configuration." + server_key + ".priority", 1); if (server_id == myid) { myhostname = hostname; diff --git a/src/Coordination/ya.make b/src/Coordination/ya.make index 833ca27f2f4..470fe8c75be 100644 --- a/src/Coordination/ya.make +++ b/src/Coordination/ya.make @@ -5,20 +5,10 @@ LIBRARY() PEERDIR( clickhouse/src/Common - contrib/libs/NuRaft ) - SRCS( - InMemoryLogStore.cpp - InMemoryStateManager.cpp - NuKeeperServer.cpp - NuKeeperStateMachine.cpp - NuKeeperStorage.cpp NuKeeperStorageDispatcher.cpp - NuKeeperStorageSerializer.cpp - SummingStateMachine.cpp - WriteBufferFromNuraftBuffer.cpp ) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 983ac733849..b913c3ed396 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1591,7 +1591,7 @@ void Context::initializeNuKeeperStorageDispatcher() const throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to initialize NuKeeper multiple times"); const auto & config = getConfigRef(); - if (config.has("nu_keeper_server")) + if (config.has("test_keeper_server")) { shared->nu_keeper_storage_dispatcher = std::make_shared(); shared->nu_keeper_storage_dispatcher->initialize(config); diff --git a/tests/config/config.d/nu_keeper_port.xml b/tests/config/config.d/test_keeper_port.xml similarity index 88% rename from tests/config/config.d/nu_keeper_port.xml rename to tests/config/config.d/test_keeper_port.xml index afd22955a33..fff60d749f6 100644 --- a/tests/config/config.d/nu_keeper_port.xml +++ b/tests/config/config.d/test_keeper_port.xml @@ -1,5 +1,5 @@ - + 9181 10000 30000 @@ -11,5 +11,5 @@ 44444 - + diff --git a/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml b/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml index afd22955a33..fff60d749f6 100644 --- a/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml +++ b/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml @@ -1,5 +1,5 @@ - + 9181 10000 30000 @@ -11,5 +11,5 @@ 44444 - + diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml index fde0d511886..81f68f50c7c 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml @@ -1,5 +1,5 @@ - + 9181 10000 30000 @@ -27,5 +27,5 @@ 1 - + diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml index c6f4e7b5a22..73340973367 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml @@ -1,5 +1,5 @@ - + 9181 10000 30000 @@ -27,5 +27,5 @@ 1 - + diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml index d1e8830c480..fbc51489d11 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml @@ -1,5 +1,5 @@ - + 9181 10000 30000 @@ -27,5 +27,5 @@ 1 - + diff --git a/tests/integration/test_testkeeper_multinode/test.py b/tests/integration/test_testkeeper_multinode/test.py index fe568e7252d..7b9430b2368 100644 --- a/tests/integration/test_testkeeper_multinode/test.py +++ b/tests/integration/test_testkeeper_multinode/test.py @@ -183,14 +183,14 @@ def test_blocade_leader_twice(started_cluster): # Total network partition pm.partition_instances(node3, node2) - for i in range(30): + for i in range(10): try: node3.query("INSERT INTO t2 SELECT rand() FROM numbers(100)") assert False, "Node3 became leader?" except Exception as ex: time.sleep(0.5) - for i in range(30): + for i in range(10): try: node2.query("INSERT INTO t2 SELECT rand() FROM numbers(100)") assert False, "Node2 became leader?" From b4a3795473b4d5e446e39692de79722ca1a40eba Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 2 Feb 2021 15:38:08 +0300 Subject: [PATCH 49/86] Fix config path --- tests/config/install.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/install.sh b/tests/config/install.sh index 6f620ef6404..9965e1fb1ad 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -29,7 +29,7 @@ ln -sf $SRC_PATH/config.d/graphite.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/database_atomic.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/max_concurrent_queries.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/test_cluster_with_incorrect_pw.xml $DEST_SERVER_PATH/config.d/ -ln -sf $SRC_PATH/config.d/nu_keeper_port.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/test_keeper_port.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/logging_no_rotate.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/tcp_with_proxy.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/top_level_domains_lists.xml $DEST_SERVER_PATH/config.d/ From 94201ebf0411b18fecb0a8d63fbb2ec7b9bfb953 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 2 Feb 2021 15:41:09 +0300 Subject: [PATCH 50/86] More info in test --- tests/integration/test_testkeeper_multinode/test.py | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/tests/integration/test_testkeeper_multinode/test.py b/tests/integration/test_testkeeper_multinode/test.py index 7b9430b2368..16ca00124a5 100644 --- a/tests/integration/test_testkeeper_multinode/test.py +++ b/tests/integration/test_testkeeper_multinode/test.py @@ -237,5 +237,15 @@ def test_blocade_leader_twice(started_cluster): assert False, "Cannot reconnect for node{}".format(n + 1) assert node1.query("SELECT COUNT() FROM t2") == "510\n" + if node2.query("SELECT COUNT() FROM t2") != "510\n": + print(node2.query("SELECT * FROM system.replication_queue FORMAT Vertical")) + print("Replicas") + print(node2.query("SELECT * FROM system.replicas FORMAT Vertical")) + print("Replica 2 info") + print(node2.query("SELECT * FROM system.zookeeper WHERE path = '/clickhouse/t2/replicas/2' FORMAT Vertical")) + print("Queue") + print(node2.query("SELECT * FROM system.zookeeper WHERE path = '/clickhouse/t2/replicas/2/queue' FORMAT Vertical")) + print("Log") + print(node2.query("SELECT * FROM system.zookeeper WHERE path = '/clickhouse/t2/log' FORMAT Vertical")) assert node2.query("SELECT COUNT() FROM t2") == "510\n" assert node3.query("SELECT COUNT() FROM t2") == "510\n" From 1e0a528bac833fd9aef353483af3b4739aa3e3cf Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 2 Feb 2021 22:33:57 +0300 Subject: [PATCH 51/86] Fix possible test flakyness --- tests/integration/test_testkeeper_multinode/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_testkeeper_multinode/test.py b/tests/integration/test_testkeeper_multinode/test.py index 16ca00124a5..0a28b76750b 100644 --- a/tests/integration/test_testkeeper_multinode/test.py +++ b/tests/integration/test_testkeeper_multinode/test.py @@ -228,6 +228,7 @@ def test_blocade_leader_twice(started_cluster): for node in [node1, node2, node3]: for i in range(100): try: + node.query("SYSTEM RESTART REPLICA t2", timeout=10) node.query("SYSTEM SYNC REPLICA t2", timeout=10) break except Exception as ex: From 6743dd46562b43570fe1c57dafb59547c1d5ed89 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 2 Feb 2021 22:56:07 +0300 Subject: [PATCH 52/86] Same for the first test --- tests/integration/test_testkeeper_multinode/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_testkeeper_multinode/test.py b/tests/integration/test_testkeeper_multinode/test.py index 0a28b76750b..cb457e24435 100644 --- a/tests/integration/test_testkeeper_multinode/test.py +++ b/tests/integration/test_testkeeper_multinode/test.py @@ -119,6 +119,7 @@ def test_blocade_leader(started_cluster): for n, node in enumerate([node1, node2, node3]): for i in range(100): try: + node.query("SYSTEM RESTART REPLICA t1", timeout=10) node.query("SYSTEM SYNC REPLICA t1", timeout=10) break except Exception as ex: From c334bdca1f3a865425d4886bf3c543fbeb6f77d1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 3 Feb 2021 10:11:19 +0300 Subject: [PATCH 53/86] Fix NuKeeper server parameters --- src/Coordination/NuKeeperServer.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index bb74ea19aa7..1d99bf54ec8 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -45,9 +45,9 @@ void NuKeeperServer::addServer(int server_id_, const std::string & server_uri_, void NuKeeperServer::startup() { nuraft::raft_params params; - params.heart_beat_interval_ = 1000; - params.election_timeout_lower_bound_ = 500; - params.election_timeout_upper_bound_ = 1000; + params.heart_beat_interval_ = 500; + params.election_timeout_lower_bound_ = 1000; + params.election_timeout_upper_bound_ = 2000; params.reserved_log_items_ = 5000; params.snapshot_distance_ = 5000; params.client_req_timeout_ = 10000; @@ -184,7 +184,7 @@ NuKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const NuKeeper auto response = request->makeResponse(); response->xid = request->xid; response->zxid = 0; /// FIXME what we can do with it? - response->error = Coordination::Error::ZSESSIONEXPIRED; + response->error = Coordination::Error::ZOPERATIONTIMEOUT; responses.push_back(DB::NuKeeperStorage::ResponseForSession{session_id, response}); } return responses; From 0c3ef018bbd62f8c8570bb6649427d716bc8af88 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 3 Feb 2021 10:15:57 +0300 Subject: [PATCH 54/86] Fix ya.make --- src/Coordination/ya.make | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Coordination/ya.make b/src/Coordination/ya.make index 470fe8c75be..f3eae68806c 100644 --- a/src/Coordination/ya.make +++ b/src/Coordination/ya.make @@ -8,8 +8,6 @@ PEERDIR( ) SRCS( - NuKeeperStorageDispatcher.cpp - ) END() From 1ff87ac6f90452d4a71494c2327d4a6781a55b37 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 3 Feb 2021 23:32:15 +0300 Subject: [PATCH 55/86] Add background session lifetime control --- src/Common/ZooKeeper/ZooKeeperCommon.h | 2 +- src/Coordination/NuKeeperServer.cpp | 13 +++- src/Coordination/NuKeeperServer.h | 4 +- src/Coordination/NuKeeperStateMachine.cpp | 23 ++++-- src/Coordination/NuKeeperStateMachine.h | 4 +- src/Coordination/NuKeeperStorage.cpp | 15 +++- src/Coordination/NuKeeperStorage.h | 18 ++++- .../NuKeeperStorageDispatcher.cpp | 42 +++++++++- src/Coordination/NuKeeperStorageDispatcher.h | 9 ++- src/Coordination/SessionExpiryQueue.cpp | 77 +++++++++++++++++++ src/Coordination/SessionExpiryQueue.h | 43 +++++++++++ src/Coordination/ya.make.in | 12 +++ src/Server/NuKeeperTCPHandler.cpp | 36 ++++----- src/Server/NuKeeperTCPHandler.h | 3 +- .../configs/enable_test_keeper1.xml | 4 +- .../configs/enable_test_keeper2.xml | 4 +- .../configs/enable_test_keeper3.xml | 4 +- 17 files changed, 261 insertions(+), 52 deletions(-) create mode 100644 src/Coordination/SessionExpiryQueue.cpp create mode 100644 src/Coordination/SessionExpiryQueue.h create mode 100644 src/Coordination/ya.make.in diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index b2c18c31798..84d7a0823ec 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -72,7 +72,7 @@ struct ZooKeeperHeartbeatRequest final : ZooKeeperRequest void writeImpl(WriteBuffer &) const override {} void readImpl(ReadBuffer &) override {} ZooKeeperResponsePtr makeResponse() const override; - bool isReadRequest() const override { return true; } + bool isReadRequest() const override { return false; } }; struct ZooKeeperHeartbeatResponse final : ZooKeeperResponse diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index 1d99bf54ec8..335f577beeb 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -24,7 +24,7 @@ NuKeeperServer::NuKeeperServer(int server_id_, const std::string & hostname_, in , hostname(hostname_) , port(port_) , endpoint(hostname + ":" + std::to_string(port)) - , state_machine(nuraft::cs_new()) + , state_machine(nuraft::cs_new(500 /* FIXME */)) , state_manager(nuraft::cs_new(server_id, endpoint)) { } @@ -214,12 +214,12 @@ NuKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const NuKeeper } } -int64_t NuKeeperServer::getSessionID() +int64_t NuKeeperServer::getSessionID(long session_timeout_ms) { - auto entry = nuraft::buffer::alloc(sizeof(int64_t)); + auto entry = nuraft::buffer::alloc(sizeof(long)); /// Just special session request nuraft::buffer_serializer bs(entry); - bs.put_i64(0); + bs.put_i64(session_timeout_ms); std::lock_guard lock(append_entries_mutex); @@ -275,4 +275,9 @@ void NuKeeperServer::waitForCatchUp() const } } +std::unordered_set NuKeeperServer::getDeadSessions() +{ + return state_machine->getDeadSessions(); +} + } diff --git a/src/Coordination/NuKeeperServer.h b/src/Coordination/NuKeeperServer.h index 352836dfc27..962863f591e 100644 --- a/src/Coordination/NuKeeperServer.h +++ b/src/Coordination/NuKeeperServer.h @@ -46,7 +46,9 @@ public: NuKeeperStorage::ResponsesForSessions putRequests(const NuKeeperStorage::RequestsForSessions & requests); - int64_t getSessionID(); + int64_t getSessionID(long session_timeout_ms); + + std::unordered_set getDeadSessions(); void addServer(int server_id_, const std::string & server_uri, bool can_become_leader_, int32_t priority); diff --git a/src/Coordination/NuKeeperStateMachine.cpp b/src/Coordination/NuKeeperStateMachine.cpp index b6521e1d648..8e22da81081 100644 --- a/src/Coordination/NuKeeperStateMachine.cpp +++ b/src/Coordination/NuKeeperStateMachine.cpp @@ -43,8 +43,9 @@ nuraft::ptr writeResponses(NuKeeperStorage::ResponsesForSessions } -NuKeeperStateMachine::NuKeeperStateMachine() - : last_committed_idx(0) +NuKeeperStateMachine::NuKeeperStateMachine(long tick_time) + : storage(tick_time) + , last_committed_idx(0) , log(&Poco::Logger::get("NuRaftStateMachine")) { LOG_DEBUG(log, "Created nukeeper state machine"); @@ -52,15 +53,19 @@ NuKeeperStateMachine::NuKeeperStateMachine() nuraft::ptr NuKeeperStateMachine::commit(const size_t log_idx, nuraft::buffer & data) { - if (data.size() == sizeof(size_t)) + if (data.size() == sizeof(long)) { - LOG_DEBUG(log, "Session ID response {}", log_idx); + nuraft::buffer_serializer timeout_data(data); + long session_timeout_ms = timeout_data.get_i64(); auto response = nuraft::buffer::alloc(sizeof(size_t)); + int64_t session_id; nuraft::buffer_serializer bs(response); { std::lock_guard lock(storage_lock); - bs.put_i64(storage.getSessionID()); + session_id = storage.getSessionID(session_timeout_ms); + bs.put_i64(session_id); } + LOG_DEBUG(log, "Session ID response {} with timeout {}", session_id, session_timeout_ms); last_committed_idx = log_idx; return response; } @@ -121,7 +126,7 @@ NuKeeperStateMachine::StorageSnapshotPtr NuKeeperStateMachine::readSnapshot(nura NuKeeperStorageSerializer serializer; ReadBufferFromNuraftBuffer reader(in); - NuKeeperStorage new_storage; + NuKeeperStorage new_storage(500 /*FIXME*/); serializer.deserialize(new_storage, reader); return std::make_shared(ss, new_storage); } @@ -229,4 +234,10 @@ NuKeeperStorage::ResponsesForSessions NuKeeperStateMachine::processReadRequest(c return storage.processRequest(request_for_session.request, request_for_session.session_id); } +std::unordered_set NuKeeperStateMachine::getDeadSessions() +{ + std::lock_guard lock(storage_lock); + return storage.getDeadSessions(); +} + } diff --git a/src/Coordination/NuKeeperStateMachine.h b/src/Coordination/NuKeeperStateMachine.h index 41c28caa76c..380588a39f0 100644 --- a/src/Coordination/NuKeeperStateMachine.h +++ b/src/Coordination/NuKeeperStateMachine.h @@ -10,7 +10,7 @@ namespace DB class NuKeeperStateMachine : public nuraft::state_machine { public: - NuKeeperStateMachine(); + NuKeeperStateMachine(long tick_time); nuraft::ptr pre_commit(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override { return nullptr; } @@ -49,6 +49,8 @@ public: NuKeeperStorage::ResponsesForSessions processReadRequest(const NuKeeperStorage::RequestForSession & request_for_session); + std::unordered_set getDeadSessions(); + private: struct StorageSnapshot { diff --git a/src/Coordination/NuKeeperStorage.cpp b/src/Coordination/NuKeeperStorage.cpp index 9a8b96d63a3..3b52b47c4bf 100644 --- a/src/Coordination/NuKeeperStorage.cpp +++ b/src/Coordination/NuKeeperStorage.cpp @@ -67,7 +67,8 @@ static NuKeeperStorage::ResponsesForSessions processWatchesImpl(const String & p return result; } -NuKeeperStorage::NuKeeperStorage() +NuKeeperStorage::NuKeeperStorage(long tick_time_ms) + : session_expiry_queue(tick_time_ms) { container.emplace("/", Node()); } @@ -638,6 +639,18 @@ NuKeeperStorage::ResponsesForSessions NuKeeperStorage::processRequest(const Coor auto response = std::make_shared(); response->xid = zk_request->xid; response->zxid = getZXID(); + session_expiry_queue.remove(session_id); + session_and_timeout.erase(session_id); + results.push_back(ResponseForSession{session_id, response}); + } + else if (zk_request->getOpNum() == Coordination::OpNum::Heartbeat) + { + session_expiry_queue.update(session_id, session_and_timeout[session_id]); + NuKeeperStorageRequestPtr storage_request = NuKeeperWrapperFactory::instance().get(zk_request); + auto [response, _] = storage_request->process(container, ephemerals, zxid, session_id); + response->xid = zk_request->xid; + response->zxid = getZXID(); + results.push_back(ResponseForSession{session_id, response}); } else diff --git a/src/Coordination/NuKeeperStorage.h b/src/Coordination/NuKeeperStorage.h index dce00391bce..cf881687dcb 100644 --- a/src/Coordination/NuKeeperStorage.h +++ b/src/Coordination/NuKeeperStorage.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -50,6 +51,7 @@ public: using Container = std::map; using Ephemerals = std::unordered_map>; using SessionAndWatcher = std::unordered_map>; + using SessionAndTimeout = std::unordered_map; using SessionIDs = std::vector; using Watches = std::map; @@ -57,6 +59,8 @@ public: Container container; Ephemerals ephemerals; SessionAndWatcher sessions_and_watchers; + SessionExpiryQueue session_expiry_queue; + SessionAndTimeout session_and_timeout; int64_t zxid{0}; bool finalized{false}; @@ -72,15 +76,23 @@ public: } public: - NuKeeperStorage(); + NuKeeperStorage(long tick_time_ms); - int64_t getSessionID() + int64_t getSessionID(long session_timeout_ms) { - return session_id_counter++; + auto result = session_id_counter++; + session_and_timeout.emplace(result, session_timeout_ms); + session_expiry_queue.update(result, session_timeout_ms); + return result; } ResponsesForSessions processRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id); ResponsesForSessions finalize(const RequestsForSessions & expired_requests); + + std::unordered_set getDeadSessions() + { + return session_expiry_queue.getExpiredSessions(); + } }; } diff --git a/src/Coordination/NuKeeperStorageDispatcher.cpp b/src/Coordination/NuKeeperStorageDispatcher.cpp index 9988e0ac476..cf36fd40bc3 100644 --- a/src/Coordination/NuKeeperStorageDispatcher.cpp +++ b/src/Coordination/NuKeeperStorageDispatcher.cpp @@ -59,7 +59,6 @@ void NuKeeperStorageDispatcher::setResponse(int64_t session_id, const Coordinati bool NuKeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id) { - { std::lock_guard lock(session_to_response_callback_mutex); if (session_to_response_callback.count(session_id) == 0) @@ -171,6 +170,7 @@ void NuKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigurati } processing_thread = ThreadFromGlobalPool([this] { processingThread(); }); + session_cleaner_thread = ThreadFromGlobalPool([this] { sessionCleanerTask(); }); LOG_DEBUG(log, "Dispatcher initialized"); } @@ -188,6 +188,9 @@ void NuKeeperStorageDispatcher::shutdown() LOG_DEBUG(log, "Shutting down storage dispatcher"); shutdown_called = true; + if (session_cleaner_thread.joinable()) + session_cleaner_thread.join(); + if (processing_thread.joinable()) processing_thread.join(); } @@ -225,6 +228,43 @@ void NuKeeperStorageDispatcher::registerSession(int64_t session_id, ZooKeeperRes throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Session with id {} already registered in dispatcher", session_id); } +void NuKeeperStorageDispatcher::sessionCleanerTask() +{ + while (true) + { + if (shutdown_called) + return; + + try + { + if (isLeader()) + { + auto dead_sessions = server->getDeadSessions(); + for (int64_t dead_session : dead_sessions) + { + LOG_INFO(log, "Found dead session {}, will try to close it", dead_session); + Coordination::ZooKeeperRequestPtr request = Coordination::ZooKeeperRequestFactory::instance().get(Coordination::OpNum::Close); + request->xid = Coordination::CLOSE_XID; + putRequest(request, dead_session); + { + std::lock_guard lock(session_to_response_callback_mutex); + auto session_it = session_to_response_callback.find(dead_session); + if (session_it != session_to_response_callback.end()) + session_to_response_callback.erase(session_it); + } + } + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + /*FIXME*/ + std::this_thread::sleep_for(std::chrono::milliseconds(500)); + } +} + void NuKeeperStorageDispatcher::finishSession(int64_t session_id) { std::lock_guard lock(session_to_response_callback_mutex); diff --git a/src/Coordination/NuKeeperStorageDispatcher.h b/src/Coordination/NuKeeperStorageDispatcher.h index c292cd99c4f..dfd36b39537 100644 --- a/src/Coordination/NuKeeperStorageDispatcher.h +++ b/src/Coordination/NuKeeperStorageDispatcher.h @@ -27,7 +27,6 @@ class NuKeeperStorageDispatcher private: Poco::Timespan operation_timeout{0, Coordination::DEFAULT_OPERATION_TIMEOUT_MS * 1000}; - std::mutex push_request_mutex; using RequestsQueue = ConcurrentBoundedQueue; @@ -40,12 +39,15 @@ private: ThreadFromGlobalPool processing_thread; + ThreadFromGlobalPool session_cleaner_thread; + std::unique_ptr server; Poco::Logger * log; private: void processingThread(); + void sessionCleanerTask(); void setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response); public: @@ -69,15 +71,14 @@ public: return server->isLeaderAlive(); } - int64_t getSessionID() + int64_t getSessionID(long session_timeout_ms) { - return server->getSessionID(); + return server->getSessionID(session_timeout_ms); } void registerSession(int64_t session_id, ZooKeeperResponseCallback callback); /// Call if we don't need any responses for this session no more (session was expired) void finishSession(int64_t session_id); - }; } diff --git a/src/Coordination/SessionExpiryQueue.cpp b/src/Coordination/SessionExpiryQueue.cpp new file mode 100644 index 00000000000..45ceaee52fe --- /dev/null +++ b/src/Coordination/SessionExpiryQueue.cpp @@ -0,0 +1,77 @@ +#include +#include +namespace DB +{ + +bool SessionExpiryQueue::remove(int64_t session_id) +{ + auto session_it = session_to_timeout.find(session_id); + if (session_it != session_to_timeout.end()) + { + auto set_it = expiry_to_sessions.find(session_it->second); + if (set_it != expiry_to_sessions.end()) + set_it->second.erase(session_id); + + return true; + } + + return false; +} + +bool SessionExpiryQueue::update(int64_t session_id, long timeout_ms) +{ + auto session_it = session_to_timeout.find(session_id); + long now = getNowMilliseconds(); + long new_expiry_time = roundToNextInterval(now + timeout_ms); + + if (session_it != session_to_timeout.end()) + { + if (new_expiry_time == session_it->second) + return false; + + auto set_it = expiry_to_sessions.find(new_expiry_time); + if (set_it == expiry_to_sessions.end()) + std::tie(set_it, std::ignore) = expiry_to_sessions.emplace(new_expiry_time, std::unordered_set()); + + set_it->second.insert(session_id); + long prev_expiry_time = session_it->second; + + if (prev_expiry_time != new_expiry_time) + { + auto prev_set_it = expiry_to_sessions.find(prev_expiry_time); + if (prev_set_it != expiry_to_sessions.end()) + prev_set_it->second.erase(session_id); + } + session_it->second = new_expiry_time; + return true; + } + else + { + session_to_timeout[session_id] = new_expiry_time; + auto set_it = expiry_to_sessions.find(new_expiry_time); + if (set_it == expiry_to_sessions.end()) + std::tie(set_it, std::ignore) = expiry_to_sessions.emplace(new_expiry_time, std::unordered_set()); + set_it->second.insert(session_id); + return false; + } +} + +std::unordered_set SessionExpiryQueue::getExpiredSessions() +{ + long now = getNowMilliseconds(); + if (now < next_expiration_time) + return {}; + + auto set_it = expiry_to_sessions.find(next_expiration_time); + long new_expiration_time = next_expiration_time + expiration_interval; + next_expiration_time = new_expiration_time; + if (set_it != expiry_to_sessions.end()) + { + auto result = set_it->second; + expiry_to_sessions.erase(set_it); + return result; + } + return {}; +} + +} diff --git a/src/Coordination/SessionExpiryQueue.h b/src/Coordination/SessionExpiryQueue.h new file mode 100644 index 00000000000..4fb254526e7 --- /dev/null +++ b/src/Coordination/SessionExpiryQueue.h @@ -0,0 +1,43 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +class SessionExpiryQueue +{ +private: + std::unordered_map session_to_timeout; + std::unordered_map> expiry_to_sessions; + + long expiration_interval; + long next_expiration_time; + + static long getNowMilliseconds() + { + using namespace std::chrono; + return duration_cast(system_clock::now().time_since_epoch()).count(); + } + + long roundToNextInterval(long time) const + { + return (time / expiration_interval + 1) * expiration_interval; + } + +public: + explicit SessionExpiryQueue(long expiration_interval_) + : expiration_interval(expiration_interval_) + , next_expiration_time(roundToNextInterval(getNowMilliseconds())) + { + } + + bool remove(int64_t session_id); + + bool update(int64_t session_id, long timeout_ms); + + std::unordered_set getExpiredSessions(); +}; + +} diff --git a/src/Coordination/ya.make.in b/src/Coordination/ya.make.in new file mode 100644 index 00000000000..ba5f8bcbea4 --- /dev/null +++ b/src/Coordination/ya.make.in @@ -0,0 +1,12 @@ +OWNER(g:clickhouse) + +LIBRARY() + +PEERDIR( + clickhouse/src/Common +) + +SRCS( +) + +END() diff --git a/src/Server/NuKeeperTCPHandler.cpp b/src/Server/NuKeeperTCPHandler.cpp index 6deee5094ca..9d39c317356 100644 --- a/src/Server/NuKeeperTCPHandler.cpp +++ b/src/Server/NuKeeperTCPHandler.cpp @@ -230,8 +230,8 @@ NuKeeperTCPHandler::NuKeeperTCPHandler(IServer & server_, const Poco::Net::Strea , log(&Poco::Logger::get("NuKeeperTCPHandler")) , global_context(server.context()) , nu_keeper_storage_dispatcher(global_context.getNuKeeperStorageDispatcher()) - , operation_timeout(0, global_context.getConfigRef().getUInt("nu_keeper_server.operation_timeout_ms", Coordination::DEFAULT_OPERATION_TIMEOUT_MS) * 1000) - , session_timeout(0, global_context.getConfigRef().getUInt("nu_keeper_server.session_timeout_ms", Coordination::DEFAULT_SESSION_TIMEOUT_MS) * 1000) + , operation_timeout(0, global_context.getConfigRef().getUInt("test_keeper_server.operation_timeout_ms", Coordination::DEFAULT_OPERATION_TIMEOUT_MS) * 1000) + , session_timeout(0, global_context.getConfigRef().getUInt("test_keeper_server.session_timeout_ms", Coordination::DEFAULT_SESSION_TIMEOUT_MS) * 1000) , poll_wrapper(std::make_unique(socket_)) , responses(std::make_unique()) { @@ -245,7 +245,7 @@ void NuKeeperTCPHandler::sendHandshake(bool has_leader) else /// Specially ignore connections if we are not leader, client will throw exception Coordination::write(42, *out); - Coordination::write(Coordination::DEFAULT_SESSION_TIMEOUT_MS, *out); + Coordination::write(static_cast(session_timeout.totalMilliseconds()), *out); Coordination::write(session_id, *out); std::array passwd{}; Coordination::write(passwd, *out); @@ -257,15 +257,14 @@ void NuKeeperTCPHandler::run() runImpl(); } -void NuKeeperTCPHandler::receiveHandshake() +Poco::Timespan NuKeeperTCPHandler::receiveHandshake() { int32_t handshake_length; int32_t protocol_version; int64_t last_zxid_seen; - int32_t timeout; + int32_t timeout_ms; int64_t previous_session_id = 0; /// We don't support session restore. So previous session_id is always zero. std::array passwd {}; - Coordination::read(handshake_length, *in); if (handshake_length != Coordination::CLIENT_HANDSHAKE_LENGTH && handshake_length != Coordination::CLIENT_HANDSHAKE_LENGTH_WITH_READONLY) throw Exception("Unexpected handshake length received: " + toString(handshake_length), ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT); @@ -280,7 +279,7 @@ void NuKeeperTCPHandler::receiveHandshake() if (last_zxid_seen != 0) throw Exception("Non zero last_zxid_seen is not supported", ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT); - Coordination::read(timeout, *in); + Coordination::read(timeout_ms, *in); Coordination::read(previous_session_id, *in); if (previous_session_id != 0) @@ -291,6 +290,8 @@ void NuKeeperTCPHandler::receiveHandshake() int8_t readonly; if (handshake_length == Coordination::CLIENT_HANDSHAKE_LENGTH_WITH_READONLY) Coordination::read(readonly, *in); + + return Poco::Timespan(0, timeout_ms * 1000); } @@ -316,7 +317,9 @@ void NuKeeperTCPHandler::runImpl() try { - receiveHandshake(); + auto client_timeout = receiveHandshake(); + if (client_timeout != 0) + session_timeout = std::min(client_timeout, session_timeout); } catch (const Exception & e) /// Typical for an incorrect username, password, or address. { @@ -328,7 +331,7 @@ void NuKeeperTCPHandler::runImpl() { try { - session_id = nu_keeper_storage_dispatcher->getSessionID(); + session_id = nu_keeper_storage_dispatcher->getSessionID(session_timeout.totalMilliseconds()); } catch (const Exception & e) { @@ -416,7 +419,7 @@ void NuKeeperTCPHandler::runImpl() if (session_stopwatch.elapsedMicroseconds() > static_cast(session_timeout.totalMicroseconds())) { LOG_DEBUG(log, "Session #{} expired", session_id); - finish(); + nu_keeper_storage_dispatcher->finishSession(session_id); break; } } @@ -424,21 +427,10 @@ void NuKeeperTCPHandler::runImpl() catch (const Exception & ex) { LOG_INFO(log, "Got exception processing session #{}: {}", session_id, getExceptionMessage(ex, true)); - finish(); + nu_keeper_storage_dispatcher->finishSession(session_id); } } -void NuKeeperTCPHandler::finish() -{ - Coordination::ZooKeeperRequestPtr request = Coordination::ZooKeeperRequestFactory::instance().get(Coordination::OpNum::Close); - request->xid = close_xid; - /// Put close request (so storage will remove all info about session) - nu_keeper_storage_dispatcher->putRequest(request, session_id); - /// We don't need any callbacks because session can be already dead and - /// nobody wait for response - nu_keeper_storage_dispatcher->finishSession(session_id); -} - std::pair NuKeeperTCPHandler::receiveRequest() { int32_t length; diff --git a/src/Server/NuKeeperTCPHandler.h b/src/Server/NuKeeperTCPHandler.h index 1874b8cd309..641d2f78e1f 100644 --- a/src/Server/NuKeeperTCPHandler.h +++ b/src/Server/NuKeeperTCPHandler.h @@ -53,10 +53,9 @@ private: void runImpl(); void sendHandshake(bool has_leader); - void receiveHandshake(); + Poco::Timespan receiveHandshake(); std::pair receiveRequest(); - void finish(); }; } diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml index 81f68f50c7c..e1b6da40338 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml @@ -1,8 +1,8 @@ 9181 - 10000 - 30000 + 5000 + 10000 1 diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml index 73340973367..7622aa164da 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml @@ -1,8 +1,8 @@ 9181 - 10000 - 30000 + 5000 + 10000 2 diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml index fbc51489d11..1edbfa7271e 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml @@ -1,8 +1,8 @@ 9181 - 10000 - 30000 + 5000 + 10000 3 From e7a83868dd16b279f6736a827eb4519fce7b0fb1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 4 Feb 2021 11:28:11 +0300 Subject: [PATCH 56/86] Fix build --- src/Coordination/NuKeeperServer.cpp | 4 ++-- src/Coordination/NuKeeperServer.h | 2 +- src/Coordination/NuKeeperStateMachine.cpp | 6 +++--- src/Coordination/NuKeeperStorage.cpp | 2 +- src/Coordination/NuKeeperStorage.h | 4 ++-- src/Coordination/SessionExpiryQueue.cpp | 12 ++++++------ src/Coordination/SessionExpiryQueue.h | 16 ++++++++-------- src/Coordination/tests/gtest_for_build.cpp | 4 ++-- .../test_testkeeper_multinode/test.py | 5 ++++- 9 files changed, 29 insertions(+), 26 deletions(-) diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index 335f577beeb..d700956c522 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -214,9 +214,9 @@ NuKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const NuKeeper } } -int64_t NuKeeperServer::getSessionID(long session_timeout_ms) +int64_t NuKeeperServer::getSessionID(int64_t session_timeout_ms) { - auto entry = nuraft::buffer::alloc(sizeof(long)); + auto entry = nuraft::buffer::alloc(sizeof(int64_t)); /// Just special session request nuraft::buffer_serializer bs(entry); bs.put_i64(session_timeout_ms); diff --git a/src/Coordination/NuKeeperServer.h b/src/Coordination/NuKeeperServer.h index 962863f591e..32ca61e924f 100644 --- a/src/Coordination/NuKeeperServer.h +++ b/src/Coordination/NuKeeperServer.h @@ -46,7 +46,7 @@ public: NuKeeperStorage::ResponsesForSessions putRequests(const NuKeeperStorage::RequestsForSessions & requests); - int64_t getSessionID(long session_timeout_ms); + int64_t getSessionID(int64_t session_timeout_ms); std::unordered_set getDeadSessions(); diff --git a/src/Coordination/NuKeeperStateMachine.cpp b/src/Coordination/NuKeeperStateMachine.cpp index 8e22da81081..f7b7ba3c567 100644 --- a/src/Coordination/NuKeeperStateMachine.cpp +++ b/src/Coordination/NuKeeperStateMachine.cpp @@ -53,11 +53,11 @@ NuKeeperStateMachine::NuKeeperStateMachine(long tick_time) nuraft::ptr NuKeeperStateMachine::commit(const size_t log_idx, nuraft::buffer & data) { - if (data.size() == sizeof(long)) + if (data.size() == sizeof(int64_t)) { nuraft::buffer_serializer timeout_data(data); - long session_timeout_ms = timeout_data.get_i64(); - auto response = nuraft::buffer::alloc(sizeof(size_t)); + int64_t session_timeout_ms = timeout_data.get_i64(); + auto response = nuraft::buffer::alloc(sizeof(int64_t)); int64_t session_id; nuraft::buffer_serializer bs(response); { diff --git a/src/Coordination/NuKeeperStorage.cpp b/src/Coordination/NuKeeperStorage.cpp index 3b52b47c4bf..45701b63b8b 100644 --- a/src/Coordination/NuKeeperStorage.cpp +++ b/src/Coordination/NuKeeperStorage.cpp @@ -67,7 +67,7 @@ static NuKeeperStorage::ResponsesForSessions processWatchesImpl(const String & p return result; } -NuKeeperStorage::NuKeeperStorage(long tick_time_ms) +NuKeeperStorage::NuKeeperStorage(int64_t tick_time_ms) : session_expiry_queue(tick_time_ms) { container.emplace("/", Node()); diff --git a/src/Coordination/NuKeeperStorage.h b/src/Coordination/NuKeeperStorage.h index cf881687dcb..6f709a6f480 100644 --- a/src/Coordination/NuKeeperStorage.h +++ b/src/Coordination/NuKeeperStorage.h @@ -76,9 +76,9 @@ public: } public: - NuKeeperStorage(long tick_time_ms); + NuKeeperStorage(int64_t tick_time_ms); - int64_t getSessionID(long session_timeout_ms) + int64_t getSessionID(int64_t session_timeout_ms) { auto result = session_id_counter++; session_and_timeout.emplace(result, session_timeout_ms); diff --git a/src/Coordination/SessionExpiryQueue.cpp b/src/Coordination/SessionExpiryQueue.cpp index 45ceaee52fe..f90cd089be8 100644 --- a/src/Coordination/SessionExpiryQueue.cpp +++ b/src/Coordination/SessionExpiryQueue.cpp @@ -18,11 +18,11 @@ bool SessionExpiryQueue::remove(int64_t session_id) return false; } -bool SessionExpiryQueue::update(int64_t session_id, long timeout_ms) +bool SessionExpiryQueue::update(int64_t session_id, int64_t timeout_ms) { auto session_it = session_to_timeout.find(session_id); - long now = getNowMilliseconds(); - long new_expiry_time = roundToNextInterval(now + timeout_ms); + int64_t now = getNowMilliseconds(); + int64_t new_expiry_time = roundToNextInterval(now + timeout_ms); if (session_it != session_to_timeout.end()) { @@ -34,7 +34,7 @@ bool SessionExpiryQueue::update(int64_t session_id, long timeout_ms) std::tie(set_it, std::ignore) = expiry_to_sessions.emplace(new_expiry_time, std::unordered_set()); set_it->second.insert(session_id); - long prev_expiry_time = session_it->second; + int64_t prev_expiry_time = session_it->second; if (prev_expiry_time != new_expiry_time) { @@ -58,12 +58,12 @@ bool SessionExpiryQueue::update(int64_t session_id, long timeout_ms) std::unordered_set SessionExpiryQueue::getExpiredSessions() { - long now = getNowMilliseconds(); + int64_t now = getNowMilliseconds(); if (now < next_expiration_time) return {}; auto set_it = expiry_to_sessions.find(next_expiration_time); - long new_expiration_time = next_expiration_time + expiration_interval; + int64_t new_expiration_time = next_expiration_time + expiration_interval; next_expiration_time = new_expiration_time; if (set_it != expiry_to_sessions.end()) { diff --git a/src/Coordination/SessionExpiryQueue.h b/src/Coordination/SessionExpiryQueue.h index 4fb254526e7..3b4ad6dde88 100644 --- a/src/Coordination/SessionExpiryQueue.h +++ b/src/Coordination/SessionExpiryQueue.h @@ -9,25 +9,25 @@ namespace DB class SessionExpiryQueue { private: - std::unordered_map session_to_timeout; - std::unordered_map> expiry_to_sessions; + std::unordered_map session_to_timeout; + std::unordered_map> expiry_to_sessions; - long expiration_interval; - long next_expiration_time; + int64_t expiration_interval; + int64_t next_expiration_time; - static long getNowMilliseconds() + static int64_t getNowMilliseconds() { using namespace std::chrono; return duration_cast(system_clock::now().time_since_epoch()).count(); } - long roundToNextInterval(long time) const + int64_t roundToNextInterval(int64_t time) const { return (time / expiration_interval + 1) * expiration_interval; } public: - explicit SessionExpiryQueue(long expiration_interval_) + explicit SessionExpiryQueue(int64_t expiration_interval_) : expiration_interval(expiration_interval_) , next_expiration_time(roundToNextInterval(getNowMilliseconds())) { @@ -35,7 +35,7 @@ public: bool remove(int64_t session_id); - bool update(int64_t session_id, long timeout_ms); + bool update(int64_t session_id, int64_t timeout_ms); std::unordered_set getExpiredSessions(); }; diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index d2f4938dfd3..956b12d6e08 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -310,7 +310,7 @@ DB::NuKeeperStorage::ResponsesForSessions getZooKeeperResponses(nuraft::ptr Date: Thu, 4 Feb 2021 12:39:07 +0300 Subject: [PATCH 57/86] Fix build one more time --- src/Coordination/NuKeeperStateMachine.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/NuKeeperStateMachine.h b/src/Coordination/NuKeeperStateMachine.h index 380588a39f0..bfb67f10a67 100644 --- a/src/Coordination/NuKeeperStateMachine.h +++ b/src/Coordination/NuKeeperStateMachine.h @@ -10,7 +10,7 @@ namespace DB class NuKeeperStateMachine : public nuraft::state_machine { public: - NuKeeperStateMachine(long tick_time); + NuKeeperStateMachine(long tick_time = 500); nuraft::ptr pre_commit(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override { return nullptr; } From da51ea179464ea96156f8205312a202f9956db9e Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 4 Feb 2021 15:07:41 +0300 Subject: [PATCH 58/86] Simplify shutdown and requests processing --- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 7 ++ src/Common/ZooKeeper/ZooKeeperCommon.h | 4 +- src/Coordination/NuKeeperCommon.h | 24 ++++++ src/Coordination/NuKeeperServer.cpp | 83 +++++++------------ src/Coordination/NuKeeperServer.h | 12 +-- src/Coordination/NuKeeperStateMachine.cpp | 8 +- src/Coordination/NuKeeperStateMachine.h | 2 + src/Coordination/NuKeeperStorage.cpp | 48 ++--------- src/Coordination/NuKeeperStorage.h | 3 +- .../NuKeeperStorageDispatcher.cpp | 19 ++--- src/Coordination/SessionExpiryQueue.cpp | 6 ++ src/Coordination/SessionExpiryQueue.h | 2 + src/Server/NuKeeperTCPHandler.cpp | 13 +-- 13 files changed, 105 insertions(+), 126 deletions(-) create mode 100644 src/Coordination/NuKeeperCommon.h diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index 278d36f9245..2d32cd75624 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -51,6 +51,13 @@ void ZooKeeperWatchResponse::writeImpl(WriteBuffer & out) const Coordination::write(path, out); } +void ZooKeeperWatchResponse::write(WriteBuffer & out) const +{ + if (error == Error::ZOK) + ZooKeeperResponse::write(out); + /// skip bad responses for watches +} + void ZooKeeperAuthRequest::writeImpl(WriteBuffer & out) const { Coordination::write(type, out); diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index 84d7a0823ec..8bc1cde8cd7 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -30,7 +30,7 @@ struct ZooKeeperResponse : virtual Response virtual ~ZooKeeperResponse() override = default; virtual void readImpl(ReadBuffer &) = 0; virtual void writeImpl(WriteBuffer &) const = 0; - void write(WriteBuffer & out) const; + virtual void write(WriteBuffer & out) const; virtual OpNum getOpNum() const = 0; }; @@ -88,6 +88,8 @@ struct ZooKeeperWatchResponse final : WatchResponse, ZooKeeperResponse void writeImpl(WriteBuffer & out) const override; + void write(WriteBuffer & out) const override; + OpNum getOpNum() const override { throw Exception("OpNum for watch response doesn't exist", Error::ZRUNTIMEINCONSISTENCY); diff --git a/src/Coordination/NuKeeperCommon.h b/src/Coordination/NuKeeperCommon.h new file mode 100644 index 00000000000..14fc612093c --- /dev/null +++ b/src/Coordination/NuKeeperCommon.h @@ -0,0 +1,24 @@ +#pragma once + +#include + +namespace DB +{ + +struct NuKeeperRequest +{ + int64_t session_id; + Coordination::ZooKeeperRequestPtr request; +}; + +using NuKeeperRequests = std::vector; + +struct NuKeeperResponse +{ + int64_t session_id; + Coordination::ZooKeeperRequestPtr response; +}; + +using NuKeeperResponses = std::vector; + +} diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index d700956c522..3910376ebda 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -17,6 +17,7 @@ namespace ErrorCodes { extern const int TIMEOUT_EXCEEDED; extern const int RAFT_ERROR; + extern const int LOGICAL_ERROR; } NuKeeperServer::NuKeeperServer(int server_id_, const std::string & hostname_, int port_) @@ -75,24 +76,11 @@ void NuKeeperServer::startup() throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Cannot start RAFT server within startup timeout"); } -NuKeeperStorage::ResponsesForSessions NuKeeperServer::shutdown(const NuKeeperStorage::RequestsForSessions & expired_requests) +void NuKeeperServer::shutdown() { - NuKeeperStorage::ResponsesForSessions responses; - if (isLeader()) - { - try - { - responses = putRequests(expired_requests); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - + state_machine->shutdownStorage(); if (!launcher.shutdown(5)) LOG_WARNING(&Poco::Logger::get("NuKeeperServer"), "Failed to shutdown RAFT server in {} seconds", 5); - return responses; } namespace @@ -106,12 +94,11 @@ nuraft::ptr getZooKeeperLogEntry(int64_t session_id, const Coord return buf.getBuffer(); } -} - -NuKeeperStorage::ResponsesForSessions NuKeeperServer::readZooKeeperResponses(nuraft::ptr & buffer) +NuKeeperStorage::ResponsesForSessions readZooKeeperResponses(nuraft::ptr & buffer, const Coordination::ZooKeeperRequestPtr & request) { DB::NuKeeperStorage::ResponsesForSessions results; DB::ReadBufferFromNuraftBuffer buf(buffer); + bool response_found = false; while (!buf.eof()) { @@ -122,7 +109,6 @@ NuKeeperStorage::ResponsesForSessions NuKeeperServer::readZooKeeperResponses(nur int64_t zxid; Coordination::Error err; - /// FIXME (alesap) We don't need to parse responses here Coordination::read(length, buf); Coordination::read(xid, buf); Coordination::read(zxid, buf); @@ -133,17 +119,11 @@ NuKeeperStorage::ResponsesForSessions NuKeeperServer::readZooKeeperResponses(nur response = std::make_shared(); else { - auto session_xids = ops_mapping.find(session_id); - if (session_xids == ops_mapping.end()) - throw Exception(ErrorCodes::RAFT_ERROR, "Unknown session id {}", session_id); - auto response_it = session_xids->second.find(xid); - if (response_it == session_xids->second.end()) - throw Exception(ErrorCodes::RAFT_ERROR, "Unknown xid {} for session id {}", xid, session_id); + if (response_found) + throw Exception(ErrorCodes::LOGICAL_ERROR, "More than one non-watch response for single request with xid {}, response xid {}", request->xid, xid); - response = response_it->second; - ops_mapping[session_id].erase(response_it); - if (ops_mapping[session_id].empty()) - ops_mapping.erase(session_xids); + response_found = true; + response = request->makeResponse(); } if (err == Coordination::Error::ZOK && (xid == Coordination::WATCH_XID || response->getOpNum() != Coordination::OpNum::Close)) @@ -158,20 +138,19 @@ NuKeeperStorage::ResponsesForSessions NuKeeperServer::readZooKeeperResponses(nur return results; } -NuKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const NuKeeperStorage::RequestsForSessions & requests) +} + +NuKeeperStorage::ResponsesForSessions NuKeeperServer::putRequest(const NuKeeperStorage::RequestForSession & request_for_session) { - if (isLeaderAlive() && requests.size() == 1 && requests[0].request->isReadRequest()) + auto [session_id, request] = request_for_session; + if (isLeaderAlive() && request_for_session.request->isReadRequest()) { - return state_machine->processReadRequest(requests[0]); + return state_machine->processReadRequest(request_for_session); } else { std::vector> entries; - for (const auto & [session_id, request] : requests) - { - ops_mapping[session_id][request->xid] = request->makeResponse(); - entries.push_back(getZooKeeperLogEntry(session_id, request)); - } + entries.push_back(getZooKeeperLogEntry(session_id, request)); std::lock_guard lock(append_entries_mutex); @@ -179,28 +158,22 @@ NuKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const NuKeeper if (!result->get_accepted()) { NuKeeperStorage::ResponsesForSessions responses; - for (const auto & [session_id, request] : requests) - { - auto response = request->makeResponse(); - response->xid = request->xid; - response->zxid = 0; /// FIXME what we can do with it? - response->error = Coordination::Error::ZOPERATIONTIMEOUT; - responses.push_back(DB::NuKeeperStorage::ResponseForSession{session_id, response}); - } + auto response = request->makeResponse(); + response->xid = request->xid; + response->zxid = 0; + response->error = Coordination::Error::ZOPERATIONTIMEOUT; + responses.push_back(DB::NuKeeperStorage::ResponseForSession{session_id, response}); return responses; } if (result->get_result_code() == nuraft::cmd_result_code::TIMEOUT) { NuKeeperStorage::ResponsesForSessions responses; - for (const auto & [session_id, request] : requests) - { - auto response = request->makeResponse(); - response->xid = request->xid; - response->zxid = 0; /// FIXME what we can do with it? - response->error = Coordination::Error::ZOPERATIONTIMEOUT; - responses.push_back(DB::NuKeeperStorage::ResponseForSession{session_id, response}); - } + auto response = request->makeResponse(); + response->xid = request->xid; + response->zxid = 0; + response->error = Coordination::Error::ZOPERATIONTIMEOUT; + responses.push_back(DB::NuKeeperStorage::ResponseForSession{session_id, response}); return responses; } else if (result->get_result_code() != nuraft::cmd_result_code::OK) @@ -210,7 +183,7 @@ NuKeeperStorage::ResponsesForSessions NuKeeperServer::putRequests(const NuKeeper if (result_buf == nullptr) throw Exception(ErrorCodes::RAFT_ERROR, "Received nullptr from RAFT leader"); - return readZooKeeperResponses(result_buf); + return readZooKeeperResponses(result_buf, request); } } @@ -250,7 +223,7 @@ bool NuKeeperServer::isLeaderAlive() const bool NuKeeperServer::waitForServer(int32_t id) const { - for (size_t i = 0; i < 10; ++i) + for (size_t i = 0; i < 50; ++i) { if (raft_instance->get_srv_config(id) != nullptr) return true; diff --git a/src/Coordination/NuKeeperServer.h b/src/Coordination/NuKeeperServer.h index 32ca61e924f..358a4212967 100644 --- a/src/Coordination/NuKeeperServer.h +++ b/src/Coordination/NuKeeperServer.h @@ -29,14 +29,6 @@ private: nuraft::ptr raft_instance; - using XIDToOp = std::unordered_map; - - using SessionIDOps = std::unordered_map; - - SessionIDOps ops_mapping; - - NuKeeperStorage::ResponsesForSessions readZooKeeperResponses(nuraft::ptr & buffer); - std::mutex append_entries_mutex; public: @@ -44,7 +36,7 @@ public: void startup(); - NuKeeperStorage::ResponsesForSessions putRequests(const NuKeeperStorage::RequestsForSessions & requests); + NuKeeperStorage::ResponsesForSessions putRequest(const NuKeeperStorage::RequestForSession & request); int64_t getSessionID(int64_t session_timeout_ms); @@ -60,7 +52,7 @@ public: void waitForServers(const std::vector & ids) const; void waitForCatchUp() const; - NuKeeperStorage::ResponsesForSessions shutdown(const NuKeeperStorage::RequestsForSessions & expired_requests); + void shutdown(); }; } diff --git a/src/Coordination/NuKeeperStateMachine.cpp b/src/Coordination/NuKeeperStateMachine.cpp index f7b7ba3c567..092b2b0580f 100644 --- a/src/Coordination/NuKeeperStateMachine.cpp +++ b/src/Coordination/NuKeeperStateMachine.cpp @@ -43,7 +43,7 @@ nuraft::ptr writeResponses(NuKeeperStorage::ResponsesForSessions } -NuKeeperStateMachine::NuKeeperStateMachine(long tick_time) +NuKeeperStateMachine::NuKeeperStateMachine(int64_t tick_time) : storage(tick_time) , last_committed_idx(0) , log(&Poco::Logger::get("NuRaftStateMachine")) @@ -240,4 +240,10 @@ std::unordered_set NuKeeperStateMachine::getDeadSessions() return storage.getDeadSessions(); } +void NuKeeperStateMachine::shutdownStorage() +{ + std::lock_guard lock(storage_lock); + storage.finalize(); +} + } diff --git a/src/Coordination/NuKeeperStateMachine.h b/src/Coordination/NuKeeperStateMachine.h index bfb67f10a67..e45c197db8c 100644 --- a/src/Coordination/NuKeeperStateMachine.h +++ b/src/Coordination/NuKeeperStateMachine.h @@ -51,6 +51,8 @@ public: std::unordered_set getDeadSessions(); + void shutdownStorage(); + private: struct StorageSnapshot { diff --git a/src/Coordination/NuKeeperStorage.cpp b/src/Coordination/NuKeeperStorage.cpp index 45701b63b8b..679426a1a64 100644 --- a/src/Coordination/NuKeeperStorage.cpp +++ b/src/Coordination/NuKeeperStorage.cpp @@ -513,50 +513,23 @@ struct NuKeeperStorageCloseRequest final : public NuKeeperStorageRequest } }; -NuKeeperStorage::ResponsesForSessions NuKeeperStorage::finalize(const RequestsForSessions & expired_requests) +void NuKeeperStorage::finalize() { if (finalized) throw DB::Exception("Testkeeper storage already finalized", ErrorCodes::LOGICAL_ERROR); finalized = true; - /// TODO delete ephemerals - ResponsesForSessions finalize_results; - auto finish_watch = [] (const auto & watch_pair) -> ResponsesForSessions - { - ResponsesForSessions results; - std::shared_ptr response = std::make_shared(); - response->type = Coordination::SESSION; - response->state = Coordination::EXPIRED_SESSION; - response->error = Coordination::Error::ZSESSIONEXPIRED; + for (const auto & [session_id, ephemerals] : ephemerals) + for (const String & ephemeral_path : ephemerals) + container.erase(ephemeral_path); - for (auto & watcher_session : watch_pair.second) - results.push_back(ResponseForSession{watcher_session, response}); - return results; - }; - - for (auto & path_watch : watches) - { - auto watch_responses = finish_watch(path_watch); - finalize_results.insert(finalize_results.end(), watch_responses.begin(), watch_responses.end()); - } + ephemerals.clear(); watches.clear(); - for (auto & path_watch : list_watches) - { - auto list_watch_responses = finish_watch(path_watch); - finalize_results.insert(finalize_results.end(), list_watch_responses.begin(), list_watch_responses.end()); - } list_watches.clear(); sessions_and_watchers.clear(); - - for (const auto & [session_id, zk_request] : expired_requests) - { - auto response = zk_request->makeResponse(); - response->error = Coordination::Error::ZSESSIONEXPIRED; - finalize_results.push_back(ResponseForSession{session_id, response}); - } - return finalize_results; + session_expiry_queue.clear(); } @@ -675,15 +648,6 @@ NuKeeperStorage::ResponsesForSessions NuKeeperStorage::processRequest(const Coor watches[zk_request->getPath()].emplace_back(session_id); sessions_and_watchers[session_id].emplace(zk_request->getPath()); } - else - { - std::shared_ptr watch_response = std::make_shared(); - watch_response->path = zk_request->getPath(); - watch_response->xid = -1; - watch_response->error = response->error; - watch_response->type = Coordination::Event::NOTWATCHING; - results.push_back(ResponseForSession{session_id, watch_response}); - } } if (response->error == Coordination::Error::ZOK) diff --git a/src/Coordination/NuKeeperStorage.h b/src/Coordination/NuKeeperStorage.h index 6f709a6f480..20ab1982b4e 100644 --- a/src/Coordination/NuKeeperStorage.h +++ b/src/Coordination/NuKeeperStorage.h @@ -87,7 +87,8 @@ public: } ResponsesForSessions processRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id); - ResponsesForSessions finalize(const RequestsForSessions & expired_requests); + + void finalize(); std::unordered_set getDeadSessions() { diff --git a/src/Coordination/NuKeeperStorageDispatcher.cpp b/src/Coordination/NuKeeperStorageDispatcher.cpp index cf36fd40bc3..fbf54106316 100644 --- a/src/Coordination/NuKeeperStorageDispatcher.cpp +++ b/src/Coordination/NuKeeperStorageDispatcher.cpp @@ -32,7 +32,7 @@ void NuKeeperStorageDispatcher::processingThread() try { - auto responses = server->putRequests({request}); + auto responses = server->putRequest(request); for (const auto & response_for_session : responses) setResponse(response_for_session.session_id, response_for_session.response); } @@ -196,17 +196,16 @@ void NuKeeperStorageDispatcher::shutdown() } if (server) + server->shutdown(); + + NuKeeperStorage::RequestForSession request_for_session; + while (requests_queue.tryPop(request_for_session)) { - NuKeeperStorage::RequestsForSessions expired_requests; - NuKeeperStorage::RequestForSession request; - while (requests_queue.tryPop(request)) - expired_requests.push_back(NuKeeperStorage::RequestForSession{request}); - - auto expired_responses = server->shutdown(expired_requests); - - for (const auto & response_for_session : expired_responses) - setResponse(response_for_session.session_id, response_for_session.response); + auto response = request_for_session.request->makeResponse(); + response->error = Coordination::Error::ZSESSIONEXPIRED; + setResponse(request_for_session.session_id, response); } + session_to_response_callback.clear(); } catch (...) { diff --git a/src/Coordination/SessionExpiryQueue.cpp b/src/Coordination/SessionExpiryQueue.cpp index f90cd089be8..51837087af5 100644 --- a/src/Coordination/SessionExpiryQueue.cpp +++ b/src/Coordination/SessionExpiryQueue.cpp @@ -74,4 +74,10 @@ std::unordered_set SessionExpiryQueue::getExpiredSessions() return {}; } +void SessionExpiryQueue::clear() +{ + session_to_timeout.clear(); + expiry_to_sessions.clear(); +} + } diff --git a/src/Coordination/SessionExpiryQueue.h b/src/Coordination/SessionExpiryQueue.h index 3b4ad6dde88..dff629a2432 100644 --- a/src/Coordination/SessionExpiryQueue.h +++ b/src/Coordination/SessionExpiryQueue.h @@ -38,6 +38,8 @@ public: bool update(int64_t session_id, int64_t timeout_ms); std::unordered_set getExpiredSessions(); + + void clear(); }; } diff --git a/src/Server/NuKeeperTCPHandler.cpp b/src/Server/NuKeeperTCPHandler.cpp index 9d39c317356..706b57ee71d 100644 --- a/src/Server/NuKeeperTCPHandler.cpp +++ b/src/Server/NuKeeperTCPHandler.cpp @@ -404,12 +404,13 @@ void NuKeeperTCPHandler::runImpl() LOG_DEBUG(log, "Session #{} successfully closed", session_id); return; } - - if (response->error == Coordination::Error::ZOK) - response->write(*out); - else if (response->xid != Coordination::WATCH_XID) - response->write(*out); - /// skipping bad response for watch + response->write(*out); + if (response->error == Coordination::Error::ZSESSIONEXPIRED) + { + LOG_DEBUG(log, "Session #{} expired because server shutting down or quorum is not alive", session_id); + nu_keeper_storage_dispatcher->finishSession(session_id); + return; + } result.ready_responses_count--; } From d85e9b496c0292675778f88dbddaa99dc030de52 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 4 Feb 2021 16:22:30 +0300 Subject: [PATCH 59/86] Fix gcc-10 build --- src/Coordination/NuKeeperStorage.cpp | 4 ++-- tests/integration/test_testkeeper_multinode/test.py | 12 +++++++++++- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/src/Coordination/NuKeeperStorage.cpp b/src/Coordination/NuKeeperStorage.cpp index 679426a1a64..ef59e717b4c 100644 --- a/src/Coordination/NuKeeperStorage.cpp +++ b/src/Coordination/NuKeeperStorage.cpp @@ -520,8 +520,8 @@ void NuKeeperStorage::finalize() finalized = true; - for (const auto & [session_id, ephemerals] : ephemerals) - for (const String & ephemeral_path : ephemerals) + for (const auto & [session_id, ephemerals_paths] : ephemerals) + for (const String & ephemeral_path : ephemerals_paths) container.erase(ephemeral_path); ephemerals.clear(); diff --git a/tests/integration/test_testkeeper_multinode/test.py b/tests/integration/test_testkeeper_multinode/test.py index d815af7a63e..caba7ecddd9 100644 --- a/tests/integration/test_testkeeper_multinode/test.py +++ b/tests/integration/test_testkeeper_multinode/test.py @@ -124,6 +124,11 @@ def test_blocade_leader(started_cluster): node.query("SYSTEM SYNC REPLICA t1", timeout=10) break except Exception as ex: + try: + node.query("ATTACH TABLE t1") + except Exception as attach_ex: + print("Got exception node{}".format(n + 1), smaller_exception(attach_ex)) + print("Got exception node{}".format(n + 1), smaller_exception(ex)) time.sleep(0.5) else: @@ -229,13 +234,18 @@ def test_blocade_leader_twice(started_cluster): else: assert False, "Cannot reconnect for node{}".format(n + 1) - for node in [node1, node2, node3]: + for n, node in enumerate([node1, node2, node3]): for i in range(100): try: node.query("SYSTEM RESTART REPLICA t2", timeout=10) node.query("SYSTEM SYNC REPLICA t2", timeout=10) break except Exception as ex: + try: + node.query("ATTACH TABLE t2") + except Exception as attach_ex: + print("Got exception node{}".format(n + 1), smaller_exception(attach_ex)) + print("Got exception node{}".format(n + 1), smaller_exception(ex)) time.sleep(0.5) else: From 933105a6678f7db1e520f77434acf03c013dce7f Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 4 Feb 2021 16:31:38 +0300 Subject: [PATCH 60/86] Fix session timeout --- tests/integration/test_testkeeper_back_to_back/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_testkeeper_back_to_back/test.py b/tests/integration/test_testkeeper_back_to_back/test.py index d3a9b742cdd..0f2c1ed19a5 100644 --- a/tests/integration/test_testkeeper_back_to_back/test.py +++ b/tests/integration/test_testkeeper_back_to_back/test.py @@ -25,7 +25,7 @@ def get_fake_zk(): global _fake_zk_instance if not _fake_zk_instance: print("node", cluster.get_instance_ip("node")) - _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip("node") + ":9181") + _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip("node") + ":9181", timeout=30.0) def reset_last_zxid_listener(state): print("Fake zk callback called for state", state) global _fake_zk_instance From c5312bf362929d95b2269c9c7c707adda20a5f84 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 4 Feb 2021 22:29:46 +0300 Subject: [PATCH 61/86] Trying to disable suspicious parameter --- src/Coordination/NuKeeperServer.cpp | 7 +++++-- src/Coordination/NuKeeperServer.h | 2 +- src/Coordination/NuKeeperStorageDispatcher.cpp | 3 ++- src/Server/NuKeeperTCPHandler.cpp | 2 ++ 4 files changed, 10 insertions(+), 4 deletions(-) diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index 3910376ebda..aa1747ca3e6 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -43,7 +43,7 @@ void NuKeeperServer::addServer(int server_id_, const std::string & server_uri_, } -void NuKeeperServer::startup() +void NuKeeperServer::startup(int64_t operation_timeout_ms) { nuraft::raft_params params; params.heart_beat_interval_ = 500; @@ -51,8 +51,10 @@ void NuKeeperServer::startup() params.election_timeout_upper_bound_ = 2000; params.reserved_log_items_ = 5000; params.snapshot_distance_ = 5000; - params.client_req_timeout_ = 10000; + params.client_req_timeout_ = operation_timeout_ms; params.auto_forwarding_ = true; + /// For some reason may lead to a very long timeouts + params.use_bg_thread_for_urgent_commit_ = false; params.return_method_ = nuraft::raft_params::blocking; nuraft::asio_service::options asio_opts{}; @@ -197,6 +199,7 @@ int64_t NuKeeperServer::getSessionID(int64_t session_timeout_ms) std::lock_guard lock(append_entries_mutex); auto result = raft_instance->append_entries({entry}); + if (!result->get_accepted()) throw Exception(ErrorCodes::RAFT_ERROR, "Cannot send session_id request to RAFT"); diff --git a/src/Coordination/NuKeeperServer.h b/src/Coordination/NuKeeperServer.h index 358a4212967..6151cd095e0 100644 --- a/src/Coordination/NuKeeperServer.h +++ b/src/Coordination/NuKeeperServer.h @@ -34,7 +34,7 @@ private: public: NuKeeperServer(int server_id_, const std::string & hostname_, int port_); - void startup(); + void startup(int64_t operation_timeout_ms); NuKeeperStorage::ResponsesForSessions putRequest(const NuKeeperStorage::RequestForSession & request); diff --git a/src/Coordination/NuKeeperStorageDispatcher.cpp b/src/Coordination/NuKeeperStorageDispatcher.cpp index fbf54106316..e327272cab1 100644 --- a/src/Coordination/NuKeeperStorageDispatcher.cpp +++ b/src/Coordination/NuKeeperStorageDispatcher.cpp @@ -111,6 +111,7 @@ void NuKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigurati int myport; int32_t my_priority = 1; + operation_timeout = Poco::Timespan(0, config.getUInt("test_keeper_server.operation_timeout_ms", Coordination::DEFAULT_OPERATION_TIMEOUT_MS) * 1000); Poco::Util::AbstractConfiguration::Keys keys; config.keys("test_keeper_server.raft_configuration", keys); bool my_can_become_leader = true; @@ -141,7 +142,7 @@ void NuKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigurati server = std::make_unique(myid, myhostname, myport); try { - server->startup(); + server->startup(operation_timeout.totalMilliseconds()); if (shouldBuildQuorum(myid, my_priority, my_can_become_leader, server_configs)) { for (const auto & [id, hostname, port, can_become_leader, priority] : server_configs) diff --git a/src/Server/NuKeeperTCPHandler.cpp b/src/Server/NuKeeperTCPHandler.cpp index 706b57ee71d..31ffc744aaa 100644 --- a/src/Server/NuKeeperTCPHandler.cpp +++ b/src/Server/NuKeeperTCPHandler.cpp @@ -331,7 +331,9 @@ void NuKeeperTCPHandler::runImpl() { try { + LOG_INFO(log, "Requesting session ID for the new client"); session_id = nu_keeper_storage_dispatcher->getSessionID(session_timeout.totalMilliseconds()); + LOG_INFO(log, "Received session ID {}", session_id); } catch (const Exception & e) { From ab98040003b5e6c3e324f19b6c11c26fb0c8c96e Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 5 Feb 2021 10:15:28 +0300 Subject: [PATCH 62/86] More logs --- src/Coordination/LoggerWrapper.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/LoggerWrapper.h b/src/Coordination/LoggerWrapper.h index 00d4c6544a5..fcc24edea14 100644 --- a/src/Coordination/LoggerWrapper.h +++ b/src/Coordination/LoggerWrapper.h @@ -11,7 +11,7 @@ class LoggerWrapper : public nuraft::logger public: LoggerWrapper(const std::string & name) : log(&Poco::Logger::get(name)) - , level(4) + , level(6) { log->setLevel(level); } From 786e687b2fa2d77784b4569ecd95e8170c743e58 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 8 Feb 2021 14:01:50 +0300 Subject: [PATCH 63/86] Trying to avoid unlimited wait --- contrib/NuRaft | 2 +- src/Coordination/NuKeeperServer.cpp | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index c6f8528ead6..7adf7ae33e7 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit c6f8528ead61f7e4565164c6f15afef221235aa8 +Subproject commit 7adf7ae33e7d5c307342431b577c8ab1025ee793 diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index aa1747ca3e6..6111bdb2dd9 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -53,8 +53,6 @@ void NuKeeperServer::startup(int64_t operation_timeout_ms) params.snapshot_distance_ = 5000; params.client_req_timeout_ = operation_timeout_ms; params.auto_forwarding_ = true; - /// For some reason may lead to a very long timeouts - params.use_bg_thread_for_urgent_commit_ = false; params.return_method_ = nuraft::raft_params::blocking; nuraft::asio_service::options asio_opts{}; From add89c17f2f0ecbf83bda559101301cef9f15b99 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 8 Feb 2021 16:06:55 +0300 Subject: [PATCH 64/86] Less serde in responses, watches on followers --- src/Coordination/NuKeeperServer.cpp | 69 ++------- src/Coordination/NuKeeperServer.h | 6 +- src/Coordination/NuKeeperStateMachine.cpp | 18 ++- src/Coordination/NuKeeperStateMachine.h | 9 +- .../NuKeeperStorageDispatcher.cpp | 53 +++++-- src/Coordination/NuKeeperStorageDispatcher.h | 7 +- src/Coordination/ThreadSafeQueue.h | 45 ++++++ src/Coordination/tests/gtest_for_build.cpp | 131 ------------------ src/Server/NuKeeperTCPHandler.cpp | 30 ---- src/Server/NuKeeperTCPHandler.h | 5 +- 10 files changed, 125 insertions(+), 248 deletions(-) create mode 100644 src/Coordination/ThreadSafeQueue.h diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index 6111bdb2dd9..cbd52b98377 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -17,16 +17,16 @@ namespace ErrorCodes { extern const int TIMEOUT_EXCEEDED; extern const int RAFT_ERROR; - extern const int LOGICAL_ERROR; } -NuKeeperServer::NuKeeperServer(int server_id_, const std::string & hostname_, int port_) +NuKeeperServer::NuKeeperServer(int server_id_, const std::string & hostname_, int port_, ResponsesQueue & responses_queue_) : server_id(server_id_) , hostname(hostname_) , port(port_) , endpoint(hostname + ":" + std::to_string(port)) - , state_machine(nuraft::cs_new(500 /* FIXME */)) + , state_machine(nuraft::cs_new(responses_queue_)) , state_manager(nuraft::cs_new(server_id, endpoint)) + , responses_queue(responses_queue_) { } @@ -53,6 +53,7 @@ void NuKeeperServer::startup(int64_t operation_timeout_ms) params.snapshot_distance_ = 5000; params.client_req_timeout_ = operation_timeout_ms; params.auto_forwarding_ = true; + params.auto_forwarding_req_timeout_ = operation_timeout_ms * 2; params.return_method_ = nuraft::raft_params::blocking; nuraft::asio_service::options asio_opts{}; @@ -94,58 +95,14 @@ nuraft::ptr getZooKeeperLogEntry(int64_t session_id, const Coord return buf.getBuffer(); } -NuKeeperStorage::ResponsesForSessions readZooKeeperResponses(nuraft::ptr & buffer, const Coordination::ZooKeeperRequestPtr & request) -{ - DB::NuKeeperStorage::ResponsesForSessions results; - DB::ReadBufferFromNuraftBuffer buf(buffer); - bool response_found = false; - - while (!buf.eof()) - { - int64_t session_id; - DB::readIntBinary(session_id, buf); - int32_t length; - Coordination::XID xid; - int64_t zxid; - Coordination::Error err; - - Coordination::read(length, buf); - Coordination::read(xid, buf); - Coordination::read(zxid, buf); - Coordination::read(err, buf); - Coordination::ZooKeeperResponsePtr response; - - if (xid == Coordination::WATCH_XID) - response = std::make_shared(); - else - { - if (response_found) - throw Exception(ErrorCodes::LOGICAL_ERROR, "More than one non-watch response for single request with xid {}, response xid {}", request->xid, xid); - - response_found = true; - response = request->makeResponse(); - } - - if (err == Coordination::Error::ZOK && (xid == Coordination::WATCH_XID || response->getOpNum() != Coordination::OpNum::Close)) - response->readImpl(buf); - - response->xid = xid; - response->zxid = zxid; - response->error = err; - - results.push_back(DB::NuKeeperStorage::ResponseForSession{session_id, response}); - } - return results; } -} - -NuKeeperStorage::ResponsesForSessions NuKeeperServer::putRequest(const NuKeeperStorage::RequestForSession & request_for_session) +void NuKeeperServer::putRequest(const NuKeeperStorage::RequestForSession & request_for_session) { auto [session_id, request] = request_for_session; - if (isLeaderAlive() && request_for_session.request->isReadRequest()) + if (isLeaderAlive() && request->isReadRequest()) { - return state_machine->processReadRequest(request_for_session); + state_machine->processReadRequest(request_for_session); } else { @@ -162,8 +119,7 @@ NuKeeperStorage::ResponsesForSessions NuKeeperServer::putRequest(const NuKeeperS response->xid = request->xid; response->zxid = 0; response->error = Coordination::Error::ZOPERATIONTIMEOUT; - responses.push_back(DB::NuKeeperStorage::ResponseForSession{session_id, response}); - return responses; + responses_queue.push(DB::NuKeeperStorage::ResponseForSession{session_id, response}); } if (result->get_result_code() == nuraft::cmd_result_code::TIMEOUT) @@ -173,17 +129,10 @@ NuKeeperStorage::ResponsesForSessions NuKeeperServer::putRequest(const NuKeeperS response->xid = request->xid; response->zxid = 0; response->error = Coordination::Error::ZOPERATIONTIMEOUT; - responses.push_back(DB::NuKeeperStorage::ResponseForSession{session_id, response}); - return responses; + responses_queue.push(DB::NuKeeperStorage::ResponseForSession{session_id, response}); } else if (result->get_result_code() != nuraft::cmd_result_code::OK) throw Exception(ErrorCodes::RAFT_ERROR, "Requests result failed with code {} and message: '{}'", result->get_result_code(), result->get_result_str()); - - auto result_buf = result->get(); - if (result_buf == nullptr) - throw Exception(ErrorCodes::RAFT_ERROR, "Received nullptr from RAFT leader"); - - return readZooKeeperResponses(result_buf, request); } } diff --git a/src/Coordination/NuKeeperServer.h b/src/Coordination/NuKeeperServer.h index 6151cd095e0..5646bbbd002 100644 --- a/src/Coordination/NuKeeperServer.h +++ b/src/Coordination/NuKeeperServer.h @@ -31,12 +31,14 @@ private: std::mutex append_entries_mutex; + ResponsesQueue & responses_queue; + public: - NuKeeperServer(int server_id_, const std::string & hostname_, int port_); + NuKeeperServer(int server_id_, const std::string & hostname_, int port_, ResponsesQueue & responses_queue_); void startup(int64_t operation_timeout_ms); - NuKeeperStorage::ResponsesForSessions putRequest(const NuKeeperStorage::RequestForSession & request); + void putRequest(const NuKeeperStorage::RequestForSession & request); int64_t getSessionID(int64_t session_timeout_ms); diff --git a/src/Coordination/NuKeeperStateMachine.cpp b/src/Coordination/NuKeeperStateMachine.cpp index 092b2b0580f..7896caad568 100644 --- a/src/Coordination/NuKeeperStateMachine.cpp +++ b/src/Coordination/NuKeeperStateMachine.cpp @@ -43,8 +43,9 @@ nuraft::ptr writeResponses(NuKeeperStorage::ResponsesForSessions } -NuKeeperStateMachine::NuKeeperStateMachine(int64_t tick_time) +NuKeeperStateMachine::NuKeeperStateMachine(ResponsesQueue & responses_queue_, long tick_time) : storage(tick_time) + , responses_queue(responses_queue_) , last_committed_idx(0) , log(&Poco::Logger::get("NuRaftStateMachine")) { @@ -76,10 +77,12 @@ nuraft::ptr NuKeeperStateMachine::commit(const size_t log_idx, n { std::lock_guard lock(storage_lock); responses_for_sessions = storage.processRequest(request_for_session.request, request_for_session.session_id); + for (auto & response_for_session : responses_for_sessions) + responses_queue.push(response_for_session); } last_committed_idx = log_idx; - return writeResponses(responses_for_sessions); + return nullptr; } } @@ -228,10 +231,15 @@ int NuKeeperStateMachine::read_logical_snp_obj( return 0; } -NuKeeperStorage::ResponsesForSessions NuKeeperStateMachine::processReadRequest(const NuKeeperStorage::RequestForSession & request_for_session) +void NuKeeperStateMachine::processReadRequest(const NuKeeperStorage::RequestForSession & request_for_session) { - std::lock_guard lock(storage_lock); - return storage.processRequest(request_for_session.request, request_for_session.session_id); + NuKeeperStorage::ResponsesForSessions responses; + { + std::lock_guard lock(storage_lock); + responses = storage.processRequest(request_for_session.request, request_for_session.session_id); + } + for (const auto & response : responses) + responses_queue.push(response); } std::unordered_set NuKeeperStateMachine::getDeadSessions() diff --git a/src/Coordination/NuKeeperStateMachine.h b/src/Coordination/NuKeeperStateMachine.h index e45c197db8c..6dfb9ff4c3a 100644 --- a/src/Coordination/NuKeeperStateMachine.h +++ b/src/Coordination/NuKeeperStateMachine.h @@ -3,14 +3,17 @@ #include #include #include +#include namespace DB { +using ResponsesQueue = ThreadSafeQueue; + class NuKeeperStateMachine : public nuraft::state_machine { public: - NuKeeperStateMachine(long tick_time = 500); + NuKeeperStateMachine(ResponsesQueue & responses_queue_, long tick_time = 500); nuraft::ptr pre_commit(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override { return nullptr; } @@ -47,7 +50,7 @@ public: return storage; } - NuKeeperStorage::ResponsesForSessions processReadRequest(const NuKeeperStorage::RequestForSession & request_for_session); + void processReadRequest(const NuKeeperStorage::RequestForSession & request_for_session); std::unordered_set getDeadSessions(); @@ -74,6 +77,8 @@ private: static void writeSnapshot(const StorageSnapshotPtr & snapshot, nuraft::ptr & out); NuKeeperStorage storage; + + ResponsesQueue & responses_queue; /// Mutex for snapshots std::mutex snapshots_lock; diff --git a/src/Coordination/NuKeeperStorageDispatcher.cpp b/src/Coordination/NuKeeperStorageDispatcher.cpp index e327272cab1..86bdae9cc37 100644 --- a/src/Coordination/NuKeeperStorageDispatcher.cpp +++ b/src/Coordination/NuKeeperStorageDispatcher.cpp @@ -16,9 +16,9 @@ NuKeeperStorageDispatcher::NuKeeperStorageDispatcher() { } -void NuKeeperStorageDispatcher::processingThread() +void NuKeeperStorageDispatcher::requestThread() { - setThreadName("NuKeeperSProc"); + setThreadName("NuKeeperReqT"); while (!shutdown_called) { NuKeeperStorage::RequestForSession request; @@ -32,9 +32,33 @@ void NuKeeperStorageDispatcher::processingThread() try { - auto responses = server->putRequest(request); - for (const auto & response_for_session : responses) - setResponse(response_for_session.session_id, response_for_session.response); + server->putRequest(request); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + } +} + +void NuKeeperStorageDispatcher::responseThread() +{ + setThreadName("NuKeeperRspT"); + while (!shutdown_called) + { + NuKeeperStorage::ResponseForSession response_for_session; + + UInt64 max_wait = UInt64(operation_timeout.totalMilliseconds()); + + if (responses_queue.tryPop(response_for_session, max_wait)) + { + if (shutdown_called) + break; + + try + { + setResponse(response_for_session.session_id, response_for_session.response); } catch (...) { @@ -139,7 +163,7 @@ void NuKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigurati ids.push_back(server_id); } - server = std::make_unique(myid, myhostname, myport); + server = std::make_unique(myid, myhostname, myport, responses_queue); try { server->startup(operation_timeout.totalMilliseconds()); @@ -170,7 +194,8 @@ void NuKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigurati throw; } - processing_thread = ThreadFromGlobalPool([this] { processingThread(); }); + request_thread = ThreadFromGlobalPool([this] { requestThread(); }); + responses_thread = ThreadFromGlobalPool([this] { responseThread(); }); session_cleaner_thread = ThreadFromGlobalPool([this] { sessionCleanerTask(); }); LOG_DEBUG(log, "Dispatcher initialized"); @@ -192,8 +217,11 @@ void NuKeeperStorageDispatcher::shutdown() if (session_cleaner_thread.joinable()) session_cleaner_thread.join(); - if (processing_thread.joinable()) - processing_thread.join(); + if (request_thread.joinable()) + request_thread.join(); + + if (responses_thread.joinable()) + responses_thread.join(); } if (server) @@ -246,12 +274,7 @@ void NuKeeperStorageDispatcher::sessionCleanerTask() Coordination::ZooKeeperRequestPtr request = Coordination::ZooKeeperRequestFactory::instance().get(Coordination::OpNum::Close); request->xid = Coordination::CLOSE_XID; putRequest(request, dead_session); - { - std::lock_guard lock(session_to_response_callback_mutex); - auto session_it = session_to_response_callback.find(dead_session); - if (session_it != session_to_response_callback.end()) - session_to_response_callback.erase(session_it); - } + finishSession(dead_session); } } } diff --git a/src/Coordination/NuKeeperStorageDispatcher.h b/src/Coordination/NuKeeperStorageDispatcher.h index dfd36b39537..6820247a5af 100644 --- a/src/Coordination/NuKeeperStorageDispatcher.h +++ b/src/Coordination/NuKeeperStorageDispatcher.h @@ -31,13 +31,15 @@ private: using RequestsQueue = ConcurrentBoundedQueue; RequestsQueue requests_queue{1}; + ResponsesQueue responses_queue; std::atomic shutdown_called{false}; using SessionToResponseCallback = std::unordered_map; std::mutex session_to_response_callback_mutex; SessionToResponseCallback session_to_response_callback; - ThreadFromGlobalPool processing_thread; + ThreadFromGlobalPool request_thread; + ThreadFromGlobalPool responses_thread; ThreadFromGlobalPool session_cleaner_thread; @@ -46,7 +48,8 @@ private: Poco::Logger * log; private: - void processingThread(); + void requestThread(); + void responseThread(); void sessionCleanerTask(); void setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response); diff --git a/src/Coordination/ThreadSafeQueue.h b/src/Coordination/ThreadSafeQueue.h new file mode 100644 index 00000000000..d36e25244bb --- /dev/null +++ b/src/Coordination/ThreadSafeQueue.h @@ -0,0 +1,45 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/// Queue with mutex and condvar. As simple as possible. +template +class ThreadSafeQueue +{ +private: + mutable std::mutex queue_mutex; + std::condition_variable cv; + std::queue queue; +public: + + void push(const T & response) + { + std::lock_guard lock(queue_mutex); + queue.push(response); + cv.notify_one(); + } + + bool tryPop(T & response, int64_t timeout_ms = 0) + { + std::unique_lock lock(queue_mutex); + if (!cv.wait_for(lock, + std::chrono::milliseconds(timeout_ms), [this] { return !queue.empty(); })) + return false; + + response = queue.front(); + queue.pop(); + return true; + } + + size_t size() const + { + std::lock_guard lock(queue_mutex); + return queue.size(); + } +}; + +} diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index 956b12d6e08..baba7fc115e 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -272,9 +272,6 @@ TEST(CoordinationTest, TestSummingRaft3) s3.launcher.shutdown(5); } -using NuKeeperRaftServer = SimpliestRaftServer; - - nuraft::ptr getZooKeeperLogEntry(int64_t session_id, const Coordination::ZooKeeperRequestPtr & request) { DB::WriteBufferFromNuraftBuffer buf; @@ -337,132 +334,4 @@ TEST(CoordinationTest, TestStorageSerialization) EXPECT_EQ(new_storage.ephemerals[1].size(), 1); } -/// Code with obvious races, but I don't want to make it -/// more complex to avoid races. -#if defined(__has_feature) -# if ! __has_feature(thread_sanitizer) - -TEST(CoordinationTest, TestNuKeeperRaft) -{ - NuKeeperRaftServer s1(1, "localhost", 44447); - NuKeeperRaftServer s2(2, "localhost", 44448); - NuKeeperRaftServer s3(3, "localhost", 44449); - - nuraft::srv_config first_config(1, "localhost:44447"); - auto ret1 = s2.raft_instance->add_srv(first_config); - - EXPECT_TRUE(ret1->get_accepted()) << "failed to add server: " << ret1->get_result_str() << std::endl; - - while (s1.raft_instance->get_leader() != 2) - { - std::cout << "Waiting s1 to join to s2 quorum\n"; - std::this_thread::sleep_for(std::chrono::milliseconds(100)); - } - - nuraft::srv_config third_config(3, "localhost:44449"); - auto ret3 = s2.raft_instance->add_srv(third_config); - - EXPECT_TRUE(ret3->get_accepted()) << "failed to add server: " << ret3->get_result_str() << std::endl; - - while (s3.raft_instance->get_leader() != 2) - { - std::cout << "Waiting s3 to join to s2 quorum\n"; - std::this_thread::sleep_for(std::chrono::milliseconds(100)); - } - - /// S2 is leader - EXPECT_EQ(s1.raft_instance->get_leader(), 2); - EXPECT_EQ(s2.raft_instance->get_leader(), 2); - EXPECT_EQ(s3.raft_instance->get_leader(), 2); - - int64_t session_id = 34; - std::shared_ptr create_request = std::make_shared(); - create_request->path = "/hello"; - create_request->data = "world"; - - auto entry1 = getZooKeeperLogEntry(session_id, create_request); - auto ret_leader = s2.raft_instance->append_entries({entry1}); - - EXPECT_TRUE(ret_leader->get_accepted()) << "failed to replicate create entry:" << ret_leader->get_result_code(); - EXPECT_EQ(ret_leader->get_result_code(), nuraft::cmd_result_code::OK) << "failed to replicate create entry:" << ret_leader->get_result_code(); - - auto * result = ret_leader.get(); - - auto responses = getZooKeeperResponses(result->get(), create_request); - - EXPECT_EQ(responses.size(), 1); - EXPECT_EQ(responses[0].session_id, 34); - EXPECT_EQ(responses[0].response->getOpNum(), Coordination::OpNum::Create); - EXPECT_EQ(dynamic_cast(responses[0].response.get())->path_created, "/hello"); - - while (s1.state_machine->getStorage().container.count("/hello") == 0) - { - std::cout << "Waiting s1 to apply entry\n"; - std::this_thread::sleep_for(std::chrono::milliseconds(100)); - } - - while (s2.state_machine->getStorage().container.count("/hello") == 0) - { - std::cout << "Waiting s2 to apply entry\n"; - std::this_thread::sleep_for(std::chrono::milliseconds(100)); - } - - while (s3.state_machine->getStorage().container.count("/hello") == 0) - { - std::cout << "Waiting s3 to apply entry\n"; - std::this_thread::sleep_for(std::chrono::milliseconds(100)); - } - - EXPECT_EQ(s1.state_machine->getStorage().container["/hello"].data, "world"); - EXPECT_EQ(s2.state_machine->getStorage().container["/hello"].data, "world"); - EXPECT_EQ(s3.state_machine->getStorage().container["/hello"].data, "world"); - - std::shared_ptr get_request = std::make_shared(); - get_request->path = "/hello"; - auto entry2 = getZooKeeperLogEntry(session_id, get_request); - auto ret_leader_get = s2.raft_instance->append_entries({entry2}); - - EXPECT_TRUE(ret_leader_get->get_accepted()) << "failed to replicate create entry: " << ret_leader_get->get_result_code(); - EXPECT_EQ(ret_leader_get->get_result_code(), nuraft::cmd_result_code::OK) << "failed to replicate create entry: " << ret_leader_get->get_result_code(); - - auto * result_get = ret_leader_get.get(); - - auto get_responses = getZooKeeperResponses(result_get->get(), get_request); - - EXPECT_EQ(get_responses.size(), 1); - EXPECT_EQ(get_responses[0].session_id, 34); - EXPECT_EQ(get_responses[0].response->getOpNum(), Coordination::OpNum::Get); - EXPECT_EQ(dynamic_cast(get_responses[0].response.get())->data, "world"); - - - NuKeeperRaftServer s4(4, "localhost", 44450); - nuraft::srv_config fourth_config(4, "localhost:44450"); - auto ret4 = s2.raft_instance->add_srv(fourth_config); - while (s4.raft_instance->get_leader() != 2) - { - std::cout << "Waiting s1 to join to s2 quorum\n"; - std::this_thread::sleep_for(std::chrono::milliseconds(100)); - } - - /// Applied snapshot - EXPECT_EQ(s4.raft_instance->get_leader(), 2); - - while (s4.state_machine->getStorage().container.count("/hello") == 0) - { - std::cout << "Waiting s4 to apply entry\n"; - std::this_thread::sleep_for(std::chrono::milliseconds(100)); - } - - EXPECT_EQ(s4.state_machine->getStorage().container["/hello"].data, "world"); - - s1.launcher.shutdown(5); - s2.launcher.shutdown(5); - s3.launcher.shutdown(5); - s4.launcher.shutdown(5); -} - -# endif - -#endif - #endif diff --git a/src/Server/NuKeeperTCPHandler.cpp b/src/Server/NuKeeperTCPHandler.cpp index 31ffc744aaa..e855e2c68f7 100644 --- a/src/Server/NuKeeperTCPHandler.cpp +++ b/src/Server/NuKeeperTCPHandler.cpp @@ -45,36 +45,6 @@ struct PollResult bool error{false}; }; -/// Queue with mutex. As simple as possible. -class ThreadSafeResponseQueue -{ -private: - mutable std::mutex queue_mutex; - std::queue queue; -public: - void push(const Coordination::ZooKeeperResponsePtr & response) - { - std::lock_guard lock(queue_mutex); - queue.push(response); - } - bool tryPop(Coordination::ZooKeeperResponsePtr & response) - { - std::lock_guard lock(queue_mutex); - if (!queue.empty()) - { - response = queue.front(); - queue.pop(); - return true; - } - return false; - } - size_t size() const - { - std::lock_guard lock(queue_mutex); - return queue.size(); - } -}; - struct SocketInterruptablePollWrapper { int sockfd; diff --git a/src/Server/NuKeeperTCPHandler.h b/src/Server/NuKeeperTCPHandler.h index 641d2f78e1f..241867a1d99 100644 --- a/src/Server/NuKeeperTCPHandler.h +++ b/src/Server/NuKeeperTCPHandler.h @@ -16,6 +16,7 @@ #include #include #include +#include #include namespace DB @@ -23,7 +24,9 @@ namespace DB struct SocketInterruptablePollWrapper; using SocketInterruptablePollWrapperPtr = std::unique_ptr; -class ThreadSafeResponseQueue; + +using ThreadSafeResponseQueue = ThreadSafeQueue; + using ThreadSafeResponseQueuePtr = std::unique_ptr; class NuKeeperTCPHandler : public Poco::Net::TCPServerConnection From 2413d6bd381b79f680399feca023f4a6b7873f9c Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 8 Feb 2021 16:26:06 +0300 Subject: [PATCH 65/86] Test multinode watches --- .../test_testkeeper_multinode/test.py | 64 +++++++++++++++++++ 1 file changed, 64 insertions(+) diff --git a/tests/integration/test_testkeeper_multinode/test.py b/tests/integration/test_testkeeper_multinode/test.py index caba7ecddd9..ff001fb75ee 100644 --- a/tests/integration/test_testkeeper_multinode/test.py +++ b/tests/integration/test_testkeeper_multinode/test.py @@ -42,6 +42,70 @@ def test_simple_replicated_table(started_cluster): assert node3.query("SELECT COUNT() FROM t") == "10\n" +def get_fake_zk(nodename): + _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=30.0) + def reset_last_zxid_listener(state): + print("Fake zk callback called for state", state) + _fake_zk_instance.last_zxid = 0 + + _fake_zk_instance.add_listener(reset_last_zxid_listener) + _fake_zk_instance.start() + return _fake_zk_instance + +def test_watch_on_follower(started_cluster): + try: + node1_zk = get_fake_zk("node1") + node2_zk = get_fake_zk("node2") + node3_zk = get_fake_zk("node3") + + node1_zk.create("/test_data_watches") + node2_zk.set("/test_data_watches", b"hello") + node3_zk.set("/test_data_watches", b"world") + + node1_data = None + def node1_callback(event): + print("node1 data watch called") + nonlocal node1_data + node1_data = event + + node1_zk.get("/test_data_watches", watch=node1_callback) + + node2_data = None + def node2_callback(event): + print("node2 data watch called") + nonlocal node2_data + node2_data = event + + node2_zk.get("/test_data_watches", watch=node2_callback) + + node3_data = None + def node3_callback(event): + print("node3 data watch called") + nonlocal node3_data + node3_data = event + + node3_zk.get("/test_data_watches", watch=node3_callback) + + node1_zk.set("/test_data_watches", b"somevalue") + time.sleep(3) + + print(node1_data) + print(node2_data) + print(node3_data) + + assert node1_data == node2_data + assert node3_data == node2_data + + finally: + try: + for zk_conn in [node1_zk, node2_zk, node3_zk]: + zk_conn.stop() + zk_conn.close() + except: + pass + + + # in extremely rare case it can take more than 5 minutes in debug build with sanitizer @pytest.mark.timeout(600) def test_blocade_leader(started_cluster): From d57613aa188e38f70d386cc53cdf1eb51bd90f55 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 8 Feb 2021 16:50:03 +0300 Subject: [PATCH 66/86] Fix 0_o build in arcadia --- src/Coordination/InMemoryStateManager.h | 2 +- src/Coordination/LoggerWrapper.h | 2 +- src/Coordination/NuKeeperServer.h | 2 +- src/Coordination/NuKeeperStateMachine.h | 2 +- src/Coordination/ReadBufferFromNuraftBuffer.h | 2 +- src/Coordination/SummingStateMachine.h | 2 +- src/Coordination/WriteBufferFromNuraftBuffer.h | 2 +- src/Coordination/tests/gtest_for_build.cpp | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Coordination/InMemoryStateManager.h b/src/Coordination/InMemoryStateManager.h index 32eea343465..7446073c9c9 100644 --- a/src/Coordination/InMemoryStateManager.h +++ b/src/Coordination/InMemoryStateManager.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include // Y_IGNORE namespace DB { diff --git a/src/Coordination/LoggerWrapper.h b/src/Coordination/LoggerWrapper.h index fcc24edea14..c8da2372a91 100644 --- a/src/Coordination/LoggerWrapper.h +++ b/src/Coordination/LoggerWrapper.h @@ -1,6 +1,6 @@ #pragma once -#include +#include // Y_IGNORE #include namespace DB diff --git a/src/Coordination/NuKeeperServer.h b/src/Coordination/NuKeeperServer.h index 5646bbbd002..6fa2ae44ce2 100644 --- a/src/Coordination/NuKeeperServer.h +++ b/src/Coordination/NuKeeperServer.h @@ -1,6 +1,6 @@ #pragma once -#include +#include // Y_IGNORE #include #include #include diff --git a/src/Coordination/NuKeeperStateMachine.h b/src/Coordination/NuKeeperStateMachine.h index 6dfb9ff4c3a..b12903b6929 100644 --- a/src/Coordination/NuKeeperStateMachine.h +++ b/src/Coordination/NuKeeperStateMachine.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include // Y_IGNORE #include #include diff --git a/src/Coordination/ReadBufferFromNuraftBuffer.h b/src/Coordination/ReadBufferFromNuraftBuffer.h index cc01d3c8f39..3817e217881 100644 --- a/src/Coordination/ReadBufferFromNuraftBuffer.h +++ b/src/Coordination/ReadBufferFromNuraftBuffer.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include // Y_IGNORE namespace DB { diff --git a/src/Coordination/SummingStateMachine.h b/src/Coordination/SummingStateMachine.h index 9aca02c6bdc..c8594ba7e8d 100644 --- a/src/Coordination/SummingStateMachine.h +++ b/src/Coordination/SummingStateMachine.h @@ -1,6 +1,6 @@ #pragma once -#include +#include // Y_IGNORE #include #include #include diff --git a/src/Coordination/WriteBufferFromNuraftBuffer.h b/src/Coordination/WriteBufferFromNuraftBuffer.h index 47a01fbc2a4..d037a0e6a27 100644 --- a/src/Coordination/WriteBufferFromNuraftBuffer.h +++ b/src/Coordination/WriteBufferFromNuraftBuffer.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include // Y_IGNORE namespace DB { diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index baba7fc115e..82affd38062 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -20,7 +20,7 @@ #include #include #include -#include +#include // Y_IGNORE #include From f2feeb9b192d6d9444d09822a37c9fab103fbc91 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 8 Feb 2021 18:20:12 +0300 Subject: [PATCH 67/86] Missing fix --- src/Coordination/InMemoryLogStore.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/InMemoryLogStore.h b/src/Coordination/InMemoryLogStore.h index 37f76f056ba..425b056a81d 100644 --- a/src/Coordination/InMemoryLogStore.h +++ b/src/Coordination/InMemoryLogStore.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include // Y_IGNORE namespace DB { From b8baf3a4432166fa66c243236962b9a42a3855bb Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 8 Feb 2021 22:40:44 +0300 Subject: [PATCH 68/86] Fix some warnings --- src/Coordination/NuKeeperStateMachine.cpp | 2 +- src/Coordination/NuKeeperStateMachine.h | 2 +- src/Coordination/NuKeeperStorage.cpp | 1 - src/Server/NuKeeperTCPHandler.h | 2 +- 4 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Coordination/NuKeeperStateMachine.cpp b/src/Coordination/NuKeeperStateMachine.cpp index 7896caad568..9be8e889fa3 100644 --- a/src/Coordination/NuKeeperStateMachine.cpp +++ b/src/Coordination/NuKeeperStateMachine.cpp @@ -43,7 +43,7 @@ nuraft::ptr writeResponses(NuKeeperStorage::ResponsesForSessions } -NuKeeperStateMachine::NuKeeperStateMachine(ResponsesQueue & responses_queue_, long tick_time) +NuKeeperStateMachine::NuKeeperStateMachine(ResponsesQueue & responses_queue_, int64_t tick_time) : storage(tick_time) , responses_queue(responses_queue_) , last_committed_idx(0) diff --git a/src/Coordination/NuKeeperStateMachine.h b/src/Coordination/NuKeeperStateMachine.h index b12903b6929..5f3065ee144 100644 --- a/src/Coordination/NuKeeperStateMachine.h +++ b/src/Coordination/NuKeeperStateMachine.h @@ -13,7 +13,7 @@ using ResponsesQueue = ThreadSafeQueue; class NuKeeperStateMachine : public nuraft::state_machine { public: - NuKeeperStateMachine(ResponsesQueue & responses_queue_, long tick_time = 500); + NuKeeperStateMachine(ResponsesQueue & responses_queue_, int64_t tick_time = 500); nuraft::ptr pre_commit(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override { return nullptr; } diff --git a/src/Coordination/NuKeeperStorage.cpp b/src/Coordination/NuKeeperStorage.cpp index ef59e717b4c..a86b7432cbf 100644 --- a/src/Coordination/NuKeeperStorage.cpp +++ b/src/Coordination/NuKeeperStorage.cpp @@ -132,7 +132,6 @@ struct NuKeeperStorageCreateRequest final : public NuKeeperStorageRequest else { NuKeeperStorage::Node created_node; - created_node.seq_num = 0; created_node.stat.czxid = zxid; created_node.stat.mzxid = zxid; created_node.stat.ctime = std::chrono::system_clock::now().time_since_epoch() / std::chrono::milliseconds(1); diff --git a/src/Server/NuKeeperTCPHandler.h b/src/Server/NuKeeperTCPHandler.h index 241867a1d99..03a857ad1d7 100644 --- a/src/Server/NuKeeperTCPHandler.h +++ b/src/Server/NuKeeperTCPHandler.h @@ -41,7 +41,7 @@ private: std::shared_ptr nu_keeper_storage_dispatcher; Poco::Timespan operation_timeout; Poco::Timespan session_timeout; - int64_t session_id; + int64_t session_id{-1}; Stopwatch session_stopwatch; SocketInterruptablePollWrapperPtr poll_wrapper; From b130fbfd788fc013113e158225c29ff65594d410 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 9 Feb 2021 17:47:18 +0300 Subject: [PATCH 69/86] Add coordination settings --- src/Coordination/NuKeeperServer.cpp | 40 +++++++++++------- src/Coordination/NuKeeperServer.h | 12 ++++-- src/Coordination/NuKeeperStateMachine.cpp | 27 ++++++------ src/Coordination/NuKeeperStateMachine.h | 7 +++- .../NuKeeperStorageDispatcher.cpp | 22 +++++----- src/Coordination/NuKeeperStorageDispatcher.h | 4 +- tests/config/config.d/test_keeper_port.xml | 8 +++- .../configs/enable_test_keeper.xml | 8 +++- .../configs/enable_test_keeper1.xml | 8 +++- .../configs/enable_test_keeper2.xml | 8 +++- .../configs/enable_test_keeper3.xml | 8 +++- .../test_testkeeper_multinode/test.py | 42 ++++++++++++++++++- 12 files changed, 139 insertions(+), 55 deletions(-) diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index cbd52b98377..40508b08761 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -19,12 +19,16 @@ namespace ErrorCodes extern const int RAFT_ERROR; } -NuKeeperServer::NuKeeperServer(int server_id_, const std::string & hostname_, int port_, ResponsesQueue & responses_queue_) +NuKeeperServer::NuKeeperServer( + int server_id_, const std::string & hostname_, int port_, + const CoordinationSettingsPtr & coordination_settings_, + ResponsesQueue & responses_queue_) : server_id(server_id_) , hostname(hostname_) , port(port_) , endpoint(hostname + ":" + std::to_string(port)) - , state_machine(nuraft::cs_new(responses_queue_)) + , coordination_settings(coordination_settings_) + , state_machine(nuraft::cs_new(responses_queue_, coordination_settings)) , state_manager(nuraft::cs_new(server_id, endpoint)) , responses_queue(responses_queue_) { @@ -43,17 +47,18 @@ void NuKeeperServer::addServer(int server_id_, const std::string & server_uri_, } -void NuKeeperServer::startup(int64_t operation_timeout_ms) +void NuKeeperServer::startup() { nuraft::raft_params params; - params.heart_beat_interval_ = 500; - params.election_timeout_lower_bound_ = 1000; - params.election_timeout_upper_bound_ = 2000; - params.reserved_log_items_ = 5000; - params.snapshot_distance_ = 5000; - params.client_req_timeout_ = operation_timeout_ms; - params.auto_forwarding_ = true; - params.auto_forwarding_req_timeout_ = operation_timeout_ms * 2; + params.heart_beat_interval_ = coordination_settings->heart_beat_interval_ms.totalMilliseconds(); + params.election_timeout_lower_bound_ = coordination_settings->election_timeout_lower_bound_ms.totalMilliseconds(); + params.election_timeout_upper_bound_ = coordination_settings->election_timeout_upper_bound_ms.totalMilliseconds(); + params.reserved_log_items_ = coordination_settings->reserved_log_items; + params.snapshot_distance_ = coordination_settings->snapshot_distance; + params.client_req_timeout_ = coordination_settings->operation_timeout_ms.totalMilliseconds(); + params.auto_forwarding_ = coordination_settings->auto_forwarding; + params.auto_forwarding_req_timeout_ = coordination_settings->operation_timeout_ms.totalMilliseconds() * 2; + params.return_method_ = nuraft::raft_params::blocking; nuraft::asio_service::options asio_opts{}; @@ -65,6 +70,7 @@ void NuKeeperServer::startup(int64_t operation_timeout_ms) if (!raft_instance) throw Exception(ErrorCodes::RAFT_ERROR, "Cannot allocate RAFT instance"); + /// FIXME static constexpr auto MAX_RETRY = 100; for (size_t i = 0; i < MAX_RETRY; ++i) { @@ -80,7 +86,7 @@ void NuKeeperServer::startup(int64_t operation_timeout_ms) void NuKeeperServer::shutdown() { state_machine->shutdownStorage(); - if (!launcher.shutdown(5)) + if (!launcher.shutdown(coordination_settings->shutdown_timeout.totalSeconds())) LOG_WARNING(&Poco::Logger::get("NuKeeperServer"), "Failed to shutdown RAFT server in {} seconds", 5); } @@ -173,6 +179,7 @@ bool NuKeeperServer::isLeaderAlive() const bool NuKeeperServer::waitForServer(int32_t id) const { + /// FIXME for (size_t i = 0; i < 50; ++i) { if (raft_instance->get_srv_config(id) != nullptr) @@ -180,17 +187,22 @@ bool NuKeeperServer::waitForServer(int32_t id) const LOG_DEBUG(&Poco::Logger::get("NuRaftInit"), "Waiting for server {} to join the cluster", id); std::this_thread::sleep_for(std::chrono::milliseconds(100)); } + + LOG_DEBUG(&Poco::Logger::get("NuRaftInit"), "Cannot wait for server {}", id); return false; } -void NuKeeperServer::waitForServers(const std::vector & ids) const +bool NuKeeperServer::waitForServers(const std::vector & ids) const { for (int32_t id : ids) - waitForServer(id); + if (!waitForServer(id)) + return false; + return true; } void NuKeeperServer::waitForCatchUp() const { + /// FIXME while (raft_instance->is_catching_up() || raft_instance->is_receiving_snapshot() || raft_instance->is_leader()) { LOG_DEBUG(&Poco::Logger::get("NuRaftInit"), "Waiting current RAFT instance to catch up"); diff --git a/src/Coordination/NuKeeperServer.h b/src/Coordination/NuKeeperServer.h index 6fa2ae44ce2..bb5870fe89a 100644 --- a/src/Coordination/NuKeeperServer.h +++ b/src/Coordination/NuKeeperServer.h @@ -5,6 +5,7 @@ #include #include #include +#include #include namespace DB @@ -21,6 +22,8 @@ private: std::string endpoint; + CoordinationSettingsPtr coordination_settings; + nuraft::ptr state_machine; nuraft::ptr state_manager; @@ -34,9 +37,12 @@ private: ResponsesQueue & responses_queue; public: - NuKeeperServer(int server_id_, const std::string & hostname_, int port_, ResponsesQueue & responses_queue_); + NuKeeperServer( + int server_id_, const std::string & hostname_, int port_, + const CoordinationSettingsPtr & coordination_settings_, + ResponsesQueue & responses_queue_); - void startup(int64_t operation_timeout_ms); + void startup(); void putRequest(const NuKeeperStorage::RequestForSession & request); @@ -51,7 +57,7 @@ public: bool isLeaderAlive() const; bool waitForServer(int32_t server_id) const; - void waitForServers(const std::vector & ids) const; + bool waitForServers(const std::vector & ids) const; void waitForCatchUp() const; void shutdown(); diff --git a/src/Coordination/NuKeeperStateMachine.cpp b/src/Coordination/NuKeeperStateMachine.cpp index 9be8e889fa3..d282f57ce73 100644 --- a/src/Coordination/NuKeeperStateMachine.cpp +++ b/src/Coordination/NuKeeperStateMachine.cpp @@ -8,8 +8,6 @@ namespace DB { -static constexpr int MAX_SNAPSHOTS = 3; - NuKeeperStorage::RequestForSession parseRequest(nuraft::buffer & data) { ReadBufferFromNuraftBuffer buffer(data); @@ -43,8 +41,9 @@ nuraft::ptr writeResponses(NuKeeperStorage::ResponsesForSessions } -NuKeeperStateMachine::NuKeeperStateMachine(ResponsesQueue & responses_queue_, int64_t tick_time) - : storage(tick_time) +NuKeeperStateMachine::NuKeeperStateMachine(ResponsesQueue & responses_queue_, const CoordinationSettingsPtr & coordination_settings_) + : coordination_settings(coordination_settings_) + , storage(coordination_settings->dead_session_check_period_ms.totalMilliseconds()) , responses_queue(responses_queue_) , last_committed_idx(0) , log(&Poco::Logger::get("NuRaftStateMachine")) @@ -129,7 +128,7 @@ NuKeeperStateMachine::StorageSnapshotPtr NuKeeperStateMachine::readSnapshot(nura NuKeeperStorageSerializer serializer; ReadBufferFromNuraftBuffer reader(in); - NuKeeperStorage new_storage(500 /*FIXME*/); + NuKeeperStorage new_storage(coordination_settings->dead_session_check_period_ms.totalMilliseconds()); serializer.deserialize(new_storage, reader); return std::make_shared(ss, new_storage); } @@ -153,15 +152,19 @@ void NuKeeperStateMachine::create_snapshot( { std::lock_guard lock(snapshots_lock); snapshots[s.get_last_log_idx()] = snapshot; - int num = snapshots.size(); - auto entry = snapshots.begin(); - - for (int i = 0; i < num - MAX_SNAPSHOTS; ++i) + size_t num = snapshots.size(); + if (num > coordination_settings->max_stored_snapshots) { - if (entry == snapshots.end()) - break; - entry = snapshots.erase(entry); + auto entry = snapshots.begin(); + + for (size_t i = 0; i < num - coordination_settings->max_stored_snapshots; ++i) + { + if (entry == snapshots.end()) + break; + entry = snapshots.erase(entry); + } } + } nuraft::ptr except(nullptr); bool ret = true; diff --git a/src/Coordination/NuKeeperStateMachine.h b/src/Coordination/NuKeeperStateMachine.h index 5f3065ee144..87748db20a5 100644 --- a/src/Coordination/NuKeeperStateMachine.h +++ b/src/Coordination/NuKeeperStateMachine.h @@ -4,6 +4,7 @@ #include // Y_IGNORE #include #include +#include namespace DB { @@ -13,7 +14,7 @@ using ResponsesQueue = ThreadSafeQueue; class NuKeeperStateMachine : public nuraft::state_machine { public: - NuKeeperStateMachine(ResponsesQueue & responses_queue_, int64_t tick_time = 500); + NuKeeperStateMachine(ResponsesQueue & responses_queue_, const CoordinationSettingsPtr & coordination_settings_); nuraft::ptr pre_commit(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override { return nullptr; } @@ -72,10 +73,12 @@ private: StorageSnapshotPtr createSnapshotInternal(nuraft::snapshot & s); - static StorageSnapshotPtr readSnapshot(nuraft::snapshot & s, nuraft::buffer & in); + StorageSnapshotPtr readSnapshot(nuraft::snapshot & s, nuraft::buffer & in); static void writeSnapshot(const StorageSnapshotPtr & snapshot, nuraft::ptr & out); + CoordinationSettingsPtr coordination_settings; + NuKeeperStorage storage; ResponsesQueue & responses_queue; diff --git a/src/Coordination/NuKeeperStorageDispatcher.cpp b/src/Coordination/NuKeeperStorageDispatcher.cpp index 86bdae9cc37..914985ee534 100644 --- a/src/Coordination/NuKeeperStorageDispatcher.cpp +++ b/src/Coordination/NuKeeperStorageDispatcher.cpp @@ -12,7 +12,8 @@ namespace ErrorCodes } NuKeeperStorageDispatcher::NuKeeperStorageDispatcher() - : log(&Poco::Logger::get("NuKeeperDispatcher")) + : coordination_settings(std::make_shared()) + , log(&Poco::Logger::get("NuKeeperDispatcher")) { } @@ -23,7 +24,7 @@ void NuKeeperStorageDispatcher::requestThread() { NuKeeperStorage::RequestForSession request; - UInt64 max_wait = UInt64(operation_timeout.totalMilliseconds()); + UInt64 max_wait = UInt64(coordination_settings->operation_timeout_ms.totalMilliseconds()); if (requests_queue.tryPop(request, max_wait)) { @@ -49,7 +50,7 @@ void NuKeeperStorageDispatcher::responseThread() { NuKeeperStorage::ResponseForSession response_for_session; - UInt64 max_wait = UInt64(operation_timeout.totalMilliseconds()); + UInt64 max_wait = UInt64(coordination_settings->operation_timeout_ms.totalMilliseconds()); if (responses_queue.tryPop(response_for_session, max_wait)) { @@ -97,7 +98,7 @@ bool NuKeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperRequestP /// Put close requests without timeouts if (request->getOpNum() == Coordination::OpNum::Close) requests_queue.push(std::move(request_info)); - else if (!requests_queue.tryPush(std::move(request_info), operation_timeout.totalMilliseconds())) + else if (!requests_queue.tryPush(std::move(request_info), coordination_settings->operation_timeout_ms.totalMilliseconds())) throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::TIMEOUT_EXCEEDED); return true; } @@ -134,8 +135,8 @@ void NuKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigurati std::string myhostname; int myport; int32_t my_priority = 1; + coordination_settings->loadFromConfig("test_keeper_server.coordination_settings", config); - operation_timeout = Poco::Timespan(0, config.getUInt("test_keeper_server.operation_timeout_ms", Coordination::DEFAULT_OPERATION_TIMEOUT_MS) * 1000); Poco::Util::AbstractConfiguration::Keys keys; config.keys("test_keeper_server.raft_configuration", keys); bool my_can_become_leader = true; @@ -163,10 +164,10 @@ void NuKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigurati ids.push_back(server_id); } - server = std::make_unique(myid, myhostname, myport, responses_queue); + server = std::make_unique(myid, myhostname, myport, coordination_settings, responses_queue); try { - server->startup(operation_timeout.totalMilliseconds()); + server->startup(); if (shouldBuildQuorum(myid, my_priority, my_can_become_leader, server_configs)) { for (const auto & [id, hostname, port, can_become_leader, priority] : server_configs) @@ -183,8 +184,8 @@ void NuKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigurati } else { - LOG_DEBUG(log, "Waiting for {} servers to build cluster", ids.size()); - server->waitForServers(ids); + while (!server->waitForServers(ids)) + LOG_DEBUG(log, "Waiting for {} servers to build cluster", ids.size()); server->waitForCatchUp(); } } @@ -283,8 +284,7 @@ void NuKeeperStorageDispatcher::sessionCleanerTask() tryLogCurrentException(__PRETTY_FUNCTION__); } - /*FIXME*/ - std::this_thread::sleep_for(std::chrono::milliseconds(500)); + std::this_thread::sleep_for(std::chrono::milliseconds(coordination_settings->dead_session_check_period_ms.totalMilliseconds())); } } diff --git a/src/Coordination/NuKeeperStorageDispatcher.h b/src/Coordination/NuKeeperStorageDispatcher.h index 6820247a5af..62144b92a7a 100644 --- a/src/Coordination/NuKeeperStorageDispatcher.h +++ b/src/Coordination/NuKeeperStorageDispatcher.h @@ -14,6 +14,7 @@ #include #include #include +#include namespace DB @@ -25,10 +26,9 @@ class NuKeeperStorageDispatcher { private: - Poco::Timespan operation_timeout{0, Coordination::DEFAULT_OPERATION_TIMEOUT_MS * 1000}; - std::mutex push_request_mutex; + CoordinationSettingsPtr coordination_settings; using RequestsQueue = ConcurrentBoundedQueue; RequestsQueue requests_queue{1}; ResponsesQueue responses_queue; diff --git a/tests/config/config.d/test_keeper_port.xml b/tests/config/config.d/test_keeper_port.xml index fff60d749f6..6ca00a972d4 100644 --- a/tests/config/config.d/test_keeper_port.xml +++ b/tests/config/config.d/test_keeper_port.xml @@ -1,9 +1,13 @@ 9181 - 10000 - 30000 1 + + + 10000 + 30000 + + 1 diff --git a/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml b/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml index fff60d749f6..00a593051f9 100644 --- a/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml +++ b/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml @@ -1,9 +1,13 @@ 9181 - 10000 - 30000 1 + + + 5000 + 10000 + + 1 diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml index e1b6da40338..75065bb2a7a 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml @@ -1,9 +1,13 @@ 9181 - 5000 - 10000 1 + + + 5000 + 10000 + + 1 diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml index 7622aa164da..18937dd4910 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml @@ -1,9 +1,13 @@ 9181 - 5000 - 10000 2 + + + 5000 + 10000 + + 1 diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml index 1edbfa7271e..5330367cd89 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml @@ -1,9 +1,13 @@ 9181 - 5000 - 10000 3 + + + 5000 + 10000 + + 1 diff --git a/tests/integration/test_testkeeper_multinode/test.py b/tests/integration/test_testkeeper_multinode/test.py index ff001fb75ee..05879613ba6 100644 --- a/tests/integration/test_testkeeper_multinode/test.py +++ b/tests/integration/test_testkeeper_multinode/test.py @@ -52,6 +52,47 @@ def get_fake_zk(nodename): _fake_zk_instance.start() return _fake_zk_instance +def test_read_write_multinode(started_cluster): + try: + node1_zk = get_fake_zk("node1") + node2_zk = get_fake_zk("node2") + node3_zk = get_fake_zk("node3") + + node1_zk.create("/test_read_write_multinode_node1", b"somedata1") + node2_zk.create("/test_read_write_multinode_node2", b"somedata2") + node3_zk.create("/test_read_write_multinode_node3", b"somedata3") + + # stale reads are allowed + while node1_zk.exists("/test_read_write_multinode_node2") is None: + time.sleep(0.1) + + while node1_zk.exists("/test_read_write_multinode_node3") is None: + time.sleep(0.1) + + while node2_zk.exists("/test_read_write_multinode_node3") is None: + time.sleep(0.1) + + assert node3_zk.get("/test_read_write_multinode_node1")[0] == b"somedata1" + assert node2_zk.get("/test_read_write_multinode_node1")[0] == b"somedata1" + assert node1_zk.get("/test_read_write_multinode_node1")[0] == b"somedata1" + + assert node3_zk.get("/test_read_write_multinode_node2")[0] == b"somedata2" + assert node2_zk.get("/test_read_write_multinode_node2")[0] == b"somedata2" + assert node1_zk.get("/test_read_write_multinode_node2")[0] == b"somedata2" + + assert node3_zk.get("/test_read_write_multinode_node3")[0] == b"somedata3" + assert node2_zk.get("/test_read_write_multinode_node3")[0] == b"somedata3" + assert node1_zk.get("/test_read_write_multinode_node3")[0] == b"somedata3" + + finally: + try: + for zk_conn in [node1_zk, node2_zk, node3_zk]: + zk_conn.stop() + zk_conn.close() + except: + pass + + def test_watch_on_follower(started_cluster): try: node1_zk = get_fake_zk("node1") @@ -105,7 +146,6 @@ def test_watch_on_follower(started_cluster): pass - # in extremely rare case it can take more than 5 minutes in debug build with sanitizer @pytest.mark.timeout(600) def test_blocade_leader(started_cluster): From c78f3ba204683d2a7b22c050cd8821426b25965a Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 9 Feb 2021 18:39:15 +0300 Subject: [PATCH 70/86] Missed file --- src/Coordination/CoordinationSettings.cpp | 35 ++++++++++++++++++++ src/Coordination/CoordinationSettings.h | 40 +++++++++++++++++++++++ 2 files changed, 75 insertions(+) create mode 100644 src/Coordination/CoordinationSettings.cpp create mode 100644 src/Coordination/CoordinationSettings.h diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp new file mode 100644 index 00000000000..cd46817e82f --- /dev/null +++ b/src/Coordination/CoordinationSettings.cpp @@ -0,0 +1,35 @@ +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int UNKNOWN_SETTING; +} + +IMPLEMENT_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS) + +void CoordinationSettings::loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config) +{ + if (!config.has(config_elem)) + return; + + Poco::Util::AbstractConfiguration::Keys config_keys; + config.keys(config_elem, config_keys); + + try + { + for (const String & key : config_keys) + set(key, config.getString(config_elem + "." + key)); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_SETTING) + e.addMessage("in Coordination settings config"); + throw; + } +} + +} diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h new file mode 100644 index 00000000000..374d432f2db --- /dev/null +++ b/src/Coordination/CoordinationSettings.h @@ -0,0 +1,40 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +struct Settings; + +/** These settings represent fine tunes for internal details of Coordination storages + * and should not be changed by the user without a reason. + */ + +#define LIST_OF_COORDINATION_SETTINGS(M) \ + M(Milliseconds, session_timeout_ms, Coordination::DEFAULT_SESSION_TIMEOUT_MS, "Default client session timeout", 0) \ + M(Milliseconds, operation_timeout_ms, Coordination::DEFAULT_OPERATION_TIMEOUT_MS, "Default client operation timeout", 0) \ + M(Milliseconds, dead_session_check_period_ms, 500, "How often leader will check sessions to consider them dead and remove", 0) \ + 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, "Lower bound of election timer (avoid too often leader elections)", 0) \ + M(UInt64, reserved_log_items, 5000, "How many log items to store (don't remove during compaction)", 0) \ + M(UInt64, snapshot_distance, 5000, "How many log items we have to collect to write new snapshot", 0) \ + M(UInt64, max_stored_snapshots, 3, "How many snapshots we want to store", 0) \ + M(Bool, auto_forwarding, true, "Allow to forward write requests from followers to leader", 0) \ + M(Milliseconds, shutdown_timeout, 5000, "How many time we will until RAFT shutdown", 0) + +DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS) + + +struct CoordinationSettings : public BaseSettings +{ + void loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config); +}; + +using CoordinationSettingsPtr = std::shared_ptr; + +} From 51c221f993ce1cd7e6500defbeb05458aee2bd1e Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 9 Feb 2021 21:29:06 +0300 Subject: [PATCH 71/86] Fix outdated session kill --- .../NuKeeperStorageDispatcher.cpp | 8 ++- .../test_testkeeper_multinode/test.py | 69 ++++++++++++++----- 2 files changed, 59 insertions(+), 18 deletions(-) diff --git a/src/Coordination/NuKeeperStorageDispatcher.cpp b/src/Coordination/NuKeeperStorageDispatcher.cpp index 914985ee534..8ca5d3fff13 100644 --- a/src/Coordination/NuKeeperStorageDispatcher.cpp +++ b/src/Coordination/NuKeeperStorageDispatcher.cpp @@ -274,7 +274,13 @@ void NuKeeperStorageDispatcher::sessionCleanerTask() LOG_INFO(log, "Found dead session {}, will try to close it", dead_session); Coordination::ZooKeeperRequestPtr request = Coordination::ZooKeeperRequestFactory::instance().get(Coordination::OpNum::Close); request->xid = Coordination::CLOSE_XID; - putRequest(request, dead_session); + NuKeeperStorage::RequestForSession request_info; + request_info.request = request; + request_info.session_id = dead_session; + { + std::lock_guard lock(push_request_mutex); + requests_queue.push(std::move(request_info)); + } finishSession(dead_session); } } diff --git a/tests/integration/test_testkeeper_multinode/test.py b/tests/integration/test_testkeeper_multinode/test.py index 05879613ba6..51f60df7719 100644 --- a/tests/integration/test_testkeeper_multinode/test.py +++ b/tests/integration/test_testkeeper_multinode/test.py @@ -27,23 +27,8 @@ def started_cluster(): def smaller_exception(ex): return '\n'.join(str(ex).split('\n')[0:2]) -def test_simple_replicated_table(started_cluster): - - for i, node in enumerate([node1, node2, node3]): - node.query("CREATE TABLE t (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t', '{}') ORDER BY tuple()".format(i + 1)) - - node2.query("INSERT INTO t SELECT number FROM numbers(10)") - - node1.query("SYSTEM SYNC REPLICA t", timeout=10) - node3.query("SYSTEM SYNC REPLICA t", timeout=10) - - assert node1.query("SELECT COUNT() FROM t") == "10\n" - assert node2.query("SELECT COUNT() FROM t") == "10\n" - assert node3.query("SELECT COUNT() FROM t") == "10\n" - - -def get_fake_zk(nodename): - _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=30.0) +def get_fake_zk(nodename, timeout=30.0): + _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout) def reset_last_zxid_listener(state): print("Fake zk callback called for state", state) _fake_zk_instance.last_zxid = 0 @@ -146,6 +131,56 @@ def test_watch_on_follower(started_cluster): pass +def test_session_expiration(started_cluster): + try: + node1_zk = get_fake_zk("node1") + node2_zk = get_fake_zk("node2") + node3_zk = get_fake_zk("node3", timeout=3.0) + + node3_zk.create("/test_ephemeral_node", b"world", ephemeral=True) + + with PartitionManager() as pm: + pm.partition_instances(node3, node2) + pm.partition_instances(node3, node1) + node3_zk.stop() + node3_zk.close() + time.sleep(5) + + assert node1_zk.exists("/test_ephemeral_node") is None + assert node2_zk.exists("/test_ephemeral_node") is None + + finally: + try: + for zk_conn in [node1_zk, node2_zk, node3_zk]: + try: + zk_conn.stop() + zk_conn.close() + except: + pass + except: + pass + +def test_simple_replicated_table(started_cluster): + # something may be wrong after partition in other tests + # so create with retry + for i, node in enumerate([node1, node2, node3]): + for i in range(100): + try: + node.query("CREATE TABLE t (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t', '{}') ORDER BY tuple()".format(i + 1)) + break + except: + time.sleep(0.1) + + node2.query("INSERT INTO t SELECT number FROM numbers(10)") + + node1.query("SYSTEM SYNC REPLICA t", timeout=10) + node3.query("SYSTEM SYNC REPLICA t", timeout=10) + + assert node1.query("SELECT COUNT() FROM t") == "10\n" + assert node2.query("SELECT COUNT() FROM t") == "10\n" + assert node3.query("SELECT COUNT() FROM t") == "10\n" + + # in extremely rare case it can take more than 5 minutes in debug build with sanitizer @pytest.mark.timeout(600) def test_blocade_leader(started_cluster): From 7848f0202c6a1b076a3607c9fe2911a9b615d644 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 10 Feb 2021 00:02:34 +0300 Subject: [PATCH 72/86] One more test --- .../test_testkeeper_multinode/test.py | 32 +++++++++++++++++-- 1 file changed, 29 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_testkeeper_multinode/test.py b/tests/integration/test_testkeeper_multinode/test.py index 51f60df7719..70968842f4d 100644 --- a/tests/integration/test_testkeeper_multinode/test.py +++ b/tests/integration/test_testkeeper_multinode/test.py @@ -8,9 +8,9 @@ from multiprocessing.dummy import Pool from helpers.network import PartitionManager cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', main_configs=['configs/enable_test_keeper1.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml']) -node2 = cluster.add_instance('node2', main_configs=['configs/enable_test_keeper2.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml']) -node3 = cluster.add_instance('node3', main_configs=['configs/enable_test_keeper3.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml']) +node1 = cluster.add_instance('node1', main_configs=['configs/enable_test_keeper1.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True) +node2 = cluster.add_instance('node2', main_configs=['configs/enable_test_keeper2.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True) +node3 = cluster.add_instance('node3', main_configs=['configs/enable_test_keeper3.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True) from kazoo.client import KazooClient @@ -160,6 +160,32 @@ def test_session_expiration(started_cluster): except: pass + +def test_follower_restart(started_cluster): + try: + node1_zk = get_fake_zk("node1") + + node1_zk.create("/test_restart_node", b"hello") + + node3.restart_clickhouse(kill=True) + + node3_zk = get_fake_zk("node3") + + # got data from log + assert node3_zk.get("/test_restart_node")[0] == b"hello" + + finally: + try: + for zk_conn in [node1_zk, node3_zk]: + try: + zk_conn.stop() + zk_conn.close() + except: + pass + except: + pass + + def test_simple_replicated_table(started_cluster): # something may be wrong after partition in other tests # so create with retry From c95140d906401c8c133838c89369ef79d5ec8745 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 10 Feb 2021 12:28:53 +0300 Subject: [PATCH 73/86] Better startup and non-verbose logging by default --- src/Coordination/CoordinationSettings.h | 5 +- src/Coordination/LoggerWrapper.h | 5 +- src/Coordination/NuKeeperServer.cpp | 58 +++++++++---------- src/Coordination/NuKeeperServer.h | 12 +++- .../NuKeeperStorageDispatcher.cpp | 17 ++++-- .../configs/enable_test_keeper.xml | 1 + .../configs/enable_test_keeper1.xml | 1 + .../configs/enable_test_keeper2.xml | 1 + .../configs/enable_test_keeper3.xml | 1 + .../test_testkeeper_multinode/test.py | 5 +- 10 files changed, 63 insertions(+), 43 deletions(-) diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 374d432f2db..441e1a5936f 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -25,7 +26,9 @@ struct Settings; M(UInt64, snapshot_distance, 5000, "How many log items we have to collect to write new snapshot", 0) \ M(UInt64, max_stored_snapshots, 3, "How many snapshots we want to store", 0) \ M(Bool, auto_forwarding, true, "Allow to forward write requests from followers to leader", 0) \ - M(Milliseconds, shutdown_timeout, 5000, "How many time we will until RAFT shutdown", 0) + M(Milliseconds, shutdown_timeout, 5000, "How many time we will until RAFT shutdown", 0) \ + M(Milliseconds, startup_timeout, 30000, "How many time we will until RAFT to start", 0) \ + M(LogsLevel, raft_logs_level, LogsLevel::information, "Log internal RAFT logs into main server log level. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'fatal', 'none'", 0) DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS) diff --git a/src/Coordination/LoggerWrapper.h b/src/Coordination/LoggerWrapper.h index c8da2372a91..755b72c06cc 100644 --- a/src/Coordination/LoggerWrapper.h +++ b/src/Coordination/LoggerWrapper.h @@ -2,6 +2,7 @@ #include // Y_IGNORE #include +#include namespace DB { @@ -9,9 +10,9 @@ namespace DB class LoggerWrapper : public nuraft::logger { public: - LoggerWrapper(const std::string & name) + LoggerWrapper(const std::string & name, LogsLevel level_) : log(&Poco::Logger::get(name)) - , level(6) + , level(static_cast(level_)) { log->setLevel(level); } diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index 40508b08761..314a1412313 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -47,7 +47,7 @@ void NuKeeperServer::addServer(int server_id_, const std::string & server_uri_, } -void NuKeeperServer::startup() +void NuKeeperServer::startup(bool should_build_quorum) { nuraft::raft_params params; params.heart_beat_interval_ = coordination_settings->heart_beat_interval_ms.totalMilliseconds(); @@ -62,25 +62,19 @@ void NuKeeperServer::startup() params.return_method_ = nuraft::raft_params::blocking; nuraft::asio_service::options asio_opts{}; + nuraft::raft_server::init_options init_options; + init_options.skip_initial_election_timeout_ = !should_build_quorum; + init_options.raft_callback_ = [this] (nuraft::cb_func::Type type, nuraft::cb_func::Param * param) + { + return callbackFunc(type, param); + }; raft_instance = launcher.init( - state_machine, state_manager, nuraft::cs_new("RaftInstance"), port, - asio_opts, params); + state_machine, state_manager, nuraft::cs_new("RaftInstance", coordination_settings->raft_logs_level), port, + asio_opts, params, init_options); if (!raft_instance) throw Exception(ErrorCodes::RAFT_ERROR, "Cannot allocate RAFT instance"); - - /// FIXME - static constexpr auto MAX_RETRY = 100; - for (size_t i = 0; i < MAX_RETRY; ++i) - { - if (raft_instance->is_initialized()) - return; - - std::this_thread::sleep_for(std::chrono::milliseconds(100)); - } - - throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Cannot start RAFT server within startup timeout"); } void NuKeeperServer::shutdown() @@ -177,10 +171,22 @@ bool NuKeeperServer::isLeaderAlive() const return raft_instance->is_leader_alive(); } + +nuraft::cb_func::ReturnCode NuKeeperServer::callbackFunc(nuraft::cb_func::Type type, nuraft::cb_func::Param * /* param */) +{ + if (type == nuraft::cb_func::Type::BecomeFresh || type == nuraft::cb_func::Type::BecomeLeader) + { + std::unique_lock lock(initialized_mutex); + initialized_flag = true; + initialized_cv.notify_all(); + } + return nuraft::cb_func::ReturnCode::Ok; +} + bool NuKeeperServer::waitForServer(int32_t id) const { /// FIXME - for (size_t i = 0; i < 50; ++i) + for (size_t i = 0; i < 30; ++i) { if (raft_instance->get_srv_config(id) != nullptr) return true; @@ -192,22 +198,12 @@ bool NuKeeperServer::waitForServer(int32_t id) const return false; } -bool NuKeeperServer::waitForServers(const std::vector & ids) const +void NuKeeperServer::waitInit() { - for (int32_t id : ids) - if (!waitForServer(id)) - return false; - return true; -} - -void NuKeeperServer::waitForCatchUp() const -{ - /// FIXME - while (raft_instance->is_catching_up() || raft_instance->is_receiving_snapshot() || raft_instance->is_leader()) - { - LOG_DEBUG(&Poco::Logger::get("NuRaftInit"), "Waiting current RAFT instance to catch up"); - std::this_thread::sleep_for(std::chrono::milliseconds(100)); - } + std::unique_lock lock(initialized_mutex); + int64_t timeout = coordination_settings->startup_timeout.totalMilliseconds(); + if (!initialized_cv.wait_for(lock, std::chrono::milliseconds(timeout), [&] { return initialized_flag; })) + throw Exception(ErrorCodes::RAFT_ERROR, "Failed to wait RAFT initialization"); } std::unordered_set NuKeeperServer::getDeadSessions() diff --git a/src/Coordination/NuKeeperServer.h b/src/Coordination/NuKeeperServer.h index bb5870fe89a..ce6dd2f0fbb 100644 --- a/src/Coordination/NuKeeperServer.h +++ b/src/Coordination/NuKeeperServer.h @@ -36,13 +36,19 @@ private: ResponsesQueue & responses_queue; + std::mutex initialized_mutex; + bool initialized_flag = false; + std::condition_variable initialized_cv; + + nuraft::cb_func::ReturnCode callbackFunc(nuraft::cb_func::Type type, nuraft::cb_func::Param * param); + public: NuKeeperServer( int server_id_, const std::string & hostname_, int port_, const CoordinationSettingsPtr & coordination_settings_, ResponsesQueue & responses_queue_); - void startup(); + void startup(bool should_build_quorum); void putRequest(const NuKeeperStorage::RequestForSession & request); @@ -57,8 +63,8 @@ public: bool isLeaderAlive() const; bool waitForServer(int32_t server_id) const; - bool waitForServers(const std::vector & ids) const; - void waitForCatchUp() const; + + void waitInit(); void shutdown(); }; diff --git a/src/Coordination/NuKeeperStorageDispatcher.cpp b/src/Coordination/NuKeeperStorageDispatcher.cpp index 8ca5d3fff13..300604e0f6e 100644 --- a/src/Coordination/NuKeeperStorageDispatcher.cpp +++ b/src/Coordination/NuKeeperStorageDispatcher.cpp @@ -167,9 +167,12 @@ void NuKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigurati server = std::make_unique(myid, myhostname, myport, coordination_settings, responses_queue); try { - server->startup(); - if (shouldBuildQuorum(myid, my_priority, my_can_become_leader, server_configs)) + bool should_build_quorum = shouldBuildQuorum(myid, my_priority, my_can_become_leader, server_configs); + server->startup(should_build_quorum); + if (should_build_quorum) { + + server->waitInit(); for (const auto & [id, hostname, port, can_become_leader, priority] : server_configs) { LOG_DEBUG(log, "Adding server with id {} ({}:{})", id, hostname, port); @@ -181,12 +184,15 @@ void NuKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigurati LOG_DEBUG(log, "Server with id {} ({}:{}) added to cluster", id, hostname, port); } + + if (server_configs.size() > 1) + LOG_DEBUG(log, "All servers were added to quorum"); } else { - while (!server->waitForServers(ids)) - LOG_DEBUG(log, "Waiting for {} servers to build cluster", ids.size()); - server->waitForCatchUp(); + LOG_DEBUG(log, "Waiting as follower"); + server->waitInit(); + LOG_DEBUG(log, "Follower became fresh"); } } catch (...) @@ -282,6 +288,7 @@ void NuKeeperStorageDispatcher::sessionCleanerTask() requests_queue.push(std::move(request_info)); } finishSession(dead_session); + LOG_INFO(log, "Dead session close request pushed"); } } } diff --git a/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml b/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml index 00a593051f9..1a441909998 100644 --- a/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml +++ b/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml @@ -6,6 +6,7 @@ 5000 10000 + trace diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml index 75065bb2a7a..3ae44f926d0 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml @@ -6,6 +6,7 @@ 5000 10000 + trace diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml index 18937dd4910..7674c755511 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml @@ -6,6 +6,7 @@ 5000 10000 + trace diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml index 5330367cd89..59dde3bc1b1 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml @@ -6,6 +6,7 @@ 5000 10000 + trace diff --git a/tests/integration/test_testkeeper_multinode/test.py b/tests/integration/test_testkeeper_multinode/test.py index 70968842f4d..e2b0537d5ec 100644 --- a/tests/integration/test_testkeeper_multinode/test.py +++ b/tests/integration/test_testkeeper_multinode/test.py @@ -144,7 +144,10 @@ def test_session_expiration(started_cluster): pm.partition_instances(node3, node1) node3_zk.stop() node3_zk.close() - time.sleep(5) + for _ in range(100): + if node1_zk.exists("/test_ephemeral_node") is None and node2_zk.exists("/test_ephemeral_node") is None: + break + time.sleep(0.1) assert node1_zk.exists("/test_ephemeral_node") is None assert node2_zk.exists("/test_ephemeral_node") is None From 63080a0b5e6d09b1e9336ccb8023e6e6f5d7569b Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 10 Feb 2021 12:31:01 +0300 Subject: [PATCH 74/86] Redundant space --- cmake/find/nuraft.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/find/nuraft.cmake b/cmake/find/nuraft.cmake index bcc656de129..7fa5251946e 100644 --- a/cmake/find/nuraft.cmake +++ b/cmake/find/nuraft.cmake @@ -1,6 +1,6 @@ option(ENABLE_NURAFT "Enable NuRaft" ${ENABLE_LIBRARIES}) - if (NOT ENABLE_NURAFT) +if (NOT ENABLE_NURAFT) return() endif() From 57d8d81d5946ff8f70c07174aae5a9ef99585099 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 10 Feb 2021 13:02:09 +0300 Subject: [PATCH 75/86] Fix style --- src/Coordination/NuKeeperServer.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index 314a1412313..0d4bdcc60fe 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -15,7 +15,6 @@ namespace DB namespace ErrorCodes { - extern const int TIMEOUT_EXCEEDED; extern const int RAFT_ERROR; } From 0d179e021bf4681f8d6e15d927ac2a296a89d6c1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 10 Feb 2021 13:15:42 +0300 Subject: [PATCH 76/86] Add sync cmd --- src/Common/ZooKeeper/ZooKeeperConstants.cpp | 3 +++ src/Common/ZooKeeper/ZooKeeperConstants.h | 1 + 2 files changed, 4 insertions(+) diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.cpp b/src/Common/ZooKeeper/ZooKeeperConstants.cpp index b4cb9feb518..295094b336b 100644 --- a/src/Common/ZooKeeper/ZooKeeperConstants.cpp +++ b/src/Common/ZooKeeper/ZooKeeperConstants.cpp @@ -15,6 +15,7 @@ static const std::unordered_set VALID_OPERATIONS = static_cast(OpNum::Get), static_cast(OpNum::Set), static_cast(OpNum::SimpleList), + static_cast(OpNum::Sync), static_cast(OpNum::Heartbeat), static_cast(OpNum::List), static_cast(OpNum::Check), @@ -48,6 +49,8 @@ std::string toString(OpNum op_num) return "Check"; case OpNum::Multi: return "Multi"; + case OpNum::Sync: + return "Sync"; case OpNum::Heartbeat: return "Heartbeat"; case OpNum::Auth: diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.h b/src/Common/ZooKeeper/ZooKeeperConstants.h index 8a20330a2d7..81ca6c6a460 100644 --- a/src/Common/ZooKeeper/ZooKeeperConstants.h +++ b/src/Common/ZooKeeper/ZooKeeperConstants.h @@ -24,6 +24,7 @@ enum class OpNum : int32_t Get = 4, Set = 5, SimpleList = 8, + Sync = 9, Heartbeat = 11, List = 12, Check = 13, From e07bdad5c0919757e5376d16b05efaaf214a8b28 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 10 Feb 2021 14:44:21 +0300 Subject: [PATCH 77/86] Fix test build --- src/Coordination/tests/gtest_for_build.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index 82affd38062..c6f29831618 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -86,7 +86,7 @@ struct SimpliestRaftServer params.return_method_ = nuraft::raft_params::blocking; raft_instance = launcher.init( - state_machine, state_manager, nuraft::cs_new("ToyRaftLogger"), port, + state_machine, state_manager, nuraft::cs_new("ToyRaftLogger", DB::LogsLevel::trace), port, nuraft::asio_service::options{}, params); if (!raft_instance) From b4d53886a399b1728517c10838f3a2f5b3b3b35b Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 10 Feb 2021 16:01:05 +0300 Subject: [PATCH 78/86] Add sync request/response --- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 22 ++++++++++++++++++++++ src/Common/ZooKeeper/ZooKeeperCommon.h | 19 +++++++++++++++++++ src/Coordination/NuKeeperStorage.cpp | 12 ++++++++++++ 3 files changed, 53 insertions(+) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index 2d32cd75624..56f9de31ec8 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -37,6 +37,26 @@ void ZooKeeperRequest::write(WriteBuffer & out) const out.next(); } +void ZooKeeperSyncRequest::writeImpl(WriteBuffer & out) const +{ + Coordination::write(path, out); +} + +void ZooKeeperSyncRequest::readImpl(ReadBuffer & in) +{ + Coordination::read(path, in); +} + +void ZooKeeperSyncResponse::readImpl(ReadBuffer & in) +{ + Coordination::read(path, in); +} + +void ZooKeeperSyncResponse::writeImpl(WriteBuffer & out) const +{ + Coordination::write(path, out); +} + void ZooKeeperWatchResponse::readImpl(ReadBuffer & in) { Coordination::read(type, in); @@ -423,6 +443,7 @@ void ZooKeeperMultiResponse::writeImpl(WriteBuffer & out) const } ZooKeeperResponsePtr ZooKeeperHeartbeatRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperSyncRequest::makeResponse() const { return std::make_shared(); } ZooKeeperResponsePtr ZooKeeperAuthRequest::makeResponse() const { return std::make_shared(); } ZooKeeperResponsePtr ZooKeeperCreateRequest::makeResponse() const { return std::make_shared(); } ZooKeeperResponsePtr ZooKeeperRemoveRequest::makeResponse() const { return std::make_shared(); } @@ -478,6 +499,7 @@ void registerZooKeeperRequest(ZooKeeperRequestFactory & factory) ZooKeeperRequestFactory::ZooKeeperRequestFactory() { registerZooKeeperRequest(*this); + registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index 8bc1cde8cd7..92b1e7c9858 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -75,6 +75,25 @@ struct ZooKeeperHeartbeatRequest final : ZooKeeperRequest bool isReadRequest() const override { return false; } }; +struct ZooKeeperSyncRequest final : ZooKeeperRequest +{ + String path; + String getPath() const override { return path; } + OpNum getOpNum() const override { return OpNum::Sync; } + void writeImpl(WriteBuffer & out) const override; + void readImpl(ReadBuffer & in) override; + ZooKeeperResponsePtr makeResponse() const override; + bool isReadRequest() const override { return false; } +}; + +struct ZooKeeperSyncResponse final : ZooKeeperResponse +{ + String path; + void readImpl(ReadBuffer & in) override; + void writeImpl(WriteBuffer & out) const override; + OpNum getOpNum() const override { return OpNum::Sync; } +}; + struct ZooKeeperHeartbeatResponse final : ZooKeeperResponse { void readImpl(ReadBuffer &) override {} diff --git a/src/Coordination/NuKeeperStorage.cpp b/src/Coordination/NuKeeperStorage.cpp index a86b7432cbf..631f975cddc 100644 --- a/src/Coordination/NuKeeperStorage.cpp +++ b/src/Coordination/NuKeeperStorage.cpp @@ -97,6 +97,17 @@ struct NuKeeperStorageHeartbeatRequest final : public NuKeeperStorageRequest } }; +struct NuKeeperStorageSyncRequest final : public NuKeeperStorageRequest +{ + using NuKeeperStorageRequest::NuKeeperStorageRequest; + std::pair process(NuKeeperStorage::Container & /* container */, NuKeeperStorage::Ephemerals & /* ephemerals */, int64_t /* zxid */, int64_t /* session_id */) const override + { + auto response = zk_request->makeResponse(); + dynamic_cast(response.get())->path = dynamic_cast(zk_request.get())->path; + return {response, {}}; + } +}; + struct NuKeeperStorageCreateRequest final : public NuKeeperStorageRequest { using NuKeeperStorageRequest::NuKeeperStorageRequest; @@ -575,6 +586,7 @@ void registerNuKeeperRequestWrapper(NuKeeperWrapperFactory & factory) NuKeeperWrapperFactory::NuKeeperWrapperFactory() { registerNuKeeperRequestWrapper(*this); + registerNuKeeperRequestWrapper(*this); //registerNuKeeperRequestWrapper(*this); registerNuKeeperRequestWrapper(*this); registerNuKeeperRequestWrapper(*this); From b49b7f859d0c7edeee539286cdc4051226971e78 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 11 Feb 2021 12:17:57 +0300 Subject: [PATCH 79/86] Simplify startup with fixed config --- src/Coordination/InMemoryStateManager.cpp | 38 +++++++- src/Coordination/InMemoryStateManager.h | 14 ++- src/Coordination/NuKeeperServer.cpp | 39 +------- src/Coordination/NuKeeperServer.h | 15 +--- .../NuKeeperStorageDispatcher.cpp | 90 ++----------------- .../configs/enable_test_keeper1.xml | 6 +- .../configs/enable_test_keeper2.xml | 6 +- .../configs/enable_test_keeper3.xml | 6 +- 8 files changed, 67 insertions(+), 147 deletions(-) diff --git a/src/Coordination/InMemoryStateManager.cpp b/src/Coordination/InMemoryStateManager.cpp index 15a1f7aa622..d90c7e46f0d 100644 --- a/src/Coordination/InMemoryStateManager.cpp +++ b/src/Coordination/InMemoryStateManager.cpp @@ -1,16 +1,46 @@ #include +#include namespace DB { -InMemoryStateManager::InMemoryStateManager(int my_server_id_, const std::string & endpoint_) +namespace ErrorCodes +{ + extern const int RAFT_ERROR; +} + +InMemoryStateManager::InMemoryStateManager( + int my_server_id_, + const std::string & config_prefix, + const Poco::Util::AbstractConfiguration & config) : my_server_id(my_server_id_) - , endpoint(endpoint_) , log_store(nuraft::cs_new()) - , server_config(nuraft::cs_new(my_server_id, endpoint)) , cluster_config(nuraft::cs_new()) { - cluster_config->get_servers().push_back(server_config); + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(config_prefix, keys); + + for (const auto & server_key : keys) + { + std::string full_prefix = config_prefix + "." + server_key; + int server_id = config.getInt(full_prefix + ".id"); + std::string hostname = config.getString(full_prefix + ".hostname"); + int port = config.getInt(full_prefix + ".port"); + bool can_become_leader = config.getBool(full_prefix + ".can_become_leader", true); + int32_t priority = config.getInt(full_prefix + ".priority", 1); + + auto endpoint = hostname + ":" + std::to_string(port); + auto peer_config = nuraft::cs_new(server_id, 0, endpoint, "", !can_become_leader, priority); + if (server_id == my_server_id) + { + my_server_config = peer_config; + my_port = port; + } + + cluster_config->get_servers().push_back(peer_config); + } + if (!my_server_config) + throw Exception(ErrorCodes::RAFT_ERROR, "Our server id {} not found in raft_configuration section"); } void InMemoryStateManager::save_config(const nuraft::cluster_config & config) diff --git a/src/Coordination/InMemoryStateManager.h b/src/Coordination/InMemoryStateManager.h index 7446073c9c9..b48b5188f36 100644 --- a/src/Coordination/InMemoryStateManager.h +++ b/src/Coordination/InMemoryStateManager.h @@ -4,6 +4,7 @@ #include #include #include // Y_IGNORE +#include namespace DB { @@ -11,7 +12,10 @@ namespace DB class InMemoryStateManager : public nuraft::state_mgr { public: - InMemoryStateManager(int server_id_, const std::string & endpoint_); + InMemoryStateManager( + int server_id_, + const std::string & config_prefix, + const Poco::Util::AbstractConfiguration & config); nuraft::ptr load_config() override { return cluster_config; } @@ -25,15 +29,17 @@ public: Int32 server_id() override { return my_server_id; } - nuraft::ptr get_srv_config() const { return server_config; } + nuraft::ptr get_srv_config() const { return my_server_config; } void system_exit(const int /* exit_code */) override {} + int getPort() const { return my_port; } + private: int my_server_id; - std::string endpoint; + int my_port; nuraft::ptr log_store; - nuraft::ptr server_config; + nuraft::ptr my_server_config; nuraft::ptr cluster_config; nuraft::ptr server_state; }; diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index 0d4bdcc60fe..c7deebfdb96 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -19,33 +19,18 @@ namespace ErrorCodes } NuKeeperServer::NuKeeperServer( - int server_id_, const std::string & hostname_, int port_, + int server_id_, const CoordinationSettingsPtr & coordination_settings_, + const Poco::Util::AbstractConfiguration & config, ResponsesQueue & responses_queue_) : server_id(server_id_) - , hostname(hostname_) - , port(port_) - , endpoint(hostname + ":" + std::to_string(port)) , coordination_settings(coordination_settings_) , state_machine(nuraft::cs_new(responses_queue_, coordination_settings)) - , state_manager(nuraft::cs_new(server_id, endpoint)) + , state_manager(nuraft::cs_new(server_id, "test_keeper_server.raft_configuration", config)) , responses_queue(responses_queue_) { } -void NuKeeperServer::addServer(int server_id_, const std::string & server_uri_, bool can_become_leader_, int32_t priority) -{ - nuraft::srv_config config(server_id_, 0, server_uri_, "", /* learner = */ !can_become_leader_, priority); - auto ret1 = raft_instance->add_srv(config); - auto code = ret1->get_result_code(); - if (code == nuraft::cmd_result_code::TIMEOUT - || code == nuraft::cmd_result_code::BAD_REQUEST - || code == nuraft::cmd_result_code::NOT_LEADER - || code == nuraft::cmd_result_code::FAILED) - throw Exception(ErrorCodes::RAFT_ERROR, "Cannot add server to RAFT quorum with code {}, message '{}'", ret1->get_result_code(), ret1->get_result_str()); -} - - void NuKeeperServer::startup(bool should_build_quorum) { nuraft::raft_params params; @@ -69,7 +54,7 @@ void NuKeeperServer::startup(bool should_build_quorum) }; raft_instance = launcher.init( - state_machine, state_manager, nuraft::cs_new("RaftInstance", coordination_settings->raft_logs_level), port, + state_machine, state_manager, nuraft::cs_new("RaftInstance", coordination_settings->raft_logs_level), state_manager->getPort(), asio_opts, params, init_options); if (!raft_instance) @@ -170,7 +155,6 @@ bool NuKeeperServer::isLeaderAlive() const return raft_instance->is_leader_alive(); } - nuraft::cb_func::ReturnCode NuKeeperServer::callbackFunc(nuraft::cb_func::Type type, nuraft::cb_func::Param * /* param */) { if (type == nuraft::cb_func::Type::BecomeFresh || type == nuraft::cb_func::Type::BecomeLeader) @@ -182,21 +166,6 @@ nuraft::cb_func::ReturnCode NuKeeperServer::callbackFunc(nuraft::cb_func::Type t return nuraft::cb_func::ReturnCode::Ok; } -bool NuKeeperServer::waitForServer(int32_t id) const -{ - /// FIXME - for (size_t i = 0; i < 30; ++i) - { - if (raft_instance->get_srv_config(id) != nullptr) - return true; - LOG_DEBUG(&Poco::Logger::get("NuRaftInit"), "Waiting for server {} to join the cluster", id); - std::this_thread::sleep_for(std::chrono::milliseconds(100)); - } - - LOG_DEBUG(&Poco::Logger::get("NuRaftInit"), "Cannot wait for server {}", id); - return false; -} - void NuKeeperServer::waitInit() { std::unique_lock lock(initialized_mutex); diff --git a/src/Coordination/NuKeeperServer.h b/src/Coordination/NuKeeperServer.h index ce6dd2f0fbb..a37d4d9127a 100644 --- a/src/Coordination/NuKeeperServer.h +++ b/src/Coordination/NuKeeperServer.h @@ -16,17 +16,11 @@ class NuKeeperServer private: int server_id; - std::string hostname; - - int port; - - std::string endpoint; - CoordinationSettingsPtr coordination_settings; nuraft::ptr state_machine; - nuraft::ptr state_manager; + nuraft::ptr state_manager; nuraft::raft_launcher launcher; @@ -44,8 +38,9 @@ private: public: NuKeeperServer( - int server_id_, const std::string & hostname_, int port_, + int server_id_, const CoordinationSettingsPtr & coordination_settings_, + const Poco::Util::AbstractConfiguration & config, ResponsesQueue & responses_queue_); void startup(bool should_build_quorum); @@ -56,14 +51,10 @@ public: std::unordered_set getDeadSessions(); - void addServer(int server_id_, const std::string & server_uri, bool can_become_leader_, int32_t priority); - bool isLeader() const; bool isLeaderAlive() const; - bool waitForServer(int32_t server_id) const; - void waitInit(); void shutdown(); diff --git a/src/Coordination/NuKeeperStorageDispatcher.cpp b/src/Coordination/NuKeeperStorageDispatcher.cpp index 300604e0f6e..9dc420830ad 100644 --- a/src/Coordination/NuKeeperStorageDispatcher.cpp +++ b/src/Coordination/NuKeeperStorageDispatcher.cpp @@ -103,97 +103,21 @@ bool NuKeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperRequestP return true; } -namespace -{ - bool shouldBuildQuorum(int32_t myid, int32_t my_priority, bool my_can_become_leader, const std::vector> & server_configs) - { - if (!my_can_become_leader) - return false; - - int32_t minid = myid; - bool has_equal_priority = false; - for (const auto & [id, hostname, port, can_become_leader, priority] : server_configs) - { - if (my_priority < priority) - return false; - else if (my_priority == priority) - has_equal_priority = true; - minid = std::min(minid, id); - } - - if (has_equal_priority) - return minid == myid; - else - return true; - } -} - void NuKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfiguration & config) { LOG_DEBUG(log, "Initializing storage dispatcher"); int myid = config.getInt("test_keeper_server.server_id"); - std::string myhostname; - int myport; - int32_t my_priority = 1; + coordination_settings->loadFromConfig("test_keeper_server.coordination_settings", config); - Poco::Util::AbstractConfiguration::Keys keys; - config.keys("test_keeper_server.raft_configuration", keys); - bool my_can_become_leader = true; - - std::vector> server_configs; - std::vector ids; - for (const auto & server_key : keys) - { - int server_id = config.getInt("test_keeper_server.raft_configuration." + server_key + ".id"); - std::string hostname = config.getString("test_keeper_server.raft_configuration." + server_key + ".hostname"); - int port = config.getInt("test_keeper_server.raft_configuration." + server_key + ".port"); - bool can_become_leader = config.getBool("test_keeper_server.raft_configuration." + server_key + ".can_become_leader", true); - int32_t priority = config.getInt("test_keeper_server.raft_configuration." + server_key + ".priority", 1); - if (server_id == myid) - { - myhostname = hostname; - myport = port; - my_can_become_leader = can_become_leader; - my_priority = priority; - } - else - { - server_configs.emplace_back(server_id, hostname, port, can_become_leader, priority); - } - ids.push_back(server_id); - } - - server = std::make_unique(myid, myhostname, myport, coordination_settings, responses_queue); + server = std::make_unique(myid, coordination_settings, config, responses_queue); try { - bool should_build_quorum = shouldBuildQuorum(myid, my_priority, my_can_become_leader, server_configs); - server->startup(should_build_quorum); - if (should_build_quorum) - { - - server->waitInit(); - for (const auto & [id, hostname, port, can_become_leader, priority] : server_configs) - { - LOG_DEBUG(log, "Adding server with id {} ({}:{})", id, hostname, port); - do - { - server->addServer(id, hostname + ":" + std::to_string(port), can_become_leader, priority); - } - while (!server->waitForServer(id)); - - LOG_DEBUG(log, "Server with id {} ({}:{}) added to cluster", id, hostname, port); - } - - if (server_configs.size() > 1) - LOG_DEBUG(log, "All servers were added to quorum"); - } - else - { - LOG_DEBUG(log, "Waiting as follower"); - server->waitInit(); - LOG_DEBUG(log, "Follower became fresh"); - } + LOG_DEBUG(log, "Waiting server to initialize"); + server->startup(true); + LOG_DEBUG(log, "Server intialized, waiting for quorum"); + server->waitInit(); + LOG_DEBUG(log, "Quorum initialized"); } catch (...) { diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml index 3ae44f926d0..6ff7b1f2b79 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml @@ -15,21 +15,21 @@ node1 44444 true - 3 + 100 2 node2 44444 true - 2 + 20 3 node3 44444 true - 1 + 10 diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml index 7674c755511..65956104f2b 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml @@ -15,21 +15,21 @@ node1 44444 true - 3 + 100 2 node2 44444 true - 2 + 20 3 node3 44444 true - 1 + 10 diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml index 59dde3bc1b1..d2279ef00a4 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml @@ -15,21 +15,21 @@ node1 44444 true - 3 + 100 2 node2 44444 true - 2 + 20 3 node3 44444 true - 1 + 10 From 74630acff59879b76cf682d0957151d7cae75044 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 11 Feb 2021 12:49:49 +0300 Subject: [PATCH 80/86] More debug in test --- .../NuKeeperStorageDispatcher.cpp | 1 + .../configs/enable_test_keeper1.xml | 6 +-- .../configs/enable_test_keeper2.xml | 6 +-- .../configs/enable_test_keeper3.xml | 6 +-- .../test_testkeeper_multinode/test.py | 37 ++++++++++++++----- 5 files changed, 37 insertions(+), 19 deletions(-) diff --git a/src/Coordination/NuKeeperStorageDispatcher.cpp b/src/Coordination/NuKeeperStorageDispatcher.cpp index 9dc420830ad..76db01eb70f 100644 --- a/src/Coordination/NuKeeperStorageDispatcher.cpp +++ b/src/Coordination/NuKeeperStorageDispatcher.cpp @@ -116,6 +116,7 @@ void NuKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigurati LOG_DEBUG(log, "Waiting server to initialize"); server->startup(true); LOG_DEBUG(log, "Server intialized, waiting for quorum"); + server->waitInit(); LOG_DEBUG(log, "Quorum initialized"); } diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml index 6ff7b1f2b79..3ae44f926d0 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml @@ -15,21 +15,21 @@ node1 44444 true - 100 + 3 2 node2 44444 true - 20 + 2 3 node3 44444 true - 10 + 1 diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml index 65956104f2b..7674c755511 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml @@ -15,21 +15,21 @@ node1 44444 true - 100 + 3 2 node2 44444 true - 20 + 2 3 node3 44444 true - 10 + 1 diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml index d2279ef00a4..59dde3bc1b1 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml @@ -15,21 +15,21 @@ node1 44444 true - 100 + 3 2 node2 44444 true - 20 + 2 3 node3 44444 true - 10 + 1 diff --git a/tests/integration/test_testkeeper_multinode/test.py b/tests/integration/test_testkeeper_multinode/test.py index e2b0537d5ec..7063c42f31a 100644 --- a/tests/integration/test_testkeeper_multinode/test.py +++ b/tests/integration/test_testkeeper_multinode/test.py @@ -307,6 +307,19 @@ def test_blocade_leader(started_cluster): assert node3.query("SELECT COUNT() FROM t1") == "310\n" +def dump_zk(node, zk_path, replica_path): + print(node.query("SELECT * FROM system.replication_queue FORMAT Vertical")) + print("Replicas") + print(node.query("SELECT * FROM system.replicas FORMAT Vertical")) + print("Replica 2 info") + print(node.query("SELECT * FROM system.zookeeper WHERE path = '{}' FORMAT Vertical".format(zk_path))) + print("Queue") + print(node.query("SELECT * FROM system.zookeeper WHERE path = '{}/queue' FORMAT Vertical".format(replica_path))) + print("Log") + print(node.query("SELECT * FROM system.zookeeper WHERE path = '{}/log' FORMAT Vertical".format(zk_path))) + print("Parts") + print(node.query("SELECT name FROM system.zookeeper WHERE path = '{}/parts' FORMAT Vertical".format(replica_path))) + # in extremely rare case it can take more than 5 minutes in debug build with sanitizer @pytest.mark.timeout(600) def test_blocade_leader_twice(started_cluster): @@ -339,6 +352,8 @@ def test_blocade_leader_twice(started_cluster): print("Got exception node2", smaller_exception(ex)) time.sleep(0.5) else: + for num, node in enumerate([node1, node2, node3]): + dump_zk(node, '/clickhouse/t2', '/clickhouse/t2/replicas/{}'.format(num + 1)) assert False, "Cannot reconnect for node2" for i in range(100): @@ -354,6 +369,8 @@ def test_blocade_leader_twice(started_cluster): print("Got exception node3", smaller_exception(ex)) time.sleep(0.5) else: + for num, node in enumerate([node1, node2, node3]): + dump_zk(node, '/clickhouse/t2', '/clickhouse/t2/replicas/{}'.format(num + 1)) assert False, "Cannot reconnect for node3" @@ -389,6 +406,8 @@ def test_blocade_leader_twice(started_cluster): print("Got exception node{}".format(n + 1), smaller_exception(ex)) time.sleep(0.5) else: + for num, node in enumerate([node1, node2, node3]): + dump_zk(node, '/clickhouse/t2', '/clickhouse/t2/replicas/{}'.format(num + 1)) assert False, "Cannot reconnect for node{}".format(n + 1) for n, node in enumerate([node1, node2, node3]): @@ -400,12 +419,14 @@ def test_blocade_leader_twice(started_cluster): print("Got exception node{}".format(n + 1), smaller_exception(ex)) time.sleep(0.5) else: + for num, node in enumerate([node1, node2, node3]): + dump_zk(node, '/clickhouse/t2', '/clickhouse/t2/replicas/{}'.format(num + 1)) assert False, "Cannot reconnect for node{}".format(n + 1) for n, node in enumerate([node1, node2, node3]): for i in range(100): try: - node.query("SYSTEM RESTART REPLICA t2", timeout=10) + node.query("SYSTEM RESTART REPLICA t2") node.query("SYSTEM SYNC REPLICA t2", timeout=10) break except Exception as ex: @@ -417,18 +438,14 @@ def test_blocade_leader_twice(started_cluster): print("Got exception node{}".format(n + 1), smaller_exception(ex)) time.sleep(0.5) else: + for num, node in enumerate([node1, node2, node3]): + dump_zk(node, '/clickhouse/t2', '/clickhouse/t2/replicas/{}'.format(num + 1)) assert False, "Cannot reconnect for node{}".format(n + 1) assert node1.query("SELECT COUNT() FROM t2") == "510\n" if node2.query("SELECT COUNT() FROM t2") != "510\n": - print(node2.query("SELECT * FROM system.replication_queue FORMAT Vertical")) - print("Replicas") - print(node2.query("SELECT * FROM system.replicas FORMAT Vertical")) - print("Replica 2 info") - print(node2.query("SELECT * FROM system.zookeeper WHERE path = '/clickhouse/t2/replicas/2' FORMAT Vertical")) - print("Queue") - print(node2.query("SELECT * FROM system.zookeeper WHERE path = '/clickhouse/t2/replicas/2/queue' FORMAT Vertical")) - print("Log") - print(node2.query("SELECT * FROM system.zookeeper WHERE path = '/clickhouse/t2/log' FORMAT Vertical")) + for num, node in enumerate([node1, node2, node3]): + dump_zk(node, '/clickhouse/t2', '/clickhouse/t2/replicas/{}'.format(num + 1)) + assert node2.query("SELECT COUNT() FROM t2") == "510\n" assert node3.query("SELECT COUNT() FROM t2") == "510\n" From 0acd01836148b9e8cfb97b04bf31f9cd899a56dc Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 11 Feb 2021 12:58:02 +0300 Subject: [PATCH 81/86] Fix typo --- src/Coordination/NuKeeperStorageDispatcher.cpp | 2 +- .../integration/test_testkeeper_multinode/test.py | 14 +++++++++++++- 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/src/Coordination/NuKeeperStorageDispatcher.cpp b/src/Coordination/NuKeeperStorageDispatcher.cpp index 76db01eb70f..042f0d2ffb9 100644 --- a/src/Coordination/NuKeeperStorageDispatcher.cpp +++ b/src/Coordination/NuKeeperStorageDispatcher.cpp @@ -115,7 +115,7 @@ void NuKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigurati { LOG_DEBUG(log, "Waiting server to initialize"); server->startup(true); - LOG_DEBUG(log, "Server intialized, waiting for quorum"); + LOG_DEBUG(log, "Server initialized, waiting for quorum"); server->waitInit(); LOG_DEBUG(log, "Quorum initialized"); diff --git a/tests/integration/test_testkeeper_multinode/test.py b/tests/integration/test_testkeeper_multinode/test.py index 7063c42f31a..f161c28ee83 100644 --- a/tests/integration/test_testkeeper_multinode/test.py +++ b/tests/integration/test_testkeeper_multinode/test.py @@ -242,6 +242,8 @@ def test_blocade_leader(started_cluster): print("Got exception node2", smaller_exception(ex)) time.sleep(0.5) else: + for num, node in enumerate([node1, node2, node3]): + dump_zk(node, '/clickhouse/t1', '/clickhouse/t1/replicas/{}'.format(num + 1)) assert False, "Cannot insert anything node2" for i in range(100): @@ -257,6 +259,8 @@ def test_blocade_leader(started_cluster): print("Got exception node3", smaller_exception(ex)) time.sleep(0.5) else: + for num, node in enumerate([node1, node2, node3]): + dump_zk(node, '/clickhouse/t1', '/clickhouse/t1/replicas/{}'.format(num + 1)) assert False, "Cannot insert anything node3" for n, node in enumerate([node1, node2, node3]): @@ -283,12 +287,14 @@ def test_blocade_leader(started_cluster): print("Got exception node1", smaller_exception(ex)) time.sleep(0.5) else: + for num, node in enumerate([node1, node2, node3]): + dump_zk(node, '/clickhouse/t1', '/clickhouse/t1/replicas/{}'.format(num + 1)) assert False, "Cannot insert anything node1" for n, node in enumerate([node1, node2, node3]): for i in range(100): try: - node.query("SYSTEM RESTART REPLICA t1", timeout=10) + node.query("SYSTEM RESTART REPLICA t1") node.query("SYSTEM SYNC REPLICA t1", timeout=10) break except Exception as ex: @@ -300,8 +306,14 @@ def test_blocade_leader(started_cluster): print("Got exception node{}".format(n + 1), smaller_exception(ex)) time.sleep(0.5) else: + for num, node in enumerate([node1, node2, node3]): + dump_zk(node, '/clickhouse/t1', '/clickhouse/t1/replicas/{}'.format(num + 1)) assert False, "Cannot sync replica node{}".format(n+1) + if node1.query("SELECT COUNT() FROM t1") != "310\n": + for num, node in enumerate([node1, node2, node3]): + dump_zk(node, '/clickhouse/t1', '/clickhouse/t1/replicas/{}'.format(num + 1)) + assert node1.query("SELECT COUNT() FROM t1") == "310\n" assert node2.query("SELECT COUNT() FROM t1") == "310\n" assert node3.query("SELECT COUNT() FROM t1") == "310\n" From 99a471e047877b953920ff2d3ab8e73e5030c6be Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 11 Feb 2021 13:25:10 +0300 Subject: [PATCH 82/86] Add ability to start as follower --- src/Coordination/InMemoryStateManager.cpp | 6 ++++++ src/Coordination/InMemoryStateManager.h | 6 ++++++ src/Coordination/NuKeeperServer.cpp | 4 ++-- src/Coordination/NuKeeperServer.h | 2 +- src/Coordination/NuKeeperStorageDispatcher.cpp | 2 +- .../configs/enable_test_keeper1.xml | 2 ++ .../configs/enable_test_keeper2.xml | 2 ++ .../configs/enable_test_keeper3.xml | 2 ++ 8 files changed, 22 insertions(+), 4 deletions(-) diff --git a/src/Coordination/InMemoryStateManager.cpp b/src/Coordination/InMemoryStateManager.cpp index d90c7e46f0d..a6db3271bc1 100644 --- a/src/Coordination/InMemoryStateManager.cpp +++ b/src/Coordination/InMemoryStateManager.cpp @@ -28,6 +28,9 @@ InMemoryStateManager::InMemoryStateManager( int port = config.getInt(full_prefix + ".port"); bool can_become_leader = config.getBool(full_prefix + ".can_become_leader", true); int32_t priority = config.getInt(full_prefix + ".priority", 1); + bool start_as_follower = config.getBool(full_prefix + ".start_as_follower", false); + if (start_as_follower) + start_as_follower_servers.insert(server_id); auto endpoint = hostname + ":" + std::to_string(port); auto peer_config = nuraft::cs_new(server_id, 0, endpoint, "", !can_become_leader, priority); @@ -41,6 +44,9 @@ InMemoryStateManager::InMemoryStateManager( } if (!my_server_config) throw Exception(ErrorCodes::RAFT_ERROR, "Our server id {} not found in raft_configuration section"); + + if (start_as_follower_servers.size() == cluster_config->get_servers().size()) + throw Exception(ErrorCodes::RAFT_ERROR, "At least one of servers should be able to start as leader (without )"); } void InMemoryStateManager::save_config(const nuraft::cluster_config & config) diff --git a/src/Coordination/InMemoryStateManager.h b/src/Coordination/InMemoryStateManager.h index b48b5188f36..a4537602b36 100644 --- a/src/Coordination/InMemoryStateManager.h +++ b/src/Coordination/InMemoryStateManager.h @@ -35,9 +35,15 @@ public: int getPort() const { return my_port; } + bool shouldStartAsFollower() const + { + return start_as_follower_servers.count(my_server_id); + } + private: int my_server_id; int my_port; + std::unordered_set start_as_follower_servers; nuraft::ptr log_store; nuraft::ptr my_server_config; nuraft::ptr cluster_config; diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index c7deebfdb96..7464a06e86f 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -31,7 +31,7 @@ NuKeeperServer::NuKeeperServer( { } -void NuKeeperServer::startup(bool should_build_quorum) +void NuKeeperServer::startup() { nuraft::raft_params params; params.heart_beat_interval_ = coordination_settings->heart_beat_interval_ms.totalMilliseconds(); @@ -47,7 +47,7 @@ void NuKeeperServer::startup(bool should_build_quorum) nuraft::asio_service::options asio_opts{}; nuraft::raft_server::init_options init_options; - init_options.skip_initial_election_timeout_ = !should_build_quorum; + init_options.skip_initial_election_timeout_ = state_manager->shouldStartAsFollower(); init_options.raft_callback_ = [this] (nuraft::cb_func::Type type, nuraft::cb_func::Param * param) { return callbackFunc(type, param); diff --git a/src/Coordination/NuKeeperServer.h b/src/Coordination/NuKeeperServer.h index a37d4d9127a..a8d269eb9eb 100644 --- a/src/Coordination/NuKeeperServer.h +++ b/src/Coordination/NuKeeperServer.h @@ -43,7 +43,7 @@ public: const Poco::Util::AbstractConfiguration & config, ResponsesQueue & responses_queue_); - void startup(bool should_build_quorum); + void startup(); void putRequest(const NuKeeperStorage::RequestForSession & request); diff --git a/src/Coordination/NuKeeperStorageDispatcher.cpp b/src/Coordination/NuKeeperStorageDispatcher.cpp index 042f0d2ffb9..570087757ad 100644 --- a/src/Coordination/NuKeeperStorageDispatcher.cpp +++ b/src/Coordination/NuKeeperStorageDispatcher.cpp @@ -114,7 +114,7 @@ void NuKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigurati try { LOG_DEBUG(log, "Waiting server to initialize"); - server->startup(true); + server->startup(); LOG_DEBUG(log, "Server initialized, waiting for quorum"); server->waitInit(); diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml index 3ae44f926d0..4ad76889d1e 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml @@ -22,6 +22,7 @@ node2 44444 true + true 2 @@ -29,6 +30,7 @@ node3 44444 true + true 1 diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml index 7674c755511..a1954a1e639 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml @@ -22,6 +22,7 @@ node2 44444 true + true 2 @@ -29,6 +30,7 @@ node3 44444 true + true 1 diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml index 59dde3bc1b1..88d2358138f 100644 --- a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml +++ b/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml @@ -22,6 +22,7 @@ node2 44444 true + true 2 @@ -29,6 +30,7 @@ node3 44444 true + true 1 From 447fcfa1c9763431d81a0e9af85f2588fd092555 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 11 Feb 2021 15:12:01 +0300 Subject: [PATCH 83/86] Fix build --- src/Coordination/InMemoryStateManager.cpp | 10 ++++++++++ src/Coordination/InMemoryStateManager.h | 5 +++++ src/Coordination/tests/gtest_for_build.cpp | 3 +-- 3 files changed, 16 insertions(+), 2 deletions(-) diff --git a/src/Coordination/InMemoryStateManager.cpp b/src/Coordination/InMemoryStateManager.cpp index a6db3271bc1..69e93578cc1 100644 --- a/src/Coordination/InMemoryStateManager.cpp +++ b/src/Coordination/InMemoryStateManager.cpp @@ -9,6 +9,16 @@ namespace ErrorCodes extern const int RAFT_ERROR; } +InMemoryStateManager::InMemoryStateManager(int server_id_, const std::string & host, int port) + : my_server_id(server_id_) + , my_port(port) + , log_store(nuraft::cs_new()) + , cluster_config(nuraft::cs_new()) +{ + auto peer_config = nuraft::cs_new(my_server_id, host + ":" + std::to_string(port)); + cluster_config->get_servers().push_back(peer_config); +} + InMemoryStateManager::InMemoryStateManager( int my_server_id_, const std::string & config_prefix, diff --git a/src/Coordination/InMemoryStateManager.h b/src/Coordination/InMemoryStateManager.h index a4537602b36..2a5c2f00dba 100644 --- a/src/Coordination/InMemoryStateManager.h +++ b/src/Coordination/InMemoryStateManager.h @@ -17,6 +17,11 @@ public: const std::string & config_prefix, const Poco::Util::AbstractConfiguration & config); + InMemoryStateManager( + int server_id_, + const std::string & host, + int port); + nuraft::ptr load_config() override { return cluster_config; } void save_config(const nuraft::cluster_config & config) override; diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index c6f29831618..ed9777350c5 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -27,7 +27,6 @@ TEST(CoordinationTest, BuildTest) { DB::InMemoryLogStore store; - DB::InMemoryStateManager state_manager(1, "localhost:12345"); DB::SummingStateMachine machine; EXPECT_EQ(1, 1); } @@ -74,7 +73,7 @@ struct SimpliestRaftServer , port(port_) , endpoint(hostname + ":" + std::to_string(port)) , state_machine(nuraft::cs_new()) - , state_manager(nuraft::cs_new(server_id, endpoint)) + , state_manager(nuraft::cs_new(server_id, hostname, port)) { nuraft::raft_params params; params.heart_beat_interval_ = 100; From df181b534e53d64196dfede15a491387cf4f9c63 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 11 Feb 2021 23:29:01 +0300 Subject: [PATCH 84/86] Better connection reset --- tests/integration/test_testkeeper_multinode/test.py | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_testkeeper_multinode/test.py b/tests/integration/test_testkeeper_multinode/test.py index f161c28ee83..c9bde5c5a02 100644 --- a/tests/integration/test_testkeeper_multinode/test.py +++ b/tests/integration/test_testkeeper_multinode/test.py @@ -12,7 +12,7 @@ node1 = cluster.add_instance('node1', main_configs=['configs/enable_test_keeper1 node2 = cluster.add_instance('node2', main_configs=['configs/enable_test_keeper2.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True) node3 = cluster.add_instance('node3', main_configs=['configs/enable_test_keeper3.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True) -from kazoo.client import KazooClient +from kazoo.client import KazooClient, KazooState @pytest.fixture(scope="module") def started_cluster(): @@ -29,11 +29,13 @@ def smaller_exception(ex): def get_fake_zk(nodename, timeout=30.0): _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout) - def reset_last_zxid_listener(state): + def reset_listener(state): + nonlocal _fake_zk_instance print("Fake zk callback called for state", state) - _fake_zk_instance.last_zxid = 0 + if state != KazooState.CONNECTED: + _fake_zk_instance._reset() - _fake_zk_instance.add_listener(reset_last_zxid_listener) + _fake_zk_instance.add_listener(reset_listener) _fake_zk_instance.start() return _fake_zk_instance @@ -135,7 +137,7 @@ def test_session_expiration(started_cluster): try: node1_zk = get_fake_zk("node1") node2_zk = get_fake_zk("node2") - node3_zk = get_fake_zk("node3", timeout=3.0) + node3_zk = get_fake_zk("node3", timeout=5.0) node3_zk.create("/test_ephemeral_node", b"world", ephemeral=True) From c883b7d154d8f4b87129a2a458ca07187fd900eb Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 12 Feb 2021 11:50:20 +0300 Subject: [PATCH 85/86] Split tests to make them stable --- .../__init__.py | 0 .../configs/enable_test_keeper1.xml | 0 .../configs/enable_test_keeper2.xml | 0 .../configs/enable_test_keeper3.xml | 0 .../configs/log_conf.xml | 0 .../configs/use_test_keeper.xml | 0 .../test.py | 198 ++------------- .../__init__.py | 1 + .../configs/enable_test_keeper1.xml | 38 +++ .../configs/enable_test_keeper2.xml | 38 +++ .../configs/enable_test_keeper3.xml | 38 +++ .../configs/log_conf.xml | 12 + .../configs/use_test_keeper.xml | 16 ++ .../test_testkeeper_multinode_simple/test.py | 239 ++++++++++++++++++ 14 files changed, 408 insertions(+), 172 deletions(-) rename tests/integration/{test_testkeeper_multinode => test_testkeeper_multinode_blocade_leader}/__init__.py (100%) rename tests/integration/{test_testkeeper_multinode => test_testkeeper_multinode_blocade_leader}/configs/enable_test_keeper1.xml (100%) rename tests/integration/{test_testkeeper_multinode => test_testkeeper_multinode_blocade_leader}/configs/enable_test_keeper2.xml (100%) rename tests/integration/{test_testkeeper_multinode => test_testkeeper_multinode_blocade_leader}/configs/enable_test_keeper3.xml (100%) rename tests/integration/{test_testkeeper_multinode => test_testkeeper_multinode_blocade_leader}/configs/log_conf.xml (100%) rename tests/integration/{test_testkeeper_multinode => test_testkeeper_multinode_blocade_leader}/configs/use_test_keeper.xml (100%) rename tests/integration/{test_testkeeper_multinode => test_testkeeper_multinode_blocade_leader}/test.py (69%) create mode 100644 tests/integration/test_testkeeper_multinode_simple/__init__.py create mode 100644 tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper1.xml create mode 100644 tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper2.xml create mode 100644 tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper3.xml create mode 100644 tests/integration/test_testkeeper_multinode_simple/configs/log_conf.xml create mode 100644 tests/integration/test_testkeeper_multinode_simple/configs/use_test_keeper.xml create mode 100644 tests/integration/test_testkeeper_multinode_simple/test.py diff --git a/tests/integration/test_testkeeper_multinode/__init__.py b/tests/integration/test_testkeeper_multinode_blocade_leader/__init__.py similarity index 100% rename from tests/integration/test_testkeeper_multinode/__init__.py rename to tests/integration/test_testkeeper_multinode_blocade_leader/__init__.py diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml b/tests/integration/test_testkeeper_multinode_blocade_leader/configs/enable_test_keeper1.xml similarity index 100% rename from tests/integration/test_testkeeper_multinode/configs/enable_test_keeper1.xml rename to tests/integration/test_testkeeper_multinode_blocade_leader/configs/enable_test_keeper1.xml diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml b/tests/integration/test_testkeeper_multinode_blocade_leader/configs/enable_test_keeper2.xml similarity index 100% rename from tests/integration/test_testkeeper_multinode/configs/enable_test_keeper2.xml rename to tests/integration/test_testkeeper_multinode_blocade_leader/configs/enable_test_keeper2.xml diff --git a/tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml b/tests/integration/test_testkeeper_multinode_blocade_leader/configs/enable_test_keeper3.xml similarity index 100% rename from tests/integration/test_testkeeper_multinode/configs/enable_test_keeper3.xml rename to tests/integration/test_testkeeper_multinode_blocade_leader/configs/enable_test_keeper3.xml diff --git a/tests/integration/test_testkeeper_multinode/configs/log_conf.xml b/tests/integration/test_testkeeper_multinode_blocade_leader/configs/log_conf.xml similarity index 100% rename from tests/integration/test_testkeeper_multinode/configs/log_conf.xml rename to tests/integration/test_testkeeper_multinode_blocade_leader/configs/log_conf.xml diff --git a/tests/integration/test_testkeeper_multinode/configs/use_test_keeper.xml b/tests/integration/test_testkeeper_multinode_blocade_leader/configs/use_test_keeper.xml similarity index 100% rename from tests/integration/test_testkeeper_multinode/configs/use_test_keeper.xml rename to tests/integration/test_testkeeper_multinode_blocade_leader/configs/use_test_keeper.xml diff --git a/tests/integration/test_testkeeper_multinode/test.py b/tests/integration/test_testkeeper_multinode_blocade_leader/test.py similarity index 69% rename from tests/integration/test_testkeeper_multinode/test.py rename to tests/integration/test_testkeeper_multinode_blocade_leader/test.py index c9bde5c5a02..899f7212660 100644 --- a/tests/integration/test_testkeeper_multinode/test.py +++ b/tests/integration/test_testkeeper_multinode_blocade_leader/test.py @@ -27,6 +27,30 @@ def started_cluster(): def smaller_exception(ex): return '\n'.join(str(ex).split('\n')[0:2]) +def wait_node(node): + for _ in range(100): + zk = None + try: + node.query("SELECT * FROM system.zookeeper WHERE path = '/'") + zk = get_fake_zk(node.name, timeout=30.0) + zk.create("/test", sequence=True) + print("node", node.name, "ready") + break + except Exception as ex: + time.sleep(0.2) + print("Waiting until", node.name, "will be ready, exception", ex) + finally: + if zk: + zk.stop() + zk.close() + else: + raise Exception("Can't wait node", node.name, "to become ready") + +def wait_nodes(): + for node in [node1, node2, node3]: + wait_node(node) + + def get_fake_zk(nodename, timeout=30.0): _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout) def reset_listener(state): @@ -39,182 +63,11 @@ def get_fake_zk(nodename, timeout=30.0): _fake_zk_instance.start() return _fake_zk_instance -def test_read_write_multinode(started_cluster): - try: - node1_zk = get_fake_zk("node1") - node2_zk = get_fake_zk("node2") - node3_zk = get_fake_zk("node3") - - node1_zk.create("/test_read_write_multinode_node1", b"somedata1") - node2_zk.create("/test_read_write_multinode_node2", b"somedata2") - node3_zk.create("/test_read_write_multinode_node3", b"somedata3") - - # stale reads are allowed - while node1_zk.exists("/test_read_write_multinode_node2") is None: - time.sleep(0.1) - - while node1_zk.exists("/test_read_write_multinode_node3") is None: - time.sleep(0.1) - - while node2_zk.exists("/test_read_write_multinode_node3") is None: - time.sleep(0.1) - - assert node3_zk.get("/test_read_write_multinode_node1")[0] == b"somedata1" - assert node2_zk.get("/test_read_write_multinode_node1")[0] == b"somedata1" - assert node1_zk.get("/test_read_write_multinode_node1")[0] == b"somedata1" - - assert node3_zk.get("/test_read_write_multinode_node2")[0] == b"somedata2" - assert node2_zk.get("/test_read_write_multinode_node2")[0] == b"somedata2" - assert node1_zk.get("/test_read_write_multinode_node2")[0] == b"somedata2" - - assert node3_zk.get("/test_read_write_multinode_node3")[0] == b"somedata3" - assert node2_zk.get("/test_read_write_multinode_node3")[0] == b"somedata3" - assert node1_zk.get("/test_read_write_multinode_node3")[0] == b"somedata3" - - finally: - try: - for zk_conn in [node1_zk, node2_zk, node3_zk]: - zk_conn.stop() - zk_conn.close() - except: - pass - - -def test_watch_on_follower(started_cluster): - try: - node1_zk = get_fake_zk("node1") - node2_zk = get_fake_zk("node2") - node3_zk = get_fake_zk("node3") - - node1_zk.create("/test_data_watches") - node2_zk.set("/test_data_watches", b"hello") - node3_zk.set("/test_data_watches", b"world") - - node1_data = None - def node1_callback(event): - print("node1 data watch called") - nonlocal node1_data - node1_data = event - - node1_zk.get("/test_data_watches", watch=node1_callback) - - node2_data = None - def node2_callback(event): - print("node2 data watch called") - nonlocal node2_data - node2_data = event - - node2_zk.get("/test_data_watches", watch=node2_callback) - - node3_data = None - def node3_callback(event): - print("node3 data watch called") - nonlocal node3_data - node3_data = event - - node3_zk.get("/test_data_watches", watch=node3_callback) - - node1_zk.set("/test_data_watches", b"somevalue") - time.sleep(3) - - print(node1_data) - print(node2_data) - print(node3_data) - - assert node1_data == node2_data - assert node3_data == node2_data - - finally: - try: - for zk_conn in [node1_zk, node2_zk, node3_zk]: - zk_conn.stop() - zk_conn.close() - except: - pass - - -def test_session_expiration(started_cluster): - try: - node1_zk = get_fake_zk("node1") - node2_zk = get_fake_zk("node2") - node3_zk = get_fake_zk("node3", timeout=5.0) - - node3_zk.create("/test_ephemeral_node", b"world", ephemeral=True) - - with PartitionManager() as pm: - pm.partition_instances(node3, node2) - pm.partition_instances(node3, node1) - node3_zk.stop() - node3_zk.close() - for _ in range(100): - if node1_zk.exists("/test_ephemeral_node") is None and node2_zk.exists("/test_ephemeral_node") is None: - break - time.sleep(0.1) - - assert node1_zk.exists("/test_ephemeral_node") is None - assert node2_zk.exists("/test_ephemeral_node") is None - - finally: - try: - for zk_conn in [node1_zk, node2_zk, node3_zk]: - try: - zk_conn.stop() - zk_conn.close() - except: - pass - except: - pass - - -def test_follower_restart(started_cluster): - try: - node1_zk = get_fake_zk("node1") - - node1_zk.create("/test_restart_node", b"hello") - - node3.restart_clickhouse(kill=True) - - node3_zk = get_fake_zk("node3") - - # got data from log - assert node3_zk.get("/test_restart_node")[0] == b"hello" - - finally: - try: - for zk_conn in [node1_zk, node3_zk]: - try: - zk_conn.stop() - zk_conn.close() - except: - pass - except: - pass - - -def test_simple_replicated_table(started_cluster): - # something may be wrong after partition in other tests - # so create with retry - for i, node in enumerate([node1, node2, node3]): - for i in range(100): - try: - node.query("CREATE TABLE t (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t', '{}') ORDER BY tuple()".format(i + 1)) - break - except: - time.sleep(0.1) - - node2.query("INSERT INTO t SELECT number FROM numbers(10)") - - node1.query("SYSTEM SYNC REPLICA t", timeout=10) - node3.query("SYSTEM SYNC REPLICA t", timeout=10) - - assert node1.query("SELECT COUNT() FROM t") == "10\n" - assert node2.query("SELECT COUNT() FROM t") == "10\n" - assert node3.query("SELECT COUNT() FROM t") == "10\n" - # in extremely rare case it can take more than 5 minutes in debug build with sanitizer @pytest.mark.timeout(600) def test_blocade_leader(started_cluster): + wait_nodes() for i, node in enumerate([node1, node2, node3]): node.query("CREATE TABLE t1 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t1', '{}') ORDER BY tuple()".format(i + 1)) @@ -337,6 +190,7 @@ def dump_zk(node, zk_path, replica_path): # in extremely rare case it can take more than 5 minutes in debug build with sanitizer @pytest.mark.timeout(600) def test_blocade_leader_twice(started_cluster): + wait_nodes() for i, node in enumerate([node1, node2, node3]): node.query("CREATE TABLE t2 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t2', '{}') ORDER BY tuple()".format(i + 1)) diff --git a/tests/integration/test_testkeeper_multinode_simple/__init__.py b/tests/integration/test_testkeeper_multinode_simple/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_testkeeper_multinode_simple/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper1.xml b/tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper1.xml new file mode 100644 index 00000000000..4ad76889d1e --- /dev/null +++ b/tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper1.xml @@ -0,0 +1,38 @@ + + + 9181 + 1 + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + true + 3 + + + 2 + node2 + 44444 + true + true + 2 + + + 3 + node3 + 44444 + true + true + 1 + + + + diff --git a/tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper2.xml b/tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper2.xml new file mode 100644 index 00000000000..a1954a1e639 --- /dev/null +++ b/tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper2.xml @@ -0,0 +1,38 @@ + + + 9181 + 2 + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + true + 3 + + + 2 + node2 + 44444 + true + true + 2 + + + 3 + node3 + 44444 + true + true + 1 + + + + diff --git a/tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper3.xml b/tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper3.xml new file mode 100644 index 00000000000..88d2358138f --- /dev/null +++ b/tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper3.xml @@ -0,0 +1,38 @@ + + + 9181 + 3 + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + true + 3 + + + 2 + node2 + 44444 + true + true + 2 + + + 3 + node3 + 44444 + true + true + 1 + + + + diff --git a/tests/integration/test_testkeeper_multinode_simple/configs/log_conf.xml b/tests/integration/test_testkeeper_multinode_simple/configs/log_conf.xml new file mode 100644 index 00000000000..318a6bca95d --- /dev/null +++ b/tests/integration/test_testkeeper_multinode_simple/configs/log_conf.xml @@ -0,0 +1,12 @@ + + 3 + + trace + /var/log/clickhouse-server/log.log + /var/log/clickhouse-server/log.err.log + 1000M + 10 + /var/log/clickhouse-server/stderr.log + /var/log/clickhouse-server/stdout.log + + diff --git a/tests/integration/test_testkeeper_multinode_simple/configs/use_test_keeper.xml b/tests/integration/test_testkeeper_multinode_simple/configs/use_test_keeper.xml new file mode 100644 index 00000000000..b6139005d2f --- /dev/null +++ b/tests/integration/test_testkeeper_multinode_simple/configs/use_test_keeper.xml @@ -0,0 +1,16 @@ + + + + node1 + 9181 + + + node2 + 9181 + + + node3 + 9181 + + + diff --git a/tests/integration/test_testkeeper_multinode_simple/test.py b/tests/integration/test_testkeeper_multinode_simple/test.py new file mode 100644 index 00000000000..a7ece4bbd56 --- /dev/null +++ b/tests/integration/test_testkeeper_multinode_simple/test.py @@ -0,0 +1,239 @@ +import pytest +from helpers.cluster import ClickHouseCluster +import random +import string +import os +import time +from multiprocessing.dummy import Pool +from helpers.network import PartitionManager + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', main_configs=['configs/enable_test_keeper1.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True) +node2 = cluster.add_instance('node2', main_configs=['configs/enable_test_keeper2.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True) +node3 = cluster.add_instance('node3', main_configs=['configs/enable_test_keeper3.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True) + +from kazoo.client import KazooClient, KazooState + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + +def smaller_exception(ex): + return '\n'.join(str(ex).split('\n')[0:2]) + +def wait_node(node): + for _ in range(100): + zk = None + try: + node.query("SELECT * FROM system.zookeeper WHERE path = '/'") + zk = get_fake_zk(node.name, timeout=30.0) + zk.create("/test", sequence=True) + print("node", node.name, "ready") + break + except Exception as ex: + time.sleep(0.2) + print("Waiting until", node.name, "will be ready, exception", ex) + finally: + if zk: + zk.stop() + zk.close() + else: + raise Exception("Can't wait node", node.name, "to become ready") + +def wait_nodes(): + for node in [node1, node2, node3]: + wait_node(node) + + +def get_fake_zk(nodename, timeout=30.0): + _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout) + def reset_listener(state): + nonlocal _fake_zk_instance + print("Fake zk callback called for state", state) + if state != KazooState.CONNECTED: + _fake_zk_instance._reset() + + _fake_zk_instance.add_listener(reset_listener) + _fake_zk_instance.start() + return _fake_zk_instance + +def test_read_write_multinode(started_cluster): + try: + wait_nodes() + node1_zk = get_fake_zk("node1") + node2_zk = get_fake_zk("node2") + node3_zk = get_fake_zk("node3") + + node1_zk.create("/test_read_write_multinode_node1", b"somedata1") + node2_zk.create("/test_read_write_multinode_node2", b"somedata2") + node3_zk.create("/test_read_write_multinode_node3", b"somedata3") + + # stale reads are allowed + while node1_zk.exists("/test_read_write_multinode_node2") is None: + time.sleep(0.1) + + while node1_zk.exists("/test_read_write_multinode_node3") is None: + time.sleep(0.1) + + while node2_zk.exists("/test_read_write_multinode_node3") is None: + time.sleep(0.1) + + assert node3_zk.get("/test_read_write_multinode_node1")[0] == b"somedata1" + assert node2_zk.get("/test_read_write_multinode_node1")[0] == b"somedata1" + assert node1_zk.get("/test_read_write_multinode_node1")[0] == b"somedata1" + + assert node3_zk.get("/test_read_write_multinode_node2")[0] == b"somedata2" + assert node2_zk.get("/test_read_write_multinode_node2")[0] == b"somedata2" + assert node1_zk.get("/test_read_write_multinode_node2")[0] == b"somedata2" + + assert node3_zk.get("/test_read_write_multinode_node3")[0] == b"somedata3" + assert node2_zk.get("/test_read_write_multinode_node3")[0] == b"somedata3" + assert node1_zk.get("/test_read_write_multinode_node3")[0] == b"somedata3" + + finally: + try: + for zk_conn in [node1_zk, node2_zk, node3_zk]: + zk_conn.stop() + zk_conn.close() + except: + pass + + +def test_watch_on_follower(started_cluster): + try: + wait_nodes() + node1_zk = get_fake_zk("node1") + node2_zk = get_fake_zk("node2") + node3_zk = get_fake_zk("node3") + + node1_zk.create("/test_data_watches") + node2_zk.set("/test_data_watches", b"hello") + node3_zk.set("/test_data_watches", b"world") + + node1_data = None + def node1_callback(event): + print("node1 data watch called") + nonlocal node1_data + node1_data = event + + node1_zk.get("/test_data_watches", watch=node1_callback) + + node2_data = None + def node2_callback(event): + print("node2 data watch called") + nonlocal node2_data + node2_data = event + + node2_zk.get("/test_data_watches", watch=node2_callback) + + node3_data = None + def node3_callback(event): + print("node3 data watch called") + nonlocal node3_data + node3_data = event + + node3_zk.get("/test_data_watches", watch=node3_callback) + + node1_zk.set("/test_data_watches", b"somevalue") + time.sleep(3) + + print(node1_data) + print(node2_data) + print(node3_data) + + assert node1_data == node2_data + assert node3_data == node2_data + + finally: + try: + for zk_conn in [node1_zk, node2_zk, node3_zk]: + zk_conn.stop() + zk_conn.close() + except: + pass + + +def test_session_expiration(started_cluster): + try: + wait_nodes() + node1_zk = get_fake_zk("node1") + node2_zk = get_fake_zk("node2") + node3_zk = get_fake_zk("node3", timeout=3.0) + print("Node3 session id", node3_zk._session_id) + + node3_zk.create("/test_ephemeral_node", b"world", ephemeral=True) + + with PartitionManager() as pm: + pm.partition_instances(node3, node2) + pm.partition_instances(node3, node1) + node3_zk.stop() + node3_zk.close() + for _ in range(100): + if node1_zk.exists("/test_ephemeral_node") is None and node2_zk.exists("/test_ephemeral_node") is None: + break + print("Node1 exists", node1_zk.exists("/test_ephemeral_node")) + print("Node2 exists", node2_zk.exists("/test_ephemeral_node")) + time.sleep(0.1) + node1_zk.sync("/") + node2_zk.sync("/") + + assert node1_zk.exists("/test_ephemeral_node") is None + assert node2_zk.exists("/test_ephemeral_node") is None + + finally: + try: + for zk_conn in [node1_zk, node2_zk, node3_zk]: + try: + zk_conn.stop() + zk_conn.close() + except: + pass + except: + pass + + +def test_follower_restart(started_cluster): + try: + wait_nodes() + node1_zk = get_fake_zk("node1") + + node1_zk.create("/test_restart_node", b"hello") + + node3.restart_clickhouse(kill=True) + + node3_zk = get_fake_zk("node3") + + # got data from log + assert node3_zk.get("/test_restart_node")[0] == b"hello" + + finally: + try: + for zk_conn in [node1_zk, node3_zk]: + try: + zk_conn.stop() + zk_conn.close() + except: + pass + except: + pass + + +def test_simple_replicated_table(started_cluster): + wait_nodes() + for i, node in enumerate([node1, node2, node3]): + node.query("CREATE TABLE t (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t', '{}') ORDER BY tuple()".format(i + 1)) + + node2.query("INSERT INTO t SELECT number FROM numbers(10)") + + node1.query("SYSTEM SYNC REPLICA t", timeout=10) + node3.query("SYSTEM SYNC REPLICA t", timeout=10) + + assert node1.query("SELECT COUNT() FROM t") == "10\n" + assert node2.query("SELECT COUNT() FROM t") == "10\n" + assert node3.query("SELECT COUNT() FROM t") == "10\n" From 6aecb62416ece880cbb8ee3a803e14d841388dde Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 12 Feb 2021 12:17:10 +0300 Subject: [PATCH 86/86] Replace database with ordinary --- .../test.py | 98 ++++++++++--------- 1 file changed, 50 insertions(+), 48 deletions(-) diff --git a/tests/integration/test_testkeeper_multinode_blocade_leader/test.py b/tests/integration/test_testkeeper_multinode_blocade_leader/test.py index 899f7212660..3b2867ef3c7 100644 --- a/tests/integration/test_testkeeper_multinode_blocade_leader/test.py +++ b/tests/integration/test_testkeeper_multinode_blocade_leader/test.py @@ -69,16 +69,17 @@ def get_fake_zk(nodename, timeout=30.0): def test_blocade_leader(started_cluster): wait_nodes() for i, node in enumerate([node1, node2, node3]): - node.query("CREATE TABLE t1 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t1', '{}') ORDER BY tuple()".format(i + 1)) + node.query("CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary") + node.query("CREATE TABLE ordinary.t1 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t1', '{}') ORDER BY tuple()".format(i + 1)) - node2.query("INSERT INTO t1 SELECT number FROM numbers(10)") + node2.query("INSERT INTO ordinary.t1 SELECT number FROM numbers(10)") - node1.query("SYSTEM SYNC REPLICA t1", timeout=10) - node3.query("SYSTEM SYNC REPLICA t1", timeout=10) + node1.query("SYSTEM SYNC REPLICA ordinary.t1", timeout=10) + node3.query("SYSTEM SYNC REPLICA ordinary.t1", timeout=10) - assert node1.query("SELECT COUNT() FROM t1") == "10\n" - assert node2.query("SELECT COUNT() FROM t1") == "10\n" - assert node3.query("SELECT COUNT() FROM t1") == "10\n" + assert node1.query("SELECT COUNT() FROM ordinary.t1") == "10\n" + assert node2.query("SELECT COUNT() FROM ordinary.t1") == "10\n" + assert node3.query("SELECT COUNT() FROM ordinary.t1") == "10\n" with PartitionManager() as pm: pm.partition_instances(node2, node1) @@ -86,12 +87,12 @@ def test_blocade_leader(started_cluster): for i in range(100): try: - node2.query("SYSTEM RESTART REPLICA t1") - node2.query("INSERT INTO t1 SELECT rand() FROM numbers(100)") + node2.query("SYSTEM RESTART REPLICA ordinary.t1") + node2.query("INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100)") break except Exception as ex: try: - node2.query("ATTACH TABLE t1") + node2.query("ATTACH TABLE ordinary.t1") except Exception as attach_ex: print("Got exception node2", smaller_exception(attach_ex)) print("Got exception node2", smaller_exception(ex)) @@ -103,12 +104,12 @@ def test_blocade_leader(started_cluster): for i in range(100): try: - node3.query("SYSTEM RESTART REPLICA t1") - node3.query("INSERT INTO t1 SELECT rand() FROM numbers(100)") + node3.query("SYSTEM RESTART REPLICA ordinary.t1") + node3.query("INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100)") break except Exception as ex: try: - node3.query("ATTACH TABLE t1") + node3.query("ATTACH TABLE ordinary.t1") except Exception as attach_ex: print("Got exception node3", smaller_exception(attach_ex)) print("Got exception node3", smaller_exception(ex)) @@ -121,11 +122,11 @@ def test_blocade_leader(started_cluster): for n, node in enumerate([node1, node2, node3]): for i in range(100): try: - node.query("SYSTEM RESTART REPLICA t1") + node.query("SYSTEM RESTART REPLICA ordinary.t1") break except Exception as ex: try: - node.query("ATTACH TABLE t1") + node.query("ATTACH TABLE ordinary.t1") except Exception as attach_ex: print("Got exception node{}".format(n + 1), smaller_exception(attach_ex)) @@ -136,7 +137,7 @@ def test_blocade_leader(started_cluster): for i in range(100): try: - node1.query("INSERT INTO t1 SELECT rand() FROM numbers(100)") + node1.query("INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100)") break except Exception as ex: print("Got exception node1", smaller_exception(ex)) @@ -149,12 +150,12 @@ def test_blocade_leader(started_cluster): for n, node in enumerate([node1, node2, node3]): for i in range(100): try: - node.query("SYSTEM RESTART REPLICA t1") - node.query("SYSTEM SYNC REPLICA t1", timeout=10) + node.query("SYSTEM RESTART REPLICA ordinary.t1") + node.query("SYSTEM SYNC REPLICA ordinary.t1", timeout=10) break except Exception as ex: try: - node.query("ATTACH TABLE t1") + node.query("ATTACH TABLE ordinary.t1") except Exception as attach_ex: print("Got exception node{}".format(n + 1), smaller_exception(attach_ex)) @@ -165,13 +166,13 @@ def test_blocade_leader(started_cluster): dump_zk(node, '/clickhouse/t1', '/clickhouse/t1/replicas/{}'.format(num + 1)) assert False, "Cannot sync replica node{}".format(n+1) - if node1.query("SELECT COUNT() FROM t1") != "310\n": + if node1.query("SELECT COUNT() FROM ordinary.t1") != "310\n": for num, node in enumerate([node1, node2, node3]): dump_zk(node, '/clickhouse/t1', '/clickhouse/t1/replicas/{}'.format(num + 1)) - assert node1.query("SELECT COUNT() FROM t1") == "310\n" - assert node2.query("SELECT COUNT() FROM t1") == "310\n" - assert node3.query("SELECT COUNT() FROM t1") == "310\n" + assert node1.query("SELECT COUNT() FROM ordinary.t1") == "310\n" + assert node2.query("SELECT COUNT() FROM ordinary.t1") == "310\n" + assert node3.query("SELECT COUNT() FROM ordinary.t1") == "310\n" def dump_zk(node, zk_path, replica_path): @@ -192,16 +193,17 @@ def dump_zk(node, zk_path, replica_path): def test_blocade_leader_twice(started_cluster): wait_nodes() for i, node in enumerate([node1, node2, node3]): - node.query("CREATE TABLE t2 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t2', '{}') ORDER BY tuple()".format(i + 1)) + node.query("CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary") + node.query("CREATE TABLE ordinary.t2 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t2', '{}') ORDER BY tuple()".format(i + 1)) - node2.query("INSERT INTO t2 SELECT number FROM numbers(10)") + node2.query("INSERT INTO ordinary.t2 SELECT number FROM numbers(10)") - node1.query("SYSTEM SYNC REPLICA t2", timeout=10) - node3.query("SYSTEM SYNC REPLICA t2", timeout=10) + node1.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) + node3.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) - assert node1.query("SELECT COUNT() FROM t2") == "10\n" - assert node2.query("SELECT COUNT() FROM t2") == "10\n" - assert node3.query("SELECT COUNT() FROM t2") == "10\n" + assert node1.query("SELECT COUNT() FROM ordinary.t2") == "10\n" + assert node2.query("SELECT COUNT() FROM ordinary.t2") == "10\n" + assert node3.query("SELECT COUNT() FROM ordinary.t2") == "10\n" with PartitionManager() as pm: pm.partition_instances(node2, node1) @@ -209,12 +211,12 @@ def test_blocade_leader_twice(started_cluster): for i in range(100): try: - node2.query("SYSTEM RESTART REPLICA t2") - node2.query("INSERT INTO t2 SELECT rand() FROM numbers(100)") + node2.query("SYSTEM RESTART REPLICA ordinary.t2") + node2.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100)") break except Exception as ex: try: - node2.query("ATTACH TABLE t2") + node2.query("ATTACH TABLE ordinary.t2") except Exception as attach_ex: print("Got exception node2", smaller_exception(attach_ex)) print("Got exception node2", smaller_exception(ex)) @@ -226,12 +228,12 @@ def test_blocade_leader_twice(started_cluster): for i in range(100): try: - node3.query("SYSTEM RESTART REPLICA t2") - node3.query("INSERT INTO t2 SELECT rand() FROM numbers(100)") + node3.query("SYSTEM RESTART REPLICA ordinary.t2") + node3.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100)") break except Exception as ex: try: - node3.query("ATTACH TABLE t2") + node3.query("ATTACH TABLE ordinary.t2") except Exception as attach_ex: print("Got exception node3", smaller_exception(attach_ex)) print("Got exception node3", smaller_exception(ex)) @@ -247,14 +249,14 @@ def test_blocade_leader_twice(started_cluster): for i in range(10): try: - node3.query("INSERT INTO t2 SELECT rand() FROM numbers(100)") + node3.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100)") assert False, "Node3 became leader?" except Exception as ex: time.sleep(0.5) for i in range(10): try: - node2.query("INSERT INTO t2 SELECT rand() FROM numbers(100)") + node2.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100)") assert False, "Node2 became leader?" except Exception as ex: time.sleep(0.5) @@ -263,11 +265,11 @@ def test_blocade_leader_twice(started_cluster): for n, node in enumerate([node1, node2, node3]): for i in range(100): try: - node.query("SYSTEM RESTART REPLICA t2") + node.query("SYSTEM RESTART REPLICA ordinary.t2") break except Exception as ex: try: - node.query("ATTACH TABLE t2") + node.query("ATTACH TABLE ordinary.t2") except Exception as attach_ex: print("Got exception node{}".format(n + 1), smaller_exception(attach_ex)) @@ -281,7 +283,7 @@ def test_blocade_leader_twice(started_cluster): for n, node in enumerate([node1, node2, node3]): for i in range(100): try: - node.query("INSERT INTO t2 SELECT rand() FROM numbers(100)") + node.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100)") break except Exception as ex: print("Got exception node{}".format(n + 1), smaller_exception(ex)) @@ -294,12 +296,12 @@ def test_blocade_leader_twice(started_cluster): for n, node in enumerate([node1, node2, node3]): for i in range(100): try: - node.query("SYSTEM RESTART REPLICA t2") - node.query("SYSTEM SYNC REPLICA t2", timeout=10) + node.query("SYSTEM RESTART REPLICA ordinary.t2") + node.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) break except Exception as ex: try: - node.query("ATTACH TABLE t2") + node.query("ATTACH TABLE ordinary.t2") except Exception as attach_ex: print("Got exception node{}".format(n + 1), smaller_exception(attach_ex)) @@ -310,10 +312,10 @@ def test_blocade_leader_twice(started_cluster): dump_zk(node, '/clickhouse/t2', '/clickhouse/t2/replicas/{}'.format(num + 1)) assert False, "Cannot reconnect for node{}".format(n + 1) - assert node1.query("SELECT COUNT() FROM t2") == "510\n" - if node2.query("SELECT COUNT() FROM t2") != "510\n": + assert node1.query("SELECT COUNT() FROM ordinary.t2") == "510\n" + if node2.query("SELECT COUNT() FROM ordinary.t2") != "510\n": for num, node in enumerate([node1, node2, node3]): dump_zk(node, '/clickhouse/t2', '/clickhouse/t2/replicas/{}'.format(num + 1)) - assert node2.query("SELECT COUNT() FROM t2") == "510\n" - assert node3.query("SELECT COUNT() FROM t2") == "510\n" + assert node2.query("SELECT COUNT() FROM ordinary.t2") == "510\n" + assert node3.query("SELECT COUNT() FROM ordinary.t2") == "510\n"