Wrap the entire Libp2p object in an Arc

Makes `Clone` calls significantly cheaper as now only the outer Arc is cloned
(the inner ones have been removed). Also wraps uses of Serai in an Arc as we
shouldn't actually need/want multiple caller connection pools.
This commit is contained in:
Luke Parker
2025-01-10 01:20:26 -05:00
parent 23122712cb
commit 2a3eaf4d7e
6 changed files with 59 additions and 37 deletions

View File

@@ -1,5 +1,5 @@
use core::future::Future; use core::future::Future;
use std::collections::HashMap; use std::{sync::Arc, collections::HashMap};
use serai_client::{ use serai_client::{
primitives::{SeraiAddress, Amount}, primitives::{SeraiAddress, Amount},
@@ -57,7 +57,7 @@ async fn block_has_events_justifying_a_cosign(
/// A task to determine which blocks we should intend to cosign. /// A task to determine which blocks we should intend to cosign.
pub(crate) struct CosignIntendTask<D: Db> { pub(crate) struct CosignIntendTask<D: Db> {
pub(crate) db: D, pub(crate) db: D,
pub(crate) serai: Serai, pub(crate) serai: Arc<Serai>,
} }
impl<D: Db> ContinuallyRan for CosignIntendTask<D> { impl<D: Db> ContinuallyRan for CosignIntendTask<D> {

View File

@@ -3,7 +3,7 @@
#![deny(missing_docs)] #![deny(missing_docs)]
use core::{fmt::Debug, future::Future}; use core::{fmt::Debug, future::Future};
use std::collections::HashMap; use std::{sync::Arc, collections::HashMap};
use blake2::{Digest, Blake2s256}; use blake2::{Digest, Blake2s256};
@@ -240,7 +240,7 @@ impl<D: Db> Cosigning<D> {
/// only used once at any given time. /// only used once at any given time.
pub fn spawn<R: RequestNotableCosigns>( pub fn spawn<R: RequestNotableCosigns>(
db: D, db: D,
serai: Serai, serai: Arc<Serai>,
request: R, request: R,
tasks_to_run_upon_cosigning: Vec<TaskHandle>, tasks_to_run_upon_cosigning: Vec<TaskHandle>,
) -> Self { ) -> Self {
@@ -334,10 +334,9 @@ impl<D: Db> Cosigning<D> {
} }
} }
/// Intake a cosign from the Serai network. /// Intake a cosign.
/// ///
/// - Returns Err(_) if there was an error trying to validate the cosign and it should be retired /// - Returns Err(_) if there was an error trying to validate the cosign.
/// later.
/// - Returns Ok(true) if the cosign was successfully handled or could not be handled at this /// - Returns Ok(true) if the cosign was successfully handled or could not be handled at this
/// time. /// time.
/// - Returns Ok(false) if the cosign was invalid. /// - Returns Ok(false) if the cosign was invalid.

View File

@@ -1,5 +1,5 @@
use core::future::Future; use core::future::Future;
use std::collections::HashSet; use std::{sync::Arc, collections::HashSet};
use rand_core::{RngCore, OsRng}; use rand_core::{RngCore, OsRng};
@@ -29,14 +29,18 @@ const TARGET_PEERS_PER_NETWORK: usize = 5;
// TODO const TARGET_DIALED_PEERS_PER_NETWORK: usize = 3; // TODO const TARGET_DIALED_PEERS_PER_NETWORK: usize = 3;
pub(crate) struct DialTask { pub(crate) struct DialTask {
serai: Serai, serai: Arc<Serai>,
validators: Validators, validators: Validators,
peers: Peers, peers: Peers,
to_dial: mpsc::UnboundedSender<DialOpts>, to_dial: mpsc::UnboundedSender<DialOpts>,
} }
impl DialTask { impl DialTask {
pub(crate) fn new(serai: Serai, peers: Peers, to_dial: mpsc::UnboundedSender<DialOpts>) -> Self { pub(crate) fn new(
serai: Arc<Serai>,
peers: Peers,
to_dial: mpsc::UnboundedSender<DialOpts>,
) -> Self {
DialTask { serai: serai.clone(), validators: Validators::new(serai).0, peers, to_dial } DialTask { serai: serai.clone(), validators: Validators::new(serai).0, peers, to_dial }
} }
} }

View File

@@ -131,33 +131,35 @@ struct Behavior {
gossip: gossip::Behavior, gossip: gossip::Behavior,
} }
/// The libp2p-backed P2P implementation.
///
/// The P2p trait implementation does not support backpressure and is expected to be fully
/// utilized. Failure to poll the entire API will cause unbounded memory growth.
#[allow(clippy::type_complexity)] #[allow(clippy::type_complexity)]
#[derive(Clone)] struct Libp2pInner {
pub struct Libp2p {
peers: Peers, peers: Peers,
gossip: mpsc::UnboundedSender<Message>, gossip: mpsc::UnboundedSender<Message>,
outbound_requests: mpsc::UnboundedSender<(PeerId, Request, oneshot::Sender<Response>)>, outbound_requests: mpsc::UnboundedSender<(PeerId, Request, oneshot::Sender<Response>)>,
tributary_gossip: Arc<Mutex<mpsc::UnboundedReceiver<([u8; 32], Vec<u8>)>>>, tributary_gossip: Mutex<mpsc::UnboundedReceiver<([u8; 32], Vec<u8>)>>,
signed_cosigns: Arc<Mutex<mpsc::UnboundedReceiver<SignedCosign>>>, signed_cosigns: Mutex<mpsc::UnboundedReceiver<SignedCosign>>,
signed_cosigns_send: mpsc::UnboundedSender<SignedCosign>, signed_cosigns_send: mpsc::UnboundedSender<SignedCosign>,
heartbeat_requests: Arc<Mutex<mpsc::UnboundedReceiver<(RequestId, ValidatorSet, [u8; 32])>>>, heartbeat_requests: Mutex<mpsc::UnboundedReceiver<(RequestId, ValidatorSet, [u8; 32])>>,
notable_cosign_requests: Arc<Mutex<mpsc::UnboundedReceiver<(RequestId, [u8; 32])>>>, notable_cosign_requests: Mutex<mpsc::UnboundedReceiver<(RequestId, [u8; 32])>>,
inbound_request_responses: mpsc::UnboundedSender<(RequestId, Response)>, inbound_request_responses: mpsc::UnboundedSender<(RequestId, Response)>,
} }
/// The libp2p-backed P2P implementation.
///
/// The P2p trait implementation does not support backpressure and is expected to be fully
/// utilized. Failure to poll the entire API will cause unbounded memory growth.
#[derive(Clone)]
pub struct Libp2p(Arc<Libp2pInner>);
impl Libp2p { impl Libp2p {
/// Create a new libp2p-backed P2P instance. /// Create a new libp2p-backed P2P instance.
/// ///
/// This will spawn all of the internal tasks necessary for functioning. /// This will spawn all of the internal tasks necessary for functioning.
pub fn new(serai_key: &Zeroizing<Keypair>, serai: Serai) -> Libp2p { pub fn new(serai_key: &Zeroizing<Keypair>, serai: Arc<Serai>) -> Libp2p {
// Define the object we track peers with // Define the object we track peers with
let peers = Peers { peers: Arc::new(RwLock::new(HashMap::new())) }; let peers = Peers { peers: Arc::new(RwLock::new(HashMap::new())) };
@@ -239,21 +241,21 @@ impl Libp2p {
inbound_request_responses_recv, inbound_request_responses_recv,
); );
Libp2p { Libp2p(Arc::new(Libp2pInner {
peers, peers,
gossip: gossip_send, gossip: gossip_send,
outbound_requests: outbound_requests_send, outbound_requests: outbound_requests_send,
tributary_gossip: Arc::new(Mutex::new(tributary_gossip_recv)), tributary_gossip: Mutex::new(tributary_gossip_recv),
signed_cosigns: Arc::new(Mutex::new(signed_cosigns_recv)), signed_cosigns: Mutex::new(signed_cosigns_recv),
signed_cosigns_send, signed_cosigns_send,
heartbeat_requests: Arc::new(Mutex::new(heartbeat_requests_recv)), heartbeat_requests: Mutex::new(heartbeat_requests_recv),
notable_cosign_requests: Arc::new(Mutex::new(notable_cosign_requests_recv)), notable_cosign_requests: Mutex::new(notable_cosign_requests_recv),
inbound_request_responses: inbound_request_responses_send, inbound_request_responses: inbound_request_responses_send,
} }))
} }
} }
@@ -261,6 +263,7 @@ impl tributary::P2p for Libp2p {
fn broadcast(&self, tributary: [u8; 32], message: Vec<u8>) -> impl Send + Future<Output = ()> { fn broadcast(&self, tributary: [u8; 32], message: Vec<u8>) -> impl Send + Future<Output = ()> {
async move { async move {
self self
.0
.gossip .gossip
.send(Message::Tributary { tributary, message }) .send(Message::Tributary { tributary, message })
.expect("gossip recv channel was dropped?"); .expect("gossip recv channel was dropped?");
@@ -281,7 +284,7 @@ impl serai_cosign::RequestNotableCosigns for Libp2p {
let request = Request::NotableCosigns { global_session }; let request = Request::NotableCosigns { global_session };
let peers = self.peers.peers.read().await.clone(); let peers = self.0.peers.peers.read().await.clone();
// HashSet of all peers // HashSet of all peers
let peers = peers.into_values().flat_map(<_>::into_iter).collect::<HashSet<_>>(); let peers = peers.into_values().flat_map(<_>::into_iter).collect::<HashSet<_>>();
// Vec of all peers // Vec of all peers
@@ -297,6 +300,7 @@ impl serai_cosign::RequestNotableCosigns for Libp2p {
let (sender, receiver) = oneshot::channel(); let (sender, receiver) = oneshot::channel();
self self
.0
.outbound_requests .outbound_requests
.send((peer, request, sender)) .send((peer, request, sender))
.expect("outbound requests recv channel was dropped?"); .expect("outbound requests recv channel was dropped?");
@@ -310,6 +314,7 @@ impl serai_cosign::RequestNotableCosigns for Libp2p {
{ {
for cosign in cosigns { for cosign in cosigns {
self self
.0
.signed_cosigns_send .signed_cosigns_send
.send(cosign) .send(cosign)
.expect("signed_cosigns recv in this object was dropped?"); .expect("signed_cosigns recv in this object was dropped?");
@@ -327,22 +332,29 @@ impl serai_coordinator_p2p::P2p for Libp2p {
fn peers(&self, network: NetworkId) -> impl Send + Future<Output = Vec<Self::Peer<'_>>> { fn peers(&self, network: NetworkId) -> impl Send + Future<Output = Vec<Self::Peer<'_>>> {
async move { async move {
let Some(peer_ids) = self.peers.peers.read().await.get(&network).cloned() else { let Some(peer_ids) = self.0.peers.peers.read().await.get(&network).cloned() else {
return vec![]; return vec![];
}; };
let mut res = vec![]; let mut res = vec![];
for id in peer_ids { for id in peer_ids {
res.push(Peer { outbound_requests: &self.outbound_requests, id }); res.push(Peer { outbound_requests: &self.0.outbound_requests, id });
} }
res res
} }
} }
fn publish_cosign(&self, cosign: SignedCosign) -> impl Send + Future<Output = ()> {
async move {
self.0.gossip.send(Message::Cosign(cosign)).expect("gossip recv channel was dropped?");
}
}
fn heartbeat( fn heartbeat(
&self, &self,
) -> impl Send + Future<Output = (Heartbeat, oneshot::Sender<Vec<TributaryBlockWithCommit>>)> { ) -> impl Send + Future<Output = (Heartbeat, oneshot::Sender<Vec<TributaryBlockWithCommit>>)> {
async move { async move {
let (request_id, set, latest_block_hash) = self let (request_id, set, latest_block_hash) = self
.0
.heartbeat_requests .heartbeat_requests
.lock() .lock()
.await .await
@@ -351,7 +363,7 @@ impl serai_coordinator_p2p::P2p for Libp2p {
.expect("heartbeat_requests_send was dropped?"); .expect("heartbeat_requests_send was dropped?");
let (sender, receiver) = oneshot::channel(); let (sender, receiver) = oneshot::channel();
tokio::spawn({ tokio::spawn({
let respond = self.inbound_request_responses.clone(); let respond = self.0.inbound_request_responses.clone();
async move { async move {
// The swarm task expects us to respond to every request. If the caller drops this // The swarm task expects us to respond to every request. If the caller drops this
// channel, we'll receive `Err` and respond with `vec![]`, safely satisfying that bound // channel, we'll receive `Err` and respond with `vec![]`, safely satisfying that bound
@@ -375,6 +387,7 @@ impl serai_coordinator_p2p::P2p for Libp2p {
) -> impl Send + Future<Output = ([u8; 32], oneshot::Sender<Vec<SignedCosign>>)> { ) -> impl Send + Future<Output = ([u8; 32], oneshot::Sender<Vec<SignedCosign>>)> {
async move { async move {
let (request_id, global_session) = self let (request_id, global_session) = self
.0
.notable_cosign_requests .notable_cosign_requests
.lock() .lock()
.await .await
@@ -383,7 +396,7 @@ impl serai_coordinator_p2p::P2p for Libp2p {
.expect("notable_cosign_requests_send was dropped?"); .expect("notable_cosign_requests_send was dropped?");
let (sender, receiver) = oneshot::channel(); let (sender, receiver) = oneshot::channel();
tokio::spawn({ tokio::spawn({
let respond = self.inbound_request_responses.clone(); let respond = self.0.inbound_request_responses.clone();
async move { async move {
let response = if let Ok(notable_cosigns) = receiver.await { let response = if let Ok(notable_cosigns) = receiver.await {
Response::NotableCosigns(notable_cosigns) Response::NotableCosigns(notable_cosigns)
@@ -401,13 +414,14 @@ impl serai_coordinator_p2p::P2p for Libp2p {
fn tributary_message(&self) -> impl Send + Future<Output = ([u8; 32], Vec<u8>)> { fn tributary_message(&self) -> impl Send + Future<Output = ([u8; 32], Vec<u8>)> {
async move { async move {
self.tributary_gossip.lock().await.recv().await.expect("tributary_gossip send was dropped?") self.0.tributary_gossip.lock().await.recv().await.expect("tributary_gossip send was dropped?")
} }
} }
fn cosign(&self) -> impl Send + Future<Output = SignedCosign> { fn cosign(&self) -> impl Send + Future<Output = SignedCosign> {
async move { async move {
self self
.0
.signed_cosigns .signed_cosigns
.lock() .lock()
.await .await

View File

@@ -21,7 +21,7 @@ pub(crate) struct Changes {
} }
pub(crate) struct Validators { pub(crate) struct Validators {
serai: Serai, serai: Arc<Serai>,
// A cache for which session we're populated with the validators of // A cache for which session we're populated with the validators of
sessions: HashMap<NetworkId, Session>, sessions: HashMap<NetworkId, Session>,
@@ -35,7 +35,7 @@ pub(crate) struct Validators {
} }
impl Validators { impl Validators {
pub(crate) fn new(serai: Serai) -> (Self, mpsc::UnboundedReceiver<Changes>) { pub(crate) fn new(serai: Arc<Serai>) -> (Self, mpsc::UnboundedReceiver<Changes>) {
let (send, recv) = mpsc::unbounded_channel(); let (send, recv) = mpsc::unbounded_channel();
let validators = Validators { let validators = Validators {
serai, serai,
@@ -148,7 +148,7 @@ impl Validators {
/// Update the view of the validators. /// Update the view of the validators.
pub(crate) async fn update(&mut self) -> Result<(), String> { pub(crate) async fn update(&mut self) -> Result<(), String> {
let session_changes = Self::session_changes(&self.serai, &self.sessions).await?; let session_changes = Self::session_changes(&*self.serai, &self.sessions).await?;
self.incorporate_session_changes(session_changes); self.incorporate_session_changes(session_changes);
Ok(()) Ok(())
} }
@@ -174,7 +174,9 @@ impl UpdateValidatorsTask {
/// Spawn a new instance of the UpdateValidatorsTask. /// Spawn a new instance of the UpdateValidatorsTask.
/// ///
/// This returns a reference to the Validators it updates after spawning itself. /// This returns a reference to the Validators it updates after spawning itself.
pub(crate) fn spawn(serai: Serai) -> (Arc<RwLock<Validators>>, mpsc::UnboundedReceiver<Changes>) { pub(crate) fn spawn(
serai: Arc<Serai>,
) -> (Arc<RwLock<Validators>>, mpsc::UnboundedReceiver<Changes>) {
// The validators which will be updated // The validators which will be updated
let (validators, changes) = Validators::new(serai); let (validators, changes) = Validators::new(serai);
let validators = Arc::new(RwLock::new(validators)); let validators = Arc::new(RwLock::new(validators));

View File

@@ -56,6 +56,9 @@ pub trait P2p: Send + Sync + Clone + tributary::P2p + serai_cosign::RequestNotab
/// Fetch the peers for this network. /// Fetch the peers for this network.
fn peers(&self, network: NetworkId) -> impl Send + Future<Output = Vec<Self::Peer<'_>>>; fn peers(&self, network: NetworkId) -> impl Send + Future<Output = Vec<Self::Peer<'_>>>;
/// Broadcast a cosign.
fn publish_cosign(&self, cosign: SignedCosign) -> impl Send + Future<Output = ()>;
/// A cancel-safe future for the next heartbeat received over the P2P network. /// A cancel-safe future for the next heartbeat received over the P2P network.
/// ///
/// Yields the validator set its for, the latest block hash observed, and a channel to return the /// Yields the validator set its for, the latest block hash observed, and a channel to return the