From a034382623dbba2a0fdc916bb6af1fb7f9bd7147 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E7=82=8E=E6=B3=BC?= Date: Mon, 20 Nov 2023 15:31:33 +0800 Subject: [PATCH] Change: remove `N, LS` from `Raft` - `Raft`: is a control handle of `RaftCore` and it does not directly rely on `N: RaftNetworkFactory` and `LS: RaftLogStorage`. Thus these two type should not be part of `Raft`. In this commit, we remove `N, LS` from `Raft`, `RaftInner` and `RaftMsg`. Type `N, LS` now is only used by `Raft::new()` which needs these two types to create `RaftCore`. - `Raft::external_request()`: Another change is the signature of the `Fn` passed to `Raft::external_request()` changes from `FnOnce(&RaftState, &mut LS, &mut N)` to `FnOnce(&RaftState)`. - Fix: the `FnOnce` passed to `Raft::external_request()` should always be `Send`, unoptionally. Because it is used to send from `Raft` to `RaftCore`. - Fix: #939 --- cluster_benchmark/tests/benchmark/network.rs | 2 +- examples/raft-kv-memstore/src/lib.rs | 2 +- examples/raft-kv-rocksdb/src/lib.rs | 2 +- openraft/src/core/raft_core.rs | 8 +-- openraft/src/core/raft_msg/mod.rs | 32 +++-------- openraft/src/raft/external_request.rs | 9 ++++ openraft/src/raft/mod.rs | 53 ++++++++----------- openraft/src/raft/raft_inner.rs | 18 ++----- openraft/src/raft/runtime_config_handle.rs | 20 +++---- openraft/src/raft/trigger.rs | 20 +++---- openraft/src/type_config.rs | 20 +++++++ .../append_entries/t10_see_higher_vote.rs | 2 +- .../append_entries/t60_enable_heartbeat.rs | 2 +- .../t61_heartbeat_reject_vote.rs | 4 +- tests/tests/fixtures/mod.rs | 8 ++- tests/tests/life_cycle/t10_initialization.rs | 10 ++-- tests/tests/life_cycle/t11_shutdown.rs | 2 +- .../tests/membership/t20_change_membership.rs | 2 +- 18 files changed, 95 insertions(+), 121 deletions(-) create mode 100644 openraft/src/raft/external_request.rs diff --git a/cluster_benchmark/tests/benchmark/network.rs b/cluster_benchmark/tests/benchmark/network.rs index 42b06583e..7b75714ac 100644 --- a/cluster_benchmark/tests/benchmark/network.rs +++ b/cluster_benchmark/tests/benchmark/network.rs @@ -29,7 +29,7 @@ use crate::store::NodeId; use crate::store::StateMachineStore; use crate::store::TypeConfig as MemConfig; -pub type BenchRaft = Raft, Arc>; +pub type BenchRaft = Raft>; #[derive(Clone)] pub struct Router { diff --git a/examples/raft-kv-memstore/src/lib.rs b/examples/raft-kv-memstore/src/lib.rs index 328a52ad6..8176d91ea 100644 --- a/examples/raft-kv-memstore/src/lib.rs +++ b/examples/raft-kv-memstore/src/lib.rs @@ -37,7 +37,7 @@ openraft::declare_raft_types!( pub type LogStore = Adaptor>; pub type StateMachineStore = Adaptor>; -pub type Raft = openraft::Raft; +pub type Raft = openraft::Raft; pub mod typ { use openraft::BasicNode; diff --git a/examples/raft-kv-rocksdb/src/lib.rs b/examples/raft-kv-rocksdb/src/lib.rs index afbc82672..dd9e2e908 100644 --- a/examples/raft-kv-rocksdb/src/lib.rs +++ b/examples/raft-kv-rocksdb/src/lib.rs @@ -47,7 +47,7 @@ openraft::declare_raft_types!( pub type LogStore = Adaptor>; pub type StateMachineStore = Adaptor>; -pub type ExampleRaft = openraft::Raft; +pub type ExampleRaft = openraft::Raft; type Server = tide::Server>; diff --git a/openraft/src/core/raft_core.rs b/openraft/src/core/raft_core.rs index 972a51872..c7c898de0 100644 --- a/openraft/src/core/raft_core.rs +++ b/openraft/src/core/raft_core.rs @@ -186,8 +186,8 @@ where pub(crate) leader_data: Option>, #[allow(dead_code)] - pub(crate) tx_api: mpsc::UnboundedSender>, - pub(crate) rx_api: mpsc::UnboundedReceiver>, + pub(crate) tx_api: mpsc::UnboundedSender>, + pub(crate) rx_api: mpsc::UnboundedReceiver>, /// A Sender to send callback by other components to [`RaftCore`], when an action is finished, /// such as flushing log to disk, or applying log entries to state machine. @@ -1063,7 +1063,7 @@ where // TODO: Make this method non-async. It does not need to run any async command in it. #[tracing::instrument(level = "debug", skip(self, msg), fields(state = debug(self.engine.state.server_state), id=display(self.id)))] - pub(crate) async fn handle_api_msg(&mut self, msg: RaftMsg) { + pub(crate) async fn handle_api_msg(&mut self, msg: RaftMsg) { tracing::debug!("recv from rx_api: {}", msg.summary()); match msg { @@ -1120,7 +1120,7 @@ where self.change_membership(changes, retain, tx); } RaftMsg::ExternalRequest { req } => { - req(&self.engine.state, &mut self.log_store, &mut self.network); + req(&self.engine.state); } RaftMsg::ExternalCommand { cmd } => { tracing::info!(cmd = debug(&cmd), "received RaftMsg::ExternalCommand: {}", func_name!()); diff --git a/openraft/src/core/raft_msg/mod.rs b/openraft/src/core/raft_msg/mod.rs index a18aa70ea..e7b1c3c69 100644 --- a/openraft/src/core/raft_msg/mod.rs +++ b/openraft/src/core/raft_msg/mod.rs @@ -10,17 +10,14 @@ use crate::error::InitializeError; use crate::error::InstallSnapshotError; use crate::raft::AppendEntriesRequest; use crate::raft::AppendEntriesResponse; +use crate::raft::BoxCoreFn; use crate::raft::ClientWriteResponse; use crate::raft::InstallSnapshotRequest; use crate::raft::InstallSnapshotResponse; use crate::raft::VoteRequest; use crate::raft::VoteResponse; -use crate::storage::RaftLogStorage; -use crate::AsyncRuntime; use crate::ChangeMembers; use crate::MessageSummary; -use crate::RaftNetworkFactory; -use crate::RaftState; use crate::RaftTypeConfig; pub(crate) mod external_command; @@ -44,11 +41,8 @@ pub(crate) type ClientWriteTx = /// A message sent by application to the [`RaftCore`]. /// /// [`RaftCore`]: crate::core::RaftCore -pub(crate) enum RaftMsg -where - C: RaftTypeConfig, - N: RaftNetworkFactory, - LS: RaftLogStorage, +pub(crate) enum RaftMsg +where C: RaftTypeConfig { AppendEntries { rpc: AppendEntriesRequest, @@ -89,19 +83,8 @@ where tx: ResultSender, ClientWriteError>, }, - #[allow(clippy::type_complexity)] ExternalRequest { - #[cfg(not(feature = "singlethreaded"))] - req: Box< - dyn FnOnce(&RaftState::Instant>, &mut LS, &mut N) - + Send - + 'static, - >, - #[cfg(feature = "singlethreaded")] - req: Box< - dyn FnOnce(&RaftState::Instant>, &mut LS, &mut N) - + 'static, - >, + req: BoxCoreFn, }, ExternalCommand { @@ -109,11 +92,8 @@ where }, } -impl MessageSummary> for RaftMsg -where - C: RaftTypeConfig, - N: RaftNetworkFactory, - LS: RaftLogStorage, +impl MessageSummary> for RaftMsg +where C: RaftTypeConfig { fn summary(&self) -> String { match self { diff --git a/openraft/src/raft/external_request.rs b/openraft/src/raft/external_request.rs new file mode 100644 index 000000000..c9b9b425e --- /dev/null +++ b/openraft/src/raft/external_request.rs @@ -0,0 +1,9 @@ +//! Defines API for application to send request to access Raft core. + +use crate::type_config::alias::InstantOf; +use crate::type_config::alias::NodeIdOf; +use crate::type_config::alias::NodeOf; +use crate::RaftState; + +/// Boxed trait object for external request function run in `RaftCore` task. +pub(crate) type BoxCoreFn = Box, NodeOf, InstantOf>) + Send + 'static>; diff --git a/openraft/src/raft/mod.rs b/openraft/src/raft/mod.rs index 6203f2150..df8401ea8 100644 --- a/openraft/src/raft/mod.rs +++ b/openraft/src/raft/mod.rs @@ -1,10 +1,13 @@ //! Public Raft interface and data types. +mod external_request; mod message; mod raft_inner; mod runtime_config_handle; mod trigger; +pub(crate) use self::external_request::BoxCoreFn; + pub(in crate::raft) mod core_state; use std::fmt::Debug; @@ -59,7 +62,6 @@ use crate::ChangeMembers; use crate::LogId; use crate::LogIdOptionExt; use crate::MessageSummary; -use crate::OptionalSend; use crate::RaftState; pub use crate::RaftTypeConfig; use crate::StorageHelper; @@ -124,22 +126,18 @@ macro_rules! declare_raft_types { /// `shutdown` method should be called on this type to await the shutdown of the node. If the parent /// application needs to shutdown the Raft node for any reason, calling `shutdown` will do the /// trick. -pub struct Raft +pub struct Raft where C: RaftTypeConfig, - N: RaftNetworkFactory, - LS: RaftLogStorage, SM: RaftStateMachine, { - inner: Arc>, + inner: Arc>, _phantom: PhantomData, } -impl Clone for Raft +impl Clone for Raft where C: RaftTypeConfig, - N: RaftNetworkFactory, - LS: RaftLogStorage, SM: RaftStateMachine, { fn clone(&self) -> Self { @@ -159,11 +157,9 @@ where // // Notably, the state machine, log storage and network factory DO NOT have to be `Send`, those // are only used within Raft task(s) on a single thread. -unsafe impl Send for Raft +unsafe impl Send for Raft where C: RaftTypeConfig, - N: RaftNetworkFactory, - LS: RaftLogStorage, SM: RaftStateMachine, C::D: Send, C::Entry: Send, @@ -177,11 +173,9 @@ where // SAFETY: Even for a single-threaded Raft, the API object is MT-capable. // // See above for details. -unsafe impl Sync for Raft +unsafe impl Sync for Raft where C: RaftTypeConfig + Send, - N: RaftNetworkFactory, - LS: RaftLogStorage, SM: RaftStateMachine, C::D: Send, C::Entry: Send, @@ -191,11 +185,9 @@ where { } -impl Raft +impl Raft where C: RaftTypeConfig, - N: RaftNetworkFactory, - LS: RaftLogStorage, SM: RaftStateMachine, { /// Create and spawn a new Raft task. @@ -218,13 +210,17 @@ where /// An implementation of the `RaftStorage` trait which will be used by Raft for data storage. /// See the docs on the `RaftStorage` trait for more details. #[tracing::instrument(level="debug", skip_all, fields(cluster=%config.cluster_name))] - pub async fn new( + pub async fn new( id: C::NodeId, config: Arc, network: N, mut log_store: LS, mut state_machine: SM, - ) -> Result> { + ) -> Result> + where + N: RaftNetworkFactory, + LS: RaftLogStorage, + { let (tx_api, rx_api) = mpsc::unbounded_channel(); let (tx_notify, rx_notify) = mpsc::unbounded_channel(); let (tx_metrics, rx_metrics) = watch::channel(RaftMetrics::new_initial(id)); @@ -310,7 +306,7 @@ where /// let raft = Raft::new(...).await?; /// raft.runtime_config().heartbeat(true); /// ``` - pub fn runtime_config(&self) -> RuntimeConfigHandle { + pub fn runtime_config(&self) -> RuntimeConfigHandle { RuntimeConfigHandle::new(self.inner.as_ref()) } @@ -337,7 +333,7 @@ where /// let raft = Raft::new(...).await?; /// raft.trigger().elect().await?; /// ``` - pub fn trigger(&self) -> Trigger { + pub fn trigger(&self) -> Trigger { Trigger::new(self.inner.as_ref()) } @@ -694,7 +690,7 @@ where #[tracing::instrument(level = "debug", skip(self, mes, rx))] pub(crate) async fn call_core( &self, - mes: RaftMsg, + mes: RaftMsg, rx: oneshot::Receiver>, ) -> Result> where @@ -738,15 +734,10 @@ where /// /// If the API channel is already closed (Raft is in shutdown), then the request functor is /// destroyed right away and not called at all. - pub fn external_request< - F: FnOnce(&RaftState::Instant>, &mut LS, &mut N) - + OptionalSend - + 'static, - >( - &self, - req: F, - ) { - let _ignore_error = self.inner.tx_api.send(RaftMsg::ExternalRequest { req: Box::new(req) }); + pub fn external_request(&self, req: F) + where F: FnOnce(&RaftState::Instant>) + Send + 'static { + let req: BoxCoreFn = Box::new(req); + let _ignore_error = self.inner.tx_api.send(RaftMsg::ExternalRequest { req }); } /// Get a handle to the metrics channel. diff --git a/openraft/src/raft/raft_inner.rs b/openraft/src/raft/raft_inner.rs index 0bcb0afcf..69712ce4e 100644 --- a/openraft/src/raft/raft_inner.rs +++ b/openraft/src/raft/raft_inner.rs @@ -12,26 +12,21 @@ use crate::core::raft_msg::RaftMsg; use crate::core::TickHandle; use crate::error::Fatal; use crate::raft::core_state::CoreState; -use crate::storage::RaftLogStorage; use crate::AsyncRuntime; use crate::Config; use crate::RaftMetrics; -use crate::RaftNetworkFactory; use crate::RaftTypeConfig; /// RaftInner is the internal handle and provides internally used APIs to communicate with /// `RaftCore`. -pub(in crate::raft) struct RaftInner -where - C: RaftTypeConfig, - N: RaftNetworkFactory, - LS: RaftLogStorage, +pub(in crate::raft) struct RaftInner +where C: RaftTypeConfig { pub(in crate::raft) id: C::NodeId, pub(in crate::raft) config: Arc, pub(in crate::raft) runtime_config: Arc, pub(in crate::raft) tick_handle: TickHandle, - pub(in crate::raft) tx_api: mpsc::UnboundedSender>, + pub(in crate::raft) tx_api: mpsc::UnboundedSender>, pub(in crate::raft) rx_metrics: watch::Receiver>, // TODO(xp): it does not need to be a async mutex. @@ -40,11 +35,8 @@ where pub(in crate::raft) core_state: Mutex>, } -impl RaftInner -where - C: RaftTypeConfig, - N: RaftNetworkFactory, - LS: RaftLogStorage, +impl RaftInner +where C: RaftTypeConfig { /// Send an [`ExternalCommand`] to RaftCore to execute in the `RaftCore` thread. /// diff --git a/openraft/src/raft/runtime_config_handle.rs b/openraft/src/raft/runtime_config_handle.rs index eb0e6ea43..4a29c78a9 100644 --- a/openraft/src/raft/runtime_config_handle.rs +++ b/openraft/src/raft/runtime_config_handle.rs @@ -3,30 +3,22 @@ use std::sync::atomic::Ordering; use crate::raft::RaftInner; -use crate::storage::RaftLogStorage; -use crate::RaftNetworkFactory; use crate::RaftTypeConfig; /// RuntimeConfigHandle is an interface to update runtime config. /// /// These config are mainly designed for testing purpose and special use cases. /// Usually you don't need to change runtime config. -pub struct RuntimeConfigHandle<'r, C, N, LS> -where - C: RaftTypeConfig, - N: RaftNetworkFactory, - LS: RaftLogStorage, +pub struct RuntimeConfigHandle<'r, C> +where C: RaftTypeConfig { - raft_inner: &'r RaftInner, + raft_inner: &'r RaftInner, } -impl<'r, C, N, LS> RuntimeConfigHandle<'r, C, N, LS> -where - C: RaftTypeConfig, - N: RaftNetworkFactory, - LS: RaftLogStorage, +impl<'r, C> RuntimeConfigHandle<'r, C> +where C: RaftTypeConfig { - pub(in crate::raft) fn new(raft_inner: &'r RaftInner) -> Self { + pub(in crate::raft) fn new(raft_inner: &'r RaftInner) -> Self { Self { raft_inner } } diff --git a/openraft/src/raft/trigger.rs b/openraft/src/raft/trigger.rs index ad388ba59..e8853d1a9 100644 --- a/openraft/src/raft/trigger.rs +++ b/openraft/src/raft/trigger.rs @@ -3,27 +3,19 @@ use crate::core::raft_msg::external_command::ExternalCommand; use crate::error::Fatal; use crate::raft::RaftInner; -use crate::storage::RaftLogStorage; -use crate::RaftNetworkFactory; use crate::RaftTypeConfig; /// Trigger is an interface to trigger an action to RaftCore by external caller. -pub struct Trigger<'r, C, N, LS> -where - C: RaftTypeConfig, - N: RaftNetworkFactory, - LS: RaftLogStorage, +pub struct Trigger<'r, C> +where C: RaftTypeConfig { - raft_inner: &'r RaftInner, + raft_inner: &'r RaftInner, } -impl<'r, C, N, LS> Trigger<'r, C, N, LS> -where - C: RaftTypeConfig, - N: RaftNetworkFactory, - LS: RaftLogStorage, +impl<'r, C> Trigger<'r, C> +where C: RaftTypeConfig { - pub(in crate::raft) fn new(raft_inner: &'r RaftInner) -> Self { + pub(in crate::raft) fn new(raft_inner: &'r RaftInner) -> Self { Self { raft_inner } } diff --git a/openraft/src/type_config.rs b/openraft/src/type_config.rs index 9436a9ae4..b7152ea97 100644 --- a/openraft/src/type_config.rs +++ b/openraft/src/type_config.rs @@ -67,3 +67,23 @@ pub trait RaftTypeConfig: /// Asynchronous runtime type. type AsyncRuntime: AsyncRuntime; } + +#[allow(dead_code)] +pub(crate) mod alias { + //! Type alias for types used in `RaftTypeConfig`. + + pub(crate) type DOf = ::D; + pub(crate) type ROf = ::R; + pub(crate) type NodeIdOf = ::NodeId; + pub(crate) type NodeOf = ::Node; + pub(crate) type EntryOf = ::Entry; + pub(crate) type SnapshotDataOf = ::SnapshotData; + pub(crate) type AsyncRuntimeOf = ::AsyncRuntime; + + pub(crate) type JoinErrorOf = as crate::AsyncRuntime>::JoinError; + pub(crate) type JoinHandleOf = as crate::AsyncRuntime>::JoinHandle; + pub(crate) type SleepOf = as crate::AsyncRuntime>::Sleep; + pub(crate) type InstantOf = as crate::AsyncRuntime>::Instant; + pub(crate) type TimeoutErrorOf = as crate::AsyncRuntime>::TimeoutError; + pub(crate) type TimeoutOf = as crate::AsyncRuntime>::Timeout; +} diff --git a/tests/tests/append_entries/t10_see_higher_vote.rs b/tests/tests/append_entries/t10_see_higher_vote.rs index 6143d6db5..157a09dac 100644 --- a/tests/tests/append_entries/t10_see_higher_vote.rs +++ b/tests/tests/append_entries/t10_see_higher_vote.rs @@ -78,7 +78,7 @@ async fn append_sees_higher_vote() -> Result<()> { .state(ServerState::Follower, "node-0 becomes follower due to a higher vote") .await?; - router.external_request(0, |st, _, _| { + router.external_request(0, |st| { assert_eq!(&Vote::new(10, 1), st.vote_ref(), "higher vote is stored"); }); } diff --git a/tests/tests/append_entries/t60_enable_heartbeat.rs b/tests/tests/append_entries/t60_enable_heartbeat.rs index 9e5b29e8f..e57fe2cf9 100644 --- a/tests/tests/append_entries/t60_enable_heartbeat.rs +++ b/tests/tests/append_entries/t60_enable_heartbeat.rs @@ -41,7 +41,7 @@ async fn enable_heartbeat() -> Result<()> { .await?; // leader lease is extended. - router.external_request(node_id, move |state, _store, _net| { + router.external_request(node_id, move |state| { assert!(state.vote_last_modified() > Some(now)); }); } diff --git a/tests/tests/append_entries/t61_heartbeat_reject_vote.rs b/tests/tests/append_entries/t61_heartbeat_reject_vote.rs index 2b2f90375..6e6121320 100644 --- a/tests/tests/append_entries/t61_heartbeat_reject_vote.rs +++ b/tests/tests/append_entries/t61_heartbeat_reject_vote.rs @@ -38,7 +38,7 @@ async fn heartbeat_reject_vote() -> Result<()> { { let m = vote_modified_time.clone(); - router.external_request(1, move |state, _store, _net| { + router.external_request(1, move |state| { let mut l = m.lock().unwrap(); *l = state.vote_last_modified(); assert!(state.vote_last_modified() > Some(now)); @@ -49,7 +49,7 @@ async fn heartbeat_reject_vote() -> Result<()> { let m = vote_modified_time.clone(); - router.external_request(1, move |state, _store, _net| { + router.external_request(1, move |state| { let l = m.lock().unwrap(); assert!(state.vote_last_modified() > Some(now)); assert!(state.vote_last_modified() > *l); diff --git a/tests/tests/fixtures/mod.rs b/tests/tests/fixtures/mod.rs index b54b6452b..89c280243 100644 --- a/tests/tests/fixtures/mod.rs +++ b/tests/tests/fixtures/mod.rs @@ -72,7 +72,7 @@ pub type MemLogStore = Adaptor>; pub type MemStateMachine = Adaptor>; /// A concrete Raft type used during testing. -pub type MemRaft = Raft; +pub type MemRaft = Raft; pub fn init_default_ut_tracing() { static START: Once = Once::new(); @@ -280,7 +280,7 @@ impl TypedRaftRouter { // (since they are awaited). #[allow(clippy::single_element_loop)] for node in [0] { - self.external_request(node, |s, _sto, _net| { + self.external_request(node, |s| { assert_eq!(s.server_state, ServerState::Learner); }); } @@ -637,9 +637,7 @@ impl TypedRaftRouter { } /// Send external request to the particular node. - pub fn external_request< - F: FnOnce(&RaftState, &mut MemLogStore, &mut TypedRaftRouter) + Send + 'static, - >( + pub fn external_request) + Send + 'static>( &self, target: MemNodeId, req: F, diff --git a/tests/tests/life_cycle/t10_initialization.rs b/tests/tests/life_cycle/t10_initialization.rs index 8c9079f40..5a9b0b43d 100644 --- a/tests/tests/life_cycle/t10_initialization.rs +++ b/tests/tests/life_cycle/t10_initialization.rs @@ -69,7 +69,7 @@ async fn initialization() -> anyhow::Result<()> { // before other requests in the Raft core API queue, which definitely are executed // (since they are awaited). for node in [0, 1, 2] { - router.external_request(node, |s, _sto, _net| { + router.external_request(node, |s| { assert_eq!(s.server_state, ServerState::Learner); }); } @@ -88,7 +88,7 @@ async fn initialization() -> anyhow::Result<()> { tracing::info!(log_index, "--- check membership state"); for node_id in [0, 1, 2] { - router.external_request(node_id, move |s, _sto, _net| { + router.external_request(node_id, move |s| { let want = EffectiveMembership::new( Some(LogId::new(CommittedLeaderId::new(0, 0), 0)), Membership::new(vec![btreeset! {0,1,2}], None), @@ -147,7 +147,7 @@ async fn initialization() -> anyhow::Result<()> { let mut follower_count = 0; for node in [0, 1, 2] { let (tx, rx) = oneshot::channel(); - router.external_request(node, |s, _sm, _net| tx.send(s.server_state).unwrap()); + router.external_request(node, |s| tx.send(s.server_state).unwrap()); match rx.await.unwrap() { ServerState::Leader => { assert!(!found_leader); @@ -176,7 +176,7 @@ async fn initialize_err_target_not_include_target() -> anyhow::Result<()> { router.new_raft_node(1).await; for node in [0, 1] { - router.external_request(node, |s, _sto, _net| { + router.external_request(node, |s| { assert_eq!(s.server_state, ServerState::Learner); }); } @@ -210,7 +210,7 @@ async fn initialize_err_not_allowed() -> anyhow::Result<()> { router.new_raft_node(0).await; for node in [0] { - router.external_request(node, |s, _sto, _net| { + router.external_request(node, |s| { assert_eq!(s.server_state, ServerState::Learner); }); } diff --git a/tests/tests/life_cycle/t11_shutdown.rs b/tests/tests/life_cycle/t11_shutdown.rs index 45f6ce669..7764cdaf9 100644 --- a/tests/tests/life_cycle/t11_shutdown.rs +++ b/tests/tests/life_cycle/t11_shutdown.rs @@ -55,7 +55,7 @@ async fn return_error_after_panic() -> Result<()> { tracing::info!(log_index, "--- panic the RaftCore"); { - router.external_request(0, |_s, _sto, _net| { + router.external_request(0, |_s| { panic!("foo"); }); } diff --git a/tests/tests/membership/t20_change_membership.rs b/tests/tests/membership/t20_change_membership.rs index 631813a85..577755f29 100644 --- a/tests/tests/membership/t20_change_membership.rs +++ b/tests/tests/membership/t20_change_membership.rs @@ -36,7 +36,7 @@ async fn update_membership_state() -> anyhow::Result<()> { for node_id in [0, 1, 2, 3, 4] { router.wait(&node_id, timeout()).log(Some(log_index), "change-membership log applied").await?; - router.external_request(node_id, move |st, _, _| { + router.external_request(node_id, move |st| { tracing::debug!("--- got state: {:?}", st); assert_eq!(st.membership_state.committed().log_id().index(), Some(log_index)); assert_eq!(st.membership_state.effective().log_id().index(), Some(log_index));