diff --git a/cluster_benchmark/tests/benchmark/store.rs b/cluster_benchmark/tests/benchmark/store.rs index 10ac2f528..30b7e27e2 100644 --- a/cluster_benchmark/tests/benchmark/store.rs +++ b/cluster_benchmark/tests/benchmark/store.rs @@ -8,6 +8,7 @@ use std::sync::atomic::AtomicU64; use std::sync::atomic::Ordering; use std::sync::Arc; +use openraft::alias::LogIdOf; use openraft::alias::SnapshotDataOf; use openraft::storage::IOFlushed; use openraft::storage::LogState; @@ -18,7 +19,6 @@ use openraft::storage::RaftStateMachine; use openraft::storage::Snapshot; use openraft::Entry; use openraft::EntryPayload; -use openraft::LogId; use openraft::OptionalSend; use openraft::RaftLogId; use openraft::SnapshotMeta; @@ -61,14 +61,14 @@ pub struct StoredSnapshot { #[derive(Serialize, Deserialize, Debug, Default, Clone)] pub struct StateMachine { - pub last_applied_log: Option>, + pub last_applied_log: Option>, pub last_membership: StoredMembership, } pub struct LogStore { vote: RwLock>>, log: RwLock>>, - last_purged_log_id: RwLock>>, + last_purged_log_id: RwLock>>, } impl LogStore { @@ -212,7 +212,7 @@ impl RaftLogStorage for Arc { } #[tracing::instrument(level = "debug", skip(self))] - async fn truncate(&mut self, log_id: LogId) -> Result<(), StorageError> { + async fn truncate(&mut self, log_id: LogIdOf) -> Result<(), StorageError> { let mut log = self.log.write().await; log.split_off(&log_id.index); @@ -220,7 +220,7 @@ impl RaftLogStorage for Arc { } #[tracing::instrument(level = "debug", skip_all)] - async fn purge(&mut self, log_id: LogId) -> Result<(), StorageError> { + async fn purge(&mut self, log_id: LogIdOf) -> Result<(), StorageError> { { let mut p = self.last_purged_log_id.write().await; *p = Some(log_id); @@ -253,7 +253,7 @@ impl RaftLogStorage for Arc { impl RaftStateMachine for Arc { async fn applied_state( &mut self, - ) -> Result<(Option>, StoredMembership), StorageError> { + ) -> Result<(Option>, StoredMembership), StorageError> { let sm = self.sm.read().await; Ok((sm.last_applied_log, sm.last_membership.clone())) } diff --git a/examples/memstore/src/log_store.rs b/examples/memstore/src/log_store.rs index d2a380eb1..f5b6dcefc 100644 --- a/examples/memstore/src/log_store.rs +++ b/examples/memstore/src/log_store.rs @@ -6,9 +6,9 @@ use std::fmt::Debug; use std::ops::RangeBounds; use std::sync::Arc; +use openraft::alias::LogIdOf; use openraft::alias::VoteOf; use openraft::storage::IOFlushed; -use openraft::LogId; use openraft::LogState; use openraft::RaftLogId; use openraft::RaftTypeConfig; @@ -24,13 +24,13 @@ pub struct LogStore { #[derive(Debug)] pub struct LogStoreInner { /// The last purged log id. - last_purged_log_id: Option>, + last_purged_log_id: Option>, /// The Raft log. log: BTreeMap, /// The commit log id. - committed: Option>, + committed: Option>, /// The current granted vote. vote: Option>, @@ -75,12 +75,12 @@ impl LogStoreInner { }) } - async fn save_committed(&mut self, committed: Option>) -> Result<(), StorageError> { + async fn save_committed(&mut self, committed: Option>) -> Result<(), StorageError> { self.committed = committed; Ok(()) } - async fn read_committed(&mut self) -> Result>, StorageError> { + async fn read_committed(&mut self) -> Result>, StorageError> { Ok(self.committed.clone()) } @@ -104,7 +104,7 @@ impl LogStoreInner { Ok(()) } - async fn truncate(&mut self, log_id: LogId) -> Result<(), StorageError> { + async fn truncate(&mut self, log_id: LogIdOf) -> Result<(), StorageError> { let keys = self.log.range(log_id.index..).map(|(k, _v)| *k).collect::>(); for key in keys { self.log.remove(&key); @@ -113,7 +113,7 @@ impl LogStoreInner { Ok(()) } - async fn purge(&mut self, log_id: LogId) -> Result<(), StorageError> { + async fn purge(&mut self, log_id: LogIdOf) -> Result<(), StorageError> { { let ld = &mut self.last_purged_log_id; assert!(ld.as_ref() <= Some(&log_id)); @@ -135,10 +135,10 @@ mod impl_log_store { use std::fmt::Debug; use std::ops::RangeBounds; + use openraft::alias::LogIdOf; use openraft::alias::VoteOf; use openraft::storage::IOFlushed; use openraft::storage::RaftLogStorage; - use openraft::LogId; use openraft::LogState; use openraft::RaftLogReader; use openraft::RaftTypeConfig; @@ -173,12 +173,12 @@ mod impl_log_store { inner.get_log_state().await } - async fn save_committed(&mut self, committed: Option>) -> Result<(), StorageError> { + async fn save_committed(&mut self, committed: Option>) -> Result<(), StorageError> { let mut inner = self.inner.lock().await; inner.save_committed(committed).await } - async fn read_committed(&mut self) -> Result>, StorageError> { + async fn read_committed(&mut self) -> Result>, StorageError> { let mut inner = self.inner.lock().await; inner.read_committed().await } @@ -194,12 +194,12 @@ mod impl_log_store { inner.append(entries, callback).await } - async fn truncate(&mut self, log_id: LogId) -> Result<(), StorageError> { + async fn truncate(&mut self, log_id: LogIdOf) -> Result<(), StorageError> { let mut inner = self.inner.lock().await; inner.truncate(log_id).await } - async fn purge(&mut self, log_id: LogId) -> Result<(), StorageError> { + async fn purge(&mut self, log_id: LogIdOf) -> Result<(), StorageError> { let mut inner = self.inner.lock().await; inner.purge(log_id).await } diff --git a/openraft/src/core/heartbeat/event.rs b/openraft/src/core/heartbeat/event.rs index 423bcfa7f..2a81380cb 100644 --- a/openraft/src/core/heartbeat/event.rs +++ b/openraft/src/core/heartbeat/event.rs @@ -4,7 +4,7 @@ use crate::display_ext::DisplayInstantExt; use crate::display_ext::DisplayOptionExt; use crate::replication::ReplicationSessionId; use crate::type_config::alias::InstantOf; -use crate::LogId; +use crate::type_config::alias::LogIdOf; use crate::RaftTypeConfig; /// The information for broadcasting a heartbeat. @@ -30,13 +30,13 @@ where C: RaftTypeConfig /// /// When there are no new logs to replicate, the Leader sends a heartbeat to replicate committed /// log id to followers to update their committed log id. - pub(crate) committed: Option>, + pub(crate) committed: Option>, } impl HeartbeatEvent where C: RaftTypeConfig { - pub(crate) fn new(time: InstantOf, session_id: ReplicationSessionId, committed: Option>) -> Self { + pub(crate) fn new(time: InstantOf, session_id: ReplicationSessionId, committed: Option>) -> Self { Self { time, session_id, diff --git a/openraft/src/core/notification.rs b/openraft/src/core/notification.rs index 8b182f5c0..3d31b68b5 100644 --- a/openraft/src/core/notification.rs +++ b/openraft/src/core/notification.rs @@ -39,7 +39,7 @@ where C: RaftTypeConfig leader_vote: CommittedVote, // TODO: need this? // /// The cluster this replication works for. - // membership_log_id: Option>, + // membership_log_id: Option>, }, /// [`StorageError`] error has taken place locally(not on remote node), diff --git a/openraft/src/core/raft_core.rs b/openraft/src/core/raft_core.rs index 08a6001b9..a1f38c8a5 100644 --- a/openraft/src/core/raft_core.rs +++ b/openraft/src/core/raft_core.rs @@ -86,6 +86,7 @@ use crate::runtime::RaftRuntime; use crate::storage::IOFlushed; use crate::storage::RaftLogStorage; use crate::type_config::alias::InstantOf; +use crate::type_config::alias::LogIdOf; use crate::type_config::alias::MpscUnboundedReceiverOf; use crate::type_config::alias::MpscUnboundedSenderOf; use crate::type_config::alias::OneshotReceiverOf; @@ -101,7 +102,6 @@ use crate::vote::RaftLeaderId; use crate::vote::RaftVote; use crate::ChangeMembers; use crate::Instant; -use crate::LogId; use crate::Membership; use crate::OptionalSend; use crate::RaftTypeConfig; @@ -110,7 +110,7 @@ use crate::StorageError; /// A temp struct to hold the data for a node that is being applied. #[derive(Debug)] pub(crate) struct ApplyingEntry { - log_id: LogId, + log_id: LogIdOf, membership: Option>, } @@ -127,7 +127,7 @@ where C: RaftTypeConfig } impl ApplyingEntry { - pub(crate) fn new(log_id: LogId, membership: Option>) -> Self { + pub(crate) fn new(log_id: LogIdOf, membership: Option>) -> Self { Self { log_id, membership } } } @@ -136,7 +136,7 @@ impl ApplyingEntry { pub(crate) struct ApplyResult { pub(crate) since: u64, pub(crate) end: u64, - pub(crate) last_applied: LogId, + pub(crate) last_applied: LogIdOf, pub(crate) applying_entries: Vec>, pub(crate) apply_results: Vec, } @@ -469,7 +469,7 @@ where } }; - let ent = C::Entry::new_membership(LogId::default(), new_membership); + let ent = C::Entry::new_membership(LogIdOf::::default(), new_membership); self.write_entry(ent, Some(tx)); } @@ -548,7 +548,7 @@ where .map(|(id, p)| { ( id.clone(), - as Borrow>>>::borrow(p).clone(), + as Borrow>>>::borrow(p).clone(), ) }) .collect(), @@ -670,7 +670,7 @@ where let membership = Membership::from(member_nodes); - let entry = C::Entry::new_membership(LogId::default(), membership); + let entry = C::Entry::new_membership(LogIdOf::::default(), membership); let res = self.engine.initialize(entry); // If there is an error, respond at once. @@ -761,8 +761,8 @@ where #[tracing::instrument(level = "debug", skip_all)] pub(crate) async fn apply_to_state_machine( &mut self, - first: LogId, - last: LogId, + first: LogIdOf, + last: LogIdOf, ) -> Result<(), StorageError> { tracing::debug!("{}: {}..={}", func_name!(), first, last); diff --git a/openraft/src/engine/command.rs b/openraft/src/engine/command.rs index 8afe8a894..f92ac0cfc 100644 --- a/openraft/src/engine/command.rs +++ b/openraft/src/engine/command.rs @@ -20,10 +20,10 @@ use crate::raft::VoteResponse; use crate::raft_state::IOId; use crate::replication::request::Replicate; use crate::replication::ReplicationSessionId; +use crate::type_config::alias::LogIdOf; 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; @@ -65,12 +65,12 @@ where C: RaftTypeConfig }, /// Replicate the committed log id to other nodes - ReplicateCommitted { committed: Option> }, + ReplicateCommitted { committed: Option> }, /// Broadcast heartbeat to all other nodes. BroadcastHeartbeat { session_id: ReplicationSessionId, - committed: Option>, + committed: Option>, }, /// Save the committed log id to [`RaftLogStorage`]. @@ -79,7 +79,7 @@ where C: RaftTypeConfig /// latest state. /// /// [`RaftLogStorage`]: crate::storage::RaftLogStorage - SaveCommitted { committed: LogId }, + SaveCommitted { committed: LogIdOf }, /// Commit log entries that are already persisted in the store, upto `upto`, inclusive. /// @@ -91,8 +91,8 @@ where C: RaftTypeConfig /// [`RaftLogStorage::save_committed()`]: crate::storage::RaftLogStorage::save_committed /// [`RaftStateMachine::apply()`]: crate::storage::RaftStateMachine::apply Apply { - already_committed: Option>, - upto: LogId, + already_committed: Option>, + upto: LogIdOf, }, /// Replicate log entries or snapshot to a target. @@ -118,11 +118,11 @@ where C: RaftTypeConfig SendVote { vote_req: VoteRequest }, /// Purge log from the beginning to `upto`, inclusive. - PurgeLog { upto: LogId }, + PurgeLog { upto: LogIdOf }, /// Delete logs that conflict with the leader from a follower/learner since log id `since`, /// inclusive. - TruncateLog { since: LogId }, + TruncateLog { since: LogIdOf }, /// A command send to state machine worker [`sm::worker::Worker`]. /// @@ -296,14 +296,14 @@ where C: RaftTypeConfig /// This is only used by [`Raft::initialize()`], because when initializing there is no leader. /// /// [`Raft::initialize()`]: `crate::Raft::initialize()` - LogFlushed { log_id: Option> }, + LogFlushed { log_id: Option> }, /// Wait until the log is applied to the state machine. #[allow(dead_code)] - Applied { log_id: Option> }, + Applied { log_id: Option> }, /// Wait until snapshot is built and includes the log id. - Snapshot { log_id: Option> }, + Snapshot { log_id: Option> }, } impl fmt::Display for Condition diff --git a/openraft/src/engine/engine_impl.rs b/openraft/src/engine/engine_impl.rs index a6b51bcd4..00ceeab18 100644 --- a/openraft/src/engine/engine_impl.rs +++ b/openraft/src/engine/engine_impl.rs @@ -46,6 +46,7 @@ use crate::raft_state::RaftState; use crate::storage::Snapshot; use crate::storage::SnapshotMeta; use crate::type_config::alias::LeaderIdOf; +use crate::type_config::alias::LogIdOf; use crate::type_config::alias::ResponderOf; use crate::type_config::alias::SnapshotDataOf; use crate::type_config::alias::VoteOf; @@ -54,7 +55,6 @@ use crate::vote::raft_vote::RaftVoteExt; use crate::vote::RaftLeaderId; use crate::vote::RaftTerm; use crate::vote::RaftVote; -use crate::LogId; use crate::LogIdOptionExt; use crate::Membership; use crate::RaftLogId; @@ -192,7 +192,7 @@ where C: RaftTypeConfig self.check_initialize()?; // The very first log id - entry.set_log_id(&LogId::default()); + entry.set_log_id(&LogIdOf::::default()); let m = entry.get_membership().expect("the only log entry for initializing has to be membership log"); self.check_members_contain_me(m)?; @@ -383,7 +383,7 @@ where C: RaftTypeConfig pub(crate) fn handle_append_entries( &mut self, vote: &VoteOf, - prev_log_id: Option>, + prev_log_id: Option>, entries: Vec, tx: Option>, ) -> bool { @@ -422,7 +422,7 @@ where C: RaftTypeConfig pub(crate) fn append_entries( &mut self, vote: &VoteOf, - prev_log_id: Option>, + prev_log_id: Option>, entries: Vec, ) -> Result<(), RejectAppendEntries> { self.vote_handler().update_vote(vote)?; @@ -438,7 +438,7 @@ where C: RaftTypeConfig /// Commit entries for follower/learner. #[tracing::instrument(level = "debug", skip_all)] - pub(crate) fn handle_commit_entries(&mut self, leader_committed: Option>) { + pub(crate) fn handle_commit_entries(&mut self, leader_committed: Option>) { tracing::debug!( leader_committed = display(leader_committed.display()), my_accepted = display(self.state.accepted_io().display()), @@ -658,7 +658,7 @@ where C: RaftTypeConfig self.leader_handler() .unwrap() - .leader_append_entries(vec![C::Entry::new_blank(LogId::::default())]); + .leader_append_entries(vec![C::Entry::new_blank(LogIdOf::::default())]); } /// Check if a raft node is in a state that allows to initialize. diff --git a/openraft/src/engine/handler/following_handler/mod.rs b/openraft/src/engine/handler/following_handler/mod.rs index 1a0a9d4bc..dffb05d64 100644 --- a/openraft/src/engine/handler/following_handler/mod.rs +++ b/openraft/src/engine/handler/following_handler/mod.rs @@ -16,9 +16,9 @@ use crate::error::RejectAppendEntries; use crate::raft_state::IOId; use crate::raft_state::LogStateReader; use crate::storage::Snapshot; +use crate::type_config::alias::LogIdOf; use crate::vote::committed::CommittedVote; use crate::EffectiveMembership; -use crate::LogId; use crate::LogIdOptionExt; use crate::RaftLogId; use crate::RaftState; @@ -59,7 +59,7 @@ where C: RaftTypeConfig /// /// Also clean conflicting entries and update membership state. #[tracing::instrument(level = "debug", skip_all)] - pub(crate) fn append_entries(&mut self, prev_log_id: Option>, mut entries: Vec) { + pub(crate) fn append_entries(&mut self, prev_log_id: Option>, mut entries: Vec) { tracing::debug!( "{}: local last_log_id: {}, request: prev_log_id: {}, entries: {}", func_name!(), @@ -113,7 +113,7 @@ where C: RaftTypeConfig /// If not, truncate the local log and return an error. pub(crate) fn ensure_log_consecutive( &mut self, - prev_log_id: Option<&LogId>, + prev_log_id: Option<&LogIdOf>, ) -> Result<(), RejectAppendEntries> { if let Some(prev) = prev_log_id { if !self.state.has_log_id(prev) { @@ -161,7 +161,7 @@ where C: RaftTypeConfig /// Commit entries that are already committed by the leader. #[tracing::instrument(level = "debug", skip_all)] - pub(crate) fn commit_entries(&mut self, leader_committed: Option>) { + pub(crate) fn commit_entries(&mut self, leader_committed: Option>) { let accepted = self.state.accepted_io().cloned(); let accepted = accepted.and_then(|x| x.last_log_id().cloned()); let committed = std::cmp::min(accepted.clone(), leader_committed.clone()); diff --git a/openraft/src/engine/handler/log_handler/calc_purge_upto_test.rs b/openraft/src/engine/handler/log_handler/calc_purge_upto_test.rs index ca98c730a..2b34f44eb 100644 --- a/openraft/src/engine/handler/log_handler/calc_purge_upto_test.rs +++ b/openraft/src/engine/handler/log_handler/calc_purge_upto_test.rs @@ -2,10 +2,11 @@ use crate::engine::testing::UTConfig; use crate::engine::Engine; use crate::engine::LogIdList; use crate::type_config::alias::LeaderIdOf; +use crate::type_config::alias::LogIdOf; use crate::vote::RaftLeaderIdExt; use crate::LogId; -fn log_id(term: u64, index: u64) -> LogId { +fn log_id(term: u64, index: u64) -> LogIdOf { LogId { leader_id: LeaderIdOf::::new_committed(term, 0), index, diff --git a/openraft/src/engine/handler/log_handler/mod.rs b/openraft/src/engine/handler/log_handler/mod.rs index bb3d67ec2..79db5c05b 100644 --- a/openraft/src/engine/handler/log_handler/mod.rs +++ b/openraft/src/engine/handler/log_handler/mod.rs @@ -3,7 +3,7 @@ use crate::engine::Command; use crate::engine::EngineConfig; use crate::engine::EngineOutput; use crate::raft_state::LogStateReader; -use crate::LogId; +use crate::type_config::alias::LogIdOf; use crate::LogIdOptionExt; use crate::RaftState; use crate::RaftTypeConfig; @@ -61,7 +61,7 @@ where C: RaftTypeConfig /// Update the log id it expect to purge up to. It won't trigger purge immediately. #[tracing::instrument(level = "debug", skip_all)] - pub(crate) fn update_purge_upto(&mut self, purge_upto: LogId) { + pub(crate) fn update_purge_upto(&mut self, purge_upto: LogIdOf) { debug_assert!(self.state.purge_upto() <= Some(&purge_upto)); self.state.purge_upto = Some(purge_upto); } @@ -74,7 +74,7 @@ where C: RaftTypeConfig /// `max_keep` specifies the number of applied logs to keep. /// `max_keep==0` means every applied log can be purged. #[tracing::instrument(level = "debug", skip_all)] - pub(crate) fn calc_purge_upto(&self) -> Option> { + pub(crate) fn calc_purge_upto(&self) -> Option> { let st = &self.state; let max_keep = self.config.max_in_snapshot_log_to_keep; let batch_size = self.config.purge_batch_size; diff --git a/openraft/src/engine/handler/replication_handler/mod.rs b/openraft/src/engine/handler/replication_handler/mod.rs index 6cce15526..f6b317cbe 100644 --- a/openraft/src/engine/handler/replication_handler/mod.rs +++ b/openraft/src/engine/handler/replication_handler/mod.rs @@ -19,9 +19,9 @@ use crate::raft_state::LogStateReader; use crate::replication::request::Replicate; use crate::replication::response::ReplicationResult; use crate::type_config::alias::InstantOf; +use crate::type_config::alias::LogIdOf; use crate::vote::raft_vote::RaftVoteExt; use crate::EffectiveMembership; -use crate::LogId; use crate::LogIdOptionExt; use crate::Membership; use crate::RaftState; @@ -57,7 +57,7 @@ where C: RaftTypeConfig /// /// It is called by the leader when a new membership log is appended to log store. #[tracing::instrument(level = "debug", skip_all)] - pub(crate) fn append_membership(&mut self, log_id: &LogId, m: &Membership) { + pub(crate) fn append_membership(&mut self, log_id: &LogIdOf, m: &Membership) { tracing::debug!("update effective membership: log_id:{} {}", log_id, m); debug_assert!( @@ -150,7 +150,7 @@ where C: RaftTypeConfig /// Update progress when replicated data(logs or snapshot) matches on follower/learner and is /// accepted. #[tracing::instrument(level = "debug", skip_all)] - pub(crate) fn update_matching(&mut self, node_id: C::NodeId, log_id: Option>) { + pub(crate) fn update_matching(&mut self, node_id: C::NodeId, log_id: Option>) { tracing::debug!( node_id = display(&node_id), log_id = display(log_id.display()), @@ -183,7 +183,7 @@ where C: RaftTypeConfig /// /// In raft a log that is granted and in the leader term is committed. #[tracing::instrument(level = "debug", skip_all)] - pub(crate) fn try_commit_quorum_accepted(&mut self, granted: Option>) { + pub(crate) fn try_commit_quorum_accepted(&mut self, granted: Option>) { // Only when the log id is proposed by current leader, it is committed. if let Some(ref c) = granted { if !self.state.vote_ref().is_same_leader(c.committed_leader_id()) { @@ -214,7 +214,7 @@ where C: RaftTypeConfig /// Update progress when replicated data(logs or snapshot) does not match follower/learner state /// and is rejected. #[tracing::instrument(level = "debug", skip_all)] - pub(crate) fn update_conflicting(&mut self, target: C::NodeId, conflict: LogId) { + pub(crate) fn update_conflicting(&mut self, target: C::NodeId, conflict: LogIdOf) { // TODO(2): test it? let prog_entry = self.leader.progress.get_mut(&target).unwrap(); @@ -391,7 +391,7 @@ where C: RaftTypeConfig /// /// Writing to local log store does not have to wait for a replication response from remote /// node. Thus it can just be done in a fast-path. - pub(crate) fn update_local_progress(&mut self, upto: Option>) { + pub(crate) fn update_local_progress(&mut self, upto: Option>) { tracing::debug!(upto = display(upto.display()), "{}", func_name!()); if upto.is_none() { diff --git a/openraft/src/engine/handler/vote_handler/mod.rs b/openraft/src/engine/handler/vote_handler/mod.rs index 7dd55b954..e5830c78d 100644 --- a/openraft/src/engine/handler/vote_handler/mod.rs +++ b/openraft/src/engine/handler/vote_handler/mod.rs @@ -18,12 +18,12 @@ use crate::proposer::CandidateState; use crate::proposer::LeaderState; use crate::raft_state::IOId; use crate::raft_state::LogStateReader; +use crate::type_config::alias::LogIdOf; use crate::type_config::alias::VoteOf; use crate::type_config::TypeConfigExt; use crate::vote::raft_vote::RaftVoteExt; use crate::vote::RaftLeaderId; use crate::vote::RaftVote; -use crate::LogId; use crate::OptionalSend; use crate::RaftState; use crate::RaftTypeConfig; @@ -213,7 +213,7 @@ where C: RaftTypeConfig // If the leader has not yet proposed any log, propose a blank log and initiate replication; // Otherwise, just initiate replication. if last_log_id < noop_log_id { - self.leader_handler().leader_append_entries(vec![C::Entry::new_blank(LogId::::default())]); + self.leader_handler().leader_append_entries(vec![C::Entry::new_blank(LogIdOf::::default())]); } else { self.replication_handler().initiate_replication(); } diff --git a/openraft/src/engine/log_id_list.rs b/openraft/src/engine/log_id_list.rs index 8b395bd38..20971cfd3 100644 --- a/openraft/src/engine/log_id_list.rs +++ b/openraft/src/engine/log_id_list.rs @@ -4,7 +4,6 @@ use crate::engine::leader_log_ids::LeaderLogIds; use crate::log_id::RaftLogId; use crate::storage::RaftLogReaderExt; use crate::type_config::alias::LogIdOf; -use crate::LogId; use crate::LogIdOptionExt; use crate::RaftLogReader; use crate::RaftTypeConfig; @@ -22,7 +21,7 @@ use crate::StorageError; pub struct LogIdList where C: RaftTypeConfig { - key_log_ids: Vec>, + key_log_ids: Vec>, } impl LogIdList @@ -49,7 +48,7 @@ where C: RaftTypeConfig /// A-------C-------C : find(A,C) /// ``` pub(crate) async fn get_key_log_ids( - range: RangeInclusive>, + range: RangeInclusive>, sto: &mut LR, ) -> Result>, StorageError> where @@ -122,7 +121,7 @@ where C: RaftTypeConfig /// Create a new `LogIdList`. /// /// It stores the last purged log id, and a series of key log ids. - pub fn new(key_log_ids: impl IntoIterator>) -> Self { + pub fn new(key_log_ids: impl IntoIterator>) -> Self { Self { key_log_ids: key_log_ids.into_iter().collect(), } @@ -180,7 +179,7 @@ where C: RaftTypeConfig /// /// NOTE: The last two in `key_log_ids` may be with the same `leader_id`, because `last_log_id` /// always present in `log_ids`. - pub(crate) fn append(&mut self, new_log_id: LogId) { + pub(crate) fn append(&mut self, new_log_id: LogIdOf) { let l = self.key_log_ids.len(); if l == 0 { self.key_log_ids.push(new_log_id); @@ -240,14 +239,14 @@ where C: RaftTypeConfig if let Some(last) = last { let (last_leader_id, last_index) = (last.leader_id.clone(), last.index); if last_index < at - 1 { - self.append(LogId::new(last_leader_id, at - 1)); + self.append(LogIdOf::::new(last_leader_id, at - 1)); } } } /// Purge log ids upto the log with index `upto_index`, inclusive. #[allow(dead_code)] - pub(crate) fn purge(&mut self, upto: &LogId) { + pub(crate) fn purge(&mut self, upto: &LogIdOf) { let last = self.last().cloned(); // When installing snapshot it may need to purge across the `last_log_id`. @@ -281,32 +280,32 @@ where C: RaftTypeConfig /// It will return `last_purged_log_id` if index is at the last purged index. // leader_id: Copy is feature gated #[allow(clippy::clone_on_copy)] - pub(crate) fn get(&self, index: u64) -> Option> { + pub(crate) fn get(&self, index: u64) -> Option> { let res = self.key_log_ids.binary_search_by(|log_id| log_id.index.cmp(&index)); match res { - Ok(i) => Some(LogId::new(self.key_log_ids[i].leader_id.clone(), index)), + Ok(i) => Some(LogIdOf::::new(self.key_log_ids[i].leader_id.clone(), index)), Err(i) => { if i == 0 || i == self.key_log_ids.len() { None } else { - Some(LogId::new(self.key_log_ids[i - 1].leader_id.clone(), index)) + Some(LogIdOf::::new(self.key_log_ids[i - 1].leader_id.clone(), index)) } } } } - pub(crate) fn first(&self) -> Option<&LogId> { + pub(crate) fn first(&self) -> Option<&LogIdOf> { self.key_log_ids.first() } - pub(crate) fn last(&self) -> Option<&LogId> { + pub(crate) fn last(&self) -> Option<&LogIdOf> { self.key_log_ids.last() } // This method will only be used under feature tokio-rt #[cfg_attr(not(feature = "tokio-rt"), allow(dead_code))] - pub(crate) fn key_log_ids(&self) -> &[LogId] { + pub(crate) fn key_log_ids(&self) -> &[LogIdOf] { &self.key_log_ids } diff --git a/openraft/src/engine/tests/initialize_test.rs b/openraft/src/engine/tests/initialize_test.rs index bce6ca47f..3a1e6ece6 100644 --- a/openraft/src/engine/tests/initialize_test.rs +++ b/openraft/src/engine/tests/initialize_test.rs @@ -15,11 +15,11 @@ use crate::error::NotInMembers; use crate::raft::VoteRequest; use crate::raft_state::LogStateReader; use crate::testing::log_id; +use crate::type_config::alias::LogIdOf; use crate::type_config::TypeConfigExt; use crate::utime::Leased; use crate::vote::raft_vote::RaftVoteExt; use crate::Entry; -use crate::LogId; use crate::Membership; use crate::Vote; @@ -36,7 +36,7 @@ fn test_initialize_single_node() -> anyhow::Result<()> { let log_id0 = log_id(0, 0, 0); let m1 = || Membership::::new_with_defaults(vec![btreeset! {1}], []); - let entry = Entry::::new_membership(LogId::default(), m1()); + let entry = Entry::::new_membership(LogIdOf::::default(), m1()); tracing::info!("--- ok: init empty node 1 with membership(1,2)"); tracing::info!("--- expect OK result, check output commands and state changes"); @@ -57,7 +57,7 @@ fn test_initialize_single_node() -> anyhow::Result<()> { vec![ Command::AppendInputEntries { committed_vote: Vote::default().into_committed(), - entries: vec![Entry::::new_membership(LogId::default(), m1())], + entries: vec![Entry::::new_membership(LogIdOf::::default(), m1())], }, // When update the effective membership, the engine set it to Follower. // But when initializing, it will switch to Candidate at once, in the last output @@ -86,7 +86,7 @@ fn test_initialize() -> anyhow::Result<()> { let log_id0 = log_id(0, 0, 0); let m12 = || Membership::::new_with_defaults(vec![btreeset! {1,2}], []); - let entry = || Entry::::new_membership(LogId::default(), m12()); + let entry = || Entry::::new_membership(LogIdOf::::default(), m12()); tracing::info!("--- ok: init empty node 1 with membership(1,2)"); tracing::info!("--- expect OK result, check output commands and state changes"); @@ -107,7 +107,7 @@ fn test_initialize() -> anyhow::Result<()> { vec![ Command::AppendInputEntries { committed_vote: Vote::default().into_committed(), - entries: vec![Entry::new_membership(LogId::default(), m12())], + entries: vec![Entry::new_membership(LogIdOf::::default(), m12())], }, // When update the effective membership, the engine set it to Follower. // But when initializing, it will switch to Candidate at once, in the last output diff --git a/openraft/src/entry/mod.rs b/openraft/src/entry/mod.rs index 7cd013b55..812540621 100644 --- a/openraft/src/entry/mod.rs +++ b/openraft/src/entry/mod.rs @@ -4,7 +4,6 @@ use std::fmt; use std::fmt::Debug; use crate::log_id::RaftLogId; -use crate::LogId; use crate::Membership; use crate::RaftTypeConfig; @@ -16,12 +15,14 @@ pub use traits::FromAppData; pub use traits::RaftEntry; pub use traits::RaftPayload; +use crate::type_config::alias::LogIdOf; + /// A Raft log entry. #[cfg_attr(feature = "serde", derive(serde::Deserialize, serde::Serialize), serde(bound = ""))] pub struct Entry where C: RaftTypeConfig { - pub log_id: LogId, + pub log_id: LogIdOf, /// This entry's payload. pub payload: EntryPayload, @@ -53,7 +54,7 @@ where C: RaftTypeConfig { fn default() -> Self { Self { - log_id: LogId::default(), + log_id: LogIdOf::::default(), payload: EntryPayload::Blank, } } @@ -100,11 +101,11 @@ where C: RaftTypeConfig impl RaftLogId for Entry where C: RaftTypeConfig { - fn get_log_id(&self) -> &LogId { + fn get_log_id(&self) -> &LogIdOf { &self.log_id } - fn set_log_id(&mut self, log_id: &LogId) { + fn set_log_id(&mut self, log_id: &LogIdOf) { self.log_id = log_id.clone(); } } @@ -112,14 +113,14 @@ where C: RaftTypeConfig impl RaftEntry for Entry where C: RaftTypeConfig { - fn new_blank(log_id: LogId) -> Self { + fn new_blank(log_id: LogIdOf) -> Self { Self { log_id, payload: EntryPayload::Blank, } } - fn new_membership(log_id: LogId, m: Membership) -> Self { + fn new_membership(log_id: LogIdOf, m: Membership) -> Self { Self { log_id, payload: EntryPayload::Membership(m), @@ -132,7 +133,7 @@ where C: RaftTypeConfig { fn from_app_data(d: C::D) -> Self { Entry { - log_id: LogId::default(), + log_id: LogIdOf::::default(), payload: EntryPayload::Normal(d), } } diff --git a/openraft/src/entry/traits.rs b/openraft/src/entry/traits.rs index 53c80bb39..944b7d214 100644 --- a/openraft/src/entry/traits.rs +++ b/openraft/src/entry/traits.rs @@ -2,7 +2,7 @@ use std::fmt::Debug; use std::fmt::Display; use crate::log_id::RaftLogId; -use crate::LogId; +use crate::type_config::alias::LogIdOf; use crate::Membership; use crate::OptionalSend; use crate::OptionalSerde; @@ -29,12 +29,12 @@ where /// Create a new blank log entry. /// /// The returned instance must return `true` for `Self::is_blank()`. - fn new_blank(log_id: LogId) -> Self; + fn new_blank(log_id: LogIdOf) -> Self; /// Create a new membership log entry. /// /// The returned instance must return `Some()` for `Self::get_membership()`. - fn new_membership(log_id: LogId, m: Membership) -> Self; + fn new_membership(log_id: LogIdOf, m: Membership) -> Self; } /// Build a raft log entry from app data. diff --git a/openraft/src/error.rs b/openraft/src/error.rs index 5368cbf21..9195ebfab 100644 --- a/openraft/src/error.rs +++ b/openraft/src/error.rs @@ -29,8 +29,8 @@ use crate::network::RPCTypes; use crate::raft::AppendEntriesResponse; use crate::raft_types::SnapshotSegmentId; use crate::try_as_ref::TryAsRef; +use crate::type_config::alias::LogIdOf; use crate::type_config::alias::VoteOf; -use crate::LogId; use crate::Membership; use crate::RaftTypeConfig; use crate::StorageError; @@ -587,8 +587,8 @@ pub struct QuorumNotEnough { #[cfg_attr(feature = "serde", derive(serde::Deserialize, serde::Serialize), serde(bound = ""))] #[error("the cluster is already undergoing a configuration change at log {membership_log_id:?}, last committed membership log id: {committed:?}")] pub struct InProgress { - pub committed: Option>, - pub membership_log_id: Option>, + pub committed: Option>, + pub membership_log_id: Option>, } #[derive(Debug, Clone, PartialEq, Eq, thiserror::Error)] @@ -602,7 +602,7 @@ pub struct LearnerNotFound { #[cfg_attr(feature = "serde", derive(serde::Deserialize, serde::Serialize), serde(bound = ""))] #[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 last_log_id: Option>, pub vote: VoteOf, } @@ -640,7 +640,7 @@ pub(crate) enum RejectVoteRequest { #[allow(dead_code)] #[error("reject vote request by a greater last-log-id: {0:?}")] - ByLastLogId(Option>), + ByLastLogId(Option>), } impl From> for AppendEntriesResponse @@ -662,7 +662,10 @@ pub(crate) enum RejectAppendEntries { ByVote(VoteOf), #[error("reject AppendEntries because of conflicting log-id: {local:?}; expect to be: {expect:?}")] - ByConflictingLogId { expect: LogId, local: Option> }, + ByConflictingLogId { + expect: LogIdOf, + local: Option>, + }, } impl From> for RejectAppendEntries diff --git a/openraft/src/log_id/log_id_option_ext.rs b/openraft/src/log_id/log_id_option_ext.rs index c8f412d56..8e36973b4 100644 --- a/openraft/src/log_id/log_id_option_ext.rs +++ b/openraft/src/log_id/log_id_option_ext.rs @@ -1,4 +1,4 @@ -use crate::LogId; +use crate::type_config::alias::LogIdOf; use crate::RaftTypeConfig; /// This helper trait extracts information from an `Option`. @@ -12,7 +12,7 @@ pub trait LogIdOptionExt { fn next_index(&self) -> u64; } -impl LogIdOptionExt for Option> +impl LogIdOptionExt for Option> where C: RaftTypeConfig { fn index(&self) -> Option { @@ -27,7 +27,7 @@ where C: RaftTypeConfig } } -impl LogIdOptionExt for Option<&LogId> +impl LogIdOptionExt for Option<&LogIdOf> where C: RaftTypeConfig { fn index(&self) -> Option { diff --git a/openraft/src/log_id/raft_log_id.rs b/openraft/src/log_id/raft_log_id.rs index 06346b33c..06feb8f8c 100644 --- a/openraft/src/log_id/raft_log_id.rs +++ b/openraft/src/log_id/raft_log_id.rs @@ -1,5 +1,5 @@ use crate::type_config::alias::CommittedLeaderIdOf; -use crate::LogId; +use crate::type_config::alias::LogIdOf; use crate::RaftTypeConfig; /// Defines API to operate an object that contains a log-id, such as a log entry or a log id. @@ -17,8 +17,8 @@ where C: RaftTypeConfig } /// Return a reference to the log-id it stores. - fn get_log_id(&self) -> &LogId; + fn get_log_id(&self) -> &LogIdOf; /// Update the log id it contains. - fn set_log_id(&mut self, log_id: &LogId); + fn set_log_id(&mut self, log_id: &LogIdOf); } diff --git a/openraft/src/log_id_range.rs b/openraft/src/log_id_range.rs index ef7b01ea4..acce6d89b 100644 --- a/openraft/src/log_id_range.rs +++ b/openraft/src/log_id_range.rs @@ -6,7 +6,7 @@ use validit::Validate; use crate::display_ext::DisplayOptionExt; use crate::type_config::alias::CommittedLeaderIdOf; -use crate::LogId; +use crate::type_config::alias::LogIdOf; use crate::LogIdOptionExt; use crate::RaftTypeConfig; @@ -21,10 +21,10 @@ pub(crate) struct LogIdRange where C: RaftTypeConfig { /// The prev log id before the first to send, exclusive. - pub(crate) prev: Option>, + pub(crate) prev: Option>, /// The last log id to send, inclusive. - pub(crate) last: Option>, + pub(crate) last: Option>, } impl Copy for LogIdRange @@ -54,7 +54,7 @@ where C: RaftTypeConfig impl LogIdRange where C: RaftTypeConfig { - pub(crate) fn new(prev: Option>, last: Option>) -> Self { + pub(crate) fn new(prev: Option>, last: Option>) -> Self { Self { prev, last } } @@ -71,9 +71,9 @@ mod tests { use crate::engine::testing::UTConfig; use crate::log_id_range::LogIdRange; use crate::testing; - use crate::LogId; + use crate::type_config::alias::LogIdOf; - fn log_id(index: u64) -> LogId { + fn log_id(index: u64) -> LogIdOf { testing::log_id(1, 1, index) } diff --git a/openraft/src/membership/effective_membership.rs b/openraft/src/membership/effective_membership.rs index 7afc17e4e..66e00d66a 100644 --- a/openraft/src/membership/effective_membership.rs +++ b/openraft/src/membership/effective_membership.rs @@ -7,7 +7,7 @@ use crate::display_ext::DisplayOptionExt; use crate::log_id::RaftLogId; use crate::quorum::Joint; use crate::quorum::QuorumSet; -use crate::LogId; +use crate::type_config::alias::LogIdOf; use crate::Membership; use crate::RaftTypeConfig; use crate::StoredMembership; @@ -65,11 +65,11 @@ where impl EffectiveMembership where C: RaftTypeConfig { - pub(crate) fn new_arc(log_id: Option>, membership: Membership) -> Arc { + pub(crate) fn new_arc(log_id: Option>, membership: Membership) -> Arc { Arc::new(Self::new(log_id, membership)) } - pub fn new(log_id: Option>, membership: Membership) -> Self { + pub fn new(log_id: Option>, membership: Membership) -> Self { let voter_ids = membership.voter_ids().collect(); let configs = membership.get_joint_config(); @@ -95,7 +95,7 @@ where C: RaftTypeConfig &self.stored_membership } - pub fn log_id(&self) -> &Option> { + pub fn log_id(&self) -> &Option> { self.stored_membership.log_id() } diff --git a/openraft/src/membership/stored_membership.rs b/openraft/src/membership/stored_membership.rs index 78dd5ed61..c38b8afc2 100644 --- a/openraft/src/membership/stored_membership.rs +++ b/openraft/src/membership/stored_membership.rs @@ -1,7 +1,7 @@ use std::fmt; use crate::display_ext::DisplayOption; -use crate::LogId; +use crate::type_config::alias::LogIdOf; use crate::Membership; use crate::RaftTypeConfig; @@ -21,7 +21,7 @@ pub struct StoredMembership where C: RaftTypeConfig { /// The id of the log that stores this membership config - log_id: Option>, + log_id: Option>, /// Membership config membership: Membership, @@ -30,11 +30,11 @@ where C: RaftTypeConfig impl StoredMembership where C: RaftTypeConfig { - pub fn new(log_id: Option>, membership: Membership) -> Self { + pub fn new(log_id: Option>, membership: Membership) -> Self { Self { log_id, membership } } - pub fn log_id(&self) -> &Option> { + pub fn log_id(&self) -> &Option> { &self.log_id } diff --git a/openraft/src/metrics/metric.rs b/openraft/src/metrics/metric.rs index 9081e0b9f..428149ba9 100644 --- a/openraft/src/metrics/metric.rs +++ b/openraft/src/metrics/metric.rs @@ -2,8 +2,8 @@ use std::cmp::Ordering; use crate::base::ord_by::OrdBy; use crate::metrics::metric_display::MetricDisplay; +use crate::type_config::alias::LogIdOf; use crate::type_config::alias::VoteOf; -use crate::LogId; use crate::LogIdOptionExt; use crate::RaftMetrics; use crate::RaftTypeConfig; @@ -18,10 +18,10 @@ where C: RaftTypeConfig Term(C::Term), Vote(VoteOf), LastLogIndex(Option), - Applied(Option>), + Applied(Option>), AppliedIndex(Option), - Snapshot(Option>), - Purged(Option>), + Snapshot(Option>), + Purged(Option>), } impl Metric diff --git a/openraft/src/metrics/raft_metrics.rs b/openraft/src/metrics/raft_metrics.rs index cbe18aa71..fea2e4422 100644 --- a/openraft/src/metrics/raft_metrics.rs +++ b/openraft/src/metrics/raft_metrics.rs @@ -9,10 +9,10 @@ use crate::metrics::HeartbeatMetrics; use crate::metrics::ReplicationMetrics; use crate::metrics::SerdeInstant; use crate::type_config::alias::InstantOf; +use crate::type_config::alias::LogIdOf; use crate::type_config::alias::SerdeInstantOf; use crate::type_config::alias::VoteOf; use crate::Instant; -use crate::LogId; use crate::RaftTypeConfig; use crate::StoredMembership; @@ -38,17 +38,17 @@ pub struct RaftMetrics { pub last_log_index: Option, /// The last log index has been applied to this Raft node's state machine. - pub last_applied: Option>, + pub last_applied: Option>, /// The id of the last log included in snapshot. /// If there is no snapshot, it is (0,0). - pub snapshot: Option>, + pub snapshot: Option>, /// The last log id that has purged from storage, inclusive. /// /// `purged` is also the first log id Openraft knows, although the corresponding log entry has /// already been deleted. - pub purged: Option>, + pub purged: Option>, // --- // --- cluster --- @@ -189,10 +189,10 @@ where C: RaftTypeConfig #[derive(Clone, Debug, Default, PartialEq, Eq)] #[cfg_attr(feature = "serde", derive(serde::Deserialize, serde::Serialize), serde(bound = ""))] pub struct RaftDataMetrics { - pub last_log: Option>, - pub last_applied: Option>, - pub snapshot: Option>, - pub purged: Option>, + pub last_log: Option>, + pub last_applied: Option>, + pub snapshot: Option>, + pub purged: Option>, /// For a leader, it is the elapsed time in milliseconds since the most recently acknowledged /// timestamp by a quorum. diff --git a/openraft/src/metrics/wait.rs b/openraft/src/metrics/wait.rs index 22a7163a3..db62e8657 100644 --- a/openraft/src/metrics/wait.rs +++ b/openraft/src/metrics/wait.rs @@ -8,10 +8,10 @@ use crate::core::ServerState; use crate::metrics::Condition; use crate::metrics::Metric; use crate::metrics::RaftMetrics; +use crate::type_config::alias::LogIdOf; 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; @@ -212,7 +212,7 @@ where C: RaftTypeConfig #[tracing::instrument(level = "trace", skip(self), fields(msg=msg.to_string().as_str()))] pub async fn snapshot( &self, - snapshot_last_log_id: LogId, + snapshot_last_log_id: LogIdOf, msg: impl ToString, ) -> Result, WaitError> { self.eq(Metric::Snapshot(Some(snapshot_last_log_id)), msg).await @@ -220,7 +220,7 @@ where C: RaftTypeConfig /// Wait for `purged` to become `want` or timeout. #[tracing::instrument(level = "trace", skip(self), fields(msg=msg.to_string().as_str()))] - pub async fn purged(&self, want: Option>, msg: impl ToString) -> Result, WaitError> { + pub async fn purged(&self, want: Option>, msg: impl ToString) -> Result, WaitError> { self.eq(Metric::Purged(want), msg).await } diff --git a/openraft/src/progress/entry/mod.rs b/openraft/src/progress/entry/mod.rs index bdefd983c..3b8f5b1fa 100644 --- a/openraft/src/progress/entry/mod.rs +++ b/openraft/src/progress/entry/mod.rs @@ -13,7 +13,7 @@ use crate::engine::EngineConfig; use crate::progress::entry::update::Updater; use crate::progress::inflight::Inflight; use crate::raft_state::LogStateReader; -use crate::LogId; +use crate::type_config::alias::LogIdOf; use crate::LogIdOptionExt; use crate::RaftTypeConfig; @@ -24,7 +24,7 @@ pub(crate) struct ProgressEntry where C: RaftTypeConfig { /// The id of the last matching log on the target following node. - pub(crate) matching: Option>, + pub(crate) matching: Option>, /// The data being transmitted in flight. /// @@ -48,7 +48,7 @@ impl ProgressEntry where C: RaftTypeConfig { #[allow(dead_code)] - pub(crate) fn new(matching: Option>) -> Self { + pub(crate) fn new(matching: Option>) -> Self { Self { matching: matching.clone(), inflight: Inflight::None, @@ -85,7 +85,7 @@ where C: RaftTypeConfig /// Return if a range of log id `..=log_id` is inflight sending. /// /// `prev_log_id` is never inflight. - pub(crate) fn is_log_range_inflight(&self, upto: &LogId) -> bool { + pub(crate) fn is_log_range_inflight(&self, upto: &LogIdOf) -> bool { match &self.inflight { Inflight::None => false, Inflight::Logs { log_id_range, .. } => { @@ -175,10 +175,10 @@ where C: RaftTypeConfig } } -impl Borrow>> for ProgressEntry +impl Borrow>> for ProgressEntry where C: RaftTypeConfig { - fn borrow(&self) -> &Option> { + fn borrow(&self) -> &Option> { &self.matching } } diff --git a/openraft/src/progress/entry/tests.rs b/openraft/src/progress/entry/tests.rs index d7e99384a..01257d0f9 100644 --- a/openraft/src/progress/entry/tests.rs +++ b/openraft/src/progress/entry/tests.rs @@ -6,10 +6,11 @@ use crate::progress::entry::ProgressEntry; use crate::progress::inflight::Inflight; use crate::raft_state::LogStateReader; use crate::type_config::alias::LeaderIdOf; +use crate::type_config::alias::LogIdOf; use crate::vote::RaftLeaderIdExt; use crate::LogId; -fn log_id(index: u64) -> LogId { +fn log_id(index: u64) -> LogIdOf { LogId { leader_id: LeaderIdOf::::new_committed(1, 1), index, @@ -88,10 +89,10 @@ fn test_update_conflicting() -> anyhow::Result<()> { /// LogStateReader impl for testing struct LogState { - last: Option>, - snap_last: Option>, - purge_upto: Option>, - purged: Option>, + last: Option>, + snap_last: Option>, + purge_upto: Option>, + purged: Option>, } impl LogState { @@ -108,7 +109,7 @@ impl LogState { } impl LogStateReader for LogState { - fn get_log_id(&self, index: u64) -> Option> { + fn get_log_id(&self, index: u64) -> Option> { let x = Some(log_id(index)); if x >= self.purged && x <= self.last { x @@ -117,35 +118,35 @@ impl LogStateReader for LogState { } } - fn last_log_id(&self) -> Option<&LogId> { + fn last_log_id(&self) -> Option<&LogIdOf> { self.last.as_ref() } - fn committed(&self) -> Option<&LogId> { + fn committed(&self) -> Option<&LogIdOf> { unimplemented!("testing") } - fn io_applied(&self) -> Option<&LogId> { + fn io_applied(&self) -> Option<&LogIdOf> { todo!() } - fn io_snapshot_last_log_id(&self) -> Option<&LogId> { + fn io_snapshot_last_log_id(&self) -> Option<&LogIdOf> { todo!() } - fn io_purged(&self) -> Option<&LogId> { + fn io_purged(&self) -> Option<&LogIdOf> { todo!() } - fn snapshot_last_log_id(&self) -> Option<&LogId> { + fn snapshot_last_log_id(&self) -> Option<&LogIdOf> { self.snap_last.as_ref() } - fn purge_upto(&self) -> Option<&LogId> { + fn purge_upto(&self) -> Option<&LogIdOf> { self.purge_upto.as_ref() } - fn last_purged_log_id(&self) -> Option<&LogId> { + fn last_purged_log_id(&self) -> Option<&LogIdOf> { self.purged.as_ref() } } diff --git a/openraft/src/progress/entry/update.rs b/openraft/src/progress/entry/update.rs index 2ea98b618..546151854 100644 --- a/openraft/src/progress/entry/update.rs +++ b/openraft/src/progress/entry/update.rs @@ -1,7 +1,7 @@ use crate::display_ext::DisplayOptionExt; use crate::engine::EngineConfig; use crate::progress::entry::ProgressEntry; -use crate::LogId; +use crate::type_config::alias::LogIdOf; use crate::LogIdOptionExt; use crate::RaftTypeConfig; @@ -76,7 +76,7 @@ where C: RaftTypeConfig } } - pub(crate) fn update_matching(&mut self, matching: Option>) { + pub(crate) fn update_matching(&mut self, matching: Option>) { tracing::debug!( "update_matching: current progress_entry: {}; matching: {}", self.entry, diff --git a/openraft/src/progress/inflight/mod.rs b/openraft/src/progress/inflight/mod.rs index aca65f7d4..363137581 100644 --- a/openraft/src/progress/inflight/mod.rs +++ b/openraft/src/progress/inflight/mod.rs @@ -10,7 +10,7 @@ use validit::Validate; use crate::display_ext::DisplayOptionExt; use crate::log_id_range::LogIdRange; use crate::type_config::alias::CommittedLeaderIdOf; -use crate::LogId; +use crate::type_config::alias::LogIdOf; use crate::LogIdOptionExt; use crate::RaftTypeConfig; @@ -35,7 +35,7 @@ where C: RaftTypeConfig /// The last log id snapshot includes. /// /// It is None, if the snapshot is empty. - last_log_id: Option>, + last_log_id: Option>, }, } @@ -76,7 +76,7 @@ impl Inflight where C: RaftTypeConfig { /// Create inflight state for sending logs. - pub(crate) fn logs(prev: Option>, last: Option>) -> Self { + pub(crate) fn logs(prev: Option>, last: Option>) -> Self { #![allow(clippy::nonminimal_bool)] if !(prev < last) { Self::None @@ -88,7 +88,7 @@ where C: RaftTypeConfig } /// Create inflight state for sending snapshot. - pub(crate) fn snapshot(snapshot_last_log_id: Option>) -> Self { + pub(crate) fn snapshot(snapshot_last_log_id: Option>) -> Self { Self::Snapshot { last_log_id: snapshot_last_log_id, } @@ -111,7 +111,7 @@ where C: RaftTypeConfig } /// Update inflight state when log upto `upto` is acknowledged by a follower/learner. - pub(crate) fn ack(&mut self, upto: Option>) { + pub(crate) fn ack(&mut self, upto: Option>) { match self { Inflight::None => { unreachable!("no inflight data") diff --git a/openraft/src/progress/inflight/tests.rs b/openraft/src/progress/inflight/tests.rs index 04ffe982a..61409f1ea 100644 --- a/openraft/src/progress/inflight/tests.rs +++ b/openraft/src/progress/inflight/tests.rs @@ -4,10 +4,11 @@ use crate::engine::testing::UTConfig; use crate::log_id_range::LogIdRange; use crate::progress::Inflight; use crate::type_config::alias::LeaderIdOf; +use crate::type_config::alias::LogIdOf; use crate::vote::RaftLeaderIdExt; use crate::LogId; -fn log_id(index: u64) -> LogId { +fn log_id(index: u64) -> LogIdOf { LogId { leader_id: LeaderIdOf::::new_committed(1, 1), index, diff --git a/openraft/src/proposer/candidate.rs b/openraft/src/proposer/candidate.rs index 493ad6b81..adb497af9 100644 --- a/openraft/src/proposer/candidate.rs +++ b/openraft/src/proposer/candidate.rs @@ -12,7 +12,6 @@ use crate::type_config::alias::LogIdOf; use crate::type_config::alias::VoteOf; use crate::vote::raft_vote::RaftVoteExt; use crate::vote::RaftVote; -use crate::LogId; use crate::RaftTypeConfig; /// Candidate: voting state. @@ -82,7 +81,7 @@ where &self.vote } - pub(crate) fn last_log_id(&self) -> Option<&LogId> { + pub(crate) fn last_log_id(&self) -> Option<&LogIdOf> { self.last_log_id.as_ref() } diff --git a/openraft/src/proposer/leader.rs b/openraft/src/proposer/leader.rs index 886316206..a503dd9d0 100644 --- a/openraft/src/proposer/leader.rs +++ b/openraft/src/proposer/leader.rs @@ -11,7 +11,6 @@ use crate::type_config::alias::LogIdOf; use crate::type_config::TypeConfigExt; use crate::vote::committed::CommittedVote; use crate::vote::raft_vote::RaftVoteExt; -use crate::LogId; use crate::LogIdOptionExt; use crate::RaftLogId; use crate::RaftTypeConfig; @@ -113,7 +112,7 @@ where first.cloned() } else { // Set to a log id that will be proposed. - Some(LogId::new( + Some(LogIdOf::::new( vote.committed_leader_id(), last_leader_log_id.last().next_index(), )) @@ -171,7 +170,7 @@ where let committed_leader_id = self.committed_vote.committed_leader_id(); - let first = LogId::new(committed_leader_id, self.last_log_id().next_index()); + let first = LogIdOf::::new(committed_leader_id, self.last_log_id().next_index()); let mut last = first.clone(); for entry in entries { diff --git a/openraft/src/raft/message/append_entries.rs b/openraft/src/raft/message/append_entries.rs index a8c582b76..06b7bf9e5 100644 --- a/openraft/src/raft/message/append_entries.rs +++ b/openraft/src/raft/message/append_entries.rs @@ -2,8 +2,8 @@ use std::fmt; use crate::display_ext::DisplayOptionExt; use crate::display_ext::DisplaySlice; +use crate::type_config::alias::LogIdOf; use crate::type_config::alias::VoteOf; -use crate::LogId; use crate::RaftTypeConfig; /// An RPC sent by a cluster leader to replicate log entries (§5.3), and as a heartbeat (§5.2). @@ -18,7 +18,7 @@ use crate::RaftTypeConfig; pub struct AppendEntriesRequest { pub vote: VoteOf, - pub prev_log_id: Option>, + pub prev_log_id: Option>, /// The new log entries to store. /// @@ -27,7 +27,7 @@ pub struct AppendEntriesRequest { pub entries: Vec, /// The leader's committed log id. - pub leader_commit: Option>, + pub leader_commit: Option>, } impl fmt::Debug for AppendEntriesRequest { @@ -86,7 +86,7 @@ pub enum AppendEntriesResponse { /// /// [`RPCError`]: crate::error::RPCError /// [`RaftNetwork::append_entries`]: crate::network::RaftNetwork::append_entries - PartialSuccess(Option>), + PartialSuccess(Option>), /// The first log id([`AppendEntriesRequest::prev_log_id`]) of the entries to send does not /// match on the remote target node. diff --git a/openraft/src/raft/message/client_write.rs b/openraft/src/raft/message/client_write.rs index 8ff2adad6..5eb7d457c 100644 --- a/openraft/src/raft/message/client_write.rs +++ b/openraft/src/raft/message/client_write.rs @@ -5,7 +5,7 @@ use openraft_macros::since; use crate::display_ext::DisplayOptionExt; use crate::error::ClientWriteError; -use crate::LogId; +use crate::type_config::alias::LogIdOf; use crate::Membership; use crate::RaftTypeConfig; @@ -20,7 +20,7 @@ pub type ClientWriteResult = Result, ClientWriteError< )] pub struct ClientWriteResponse { /// The id of the log that is applied. - pub log_id: LogId, + pub log_id: LogIdOf, /// Application specific response data. pub data: C::R, @@ -35,7 +35,7 @@ where C: RaftTypeConfig /// Create a new instance of `ClientWriteResponse`. #[allow(dead_code)] #[since(version = "0.9.5")] - pub(crate) fn new_app_response(log_id: LogId, data: C::R) -> Self { + pub(crate) fn new_app_response(log_id: LogIdOf, data: C::R) -> Self { Self { log_id, data, @@ -44,7 +44,7 @@ where C: RaftTypeConfig } #[since(version = "0.9.5")] - pub fn log_id(&self) -> &LogId { + pub fn log_id(&self) -> &LogIdOf { &self.log_id } diff --git a/openraft/src/raft/message/transfer_leader.rs b/openraft/src/raft/message/transfer_leader.rs index 9cc135df1..5bbc75d8d 100644 --- a/openraft/src/raft/message/transfer_leader.rs +++ b/openraft/src/raft/message/transfer_leader.rs @@ -1,8 +1,8 @@ use std::fmt; use crate::display_ext::DisplayOptionExt; +use crate::type_config::alias::LogIdOf; use crate::type_config::alias::VoteOf; -use crate::LogId; use crate::RaftTypeConfig; #[derive(Clone, Debug)] @@ -18,13 +18,13 @@ where C: RaftTypeConfig pub(crate) to_node_id: C::NodeId, /// The last log id the `to_node_id` node should at least have to become Leader. - pub(crate) last_log_id: Option>, + pub(crate) last_log_id: Option>, } impl TransferLeaderRequest where C: RaftTypeConfig { - pub fn new(from: VoteOf, 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, @@ -45,7 +45,7 @@ where C: RaftTypeConfig /// The last log id on the `to_node_id` node should at least have to become Leader. /// /// This is the last log id on the Leader when the leadership is transferred. - pub fn last_log_id(&self) -> Option<&LogId> { + pub fn last_log_id(&self) -> Option<&LogIdOf> { self.last_log_id.as_ref() } } diff --git a/openraft/src/raft/message/vote.rs b/openraft/src/raft/message/vote.rs index e83f6b90e..58bd4c06f 100644 --- a/openraft/src/raft/message/vote.rs +++ b/openraft/src/raft/message/vote.rs @@ -2,8 +2,8 @@ use std::borrow::Borrow; use std::fmt; use crate::display_ext::DisplayOptionExt; +use crate::type_config::alias::LogIdOf; use crate::type_config::alias::VoteOf; -use crate::LogId; use crate::RaftTypeConfig; /// An RPC sent by candidates to gather votes (§5.2). @@ -11,7 +11,7 @@ use crate::RaftTypeConfig; #[cfg_attr(feature = "serde", derive(serde::Deserialize, serde::Serialize), serde(bound = ""))] pub struct VoteRequest { pub vote: VoteOf, - pub last_log_id: Option>, + pub last_log_id: Option>, } impl fmt::Display for VoteRequest @@ -25,7 +25,7 @@ where C: RaftTypeConfig impl VoteRequest where C: RaftTypeConfig { - pub fn new(vote: VoteOf, last_log_id: Option>) -> Self { + pub fn new(vote: VoteOf, last_log_id: Option>) -> Self { Self { vote, last_log_id } } } @@ -45,13 +45,13 @@ pub struct VoteResponse { pub vote_granted: bool, /// The last log id stored on the remote voter. - pub last_log_id: Option>, + pub last_log_id: Option>, } 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, diff --git a/openraft/src/raft/mod.rs b/openraft/src/raft/mod.rs index b866c7bc0..19d03ab43 100644 --- a/openraft/src/raft/mod.rs +++ b/openraft/src/raft/mod.rs @@ -84,13 +84,13 @@ use crate::storage::RaftLogStorage; use crate::storage::RaftStateMachine; use crate::storage::Snapshot; use crate::type_config::alias::JoinErrorOf; +use crate::type_config::alias::LogIdOf; 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; use crate::LogIdOptionExt; use crate::LogIndexOptionExt; use crate::OptionalSend; @@ -542,7 +542,7 @@ where C: RaftTypeConfig /// ``` /// Read more about how it works: [Read Operation](crate::docs::protocol::read) #[tracing::instrument(level = "debug", skip(self))] - pub async fn ensure_linearizable(&self) -> Result>, RaftError>> { + pub async fn ensure_linearizable(&self) -> Result>, RaftError>> { let (read_log_id, applied) = self.get_read_log_id().await?; if read_log_id.index() > applied.index() { @@ -591,7 +591,7 @@ where C: RaftTypeConfig #[tracing::instrument(level = "debug", skip(self))] pub async fn get_read_log_id( &self, - ) -> Result<(Option>, Option>), RaftError>> { + ) -> Result<(Option>, Option>), RaftError>> { let (tx, rx) = C::oneshot(); let (read_log_id, applied) = self.inner.call_core(RaftMsg::CheckIsLeaderRequest { tx }, rx).await?; Ok((read_log_id, applied)) @@ -782,8 +782,8 @@ where C: RaftTypeConfig &self, metrics: &RaftMetrics, node_id: &C::NodeId, - membership_log_id: Option<&LogId>, - ) -> Result>, ()> { + membership_log_id: Option<&LogIdOf>, + ) -> Result>, ()> { if metrics.membership_config.log_id().as_ref() < membership_log_id { // Waiting for the latest metrics to report. return Err(()); diff --git a/openraft/src/raft_state/io_state.rs b/openraft/src/raft_state/io_state.rs index d08623a9d..83e1dd3a9 100644 --- a/openraft/src/raft_state/io_state.rs +++ b/openraft/src/raft_state/io_state.rs @@ -7,8 +7,8 @@ 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::LogIdOf; use crate::type_config::alias::VoteOf; -use crate::LogId; use crate::RaftTypeConfig; pub(crate) mod io_id; @@ -69,17 +69,17 @@ where C: RaftTypeConfig pub(crate) io_progress: Valid>>, /// The last log id that has been applied to state machine. - pub(crate) applied: Option>, + pub(crate) applied: Option>, /// The last log id in the currently persisted snapshot. - pub(crate) snapshot: Option>, + pub(crate) snapshot: Option>, /// The last log id that has been purged from storage. /// /// `RaftState::last_purged_log_id()` /// is just the log id that is going to be purged, i.e., there is a `PurgeLog` command queued to /// be executed, and it may not be the actually purged log id. - pub(crate) purged: Option>, + pub(crate) purged: Option>, } impl Validate for IOState @@ -104,9 +104,9 @@ where C: RaftTypeConfig { pub(crate) fn new( vote: &VoteOf, - applied: Option>, - snapshot: Option>, - purged: Option>, + applied: Option>, + snapshot: Option>, + purged: Option>, ) -> Self { let mut io_progress = Valid::new(IOProgress::default()); @@ -123,7 +123,7 @@ where C: RaftTypeConfig } } - pub(crate) fn update_applied(&mut self, log_id: Option>) { + pub(crate) fn update_applied(&mut self, log_id: Option>) { tracing::debug!(applied = display(DisplayOption(&log_id)), "{}", func_name!()); // TODO: should we update flushed if applied is newer? @@ -137,11 +137,11 @@ where C: RaftTypeConfig self.applied = log_id; } - pub(crate) fn applied(&self) -> Option<&LogId> { + pub(crate) fn applied(&self) -> Option<&LogIdOf> { self.applied.as_ref() } - pub(crate) fn update_snapshot(&mut self, log_id: Option>) { + pub(crate) fn update_snapshot(&mut self, log_id: Option>) { tracing::debug!(snapshot = display(DisplayOption(&log_id)), "{}", func_name!()); debug_assert!( @@ -154,7 +154,7 @@ where C: RaftTypeConfig self.snapshot = log_id; } - pub(crate) fn snapshot(&self) -> Option<&LogId> { + pub(crate) fn snapshot(&self) -> Option<&LogIdOf> { self.snapshot.as_ref() } @@ -166,11 +166,11 @@ where C: RaftTypeConfig self.building_snapshot } - pub(crate) fn update_purged(&mut self, log_id: Option>) { + pub(crate) fn update_purged(&mut self, log_id: Option>) { self.purged = log_id; } - pub(crate) fn purged(&self) -> Option<&LogId> { + pub(crate) fn purged(&self) -> Option<&LogIdOf> { self.purged.as_ref() } } diff --git a/openraft/src/raft_state/io_state/io_id.rs b/openraft/src/raft_state/io_state/io_id.rs index d2fa43c96..7a8914655 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::LogIdOf; use crate::type_config::alias::VoteOf; use crate::vote::committed::CommittedVote; use crate::vote::non_committed::NonCommittedVote; @@ -10,7 +11,6 @@ use crate::vote::ref_vote::RefVote; use crate::vote::RaftVote; use crate::ErrorSubject; use crate::ErrorVerb; -use crate::LogId; use crate::RaftTypeConfig; /// An ID to uniquely identify a monotonic increasing io operation to [`RaftLogStorage`]. @@ -82,7 +82,7 @@ where C: RaftTypeConfig Self::Vote(vote) } - pub(crate) fn new_log_io(committed_vote: CommittedVote, last_log_id: Option>) -> Self { + pub(crate) fn new_log_io(committed_vote: CommittedVote, last_log_id: Option>) -> Self { Self::Log(LogIOId::new(committed_vote, last_log_id)) } @@ -103,7 +103,7 @@ where C: RaftTypeConfig } } - pub(crate) fn last_log_id(&self) -> Option<&LogId> { + pub(crate) fn last_log_id(&self) -> Option<&LogIdOf> { match self { Self::Vote(_) => None, Self::Log(log_io_id) => log_io_id.log_id.as_ref(), diff --git a/openraft/src/raft_state/io_state/log_io_id.rs b/openraft/src/raft_state/io_state/log_io_id.rs index 66b1159ba..2a801857f 100644 --- a/openraft/src/raft_state/io_state/log_io_id.rs +++ b/openraft/src/raft_state/io_state/log_io_id.rs @@ -1,8 +1,8 @@ use std::fmt; use crate::display_ext::DisplayOptionExt; +use crate::type_config::alias::LogIdOf; use crate::vote::committed::CommittedVote; -use crate::LogId; use crate::RaftTypeConfig; /// A monotonic increasing id for log append io operation. @@ -30,7 +30,7 @@ where C: RaftTypeConfig pub(crate) committed_vote: CommittedVote, /// The last log id that has been flushed to storage. - pub(crate) log_id: Option>, + pub(crate) log_id: Option>, } impl fmt::Display for LogIOId @@ -44,7 +44,7 @@ where C: RaftTypeConfig impl LogIOId where C: RaftTypeConfig { - pub(crate) fn new(committed_vote: CommittedVote, log_id: Option>) -> Self { + pub(crate) fn new(committed_vote: CommittedVote, log_id: Option>) -> Self { Self { committed_vote, log_id } } } diff --git a/openraft/src/raft_state/log_state_reader.rs b/openraft/src/raft_state/log_state_reader.rs index ec9e28558..e44957a7c 100644 --- a/openraft/src/raft_state/log_state_reader.rs +++ b/openraft/src/raft_state/log_state_reader.rs @@ -1,4 +1,4 @@ -use crate::LogId; +use crate::type_config::alias::LogIdOf; use crate::LogIdOptionExt; use crate::RaftTypeConfig; @@ -9,7 +9,7 @@ pub(crate) trait LogStateReader where C: RaftTypeConfig { /// Get previous log id, i.e., the log id at index - 1 - fn prev_log_id(&self, index: u64) -> Option> { + fn prev_log_id(&self, index: u64) -> Option> { if index == 0 { None } else { @@ -20,7 +20,7 @@ where C: RaftTypeConfig /// Return if a log id exists. /// /// It assumes a committed log will always get positive return value, according to raft spec. - fn has_log_id(&self, log_id: &LogId) -> bool { + fn has_log_id(&self, log_id: &LogIdOf) -> bool { if log_id.index < self.committed().next_index() { debug_assert!(Some(log_id) <= self.committed()); return true; @@ -39,40 +39,40 @@ where C: RaftTypeConfig /// It will return `last_purged_log_id` if index is at the last purged index. /// If the log at the specified index is smaller than `last_purged_log_id`, or greater than /// `last_log_id`, it returns None. - fn get_log_id(&self, index: u64) -> Option>; + fn get_log_id(&self, index: u64) -> Option>; /// The last known log id in the store. /// /// The range of all stored log ids are `(last_purged_log_id(), last_log_id()]`, left open right /// close. - fn last_log_id(&self) -> Option<&LogId>; + fn last_log_id(&self) -> Option<&LogIdOf>; /// The last known committed log id, i.e., the id of the log that is accepted by a quorum of /// voters. - fn committed(&self) -> Option<&LogId>; + fn committed(&self) -> Option<&LogIdOf>; /// The last known applied log id, i.e., the id of the log that is applied to state machine. /// /// This is actually happened io-state which might fall behind committed log id. - fn io_applied(&self) -> Option<&LogId>; + fn io_applied(&self) -> Option<&LogIdOf>; /// The last log id in the last persisted snapshot. /// /// This is actually happened io-state which might fall behind `Self::snapshot_last_log_id()`. - fn io_snapshot_last_log_id(&self) -> Option<&LogId>; + fn io_snapshot_last_log_id(&self) -> Option<&LogIdOf>; /// The last known purged log id, inclusive. /// /// This is actually purged log id from storage. - fn io_purged(&self) -> Option<&LogId>; + fn io_purged(&self) -> Option<&LogIdOf>; /// Return the last log id the snapshot includes. - fn snapshot_last_log_id(&self) -> Option<&LogId>; + fn snapshot_last_log_id(&self) -> Option<&LogIdOf>; /// Return the log id it wants to purge up to. /// /// Logs may not be able to be purged at once because they are in use by replication tasks. - fn purge_upto(&self) -> Option<&LogId>; + fn purge_upto(&self) -> Option<&LogIdOf>; /// The greatest log id that has been purged after being applied to state machine, i.e., the /// oldest known log id. @@ -81,5 +81,5 @@ where C: RaftTypeConfig /// left open and right close. /// /// `last_purged_log_id == last_log_id` means there is no log entry in the storage. - fn last_purged_log_id(&self) -> Option<&LogId>; + fn last_purged_log_id(&self) -> Option<&LogIdOf>; } diff --git a/openraft/src/raft_state/membership_state/mod.rs b/openraft/src/raft_state/membership_state/mod.rs index e0e0050b7..9b104e239 100644 --- a/openraft/src/raft_state/membership_state/mod.rs +++ b/openraft/src/raft_state/membership_state/mod.rs @@ -5,7 +5,6 @@ use std::sync::Arc; use validit::Validate; use crate::EffectiveMembership; -use crate::LogId; use crate::LogIdOptionExt; use crate::RaftTypeConfig; @@ -17,6 +16,8 @@ mod membership_state_test; pub(crate) use change_handler::ChangeHandler; +use crate::type_config::alias::LogIdOf; + /// The state of membership configs a raft node needs to know. /// /// A raft node needs to store at most 2 membership config log: @@ -80,7 +81,7 @@ where C: RaftTypeConfig } /// Update membership state if the specified committed_log_id is greater than `self.effective` - pub(crate) fn commit(&mut self, committed_log_id: &Option>) { + pub(crate) fn commit(&mut self, committed_log_id: &Option>) { if committed_log_id >= self.effective().log_id() { debug_assert!(committed_log_id.index() >= self.effective().log_id().index()); self.committed = self.effective.clone(); diff --git a/openraft/src/raft_state/mod.rs b/openraft/src/raft_state/mod.rs index 05346f754..ba485a79b 100644 --- a/openraft/src/raft_state/mod.rs +++ b/openraft/src/raft_state/mod.rs @@ -9,7 +9,6 @@ use crate::error::ForwardToLeader; use crate::log_id::RaftLogId; use crate::storage::SnapshotMeta; use crate::utime::Leased; -use crate::LogId; use crate::LogIdOptionExt; use crate::RaftTypeConfig; use crate::ServerState; @@ -61,7 +60,7 @@ where C: RaftTypeConfig /// of the leader. /// /// - A quorum could be a uniform quorum or joint quorum. - pub committed: Option>, + pub committed: Option>, pub(crate) purged_next: u64, @@ -86,7 +85,7 @@ where C: RaftTypeConfig /// /// If a log is in use by a replication task, the purge is postponed and is stored in this /// field. - pub(crate) purge_upto: Option>, + pub(crate) purge_upto: Option>, } impl Default for RaftState @@ -110,39 +109,39 @@ where C: RaftTypeConfig impl LogStateReader for RaftState where C: RaftTypeConfig { - fn get_log_id(&self, index: u64) -> Option> { + fn get_log_id(&self, index: u64) -> Option> { self.log_ids.get(index) } - fn last_log_id(&self) -> Option<&LogId> { + fn last_log_id(&self) -> Option<&LogIdOf> { self.log_ids.last() } - fn committed(&self) -> Option<&LogId> { + fn committed(&self) -> Option<&LogIdOf> { self.committed.as_ref() } - fn io_applied(&self) -> Option<&LogId> { + fn io_applied(&self) -> Option<&LogIdOf> { self.io_state.applied() } - fn io_snapshot_last_log_id(&self) -> Option<&LogId> { + fn io_snapshot_last_log_id(&self) -> Option<&LogIdOf> { self.io_state.snapshot() } - fn io_purged(&self) -> Option<&LogId> { + fn io_purged(&self) -> Option<&LogIdOf> { self.io_state.purged() } - fn snapshot_last_log_id(&self) -> Option<&LogId> { + fn snapshot_last_log_id(&self) -> Option<&LogIdOf> { self.snapshot_meta.last_log_id.as_ref() } - fn purge_upto(&self) -> Option<&LogId> { + fn purge_upto(&self) -> Option<&LogIdOf> { self.purge_upto.as_ref() } - fn last_purged_log_id(&self) -> Option<&LogId> { + fn last_purged_log_id(&self) -> Option<&LogIdOf> { if self.purged_next == 0 { return None; } diff --git a/openraft/src/raft_state/tests/log_state_reader_test.rs b/openraft/src/raft_state/tests/log_state_reader_test.rs index 6fc9675fc..7a239684f 100644 --- a/openraft/src/raft_state/tests/log_state_reader_test.rs +++ b/openraft/src/raft_state/tests/log_state_reader_test.rs @@ -2,10 +2,10 @@ use crate::engine::testing::UTConfig; use crate::engine::LogIdList; use crate::raft_state::LogStateReader; use crate::testing; -use crate::LogId; +use crate::type_config::alias::LogIdOf; use crate::RaftState; -fn log_id(term: u64, index: u64) -> LogId { +fn log_id(term: u64, index: u64) -> LogIdOf { testing::log_id(term, 0, index) } diff --git a/openraft/src/raft_state/tests/validate_test.rs b/openraft/src/raft_state/tests/validate_test.rs index c1b83a7c6..c6993090f 100644 --- a/openraft/src/raft_state/tests/validate_test.rs +++ b/openraft/src/raft_state/tests/validate_test.rs @@ -4,10 +4,10 @@ use crate::engine::testing::UTConfig; use crate::engine::LogIdList; use crate::storage::SnapshotMeta; use crate::testing; -use crate::LogId; +use crate::type_config::alias::LogIdOf; use crate::RaftState; -fn log_id(term: u64, index: u64) -> LogId { +fn log_id(term: u64, index: u64) -> LogIdOf { testing::log_id(term, 0, index) } diff --git a/openraft/src/replication/mod.rs b/openraft/src/replication/mod.rs index 55b90f23c..5ea47efae 100644 --- a/openraft/src/replication/mod.rs +++ b/openraft/src/replication/mod.rs @@ -59,7 +59,6 @@ use crate::type_config::alias::VoteOf; use crate::type_config::async_runtime::mutex::Mutex; use crate::type_config::TypeConfigExt; use crate::vote::raft_vote::RaftVoteExt; -use crate::LogId; use crate::RaftLogId; use crate::RaftNetworkFactory; use crate::RaftTypeConfig; @@ -136,10 +135,10 @@ where config: Arc, /// The log id of the highest log entry which is known to be committed in the cluster. - committed: Option>, + committed: Option>, /// Last matching log id on a follower/learner - matching: Option>, + matching: Option>, /// Next replication action to run. next_action: Option>, @@ -163,8 +162,8 @@ where target: C::NodeId, session_id: ReplicationSessionId, config: Arc, - committed: Option>, - matching: Option>, + committed: Option>, + matching: Option>, network: N::Network, snapshot_network: N::Network, log_reader: LS::LogReader, @@ -818,7 +817,7 @@ where } /// If there are more logs to send, it returns a new `Some(Data::Logs)` to send. - fn next_action_to_send(&mut self, matching: Option>, log_ids: LogIdRange) -> Option> { + fn next_action_to_send(&mut self, matching: Option>, log_ids: LogIdRange) -> Option> { if matching < log_ids.last { Some(Data::new_logs(LogIdRange::new(matching, log_ids.last))) } else { @@ -827,7 +826,7 @@ where } /// Check if partial success result(`matching`) is valid for a given log range to send. - fn debug_assert_partial_success(to_send: &LogIdRange, matching: &Option>) { + fn debug_assert_partial_success(to_send: &LogIdRange, matching: &Option>) { debug_assert!( matching <= &to_send.last, "matching ({}) should be <= last_log_id ({})", diff --git a/openraft/src/replication/replication_session_id.rs b/openraft/src/replication/replication_session_id.rs index 439d6b0f0..5bac54662 100644 --- a/openraft/src/replication/replication_session_id.rs +++ b/openraft/src/replication/replication_session_id.rs @@ -2,9 +2,9 @@ use std::fmt::Display; use std::fmt::Formatter; use crate::display_ext::DisplayOptionExt; +use crate::type_config::alias::LogIdOf; use crate::type_config::alias::VoteOf; use crate::vote::committed::CommittedVote; -use crate::LogId; use crate::RaftTypeConfig; /// Uniquely identifies a replication session. @@ -38,7 +38,7 @@ where C: RaftTypeConfig pub(crate) leader_vote: CommittedVote, /// The log id of the membership log this replication works for. - pub(crate) membership_log_id: Option>, + pub(crate) membership_log_id: Option>, } impl Display for ReplicationSessionId @@ -57,7 +57,7 @@ where C: RaftTypeConfig impl ReplicationSessionId where C: RaftTypeConfig { - pub(crate) fn new(vote: CommittedVote, membership_log_id: Option>) -> Self { + pub(crate) fn new(vote: CommittedVote, membership_log_id: Option>) -> Self { Self { leader_vote: vote, membership_log_id, diff --git a/openraft/src/replication/request.rs b/openraft/src/replication/request.rs index 6212381b9..0b39b3768 100644 --- a/openraft/src/replication/request.rs +++ b/openraft/src/replication/request.rs @@ -9,7 +9,7 @@ pub(crate) enum Replicate where C: RaftTypeConfig { /// Inform replication stream to forward the committed log id to followers/learners. - Committed(Option>), + Committed(Option>), /// Send a chunk of data, e.g., logs or snapshot. Data(Data), @@ -47,7 +47,6 @@ use crate::raft::SnapshotResponse; use crate::replication::callbacks::SnapshotCallback; use crate::storage::SnapshotMeta; use crate::type_config::alias::InstantOf; -use crate::LogId; use crate::RaftTypeConfig; /// Request to replicate a chunk of data, logs or snapshot. diff --git a/openraft/src/storage/callback.rs b/openraft/src/storage/callback.rs index 793bb7b15..6599b6ed0 100644 --- a/openraft/src/storage/callback.rs +++ b/openraft/src/storage/callback.rs @@ -5,12 +5,12 @@ use std::io; use crate::async_runtime::MpscUnboundedSender; use crate::async_runtime::MpscUnboundedWeakSender; use crate::core::notification::Notification; +use crate::type_config::alias::LogIdOf; use crate::type_config::alias::MpscUnboundedWeakSenderOf; use crate::type_config::alias::OneshotSenderOf; use crate::type_config::async_runtime::oneshot::OneshotSender; use crate::ErrorSubject; use crate::ErrorVerb; -use crate::LogId; use crate::RaftTypeConfig; use crate::StorageError; @@ -101,8 +101,8 @@ where C: RaftTypeConfig pub struct LogApplied where C: RaftTypeConfig { - last_log_id: LogId, - tx: OneshotSenderOf, Vec), StorageError>>, + last_log_id: LogIdOf, + tx: OneshotSenderOf, Vec), StorageError>>, } impl LogApplied @@ -110,8 +110,8 @@ where C: RaftTypeConfig { #[allow(dead_code)] pub(crate) fn new( - last_log_id: LogId, - tx: OneshotSenderOf, Vec), StorageError>>, + last_log_id: LogIdOf, + tx: OneshotSenderOf, Vec), StorageError>>, ) -> Self { Self { last_log_id, tx } } diff --git a/openraft/src/storage/log_reader_ext.rs b/openraft/src/storage/log_reader_ext.rs index f355591fa..9bc9645e1 100644 --- a/openraft/src/storage/log_reader_ext.rs +++ b/openraft/src/storage/log_reader_ext.rs @@ -1,7 +1,7 @@ use anyerror::AnyError; use openraft_macros::add_async_trait; -use crate::LogId; +use crate::type_config::alias::LogIdOf; use crate::RaftLogId; use crate::RaftLogReader; use crate::RaftTypeConfig; @@ -20,7 +20,7 @@ where C: RaftTypeConfig } /// Get the log id of the entry at `index`. - async fn get_log_id(&mut self, log_index: u64) -> Result, StorageError> { + async fn get_log_id(&mut self, log_index: u64) -> Result, StorageError> { let entries = self.try_get_log_entries(log_index..=log_index).await?; if entries.is_empty() { diff --git a/openraft/src/storage/log_state.rs b/openraft/src/storage/log_state.rs index 9b456e11c..0c36df879 100644 --- a/openraft/src/storage/log_state.rs +++ b/openraft/src/storage/log_state.rs @@ -1,4 +1,4 @@ -use crate::LogId; +use crate::type_config::alias::LogIdOf; use crate::RaftTypeConfig; /// The state about logs. @@ -7,9 +7,9 @@ use crate::RaftTypeConfig; #[derive(Clone, Debug, Default, PartialEq, Eq)] pub struct LogState { /// The greatest log id that has been purged after being applied to state machine. - pub last_purged_log_id: Option>, + pub last_purged_log_id: Option>, /// The log id of the last present entry if there are any entries. /// Otherwise the same value as `last_purged_log_id`. - pub last_log_id: Option>, + pub last_log_id: Option>, } diff --git a/openraft/src/storage/snapshot_meta.rs b/openraft/src/storage/snapshot_meta.rs index 6eedf3c58..006a909d9 100644 --- a/openraft/src/storage/snapshot_meta.rs +++ b/openraft/src/storage/snapshot_meta.rs @@ -2,7 +2,7 @@ use std::fmt; use crate::display_ext::DisplayOption; use crate::storage::SnapshotSignature; -use crate::LogId; +use crate::type_config::alias::LogIdOf; use crate::RaftTypeConfig; use crate::SnapshotId; use crate::StoredMembership; @@ -18,7 +18,7 @@ pub struct SnapshotMeta where C: RaftTypeConfig { /// Log entries upto which this snapshot includes, inclusive. - pub last_log_id: Option>, + pub last_log_id: Option>, /// The last applied membership config. pub last_membership: StoredMembership, @@ -55,7 +55,7 @@ where C: RaftTypeConfig } /// Returns a ref to the id of the last log that is included in this snapshot. - pub fn last_log_id(&self) -> Option<&LogId> { + pub fn last_log_id(&self) -> Option<&LogIdOf> { self.last_log_id.as_ref() } } diff --git a/openraft/src/storage/snapshot_signature.rs b/openraft/src/storage/snapshot_signature.rs index b618136b6..6db95721e 100644 --- a/openraft/src/storage/snapshot_signature.rs +++ b/openraft/src/storage/snapshot_signature.rs @@ -1,4 +1,4 @@ -use crate::LogId; +use crate::type_config::alias::LogIdOf; use crate::RaftTypeConfig; use crate::SnapshotId; @@ -9,10 +9,10 @@ pub struct SnapshotSignature where C: RaftTypeConfig { /// Log entries upto which this snapshot includes, inclusive. - pub last_log_id: Option>, + pub last_log_id: Option>, /// The last applied membership log id. - pub last_membership_log_id: Option>, + pub last_membership_log_id: Option>, /// To identify a snapshot when transferring. pub snapshot_id: SnapshotId, diff --git a/openraft/src/storage/v2/raft_log_reader.rs b/openraft/src/storage/v2/raft_log_reader.rs index 20b49705b..0d960999e 100644 --- a/openraft/src/storage/v2/raft_log_reader.rs +++ b/openraft/src/storage/v2/raft_log_reader.rs @@ -6,8 +6,8 @@ use openraft_macros::add_async_trait; use openraft_macros::since; use crate::engine::LogIdList; +use crate::type_config::alias::LogIdOf; use crate::type_config::alias::VoteOf; -use crate::LogId; use crate::OptionalSend; use crate::OptionalSync; use crate::RaftTypeConfig; @@ -104,7 +104,7 @@ where C: RaftTypeConfig /// /// [`RaftLogStorage`]: crate::storage::RaftLogStorage #[since(version = "0.10.0")] - async fn get_key_log_ids(&mut self, range: RangeInclusive>) -> Result>, StorageError> { + async fn get_key_log_ids(&mut self, range: RangeInclusive>) -> Result>, StorageError> { LogIdList::get_key_log_ids(range, self).await } } diff --git a/openraft/src/storage/v2/raft_log_storage.rs b/openraft/src/storage/v2/raft_log_storage.rs index 0495c5618..b311b3f52 100644 --- a/openraft/src/storage/v2/raft_log_storage.rs +++ b/openraft/src/storage/v2/raft_log_storage.rs @@ -2,8 +2,8 @@ use openraft_macros::add_async_trait; use crate::storage::IOFlushed; use crate::storage::LogState; +use crate::type_config::alias::LogIdOf; use crate::type_config::alias::VoteOf; -use crate::LogId; use crate::OptionalSend; use crate::OptionalSync; use crate::RaftLogReader; @@ -67,13 +67,13 @@ where C: RaftTypeConfig /// See: [`docs::data::log_pointers`]. /// /// [`docs::data::log_pointers`]: `crate::docs::data::log_pointers#optionally-persisted-committed` - async fn save_committed(&mut self, _committed: Option>) -> Result<(), StorageError> { + async fn save_committed(&mut self, _committed: Option>) -> Result<(), StorageError> { // By default `committed` log id is not saved Ok(()) } /// Return the last saved committed log id by [`Self::save_committed`]. - async fn read_committed(&mut self) -> Result>, StorageError> { + async fn read_committed(&mut self) -> Result>, StorageError> { // By default `committed` log id is not saved and this method just return None. Ok(None) } @@ -106,12 +106,12 @@ where C: RaftTypeConfig /// ### To ensure correctness: /// /// - It must not leave a **hole** in logs. - async fn truncate(&mut self, log_id: LogId) -> Result<(), StorageError>; + async fn truncate(&mut self, log_id: LogIdOf) -> Result<(), StorageError>; /// Purge logs upto `log_id`, inclusive /// /// ### To ensure correctness: /// /// - It must not leave a **hole** in logs. - async fn purge(&mut self, log_id: LogId) -> Result<(), StorageError>; + async fn purge(&mut self, log_id: LogIdOf) -> Result<(), StorageError>; } diff --git a/openraft/src/storage/v2/raft_state_machine.rs b/openraft/src/storage/v2/raft_state_machine.rs index 9b28f6d46..c1c4c1dd8 100644 --- a/openraft/src/storage/v2/raft_state_machine.rs +++ b/openraft/src/storage/v2/raft_state_machine.rs @@ -2,7 +2,7 @@ use openraft_macros::add_async_trait; use crate::storage::Snapshot; use crate::storage::SnapshotMeta; -use crate::LogId; +use crate::type_config::alias::LogIdOf; use crate::OptionalSend; use crate::OptionalSync; use crate::RaftSnapshotBuilder; @@ -35,7 +35,7 @@ where C: RaftTypeConfig /// last-applied-log-id. /// Because upon startup, the last membership will be loaded by scanning logs from the /// `last-applied-log-id`. - async fn applied_state(&mut self) -> Result<(Option>, StoredMembership), StorageError>; + async fn applied_state(&mut self) -> Result<(Option>, StoredMembership), StorageError>; /// Apply the given payload of entries to the state machine. /// diff --git a/openraft/src/storage_error.rs b/openraft/src/storage_error.rs index 04e1436c3..37c5100f6 100644 --- a/openraft/src/storage_error.rs +++ b/openraft/src/storage_error.rs @@ -3,7 +3,7 @@ use std::fmt; use anyerror::AnyError; use crate::storage::SnapshotSignature; -use crate::LogId; +use crate::type_config::alias::LogIdOf; use crate::RaftTypeConfig; /// Convert error to StorageError::IO(); @@ -48,13 +48,13 @@ where C: RaftTypeConfig Logs, /// Error about a single log entry - Log(LogId), + Log(LogIdOf), /// Error about a single log entry without knowing the log term. LogIndex(u64), /// Error happened when applying a log entry - Apply(LogId), + Apply(LogIdOf), /// Error happened when operating state machine. StateMachine, @@ -137,7 +137,7 @@ where C: RaftTypeConfig } } - pub fn write_log_entry(log_id: LogId, source: impl Into) -> Self { + pub fn write_log_entry(log_id: LogIdOf, source: impl Into) -> Self { Self::new(ErrorSubject::Log(log_id), ErrorVerb::Write, source) } @@ -145,7 +145,7 @@ where C: RaftTypeConfig Self::new(ErrorSubject::LogIndex(log_index), ErrorVerb::Read, source) } - pub fn read_log_entry(log_id: LogId, source: impl Into) -> Self { + pub fn read_log_entry(log_id: LogIdOf, source: impl Into) -> Self { Self::new(ErrorSubject::Log(log_id), ErrorVerb::Read, source) } @@ -165,7 +165,7 @@ where C: RaftTypeConfig Self::new(ErrorSubject::Vote, ErrorVerb::Read, source) } - pub fn apply(log_id: LogId, source: impl Into) -> Self { + pub fn apply(log_id: LogIdOf, source: impl Into) -> Self { Self::new(ErrorSubject::Apply(log_id), ErrorVerb::Write, source) } diff --git a/openraft/src/testing/common.rs b/openraft/src/testing/common.rs index 4ea87291e..d4bc35f71 100644 --- a/openraft/src/testing/common.rs +++ b/openraft/src/testing/common.rs @@ -3,12 +3,13 @@ use std::collections::BTreeSet; use crate::entry::RaftEntry; +use crate::type_config::alias::LogIdOf; use crate::vote::RaftLeaderIdExt; use crate::LogId; use crate::RaftTypeConfig; /// Builds a log id, for testing purposes. -pub fn log_id(term: u64, node_id: C::NodeId, index: u64) -> LogId +pub fn log_id(term: u64, node_id: C::NodeId, index: u64) -> LogIdOf where C: RaftTypeConfig, C::Term: From, diff --git a/openraft/src/testing/log/suite.rs b/openraft/src/testing/log/suite.rs index fb4ea693b..bf6ed7c9a 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::LogIdOf; use crate::type_config::alias::VoteOf; use crate::type_config::TypeConfigExt; use crate::vote::raft_vote::RaftVoteExt; @@ -53,7 +54,7 @@ trait ReaderExt: RaftLogStorage where C: RaftTypeConfig { /// Proxy method to invoke [`RaftLogReaderExt::get_log_id`]. - async fn get_log_id(&mut self, log_index: u64) -> Result, StorageError> { + async fn get_log_id(&mut self, log_index: u64) -> Result, StorageError> { self.get_log_reader().await.get_log_id(log_index).await } @@ -642,7 +643,7 @@ where tracing::info!("--- empty store, expect []"); { let initial = StorageHelper::new(&mut store, &mut sm).get_initial_state().await?; - assert_eq!(Vec::>::new(), initial.log_ids.key_log_ids()); + assert_eq!(Vec::>::new(), initial.log_ids.key_log_ids()); } tracing::info!("--- log terms: [0], last_purged_log_id is None, expect [(0,0)]"); @@ -1374,7 +1375,7 @@ where } /// Create a log id with node id 0 for testing. -fn log_id_0(term: impl Into, index: u64) -> LogId +fn log_id_0(term: impl Into, index: u64) -> LogIdOf where C: RaftTypeConfig, C::NodeId: From, @@ -1457,7 +1458,7 @@ where Ok(()) } -fn log_id(term: u64, node_id: u64, index: u64) -> LogId +fn log_id(term: u64, node_id: u64, index: u64) -> LogIdOf where C: RaftTypeConfig, C::Term: From, diff --git a/tests/tests/client_api/t16_with_state_machine.rs b/tests/tests/client_api/t16_with_state_machine.rs index 0d3a44ce8..89504ca0f 100644 --- a/tests/tests/client_api/t16_with_state_machine.rs +++ b/tests/tests/client_api/t16_with_state_machine.rs @@ -2,6 +2,7 @@ use std::sync::Arc; use anyhow::Result; use maplit::btreeset; +use openraft::alias::LogIdOf; use openraft::error::Fatal; use openraft::storage::RaftStateMachine; use openraft::storage::Snapshot; @@ -9,7 +10,6 @@ use openraft::storage::SnapshotMeta; use openraft::testing::log_id; use openraft::Config; use openraft::Entry; -use openraft::LogId; use openraft::OptionalSend; use openraft::RaftSnapshotBuilder; use openraft::RaftTypeConfig; @@ -98,7 +98,7 @@ async fn with_state_machine_wrong_sm_type() -> Result<()> { impl RaftStateMachine for FooSM { type SnapshotBuilder = Self; - async fn applied_state(&mut self) -> Result<(Option>, StoredMembership), Err> { + async fn applied_state(&mut self) -> Result<(Option>, StoredMembership), Err> { todo!() } diff --git a/tests/tests/fixtures/mod.rs b/tests/tests/fixtures/mod.rs index 640ee4e52..b10f671b4 100644 --- a/tests/tests/fixtures/mod.rs +++ b/tests/tests/fixtures/mod.rs @@ -48,7 +48,6 @@ use openraft::storage::RaftLogStorage; use openraft::storage::RaftStateMachine; use openraft::storage::Snapshot; use openraft::Config; -use openraft::LogId; use openraft::LogIdOptionExt; use openraft::OptionalSend; use openraft::RPCTypes; @@ -185,6 +184,7 @@ impl fmt::Display for Direction { } } +use openraft::alias::LogIdOf; use openraft::alias::VoteOf; use openraft::network::v2::RaftNetworkV2; use openraft::vote::RaftLeaderId; @@ -729,7 +729,7 @@ impl TypedRaftRouter { pub async fn wait_for_snapshot( &self, node_ids: &BTreeSet, - want: LogId, + want: LogIdOf, timeout: Option, msg: &str, ) -> anyhow::Result<()> { @@ -867,7 +867,7 @@ impl TypedRaftRouter { expect_term: u64, expect_last_log: u64, expect_voted_for: Option, - expect_sm_last_applied_log: LogId, + expect_sm_last_applied_log: LogIdOf, expect_snapshot: &Option<(ValueTest, u64)>, ) -> anyhow::Result<()> { let last_log_id = storage.get_log_state().await?.last_log_id; @@ -959,7 +959,7 @@ impl TypedRaftRouter { expect_term: u64, expect_last_log: u64, expect_voted_for: Option, - expect_sm_last_applied_log: LogId, + expect_sm_last_applied_log: LogIdOf, expect_snapshot: Option<(ValueTest, u64)>, ) -> anyhow::Result<()> { let node_ids = {