From f8dd4d5aead8767a3f0b73b2a9ea89dd2089947c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E7=82=8E=E6=B3=BC?= Date: Thu, 2 Jan 2025 15:10:51 +0800 Subject: [PATCH] Chore: Replace `Vote` with `VoteOf` Introduce the type alias `VoteOf` to simplify future migration from the fixed type `Vote` to an associated type `C::Vote`. By defining `VoteOf = Vote` now, and later updating it to `VoteOf = C::Vote`, the migration can be handled with minimal code changes, avoiding large-scale modifications in the future. --- examples/memstore/src/log_store.rs | 14 +++++++------- openraft/src/core/notification.rs | 4 ++-- openraft/src/core/raft_msg/mod.rs | 6 +++--- openraft/src/engine/command.rs | 4 ++-- openraft/src/engine/engine_impl.rs | 9 +++++---- openraft/src/engine/handler/vote_handler/mod.rs | 6 +++--- openraft/src/error.rs | 12 ++++++------ openraft/src/metrics/metric.rs | 4 ++-- openraft/src/metrics/raft_metrics.rs | 5 +++-- openraft/src/metrics/wait.rs | 4 ++-- openraft/src/network/snapshot_transport.rs | 8 ++++---- openraft/src/network/v2/adapt_v1.rs | 4 ++-- openraft/src/network/v2/network.rs | 4 ++-- openraft/src/proposer/candidate.rs | 8 ++++---- openraft/src/raft/message/append_entries.rs | 6 +++--- openraft/src/raft/message/install_snapshot.rs | 10 +++++----- openraft/src/raft/message/transfer_leader.rs | 8 ++++---- openraft/src/raft/message/vote.rs | 12 ++++++------ openraft/src/raft/mod.rs | 4 ++-- openraft/src/raft_state/io_state.rs | 4 ++-- openraft/src/raft_state/io_state/io_id.rs | 6 +++--- openraft/src/raft_state/mod.rs | 7 ++++--- openraft/src/raft_state/vote_state_reader.rs | 4 ++-- openraft/src/replication/mod.rs | 4 ++-- openraft/src/replication/replication_session_id.rs | 4 ++-- openraft/src/storage/v2/raft_log_reader.rs | 4 ++-- openraft/src/storage/v2/raft_log_storage.rs | 4 ++-- openraft/src/storage/v2/raft_log_storage_ext.rs | 4 ++-- openraft/src/testing/log/suite.rs | 5 +++-- openraft/src/vote/committed.rs | 8 ++++---- openraft/src/vote/non_committed.rs | 8 ++++---- 31 files changed, 99 insertions(+), 95 deletions(-) diff --git a/examples/memstore/src/log_store.rs b/examples/memstore/src/log_store.rs index 4f667453f..d2a380eb1 100644 --- a/examples/memstore/src/log_store.rs +++ b/examples/memstore/src/log_store.rs @@ -6,13 +6,13 @@ use std::fmt::Debug; use std::ops::RangeBounds; use std::sync::Arc; +use openraft::alias::VoteOf; use openraft::storage::IOFlushed; use openraft::LogId; use openraft::LogState; use openraft::RaftLogId; use openraft::RaftTypeConfig; use openraft::StorageError; -use openraft::Vote; use tokio::sync::Mutex; /// RaftLogStore implementation with a in-memory storage @@ -33,7 +33,7 @@ pub struct LogStoreInner { committed: Option>, /// The current granted vote. - vote: Option>, + vote: Option>, } impl Default for LogStoreInner { @@ -84,12 +84,12 @@ impl LogStoreInner { Ok(self.committed.clone()) } - async fn save_vote(&mut self, vote: &Vote) -> Result<(), StorageError> { + async fn save_vote(&mut self, vote: &VoteOf) -> Result<(), StorageError> { self.vote = Some(vote.clone()); Ok(()) } - async fn read_vote(&mut self) -> Result>, StorageError> { + async fn read_vote(&mut self) -> Result>, StorageError> { Ok(self.vote.clone()) } @@ -135,6 +135,7 @@ mod impl_log_store { use std::fmt::Debug; use std::ops::RangeBounds; + use openraft::alias::VoteOf; use openraft::storage::IOFlushed; use openraft::storage::RaftLogStorage; use openraft::LogId; @@ -142,7 +143,6 @@ mod impl_log_store { use openraft::RaftLogReader; use openraft::RaftTypeConfig; use openraft::StorageError; - use openraft::Vote; use crate::log_store::LogStore; @@ -157,7 +157,7 @@ mod impl_log_store { inner.try_get_log_entries(range).await } - async fn read_vote(&mut self) -> Result>, StorageError> { + async fn read_vote(&mut self) -> Result>, StorageError> { let mut inner = self.inner.lock().await; inner.read_vote().await } @@ -183,7 +183,7 @@ mod impl_log_store { inner.read_committed().await } - async fn save_vote(&mut self, vote: &Vote) -> Result<(), StorageError> { + async fn save_vote(&mut self, vote: &VoteOf) -> Result<(), StorageError> { let mut inner = self.inner.lock().await; inner.save_vote(vote).await } diff --git a/openraft/src/core/notification.rs b/openraft/src/core/notification.rs index f45342424..8b182f5c0 100644 --- a/openraft/src/core/notification.rs +++ b/openraft/src/core/notification.rs @@ -7,11 +7,11 @@ use crate::raft_state::IOId; use crate::replication; use crate::replication::ReplicationSessionId; use crate::type_config::alias::InstantOf; +use crate::type_config::alias::VoteOf; use crate::vote::committed::CommittedVote; use crate::vote::non_committed::NonCommittedVote; use crate::RaftTypeConfig; use crate::StorageError; -use crate::Vote; /// A message coming from the internal components. pub(crate) enum Notification @@ -33,7 +33,7 @@ where C: RaftTypeConfig target: C::NodeId, /// The higher vote observed. - higher: Vote, + higher: VoteOf, /// The Leader that sent replication request. leader_vote: CommittedVote, diff --git a/openraft/src/core/raft_msg/mod.rs b/openraft/src/core/raft_msg/mod.rs index e2317bf67..70df7399d 100644 --- a/openraft/src/core/raft_msg/mod.rs +++ b/openraft/src/core/raft_msg/mod.rs @@ -16,10 +16,10 @@ use crate::type_config::alias::LogIdOf; use crate::type_config::alias::OneshotSenderOf; use crate::type_config::alias::ResponderOf; use crate::type_config::alias::SnapshotDataOf; +use crate::type_config::alias::VoteOf; use crate::ChangeMembers; use crate::RaftState; use crate::RaftTypeConfig; -use crate::Vote; pub(crate) mod external_command; @@ -52,7 +52,7 @@ where C: RaftTypeConfig }, InstallFullSnapshot { - vote: Vote, + vote: VoteOf, snapshot: Snapshot, tx: ResultSender>, }, @@ -101,7 +101,7 @@ where C: RaftTypeConfig /// Otherwise, just reset Leader lease so that the node `to` can become Leader. HandleTransferLeader { /// The vote of the Leader that is transferring the leadership. - from: Vote, + from: VoteOf, /// The assigned node to be the next Leader. to: C::NodeId, }, diff --git a/openraft/src/engine/command.rs b/openraft/src/engine/command.rs index f95107ccb..8afe8a894 100644 --- a/openraft/src/engine/command.rs +++ b/openraft/src/engine/command.rs @@ -21,11 +21,11 @@ use crate::raft_state::IOId; use crate::replication::request::Replicate; use crate::replication::ReplicationSessionId; use crate::type_config::alias::OneshotSenderOf; +use crate::type_config::alias::VoteOf; use crate::vote::committed::CommittedVote; use crate::LogId; use crate::OptionalSend; use crate::RaftTypeConfig; -use crate::Vote; /// Commands to send to `RaftRuntime` to execute, to update the application state. #[derive(Debug)] @@ -112,7 +112,7 @@ where C: RaftTypeConfig }, /// Save vote to storage - SaveVote { vote: Vote }, + SaveVote { vote: VoteOf }, /// Send vote to all other members SendVote { vote_req: VoteRequest }, diff --git a/openraft/src/engine/engine_impl.rs b/openraft/src/engine/engine_impl.rs index 38587d2a2..071e5de5a 100644 --- a/openraft/src/engine/engine_impl.rs +++ b/openraft/src/engine/engine_impl.rs @@ -46,6 +46,7 @@ use crate::storage::Snapshot; use crate::storage::SnapshotMeta; use crate::type_config::alias::ResponderOf; use crate::type_config::alias::SnapshotDataOf; +use crate::type_config::alias::VoteOf; use crate::type_config::TypeConfigExt; use crate::vote::RaftLeaderId; use crate::vote::RaftTerm; @@ -113,7 +114,7 @@ where C: RaftTypeConfig /// /// The candidate `last_log_id` is initialized with the attributes of Acceptor part: /// [`RaftState`] - pub(crate) fn new_candidate(&mut self, vote: Vote) -> &mut Candidate> { + pub(crate) fn new_candidate(&mut self, vote: VoteOf) -> &mut Candidate> { let now = C::now(); let last_log_id = self.state.last_log_id().cloned(); @@ -380,7 +381,7 @@ where C: RaftTypeConfig #[tracing::instrument(level = "debug", skip_all)] pub(crate) fn handle_append_entries( &mut self, - vote: &Vote, + vote: &VoteOf, prev_log_id: Option>, entries: Vec, tx: Option>, @@ -419,7 +420,7 @@ where C: RaftTypeConfig pub(crate) fn append_entries( &mut self, - vote: &Vote, + vote: &VoteOf, prev_log_id: Option>, entries: Vec, ) -> Result<(), RejectAppendEntries> { @@ -453,7 +454,7 @@ where C: RaftTypeConfig #[tracing::instrument(level = "debug", skip_all)] pub(crate) fn handle_install_full_snapshot( &mut self, - vote: Vote, + vote: VoteOf, snapshot: Snapshot, tx: ResultSender>, ) { diff --git a/openraft/src/engine/handler/vote_handler/mod.rs b/openraft/src/engine/handler/vote_handler/mod.rs index f20da67c8..461a06e6a 100644 --- a/openraft/src/engine/handler/vote_handler/mod.rs +++ b/openraft/src/engine/handler/vote_handler/mod.rs @@ -17,13 +17,13 @@ use crate::proposer::CandidateState; use crate::proposer::LeaderState; use crate::raft_state::IOId; use crate::raft_state::LogStateReader; +use crate::type_config::alias::VoteOf; use crate::type_config::TypeConfigExt; use crate::vote::RaftLeaderId; use crate::LogId; use crate::OptionalSend; use crate::RaftState; use crate::RaftTypeConfig; -use crate::Vote; #[cfg(test)] mod accept_vote_test; @@ -60,7 +60,7 @@ where C: RaftTypeConfig #[tracing::instrument(level = "debug", skip_all)] pub(crate) fn accept_vote( &mut self, - vote: &Vote, + vote: &VoteOf, tx: ResultSender, f: F, ) -> Option> @@ -99,7 +99,7 @@ where C: RaftTypeConfig /// Note: This method does not check last-log-id. handle-vote-request has to deal with /// last-log-id itself. #[tracing::instrument(level = "debug", skip_all)] - pub(crate) fn update_vote(&mut self, vote: &Vote) -> Result<(), RejectVoteRequest> { + pub(crate) fn update_vote(&mut self, vote: &VoteOf) -> Result<(), RejectVoteRequest> { // Partial ord compare: // Vote does not have to be total ord. // `!(a >= b)` does not imply `a < b`. diff --git a/openraft/src/error.rs b/openraft/src/error.rs index 18265319a..5368cbf21 100644 --- a/openraft/src/error.rs +++ b/openraft/src/error.rs @@ -29,11 +29,11 @@ use crate::network::RPCTypes; use crate::raft::AppendEntriesResponse; use crate::raft_types::SnapshotSegmentId; use crate::try_as_ref::TryAsRef; +use crate::type_config::alias::VoteOf; use crate::LogId; use crate::Membership; use crate::RaftTypeConfig; use crate::StorageError; -use crate::Vote; /// RaftError is returned by API methods of `Raft`. /// @@ -346,8 +346,8 @@ where #[cfg_attr(feature = "serde", derive(serde::Deserialize, serde::Serialize), serde(bound = ""))] #[error("seen a higher vote: {higher} GT mine: {sender_vote}")] pub(crate) struct HigherVote { - pub(crate) higher: Vote, - pub(crate) sender_vote: Vote, + pub(crate) higher: VoteOf, + pub(crate) sender_vote: VoteOf, } /// Error that indicates a **temporary** network error and when it is returned, Openraft will retry @@ -603,7 +603,7 @@ pub struct LearnerNotFound { #[error("not allowed to initialize due to current raft state: last_log_id: {last_log_id:?} vote: {vote}")] pub struct NotAllowed { pub last_log_id: Option>, - pub vote: Vote, + pub vote: VoteOf, } #[derive(Debug, Clone, PartialEq, Eq, thiserror::Error)] @@ -636,7 +636,7 @@ pub enum NoForward {} #[cfg_attr(feature = "serde", derive(serde::Deserialize, serde::Serialize), serde(bound = ""))] pub(crate) enum RejectVoteRequest { #[error("reject vote request by a greater vote: {0}")] - ByVote(Vote), + ByVote(VoteOf), #[allow(dead_code)] #[error("reject vote request by a greater last-log-id: {0:?}")] @@ -659,7 +659,7 @@ where C: RaftTypeConfig #[derive(Debug, Clone, PartialEq, Eq, thiserror::Error)] pub(crate) enum RejectAppendEntries { #[error("reject AppendEntries by a greater vote: {0}")] - ByVote(Vote), + ByVote(VoteOf), #[error("reject AppendEntries because of conflicting log-id: {local:?}; expect to be: {expect:?}")] ByConflictingLogId { expect: LogId, local: Option> }, diff --git a/openraft/src/metrics/metric.rs b/openraft/src/metrics/metric.rs index 7959b7646..d1cb79fd8 100644 --- a/openraft/src/metrics/metric.rs +++ b/openraft/src/metrics/metric.rs @@ -1,11 +1,11 @@ use std::cmp::Ordering; use crate::metrics::metric_display::MetricDisplay; +use crate::type_config::alias::VoteOf; use crate::LogId; use crate::LogIdOptionExt; use crate::RaftMetrics; use crate::RaftTypeConfig; -use crate::Vote; /// A metric entry of a Raft node. /// @@ -15,7 +15,7 @@ pub enum Metric where C: RaftTypeConfig { Term(C::Term), - Vote(Vote), + Vote(VoteOf), LastLogIndex(Option), Applied(Option>), AppliedIndex(Option), diff --git a/openraft/src/metrics/raft_metrics.rs b/openraft/src/metrics/raft_metrics.rs index 8a4735ab7..929bb545c 100644 --- a/openraft/src/metrics/raft_metrics.rs +++ b/openraft/src/metrics/raft_metrics.rs @@ -10,6 +10,7 @@ use crate::metrics::ReplicationMetrics; use crate::metrics::SerdeInstant; use crate::type_config::alias::InstantOf; use crate::type_config::alias::SerdeInstantOf; +use crate::type_config::alias::VoteOf; use crate::Instant; use crate::LogId; use crate::RaftTypeConfig; @@ -32,7 +33,7 @@ pub struct RaftMetrics { pub current_term: C::Term, /// The last flushed vote. - pub vote: Vote, + pub vote: VoteOf, /// The last log index has been appended to this Raft node's log. pub last_log_index: Option, @@ -280,7 +281,7 @@ where C: RaftTypeConfig #[cfg_attr(feature = "serde", derive(serde::Deserialize, serde::Serialize), serde(bound = ""))] pub struct RaftServerMetrics { pub id: C::NodeId, - pub vote: Vote, + pub vote: VoteOf, pub state: ServerState, pub current_leader: Option, diff --git a/openraft/src/metrics/wait.rs b/openraft/src/metrics/wait.rs index 2428411a4..22a7163a3 100644 --- a/openraft/src/metrics/wait.rs +++ b/openraft/src/metrics/wait.rs @@ -8,12 +8,12 @@ use crate::core::ServerState; use crate::metrics::Condition; use crate::metrics::Metric; use crate::metrics::RaftMetrics; +use crate::type_config::alias::VoteOf; use crate::type_config::alias::WatchReceiverOf; use crate::type_config::TypeConfigExt; use crate::LogId; use crate::OptionalSend; use crate::RaftTypeConfig; -use crate::Vote; // Error variants related to metrics. #[derive(Debug, thiserror::Error)] @@ -93,7 +93,7 @@ where C: RaftTypeConfig /// Wait for `vote` to become `want` or timeout. #[tracing::instrument(level = "trace", skip(self), fields(msg=msg.to_string().as_str()))] - pub async fn vote(&self, want: Vote, msg: impl ToString) -> Result, WaitError> { + pub async fn vote(&self, want: VoteOf, msg: impl ToString) -> Result, WaitError> { self.eq(Metric::Vote(want), msg).await } diff --git a/openraft/src/network/snapshot_transport.rs b/openraft/src/network/snapshot_transport.rs index 89638fefe..720eb8385 100644 --- a/openraft/src/network/snapshot_transport.rs +++ b/openraft/src/network/snapshot_transport.rs @@ -28,6 +28,7 @@ mod tokio_rt { use crate::raft::InstallSnapshotRequest; use crate::raft::SnapshotResponse; use crate::storage::Snapshot; + use crate::type_config::alias::VoteOf; use crate::type_config::TypeConfigExt; use crate::ErrorSubject; use crate::ErrorVerb; @@ -37,7 +38,6 @@ mod tokio_rt { use crate::RaftTypeConfig; use crate::StorageError; use crate::ToStorageResult; - use crate::Vote; /// This chunk based implementation requires `SnapshotData` to be `AsyncRead + AsyncSeek`. impl SnapshotTransport for Chunked @@ -45,7 +45,7 @@ mod tokio_rt { { async fn send_snapshot( net: &mut Net, - vote: Vote, + vote: VoteOf, mut snapshot: Snapshot, mut cancel: impl Future + OptionalSend + 'static, option: RPCOption, @@ -272,12 +272,12 @@ use crate::network::RPCOption; use crate::raft::InstallSnapshotRequest; use crate::raft::SnapshotResponse; use crate::storage::Snapshot; +use crate::type_config::alias::VoteOf; use crate::OptionalSend; use crate::Raft; use crate::RaftNetwork; use crate::RaftTypeConfig; use crate::SnapshotId; -use crate::Vote; /// Send and Receive snapshot by chunks. pub struct Chunked {} @@ -299,7 +299,7 @@ pub trait SnapshotTransport { // TODO: consider removing dependency on RaftNetwork async fn send_snapshot( net: &mut Net, - vote: Vote, + vote: VoteOf, snapshot: Snapshot, cancel: impl Future + OptionalSend + 'static, option: RPCOption, diff --git a/openraft/src/network/v2/adapt_v1.rs b/openraft/src/network/v2/adapt_v1.rs index 7865ba7a5..3eb6eb127 100644 --- a/openraft/src/network/v2/adapt_v1.rs +++ b/openraft/src/network/v2/adapt_v1.rs @@ -13,10 +13,10 @@ use crate::raft::SnapshotResponse; use crate::raft::VoteRequest; use crate::raft::VoteResponse; use crate::storage::Snapshot; +use crate::type_config::alias::VoteOf; use crate::OptionalSend; use crate::RaftNetwork; use crate::RaftTypeConfig; -use crate::Vote; impl RaftNetworkV2 for V1 where @@ -38,7 +38,7 @@ where async fn full_snapshot( &mut self, - vote: Vote, + vote: VoteOf, snapshot: Snapshot, cancel: impl Future + OptionalSend + 'static, option: RPCOption, diff --git a/openraft/src/network/v2/network.rs b/openraft/src/network/v2/network.rs index 031858031..d8267c18f 100644 --- a/openraft/src/network/v2/network.rs +++ b/openraft/src/network/v2/network.rs @@ -18,10 +18,10 @@ use crate::raft::SnapshotResponse; use crate::raft::VoteRequest; use crate::raft::VoteResponse; use crate::storage::Snapshot; +use crate::type_config::alias::VoteOf; use crate::OptionalSend; use crate::OptionalSync; use crate::RaftTypeConfig; -use crate::Vote; /// A trait defining the interface for a Raft network between cluster members. /// @@ -75,7 +75,7 @@ where C: RaftTypeConfig /// [`Raft::install_full_snapshot()`]: crate::raft::Raft::install_full_snapshot async fn full_snapshot( &mut self, - vote: Vote, + vote: VoteOf, snapshot: Snapshot, cancel: impl Future + OptionalSend + 'static, option: RPCOption, diff --git a/openraft/src/proposer/candidate.rs b/openraft/src/proposer/candidate.rs index 21f802a63..4b1582276 100644 --- a/openraft/src/proposer/candidate.rs +++ b/openraft/src/proposer/candidate.rs @@ -9,9 +9,9 @@ use crate::proposer::Leader; use crate::quorum::QuorumSet; use crate::type_config::alias::InstantOf; use crate::type_config::alias::LogIdOf; +use crate::type_config::alias::VoteOf; use crate::LogId; use crate::RaftTypeConfig; -use crate::Vote; /// Candidate: voting state. #[derive(Clone, Debug)] @@ -25,7 +25,7 @@ where starting_time: InstantOf, /// The vote. - vote: Vote, + vote: VoteOf, last_log_id: Option>, @@ -61,7 +61,7 @@ where { pub(crate) fn new( starting_time: InstantOf, - vote: Vote, + vote: VoteOf, last_log_id: Option>, quorum_set: QS, learner_ids: impl IntoIterator, @@ -76,7 +76,7 @@ where } } - pub(crate) fn vote_ref(&self) -> &Vote { + pub(crate) fn vote_ref(&self) -> &VoteOf { &self.vote } diff --git a/openraft/src/raft/message/append_entries.rs b/openraft/src/raft/message/append_entries.rs index 484d56559..a8c582b76 100644 --- a/openraft/src/raft/message/append_entries.rs +++ b/openraft/src/raft/message/append_entries.rs @@ -2,9 +2,9 @@ use std::fmt; use crate::display_ext::DisplayOptionExt; use crate::display_ext::DisplaySlice; +use crate::type_config::alias::VoteOf; use crate::LogId; use crate::RaftTypeConfig; -use crate::Vote; /// An RPC sent by a cluster leader to replicate log entries (§5.3), and as a heartbeat (§5.2). /// @@ -16,7 +16,7 @@ use crate::Vote; #[derive(Clone)] #[cfg_attr(feature = "serde", derive(serde::Deserialize, serde::Serialize), serde(bound = ""))] pub struct AppendEntriesRequest { - pub vote: Vote, + pub vote: VoteOf, pub prev_log_id: Option>, @@ -95,7 +95,7 @@ pub enum AppendEntriesResponse { /// Seen a vote `v` that does not hold `mine_vote >= v`. /// And a leader's vote(committed vote) must be total order with other vote. /// Therefore it has to be a higher vote: `mine_vote < v` - HigherVote(Vote), + HigherVote(VoteOf), } impl AppendEntriesResponse diff --git a/openraft/src/raft/message/install_snapshot.rs b/openraft/src/raft/message/install_snapshot.rs index bb1bc7bac..532596a76 100644 --- a/openraft/src/raft/message/install_snapshot.rs +++ b/openraft/src/raft/message/install_snapshot.rs @@ -1,15 +1,15 @@ use std::fmt; use crate::storage::SnapshotMeta; +use crate::type_config::alias::VoteOf; use crate::RaftTypeConfig; -use crate::Vote; /// An RPC sent by the Raft leader to send chunks of a snapshot to a follower (§7). #[derive(Clone, Debug)] #[derive(PartialEq, Eq)] #[cfg_attr(feature = "serde", derive(serde::Deserialize, serde::Serialize), serde(bound = ""))] pub struct InstallSnapshotRequest { - pub vote: Vote, + pub vote: VoteOf, /// Metadata of a snapshot: snapshot_id, last_log_ed membership etc. pub meta: SnapshotMeta, @@ -44,7 +44,7 @@ impl fmt::Display for InstallSnapshotRequest { #[display("{{vote:{}}}", vote)] #[cfg_attr(feature = "serde", derive(serde::Deserialize, serde::Serialize), serde(bound = ""))] pub struct InstallSnapshotResponse { - pub vote: Vote, + pub vote: VoteOf, } /// The response to `Raft::install_full_snapshot` API. @@ -54,11 +54,11 @@ pub struct InstallSnapshotResponse { #[display("SnapshotResponse{{vote:{}}}", vote)] #[cfg_attr(feature = "serde", derive(serde::Deserialize, serde::Serialize), serde(bound = ""))] pub struct SnapshotResponse { - pub vote: Vote, + pub vote: VoteOf, } impl SnapshotResponse { - pub fn new(vote: Vote) -> Self { + pub fn new(vote: VoteOf) -> Self { Self { vote } } } diff --git a/openraft/src/raft/message/transfer_leader.rs b/openraft/src/raft/message/transfer_leader.rs index f5f7ed474..9cc135df1 100644 --- a/openraft/src/raft/message/transfer_leader.rs +++ b/openraft/src/raft/message/transfer_leader.rs @@ -1,9 +1,9 @@ use std::fmt; use crate::display_ext::DisplayOptionExt; +use crate::type_config::alias::VoteOf; use crate::LogId; use crate::RaftTypeConfig; -use crate::Vote; #[derive(Clone, Debug)] #[derive(PartialEq, Eq)] @@ -12,7 +12,7 @@ pub struct TransferLeaderRequest where C: RaftTypeConfig { /// The vote of the Leader that is transferring the leadership. - pub(crate) from_leader: Vote, + pub(crate) from_leader: VoteOf, /// The assigned node to be the next Leader. pub(crate) to_node_id: C::NodeId, @@ -24,7 +24,7 @@ where C: RaftTypeConfig impl TransferLeaderRequest where C: RaftTypeConfig { - pub fn new(from: Vote, to: C::NodeId, last_log_id: Option>) -> Self { + pub fn new(from: VoteOf, to: C::NodeId, last_log_id: Option>) -> Self { Self { from_leader: from, to_node_id: to, @@ -33,7 +33,7 @@ where C: RaftTypeConfig } /// From which Leader the leadership is transferred. - pub fn from_leader(&self) -> &Vote { + pub fn from_leader(&self) -> &VoteOf { &self.from_leader } diff --git a/openraft/src/raft/message/vote.rs b/openraft/src/raft/message/vote.rs index 7737ebf0c..e83f6b90e 100644 --- a/openraft/src/raft/message/vote.rs +++ b/openraft/src/raft/message/vote.rs @@ -2,15 +2,15 @@ use std::borrow::Borrow; use std::fmt; use crate::display_ext::DisplayOptionExt; +use crate::type_config::alias::VoteOf; use crate::LogId; use crate::RaftTypeConfig; -use crate::Vote; /// An RPC sent by candidates to gather votes (§5.2). #[derive(Debug, Clone, PartialEq, Eq)] #[cfg_attr(feature = "serde", derive(serde::Deserialize, serde::Serialize), serde(bound = ""))] pub struct VoteRequest { - pub vote: Vote, + pub vote: VoteOf, pub last_log_id: Option>, } @@ -25,7 +25,7 @@ where C: RaftTypeConfig impl VoteRequest where C: RaftTypeConfig { - pub fn new(vote: Vote, last_log_id: Option>) -> Self { + pub fn new(vote: VoteOf, last_log_id: Option>) -> Self { Self { vote, last_log_id } } } @@ -39,7 +39,7 @@ pub struct VoteResponse { /// /// `vote` that equals the candidate.vote does not mean the vote is granted. /// The `vote` may be updated when a previous Leader sees a higher vote. - pub vote: Vote, + pub vote: VoteOf, /// It is true if a node accepted and saved the VoteRequest. pub vote_granted: bool, @@ -51,7 +51,7 @@ pub struct VoteResponse { impl VoteResponse where C: RaftTypeConfig { - pub fn new(vote: impl Borrow>, last_log_id: Option>, granted: bool) -> Self { + pub fn new(vote: impl Borrow>, last_log_id: Option>, granted: bool) -> Self { Self { vote: vote.borrow().clone(), vote_granted: granted, @@ -61,7 +61,7 @@ where C: RaftTypeConfig /// Returns `true` if the response indicates that the target node has granted a vote to the /// candidate. - pub fn is_granted_to(&self, candidate_vote: &Vote) -> bool { + pub fn is_granted_to(&self, candidate_vote: &VoteOf) -> bool { &self.vote == candidate_vote } } diff --git a/openraft/src/raft/mod.rs b/openraft/src/raft/mod.rs index 061d3f79a..77824c59f 100644 --- a/openraft/src/raft/mod.rs +++ b/openraft/src/raft/mod.rs @@ -86,6 +86,7 @@ use crate::type_config::alias::JoinErrorOf; use crate::type_config::alias::ResponderOf; use crate::type_config::alias::ResponderReceiverOf; use crate::type_config::alias::SnapshotDataOf; +use crate::type_config::alias::VoteOf; use crate::type_config::alias::WatchReceiverOf; use crate::type_config::TypeConfigExt; use crate::LogId; @@ -96,7 +97,6 @@ use crate::RaftNetworkFactory; use crate::RaftState; pub use crate::RaftTypeConfig; use crate::StorageHelper; -use crate::Vote; /// Define types for a Raft type configuration. /// @@ -424,7 +424,7 @@ where C: RaftTypeConfig #[tracing::instrument(level = "debug", skip_all)] pub async fn install_full_snapshot( &self, - vote: Vote, + vote: VoteOf, snapshot: Snapshot, ) -> Result, Fatal> { tracing::info!("Raft::install_full_snapshot()"); diff --git a/openraft/src/raft_state/io_state.rs b/openraft/src/raft_state/io_state.rs index 9ad261663..d08623a9d 100644 --- a/openraft/src/raft_state/io_state.rs +++ b/openraft/src/raft_state/io_state.rs @@ -7,9 +7,9 @@ use validit::Validate; use crate::display_ext::DisplayOption; use crate::raft_state::io_state::io_progress::IOProgress; use crate::raft_state::IOId; +use crate::type_config::alias::VoteOf; use crate::LogId; use crate::RaftTypeConfig; -use crate::Vote; pub(crate) mod io_id; pub(crate) mod io_progress; @@ -103,7 +103,7 @@ impl IOState where C: RaftTypeConfig { pub(crate) fn new( - vote: &Vote, + vote: &VoteOf, applied: Option>, snapshot: Option>, purged: Option>, diff --git a/openraft/src/raft_state/io_state/io_id.rs b/openraft/src/raft_state/io_state/io_id.rs index cfaf840b9..93d4e9bd0 100644 --- a/openraft/src/raft_state/io_state/io_id.rs +++ b/openraft/src/raft_state/io_state/io_id.rs @@ -2,6 +2,7 @@ use std::cmp::Ordering; use std::fmt; use crate::raft_state::io_state::log_io_id::LogIOId; +use crate::type_config::alias::VoteOf; use crate::vote::committed::CommittedVote; use crate::vote::non_committed::NonCommittedVote; use crate::vote::ref_vote::RefVote; @@ -9,7 +10,6 @@ use crate::ErrorSubject; use crate::ErrorVerb; use crate::LogId; use crate::RaftTypeConfig; -use crate::Vote; /// An ID to uniquely identify a monotonic increasing io operation to [`RaftLogStorage`]. /// @@ -68,7 +68,7 @@ where C: RaftTypeConfig impl IOId where C: RaftTypeConfig { - pub(crate) fn new(vote: &Vote) -> Self { + pub(crate) fn new(vote: &VoteOf) -> Self { if vote.is_committed() { Self::new_log_io(vote.clone().into_committed(), None) } else { @@ -87,7 +87,7 @@ where C: RaftTypeConfig /// Returns the vote the io operation is submitted by. #[allow(clippy::wrong_self_convention)] // The above lint is disabled because in future Vote may not be `Copy` - pub(crate) fn to_vote(&self) -> Vote { + pub(crate) fn to_vote(&self) -> VoteOf { match self { Self::Vote(non_committed_vote) => non_committed_vote.clone().into_vote(), Self::Log(log_io_id) => log_io_id.committed_vote.clone().into_vote(), diff --git a/openraft/src/raft_state/mod.rs b/openraft/src/raft_state/mod.rs index 0b9fee114..8cf870907 100644 --- a/openraft/src/raft_state/mod.rs +++ b/openraft/src/raft_state/mod.rs @@ -41,6 +41,7 @@ use crate::proposer::Leader; use crate::proposer::LeaderQuorumSet; use crate::type_config::alias::InstantOf; use crate::type_config::alias::LogIdOf; +use crate::type_config::alias::VoteOf; use crate::vote::RaftLeaderId; /// A struct used to represent the raft state which a Raft node needs. @@ -50,7 +51,7 @@ pub struct RaftState where C: RaftTypeConfig { /// The vote state of this node. - pub(crate) vote: Leased, InstantOf>, + pub(crate) vote: Leased, InstantOf>, /// The LogId of the last log committed(AKA applied) to the state machine. /// @@ -150,7 +151,7 @@ where C: RaftTypeConfig impl VoteStateReader for RaftState where C: RaftTypeConfig { - fn vote_ref(&self) -> &Vote { + fn vote_ref(&self) -> &VoteOf { self.vote.deref() } } @@ -188,7 +189,7 @@ impl RaftState where C: RaftTypeConfig { /// Get a reference to the current vote. - pub fn vote_ref(&self) -> &Vote { + pub fn vote_ref(&self) -> &VoteOf { self.vote.deref() } diff --git a/openraft/src/raft_state/vote_state_reader.rs b/openraft/src/raft_state/vote_state_reader.rs index 765b76a8a..4162c9595 100644 --- a/openraft/src/raft_state/vote_state_reader.rs +++ b/openraft/src/raft_state/vote_state_reader.rs @@ -1,5 +1,5 @@ +use crate::type_config::alias::VoteOf; use crate::RaftTypeConfig; -use crate::Vote; // TODO: remove it? /// APIs to get vote. @@ -8,5 +8,5 @@ pub(crate) trait VoteStateReader where C: RaftTypeConfig { /// Get a reference to the current vote. - fn vote_ref(&self) -> &Vote; + fn vote_ref(&self) -> &VoteOf; } diff --git a/openraft/src/replication/mod.rs b/openraft/src/replication/mod.rs index 2a58b50ef..81c9d17d5 100644 --- a/openraft/src/replication/mod.rs +++ b/openraft/src/replication/mod.rs @@ -54,6 +54,7 @@ use crate::type_config::alias::MpscUnboundedWeakSenderOf; use crate::type_config::alias::MutexOf; use crate::type_config::alias::OneshotReceiverOf; use crate::type_config::alias::OneshotSenderOf; +use crate::type_config::alias::VoteOf; use crate::type_config::async_runtime::mutex::Mutex; use crate::type_config::TypeConfigExt; use crate::vote::RaftLeaderId; @@ -62,7 +63,6 @@ use crate::RaftLogId; use crate::RaftNetworkFactory; use crate::RaftTypeConfig; use crate::StorageError; -use crate::Vote; /// The handle to a spawned replication stream. pub(crate) struct ReplicationHandle @@ -748,7 +748,7 @@ where async fn send_snapshot( network: Arc>, - vote: Vote, + vote: VoteOf, snapshot: Snapshot, option: RPCOption, cancel: OneshotReceiverOf, diff --git a/openraft/src/replication/replication_session_id.rs b/openraft/src/replication/replication_session_id.rs index 21b3edf8f..439d6b0f0 100644 --- a/openraft/src/replication/replication_session_id.rs +++ b/openraft/src/replication/replication_session_id.rs @@ -2,10 +2,10 @@ use std::fmt::Display; use std::fmt::Formatter; use crate::display_ext::DisplayOptionExt; +use crate::type_config::alias::VoteOf; use crate::vote::committed::CommittedVote; use crate::LogId; use crate::RaftTypeConfig; -use crate::Vote; /// Uniquely identifies a replication session. /// @@ -68,7 +68,7 @@ where C: RaftTypeConfig self.leader_vote.clone() } - pub(crate) fn vote(&self) -> Vote { + pub(crate) fn vote(&self) -> VoteOf { self.leader_vote.clone().into_vote() } } diff --git a/openraft/src/storage/v2/raft_log_reader.rs b/openraft/src/storage/v2/raft_log_reader.rs index 59a60318c..20b49705b 100644 --- a/openraft/src/storage/v2/raft_log_reader.rs +++ b/openraft/src/storage/v2/raft_log_reader.rs @@ -6,12 +6,12 @@ use openraft_macros::add_async_trait; use openraft_macros::since; use crate::engine::LogIdList; +use crate::type_config::alias::VoteOf; use crate::LogId; use crate::OptionalSend; use crate::OptionalSync; use crate::RaftTypeConfig; use crate::StorageError; -use crate::Vote; /// A trait defining the interface for a Raft log subsystem. /// /// This interface is accessed read-only by replication sub task: `ReplicationCore`. @@ -50,7 +50,7 @@ where C: RaftTypeConfig /// See: [log-stream](`crate::docs::protocol::replication::log_stream`) /// /// [`RaftLogStorage::save_vote`]: crate::storage::RaftLogStorage::save_vote - async fn read_vote(&mut self) -> Result>, StorageError>; + async fn read_vote(&mut self) -> Result>, StorageError>; /// Returns log entries within range `[start, end)`, `end` is exclusive, /// potentially limited by implementation-defined constraints. diff --git a/openraft/src/storage/v2/raft_log_storage.rs b/openraft/src/storage/v2/raft_log_storage.rs index 8cc77bbe8..0495c5618 100644 --- a/openraft/src/storage/v2/raft_log_storage.rs +++ b/openraft/src/storage/v2/raft_log_storage.rs @@ -2,13 +2,13 @@ use openraft_macros::add_async_trait; use crate::storage::IOFlushed; use crate::storage::LogState; +use crate::type_config::alias::VoteOf; use crate::LogId; use crate::OptionalSend; use crate::OptionalSync; use crate::RaftLogReader; use crate::RaftTypeConfig; use crate::StorageError; -use crate::Vote; /// API for log store. /// @@ -52,7 +52,7 @@ where C: RaftTypeConfig /// ### To ensure correctness: /// /// The vote must be persisted on disk before returning. - async fn save_vote(&mut self, vote: &Vote) -> Result<(), StorageError>; + async fn save_vote(&mut self, vote: &VoteOf) -> Result<(), StorageError>; /// Saves the last committed log id to storage. /// diff --git a/openraft/src/storage/v2/raft_log_storage_ext.rs b/openraft/src/storage/v2/raft_log_storage_ext.rs index b2b795213..5ce52bdd0 100644 --- a/openraft/src/storage/v2/raft_log_storage_ext.rs +++ b/openraft/src/storage/v2/raft_log_storage_ext.rs @@ -7,11 +7,11 @@ use crate::log_id::RaftLogId; use crate::raft_state::io_state::io_id::IOId; use crate::storage::IOFlushed; use crate::storage::RaftLogStorage; +use crate::type_config::alias::VoteOf; use crate::type_config::TypeConfigExt; use crate::OptionalSend; use crate::RaftTypeConfig; use crate::StorageError; -use crate::Vote; /// Extension trait for RaftLogStorage to provide utility methods. /// @@ -34,7 +34,7 @@ where C: RaftTypeConfig let (tx, mut rx) = C::mpsc_unbounded(); - let io_id = IOId::::new_log_io(Vote::::default().into_committed(), Some(last_log_id)); + let io_id = IOId::::new_log_io(VoteOf::::default().into_committed(), Some(last_log_id)); let notify = Notification::LocalIO { io_id }; let callback = IOFlushed::::new(notify, tx.downgrade()); diff --git a/openraft/src/testing/log/suite.rs b/openraft/src/testing/log/suite.rs index cc067a4fb..21fd00f2f 100644 --- a/openraft/src/testing/log/suite.rs +++ b/openraft/src/testing/log/suite.rs @@ -23,6 +23,7 @@ use crate::storage::RaftLogStorage; use crate::storage::RaftStateMachine; use crate::storage::StorageHelper; use crate::testing::log::StoreBuilder; +use crate::type_config::alias::VoteOf; use crate::type_config::TypeConfigExt; use crate::vote::RaftLeaderIdExt; use crate::LogId; @@ -70,7 +71,7 @@ where C: RaftTypeConfig } /// Proxy method to invoke [`RaftLogReader::read_vote`]. - async fn read_vote(&mut self) -> Result>, StorageError> { + async fn read_vote(&mut self) -> Result>, StorageError> { self.get_log_reader().await.read_vote().await } @@ -1444,7 +1445,7 @@ where let (tx, mut rx) = C::mpsc_unbounded(); // Dummy log io id for blocking append - let io_id = IOId::::new_log_io(Vote::::default().into_committed(), Some(last_log_id)); + let io_id = IOId::::new_log_io(VoteOf::::default().into_committed(), Some(last_log_id)); let notify = Notification::LocalIO { io_id }; let cb = IOFlushed::new(notify, tx.downgrade()); diff --git a/openraft/src/vote/committed.rs b/openraft/src/vote/committed.rs index 8d114161c..07f4745f2 100644 --- a/openraft/src/vote/committed.rs +++ b/openraft/src/vote/committed.rs @@ -2,10 +2,10 @@ use std::cmp::Ordering; use std::fmt; use crate::type_config::alias::CommittedLeaderIdOf; +use crate::type_config::alias::VoteOf; use crate::vote::ref_vote::RefVote; use crate::vote::RaftLeaderId; use crate::RaftTypeConfig; -use crate::Vote; /// Represents a committed Vote that has been accepted by a quorum. /// @@ -16,7 +16,7 @@ use crate::Vote; pub(crate) struct CommittedVote where C: RaftTypeConfig { - vote: Vote, + vote: VoteOf, } /// The `CommittedVote` is totally ordered. @@ -37,7 +37,7 @@ where C: RaftTypeConfig impl CommittedVote where C: RaftTypeConfig { - pub(crate) fn new(mut vote: Vote) -> Self { + pub(crate) fn new(mut vote: VoteOf) -> Self { vote.committed = true; Self { vote } } @@ -46,7 +46,7 @@ where C: RaftTypeConfig self.vote.leader_id().to_committed() } - pub(crate) fn into_vote(self) -> Vote { + pub(crate) fn into_vote(self) -> VoteOf { self.vote } diff --git a/openraft/src/vote/non_committed.rs b/openraft/src/vote/non_committed.rs index 1c9b30c15..06195fd6e 100644 --- a/openraft/src/vote/non_committed.rs +++ b/openraft/src/vote/non_committed.rs @@ -1,9 +1,9 @@ use std::fmt; use crate::type_config::alias::LeaderIdOf; +use crate::type_config::alias::VoteOf; use crate::vote::ref_vote::RefVote; use crate::RaftTypeConfig; -use crate::Vote; /// Represents a non-committed Vote that has **NOT** been granted by a quorum. /// @@ -14,13 +14,13 @@ use crate::Vote; pub(crate) struct NonCommittedVote where C: RaftTypeConfig { - vote: Vote, + vote: VoteOf, } impl NonCommittedVote where C: RaftTypeConfig { - pub(crate) fn new(vote: Vote) -> Self { + pub(crate) fn new(vote: VoteOf) -> Self { debug_assert!(!vote.committed); Self { vote } } @@ -29,7 +29,7 @@ where C: RaftTypeConfig &self.vote.leader_id } - pub(crate) fn into_vote(self) -> Vote { + pub(crate) fn into_vote(self) -> VoteOf { self.vote }