cuprate/p2p/p2p-core/src/client.rs

231 lines
7.5 KiB
Rust
Raw Normal View History

2024-01-13 00:07:35 +00:00
use std::{
fmt::{Debug, Display, Formatter},
sync::Arc,
task::{ready, Context, Poll},
2024-01-13 00:07:35 +00:00
};
use futures::channel::oneshot;
use tokio::{
sync::{mpsc, OwnedSemaphorePermit, Semaphore},
task::JoinHandle,
};
use tokio_util::sync::PollSemaphore;
P2P: Block downloader (#132) * impl async buffer * clippy * p2p changes * clippy * a few more docs * init cuprate-p2p * remove some unrelated code and add some docs * start documenting client_pool.rs * add more docs * typo * fix docs * use JoinSet in connection maintainer * small changes * add peer sync state svc * add broadcast svc * add more docs * add some tests * add a test * fix merge * add another test * unify PeerDisconnectFut and add more docs * start network init * add an inbound connection server * remove crate doc for now * fix address book docs * fix leak in client pool * correct comment * fix merge + add some docs * review comments * init block downloader * fix doc * initial chain search * add chain_tracker * move block downloader to struct * spawn task whe getting blocks * check for free peers and handle batch response * add test bin * working block downloader * dynamic batch sizes * dandelion_tower -> dandelion-tower * fix async-buffer builds * check if incoming peers are banned * add interface methods * update docs * use a JoinSet for background network tasks * dynamic batch size changes * Keep a longer of queue of blocks to get * more checks on incoming data * fix merge * fix imports * add more docs * add some limits on messages * keep peers that dont have the current need data * fix clippy * fix .lock * fix stopping the block downloader * clean up API and add more docs * tracing + bug fixes * fix panic * doc changes * remove test_init * remove spammy log * fix previous merge * add a test * fix test * remove test unwrap * order imports correctly * clean up test * add a timeout * fix tests * review fixes * make `BlockDownloader` pub * make `initial_chain_search` pub * make `block_downloader` private * Apply suggestions from code review Co-authored-by: hinto-janai <hinto.janai@protonmail.com> * split some sections into separate modules * split chain requests * sort imports * check previous ID is correct * fix typos * Apply suggestions from code review Co-authored-by: hinto-janai <hinto.janai@protonmail.com> --------- Co-authored-by: hinto-janai <hinto.janai@protonmail.com>
2024-06-22 00:29:40 +00:00
use tower::{Service, ServiceExt};
use tracing::Instrument;
2024-01-13 00:07:35 +00:00
2024-01-22 01:56:34 +00:00
use cuprate_helper::asynch::InfallibleOneshotReceiver;
use cuprate_pruning::PruningSeed;
2024-01-13 00:07:35 +00:00
use crate::{
P2P: Block downloader (#132) * impl async buffer * clippy * p2p changes * clippy * a few more docs * init cuprate-p2p * remove some unrelated code and add some docs * start documenting client_pool.rs * add more docs * typo * fix docs * use JoinSet in connection maintainer * small changes * add peer sync state svc * add broadcast svc * add more docs * add some tests * add a test * fix merge * add another test * unify PeerDisconnectFut and add more docs * start network init * add an inbound connection server * remove crate doc for now * fix address book docs * fix leak in client pool * correct comment * fix merge + add some docs * review comments * init block downloader * fix doc * initial chain search * add chain_tracker * move block downloader to struct * spawn task whe getting blocks * check for free peers and handle batch response * add test bin * working block downloader * dynamic batch sizes * dandelion_tower -> dandelion-tower * fix async-buffer builds * check if incoming peers are banned * add interface methods * update docs * use a JoinSet for background network tasks * dynamic batch size changes * Keep a longer of queue of blocks to get * more checks on incoming data * fix merge * fix imports * add more docs * add some limits on messages * keep peers that dont have the current need data * fix clippy * fix .lock * fix stopping the block downloader * clean up API and add more docs * tracing + bug fixes * fix panic * doc changes * remove test_init * remove spammy log * fix previous merge * add a test * fix test * remove test unwrap * order imports correctly * clean up test * add a timeout * fix tests * review fixes * make `BlockDownloader` pub * make `initial_chain_search` pub * make `block_downloader` private * Apply suggestions from code review Co-authored-by: hinto-janai <hinto.janai@protonmail.com> * split some sections into separate modules * split chain requests * sort imports * check previous ID is correct * fix typos * Apply suggestions from code review Co-authored-by: hinto-janai <hinto.janai@protonmail.com> --------- Co-authored-by: hinto-janai <hinto.janai@protonmail.com>
2024-06-22 00:29:40 +00:00
handles::{ConnectionGuard, ConnectionHandle},
ConnectionDirection, NetworkZone, PeerError, PeerRequest, PeerResponse, SharedError,
2024-01-13 00:07:35 +00:00
};
mod connection;
mod connector;
pub mod handshaker;
mod request_handler;
mod timeout_monitor;
pub use connector::{ConnectRequest, Connector};
pub use handshaker::{DoHandshakeRequest, HandshakeError, HandshakerBuilder};
/// An internal identifier for a given peer, will be their address if known
/// or a random u128 if not.
#[derive(Debug, Copy, Clone, PartialEq, Eq, Hash)]
pub enum InternalPeerID<A> {
/// A known address.
KnownAddr(A),
/// An unknown address (probably an inbound anonymity network connection).
Unknown(u128),
}
2024-01-13 00:07:35 +00:00
impl<A: Display> Display for InternalPeerID<A> {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
match self {
InternalPeerID::KnownAddr(addr) => addr.fmt(f),
InternalPeerID::Unknown(id) => f.write_str(&format!("Unknown, ID: {id}")),
2024-01-13 00:07:35 +00:00
}
}
}
/// Information on a connected peer.
#[derive(Debug, Clone)]
pub struct PeerInformation<A> {
/// The internal peer ID of this peer.
pub id: InternalPeerID<A>,
/// The [`ConnectionHandle`] for this peer, allows banning this peer and checking if it is still
/// alive.
pub handle: ConnectionHandle,
/// The direction of this connection (inbound|outbound).
pub direction: ConnectionDirection,
/// The peers pruning seed.
pub pruning_seed: PruningSeed,
}
/// This represents a connection to a peer.
///
/// It allows sending requests to the peer, but does only does minimal checks that the data returned
/// is the data asked for, i.e. for a certain request the only thing checked will be that the response
/// is the correct response for that request, not that the response contains the correct data.
2024-01-13 00:07:35 +00:00
pub struct Client<Z: NetworkZone> {
/// Information on the connected peer.
pub info: PeerInformation<Z::Addr>,
2024-01-13 00:07:35 +00:00
/// The channel to the [`Connection`](connection::Connection) task.
connection_tx: mpsc::Sender<connection::ConnectionTaskRequest>,
/// The [`JoinHandle`] of the spawned connection task.
2024-01-13 00:07:35 +00:00
connection_handle: JoinHandle<()>,
/// The [`JoinHandle`] of the spawned timeout monitor task.
timeout_handle: JoinHandle<Result<(), tower::BoxError>>,
2024-01-13 00:07:35 +00:00
/// The semaphore that limits the requests sent to the peer.
semaphore: PollSemaphore,
/// A permit for the semaphore, will be [`Some`] after `poll_ready` returns ready.
permit: Option<OwnedSemaphorePermit>,
/// The error slot shared between the [`Client`] and [`Connection`](connection::Connection).
2024-01-13 00:07:35 +00:00
error: SharedError<PeerError>,
}
impl<Z: NetworkZone> Client<Z> {
/// Creates a new [`Client`].
pub(crate) fn new(
info: PeerInformation<Z::Addr>,
2024-01-13 00:07:35 +00:00
connection_tx: mpsc::Sender<connection::ConnectionTaskRequest>,
connection_handle: JoinHandle<()>,
timeout_handle: JoinHandle<Result<(), tower::BoxError>>,
semaphore: Arc<Semaphore>,
2024-01-13 00:07:35 +00:00
error: SharedError<PeerError>,
) -> Self {
Self {
info,
connection_tx,
timeout_handle,
semaphore: PollSemaphore::new(semaphore),
permit: None,
2024-01-13 00:07:35 +00:00
connection_handle,
error,
}
}
/// Internal function to set an error on the [`SharedError`].
2024-01-13 00:07:35 +00:00
fn set_err(&self, err: PeerError) -> tower::BoxError {
let err_str = err.to_string();
match self.error.try_insert_err(err) {
Ok(_) => err_str,
Err(e) => e.to_string(),
}
.into()
}
}
impl<Z: NetworkZone> Service<PeerRequest> for Client<Z> {
type Response = PeerResponse;
type Error = tower::BoxError;
type Future = InfallibleOneshotReceiver<Result<Self::Response, Self::Error>>;
fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
if let Some(err) = self.error.try_get_err() {
return Poll::Ready(Err(err.to_string().into()));
}
if self.connection_handle.is_finished() || self.timeout_handle.is_finished() {
2024-01-13 00:07:35 +00:00
let err = self.set_err(PeerError::ClientChannelClosed);
return Poll::Ready(Err(err));
}
if self.permit.is_some() {
return Poll::Ready(Ok(()));
}
let permit = ready!(self.semaphore.poll_acquire(cx))
.expect("Client semaphore should not be closed!");
self.permit = Some(permit);
Poll::Ready(Ok(()))
2024-01-13 00:07:35 +00:00
}
fn call(&mut self, request: PeerRequest) -> Self::Future {
let permit = self
.permit
.take()
.expect("poll_ready did not return ready before call to call");
2024-01-13 00:07:35 +00:00
let (tx, rx) = oneshot::channel();
let req = connection::ConnectionTaskRequest {
response_channel: tx,
request,
permit: Some(permit),
2024-01-13 00:07:35 +00:00
};
P2P: Block downloader (#132) * impl async buffer * clippy * p2p changes * clippy * a few more docs * init cuprate-p2p * remove some unrelated code and add some docs * start documenting client_pool.rs * add more docs * typo * fix docs * use JoinSet in connection maintainer * small changes * add peer sync state svc * add broadcast svc * add more docs * add some tests * add a test * fix merge * add another test * unify PeerDisconnectFut and add more docs * start network init * add an inbound connection server * remove crate doc for now * fix address book docs * fix leak in client pool * correct comment * fix merge + add some docs * review comments * init block downloader * fix doc * initial chain search * add chain_tracker * move block downloader to struct * spawn task whe getting blocks * check for free peers and handle batch response * add test bin * working block downloader * dynamic batch sizes * dandelion_tower -> dandelion-tower * fix async-buffer builds * check if incoming peers are banned * add interface methods * update docs * use a JoinSet for background network tasks * dynamic batch size changes * Keep a longer of queue of blocks to get * more checks on incoming data * fix merge * fix imports * add more docs * add some limits on messages * keep peers that dont have the current need data * fix clippy * fix .lock * fix stopping the block downloader * clean up API and add more docs * tracing + bug fixes * fix panic * doc changes * remove test_init * remove spammy log * fix previous merge * add a test * fix test * remove test unwrap * order imports correctly * clean up test * add a timeout * fix tests * review fixes * make `BlockDownloader` pub * make `initial_chain_search` pub * make `block_downloader` private * Apply suggestions from code review Co-authored-by: hinto-janai <hinto.janai@protonmail.com> * split some sections into separate modules * split chain requests * sort imports * check previous ID is correct * fix typos * Apply suggestions from code review Co-authored-by: hinto-janai <hinto.janai@protonmail.com> --------- Co-authored-by: hinto-janai <hinto.janai@protonmail.com>
2024-06-22 00:29:40 +00:00
if let Err(e) = self.connection_tx.try_send(req) {
// The connection task could have closed between a call to `poll_ready` and the call to
// `call`, which means if we don't handle the error here the receiver would panic.
use mpsc::error::TrySendError;
match e {
TrySendError::Closed(req) | TrySendError::Full(req) => {
self.set_err(PeerError::ClientChannelClosed);
let _ = req
.response_channel
.send(Err(PeerError::ClientChannelClosed.into()));
}
}
}
2024-01-13 00:07:35 +00:00
rx.into()
}
}
P2P: Block downloader (#132) * impl async buffer * clippy * p2p changes * clippy * a few more docs * init cuprate-p2p * remove some unrelated code and add some docs * start documenting client_pool.rs * add more docs * typo * fix docs * use JoinSet in connection maintainer * small changes * add peer sync state svc * add broadcast svc * add more docs * add some tests * add a test * fix merge * add another test * unify PeerDisconnectFut and add more docs * start network init * add an inbound connection server * remove crate doc for now * fix address book docs * fix leak in client pool * correct comment * fix merge + add some docs * review comments * init block downloader * fix doc * initial chain search * add chain_tracker * move block downloader to struct * spawn task whe getting blocks * check for free peers and handle batch response * add test bin * working block downloader * dynamic batch sizes * dandelion_tower -> dandelion-tower * fix async-buffer builds * check if incoming peers are banned * add interface methods * update docs * use a JoinSet for background network tasks * dynamic batch size changes * Keep a longer of queue of blocks to get * more checks on incoming data * fix merge * fix imports * add more docs * add some limits on messages * keep peers that dont have the current need data * fix clippy * fix .lock * fix stopping the block downloader * clean up API and add more docs * tracing + bug fixes * fix panic * doc changes * remove test_init * remove spammy log * fix previous merge * add a test * fix test * remove test unwrap * order imports correctly * clean up test * add a timeout * fix tests * review fixes * make `BlockDownloader` pub * make `initial_chain_search` pub * make `block_downloader` private * Apply suggestions from code review Co-authored-by: hinto-janai <hinto.janai@protonmail.com> * split some sections into separate modules * split chain requests * sort imports * check previous ID is correct * fix typos * Apply suggestions from code review Co-authored-by: hinto-janai <hinto.janai@protonmail.com> --------- Co-authored-by: hinto-janai <hinto.janai@protonmail.com>
2024-06-22 00:29:40 +00:00
/// Creates a mock [`Client`] for testing purposes.
///
/// `request_handler` will be used to handle requests sent to the [`Client`]
pub fn mock_client<Z: NetworkZone, S>(
info: PeerInformation<Z::Addr>,
connection_guard: ConnectionGuard,
mut request_handler: S,
) -> Client<Z>
where
S: Service<PeerRequest, Response = PeerResponse, Error = tower::BoxError> + Send + 'static,
S::Future: Send + 'static,
P2P: Block downloader (#132) * impl async buffer * clippy * p2p changes * clippy * a few more docs * init cuprate-p2p * remove some unrelated code and add some docs * start documenting client_pool.rs * add more docs * typo * fix docs * use JoinSet in connection maintainer * small changes * add peer sync state svc * add broadcast svc * add more docs * add some tests * add a test * fix merge * add another test * unify PeerDisconnectFut and add more docs * start network init * add an inbound connection server * remove crate doc for now * fix address book docs * fix leak in client pool * correct comment * fix merge + add some docs * review comments * init block downloader * fix doc * initial chain search * add chain_tracker * move block downloader to struct * spawn task whe getting blocks * check for free peers and handle batch response * add test bin * working block downloader * dynamic batch sizes * dandelion_tower -> dandelion-tower * fix async-buffer builds * check if incoming peers are banned * add interface methods * update docs * use a JoinSet for background network tasks * dynamic batch size changes * Keep a longer of queue of blocks to get * more checks on incoming data * fix merge * fix imports * add more docs * add some limits on messages * keep peers that dont have the current need data * fix clippy * fix .lock * fix stopping the block downloader * clean up API and add more docs * tracing + bug fixes * fix panic * doc changes * remove test_init * remove spammy log * fix previous merge * add a test * fix test * remove test unwrap * order imports correctly * clean up test * add a timeout * fix tests * review fixes * make `BlockDownloader` pub * make `initial_chain_search` pub * make `block_downloader` private * Apply suggestions from code review Co-authored-by: hinto-janai <hinto.janai@protonmail.com> * split some sections into separate modules * split chain requests * sort imports * check previous ID is correct * fix typos * Apply suggestions from code review Co-authored-by: hinto-janai <hinto.janai@protonmail.com> --------- Co-authored-by: hinto-janai <hinto.janai@protonmail.com>
2024-06-22 00:29:40 +00:00
{
let (tx, mut rx) = mpsc::channel(1);
let task_span = tracing::error_span!("mock_connection", addr = %info.id);
let task_handle = tokio::spawn(
async move {
let _guard = connection_guard;
loop {
let Some(req): Option<connection::ConnectionTaskRequest> = rx.recv().await else {
tracing::debug!("Channel closed, closing mock connection");
return;
};
tracing::debug!("Received new request: {:?}", req.request.id());
let res = request_handler
.ready()
.await
.unwrap()
.call(req.request)
.await
.unwrap();
tracing::debug!("Sending back response");
let _ = req.response_channel.send(Ok(res));
}
}
.instrument(task_span),
);
let timeout_task = tokio::spawn(futures::future::pending());
let semaphore = Arc::new(Semaphore::new(1));
let error_slot = SharedError::new();
Client::new(info, tx, task_handle, timeout_task, semaphore, error_slot)
}