diff --git a/examples/raft-kv-memstore-generic-snapshot-data/README.md b/examples/raft-kv-memstore-generic-snapshot-data/README.md index da9d63e07..ad1f1ef03 100644 --- a/examples/raft-kv-memstore-generic-snapshot-data/README.md +++ b/examples/raft-kv-memstore-generic-snapshot-data/README.md @@ -7,7 +7,7 @@ This example is similar to the basic raft-kv-memstore example but focuses on how to handle snapshot with `generic-snapshot-data` enabled. Other aspects are minimized. -To send a complete snapshot, Refer to implementation of `RaftNetwork::snapshot()` in this example. +To send a complete snapshot, Refer to implementation of `RaftNetwork::full_snapshot()` in this example. To receive a complete snapshot, Refer to implementation of `api::snapshot()` in this example. diff --git a/examples/raft-kv-memstore-generic-snapshot-data/src/api.rs b/examples/raft-kv-memstore-generic-snapshot-data/src/api.rs index f3e610f6a..89394adc5 100644 --- a/examples/raft-kv-memstore-generic-snapshot-data/src/api.rs +++ b/examples/raft-kv-memstore-generic-snapshot-data/src/api.rs @@ -60,7 +60,7 @@ pub async fn snapshot(app: &mut App, req: String) -> String { }; let res = app .raft - .install_complete_snapshot(vote, snapshot) + .install_full_snapshot(vote, snapshot) .await .map_err(typ::RaftError::::Fatal); encode(res) diff --git a/examples/raft-kv-memstore-generic-snapshot-data/src/network.rs b/examples/raft-kv-memstore-generic-snapshot-data/src/network.rs index 45e459433..20842ac8c 100644 --- a/examples/raft-kv-memstore-generic-snapshot-data/src/network.rs +++ b/examples/raft-kv-memstore-generic-snapshot-data/src/network.rs @@ -51,7 +51,7 @@ impl RaftNetwork for Connection { } /// A real application should replace this method with customized implementation. - async fn snapshot( + async fn full_snapshot( &mut self, vote: Vote, snapshot: Snapshot, diff --git a/examples/raft-kv-memstore-generic-snapshot-data/tests/cluster/test_cluster.rs b/examples/raft-kv-memstore-generic-snapshot-data/tests/cluster/test_cluster.rs index 9f14ee922..8affab779 100644 --- a/examples/raft-kv-memstore-generic-snapshot-data/tests/cluster/test_cluster.rs +++ b/examples/raft-kv-memstore-generic-snapshot-data/tests/cluster/test_cluster.rs @@ -37,9 +37,9 @@ pub fn log_panic(panic: &PanicInfo) { /// /// - Setup a single node cluster, write some logs, take a snapshot; /// - Add a learner node-2 to receive snapshot replication, via the complete-snapshot API: -/// - The sending end sends snapshot with `RaftNetwork::snapshot()`; +/// - The sending end sends snapshot with `RaftNetwork::full_snapshot()`; /// - The receiving end deliver the received snapshot to `Raft` with -/// `Raft::install_complete_snapshot()`. +/// `Raft::install_full_snapshot()`. #[tokio::test] async fn test_cluster() { std::panic::set_hook(Box::new(|panic| { diff --git a/examples/raft-kv-memstore-opendal-snapshot-data/README.md b/examples/raft-kv-memstore-opendal-snapshot-data/README.md index 41fb2ce5a..e871f2f18 100644 --- a/examples/raft-kv-memstore-opendal-snapshot-data/README.md +++ b/examples/raft-kv-memstore-opendal-snapshot-data/README.md @@ -9,7 +9,7 @@ This example is similar to the basic raft-kv-memstore example but focuses on how to store and fetch snapshot data from remote storage. Other aspects are minimized. -To send a complete snapshot, Refer to implementation of `RaftNetwork::snapshot()` in this example. +To send a complete snapshot, Refer to implementation of `RaftNetwork::full_snapshot()` in this example. To receive a complete snapshot, Refer to implementation of `api::snapshot()` in this example. diff --git a/examples/raft-kv-memstore-opendal-snapshot-data/src/api.rs b/examples/raft-kv-memstore-opendal-snapshot-data/src/api.rs index f3e610f6a..89394adc5 100644 --- a/examples/raft-kv-memstore-opendal-snapshot-data/src/api.rs +++ b/examples/raft-kv-memstore-opendal-snapshot-data/src/api.rs @@ -60,7 +60,7 @@ pub async fn snapshot(app: &mut App, req: String) -> String { }; let res = app .raft - .install_complete_snapshot(vote, snapshot) + .install_full_snapshot(vote, snapshot) .await .map_err(typ::RaftError::::Fatal); encode(res) diff --git a/examples/raft-kv-memstore-opendal-snapshot-data/src/network.rs b/examples/raft-kv-memstore-opendal-snapshot-data/src/network.rs index 45e459433..20842ac8c 100644 --- a/examples/raft-kv-memstore-opendal-snapshot-data/src/network.rs +++ b/examples/raft-kv-memstore-opendal-snapshot-data/src/network.rs @@ -51,7 +51,7 @@ impl RaftNetwork for Connection { } /// A real application should replace this method with customized implementation. - async fn snapshot( + async fn full_snapshot( &mut self, vote: Vote, snapshot: Snapshot, diff --git a/examples/raft-kv-memstore-opendal-snapshot-data/tests/cluster/test_cluster.rs b/examples/raft-kv-memstore-opendal-snapshot-data/tests/cluster/test_cluster.rs index c9d576ab6..87682a428 100644 --- a/examples/raft-kv-memstore-opendal-snapshot-data/tests/cluster/test_cluster.rs +++ b/examples/raft-kv-memstore-opendal-snapshot-data/tests/cluster/test_cluster.rs @@ -38,9 +38,9 @@ pub fn log_panic(panic: &PanicInfo) { /// /// - Setup a single node cluster, write some logs, take a snapshot; /// - Add a learner node-2 to receive snapshot replication, via the complete-snapshot API: -/// - The sending end sends snapshot with `RaftNetwork::snapshot()`; +/// - The sending end sends snapshot with `RaftNetwork::full_snapshot()`; /// - The receiving end deliver the received snapshot to `Raft` with -/// `Raft::install_complete_snapshot()`. +/// `Raft::install_full_snapshot()`. #[tokio::test] async fn test_cluster() { std::panic::set_hook(Box::new(|panic| { diff --git a/openraft/src/core/raft_core.rs b/openraft/src/core/raft_core.rs index c93d6592d..9ce86f542 100644 --- a/openraft/src/core/raft_core.rs +++ b/openraft/src/core/raft_core.rs @@ -1115,8 +1115,8 @@ where RaftMsg::BeginReceivingSnapshot { vote, tx } => { self.engine.handle_begin_receiving_snapshot(vote, tx); } - RaftMsg::InstallCompleteSnapshot { vote, snapshot, tx } => { - self.engine.handle_install_complete_snapshot(vote, snapshot, tx); + RaftMsg::InstallFullSnapshot { vote, snapshot, tx } => { + self.engine.handle_install_full_snapshot(vote, snapshot, tx); } RaftMsg::CheckIsLeaderRequest { tx } => { if self.engine.state.is_leader(&self.engine.config.id) { diff --git a/openraft/src/core/raft_msg/mod.rs b/openraft/src/core/raft_msg/mod.rs index 778cef933..b6679d43a 100644 --- a/openraft/src/core/raft_msg/mod.rs +++ b/openraft/src/core/raft_msg/mod.rs @@ -61,7 +61,7 @@ where C: RaftTypeConfig tx: VoteTx, }, - InstallCompleteSnapshot { + InstallFullSnapshot { vote: Vote, snapshot: Snapshot, tx: ResultSender>, @@ -124,8 +124,8 @@ where C: RaftTypeConfig RaftMsg::BeginReceivingSnapshot { vote, .. } => { format!("BeginReceivingSnapshot: vote: {}", vote) } - RaftMsg::InstallCompleteSnapshot { vote, snapshot, .. } => { - format!("InstallCompleteSnapshot: vote: {}, snapshot: {}", vote, snapshot) + RaftMsg::InstallFullSnapshot { vote, snapshot, .. } => { + format!("InstallFullSnapshot: vote: {}, snapshot: {}", vote, snapshot) } RaftMsg::ClientWriteRequest { .. } => "ClientWriteRequest".to_string(), RaftMsg::CheckIsLeaderRequest { .. } => "CheckIsLeaderRequest".to_string(), diff --git a/openraft/src/core/sm/command.rs b/openraft/src/core/sm/command.rs index b885fd941..2c49c2597 100644 --- a/openraft/src/core/sm/command.rs +++ b/openraft/src/core/sm/command.rs @@ -64,8 +64,8 @@ where C: RaftTypeConfig Command::new(payload) } - pub(crate) fn install_complete_snapshot(snapshot: Snapshot) -> Self { - let payload = CommandPayload::InstallCompleteSnapshot { snapshot }; + pub(crate) fn install_full_snapshot(snapshot: Snapshot) -> Self { + let payload = CommandPayload::InstallFullSnapshot { snapshot }; Command::new(payload) } @@ -98,7 +98,7 @@ where C: RaftTypeConfig tx: ResultSender>, HigherVote>, }, - InstallCompleteSnapshot { + InstallFullSnapshot { snapshot: Snapshot, }, @@ -115,8 +115,8 @@ where C: RaftTypeConfig match self { CommandPayload::BuildSnapshot => write!(f, "BuildSnapshot"), CommandPayload::GetSnapshot { .. } => write!(f, "GetSnapshot"), - CommandPayload::InstallCompleteSnapshot { snapshot } => { - write!(f, "InstallCompleteSnapshot: meta: {:?}", snapshot.meta) + CommandPayload::InstallFullSnapshot { snapshot } => { + write!(f, "InstallFullSnapshot: meta: {:?}", snapshot.meta) } CommandPayload::BeginReceivingSnapshot { .. } => { write!(f, "BeginReceivingSnapshot") @@ -136,8 +136,8 @@ where C: RaftTypeConfig (CommandPayload::GetSnapshot { .. }, CommandPayload::GetSnapshot { .. }) => true, (CommandPayload::BeginReceivingSnapshot { .. }, CommandPayload::BeginReceivingSnapshot { .. }) => true, ( - CommandPayload::InstallCompleteSnapshot { snapshot: s1 }, - CommandPayload::InstallCompleteSnapshot { snapshot: s2 }, + CommandPayload::InstallFullSnapshot { snapshot: s1 }, + CommandPayload::InstallFullSnapshot { snapshot: s2 }, ) => s1.meta == s2.meta, (CommandPayload::Apply { entries: entries1 }, CommandPayload::Apply { entries: entries2 }) => { // Entry may not be `Eq`, we just compare log id. diff --git a/openraft/src/core/sm/mod.rs b/openraft/src/core/sm/mod.rs index d88c30f04..05a9d6987 100644 --- a/openraft/src/core/sm/mod.rs +++ b/openraft/src/core/sm/mod.rs @@ -124,7 +124,7 @@ where self.get_snapshot(tx).await?; // GetSnapshot does not respond to RaftCore } - CommandPayload::InstallCompleteSnapshot { snapshot } => { + CommandPayload::InstallFullSnapshot { snapshot } => { tracing::info!("{}: install complete snapshot", func_name!()); let meta = snapshot.meta.clone(); diff --git a/openraft/src/docs/feature_flags/feature-flags.md b/openraft/src/docs/feature_flags/feature-flags.md index 21aa533d4..29fb35be4 100644 --- a/openraft/src/docs/feature_flags/feature-flags.md +++ b/openraft/src/docs/feature_flags/feature-flags.md @@ -35,11 +35,11 @@ This feature is introduced in 0.9.0 On the sending end (leader that sends snapshot to follower): -- Without `generic-snapshot-data`: [`RaftNetwork::snapshot()`] +- Without `generic-snapshot-data`: [`RaftNetwork::full_snapshot()`] provides a default implementation that invokes the chunk-based API [`RaftNetwork::install_snapshot()`] for transmit. -- With `generic-snapshot-data` enabled: [`RaftNetwork::snapshot()`] +- With `generic-snapshot-data` enabled: [`RaftNetwork::full_snapshot()`] must be implemented to provide application customized snapshot transmission. Application does not need to implement [`RaftNetwork::install_snapshot()`]. @@ -96,5 +96,5 @@ emit log record. See: [tracing doc: emitting-log-records](https://docs.rs/tracing/latest/tracing/#emitting-log-records) -[`RaftNetwork::snapshot()`]: crate::network::RaftNetwork::snapshot +[`RaftNetwork::full_snapshot()`]: crate::network::RaftNetwork::full_snapshot [`RaftNetwork::install_snapshot()`]: crate::network::RaftNetwork::install_snapshot \ No newline at end of file diff --git a/openraft/src/docs/getting_started/getting-started.md b/openraft/src/docs/getting_started/getting-started.md index 1d519d6bb..6bf00cd5f 100644 --- a/openraft/src/docs/getting_started/getting-started.md +++ b/openraft/src/docs/getting_started/getting-started.md @@ -184,11 +184,11 @@ and receiving messages between Raft nodes. Here is the list of methods that need to be implemented for the [`RaftNetwork`] trait: -| [`RaftNetwork`] method | forward request | to target | -|------------------------|--------------------------|---------------------------------------------------| -| [`append_entries()`] | [`AppendEntriesRequest`] | remote node [`Raft::append_entries()`] | -| [`snapshot()`] | [`Snapshot`] | remote node [`Raft::install_complete_snapshot()`] | -| [`vote()`] | [`VoteRequest`] | remote node [`Raft::vote()`] | +| [`RaftNetwork`] method | forward request | to target | +|------------------------|--------------------------|------------------------------------------------| +| [`append_entries()`] | [`AppendEntriesRequest`] | remote node [`Raft::append_entries()`] | +| [`full_snapshot()`] | [`Snapshot`] | remote node [`Raft::install_full_snapshot()`] | +| [`vote()`] | [`VoteRequest`] | remote node [`Raft::vote()`] | [Mem KV Network](https://github.com/datafuselabs/openraft/blob/main/examples/raft-kv-memstore/src/network/raft_network_impl.rs) demonstrates how to forward messages to other Raft nodes using [`reqwest`](https://docs.rs/reqwest/latest/reqwest/) as network transport layer. @@ -351,7 +351,7 @@ Additionally, two test scripts for setting up a cluster are available: [`Raft`]: `crate::Raft` [`Raft::append_entries()`]: `crate::Raft::append_entries` [`Raft::vote()`]: `crate::Raft::vote` -[`Raft::install_complete_snapshot()`]: `crate::Raft::install_complete_snapshot` +[`Raft::install_full_snapshot()`]: `crate::Raft::install_full_snapshot` [`AppendEntriesRequest`]: `crate::raft::AppendEntriesRequest` [`VoteRequest`]: `crate::raft::VoteRequest` @@ -398,7 +398,7 @@ Additionally, two test scripts for setting up a cluster are available: [`RaftNetwork`]: `crate::network::RaftNetwork` [`append_entries()`]: `crate::RaftNetwork::append_entries` [`vote()`]: `crate::RaftNetwork::vote` -[`snapshot()`]: `crate::RaftNetwork::snapshot` +[`full_snapshot()`]: `crate::RaftNetwork::full_snapshot` [`RaftSnapshotBuilder`]: `crate::storage::RaftSnapshotBuilder` diff --git a/openraft/src/engine/command.rs b/openraft/src/engine/command.rs index 2d11f3001..a61ff97f4 100644 --- a/openraft/src/engine/command.rs +++ b/openraft/src/engine/command.rs @@ -230,7 +230,7 @@ where AppendEntries(ValueSender, Infallible>>), ReceiveSnapshotChunk(ValueSender>), InstallSnapshot(ValueSender, InstallSnapshotError>>), - InstallCompleteSnapshot(ValueSender, Infallible>>), + InstallFullSnapshot(ValueSender, Infallible>>), Initialize(ValueSender>>), } @@ -253,7 +253,7 @@ where Respond::AppendEntries(x) => x.send(), Respond::ReceiveSnapshotChunk(x) => x.send(), Respond::InstallSnapshot(x) => x.send(), - Respond::InstallCompleteSnapshot(x) => x.send(), + Respond::InstallFullSnapshot(x) => x.send(), Respond::Initialize(x) => x.send(), } } diff --git a/openraft/src/engine/engine_impl.rs b/openraft/src/engine/engine_impl.rs index b5db75f52..8fbe07c0a 100644 --- a/openraft/src/engine/engine_impl.rs +++ b/openraft/src/engine/engine_impl.rs @@ -450,7 +450,7 @@ where C: RaftTypeConfig /// Install a completely received snapshot on a follower. #[tracing::instrument(level = "debug", skip_all)] - pub(crate) fn handle_install_complete_snapshot( + pub(crate) fn handle_install_full_snapshot( &mut self, vote: Vote, snapshot: Snapshot, @@ -467,7 +467,7 @@ where C: RaftTypeConfig }; let mut fh = self.following_handler(); - fh.install_complete_snapshot(snapshot); + fh.install_full_snapshot(snapshot); let res = Ok(SnapshotResponse { vote: *self.state.vote_ref(), }); diff --git a/openraft/src/engine/handler/following_handler/install_snapshot_test.rs b/openraft/src/engine/handler/following_handler/install_snapshot_test.rs index ade389fcc..57eade688 100644 --- a/openraft/src/engine/handler/following_handler/install_snapshot_test.rs +++ b/openraft/src/engine/handler/following_handler/install_snapshot_test.rs @@ -56,7 +56,7 @@ fn test_install_snapshot_lt_last_snapshot() -> anyhow::Result<()> { // `snapshot_meta.last_log_id`. let mut eng = eng(); - eng.following_handler().install_complete_snapshot(Snapshot { + eng.following_handler().install_full_snapshot(Snapshot { meta: SnapshotMeta { last_log_id: Some(log_id(2, 1, 2)), last_membership: StoredMembership::new(Some(log_id(1, 1, 1)), m1234()), @@ -86,7 +86,7 @@ fn test_install_snapshot_lt_committed() -> anyhow::Result<()> { // Although in this case the state machine is not affected. let mut eng = eng(); - eng.following_handler().install_complete_snapshot(Snapshot { + eng.following_handler().install_full_snapshot(Snapshot { meta: SnapshotMeta { last_log_id: Some(log_id(4, 1, 5)), last_membership: StoredMembership::new(Some(log_id(1, 1, 1)), m1234()), @@ -113,7 +113,7 @@ fn test_install_snapshot_not_conflict() -> anyhow::Result<()> { // Snapshot will be installed and there are no conflicting logs. let mut eng = eng(); - eng.following_handler().install_complete_snapshot(Snapshot { + eng.following_handler().install_full_snapshot(Snapshot { meta: SnapshotMeta { last_log_id: Some(log_id(4, 1, 6)), last_membership: StoredMembership::new(Some(log_id(1, 1, 1)), m1234()), @@ -140,7 +140,7 @@ fn test_install_snapshot_not_conflict() -> anyhow::Result<()> { vec![ // Command::from( - sm::Command::install_complete_snapshot(Snapshot { + sm::Command::install_full_snapshot(Snapshot { meta: SnapshotMeta { last_log_id: Some(log_id(4, 1, 6)), last_membership: StoredMembership::new(Some(log_id(1, 1, 1)), m1234()), @@ -187,7 +187,7 @@ fn test_install_snapshot_conflict() -> anyhow::Result<()> { eng }; - eng.following_handler().install_complete_snapshot(Snapshot { + eng.following_handler().install_full_snapshot(Snapshot { meta: SnapshotMeta { last_log_id: Some(log_id(5, 1, 6)), last_membership: StoredMembership::new(Some(log_id(1, 1, 1)), m1234()), @@ -215,7 +215,7 @@ fn test_install_snapshot_conflict() -> anyhow::Result<()> { // Command::DeleteConflictLog { since: log_id(2, 1, 4) }, Command::from( - sm::Command::install_complete_snapshot(Snapshot { + sm::Command::install_full_snapshot(Snapshot { meta: SnapshotMeta { last_log_id: Some(log_id(5, 1, 6)), last_membership: StoredMembership::new(Some(log_id(1, 1, 1)), m1234()), @@ -238,7 +238,7 @@ fn test_install_snapshot_advance_last_log_id() -> anyhow::Result<()> { // Snapshot will be installed and there are no conflicting logs. let mut eng = eng(); - eng.following_handler().install_complete_snapshot(Snapshot { + eng.following_handler().install_full_snapshot(Snapshot { meta: SnapshotMeta { last_log_id: Some(log_id(100, 1, 100)), last_membership: StoredMembership::new(Some(log_id(1, 1, 1)), m1234()), @@ -268,7 +268,7 @@ fn test_install_snapshot_advance_last_log_id() -> anyhow::Result<()> { assert_eq!( vec![ Command::from( - sm::Command::install_complete_snapshot(Snapshot { + sm::Command::install_full_snapshot(Snapshot { meta: SnapshotMeta { last_log_id: Some(log_id(100, 1, 100)), last_membership: StoredMembership::new(Some(log_id(1, 1, 1)), m1234()), @@ -293,7 +293,7 @@ fn test_install_snapshot_update_accepted() -> anyhow::Result<()> { // Snapshot will be installed and `accepted` should be updated. let mut eng = eng(); - eng.following_handler().install_complete_snapshot(Snapshot { + eng.following_handler().install_full_snapshot(Snapshot { meta: SnapshotMeta { last_log_id: Some(log_id(100, 1, 100)), last_membership: StoredMembership::new(Some(log_id(1, 1, 1)), m1234()), diff --git a/openraft/src/engine/handler/following_handler/mod.rs b/openraft/src/engine/handler/following_handler/mod.rs index 7897e1bd7..8780dd77b 100644 --- a/openraft/src/engine/handler/following_handler/mod.rs +++ b/openraft/src/engine/handler/following_handler/mod.rs @@ -243,9 +243,9 @@ where C: RaftTypeConfig /// Refer to [`snapshot_replication`](crate::docs::protocol::replication::snapshot_replication) /// for the reason the following workflow is needed. #[tracing::instrument(level = "debug", skip_all)] - pub(crate) fn install_complete_snapshot(&mut self, snapshot: Snapshot) { + pub(crate) fn install_full_snapshot(&mut self, snapshot: Snapshot) { let meta = &snapshot.meta; - tracing::info!("install_complete_snapshot: meta:{:?}", meta); + tracing::info!("install_full_snapshot: meta:{:?}", meta); let snap_last_log_id = meta.last_log_id; @@ -285,7 +285,7 @@ where C: RaftTypeConfig meta.last_membership.clone(), )); - self.output.push_command(Command::from(sm::Command::install_complete_snapshot(snapshot))); + self.output.push_command(Command::from(sm::Command::install_full_snapshot(snapshot))); self.state.purge_upto = Some(snap_last_log_id); self.log_handler().purge_log(); diff --git a/openraft/src/network/network.rs b/openraft/src/network/network.rs index 5131db833..59eb860e9 100644 --- a/openraft/src/network/network.rs +++ b/openraft/src/network/network.rs @@ -72,12 +72,12 @@ where C: RaftTypeConfig /// /// The `vote` is the leader vote which is used to check if the leader is still valid by a /// follower. - /// When the follower finished receiving snapshot, it calls `Raft::install_complete_snapshot()` + /// When the follower finished receiving snapshot, it calls `Raft::install_full_snapshot()` /// with this vote. /// /// `cancel` get `Ready` when the caller decides to cancel this snapshot transmission. #[cfg(feature = "generic-snapshot-data")] - async fn snapshot( + async fn full_snapshot( &mut self, vote: Vote, snapshot: Snapshot, @@ -86,7 +86,7 @@ where C: RaftTypeConfig ) -> Result, StreamingError>>; #[cfg(not(feature = "generic-snapshot-data"))] - async fn snapshot( + async fn full_snapshot( &mut self, vote: Vote, snapshot: Snapshot, diff --git a/openraft/src/network/stream_snapshot.rs b/openraft/src/network/stream_snapshot.rs index c625ea52d..3245c1b4a 100644 --- a/openraft/src/network/stream_snapshot.rs +++ b/openraft/src/network/stream_snapshot.rs @@ -58,7 +58,7 @@ impl SnapshotTransport for Chunked { /// Stream snapshot by chunks. /// /// This function is for backward compatibility and provides a default implement for - /// `RaftNetwork::snapshot()` upon `RafNetwork::install_snapshot()`. This implementation + /// `RaftNetwork::full_snapshot()` upon `RafNetwork::install_snapshot()`. This implementation /// requires `SnapshotData` to be `AsyncRead + AsyncSeek`. /// /// The argument `vote` is the leader's vote which is used to check if the leader is still valid @@ -96,7 +96,7 @@ impl SnapshotTransport for Chunked { snapshot.snapshot.seek(SeekFrom::Start(offset)).await.sto_res(subject_verb)?; // Safe unwrap(): this function is called only by default implementation of - // `RaftNetwork::snapshot()` and it is always set. + // `RaftNetwork::full_snapshot()` and it is always set. let chunk_size = option.snapshot_chunk_size().unwrap(); let mut buf = Vec::with_capacity(chunk_size); while buf.capacity() > buf.len() { diff --git a/openraft/src/raft/message/install_snapshot.rs b/openraft/src/raft/message/install_snapshot.rs index 7600e74e4..f48806db9 100644 --- a/openraft/src/raft/message/install_snapshot.rs +++ b/openraft/src/raft/message/install_snapshot.rs @@ -55,7 +55,7 @@ pub struct InstallSnapshotResponse { pub vote: Vote, } -/// The response to `Raft::install_complete_snapshot` API. +/// The response to `Raft::install_full_snapshot` API. #[derive(Debug)] #[derive(PartialEq, Eq)] #[derive(derive_more::Display)] diff --git a/openraft/src/raft/mod.rs b/openraft/src/raft/mod.rs index c1d403239..5ae484b05 100644 --- a/openraft/src/raft/mod.rs +++ b/openraft/src/raft/mod.rs @@ -383,15 +383,15 @@ where C: RaftTypeConfig /// The application receives a snapshot from the leader, in chunks or a stream, and /// then rebuild a snapshot, then pass the snapshot to Raft to install. #[tracing::instrument(level = "debug", skip_all)] - pub async fn install_complete_snapshot( + pub async fn install_full_snapshot( &self, vote: Vote, snapshot: Snapshot, ) -> Result, Fatal> { - tracing::info!("Raft::install_complete_snapshot()"); + tracing::info!("Raft::install_full_snapshot()"); let (tx, rx) = oneshot::channel(); - let res = self.inner.call_core(RaftMsg::InstallCompleteSnapshot { vote, snapshot, tx }, rx).await; + let res = self.inner.call_core(RaftMsg::InstallFullSnapshot { vote, snapshot, tx }, rx).await; match res { Ok(x) => Ok(x), Err(e) => { @@ -464,7 +464,7 @@ where C: RaftTypeConfig let snapshot = Chunked::receive_snapshot(&mut *streaming, req).await?; if let Some(snapshot) = snapshot { - let resp = self.install_complete_snapshot(req_vote, snapshot).await?; + let resp = self.install_full_snapshot(req_vote, snapshot).await?; Ok(resp.into()) } else { diff --git a/openraft/src/replication/mod.rs b/openraft/src/replication/mod.rs index 6af7aa176..d0927629a 100644 --- a/openraft/src/replication/mod.rs +++ b/openraft/src/replication/mod.rs @@ -806,7 +806,7 @@ where ReplicationClosed::new("ReplicationCore is dropped") }; - let res = net.snapshot(vote, snapshot, cancel, option).await; + let res = net.full_snapshot(vote, snapshot, cancel, option).await; if let Err(e) = &res { tracing::warn!(error = display(e), "failed to send snapshot"); } diff --git a/tests/tests/client_api/main.rs b/tests/tests/client_api/main.rs index a8a378cdf..94711e97d 100644 --- a/tests/tests/client_api/main.rs +++ b/tests/tests/client_api/main.rs @@ -12,7 +12,7 @@ mod t11_client_reads; mod t12_trigger_purge_log; mod t13_begin_receiving_snapshot; mod t13_get_snapshot; -mod t13_install_complete_snapshot; +mod t13_install_full_snapshot; mod t13_trigger_snapshot; mod t16_with_raft_state; mod t50_lagging_network_write; diff --git a/tests/tests/client_api/t13_install_complete_snapshot.rs b/tests/tests/client_api/t13_install_full_snapshot.rs similarity index 89% rename from tests/tests/client_api/t13_install_complete_snapshot.rs rename to tests/tests/client_api/t13_install_full_snapshot.rs index a647839ba..e40098433 100644 --- a/tests/tests/client_api/t13_install_complete_snapshot.rs +++ b/tests/tests/client_api/t13_install_full_snapshot.rs @@ -10,7 +10,7 @@ use crate::fixtures::init_default_ut_tracing; use crate::fixtures::RaftRouter; #[async_entry::test(worker_threads = 8, init = "init_default_ut_tracing()", tracing_span = "debug")] -async fn install_complete_snapshot() -> anyhow::Result<()> { +async fn install_full_snapshot() -> anyhow::Result<()> { let config = Arc::new( Config { enable_heartbeat: false, @@ -51,7 +51,7 @@ async fn install_complete_snapshot() -> anyhow::Result<()> { { let n1 = router.get_raft_handle(&1)?; - let resp = n1.install_complete_snapshot(Vote::new(0, 0), snap.clone()).await?; + let resp = n1.install_full_snapshot(Vote::new(0, 0), snap.clone()).await?; assert_eq!( Vote::new_committed(1, 0), resp.vote, @@ -73,7 +73,7 @@ async fn install_complete_snapshot() -> anyhow::Result<()> { { let n1 = router.get_raft_handle(&1)?; - let resp = n1.install_complete_snapshot(Vote::new_committed(1, 0), snap.clone()).await?; + let resp = n1.install_full_snapshot(Vote::new_committed(1, 0), snap.clone()).await?; assert_eq!(Vote::new_committed(1, 0), resp.vote,); n1.with_raft_state(move |state| { assert_eq!( @@ -88,7 +88,7 @@ async fn install_complete_snapshot() -> anyhow::Result<()> { { let n2 = router.get_raft_handle(&2)?; - let resp = n2.install_complete_snapshot(Vote::new_committed(1, 0), snap.clone()).await?; + let resp = n2.install_full_snapshot(Vote::new_committed(1, 0), snap.clone()).await?; assert_eq!(Vote::new_committed(1, 0), resp.vote,); n2.with_raft_state(move |state| { assert_eq!(