use std::future::Future;
use std::time::Duration;
use openraft_macros::add_async_trait;
use crate::error::Fatal;
use crate::error::RPCError;
use crate::error::RaftError;
use crate::error::ReplicationClosed;
use crate::error::StreamingError;
use crate::network::rpc_option::RPCOption;
use crate::network::Backoff;
use crate::raft::AppendEntriesRequest;
use crate::raft::AppendEntriesResponse;
use crate::raft::SnapshotResponse;
use crate::raft::VoteRequest;
use crate::raft::VoteResponse;
use crate::OptionalSend;
use crate::OptionalSync;
use crate::RaftTypeConfig;
use crate::Snapshot;
use crate::Vote;
#[add_async_trait]
pub trait RaftNetwork<C>: OptionalSend + OptionalSync + 'static
where C: RaftTypeConfig
{
async fn append_entries(
&mut self,
rpc: AppendEntriesRequest<C>,
option: RPCOption,
) -> Result<AppendEntriesResponse<C::NodeId>, RPCError<C::NodeId, C::Node, RaftError<C::NodeId>>>;
#[cfg(feature = "generic-snapshot-data")]
#[deprecated(
since = "0.9.0",
note = "with `generic-snapshot-data` enabled, use `full_snapshot()` instead to send full snapshot"
)]
async fn install_snapshot(
&mut self,
_rpc: crate::raft::InstallSnapshotRequest<C>,
_option: RPCOption,
) -> Result<
crate::raft::InstallSnapshotResponse<C::NodeId>,
RPCError<C::NodeId, C::Node, RaftError<C::NodeId, crate::error::InstallSnapshotError>>,
> {
unimplemented!()
}
#[cfg(not(feature = "generic-snapshot-data"))]
async fn install_snapshot(
&mut self,
_rpc: crate::raft::InstallSnapshotRequest<C>,
_option: RPCOption,
) -> Result<
crate::raft::InstallSnapshotResponse<C::NodeId>,
RPCError<C::NodeId, C::Node, RaftError<C::NodeId, crate::error::InstallSnapshotError>>,
>;
async fn vote(
&mut self,
rpc: VoteRequest<C::NodeId>,
option: RPCOption,
) -> Result<VoteResponse<C::NodeId>, RPCError<C::NodeId, C::Node, RaftError<C::NodeId>>>;
#[cfg(feature = "generic-snapshot-data")]
async fn full_snapshot(
&mut self,
vote: Vote<C::NodeId>,
snapshot: Snapshot<C>,
cancel: impl Future<Output = ReplicationClosed> + OptionalSend + 'static,
option: RPCOption,
) -> Result<SnapshotResponse<C::NodeId>, StreamingError<C, Fatal<C::NodeId>>>;
#[cfg(not(feature = "generic-snapshot-data"))]
async fn full_snapshot(
&mut self,
vote: Vote<C::NodeId>,
snapshot: Snapshot<C>,
cancel: impl Future<Output = ReplicationClosed> + OptionalSend + 'static,
option: RPCOption,
) -> Result<SnapshotResponse<C::NodeId>, StreamingError<C, Fatal<C::NodeId>>> {
use crate::network::snapshot_transport::Chunked;
use crate::network::snapshot_transport::SnapshotTransport;
let resp = Chunked::send_snapshot(self, vote, snapshot, cancel, option).await?;
Ok(resp)
}
fn backoff(&self) -> Backoff {
Backoff::new(std::iter::repeat(Duration::from_millis(500)))
}
}