From 7bfe16af51ec39713c3703fd45b759f169aa51ef Mon Sep 17 00:00:00 2001 From: longfar Date: Mon, 24 Jun 2024 20:20:12 +0800 Subject: [PATCH 01/19] refactor: remove the singleton of PRaft --- src/base_cmd.cc | 26 +++++++++++--------------- src/client.cc | 11 +++++------ src/cmd_admin.cc | 34 ++++++++++++++++++++++------------ src/cmd_admin.h | 3 +++ src/cmd_raft.cc | 36 ++++++++++++++++++++---------------- src/cmd_raft.h | 7 +++++++ src/db.cc | 17 +++++++---------- src/db.h | 8 +++++--- src/pikiwidb.cc | 3 --- src/praft/praft.cc | 27 +++++++++++---------------- src/praft/praft.h | 20 +++++++++----------- tests/consistency_test.go | 4 ++-- 12 files changed, 102 insertions(+), 94 deletions(-) diff --git a/src/base_cmd.cc b/src/base_cmd.cc index 4e53d6873..5fcd499ff 100644 --- a/src/base_cmd.cc +++ b/src/base_cmd.cc @@ -9,15 +9,12 @@ #include "base_cmd.h" -#include "fmt/core.h" - +#include "pstd/log.h" #include "praft/praft.h" -#include "common.h" #include "config.h" -#include "log.h" #include "pikiwidb.h" -#include "praft/praft.h" +#include "store.h" namespace pikiwidb { @@ -41,19 +38,18 @@ std::vector BaseCmd::CurrentKey(PClient* client) const { return std void BaseCmd::Execute(PClient* client) { DEBUG("execute command: {}", client->CmdName()); - // read consistency (lease read) / write redirection - if (g_config.use_raft.load(std::memory_order_relaxed) && (HasFlag(kCmdFlagsReadonly) || HasFlag(kCmdFlagsWrite))) { - if (!PRAFT.IsInitialized()) { + if (g_config.use_raft.load()) { + auto praft = PSTORE.GetBackend(client->GetCurrentDB())->GetPRaft(); + // 1. If PRAFT is not initialized yet, return an error message to the client for both read and write commands. + if (!praft->IsInitialized() && (HasFlag(kCmdFlagsReadonly) || HasFlag(kCmdFlagsWrite))) { + DEBUG("drop command: {}", client->CmdName()); return client->SetRes(CmdRes::kErrOther, "PRAFT is not initialized"); } - if (!PRAFT.IsLeader()) { - auto leader_addr = PRAFT.GetLeaderAddress(); - if (leader_addr.empty()) { - return client->SetRes(CmdRes::kErrOther, std::string("-CLUSTERDOWN No Raft leader")); - } - - return client->SetRes(CmdRes::kErrOther, fmt::format("-MOVED {}", leader_addr)); + // 2. If PRAFT is initialized and the current node is not the leader, return a redirection message for write + // commands. + if (HasFlag(kCmdFlagsWrite) && !praft->IsLeader()) { + return client->SetRes(CmdRes::kErrOther, fmt::format("MOVED {}", praft->GetLeaderAddress())); } } diff --git a/src/client.cc b/src/client.cc index 8b3a1fbf8..22422a3fc 100644 --- a/src/client.cc +++ b/src/client.cc @@ -13,16 +13,14 @@ #include #include "fmt/core.h" + #include "praft/praft.h" #include "pstd/log.h" #include "pstd/pstd_string.h" #include "base_cmd.h" #include "config.h" -#include "env.h" #include "pikiwidb.h" -#include "pstd_string.h" -#include "slow_log.h" #include "store.h" namespace pikiwidb { @@ -280,7 +278,7 @@ int PClient::handlePacket(const char* start, int bytes) { if (isPeerMaster()) { if (isClusterCmdTarget()) { // Proccees the packet at one turn. - int len = PRAFT.ProcessClusterCmdResponse(this, start, bytes); // @todo + int len = PSTORE.GetBackend(dbno_)->GetPRaft()->ProcessClusterCmdResponse(this, start, bytes); // @todo if (len > 0) { return len; } @@ -448,7 +446,7 @@ void PClient::OnConnect() { } if (isClusterCmdTarget()) { - PRAFT.SendNodeRequest(this); + PSTORE.GetBackend(dbno_)->GetPRaft()->SendNodeRequest(this); } } else { if (g_config.password.empty()) { @@ -511,7 +509,8 @@ bool PClient::isPeerMaster() const { } bool PClient::isClusterCmdTarget() const { - return PRAFT.GetClusterCmdCtx().GetPeerIp() == PeerIP() && PRAFT.GetClusterCmdCtx().GetPort() == PeerPort(); + auto praft = PSTORE.GetBackend(dbno_)->GetPRaft(); + return praft->GetClusterCmdCtx().GetPeerIp() == PeerIP() && praft->GetClusterCmdCtx().GetPort() == PeerPort(); } uint64_t PClient::uniqueID() const { return GetConnId(); } diff --git a/src/cmd_admin.cc b/src/cmd_admin.cc index be7872747..2eb0f1fce 100644 --- a/src/cmd_admin.cc +++ b/src/cmd_admin.cc @@ -171,6 +171,11 @@ const std::string InfoCmd::kRaftSection = "raft"; InfoCmd::InfoCmd(const std::string& name, int16_t arity) : BaseCmd(name, arity, kCmdFlagsAdmin, kAclCategoryAdmin) {} +// bool InfoCmd::DoInitial(PClient* client) { +// praft_ = PSTORE.GetBackend(client->GetCurrentDB())->GetPRaft(); +// return true; +// } + bool InfoCmd::DoInitial(PClient* client) { size_t argc = client->argv_.size(); if (argc == 1) { @@ -178,7 +183,7 @@ bool InfoCmd::DoInitial(PClient* client) { return true; } - std::string argv_ = client->argv_[1].data(); + std::string argv_ = client->argv_[1]; // convert section to lowercase std::transform(argv_.begin(), argv_.end(), argv_.begin(), [](unsigned char c) { return std::tolower(c); }); if (argc == 2) { @@ -260,23 +265,28 @@ void InfoCmd::DoCmd(PClient* client) { raft_num_voting_nodes:2 raft_node1:id=1733428433,state=connected,voting=yes,addr=localhost,port=5001,last_conn_secs=5,conn_errors=0,conn_oks=1 */ -void InfoCmd::InfoRaft(std::string& message) { - if (!PRAFT.IsInitialized()) { - message += "-ERR Not a cluster member.\r\n"; - return; +void InfoCmd::InfoRaft(PClient* client) { + if (client->argv_.size() != 2) { + return client->SetRes(CmdRes::kWrongNum, client->CmdName()); } - auto node_status = PRAFT.GetNodeStatus(); + praft_ = PSTORE.GetBackend(client->GetCurrentDB())->GetPRaft(); + assert(praft_); + if (!praft_->IsInitialized()) { + return client->SetRes(CmdRes::kErrOther, "Don't already cluster member"); + } + + auto node_status = praft_->GetNodeStatus(); + std::string message; if (node_status.state == braft::State::STATE_END) { message += "-ERR Node is not initialized.\r\n"; return; } std::stringstream tmp_stream; - - tmp_stream << "raft_group_id:" << PRAFT.GetGroupID() << "\r\n"; - tmp_stream << "raft_node_id:" << PRAFT.GetNodeID() << "\r\n"; - tmp_stream << "raft_peer_id:" << PRAFT.GetPeerID() << "\r\n"; + tmp_stream << "raft_group_id:" << praft_->GetGroupID() << "\r\n"; + tmp_stream << "raft_node_id:" << praft_->GetNodeID() << "\r\n"; + tmp_stream << "raft_peer_id:" << praft_->GetPeerID() << "\r\n"; if (braft::is_active_state(node_status.state)) { tmp_stream << "raft_state:up\r\n"; } else { @@ -286,9 +296,9 @@ void InfoCmd::InfoRaft(std::string& message) { tmp_stream << "raft_leader_id:" << node_status.leader_id.to_string() << "\r\n"; tmp_stream << "raft_current_term:" << std::to_string(node_status.term) << "\r\n"; - if (PRAFT.IsLeader()) { + if (praft_->IsLeader()) { std::vector peers; - auto status = PRAFT.GetListPeers(&peers); + auto status = praft_->GetListPeers(&peers); if (!status.ok()) { tmp_stream.str("-ERR "); tmp_stream << status.error_str() << "\r\n"; diff --git a/src/cmd_admin.h b/src/cmd_admin.h index c79c0561e..c65e7711d 100644 --- a/src/cmd_admin.h +++ b/src/cmd_admin.h @@ -166,6 +166,9 @@ class InfoCmd : public BaseCmd { std::string FormatCommandStatLine(const CommandStatistics& stats); double MethodofTotalTimeCalculation(const uint64_t time_consuming); double MethodofCommandStatistics(const uint64_t time_consuming, const uint64_t frequency); + + private: + PRaft* praft_ = nullptr; }; class CmdDebug : public BaseCmdGroup { diff --git a/src/cmd_raft.cc b/src/cmd_raft.cc index e91c93405..8cc12d690 100644 --- a/src/cmd_raft.cc +++ b/src/cmd_raft.cc @@ -16,7 +16,6 @@ #include #include -// #include "net/event_loop.h" #include "praft/praft.h" #include "pstd/log.h" #include "pstd/pstd_string.h" @@ -25,6 +24,7 @@ #include "config.h" #include "pikiwidb.h" #include "replication.h" +#include "store.h" namespace pikiwidb { @@ -39,10 +39,12 @@ bool RaftNodeCmd::DoInitial(PClient* client) { client->SetRes(CmdRes::kErrOther, "RAFT.NODE supports ADD / REMOVE / DOSNAPSHOT only"); return false; } + praft_ = PSTORE.GetBackend(client->GetCurrentDB())->GetPRaft(); return true; } void RaftNodeCmd::DoCmd(PClient* client) { + assert(praft_); auto cmd = client->argv_[1]; pstd::StringToUpper(cmd); if (cmd == kAddCmd) { @@ -58,8 +60,8 @@ void RaftNodeCmd::DoCmd(PClient* client) { void RaftNodeCmd::DoCmdAdd(PClient* client) { // Check whether it is a leader. If it is not a leader, return the leader information - if (!PRAFT.IsLeader()) { - client->SetRes(CmdRes::kWrongLeader, PRAFT.GetLeaderID()); + if (!praft_->IsLeader()) { + client->SetRes(CmdRes::kWrongLeader, praft_->GetLeaderID()); return; } @@ -70,7 +72,7 @@ void RaftNodeCmd::DoCmdAdd(PClient* client) { // RedisRaft has nodeid, but in Braft, NodeId is IP:Port. // So we do not need to parse and use nodeid like redis; - auto s = PRAFT.AddPeer(client->argv_[3]); + auto s = praft_->AddPeer(client->argv_[3]); if (s.ok()) { client->SetRes(CmdRes::kOK); } else { @@ -80,7 +82,7 @@ void RaftNodeCmd::DoCmdAdd(PClient* client) { void RaftNodeCmd::DoCmdRemove(PClient* client) { // If the node has been initialized, it needs to close the previous initialization and rejoin the other group - if (!PRAFT.IsInitialized()) { + if (!praft_->IsInitialized()) { client->SetRes(CmdRes::kErrOther, "Don't already cluster member"); return; } @@ -91,9 +93,9 @@ void RaftNodeCmd::DoCmdRemove(PClient* client) { } // Check whether it is a leader. If it is not a leader, send remove request to leader - if (!PRAFT.IsLeader()) { + if (!praft_->IsLeader()) { // Get the leader information - braft::PeerId leader_peer_id(PRAFT.GetLeaderID()); + braft::PeerId leader_peer_id(praft_->GetLeaderID()); // @todo There will be an unreasonable address, need to consider how to deal with it if (leader_peer_id.is_empty()) { client->SetRes(CmdRes::kErrOther, @@ -106,11 +108,11 @@ void RaftNodeCmd::DoCmdRemove(PClient* client) { auto port = leader_peer_id.addr.port - pikiwidb::g_config.raft_port_offset; auto peer_id = client->argv_[2]; auto ret = - PRAFT.GetClusterCmdCtx().Set(ClusterCmdType::kRemove, client, std::move(peer_ip), port, std::move(peer_id)); + praft_->GetClusterCmdCtx().Set(ClusterCmdType::kRemove, client, std::move(peer_ip), port, std::move(peer_id)); if (!ret) { // other clients have removed return client->SetRes(CmdRes::kErrOther, "Other clients have removed"); } - PRAFT.GetClusterCmdCtx().ConnectTargetNode(); + praft_->GetClusterCmdCtx().ConnectTargetNode(); INFO("Sent remove request to leader successfully"); // Not reply any message here, we will reply after the connection is established. @@ -118,7 +120,7 @@ void RaftNodeCmd::DoCmdRemove(PClient* client) { return; } - auto s = PRAFT.RemovePeer(client->argv_[2]); + auto s = praft_->RemovePeer(client->argv_[2]); if (s.ok()) { client->SetRes(CmdRes::kOK); } else { @@ -129,7 +131,7 @@ void RaftNodeCmd::DoCmdRemove(PClient* client) { void RaftNodeCmd::DoCmdSnapshot(PClient* client) { auto self_snapshot_index = PSTORE.GetBackend(client->GetCurrentDB())->GetStorage()->GetSmallestFlushedLogIndex(); INFO("DoCmdSnapshot self_snapshot_index:{}", self_snapshot_index); - auto s = PRAFT.DoSnapshot(self_snapshot_index); + auto s = praft_->DoSnapshot(self_snapshot_index); if (s.ok()) { client->SetRes(CmdRes::kOK); } @@ -145,11 +147,13 @@ bool RaftClusterCmd::DoInitial(PClient* client) { client->SetRes(CmdRes::kErrOther, "RAFT.CLUSTER supports INIT/JOIN only"); return false; } + praft_ = PSTORE.GetBackend(client->GetCurrentDB())->GetPRaft(); return true; } void RaftClusterCmd::DoCmd(PClient* client) { - if (PRAFT.IsInitialized()) { + assert(praft_); + if (praft_->IsInitialized()) { return client->SetRes(CmdRes::kErrOther, "Already cluster member"); } @@ -177,7 +181,7 @@ void RaftClusterCmd::DoCmdInit(PClient* client) { } else { cluster_id = pstd::RandomHexChars(RAFT_GROUPID_LEN); } - auto s = PRAFT.Init(cluster_id, false); + auto s = praft_->Init(cluster_id, false); if (!s.ok()) { return client->SetRes(CmdRes::kErrOther, fmt::format("Failed to init node: ", s.error_str())); } @@ -197,7 +201,7 @@ static inline std::optional> GetIpAndPortFromEnd void RaftClusterCmd::DoCmdJoin(PClient* client) { // If the node has been initialized, it needs to close the previous initialization and rejoin the other group - if (PRAFT.IsInitialized()) { + if (praft_->IsInitialized()) { return client->SetRes(CmdRes::kErrOther, "A node that has been added to a cluster must be removed \ from the old cluster before it can be added to the new cluster"); @@ -224,11 +228,11 @@ void RaftClusterCmd::DoCmdJoin(PClient* client) { auto& [peer_ip, port] = *ip_port; // Connect target - auto ret = PRAFT.GetClusterCmdCtx().Set(ClusterCmdType::kJoin, client, std::move(peer_ip), port); + auto ret = praft_->GetClusterCmdCtx().Set(ClusterCmdType::kJoin, client, std::move(peer_ip), port); if (!ret) { // other clients have joined return client->SetRes(CmdRes::kErrOther, "Other clients have joined"); } - PRAFT.GetClusterCmdCtx().ConnectTargetNode(); + praft_->GetClusterCmdCtx().ConnectTargetNode(); INFO("Sent join request to leader successfully"); // Not reply any message here, we will reply after the connection is established. diff --git a/src/cmd_raft.h b/src/cmd_raft.h index 27ae19d16..ad5898a31 100644 --- a/src/cmd_raft.h +++ b/src/cmd_raft.h @@ -15,6 +15,7 @@ #include "base_cmd.h" namespace pikiwidb { +class PRaft; /* RAFT.NODE ADD [id] [address:port] * Add a new node to the cluster. The [id] can be an explicit non-zero value, @@ -50,6 +51,9 @@ class RaftNodeCmd : public BaseCmd { void DoCmdRemove(PClient *client); void DoCmdSnapshot(PClient *client); +private: + PRaft* praft_ = nullptr; + static constexpr std::string_view kAddCmd = "ADD"; static constexpr std::string_view kRemoveCmd = "REMOVE"; static constexpr std::string_view kDoSnapshot = "DOSNAPSHOT"; @@ -79,6 +83,9 @@ class RaftClusterCmd : public BaseCmd { void DoCmdInit(PClient *client); void DoCmdJoin(PClient *client); +private: + PRaft* praft_ = nullptr; + static constexpr std::string_view kInitCmd = "INIT"; static constexpr std::string_view kJoinCmd = "JOIN"; }; diff --git a/src/db.cc b/src/db.cc index e736c594e..09fe15ed7 100644 --- a/src/db.cc +++ b/src/db.cc @@ -9,7 +9,6 @@ */ #include "db.h" -#include #include "config.h" #include "praft/praft.h" @@ -28,21 +27,18 @@ rocksdb::Status DB::Open() { storage::StorageOptions storage_options; storage_options.options = g_config.GetRocksDBOptions(); storage_options.table_options = g_config.GetRocksDBBlockBasedTableOptions(); - storage_options.options.ttl = g_config.rocksdb_ttl_second.load(std::memory_order_relaxed); storage_options.options.periodic_compaction_seconds = g_config.rocksdb_periodic_second.load(std::memory_order_relaxed); - storage_options.small_compaction_threshold = g_config.small_compaction_threshold.load(); storage_options.small_compaction_duration_threshold = g_config.small_compaction_duration_threshold.load(); if (g_config.use_raft.load(std::memory_order_relaxed)) { - storage_options.append_log_function = [&r = PRAFT](const Binlog& log, std::promise&& promise) { + storage_options.append_log_function = [&r = *praft_](const Binlog& log, std::promise&& promise) { r.AppendLog(log, std::move(promise)); }; - storage_options.do_snapshot_function = [raft = &pikiwidb::PRAFT](auto&& self_snapshot_index, auto&& is_sync) { - raft->DoSnapshot(std::forward(self_snapshot_index), - std::forward(is_sync)); + storage_options.do_snapshot_function = [&r = *praft_](int64_t self_snapshot_index, bool is_sync) { + return r.DoSnapshot(self_snapshot_index, is_sync); }; } @@ -120,11 +116,12 @@ void DB::LoadDBFromCheckpoint(const std::string& checkpoint_path, bool sync [[ma storage_options.options.periodic_compaction_seconds = g_config.rocksdb_periodic_second.load(std::memory_order_relaxed); if (g_config.use_raft.load(std::memory_order_relaxed)) { - storage_options.append_log_function = [&r = PRAFT](const Binlog& log, std::promise&& promise) { + storage_options.append_log_function = [&r = *praft_](const Binlog& log, std::promise&& promise) { r.AppendLog(log, std::move(promise)); }; - storage_options.do_snapshot_function = - std::bind(&pikiwidb::PRaft::DoSnapshot, &pikiwidb::PRAFT, std::placeholders::_1, std::placeholders::_2); + storage_options.do_snapshot_function = [&r = *praft_](int64_t self_snapshot_index, bool is_sync) { + return r.DoSnapshot(self_snapshot_index, is_sync); + }; } if (auto s = storage_->Open(storage_options, db_path_); !s.ok()) { diff --git a/src/db.h b/src/db.h index 7326606c2..387d65192 100644 --- a/src/db.h +++ b/src/db.h @@ -10,11 +10,10 @@ #pragma once -#include +#include #include -#include "pstd/log.h" -#include "pstd/noncopyable.h" +#include "praft/praft.h" #include "storage/storage.h" namespace pikiwidb { @@ -42,6 +41,8 @@ class DB { int GetDbIndex() { return db_index_; } + PRaft* GetPRaft() { return praft_.get(); } + private: const int db_index_ = 0; const std::string db_path_; @@ -53,6 +54,7 @@ class DB { */ std::shared_mutex storage_mutex_; std::unique_ptr storage_; + std::unique_ptr praft_{std::make_unique()}; bool opened_ = false; }; diff --git a/src/pikiwidb.cc b/src/pikiwidb.cc index 93f8f1a11..067082f6f 100644 --- a/src/pikiwidb.cc +++ b/src/pikiwidb.cc @@ -240,9 +240,6 @@ void PikiwiDB::Run() { } void PikiwiDB::Stop() { - pikiwidb::PRAFT.ShutDown(); - pikiwidb::PRAFT.Join(); - pikiwidb::PRAFT.Clear(); cmd_threads_.Stop(); event_server_->StopServer(); } diff --git a/src/praft/praft.cc b/src/praft/praft.cc index f0653427a..41135a26d 100644 --- a/src/praft/praft.cc +++ b/src/praft/praft.cc @@ -72,25 +72,20 @@ void ClusterCmdContext::ConnectTargetNode() { auto ip = PREPL.GetMasterAddr().GetIP(); auto port = PREPL.GetMasterAddr().GetPort(); if (ip == peer_ip_ && port == port_ && PREPL.GetMasterState() == kPReplStateConnected) { - PRAFT.SendNodeRequest(PREPL.GetMaster()); + praft_->SendNodeRequest(PREPL.GetMaster()); return; } // reconnect auto fail_cb = [&](const std::string& err) { INFO("Failed to connect to cluster node, err: {}", err); - PRAFT.OnClusterCmdConnectionFailed(err); + praft_->OnClusterCmdConnectionFailed(err); }; PREPL.SetFailCallback(fail_cb); PREPL.SetMasterState(kPReplStateNone); PREPL.SetMasterAddr(peer_ip_.c_str(), port_); } -PRaft& PRaft::Instance() { - static PRaft store; - return store; -} - butil::Status PRaft::Init(std::string& group_id, bool initial_conf_is_null) { if (node_ && server_) { return {0, "OK"}; @@ -99,7 +94,7 @@ butil::Status PRaft::Init(std::string& group_id, bool initial_conf_is_null) { server_ = std::make_unique(); auto port = g_config.port + pikiwidb::g_config.raft_port_offset; // Add your service into RPC server - DummyServiceImpl service(&PRAFT); + DummyServiceImpl service(this); if (server_->AddService(&service, brpc::SERVER_DOESNT_OWN_SERVICE) != 0) { server_.reset(); return ERROR_LOG_AND_STATUS("Failed to add service"); @@ -349,10 +344,10 @@ int PRaft::ProcessClusterCmdResponse(PClient* client, const char* start, int len int ret = 0; switch (cluster_cmd_type) { case ClusterCmdType::kJoin: - ret = PRAFT.ProcessClusterJoinCmdResponse(client, start, len); + ret = ProcessClusterJoinCmdResponse(client, start, len); break; case ClusterCmdType::kRemove: - ret = PRAFT.ProcessClusterRemoveCmdResponse(client, start, len); + ret = ProcessClusterRemoveCmdResponse(client, start, len); break; default: client->SetRes(CmdRes::kErrOther, "RAFT.CLUSTER response supports JOIN/REMOVE only"); @@ -419,14 +414,14 @@ void PRaft::LeaderRedirection(PClient* join_client, const std::string& reply) { // Reset the target of the connection cluster_cmd_ctx_.Clear(); - auto ret = PRAFT.GetClusterCmdCtx().Set(ClusterCmdType::kJoin, join_client, std::move(peer_ip), port); + auto ret = GetClusterCmdCtx().Set(ClusterCmdType::kJoin, join_client, std::move(peer_ip), port); if (!ret) { // other clients have joined join_client->SetRes(CmdRes::kErrOther, "Other clients have joined"); join_client->SendPacket(); // join_client->Clear(); return; } - PRAFT.GetClusterCmdCtx().ConnectTargetNode(); + GetClusterCmdCtx().ConnectTargetNode(); // Not reply any message here, we will reply after the connection is established. join_client->Clear(); @@ -441,7 +436,7 @@ void PRaft::InitializeNodeBeforeAdd(PClient* client, PClient* join_client, const if (group_id_end != std::string::npos) { std::string raft_group_id = reply.substr(group_id_start, group_id_end - group_id_start); // initialize the slave node - auto s = PRAFT.Init(raft_group_id, true); + auto s = Init(raft_group_id, true); if (!s.ok()) { join_client->SetRes(CmdRes::kErrOther, s.error_str()); join_client->SendPacket(); @@ -451,7 +446,7 @@ void PRaft::InitializeNodeBeforeAdd(PClient* client, PClient* join_client, const return; } - PRAFT.SendNodeAddRequest(client); + SendNodeAddRequest(client); } else { ERROR("Joined Raft cluster fail, because of invalid raft_group_id"); join_client->SetRes(CmdRes::kErrOther, "Invalid raft_group_id"); @@ -472,7 +467,7 @@ int PRaft::ProcessClusterJoinCmdResponse(PClient* client, const char* start, int std::string reply(start, len); if (reply.find(OK_STR) != std::string::npos) { - INFO("Joined Raft cluster, node id: {}, group_id: {}", PRAFT.GetNodeID(), PRAFT.group_id_); + INFO("Joined Raft cluster, node id: {}, group_id: {}", GetNodeID(), group_id_); join_client->SetRes(CmdRes::kOK); join_client->SendPacket(); // join_client->Clear(); @@ -506,7 +501,7 @@ int PRaft::ProcessClusterRemoveCmdResponse(PClient* client, const char* start, i std::string reply(start, len); if (reply.find(OK_STR) != std::string::npos) { - INFO("Removed Raft cluster, node id: {}, group_id: {}", PRAFT.GetNodeID(), PRAFT.group_id_); + INFO("Removed Raft cluster, node id: {}, group_id: {}", GetNodeID(), group_id_); ShutDown(); Join(); Clear(); diff --git a/src/praft/praft.h b/src/praft/praft.h index 1d2148af7..025beab0f 100644 --- a/src/praft/praft.h +++ b/src/praft/praft.h @@ -7,11 +7,9 @@ #pragma once -#include #include #include #include -#include #include #include "braft/file_system_adaptor.h" @@ -34,10 +32,9 @@ namespace pikiwidb { #define RAFT_GROUP_ID "raft_group_id:" #define NOT_LEADER "Not leader" -#define PRAFT PRaft::Instance() - -// class EventLoop; +class EventLoop; class Binlog; +class PRaft; enum ClusterCmdType { kNone, @@ -46,10 +43,8 @@ enum ClusterCmdType { }; class ClusterCmdContext { - friend class PRaft; - public: - ClusterCmdContext() = default; + ClusterCmdContext(PRaft* raft) : praft_(raft) {} ~ClusterCmdContext() = default; bool Set(ClusterCmdType cluster_cmd_type, PClient* client, std::string&& peer_ip, int port, @@ -69,6 +64,7 @@ class ClusterCmdContext { void ConnectTargetNode(); private: + PRaft* praft_; ClusterCmdType cluster_cmd_type_ = ClusterCmdType::kNone; std::mutex mtx_; PClient* client_ = nullptr; @@ -95,9 +91,11 @@ class PRaftWriteDoneClosure : public braft::Closure { class PRaft : public braft::StateMachine { public: PRaft() = default; - ~PRaft() override = default; - - static PRaft& Instance(); + ~PRaft() override { + ShutDown(); + Join(); + Clear(); + } //===--------------------------------------------------------------------===// // Braft API diff --git a/tests/consistency_test.go b/tests/consistency_test.go index ccecce14d..c60d8b670 100644 --- a/tests/consistency_test.go +++ b/tests/consistency_test.go @@ -898,12 +898,12 @@ var _ = Describe("Consistency", Ordered, func() { } { // set write on leader - set, err := leader.SetEx(ctx, testKey, testValue, 3).Result() + set, err := leader.SetEx(ctx, testKey, testValue, 3*time.Second).Result() Expect(err).NotTo(HaveOccurred()) Expect(set).To(Equal("OK")) // read check - time.Sleep(10 * time.Second) + time.Sleep(5 * time.Second) readChecker(func(c *redis.Client) { _, err := c.Get(ctx, testKey).Result() Expect(err).To(Equal(redis.Nil)) From 62415c0fe0399bd98fad092771d01c14712d2a6c Mon Sep 17 00:00:00 2001 From: longfar Date: Fri, 28 Jun 2024 09:26:00 +0800 Subject: [PATCH 02/19] fix --- etc/conf/pikiwidb.conf | 4 ++-- src/cmd_raft.cc | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/etc/conf/pikiwidb.conf b/etc/conf/pikiwidb.conf index 38d7371cc..72f8fbaa5 100644 --- a/etc/conf/pikiwidb.conf +++ b/etc/conf/pikiwidb.conf @@ -35,7 +35,7 @@ logfile stdout # Set the number of databases. The default database is DB 0, you can select # a different one on a per-connection basis using SELECT where # dbid is a number between 0 and 'databases'-1 -databases 16 +databases 1 ################################ SNAPSHOTTING ################################# # @@ -343,6 +343,6 @@ rocksdb-ttl-second 604800 rocksdb-periodic-second 259200; ############################### RAFT ############################### -use-raft no +use-raft yes # Braft relies on brpc to communicate via the default port number plus the port offset raft-port-offset 10 diff --git a/src/cmd_raft.cc b/src/cmd_raft.cc index 8cc12d690..194eea1ff 100644 --- a/src/cmd_raft.cc +++ b/src/cmd_raft.cc @@ -183,7 +183,7 @@ void RaftClusterCmd::DoCmdInit(PClient* client) { } auto s = praft_->Init(cluster_id, false); if (!s.ok()) { - return client->SetRes(CmdRes::kErrOther, fmt::format("Failed to init node: ", s.error_str())); + return client->SetRes(CmdRes::kErrOther, fmt::format("Failed to init node: {}", s.error_str())); } client->SetRes(CmdRes::kOK); } From 450c1f284ea98241a67ad0f3e99a60811651355d Mon Sep 17 00:00:00 2001 From: longfar Date: Fri, 28 Jun 2024 17:41:13 +0800 Subject: [PATCH 03/19] feat: move rpc server to PStore from PRaft --- src/praft/praft.cc | 97 +++++++++++----------------------------------- src/praft/praft.h | 7 ++-- src/store.cc | 16 ++++++-- src/store.h | 13 +++++-- 4 files changed, 48 insertions(+), 85 deletions(-) diff --git a/src/praft/praft.cc b/src/praft/praft.cc index 41135a26d..3611a7628 100644 --- a/src/praft/praft.cc +++ b/src/praft/praft.cc @@ -10,9 +10,8 @@ #include #include "braft/raft.h" -#include "braft/snapshot.h" #include "braft/util.h" -#include "brpc/server.h" +#include "butil/endpoint.h" #include "gflags/gflags.h" #include "pstd/log.h" @@ -86,51 +85,11 @@ void ClusterCmdContext::ConnectTargetNode() { PREPL.SetMasterAddr(peer_ip_.c_str(), port_); } -butil::Status PRaft::Init(std::string& group_id, bool initial_conf_is_null) { - if (node_ && server_) { +butil::Status PRaft::Init(const std::string& group_id, bool initial_conf_is_null) { + if (node_) { return {0, "OK"}; } - server_ = std::make_unique(); - auto port = g_config.port + pikiwidb::g_config.raft_port_offset; - // Add your service into RPC server - DummyServiceImpl service(this); - if (server_->AddService(&service, brpc::SERVER_DOESNT_OWN_SERVICE) != 0) { - server_.reset(); - return ERROR_LOG_AND_STATUS("Failed to add service"); - } - // raft can share the same RPC server. Notice the second parameter, because - // adding services into a running server is not allowed and the listen - // address of this server is impossible to get before the server starts. You - // have to specify the address of the server. - if (braft::add_service(server_.get(), port) != 0) { - server_.reset(); - return ERROR_LOG_AND_STATUS("Failed to add raft service"); - } - - // It's recommended to start the server before Counter is started to avoid - // the case that it becomes the leader while the service is unreacheable by - // clients. - // Notice the default options of server is used here. Check out details from - // the doc of brpc if you would like change some option; - if (server_->Start(port, nullptr) != 0) { - server_.reset(); - return ERROR_LOG_AND_STATUS("Failed to start server"); - } - // It's ok to start PRaft; - assert(group_id.size() == RAFT_GROUPID_LEN); - this->group_id_ = group_id; - - // FIXME: g_config.ip is default to 127.0.0.0, which may not work in cluster. - raw_addr_ = g_config.ip.ToString() + ":" + std::to_string(port); - butil::ip_t ip; - auto ret = butil::str2ip(g_config.ip.ToString().c_str(), &ip); - if (ret != 0) { - server_.reset(); - return ERROR_LOG_AND_STATUS("Failed to convert str_ip to butil::ip_t"); - } - butil::EndPoint addr(ip, port); - // Default init in one node. // initial_conf takes effect only when the replication group is started from an empty node. // The Configuration is restored from the snapshot and log files when the data in the replication group is not empty. @@ -139,37 +98,39 @@ butil::Status PRaft::Init(std::string& group_id, bool initial_conf_is_null) { // Set initial_conf to empty for other nodes. // You can also start empty nodes simultaneously by setting the same inital_conf(ip:port of multiple nodes) for // multiple nodes. - std::string initial_conf; + braft::NodeOptions node_options; if (!initial_conf_is_null) { - initial_conf = raw_addr_ + ":0,"; - } - if (node_options_.initial_conf.parse_from(initial_conf) != 0) { - server_.reset(); - return ERROR_LOG_AND_STATUS("Failed to parse configuration"); + auto endpoint_str = butil::endpoint2str(PSTORE.GetEndPoint()); + std::string initial_conf = fmt::format("{}:0,", endpoint_str.c_str()); + if (node_options.initial_conf.parse_from(initial_conf) != 0) { + return ERROR_LOG_AND_STATUS("Failed to parse configuration"); + } } + node_options.fsm = this; + node_options.node_owns_fsm = false; + node_options.snapshot_interval_s = 0; + auto prefix = fmt::format("local://{}{}/{}", g_config.db_path.ToString(), "_praft", db_id_); + node_options.log_uri = prefix + "/log"; + node_options.raft_meta_uri = prefix + "/raft_meta"; + node_options.snapshot_uri = prefix + "/snapshot"; + snapshot_adaptor_ = new PPosixFileSystemAdaptor(); + node_options.snapshot_file_system_adaptor = &snapshot_adaptor_; + // node_options_.election_timeout_ms = FLAGS_election_timeout_ms; - node_options_.fsm = this; - node_options_.node_owns_fsm = false; - node_options_.snapshot_interval_s = 0; - std::string prefix = "local://" + g_config.db_path.ToString() + std::to_string(db_id_) + "/_praft"; - node_options_.log_uri = prefix + "/log"; - node_options_.raft_meta_uri = prefix + "/raft_meta"; - node_options_.snapshot_uri = prefix + "/snapshot"; // node_options_.disable_cli = FLAGS_disable_cli; - snapshot_adaptor_ = new PPosixFileSystemAdaptor(); - node_options_.snapshot_file_system_adaptor = &snapshot_adaptor_; - node_ = std::make_unique("pikiwidb", braft::PeerId(addr)); // group_id - if (node_->init(node_options_) != 0) { - server_.reset(); + node_ = std::make_unique(group_id, braft::PeerId(PSTORE.GetEndPoint())); // group_id + if (node_->init(node_options) != 0) { node_.reset(); return ERROR_LOG_AND_STATUS("Failed to init raft node"); } + group_id_ = group_id; // enable leader lease braft::FLAGS_raft_enable_leader_lease = true; + INFO("Initialized praft successfully: node_id={}", GetNodeID()); return {0, "OK"}; } @@ -594,10 +555,6 @@ void PRaft::ShutDown() { if (node_) { node_->shutdown(nullptr); } - - if (server_) { - server_->Stop(0); - } } // Blocking this thread until the node is eventually down. @@ -605,10 +562,6 @@ void PRaft::Join() { if (node_) { node_->join(); } - - if (server_) { - server_->Join(); - } } void PRaft::AppendLog(const Binlog& log, std::promise&& promise) { @@ -634,10 +587,6 @@ void PRaft::Clear() { if (node_) { node_.reset(); } - - if (server_) { - server_.reset(); - } } void PRaft::on_apply(braft::Iterator& iter) { diff --git a/src/praft/praft.h b/src/praft/praft.h index 025beab0f..41a91b15d 100644 --- a/src/praft/praft.h +++ b/src/praft/praft.h @@ -100,7 +100,7 @@ class PRaft : public braft::StateMachine { //===--------------------------------------------------------------------===// // Braft API //===--------------------------------------------------------------------===// - butil::Status Init(std::string& group_id, bool initial_conf_is_null); + butil::Status Init(const std::string& group_id, bool initial_conf_is_null); butil::Status AddPeer(const std::string& peer); butil::Status RemovePeer(const std::string& peer); butil::Status DoSnapshot(int64_t self_snapshot_index = 0, bool is_sync = true); @@ -140,7 +140,7 @@ class PRaft : public braft::StateMachine { storage::LogIndex GetTerm(uint64_t log_index); storage::LogIndex GetLastLogIndex(bool is_flush = false); - bool IsInitialized() const { return node_ != nullptr && server_ != nullptr; } + bool IsInitialized() const { return node_ != nullptr; } private: void on_apply(braft::Iterator& iter) override; @@ -157,14 +157,13 @@ class PRaft : public braft::StateMachine { void on_start_following(const ::braft::LeaderChangeContext& ctx) override; private: - std::unique_ptr server_{nullptr}; // brpc std::unique_ptr node_{nullptr}; butil::atomic leader_term_ = -1; braft::NodeOptions node_options_; // options for raft node std::string raw_addr_; // ip:port of this node scoped_refptr snapshot_adaptor_ = nullptr; - ClusterCmdContext cluster_cmd_ctx_; // context for cluster join/remove command + ClusterCmdContext cluster_cmd_ctx_{this}; // context for cluster join/remove command std::string group_id_; // group id int db_id_ = 0; // db_id diff --git a/src/store.cc b/src/store.cc index 6f0500091..2ba57ae58 100644 --- a/src/store.cc +++ b/src/store.cc @@ -10,9 +10,6 @@ #include "store.h" -#include -#include - #include "config.h" #include "db.h" #include "pstd/log.h" @@ -36,6 +33,19 @@ void PStore::Init(int db_number) { backends_.push_back(std::move(db)); INFO("Open DB_{} success!", i); } + auto ip = g_config.ip.ToString(); + butil::ip_t rpc_ip; + butil::str2ip(ip.c_str(), &rpc_ip); + auto rpc_port = + g_config.port.load(std::memory_order_relaxed) + g_config.raft_port_offset.load(std::memory_order_relaxed); + endpoint_ = butil::EndPoint(rpc_ip, rpc_port); + if (braft::add_service(GetRpcServer(), endpoint_) != 0) { + return ERROR("Failed to add raft service to rpc server"); + } + if (rpc_server_->Start(endpoint_, nullptr) != 0) { + return ERROR("Failed to start rpc server"); + } + INFO("Started RPC server successfully"); INFO("STORE Init success!"); } diff --git a/src/store.h b/src/store.h index 4383cb392..125ca30f7 100644 --- a/src/store.h +++ b/src/store.h @@ -12,13 +12,13 @@ #define GLOG_NO_ABBREVIATED_SEVERITIES -#include -#include +#include #include -#include "common.h" +#include "brpc/server.h" +#include "butil/endpoint.h" + #include "db.h" -#include "storage/storage.h" namespace pikiwidb { @@ -59,11 +59,16 @@ class PStore { void HandleTaskSpecificDB(const TasksVector& tasks); int GetDBNumber() const { return db_number_; } + brpc::Server* GetRpcServer() const { return rpc_server_.get(); } + const butil::EndPoint& GetEndPoint() const { return endpoint_; } private: PStore() = default; + int db_number_ = 0; std::vector> backends_; + butil::EndPoint endpoint_; + std::unique_ptr rpc_server_{std::make_unique()}; }; #define PSTORE PStore::Instance() From e5bd3902a80351dba59e7300ed305d04f89661d5 Mon Sep 17 00:00:00 2001 From: longfar Date: Mon, 8 Jul 2024 19:45:33 +0800 Subject: [PATCH 04/19] tmp --- src/cmd_raft.cc | 44 +++++++++++++++++++++------------- src/cmd_raft.h | 15 ++++++------ src/praft/praft.cc | 60 ++++++++++++++++++++++++---------------------- src/praft/praft.h | 2 +- src/store.h | 17 +++++++++++++ 5 files changed, 85 insertions(+), 53 deletions(-) diff --git a/src/cmd_raft.cc b/src/cmd_raft.cc index 194eea1ff..6a6c8ff6b 100644 --- a/src/cmd_raft.cc +++ b/src/cmd_raft.cc @@ -39,6 +39,7 @@ bool RaftNodeCmd::DoInitial(PClient* client) { client->SetRes(CmdRes::kErrOther, "RAFT.NODE supports ADD / REMOVE / DOSNAPSHOT only"); return false; } + group_id_ = client->argv_[2]; praft_ = PSTORE.GetBackend(client->GetCurrentDB())->GetPRaft(); return true; } @@ -52,6 +53,7 @@ void RaftNodeCmd::DoCmd(PClient* client) { } else if (cmd == kRemoveCmd) { DoCmdRemove(client); } else if (cmd == kDoSnapshot) { + assert(0); // TODO(longfar): add group id in arguments DoCmdSnapshot(client); } else { client->SetRes(CmdRes::kErrOther, "RAFT.NODE supports ADD / REMOVE / DOSNAPSHOT only"); @@ -59,8 +61,11 @@ void RaftNodeCmd::DoCmd(PClient* client) { } void RaftNodeCmd::DoCmdAdd(PClient* client) { + auto db = PSTORE.GetDBByGroupID(group_id_); + assert(db); + auto praft = db->GetPRaft(); // Check whether it is a leader. If it is not a leader, return the leader information - if (!praft_->IsLeader()) { + if (!praft->IsLeader()) { client->SetRes(CmdRes::kWrongLeader, praft_->GetLeaderID()); return; } @@ -72,7 +77,7 @@ void RaftNodeCmd::DoCmdAdd(PClient* client) { // RedisRaft has nodeid, but in Braft, NodeId is IP:Port. // So we do not need to parse and use nodeid like redis; - auto s = praft_->AddPeer(client->argv_[3]); + auto s = praft->AddPeer(client->argv_[3]); if (s.ok()) { client->SetRes(CmdRes::kOK); } else { @@ -171,21 +176,22 @@ void RaftClusterCmd::DoCmdInit(PClient* client) { return client->SetRes(CmdRes::kWrongNum, client->CmdName()); } - std::string cluster_id; + std::string group_id; if (client->argv_.size() == 3) { - cluster_id = client->argv_[2]; - if (cluster_id.size() != RAFT_GROUPID_LEN) { + group_id = client->argv_[2]; + if (group_id.size() != RAFT_GROUPID_LEN) { return client->SetRes(CmdRes::kInvalidParameter, "Cluster id must be " + std::to_string(RAFT_GROUPID_LEN) + " characters"); } } else { - cluster_id = pstd::RandomHexChars(RAFT_GROUPID_LEN); + group_id = pstd::RandomHexChars(RAFT_GROUPID_LEN); } - auto s = praft_->Init(cluster_id, false); + auto s = praft_->Init(group_id, false); if (!s.ok()) { return client->SetRes(CmdRes::kErrOther, fmt::format("Failed to init node: {}", s.error_str())); } - client->SetRes(CmdRes::kOK); + PSTORE.AddRegion(praft_->GetGroupID(), client->GetCurrentDB()); + client->SetLineString(fmt::format("+OK {}", group_id)); } static inline std::optional> GetIpAndPortFromEndPoint(const std::string& endpoint) { @@ -207,16 +213,22 @@ void RaftClusterCmd::DoCmdJoin(PClient* client) { from the old cluster before it can be added to the new cluster"); } - if (client->argv_.size() < 3) { - return client->SetRes(CmdRes::kWrongNum, client->CmdName()); - } + // if (client->argv_.size() < 3) { + // return client->SetRes(CmdRes::kWrongNum, client->CmdName()); + // } - // (KKorpse)TODO: Support multiple nodes join at the same time. - if (client->argv_.size() > 3) { - return client->SetRes(CmdRes::kInvalidParameter, "Too many arguments"); - } + // // (KKorpse)TODO: Support multiple nodes join at the same time. + // if (client->argv_.size() > 3) { + // return client->SetRes(CmdRes::kInvalidParameter, "Too many arguments"); + // } + assert(client->argv_.size() == 4); + auto group_id = client->argv_[2]; + auto addr = client->argv_[3]; + + // init raft + auto s = praft_->Init(group_id, true); + assert(s.ok()); - auto addr = client->argv_[2]; if (braft::PeerId(addr).is_empty()) { return client->SetRes(CmdRes::kErrOther, fmt::format("Invalid ip::port: {}", addr)); } diff --git a/src/cmd_raft.h b/src/cmd_raft.h index ad5898a31..eedbfab0d 100644 --- a/src/cmd_raft.h +++ b/src/cmd_raft.h @@ -17,7 +17,8 @@ namespace pikiwidb { class PRaft; -/* RAFT.NODE ADD [id] [address:port] +/* + * RAFT.NODE ADD [id] [address:port] * Add a new node to the cluster. The [id] can be an explicit non-zero value, * or zero to let the cluster choose one. * Reply: @@ -53,21 +54,21 @@ class RaftNodeCmd : public BaseCmd { private: PRaft* praft_ = nullptr; + std::string group_id_; static constexpr std::string_view kAddCmd = "ADD"; static constexpr std::string_view kRemoveCmd = "REMOVE"; static constexpr std::string_view kDoSnapshot = "DOSNAPSHOT"; }; -/* RAFT.CLUSTER INIT +/* + * RAFT.CLUSTER INIT [group_id] * Initializes a new Raft cluster. - * is an optional 32 character string, if set, cluster will use it for the id * Reply: - * +OK [group_id] + * +OK * - * RAFT.CLUSTER JOIN [addr:port] - * Join an existing cluster. - * The operation is asynchronous and may take place/retry in the background. + * RAFT.CLUSTER JOIN + * Join an existing cluster. The operation is asynchronous and may take place/retry in the background. * Reply: * +OK */ diff --git a/src/praft/praft.cc b/src/praft/praft.cc index 3611a7628..e13dcae69 100644 --- a/src/praft/praft.cc +++ b/src/praft/praft.cc @@ -251,9 +251,10 @@ void PRaft::SendNodeRequest(PClient* client) { auto cluster_cmd_type = cluster_cmd_ctx_.GetClusterCmdType(); switch (cluster_cmd_type) { - case ClusterCmdType::kJoin: - SendNodeInfoRequest(client, "DATA"); - break; + case ClusterCmdType::kJoin: { + // SendNodeInfoRequest(client, "DATA"); + SendNodeAddRequest(client); + } break; case ClusterCmdType::kRemove: SendNodeRemoveRequest(client); break; @@ -276,17 +277,11 @@ void PRaft::SendNodeAddRequest(PClient* client) { assert(client); // Node id in braft are ip:port, the node id param in RAFT.NODE ADD cmd will be ignored. - int unused_node_id = 0; auto port = g_config.port + pikiwidb::g_config.raft_port_offset; auto raw_addr = g_config.ip.ToString() + ":" + std::to_string(port); - UnboundedBuffer req; - req.PushData("RAFT.NODE ADD ", 14); - req.PushData(std::to_string(unused_node_id).c_str(), std::to_string(unused_node_id).size()); - req.PushData(" ", 1); - req.PushData(raw_addr.data(), raw_addr.size()); - req.PushData("\r\n", 2); - client->SendPacket(req); - // client->Clear(); + auto msg = fmt::format("RAFT.NODE ADD {} {}\r\n", group_id_, raw_addr); + client->SendPacket(std::move(msg)); + client->Clear(); } void PRaft::SendNodeRemoveRequest(PClient* client) { @@ -388,34 +383,36 @@ void PRaft::LeaderRedirection(PClient* join_client, const std::string& reply) { join_client->Clear(); } -void PRaft::InitializeNodeBeforeAdd(PClient* client, PClient* join_client, const std::string& reply) { +bool PRaft::InitializeNodeBeforeAdd(PClient* client, PClient* join_client, const std::string& reply) { std::string prefix = RAFT_GROUP_ID; std::string::size_type prefix_length = prefix.length(); std::string::size_type group_id_start = reply.find(prefix); group_id_start += prefix_length; // locate the start location of "raft_group_id" std::string::size_type group_id_end = reply.find("\r\n", group_id_start); - if (group_id_end != std::string::npos) { - std::string raft_group_id = reply.substr(group_id_start, group_id_end - group_id_start); - // initialize the slave node - auto s = Init(raft_group_id, true); - if (!s.ok()) { - join_client->SetRes(CmdRes::kErrOther, s.error_str()); - join_client->SendPacket(); - // join_client->Clear(); - // If the join fails, clear clusterContext and set it again by using the join command - cluster_cmd_ctx_.Clear(); - return; - } - - SendNodeAddRequest(client); - } else { + if (group_id_end == std::string::npos) { // can't find group id ERROR("Joined Raft cluster fail, because of invalid raft_group_id"); join_client->SetRes(CmdRes::kErrOther, "Invalid raft_group_id"); join_client->SendPacket(); // join_client->Clear(); // If the join fails, clear clusterContext and set it again by using the join command cluster_cmd_ctx_.Clear(); + return false; } + + std::string raft_group_id = reply.substr(group_id_start, group_id_end - group_id_start); + // initialize the slave node + auto s = Init(raft_group_id, true); + if (!s.ok()) { + join_client->SetRes(CmdRes::kErrOther, s.error_str()); + join_client->SendPacket(); + join_client->Clear(); + // If the join fails, clear clusterContext and set it again by using the join command + cluster_cmd_ctx_.Clear(); + ERROR("Failed to init raft: {}", s.error_cstr()); + return false; + } + INFO("Init raft successfully, groupid={}", raft_group_id); + return true; } int PRaft::ProcessClusterJoinCmdResponse(PClient* client, const char* start, int len) { @@ -439,7 +436,12 @@ int PRaft::ProcessClusterJoinCmdResponse(PClient* client, const char* start, int } else if (reply.find(WRONG_LEADER) != std::string::npos) { LeaderRedirection(join_client, reply); } else if (reply.find(RAFT_GROUP_ID) != std::string::npos) { - InitializeNodeBeforeAdd(client, join_client, reply); + auto res = InitializeNodeBeforeAdd(client, join_client, reply); + if (!res) { + ERROR("Failed to initialize node before add"); + return len; + } + SendNodeAddRequest(client); } else { ERROR("Joined Raft cluster fail, str: {}", reply); join_client->SetRes(CmdRes::kErrOther, reply); diff --git a/src/praft/praft.h b/src/praft/praft.h index 41a91b15d..296037b70 100644 --- a/src/praft/praft.h +++ b/src/praft/praft.h @@ -122,7 +122,7 @@ class PRaft : public braft::StateMachine { int ProcessClusterCmdResponse(PClient* client, const char* start, int len); void CheckRocksDBConfiguration(PClient* client, PClient* join_client, const std::string& reply); void LeaderRedirection(PClient* join_client, const std::string& reply); - void InitializeNodeBeforeAdd(PClient* client, PClient* join_client, const std::string& reply); + bool InitializeNodeBeforeAdd(PClient* client, PClient* join_client, const std::string& reply); int ProcessClusterJoinCmdResponse(PClient* client, const char* start, int len); int ProcessClusterRemoveCmdResponse(PClient* client, const char* start, int len); diff --git a/src/store.h b/src/store.h index 125ca30f7..0a00bc369 100644 --- a/src/store.h +++ b/src/store.h @@ -10,6 +10,11 @@ #pragma once +#include +#include +#include +#include +#include "praft/praft.h" #define GLOG_NO_ABBREVIATED_SEVERITIES #include @@ -61,6 +66,17 @@ class PStore { int GetDBNumber() const { return db_number_; } brpc::Server* GetRpcServer() const { return rpc_server_.get(); } const butil::EndPoint& GetEndPoint() const { return endpoint_; } + void AddRegion(const std::string& group_id, uint32_t dbno) { + assert(!db_map_.contains(group_id)); + db_map_.emplace(group_id, dbno); + } + DB* GetDBByGroupID(const std::string& group_id) const { + auto it = db_map_.find(group_id); + if (it == db_map_.end()) { + return nullptr; + } + return backends_[it->second].get(); + } private: PStore() = default; @@ -69,6 +85,7 @@ class PStore { std::vector> backends_; butil::EndPoint endpoint_; std::unique_ptr rpc_server_{std::make_unique()}; + std::unordered_map db_map_; }; #define PSTORE PStore::Instance() From 5042a4dcd59f9f43d66ba5a54f97533e626ff239 Mon Sep 17 00:00:00 2001 From: longfar Date: Sat, 13 Jul 2024 14:24:52 +0800 Subject: [PATCH 05/19] tmp --- src/cmd_admin.cc | 2 +- src/cmd_raft.cc | 4 ++-- src/db.cc | 4 +++- src/db.h | 2 +- src/praft/praft.cc | 6 ++++-- src/praft/praft.h | 2 +- src/store.h | 4 ++-- 7 files changed, 14 insertions(+), 10 deletions(-) diff --git a/src/cmd_admin.cc b/src/cmd_admin.cc index 2eb0f1fce..51eddfa3e 100644 --- a/src/cmd_admin.cc +++ b/src/cmd_admin.cc @@ -120,7 +120,7 @@ void FlushallCmd::DoCmd(PClient* client) { } SelectCmd::SelectCmd(const std::string& name, int16_t arity) - : BaseCmd(name, arity, kCmdFlagsAdmin | kCmdFlagsReadonly, kAclCategoryAdmin) {} + : BaseCmd(name, arity, kCmdFlagsAdmin, kAclCategoryAdmin) {} bool SelectCmd::DoInitial(PClient* client) { return true; } diff --git a/src/cmd_raft.cc b/src/cmd_raft.cc index 6a6c8ff6b..d70662093 100644 --- a/src/cmd_raft.cc +++ b/src/cmd_raft.cc @@ -53,7 +53,7 @@ void RaftNodeCmd::DoCmd(PClient* client) { } else if (cmd == kRemoveCmd) { DoCmdRemove(client); } else if (cmd == kDoSnapshot) { - assert(0); // TODO(longfar): add group id in arguments + assert(0); // TODO(longfar): add group id in arguments DoCmdSnapshot(client); } else { client->SetRes(CmdRes::kErrOther, "RAFT.NODE supports ADD / REMOVE / DOSNAPSHOT only"); @@ -61,6 +61,7 @@ void RaftNodeCmd::DoCmd(PClient* client) { } void RaftNodeCmd::DoCmdAdd(PClient* client) { + DEBUG("Received RAFT.NODE ADD cmd from {}", client->PeerIP()); auto db = PSTORE.GetDBByGroupID(group_id_); assert(db); auto praft = db->GetPRaft(); @@ -245,7 +246,6 @@ void RaftClusterCmd::DoCmdJoin(PClient* client) { return client->SetRes(CmdRes::kErrOther, "Other clients have joined"); } praft_->GetClusterCmdCtx().ConnectTargetNode(); - INFO("Sent join request to leader successfully"); // Not reply any message here, we will reply after the connection is established. client->Clear(); diff --git a/src/db.cc b/src/db.cc index 09fe15ed7..b00cb6cc3 100644 --- a/src/db.cc +++ b/src/db.cc @@ -19,7 +19,9 @@ extern pikiwidb::PConfig g_config; namespace pikiwidb { DB::DB(int db_index, const std::string& db_path) - : db_index_(db_index), db_path_(db_path + std::to_string(db_index_) + '/') {} + : db_index_(db_index), + db_path_(db_path + std::to_string(db_index_) + '/'), + praft_(std::make_unique(db_index)) {} DB::~DB() { INFO("DB{} is closing...", db_index_); } diff --git a/src/db.h b/src/db.h index 387d65192..67e62dcbe 100644 --- a/src/db.h +++ b/src/db.h @@ -54,7 +54,7 @@ class DB { */ std::shared_mutex storage_mutex_; std::unique_ptr storage_; - std::unique_ptr praft_{std::make_unique()}; + std::unique_ptr praft_{nullptr}; bool opened_ = false; }; diff --git a/src/praft/praft.cc b/src/praft/praft.cc index e13dcae69..4ff86ecfb 100644 --- a/src/praft/praft.cc +++ b/src/praft/praft.cc @@ -120,7 +120,7 @@ butil::Status PRaft::Init(const std::string& group_id, bool initial_conf_is_null // node_options_.election_timeout_ms = FLAGS_election_timeout_ms; // node_options_.disable_cli = FLAGS_disable_cli; - node_ = std::make_unique(group_id, braft::PeerId(PSTORE.GetEndPoint())); // group_id + node_ = std::make_unique(group_id, braft::PeerId(PSTORE.GetEndPoint(), db_id_)); // group_id if (node_->init(node_options) != 0) { node_.reset(); return ERROR_LOG_AND_STATUS("Failed to init raft node"); @@ -253,7 +253,8 @@ void PRaft::SendNodeRequest(PClient* client) { switch (cluster_cmd_type) { case ClusterCmdType::kJoin: { // SendNodeInfoRequest(client, "DATA"); - SendNodeAddRequest(client); + SendNodeInfoRequest(client, "DATA"); + // SendNodeAddRequest(client); } break; case ClusterCmdType::kRemove: SendNodeRemoveRequest(client); @@ -281,6 +282,7 @@ void PRaft::SendNodeAddRequest(PClient* client) { auto raw_addr = g_config.ip.ToString() + ":" + std::to_string(port); auto msg = fmt::format("RAFT.NODE ADD {} {}\r\n", group_id_, raw_addr); client->SendPacket(std::move(msg)); + INFO("Sent join request to leader successfully"); client->Clear(); } diff --git a/src/praft/praft.h b/src/praft/praft.h index 296037b70..c83ff6372 100644 --- a/src/praft/praft.h +++ b/src/praft/praft.h @@ -90,7 +90,7 @@ class PRaftWriteDoneClosure : public braft::Closure { class PRaft : public braft::StateMachine { public: - PRaft() = default; + PRaft(uint32_t db_id) : db_id_(db_id) {} ~PRaft() override { ShutDown(); Join(); diff --git a/src/store.h b/src/store.h index 0a00bc369..09e1b7bda 100644 --- a/src/store.h +++ b/src/store.h @@ -42,8 +42,8 @@ struct TaskContext { bool sync = false; TaskContext() = delete; TaskContext(TaskType t, bool s = false) : type(t), sync(s) {} - TaskContext(TaskType t, int d, bool s = false) : type(t), db(d), sync(s) {} - TaskContext(TaskType t, int d, const std::map& a, bool s = false) + TaskContext(TaskType t, uint32_t d, bool s = false) : type(t), db(d), sync(s) {} + TaskContext(TaskType t, uint32_t d, const std::map& a, bool s = false) : type(t), db(d), args(a), sync(s) {} }; From a44050d84d40ed0e2d37606238983de70ded935d Mon Sep 17 00:00:00 2001 From: longfar Date: Thu, 26 Sep 2024 10:46:27 +0800 Subject: [PATCH 06/19] fix: compile error and warnings --- src/cmd_admin.cc | 18 +++++++++++------- src/cmd_admin.h | 2 +- src/pikiwidb.cc | 2 +- src/praft/praft.cc | 2 +- src/praft/praft.h | 2 +- src/praft/psnapshot.cc | 2 +- src/praft/psnapshot.h | 6 ++++-- src/store.h | 4 ++-- 8 files changed, 22 insertions(+), 16 deletions(-) diff --git a/src/cmd_admin.cc b/src/cmd_admin.cc index 51eddfa3e..19d9c4f21 100644 --- a/src/cmd_admin.cc +++ b/src/cmd_admin.cc @@ -11,31 +11,31 @@ management of the PikiwiDB. */ +#include "cmd_admin.h" #include #include #include #include + #include #include - #include #include +#include #include #include #include -#include "cmd_admin.h" -#include "db.h" #include "braft/raft.h" -#include "pstd_string.h" #include "rocksdb/version.h" -#include "pikiwidb.h" #include "praft/praft.h" #include "pstd/env.h" +#include "pstd/pstd_string.h" -#include "cmd_table_manager.h" +#include "db.h" +#include "pikiwidb.h" #include "slow_log.h" #include "store.h" @@ -242,7 +242,7 @@ void InfoCmd::DoCmd(PClient* client) { InfoCommandStats(client, info); break; case kInfoRaft: - InfoRaft(info); + InfoRaft(client); break; default: break; @@ -329,8 +329,12 @@ void InfoCmd::InfoServer(std::string& info) { tmp_stream << "# Server\r\n"; tmp_stream << "PikiwiDB_version:" << version << "\r\n"; +#ifdef KPIKIWIDB_GIT_COMMIT_ID tmp_stream << "PikiwiDB_build_git_sha:" << KPIKIWIDB_GIT_COMMIT_ID << "\r\n"; +#endif +#ifdef KPIKIWIDB_BUILD_DATE tmp_stream << "Pikiwidb_build_compile_date: " << KPIKIWIDB_BUILD_DATE << "\r\n"; +#endif tmp_stream << "os:" << host_info.sysname << " " << host_info.release << " " << host_info.machine << "\r\n"; tmp_stream << "arch_bits:" << (reinterpret_cast(&host_info.machine) + strlen(host_info.machine) - 2) << "\r\n"; tmp_stream << "process_id:" << getpid() << "\r\n"; diff --git a/src/cmd_admin.h b/src/cmd_admin.h index c65e7711d..661e81039 100644 --- a/src/cmd_admin.h +++ b/src/cmd_admin.h @@ -160,7 +160,7 @@ class InfoCmd : public BaseCmd { void InfoServer(std::string& info); void InfoStats(std::string& info); void InfoCPU(std::string& info); - void InfoRaft(std::string& info); + void InfoRaft(PClient* client); void InfoData(std::string& info); void InfoCommandStats(PClient* client, std::string& info); std::string FormatCommandStatLine(const CommandStatistics& stats); diff --git a/src/pikiwidb.cc b/src/pikiwidb.cc index 067082f6f..5b6742315 100644 --- a/src/pikiwidb.cc +++ b/src/pikiwidb.cc @@ -130,7 +130,7 @@ bool PikiwiDB::ParseArgs(int argc, char* argv[]) { unsigned int optarg_long = static_cast(strlen(optarg)); char* str = (char*)calloc(optarg_long, sizeof(char*)); if (str) { - if (sscanf(optarg, "%s:%d", str, &master_port_) != 2) { + if (sscanf(optarg, "%s:%hd", str, &master_port_) != 2) { ERROR("Invalid slaveof format."); free(str); return false; diff --git a/src/praft/praft.cc b/src/praft/praft.cc index 4ff86ecfb..8ddd86b1c 100644 --- a/src/praft/praft.cc +++ b/src/praft/praft.cc @@ -114,7 +114,7 @@ butil::Status PRaft::Init(const std::string& group_id, bool initial_conf_is_null node_options.log_uri = prefix + "/log"; node_options.raft_meta_uri = prefix + "/raft_meta"; node_options.snapshot_uri = prefix + "/snapshot"; - snapshot_adaptor_ = new PPosixFileSystemAdaptor(); + snapshot_adaptor_ = new PPosixFileSystemAdaptor(this); node_options.snapshot_file_system_adaptor = &snapshot_adaptor_; // node_options_.election_timeout_ms = FLAGS_election_timeout_ms; diff --git a/src/praft/praft.h b/src/praft/praft.h index c83ff6372..916c5be1b 100644 --- a/src/praft/praft.h +++ b/src/praft/praft.h @@ -90,7 +90,7 @@ class PRaftWriteDoneClosure : public braft::Closure { class PRaft : public braft::StateMachine { public: - PRaft(uint32_t db_id) : db_id_(db_id) {} + PRaft(int did) : db_id_(did) {} ~PRaft() override { ShutDown(); Join(); diff --git a/src/praft/psnapshot.cc b/src/praft/psnapshot.cc index 17e389aca..a0b4f9152 100644 --- a/src/praft/psnapshot.cc +++ b/src/praft/psnapshot.cc @@ -83,7 +83,7 @@ braft::FileAdaptor* PPosixFileSystemAdaptor::open(const std::string& path, int o auto& new_meta = const_cast(snapshot_meta_memtable.meta()); auto last_log_index = PSTORE.GetBackend(db_id)->GetStorage()->GetSmallestFlushedLogIndex(); new_meta.set_last_included_index(last_log_index); - auto last_log_term = PRAFT.GetTerm(last_log_index); + auto last_log_term = praft_->GetTerm(last_log_index); new_meta.set_last_included_term(last_log_term); INFO("Succeed to fix db_{} snapshot meta: {}, {}", db_id, last_log_index, last_log_term); diff --git a/src/praft/psnapshot.h b/src/praft/psnapshot.h index 7aee35c58..2779f77d8 100644 --- a/src/praft/psnapshot.h +++ b/src/praft/psnapshot.h @@ -12,6 +12,7 @@ #include "braft/file_system_adaptor.h" #include "braft/macros.h" #include "braft/snapshot.h" +#include "praft/praft.h" #define PRAFT_SNAPSHOT_META_FILE "__raft_snapshot_meta" #define PRAFT_SNAPSHOT_PATH "snapshot/snapshot_" @@ -21,8 +22,8 @@ namespace pikiwidb { class PPosixFileSystemAdaptor : public braft::PosixFileSystemAdaptor { public: - PPosixFileSystemAdaptor() {} - ~PPosixFileSystemAdaptor() {} + explicit PPosixFileSystemAdaptor(PRaft* praft) : praft_(praft) {} + ~PPosixFileSystemAdaptor() override = default; braft::FileAdaptor* open(const std::string& path, int oflag, const ::google::protobuf::Message* file_meta, butil::File::Error* e) override; @@ -31,6 +32,7 @@ class PPosixFileSystemAdaptor : public braft::PosixFileSystemAdaptor { private: braft::raft_mutex_t mutex_; + PRaft* praft_{nullptr}; }; } // namespace pikiwidb diff --git a/src/store.h b/src/store.h index 09e1b7bda..6711e70a6 100644 --- a/src/store.h +++ b/src/store.h @@ -42,8 +42,8 @@ struct TaskContext { bool sync = false; TaskContext() = delete; TaskContext(TaskType t, bool s = false) : type(t), sync(s) {} - TaskContext(TaskType t, uint32_t d, bool s = false) : type(t), db(d), sync(s) {} - TaskContext(TaskType t, uint32_t d, const std::map& a, bool s = false) + TaskContext(TaskType t, int32_t d, bool s = false) : type(t), db(d), sync(s) {} + TaskContext(TaskType t, int32_t d, const std::map& a, bool s = false) : type(t), db(d), args(a), sync(s) {} }; From 997704001da011f73f704e6d877ec77aa1fac6fb Mon Sep 17 00:00:00 2001 From: longfar Date: Thu, 26 Sep 2024 10:50:32 +0800 Subject: [PATCH 07/19] feat: brpc command --- etc/conf/pikiwidb.conf | 2 +- src/CMakeLists.txt | 2 +- src/cmd_raft.cc | 76 +++++++++++++++++++++++--------------- src/praft/praft.cc | 29 +++++++++++++-- src/praft/praft.h | 1 + src/praft/praft.proto | 18 ++++++--- src/praft/praft_service.cc | 36 ++++++++++++++++++ src/praft/praft_service.h | 14 +++---- src/store.cc | 7 +++- src/store.h | 5 +++ 10 files changed, 139 insertions(+), 51 deletions(-) create mode 100644 src/praft/praft_service.cc diff --git a/etc/conf/pikiwidb.conf b/etc/conf/pikiwidb.conf index 72f8fbaa5..3570d42a7 100644 --- a/etc/conf/pikiwidb.conf +++ b/etc/conf/pikiwidb.conf @@ -35,7 +35,7 @@ logfile stdout # Set the number of databases. The default database is DB 0, you can select # a different one on a per-connection basis using SELECT where # dbid is a number between 0 and 'databases'-1 -databases 1 +databases 2 ################################ SNAPSHOTTING ################################# # diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 9e066544e..c3794d9e8 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -30,7 +30,7 @@ TARGET_INCLUDE_DIRECTORIES(pikiwidb PRIVATE ${rocksdb_SOURCE_DIR}/include PRIVATE ${BRAFT_INCLUDE_DIR} PRIVATE ${BRPC_INCLUDE_DIR} - PRIVATE ${LIBEVENT_INCLUDE_DIRS} + PRIVATE ${PROTO_OUTPUT_DIR} ) ADD_DEPENDENCIES(pikiwidb diff --git a/src/cmd_raft.cc b/src/cmd_raft.cc index d70662093..d9efa820f 100644 --- a/src/cmd_raft.cc +++ b/src/cmd_raft.cc @@ -15,10 +15,13 @@ #include #include #include +#include #include "praft/praft.h" #include "pstd/log.h" #include "pstd/pstd_string.h" +#include "brpc/channel.h" +#include "praft.pb.h" #include "client.h" #include "config.h" @@ -187,11 +190,11 @@ void RaftClusterCmd::DoCmdInit(PClient* client) { } else { group_id = pstd::RandomHexChars(RAFT_GROUPID_LEN); } + PSTORE.AddRegion(group_id, client->GetCurrentDB()); auto s = praft_->Init(group_id, false); if (!s.ok()) { return client->SetRes(CmdRes::kErrOther, fmt::format("Failed to init node: {}", s.error_str())); } - PSTORE.AddRegion(praft_->GetGroupID(), client->GetCurrentDB()); client->SetLineString(fmt::format("+OK {}", group_id)); } @@ -207,48 +210,61 @@ static inline std::optional> GetIpAndPortFromEnd } void RaftClusterCmd::DoCmdJoin(PClient* client) { - // If the node has been initialized, it needs to close the previous initialization and rejoin the other group - if (praft_->IsInitialized()) { - return client->SetRes(CmdRes::kErrOther, - "A node that has been added to a cluster must be removed \ - from the old cluster before it can be added to the new cluster"); - } - - // if (client->argv_.size() < 3) { - // return client->SetRes(CmdRes::kWrongNum, client->CmdName()); - // } - - // // (KKorpse)TODO: Support multiple nodes join at the same time. - // if (client->argv_.size() > 3) { - // return client->SetRes(CmdRes::kInvalidParameter, "Too many arguments"); - // } assert(client->argv_.size() == 4); auto group_id = client->argv_[2]; auto addr = client->argv_[3]; - // init raft - auto s = praft_->Init(group_id, true); - assert(s.ok()); - - if (braft::PeerId(addr).is_empty()) { - return client->SetRes(CmdRes::kErrOther, fmt::format("Invalid ip::port: {}", addr)); + if (group_id.size() != RAFT_GROUPID_LEN) { + return client->SetRes(CmdRes::kInvalidParameter, + "Cluster id must be " + std::to_string(RAFT_GROUPID_LEN) + " characters"); } - auto ip_port = GetIpAndPortFromEndPoint(addr); if (!ip_port.has_value()) { return client->SetRes(CmdRes::kErrOther, fmt::format("Invalid ip::port: {}", addr)); } - auto& [peer_ip, port] = *ip_port; + auto& [ip, port] = *ip_port; + PSTORE.AddRegion(group_id, client->GetCurrentDB()); + auto s = praft_->Init(group_id, true); + assert(s.ok()); - // Connect target - auto ret = praft_->GetClusterCmdCtx().Set(ClusterCmdType::kJoin, client, std::move(peer_ip), port); - if (!ret) { // other clients have joined - return client->SetRes(CmdRes::kErrOther, "Other clients have joined"); + brpc::ChannelOptions options; + options.connection_type = brpc::CONNECTION_TYPE_SINGLE; + options.max_retry = 0; + options.connect_timeout_ms = 200; + + brpc::Channel add_node_channel; + if (0 != add_node_channel.Init(addr.c_str(), &options)) { + ERROR("Fail to init add_node_channel to praft service!"); + // 失败的情况下,应该取消 Init。 + // 并且 Remove Region。 + client->SetRes(CmdRes::kErrOther, "Fail to init channel."); + return; + } + + brpc::Controller cntl; + NodeAddRequest request; + NodeAddResponse response; + auto end_point = fmt::format("{}:{}", ip, std::to_string(port)); + request.set_groupid(group_id); + request.set_endpoint(end_point); + request.set_index(client->GetCurrentDB()); + request.set_role(0); // 0 : !witness + PRaftService_Stub stub(&add_node_channel); + stub.AddNode(&cntl, &request, &response, NULL); + + if (cntl.Failed()) { + client->SetRes(CmdRes::kErrOther, "Failed to send add node rpc"); + return; + } + if (response.success()) { + client->SetRes(CmdRes::kOK, "Add Node Success"); + return; } - praft_->GetClusterCmdCtx().ConnectTargetNode(); + // 这里需要删除 Region。并且取消 初始化 。 + client->SetRes(CmdRes::kErrOther, "Failed to Add Node"); // Not reply any message here, we will reply after the connection is established. - client->Clear(); + // client->Clear(); } } // namespace pikiwidb diff --git a/src/praft/praft.cc b/src/praft/praft.cc index 8ddd86b1c..a79b3cbc9 100644 --- a/src/praft/praft.cc +++ b/src/praft/praft.cc @@ -71,6 +71,7 @@ void ClusterCmdContext::ConnectTargetNode() { auto ip = PREPL.GetMasterAddr().GetIP(); auto port = PREPL.GetMasterAddr().GetPort(); if (ip == peer_ip_ && port == port_ && PREPL.GetMasterState() == kPReplStateConnected) { + std::cout<<"已经建立连接了, 直接发送" <SendNodeRequest(PREPL.GetMaster()); return; } @@ -101,7 +102,7 @@ butil::Status PRaft::Init(const std::string& group_id, bool initial_conf_is_null braft::NodeOptions node_options; if (!initial_conf_is_null) { auto endpoint_str = butil::endpoint2str(PSTORE.GetEndPoint()); - std::string initial_conf = fmt::format("{}:0,", endpoint_str.c_str()); + std::string initial_conf = fmt::format("{}:{},", endpoint_str.c_str(), db_id_); if (node_options.initial_conf.parse_from(initial_conf) != 0) { return ERROR_LOG_AND_STATUS("Failed to parse configuration"); } @@ -253,8 +254,8 @@ void PRaft::SendNodeRequest(PClient* client) { switch (cluster_cmd_type) { case ClusterCmdType::kJoin: { // SendNodeInfoRequest(client, "DATA"); - SendNodeInfoRequest(client, "DATA"); - // SendNodeAddRequest(client); + // SendNodeInfoRequest(client, "DATA"); + SendNodeAddRequest(client); } break; case ClusterCmdType::kRemove: SendNodeRemoveRequest(client); @@ -279,7 +280,7 @@ void PRaft::SendNodeAddRequest(PClient* client) { // Node id in braft are ip:port, the node id param in RAFT.NODE ADD cmd will be ignored. auto port = g_config.port + pikiwidb::g_config.raft_port_offset; - auto raw_addr = g_config.ip.ToString() + ":" + std::to_string(port); + auto raw_addr = g_config.ip.ToString() + ":" + std::to_string(port) + ":" + std::to_string(db_id_); auto msg = fmt::format("RAFT.NODE ADD {} {}\r\n", group_id_, raw_addr); client->SendPacket(std::move(msg)); INFO("Sent join request to leader successfully"); @@ -508,6 +509,26 @@ butil::Status PRaft::AddPeer(const std::string& peer) { return {0, "OK"}; } +butil::Status PRaft::AddPeer(const std::string& endpoint, int index) { + if (!node_) { + ERROR_LOG_AND_STATUS("Node is not initialized"); + } + + braft::SynchronizedClosure done; + butil::EndPoint ep; + butil::str2endpoint(endpoint.c_str(), &ep); + braft::PeerId peer_id(ep, index); + node_->add_peer(peer_id, &done); + done.wait(); + + if (!done.status().ok()) { + // WARN("Failed to add peer {} to node {}, status: {}", end_point, node_->node_id().to_string(), done.status().error_str()); + WARN("Failed to add"); + return done.status(); + } + return done.status(); +} + butil::Status PRaft::RemovePeer(const std::string& peer) { if (!node_) { return ERROR_LOG_AND_STATUS("Node is not initialized"); diff --git a/src/praft/praft.h b/src/praft/praft.h index 916c5be1b..089bc012a 100644 --- a/src/praft/praft.h +++ b/src/praft/praft.h @@ -102,6 +102,7 @@ class PRaft : public braft::StateMachine { //===--------------------------------------------------------------------===// butil::Status Init(const std::string& group_id, bool initial_conf_is_null); butil::Status AddPeer(const std::string& peer); + butil::Status AddPeer(const std::string& endpoint, int index); butil::Status RemovePeer(const std::string& peer); butil::Status DoSnapshot(int64_t self_snapshot_index = 0, bool is_sync = true); diff --git a/src/praft/praft.proto b/src/praft/praft.proto index 61a495f21..24bb42580 100644 --- a/src/praft/praft.proto +++ b/src/praft/praft.proto @@ -2,12 +2,18 @@ syntax="proto3"; package pikiwidb; option cc_generic_services = true; -message DummyRequest { -}; +message NodeAddRequest { + string GroupID = 1; + string EndPoint = 2; + uint32 index = 3; + uint32 role = 4; +} -message DummyResponse { -}; +message NodeAddResponse { + bool success = 1; +} -service DummyService { - rpc DummyMethod(DummyRequest) returns (DummyResponse); +service PRaftService { + rpc AddNode(NodeAddRequest) returns (NodeAddResponse); + }; diff --git a/src/praft/praft_service.cc b/src/praft/praft_service.cc new file mode 100644 index 000000000..b3413705f --- /dev/null +++ b/src/praft/praft_service.cc @@ -0,0 +1,36 @@ +/* + * Copyright (c) 2024-present, Qihoo, Inc. All rights reserved. + * This source code is licensed under the BSD-style license found in the + * LICENSE file in the root directory of this source tree. An additional grant + * of patent rights can be found in the PATENTS file in the same directory. + */ + +#include "praft_service.h" + +#include "fmt/format.h" +#include "store.h" + + +namespace pikiwidb { + void PRaftServiceImpl::AddNode(::google::protobuf::RpcController* controller, + const ::pikiwidb::NodeAddRequest* request, + ::pikiwidb::NodeAddResponse* response, + ::google::protobuf::Closure* done) { + brpc::ClosureGuard done_guard(done); + auto groupid = request->groupid(); + auto db_ptr = PSTORE.GetDBByGroupID(groupid); + auto praft_ptr = db_ptr->GetPRaft(); + auto end_point = request->endpoint(); + auto index = request->index(); + auto role = request->role(); + + auto status = praft_ptr->AddPeer(end_point, index); + if (!status.ok()) { + std::cout<<"add node fail!"<set_success(false); + return; + } + std::cout<<"add node success!"<set_success(true); +} +} \ No newline at end of file diff --git a/src/praft/praft_service.h b/src/praft/praft_service.h index db08c7b7b..082200c2f 100644 --- a/src/praft/praft_service.h +++ b/src/praft/praft_service.h @@ -12,15 +12,13 @@ namespace pikiwidb { class PRaft; - -class DummyServiceImpl : public DummyService { +class PRaftServiceImpl : public PRaftService { public: - explicit DummyServiceImpl(PRaft* praft) : praft_(praft) {} - void DummyMethod(::google::protobuf::RpcController* controller, const ::pikiwidb::DummyRequest* request, - ::pikiwidb::DummyResponse* response, ::google::protobuf::Closure* done) override {} - - private: - PRaft* praft_ = nullptr; + PRaftServiceImpl() = default; + void AddNode(::google::protobuf::RpcController* controller, + const ::pikiwidb::NodeAddRequest* request, + ::pikiwidb::NodeAddResponse* response, + ::google::protobuf::Closure* done); }; } // namespace pikiwidb diff --git a/src/store.cc b/src/store.cc index 2ba57ae58..dc89d173d 100644 --- a/src/store.cc +++ b/src/store.cc @@ -14,9 +14,9 @@ #include "db.h" #include "pstd/log.h" #include "pstd/pstd_string.h" +#include "praft/praft_service.h" namespace pikiwidb { - PStore::~PStore() { INFO("STORE is closing..."); } PStore& PStore::Instance() { @@ -33,6 +33,7 @@ void PStore::Init(int db_number) { backends_.push_back(std::move(db)); INFO("Open DB_{} success!", i); } + auto ip = g_config.ip.ToString(); butil::ip_t rpc_ip; butil::str2ip(ip.c_str(), &rpc_ip); @@ -42,6 +43,10 @@ void PStore::Init(int db_number) { if (braft::add_service(GetRpcServer(), endpoint_) != 0) { return ERROR("Failed to add raft service to rpc server"); } + if (0 != rpc_server_->AddService(dynamic_cast(praft_service_.get()), brpc::SERVER_OWNS_SERVICE)) { + return ERROR("Failed to add praft service to rpc server"); + } + if (rpc_server_->Start(endpoint_, nullptr) != 0) { return ERROR("Failed to start rpc server"); } diff --git a/src/store.h b/src/store.h index 6711e70a6..07e2f1501 100644 --- a/src/store.h +++ b/src/store.h @@ -14,7 +14,10 @@ #include #include #include + #include "praft/praft.h" +#include "praft/praft_service.h" + #define GLOG_NO_ABBREVIATED_SEVERITIES #include @@ -26,6 +29,7 @@ #include "db.h" namespace pikiwidb { +class RaftServiceImpl; enum TaskType { kCheckpoint = 0, kLoadDBFromCheckpoint, kEmpty }; @@ -84,6 +88,7 @@ class PStore { int db_number_ = 0; std::vector> backends_; butil::EndPoint endpoint_; + std::unique_ptr praft_service_{std::make_unique()}; std::unique_ptr rpc_server_{std::make_unique()}; std::unordered_map db_map_; }; From 425d2c59f8a31101eb929e11775d725ba00151ab Mon Sep 17 00:00:00 2001 From: longfar Date: Thu, 26 Sep 2024 11:04:53 +0800 Subject: [PATCH 08/19] feat: use brpc to send node add message --- src/client.h | 2 + src/cmd_raft.cc | 92 ++++++++++++++++++++++++-------------- src/cmd_raft.h | 14 +++--- src/praft/praft.cc | 6 +-- src/praft/praft_service.cc | 38 +++++++--------- src/praft/praft_service.h | 6 +-- src/store.cc | 56 ++++++++++++++++++----- src/store.h | 35 ++++++++++----- 8 files changed, 156 insertions(+), 93 deletions(-) diff --git a/src/client.h b/src/client.h index 3da74f70b..44b0cf291 100644 --- a/src/client.h +++ b/src/client.h @@ -154,6 +154,8 @@ enum class ClientState { kClosed, }; +const int kChannelTimeoutMS = 200; + class DB; struct PSlaveInfo; diff --git a/src/cmd_raft.cc b/src/cmd_raft.cc index d9efa820f..bc8835b78 100644 --- a/src/cmd_raft.cc +++ b/src/cmd_raft.cc @@ -15,17 +15,18 @@ #include #include #include -#include + +#include "brpc/channel.h" +#include "fmt/format.h" #include "praft/praft.h" #include "pstd/log.h" #include "pstd/pstd_string.h" -#include "brpc/channel.h" -#include "praft.pb.h" #include "client.h" #include "config.h" #include "pikiwidb.h" +#include "praft.pb.h" #include "replication.h" #include "store.h" @@ -138,9 +139,7 @@ void RaftNodeCmd::DoCmdRemove(PClient* client) { } void RaftNodeCmd::DoCmdSnapshot(PClient* client) { - auto self_snapshot_index = PSTORE.GetBackend(client->GetCurrentDB())->GetStorage()->GetSmallestFlushedLogIndex(); - INFO("DoCmdSnapshot self_snapshot_index:{}", self_snapshot_index); - auto s = praft_->DoSnapshot(self_snapshot_index); + auto s = praft_->DoSnapshot(); if (s.ok()) { client->SetRes(CmdRes::kOK); } @@ -156,6 +155,7 @@ bool RaftClusterCmd::DoInitial(PClient* client) { client->SetRes(CmdRes::kErrOther, "RAFT.CLUSTER supports INIT/JOIN only"); return false; } + praft_ = PSTORE.GetBackend(client->GetCurrentDB())->GetPRaft(); return true; } @@ -190,12 +190,19 @@ void RaftClusterCmd::DoCmdInit(PClient* client) { } else { group_id = pstd::RandomHexChars(RAFT_GROUPID_LEN); } - PSTORE.AddRegion(group_id, client->GetCurrentDB()); - auto s = praft_->Init(group_id, false); - if (!s.ok()) { - return client->SetRes(CmdRes::kErrOther, fmt::format("Failed to init node: {}", s.error_str())); + + auto add_region_success = PSTORE.AddRegion(group_id, client->GetCurrentDB()); + if (add_region_success) { + auto s = praft_->Init(group_id, false); + if (!s.ok()) { + PSTORE.RemoveRegion(group_id); + ClearPaftCtx(); + return client->SetRes(CmdRes::kErrOther, fmt::format("Failed to init raft node: {}", s.error_str())); + } + client->SetLineString(fmt::format("+OK {}", group_id)); + } else { + client->SetRes(CmdRes::kErrOther, fmt::format("The current GroupID {} already exists", group_id)); } - client->SetLineString(fmt::format("+OK {}", group_id)); } static inline std::optional> GetIpAndPortFromEndPoint(const std::string& endpoint) { @@ -213,46 +220,59 @@ void RaftClusterCmd::DoCmdJoin(PClient* client) { assert(client->argv_.size() == 4); auto group_id = client->argv_[2]; auto addr = client->argv_[3]; + butil::EndPoint endpoint; + if (0 != butil::str2endpoint(addr.c_str(), &endpoint)) { + ERROR("Wrong endpoint format: {}", addr); + return client->SetRes(CmdRes::kErrOther, "Wrong endpoint format"); + } + endpoint.port += g_config.raft_port_offset; if (group_id.size() != RAFT_GROUPID_LEN) { return client->SetRes(CmdRes::kInvalidParameter, - "Cluster id must be " + std::to_string(RAFT_GROUPID_LEN) + " characters"); + "Cluster id must be " + std::to_string(RAFT_GROUPID_LEN) + " characters"); } - auto ip_port = GetIpAndPortFromEndPoint(addr); - if (!ip_port.has_value()) { - return client->SetRes(CmdRes::kErrOther, fmt::format("Invalid ip::port: {}", addr)); + + auto add_region_success = PSTORE.AddRegion(group_id, client->GetCurrentDB()); + if (add_region_success) { + auto s = praft_->Init(group_id, false); + if (!s.ok()) { + PSTORE.RemoveRegion(group_id); + ClearPaftCtx(); + return client->SetRes(CmdRes::kErrOther, fmt::format("Failed to init raft node: {}", s.error_str())); + } + } else { + client->SetRes(CmdRes::kErrOther, fmt::format("The current GroupID {} already exists", group_id)); } - auto& [ip, port] = *ip_port; - PSTORE.AddRegion(group_id, client->GetCurrentDB()); - auto s = praft_->Init(group_id, true); - assert(s.ok()); brpc::ChannelOptions options; options.connection_type = brpc::CONNECTION_TYPE_SINGLE; options.max_retry = 0; - options.connect_timeout_ms = 200; + options.connect_timeout_ms = kChannelTimeoutMS; brpc::Channel add_node_channel; - if (0 != add_node_channel.Init(addr.c_str(), &options)) { + if (0 != add_node_channel.Init(endpoint, &options)) { + PSTORE.RemoveRegion(group_id); + ClearPaftCtx(); ERROR("Fail to init add_node_channel to praft service!"); - // 失败的情况下,应该取消 Init。 - // 并且 Remove Region。 - client->SetRes(CmdRes::kErrOther, "Fail to init channel."); + client->SetRes(CmdRes::kErrOther, "Fail to init add_node_channel."); return; } brpc::Controller cntl; NodeAddRequest request; NodeAddResponse response; - auto end_point = fmt::format("{}:{}", ip, std::to_string(port)); + auto end_point = butil::endpoint2str(PSTORE.GetEndPoint()).c_str(); request.set_groupid(group_id); - request.set_endpoint(end_point); + request.set_endpoint(std::string(end_point)); request.set_index(client->GetCurrentDB()); - request.set_role(0); // 0 : !witness + request.set_role(0); PRaftService_Stub stub(&add_node_channel); stub.AddNode(&cntl, &request, &response, NULL); - + if (cntl.Failed()) { + PSTORE.RemoveRegion(group_id); + ClearPaftCtx(); + ERROR("Fail to send add node rpc to target server {}", addr); client->SetRes(CmdRes::kErrOther, "Failed to send add node rpc"); return; } @@ -260,11 +280,17 @@ void RaftClusterCmd::DoCmdJoin(PClient* client) { client->SetRes(CmdRes::kOK, "Add Node Success"); return; } - // 这里需要删除 Region。并且取消 初始化 。 - client->SetRes(CmdRes::kErrOther, "Failed to Add Node"); - - // Not reply any message here, we will reply after the connection is established. - // client->Clear(); + PSTORE.RemoveRegion(group_id); + ClearPaftCtx(); + ERROR("Add node request return false"); + client->SetRes(CmdRes::kErrOther, "Failed to Add Node"); } +void RaftClusterCmd::ClearPaftCtx() { + assert(praft_); + praft_->ShutDown(); + praft_->Join(); + praft_->Clear(); + praft_ = nullptr; +} } // namespace pikiwidb diff --git a/src/cmd_raft.h b/src/cmd_raft.h index eedbfab0d..d994271a5 100644 --- a/src/cmd_raft.h +++ b/src/cmd_raft.h @@ -17,7 +17,7 @@ namespace pikiwidb { class PRaft; -/* +/* * RAFT.NODE ADD [id] [address:port] * Add a new node to the cluster. The [id] can be an explicit non-zero value, * or zero to let the cluster choose one. @@ -52,8 +52,8 @@ class RaftNodeCmd : public BaseCmd { void DoCmdRemove(PClient *client); void DoCmdSnapshot(PClient *client); -private: - PRaft* praft_ = nullptr; + private: + PRaft *praft_ = nullptr; std::string group_id_; static constexpr std::string_view kAddCmd = "ADD"; @@ -61,7 +61,7 @@ class RaftNodeCmd : public BaseCmd { static constexpr std::string_view kDoSnapshot = "DOSNAPSHOT"; }; -/* +/* * RAFT.CLUSTER INIT [group_id] * Initializes a new Raft cluster. * Reply: @@ -84,8 +84,10 @@ class RaftClusterCmd : public BaseCmd { void DoCmdInit(PClient *client); void DoCmdJoin(PClient *client); -private: - PRaft* praft_ = nullptr; + void ClearPaftCtx(); + + private: + PRaft *praft_ = nullptr; static constexpr std::string_view kInitCmd = "INIT"; static constexpr std::string_view kJoinCmd = "JOIN"; diff --git a/src/praft/praft.cc b/src/praft/praft.cc index a79b3cbc9..405ac28d7 100644 --- a/src/praft/praft.cc +++ b/src/praft/praft.cc @@ -71,7 +71,6 @@ void ClusterCmdContext::ConnectTargetNode() { auto ip = PREPL.GetMasterAddr().GetIP(); auto port = PREPL.GetMasterAddr().GetPort(); if (ip == peer_ip_ && port == port_ && PREPL.GetMasterState() == kPReplStateConnected) { - std::cout<<"已经建立连接了, 直接发送" <SendNodeRequest(PREPL.GetMaster()); return; } @@ -510,7 +509,7 @@ butil::Status PRaft::AddPeer(const std::string& peer) { } butil::Status PRaft::AddPeer(const std::string& endpoint, int index) { - if (!node_) { + if (!node_) { ERROR_LOG_AND_STATUS("Node is not initialized"); } @@ -522,8 +521,7 @@ butil::Status PRaft::AddPeer(const std::string& endpoint, int index) { done.wait(); if (!done.status().ok()) { - // WARN("Failed to add peer {} to node {}, status: {}", end_point, node_->node_id().to_string(), done.status().error_str()); - WARN("Failed to add"); + WARN("Failed to add peer {} to node {}, status: {}", ep, node_->node_id().to_string(), done.status().error_str()); return done.status(); } return done.status(); diff --git a/src/praft/praft_service.cc b/src/praft/praft_service.cc index b3413705f..6e2c9559b 100644 --- a/src/praft/praft_service.cc +++ b/src/praft/praft_service.cc @@ -10,27 +10,21 @@ #include "fmt/format.h" #include "store.h" - namespace pikiwidb { - void PRaftServiceImpl::AddNode(::google::protobuf::RpcController* controller, - const ::pikiwidb::NodeAddRequest* request, - ::pikiwidb::NodeAddResponse* response, - ::google::protobuf::Closure* done) { - brpc::ClosureGuard done_guard(done); - auto groupid = request->groupid(); - auto db_ptr = PSTORE.GetDBByGroupID(groupid); - auto praft_ptr = db_ptr->GetPRaft(); - auto end_point = request->endpoint(); - auto index = request->index(); - auto role = request->role(); - - auto status = praft_ptr->AddPeer(end_point, index); - if (!status.ok()) { - std::cout<<"add node fail!"<set_success(false); - return; - } - std::cout<<"add node success!"<set_success(true); +void PRaftServiceImpl::AddNode(::google::protobuf::RpcController* controller, const ::pikiwidb::NodeAddRequest* request, + ::pikiwidb::NodeAddResponse* response, ::google::protobuf::Closure* done) { + brpc::ClosureGuard done_guard(done); + auto groupid = request->groupid(); + auto db_ptr = PSTORE.GetDBByGroupID(groupid); + auto praft_ptr = db_ptr->GetPRaft(); + auto end_point = request->endpoint(); + auto index = request->index(); + auto role = request->role(); + auto status = praft_ptr->AddPeer(end_point, index); + if (!status.ok()) { + response->set_success(false); + return; + } + response->set_success(true); } -} \ No newline at end of file +} // namespace pikiwidb \ No newline at end of file diff --git a/src/praft/praft_service.h b/src/praft/praft_service.h index 082200c2f..8db7634f8 100644 --- a/src/praft/praft_service.h +++ b/src/praft/praft_service.h @@ -15,10 +15,8 @@ class PRaft; class PRaftServiceImpl : public PRaftService { public: PRaftServiceImpl() = default; - void AddNode(::google::protobuf::RpcController* controller, - const ::pikiwidb::NodeAddRequest* request, - ::pikiwidb::NodeAddResponse* response, - ::google::protobuf::Closure* done); + void AddNode(::google::protobuf::RpcController* controller, const ::pikiwidb::NodeAddRequest* request, + ::pikiwidb::NodeAddResponse* response, ::google::protobuf::Closure* done); }; } // namespace pikiwidb diff --git a/src/store.cc b/src/store.cc index dc89d173d..59c000f14 100644 --- a/src/store.cc +++ b/src/store.cc @@ -12,9 +12,9 @@ #include "config.h" #include "db.h" +#include "praft/praft_service.h" #include "pstd/log.h" #include "pstd/pstd_string.h" -#include "praft/praft_service.h" namespace pikiwidb { PStore::~PStore() { INFO("STORE is closing..."); } @@ -33,25 +33,29 @@ void PStore::Init(int db_number) { backends_.push_back(std::move(db)); INFO("Open DB_{} success!", i); } - - auto ip = g_config.ip.ToString(); - butil::ip_t rpc_ip; - butil::str2ip(ip.c_str(), &rpc_ip); + + auto rpc_ip = g_config.ip.ToString(); auto rpc_port = g_config.port.load(std::memory_order_relaxed) + g_config.raft_port_offset.load(std::memory_order_relaxed); - endpoint_ = butil::EndPoint(rpc_ip, rpc_port); - if (braft::add_service(GetRpcServer(), endpoint_) != 0) { + + if (0 != butil::str2endpoint(fmt::format("{}:{}", rpc_ip, std::to_string(rpc_port)).c_str(), &endpoint_)) { + return ERROR("Wrong endpoint format"); + } + + if (0 != braft::add_service(GetRpcServer(), endpoint_)) { return ERROR("Failed to add raft service to rpc server"); } - if (0 != rpc_server_->AddService(dynamic_cast(praft_service_.get()), brpc::SERVER_OWNS_SERVICE)) { + + if (0 != rpc_server_->AddService(dynamic_cast(praft_service_.get()), + brpc::SERVER_OWNS_SERVICE)) { return ERROR("Failed to add praft service to rpc server"); } - - if (rpc_server_->Start(endpoint_, nullptr) != 0) { + + if (0 != rpc_server_->Start(endpoint_, nullptr)) { return ERROR("Failed to start rpc server"); } - INFO("Started RPC server successfully"); - INFO("STORE Init success!"); + INFO("Started RPC server successfully on addr {}", butil::endpoint2str(endpoint_).c_str()); + INFO("PSTORE Init success!"); } void PStore::HandleTaskSpecificDB(const TasksVector& tasks) { @@ -91,4 +95,32 @@ void PStore::HandleTaskSpecificDB(const TasksVector& tasks) { } }); } + +bool PStore::AddRegion(const std::string& group_id, uint32_t dbno) { + std::lock_guard lock(rw_mutex_); + if (region_map_.find(group_id) != region_map_.end()) { + return false; + } + region_map_.emplace(group_id, dbno); + return true; +} + +bool PStore::RemoveRegion(const std::string& group_id) { + std::lock_guard lock(rw_mutex_); + if (region_map_.find(group_id) != region_map_.end()) { + region_map_.erase(group_id); + return true; + } + return false; +} + +DB* PStore::GetDBByGroupID(const std::string& group_id) const { + std::shared_lock lock(rw_mutex_); + auto it = region_map_.find(group_id); + if (it == region_map_.end()) { + return nullptr; + } + return backends_[it->second].get(); +} + } // namespace pikiwidb diff --git a/src/store.h b/src/store.h index 07e2f1501..f4ed7a5d2 100644 --- a/src/store.h +++ b/src/store.h @@ -12,6 +12,7 @@ #include #include +#include #include #include @@ -68,19 +69,27 @@ class PStore { void HandleTaskSpecificDB(const TasksVector& tasks); int GetDBNumber() const { return db_number_; } + brpc::Server* GetRpcServer() const { return rpc_server_.get(); } + const butil::EndPoint& GetEndPoint() const { return endpoint_; } - void AddRegion(const std::string& group_id, uint32_t dbno) { - assert(!db_map_.contains(group_id)); - db_map_.emplace(group_id, dbno); - } - DB* GetDBByGroupID(const std::string& group_id) const { - auto it = db_map_.find(group_id); - if (it == db_map_.end()) { - return nullptr; - } - return backends_[it->second].get(); - } + + /** + * return true if add group_id -> dbno into region_map_ success. + * return false if group_id -> dbno already exists in region_map_. + */ + bool AddRegion(const std::string& group_id, uint32_t dbno); + + /** + * return true if remove group_id -> dbno from region_map_ success. + * return false if group_id -> dbno do not exists in region_map_. + */ + bool RemoveRegion(const std::string& group_id); + + /** + * return nullptr if group_id -> dbno do not existed in region_map_. + */ + DB* GetDBByGroupID(const std::string& group_id) const; private: PStore() = default; @@ -90,7 +99,9 @@ class PStore { butil::EndPoint endpoint_; std::unique_ptr praft_service_{std::make_unique()}; std::unique_ptr rpc_server_{std::make_unique()}; - std::unordered_map db_map_; + + mutable std::shared_mutex rw_mutex_; + std::unordered_map region_map_; }; #define PSTORE PStore::Instance() From c6f100c6327ef58853580a9e1091517bda2a5cdd Mon Sep 17 00:00:00 2001 From: longfar Date: Thu, 26 Sep 2024 11:19:09 +0800 Subject: [PATCH 09/19] feat: add node redirect feature --- src/cmd_raft.cc | 75 +++++++++++++++++++++++++------------- src/praft/praft.cc | 3 +- src/praft/praft.proto | 6 ++- src/praft/praft_service.cc | 30 +++++++++++++-- src/store.h | 7 ++++ 5 files changed, 89 insertions(+), 32 deletions(-) diff --git a/src/cmd_raft.cc b/src/cmd_raft.cc index bc8835b78..544a10d8d 100644 --- a/src/cmd_raft.cc +++ b/src/cmd_raft.cc @@ -249,40 +249,63 @@ void RaftClusterCmd::DoCmdJoin(PClient* client) { options.max_retry = 0; options.connect_timeout_ms = kChannelTimeoutMS; - brpc::Channel add_node_channel; - if (0 != add_node_channel.Init(endpoint, &options)) { - PSTORE.RemoveRegion(group_id); - ClearPaftCtx(); - ERROR("Fail to init add_node_channel to praft service!"); - client->SetRes(CmdRes::kErrOther, "Fail to init add_node_channel."); - return; - } - - brpc::Controller cntl; NodeAddRequest request; NodeAddResponse response; + auto end_point = butil::endpoint2str(PSTORE.GetEndPoint()).c_str(); - request.set_groupid(group_id); + request.set_group_id(group_id); request.set_endpoint(std::string(end_point)); request.set_index(client->GetCurrentDB()); request.set_role(0); - PRaftService_Stub stub(&add_node_channel); - stub.AddNode(&cntl, &request, &response, NULL); - - if (cntl.Failed()) { - PSTORE.RemoveRegion(group_id); - ClearPaftCtx(); - ERROR("Fail to send add node rpc to target server {}", addr); - client->SetRes(CmdRes::kErrOther, "Failed to send add node rpc"); - return; - } - if (response.success()) { - client->SetRes(CmdRes::kOK, "Add Node Success"); - return; - } + + int retry_count = 0; + + do { + brpc::Channel add_node_channel; + if (0 != add_node_channel.Init(endpoint, &options)) { + PSTORE.RemoveRegion(group_id); + ClearPaftCtx(); + ERROR("Fail to init add_node_channel to praft service!"); + client->SetRes(CmdRes::kErrOther, "Fail to init add_node_channel."); + return; + } + + brpc::Controller cntl; + PRaftService_Stub stub(&add_node_channel); + stub.AddNode(&cntl, &request, &response, NULL); + + if (cntl.Failed()) { + PSTORE.RemoveRegion(group_id); + ClearPaftCtx(); + ERROR("Fail to send add node rpc to target server {}", addr); + client->SetRes(CmdRes::kErrOther, "Failed to send add node rpc"); + return; + } + + if (response.success()) { + client->SetRes(CmdRes::kOK, "Add Node Success"); + return; + } + + switch (response.error_code()) { + case PRaftErrorCode::kErrorReDirect: { + butil::str2endpoint(response.leader_endpoint().c_str(), &endpoint); + endpoint.port += g_config.raft_port_offset; + break; + } + default: { + ERROR("Add node request return false"); + PSTORE.RemoveRegion(group_id); + ClearPaftCtx(); + client->SetRes(CmdRes::kErrOther, "Failed to Add Node"); + return; + } + } + } while (!response.success() && ++retry_count <= 3); + + ERROR("Add node request return false"); PSTORE.RemoveRegion(group_id); ClearPaftCtx(); - ERROR("Add node request return false"); client->SetRes(CmdRes::kErrOther, "Failed to Add Node"); } diff --git a/src/praft/praft.cc b/src/praft/praft.cc index 405ac28d7..25c2a99f7 100644 --- a/src/praft/praft.cc +++ b/src/praft/praft.cc @@ -521,7 +521,8 @@ butil::Status PRaft::AddPeer(const std::string& endpoint, int index) { done.wait(); if (!done.status().ok()) { - WARN("Failed to add peer {} to node {}, status: {}", ep, node_->node_id().to_string(), done.status().error_str()); + // WARN("Failed to add peer {} to node {}, status: {}", ep, node_->node_id().to_string(), + // done.status().error_str()); return done.status(); } return done.status(); diff --git a/src/praft/praft.proto b/src/praft/praft.proto index 24bb42580..5636baf1a 100644 --- a/src/praft/praft.proto +++ b/src/praft/praft.proto @@ -3,14 +3,16 @@ package pikiwidb; option cc_generic_services = true; message NodeAddRequest { - string GroupID = 1; - string EndPoint = 2; + string group_id = 1; + string endpoint = 2; uint32 index = 3; uint32 role = 4; } message NodeAddResponse { bool success = 1; + uint32 error_code = 2; + string leader_endpoint = 3; } service PRaftService { diff --git a/src/praft/praft_service.cc b/src/praft/praft_service.cc index 6e2c9559b..305f5cfca 100644 --- a/src/praft/praft_service.cc +++ b/src/praft/praft_service.cc @@ -14,15 +14,39 @@ namespace pikiwidb { void PRaftServiceImpl::AddNode(::google::protobuf::RpcController* controller, const ::pikiwidb::NodeAddRequest* request, ::pikiwidb::NodeAddResponse* response, ::google::protobuf::Closure* done) { brpc::ClosureGuard done_guard(done); - auto groupid = request->groupid(); - auto db_ptr = PSTORE.GetDBByGroupID(groupid); - auto praft_ptr = db_ptr->GetPRaft(); + auto groupid = request->group_id(); auto end_point = request->endpoint(); auto index = request->index(); auto role = request->role(); + + auto db_ptr = PSTORE.GetDBByGroupID(groupid); + if (!db_ptr) { + response->set_success(false); + response->set_error_code(static_cast(PRaftErrorCode::kErrorDisMatch)); + response->set_leader_endpoint(end_point); + return; + } + auto praft_ptr = db_ptr->GetPRaft(); + if (!praft_ptr) { + response->set_success(false); + response->set_error_code(static_cast(PRaftErrorCode::kErrorDisMatch)); + response->set_leader_endpoint(end_point); + return; + } + + if (!praft_ptr->IsLeader()) { + response->set_success(false); + response->set_error_code(static_cast(PRaftErrorCode::kErrorReDirect)); + std::cout << "leader addr = " << praft_ptr->GetLeaderAddress() << std::endl; + response->set_leader_endpoint(praft_ptr->GetLeaderAddress()); + return; + } + auto status = praft_ptr->AddPeer(end_point, index); if (!status.ok()) { response->set_success(false); + response->set_error_code(static_cast(PRaftErrorCode::kErrorAddNode)); + response->set_leader_endpoint(praft_ptr->GetLeaderAddress()); return; } response->set_success(true); diff --git a/src/store.h b/src/store.h index f4ed7a5d2..144706008 100644 --- a/src/store.h +++ b/src/store.h @@ -54,6 +54,13 @@ struct TaskContext { using TasksVector = std::vector; +enum PRaftErrorCode { + kErrorDisMatch = 0, + kErrorAddNode, + kErrorRemoveNode, + kErrorReDirect, +}; + class PStore { public: static PStore& Instance(); From 042026594b4640c1aa61c07a0c8c79411a4f132b Mon Sep 17 00:00:00 2001 From: longfar Date: Thu, 26 Sep 2024 12:12:06 +0800 Subject: [PATCH 10/19] fix: endpoint bug --- src/cmd_raft.cc | 89 +++++++++++++++++++++++++++----------- src/config.h | 2 +- src/praft/praft.cc | 20 +++++++++ src/praft/praft.h | 1 + src/praft/praft.proto | 15 ++++++- src/praft/praft_service.cc | 43 +++++++++++++++++- src/praft/praft_service.h | 3 ++ 7 files changed, 145 insertions(+), 28 deletions(-) diff --git a/src/cmd_raft.cc b/src/cmd_raft.cc index 544a10d8d..ca0c4a780 100644 --- a/src/cmd_raft.cc +++ b/src/cmd_raft.cc @@ -17,7 +17,6 @@ #include #include "brpc/channel.h" -#include "fmt/format.h" #include "praft/praft.h" #include "pstd/log.h" @@ -113,24 +112,64 @@ void RaftNodeCmd::DoCmdRemove(PClient* client) { return; } - // Connect target - std::string peer_ip = butil::ip2str(leader_peer_id.addr.ip).c_str(); - auto port = leader_peer_id.addr.port - pikiwidb::g_config.raft_port_offset; - auto peer_id = client->argv_[2]; - auto ret = - praft_->GetClusterCmdCtx().Set(ClusterCmdType::kRemove, client, std::move(peer_ip), port, std::move(peer_id)); - if (!ret) { // other clients have removed - return client->SetRes(CmdRes::kErrOther, "Other clients have removed"); - } - praft_->GetClusterCmdCtx().ConnectTargetNode(); - INFO("Sent remove request to leader successfully"); + brpc::ChannelOptions options; + options.connection_type = brpc::CONNECTION_TYPE_SINGLE; + options.max_retry = 0; + options.connect_timeout_ms = kChannelTimeoutMS; + + NodeRemoveRequest request; + NodeRemoveResponse response; + + request.set_group_id(praft_->GetGroupID()); + request.set_endpoint(client->argv_[2]); + request.set_index(client->GetCurrentDB()); + request.set_role(0); + + auto endpoint = leader_peer_id.addr; + int retry_count = 0; + do { + brpc::Channel remove_node_channel; + if (0 != remove_node_channel.Init(endpoint, &options)) { + ERROR("Fail to init remove_node_channel to praft service!"); + client->SetRes(CmdRes::kErrOther, "Fail to init remove_node_channel."); + return; + } + + brpc::Controller cntl; + PRaftService_Stub stub(&remove_node_channel); + stub.RemoveNode(&cntl, &request, &response, nullptr); + + if (cntl.Failed()) { + ERROR("Fail to send remove node rpc to target server {}", butil::endpoint2str(endpoint).c_str()); + client->SetRes(CmdRes::kErrOther, "Failed to send remove node rpc"); + return; + } + + if (response.success()) { + client->SetRes(CmdRes::kOK, "Remove Node Success"); + return; + } + + switch (response.error_code()) { + case PRaftErrorCode::kErrorReDirect: { + butil::str2endpoint(response.leader_endpoint().c_str(), &endpoint); + endpoint.port += g_config.raft_port_offset; + break; + } + default: { + ERROR("Remove node request return false"); + client->SetRes(CmdRes::kErrOther, "Failed to Remove Node"); + return; + } + } + } while (!response.success() && ++retry_count <= 3); - // Not reply any message here, we will reply after the connection is established. - client->Clear(); + ERROR("Remove node request return false"); + client->SetRes(CmdRes::kErrOther, "Failed to Remove Node"); return; } - auto s = praft_->RemovePeer(client->argv_[2]); + auto s = praft_->RemovePeer(client->argv_[2], client->GetCurrentDB()); if (s.ok()) { client->SetRes(CmdRes::kOK); } else { @@ -220,12 +259,12 @@ void RaftClusterCmd::DoCmdJoin(PClient* client) { assert(client->argv_.size() == 4); auto group_id = client->argv_[2]; auto addr = client->argv_[3]; - butil::EndPoint endpoint; - if (0 != butil::str2endpoint(addr.c_str(), &endpoint)) { + butil::EndPoint tar_ep; + if (0 != butil::str2endpoint(addr.c_str(), &tar_ep)) { ERROR("Wrong endpoint format: {}", addr); return client->SetRes(CmdRes::kErrOther, "Wrong endpoint format"); } - endpoint.port += g_config.raft_port_offset; + tar_ep.port += g_config.raft_port_offset; if (group_id.size() != RAFT_GROUPID_LEN) { return client->SetRes(CmdRes::kInvalidParameter, @@ -252,9 +291,9 @@ void RaftClusterCmd::DoCmdJoin(PClient* client) { NodeAddRequest request; NodeAddResponse response; - auto end_point = butil::endpoint2str(PSTORE.GetEndPoint()).c_str(); + auto self_ep = butil::endpoint2str(PSTORE.GetEndPoint()); request.set_group_id(group_id); - request.set_endpoint(std::string(end_point)); + request.set_endpoint(self_ep.c_str()); request.set_index(client->GetCurrentDB()); request.set_role(0); @@ -262,7 +301,7 @@ void RaftClusterCmd::DoCmdJoin(PClient* client) { do { brpc::Channel add_node_channel; - if (0 != add_node_channel.Init(endpoint, &options)) { + if (0 != add_node_channel.Init(tar_ep, &options)) { PSTORE.RemoveRegion(group_id); ClearPaftCtx(); ERROR("Fail to init add_node_channel to praft service!"); @@ -272,12 +311,12 @@ void RaftClusterCmd::DoCmdJoin(PClient* client) { brpc::Controller cntl; PRaftService_Stub stub(&add_node_channel); - stub.AddNode(&cntl, &request, &response, NULL); + stub.AddNode(&cntl, &request, &response, nullptr); if (cntl.Failed()) { PSTORE.RemoveRegion(group_id); ClearPaftCtx(); - ERROR("Fail to send add node rpc to target server {}", addr); + ERROR("Fail to send add node rpc to target server {}, ErrCode={}", addr, cntl.ErrorCode()); client->SetRes(CmdRes::kErrOther, "Failed to send add node rpc"); return; } @@ -289,8 +328,8 @@ void RaftClusterCmd::DoCmdJoin(PClient* client) { switch (response.error_code()) { case PRaftErrorCode::kErrorReDirect: { - butil::str2endpoint(response.leader_endpoint().c_str(), &endpoint); - endpoint.port += g_config.raft_port_offset; + butil::str2endpoint(response.leader_endpoint().c_str(), &tar_ep); + tar_ep.port += g_config.raft_port_offset; break; } default: { diff --git a/src/config.h b/src/config.h index ad88661de..0b7941d13 100644 --- a/src/config.h +++ b/src/config.h @@ -281,7 +281,7 @@ class PConfig { AtomicString run_id; // The number of databases. - std::atomic databases = 16; + std::atomic databases = 3; /* * For Network I/O threads, in future version, we may delete diff --git a/src/praft/praft.cc b/src/praft/praft.cc index 25c2a99f7..bf5532287 100644 --- a/src/praft/praft.cc +++ b/src/praft/praft.cc @@ -546,6 +546,26 @@ butil::Status PRaft::RemovePeer(const std::string& peer) { return {0, "OK"}; } +butil::Status PRaft::RemovePeer(const std::string& endpoint, int index) { + if (!node_) { + return ERROR_LOG_AND_STATUS("Node is not initialized"); + } + + braft::SynchronizedClosure done; + butil::EndPoint ep; + butil::str2endpoint(endpoint.c_str(), &ep); + ep.port += g_config.raft_port_offset; + braft::PeerId peer_id(ep, index); + node_->remove_peer(peer_id, &done); + done.wait(); + + if (!done.status().ok()) { + WARN("Failed to remove peer, status: {}", done.status().error_str()); + return done.status(); + } + return done.status(); +} + butil::Status PRaft::DoSnapshot(int64_t self_snapshot_index, bool is_sync) { if (!node_) { return ERROR_LOG_AND_STATUS("Node is not initialized"); diff --git a/src/praft/praft.h b/src/praft/praft.h index 089bc012a..4ebec2777 100644 --- a/src/praft/praft.h +++ b/src/praft/praft.h @@ -104,6 +104,7 @@ class PRaft : public braft::StateMachine { butil::Status AddPeer(const std::string& peer); butil::Status AddPeer(const std::string& endpoint, int index); butil::Status RemovePeer(const std::string& peer); + butil::Status RemovePeer(const std::string& endpoint, int index); butil::Status DoSnapshot(int64_t self_snapshot_index = 0, bool is_sync = true); void ShutDown(); diff --git a/src/praft/praft.proto b/src/praft/praft.proto index 5636baf1a..9bb140e79 100644 --- a/src/praft/praft.proto +++ b/src/praft/praft.proto @@ -15,7 +15,20 @@ message NodeAddResponse { string leader_endpoint = 3; } +message NodeRemoveRequest { + string group_id = 1; + string endpoint = 2; + uint32 index = 3; + uint32 role = 4; +} + +message NodeRemoveResponse { + bool success = 1; + uint32 error_code = 2; + string leader_endpoint = 3; +} + service PRaftService { rpc AddNode(NodeAddRequest) returns (NodeAddResponse); - + rpc RemoveNode(NodeRemoveRequest) returns (NodeRemoveResponse); }; diff --git a/src/praft/praft_service.cc b/src/praft/praft_service.cc index 305f5cfca..c71305aa8 100644 --- a/src/praft/praft_service.cc +++ b/src/praft/praft_service.cc @@ -37,7 +37,6 @@ void PRaftServiceImpl::AddNode(::google::protobuf::RpcController* controller, co if (!praft_ptr->IsLeader()) { response->set_success(false); response->set_error_code(static_cast(PRaftErrorCode::kErrorReDirect)); - std::cout << "leader addr = " << praft_ptr->GetLeaderAddress() << std::endl; response->set_leader_endpoint(praft_ptr->GetLeaderAddress()); return; } @@ -51,4 +50,46 @@ void PRaftServiceImpl::AddNode(::google::protobuf::RpcController* controller, co } response->set_success(true); } + +void PRaftServiceImpl::RemoveNode(::google::protobuf::RpcController* controller, + const ::pikiwidb::NodeRemoveRequest* request, + ::pikiwidb::NodeRemoveResponse* response, ::google::protobuf::Closure* done) { + brpc::ClosureGuard done_guard(done); + auto groupid = request->group_id(); + auto end_point = request->endpoint(); + auto index = request->index(); + auto role = request->role(); + + auto db_ptr = PSTORE.GetDBByGroupID(groupid); + if (!db_ptr) { + response->set_success(false); + response->set_error_code(static_cast(PRaftErrorCode::kErrorDisMatch)); + response->set_leader_endpoint(end_point); + return; + } + auto praft_ptr = db_ptr->GetPRaft(); + if (!praft_ptr) { + response->set_success(false); + response->set_error_code(static_cast(PRaftErrorCode::kErrorDisMatch)); + response->set_leader_endpoint(end_point); + return; + } + + if (!praft_ptr->IsLeader()) { + response->set_success(false); + response->set_error_code(static_cast(PRaftErrorCode::kErrorReDirect)); + response->set_leader_endpoint(praft_ptr->GetLeaderAddress()); + return; + } + + auto status = praft_ptr->RemovePeer(end_point, index); + if (!status.ok()) { + response->set_success(false); + response->set_error_code(static_cast(PRaftErrorCode::kErrorAddNode)); + response->set_leader_endpoint(praft_ptr->GetLeaderAddress()); + return; + } + response->set_success(true); +} + } // namespace pikiwidb \ No newline at end of file diff --git a/src/praft/praft_service.h b/src/praft/praft_service.h index 8db7634f8..70e72f1c4 100644 --- a/src/praft/praft_service.h +++ b/src/praft/praft_service.h @@ -17,6 +17,9 @@ class PRaftServiceImpl : public PRaftService { PRaftServiceImpl() = default; void AddNode(::google::protobuf::RpcController* controller, const ::pikiwidb::NodeAddRequest* request, ::pikiwidb::NodeAddResponse* response, ::google::protobuf::Closure* done); + + void RemoveNode(::google::protobuf::RpcController* controller, const ::pikiwidb::NodeRemoveRequest* request, + ::pikiwidb::NodeRemoveResponse* response, ::google::protobuf::Closure* done); }; } // namespace pikiwidb From bad54e1b5d9c9bc9ae4786cd76d777a923424bfe Mon Sep 17 00:00:00 2001 From: longfar Date: Sun, 29 Sep 2024 11:14:13 +0800 Subject: [PATCH 11/19] fix: no reply of info raft command --- src/cmd_admin.cc | 5 ++--- src/cmd_admin.h | 2 +- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/cmd_admin.cc b/src/cmd_admin.cc index 19d9c4f21..a63d4280f 100644 --- a/src/cmd_admin.cc +++ b/src/cmd_admin.cc @@ -242,7 +242,7 @@ void InfoCmd::DoCmd(PClient* client) { InfoCommandStats(client, info); break; case kInfoRaft: - InfoRaft(client); + InfoRaft(client, info); break; default: break; @@ -265,7 +265,7 @@ void InfoCmd::DoCmd(PClient* client) { raft_num_voting_nodes:2 raft_node1:id=1733428433,state=connected,voting=yes,addr=localhost,port=5001,last_conn_secs=5,conn_errors=0,conn_oks=1 */ -void InfoCmd::InfoRaft(PClient* client) { +void InfoCmd::InfoRaft(PClient* client, std::string& message) { if (client->argv_.size() != 2) { return client->SetRes(CmdRes::kWrongNum, client->CmdName()); } @@ -277,7 +277,6 @@ void InfoCmd::InfoRaft(PClient* client) { } auto node_status = praft_->GetNodeStatus(); - std::string message; if (node_status.state == braft::State::STATE_END) { message += "-ERR Node is not initialized.\r\n"; return; diff --git a/src/cmd_admin.h b/src/cmd_admin.h index 661e81039..c3d5770c8 100644 --- a/src/cmd_admin.h +++ b/src/cmd_admin.h @@ -160,7 +160,7 @@ class InfoCmd : public BaseCmd { void InfoServer(std::string& info); void InfoStats(std::string& info); void InfoCPU(std::string& info); - void InfoRaft(PClient* client); + void InfoRaft(PClient* client, std::string& info); void InfoData(std::string& info); void InfoCommandStats(PClient* client, std::string& info); std::string FormatCommandStatLine(const CommandStatistics& stats); From ff2f9db6d78f35683642dfec158efa80407fefc7 Mon Sep 17 00:00:00 2001 From: longfar Date: Sun, 29 Sep 2024 17:04:16 +0800 Subject: [PATCH 12/19] feat: add pd and proxy --- ppd/main.cc | 20 +++++ ppd/pd.proto | 84 +++++++++++++++++++++ ppd/pd_service.cc | 52 +++++++++++++ ppd/pd_service.h | 37 +++++++++ pproxy/main.cc | 20 +++++ pproxy/proxy.proto | 28 +++++++ pproxy/proxy_service.cc | 35 +++++++++ pproxy/proxy_service.h | 14 ++++ pproxy/router.h | 161 ++++++++++++++++++++++++++++++++++++++++ pproxy/task_manager.cc | 84 +++++++++++++++++++++ pproxy/task_manager.h | 74 ++++++++++++++++++ 11 files changed, 609 insertions(+) create mode 100644 ppd/main.cc create mode 100644 ppd/pd.proto create mode 100644 ppd/pd_service.cc create mode 100644 ppd/pd_service.h create mode 100644 pproxy/main.cc create mode 100644 pproxy/proxy.proto create mode 100644 pproxy/proxy_service.cc create mode 100644 pproxy/proxy_service.h create mode 100644 pproxy/router.h create mode 100644 pproxy/task_manager.cc create mode 100644 pproxy/task_manager.h diff --git a/ppd/main.cc b/ppd/main.cc new file mode 100644 index 000000000..1b87a6b0e --- /dev/null +++ b/ppd/main.cc @@ -0,0 +1,20 @@ +#include "pd_service.h" + +int main(int argc, char* argv[]) { + brpc::Server server; + + PlacementDriverServiceImpl service; + if (server.AddService(&service, brpc::SERVER_OWNS_SERVICE) != 0) { + fprintf(stderr, "Fail to add service!\n"); + return -1; + } + + // 启动服务 + if (server.Start(8080, nullptr) != 0) { + fprintf(stderr, "Fail to start server!\n"); + return -1; + } + + server.RunUntilAskedToQuit(); + return 0; +} \ No newline at end of file diff --git a/ppd/pd.proto b/ppd/pd.proto new file mode 100644 index 000000000..7d3ab454e --- /dev/null +++ b/ppd/pd.proto @@ -0,0 +1,84 @@ +syntax="proto3"; +package pikiwidb; +option cc_generic_services = true; + +import "store.proto"; + +message GetClusterInfoRequest { +}; + +message GetClusterInfoResponse { + bool success = 1; + repeated Store store = 2; +}; + +message Store { + int64 store_id = 1; + string ip = 2; + int32 port = 3; + StoreState state = 4; + repeated Region region = 5; +}; + +message Region { + int64 region_id = 1; + optional string start_key = 2; + optional string end_key = 3; + repeated RegionEpoch region_epoch = 4; + repeated Peer peers = 5; +}; + +message RegionEpoch { + int64 conf_change_ver = 1; // conf change version + int64 region_ver = 2; // region version (split or merge) +}; + +enum StoreState { + UP = 0; + OFFLINE = 1; + TOMBSTONE = 2; +}; + +message CreateAllRegionsRequest { + int64 regions_count = 1; + int32 region_peers_count = 2; + repeated RegionOptions regionOptions = 3; +}; + +message CreateAllRegionsResponse { + bool success = 1; +}; + +message DeleteAllRegionsRequest { +}; + +message DeleteAllRegionsResponse { + bool success = 1; +}; + +message AddStoreRequest { + string ip = 1; + int32 port = 2; +}; + +message AddStoreResponse { + bool success = 1; + optional int64 store_id = 2; + optional string redirect = 3; +}; + +message RemoveStoreRequest { + int64 store_id = 1; +}; + +message RemoveStoreResponse { + bool success = 1; +}; + +service PlacementDriverService { + rpc CreateAllRegions(CreateAllRegionsRequest) returns (CreateAllRegionsResponse); + rpc DeleteAllRegions(DeleteAllRegionsRequest) returns (DeleteAllRegionsResponse); + rpc AddStore(AddStoreRequest) returns (AddStoreResponse); + rpc RemoveStore(RemoveStoreRequest) returns (RemoveStoreResponse); + rpc GetClusterInfo(GetClusterInfoRequest) returns (GetClusterInfoResponse); +}; \ No newline at end of file diff --git a/ppd/pd_service.cc b/ppd/pd_service.cc new file mode 100644 index 000000000..3bf016baa --- /dev/null +++ b/ppd/pd_service.cc @@ -0,0 +1,52 @@ +#include "pd_service.h" + +#include "pd_server.h" + +namespace pikiwidb { +void PlacementDriverServiceImpl::CreateAllRegions(::google::protobuf::RpcController* controller, + const ::pikiwidb::CreateAllRegionsRequest* request, + ::pikiwidb::CreateAllRegionsResponse* response, + ::google::protobuf::Closure* done) {} + +void PlacementDriverServiceImpl::DeleteAllRegions(::google::protobuf::RpcController* controller, + const ::pikiwidb::DeleteAllRegionsRequest* request, + ::pikiwidb::DeleteAllRegionsResponse* response, + ::google::protobuf::Closure* done) {} + +void PlacementDriverServiceImpl::AddStore(::google::protobuf::RpcController* controller, + const ::pikiwidb::AddStoreRequest* request, + ::pikiwidb::AddStoreResponse* response, ::google::protobuf::Closure* done) { + brpc::ClosureGuard done_guard(done); + auto [success, store_id] = PDSERVER.AddStore(request->ip(), request->port()); + if (!success) { + response->set_success(false); + return; + } + + response->set_success(true); + response->set_store_id(store_id); +} + +void PlacementDriverServiceImpl::RemoveStore(::google::protobuf::RpcController* controller, + const ::pikiwidb::RemoveStoreRequest* request, + ::pikiwidb::RemoveStoreResponse* response, + ::google::protobuf::Closure* done) {} + +void PlacementDriverServiceImpl::GetClusterInfo(::google::protobuf::RpcController* controller, + const ::pikiwidb::GetClusterInfoRequest* request, + ::pikiwidb::GetClusterInfoResponse* response, + ::google::protobuf::Closure* done) { + brpc::ClosureGuard done_guard(done); + PDSERVER.GetClusterInfo(response); +} + +void PlacementDriverServiceImpl::OpenPDScheduling(::google::protobuf::RpcController* controller, + const ::pikiwidb::OpenPDSchedulingRequest* request, + ::pikiwidb::OpenPDSchedulingResponse* response, + ::google::protobuf::Closure* done) {} + +void PlacementDriverServiceImpl::ClosePDScheduling(::google::protobuf::RpcController* controller, + const ::pikiwidb::ClosePDSchedulingRequest* request, + ::pikiwidb::ClosePDSchedulingResponse* response, + ::google::protobuf::Closure* done) {} +} // namespace pikiwidb \ No newline at end of file diff --git a/ppd/pd_service.h b/ppd/pd_service.h new file mode 100644 index 000000000..282ac3bca --- /dev/null +++ b/ppd/pd_service.h @@ -0,0 +1,37 @@ +#pragma once + +#include "pd.pb.h" + +namespace pikiwidb { + +class PlacementDriverServiceImpl : public PlacementDriverService { + public: + PlacementDriverServiceImpl() = default; + + void CreateAllRegions(::google::protobuf::RpcController* controller, + const ::pikiwidb::CreateAllRegionsRequest* request, + ::pikiwidb::CreateAllRegionsResponse* response, ::google::protobuf::Closure* done) override; + + void DeleteAllRegions(::google::protobuf::RpcController* controller, + const ::pikiwidb::DeleteAllRegionsRequest* request, + ::pikiwidb::DeleteAllRegionsResponse* response, ::google::protobuf::Closure* done) override; + + void AddStore(::google::protobuf::RpcController* controller, const ::pikiwidb::AddStoreRequest* request, + ::pikiwidb::AddStoreResponse* response, ::google::protobuf::Closure* done) override; + + void RemoveStore(::google::protobuf::RpcController* controller, const ::pikiwidb::RemoveStoreRequest* request, + ::pikiwidb::RemoveStoreResponse* response, ::google::protobuf::Closure* done) override; + + void GetClusterInfo(::google::protobuf::RpcController* controller, const ::pikiwidb::GetClusterInfoRequest* request, + ::pikiwidb::GetClusterInfoResponse* response, ::google::protobuf::Closure* done) override; + + void OpenPDScheduling(::google::protobuf::RpcController* controller, + const ::pikiwidb::OpenPDSchedulingRequest* request, + ::pikiwidb::OpenPDSchedulingResponse* response, ::google::protobuf::Closure* done) override; + + void ClosePDScheduling(::google::protobuf::RpcController* controller, + const ::pikiwidb::ClosePDSchedulingRequest* request, + ::pikiwidb::ClosePDSchedulingResponse* response, ::google::protobuf::Closure* done) override; +}; + +} // namespace pikiwidb \ No newline at end of file diff --git a/pproxy/main.cc b/pproxy/main.cc new file mode 100644 index 000000000..9304fcb55 --- /dev/null +++ b/pproxy/main.cc @@ -0,0 +1,20 @@ +#include "proxy_service.h" + +int main(int argc, char* argv[]) { + brpc::Server server; + + ProxyServiceImpl service; + if (server.AddService(&service, brpc::SERVER_OWNS_SERVICE) != 0) { + fprintf(stderr, "Fail to add service!\n"); + return -1; + } + + // 启动服务 + if (server.Start(8080, nullptr) != 0) { + fprintf(stderr, "Fail to start server!\n"); + return -1; + } + + server.RunUntilAskedToQuit(); + return 0; +} \ No newline at end of file diff --git a/pproxy/proxy.proto b/pproxy/proxy.proto new file mode 100644 index 000000000..c899cfd81 --- /dev/null +++ b/pproxy/proxy.proto @@ -0,0 +1,28 @@ +syntax = "proto3"; +package pikiwidb.proxy; +option cc_generic_services = true; + +// 定义请求和响应 +message RunCommandRequest { + string command = 1; +} + +message RunCommandResponse { + string output = 1; +} +message GetRouteInfoRequest { +} +message GetRouteInfoResponse { + message RouteInfo { + string group_id = 1; + string endpoint = 2; + int32 role = 3; + } + repeated RouteInfo infos = 1; +} + +// 定义服务 +service ProxyService { + rpc RunCommand(RunCommandRequest) returns (RunCommandResponse); + rpc GetRouteInfo(GetRouteInfoRequest) returns (GetRouteInfoResponse); +} diff --git a/pproxy/proxy_service.cc b/pproxy/proxy_service.cc new file mode 100644 index 000000000..2bb4c4a3a --- /dev/null +++ b/pproxy/proxy_service.cc @@ -0,0 +1,35 @@ +#include "proxy_service.h" + +namespace pikiwidb::proxy { +void ProxyServiceImpl::RunCommand(::google::protobuf::RpcController* cntl, + const pikiwidb::proxy::RunCommandRequest* request, + pikiwidb::proxy::RunCommandResponse* response, ::google::protobuf::Closure* done) { + std::string command = request->command(); + std::string output = ExecuteCommand(command); + + response->set_output(output); + + done->Run(); +} +void ProxyServiceImpl::GetRouteINfo(::google::protobuf::RpcController* cntl, + const pikiwidb::proxy::GetRouteInfoRequest* request, + pikiwidb::proxy::GetRouteInfoResponse* response, + ::google::protobuf::Closure* done) { +} + +std::string ProxyServiceImpl::ExecuteCommand(const std::string& command) { + std::array buffer; + std::string result; + + // 使用 popen 执行命令 + std::unique_ptr pipe(popen(command.c_str(), "r"), pclose); + if (!pipe) { + return "popen() failed!"; + } + while (fgets(buffer.data(), buffer.size(), pipe.get()) != nullptr) { + result += buffer.data(); + } + return result; +} + +} // namespace pikiwidb::proxy \ No newline at end of file diff --git a/pproxy/proxy_service.h b/pproxy/proxy_service.h new file mode 100644 index 000000000..3b6be94dd --- /dev/null +++ b/pproxy/proxy_service.h @@ -0,0 +1,14 @@ +#pragma once + +#include "proxy.pb.h" + +class ProxyServiceImpl : public ProxyService { + public: + void RunCommand(::google::protobuf::RpcController* cntl, const pikiwidb::proxy::RunCommandRequest* request, + pikiwidb::proxy::RunCommandResponse* response, ::google::protobuf::Closure* done) override; + void GetRouteInfo(::google::protobuf::RpcController* cntl, const pikiwidb::proxy::GetRouteInfoRequest* request, + pikiwidb::proxy::GetRouteInfoResponse* response, ::google::protobuf::Closure* done) override; + + private: + std::string ExecuteCommand(const std::string& command); +}; diff --git a/pproxy/router.h b/pproxy/router.h new file mode 100644 index 000000000..a9981c88b --- /dev/null +++ b/pproxy/router.h @@ -0,0 +1,161 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +class Router { + public: + Router() { + // maximum 100 parameters + params.reserve(100); + } + + Router *add(const char *method, const char *pattern, std::function &)> handler) { + // step over any initial slash + if (pattern[0] == '/') { + pattern++; + } + + std::vector nodes; + // nodes.push_back(method); + + const char *stop; + const char *start = pattern; + const char *end_ptr = pattern + strlen(pattern); + do { + stop = getNextSegment(start, end_ptr); + + // std::cout << "Segment(" << std::string(start, stop - start) << ")" << std::endl; + + nodes.emplace_back(start, stop - start); + + start = stop + 1; + } while (stop != end_ptr); + + // if pattern starts with / then move 1+ and run inline slash parser + + add(nodes, handlers.size()); + handlers.push_back(handler); + + compile(); + return this; + } + + void compile() { + compiled_tree.clear(); + compile_tree(tree); + } + + void route(const char *method, unsigned int method_length, const char *url, unsigned int url_length, userData) { + handlers[lookup(url, url_length)](userData, params); + params.clear(); + } + + private: + std::vector &)>> handlers; + std::vector params; + + struct Node { + std::string name; + std::map children; + short handler; + }; + + Node *tree = new Node({"GET", {}, -1}); + std::string compiled_tree; + + void add(const std::vector &route, short handler) { + Node *parent = tree; + for (const std::string &node : route) { + if (parent->children.find(node) == parent->children.end()) { + parent->children[node] = new Node({node, {}, handler}); + } + parent = parent->children[node]; + } + } + + unsigned short compile_tree(Node *n) { + unsigned short nodeLength = 6 + n->name.length(); + for (const auto &c : n->children) { + nodeLength += compile_tree(c.second); + } + + unsigned short nodeNameLength = n->name.length(); + + std::string compiledNode; + compiledNode.append(reinterpret_cast(&nodeLength), sizeof(nodeLength)); + compiledNode.append(reinterpret_cast(&nodeNameLength), sizeof(nodeNameLength)); + compiledNode.append(reinterpret_cast(&n->handler), sizeof(n->handler)); + compiledNode.append(n->name.data(), n->name.length()); + + compiled_tree = compiledNode + compiled_tree; + return nodeLength; + } + + inline const char *find_node(const char *parent_node, const char *name, int name_length) { + unsigned short nodeLength = *(unsigned short *)&parent_node[0]; + unsigned short nodeNameLength = *(unsigned short *)&parent_node[2]; + + // std::cout << "Finding node: <" << std::string(name, name_length) << ">" << std::endl; + + const char *stoppp = parent_node + nodeLength; + for (const char *candidate = parent_node + 6 + nodeNameLength; candidate < stoppp;) { + unsigned short nodeLength = *(unsigned short *)&candidate[0]; + unsigned short nodeNameLength = *(unsigned short *)&candidate[2]; + + // whildcard, parameter, equal + if (nodeNameLength == 0) { + return candidate; + } else if (candidate[6] == ':') { + // parameter + + // todo: push this pointer on the stack of args! + params.push_back(std::string_view({name, static_cast(name_length)})); + + return candidate; + } else if (nodeNameLength == name_length && !memcmp(candidate + 6, name, name_length)) { + return candidate; + } + + candidate = candidate + nodeLength; + } + + return nullptr; + } + + // returns next slash from start or end + inline const char *getNextSegment(const char *start, const char *end) { + const char *stop = static_cast(memchr(start, '/', end - start)); + return stop ? stop : end; + } + + // should take method also! + inline int lookup(const char *url, int length) { + // all urls start with / + url++; + length--; + + const char *treeStart = static_cast(compiled_tree.data()); + + const char *stop; + const char *start = url; + const char *end_ptr = url + length; + do { + stop = getNextSegment(start, end_ptr); + + // std::cout << "Matching(" << std::string(start, stop - start) << ")" << std::endl; + + if (nullptr == (treeStart = find_node(treeStart, start, stop - start))) { + return -1; + } + + start = stop + 1; + } while (stop != end_ptr); + + return *(short *)&treeStart[4]; + } +}; \ No newline at end of file diff --git a/pproxy/task_manager.cc b/pproxy/task_manager.cc new file mode 100644 index 000000000..65207950f --- /dev/null +++ b/pproxy/task_manager.cc @@ -0,0 +1,84 @@ +#include "task_manager.h" + +#include +#include + +Task::Status Task::TextToStatus(const std::string& input) { + if (input == "pending") { + return Task::pending; + } else if (input == "completed") { + return Task::completed; + } else if (input == "deleted") { + return Task::deleted; + } else if (input == "recurring") { + return Task::recurring; + } else if (input == "waiting") { + return Task::waiting; + } + + return Task::pending; +} +std::string Task::StatusToText(Status s) { + if (s == Task::pending) { + return "pending"; + } else if (s == Task::completed) { + return "completed"; + } else if (s == Task::deleted) { + return "deleted"; + } else if (s == Task::recurring) { + return "recurring"; + } else if (s == Task::waiting) { + return "waiting"; + } + return "pending"; +} + +TaskManager::TaskManager(std::shared_ptr threadpool, size_t maxWorkers) + : _threadpool(std::move(threadpool)), _maxWorkers(maxWorkers) {} + +std::future TaskManager::stop() { + auto task = std::make_shared>([this] { + std::unique_lock guard(_mutex); + bool isLast = _workerCount == 1; + + // Guarantee that the task finishes last. + while (!isLast) { + guard.unlock(); + std::this_thread::sleep_for(std::chrono::milliseconds(1)); + guard.lock(); + isLast = _workerCount == 1; + } + }); + auto future = task->get_future(); + + // Adding a new task and expecting the future guarantees that the last batch of tasks is being executed. + auto functor = [task = std::move(task)]() mutable { (*task)(); }; + std::lock_guard guard(_mutex); + + _stopped = true; + _tasks.emplace(std::move(functor)); + this->processTasks(); + + return future; +} + +void TaskManager::addTask(std::function functor) { + std::lock_guard guard(_mutex); + + if (_stopped) { + return; + } + _tasks.emplace(std::move(functor), Clock::now()); + this->processTasks(); +} + +void TaskManager::processTasks() { + if (_tasks.empty() || _workerCount == _maxWorkers) { + return; + } + auto task = std::move(_tasks.front()); + _tasks.pop(); + + ++_workerCount; + _threadpool->execute(std::move(task)); +} \ No newline at end of file diff --git a/pproxy/task_manager.h b/pproxy/task_manager.h new file mode 100644 index 000000000..ad1857d29 --- /dev/null +++ b/pproxy/task_manager.h @@ -0,0 +1,74 @@ +#pragma once + +#include +#include +#include +#include +#include + +class Task { + public: + Task(); + Task(const Task&); + Task& operator=(const Task&); + bool operator==(const Task&); + Task(const std::string&); + ~Task(); + + enum Status { pending, completed, deleted, recurring, waiting }; + + static Status TextToStatus(const std::string& input); + static std::string StatusToText(Status s); + + void SetEntry(); + + Status GetStatus() const; + void SetStatus(Status); + + private: + int determineVersion(const std::string&); + void legacyParse(const std::string&); + int id; +}; + +class TaskManager { + public: + TaskManager(std::shared_ptr threadpool, size_t maxWorkers); + + std::future Stop(); + + template + auto Push(F&& function, Args&&... args) // + -> std::future::type> { + using ReturnType = typename std::result_of::type; + + auto task = std::make_shared>( // + std::bind(std::forward(function), std::forward(args)...)); + auto future = task->get_future(); + + auto functor = [this, task = std::move(task)]() mutable { + (*task)(); + { + std::lock_guard guard(_mutex); + + --_workerCount; + this->processTasks(); + } + }; + this->addTask(std::move(functor)); + + return future; + } + + private: + void addTask(std::function functor); + void processTasks(); + + private: + std::shared_ptr<::Threadpool> _threadpool; + std::queue _tasks; + std::mutex _mutex; + size_t _maxWorkers; + size_t _workerCount{0}; + bool _stopped{false}; +}; \ No newline at end of file From 5245b11b4bd84f7305b154c50191159de9abf8c6 Mon Sep 17 00:00:00 2001 From: longfar Date: Sun, 29 Sep 2024 17:10:05 +0800 Subject: [PATCH 13/19] fix: code format --- src/base_cmd.cc | 2 +- src/praft/praft.h | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/base_cmd.cc b/src/base_cmd.cc index 5fcd499ff..e247ad084 100644 --- a/src/base_cmd.cc +++ b/src/base_cmd.cc @@ -9,8 +9,8 @@ #include "base_cmd.h" -#include "pstd/log.h" #include "praft/praft.h" +#include "pstd/log.h" #include "config.h" #include "pikiwidb.h" diff --git a/src/praft/praft.h b/src/praft/praft.h index 4ebec2777..f31e5fdc5 100644 --- a/src/praft/praft.h +++ b/src/praft/praft.h @@ -166,8 +166,8 @@ class PRaft : public braft::StateMachine { scoped_refptr snapshot_adaptor_ = nullptr; ClusterCmdContext cluster_cmd_ctx_{this}; // context for cluster join/remove command - std::string group_id_; // group id - int db_id_ = 0; // db_id + std::string group_id_; // group id + int db_id_ = 0; // db_id bool is_node_first_start_up_ = true; }; From d242a2d3ac9a8fe6cfa799ab5155aa05ec1a0334 Mon Sep 17 00:00:00 2001 From: longfar Date: Sun, 20 Oct 2024 16:19:05 +0800 Subject: [PATCH 14/19] refactor: add liscence --- ppd/main.cc | 7 +++++++ ppd/pd_service.cc | 7 +++++++ ppd/pd_service.h | 7 +++++++ pproxy/main.cc | 7 +++++++ pproxy/proxy_service.cc | 7 +++++++ pproxy/proxy_service.h | 7 +++++++ pproxy/router.h | 7 +++++++ pproxy/task_manager.cc | 7 +++++++ pproxy/task_manager.h | 7 +++++++ 9 files changed, 63 insertions(+) diff --git a/ppd/main.cc b/ppd/main.cc index 1b87a6b0e..2d6937b06 100644 --- a/ppd/main.cc +++ b/ppd/main.cc @@ -1,3 +1,10 @@ +/* + * Copyright (c) 2023-present, OpenAtom Foundation, Inc. All rights reserved. + * This source code is licensed under the BSD-style license found in the + * LICENSE file in the root directory of this source tree. An additional grant + * of patent rights can be found in the PATENTS file in the same directory. + */ + #include "pd_service.h" int main(int argc, char* argv[]) { diff --git a/ppd/pd_service.cc b/ppd/pd_service.cc index 3bf016baa..dcffc0588 100644 --- a/ppd/pd_service.cc +++ b/ppd/pd_service.cc @@ -1,3 +1,10 @@ +/* + * Copyright (c) 2023-present, OpenAtom Foundation, Inc. All rights reserved. + * This source code is licensed under the BSD-style license found in the + * LICENSE file in the root directory of this source tree. An additional grant + * of patent rights can be found in the PATENTS file in the same directory. + */ + #include "pd_service.h" #include "pd_server.h" diff --git a/ppd/pd_service.h b/ppd/pd_service.h index 282ac3bca..37bcbf113 100644 --- a/ppd/pd_service.h +++ b/ppd/pd_service.h @@ -1,3 +1,10 @@ +/* + * Copyright (c) 2023-present, OpenAtom Foundation, Inc. All rights reserved. + * This source code is licensed under the BSD-style license found in the + * LICENSE file in the root directory of this source tree. An additional grant + * of patent rights can be found in the PATENTS file in the same directory. + */ + #pragma once #include "pd.pb.h" diff --git a/pproxy/main.cc b/pproxy/main.cc index 9304fcb55..d325e4834 100644 --- a/pproxy/main.cc +++ b/pproxy/main.cc @@ -1,3 +1,10 @@ +/* + * Copyright (c) 2023-present, OpenAtom Foundation, Inc. All rights reserved. + * This source code is licensed under the BSD-style license found in the + * LICENSE file in the root directory of this source tree. An additional grant + * of patent rights can be found in the PATENTS file in the same directory. + */ + #include "proxy_service.h" int main(int argc, char* argv[]) { diff --git a/pproxy/proxy_service.cc b/pproxy/proxy_service.cc index 2bb4c4a3a..24be1b01e 100644 --- a/pproxy/proxy_service.cc +++ b/pproxy/proxy_service.cc @@ -1,3 +1,10 @@ +/* + * Copyright (c) 2023-present, OpenAtom Foundation, Inc. All rights reserved. + * This source code is licensed under the BSD-style license found in the + * LICENSE file in the root directory of this source tree. An additional grant + * of patent rights can be found in the PATENTS file in the same directory. + */ + #include "proxy_service.h" namespace pikiwidb::proxy { diff --git a/pproxy/proxy_service.h b/pproxy/proxy_service.h index 3b6be94dd..3936cb200 100644 --- a/pproxy/proxy_service.h +++ b/pproxy/proxy_service.h @@ -1,3 +1,10 @@ +/* + * Copyright (c) 2023-present, OpenAtom Foundation, Inc. All rights reserved. + * This source code is licensed under the BSD-style license found in the + * LICENSE file in the root directory of this source tree. An additional grant + * of patent rights can be found in the PATENTS file in the same directory. + */ + #pragma once #include "proxy.pb.h" diff --git a/pproxy/router.h b/pproxy/router.h index a9981c88b..fad1e6cd1 100644 --- a/pproxy/router.h +++ b/pproxy/router.h @@ -1,3 +1,10 @@ +/* + * Copyright (c) 2023-present, OpenAtom Foundation, Inc. All rights reserved. + * This source code is licensed under the BSD-style license found in the + * LICENSE file in the root directory of this source tree. An additional grant + * of patent rights can be found in the PATENTS file in the same directory. + */ + #pragma once #include diff --git a/pproxy/task_manager.cc b/pproxy/task_manager.cc index 65207950f..a21053b85 100644 --- a/pproxy/task_manager.cc +++ b/pproxy/task_manager.cc @@ -1,3 +1,10 @@ +/* + * Copyright (c) 2023-present, OpenAtom Foundation, Inc. All rights reserved. + * This source code is licensed under the BSD-style license found in the + * LICENSE file in the root directory of this source tree. An additional grant + * of patent rights can be found in the PATENTS file in the same directory. + */ + #include "task_manager.h" #include diff --git a/pproxy/task_manager.h b/pproxy/task_manager.h index ad1857d29..8d70cf5e7 100644 --- a/pproxy/task_manager.h +++ b/pproxy/task_manager.h @@ -1,3 +1,10 @@ +/* + * Copyright (c) 2023-present, OpenAtom Foundation, Inc. All rights reserved. + * This source code is licensed under the BSD-style license found in the + * LICENSE file in the root directory of this source tree. An additional grant + * of patent rights can be found in the PATENTS file in the same directory. + */ + #pragma once #include From 5dff21ea5dcc83f35050de225376148bd62c1798 Mon Sep 17 00:00:00 2001 From: longfar Date: Mon, 21 Oct 2024 10:57:42 +0800 Subject: [PATCH 15/19] refactor: move to cc file --- pproxy/router.cc | 145 +++++++++++++++++++++++++++++++++++++++++++++++ pproxy/router.h | 136 +++----------------------------------------- 2 files changed, 154 insertions(+), 127 deletions(-) create mode 100644 pproxy/router.cc diff --git a/pproxy/router.cc b/pproxy/router.cc new file mode 100644 index 000000000..c8490a71d --- /dev/null +++ b/pproxy/router.cc @@ -0,0 +1,145 @@ +/* + * Copyright (c) 2023-present, OpenAtom Foundation, Inc. All rights reserved. + * This source code is licensed under the BSD-style license found in the + * LICENSE file in the root directory of this source tree. An additional grant + * of patent rights can be found in the PATENTS file in the same directory. + */ + +#include "router.h" +#include + +Router::Router() { + // maximum 100 parameters + params.reserve(100); +} + +Router *Router::add(const char *method, const char *pattern, + std::function &)> handler) { + // step over any initial slash + if (pattern[0] == '/') { + pattern++; + } + + std::vector nodes; + // nodes.push_back(method); + + const char *stop; + const char *start = pattern; + const char *end_ptr = pattern + strlen(pattern); + do { + stop = getNextSegment(start, end_ptr); + + // std::cout << "Segment(" << std::string(start, stop - start) << ")" << std::endl; + + nodes.emplace_back(start, stop - start); + + start = stop + 1; + } while (stop != end_ptr); + + // if pattern starts with / then move 1+ and run inline slash parser + + add(nodes, handlers.size()); + handlers.push_back(handler); + + compile(); + return this; +} + +void Router::compile() { + compiled_tree.clear(); + compile_tree(tree); +} + +void Router::route(const char *method, unsigned int method_length, const char *url, unsigned int url_length, userData) { + handlers[lookup(url, url_length)](userData, params); + params.clear(); +} + +void Router::add(const std::vector &route, short handler) { + Node *parent = tree; + for (const std::string &node : route) { + if (parent->children.find(node) == parent->children.end()) { + parent->children[node] = new Node({node, {}, handler}); + } + parent = parent->children[node]; + } +} + +unsigned short Router::compile_tree(Node *n) { + unsigned short nodeLength = 6 + n->name.length(); + for (const auto &c : n->children) { + nodeLength += compile_tree(c.second); + } + + unsigned short nodeNameLength = n->name.length(); + + std::string compiledNode; + compiledNode.append(reinterpret_cast(&nodeLength), sizeof(nodeLength)); + compiledNode.append(reinterpret_cast(&nodeNameLength), sizeof(nodeNameLength)); + compiledNode.append(reinterpret_cast(&n->handler), sizeof(n->handler)); + compiledNode.append(n->name.data(), n->name.length()); + + compiled_tree = compiledNode + compiled_tree; + return nodeLength; +} + +const char *Router::find_node(const char *parent_node, const char *name, int name_length) { + unsigned short nodeLength = *reinterpret_cast(&parent_node[0]); + unsigned short nodeNameLength = *reinterpret_cast(&parent_node[2]); + + // std::cout << "Finding node: <" << std::string(name, name_length) << ">" << std::endl; + + const char *stoppp = parent_node + nodeLength; + for (const char *candidate = parent_node + 6 + nodeNameLength; candidate < stoppp;) { + unsigned short nodeLength = *reinterpret_cast(&candidate[0]); + unsigned short nodeNameLength = *reinterpret_cast(&candidate[2]); + + // whildcard, parameter, equal + if (nodeNameLength == 0) { + return candidate; + } else if (candidate[6] == ':') { + // parameter + + // todo: push this pointer on the stack of args! + params.push_back(std::string_view({name, static_cast(name_length)})); + + return candidate; + } else if (nodeNameLength == name_length && !memcmp(candidate + 6, name, name_length)) { + return candidate; + } + + candidate = candidate + nodeLength; + } + + return nullptr; +} + +const char *Router::getNextSegment(const char *start, const char *end) { + const char *stop = static_cast(memchr(start, '/', end - start)); + return stop ? stop : end; +} + +int Router::lookup(const char *url, int length) { + // all urls start with / + url++; + length--; + + const char *treeStart = static_cast(compiled_tree.data()); + + const char *stop; + const char *start = url; + const char *end_ptr = url + length; + do { + stop = getNextSegment(start, end_ptr); + + // std::cout << "Matching(" << std::string(start, stop - start) << ")" << std::endl; + + if (nullptr == (treeStart = find_node(treeStart, start, stop - start))) { + return -1; + } + + start = stop + 1; + } while (stop != end_ptr); + + return *reinterpret_cast(&treeStart[4]); +} diff --git a/pproxy/router.h b/pproxy/router.h index fad1e6cd1..644773b45 100644 --- a/pproxy/router.h +++ b/pproxy/router.h @@ -7,7 +7,6 @@ #pragma once -#include #include #include #include @@ -16,51 +15,13 @@ class Router { public: - Router() { - // maximum 100 parameters - params.reserve(100); - } + Router(); - Router *add(const char *method, const char *pattern, std::function &)> handler) { - // step over any initial slash - if (pattern[0] == '/') { - pattern++; - } + Router *add(const char *method, const char *pattern, std::function &)> handler); - std::vector nodes; - // nodes.push_back(method); + void compile(); - const char *stop; - const char *start = pattern; - const char *end_ptr = pattern + strlen(pattern); - do { - stop = getNextSegment(start, end_ptr); - - // std::cout << "Segment(" << std::string(start, stop - start) << ")" << std::endl; - - nodes.emplace_back(start, stop - start); - - start = stop + 1; - } while (stop != end_ptr); - - // if pattern starts with / then move 1+ and run inline slash parser - - add(nodes, handlers.size()); - handlers.push_back(handler); - - compile(); - return this; - } - - void compile() { - compiled_tree.clear(); - compile_tree(tree); - } - - void route(const char *method, unsigned int method_length, const char *url, unsigned int url_length, userData) { - handlers[lookup(url, url_length)](userData, params); - params.clear(); - } + void route(const char *method, unsigned int method_length, const char *url, unsigned int url_length, userData); private: std::vector &)>> handlers; @@ -75,94 +36,15 @@ class Router { Node *tree = new Node({"GET", {}, -1}); std::string compiled_tree; - void add(const std::vector &route, short handler) { - Node *parent = tree; - for (const std::string &node : route) { - if (parent->children.find(node) == parent->children.end()) { - parent->children[node] = new Node({node, {}, handler}); - } - parent = parent->children[node]; - } - } - - unsigned short compile_tree(Node *n) { - unsigned short nodeLength = 6 + n->name.length(); - for (const auto &c : n->children) { - nodeLength += compile_tree(c.second); - } - - unsigned short nodeNameLength = n->name.length(); - - std::string compiledNode; - compiledNode.append(reinterpret_cast(&nodeLength), sizeof(nodeLength)); - compiledNode.append(reinterpret_cast(&nodeNameLength), sizeof(nodeNameLength)); - compiledNode.append(reinterpret_cast(&n->handler), sizeof(n->handler)); - compiledNode.append(n->name.data(), n->name.length()); - - compiled_tree = compiledNode + compiled_tree; - return nodeLength; - } + void add(const std::vector &route, short handler); - inline const char *find_node(const char *parent_node, const char *name, int name_length) { - unsigned short nodeLength = *(unsigned short *)&parent_node[0]; - unsigned short nodeNameLength = *(unsigned short *)&parent_node[2]; + unsigned short compile_tree(Node *n); - // std::cout << "Finding node: <" << std::string(name, name_length) << ">" << std::endl; - - const char *stoppp = parent_node + nodeLength; - for (const char *candidate = parent_node + 6 + nodeNameLength; candidate < stoppp;) { - unsigned short nodeLength = *(unsigned short *)&candidate[0]; - unsigned short nodeNameLength = *(unsigned short *)&candidate[2]; - - // whildcard, parameter, equal - if (nodeNameLength == 0) { - return candidate; - } else if (candidate[6] == ':') { - // parameter - - // todo: push this pointer on the stack of args! - params.push_back(std::string_view({name, static_cast(name_length)})); - - return candidate; - } else if (nodeNameLength == name_length && !memcmp(candidate + 6, name, name_length)) { - return candidate; - } - - candidate = candidate + nodeLength; - } - - return nullptr; - } + const char *find_node(const char *parent_node, const char *name, int name_length); // returns next slash from start or end - inline const char *getNextSegment(const char *start, const char *end) { - const char *stop = static_cast(memchr(start, '/', end - start)); - return stop ? stop : end; - } + const char *getNextSegment(const char *start, const char *end); // should take method also! - inline int lookup(const char *url, int length) { - // all urls start with / - url++; - length--; - - const char *treeStart = static_cast(compiled_tree.data()); - - const char *stop; - const char *start = url; - const char *end_ptr = url + length; - do { - stop = getNextSegment(start, end_ptr); - - // std::cout << "Matching(" << std::string(start, stop - start) << ")" << std::endl; - - if (nullptr == (treeStart = find_node(treeStart, start, stop - start))) { - return -1; - } - - start = stop + 1; - } while (stop != end_ptr); - - return *(short *)&treeStart[4]; - } + int lookup(const char *url, int length); }; \ No newline at end of file From f1a1048fd973f63f969e861023ba9b7ba24bb811 Mon Sep 17 00:00:00 2001 From: longfar Date: Mon, 21 Oct 2024 17:05:14 +0800 Subject: [PATCH 16/19] fix: comment --- ppd/main.cc | 24 +++++++++++++++++++----- ppd/pd.proto | 17 +++++++++++++---- ppd/pd_service.cc | 2 ++ pproxy/main.cc | 24 ++++++++++++++++++------ pproxy/proxy.proto | 2 -- pproxy/proxy_service.cc | 41 ++++++++++++++++++++++++++++++++--------- pproxy/router.cc | 3 ++- pproxy/router.h | 5 +++-- pproxy/task_manager.cc | 30 ++++++++++++++++-------------- pproxy/task_manager.h | 16 ++++++++-------- src/base_cmd.cc | 2 +- src/cmd_admin.cc | 11 +++-------- src/cmd_raft.cc | 6 +++--- 13 files changed, 120 insertions(+), 63 deletions(-) diff --git a/ppd/main.cc b/ppd/main.cc index 2d6937b06..e3c79645e 100644 --- a/ppd/main.cc +++ b/ppd/main.cc @@ -5,23 +5,37 @@ * of patent rights can be found in the PATENTS file in the same directory. */ +#include "brpc/server.h" +#include "butil/errno.h" +#include "gflags/gflags.h" +#include "spdlog/spdlog.h" + #include "pd_service.h" +DEFINE_int32(port, 8080, "Port of rpc server"); +DEFINE_int32(idle_timeout_s, 60, + "Connection will be closed if there is no " + "read/write operations during the last `idle_timeout_s`"); +DEFINE_int32(max_concurrency, 0, "Limit of request processing in parallel"); + int main(int argc, char* argv[]) { + GFLAGS_NS::ParseCommandLineFlags(&argc, &argv, true); brpc::Server server; - PlacementDriverServiceImpl service; if (server.AddService(&service, brpc::SERVER_OWNS_SERVICE) != 0) { - fprintf(stderr, "Fail to add service!\n"); + spdlog::error("Failed to add service for: {}", berror()); return -1; } + brpc::ServerOptions options; + options.idle_timeout_sec = FLAGS_idle_timeout_s; + options.max_concurrency = FLAGS_max_concurrency; + // 启动服务 - if (server.Start(8080, nullptr) != 0) { - fprintf(stderr, "Fail to start server!\n"); + if (server.Start(FLAGS_port, &options) != 0) { + spdlog::error("Failed to start server for: {}", berror()); return -1; } server.RunUntilAskedToQuit(); - return 0; } \ No newline at end of file diff --git a/ppd/pd.proto b/ppd/pd.proto index 7d3ab454e..84f4f8714 100644 --- a/ppd/pd.proto +++ b/ppd/pd.proto @@ -1,8 +1,11 @@ -syntax="proto3"; +syntax = "proto3"; package pikiwidb; option cc_generic_services = true; -import "store.proto"; +message Peer { + string group_id = 1; + int32 cluster_idx = 2; +}; message GetClusterInfoRequest { }; @@ -22,8 +25,8 @@ message Store { message Region { int64 region_id = 1; - optional string start_key = 2; - optional string end_key = 3; + string start_key = 2; + string end_key = 3; repeated RegionEpoch region_epoch = 4; repeated Peer peers = 5; }; @@ -39,6 +42,12 @@ enum StoreState { TOMBSTONE = 2; }; +message RegionOptions { + string start_key = 1; + string end_key = 2; + int32 max_data_size = 3; +}; + message CreateAllRegionsRequest { int64 regions_count = 1; int32 region_peers_count = 2; diff --git a/ppd/pd_service.cc b/ppd/pd_service.cc index dcffc0588..6463eb3f1 100644 --- a/ppd/pd_service.cc +++ b/ppd/pd_service.cc @@ -8,6 +8,7 @@ #include "pd_service.h" #include "pd_server.h" +#include "spdlog/spdlog.h" namespace pikiwidb { void PlacementDriverServiceImpl::CreateAllRegions(::google::protobuf::RpcController* controller, @@ -32,6 +33,7 @@ void PlacementDriverServiceImpl::AddStore(::google::protobuf::RpcController* con response->set_success(true); response->set_store_id(store_id); + spdlog::info("add store success: {}", store_id); } void PlacementDriverServiceImpl::RemoveStore(::google::protobuf::RpcController* controller, diff --git a/pproxy/main.cc b/pproxy/main.cc index d325e4834..ff0fa5186 100644 --- a/pproxy/main.cc +++ b/pproxy/main.cc @@ -5,23 +5,35 @@ * of patent rights can be found in the PATENTS file in the same directory. */ +#include "brpc/server.h" +#include "gflags/gflags.h" +#include "spdlog/spdlog.h" + #include "proxy_service.h" +DEFINE_int32(port, 8080, "Port of rpc server"); +DEFINE_int32(idle_timeout_s, 60, + "Connection will be closed if there is no " + "read/write operations during the last `idle_timeout_s`"); +DEFINE_int32(max_concurrency, 0, "Limit of request processing in parallel"); + int main(int argc, char* argv[]) { + GFLAGS_NS::ParseCommandLineFlags(&argc, &argv, true); brpc::Server server; - ProxyServiceImpl service; if (server.AddService(&service, brpc::SERVER_OWNS_SERVICE) != 0) { - fprintf(stderr, "Fail to add service!\n"); + spdlog::error("Failed to add service for: {}", berror()); return -1; } - // 启动服务 - if (server.Start(8080, nullptr) != 0) { - fprintf(stderr, "Fail to start server!\n"); + brpc::ServerOptions options; + options.idle_timeout_sec = FLAGS_idle_timeout_s; + options.max_concurrency = FLAGS_max_concurrency; + + if (server.Start(FLAGS_port, &options) != 0) { + spdlog::error("Failed to start server for: {}", berror()); return -1; } server.RunUntilAskedToQuit(); - return 0; } \ No newline at end of file diff --git a/pproxy/proxy.proto b/pproxy/proxy.proto index c899cfd81..2340d83d1 100644 --- a/pproxy/proxy.proto +++ b/pproxy/proxy.proto @@ -2,7 +2,6 @@ syntax = "proto3"; package pikiwidb.proxy; option cc_generic_services = true; -// 定义请求和响应 message RunCommandRequest { string command = 1; } @@ -21,7 +20,6 @@ message GetRouteInfoResponse { repeated RouteInfo infos = 1; } -// 定义服务 service ProxyService { rpc RunCommand(RunCommandRequest) returns (RunCommandResponse); rpc GetRouteInfo(GetRouteInfoRequest) returns (GetRouteInfoResponse); diff --git a/pproxy/proxy_service.cc b/pproxy/proxy_service.cc index 24be1b01e..a69176610 100644 --- a/pproxy/proxy_service.cc +++ b/pproxy/proxy_service.cc @@ -7,33 +7,56 @@ #include "proxy_service.h" +#include +#include +#include + namespace pikiwidb::proxy { void ProxyServiceImpl::RunCommand(::google::protobuf::RpcController* cntl, const pikiwidb::proxy::RunCommandRequest* request, pikiwidb::proxy::RunCommandResponse* response, ::google::protobuf::Closure* done) { - std::string command = request->command(); - std::string output = ExecuteCommand(command); + std::string command = request->command(); // 检查命令是否在白名单中 - response->set_output(output); + if (!IsCommandAllowed(command)) { + response->set_error("Command not allowed"); + done->Run(); + return; + } + std::string output = ExecuteCommand(command); + if (output.empty()) { + response->set_error("Command execution failed"); + } else { + response->set_output(output); + } done->Run(); } + void ProxyServiceImpl::GetRouteINfo(::google::protobuf::RpcController* cntl, const pikiwidb::proxy::GetRouteInfoRequest* request, pikiwidb::proxy::GetRouteInfoResponse* response, - ::google::protobuf::Closure* done) { -} + ::google::protobuf::Closure* done) {} std::string ProxyServiceImpl::ExecuteCommand(const std::string& command) { + if (!IsCommandAllowed(command)) { + return "Command not allowed"; + } + std::array buffer; std::string result; - - // 使用 popen 执行命令 std::unique_ptr pipe(popen(command.c_str(), "r"), pclose); if (!pipe) { - return "popen() failed!"; + return "Failed to execute command"; } - while (fgets(buffer.data(), buffer.size(), pipe.get()) != nullptr) { + + while (true) { + if (fgets(buffer.data(), buffer.size(), pipe.get()) == nullptr) { + if (feof(pipe.get())) { + break; + } else { + return "Error reading command output"; + } + } result += buffer.data(); } return result; diff --git a/pproxy/router.cc b/pproxy/router.cc index c8490a71d..c089d033e 100644 --- a/pproxy/router.cc +++ b/pproxy/router.cc @@ -7,6 +7,7 @@ #include "router.h" #include +#include Router::Router() { // maximum 100 parameters @@ -59,7 +60,7 @@ void Router::add(const std::vector &route, short handler) { Node *parent = tree; for (const std::string &node : route) { if (parent->children.find(node) == parent->children.end()) { - parent->children[node] = new Node({node, {}, handler}); + parent->children[node] = std::shared_ptr(new Node({node, {}, handler})); } parent = parent->children[node]; } diff --git a/pproxy/router.h b/pproxy/router.h index 644773b45..2788a93f3 100644 --- a/pproxy/router.h +++ b/pproxy/router.h @@ -9,6 +9,7 @@ #include #include +#include #include #include #include @@ -29,11 +30,11 @@ class Router { struct Node { std::string name; - std::map children; + std::map> children; short handler; }; - Node *tree = new Node({"GET", {}, -1}); + std::shared_ptr tree = std::shared_ptr(new Node({"GET", {}, -1})); std::string compiled_tree; void add(const std::vector &route, short handler); diff --git a/pproxy/task_manager.cc b/pproxy/task_manager.cc index a21053b85..35346dd54 100644 --- a/pproxy/task_manager.cc +++ b/pproxy/task_manager.cc @@ -10,6 +10,8 @@ #include #include +#include "threadpool.h" + Task::Status Task::TextToStatus(const std::string& input) { if (input == "pending") { return Task::pending; @@ -45,47 +47,47 @@ TaskManager::TaskManager(std::shared_ptr threadpool, size_t maxWorke std::future TaskManager::stop() { auto task = std::make_shared>([this] { - std::unique_lock guard(_mutex); - bool isLast = _workerCount == 1; + std::unique_lock guard(mutex_); + bool isLast = workerCount_ == 1; // Guarantee that the task finishes last. while (!isLast) { guard.unlock(); std::this_thread::sleep_for(std::chrono::milliseconds(1)); guard.lock(); - isLast = _workerCount == 1; + isLast = workerCount_ == 1; } }); auto future = task->get_future(); // Adding a new task and expecting the future guarantees that the last batch of tasks is being executed. auto functor = [task = std::move(task)]() mutable { (*task)(); }; - std::lock_guard guard(_mutex); + std::lock_guard guard(mutex_); - _stopped = true; - _tasks.emplace(std::move(functor)); + stopped_ = true; + tasks_.emplace(std::move(functor)); this->processTasks(); return future; } void TaskManager::addTask(std::function functor) { - std::lock_guard guard(_mutex); + std::lock_guard guard(mutex_); - if (_stopped) { + if (stopped_) { return; } - _tasks.emplace(std::move(functor), Clock::now()); + tasks_.emplace(std::move(functor), Clock::now()); this->processTasks(); } void TaskManager::processTasks() { - if (_tasks.empty() || _workerCount == _maxWorkers) { + if (tasks_.empty() || workerCount_ == maxWorkers_) { return; } - auto task = std::move(_tasks.front()); - _tasks.pop(); + auto task = std::move(tasks_.front()); + tasks_.pop(); - ++_workerCount; - _threadpool->execute(std::move(task)); + ++workerCount_; + threadpool_->execute(std::move(task)); } \ No newline at end of file diff --git a/pproxy/task_manager.h b/pproxy/task_manager.h index 8d70cf5e7..06208105f 100644 --- a/pproxy/task_manager.h +++ b/pproxy/task_manager.h @@ -56,9 +56,9 @@ class TaskManager { auto functor = [this, task = std::move(task)]() mutable { (*task)(); { - std::lock_guard guard(_mutex); + std::lock_guard guard(mutex_); - --_workerCount; + --workerCount_; this->processTasks(); } }; @@ -72,10 +72,10 @@ class TaskManager { void processTasks(); private: - std::shared_ptr<::Threadpool> _threadpool; - std::queue _tasks; - std::mutex _mutex; - size_t _maxWorkers; - size_t _workerCount{0}; - bool _stopped{false}; + std::shared_ptr<::Threadpool> threadpool_; + std::queue tasks_; + std::mutex mutex_; + size_t maxWorkers_; + size_t workerCount_{0}; + bool stopped_{false}; }; \ No newline at end of file diff --git a/src/base_cmd.cc b/src/base_cmd.cc index e247ad084..24613c858 100644 --- a/src/base_cmd.cc +++ b/src/base_cmd.cc @@ -49,7 +49,7 @@ void BaseCmd::Execute(PClient* client) { // 2. If PRAFT is initialized and the current node is not the leader, return a redirection message for write // commands. if (HasFlag(kCmdFlagsWrite) && !praft->IsLeader()) { - return client->SetRes(CmdRes::kErrOther, fmt::format("MOVED {}", praft->GetLeaderAddress())); + return client->SetRes(CmdRes::kMoved, fmt::format("MOVED {}", praft->GetLeaderAddress())); } } diff --git a/src/cmd_admin.cc b/src/cmd_admin.cc index a63d4280f..535903239 100644 --- a/src/cmd_admin.cc +++ b/src/cmd_admin.cc @@ -171,11 +171,6 @@ const std::string InfoCmd::kRaftSection = "raft"; InfoCmd::InfoCmd(const std::string& name, int16_t arity) : BaseCmd(name, arity, kCmdFlagsAdmin, kAclCategoryAdmin) {} -// bool InfoCmd::DoInitial(PClient* client) { -// praft_ = PSTORE.GetBackend(client->GetCurrentDB())->GetPRaft(); -// return true; -// } - bool InfoCmd::DoInitial(PClient* client) { size_t argc = client->argv_.size(); if (argc == 1) { @@ -183,11 +178,11 @@ bool InfoCmd::DoInitial(PClient* client) { return true; } - std::string argv_ = client->argv_[1]; + auto argv = client->argv_[1]; // convert section to lowercase - std::transform(argv_.begin(), argv_.end(), argv_.begin(), [](unsigned char c) { return std::tolower(c); }); + std::transform(argv.begin(), argv.end(), argv.begin(), [](unsigned char c) { return std::tolower(c); }); if (argc == 2) { - auto it = sectionMap.find(argv_); + auto it = sectionMap.find(argv); if (it != sectionMap.end()) { info_section_ = it->second; } else { diff --git a/src/cmd_raft.cc b/src/cmd_raft.cc index ca0c4a780..48bf2f204 100644 --- a/src/cmd_raft.cc +++ b/src/cmd_raft.cc @@ -67,9 +67,9 @@ void RaftNodeCmd::DoCmdAdd(PClient* client) { DEBUG("Received RAFT.NODE ADD cmd from {}", client->PeerIP()); auto db = PSTORE.GetDBByGroupID(group_id_); assert(db); - auto praft = db->GetPRaft(); + auto leader = db->GetPRaft(); // Check whether it is a leader. If it is not a leader, return the leader information - if (!praft->IsLeader()) { + if (!leader->IsLeader()) { client->SetRes(CmdRes::kWrongLeader, praft_->GetLeaderID()); return; } @@ -81,7 +81,7 @@ void RaftNodeCmd::DoCmdAdd(PClient* client) { // RedisRaft has nodeid, but in Braft, NodeId is IP:Port. // So we do not need to parse and use nodeid like redis; - auto s = praft->AddPeer(client->argv_[3]); + auto s = leader->AddPeer(client->argv_[3]); if (s.ok()) { client->SetRes(CmdRes::kOK); } else { From f9287ebf9f252591a4c121c40fea33fc5eeefa3c Mon Sep 17 00:00:00 2001 From: longfar Date: Mon, 21 Oct 2024 18:06:37 +0800 Subject: [PATCH 17/19] fix: comment --- pproxy/task_manager.h | 9 ++++++--- src/cmd_admin.cc | 18 +++++++++--------- src/cmd_admin.h | 5 +---- src/praft/praft.cc | 9 ++------- 4 files changed, 18 insertions(+), 23 deletions(-) diff --git a/pproxy/task_manager.h b/pproxy/task_manager.h index 06208105f..6055a1eca 100644 --- a/pproxy/task_manager.h +++ b/pproxy/task_manager.h @@ -12,6 +12,9 @@ #include #include #include +#include + +#include "pstd/thread_pool.h" class Task { public: @@ -46,8 +49,8 @@ class TaskManager { template auto Push(F&& function, Args&&... args) // - -> std::future::type> { - using ReturnType = typename std::result_of::type; + -> std::future> { + using ReturnType = std::invoke_result_t; auto task = std::make_shared>( // std::bind(std::forward(function), std::forward(args)...)); @@ -72,7 +75,7 @@ class TaskManager { void processTasks(); private: - std::shared_ptr<::Threadpool> threadpool_; + std::shared_ptr threadpool_; std::queue tasks_; std::mutex mutex_; size_t maxWorkers_; diff --git a/src/cmd_admin.cc b/src/cmd_admin.cc index 535903239..cc3fd8368 100644 --- a/src/cmd_admin.cc +++ b/src/cmd_admin.cc @@ -265,22 +265,22 @@ void InfoCmd::InfoRaft(PClient* client, std::string& message) { return client->SetRes(CmdRes::kWrongNum, client->CmdName()); } - praft_ = PSTORE.GetBackend(client->GetCurrentDB())->GetPRaft(); - assert(praft_); - if (!praft_->IsInitialized()) { + auto praft = PSTORE.GetBackend(client->GetCurrentDB())->GetPRaft(); + assert(praft); + if (!praft->IsInitialized()) { return client->SetRes(CmdRes::kErrOther, "Don't already cluster member"); } - auto node_status = praft_->GetNodeStatus(); + auto node_status = praft->GetNodeStatus(); if (node_status.state == braft::State::STATE_END) { message += "-ERR Node is not initialized.\r\n"; return; } std::stringstream tmp_stream; - tmp_stream << "raft_group_id:" << praft_->GetGroupID() << "\r\n"; - tmp_stream << "raft_node_id:" << praft_->GetNodeID() << "\r\n"; - tmp_stream << "raft_peer_id:" << praft_->GetPeerID() << "\r\n"; + tmp_stream << "raft_group_id:" << praft->GetGroupID() << "\r\n"; + tmp_stream << "raft_node_id:" << praft->GetNodeID() << "\r\n"; + tmp_stream << "raft_peer_id:" << praft->GetPeerID() << "\r\n"; if (braft::is_active_state(node_status.state)) { tmp_stream << "raft_state:up\r\n"; } else { @@ -290,9 +290,9 @@ void InfoCmd::InfoRaft(PClient* client, std::string& message) { tmp_stream << "raft_leader_id:" << node_status.leader_id.to_string() << "\r\n"; tmp_stream << "raft_current_term:" << std::to_string(node_status.term) << "\r\n"; - if (praft_->IsLeader()) { + if (praft->IsLeader()) { std::vector peers; - auto status = praft_->GetListPeers(&peers); + auto status = praft->GetListPeers(&peers); if (!status.ok()) { tmp_stream.str("-ERR "); tmp_stream << status.error_str() << "\r\n"; diff --git a/src/cmd_admin.h b/src/cmd_admin.h index c3d5770c8..33c533e84 100644 --- a/src/cmd_admin.h +++ b/src/cmd_admin.h @@ -160,15 +160,12 @@ class InfoCmd : public BaseCmd { void InfoServer(std::string& info); void InfoStats(std::string& info); void InfoCPU(std::string& info); - void InfoRaft(PClient* client, std::string& info); + void InfoRaft(PClient* client, std::string& message); void InfoData(std::string& info); void InfoCommandStats(PClient* client, std::string& info); std::string FormatCommandStatLine(const CommandStatistics& stats); double MethodofTotalTimeCalculation(const uint64_t time_consuming); double MethodofCommandStatistics(const uint64_t time_consuming, const uint64_t frequency); - - private: - PRaft* praft_ = nullptr; }; class CmdDebug : public BaseCmdGroup { diff --git a/src/praft/praft.cc b/src/praft/praft.cc index bf5532287..2c41b5ed9 100644 --- a/src/praft/praft.cc +++ b/src/praft/praft.cc @@ -252,8 +252,6 @@ void PRaft::SendNodeRequest(PClient* client) { auto cluster_cmd_type = cluster_cmd_ctx_.GetClusterCmdType(); switch (cluster_cmd_type) { case ClusterCmdType::kJoin: { - // SendNodeInfoRequest(client, "DATA"); - // SendNodeInfoRequest(client, "DATA"); SendNodeAddRequest(client); } break; case ClusterCmdType::kRemove: @@ -448,7 +446,6 @@ int PRaft::ProcessClusterJoinCmdResponse(PClient* client, const char* start, int ERROR("Joined Raft cluster fail, str: {}", reply); join_client->SetRes(CmdRes::kErrOther, reply); join_client->SendPacket(); - // join_client->Clear(); // If the join fails, clear clusterContext and set it again by using the join command cluster_cmd_ctx_.Clear(); } @@ -473,7 +470,6 @@ int PRaft::ProcessClusterRemoveCmdResponse(PClient* client, const char* start, i remove_client->SetRes(CmdRes::kOK); remove_client->SendPacket(); - // remove_client->Clear(); } else if (reply.find(NOT_LEADER) != std::string::npos) { auto remove_client = cluster_cmd_ctx_.GetClient(); remove_client->Clear(); @@ -482,7 +478,6 @@ int PRaft::ProcessClusterRemoveCmdResponse(PClient* client, const char* start, i ERROR("Removed Raft cluster fail, str: {}", reply); remove_client->SetRes(CmdRes::kErrOther, reply); remove_client->SendPacket(); - // remove_client->Clear(); } // If the remove fails, clear clusterContext and set it again by using the join command @@ -493,7 +488,7 @@ int PRaft::ProcessClusterRemoveCmdResponse(PClient* client, const char* start, i butil::Status PRaft::AddPeer(const std::string& peer) { if (!node_) { - ERROR_LOG_AND_STATUS("Node is not initialized"); + return ERROR_LOG_AND_STATUS("Node is not initialized"); } braft::SynchronizedClosure done; @@ -510,7 +505,7 @@ butil::Status PRaft::AddPeer(const std::string& peer) { butil::Status PRaft::AddPeer(const std::string& endpoint, int index) { if (!node_) { - ERROR_LOG_AND_STATUS("Node is not initialized"); + return ERROR_LOG_AND_STATUS("Node is not initialized"); } braft::SynchronizedClosure done; From dfee101a8e3cccf6c32898e6f99f3e7c304adfbc Mon Sep 17 00:00:00 2001 From: longfar Date: Mon, 21 Oct 2024 18:11:17 +0800 Subject: [PATCH 18/19] fix: comment --- src/cmd_raft.cc | 6 ++++-- src/praft/praft.cc | 1 - 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/cmd_raft.cc b/src/cmd_raft.cc index 48bf2f204..7e2b7237b 100644 --- a/src/cmd_raft.cc +++ b/src/cmd_raft.cc @@ -11,6 +11,7 @@ */ #include "cmd_raft.h" +#include #include #include @@ -179,9 +180,10 @@ void RaftNodeCmd::DoCmdRemove(PClient* client) { void RaftNodeCmd::DoCmdSnapshot(PClient* client) { auto s = praft_->DoSnapshot(); - if (s.ok()) { - client->SetRes(CmdRes::kOK); + if (!s.ok()) { + return client->SetRes(CmdRes::kErrOther, fmt::format("do snapshot error: {}", s.error_cstr())); } + client->SetRes(CmdRes::kOK); } RaftClusterCmd::RaftClusterCmd(const std::string& name, int16_t arity) diff --git a/src/praft/praft.cc b/src/praft/praft.cc index 2c41b5ed9..274218449 100644 --- a/src/praft/praft.cc +++ b/src/praft/praft.cc @@ -393,7 +393,6 @@ bool PRaft::InitializeNodeBeforeAdd(PClient* client, PClient* join_client, const ERROR("Joined Raft cluster fail, because of invalid raft_group_id"); join_client->SetRes(CmdRes::kErrOther, "Invalid raft_group_id"); join_client->SendPacket(); - // join_client->Clear(); // If the join fails, clear clusterContext and set it again by using the join command cluster_cmd_ctx_.Clear(); return false; From 4277387fc6269844fc90eb0ed899da36abc601fa Mon Sep 17 00:00:00 2001 From: longfar Date: Mon, 21 Oct 2024 18:13:05 +0800 Subject: [PATCH 19/19] fix: comment --- src/praft/praft.cc | 1 + 1 file changed, 1 insertion(+) diff --git a/src/praft/praft.cc b/src/praft/praft.cc index 274218449..d0dcf4d11 100644 --- a/src/praft/praft.cc +++ b/src/praft/praft.cc @@ -509,6 +509,7 @@ butil::Status PRaft::AddPeer(const std::string& endpoint, int index) { braft::SynchronizedClosure done; butil::EndPoint ep; + ep.port += g_config.raft_port_offset; butil::str2endpoint(endpoint.c_str(), &ep); braft::PeerId peer_id(ep, index); node_->add_peer(peer_id, &done);