From 5e349f58911c4b45174693e50e1689b7b583ad08 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jo=C3=A3o=20Oliveira?= Date: Tue, 10 Sep 2024 23:29:25 +0100 Subject: [PATCH] feat(gossipsub): implement backpressures --- Cargo.lock | 49 +- protocols/gossipsub/CHANGELOG.md | 7 + protocols/gossipsub/Cargo.toml | 2 + protocols/gossipsub/src/behaviour.rs | 753 ++++++---- protocols/gossipsub/src/behaviour/tests.rs | 1409 ++++++++++-------- protocols/gossipsub/src/config.rs | 43 + protocols/gossipsub/src/error.rs | 3 + protocols/gossipsub/src/handler.rs | 55 +- protocols/gossipsub/src/lib.rs | 2 +- protocols/gossipsub/src/metrics.rs | 61 + protocols/gossipsub/src/peer_score.rs | 24 + protocols/gossipsub/src/peer_score/params.rs | 8 + protocols/gossipsub/src/protocol.rs | 53 +- protocols/gossipsub/src/types.rs | 390 ++++- 14 files changed, 1809 insertions(+), 1050 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 4c12e6fb984..a3105987694 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -268,6 +268,18 @@ dependencies = [ "futures-core", ] +[[package]] +name = "async-channel" +version = "2.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "89b47800b0be77592da0afd425cc03468052844aff33b84e33cc696f64e77b6a" +dependencies = [ + "concurrent-queue", + "event-listener-strategy 0.5.2", + "futures-core", + "pin-project-lite", +] + [[package]] name = "async-executor" version = "1.5.1" @@ -300,7 +312,7 @@ version = "2.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f1b6f5d7df27bd294849f8eec66ecfc63d11814df7a4f5d74168a2394467b776" dependencies = [ - "async-channel", + "async-channel 1.9.0", "async-executor", "async-io 1.13.0", "async-lock 2.7.0", @@ -364,7 +376,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "deb2ab2aa8a746e221ab826c73f48bc6ba41be6763f0855cb249eb6d154cf1d7" dependencies = [ "event-listener 3.1.0", - "event-listener-strategy", + "event-listener-strategy 0.3.0", "pin-project-lite", ] @@ -405,7 +417,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "62565bb4402e926b29953c785397c6dc0391b7b446e45008b0049eb43cec6f5d" dependencies = [ "async-attributes", - "async-channel", + "async-channel 1.9.0", "async-global-executor", "async-io 1.13.0", "async-lock 2.7.0", @@ -765,7 +777,7 @@ version = "1.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "77231a1c8f801696fc0123ec6150ce92cffb8e164a02afb9c8ddee0e9b65ad65" dependencies = [ - "async-channel", + "async-channel 1.9.0", "async-lock 2.7.0", "async-task", "atomic-waker", @@ -1027,9 +1039,9 @@ dependencies = [ [[package]] name = "concurrent-queue" -version = "2.2.0" +version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62ec6771ecfa0762d24683ee5a32ad78487a3d3afdc0fb8cae19d2c5deb50b7c" +checksum = "4ca0197aee26d1ae37445ee532fefce43251d24cc7c166799f4d46817f1d3973" dependencies = [ "crossbeam-utils", ] @@ -1563,6 +1575,17 @@ dependencies = [ "pin-project-lite", ] +[[package]] +name = "event-listener" +version = "5.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6032be9bd27023a771701cc49f9f053c751055f71efb2e0ae5c15809093675ba" +dependencies = [ + "concurrent-queue", + "parking", + "pin-project-lite", +] + [[package]] name = "event-listener-strategy" version = "0.3.0" @@ -1573,6 +1596,16 @@ dependencies = [ "pin-project-lite", ] +[[package]] +name = "event-listener-strategy" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0f214dc438f977e6d4e3500aaa277f5ad94ca83fbbd9b1a15713ce2344ccc5a1" +dependencies = [ + "event-listener 5.3.1", + "pin-project-lite", +] + [[package]] name = "fastrand" version = "1.9.0" @@ -2839,6 +2872,7 @@ dependencies = [ name = "libp2p-gossipsub" version = "0.47.1" dependencies = [ + "async-channel 2.3.1", "async-std", "asynchronous-codec", "base64 0.22.1", @@ -2848,6 +2882,7 @@ dependencies = [ "fnv", "futures", "futures-ticker", + "futures-timer", "getrandom 0.2.15", "hex", "hex_fmt", @@ -5656,7 +5691,7 @@ version = "1.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "13f2b548cd8447f8de0fdf1c592929f70f4fc7039a05e47404b0d096ec6987a1" dependencies = [ - "async-channel", + "async-channel 1.9.0", "async-executor", "async-fs", "async-io 1.13.0", diff --git a/protocols/gossipsub/CHANGELOG.md b/protocols/gossipsub/CHANGELOG.md index c47a9f40f66..2cb25e604da 100644 --- a/protocols/gossipsub/CHANGELOG.md +++ b/protocols/gossipsub/CHANGELOG.md @@ -3,6 +3,13 @@ - Attempt to publish to at least mesh_n peers when flood publish is disabled. See [PR 5578](https://github.com/libp2p/rust-libp2p/pull/5578). +- Introduce back pressure and penalize slow peers. Drop stale messages that timeout before being + delivered. + See [PR 5595](https://github.com/libp2p/rust-libp2p/pull/5595). +- Change `Behaviour::unsubscribe` and `Behaviour::report_message_validation_result` + to `bool` they don't need to be a `Result`. + See [PR 5595](https://github.com/libp2p/rust-libp2p/pull/5595). + ## 0.47.0 diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml index 665f757fcb3..0f8cda8eee5 100644 --- a/protocols/gossipsub/Cargo.toml +++ b/protocols/gossipsub/Cargo.toml @@ -14,6 +14,7 @@ categories = ["network-programming", "asynchronous"] wasm-bindgen = ["getrandom/js"] [dependencies] +async-channel = "2.3.1" asynchronous-codec = { workspace = true } base64 = "0.22.1" byteorder = "1.5.0" @@ -22,6 +23,7 @@ either = "1.11" fnv = "1.0.7" futures = { workspace = true } futures-ticker = "0.0.3" +futures-timer = "3.0.2" getrandom = "0.2.15" hex_fmt = "0.3.0" web-time = { workspace = true } diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index 16adb555a44..b8cd566d511 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -47,12 +47,6 @@ use libp2p_swarm::{ }; use web_time::{Instant, SystemTime}; -use crate::backoff::BackoffStorage; -use crate::config::{Config, ValidationMode}; -use crate::gossip_promises::GossipPromises; -use crate::handler::{Handler, HandlerEvent, HandlerIn}; -use crate::mcache::MessageCache; -use crate::metrics::{Churn, Config as MetricsConfig, Inclusion, Metrics, Penalty}; use crate::peer_score::{PeerScore, PeerScoreParams, PeerScoreThresholds, RejectReason}; use crate::protocol::SIGNING_PREFIX; use crate::subscription_filter::{AllowAllSubscriptionFilter, TopicSubscriptionFilter}; @@ -64,6 +58,21 @@ use crate::types::{ SubscriptionAction, }; use crate::types::{PeerConnections, PeerKind, RpcOut}; +use crate::{backoff::BackoffStorage, FailedMessages}; +use crate::{ + config::{Config, ValidationMode}, + types::Graft, +}; +use crate::{gossip_promises::GossipPromises, types::Prune}; +use crate::{ + handler::{Handler, HandlerEvent, HandlerIn}, + types::IWant, +}; +use crate::{mcache::MessageCache, types::IHave}; +use crate::{ + metrics::{Churn, Config as MetricsConfig, Inclusion, Metrics, Penalty}, + types::RpcSender, +}; use crate::{rpc_proto::proto, TopicScoreParams}; use crate::{PublishError, SubscriptionError, ValidationError}; use quick_protobuf::{MessageWrite, Writer}; @@ -145,6 +154,13 @@ pub enum Event { }, /// A peer that does not support gossipsub has connected. GossipsubNotSupported { peer_id: PeerId }, + /// A peer is not able to download messages in time. + SlowPeer { + /// The peer_id + peer_id: PeerId, + /// The types and amounts of failed messages that are occurring for this peer. + failed_messages: FailedMessages, + }, } /// A data structure for storing configuration for publishing messages. See [`MessageAuthenticity`] @@ -245,9 +261,6 @@ pub struct Behaviour { /// Events that need to be yielded to the outside when polling. events: VecDeque>, - /// Pools non-urgent control messages between heartbeats. - control_pool: HashMap>, - /// Information used for publishing messages. publish_config: PublishConfig, @@ -309,10 +322,6 @@ pub struct Behaviour { /// Counts the number of `IWANT` that we sent the each peer since the last heartbeat. count_sent_iwant: HashMap, - /// Keeps track of IWANT messages that we are awaiting to send. - /// This is used to prevent sending duplicate IWANT messages for the same message. - pending_iwant_msgs: HashSet, - /// Short term cache for published message ids. This is used for penalizing peers sending /// our own messages back if the messages are anonymous or use a random author. published_message_ids: DuplicateCache, @@ -327,6 +336,9 @@ pub struct Behaviour { /// Keep track of a set of internal metrics relating to gossipsub. metrics: Option, + + /// Tracks the numbers of failed messages per peer-id. + failed_messages: HashMap, } impl Behaviour @@ -434,7 +446,6 @@ where Ok(Behaviour { metrics: metrics.map(|(registry, cfg)| Metrics::new(registry, cfg)), events: VecDeque::new(), - control_pool: HashMap::new(), publish_config: privacy.into(), duplicate_cache: DuplicateCache::new(config.duplicate_cache_time()), explicit_peers: HashSet::new(), @@ -458,12 +469,12 @@ where peer_score: None, count_received_ihave: HashMap::new(), count_sent_iwant: HashMap::new(), - pending_iwant_msgs: HashSet::new(), connected_peers: HashMap::new(), published_message_ids: DuplicateCache::new(config.published_message_ids_cache_time()), config, subscription_filter, data_transform, + failed_messages: Default::default(), }) } } @@ -527,10 +538,9 @@ where } // send subscription request to all peers - for peer in self.connected_peers.keys().copied().collect::>() { - tracing::debug!(%peer, "Sending SUBSCRIBE to peer"); - let event = RpcOut::Subscribe(topic_hash.clone()); - self.send_message(peer, event); + for (peer_id, peer) in self.connected_peers.iter_mut() { + tracing::debug!(%peer_id, "Sending SUBSCRIBE to peer"); + peer.sender.subscribe(topic_hash.clone()); } // call JOIN(topic) @@ -543,22 +553,20 @@ where /// Unsubscribes from a topic. /// /// Returns [`Ok(true)`] if we were subscribed to this topic. - #[allow(clippy::unnecessary_wraps)] - pub fn unsubscribe(&mut self, topic: &Topic) -> Result { + pub fn unsubscribe(&mut self, topic: &Topic) -> bool { tracing::debug!(%topic, "Unsubscribing from topic"); let topic_hash = topic.hash(); if !self.mesh.contains_key(&topic_hash) { tracing::debug!(topic=%topic_hash, "Already unsubscribed from topic"); // we are not subscribed - return Ok(false); + return false; } // announce to all peers - for peer in self.connected_peers.keys().copied().collect::>() { - tracing::debug!(%peer, "Sending UNSUBSCRIBE to peer"); - let event = RpcOut::Unsubscribe(topic_hash.clone()); - self.send_message(peer, event); + for (peer_id, peer) in self.connected_peers.iter_mut() { + tracing::debug!(%peer_id, "Sending UNSUBSCRIBE to peer"); + peer.sender.unsubscribe(topic_hash.clone()); } // call LEAVE(topic) @@ -566,7 +574,7 @@ where self.leave(&topic_hash); tracing::debug!(topic=%topic_hash, "Unsubscribed from topic"); - Ok(true) + true } /// Publishes a message with multiple topics to the network. @@ -724,9 +732,38 @@ where } // Send to peers we know are subscribed to the topic. + let mut publish_failed = true; for peer_id in recipient_peers.iter() { - tracing::trace!(peer=%peer_id, "Sending message to peer"); - self.send_message(*peer_id, RpcOut::Publish(raw_message.clone())); + if let Some(peer) = self.connected_peers.get_mut(peer_id) { + tracing::trace!(peer=%peer_id, "Sending message to peer"); + match peer.sender.publish( + raw_message.clone(), + self.config.publish_queue_duration(), + self.metrics.as_mut(), + ) { + Ok(_) => publish_failed = false, + Err(_) => { + self.failed_messages.entry(*peer_id).or_default().priority += 1; + + tracing::warn!(peer_id=%peer_id, "Publish queue full. Could not publish to peer"); + // Downscore the peer due to failed message. + if let Some((peer_score, ..)) = &mut self.peer_score { + peer_score.failed_message_slow_peer(peer_id); + } + } + } + } else { + tracing::error!(peer_id = %peer_id, + "Could not PUBLISH, peer doesn't exist in connected peer list"); + } + } + + if recipient_peers.is_empty() { + return Err(PublishError::InsufficientPeers); + } + + if publish_failed { + return Err(PublishError::AllQueuesFull(recipient_peers.len())); } tracing::debug!(message=%msg_id, "Published message"); @@ -762,7 +799,7 @@ where msg_id: &MessageId, propagation_source: &PeerId, acceptance: MessageAcceptance, - ) -> Result { + ) -> bool { let reject_reason = match acceptance { MessageAcceptance::Accept => { let (raw_message, originating_peers) = match self.mcache.validate(msg_id) { @@ -777,7 +814,7 @@ where if let Some(metrics) = self.metrics.as_mut() { metrics.memcache_miss(); } - return Ok(false); + return false; } }; @@ -790,8 +827,8 @@ where raw_message, Some(propagation_source), originating_peers, - )?; - return Ok(true); + ); + return true; } MessageAcceptance::Reject => RejectReason::ValidationFailed, MessageAcceptance::Ignore => RejectReason::ValidationIgnored, @@ -815,10 +852,10 @@ where peer_score.reject_message(peer, msg_id, &raw_message.topic, reject_reason); } } - Ok(true) + true } else { tracing::warn!(message=%msg_id, "Rejected message not in cache"); - Ok(false) + false } } @@ -1002,17 +1039,18 @@ where for peer_id in added_peers { // Send a GRAFT control message - tracing::debug!(peer=%peer_id, "JOIN: Sending Graft message to peer"); if let Some((peer_score, ..)) = &mut self.peer_score { peer_score.graft(&peer_id, topic_hash.clone()); } - Self::control_pool_add( - &mut self.control_pool, - peer_id, - ControlAction::Graft { + if let Some(peer) = &mut self.connected_peers.get_mut(&peer_id) { + tracing::debug!(peer=%peer_id, "JOIN: Sending Graft message to peer"); + peer.sender.graft(Graft { topic_hash: topic_hash.clone(), - }, - ); + }); + } else { + tracing::error!(peer = %peer_id, + "Could not GRAFT, peer doesn't exist in connected peer list"); + } // If the peer did not previously exist in any mesh, inform the handler peer_added_to_mesh( @@ -1039,7 +1077,7 @@ where peer: &PeerId, do_px: bool, on_unsubscribe: bool, - ) -> ControlAction { + ) -> Prune { if let Some((peer_score, ..)) = &mut self.peer_score { peer_score.prune(peer, topic_hash.clone()); } @@ -1050,7 +1088,7 @@ where } Some(PeerKind::Gossipsub) => { // GossipSub v1.0 -- no peer exchange, the peer won't be able to parse it anyway - return ControlAction::Prune { + return Prune { topic_hash: topic_hash.clone(), peers: Vec::new(), backoff: None, @@ -1086,7 +1124,7 @@ where // update backoff self.backoffs.update_backoff(topic_hash, peer, backoff); - ControlAction::Prune { + Prune { topic_hash: topic_hash.clone(), peers, backoff: Some(backoff.as_secs()), @@ -1102,17 +1140,20 @@ where if let Some(m) = self.metrics.as_mut() { m.left(topic_hash) } - for peer in peers { + for peer_id in peers { // Send a PRUNE control message - tracing::debug!(%peer, "LEAVE: Sending PRUNE to peer"); - let on_unsubscribe = true; - let control = - self.make_prune(topic_hash, &peer, self.config.do_px(), on_unsubscribe); - Self::control_pool_add(&mut self.control_pool, peer, control); + let prune = self.make_prune(topic_hash, &peer_id, self.config.do_px(), true); + if let Some(peer) = &mut self.connected_peers.get_mut(&peer_id) { + tracing::debug!(%peer_id, "LEAVE: Sending PRUNE to peer"); + peer.sender.prune(prune); + } else { + tracing::error!(peer = %peer_id, + "Could not PRUNE, peer doesn't exist in connected peer list"); + } // If the peer did not previously exist in any mesh, inform the handler peer_removed_from_mesh( - peer, + peer_id, topic_hash, &self.mesh, &mut self.events, @@ -1206,10 +1247,6 @@ where return false; } - if self.pending_iwant_msgs.contains(id) { - return false; - } - self.peer_score .as_ref() .map(|(_, _, _, promises)| !promises.contains(id)) @@ -1260,11 +1297,6 @@ where iwant_ids_vec.truncate(iask); *iasked += iask; - for message_id in &iwant_ids_vec { - // Add all messages to the pending list - self.pending_iwant_msgs.insert(message_id.clone()); - } - if let Some((_, _, _, gossip_promises)) = &mut self.peer_score { gossip_promises.add_promise( *peer_id, @@ -1272,19 +1304,35 @@ where Instant::now() + self.config.iwant_followup_time(), ); } - tracing::trace!( - peer=%peer_id, - "IHAVE: Asking for the following messages from peer: {:?}", - iwant_ids_vec - ); + if let Some(peer) = &mut self.connected_peers.get_mut(peer_id) { + tracing::trace!( + peer=%peer_id, + "IHAVE: Asking for the following messages from peer: {:?}", + iwant_ids_vec + ); - Self::control_pool_add( - &mut self.control_pool, - *peer_id, - ControlAction::IWant { - message_ids: iwant_ids_vec, - }, - ); + if peer + .sender + .iwant(IWant { + message_ids: iwant_ids_vec, + }) + .is_err() + { + tracing::warn!(peer=%peer_id, "Send Queue full. Could not send IWANT"); + + if let Some((peer_score, ..)) = &mut self.peer_score { + peer_score.failed_message_slow_peer(peer_id); + } + // Increment failed message count + self.failed_messages + .entry(*peer_id) + .or_default() + .non_priority += 1; + } + } else { + tracing::error!(peer = %peer_id, + "Could not IWANT, peer doesn't exist in connected peer list"); + } } tracing::trace!(peer=%peer_id, "Completed IHAVE handling for peer"); } @@ -1318,9 +1366,30 @@ where message=%id, "IWANT: Peer has asked for message too many times; ignoring request" ); - } else { + } else if let Some(peer) = &mut self.connected_peers.get_mut(peer_id) { tracing::debug!(peer=%peer_id, "IWANT: Sending cached messages to peer"); - self.send_message(*peer_id, RpcOut::Forward(msg)); + if peer + .sender + .forward( + msg, + self.config.forward_queue_duration(), + self.metrics.as_mut(), + ) + .is_err() + { + // Downscore the peer + if let Some((peer_score, ..)) = &mut self.peer_score { + peer_score.failed_message_slow_peer(peer_id); + } + // Increment the failed message count + self.failed_messages + .entry(*peer_id) + .or_default() + .non_priority += 1; + } + } else { + tracing::error!(peer = %peer_id, + "Could not IWANT, peer doesn't exist in connected peer list"); } } } @@ -1336,14 +1405,13 @@ where let mut do_px = self.config.do_px(); - let Some(connected_peer) = self.connected_peers.get_mut(peer_id) else { - tracing::error!(peer_id = %peer_id, "Peer non-existent when handling graft"); - return; - }; - // For each topic, if a peer has grafted us, then we necessarily must be in their mesh // and they must be subscribed to the topic. Ensure we have recorded the mapping. for topic in &topics { + let Some(connected_peer) = self.connected_peers.get_mut(peer_id) else { + tracing::error!(peer_id = %peer_id, "Peer non-existent when handling graft"); + return; + }; if connected_peer.topics.insert(topic.clone()) { if let Some(m) = self.metrics.as_mut() { m.inc_topic_peers(topic); @@ -1474,12 +1542,19 @@ where if !to_prune_topics.is_empty() { // build the prune messages to send let on_unsubscribe = false; - for action in to_prune_topics + let mut sender = match self.connected_peers.get_mut(peer_id) { + Some(connected_peer) => connected_peer.sender.clone(), + None => { + tracing::error!(peer_id = %peer_id, "Peer non-existent when handling graft and obtaining a sender"); + return; + } + }; + + for prune in to_prune_topics .iter() .map(|t| self.make_prune(t, peer_id, do_px, on_unsubscribe)) - .collect::>() { - self.send_message(*peer_id, RpcOut::Control(action)); + sender.prune(prune); } // Send the prune messages to the peer tracing::debug!( @@ -1771,17 +1846,12 @@ where // forward the message to mesh peers, if no validation is required if !self.config.validate_messages() { - if self - .forward_msg( - &msg_id, - raw_message, - Some(propagation_source), - HashSet::new(), - ) - .is_err() - { - tracing::error!("Failed to forward message. Too large"); - } + self.forward_msg( + &msg_id, + raw_message, + Some(propagation_source), + HashSet::new(), + ); tracing::debug!(message=%msg_id, "Completed message handling for message"); } } @@ -1965,12 +2035,13 @@ where // If we need to send grafts to peer, do so immediately, rather than waiting for the // heartbeat. - for action in topics_to_graft - .into_iter() - .map(|topic_hash| ControlAction::Graft { topic_hash }) - .collect::>() - { - self.send_message(*propagation_source, RpcOut::Control(action)) + if let Some(peer) = &mut self.connected_peers.get_mut(propagation_source) { + for topic_hash in topics_to_graft.into_iter() { + peer.sender.graft(Graft { topic_hash }); + } + } else { + tracing::error!(peer = %propagation_source, + "Could not GRAFT, peer doesn't exist in connected peer list"); } // Notify the application of the subscriptions @@ -2001,6 +2072,16 @@ where tracing::debug!("Starting heartbeat"); let start = Instant::now(); + // Every heartbeat we sample the send queues to add to our metrics. We do this intentionally + // before we add all the gossip from this heartbeat in order to gain a true measure of + // steady-state size of the queues. + if let Some(m) = &mut self.metrics { + for sender_queue in self.connected_peers.values_mut().map(|v| &v.sender) { + m.observe_priority_queue_size(sender_queue.priority_len()); + m.observe_non_priority_queue_size(sender_queue.non_priority_len()); + } + } + self.heartbeat_ticks += 1; let mut to_graft = HashMap::new(); @@ -2370,12 +2451,20 @@ where self.send_graft_prune(to_graft, to_prune, no_px); } - // piggyback pooled control messages - self.flush_control_pool(); - // shift the memcache self.mcache.shift(); + // Report expired messages + for (peer_id, failed_messages) in self.failed_messages.drain() { + tracing::debug!("Peer couldn't consume messages: {:?}", failed_messages); + self.events + .push_back(ToSwarm::GenerateEvent(Event::SlowPeer { + peer_id, + failed_messages, + })); + } + self.failed_messages.shrink_to_fit(); + tracing::debug!("Completed Heartbeat"); if let Some(metrics) = self.metrics.as_mut() { let duration = u64::try_from(start.elapsed().as_millis()).unwrap_or(u64::MAX); @@ -2422,7 +2511,7 @@ where tracing::debug!("Gossiping IHAVE to {} peers", to_msg_peers.len()); - for peer in to_msg_peers { + for peer_id in to_msg_peers { let mut peer_message_ids = message_ids.clone(); if peer_message_ids.len() > self.config.max_ihave_length() { @@ -2434,14 +2523,30 @@ where } // send an IHAVE message - Self::control_pool_add( - &mut self.control_pool, - peer, - ControlAction::IHave { - topic_hash: topic_hash.clone(), - message_ids: peer_message_ids, - }, - ); + if let Some(peer) = &mut self.connected_peers.get_mut(&peer_id) { + if peer + .sender + .ihave(IHave { + topic_hash: topic_hash.clone(), + message_ids: peer_message_ids, + }) + .is_err() + { + tracing::warn!(peer=%peer_id, "Send Queue full. Could not send IHAVE"); + + if let Some((peer_score, ..)) = &mut self.peer_score { + peer_score.failed_message_slow_peer(&peer_id); + } + // Increment failed message count + self.failed_messages + .entry(peer_id) + .or_default() + .non_priority += 1; + } + } else { + tracing::error!(peer = %peer_id, + "Could not IHAVE, peer doesn't exist in connected peer list"); + } } } } @@ -2455,67 +2560,83 @@ where no_px: HashSet, ) { // handle the grafts and overlapping prunes per peer - for (peer, topics) in to_graft.into_iter() { + for (peer_id, topics) in to_graft.into_iter() { for topic in &topics { // inform scoring of graft if let Some((peer_score, ..)) = &mut self.peer_score { - peer_score.graft(&peer, topic.clone()); + peer_score.graft(&peer_id, topic.clone()); } // inform the handler of the peer being added to the mesh // If the peer did not previously exist in any mesh, inform the handler peer_added_to_mesh( - peer, + peer_id, vec![topic], &self.mesh, &mut self.events, &self.connected_peers, ); } - let control_msgs = topics.iter().map(|topic_hash| ControlAction::Graft { - topic_hash: topic_hash.clone(), - }); // If there are prunes associated with the same peer add them. // NOTE: In this case a peer has been added to a topic mesh, and removed from another. // It therefore must be in at least one mesh and we do not need to inform the handler // of its removal from another. + // send the control messages + let mut sender = match self.connected_peers.get_mut(&peer_id) { + Some(connected_peer) => connected_peer.sender.clone(), + None => { + tracing::error!(peer_id = %peer_id, "Peer non-existent when sending graft/prune"); + return; + } + }; + // The following prunes are not due to unsubscribing. let prunes = to_prune - .remove(&peer) + .remove(&peer_id) .into_iter() .flatten() .map(|topic_hash| { self.make_prune( &topic_hash, - &peer, - self.config.do_px() && !no_px.contains(&peer), + &peer_id, + self.config.do_px() && !no_px.contains(&peer_id), false, ) }); - // send the control messages - for msg in control_msgs.chain(prunes).collect::>() { - self.send_message(peer, RpcOut::Control(msg)); + for topic_hash in topics { + sender.graft(Graft { + topic_hash: topic_hash.clone(), + }); + } + + for prune in prunes { + sender.prune(prune); } } // handle the remaining prunes // The following prunes are not due to unsubscribing. - for (peer, topics) in to_prune.iter() { + for (peer_id, topics) in to_prune.iter() { for topic_hash in topics { let prune = self.make_prune( topic_hash, - peer, - self.config.do_px() && !no_px.contains(peer), + peer_id, + self.config.do_px() && !no_px.contains(peer_id), false, ); - self.send_message(*peer, RpcOut::Control(prune)); + if let Some(peer) = self.connected_peers.get_mut(peer_id) { + peer.sender.prune(prune); + } else { + tracing::error!(peer = %peer_id, + "Could not PRUNE, peer doesn't exist in connected peer list"); + } // inform the handler peer_removed_from_mesh( - *peer, + *peer_id, topic_hash, &self.mesh, &mut self.events, @@ -2528,14 +2649,13 @@ where /// Helper function which forwards a message to mesh\[topic\] peers. /// /// Returns true if at least one peer was messaged. - #[allow(clippy::unnecessary_wraps)] fn forward_msg( &mut self, msg_id: &MessageId, message: RawMessage, propagation_source: Option<&PeerId>, originating_peers: HashSet, - ) -> Result { + ) -> bool { // message is fully validated inform peer_score if let Some((peer_score, ..)) = &mut self.peer_score { if let Some(peer) = propagation_source { @@ -2546,49 +2666,68 @@ where tracing::debug!(message=%msg_id, "Forwarding message"); let mut recipient_peers = HashSet::new(); - { - // Populate the recipient peers mapping - - // Add explicit peers - for peer_id in &self.explicit_peers { - if let Some(peer) = self.connected_peers.get(peer_id) { - if Some(peer_id) != propagation_source - && !originating_peers.contains(peer_id) - && Some(peer_id) != message.source.as_ref() - && peer.topics.contains(&message.topic) - { - recipient_peers.insert(*peer_id); - } + // Populate the recipient peers mapping + + // Add explicit peers + for peer_id in &self.explicit_peers { + if let Some(peer) = self.connected_peers.get(peer_id) { + if Some(peer_id) != propagation_source + && !originating_peers.contains(peer_id) + && Some(peer_id) != message.source.as_ref() + && peer.topics.contains(&message.topic) + { + recipient_peers.insert(*peer_id); } } + } - // add mesh peers - let topic = &message.topic; - // mesh - if let Some(mesh_peers) = self.mesh.get(topic) { - for peer_id in mesh_peers { - if Some(peer_id) != propagation_source - && !originating_peers.contains(peer_id) - && Some(peer_id) != message.source.as_ref() - { - recipient_peers.insert(*peer_id); - } + // add mesh peers + let topic = &message.topic; + // mesh + if let Some(mesh_peers) = self.mesh.get(topic) { + for peer_id in mesh_peers { + if Some(peer_id) != propagation_source + && !originating_peers.contains(peer_id) + && Some(peer_id) != message.source.as_ref() + { + recipient_peers.insert(*peer_id); } } } // forward the message to peers if !recipient_peers.is_empty() { - let event = RpcOut::Forward(message.clone()); - - for peer in recipient_peers.iter() { - tracing::debug!(%peer, message=%msg_id, "Sending message to peer"); - self.send_message(*peer, event.clone()); + for peer_id in recipient_peers.iter() { + if let Some(peer) = self.connected_peers.get_mut(peer_id) { + tracing::debug!(%peer_id, message=%msg_id, "Sending message to peer"); + if peer + .sender + .forward( + message.clone(), + self.config.forward_queue_duration(), + self.metrics.as_mut(), + ) + .is_err() + { + // Downscore the peer + if let Some((peer_score, ..)) = &mut self.peer_score { + peer_score.failed_message_slow_peer(peer_id); + } + // Increment the failed message count + self.failed_messages + .entry(*peer_id) + .or_default() + .non_priority += 1; + } + } else { + tracing::error!(peer = %peer_id, + "Could not FORWARD, peer doesn't exist in connected peer list"); + } } tracing::debug!("Completed forwarding message"); - Ok(true) + true } else { - Ok(false) + false } } @@ -2684,49 +2823,10 @@ where } } - // adds a control action to control_pool - fn control_pool_add( - control_pool: &mut HashMap>, - peer: PeerId, - control: ControlAction, - ) { - control_pool.entry(peer).or_default().push(control); - } - - /// Takes each control action mapping and turns it into a message - fn flush_control_pool(&mut self) { - for (peer, controls) in self.control_pool.drain().collect::>() { - for msg in controls { - self.send_message(peer, RpcOut::Control(msg)); - } - } - - // This clears all pending IWANT messages - self.pending_iwant_msgs.clear(); - } - - /// Send a [`RpcOut`] message to a peer. This will wrap the message in an arc if it - /// is not already an arc. - fn send_message(&mut self, peer_id: PeerId, rpc: RpcOut) { - if let Some(m) = self.metrics.as_mut() { - if let RpcOut::Publish(ref message) | RpcOut::Forward(ref message) = rpc { - // register bytes sent on the internal metrics. - m.msg_sent(&message.topic, message.raw_protobuf_len()); - } - } - - self.events.push_back(ToSwarm::NotifyHandler { - peer_id, - event: HandlerIn::Message(rpc), - handler: NotifyHandler::Any, - }); - } - fn on_connection_established( &mut self, ConnectionEstablished { peer_id, - connection_id, endpoint, other_established, .. @@ -2754,21 +2854,6 @@ where } } - // By default we assume a peer is only a floodsub peer. - // - // The protocol negotiation occurs once a message is sent/received. Once this happens we - // update the type of peer that this is in order to determine which kind of routing should - // occur. - self.connected_peers - .entry(peer_id) - .or_insert(PeerConnections { - kind: PeerKind::Floodsub, - connections: vec![], - topics: Default::default(), - }) - .connections - .push(connection_id); - if other_established > 0 { return; // Not our first connection to this peer, hence nothing to do. } @@ -2785,8 +2870,13 @@ where tracing::debug!(peer=%peer_id, "New peer connected"); // We need to send our subscriptions to the newly-connected node. - for topic_hash in self.mesh.clone().into_keys() { - self.send_message(peer_id, RpcOut::Subscribe(topic_hash)); + if let Some(peer) = self.connected_peers.get_mut(&peer_id) { + for topic_hash in self.mesh.clone().into_keys() { + peer.sender.subscribe(topic_hash); + } + } else { + tracing::error!(peer = %peer_id, + "Could not SUBSCRIBE, peer doesn't exist in connected peer list"); } } @@ -2843,37 +2933,32 @@ where } else { // remove from mesh, topic_peers, peer_topic and the fanout tracing::debug!(peer=%peer_id, "Peer disconnected"); - { - let Some(peer) = self.connected_peers.get(&peer_id) else { - debug_assert!( - self.blacklisted_peers.contains(&peer_id), - "Disconnected node not in connected list" - ); - return; - }; - - // remove peer from all mappings - for topic in peer.topics.iter() { - // check the mesh for the topic - if let Some(mesh_peers) = self.mesh.get_mut(topic) { - // check if the peer is in the mesh and remove it - if mesh_peers.remove(&peer_id) { - if let Some(m) = self.metrics.as_mut() { - m.peers_removed(topic, Churn::Dc, 1); - m.set_mesh_peers(topic, mesh_peers.len()); - } - }; - } + let Some(connected_peer) = self.connected_peers.get(&peer_id) else { + tracing::error!(peer_id = %peer_id, "Peer non-existent when handling disconnection"); + return; + }; - if let Some(m) = self.metrics.as_mut() { - m.dec_topic_peers(topic); - } + // remove peer from all mappings + for topic in &connected_peer.topics { + // check the mesh for the topic + if let Some(mesh_peers) = self.mesh.get_mut(topic) { + // check if the peer is in the mesh and remove it + if mesh_peers.remove(&peer_id) { + if let Some(m) = self.metrics.as_mut() { + m.peers_removed(topic, Churn::Dc, 1); + m.set_mesh_peers(topic, mesh_peers.len()); + } + }; + } - // remove from fanout - self.fanout - .get_mut(topic) - .map(|peers| peers.remove(&peer_id)); + if let Some(m) = self.metrics.as_mut() { + m.dec_topic_peers(topic); } + + // remove from fanout + self.fanout + .get_mut(topic) + .map(|peers| peers.remove(&peer_id)); } // Forget px and outbound status for this peer @@ -2882,12 +2967,7 @@ where // If metrics are enabled, register the disconnection of a peer based on its protocol. if let Some(metrics) = self.metrics.as_mut() { - let peer_kind = &self - .connected_peers - .get(&peer_id) - .expect("Connected peer must be registered") - .kind; - metrics.peer_protocol_disconnected(peer_kind.clone()); + metrics.peer_protocol_disconnected(connected_peer.kind.clone()); } self.connected_peers.remove(&peer_id); @@ -2949,23 +3029,58 @@ where fn handle_established_inbound_connection( &mut self, - _: ConnectionId, - _: PeerId, + connection_id: ConnectionId, + peer_id: PeerId, _: &Multiaddr, _: &Multiaddr, ) -> Result, ConnectionDenied> { - Ok(Handler::new(self.config.protocol_config())) + // By default we assume a peer is only a floodsub peer. + // + // The protocol negotiation occurs once a message is sent/received. Once this happens we + // update the type of peer that this is in order to determine which kind of routing should + // occur. + let connected_peer = self + .connected_peers + .entry(peer_id) + .or_insert(PeerConnections { + kind: PeerKind::Floodsub, + connections: vec![], + sender: RpcSender::new(self.config.connection_handler_queue_len()), + topics: Default::default(), + }); + // Add the new connection + connected_peer.connections.push(connection_id); + + Ok(Handler::new( + self.config.protocol_config(), + connected_peer.sender.new_receiver(), + )) } fn handle_established_outbound_connection( &mut self, - _: ConnectionId, - _: PeerId, + connection_id: ConnectionId, + peer_id: PeerId, _: &Multiaddr, _: Endpoint, _: PortUse, ) -> Result, ConnectionDenied> { - Ok(Handler::new(self.config.protocol_config())) + let connected_peer = self + .connected_peers + .entry(peer_id) + .or_insert(PeerConnections { + kind: PeerKind::Floodsub, + connections: vec![], + sender: RpcSender::new(self.config.connection_handler_queue_len()), + topics: Default::default(), + }); + // Add the new connection + connected_peer.connections.push(connection_id); + + Ok(Handler::new( + self.config.protocol_config(), + connected_peer.sender.new_receiver(), + )) } fn on_connection_handler_event( @@ -3005,6 +3120,42 @@ where } } } + HandlerEvent::MessageDropped(rpc) => { + // Account for this in the scoring logic + if let Some((peer_score, _, _, _)) = &mut self.peer_score { + peer_score.failed_message_slow_peer(&propagation_source); + } + + // Keep track of expired messages for the application layer. + match rpc { + RpcOut::Publish { .. } => { + self.failed_messages + .entry(propagation_source) + .or_default() + .publish += 1; + } + RpcOut::Forward { .. } => { + self.failed_messages + .entry(propagation_source) + .or_default() + .forward += 1; + } + _ => {} // + } + + // Record metrics on the failure. + if let Some(metrics) = self.metrics.as_mut() { + match rpc { + RpcOut::Publish { message, .. } => { + metrics.publish_msg_dropped(&message.topic); + } + RpcOut::Forward { message, .. } => { + metrics.forward_msg_dropped(&message.topic); + } + _ => {} + } + } + } HandlerEvent::Message { rpc, invalid_messages, @@ -3065,21 +3216,21 @@ where let mut prune_msgs = vec![]; for control_msg in rpc.control_msgs { match control_msg { - ControlAction::IHave { + ControlAction::IHave(IHave { topic_hash, message_ids, - } => { + }) => { ihave_msgs.push((topic_hash, message_ids)); } - ControlAction::IWant { message_ids } => { + ControlAction::IWant(IWant { message_ids }) => { self.handle_iwant(&propagation_source, message_ids) } - ControlAction::Graft { topic_hash } => graft_msgs.push(topic_hash), - ControlAction::Prune { + ControlAction::Graft(Graft { topic_hash }) => graft_msgs.push(topic_hash), + ControlAction::Prune(Prune { topic_hash, peers, backoff, - } => prune_msgs.push((topic_hash, peers, backoff)), + }) => prune_msgs.push((topic_hash, peers, backoff)), } } if !ihave_msgs.is_empty() { @@ -3143,13 +3294,15 @@ fn peer_added_to_mesh( connections: &HashMap, ) { // Ensure there is an active connection - let connection_id = { - let conn = connections.get(&peer_id).expect("To be connected to peer."); - assert!( - !conn.connections.is_empty(), - "Must have at least one connection" - ); - conn.connections[0] + let connection_id = match connections.get(&peer_id) { + Some(p) => p + .connections + .first() + .expect("There should be at least one connection to a peer."), + None => { + tracing::error!(peer_id=%peer_id, "Peer not existent when added to the mesh"); + return; + } }; if let Some(peer) = connections.get(&peer_id) { @@ -3168,7 +3321,7 @@ fn peer_added_to_mesh( events.push_back(ToSwarm::NotifyHandler { peer_id, event: HandlerIn::JoinedMesh, - handler: NotifyHandler::One(connection_id), + handler: NotifyHandler::One(*connection_id), }); } @@ -3183,12 +3336,16 @@ fn peer_removed_from_mesh( connections: &HashMap, ) { // Ensure there is an active connection - let connection_id = connections - .get(&peer_id) - .expect("To be connected to peer.") - .connections - .first() - .expect("There should be at least one connection to a peer."); + let connection_id = match connections.get(&peer_id) { + Some(p) => p + .connections + .first() + .expect("There should be at least one connection to a peer."), + None => { + tracing::error!(peer_id=%peer_id, "Peer not existent when removed from mesh"); + return; + } + }; if let Some(peer) = connections.get(&peer_id) { for topic in &peer.topics { @@ -3290,7 +3447,6 @@ impl fmt::Debug for Behaviour RawMessage { - RawMessage { - source: Some(PeerId::random()), - data: vec![0; 100], - sequence_number: None, - topic: TopicHash::from_raw("test_topic"), - signature: None, - key: None, - validated: false, - } - } - - fn test_control() -> ControlAction { - ControlAction::IHave { - topic_hash: IdentTopic::new("TestTopic").hash(), - message_ids: vec![MessageId(vec![12u8]); 5], - } - } - - impl Arbitrary for RpcOut { - fn arbitrary(g: &mut Gen) -> Self { - match u8::arbitrary(g) % 5 { - 0 => RpcOut::Subscribe(IdentTopic::new("TestTopic").hash()), - 1 => RpcOut::Unsubscribe(IdentTopic::new("TestTopic").hash()), - 2 => RpcOut::Publish(test_message()), - 3 => RpcOut::Forward(test_message()), - 4 => RpcOut::Control(test_control()), - _ => panic!("outside range"), - } - } - } -} diff --git a/protocols/gossipsub/src/behaviour/tests.rs b/protocols/gossipsub/src/behaviour/tests.rs index a74566a1308..c03d5d7f377 100644 --- a/protocols/gossipsub/src/behaviour/tests.rs +++ b/protocols/gossipsub/src/behaviour/tests.rs @@ -22,6 +22,7 @@ use super::*; use crate::subscription_filter::WhitelistSubscriptionFilter; +use crate::types::RpcReceiver; use crate::{config::ConfigBuilder, types::Rpc, IdentTopic as Topic}; use async_std::net::Ipv4Addr; use byteorder::{BigEndian, ByteOrder}; @@ -53,7 +54,15 @@ where D: DataTransform + Default + Clone + Send + 'static, F: TopicSubscriptionFilter + Clone + Default + Send + 'static, { - pub(crate) fn create_network(self) -> (Behaviour, Vec, Vec) { + #[allow(clippy::type_complexity)] + pub(crate) fn create_network( + self, + ) -> ( + Behaviour, + Vec, + HashMap, + Vec, + ) { let keypair = libp2p_identity::Keypair::generate_ed25519(); // create a gossipsub struct let mut gs: Behaviour = Behaviour::new_with_subscription_filter_and_transform( @@ -81,10 +90,11 @@ where // build and connect peer_no random peers let mut peers = vec![]; + let mut receivers = HashMap::new(); let empty = vec![]; for i in 0..self.peer_no { - peers.push(add_peer( + let (peer, receiver) = add_peer( &mut gs, if self.to_subscribe { &topic_hashes @@ -93,10 +103,12 @@ where }, i < self.outbound, i < self.explicit, - )); + ); + peers.push(peer); + receivers.insert(peer, receiver); } - (gs, peers, topic_hashes) + (gs, peers, receivers, topic_hashes) } fn peer_no(mut self, peer_no: usize) -> Self { @@ -160,7 +172,7 @@ fn add_peer( topic_hashes: &[TopicHash], outbound: bool, explicit: bool, -) -> PeerId +) -> (PeerId, RpcReceiver) where D: DataTransform + Default + Clone + Send + 'static, F: TopicSubscriptionFilter + Clone + Default + Send + 'static, @@ -174,7 +186,7 @@ fn add_peer_with_addr( outbound: bool, explicit: bool, address: Multiaddr, -) -> PeerId +) -> (PeerId, RpcReceiver) where D: DataTransform + Default + Clone + Send + 'static, F: TopicSubscriptionFilter + Clone + Default + Send + 'static, @@ -196,7 +208,7 @@ fn add_peer_with_addr_and_kind( explicit: bool, address: Multiaddr, kind: Option, -) -> PeerId +) -> (PeerId, RpcReceiver) where D: DataTransform + Default + Clone + Send + 'static, F: TopicSubscriptionFilter + Clone + Default + Send + 'static, @@ -215,9 +227,22 @@ where } }; + let sender = RpcSender::new(gs.config.connection_handler_queue_len()); + let receiver = sender.new_receiver(); + let connection_id = ConnectionId::new_unchecked(0); + gs.connected_peers.insert( + peer, + PeerConnections { + kind: kind.clone().unwrap_or(PeerKind::Floodsub), + connections: vec![connection_id], + topics: Default::default(), + sender, + }, + ); + gs.on_swarm_event(FromSwarm::ConnectionEstablished(ConnectionEstablished { peer_id: peer, - connection_id: ConnectionId::new_unchecked(0), + connection_id, endpoint: &endpoint, failed_addresses: &[], other_established: 0, // first connection @@ -245,7 +270,7 @@ where &peer, ); } - peer + (peer, receiver) } fn disconnect_peer(gs: &mut Behaviour, peer_id: &PeerId) @@ -298,33 +323,39 @@ fn proto_to_message(rpc: &proto::RPC) -> Rpc { let ihave_msgs: Vec = rpc_control .ihave .into_iter() - .map(|ihave| ControlAction::IHave { - topic_hash: TopicHash::from_raw(ihave.topic_id.unwrap_or_default()), - message_ids: ihave - .message_ids - .into_iter() - .map(MessageId::from) - .collect::>(), + .map(|ihave| { + ControlAction::IHave(IHave { + topic_hash: TopicHash::from_raw(ihave.topic_id.unwrap_or_default()), + message_ids: ihave + .message_ids + .into_iter() + .map(MessageId::from) + .collect::>(), + }) }) .collect(); let iwant_msgs: Vec = rpc_control .iwant .into_iter() - .map(|iwant| ControlAction::IWant { - message_ids: iwant - .message_ids - .into_iter() - .map(MessageId::from) - .collect::>(), + .map(|iwant| { + ControlAction::IWant(IWant { + message_ids: iwant + .message_ids + .into_iter() + .map(MessageId::from) + .collect::>(), + }) }) .collect(); let graft_msgs: Vec = rpc_control .graft .into_iter() - .map(|graft| ControlAction::Graft { - topic_hash: TopicHash::from_raw(graft.topic_id.unwrap_or_default()), + .map(|graft| { + ControlAction::Graft(Graft { + topic_hash: TopicHash::from_raw(graft.topic_id.unwrap_or_default()), + }) }) .collect(); @@ -347,11 +378,11 @@ fn proto_to_message(rpc: &proto::RPC) -> Rpc { .collect::>(); let topic_hash = TopicHash::from_raw(prune.topic_id.unwrap_or_default()); - prune_msgs.push(ControlAction::Prune { + prune_msgs.push(ControlAction::Prune(Prune { topic_hash, peers, backoff: prune.backoff, - }); + })); } control_msgs.extend(ihave_msgs); @@ -387,7 +418,7 @@ fn test_subscribe() { // - run JOIN(topic) let subscribe_topic = vec![String::from("test_subscribe")]; - let (gs, _, topic_hashes) = inject_nodes1() + let (gs, _, receivers, topic_hashes) = inject_nodes1() .peer_no(20) .topics(subscribe_topic) .to_subscribe(true) @@ -399,26 +430,24 @@ fn test_subscribe() { ); // collect all the subscriptions - let subscriptions = gs - .events - .iter() - .filter(|e| { - matches!( - e, - ToSwarm::NotifyHandler { - event: HandlerIn::Message(RpcOut::Subscribe(_)), - .. + let subscriptions = receivers + .into_values() + .fold(0, |mut collected_subscriptions, c| { + let priority = c.priority.get_ref(); + while !priority.is_empty() { + if let Ok(RpcOut::Subscribe(_)) = priority.try_recv() { + collected_subscriptions += 1 } - ) - }) - .count(); + } + collected_subscriptions + }); // we sent a subscribe to all known peers assert_eq!(subscriptions, 20); } -#[test] /// Test unsubscribe. +#[test] fn test_unsubscribe() { // Unsubscribe should: // - Remove the mesh entry for topic @@ -432,7 +461,7 @@ fn test_unsubscribe() { .collect::>(); // subscribe to topic_strings - let (mut gs, _, topic_hashes) = inject_nodes1() + let (mut gs, _, receivers, topic_hashes) = inject_nodes1() .peer_no(20) .topics(topic_strings) .to_subscribe(true) @@ -462,15 +491,16 @@ fn test_unsubscribe() { ); // collect all the subscriptions - let subscriptions = gs - .events - .iter() - .fold(0, |collected_subscriptions, e| match e { - ToSwarm::NotifyHandler { - event: HandlerIn::Message(RpcOut::Subscribe(_)), - .. - } => collected_subscriptions + 1, - _ => collected_subscriptions, + let subscriptions = receivers + .into_values() + .fold(0, |mut collected_subscriptions, c| { + let priority = c.priority.get_ref(); + while !priority.is_empty() { + if let Ok(RpcOut::Subscribe(_)) = priority.try_recv() { + collected_subscriptions += 1 + } + } + collected_subscriptions }); // we sent a unsubscribe to all known peers, for two topics @@ -485,8 +515,8 @@ fn test_unsubscribe() { } } -#[test] /// Test JOIN(topic) functionality. +#[test] fn test_join() { // The Join function should: // - Remove peers from fanout[topic] @@ -503,12 +533,15 @@ fn test_join() { .map(|t| Topic::new(t.clone())) .collect::>(); - let (mut gs, _, topic_hashes) = inject_nodes1() + let (mut gs, _, mut receivers, topic_hashes) = inject_nodes1() .peer_no(20) .topics(topic_strings) .to_subscribe(true) .create_network(); + // Flush previous GRAFT messages. + receivers = flush_events(&mut gs, receivers); + // unsubscribe, then call join to invoke functionality assert!( gs.unsubscribe(&topics[0]).unwrap(), @@ -531,24 +564,36 @@ fn test_join() { "Should have added 6 nodes to the mesh" ); - fn collect_grafts( - mut collected_grafts: Vec, - (_, controls): (&PeerId, &Vec), - ) -> Vec { - for c in controls.iter() { - if let ControlAction::Graft { topic_hash: _ } = c { - collected_grafts.push(c.clone()) + fn count_grafts( + receivers: HashMap, + ) -> (usize, HashMap) { + let mut new_receivers = HashMap::new(); + let mut acc = 0; + + for (peer_id, c) in receivers.into_iter() { + let priority = c.priority.get_ref(); + while !priority.is_empty() { + if let Ok(RpcOut::Graft(_)) = priority.try_recv() { + acc += 1; + } } + new_receivers.insert( + peer_id, + RpcReceiver { + priority_len: c.priority_len, + priority: c.priority, + non_priority: c.non_priority, + }, + ); } - collected_grafts + (acc, new_receivers) } // there should be mesh_n GRAFT messages. - let graft_messages = gs.control_pool.iter().fold(vec![], collect_grafts); + let (graft_messages, mut receivers) = count_grafts(receivers); assert_eq!( - graft_messages.len(), - 6, + graft_messages, 6, "There should be 6 grafts messages sent to peers" ); @@ -557,14 +602,37 @@ fn test_join() { gs.fanout .insert(topic_hashes[1].clone(), Default::default()); let mut new_peers: Vec = vec![]; + for _ in 0..3 { let random_peer = PeerId::random(); // inform the behaviour of a new peer + let address = "/ip4/127.0.0.1".parse::().unwrap(); + gs.handle_established_inbound_connection( + ConnectionId::new_unchecked(0), + random_peer, + &address, + &address, + ) + .unwrap(); + let sender = RpcSender::new(gs.config.connection_handler_queue_len()); + let receiver = sender.new_receiver(); + let connection_id = ConnectionId::new_unchecked(0); + gs.connected_peers.insert( + random_peer, + PeerConnections { + kind: PeerKind::Floodsub, + connections: vec![connection_id], + topics: Default::default(), + sender, + }, + ); + receivers.insert(random_peer, receiver); + gs.on_swarm_event(FromSwarm::ConnectionEstablished(ConnectionEstablished { peer_id: random_peer, - connection_id: ConnectionId::new_unchecked(0), + connection_id, endpoint: &ConnectedPoint::Dialer { - address: "/ip4/127.0.0.1".parse::().unwrap(), + address, role_override: Endpoint::Dialer, port_use: PortUse::Reuse, }, @@ -594,12 +662,12 @@ fn test_join() { ); } - // there should now be 12 graft messages to be sent - let graft_messages = gs.control_pool.iter().fold(vec![], collect_grafts); + // there should now 6 graft messages to be sent + let (graft_messages, _) = count_grafts(receivers); - assert!( - graft_messages.len() == 12, - "There should be 12 grafts messages sent to peers" + assert_eq!( + graft_messages, 6, + "There should be 6 grafts messages sent to peers" ); } @@ -617,7 +685,7 @@ fn test_publish_without_flood_publishing() { .unwrap(); let publish_topic = String::from("test_publish"); - let (mut gs, _, topic_hashes) = inject_nodes1() + let (mut gs, _, receivers, topic_hashes) = inject_nodes1() .peer_no(20) .topics(vec![publish_topic.clone()]) .to_subscribe(true) @@ -644,18 +712,16 @@ fn test_publish_without_flood_publishing() { gs.publish(Topic::new(publish_topic), publish_data).unwrap(); // Collect all publish messages - let publishes = gs - .events - .into_iter() - .fold(vec![], |mut collected_publish, e| match e { - ToSwarm::NotifyHandler { - event: HandlerIn::Message(RpcOut::Publish(message)), - .. - } => { - collected_publish.push(message); - collected_publish + let publishes = receivers + .into_values() + .fold(vec![], |mut collected_publish, c| { + let priority = c.priority.get_ref(); + while !priority.is_empty() { + if let Ok(RpcOut::Publish { message, .. }) = priority.try_recv() { + collected_publish.push(message); + } } - _ => collected_publish, + collected_publish }); // Transform the inbound message @@ -699,7 +765,7 @@ fn test_fanout() { .unwrap(); let fanout_topic = String::from("test_fanout"); - let (mut gs, _, topic_hashes) = inject_nodes1() + let (mut gs, _, receivers, topic_hashes) = inject_nodes1() .peer_no(20) .topics(vec![fanout_topic.clone()]) .to_subscribe(true) @@ -731,18 +797,16 @@ fn test_fanout() { ); // Collect all publish messages - let publishes = gs - .events - .into_iter() - .fold(vec![], |mut collected_publish, e| match e { - ToSwarm::NotifyHandler { - event: HandlerIn::Message(RpcOut::Publish(message)), - .. - } => { - collected_publish.push(message); - collected_publish + let publishes = receivers + .into_values() + .fold(vec![], |mut collected_publish, c| { + let priority = c.priority.get_ref(); + while !priority.is_empty() { + if let Ok(RpcOut::Publish { message, .. }) = priority.try_recv() { + collected_publish.push(message); + } } - _ => collected_publish, + collected_publish }); // Transform the inbound message @@ -770,10 +834,10 @@ fn test_fanout() { ); } -#[test] /// Test the gossipsub NetworkBehaviour peer connection logic. +#[test] fn test_inject_connected() { - let (gs, peers, topic_hashes) = inject_nodes1() + let (gs, peers, receivers, topic_hashes) = inject_nodes1() .peer_no(20) .topics(vec![String::from("topic1"), String::from("topic2")]) .to_subscribe(true) @@ -781,26 +845,20 @@ fn test_inject_connected() { // check that our subscriptions are sent to each of the peers // collect all the SendEvents - let subscriptions = gs - .events - .into_iter() - .filter_map(|e| match e { - ToSwarm::NotifyHandler { - event: HandlerIn::Message(RpcOut::Subscribe(topic)), - peer_id, - .. - } => Some((peer_id, topic)), - _ => None, - }) - .fold( - HashMap::>::new(), - |mut subs, (peer, sub)| { - let mut peer_subs = subs.remove(&peer).unwrap_or_default(); - peer_subs.push(sub.into_string()); - subs.insert(peer, peer_subs); - subs - }, - ); + let subscriptions = receivers.into_iter().fold( + HashMap::>::new(), + |mut collected_subscriptions, (peer, c)| { + let priority = c.priority.get_ref(); + while !priority.is_empty() { + if let Ok(RpcOut::Subscribe(topic)) = priority.try_recv() { + let mut peer_subs = collected_subscriptions.remove(&peer).unwrap_or_default(); + peer_subs.push(topic.into_string()); + collected_subscriptions.insert(peer, peer_subs); + } + } + collected_subscriptions + }, + ); // check that there are two subscriptions sent to each peer for peer_subs in subscriptions.values() { @@ -822,8 +880,8 @@ fn test_inject_connected() { } } -#[test] /// Test subscription handling +#[test] fn test_handle_received_subscriptions() { // For every subscription: // SUBSCRIBE: - Add subscribed topic to peer_topics for peer. @@ -835,7 +893,7 @@ fn test_handle_received_subscriptions() { .iter() .map(|&t| String::from(t)) .collect(); - let (mut gs, peers, topic_hashes) = inject_nodes1() + let (mut gs, peers, _receivers, topic_hashes) = inject_nodes1() .peer_no(20) .topics(topics) .to_subscribe(false) @@ -914,10 +972,9 @@ fn test_handle_received_subscriptions() { &peers[0], ); - let peer = gs.connected_peers.get(&peers[0]).unwrap().clone(); - assert_eq!( - peer.topics, - topic_hashes[1..3].iter().cloned().collect::>(), + let peer = gs.connected_peers.get(&peers[0]).unwrap(); + assert!( + peer.topics == topic_hashes[1..3].iter().cloned().collect::>(), "Peer should be subscribed to two topics" ); @@ -935,8 +992,8 @@ fn test_handle_received_subscriptions() { ); } -#[test] /// Test Gossipsub.get_random_peers() function +#[test] fn test_get_random_peers() { // generate a default Config let gs_config = ConfigBuilder::default() @@ -949,25 +1006,22 @@ fn test_get_random_peers() { // create a topic and fill it with some peers let topic_hash = Topic::new("Test").hash(); let mut peers = vec![]; - for _ in 0..20 { - peers.push(PeerId::random()) - } let mut topics = BTreeSet::new(); topics.insert(topic_hash.clone()); - gs.connected_peers = peers - .iter() - .map(|p| { - ( - *p, - PeerConnections { - kind: PeerKind::Gossipsubv1_1, - connections: vec![ConnectionId::new_unchecked(0)], - topics: topics.clone(), - }, - ) - }) - .collect(); + for _ in 0..20 { + let peer_id = PeerId::random(); + peers.push(peer_id); + gs.connected_peers.insert( + peer_id, + PeerConnections { + kind: PeerKind::Gossipsubv1_1, + connections: vec![ConnectionId::new_unchecked(0)], + topics: topics.clone(), + sender: RpcSender::new(gs.config.connection_handler_queue_len()), + }, + ); + } let random_peers = get_random_peers(&gs.connected_peers, &topic_hash, 5, |_| true); assert_eq!(random_peers.len(), 5, "Expected 5 peers to be returned"); @@ -997,7 +1051,7 @@ fn test_get_random_peers() { /// Tests that the correct message is sent when a peer asks for a message in our cache. #[test] fn test_handle_iwant_msg_cached() { - let (mut gs, peers, _) = inject_nodes1() + let (mut gs, peers, receivers, _) = inject_nodes1() .peer_no(20) .topics(Vec::new()) .to_subscribe(true) @@ -1025,18 +1079,17 @@ fn test_handle_iwant_msg_cached() { gs.handle_iwant(&peers[7], vec![msg_id.clone()]); // the messages we are sending - let sent_messages = gs.events.into_iter().fold( - Vec::::new(), - |mut collected_messages, e| match e { - ToSwarm::NotifyHandler { event, .. } => { - if let HandlerIn::Message(RpcOut::Forward(message)) = event { - collected_messages.push(message); + let sent_messages = receivers + .into_values() + .fold(vec![], |mut collected_messages, c| { + let non_priority = c.non_priority.get_ref(); + while !non_priority.is_empty() { + if let Ok(RpcOut::Forward { message, .. }) = non_priority.try_recv() { + collected_messages.push(message) } - collected_messages } - _ => collected_messages, - }, - ); + collected_messages + }); assert!( sent_messages @@ -1050,7 +1103,7 @@ fn test_handle_iwant_msg_cached() { /// Tests that messages are sent correctly depending on the shifting of the message cache. #[test] fn test_handle_iwant_msg_cached_shifted() { - let (mut gs, peers, _) = inject_nodes1() + let (mut gs, peers, mut receivers, _) = inject_nodes1() .peer_no(20) .topics(Vec::new()) .to_subscribe(true) @@ -1083,19 +1136,29 @@ fn test_handle_iwant_msg_cached_shifted() { gs.handle_iwant(&peers[7], vec![msg_id.clone()]); // is the message is being sent? - let message_exists = gs.events.iter().any(|e| match e { - ToSwarm::NotifyHandler { - event: HandlerIn::Message(RpcOut::Forward(message)), - .. - } => { - gs.config.message_id( - &gs.data_transform - .inbound_transform(message.clone()) - .unwrap(), - ) == msg_id + let mut message_exists = false; + receivers = receivers.into_iter().map(|(peer_id, c)| { + let non_priority = c.non_priority.get_ref(); + while !non_priority.is_empty() { + if matches!(non_priority.try_recv(), Ok(RpcOut::Forward{message, timeout: _ }) if + gs.config.message_id( + &gs.data_transform + .inbound_transform(message.clone()) + .unwrap(), + ) == msg_id) + { + message_exists = true; + } } - _ => false, - }); + ( + peer_id, + RpcReceiver { + priority_len: c.priority_len, + priority: c.priority, + non_priority: c.non_priority, + }, + ) + }).collect(); // default history_length is 5, expect no messages after shift > 5 if shift < 5 { assert!( @@ -1111,10 +1174,10 @@ fn test_handle_iwant_msg_cached_shifted() { } } +/// tests that an event is not created when a peers asks for a message not in our cache #[test] -// tests that an event is not created when a peers asks for a message not in our cache fn test_handle_iwant_msg_not_cached() { - let (mut gs, peers, _) = inject_nodes1() + let (mut gs, peers, _, _) = inject_nodes1() .peer_no(20) .topics(Vec::new()) .to_subscribe(true) @@ -1130,10 +1193,10 @@ fn test_handle_iwant_msg_not_cached() { ); } +/// tests that an event is created when a peer shares that it has a message we want #[test] -// tests that an event is created when a peer shares that it has a message we want fn test_handle_ihave_subscribed_and_msg_not_cached() { - let (mut gs, peers, topic_hashes) = inject_nodes1() + let (mut gs, peers, mut receivers, topic_hashes) = inject_nodes1() .peer_no(20) .topics(vec![String::from("topic1")]) .to_subscribe(true) @@ -1145,15 +1208,20 @@ fn test_handle_ihave_subscribed_and_msg_not_cached() { ); // check that we sent an IWANT request for `unknown id` - let iwant_exists = match gs.control_pool.get(&peers[7]) { - Some(controls) => controls.iter().any(|c| match c { - ControlAction::IWant { message_ids } => message_ids + let mut iwant_exists = false; + let receiver = receivers.remove(&peers[7]).unwrap(); + let non_priority = receiver.non_priority.get_ref(); + while !non_priority.is_empty() { + if let Ok(RpcOut::IWant(IWant { message_ids })) = non_priority.try_recv() { + if message_ids .iter() - .any(|m| *m == MessageId::new(b"unknown id")), - _ => false, - }), - _ => false, - }; + .any(|m| *m == MessageId::new(b"unknown id")) + { + iwant_exists = true; + break; + } + } + } assert!( iwant_exists, @@ -1161,11 +1229,11 @@ fn test_handle_ihave_subscribed_and_msg_not_cached() { ); } +/// tests that an event is not created when a peer shares that it has a message that +/// we already have #[test] -// tests that an event is not created when a peer shares that it has a message that -// we already have fn test_handle_ihave_subscribed_and_msg_cached() { - let (mut gs, peers, topic_hashes) = inject_nodes1() + let (mut gs, peers, _, topic_hashes) = inject_nodes1() .peer_no(20) .topics(vec![String::from("topic1")]) .to_subscribe(true) @@ -1183,11 +1251,11 @@ fn test_handle_ihave_subscribed_and_msg_cached() { ) } +/// test that an event is not created when a peer shares that it has a message in +/// a topic that we are not subscribed to #[test] -// test that an event is not created when a peer shares that it has a message in -// a topic that we are not subscribed to fn test_handle_ihave_not_subscribed() { - let (mut gs, peers, _) = inject_nodes1() + let (mut gs, peers, _, _) = inject_nodes1() .peer_no(20) .topics(vec![]) .to_subscribe(true) @@ -1209,11 +1277,11 @@ fn test_handle_ihave_not_subscribed() { ) } +/// tests that a peer is added to our mesh when we are both subscribed +/// to the same topic #[test] -// tests that a peer is added to our mesh when we are both subscribed -// to the same topic fn test_handle_graft_is_subscribed() { - let (mut gs, peers, topic_hashes) = inject_nodes1() + let (mut gs, peers, _, topic_hashes) = inject_nodes1() .peer_no(20) .topics(vec![String::from("topic1")]) .to_subscribe(true) @@ -1227,11 +1295,11 @@ fn test_handle_graft_is_subscribed() { ); } +/// tests that a peer is not added to our mesh when they are subscribed to +/// a topic that we are not #[test] -// tests that a peer is not added to our mesh when they are subscribed to -// a topic that we are not fn test_handle_graft_is_not_subscribed() { - let (mut gs, peers, topic_hashes) = inject_nodes1() + let (mut gs, peers, _, topic_hashes) = inject_nodes1() .peer_no(20) .topics(vec![String::from("topic1")]) .to_subscribe(true) @@ -1248,15 +1316,15 @@ fn test_handle_graft_is_not_subscribed() { ); } +/// tests multiple topics in a single graft message #[test] -// tests multiple topics in a single graft message fn test_handle_graft_multiple_topics() { let topics: Vec = ["topic1", "topic2", "topic3", "topic4"] .iter() .map(|&t| String::from(t)) .collect(); - let (mut gs, peers, topic_hashes) = inject_nodes1() + let (mut gs, peers, _, topic_hashes) = inject_nodes1() .peer_no(20) .topics(topics) .to_subscribe(true) @@ -1283,10 +1351,10 @@ fn test_handle_graft_multiple_topics() { ); } +/// tests that a peer is removed from our mesh #[test] -// tests that a peer is removed from our mesh fn test_handle_prune_peer_in_mesh() { - let (mut gs, peers, topic_hashes) = inject_nodes1() + let (mut gs, peers, _, topic_hashes) = inject_nodes1() .peer_no(20) .topics(vec![String::from("topic1")]) .to_subscribe(true) @@ -1313,36 +1381,68 @@ fn test_handle_prune_peer_in_mesh() { ); } -fn count_control_msgs( - gs: &Behaviour, - mut filter: impl FnMut(&PeerId, &ControlAction) -> bool, -) -> usize { - gs.control_pool - .iter() - .map(|(peer_id, actions)| actions.iter().filter(|m| filter(peer_id, m)).count()) - .sum::() - + gs.events - .iter() - .filter(|e| match e { - ToSwarm::NotifyHandler { - peer_id, - event: HandlerIn::Message(RpcOut::Control(action)), - .. - } => filter(peer_id, action), - _ => false, - }) - .count() +fn count_control_msgs( + receivers: HashMap, + mut filter: impl FnMut(&PeerId, &RpcOut) -> bool, +) -> (usize, HashMap) { + let mut new_receivers = HashMap::new(); + let mut collected_messages = 0; + for (peer_id, c) in receivers.into_iter() { + let priority = c.priority.get_ref(); + let non_priority = c.non_priority.get_ref(); + while !priority.is_empty() || !non_priority.is_empty() { + if let Ok(rpc) = priority.try_recv() { + if filter(&peer_id, &rpc) { + collected_messages += 1; + } + } + if let Ok(rpc) = non_priority.try_recv() { + if filter(&peer_id, &rpc) { + collected_messages += 1; + } + } + } + new_receivers.insert( + peer_id, + RpcReceiver { + priority_len: c.priority_len, + priority: c.priority, + non_priority: c.non_priority, + }, + ); + } + (collected_messages, new_receivers) } -fn flush_events(gs: &mut Behaviour) { - gs.control_pool.clear(); +fn flush_events( + gs: &mut Behaviour, + receivers: HashMap, +) -> HashMap { gs.events.clear(); + let mut new_receivers = HashMap::new(); + for (peer_id, c) in receivers.into_iter() { + let priority = c.priority.get_ref(); + let non_priority = c.non_priority.get_ref(); + while !priority.is_empty() || !non_priority.is_empty() { + let _ = priority.try_recv(); + let _ = non_priority.try_recv(); + } + new_receivers.insert( + peer_id, + RpcReceiver { + priority_len: c.priority_len, + priority: c.priority, + non_priority: c.non_priority, + }, + ); + } + new_receivers } +/// tests that a peer added as explicit peer gets connected to #[test] -// tests that a peer added as explicit peer gets connected to fn test_explicit_peer_gets_connected() { - let (mut gs, _, _) = inject_nodes1() + let (mut gs, _, _, _) = inject_nodes1() .peer_no(0) .topics(Vec::new()) .to_subscribe(true) @@ -1375,7 +1475,7 @@ fn test_explicit_peer_reconnects() { .check_explicit_peers_ticks(2) .build() .unwrap(); - let (mut gs, others, _) = inject_nodes1() + let (mut gs, others, receivers, _) = inject_nodes1() .peer_no(1) .topics(Vec::new()) .to_subscribe(true) @@ -1387,7 +1487,7 @@ fn test_explicit_peer_reconnects() { //add peer as explicit peer gs.add_explicit_peer(peer); - flush_events(&mut gs); + flush_events(&mut gs, receivers); //disconnect peer disconnect_peer(&mut gs, peer); @@ -1425,7 +1525,7 @@ fn test_explicit_peer_reconnects() { #[test] fn test_handle_graft_explicit_peer() { - let (mut gs, peers, topic_hashes) = inject_nodes1() + let (mut gs, peers, receivers, topic_hashes) = inject_nodes1() .peer_no(1) .topics(vec![String::from("topic1"), String::from("topic2")]) .to_subscribe(true) @@ -1442,21 +1542,24 @@ fn test_handle_graft_explicit_peer() { assert!(gs.mesh[&topic_hashes[1]].is_empty()); //check prunes - assert!( - count_control_msgs(&gs, |peer_id, m| peer_id == peer + let (control_msgs, _) = count_control_msgs(receivers, |peer_id, m| { + peer_id == peer && match m { - ControlAction::Prune { topic_hash, .. } => - topic_hash == &topic_hashes[0] || topic_hash == &topic_hashes[1], + RpcOut::Prune(Prune { topic_hash, .. }) => { + topic_hash == &topic_hashes[0] || topic_hash == &topic_hashes[1] + } _ => false, - }) - >= 2, + } + }); + assert!( + control_msgs >= 2, "Not enough prunes sent when grafting from explicit peer" ); } #[test] fn explicit_peers_not_added_to_mesh_on_receiving_subscription() { - let (gs, peers, topic_hashes) = inject_nodes1() + let (gs, peers, receivers, topic_hashes) = inject_nodes1() .peer_no(2) .topics(vec![String::from("topic1")]) .to_subscribe(true) @@ -1471,25 +1574,27 @@ fn explicit_peers_not_added_to_mesh_on_receiving_subscription() { ); //assert that graft gets created to non-explicit peer + let (control_msgs, receivers) = count_control_msgs(receivers, |peer_id, m| { + peer_id == &peers[1] && matches!(m, RpcOut::Graft { .. }) + }); assert!( - count_control_msgs(&gs, |peer_id, m| peer_id == &peers[1] - && matches!(m, ControlAction::Graft { .. })) - >= 1, + control_msgs >= 1, "No graft message got created to non-explicit peer" ); //assert that no graft gets created to explicit peer + let (control_msgs, _) = count_control_msgs(receivers, |peer_id, m| { + peer_id == &peers[0] && matches!(m, RpcOut::Graft { .. }) + }); assert_eq!( - count_control_msgs(&gs, |peer_id, m| peer_id == &peers[0] - && matches!(m, ControlAction::Graft { .. })), - 0, + control_msgs, 0, "A graft message got created to an explicit peer" ); } #[test] fn do_not_graft_explicit_peer() { - let (mut gs, others, topic_hashes) = inject_nodes1() + let (mut gs, others, receivers, topic_hashes) = inject_nodes1() .peer_no(1) .topics(vec![String::from("topic")]) .to_subscribe(true) @@ -1503,17 +1608,18 @@ fn do_not_graft_explicit_peer() { assert_eq!(gs.mesh[&topic_hashes[0]], BTreeSet::new()); //assert that no graft gets created to explicit peer + let (control_msgs, _) = count_control_msgs(receivers, |peer_id, m| { + peer_id == &others[0] && matches!(m, RpcOut::Graft { .. }) + }); assert_eq!( - count_control_msgs(&gs, |peer_id, m| peer_id == &others[0] - && matches!(m, ControlAction::Graft { .. })), - 0, + control_msgs, 0, "A graft message got created to an explicit peer" ); } #[test] fn do_forward_messages_to_explicit_peers() { - let (mut gs, peers, topic_hashes) = inject_nodes1() + let (mut gs, peers, receivers, topic_hashes) = inject_nodes1() .peer_no(2) .topics(vec![String::from("topic1"), String::from("topic2")]) .to_subscribe(true) @@ -1533,21 +1639,16 @@ fn do_forward_messages_to_explicit_peers() { validated: true, }; gs.handle_received_message(message.clone(), &local_id); - assert_eq!( - gs.events - .iter() - .filter(|e| match e { - ToSwarm::NotifyHandler { - peer_id, - event: HandlerIn::Message(RpcOut::Forward(m)), - .. - } => { - peer_id == &peers[0] && m.data == message.data + receivers.into_iter().fold(0, |mut fwds, (peer_id, c)| { + let non_priority = c.non_priority.get_ref(); + while !non_priority.is_empty() { + if matches!(non_priority.try_recv(), Ok(RpcOut::Forward{message: m, timeout: _}) if peer_id == peers[0] && m.data == message.data) { + fwds +=1; + } } - _ => false, - }) - .count(), + fwds + }), 1, "The message did not get forwarded to the explicit peer" ); @@ -1555,7 +1656,7 @@ fn do_forward_messages_to_explicit_peers() { #[test] fn explicit_peers_not_added_to_mesh_on_subscribe() { - let (mut gs, peers, _) = inject_nodes1() + let (mut gs, peers, receivers, _) = inject_nodes1() .peer_no(2) .topics(Vec::new()) .to_subscribe(true) @@ -1583,25 +1684,27 @@ fn explicit_peers_not_added_to_mesh_on_subscribe() { assert_eq!(gs.mesh[&topic_hash], vec![peers[1]].into_iter().collect()); //assert that graft gets created to non-explicit peer + let (control_msgs, receivers) = count_control_msgs(receivers, |peer_id, m| { + peer_id == &peers[1] && matches!(m, RpcOut::Graft { .. }) + }); assert!( - count_control_msgs(&gs, |peer_id, m| peer_id == &peers[1] - && matches!(m, ControlAction::Graft { .. })) - > 0, + control_msgs > 0, "No graft message got created to non-explicit peer" ); //assert that no graft gets created to explicit peer + let (control_msgs, _) = count_control_msgs(receivers, |peer_id, m| { + peer_id == &peers[0] && matches!(m, RpcOut::Graft { .. }) + }); assert_eq!( - count_control_msgs(&gs, |peer_id, m| peer_id == &peers[0] - && matches!(m, ControlAction::Graft { .. })), - 0, + control_msgs, 0, "A graft message got created to an explicit peer" ); } #[test] fn explicit_peers_not_added_to_mesh_from_fanout_on_subscribe() { - let (mut gs, peers, _) = inject_nodes1() + let (mut gs, peers, receivers, _) = inject_nodes1() .peer_no(2) .topics(Vec::new()) .to_subscribe(true) @@ -1632,25 +1735,27 @@ fn explicit_peers_not_added_to_mesh_from_fanout_on_subscribe() { assert_eq!(gs.mesh[&topic_hash], vec![peers[1]].into_iter().collect()); //assert that graft gets created to non-explicit peer + let (control_msgs, receivers) = count_control_msgs(receivers, |peer_id, m| { + peer_id == &peers[1] && matches!(m, RpcOut::Graft { .. }) + }); assert!( - count_control_msgs(&gs, |peer_id, m| peer_id == &peers[1] - && matches!(m, ControlAction::Graft { .. })) - >= 1, + control_msgs >= 1, "No graft message got created to non-explicit peer" ); //assert that no graft gets created to explicit peer + let (control_msgs, _) = count_control_msgs(receivers, |peer_id, m| { + peer_id == &peers[0] && matches!(m, RpcOut::Graft { .. }) + }); assert_eq!( - count_control_msgs(&gs, |peer_id, m| peer_id == &peers[0] - && matches!(m, ControlAction::Graft { .. })), - 0, + control_msgs, 0, "A graft message got created to an explicit peer" ); } #[test] fn no_gossip_gets_sent_to_explicit_peers() { - let (mut gs, peers, topic_hashes) = inject_nodes1() + let (mut gs, peers, mut receivers, topic_hashes) = inject_nodes1() .peer_no(2) .topics(vec![String::from("topic1"), String::from("topic2")]) .to_subscribe(true) @@ -1679,25 +1784,24 @@ fn no_gossip_gets_sent_to_explicit_peers() { } //assert that no gossip gets sent to explicit peer - assert_eq!( - gs.control_pool - .get(&peers[0]) - .unwrap_or(&Vec::new()) - .iter() - .filter(|m| matches!(m, ControlAction::IHave { .. })) - .count(), - 0, - "Gossip got emitted to explicit peer" - ); + let receiver = receivers.remove(&peers[0]).unwrap(); + let mut gossips = 0; + let non_priority = receiver.non_priority.get_ref(); + while !non_priority.is_empty() { + if let Ok(RpcOut::IHave(_)) = non_priority.try_recv() { + gossips += 1; + } + } + assert_eq!(gossips, 0, "Gossip got emitted to explicit peer"); } -// Tests the mesh maintenance addition +/// Tests the mesh maintenance addition #[test] fn test_mesh_addition() { let config: Config = Config::default(); // Adds mesh_low peers and PRUNE 2 giving us a deficit. - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _receivers, topics) = inject_nodes1() .peer_no(config.mesh_n() + 1) .topics(vec!["test".into()]) .to_subscribe(true) @@ -1725,7 +1829,7 @@ fn test_mesh_addition() { assert_eq!(gs.mesh.get(&topics[0]).unwrap().len(), config.mesh_n()); } -// Tests the mesh maintenance subtraction +/// Tests the mesh maintenance subtraction #[test] fn test_mesh_subtraction() { let config = Config::default(); @@ -1733,7 +1837,7 @@ fn test_mesh_subtraction() { // Adds mesh_low peers and PRUNE 2 giving us a deficit. let n = config.mesh_n_high() + 10; //make all outbound connections so that we allow grafting to all - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _receivers, topics) = inject_nodes1() .peer_no(n) .topics(vec!["test".into()]) .to_subscribe(true) @@ -1757,7 +1861,7 @@ fn test_mesh_subtraction() { fn test_connect_to_px_peers_on_handle_prune() { let config: Config = Config::default(); - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _, topics) = inject_nodes1() .peer_no(1) .topics(vec!["test".into()]) .to_subscribe(true) @@ -1813,7 +1917,7 @@ fn test_send_px_and_backoff_in_prune() { let config: Config = Config::default(); //build mesh with enough peers for px - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, receivers, topics) = inject_nodes1() .peer_no(config.prune_peers() + 1) .topics(vec!["test".into()]) .to_subscribe(true) @@ -1829,24 +1933,25 @@ fn test_send_px_and_backoff_in_prune() { ); //check prune message - assert_eq!( - count_control_msgs(&gs, |peer_id, m| peer_id == &peers[0] + let (control_msgs, _) = count_control_msgs(receivers, |peer_id, m| { + peer_id == &peers[0] && match m { - ControlAction::Prune { + RpcOut::Prune(Prune { topic_hash, peers, backoff, - } => + }) => { topic_hash == &topics[0] && peers.len() == config.prune_peers() && //all peers are different peers.iter().collect::>().len() == config.prune_peers() && - backoff.unwrap() == config.prune_backoff().as_secs(), + backoff.unwrap() == config.prune_backoff().as_secs() + } _ => false, - }), - 1 - ); + } + }); + assert_eq!(control_msgs, 1); } #[test] @@ -1854,7 +1959,7 @@ fn test_prune_backoffed_peer_on_graft() { let config: Config = Config::default(); //build mesh with enough peers for px - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, receivers, topics) = inject_nodes1() .peer_no(config.prune_peers() + 1) .topics(vec!["test".into()]) .to_subscribe(true) @@ -1871,28 +1976,29 @@ fn test_prune_backoffed_peer_on_graft() { ); //ignore all messages until now - gs.events.clear(); + let receivers = flush_events(&mut gs, receivers); //handle graft gs.handle_graft(&peers[0], vec![topics[0].clone()]); //check prune message - assert_eq!( - count_control_msgs(&gs, |peer_id, m| peer_id == &peers[0] + let (control_msgs, _) = count_control_msgs(receivers, |peer_id, m| { + peer_id == &peers[0] && match m { - ControlAction::Prune { + RpcOut::Prune(Prune { topic_hash, peers, backoff, - } => + }) => { topic_hash == &topics[0] && //no px in this case peers.is_empty() && - backoff.unwrap() == config.prune_backoff().as_secs(), + backoff.unwrap() == config.prune_backoff().as_secs() + } _ => false, - }), - 1 - ); + } + }); + assert_eq!(control_msgs, 1); } #[test] @@ -1903,7 +2009,7 @@ fn test_do_not_graft_within_backoff_period() { .build() .unwrap(); //only one peer => mesh too small and will try to regraft as early as possible - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, receivers, topics) = inject_nodes1() .peer_no(1) .topics(vec!["test".into()]) .to_subscribe(true) @@ -1914,7 +2020,7 @@ fn test_do_not_graft_within_backoff_period() { gs.handle_prune(&peers[0], vec![(topics[0].clone(), Vec::new(), Some(1))]); //forget all events until now - flush_events(&mut gs); + let receivers = flush_events(&mut gs, receivers); //call heartbeat gs.heartbeat(); @@ -1927,9 +2033,10 @@ fn test_do_not_graft_within_backoff_period() { //Check that no graft got created (we have backoff_slack = 1 therefore one more heartbeat // is needed). + let (control_msgs, receivers) = + count_control_msgs(receivers, |_, m| matches!(m, RpcOut::Graft { .. })); assert_eq!( - count_control_msgs(&gs, |_, m| matches!(m, ControlAction::Graft { .. })), - 0, + control_msgs, 0, "Graft message created too early within backoff period" ); @@ -1938,8 +2045,9 @@ fn test_do_not_graft_within_backoff_period() { gs.heartbeat(); //check that graft got created + let (control_msgs, _) = count_control_msgs(receivers, |_, m| matches!(m, RpcOut::Graft { .. })); assert!( - count_control_msgs(&gs, |_, m| matches!(m, ControlAction::Graft { .. })) > 0, + control_msgs > 0, "No graft message was created after backoff period" ); } @@ -1954,7 +2062,7 @@ fn test_do_not_graft_within_default_backoff_period_after_receiving_prune_without .build() .unwrap(); //only one peer => mesh too small and will try to regraft as early as possible - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, receivers, topics) = inject_nodes1() .peer_no(1) .topics(vec!["test".into()]) .to_subscribe(true) @@ -1965,7 +2073,7 @@ fn test_do_not_graft_within_default_backoff_period_after_receiving_prune_without gs.handle_prune(&peers[0], vec![(topics[0].clone(), Vec::new(), None)]); //forget all events until now - flush_events(&mut gs); + let receivers = flush_events(&mut gs, receivers); //call heartbeat gs.heartbeat(); @@ -1976,9 +2084,10 @@ fn test_do_not_graft_within_default_backoff_period_after_receiving_prune_without //Check that no graft got created (we have backoff_slack = 1 therefore one more heartbeat // is needed). + let (control_msgs, receivers) = + count_control_msgs(receivers, |_, m| matches!(m, RpcOut::Graft { .. })); assert_eq!( - count_control_msgs(&gs, |_, m| matches!(m, ControlAction::Graft { .. })), - 0, + control_msgs, 0, "Graft message created too early within backoff period" ); @@ -1987,8 +2096,9 @@ fn test_do_not_graft_within_default_backoff_period_after_receiving_prune_without gs.heartbeat(); //check that graft got created + let (control_msgs, _) = count_control_msgs(receivers, |_, m| matches!(m, RpcOut::Graft { .. })); assert!( - count_control_msgs(&gs, |_, m| matches!(m, ControlAction::Graft { .. })) > 0, + control_msgs > 0, "No graft message was created after backoff period" ); } @@ -2007,7 +2117,7 @@ fn test_unsubscribe_backoff() { let topic = String::from("test"); // only one peer => mesh too small and will try to regraft as early as possible - let (mut gs, _, topics) = inject_nodes1() + let (mut gs, _, receivers, topics) = inject_nodes1() .peer_no(1) .topics(vec![topic.clone()]) .to_subscribe(true) @@ -2016,19 +2126,19 @@ fn test_unsubscribe_backoff() { let _ = gs.unsubscribe(&Topic::new(topic)); + let (control_msgs, receivers) = count_control_msgs(receivers, |_, m| match m { + RpcOut::Prune(Prune { backoff, .. }) => backoff == &Some(1), + _ => false, + }); assert_eq!( - count_control_msgs(&gs, |_, m| match m { - ControlAction::Prune { backoff, .. } => backoff == &Some(1), - _ => false, - }), - 1, + control_msgs, 1, "Peer should be pruned with `unsubscribe_backoff`." ); let _ = gs.subscribe(&Topic::new(topics[0].to_string())); // forget all events until now - flush_events(&mut gs); + let receivers = flush_events(&mut gs, receivers); // call heartbeat gs.heartbeat(); @@ -2041,9 +2151,10 @@ fn test_unsubscribe_backoff() { // Check that no graft got created (we have backoff_slack = 1 therefore one more heartbeat // is needed). + let (control_msgs, receivers) = + count_control_msgs(receivers, |_, m| matches!(m, RpcOut::Graft { .. })); assert_eq!( - count_control_msgs(&gs, |_, m| matches!(m, ControlAction::Graft { .. })), - 0, + control_msgs, 0, "Graft message created too early within backoff period" ); @@ -2052,8 +2163,9 @@ fn test_unsubscribe_backoff() { gs.heartbeat(); // check that graft got created + let (control_msgs, _) = count_control_msgs(receivers, |_, m| matches!(m, RpcOut::Graft { .. })); assert!( - count_control_msgs(&gs, |_, m| matches!(m, ControlAction::Graft { .. })) > 0, + control_msgs > 0, "No graft message was created after backoff period" ); } @@ -2064,7 +2176,7 @@ fn test_flood_publish() { let topic = "test"; // Adds more peers than mesh can hold to test flood publishing - let (mut gs, _, _) = inject_nodes1() + let (mut gs, _, receivers, _) = inject_nodes1() .peer_no(config.mesh_n_high() + 10) .topics(vec![topic.into()]) .to_subscribe(true) @@ -2075,17 +2187,16 @@ fn test_flood_publish() { gs.publish(Topic::new(topic), publish_data).unwrap(); // Collect all publish messages - let publishes = gs - .events - .into_iter() - .fold(vec![], |mut collected_publish, e| match e { - ToSwarm::NotifyHandler { event, .. } => { - if let HandlerIn::Message(RpcOut::Publish(message)) = event { + let publishes = receivers + .into_values() + .fold(vec![], |mut collected_publish, c| { + let priority = c.priority.get_ref(); + while !priority.is_empty() { + if let Ok(RpcOut::Publish { message, .. }) = priority.try_recv() { collected_publish.push(message); } - collected_publish } - _ => collected_publish, + collected_publish }); // Transform the inbound message @@ -2120,7 +2231,7 @@ fn test_gossip_to_at_least_gossip_lazy_peers() { //add more peers than in mesh to test gossipping //by default only mesh_n_low peers will get added to mesh - let (mut gs, _, topic_hashes) = inject_nodes1() + let (mut gs, _, receivers, topic_hashes) = inject_nodes1() .peer_no(config.mesh_n_low() + config.gossip_lazy() + 1) .topics(vec!["topic".into()]) .to_subscribe(true) @@ -2147,16 +2258,14 @@ fn test_gossip_to_at_least_gossip_lazy_peers() { let msg_id = gs.config.message_id(message); //check that exactly config.gossip_lazy() many gossip messages were sent. - assert_eq!( - count_control_msgs(&gs, |_, action| match action { - ControlAction::IHave { - topic_hash, - message_ids, - } => topic_hash == &topic_hashes[0] && message_ids.iter().any(|id| id == &msg_id), - _ => false, - }), - config.gossip_lazy() - ); + let (control_msgs, _) = count_control_msgs(receivers, |_, action| match action { + RpcOut::IHave(IHave { + topic_hash, + message_ids, + }) => topic_hash == &topic_hashes[0] && message_ids.iter().any(|id| id == &msg_id), + _ => false, + }); + assert_eq!(control_msgs, config.gossip_lazy()); } #[test] @@ -2165,7 +2274,7 @@ fn test_gossip_to_at_most_gossip_factor_peers() { //add a lot of peers let m = config.mesh_n_low() + config.gossip_lazy() * (2.0 / config.gossip_factor()) as usize; - let (mut gs, _, topic_hashes) = inject_nodes1() + let (mut gs, _, receivers, topic_hashes) = inject_nodes1() .peer_no(m) .topics(vec!["topic".into()]) .to_subscribe(true) @@ -2191,14 +2300,15 @@ fn test_gossip_to_at_most_gossip_factor_peers() { let msg_id = gs.config.message_id(message); //check that exactly config.gossip_lazy() many gossip messages were sent. + let (control_msgs, _) = count_control_msgs(receivers, |_, action| match action { + RpcOut::IHave(IHave { + topic_hash, + message_ids, + }) => topic_hash == &topic_hashes[0] && message_ids.iter().any(|id| id == &msg_id), + _ => false, + }); assert_eq!( - count_control_msgs(&gs, |_, action| match action { - ControlAction::IHave { - topic_hash, - message_ids, - } => topic_hash == &topic_hashes[0] && message_ids.iter().any(|id| id == &msg_id), - _ => false, - }), + control_msgs, ((m - config.mesh_n_low()) as f64 * config.gossip_factor()) as usize ); } @@ -2208,7 +2318,7 @@ fn test_accept_only_outbound_peer_grafts_when_mesh_full() { let config: Config = Config::default(); //enough peers to fill the mesh - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _, topics) = inject_nodes1() .peer_no(config.mesh_n_high()) .topics(vec!["test".into()]) .to_subscribe(true) @@ -2223,8 +2333,8 @@ fn test_accept_only_outbound_peer_grafts_when_mesh_full() { assert_eq!(gs.mesh[&topics[0]].len(), config.mesh_n_high()); //create an outbound and an inbound peer - let inbound = add_peer(&mut gs, &topics, false, false); - let outbound = add_peer(&mut gs, &topics, true, false); + let (inbound, _in_reciver) = add_peer(&mut gs, &topics, false, false); + let (outbound, _out_receiver) = add_peer(&mut gs, &topics, true, false); //send grafts gs.handle_graft(&inbound, vec![topics[0].clone()]); @@ -2254,7 +2364,7 @@ fn test_do_not_remove_too_many_outbound_peers() { .unwrap(); //fill the mesh with inbound connections - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _receivers, topics) = inject_nodes1() .peer_no(n) .topics(vec!["test".into()]) .to_subscribe(true) @@ -2269,7 +2379,7 @@ fn test_do_not_remove_too_many_outbound_peers() { //create m outbound connections and graft (we will accept the graft) let mut outbound = HashSet::new(); for _ in 0..m { - let peer = add_peer(&mut gs, &topics, true, false); + let (peer, _) = add_peer(&mut gs, &topics, true, false); outbound.insert(peer); gs.handle_graft(&peer, topics.clone()); } @@ -2292,7 +2402,7 @@ fn test_add_outbound_peers_if_min_is_not_satisfied() { let config: Config = Config::default(); // Fill full mesh with inbound peers - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _, topics) = inject_nodes1() .peer_no(config.mesh_n_high()) .topics(vec!["test".into()]) .to_subscribe(true) @@ -2304,8 +2414,9 @@ fn test_add_outbound_peers_if_min_is_not_satisfied() { } //create config.mesh_outbound_min() many outbound connections without grafting + let mut peers = vec![]; for _ in 0..config.mesh_outbound_min() { - add_peer(&mut gs, &topics, true, false); + peers.push(add_peer(&mut gs, &topics, true, false)); } // Nothing changed in the mesh yet @@ -2326,7 +2437,7 @@ fn test_prune_negative_scored_peers() { let config = Config::default(); //build mesh with one peer - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, receivers, topics) = inject_nodes1() .peer_no(1) .topics(vec!["test".into()]) .to_subscribe(true) @@ -2349,29 +2460,30 @@ fn test_prune_negative_scored_peers() { assert!(gs.mesh[&topics[0]].is_empty()); //check prune message - assert_eq!( - count_control_msgs(&gs, |peer_id, m| peer_id == &peers[0] + let (control_msgs, _) = count_control_msgs(receivers, |peer_id, m| { + peer_id == &peers[0] && match m { - ControlAction::Prune { + RpcOut::Prune(Prune { topic_hash, peers, backoff, - } => + }) => { topic_hash == &topics[0] && //no px in this case peers.is_empty() && - backoff.unwrap() == config.prune_backoff().as_secs(), + backoff.unwrap() == config.prune_backoff().as_secs() + } _ => false, - }), - 1 - ); + } + }); + assert_eq!(control_msgs, 1); } #[test] fn test_dont_graft_to_negative_scored_peers() { let config = Config::default(); //init full mesh - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _, topics) = inject_nodes1() .peer_no(config.mesh_n_high()) .topics(vec!["test".into()]) .to_subscribe(true) @@ -2383,8 +2495,8 @@ fn test_dont_graft_to_negative_scored_peers() { .create_network(); //add two additional peers that will not be part of the mesh - let p1 = add_peer(&mut gs, &topics, false, false); - let p2 = add_peer(&mut gs, &topics, false, false); + let (p1, _receiver1) = add_peer(&mut gs, &topics, false, false); + let (p2, _receiver2) = add_peer(&mut gs, &topics, false, false); //reduce score of p1 to negative gs.peer_score.as_mut().unwrap().0.add_penalty(&p1, 1); @@ -2410,7 +2522,7 @@ fn test_ignore_px_from_negative_scored_peer() { let config = Config::default(); //build mesh with one peer - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _, topics) = inject_nodes1() .peer_no(1) .topics(vec!["test".into()]) .to_subscribe(true) @@ -2457,7 +2569,7 @@ fn test_only_send_nonnegative_scoring_peers_in_px() { .unwrap(); // Build mesh with three peer - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, receivers, topics) = inject_nodes1() .peer_no(3) .topics(vec!["test".into()]) .to_subscribe(true) @@ -2483,21 +2595,22 @@ fn test_only_send_nonnegative_scoring_peers_in_px() { ); // Check that px in prune message only contains third peer - assert_eq!( - count_control_msgs(&gs, |peer_id, m| peer_id == &peers[1] + let (control_msgs, _) = count_control_msgs(receivers, |peer_id, m| { + peer_id == &peers[1] && match m { - ControlAction::Prune { + RpcOut::Prune(Prune { topic_hash, peers: px, .. - } => + }) => { topic_hash == &topics[0] && px.len() == 1 - && px[0].peer_id.as_ref().unwrap() == &peers[2], + && px[0].peer_id.as_ref().unwrap() == &peers[2] + } _ => false, - }), - 1 - ); + } + }); + assert_eq!(control_msgs, 1); } #[test] @@ -2510,7 +2623,7 @@ fn test_do_not_gossip_to_peers_below_gossip_threshold() { }; // Build full mesh - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, mut receivers, topics) = inject_nodes1() .peer_no(config.mesh_n_high()) .topics(vec!["test".into()]) .to_subscribe(true) @@ -2524,8 +2637,10 @@ fn test_do_not_gossip_to_peers_below_gossip_threshold() { } // Add two additional peers that will not be part of the mesh - let p1 = add_peer(&mut gs, &topics, false, false); - let p2 = add_peer(&mut gs, &topics, false, false); + let (p1, receiver1) = add_peer(&mut gs, &topics, false, false); + receivers.insert(p1, receiver1); + let (p2, receiver2) = add_peer(&mut gs, &topics, false, false); + receivers.insert(p2, receiver2); // Reduce score of p1 below peer_score_thresholds.gossip_threshold // note that penalties get squared so two penalties means a score of @@ -2556,23 +2671,21 @@ fn test_do_not_gossip_to_peers_below_gossip_threshold() { gs.emit_gossip(); // Check that exactly one gossip messages got sent and it got sent to p2 - assert_eq!( - count_control_msgs(&gs, |peer, action| match action { - ControlAction::IHave { - topic_hash, - message_ids, - } => { - if topic_hash == &topics[0] && message_ids.iter().any(|id| id == &msg_id) { - assert_eq!(peer, &p2); - true - } else { - false - } + let (control_msgs, _) = count_control_msgs(receivers, |peer, action| match action { + RpcOut::IHave(IHave { + topic_hash, + message_ids, + }) => { + if topic_hash == &topics[0] && message_ids.iter().any(|id| id == &msg_id) { + assert_eq!(peer, &p2); + true + } else { + false } - _ => false, - }), - 1 - ); + } + _ => false, + }); + assert_eq!(control_msgs, 1); } #[test] @@ -2585,7 +2698,7 @@ fn test_iwant_msg_from_peer_below_gossip_threshold_gets_ignored() { }; // Build full mesh - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, mut receivers, topics) = inject_nodes1() .peer_no(config.mesh_n_high()) .topics(vec!["test".into()]) .to_subscribe(true) @@ -2601,8 +2714,10 @@ fn test_iwant_msg_from_peer_below_gossip_threshold_gets_ignored() { } // Add two additional peers that will not be part of the mesh - let p1 = add_peer(&mut gs, &topics, false, false); - let p2 = add_peer(&mut gs, &topics, false, false); + let (p1, receiver1) = add_peer(&mut gs, &topics, false, false); + receivers.insert(p1, receiver1); + let (p2, receiver2) = add_peer(&mut gs, &topics, false, false); + receivers.insert(p2, receiver2); // Reduce score of p1 below peer_score_thresholds.gossip_threshold // note that penalties get squared so two penalties means a score of @@ -2633,18 +2748,18 @@ fn test_iwant_msg_from_peer_below_gossip_threshold_gets_ignored() { gs.handle_iwant(&p2, vec![msg_id.clone()]); // the messages we are sending - let sent_messages = gs - .events - .into_iter() - .fold(vec![], |mut collected_messages, e| match e { - ToSwarm::NotifyHandler { event, peer_id, .. } => { - if let HandlerIn::Message(RpcOut::Forward(message)) = event { - collected_messages.push((peer_id, message)); + let sent_messages = + receivers + .into_iter() + .fold(vec![], |mut collected_messages, (peer_id, c)| { + let non_priority = c.non_priority.get_ref(); + while !non_priority.is_empty() { + if let Ok(RpcOut::Forward { message, .. }) = non_priority.try_recv() { + collected_messages.push((peer_id, message)); + } } collected_messages - } - _ => collected_messages, - }); + }); //the message got sent to p2 assert!(sent_messages @@ -2673,7 +2788,7 @@ fn test_ihave_msg_from_peer_below_gossip_threshold_gets_ignored() { ..PeerScoreThresholds::default() }; //build full mesh - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, mut receivers, topics) = inject_nodes1() .peer_no(config.mesh_n_high()) .topics(vec!["test".into()]) .to_subscribe(true) @@ -2689,8 +2804,10 @@ fn test_ihave_msg_from_peer_below_gossip_threshold_gets_ignored() { } //add two additional peers that will not be part of the mesh - let p1 = add_peer(&mut gs, &topics, false, false); - let p2 = add_peer(&mut gs, &topics, false, false); + let (p1, receiver1) = add_peer(&mut gs, &topics, false, false); + receivers.insert(p1, receiver1); + let (p2, receiver2) = add_peer(&mut gs, &topics, false, false); + receivers.insert(p2, receiver2); //reduce score of p1 below peer_score_thresholds.gossip_threshold //note that penalties get squared so two penalties means a score of @@ -2720,19 +2837,18 @@ fn test_ihave_msg_from_peer_below_gossip_threshold_gets_ignored() { gs.handle_ihave(&p2, vec![(topics[0].clone(), vec![msg_id.clone()])]); // check that we sent exactly one IWANT request to p2 - assert_eq!( - count_control_msgs(&gs, |peer, c| match c { - ControlAction::IWant { message_ids } => - if message_ids.iter().any(|m| m == &msg_id) { - assert_eq!(peer, &p2); - true - } else { - false - }, - _ => false, - }), - 1 - ); + let (control_msgs, _) = count_control_msgs(receivers, |peer, c| match c { + RpcOut::IWant(IWant { message_ids }) => { + if message_ids.iter().any(|m| m == &msg_id) { + assert_eq!(peer, &p2); + true + } else { + false + } + } + _ => false, + }); + assert_eq!(control_msgs, 1); } #[test] @@ -2749,7 +2865,7 @@ fn test_do_not_publish_to_peer_below_publish_threshold() { }; //build mesh with no peers and no subscribed topics - let (mut gs, _, _) = inject_nodes1() + let (mut gs, _, mut receivers, _) = inject_nodes1() .gs_config(config) .scoring(Some((peer_score_params, peer_score_thresholds))) .create_network(); @@ -2759,8 +2875,10 @@ fn test_do_not_publish_to_peer_below_publish_threshold() { let topics = vec![topic.hash()]; //add two additional peers that will be added to the mesh - let p1 = add_peer(&mut gs, &topics, false, false); - let p2 = add_peer(&mut gs, &topics, false, false); + let (p1, receiver1) = add_peer(&mut gs, &topics, false, false); + receivers.insert(p1, receiver1); + let (p2, receiver2) = add_peer(&mut gs, &topics, false, false); + receivers.insert(p2, receiver2); //reduce score of p1 below peer_score_thresholds.publish_threshold //note that penalties get squared so two penalties means a score of @@ -2778,17 +2896,16 @@ fn test_do_not_publish_to_peer_below_publish_threshold() { gs.publish(topic, publish_data).unwrap(); // Collect all publish messages - let publishes = gs - .events + let publishes = receivers .into_iter() - .fold(vec![], |mut collected_publish, e| match e { - ToSwarm::NotifyHandler { event, peer_id, .. } => { - if let HandlerIn::Message(RpcOut::Publish(message)) = event { + .fold(vec![], |mut collected_publish, (peer_id, c)| { + let priority = c.priority.get_ref(); + while !priority.is_empty() { + if let Ok(RpcOut::Publish { message, .. }) = priority.try_recv() { collected_publish.push((peer_id, message)); } - collected_publish } - _ => collected_publish, + collected_publish }); //assert only published to p2 @@ -2806,15 +2923,17 @@ fn test_do_not_flood_publish_to_peer_below_publish_threshold() { ..PeerScoreThresholds::default() }; //build mesh with no peers - let (mut gs, _, topics) = inject_nodes1() + let (mut gs, _, mut receivers, topics) = inject_nodes1() .topics(vec!["test".into()]) .gs_config(config) .scoring(Some((peer_score_params, peer_score_thresholds))) .create_network(); //add two additional peers that will be added to the mesh - let p1 = add_peer(&mut gs, &topics, false, false); - let p2 = add_peer(&mut gs, &topics, false, false); + let (p1, receiver1) = add_peer(&mut gs, &topics, false, false); + receivers.insert(p1, receiver1); + let (p2, receiver2) = add_peer(&mut gs, &topics, false, false); + receivers.insert(p2, receiver2); //reduce score of p1 below peer_score_thresholds.publish_threshold //note that penalties get squared so two penalties means a score of @@ -2832,17 +2951,16 @@ fn test_do_not_flood_publish_to_peer_below_publish_threshold() { gs.publish(Topic::new("test"), publish_data).unwrap(); // Collect all publish messages - let publishes = gs - .events + let publishes = receivers .into_iter() - .fold(vec![], |mut collected_publish, e| match e { - ToSwarm::NotifyHandler { event, peer_id, .. } => { - if let HandlerIn::Message(RpcOut::Publish(message)) = event { - collected_publish.push((peer_id, message)); + .fold(vec![], |mut collected_publish, (peer_id, c)| { + let priority = c.priority.get_ref(); + while !priority.is_empty() { + if let Ok(RpcOut::Publish { message, .. }) = priority.try_recv() { + collected_publish.push((peer_id, message)) } - collected_publish } - _ => collected_publish, + collected_publish }); //assert only published to p2 @@ -2862,15 +2980,15 @@ fn test_ignore_rpc_from_peers_below_graylist_threshold() { }; //build mesh with no peers - let (mut gs, _, topics) = inject_nodes1() + let (mut gs, _, _, topics) = inject_nodes1() .topics(vec!["test".into()]) .gs_config(config.clone()) .scoring(Some((peer_score_params, peer_score_thresholds))) .create_network(); //add two additional peers that will be added to the mesh - let p1 = add_peer(&mut gs, &topics, false, false); - let p2 = add_peer(&mut gs, &topics, false, false); + let (p1, _receiver1) = add_peer(&mut gs, &topics, false, false); + let (p2, _receiver2) = add_peer(&mut gs, &topics, false, false); //reduce score of p1 below peer_score_thresholds.graylist_threshold //note that penalties get squared so two penalties means a score of @@ -2931,10 +3049,10 @@ fn test_ignore_rpc_from_peers_below_graylist_threshold() { topic_hash: topics[0].clone(), }; - let control_action = ControlAction::IHave { + let control_action = ControlAction::IHave(IHave { topic_hash: topics[0].clone(), message_ids: vec![config.message_id(message2)], - }; + }); //clear events gs.events.clear(); @@ -2960,10 +3078,10 @@ fn test_ignore_rpc_from_peers_below_graylist_threshold() { ToSwarm::GenerateEvent(Event::Subscribed { .. }) )); - let control_action = ControlAction::IHave { + let control_action = ControlAction::IHave(IHave { topic_hash: topics[0].clone(), message_ids: vec![config.message_id(message4)], - }; + }); //receive from p2 gs.on_connection_handler_event( @@ -2992,7 +3110,7 @@ fn test_ignore_px_from_peers_below_accept_px_threshold() { ..PeerScoreThresholds::default() }; // Build mesh with two peers - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _, topics) = inject_nodes1() .peer_no(2) .topics(vec!["test".into()]) .to_subscribe(true) @@ -3063,7 +3181,7 @@ fn test_keep_best_scoring_peers_on_oversubscription() { //build mesh with more peers than mesh can hold let n = config.mesh_n_high() + 1; - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _receivers, topics) = inject_nodes1() .peer_no(n) .topics(vec!["test".into()]) .to_subscribe(true) @@ -3123,7 +3241,7 @@ fn test_scoring_p1() { let peer_score_thresholds = PeerScoreThresholds::default(); //build mesh with one peer - let (mut gs, peers, _) = inject_nodes1() + let (mut gs, peers, _, _) = inject_nodes1() .peer_no(1) .topics(vec!["test".into()]) .to_subscribe(true) @@ -3205,7 +3323,7 @@ fn test_scoring_p2() { let peer_score_thresholds = PeerScoreThresholds::default(); //build mesh with one peer - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _, topics) = inject_nodes1() .peer_no(2) .topics(vec!["test".into()]) .to_subscribe(true) @@ -3305,7 +3423,7 @@ fn test_scoring_p3() { let peer_score_thresholds = PeerScoreThresholds::default(); //build mesh with two peers - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _, topics) = inject_nodes1() .peer_no(2) .topics(vec!["test".into()]) .to_subscribe(true) @@ -3406,7 +3524,7 @@ fn test_scoring_p3b() { let peer_score_thresholds = PeerScoreThresholds::default(); //build mesh with one peer - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _, topics) = inject_nodes1() .peer_no(1) .topics(vec!["test".into()]) .to_subscribe(true) @@ -3498,7 +3616,7 @@ fn test_scoring_p4_valid_message() { let peer_score_thresholds = PeerScoreThresholds::default(); //build mesh with two peers - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _, topics) = inject_nodes1() .peer_no(1) .topics(vec!["test".into()]) .to_subscribe(true) @@ -3557,7 +3675,7 @@ fn test_scoring_p4_invalid_signature() { let peer_score_thresholds = PeerScoreThresholds::default(); //build mesh with one peer - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _, topics) = inject_nodes1() .peer_no(1) .topics(vec!["test".into()]) .to_subscribe(true) @@ -3615,7 +3733,7 @@ fn test_scoring_p4_message_from_self() { let peer_score_thresholds = PeerScoreThresholds::default(); //build mesh with two peers - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _, topics) = inject_nodes1() .peer_no(1) .topics(vec!["test".into()]) .to_subscribe(true) @@ -3665,7 +3783,7 @@ fn test_scoring_p4_ignored_message() { let peer_score_thresholds = PeerScoreThresholds::default(); //build mesh with two peers - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _, topics) = inject_nodes1() .peer_no(1) .topics(vec!["test".into()]) .to_subscribe(true) @@ -3724,7 +3842,7 @@ fn test_scoring_p4_application_invalidated_message() { let peer_score_thresholds = PeerScoreThresholds::default(); //build mesh with two peers - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _, topics) = inject_nodes1() .peer_no(1) .topics(vec!["test".into()]) .to_subscribe(true) @@ -3786,7 +3904,7 @@ fn test_scoring_p4_application_invalid_message_from_two_peers() { let peer_score_thresholds = PeerScoreThresholds::default(); //build mesh with two peers - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _, topics) = inject_nodes1() .peer_no(2) .topics(vec!["test".into()]) .to_subscribe(true) @@ -3856,7 +3974,7 @@ fn test_scoring_p4_three_application_invalid_messages() { let peer_score_thresholds = PeerScoreThresholds::default(); //build mesh with one peer - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _, topics) = inject_nodes1() .peer_no(1) .topics(vec!["test".into()]) .to_subscribe(true) @@ -3940,7 +4058,7 @@ fn test_scoring_p4_decay() { let peer_score_thresholds = PeerScoreThresholds::default(); //build mesh with one peer - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _, topics) = inject_nodes1() .peer_no(1) .topics(vec!["test".into()]) .to_subscribe(true) @@ -3994,7 +4112,7 @@ fn test_scoring_p5() { }; //build mesh with one peer - let (mut gs, peers, _) = inject_nodes1() + let (mut gs, peers, _, _) = inject_nodes1() .peer_no(1) .topics(vec!["test".into()]) .to_subscribe(true) @@ -4020,7 +4138,7 @@ fn test_scoring_p6() { ..Default::default() }; - let (mut gs, _, _) = inject_nodes1() + let (mut gs, _, _, _) = inject_nodes1() .peer_no(0) .topics(vec![]) .to_subscribe(false) @@ -4033,20 +4151,20 @@ fn test_scoring_p6() { //create 5 peers with the same ip let addr = Multiaddr::from(Ipv4Addr::new(10, 1, 2, 3)); let peers = vec![ - add_peer_with_addr(&mut gs, &[], false, false, addr.clone()), - add_peer_with_addr(&mut gs, &[], false, false, addr.clone()), - add_peer_with_addr(&mut gs, &[], true, false, addr.clone()), - add_peer_with_addr(&mut gs, &[], true, false, addr.clone()), - add_peer_with_addr(&mut gs, &[], true, true, addr.clone()), + add_peer_with_addr(&mut gs, &[], false, false, addr.clone()).0, + add_peer_with_addr(&mut gs, &[], false, false, addr.clone()).0, + add_peer_with_addr(&mut gs, &[], true, false, addr.clone()).0, + add_peer_with_addr(&mut gs, &[], true, false, addr.clone()).0, + add_peer_with_addr(&mut gs, &[], true, true, addr.clone()).0, ]; //create 4 other peers with other ip let addr2 = Multiaddr::from(Ipv4Addr::new(10, 1, 2, 4)); let others = vec![ - add_peer_with_addr(&mut gs, &[], false, false, addr2.clone()), - add_peer_with_addr(&mut gs, &[], false, false, addr2.clone()), - add_peer_with_addr(&mut gs, &[], true, false, addr2.clone()), - add_peer_with_addr(&mut gs, &[], true, false, addr2.clone()), + add_peer_with_addr(&mut gs, &[], false, false, addr2.clone()).0, + add_peer_with_addr(&mut gs, &[], false, false, addr2.clone()).0, + add_peer_with_addr(&mut gs, &[], true, false, addr2.clone()).0, + add_peer_with_addr(&mut gs, &[], true, false, addr2.clone()).0, ]; //no penalties yet @@ -4153,7 +4271,7 @@ fn test_scoring_p7_grafts_before_backoff() { ..Default::default() }; - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _receivers, topics) = inject_nodes1() .peer_no(2) .topics(vec!["test".into()]) .to_subscribe(false) @@ -4230,7 +4348,7 @@ fn test_opportunistic_grafting() { ..Default::default() }; - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _receivers, topics) = inject_nodes1() .peer_no(5) .topics(vec!["test".into()]) .to_subscribe(false) @@ -4259,7 +4377,7 @@ fn test_opportunistic_grafting() { } //set scores for peers in the mesh - for (i, peer) in others.iter().enumerate().take(5) { + for (i, (peer, _receiver)) in others.iter().enumerate().take(5) { gs.set_application_score(peer, 0.0 + i as f64); } @@ -4299,7 +4417,7 @@ fn test_opportunistic_grafting() { ); assert!( - gs.mesh[&topics[0]].is_disjoint(&others.iter().cloned().take(2).collect()), + gs.mesh[&topics[0]].is_disjoint(&others.iter().map(|(p, _)| p).cloned().take(2).collect()), "peers below or equal to median should not be added in opportunistic grafting" ); } @@ -4307,19 +4425,19 @@ fn test_opportunistic_grafting() { #[test] fn test_ignore_graft_from_unknown_topic() { //build gossipsub without subscribing to any topics - let (mut gs, _, _) = inject_nodes1() - .peer_no(0) + let (mut gs, peers, receivers, _) = inject_nodes1() + .peer_no(1) .topics(vec![]) .to_subscribe(false) .create_network(); //handle an incoming graft for some topic - gs.handle_graft(&PeerId::random(), vec![Topic::new("test").hash()]); + gs.handle_graft(&peers[0], vec![Topic::new("test").hash()]); //assert that no prune got created + let (control_msgs, _) = count_control_msgs(receivers, |_, a| matches!(a, RpcOut::Prune { .. })); assert_eq!( - count_control_msgs(&gs, |_, a| matches!(a, ControlAction::Prune { .. })), - 0, + control_msgs, 0, "we should not prune after graft in unknown topic" ); } @@ -4328,14 +4446,15 @@ fn test_ignore_graft_from_unknown_topic() { fn test_ignore_too_many_iwants_from_same_peer_for_same_message() { let config = Config::default(); //build gossipsub with full mesh - let (mut gs, _, topics) = inject_nodes1() + let (mut gs, _, mut receivers, topics) = inject_nodes1() .peer_no(config.mesh_n_high()) .topics(vec!["test".into()]) .to_subscribe(false) .create_network(); //add another peer not in the mesh - let peer = add_peer(&mut gs, &topics, false, false); + let (peer, receiver) = add_peer(&mut gs, &topics, false, false); + receivers.insert(peer, receiver); //receive a message let mut seq = 0; @@ -4349,7 +4468,7 @@ fn test_ignore_too_many_iwants_from_same_peer_for_same_message() { gs.handle_received_message(m1, &PeerId::random()); //clear events - gs.events.clear(); + let receivers = flush_events(&mut gs, receivers); //the first gossip_retransimission many iwants return the valid message, all others are // ignored. @@ -4358,16 +4477,15 @@ fn test_ignore_too_many_iwants_from_same_peer_for_same_message() { } assert_eq!( - gs.events - .iter() - .filter(|e| matches!( - e, - ToSwarm::NotifyHandler { - event: HandlerIn::Message(RpcOut::Forward(_)), - .. + receivers.into_values().fold(0, |mut fwds, c| { + let non_priority = c.non_priority.get_ref(); + while !non_priority.is_empty() { + if let Ok(RpcOut::Forward { .. }) = non_priority.try_recv() { + fwds += 1; } - )) - .count(), + } + fwds + }), config.gossip_retransimission() as usize, "not more then gossip_retransmission many messages get sent back" ); @@ -4380,7 +4498,7 @@ fn test_ignore_too_many_ihaves() { .build() .unwrap(); //build gossipsub with full mesh - let (mut gs, _, topics) = inject_nodes1() + let (mut gs, _, mut receivers, topics) = inject_nodes1() .peer_no(config.mesh_n_high()) .topics(vec!["test".into()]) .to_subscribe(false) @@ -4388,7 +4506,8 @@ fn test_ignore_too_many_ihaves() { .create_network(); //add another peer not in the mesh - let peer = add_peer(&mut gs, &topics, false, false); + let (peer, receiver) = add_peer(&mut gs, &topics, false, false); + receivers.insert(peer, receiver); //peer has 20 messages let mut seq = 0; @@ -4416,15 +4535,18 @@ fn test_ignore_too_many_ihaves() { .collect(); //we send iwant only for the first 10 messages + let (control_msgs, receivers) = count_control_msgs(receivers, |p, action| { + p == &peer + && matches!(action, RpcOut::IWant(IWant { message_ids }) if message_ids.len() == 1 && first_ten.contains(&message_ids[0])) + }); assert_eq!( - count_control_msgs(&gs, |p, action| p == &peer - && matches!(action, ControlAction::IWant { message_ids } if message_ids.len() == 1 && first_ten.contains(&message_ids[0]))), - 10, + control_msgs, 10, "exactly the first ten ihaves should be processed and one iwant for each created" ); //after a heartbeat everything is forgotten gs.heartbeat(); + for raw_message in messages[10..].iter() { // Transform the inbound message let message = &gs @@ -4438,13 +4560,12 @@ fn test_ignore_too_many_ihaves() { ); } - //we sent iwant for all 20 messages - assert_eq!( - count_control_msgs(&gs, |p, action| p == &peer - && matches!(action, ControlAction::IWant { message_ids } if message_ids.len() == 1)), - 20, - "all 20 should get sent" - ); + //we sent iwant for all 10 messages + let (control_msgs, _) = count_control_msgs(receivers, |p, action| { + p == &peer + && matches!(action, RpcOut::IWant(IWant { message_ids }) if message_ids.len() == 1) + }); + assert_eq!(control_msgs, 10, "all 20 should get sent"); } #[test] @@ -4455,7 +4576,7 @@ fn test_ignore_too_many_messages_in_ihave() { .build() .unwrap(); //build gossipsub with full mesh - let (mut gs, _, topics) = inject_nodes1() + let (mut gs, _, mut receivers, topics) = inject_nodes1() .peer_no(config.mesh_n_high()) .topics(vec!["test".into()]) .to_subscribe(false) @@ -4463,7 +4584,8 @@ fn test_ignore_too_many_messages_in_ihave() { .create_network(); //add another peer not in the mesh - let peer = add_peer(&mut gs, &topics, false, false); + let (peer, receiver) = add_peer(&mut gs, &topics, false, false); + receivers.insert(peer, receiver); //peer has 20 messages let mut seq = 0; @@ -4488,17 +4610,18 @@ fn test_ignore_too_many_messages_in_ihave() { //we send iwant only for the first 10 messages let mut sum = 0; + let (control_msgs, receivers) = count_control_msgs(receivers, |p, rpc| match rpc { + RpcOut::IWant(IWant { message_ids }) => { + p == &peer && { + assert!(first_twelve.is_superset(&message_ids.iter().collect())); + sum += message_ids.len(); + true + } + } + _ => false, + }); assert_eq!( - count_control_msgs(&gs, |p, action| match action { - ControlAction::IWant { message_ids } => - p == &peer && { - assert!(first_twelve.is_superset(&message_ids.iter().collect())); - sum += message_ids.len(); - true - }, - _ => false, - }), - 2, + control_msgs, 2, "the third ihave should get ignored and no iwant sent" ); @@ -4511,20 +4634,19 @@ fn test_ignore_too_many_messages_in_ihave() { vec![(topics[0].clone(), message_ids[10..20].to_vec())], ); - //we sent 20 iwant messages + //we sent 10 iwant messages ids via a IWANT rpc. let mut sum = 0; - assert_eq!( - count_control_msgs(&gs, |p, action| match action { - ControlAction::IWant { message_ids } => - p == &peer && { - sum += message_ids.len(); - true - }, - _ => false, - }), - 3 - ); - assert_eq!(sum, 20, "exactly 20 iwants should get sent"); + let (control_msgs, _) = count_control_msgs(receivers, |p, rpc| match rpc { + RpcOut::IWant(IWant { message_ids }) => { + p == &peer && { + sum += message_ids.len(); + true + } + } + _ => false, + }); + assert_eq!(control_msgs, 1); + assert_eq!(sum, 10, "exactly 20 iwants should get sent"); } #[test] @@ -4535,7 +4657,7 @@ fn test_limit_number_of_message_ids_inside_ihave() { .build() .unwrap(); //build gossipsub with full mesh - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, mut receivers, topics) = inject_nodes1() .peer_no(config.mesh_n_high()) .topics(vec!["test".into()]) .to_subscribe(false) @@ -4548,8 +4670,10 @@ fn test_limit_number_of_message_ids_inside_ihave() { } //add two other peers not in the mesh - let p1 = add_peer(&mut gs, &topics, false, false); - let p2 = add_peer(&mut gs, &topics, false, false); + let (p1, receiver1) = add_peer(&mut gs, &topics, false, false); + receivers.insert(p1, receiver1); + let (p2, receiver2) = add_peer(&mut gs, &topics, false, false); + receivers.insert(p2, receiver2); //receive 200 messages from another peer let mut seq = 0; @@ -4567,22 +4691,22 @@ fn test_limit_number_of_message_ids_inside_ihave() { let mut ihaves1 = HashSet::new(); let mut ihaves2 = HashSet::new(); - assert_eq!( - count_control_msgs(&gs, |p, action| match action { - ControlAction::IHave { message_ids, .. } => { - if p == &p1 { - ihaves1 = message_ids.iter().cloned().collect(); - true - } else if p == &p2 { - ihaves2 = message_ids.iter().cloned().collect(); - true - } else { - false - } + let (control_msgs, _) = count_control_msgs(receivers, |p, action| match action { + RpcOut::IHave(IHave { message_ids, .. }) => { + if p == &p1 { + ihaves1 = message_ids.iter().cloned().collect(); + true + } else if p == &p2 { + ihaves2 = message_ids.iter().cloned().collect(); + true + } else { + false } - _ => false, - }), - 2, + } + _ => false, + }); + assert_eq!( + control_msgs, 2, "should have emitted one ihave to p1 and one to p2" ); @@ -4610,11 +4734,12 @@ fn test_limit_number_of_message_ids_inside_ihave() { #[test] fn test_iwant_penalties() { + /* use tracing_subscriber::EnvFilter; let _ = tracing_subscriber::fmt() .with_env_filter(EnvFilter::from_default_env()) .try_init(); - + */ let config = ConfigBuilder::default() .iwant_followup_time(Duration::from_secs(4)) .build() @@ -4625,7 +4750,7 @@ fn test_iwant_penalties() { }; // fill the mesh - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, _, topics) = inject_nodes1() .peer_no(2) .topics(vec!["test".into()]) .to_subscribe(false) @@ -4649,7 +4774,7 @@ fn test_iwant_penalties() { let mut first_messages = Vec::new(); let mut second_messages = Vec::new(); let mut seq = 0; - for peer in &other_peers { + for (peer, _receiver) in &other_peers { let msg1 = random_message(&mut seq, &topics); let msg2 = random_message(&mut seq, &topics); @@ -4672,19 +4797,19 @@ fn test_iwant_penalties() { } // the peers send us all the first message ids in time - for (index, peer) in other_peers.iter().enumerate() { + for (index, (peer, _receiver)) in other_peers.iter().enumerate() { gs.handle_received_message(first_messages[index].clone(), peer); } // now we do a heartbeat no penalization should have been applied yet gs.heartbeat(); - for peer in &other_peers { + for (peer, _receiver) in &other_peers { assert_eq!(gs.peer_score.as_ref().unwrap().0.score(peer), 0.0); } // receive the first twenty of the other peers then send their response - for (index, peer) in other_peers.iter().enumerate().take(20) { + for (index, (peer, _receiver)) in other_peers.iter().enumerate().take(20) { gs.handle_received_message(second_messages[index].clone(), peer); } @@ -4695,7 +4820,7 @@ fn test_iwant_penalties() { gs.heartbeat(); // now we get the second messages from the last 80 peers. - for (index, peer) in other_peers.iter().enumerate() { + for (index, (peer, _receiver)) in other_peers.iter().enumerate() { if index > 19 { gs.handle_received_message(second_messages[index].clone(), peer); } @@ -4709,7 +4834,7 @@ fn test_iwant_penalties() { let mut single_penalized = 0; let mut double_penalized = 0; - for (i, peer) in other_peers.iter().enumerate() { + for (i, (peer, _receiver)) in other_peers.iter().enumerate() { let score = gs.peer_score.as_ref().unwrap().0.score(peer); if score == 0.0 { not_penalized += 1; @@ -4737,7 +4862,7 @@ fn test_publish_to_floodsub_peers_without_flood_publish() { .flood_publish(false) .build() .unwrap(); - let (mut gs, _, topics) = inject_nodes1() + let (mut gs, _, mut receivers, topics) = inject_nodes1() .peer_no(config.mesh_n_low() - 1) .topics(vec!["test".into()]) .to_subscribe(false) @@ -4745,7 +4870,7 @@ fn test_publish_to_floodsub_peers_without_flood_publish() { .create_network(); //add two floodsub peer, one explicit, one implicit - let p1 = add_peer_with_addr_and_kind( + let (p1, receiver1) = add_peer_with_addr_and_kind( &mut gs, &topics, false, @@ -4753,7 +4878,11 @@ fn test_publish_to_floodsub_peers_without_flood_publish() { Multiaddr::empty(), Some(PeerKind::Floodsub), ); - let p2 = add_peer_with_addr_and_kind(&mut gs, &topics, false, false, Multiaddr::empty(), None); + receivers.insert(p1, receiver1); + + let (p2, receiver2) = + add_peer_with_addr_and_kind(&mut gs, &topics, false, false, Multiaddr::empty(), None); + receivers.insert(p2, receiver2); //p1 and p2 are not in the mesh assert!(!gs.mesh[&topics[0]].contains(&p1) && !gs.mesh[&topics[0]].contains(&p2)); @@ -4763,24 +4892,22 @@ fn test_publish_to_floodsub_peers_without_flood_publish() { gs.publish(Topic::new("test"), publish_data).unwrap(); // Collect publish messages to floodsub peers - let publishes = gs - .events - .iter() - .fold(vec![], |mut collected_publish, e| match e { - ToSwarm::NotifyHandler { peer_id, event, .. } => { - if peer_id == &p1 || peer_id == &p2 { - if let HandlerIn::Message(RpcOut::Publish(message)) = event { - collected_publish.push(message); - } + let publishes = receivers + .into_iter() + .fold(0, |mut collected_publish, (peer_id, c)| { + let priority = c.priority.get_ref(); + while !priority.is_empty() { + if matches!(priority.try_recv(), + Ok(RpcOut::Publish{..}) if peer_id == p1 || peer_id == p2) + { + collected_publish += 1; } - collected_publish } - _ => collected_publish, + collected_publish }); assert_eq!( - publishes.len(), - 2, + publishes, 2, "Should send a publish message to all floodsub peers" ); } @@ -4791,7 +4918,7 @@ fn test_do_not_use_floodsub_in_fanout() { .flood_publish(false) .build() .unwrap(); - let (mut gs, _, _) = inject_nodes1() + let (mut gs, _, mut receivers, _) = inject_nodes1() .peer_no(config.mesh_n_low() - 1) .topics(Vec::new()) .to_subscribe(false) @@ -4802,7 +4929,7 @@ fn test_do_not_use_floodsub_in_fanout() { let topics = vec![topic.hash()]; //add two floodsub peer, one explicit, one implicit - let p1 = add_peer_with_addr_and_kind( + let (p1, receiver1) = add_peer_with_addr_and_kind( &mut gs, &topics, false, @@ -4810,31 +4937,33 @@ fn test_do_not_use_floodsub_in_fanout() { Multiaddr::empty(), Some(PeerKind::Floodsub), ); - let p2 = add_peer_with_addr_and_kind(&mut gs, &topics, false, false, Multiaddr::empty(), None); + receivers.insert(p1, receiver1); + let (p2, receiver2) = + add_peer_with_addr_and_kind(&mut gs, &topics, false, false, Multiaddr::empty(), None); + + receivers.insert(p2, receiver2); //publish a message let publish_data = vec![0; 42]; gs.publish(Topic::new("test"), publish_data).unwrap(); // Collect publish messages to floodsub peers - let publishes = gs - .events - .iter() - .fold(vec![], |mut collected_publish, e| match e { - ToSwarm::NotifyHandler { peer_id, event, .. } => { - if peer_id == &p1 || peer_id == &p2 { - if let HandlerIn::Message(RpcOut::Publish(message)) = event { - collected_publish.push(message); - } + let publishes = receivers + .into_iter() + .fold(0, |mut collected_publish, (peer_id, c)| { + let priority = c.priority.get_ref(); + while !priority.is_empty() { + if matches!(priority.try_recv(), + Ok(RpcOut::Publish{..}) if peer_id == p1 || peer_id == p2) + { + collected_publish += 1; } - collected_publish } - _ => collected_publish, + collected_publish }); assert_eq!( - publishes.len(), - 2, + publishes, 2, "Should send a publish message to all floodsub peers" ); @@ -4846,7 +4975,7 @@ fn test_do_not_use_floodsub_in_fanout() { #[test] fn test_dont_add_floodsub_peers_to_mesh_on_join() { - let (mut gs, _, _) = inject_nodes1() + let (mut gs, _, _, _) = inject_nodes1() .peer_no(0) .topics(Vec::new()) .to_subscribe(false) @@ -4876,14 +5005,14 @@ fn test_dont_add_floodsub_peers_to_mesh_on_join() { #[test] fn test_dont_send_px_to_old_gossipsub_peers() { - let (mut gs, _, topics) = inject_nodes1() + let (mut gs, _, receivers, topics) = inject_nodes1() .peer_no(0) .topics(vec!["test".into()]) .to_subscribe(false) .create_network(); //add an old gossipsub peer - let p1 = add_peer_with_addr_and_kind( + let (p1, _receiver1) = add_peer_with_addr_and_kind( &mut gs, &topics, false, @@ -4900,20 +5029,17 @@ fn test_dont_send_px_to_old_gossipsub_peers() { ); //check that prune does not contain px - assert_eq!( - count_control_msgs(&gs, |_, m| match m { - ControlAction::Prune { peers: px, .. } => !px.is_empty(), - _ => false, - }), - 0, - "Should not send px to floodsub peers" - ); + let (control_msgs, _) = count_control_msgs(receivers, |_, m| match m { + RpcOut::Prune(Prune { peers: px, .. }) => !px.is_empty(), + _ => false, + }); + assert_eq!(control_msgs, 0, "Should not send px to floodsub peers"); } #[test] fn test_dont_send_floodsub_peers_in_px() { //build mesh with one peer - let (mut gs, peers, topics) = inject_nodes1() + let (mut gs, peers, receivers, topics) = inject_nodes1() .peer_no(1) .topics(vec!["test".into()]) .to_subscribe(true) @@ -4938,19 +5064,16 @@ fn test_dont_send_floodsub_peers_in_px() { ); //check that px in prune message is empty - assert_eq!( - count_control_msgs(&gs, |_, m| match m { - ControlAction::Prune { peers: px, .. } => !px.is_empty(), - _ => false, - }), - 0, - "Should not include floodsub peers in px" - ); + let (control_msgs, _) = count_control_msgs(receivers, |_, m| match m { + RpcOut::Prune(Prune { peers: px, .. }) => !px.is_empty(), + _ => false, + }); + assert_eq!(control_msgs, 0, "Should not include floodsub peers in px"); } #[test] fn test_dont_add_floodsub_peers_to_mesh_in_heartbeat() { - let (mut gs, _, topics) = inject_nodes1() + let (mut gs, _, _, topics) = inject_nodes1() .peer_no(0) .topics(vec!["test".into()]) .to_subscribe(false) @@ -4978,7 +5101,7 @@ fn test_dont_add_floodsub_peers_to_mesh_in_heartbeat() { // Some very basic test of public api methods. #[test] fn test_public_api() { - let (gs, peers, topic_hashes) = inject_nodes1() + let (gs, peers, _, topic_hashes) = inject_nodes1() .peer_no(4) .topics(vec![String::from("topic1")]) .to_subscribe(true) @@ -5010,7 +5133,7 @@ fn test_public_api() { fn test_subscribe_to_invalid_topic() { let t1 = Topic::new("t1"); let t2 = Topic::new("t2"); - let (mut gs, _, _) = inject_nodes::() + let (mut gs, _, _, _) = inject_nodes::() .subscription_filter(WhitelistSubscriptionFilter( vec![t1.hash()].into_iter().collect(), )) @@ -5024,7 +5147,7 @@ fn test_subscribe_to_invalid_topic() { #[test] fn test_subscribe_and_graft_with_negative_score() { //simulate a communication between two gossipsub instances - let (mut gs1, _, topic_hashes) = inject_nodes1() + let (mut gs1, _, _, topic_hashes) = inject_nodes1() .topics(vec!["test".into()]) .scoring(Some(( PeerScoreParams::default(), @@ -5032,14 +5155,14 @@ fn test_subscribe_and_graft_with_negative_score() { ))) .create_network(); - let (mut gs2, _, _) = inject_nodes1().create_network(); + let (mut gs2, _, receivers, _) = inject_nodes1().create_network(); let connection_id = ConnectionId::new_unchecked(0); let topic = Topic::new("test"); - let p2 = add_peer(&mut gs1, &Vec::new(), true, false); - let p1 = add_peer(&mut gs2, &topic_hashes, false, false); + let (p2, _receiver1) = add_peer(&mut gs1, &Vec::new(), true, false); + let (p1, _receiver2) = add_peer(&mut gs2, &topic_hashes, false, false); //add penalty to peer p2 gs1.peer_score.as_mut().unwrap().0.add_penalty(&p2, 1); @@ -5049,43 +5172,41 @@ fn test_subscribe_and_graft_with_negative_score() { //subscribe to topic in gs2 gs2.subscribe(&topic).unwrap(); - let forward_messages_to_p1 = |gs1: &mut Behaviour<_, _>, gs2: &mut Behaviour<_, _>| { - //collect messages to p1 - let messages_to_p1 = gs2.events.drain(..).filter_map(|e| match e { - ToSwarm::NotifyHandler { peer_id, event, .. } => { - if peer_id == p1 { - if let HandlerIn::Message(m) = event { - Some(m) - } else { - None - } - } else { - None + let forward_messages_to_p1 = |gs1: &mut Behaviour<_, _>, + p1: PeerId, + p2: PeerId, + connection_id: ConnectionId, + receivers: HashMap| + -> HashMap { + let new_receivers = HashMap::new(); + for (peer_id, receiver) in receivers.into_iter() { + let non_priority = receiver.non_priority.get_ref(); + match non_priority.try_recv() { + Ok(rpc) if peer_id == p1 => { + gs1.on_connection_handler_event( + p2, + connection_id, + HandlerEvent::Message { + rpc: proto_to_message(&rpc.into_protobuf()), + invalid_messages: vec![], + }, + ); } + _ => {} } - _ => None, - }); - for message in messages_to_p1 { - gs1.on_connection_handler_event( - p2, - connection_id, - HandlerEvent::Message { - rpc: proto_to_message(&message.into_protobuf()), - invalid_messages: vec![], - }, - ); } + new_receivers }; //forward the subscribe message - forward_messages_to_p1(&mut gs1, &mut gs2); + let receivers = forward_messages_to_p1(&mut gs1, p1, p2, connection_id, receivers); //heartbeats on both gs1.heartbeat(); gs2.heartbeat(); //forward messages again - forward_messages_to_p1(&mut gs1, &mut gs2); + forward_messages_to_p1(&mut gs1, p1, p2, connection_id, receivers); //nobody got penalized assert!(gs1.peer_score.as_ref().unwrap().0.score(&p2) >= original_score); @@ -5102,7 +5223,7 @@ fn test_graft_without_subscribe() { let topic = String::from("test_subscribe"); let subscribe_topic = vec![topic.clone()]; let subscribe_topic_hash = vec![Topic::new(topic.clone()).hash()]; - let (mut gs, peers, topic_hashes) = inject_nodes1() + let (mut gs, peers, _, topic_hashes) = inject_nodes1() .peer_no(1) .topics(subscribe_topic) .to_subscribe(false) diff --git a/protocols/gossipsub/src/config.rs b/protocols/gossipsub/src/config.rs index febe2514a30..37448a587c0 100644 --- a/protocols/gossipsub/src/config.rs +++ b/protocols/gossipsub/src/config.rs @@ -95,6 +95,9 @@ pub struct Config { max_ihave_messages: usize, iwant_followup_time: Duration, published_message_ids_cache_time: Duration, + connection_handler_queue_len: usize, + connection_handler_publish_duration: Duration, + connection_handler_forward_duration: Duration, } impl Config { @@ -350,6 +353,23 @@ impl Config { pub fn published_message_ids_cache_time(&self) -> Duration { self.published_message_ids_cache_time } + + /// The max number of messages a `ConnectionHandler` can buffer. The default is 5000. + pub fn connection_handler_queue_len(&self) -> usize { + self.connection_handler_queue_len + } + + /// The duration a message to be published can wait to be sent before it is abandoned. The + /// default is 5 seconds. + pub fn publish_queue_duration(&self) -> Duration { + self.connection_handler_publish_duration + } + + /// The duration a message to be forwarded can wait to be sent before it is abandoned. The + /// default is 1s. + pub fn forward_queue_duration(&self) -> Duration { + self.connection_handler_forward_duration + } } impl Default for Config { @@ -417,6 +437,9 @@ impl Default for ConfigBuilder { max_ihave_messages: 10, iwant_followup_time: Duration::from_secs(3), published_message_ids_cache_time: Duration::from_secs(10), + connection_handler_queue_len: 5000, + connection_handler_publish_duration: Duration::from_secs(5), + connection_handler_forward_duration: Duration::from_millis(1000), }, invalid_protocol: false, } @@ -782,6 +805,26 @@ impl ConfigBuilder { self } + /// The max number of messages a `ConnectionHandler` can buffer. The default is 5000. + pub fn connection_handler_queue_len(&mut self, len: usize) -> &mut Self { + self.config.connection_handler_queue_len = len; + self + } + + /// The duration a message to be published can wait to be sent before it is abandoned. The + /// default is 5 seconds. + pub fn publish_queue_duration(&mut self, duration: Duration) -> &mut Self { + self.config.connection_handler_publish_duration = duration; + self + } + + /// The duration a message to be forwarded can wait to be sent before it is abandoned. The + /// default is 1s. + pub fn forward_queue_duration(&mut self, duration: Duration) -> &mut Self { + self.config.connection_handler_forward_duration = duration; + self + } + /// Constructs a [`Config`] from the given configuration and validates the settings. pub fn build(&self) -> Result { // check all constraints on config diff --git a/protocols/gossipsub/src/error.rs b/protocols/gossipsub/src/error.rs index 8761630467b..03df3e90f17 100644 --- a/protocols/gossipsub/src/error.rs +++ b/protocols/gossipsub/src/error.rs @@ -36,6 +36,9 @@ pub enum PublishError { MessageTooLarge, /// The compression algorithm failed. TransformFailed(std::io::Error), + /// Messages could not be sent because all queues for peers were full. The usize represents the + /// number of peers that have full queues. + AllQueuesFull(usize), } impl std::fmt::Display for PublishError { diff --git a/protocols/gossipsub/src/handler.rs b/protocols/gossipsub/src/handler.rs index 88def13a521..c7c3eb2ebe1 100644 --- a/protocols/gossipsub/src/handler.rs +++ b/protocols/gossipsub/src/handler.rs @@ -20,7 +20,7 @@ use crate::protocol::{GossipsubCodec, ProtocolConfig}; use crate::rpc_proto::proto; -use crate::types::{PeerKind, RawMessage, Rpc, RpcOut}; +use crate::types::{PeerKind, RawMessage, Rpc, RpcOut, RpcReceiver}; use crate::ValidationError; use asynchronous_codec::Framed; use futures::future::Either; @@ -32,7 +32,6 @@ use libp2p_swarm::handler::{ FullyNegotiatedInbound, FullyNegotiatedOutbound, StreamUpgradeError, SubstreamProtocol, }; use libp2p_swarm::Stream; -use smallvec::SmallVec; use std::{ pin::Pin, task::{Context, Poll}, @@ -55,14 +54,14 @@ pub enum HandlerEvent { /// An inbound or outbound substream has been established with the peer and this informs over /// which protocol. This message only occurs once per connection. PeerKind(PeerKind), + /// A message to be published was dropped because it could not be sent in time. + MessageDropped(RpcOut), } /// A message sent from the behaviour to the handler. #[allow(clippy::large_enum_variant)] #[derive(Debug)] pub enum HandlerIn { - /// A gossipsub message to send. - Message(RpcOut), /// The peer has joined the mesh. JoinedMesh, /// The peer has left the mesh. @@ -94,8 +93,8 @@ pub struct EnabledHandler { /// The single long-lived inbound substream. inbound_substream: Option, - /// Queue of values that we want to send to the remote. - send_queue: SmallVec<[proto::RPC; 16]>, + /// Queue of values that we want to send to the remote + send_queue: RpcReceiver, /// Flag indicating that an outbound substream is being established to prevent duplicate /// requests. @@ -159,7 +158,7 @@ enum OutboundSubstreamState { impl Handler { /// Builds a new [`Handler`]. - pub fn new(protocol_config: ProtocolConfig) -> Self { + pub fn new(protocol_config: ProtocolConfig, message_queue: RpcReceiver) -> Self { Handler::Enabled(EnabledHandler { listen_protocol: protocol_config, inbound_substream: None, @@ -167,7 +166,7 @@ impl Handler { outbound_substream_establishing: false, outbound_substream_attempts: 0, inbound_substream_attempts: 0, - send_queue: SmallVec::new(), + send_queue: message_queue, peer_kind: None, peer_kind_sent: false, last_io_activity: Instant::now(), @@ -232,7 +231,7 @@ impl EnabledHandler { } // determine if we need to create the outbound stream - if !self.send_queue.is_empty() + if !self.send_queue.poll_is_empty(cx) && self.outbound_substream.is_none() && !self.outbound_substream_establishing { @@ -250,10 +249,31 @@ impl EnabledHandler { ) { // outbound idle state Some(OutboundSubstreamState::WaitingOutput(substream)) => { - if let Some(message) = self.send_queue.pop() { - self.send_queue.shrink_to_fit(); - self.outbound_substream = - Some(OutboundSubstreamState::PendingSend(substream, message)); + if let Poll::Ready(Some(mut message)) = self.send_queue.poll_next_unpin(cx) { + match message { + RpcOut::Publish { + message: _, + ref mut timeout, + } + | RpcOut::Forward { + message: _, + ref mut timeout, + } => { + if Pin::new(timeout).poll(cx).is_ready() { + // Inform the behaviour and end the poll. + self.outbound_substream = + Some(OutboundSubstreamState::WaitingOutput(substream)); + return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( + HandlerEvent::MessageDropped(message), + )); + } + } + _ => {} // All other messages are not time-bound. + } + self.outbound_substream = Some(OutboundSubstreamState::PendingSend( + substream, + message.into_protobuf(), + )); continue; } @@ -319,6 +339,7 @@ impl EnabledHandler { } } + // Handle inbound messages. loop { match std::mem::replace( &mut self.inbound_substream, @@ -383,6 +404,13 @@ impl EnabledHandler { } } + // Drop the next message in queue if it's stale. + if let Poll::Ready(Some(rpc)) = self.send_queue.poll_stale(cx) { + return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( + HandlerEvent::MessageDropped(rpc), + )); + } + Poll::Pending } } @@ -409,7 +437,6 @@ impl ConnectionHandler for Handler { fn on_behaviour_event(&mut self, message: HandlerIn) { match self { Handler::Enabled(handler) => match message { - HandlerIn::Message(m) => handler.send_queue.push(m.into_protobuf()), HandlerIn::JoinedMesh => { handler.in_mesh = true; } diff --git a/protocols/gossipsub/src/lib.rs b/protocols/gossipsub/src/lib.rs index 3db2fa7ce51..1d29aaa7598 100644 --- a/protocols/gossipsub/src/lib.rs +++ b/protocols/gossipsub/src/lib.rs @@ -125,7 +125,7 @@ pub use self::subscription_filter::{ }; pub use self::topic::{Hasher, Topic, TopicHash}; pub use self::transform::{DataTransform, IdentityTransform}; -pub use self::types::{Message, MessageAcceptance, MessageId, RawMessage}; +pub use self::types::{FailedMessages, Message, MessageAcceptance, MessageId, RawMessage}; #[deprecated(note = "Will be removed from the public API.")] pub type Rpc = self::types::Rpc; diff --git a/protocols/gossipsub/src/metrics.rs b/protocols/gossipsub/src/metrics.rs index 7d4acada3c7..0c762fb0e60 100644 --- a/protocols/gossipsub/src/metrics.rs +++ b/protocols/gossipsub/src/metrics.rs @@ -127,6 +127,10 @@ pub(crate) struct Metrics { ignored_messages: Family, /// The number of messages rejected by the application (validation result). rejected_messages: Family, + /// The number of publish messages dropped by the sender. + publish_messages_dropped: Family, + /// The number of forward messages dropped by the sender. + forward_messages_dropped: Family, /* Metrics regarding mesh state */ /// Number of peers in our mesh. This metric should be updated with the count of peers for a @@ -174,6 +178,11 @@ pub(crate) struct Metrics { /// The number of times we have decided that an IWANT control message is required for this /// topic. A very high metric might indicate an underperforming network. topic_iwant_msgs: Family, + + /// The size of the priority queue. + priority_queue_size: Histogram, + /// The size of the non-priority queue. + non_priority_queue_size: Histogram, } impl Metrics { @@ -222,6 +231,16 @@ impl Metrics { "Number of rejected messages received for each topic" ); + let publish_messages_dropped = register_family!( + "publish_messages_dropped_per_topic", + "Number of publish messages dropped per topic" + ); + + let forward_messages_dropped = register_family!( + "forward_messages_dropped_per_topic", + "Number of forward messages dropped per topic" + ); + let mesh_peer_counts = register_family!( "mesh_peer_counts", "Number of peers in each topic in our mesh" @@ -302,6 +321,20 @@ impl Metrics { metric }; + let priority_queue_size = Histogram::new(linear_buckets(0.0, 25.0, 100)); + registry.register( + "priority_queue_size", + "Histogram of observed priority queue sizes", + priority_queue_size.clone(), + ); + + let non_priority_queue_size = Histogram::new(linear_buckets(0.0, 25.0, 100)); + registry.register( + "non_priority_queue_size", + "Histogram of observed non-priority queue sizes", + non_priority_queue_size.clone(), + ); + Self { max_topics, max_never_subscribed_topics, @@ -312,6 +345,8 @@ impl Metrics { accepted_messages, ignored_messages, rejected_messages, + publish_messages_dropped, + forward_messages_dropped, mesh_peer_counts, mesh_peer_inclusion_events, mesh_peer_churn_events, @@ -327,6 +362,8 @@ impl Metrics { heartbeat_duration, memcache_misses, topic_iwant_msgs, + priority_queue_size, + non_priority_queue_size, } } @@ -457,6 +494,20 @@ impl Metrics { } } + /// Register dropping a Publish message over a topic. + pub(crate) fn publish_msg_dropped(&mut self, topic: &TopicHash) { + if self.register_topic(topic).is_ok() { + self.publish_messages_dropped.get_or_create(topic).inc(); + } + } + + /// Register dropping a Forward message over a topic. + pub(crate) fn forward_msg_dropped(&mut self, topic: &TopicHash) { + if self.register_topic(topic).is_ok() { + self.forward_messages_dropped.get_or_create(topic).inc(); + } + } + /// Register that a message was received (and was not a duplicate). pub(crate) fn msg_recvd(&mut self, topic: &TopicHash) { if self.register_topic(topic).is_ok() { @@ -507,6 +558,16 @@ impl Metrics { self.heartbeat_duration.observe(millis as f64); } + /// Observes a priority queue size. + pub(crate) fn observe_priority_queue_size(&mut self, len: usize) { + self.priority_queue_size.observe(len as f64); + } + + /// Observes a non-priority queue size. + pub(crate) fn observe_non_priority_queue_size(&mut self, len: usize) { + self.non_priority_queue_size.observe(len as f64); + } + /// Observe a score of a mesh peer. pub(crate) fn observe_mesh_peers_score(&mut self, topic: &TopicHash, score: f64) { if self.register_topic(topic).is_ok() { diff --git a/protocols/gossipsub/src/peer_score.rs b/protocols/gossipsub/src/peer_score.rs index ac24fc91970..789f2d53447 100644 --- a/protocols/gossipsub/src/peer_score.rs +++ b/protocols/gossipsub/src/peer_score.rs @@ -67,6 +67,8 @@ struct PeerStats { behaviour_penalty: f64, /// Application specific score. Can be manipulated by calling PeerScore::set_application_score application_score: f64, + /// Scoring based on how whether this peer consumes messages fast enough or not. + slow_peer_penalty: f64, } enum ConnectionStatus { @@ -87,6 +89,7 @@ impl Default for PeerStats { known_ips: HashSet::new(), behaviour_penalty: 0f64, application_score: 0f64, + slow_peer_penalty: 0f64, } } } @@ -338,6 +341,12 @@ impl PeerScore { let excess = peer_stats.behaviour_penalty - self.params.behaviour_penalty_threshold; let p7 = excess * excess; score += p7 * self.params.behaviour_penalty_weight; + + // Slow peer weighting + if peer_stats.slow_peer_penalty > self.params.slow_peer_threshold { + let excess = peer_stats.slow_peer_penalty - self.params.slow_peer_threshold; + score += excess * self.params.slow_peer_weight; + } } score } @@ -428,6 +437,13 @@ impl PeerScore { if peer_stats.behaviour_penalty < params_ref.decay_to_zero { peer_stats.behaviour_penalty = 0.0; } + + // decay slow peer score + peer_stats.slow_peer_penalty *= params_ref.slow_peer_decay; + if peer_stats.slow_peer_penalty < params_ref.decay_to_zero { + peer_stats.slow_peer_penalty = 0.0; + } + true }); } @@ -455,6 +471,14 @@ impl PeerScore { self.peer_ips.entry(ip).or_default().insert(*peer_id); } + /// Indicate that a peer has been too slow to consume a message. + pub(crate) fn failed_message_slow_peer(&mut self, peer_id: &PeerId) { + if let Some(peer_stats) = self.peer_stats.get_mut(peer_id) { + peer_stats.slow_peer_penalty += 1.0; + tracing::debug!(peer=%peer_id, %peer_stats.slow_peer_penalty, "[Penalty] Expired message penalty."); + } + } + /// Removes an ip from a peer pub(crate) fn remove_ip(&mut self, peer_id: &PeerId, ip: &IpAddr) { if let Some(peer_stats) = self.peer_stats.get_mut(peer_id) { diff --git a/protocols/gossipsub/src/peer_score/params.rs b/protocols/gossipsub/src/peer_score/params.rs index 35bea0e4353..a5ac1b63b51 100644 --- a/protocols/gossipsub/src/peer_score/params.rs +++ b/protocols/gossipsub/src/peer_score/params.rs @@ -148,6 +148,11 @@ pub struct PeerScoreParams { /// Time to remember counters for a disconnected peer. pub retain_score: Duration, + + /// Slow peer penalty conditions + pub slow_peer_weight: f64, + pub slow_peer_threshold: f64, + pub slow_peer_decay: f64, } impl Default for PeerScoreParams { @@ -165,6 +170,9 @@ impl Default for PeerScoreParams { decay_interval: Duration::from_secs(DEFAULT_DECAY_INTERVAL), decay_to_zero: DEFAULT_DECAY_TO_ZERO, retain_score: Duration::from_secs(3600), + slow_peer_weight: -0.2, + slow_peer_threshold: 0.0, + slow_peer_decay: 0.2, } } } diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index c13caae58b6..0353f23d6d9 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -23,7 +23,8 @@ use crate::handler::HandlerEvent; use crate::rpc_proto::proto; use crate::topic::TopicHash; use crate::types::{ - ControlAction, MessageId, PeerInfo, PeerKind, RawMessage, Rpc, Subscription, SubscriptionAction, + ControlAction, Graft, IHave, IWant, MessageId, PeerInfo, PeerKind, Prune, RawMessage, Rpc, + Subscription, SubscriptionAction, }; use crate::ValidationError; use asynchronous_codec::{Decoder, Encoder, Framed}; @@ -412,33 +413,39 @@ impl Decoder for GossipsubCodec { let ihave_msgs: Vec = rpc_control .ihave .into_iter() - .map(|ihave| ControlAction::IHave { - topic_hash: TopicHash::from_raw(ihave.topic_id.unwrap_or_default()), - message_ids: ihave - .message_ids - .into_iter() - .map(MessageId::from) - .collect::>(), + .map(|ihave| { + ControlAction::IHave(IHave { + topic_hash: TopicHash::from_raw(ihave.topic_id.unwrap_or_default()), + message_ids: ihave + .message_ids + .into_iter() + .map(MessageId::from) + .collect::>(), + }) }) .collect(); let iwant_msgs: Vec = rpc_control .iwant .into_iter() - .map(|iwant| ControlAction::IWant { - message_ids: iwant - .message_ids - .into_iter() - .map(MessageId::from) - .collect::>(), + .map(|iwant| { + ControlAction::IWant(IWant { + message_ids: iwant + .message_ids + .into_iter() + .map(MessageId::from) + .collect::>(), + }) }) .collect(); let graft_msgs: Vec = rpc_control .graft .into_iter() - .map(|graft| ControlAction::Graft { - topic_hash: TopicHash::from_raw(graft.topic_id.unwrap_or_default()), + .map(|graft| { + ControlAction::Graft(Graft { + topic_hash: TopicHash::from_raw(graft.topic_id.unwrap_or_default()), + }) }) .collect(); @@ -462,11 +469,11 @@ impl Decoder for GossipsubCodec { .collect::>(); let topic_hash = TopicHash::from_raw(prune.topic_id.unwrap_or_default()); - prune_msgs.push(ControlAction::Prune { + prune_msgs.push(ControlAction::Prune(Prune { topic_hash, peers, backoff: prune.backoff, - }); + })); } control_msgs.extend(ihave_msgs); @@ -501,7 +508,7 @@ impl Decoder for GossipsubCodec { mod tests { use super::*; use crate::config::Config; - use crate::{Behaviour, ConfigBuilder}; + use crate::{Behaviour, ConfigBuilder, MessageAuthenticity}; use crate::{IdentTopic as Topic, Version}; use libp2p_identity::Keypair; use quickcheck::*; @@ -516,8 +523,9 @@ mod tests { // generate an arbitrary GossipsubMessage using the behaviour signing functionality let config = Config::default(); let mut gs: Behaviour = - Behaviour::new(crate::MessageAuthenticity::Signed(keypair.0), config).unwrap(); - let data = (0..g.gen_range(10..10024u32)) + Behaviour::new(MessageAuthenticity::Signed(keypair.0), config).unwrap(); + let mut data_g = quickcheck::Gen::new(10024); + let data = (0..u8::arbitrary(&mut data_g)) .map(|_| u8::arbitrary(g)) .collect::>(); let topic_id = TopicId::arbitrary(g).0; @@ -530,7 +538,8 @@ mod tests { impl Arbitrary for TopicId { fn arbitrary(g: &mut Gen) -> Self { - let topic_string: String = (0..g.gen_range(20..1024u32)) + let mut data_g = quickcheck::Gen::new(1024); + let topic_string: String = (0..u8::arbitrary(&mut data_g)) .map(|_| char::arbitrary(g)) .collect::(); TopicId(Topic::new(topic_string).into()) diff --git a/protocols/gossipsub/src/types.rs b/protocols/gossipsub/src/types.rs index a88f4822ac2..c4625a2be63 100644 --- a/protocols/gossipsub/src/types.rs +++ b/protocols/gossipsub/src/types.rs @@ -19,18 +19,60 @@ // DEALINGS IN THE SOFTWARE. //! A collection of types using the Gossipsub system. +use crate::metrics::Metrics; use crate::TopicHash; +use async_channel::{Receiver, Sender}; +use futures::{stream::Peekable, Stream, StreamExt}; +use futures_timer::Delay; use libp2p_identity::PeerId; use libp2p_swarm::ConnectionId; use prometheus_client::encoding::EncodeLabelValue; use quick_protobuf::MessageWrite; use std::fmt::Debug; +use std::future::Future; +use std::pin::Pin; +use std::sync::{ + atomic::{AtomicUsize, Ordering}, + Arc, +}; +use std::task::{Context, Poll}; +use std::time::Duration; use std::{collections::BTreeSet, fmt}; use crate::rpc_proto::proto; #[cfg(feature = "serde")] use serde::{Deserialize, Serialize}; +/// The type of messages that have expired while attempting to send to a peer. +#[derive(Clone, Debug, Default)] +pub struct FailedMessages { + /// The number of publish messages that failed to be published in a heartbeat. + pub publish: usize, + /// The number of forward messages that failed to be published in a heartbeat. + pub forward: usize, + /// The number of messages that were failed to be sent to the priority queue as it was full. + pub priority: usize, + /// The number of messages that were failed to be sent to the non-priority queue as it was full. + pub non_priority: usize, +} + +impl FailedMessages { + /// The total number of messages that expired due a timeout. + pub fn total_timeout(&self) -> usize { + self.publish + self.forward + } + + /// The total number of messages that failed due to the queue being full. + pub fn total_queue_full(&self) -> usize { + self.priority + self.non_priority + } + + /// The total failed messages in a heartbeat. + pub fn total(&self) -> usize { + self.total_timeout() + self.total_queue_full() + } +} + #[derive(Debug)] /// Validation kinds from the application for received messages. pub enum MessageAcceptance { @@ -71,7 +113,7 @@ impl std::fmt::Debug for MessageId { } } -#[derive(Debug, Clone, PartialEq, Eq)] +#[derive(Debug, Clone)] pub(crate) struct PeerConnections { /// The kind of protocol the peer supports. pub(crate) kind: PeerKind, @@ -79,6 +121,8 @@ pub(crate) struct PeerConnections { pub(crate) connections: Vec, /// Subscribed topics. pub(crate) topics: BTreeSet, + /// The rpc sender to the peer. + pub(crate) sender: RpcSender, } /// Describes the types of peers that can exist in the gossipsub context. @@ -197,8 +241,8 @@ pub enum SubscriptionAction { } #[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub struct PeerInfo { - pub peer_id: Option, +pub(crate) struct PeerInfo { + pub(crate) peer_id: Option, //TODO add this when RFC: Signed Address Records got added to the spec (see pull request // https://github.com/libp2p/specs/pull/217) //pub signed_peer_record: ?, @@ -208,46 +252,68 @@ pub struct PeerInfo { #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub enum ControlAction { /// Node broadcasts known messages per topic - IHave control message. - IHave { - /// The topic of the messages. - topic_hash: TopicHash, - /// A list of known message ids (peer_id + sequence _number) as a string. - message_ids: Vec, - }, + IHave(IHave), /// The node requests specific message ids (peer_id + sequence _number) - IWant control message. - IWant { - /// A list of known message ids (peer_id + sequence _number) as a string. - message_ids: Vec, - }, + IWant(IWant), /// The node has been added to the mesh - Graft control message. - Graft { - /// The mesh topic the peer should be added to. - topic_hash: TopicHash, - }, + Graft(Graft), /// The node has been removed from the mesh - Prune control message. - Prune { - /// The mesh topic the peer should be removed from. - topic_hash: TopicHash, - /// A list of peers to be proposed to the removed peer as peer exchange - peers: Vec, - /// The backoff time in seconds before we allow to reconnect - backoff: Option, - }, + Prune(Prune), } -/// A Gossipsub RPC message sent. +/// Node broadcasts known messages per topic - IHave control message. #[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct IHave { + /// The topic of the messages. + pub(crate) topic_hash: TopicHash, + /// A list of known message ids (peer_id + sequence _number) as a string. + pub(crate) message_ids: Vec, +} + +/// The node requests specific message ids (peer_id + sequence _number) - IWant control message. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct IWant { + /// A list of known message ids (peer_id + sequence _number) as a string. + pub(crate) message_ids: Vec, +} + +/// The node has been added to the mesh - Graft control message. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct Graft { + /// The mesh topic the peer should be added to. + pub(crate) topic_hash: TopicHash, +} + +/// The node has been removed from the mesh - Prune control message. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct Prune { + /// The mesh topic the peer should be removed from. + pub(crate) topic_hash: TopicHash, + /// A list of peers to be proposed to the removed peer as peer exchange + pub(crate) peers: Vec, + /// The backoff time in seconds before we allow to reconnect + pub(crate) backoff: Option, +} + +/// A Gossipsub RPC message sent. +#[derive(Debug)] pub enum RpcOut { - /// Publish a Gossipsub message on network. - Publish(RawMessage), - /// Forward a Gossipsub message to the network. - Forward(RawMessage), + /// Publish a Gossipsub message on network. [`Delay`] tags the time we attempted to send it. + Publish { message: RawMessage, timeout: Delay }, + /// Forward a Gossipsub message on network. [`Delay`] tags the time we attempted to send it. + Forward { message: RawMessage, timeout: Delay }, /// Subscribe a topic. Subscribe(TopicHash), /// Unsubscribe a topic. Unsubscribe(TopicHash), - /// List of Gossipsub control messages. - Control(ControlAction), + /// Send a GRAFT control message. + Graft(Graft), + /// Send a PRUNE control message. + Prune(Prune), + /// Send a IHave control message. + IHave(IHave), + /// Send a IWant control message. + IWant(IWant), } impl RpcOut { @@ -262,12 +328,18 @@ impl From for proto::RPC { /// Converts the RPC into protobuf format. fn from(rpc: RpcOut) -> Self { match rpc { - RpcOut::Publish(message) => proto::RPC { + RpcOut::Publish { + message, + timeout: _, + } => proto::RPC { subscriptions: Vec::new(), publish: vec![message.into()], control: None, }, - RpcOut::Forward(message) => proto::RPC { + RpcOut::Forward { + message, + timeout: _, + } => proto::RPC { publish: vec![message.into()], subscriptions: Vec::new(), control: None, @@ -288,7 +360,7 @@ impl From for proto::RPC { }], control: None, }, - RpcOut::Control(ControlAction::IHave { + RpcOut::IHave(IHave { topic_hash, message_ids, }) => proto::RPC { @@ -304,7 +376,7 @@ impl From for proto::RPC { prune: vec![], }), }, - RpcOut::Control(ControlAction::IWant { message_ids }) => proto::RPC { + RpcOut::IWant(IWant { message_ids }) => proto::RPC { publish: Vec::new(), subscriptions: Vec::new(), control: Some(proto::ControlMessage { @@ -316,7 +388,7 @@ impl From for proto::RPC { prune: vec![], }), }, - RpcOut::Control(ControlAction::Graft { topic_hash }) => proto::RPC { + RpcOut::Graft(Graft { topic_hash }) => proto::RPC { publish: Vec::new(), subscriptions: vec![], control: Some(proto::ControlMessage { @@ -328,7 +400,7 @@ impl From for proto::RPC { prune: vec![], }), }, - RpcOut::Control(ControlAction::Prune { + RpcOut::Prune(Prune { topic_hash, peers, backoff, @@ -420,33 +492,33 @@ impl From for proto::RPC { for action in rpc.control_msgs { match action { // collect all ihave messages - ControlAction::IHave { + ControlAction::IHave(IHave { topic_hash, message_ids, - } => { + }) => { let rpc_ihave = proto::ControlIHave { topic_id: Some(topic_hash.into_string()), message_ids: message_ids.into_iter().map(|msg_id| msg_id.0).collect(), }; control.ihave.push(rpc_ihave); } - ControlAction::IWant { message_ids } => { + ControlAction::IWant(IWant { message_ids }) => { let rpc_iwant = proto::ControlIWant { message_ids: message_ids.into_iter().map(|msg_id| msg_id.0).collect(), }; control.iwant.push(rpc_iwant); } - ControlAction::Graft { topic_hash } => { + ControlAction::Graft(Graft { topic_hash }) => { let rpc_graft = proto::ControlGraft { topic_id: Some(topic_hash.into_string()), }; control.graft.push(rpc_graft); } - ControlAction::Prune { + ControlAction::Prune(Prune { topic_hash, peers, backoff, - } => { + }) => { let rpc_prune = proto::ControlPrune { topic_id: Some(topic_hash.into_string()), peers: peers @@ -514,3 +586,233 @@ impl fmt::Display for PeerKind { f.write_str(self.as_ref()) } } + +/// `RpcOut` sender that is priority aware. +#[derive(Debug, Clone)] +pub(crate) struct RpcSender { + cap: usize, + len: Arc, + pub(crate) priority_sender: Sender, + pub(crate) non_priority_sender: Sender, + priority_receiver: Receiver, + non_priority_receiver: Receiver, +} + +impl RpcSender { + /// Create a RpcSender. + pub(crate) fn new(cap: usize) -> RpcSender { + let (priority_sender, priority_receiver) = async_channel::unbounded(); + let (non_priority_sender, non_priority_receiver) = async_channel::bounded(cap / 2); + let len = Arc::new(AtomicUsize::new(0)); + RpcSender { + cap: cap / 2, + len, + priority_sender, + non_priority_sender, + priority_receiver, + non_priority_receiver, + } + } + + /// Create a new Receiver to the sender. + pub(crate) fn new_receiver(&self) -> RpcReceiver { + RpcReceiver { + priority_len: self.len.clone(), + priority: Box::pin(self.priority_receiver.clone().peekable()), + non_priority: Box::pin(self.non_priority_receiver.clone().peekable()), + } + } + + /// Send a `RpcOut::Graft` message to the `RpcReceiver` + /// this is high priority. + pub(crate) fn graft(&mut self, graft: Graft) { + self.priority_sender + .try_send(RpcOut::Graft(graft)) + .expect("Channel is unbounded and should always be open"); + } + + /// Send a `RpcOut::Prune` message to the `RpcReceiver` + /// this is high priority. + pub(crate) fn prune(&mut self, prune: Prune) { + self.priority_sender + .try_send(RpcOut::Prune(prune)) + .expect("Channel is unbounded and should always be open"); + } + + /// Send a `RpcOut::IHave` message to the `RpcReceiver` + /// this is low priority, if the queue is full an Err is returned. + #[allow(clippy::result_large_err)] + pub(crate) fn ihave(&mut self, ihave: IHave) -> Result<(), RpcOut> { + self.non_priority_sender + .try_send(RpcOut::IHave(ihave)) + .map_err(|err| err.into_inner()) + } + + /// Send a `RpcOut::IHave` message to the `RpcReceiver` + /// this is low priority, if the queue is full an Err is returned. + #[allow(clippy::result_large_err)] + pub(crate) fn iwant(&mut self, iwant: IWant) -> Result<(), RpcOut> { + self.non_priority_sender + .try_send(RpcOut::IWant(iwant)) + .map_err(|err| err.into_inner()) + } + + /// Send a `RpcOut::Subscribe` message to the `RpcReceiver` + /// this is high priority. + pub(crate) fn subscribe(&mut self, topic: TopicHash) { + self.priority_sender + .try_send(RpcOut::Subscribe(topic)) + .expect("Channel is unbounded and should always be open"); + } + + /// Send a `RpcOut::Unsubscribe` message to the `RpcReceiver` + /// this is high priority. + pub(crate) fn unsubscribe(&mut self, topic: TopicHash) { + self.priority_sender + .try_send(RpcOut::Unsubscribe(topic)) + .expect("Channel is unbounded and should always be open"); + } + + /// Send a `RpcOut::Publish` message to the `RpcReceiver` + /// this is high priority. If message sending fails, an `Err` is returned. + pub(crate) fn publish( + &mut self, + message: RawMessage, + timeout: Duration, + metrics: Option<&mut Metrics>, + ) -> Result<(), ()> { + if self.len.load(Ordering::Relaxed) >= self.cap { + return Err(()); + } + self.priority_sender + .try_send(RpcOut::Publish { + message: message.clone(), + timeout: Delay::new(timeout), + }) + .expect("Channel is unbounded and should always be open"); + self.len.fetch_add(1, Ordering::Relaxed); + + if let Some(m) = metrics { + m.msg_sent(&message.topic, message.raw_protobuf_len()); + } + + Ok(()) + } + + /// Send a `RpcOut::Forward` message to the `RpcReceiver` + /// this is high priority. If the queue is full the message is discarded. + pub(crate) fn forward( + &mut self, + message: RawMessage, + timeout: Duration, + metrics: Option<&mut Metrics>, + ) -> Result<(), ()> { + self.non_priority_sender + .try_send(RpcOut::Forward { + message: message.clone(), + timeout: Delay::new(timeout), + }) + .map_err(|_| ())?; + + if let Some(m) = metrics { + m.msg_sent(&message.topic, message.raw_protobuf_len()); + } + + Ok(()) + } + + /// Returns the current size of the priority queue. + pub(crate) fn priority_len(&self) -> usize { + self.len.load(Ordering::Relaxed) + } + + /// Returns the current size of the non-priority queue. + pub(crate) fn non_priority_len(&self) -> usize { + self.non_priority_sender.len() + } +} + +/// `RpcOut` sender that is priority aware. +#[derive(Debug)] +pub struct RpcReceiver { + /// The maximum length of the priority queue. + pub(crate) priority_len: Arc, + /// The priority queue receiver. + pub(crate) priority: Pin>>>, + /// The non priority queue receiver. + pub(crate) non_priority: Pin>>>, +} + +impl RpcReceiver { + // Peek the next message in the queues and return it if its timeout has elapsed. + // Returns `None` if there aren't any more messages on the stream or none is stale. + pub(crate) fn poll_stale(&mut self, cx: &mut Context<'_>) -> Poll> { + // Peek priority queue. + let priority = match self.priority.as_mut().poll_peek_mut(cx) { + Poll::Ready(Some(RpcOut::Publish { + message: _, + ref mut timeout, + })) => { + if Pin::new(timeout).poll(cx).is_ready() { + // Return the message. + let dropped = futures::ready!(self.priority.poll_next_unpin(cx)) + .expect("There should be a message"); + return Poll::Ready(Some(dropped)); + } + Poll::Ready(None) + } + poll => poll, + }; + + let non_priority = match self.non_priority.as_mut().poll_peek_mut(cx) { + Poll::Ready(Some(RpcOut::Forward { + message: _, + ref mut timeout, + })) => { + if Pin::new(timeout).poll(cx).is_ready() { + // Return the message. + let dropped = futures::ready!(self.non_priority.poll_next_unpin(cx)) + .expect("There should be a message"); + return Poll::Ready(Some(dropped)); + } + Poll::Ready(None) + } + poll => poll, + }; + + match (priority, non_priority) { + (Poll::Ready(None), Poll::Ready(None)) => Poll::Ready(None), + _ => Poll::Pending, + } + } + + /// Poll queues and return true if both are empty. + pub(crate) fn poll_is_empty(&mut self, cx: &mut Context<'_>) -> bool { + matches!( + ( + self.priority.as_mut().poll_peek(cx), + self.non_priority.as_mut().poll_peek(cx), + ), + (Poll::Ready(None), Poll::Ready(None)) + ) + } +} + +impl Stream for RpcReceiver { + type Item = RpcOut; + + fn poll_next( + mut self: std::pin::Pin<&mut Self>, + cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + // The priority queue is first polled. + if let Poll::Ready(rpc) = Pin::new(&mut self.priority).poll_next(cx) { + if let Some(RpcOut::Publish { .. }) = rpc { + self.priority_len.fetch_sub(1, Ordering::Relaxed); + } + return Poll::Ready(rpc); + } + // Then we poll the non priority. + Pin::new(&mut self.non_priority).poll_next(cx) + } +}