From 0afb158d74dc8e59e64e4031cc9134bf7cb3ea6b Mon Sep 17 00:00:00 2001 From: manasnagaraj Date: Wed, 27 Nov 2024 16:08:33 -0500 Subject: [PATCH 01/16] gossipsub-1.2-beta implemented --- Cargo.lock | 10 + Cargo.toml | 2 + protocols/gossipsub/Cargo.toml | 1 + protocols/gossipsub/src/behaviour.rs | 213 +++++++--- protocols/gossipsub/src/behaviour/tests.rs | 226 +++++++++- .../gossipsub/src/generated/gossipsub/pb.rs | 390 ++++++++++-------- protocols/gossipsub/src/generated/rpc.proto | 9 +- protocols/gossipsub/src/gossip_promises.rs | 8 + protocols/gossipsub/src/metrics.rs | 35 ++ protocols/gossipsub/src/protocol.rs | 25 +- protocols/gossipsub/src/rpc.rs | 4 +- protocols/gossipsub/src/types.rs | 55 ++- 12 files changed, 735 insertions(+), 243 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index e24db6e69d4..66b79c79fd4 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1927,6 +1927,15 @@ dependencies = [ "allocator-api2", ] +[[package]] +name = "hashlink" +version = "0.9.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6ba4ff7128dee98c7dc9794b6a411377e1404dba1c97deb8d1a55297bd25d8af" +dependencies = [ + "hashbrown 0.14.3", +] + [[package]] name = "heck" version = "0.4.1" @@ -2768,6 +2777,7 @@ dependencies = [ "futures", "futures-timer", "getrandom 0.2.15", + "hashlink", "hex", "hex_fmt", "libp2p-core", diff --git a/Cargo.toml b/Cargo.toml index a7f944d22fc..cd5d3af3679 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -75,6 +75,7 @@ rust-version = "1.75.0" asynchronous-codec = { version = "0.7.0" } futures-bounded = { version = "0.2.4" } futures-rustls = { version = "0.26.0", default-features = false } +hashlink = "0.9.0" libp2p = { version = "0.54.2", path = "libp2p" } libp2p-allow-block-list = { version = "0.4.2", path = "misc/allow-block-list" } libp2p-autonat = { version = "0.13.1", path = "protocols/autonat" } @@ -132,6 +133,7 @@ web-time = "1.1.0" ring = "0.17.8" rcgen = "0.11.3" + [patch.crates-io] # Patch away `libp2p-identity` in our dependency tree with the workspace version. diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml index c09286c8aa0..1926b321bd6 100644 --- a/protocols/gossipsub/Cargo.toml +++ b/protocols/gossipsub/Cargo.toml @@ -24,6 +24,7 @@ fnv = "1.0.7" futures = { workspace = true } futures-timer = "3.0.2" getrandom = "0.2.15" +hashlink.workspace = true hex_fmt = "0.3.0" web-time = { workspace = true } libp2p-core = { workspace = true } diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index fae45ed452e..c3d974ae887 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -29,7 +29,7 @@ use std::{ time::Duration, }; -use futures::FutureExt; +use futures::{FutureExt, StreamExt}; use futures_timer::Delay; use prometheus_client::registry::Registry; use rand::{seq::SliceRandom, thread_rng}; @@ -53,10 +53,7 @@ use crate::subscription_filter::{AllowAllSubscriptionFilter, TopicSubscriptionFi use crate::time_cache::DuplicateCache; use crate::topic::{Hasher, Topic, TopicHash}; use crate::transform::{DataTransform, IdentityTransform}; -use crate::types::{ - ControlAction, Message, MessageAcceptance, MessageId, PeerInfo, RawMessage, Subscription, - SubscriptionAction, -}; +use crate::types::{ControlAction, IDontWant, Message, MessageAcceptance, MessageId, PeerInfo, RawMessage, Subscription, SubscriptionAction}; use crate::types::{PeerConnections, PeerKind, RpcOut}; use crate::{backoff::BackoffStorage, FailedMessages}; use crate::{ @@ -77,10 +74,17 @@ use crate::{rpc_proto::proto, TopicScoreParams}; use crate::{PublishError, SubscriptionError, ValidationError}; use quick_protobuf::{MessageWrite, Writer}; use std::{cmp::Ordering::Equal, fmt::Debug}; +use hashlink::LinkedHashMap; #[cfg(test)] mod tests; +/// IDONTWANT cache capacity. +const IDONTWANT_CAP: usize = 10_000; + +/// IDONTWANT timeout before removal. +const IDONTWANT_TIMEOUT: Duration = Duration::new(3, 0); + /// Determines if published messages should be signed or not. /// /// Without signing, a number of privacy preserving modes can be selected. @@ -314,7 +318,7 @@ pub struct Behaviour { /// Stores optional peer score data together with thresholds, decay interval and gossip /// promises. - peer_score: Option<(PeerScore, PeerScoreThresholds, Delay, GossipPromises)>, + peer_score: Option<(PeerScore, PeerScoreThresholds, Delay)>, /// Counts the number of `IHAVE` received from each peer since the last heartbeat. count_received_ihave: HashMap, @@ -339,6 +343,9 @@ pub struct Behaviour { /// Tracks the numbers of failed messages per peer-id. failed_messages: HashMap, + + /// Tracks recently sent `IWANT` messages and checks if peers respond to them. + gossip_promises: GossipPromises, } impl Behaviour @@ -472,6 +479,7 @@ where subscription_filter, data_transform, failed_messages: Default::default(), + gossip_promises: Default::default(), }) } } @@ -905,7 +913,7 @@ where let interval = Delay::new(params.decay_interval); let peer_score = PeerScore::new_with_message_delivery_time_callback(params, callback); - self.peer_score = Some((peer_score, threshold, interval, GossipPromises::default())); + self.peer_score = Some((peer_score, threshold, interval)); Ok(()) } @@ -1169,7 +1177,7 @@ where } fn score_below_threshold_from_scores( - peer_score: &Option<(PeerScore, PeerScoreThresholds, Delay, GossipPromises)>, + peer_score: &Option<(PeerScore, PeerScoreThresholds, Delay)>, peer_id: &PeerId, threshold: impl Fn(&PeerScoreThresholds) -> f64, ) -> (bool, f64) { @@ -1230,10 +1238,7 @@ where return false; } - self.peer_score - .as_ref() - .map(|(_, _, _, promises)| !promises.contains(id)) - .unwrap_or(true) + !self.gossip_promises.contains(id) }; for (topic, ids) in ihave_msgs { @@ -1280,13 +1285,11 @@ where iwant_ids_vec.truncate(iask); *iasked += iask; - if let Some((_, _, _, gossip_promises)) = &mut self.peer_score { - gossip_promises.add_promise( - *peer_id, - &iwant_ids_vec, - Instant::now() + self.config.iwant_followup_time(), - ); - } + self.gossip_promises.add_promise( + *peer_id, + &iwant_ids_vec, + Instant::now() + self.config.iwant_followup_time(), + ); tracing::trace!( peer=%peer_id, "IHAVE: Asking for the following messages from peer: {:?}", @@ -1649,14 +1652,16 @@ where peer=%propagation_source, "Rejecting message from blacklisted peer" ); - if let Some((peer_score, .., gossip_promises)) = &mut self.peer_score { + self.gossip_promises + .reject_message(msg_id, &RejectReason::BlackListedPeer); + if let Some((peer_score, ..)) = &mut self.peer_score { peer_score.reject_message( propagation_source, msg_id, &raw_message.topic, RejectReason::BlackListedPeer, ); - gossip_promises.reject_message(msg_id, &RejectReason::BlackListedPeer); + // gossip_promises.reject_message(msg_id, &RejectReason::BlackListedPeer); } return false; } @@ -1738,6 +1743,9 @@ where // Calculate the message id on the transformed data. let msg_id = self.config.message_id(&message); + // Broadcast IDONTWANT messages. + self.send_idontwant(&raw_message, &msg_id, propagation_source); + // Check the validity of the message // Peers get penalized if this message is invalid. We don't add it to the duplicate cache // and instead continually penalize peers that repeatedly send this message. @@ -1765,9 +1773,11 @@ where // Tells score that message arrived (but is maybe not fully validated yet). // Consider the message as delivered for gossip promises. - if let Some((peer_score, .., gossip_promises)) = &mut self.peer_score { + self.gossip_promises.message_delivered(&msg_id); + + // Tells score that message arrived (but is maybe not fully validated yet). + if let Some((peer_score, ..)) = &mut self.peer_score { peer_score.validate_message(propagation_source, &msg_id, &message.topic); - gossip_promises.message_delivered(&msg_id); } // Add the message to our memcache @@ -1809,7 +1819,7 @@ where raw_message: &RawMessage, reject_reason: RejectReason, ) { - if let Some((peer_score, .., gossip_promises)) = &mut self.peer_score { + if let Some((peer_score, ..)) = &mut self.peer_score { if let Some(metrics) = self.metrics.as_mut() { metrics.register_invalid_message(&raw_message.topic); } @@ -1824,7 +1834,8 @@ where reject_reason, ); - gossip_promises.reject_message(&message_id, &reject_reason); + self.gossip_promises + .reject_message(&message_id, &reject_reason); } else { // The message is invalid, we reject it ignoring any gossip promises. If a peer is // advertising this message via an IHAVE and it's invalid it will be double @@ -1897,7 +1908,7 @@ where // if the mesh needs peers add the peer to the mesh if !self.explicit_peers.contains(propagation_source) - && matches!(peer.kind, PeerKind::Gossipsubv1_1 | PeerKind::Gossipsub) + && peer.kind.is_gossipsub() && !Self::score_below_threshold_from_scores( &self.peer_score, propagation_source, @@ -1998,8 +2009,8 @@ where /// Applies penalties to peers that did not respond to our IWANT requests. fn apply_iwant_penalties(&mut self) { - if let Some((peer_score, .., gossip_promises)) = &mut self.peer_score { - for (peer, count) in gossip_promises.get_broken_promises() { + if let Some((peer_score, ..)) = &mut self.peer_score { + for (peer, count) in self.gossip_promises.get_broken_promises() { peer_score.add_penalty(&peer, count); if let Some(metrics) = self.metrics.as_mut() { metrics.register_score_penalty(Penalty::BrokenPromise); @@ -2220,7 +2231,7 @@ where && peers.len() > 1 && self.peer_score.is_some() { - if let Some((_, thresholds, _, _)) = &self.peer_score { + if let Some((_, thresholds, _)) = &self.peer_score { // Opportunistic grafting works as follows: we check the median score of peers // in the mesh; if this score is below the opportunisticGraftThreshold, we // select a few peers at random with score over the median. @@ -2313,7 +2324,7 @@ where for (topic_hash, peers) in self.fanout.iter_mut() { let mut to_remove_peers = Vec::new(); let publish_threshold = match &self.peer_score { - Some((_, thresholds, _, _)) => thresholds.publish_threshold, + Some((_, thresholds, _)) => thresholds.publish_threshold, _ => 0.0, }; for peer_id in peers.iter() { @@ -2406,6 +2417,17 @@ where } self.failed_messages.shrink_to_fit(); + // Flush stale IDONTWANTs. + for peer in self.connected_peers.values_mut() { + while let Some((_front, instant)) = peer.dont_send.front() { + if (*instant + IDONTWANT_TIMEOUT) >= Instant::now() { + break; + } else { + peer.dont_send.pop_front(); + } + } + } + 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); @@ -2561,6 +2583,60 @@ where } } + + /// Helper function which sends an IDONTWANT message to mesh\[topic\] peers. + fn send_idontwant( + &mut self, + message: &RawMessage, + msg_id: &MessageId, + propagation_source: &PeerId, + ) { + let Some(mesh_peers) = self.mesh.get(&message.topic) else { + return; + }; + + let iwant_peers = self.gossip_promises.peers_for_message(msg_id); + + let recipient_peers = mesh_peers + .iter() + .chain(iwant_peers.iter()) + .filter(|peer_id| { + *peer_id != propagation_source && Some(*peer_id) != message.source.as_ref() + }); + + for peer_id in recipient_peers { + let Some(peer) = self.connected_peers.get_mut(peer_id) else { + tracing::error!(peer = %peer_id, + "Could not IDONTWANT, peer doesn't exist in connected peer list"); + continue; + }; + + // Only gossipsub 1.2 peers support IDONTWANT. + if peer.kind != PeerKind::Gossipsubv1_2_beta { + continue; + } + + if peer + .sender + .send_message(RpcOut::IDontWant(IDontWant { + message_ids: vec![msg_id.clone()], + })) + .is_err() + { + tracing::warn!(peer=%peer_id, "Send Queue full. Could not send IDONTWANT"); + + 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; + } + } + } + /// Helper function which forwards a message to mesh\[topic\] peers. /// /// Returns true if at least one peer was messaged. @@ -2616,16 +2692,43 @@ where } // forward the message to peers - for peer in recipient_peers.iter() { - let event = RpcOut::Forward { - message: message.clone(), - timeout: Delay::new(self.config.forward_queue_duration()), - }; - tracing::debug!(%peer, message=%msg_id, "Sending message to peer"); - self.send_message(*peer, event); + if !recipient_peers.is_empty() { + for peer_id in recipient_peers.iter() { + if let Some(peer) = self.connected_peers.get_mut(peer_id) { + if peer.dont_send.get(msg_id).is_some() { + tracing::debug!(%peer_id, message=%msg_id, "Peer doesn't want message"); + continue; + } + + tracing::debug!(%peer_id, message=%msg_id, "Sending message to peer"); + if peer + .sender + .send_message(RpcOut::Forward { + message: message.clone(), + timeout: Delay::new(self.config.forward_queue_duration()) + }) + .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"); + true + } else { + false } - tracing::debug!("Completed forwarding message"); - true } /// Constructs a [`RawMessage`] performing message signing if required. @@ -2758,7 +2861,7 @@ where failed_messages.non_priority += 1; failed_messages.forward += 1; } - RpcOut::IWant(_) | RpcOut::IHave(_) => { + RpcOut::IWant(_) | RpcOut::IHave(_) | RpcOut::IDontWant(_) => { failed_messages.non_priority += 1; } RpcOut::Graft(_) @@ -2998,6 +3101,7 @@ where connections: vec![], sender: Sender::new(self.config.connection_handler_queue_len()), topics: Default::default(), + dont_send: LinkedHashMap::new(), }); // Add the new connection connected_peer.connections.push(connection_id); @@ -3024,6 +3128,7 @@ where connections: vec![], sender: Sender::new(self.config.connection_handler_queue_len()), topics: Default::default(), + dont_send: LinkedHashMap::new(), }); // Add the new connection connected_peer.connections.push(connection_id); @@ -3073,7 +3178,7 @@ where } HandlerEvent::MessageDropped(rpc) => { // Account for this in the scoring logic - if let Some((peer_score, _, _, _)) = &mut self.peer_score { + if let Some((peer_score, _, _)) = &mut self.peer_score { peer_score.failed_message_slow_peer(&propagation_source); } @@ -3180,6 +3285,24 @@ where peers, backoff, }) => prune_msgs.push((topic_hash, peers, backoff)), + ControlAction::IDontWant(IDontWant { message_ids }) => { + let Some(peer) = self.connected_peers.get_mut(&propagation_source) + else { + tracing::error!(peer = %propagation_source, + "Could not handle IDONTWANT, peer doesn't exist in connected peer list"); + continue; + }; + if let Some(metrics) = self.metrics.as_mut() { + metrics.register_idontwant(message_ids.len()); + } + for message_id in message_ids { + peer.dont_send.insert(message_id, Instant::now()); + // Don't exceed capacity. + if peer.dont_send.len() > IDONTWANT_CAP { + peer.dont_send.pop_front(); + } + } + } } } if !ihave_msgs.is_empty() { @@ -3205,16 +3328,14 @@ where } // update scores - if let Some((peer_score, _, delay, _)) = &mut self.peer_score { - if delay.poll_unpin(cx).is_ready() { + if let Some((peer_score, _, delay)) = &mut self.peer_score { + if delay.poll_unpin(cx).is_ready(){ peer_score.refresh_scores(); - delay.reset(peer_score.params.decay_interval); } } if self.heartbeat.poll_unpin(cx).is_ready() { self.heartbeat(); - self.heartbeat.reset(self.config.heartbeat_interval()); } Poll::Pending @@ -3332,7 +3453,7 @@ fn get_random_peers_dynamic( .iter() .filter(|(_, p)| p.topics.contains(topic_hash)) .filter(|(peer_id, _)| f(peer_id)) - .filter(|(_, p)| p.kind == PeerKind::Gossipsub || p.kind == PeerKind::Gossipsubv1_1) + .filter(|(_, p)| p.kind.is_gossipsub()) .map(|(peer_id, _)| *peer_id) .collect::>(); diff --git a/protocols/gossipsub/src/behaviour/tests.rs b/protocols/gossipsub/src/behaviour/tests.rs index 9567150382a..3a19d015591 100644 --- a/protocols/gossipsub/src/behaviour/tests.rs +++ b/protocols/gossipsub/src/behaviour/tests.rs @@ -32,13 +32,7 @@ use std::net::Ipv4Addr; use std::thread::sleep; #[derive(Default, Debug)] -struct InjectNodes -// TODO: remove trait bound Default when this issue is fixed: -// https://github.com/colin-kiegel/rust-derive-builder/issues/93 -where - D: DataTransform + Default + Clone + Send + 'static, - F: TopicSubscriptionFilter + Clone + Default + Send + 'static, -{ +struct InjectNodes { peer_no: usize, topics: Vec, to_subscribe: bool, @@ -48,6 +42,7 @@ where scoring: Option<(PeerScoreParams, PeerScoreThresholds)>, data_transform: D, subscription_filter: F, + peer_kind: Option, } impl InjectNodes @@ -95,7 +90,7 @@ where let empty = vec![]; for i in 0..self.peer_no { - let (peer, receiver) = add_peer( + let (peer, receiver) = add_peer_with_addr_and_kind( &mut gs, if self.to_subscribe { &topic_hashes @@ -104,6 +99,8 @@ where }, i < self.outbound, i < self.explicit, + Multiaddr::empty(), + self.peer_kind.clone().or(Some(PeerKind::Gossipsubv1_1)), ); peers.push(peer); receivers.insert(peer, receiver); @@ -152,6 +149,11 @@ where self.subscription_filter = subscription_filter; self } + + fn peer_kind(mut self, peer_kind: PeerKind) -> Self { + self.peer_kind = Some(peer_kind); + self + } } fn inject_nodes() -> InjectNodes @@ -238,6 +240,7 @@ where connections: vec![connection_id], topics: Default::default(), sender, + dont_send: LinkedHashMap::new(), }, ); @@ -623,6 +626,7 @@ fn test_join() { connections: vec![connection_id], topics: Default::default(), sender, + dont_send: LinkedHashMap::new(), }, ); receivers.insert(random_peer, receiver); @@ -1018,6 +1022,7 @@ fn test_get_random_peers() { connections: vec![ConnectionId::new_unchecked(0)], topics: topics.clone(), sender: Sender::new(gs.config.connection_handler_queue_len()), + dont_send: LinkedHashMap::new(), }, ); } @@ -4580,9 +4585,9 @@ fn test_ignore_too_many_messages_in_ihave() { let (peer, receiver) = add_peer(&mut gs, &topics, false, false); receivers.insert(peer, receiver); - //peer has 20 messages + //peer has 30 messages let mut seq = 0; - let message_ids: Vec<_> = (0..20) + let message_ids: Vec<_> = (0..30) .map(|_| random_message(&mut seq, &topics)) .map(|msg| gs.data_transform.inbound_transform(msg).unwrap()) .map(|msg| config.message_id(&msg)) @@ -4624,7 +4629,7 @@ fn test_ignore_too_many_messages_in_ihave() { gs.heartbeat(); gs.handle_ihave( &peer, - vec![(topics[0].clone(), message_ids[10..20].to_vec())], + vec![(topics[0].clone(), message_ids[20..30].to_vec())], ); //we sent 10 iwant messages ids via a IWANT rpc. @@ -5237,6 +5242,194 @@ fn test_graft_without_subscribe() { let _ = gs.unsubscribe(&Topic::new(topic)); } +/// Test that a node sends IDONTWANT messages to the mesh peers +/// that run Gossipsub v1.2. +#[test] +fn sends_idontwant() { + let (mut gs, peers, receivers, topic_hashes) = inject_nodes1() + .peer_no(5) + .topics(vec![String::from("topic1")]) + .to_subscribe(true) + .gs_config(Config::default()) + .explicit(1) + .peer_kind(PeerKind::Gossipsubv1_2_beta) + .create_network(); + + let local_id = PeerId::random(); + + let message = RawMessage { + source: Some(peers[1]), + data: vec![12], + sequence_number: Some(0), + topic: topic_hashes[0].clone(), + signature: None, + key: None, + validated: true, + }; + gs.handle_received_message(message.clone(), &local_id); + assert_eq!( + receivers + .into_iter() + .fold(0, |mut idontwants, (peer_id, c)| { + let non_priority = c.non_priority.get_ref(); + while !non_priority.is_empty() { + if let Ok(RpcOut::IDontWant(_)) = non_priority.try_recv() { + assert_ne!(peer_id, peers[1]); + idontwants += 1; + } + } + idontwants + }), + 3, + "IDONTWANT was not sent" + ); +} + +/// Test that a node doesn't send IDONTWANT messages to the mesh peers +/// that don't run Gossipsub v1.2. +#[test] +fn doesnt_send_idontwant() { + let (mut gs, peers, receivers, topic_hashes) = inject_nodes1() + .peer_no(5) + .topics(vec![String::from("topic1")]) + .to_subscribe(true) + .gs_config(Config::default()) + .explicit(1) + .peer_kind(PeerKind::Gossipsubv1_1) + .create_network(); + + let local_id = PeerId::random(); + + let message = RawMessage { + source: Some(peers[1]), + data: vec![12], + sequence_number: Some(0), + topic: topic_hashes[0].clone(), + signature: None, + key: None, + validated: true, + }; + gs.handle_received_message(message.clone(), &local_id); + assert_eq!( + receivers + .into_iter() + .fold(0, |mut idontwants, (peer_id, c)| { + let non_priority = c.non_priority.get_ref(); + while !non_priority.is_empty() { + if matches!(non_priority.try_recv(), Ok(RpcOut::IDontWant(_)) if peer_id != peers[1]) { + idontwants += 1; + } + } + idontwants + }), + 0, + "IDONTWANT were sent" + ); +} + +/// Test that a node doesn't forward a messages to the mesh peers +/// that sent IDONTWANT. +#[test] +fn doesnt_forward_idontwant() { + let (mut gs, peers, receivers, topic_hashes) = inject_nodes1() + .peer_no(4) + .topics(vec![String::from("topic1")]) + .to_subscribe(true) + .gs_config(Config::default()) + .explicit(1) + .peer_kind(PeerKind::Gossipsubv1_2_beta) + .create_network(); + + let local_id = PeerId::random(); + + let raw_message = RawMessage { + source: Some(peers[1]), + data: vec![12], + sequence_number: Some(0), + topic: topic_hashes[0].clone(), + signature: None, + key: None, + validated: true, + }; + let message = gs + .data_transform + .inbound_transform(raw_message.clone()) + .unwrap(); + let message_id = gs.config.message_id(&message); + let peer = gs.connected_peers.get_mut(&peers[2]).unwrap(); + peer.dont_send.insert(message_id, Instant::now()); + + gs.handle_received_message(raw_message.clone(), &local_id); + assert_eq!( + receivers.into_iter().fold(0, |mut fwds, (peer_id, c)| { + let non_priority = c.non_priority.get_ref(); + while !non_priority.is_empty() { + if let Ok(RpcOut::Forward { .. }) = non_priority.try_recv() { + assert_ne!(peer_id, peers[2]); + fwds += 1; + } + } + fwds + }), + 2, + "IDONTWANT was not sent" + ); +} + +/// Test that a node parses an +/// IDONTWANT message to the respective peer. +#[test] +fn parses_idontwant() { + let (mut gs, peers, _receivers, _topic_hashes) = inject_nodes1() + .peer_no(2) + .topics(vec![String::from("topic1")]) + .to_subscribe(true) + .gs_config(Config::default()) + .explicit(1) + .peer_kind(PeerKind::Gossipsubv1_2_beta) + .create_network(); + + let message_id = MessageId::new(&[0, 1, 2, 3]); + let rpc = Rpc { + messages: vec![], + subscriptions: vec![], + control_msgs: vec![ControlAction::IDontWant(IDontWant { + message_ids: vec![message_id.clone()], + })], + }; + gs.on_connection_handler_event( + peers[1], + ConnectionId::new_unchecked(0), + HandlerEvent::Message { + rpc, + invalid_messages: vec![], + }, + ); + let peer = gs.connected_peers.get_mut(&peers[1]).unwrap(); + assert!(peer.dont_send.get(&message_id).is_some()); +} + +/// Test that a node clears stale IDONTWANT messages. +#[test] +fn clear_stale_idontwant() { + let (mut gs, peers, _receivers, _topic_hashes) = inject_nodes1() + .peer_no(4) + .topics(vec![String::from("topic1")]) + .to_subscribe(true) + .gs_config(Config::default()) + .explicit(1) + .peer_kind(PeerKind::Gossipsubv1_2_beta) + .create_network(); + + let peer = gs.connected_peers.get_mut(&peers[2]).unwrap(); + peer.dont_send + .insert(MessageId::new(&[1, 2, 3, 4]), Instant::now()); + std::thread::sleep(Duration::from_secs(3)); + gs.heartbeat(); + let peer = gs.connected_peers.get_mut(&peers[2]).unwrap(); + assert!(peer.dont_send.is_empty()); +} + #[test] fn test_all_queues_full() { let gs_config = ConfigBuilder::default() @@ -5260,6 +5453,7 @@ fn test_all_queues_full() { connections: vec![ConnectionId::new_unchecked(0)], topics: topics.clone(), sender: Sender::new(2), + dont_send: LinkedHashMap::new(), }, ); @@ -5294,6 +5488,7 @@ fn test_slow_peer_returns_failed_publish() { connections: vec![ConnectionId::new_unchecked(0)], topics: topics.clone(), sender: Sender::new(2), + dont_send: LinkedHashMap::new(), }, ); let peer_id = PeerId::random(); @@ -5305,6 +5500,7 @@ fn test_slow_peer_returns_failed_publish() { connections: vec![ConnectionId::new_unchecked(0)], topics: topics.clone(), sender: Sender::new(gs.config.connection_handler_queue_len()), + dont_send: LinkedHashMap::new(), }, ); @@ -5365,6 +5561,7 @@ fn test_slow_peer_returns_failed_ihave_handling() { connections: vec![ConnectionId::new_unchecked(0)], topics: topics.clone(), sender: Sender::new(2), + dont_send: LinkedHashMap::new(), }, ); peers.push(slow_peer_id); @@ -5380,6 +5577,7 @@ fn test_slow_peer_returns_failed_ihave_handling() { connections: vec![ConnectionId::new_unchecked(0)], topics: topics.clone(), sender: Sender::new(gs.config.connection_handler_queue_len()), + dont_send: LinkedHashMap::new(), }, ); @@ -5458,6 +5656,7 @@ fn test_slow_peer_returns_failed_iwant_handling() { connections: vec![ConnectionId::new_unchecked(0)], topics: topics.clone(), sender: Sender::new(2), + dont_send: LinkedHashMap::new(), }, ); peers.push(slow_peer_id); @@ -5473,6 +5672,7 @@ fn test_slow_peer_returns_failed_iwant_handling() { connections: vec![ConnectionId::new_unchecked(0)], topics: topics.clone(), sender: Sender::new(gs.config.connection_handler_queue_len()), + dont_send: LinkedHashMap::new(), }, ); @@ -5548,6 +5748,7 @@ fn test_slow_peer_returns_failed_forward() { connections: vec![ConnectionId::new_unchecked(0)], topics: topics.clone(), sender: Sender::new(2), + dont_send: LinkedHashMap::new(), }, ); peers.push(slow_peer_id); @@ -5563,6 +5764,7 @@ fn test_slow_peer_returns_failed_forward() { connections: vec![ConnectionId::new_unchecked(0)], topics: topics.clone(), sender: Sender::new(gs.config.connection_handler_queue_len()), + dont_send: LinkedHashMap::new(), }, ); @@ -5643,6 +5845,7 @@ fn test_slow_peer_is_downscored_on_publish() { connections: vec![ConnectionId::new_unchecked(0)], topics: topics.clone(), sender: Sender::new(2), + dont_send: LinkedHashMap::new(), }, ); gs.peer_score.as_mut().unwrap().0.add_peer(slow_peer_id); @@ -5655,6 +5858,7 @@ fn test_slow_peer_is_downscored_on_publish() { connections: vec![ConnectionId::new_unchecked(0)], topics: topics.clone(), sender: Sender::new(gs.config.connection_handler_queue_len()), + dont_send: LinkedHashMap::new(), }, ); diff --git a/protocols/gossipsub/src/generated/gossipsub/pb.rs b/protocols/gossipsub/src/generated/gossipsub/pb.rs index 9a074fd61fc..c6a86fcfe92 100644 --- a/protocols/gossipsub/src/generated/gossipsub/pb.rs +++ b/protocols/gossipsub/src/generated/gossipsub/pb.rs @@ -40,9 +40,9 @@ impl<'a> MessageRead<'a> for RPC { impl MessageWrite for RPC { fn get_size(&self) -> usize { 0 - + self.subscriptions.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() - + self.publish.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() - + self.control.as_ref().map_or(0, |m| 1 + sizeof_len((m).get_size())) + + self.subscriptions.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() + + self.publish.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() + + self.control.as_ref().map_or(0, |m| 1 + sizeof_len((m).get_size())) } fn write_message(&self, w: &mut Writer) -> Result<()> { @@ -55,43 +55,43 @@ impl MessageWrite for RPC { pub mod mod_RPC { -use super::*; - -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Debug, Default, PartialEq, Clone)] -pub struct SubOpts { - pub subscribe: Option, - pub topic_id: Option, -} - -impl<'a> MessageRead<'a> for SubOpts { - fn from_reader(r: &mut BytesReader, bytes: &'a [u8]) -> Result { - let mut msg = Self::default(); - while !r.is_eof() { - match r.next_tag(bytes) { - Ok(8) => msg.subscribe = Some(r.read_bool(bytes)?), - Ok(18) => msg.topic_id = Some(r.read_string(bytes)?.to_owned()), - Ok(t) => { r.read_unknown(bytes, t)?; } - Err(e) => return Err(e), + use super::*; + + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Debug, Default, PartialEq, Clone)] + pub struct SubOpts { + pub subscribe: Option, + pub topic_id: Option, + } + + impl<'a> MessageRead<'a> for SubOpts { + fn from_reader(r: &mut BytesReader, bytes: &'a [u8]) -> Result { + let mut msg = Self::default(); + while !r.is_eof() { + match r.next_tag(bytes) { + Ok(8) => msg.subscribe = Some(r.read_bool(bytes)?), + Ok(18) => msg.topic_id = Some(r.read_string(bytes)?.to_owned()), + Ok(t) => { r.read_unknown(bytes, t)?; } + Err(e) => return Err(e), + } } + Ok(msg) } - Ok(msg) } -} -impl MessageWrite for SubOpts { - fn get_size(&self) -> usize { - 0 - + self.subscribe.as_ref().map_or(0, |m| 1 + sizeof_varint(*(m) as u64)) - + self.topic_id.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) - } + impl MessageWrite for SubOpts { + fn get_size(&self) -> usize { + 0 + + self.subscribe.as_ref().map_or(0, |m| 1 + sizeof_varint(*(m) as u64)) + + self.topic_id.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + } - fn write_message(&self, w: &mut Writer) -> Result<()> { - if let Some(ref s) = self.subscribe { w.write_with_tag(8, |w| w.write_bool(*s))?; } - if let Some(ref s) = self.topic_id { w.write_with_tag(18, |w| w.write_string(&**s))?; } - Ok(()) + fn write_message(&self, w: &mut Writer) -> Result<()> { + if let Some(ref s) = self.subscribe { w.write_with_tag(8, |w| w.write_bool(*s))?; } + if let Some(ref s) = self.topic_id { w.write_with_tag(18, |w| w.write_string(&**s))?; } + Ok(()) + } } -} } @@ -128,12 +128,12 @@ impl<'a> MessageRead<'a> for Message { impl MessageWrite for Message { fn get_size(&self) -> usize { 0 - + self.from.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) - + self.data.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) - + self.seqno.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) - + 1 + sizeof_len((&self.topic).len()) - + self.signature.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) - + self.key.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + + self.from.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + + self.data.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + + self.seqno.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + + 1 + sizeof_len((&self.topic).len()) + + self.signature.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + + self.key.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) } fn write_message(&self, w: &mut Writer) -> Result<()> { @@ -154,6 +154,7 @@ pub struct ControlMessage { pub iwant: Vec, pub graft: Vec, pub prune: Vec, + pub idontwant: Vec, } impl<'a> MessageRead<'a> for ControlMessage { @@ -165,6 +166,7 @@ impl<'a> MessageRead<'a> for ControlMessage { Ok(18) => msg.iwant.push(r.read_message::(bytes)?), Ok(26) => msg.graft.push(r.read_message::(bytes)?), Ok(34) => msg.prune.push(r.read_message::(bytes)?), + Ok(42) => msg.idontwant.push(r.read_message::(bytes)?), Ok(t) => { r.read_unknown(bytes, t)?; } Err(e) => return Err(e), } @@ -176,10 +178,11 @@ impl<'a> MessageRead<'a> for ControlMessage { impl MessageWrite for ControlMessage { fn get_size(&self) -> usize { 0 - + self.ihave.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() - + self.iwant.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() - + self.graft.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() - + self.prune.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() + + self.ihave.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() + + self.iwant.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() + + self.graft.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() + + self.prune.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() + + self.idontwant.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() } fn write_message(&self, w: &mut Writer) -> Result<()> { @@ -187,6 +190,7 @@ impl MessageWrite for ControlMessage { for s in &self.iwant { w.write_with_tag(18, |w| w.write_message(s))?; } for s in &self.graft { w.write_with_tag(26, |w| w.write_message(s))?; } for s in &self.prune { w.write_with_tag(34, |w| w.write_message(s))?; } + for s in &self.idontwant { w.write_with_tag(42, |w| w.write_message(s))?; } Ok(()) } } @@ -216,8 +220,8 @@ impl<'a> MessageRead<'a> for ControlIHave { impl MessageWrite for ControlIHave { fn get_size(&self) -> usize { 0 - + self.topic_id.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) - + self.message_ids.iter().map(|s| 1 + sizeof_len((s).len())).sum::() + + self.topic_id.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + + self.message_ids.iter().map(|s| 1 + sizeof_len((s).len())).sum::() } fn write_message(&self, w: &mut Writer) -> Result<()> { @@ -250,7 +254,7 @@ impl<'a> MessageRead<'a> for ControlIWant { impl MessageWrite for ControlIWant { fn get_size(&self) -> usize { 0 - + self.message_ids.iter().map(|s| 1 + sizeof_len((s).len())).sum::() + + self.message_ids.iter().map(|s| 1 + sizeof_len((s).len())).sum::() } fn write_message(&self, w: &mut Writer) -> Result<()> { @@ -282,7 +286,7 @@ impl<'a> MessageRead<'a> for ControlGraft { impl MessageWrite for ControlGraft { fn get_size(&self) -> usize { 0 - + self.topic_id.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + + self.topic_id.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) } fn write_message(&self, w: &mut Writer) -> Result<()> { @@ -318,9 +322,9 @@ impl<'a> MessageRead<'a> for ControlPrune { impl MessageWrite for ControlPrune { fn get_size(&self) -> usize { 0 - + self.topic_id.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) - + self.peers.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() - + self.backoff.as_ref().map_or(0, |m| 1 + sizeof_varint(*(m) as u64)) + + self.topic_id.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + + self.peers.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() + + self.backoff.as_ref().map_or(0, |m| 1 + sizeof_varint(*(m) as u64)) } fn write_message(&self, w: &mut Writer) -> Result<()> { @@ -331,6 +335,38 @@ impl MessageWrite for ControlPrune { } } +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Debug, Default, PartialEq, Clone)] +pub struct ControlIDontWant { + pub message_ids: Vec>, +} + +impl<'a> MessageRead<'a> for ControlIDontWant { + fn from_reader(r: &mut BytesReader, bytes: &'a [u8]) -> Result { + let mut msg = Self::default(); + while !r.is_eof() { + match r.next_tag(bytes) { + Ok(10) => msg.message_ids.push(r.read_bytes(bytes)?.to_owned()), + Ok(t) => { r.read_unknown(bytes, t)?; } + Err(e) => return Err(e), + } + } + Ok(msg) + } +} + +impl MessageWrite for ControlIDontWant { + fn get_size(&self) -> usize { + 0 + + self.message_ids.iter().map(|s| 1 + sizeof_len((s).len())).sum::() + } + + fn write_message(&self, w: &mut Writer) -> Result<()> { + for s in &self.message_ids { w.write_with_tag(10, |w| w.write_bytes(&**s))?; } + Ok(()) + } +} + #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Debug, Default, PartialEq, Clone)] pub struct PeerInfo { @@ -356,8 +392,8 @@ impl<'a> MessageRead<'a> for PeerInfo { impl MessageWrite for PeerInfo { fn get_size(&self) -> usize { 0 - + self.peer_id.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) - + self.signed_peer_record.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + + self.peer_id.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + + self.signed_peer_record.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) } fn write_message(&self, w: &mut Writer) -> Result<()> { @@ -394,9 +430,9 @@ impl<'a> MessageRead<'a> for TopicDescriptor { impl MessageWrite for TopicDescriptor { fn get_size(&self) -> usize { 0 - + self.name.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) - + self.auth.as_ref().map_or(0, |m| 1 + sizeof_len((m).get_size())) - + self.enc.as_ref().map_or(0, |m| 1 + sizeof_len((m).get_size())) + + self.name.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + + self.auth.as_ref().map_or(0, |m| 1 + sizeof_len((m).get_size())) + + self.enc.as_ref().map_or(0, |m| 1 + sizeof_len((m).get_size())) } fn write_message(&self, w: &mut Writer) -> Result<()> { @@ -409,159 +445,159 @@ impl MessageWrite for TopicDescriptor { pub mod mod_TopicDescriptor { -use super::*; - -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Debug, Default, PartialEq, Clone)] -pub struct AuthOpts { - pub mode: Option, - pub keys: Vec>, -} - -impl<'a> MessageRead<'a> for AuthOpts { - fn from_reader(r: &mut BytesReader, bytes: &'a [u8]) -> Result { - let mut msg = Self::default(); - while !r.is_eof() { - match r.next_tag(bytes) { - Ok(8) => msg.mode = Some(r.read_enum(bytes)?), - Ok(18) => msg.keys.push(r.read_bytes(bytes)?.to_owned()), - Ok(t) => { r.read_unknown(bytes, t)?; } - Err(e) => return Err(e), + use super::*; + + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Debug, Default, PartialEq, Clone)] + pub struct AuthOpts { + pub mode: Option, + pub keys: Vec>, + } + + impl<'a> MessageRead<'a> for AuthOpts { + fn from_reader(r: &mut BytesReader, bytes: &'a [u8]) -> Result { + let mut msg = Self::default(); + while !r.is_eof() { + match r.next_tag(bytes) { + Ok(8) => msg.mode = Some(r.read_enum(bytes)?), + Ok(18) => msg.keys.push(r.read_bytes(bytes)?.to_owned()), + Ok(t) => { r.read_unknown(bytes, t)?; } + Err(e) => return Err(e), + } } + Ok(msg) } - Ok(msg) } -} -impl MessageWrite for AuthOpts { - fn get_size(&self) -> usize { - 0 - + self.mode.as_ref().map_or(0, |m| 1 + sizeof_varint(*(m) as u64)) - + self.keys.iter().map(|s| 1 + sizeof_len((s).len())).sum::() - } + impl MessageWrite for AuthOpts { + fn get_size(&self) -> usize { + 0 + + self.mode.as_ref().map_or(0, |m| 1 + sizeof_varint(*(m) as u64)) + + self.keys.iter().map(|s| 1 + sizeof_len((s).len())).sum::() + } - fn write_message(&self, w: &mut Writer) -> Result<()> { - if let Some(ref s) = self.mode { w.write_with_tag(8, |w| w.write_enum(*s as i32))?; } - for s in &self.keys { w.write_with_tag(18, |w| w.write_bytes(&**s))?; } - Ok(()) + fn write_message(&self, w: &mut Writer) -> Result<()> { + if let Some(ref s) = self.mode { w.write_with_tag(8, |w| w.write_enum(*s as i32))?; } + for s in &self.keys { w.write_with_tag(18, |w| w.write_bytes(&**s))?; } + Ok(()) + } } -} -pub mod mod_AuthOpts { + pub mod mod_AuthOpts { -#[derive(Debug, PartialEq, Eq, Clone, Copy)] -pub enum AuthMode { - NONE = 0, - KEY = 1, - WOT = 2, -} + #[derive(Debug, PartialEq, Eq, Clone, Copy)] + pub enum AuthMode { + NONE = 0, + KEY = 1, + WOT = 2, + } -impl Default for AuthMode { - fn default() -> Self { - AuthMode::NONE - } -} + impl Default for AuthMode { + fn default() -> Self { + AuthMode::NONE + } + } -impl From for AuthMode { - fn from(i: i32) -> Self { - match i { - 0 => AuthMode::NONE, - 1 => AuthMode::KEY, - 2 => AuthMode::WOT, - _ => Self::default(), + impl From for AuthMode { + fn from(i: i32) -> Self { + match i { + 0 => AuthMode::NONE, + 1 => AuthMode::KEY, + 2 => AuthMode::WOT, + _ => Self::default(), + } + } } - } -} -impl<'a> From<&'a str> for AuthMode { - fn from(s: &'a str) -> Self { - match s { - "NONE" => AuthMode::NONE, - "KEY" => AuthMode::KEY, - "WOT" => AuthMode::WOT, - _ => Self::default(), + impl<'a> From<&'a str> for AuthMode { + fn from(s: &'a str) -> Self { + match s { + "NONE" => AuthMode::NONE, + "KEY" => AuthMode::KEY, + "WOT" => AuthMode::WOT, + _ => Self::default(), + } + } } - } -} -} + } -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Debug, Default, PartialEq, Clone)] -pub struct EncOpts { - pub mode: Option, - pub key_hashes: Vec>, -} + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Debug, Default, PartialEq, Clone)] + pub struct EncOpts { + pub mode: Option, + pub key_hashes: Vec>, + } -impl<'a> MessageRead<'a> for EncOpts { - fn from_reader(r: &mut BytesReader, bytes: &'a [u8]) -> Result { - let mut msg = Self::default(); - while !r.is_eof() { - match r.next_tag(bytes) { - Ok(8) => msg.mode = Some(r.read_enum(bytes)?), - Ok(18) => msg.key_hashes.push(r.read_bytes(bytes)?.to_owned()), - Ok(t) => { r.read_unknown(bytes, t)?; } - Err(e) => return Err(e), + impl<'a> MessageRead<'a> for EncOpts { + fn from_reader(r: &mut BytesReader, bytes: &'a [u8]) -> Result { + let mut msg = Self::default(); + while !r.is_eof() { + match r.next_tag(bytes) { + Ok(8) => msg.mode = Some(r.read_enum(bytes)?), + Ok(18) => msg.key_hashes.push(r.read_bytes(bytes)?.to_owned()), + Ok(t) => { r.read_unknown(bytes, t)?; } + Err(e) => return Err(e), + } } + Ok(msg) } - Ok(msg) } -} -impl MessageWrite for EncOpts { - fn get_size(&self) -> usize { - 0 - + self.mode.as_ref().map_or(0, |m| 1 + sizeof_varint(*(m) as u64)) - + self.key_hashes.iter().map(|s| 1 + sizeof_len((s).len())).sum::() - } + impl MessageWrite for EncOpts { + fn get_size(&self) -> usize { + 0 + + self.mode.as_ref().map_or(0, |m| 1 + sizeof_varint(*(m) as u64)) + + self.key_hashes.iter().map(|s| 1 + sizeof_len((s).len())).sum::() + } - fn write_message(&self, w: &mut Writer) -> Result<()> { - if let Some(ref s) = self.mode { w.write_with_tag(8, |w| w.write_enum(*s as i32))?; } - for s in &self.key_hashes { w.write_with_tag(18, |w| w.write_bytes(&**s))?; } - Ok(()) + fn write_message(&self, w: &mut Writer) -> Result<()> { + if let Some(ref s) = self.mode { w.write_with_tag(8, |w| w.write_enum(*s as i32))?; } + for s in &self.key_hashes { w.write_with_tag(18, |w| w.write_bytes(&**s))?; } + Ok(()) + } } -} -pub mod mod_EncOpts { + pub mod mod_EncOpts { -#[derive(Debug, PartialEq, Eq, Clone, Copy)] -pub enum EncMode { - NONE = 0, - SHAREDKEY = 1, - WOT = 2, -} + #[derive(Debug, PartialEq, Eq, Clone, Copy)] + pub enum EncMode { + NONE = 0, + SHAREDKEY = 1, + WOT = 2, + } -impl Default for EncMode { - fn default() -> Self { - EncMode::NONE - } -} + impl Default for EncMode { + fn default() -> Self { + EncMode::NONE + } + } -impl From for EncMode { - fn from(i: i32) -> Self { - match i { - 0 => EncMode::NONE, - 1 => EncMode::SHAREDKEY, - 2 => EncMode::WOT, - _ => Self::default(), + impl From for EncMode { + fn from(i: i32) -> Self { + match i { + 0 => EncMode::NONE, + 1 => EncMode::SHAREDKEY, + 2 => EncMode::WOT, + _ => Self::default(), + } + } } - } -} -impl<'a> From<&'a str> for EncMode { - fn from(s: &'a str) -> Self { - match s { - "NONE" => EncMode::NONE, - "SHAREDKEY" => EncMode::SHAREDKEY, - "WOT" => EncMode::WOT, - _ => Self::default(), + impl<'a> From<&'a str> for EncMode { + fn from(s: &'a str) -> Self { + match s { + "NONE" => EncMode::NONE, + "SHAREDKEY" => EncMode::SHAREDKEY, + "WOT" => EncMode::WOT, + _ => Self::default(), + } + } } - } -} -} + } } diff --git a/protocols/gossipsub/src/generated/rpc.proto b/protocols/gossipsub/src/generated/rpc.proto index 2ce12f3f37f..fe4d3bc9366 100644 --- a/protocols/gossipsub/src/generated/rpc.proto +++ b/protocols/gossipsub/src/generated/rpc.proto @@ -19,8 +19,8 @@ message Message { optional bytes data = 2; optional bytes seqno = 3; required string topic = 4; - optional bytes signature = 5; - optional bytes key = 6; + optional bytes signature = 5; + optional bytes key = 6; } message ControlMessage { @@ -28,6 +28,7 @@ message ControlMessage { repeated ControlIWant iwant = 2; repeated ControlGraft graft = 3; repeated ControlPrune prune = 4; + repeated ControlIDontWant idontwant = 5; } message ControlIHave { @@ -49,6 +50,10 @@ message ControlPrune { optional uint64 backoff = 3; // gossipsub v1.1 backoff time (in seconds) } +message ControlIDontWant { + repeated bytes message_ids = 1; +} + message PeerInfo { optional bytes peer_id = 1; optional bytes signed_peer_record = 2; diff --git a/protocols/gossipsub/src/gossip_promises.rs b/protocols/gossipsub/src/gossip_promises.rs index bdf58b74fc2..34d27793719 100644 --- a/protocols/gossipsub/src/gossip_promises.rs +++ b/protocols/gossipsub/src/gossip_promises.rs @@ -41,6 +41,14 @@ impl GossipPromises { self.promises.contains_key(message) } + ///Get the peers we sent IWANT the input message id. + pub(crate) fn peers_for_message(&self, message_id: &MessageId) -> Vec { + self.promises + .get(message_id) + .map(|peers| peers.keys().copied().collect()) + .unwrap_or_default() + } + /// Track a promise to deliver a message from a list of [`MessageId`]s we are requesting. pub(crate) fn add_promise(&mut self, peer: PeerId, messages: &[MessageId], expires: Instant) { for message_id in messages { diff --git a/protocols/gossipsub/src/metrics.rs b/protocols/gossipsub/src/metrics.rs index 40af1af2cac..6d99e132cb4 100644 --- a/protocols/gossipsub/src/metrics.rs +++ b/protocols/gossipsub/src/metrics.rs @@ -181,6 +181,12 @@ pub(crate) struct Metrics { /// topic. A very high metric might indicate an underperforming network. topic_iwant_msgs: Family, + /// The number of times we have received an IDONTWANT control message. + idontwant_msgs: Counter, + + /// The number of msg_id's we have received in every IDONTWANT control message. + idontwant_msgs_ids: Counter, + /// The size of the priority queue. priority_queue_size: Histogram, /// The size of the non-priority queue. @@ -318,6 +324,27 @@ impl Metrics { "topic_iwant_msgs", "Number of times we have decided an IWANT is required for this topic" ); + + let idontwant_msgs = { + let metric = Counter::default(); + registry.register( + "idontwant_msgs", + "The number of times we have received an IDONTWANT control message", + metric.clone(), + ); + metric + }; + + let idontwant_msgs_ids = { + let metric = Counter::default(); + registry.register( + "idontwant_msgs_ids", + "The number of msg_id's we have received in every IDONTWANT control message.", + metric.clone(), + ); + metric + }; + let memcache_misses = { let metric = Counter::default(); registry.register( @@ -370,6 +397,8 @@ impl Metrics { heartbeat_duration, memcache_misses, topic_iwant_msgs, + idontwant_msgs, + idontwant_msgs_ids, priority_queue_size, non_priority_queue_size, } @@ -568,6 +597,12 @@ impl Metrics { } } + /// Register receiving an IDONTWANT msg for this topic. + pub(crate) fn register_idontwant(&mut self, msgs: usize) { + self.idontwant_msgs.inc(); + self.idontwant_msgs_ids.inc_by(msgs as u64); + } + /// Observes a heartbeat duration. pub(crate) fn observe_heartbeat_duration(&mut self, millis: u64) { self.heartbeat_duration.observe(millis as f64); diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index 8d33fe51a90..1b746fafe0a 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -22,10 +22,7 @@ use crate::config::ValidationMode; use crate::handler::HandlerEvent; use crate::rpc_proto::proto; use crate::topic::TopicHash; -use crate::types::{ - ControlAction, Graft, IHave, IWant, MessageId, PeerInfo, PeerKind, Prune, RawMessage, Rpc, - Subscription, SubscriptionAction, -}; +use crate::types::{ControlAction, Graft, IDontWant, IHave, IWant, MessageId, PeerInfo, PeerKind, Prune, RawMessage, Rpc, Subscription, SubscriptionAction}; use crate::ValidationError; use asynchronous_codec::{Decoder, Encoder, Framed}; use byteorder::{BigEndian, ByteOrder}; @@ -40,6 +37,11 @@ use std::pin::Pin; pub(crate) const SIGNING_PREFIX: &[u8] = b"libp2p-pubsub:"; +pub(crate) const GOSSIPSUB_1_2_0_BETA_PROTOCOL: ProtocolId = ProtocolId { + protocol: StreamProtocol::new("/meshsub/1.2.0"), + kind: PeerKind::Gossipsubv1_2_beta, +}; + pub(crate) const GOSSIPSUB_1_1_0_PROTOCOL: ProtocolId = ProtocolId { protocol: StreamProtocol::new("/meshsub/1.1.0"), kind: PeerKind::Gossipsubv1_1, @@ -476,10 +478,25 @@ impl Decoder for GossipsubCodec { })); } + let idontwant_msgs: Vec = rpc_control + .idontwant + .into_iter() + .map(|idontwant| { + ControlAction::IDontWant(IDontWant { + message_ids: idontwant + .message_ids + .into_iter() + .map(MessageId::from) + .collect::>(), + }) + }) + .collect(); + control_msgs.extend(ihave_msgs); control_msgs.extend(iwant_msgs); control_msgs.extend(graft_msgs); control_msgs.extend(prune_msgs); + control_msgs.extend(idontwant_msgs); } Ok(Some(HandlerEvent::Message { diff --git a/protocols/gossipsub/src/rpc.rs b/protocols/gossipsub/src/rpc.rs index c90e46a85da..2fdee7ce266 100644 --- a/protocols/gossipsub/src/rpc.rs +++ b/protocols/gossipsub/src/rpc.rs @@ -44,7 +44,7 @@ pub(crate) struct Sender { } impl Sender { - /// Create a RpcSender. + /// Create a RpcSende pub(crate) fn new(cap: usize) -> Sender { // We intentionally do not bound the channel, as we still need to send control messages // such as `GRAFT`, `PRUNE`, `SUBSCRIBE`, and `UNSUBSCRIBE`. @@ -88,7 +88,7 @@ impl Sender { | RpcOut::Prune(_) | RpcOut::Subscribe(_) | RpcOut::Unsubscribe(_) => &self.priority_sender, - RpcOut::Forward { .. } | RpcOut::IHave(_) | RpcOut::IWant(_) => { + RpcOut::Forward { .. } | RpcOut::IHave(_) | RpcOut::IWant(_) | RpcOut::IDontWant(_)=> { &self.non_priority_sender } }; diff --git a/protocols/gossipsub/src/types.rs b/protocols/gossipsub/src/types.rs index bb1916fefd0..7fe412b9aca 100644 --- a/protocols/gossipsub/src/types.rs +++ b/protocols/gossipsub/src/types.rs @@ -28,7 +28,8 @@ use prometheus_client::encoding::EncodeLabelValue; use quick_protobuf::MessageWrite; use std::fmt::Debug; use std::{collections::BTreeSet, fmt}; - +use std::time::Instant; +use hashlink::LinkedHashMap; use crate::rpc_proto::proto; #[cfg(feature = "serde")] use serde::{Deserialize, Serialize}; @@ -110,11 +111,16 @@ pub(crate) struct PeerConnections { pub(crate) topics: BTreeSet, /// The rpc sender to the connection handler(s). pub(crate) sender: Sender, + /// Don't send messages. + pub(crate) dont_send: LinkedHashMap, } /// Describes the types of peers that can exist in the gossipsub context. #[derive(Debug, Clone, PartialEq, Hash, EncodeLabelValue, Eq)] +#[allow(non_camel_case_types)] pub enum PeerKind { + /// A gossipsub 1.2 peer. + Gossipsubv1_2_beta, /// A gossipsub 1.1 peer. Gossipsubv1_1, /// A gossipsub 1.0 peer. @@ -150,6 +156,16 @@ pub struct RawMessage { pub validated: bool, } +impl PeerKind { + /// Returns true if peer speaks any gossipsub version. + pub(crate) fn is_gossipsub(&self) -> bool { + matches!( + self, + Self::Gossipsubv1_2_beta | Self::Gossipsubv1_1 | Self::Gossipsub + ) + } +} + impl RawMessage { /// Calculates the encoded length of this message (used for calculating metrics). pub fn raw_protobuf_len(&self) -> usize { @@ -246,6 +262,8 @@ pub enum ControlAction { Graft(Graft), /// The node has been removed from the mesh - Prune control message. Prune(Prune), + /// The node requests us to not forward message ids (peer_id + sequence _number) - IDontWant control message. + IDontWant(IDontWant), } /// Node broadcasts known messages per topic - IHave control message. @@ -282,6 +300,13 @@ pub struct Prune { pub(crate) backoff: Option, } +/// The node requests us to not forward message ids - IDontWant control message. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct IDontWant { + /// A list of known message ids. + pub(crate) message_ids: Vec, +} + /// A Gossipsub RPC message sent. #[derive(Debug)] pub enum RpcOut { @@ -303,6 +328,8 @@ pub enum RpcOut { IHave(IHave), /// Send a IWant control message. IWant(IWant), + /// The node requests us to not forward message ids (peer_id + sequence _number) - IDontWant control message. + IDontWant(IDontWant), } impl RpcOut { @@ -363,6 +390,7 @@ impl From for proto::RPC { iwant: vec![], graft: vec![], prune: vec![], + idontwant:vec![] }), }, RpcOut::IWant(IWant { message_ids }) => proto::RPC { @@ -375,6 +403,7 @@ impl From for proto::RPC { }], graft: vec![], prune: vec![], + idontwant:vec![] }), }, RpcOut::Graft(Graft { topic_hash }) => proto::RPC { @@ -387,6 +416,7 @@ impl From for proto::RPC { topic_id: Some(topic_hash.into_string()), }], prune: vec![], + idontwant:vec![] }), }, RpcOut::Prune(Prune { @@ -413,9 +443,23 @@ impl From for proto::RPC { .collect(), backoff, }], + idontwant:vec![] }), } } + RpcOut::IDontWant(IDontWant { message_ids }) => proto::RPC { + publish: Vec::new(), + subscriptions: Vec::new(), + control: Some(proto::ControlMessage { + ihave: vec![], + iwant: vec![], + graft: vec![], + prune: vec![], + idontwant: vec![proto::ControlIDontWant { + message_ids: message_ids.into_iter().map(|msg_id| msg_id.0).collect(), + }], + }), + }, } } } @@ -474,6 +518,8 @@ impl From for proto::RPC { iwant: Vec::new(), graft: Vec::new(), prune: Vec::new(), + idontwant: Vec::new(), + }; let empty_control_msg = rpc.control_msgs.is_empty(); @@ -522,6 +568,12 @@ impl From for proto::RPC { }; control.prune.push(rpc_prune); } + ControlAction::IDontWant(IDontWant { message_ids }) => { + let rpc_idontwant = proto::ControlIDontWant { + message_ids: message_ids.into_iter().map(|msg_id| msg_id.0).collect(), + }; + control.idontwant.push(rpc_idontwant); + } } } @@ -560,6 +612,7 @@ impl PeerKind { Self::Floodsub => "Floodsub", Self::Gossipsub => "Gossipsub v1.0", Self::Gossipsubv1_1 => "Gossipsub v1.1", + Self::Gossipsubv1_2_beta => "Gossipsub v1.2-beta", } } } From 2ecf7f2b442d703582b3d9ca0c916c3d4756926f Mon Sep 17 00:00:00 2001 From: manasnagaraj Date: Fri, 29 Nov 2024 14:33:04 -0500 Subject: [PATCH 02/16] fixing clippy issues --- protocols/gossipsub/src/behaviour.rs | 14 +++++++------- protocols/gossipsub/src/rpc.rs | 2 +- protocols/gossipsub/src/types.rs | 11 +++++------ 3 files changed, 13 insertions(+), 14 deletions(-) diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index 591564356af..5d3f0efe295 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -28,8 +28,10 @@ use std::{ time::Duration, }; +#[allow(unused_imports)] use futures::{FutureExt, StreamExt}; use futures_timer::Delay; +use hashlink::LinkedHashMap; use libp2p_core::{ multiaddr::Protocol::{Ip4, Ip6}, transport::PortUse, @@ -46,7 +48,6 @@ use prometheus_client::registry::Registry; use quick_protobuf::{MessageWrite, Writer}; use rand::{seq::SliceRandom, thread_rng}; use web_time::{Instant, SystemTime}; -use hashlink::LinkedHashMap; use crate::{ backoff::BackoffStorage, @@ -64,13 +65,13 @@ use crate::{ topic::{Hasher, Topic, TopicHash}, transform::{DataTransform, IdentityTransform}, types::{ - ControlAction, Graft, IHave, IWant, IDontWant, Message, MessageAcceptance, MessageId, PeerConnections, - PeerInfo, PeerKind, Prune, RawMessage, RpcOut, Subscription, SubscriptionAction, + ControlAction, Graft, IDontWant, IHave, IWant, Message, MessageAcceptance, MessageId, + PeerConnections, PeerInfo, PeerKind, Prune, RawMessage, RpcOut, Subscription, + SubscriptionAction, }, FailedMessages, PublishError, SubscriptionError, TopicScoreParams, ValidationError, }; - #[cfg(test)] mod tests; @@ -2583,7 +2584,6 @@ where } } - /// Helper function which sends an IDONTWANT message to mesh\[topic\] peers. fn send_idontwant( &mut self, @@ -2705,7 +2705,7 @@ where .sender .send_message(RpcOut::Forward { message: message.clone(), - timeout: Delay::new(self.config.forward_queue_duration()) + timeout: Delay::new(self.config.forward_queue_duration()), }) .is_err() { @@ -3330,7 +3330,7 @@ where // update scores if let Some((peer_score, _, delay)) = &mut self.peer_score { - if delay.poll_unpin(cx).is_ready(){ + if delay.poll_unpin(cx).is_ready() { peer_score.refresh_scores(); } } diff --git a/protocols/gossipsub/src/rpc.rs b/protocols/gossipsub/src/rpc.rs index f842bbd4a5f..f9479a7b94d 100644 --- a/protocols/gossipsub/src/rpc.rs +++ b/protocols/gossipsub/src/rpc.rs @@ -89,7 +89,7 @@ impl Sender { | RpcOut::Prune(_) | RpcOut::Subscribe(_) | RpcOut::Unsubscribe(_) => &self.priority_sender, - RpcOut::Forward { .. } | RpcOut::IHave(_) | RpcOut::IWant(_) | RpcOut::IDontWant(_)=> { + RpcOut::Forward { .. } | RpcOut::IHave(_) | RpcOut::IWant(_) | RpcOut::IDontWant(_) => { &self.non_priority_sender } }; diff --git a/protocols/gossipsub/src/types.rs b/protocols/gossipsub/src/types.rs index 874f42199cd..00fffc31992 100644 --- a/protocols/gossipsub/src/types.rs +++ b/protocols/gossipsub/src/types.rs @@ -22,6 +22,7 @@ use std::{collections::BTreeSet, fmt, fmt::Debug}; use futures_timer::Delay; +use hashlink::LinkedHashMap; use libp2p_identity::PeerId; use libp2p_swarm::ConnectionId; use prometheus_client::encoding::EncodeLabelValue; @@ -29,7 +30,6 @@ use quick_protobuf::MessageWrite; #[cfg(feature = "serde")] use serde::{Deserialize, Serialize}; use std::time::Instant; -use hashlink::LinkedHashMap; use crate::{rpc::Sender, rpc_proto::proto, TopicHash}; @@ -391,7 +391,7 @@ impl From for proto::RPC { iwant: vec![], graft: vec![], prune: vec![], - idontwant:vec![] + idontwant: vec![], }), }, RpcOut::IWant(IWant { message_ids }) => proto::RPC { @@ -404,7 +404,7 @@ impl From for proto::RPC { }], graft: vec![], prune: vec![], - idontwant:vec![] + idontwant: vec![], }), }, RpcOut::Graft(Graft { topic_hash }) => proto::RPC { @@ -417,7 +417,7 @@ impl From for proto::RPC { topic_id: Some(topic_hash.into_string()), }], prune: vec![], - idontwant:vec![] + idontwant: vec![], }), }, RpcOut::Prune(Prune { @@ -444,7 +444,7 @@ impl From for proto::RPC { .collect(), backoff, }], - idontwant:vec![] + idontwant: vec![], }), } } @@ -520,7 +520,6 @@ impl From for proto::RPC { graft: Vec::new(), prune: Vec::new(), idontwant: Vec::new(), - }; let empty_control_msg = rpc.control_msgs.is_empty(); From ac1f5f8a5f5836888a6a82fa613d480f3b3196e1 Mon Sep 17 00:00:00 2001 From: manasnagaraj Date: Fri, 29 Nov 2024 20:55:50 -0500 Subject: [PATCH 03/16] removing unintended #[allow(unused_import)] --- protocols/gossipsub/src/behaviour.rs | 3 +-- protocols/gossipsub/src/behaviour/tests.rs | 1 + 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index 5d3f0efe295..a14e6aaf5c3 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -28,8 +28,7 @@ use std::{ time::Duration, }; -#[allow(unused_imports)] -use futures::{FutureExt, StreamExt}; +use futures::FutureExt; use futures_timer::Delay; use hashlink::LinkedHashMap; use libp2p_core::{ diff --git a/protocols/gossipsub/src/behaviour/tests.rs b/protocols/gossipsub/src/behaviour/tests.rs index 34e5186b3bb..145ac1138b6 100644 --- a/protocols/gossipsub/src/behaviour/tests.rs +++ b/protocols/gossipsub/src/behaviour/tests.rs @@ -22,6 +22,7 @@ use std::{future, net::Ipv4Addr, thread::sleep}; +use futures::StreamExt; use byteorder::{BigEndian, ByteOrder}; use libp2p_core::ConnectedPoint; use rand::Rng; From 4d51013e85349348467ca277f68129614066e7d3 Mon Sep 17 00:00:00 2001 From: manasnagaraj Date: Sun, 1 Dec 2024 17:43:13 -0500 Subject: [PATCH 04/16] optimising send_idontwant and foward_msg --- protocols/gossipsub/src/behaviour.rs | 83 +++++++--------------- protocols/gossipsub/src/gossip_promises.rs | 2 +- protocols/gossipsub/src/metrics.rs | 2 +- protocols/gossipsub/src/protocol.rs | 25 ++++++- protocols/gossipsub/src/rpc.rs | 2 +- protocols/gossipsub/src/types.rs | 6 +- 6 files changed, 55 insertions(+), 65 deletions(-) diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index a14e6aaf5c3..0f129d27d10 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -1658,7 +1658,6 @@ where &raw_message.topic, RejectReason::BlackListedPeer, ); - // gossip_promises.reject_message(msg_id, &RejectReason::BlackListedPeer); } return false; } @@ -2596,43 +2595,30 @@ where let iwant_peers = self.gossip_promises.peers_for_message(msg_id); - let recipient_peers = mesh_peers + let recipient_peers: Vec= mesh_peers .iter() .chain(iwant_peers.iter()) .filter(|peer_id| { *peer_id != propagation_source && Some(*peer_id) != message.source.as_ref() - }); + }) + .cloned() + .collect(); for peer_id in recipient_peers { - let Some(peer) = self.connected_peers.get_mut(peer_id) else { + let Some(peer) = self.connected_peers.get_mut(&peer_id) else { tracing::error!(peer = %peer_id, "Could not IDONTWANT, peer doesn't exist in connected peer list"); continue; }; // Only gossipsub 1.2 peers support IDONTWANT. - if peer.kind != PeerKind::Gossipsubv1_2_beta { + if peer.kind != PeerKind::Gossipsubv1_2 { continue; } - if peer - .sender - .send_message(RpcOut::IDontWant(IDontWant { - message_ids: vec![msg_id.clone()], - })) - .is_err() - { - tracing::warn!(peer=%peer_id, "Send Queue full. Could not send IDONTWANT"); - - 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; - } + self.send_message(peer_id, RpcOut::IDontWant(IDontWant{ + message_ids: vec![msg_id.clone()], + })); } } @@ -2691,43 +2677,26 @@ where } // forward the message to peers - if !recipient_peers.is_empty() { - for peer_id in recipient_peers.iter() { - if let Some(peer) = self.connected_peers.get_mut(peer_id) { - if peer.dont_send.get(msg_id).is_some() { - tracing::debug!(%peer_id, message=%msg_id, "Peer doesn't want message"); - continue; - } - - tracing::debug!(%peer_id, message=%msg_id, "Sending message to peer"); - if peer - .sender - .send_message(RpcOut::Forward { - message: message.clone(), - timeout: Delay::new(self.config.forward_queue_duration()), - }) - .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"); + for peer_id in recipient_peers.iter() { + if let Some(peer) = self.connected_peers.get_mut(peer_id) { + if peer.dont_send.get(msg_id).is_some() { + tracing::debug!(%peer_id, message=%msg_id, "Peer doesn't want message"); + continue; } + + tracing::debug!(%peer_id, message=%msg_id, "Sending message to peer"); + + self.send_message(*peer_id,RpcOut::Forward { + message: message.clone(), + timeout: Delay::new(self.config.forward_queue_duration()), + }); + } else { + tracing::error!(peer = %peer_id, + "Could not FORWARD, peer doesn't exist in connected peer list"); } - tracing::debug!("Completed forwarding message"); - true - } else { - false } + tracing::debug!("Completed forwarding message"); + true } /// Constructs a [`RawMessage`] performing message signing if required. diff --git a/protocols/gossipsub/src/gossip_promises.rs b/protocols/gossipsub/src/gossip_promises.rs index 194cd1db0f5..284ba7cab01 100644 --- a/protocols/gossipsub/src/gossip_promises.rs +++ b/protocols/gossipsub/src/gossip_promises.rs @@ -41,7 +41,7 @@ impl GossipPromises { self.promises.contains_key(message) } - ///Get the peers we sent IWANT the input message id. + /// Get the peers we sent IWANT the input message id. pub(crate) fn peers_for_message(&self, message_id: &MessageId) -> Vec { self.promises .get(message_id) diff --git a/protocols/gossipsub/src/metrics.rs b/protocols/gossipsub/src/metrics.rs index b3a867a394f..42dedc000b7 100644 --- a/protocols/gossipsub/src/metrics.rs +++ b/protocols/gossipsub/src/metrics.rs @@ -345,7 +345,7 @@ impl Metrics { let metric = Counter::default(); registry.register( "idontwant_msgs_ids", - "The number of msg_id's we have received in every IDONTWANT control message.", + "The number of msg_id's we have received in every total of all IDONTWANT control message.", metric.clone(), ); metric diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index e4272737342..87dfcdc09f1 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -35,14 +35,20 @@ use crate::{ rpc_proto::proto, topic::TopicHash, types::{ - ControlAction, Graft, IHave, IWant, MessageId, PeerInfo, PeerKind, Prune, RawMessage, Rpc, + ControlAction, Graft, IDontWant, IHave, IWant, MessageId, PeerInfo, PeerKind, Prune, RawMessage, Rpc, Subscription, SubscriptionAction, }, ValidationError, }; + pub(crate) const SIGNING_PREFIX: &[u8] = b"libp2p-pubsub:"; +pub(crate) const GOSSIPSUB_1_2_0_PROTOCOL: ProtocolId = ProtocolId { + protocol: StreamProtocol::new("/meshsub/1.2.0"), + kind: PeerKind::Gossipsubv1_2, +}; + pub(crate) const GOSSIPSUB_1_1_0_PROTOCOL: ProtocolId = ProtocolId { protocol: StreamProtocol::new("/meshsub/1.1.0"), kind: PeerKind::Gossipsubv1_1, @@ -72,7 +78,7 @@ impl Default for ProtocolConfig { Self { max_transmit_size: 65536, validation_mode: ValidationMode::Strict, - protocol_ids: vec![GOSSIPSUB_1_1_0_PROTOCOL, GOSSIPSUB_1_0_0_PROTOCOL], + protocol_ids: vec![GOSSIPSUB_1_2_0_PROTOCOL, GOSSIPSUB_1_1_0_PROTOCOL, GOSSIPSUB_1_0_0_PROTOCOL], } } } @@ -479,10 +485,25 @@ impl Decoder for GossipsubCodec { })); } + let idontwant_msgs: Vec = rpc_control + .idontwant + .into_iter() + .map(|idontwant| { + ControlAction::IDontWant(IDontWant { + message_ids: idontwant + .message_ids + .into_iter() + .map(MessageId::from) + .collect::>(), + }) + }) + .collect(); + control_msgs.extend(ihave_msgs); control_msgs.extend(iwant_msgs); control_msgs.extend(graft_msgs); control_msgs.extend(prune_msgs); + control_msgs.extend(idontwant_msgs); } Ok(Some(HandlerEvent::Message { diff --git a/protocols/gossipsub/src/rpc.rs b/protocols/gossipsub/src/rpc.rs index f9479a7b94d..41b338267e9 100644 --- a/protocols/gossipsub/src/rpc.rs +++ b/protocols/gossipsub/src/rpc.rs @@ -45,7 +45,7 @@ pub(crate) struct Sender { } impl Sender { - /// Create a RpcSende + /// Create a RpcSender. pub(crate) fn new(cap: usize) -> Sender { // We intentionally do not bound the channel, as we still need to send control messages // such as `GRAFT`, `PRUNE`, `SUBSCRIBE`, and `UNSUBSCRIBE`. diff --git a/protocols/gossipsub/src/types.rs b/protocols/gossipsub/src/types.rs index 00fffc31992..ccf68e86888 100644 --- a/protocols/gossipsub/src/types.rs +++ b/protocols/gossipsub/src/types.rs @@ -120,7 +120,7 @@ pub(crate) struct PeerConnections { #[allow(non_camel_case_types)] pub enum PeerKind { /// A gossipsub 1.2 peer. - Gossipsubv1_2_beta, + Gossipsubv1_2, /// A gossipsub 1.1 peer. Gossipsubv1_1, /// A gossipsub 1.0 peer. @@ -161,7 +161,7 @@ impl PeerKind { pub(crate) fn is_gossipsub(&self) -> bool { matches!( self, - Self::Gossipsubv1_2_beta | Self::Gossipsubv1_1 | Self::Gossipsub + Self::Gossipsubv1_2 | Self::Gossipsubv1_1 | Self::Gossipsub ) } } @@ -612,7 +612,7 @@ impl PeerKind { Self::Floodsub => "Floodsub", Self::Gossipsub => "Gossipsub v1.0", Self::Gossipsubv1_1 => "Gossipsub v1.1", - Self::Gossipsubv1_2_beta => "Gossipsub v1.2-beta", + Self::Gossipsubv1_2 => "Gossipsub v1.2", } } } From 434e315a298f24026f49a86fe55a2a68e4304283 Mon Sep 17 00:00:00 2001 From: manasnagaraj Date: Sun, 1 Dec 2024 18:19:28 -0500 Subject: [PATCH 05/16] minor fixes --- protocols/gossipsub/src/behaviour.rs | 22 ++++++++++++++-------- protocols/gossipsub/src/behaviour/tests.rs | 9 ++++----- protocols/gossipsub/src/protocol.rs | 11 +++++++---- 3 files changed, 25 insertions(+), 17 deletions(-) diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index 0f129d27d10..7b2c5cd59e7 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -2595,7 +2595,7 @@ where let iwant_peers = self.gossip_promises.peers_for_message(msg_id); - let recipient_peers: Vec= mesh_peers + let recipient_peers: Vec = mesh_peers .iter() .chain(iwant_peers.iter()) .filter(|peer_id| { @@ -2616,9 +2616,12 @@ where continue; } - self.send_message(peer_id, RpcOut::IDontWant(IDontWant{ - message_ids: vec![msg_id.clone()], - })); + self.send_message( + peer_id, + RpcOut::IDontWant(IDontWant { + message_ids: vec![msg_id.clone()], + }), + ); } } @@ -2686,10 +2689,13 @@ where tracing::debug!(%peer_id, message=%msg_id, "Sending message to peer"); - self.send_message(*peer_id,RpcOut::Forward { - message: message.clone(), - timeout: Delay::new(self.config.forward_queue_duration()), - }); + self.send_message( + *peer_id, + RpcOut::Forward { + message: message.clone(), + timeout: Delay::new(self.config.forward_queue_duration()), + }, + ); } else { tracing::error!(peer = %peer_id, "Could not FORWARD, peer doesn't exist in connected peer list"); diff --git a/protocols/gossipsub/src/behaviour/tests.rs b/protocols/gossipsub/src/behaviour/tests.rs index 145ac1138b6..54e12e7490b 100644 --- a/protocols/gossipsub/src/behaviour/tests.rs +++ b/protocols/gossipsub/src/behaviour/tests.rs @@ -22,7 +22,6 @@ use std::{future, net::Ipv4Addr, thread::sleep}; -use futures::StreamExt; use byteorder::{BigEndian, ByteOrder}; use libp2p_core::ConnectedPoint; use rand::Rng; @@ -5285,7 +5284,7 @@ fn sends_idontwant() { .to_subscribe(true) .gs_config(Config::default()) .explicit(1) - .peer_kind(PeerKind::Gossipsubv1_2_beta) + .peer_kind(PeerKind::Gossipsubv1_2) .create_network(); let local_id = PeerId::random(); @@ -5370,7 +5369,7 @@ fn doesnt_forward_idontwant() { .to_subscribe(true) .gs_config(Config::default()) .explicit(1) - .peer_kind(PeerKind::Gossipsubv1_2_beta) + .peer_kind(PeerKind::Gossipsubv1_2) .create_network(); let local_id = PeerId::random(); @@ -5419,7 +5418,7 @@ fn parses_idontwant() { .to_subscribe(true) .gs_config(Config::default()) .explicit(1) - .peer_kind(PeerKind::Gossipsubv1_2_beta) + .peer_kind(PeerKind::Gossipsubv1_2) .create_network(); let message_id = MessageId::new(&[0, 1, 2, 3]); @@ -5451,7 +5450,7 @@ fn clear_stale_idontwant() { .to_subscribe(true) .gs_config(Config::default()) .explicit(1) - .peer_kind(PeerKind::Gossipsubv1_2_beta) + .peer_kind(PeerKind::Gossipsubv1_2) .create_network(); let peer = gs.connected_peers.get_mut(&peers[2]).unwrap(); diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index 87dfcdc09f1..7ee6d5c8245 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -35,13 +35,12 @@ use crate::{ rpc_proto::proto, topic::TopicHash, types::{ - ControlAction, Graft, IDontWant, IHave, IWant, MessageId, PeerInfo, PeerKind, Prune, RawMessage, Rpc, - Subscription, SubscriptionAction, + ControlAction, Graft, IDontWant, IHave, IWant, MessageId, PeerInfo, PeerKind, Prune, + RawMessage, Rpc, Subscription, SubscriptionAction, }, ValidationError, }; - pub(crate) const SIGNING_PREFIX: &[u8] = b"libp2p-pubsub:"; pub(crate) const GOSSIPSUB_1_2_0_PROTOCOL: ProtocolId = ProtocolId { @@ -78,7 +77,11 @@ impl Default for ProtocolConfig { Self { max_transmit_size: 65536, validation_mode: ValidationMode::Strict, - protocol_ids: vec![GOSSIPSUB_1_2_0_PROTOCOL, GOSSIPSUB_1_1_0_PROTOCOL, GOSSIPSUB_1_0_0_PROTOCOL], + protocol_ids: vec![ + GOSSIPSUB_1_2_0_PROTOCOL, + GOSSIPSUB_1_1_0_PROTOCOL, + GOSSIPSUB_1_0_0_PROTOCOL, + ], } } } From 37daf06b237a6e5d738bd3dcab96df6478f8cfa4 Mon Sep 17 00:00:00 2001 From: manasnagaraj Date: Wed, 25 Dec 2024 13:41:33 -0500 Subject: [PATCH 06/16] merge in master --- .github/ISSUE_TEMPLATE/bug_report.yml | 8 +- .github/ISSUE_TEMPLATE/enhancement.yml | 4 +- .github/ISSUE_TEMPLATE/feature_request.yml | 4 +- .github/pull_request_template.md | 4 +- .github/workflows/ci.yml | 4 +- .idea/vcs.xml | 6 + .idea/workspace.xml | 535 ++++++++++++++++++ Cargo.lock | 146 ++--- Cargo.toml | 2 + core/Cargo.toml | 5 +- examples/autonat/Cargo.toml | 1 - examples/autonatv2/Cargo.toml | 4 +- examples/chat/Cargo.toml | 2 - examples/dcutr/Cargo.toml | 1 - .../distributed-key-value-store/Cargo.toml | 2 - examples/file-sharing/Cargo.toml | 1 - examples/identify/Cargo.toml | 1 - examples/ipfs-kad/Cargo.toml | 3 - examples/ipfs-private/Cargo.toml | 2 - examples/ping/Cargo.toml | 1 - examples/relay-server/Cargo.toml | 1 - hole-punching-tests/Cargo.toml | 2 +- identity/Cargo.toml | 1 - interop-tests/Cargo.toml | 1 - libp2p/Cargo.toml | 4 +- misc/keygen/Cargo.toml | 1 - misc/memory-connection-limits/Cargo.toml | 2 - misc/multistream-select/Cargo.toml | 1 - misc/server/Cargo.toml | 4 +- misc/server/src/config.rs | 2 +- misc/webrtc-utils/Cargo.toml | 1 - protocols/autonat/Cargo.toml | 3 +- protocols/dcutr/Cargo.toml | 5 - protocols/gossipsub/Cargo.toml | 8 +- protocols/gossipsub/src/behaviour.rs | 7 +- protocols/gossipsub/src/behaviour/tests.rs | 21 +- protocols/gossipsub/src/types.rs | 9 +- protocols/identify/Cargo.toml | 1 - protocols/kad/Cargo.toml | 1 - protocols/mdns/CHANGELOG.md | 2 + protocols/mdns/Cargo.toml | 4 - protocols/mdns/src/behaviour.rs | 178 +++--- protocols/perf/Cargo.toml | 3 - protocols/ping/Cargo.toml | 2 - protocols/rendezvous/Cargo.toml | 5 - protocols/request-response/Cargo.toml | 5 - swarm-derive/Cargo.toml | 1 - swarm-test/Cargo.toml | 1 - swarm/Cargo.toml | 2 +- transports/noise/Cargo.toml | 2 - transports/plaintext/Cargo.toml | 1 - transports/quic/Cargo.toml | 4 +- transports/tcp/Cargo.toml | 2 - transports/tls/Cargo.toml | 1 - transports/webrtc-websys/Cargo.toml | 2 +- transports/webrtc/Cargo.toml | 3 - transports/websocket-websys/Cargo.toml | 1 - wasm-tests/webtransport-tests/Cargo.toml | 2 +- 58 files changed, 741 insertions(+), 291 deletions(-) create mode 100644 .idea/vcs.xml create mode 100644 .idea/workspace.xml diff --git a/.github/ISSUE_TEMPLATE/bug_report.yml b/.github/ISSUE_TEMPLATE/bug_report.yml index 1a531e3646c..40131a47a17 100644 --- a/.github/ISSUE_TEMPLATE/bug_report.yml +++ b/.github/ISSUE_TEMPLATE/bug_report.yml @@ -9,7 +9,7 @@ body: - type: textarea attributes: label: Summary - description: Please provide a short summary of the bug, along with any information you feel relevant to replicate the bug. + description: Please provide a short summary of the bug, along with any information you feel is relevant to replicate the bug. validations: required: true - type: textarea @@ -34,7 +34,7 @@ body: - type: textarea attributes: label: Possible Solution - description: Suggest a fix/reason for the bug, or ideas how to implement the addition or change. + description: Suggest a fix/reason for the bug, or ideas on how to implement the addition or change. validations: required: false - type: textarea @@ -45,11 +45,11 @@ body: required: false - type: dropdown attributes: - label: Would you like to work on fixing this bug ? + label: Would you like to work on fixing this bug? description: Any contribution towards fixing the bug is greatly appreciated. We are more than happy to provide help on the process. options: - "Yes" - "No" - Maybe validations: - required: true \ No newline at end of file + required: true diff --git a/.github/ISSUE_TEMPLATE/enhancement.yml b/.github/ISSUE_TEMPLATE/enhancement.yml index ed7aeb644b3..05330cf071c 100644 --- a/.github/ISSUE_TEMPLATE/enhancement.yml +++ b/.github/ISSUE_TEMPLATE/enhancement.yml @@ -21,11 +21,11 @@ body: required: true - type: dropdown attributes: - label: Are you planning to do it yourself in a pull request ? + label: Are you planning to do it yourself in a pull request? description: Any contribution is greatly appreciated. We are more than happy to provide help on the process. options: - "Yes" - "No" - Maybe validations: - required: true \ No newline at end of file + required: true diff --git a/.github/ISSUE_TEMPLATE/feature_request.yml b/.github/ISSUE_TEMPLATE/feature_request.yml index 6fa3e638be8..45e1da2cad0 100644 --- a/.github/ISSUE_TEMPLATE/feature_request.yml +++ b/.github/ISSUE_TEMPLATE/feature_request.yml @@ -32,11 +32,11 @@ body: required: false - type: dropdown attributes: - label: Are you planning to do it yourself in a pull request ? + label: Are you planning to do it yourself in a pull request? description: Any contribution is greatly appreciated. We are more than happy to provide help on the process. options: - "Yes" - "No" - Maybe validations: - required: true \ No newline at end of file + required: true diff --git a/.github/pull_request_template.md b/.github/pull_request_template.md index 90e8b2cda53..24eb1b75b2c 100644 --- a/.github/pull_request_template.md +++ b/.github/pull_request_template.md @@ -6,7 +6,7 @@ This section will appear as the commit message after merging. Please craft it accordingly. For a quick primer on good commit messages, check out this blog post: https://cbea.ms/git-commit/ -Please include any relevant issues in here, for example: +Please include any relevant issues here, for example: Related https://github.com/libp2p/rust-libp2p/issues/ABCD. Fixes https://github.com/libp2p/rust-libp2p/issues/XYZ. @@ -15,7 +15,7 @@ Fixes https://github.com/libp2p/rust-libp2p/issues/XYZ. ## Notes & open questions ## Change checklist diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 8b3817a7aa9..f8cff086990 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -67,7 +67,7 @@ jobs: echo "CRATE_VERSION=$CRATE_VERSION" >> $GITHUB_ENV - - name: Enforce version in `workspace.dependencies` matches latest version + - name: Enforce version in `workspace.dependencies` matches the latest version if: env.CRATE != 'libp2p' run: | SPECIFIED_VERSION=$(tq "workspace.dependencies.$CRATE.version" --file ./Cargo.toml) @@ -77,7 +77,7 @@ jobs: test "$CRATE_VERSION" = "$SPECIFIED_VERSION" || test "=$CRATE_VERSION" = "$SPECIFIED_VERSION" - - name: Enforce version in CHANGELOG.md matches version in manifest + - name: Enforce version in CHANGELOG.md matches the version in manifest run: | MANIFEST_PATH=$(cargo metadata --format-version=1 --no-deps | jq -e -r '.packages[] | select(.name == "'"$CRATE"'") | .manifest_path') DIR_TO_CRATE=$(dirname "$MANIFEST_PATH") diff --git a/.idea/vcs.xml b/.idea/vcs.xml new file mode 100644 index 00000000000..35eb1ddfbbc --- /dev/null +++ b/.idea/vcs.xml @@ -0,0 +1,6 @@ + + + + + + \ No newline at end of file diff --git a/.idea/workspace.xml b/.idea/workspace.xml new file mode 100644 index 00000000000..e0cbd3e49ab --- /dev/null +++ b/.idea/workspace.xml @@ -0,0 +1,535 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + { + "associatedIndex": 5 +} + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 1728685449599 + + + + + + \ No newline at end of file diff --git a/Cargo.lock b/Cargo.lock index 2f24420a86f..55f69c070f7 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -102,9 +102,9 @@ dependencies = [ [[package]] name = "anstyle" -version = "1.0.1" +version = "1.0.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3a30da5c5f2d5e72842e00bcb57657162cdabef0931f40e2deb9b4140440cecd" +checksum = "55cc3b69f167a1ef2e161439aa98aed94e6028e5f9a59be9a6ffb47aef1651f9" [[package]] name = "anstyle-parse" @@ -158,12 +158,6 @@ version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6b4930d2cb77ce62f89ee5d5289b4ac049559b1c45539271f5ed4fdc7db34545" -[[package]] -name = "arrayvec" -version = "0.7.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" - [[package]] name = "asn1-rs" version = "0.5.2" @@ -543,7 +537,6 @@ dependencies = [ "futures", "libp2p", "tokio", - "tracing", "tracing-subscriber", ] @@ -554,7 +547,6 @@ dependencies = [ "cfg-if", "clap", "libp2p", - "opentelemetry 0.21.0", "opentelemetry-jaeger", "opentelemetry_sdk 0.21.2", "rand 0.8.5", @@ -863,6 +855,12 @@ version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" +[[package]] +name = "cfg_aliases" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "613afe47fcd5fac7ccf1db93babcb082c5994d996f20b8b159f2ad1658eb5724" + [[package]] name = "chacha20" version = "0.9.1" @@ -891,11 +889,9 @@ dependencies = [ name = "chat-example" version = "0.1.0" dependencies = [ - "async-trait", "futures", "libp2p", "tokio", - "tracing", "tracing-subscriber", ] @@ -1268,7 +1264,6 @@ dependencies = [ "futures", "futures-timer", "libp2p", - "log", "tokio", "tracing", "tracing-subscriber", @@ -1378,11 +1373,9 @@ dependencies = [ name = "distributed-key-value-store-example" version = "0.1.0" dependencies = [ - "async-trait", "futures", "libp2p", "tokio", - "tracing", "tracing-subscriber", ] @@ -1479,6 +1472,16 @@ dependencies = [ "syn 2.0.89", ] +[[package]] +name = "env_filter" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4f2c92ceda6ceec50f43169f9ee8424fe2db276791afde7b2cd8bc084cb376ab" +dependencies = [ + "log", + "regex", +] + [[package]] name = "env_logger" version = "0.8.4" @@ -1491,15 +1494,15 @@ dependencies = [ [[package]] name = "env_logger" -version = "0.10.2" +version = "0.11.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4cd405aab171cb85d6735e5c8d9db038c17d3ca007a4d2c25f337935c3d90580" +checksum = "e13fa619b91fb2381732789fc5de83b45675e882f66623b7d8cb4f643017018d" dependencies = [ + "anstream", + "anstyle", + "env_filter", "humantime", - "is-terminal", "log", - "regex", - "termcolor", ] [[package]] @@ -1597,7 +1600,6 @@ dependencies = [ "libp2p", "serde", "tokio", - "tracing", "tracing-subscriber", ] @@ -2061,7 +2063,7 @@ version = "0.1.0" dependencies = [ "anyhow", "either", - "env_logger 0.10.2", + "env_logger 0.11.5", "futures", "libp2p", "redis", @@ -2362,7 +2364,6 @@ dependencies = [ "futures", "libp2p", "tokio", - "tracing", "tracing-subscriber", ] @@ -2504,7 +2505,6 @@ dependencies = [ "anyhow", "axum", "console_error_panic_hook", - "either", "futures", "futures-timer", "libp2p", @@ -2548,13 +2548,10 @@ name = "ipfs-kad-example" version = "0.1.0" dependencies = [ "anyhow", - "async-trait", "clap", - "env_logger 0.10.2", "futures", "libp2p", "tokio", - "tracing", "tracing-subscriber", ] @@ -2562,12 +2559,10 @@ dependencies = [ name = "ipfs-private-example" version = "0.1.0" dependencies = [ - "async-trait", "either", "futures", "libp2p", "tokio", - "tracing", "tracing-subscriber", ] @@ -2633,7 +2628,6 @@ version = "0.1.0" dependencies = [ "base64 0.22.1", "clap", - "libp2p-core", "libp2p-identity", "serde", "serde_json", @@ -2666,9 +2660,7 @@ name = "libp2p" version = "0.54.2" dependencies = [ "async-std", - "async-trait", "bytes", - "clap", "either", "futures", "futures-timer", @@ -2731,7 +2723,6 @@ version = "0.13.2" dependencies = [ "async-trait", "asynchronous-codec", - "bytes", "either", "futures", "futures-bounded", @@ -2788,11 +2779,8 @@ dependencies = [ "parking_lot", "pin-project", "quick-protobuf", - "quickcheck-ext", "rand 0.8.5", "rw-stream-sink", - "serde", - "smallvec", "thiserror 2.0.3", "tracing", "unsigned-varint 0.8.0", @@ -2804,17 +2792,13 @@ name = "libp2p-dcutr" version = "0.12.1" dependencies = [ "asynchronous-codec", - "clap", "either", "futures", "futures-bounded", "futures-timer", "libp2p-core", - "libp2p-dns", "libp2p-identify", "libp2p-identity", - "libp2p-noise", - "libp2p-ping", "libp2p-plaintext", "libp2p-relay", "libp2p-swarm", @@ -2824,7 +2808,6 @@ dependencies = [ "lru", "quick-protobuf", "quick-protobuf-codec", - "rand 0.8.5", "thiserror 2.0.3", "tokio", "tracing", @@ -2885,14 +2868,11 @@ dependencies = [ "futures-timer", "getrandom 0.2.15", "hashlink", - "hex", "hex_fmt", "libp2p-core", "libp2p-identity", - "libp2p-noise", "libp2p-swarm", "libp2p-swarm-test", - "libp2p-yamux", "prometheus-client", "quick-protobuf", "quick-protobuf-codec", @@ -2901,7 +2881,6 @@ dependencies = [ "regex", "serde", "sha2 0.10.8", - "smallvec", "tokio", "tracing", "tracing-subscriber", @@ -2922,7 +2901,6 @@ dependencies = [ "libp2p-identity", "libp2p-swarm", "libp2p-swarm-test", - "lru", "quick-protobuf", "quick-protobuf-codec", "smallvec", @@ -2936,7 +2914,6 @@ name = "libp2p-identity" version = "0.2.10" dependencies = [ "asn1_der", - "base64 0.22.1", "bs58", "criterion", "ed25519-dalek", @@ -2963,7 +2940,6 @@ dependencies = [ name = "libp2p-kad" version = "0.47.1" dependencies = [ - "arrayvec", "async-std", "asynchronous-codec", "bytes", @@ -2999,17 +2975,13 @@ version = "0.46.1" dependencies = [ "async-io", "async-std", - "data-encoding", "futures", "hickory-proto", "if-watch", "libp2p-core", "libp2p-identity", - "libp2p-noise", "libp2p-swarm", "libp2p-swarm-test", - "libp2p-tcp", - "libp2p-yamux", "rand 0.8.5", "smallvec", "socket2", @@ -3022,7 +2994,6 @@ dependencies = [ name = "libp2p-memory-connection-limits" version = "0.3.1" dependencies = [ - "async-std", "libp2p-core", "libp2p-identify", "libp2p-identity", @@ -3030,7 +3001,6 @@ dependencies = [ "libp2p-swarm-derive", "libp2p-swarm-test", "memory-stats", - "rand 0.8.5", "sysinfo", "tracing", ] @@ -3095,7 +3065,6 @@ version = "0.45.1" dependencies = [ "asynchronous-codec", "bytes", - "curve25519-dalek", "futures", "futures_ringbuf", "libp2p-core", @@ -3106,7 +3075,6 @@ dependencies = [ "quick-protobuf", "quickcheck-ext", "rand 0.8.5", - "sha2 0.10.8", "snow", "static_assertions", "thiserror 2.0.3", @@ -3127,15 +3095,12 @@ dependencies = [ "futures-timer", "libp2p", "libp2p-core", - "libp2p-dns", "libp2p-identity", - "libp2p-quic", "libp2p-swarm", "libp2p-swarm-test", "libp2p-tcp", "libp2p-tls", "libp2p-yamux", - "rand 0.8.5", "serde", "serde_json", "thiserror 2.0.3", @@ -3149,7 +3114,6 @@ dependencies = [ name = "libp2p-ping" version = "0.45.1" dependencies = [ - "either", "futures", "futures-timer", "libp2p-core", @@ -3160,7 +3124,6 @@ dependencies = [ "rand 0.8.5", "tokio", "tracing", - "tracing-subscriber", "web-time 1.1.0", ] @@ -3177,7 +3140,6 @@ dependencies = [ "quick-protobuf", "quick-protobuf-codec", "quickcheck-ext", - "rand 0.8.5", "tracing", "tracing-subscriber", ] @@ -3208,7 +3170,6 @@ name = "libp2p-quic" version = "0.11.2" dependencies = [ "async-std", - "bytes", "futures", "futures-timer", "if-watch", @@ -3219,7 +3180,6 @@ dependencies = [ "libp2p-tcp", "libp2p-tls", "libp2p-yamux", - "parking_lot", "quickcheck", "quinn", "rand 0.8.5", @@ -3270,15 +3230,10 @@ dependencies = [ "futures", "futures-timer", "libp2p-core", - "libp2p-identify", "libp2p-identity", - "libp2p-noise", - "libp2p-ping", "libp2p-request-response", "libp2p-swarm", "libp2p-swarm-test", - "libp2p-tcp", - "libp2p-yamux", "quick-protobuf", "quick-protobuf-codec", "rand 0.8.5", @@ -3299,22 +3254,17 @@ dependencies = [ "cbor4ii", "futures", "futures-bounded", - "futures-timer", "futures_ringbuf", "libp2p-core", "libp2p-identity", - "libp2p-noise", "libp2p-swarm", "libp2p-swarm-test", - "libp2p-tcp", - "libp2p-yamux", "rand 0.8.5", "serde", "serde_json", "smallvec", "tracing", "tracing-subscriber", - "web-time 1.1.0", ] [[package]] @@ -3325,11 +3275,9 @@ dependencies = [ "base64 0.22.1", "clap", "futures", - "futures-timer", "libp2p", "prometheus-client", "serde", - "serde_derive", "serde_json", "tokio", "tracing", @@ -3391,7 +3339,6 @@ name = "libp2p-swarm-derive" version = "0.35.0" dependencies = [ "heck 0.5.0", - "proc-macro2", "quote", "syn 2.0.89", ] @@ -3409,7 +3356,6 @@ dependencies = [ "libp2p-swarm", "libp2p-tcp", "libp2p-yamux", - "rand 0.8.5", "tracing", ] @@ -3424,7 +3370,6 @@ dependencies = [ "if-watch", "libc", "libp2p-core", - "libp2p-identity", "socket2", "tokio", "tracing", @@ -3437,7 +3382,6 @@ version = "0.5.0" dependencies = [ "futures", "futures-rustls", - "hex", "hex-literal", "libp2p-core", "libp2p-identity", @@ -3483,7 +3427,6 @@ name = "libp2p-webrtc" version = "0.8.0-alpha" dependencies = [ "async-trait", - "bytes", "futures", "futures-timer", "hex", @@ -3496,10 +3439,8 @@ dependencies = [ "quickcheck", "rand 0.8.5", "rcgen", - "serde", "stun 0.6.0", "thiserror 2.0.3", - "tinytemplate", "tokio", "tokio-util", "tracing", @@ -3524,7 +3465,6 @@ dependencies = [ "rand 0.8.5", "serde", "sha2 0.10.8", - "thiserror 2.0.3", "tinytemplate", "tracing", ] @@ -3583,7 +3523,6 @@ dependencies = [ "libp2p-identity", "libp2p-noise", "libp2p-yamux", - "parking_lot", "send_wrapper 0.6.0", "thiserror 2.0.3", "tracing", @@ -3922,7 +3861,6 @@ dependencies = [ "futures_ringbuf", "pin-project", "quickcheck-ext", - "rand 0.8.5", "rw-stream-sink", "smallvec", "tracing", @@ -4473,7 +4411,6 @@ dependencies = [ "futures", "libp2p", "tokio", - "tracing", "tracing-subscriber", ] @@ -4716,9 +4653,9 @@ dependencies = [ [[package]] name = "quinn" -version = "0.11.2" +version = "0.11.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e4ceeeeabace7857413798eb1ffa1e9c905a9946a57d81fb69b4b71c4d8eb3ad" +checksum = "62e96808277ec6f97351a2380e6c25114bc9e67037775464979f3037c92d05ef" dependencies = [ "async-io", "async-std", @@ -4727,36 +4664,41 @@ dependencies = [ "pin-project-lite", "quinn-proto", "quinn-udp", - "rustc-hash 1.1.0", + "rustc-hash", "rustls 0.23.11", - "thiserror 1.0.63", + "socket2", + "thiserror 2.0.3", "tokio", "tracing", ] [[package]] name = "quinn-proto" -version = "0.11.8" +version = "0.11.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fadfaed2cd7f389d0161bb73eeb07b7b78f8691047a6f3e73caaeae55310a4a6" +checksum = "a2fe5ef3495d7d2e377ff17b1a8ce2ee2ec2a18cde8b6ad6619d65d0701c135d" dependencies = [ "bytes", + "getrandom 0.2.15", "rand 0.8.5", "ring 0.17.8", - "rustc-hash 2.0.0", + "rustc-hash", "rustls 0.23.11", + "rustls-pki-types", "slab", - "thiserror 1.0.63", + "thiserror 2.0.3", "tinyvec", "tracing", + "web-time 1.1.0", ] [[package]] name = "quinn-udp" -version = "0.5.0" +version = "0.5.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cb7ad7bc932e4968523fa7d9c320ee135ff779de720e9350fee8728838551764" +checksum = "1c40286217b4ba3a71d644d752e6a0b71f13f1b6a2c5311acfcbe0c2418ed904" dependencies = [ + "cfg_aliases", "libc", "once_cell", "socket2", @@ -4986,7 +4928,6 @@ dependencies = [ "futures", "libp2p", "tokio", - "tracing", "tracing-subscriber", ] @@ -5214,12 +5155,6 @@ version = "0.1.23" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d626bb9dae77e28219937af045c257c28bfd3f69333c512553507f5f9798cb76" -[[package]] -name = "rustc-hash" -version = "1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08d43f7aa6b08d49f382cde6a7982047c3426db949b1424bc4b7ec9ae12c6ce2" - [[package]] name = "rustc-hash" version = "2.0.0" @@ -5312,6 +5247,9 @@ name = "rustls-pki-types" version = "1.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "976295e77ce332211c0d24d92c0e83e50f5c5f046d11082cea19f3df13a3562d" +dependencies = [ + "web-time 1.1.0", +] [[package]] name = "rustls-webpki" diff --git a/Cargo.toml b/Cargo.toml index e312f011a1e..c349e51f7ae 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -117,9 +117,11 @@ libp2p-yamux = { version = "0.46.0", path = "muxers/yamux" } # External dependencies async-std-resolver = { version = "0.25.0-alpha.4", default-features = false } asynchronous-codec = { version = "0.7.0" } +env_logger = "0.11" futures = "0.3.30" futures-bounded = { version = "0.2.4" } futures-rustls = { version = "0.26.0", default-features = false } +getrandom = "0.2" hickory-proto = { version = "0.25.0-alpha.4", default-features = false } hickory-resolver = { version = "0.25.0-alpha.4", default-features = false } multiaddr = "0.18.1" diff --git a/core/Cargo.toml b/core/Cargo.toml index 8ec0b0fc197..162800b96c2 100644 --- a/core/Cargo.toml +++ b/core/Cargo.toml @@ -26,8 +26,6 @@ pin-project = "1.1.5" quick-protobuf = "0.8" rand = "0.8" rw-stream-sink = { workspace = true } -serde = { version = "1", optional = true, features = ["derive"] } -smallvec = "1.13.2" thiserror = { workspace = true } tracing = { workspace = true } unsigned-varint = { workspace = true } @@ -37,11 +35,10 @@ async-std = { version = "1.6.2", features = ["attributes"] } libp2p-mplex = { path = "../muxers/mplex" } # Using `path` here because this is a cyclic dev-dependency which otherwise breaks releasing. libp2p-noise = { path = "../transports/noise" } # Using `path` here because this is a cyclic dev-dependency which otherwise breaks releasing. multihash = { workspace = true, features = ["arb"] } -quickcheck = { workspace = true } libp2p-identity = { workspace = true, features = ["ed25519", "rand"] } [features] -serde = ["multihash/serde-codec", "dep:serde", "libp2p-identity/serde"] +serde = ["multihash/serde-codec", "libp2p-identity/serde"] # Passing arguments to the docsrs builder in order to properly document cfg's. # More information: https://docs.rs/about/builds#cross-compiling diff --git a/examples/autonat/Cargo.toml b/examples/autonat/Cargo.toml index 010b76623e0..7c06b48a105 100644 --- a/examples/autonat/Cargo.toml +++ b/examples/autonat/Cargo.toml @@ -13,7 +13,6 @@ tokio = { workspace = true, features = ["full"] } clap = { version = "4.5.6", features = ["derive"] } futures = { workspace = true } libp2p = { path = "../../libp2p", features = ["tokio", "tcp", "noise", "yamux", "autonat", "identify", "macros"] } -tracing = { workspace = true } tracing-subscriber = { workspace = true, features = ["env-filter"] } [lints] diff --git a/examples/autonatv2/Cargo.toml b/examples/autonatv2/Cargo.toml index 6c862ee22e4..67e74d67a22 100644 --- a/examples/autonatv2/Cargo.toml +++ b/examples/autonatv2/Cargo.toml @@ -21,15 +21,13 @@ tokio = { version = "1.35.1", features = ["macros", "rt-multi-thread"] } tracing = "0.1.40" tracing-subscriber = { version = "0.3.18", features = ["env-filter"] } rand = "0.8.5" -opentelemetry = { version = "0.21.0", optional = true } opentelemetry_sdk = { version = "0.21.1", optional = true, features = ["rt-tokio"] } tracing-opentelemetry = { version = "0.22.0", optional = true } opentelemetry-jaeger = { version = "0.20.0", optional = true, features = ["rt-tokio"] } cfg-if = "1.0.0" [features] -jaeger = ["opentelemetry", "opentelemetry_sdk", "tracing-opentelemetry", "opentelemetry-jaeger"] -opentelemetry = ["dep:opentelemetry"] +jaeger = ["opentelemetry_sdk", "tracing-opentelemetry", "opentelemetry-jaeger"] opentelemetry_sdk = ["dep:opentelemetry_sdk"] tracing-opentelemetry = ["dep:tracing-opentelemetry"] opentelemetry-jaeger = ["dep:opentelemetry-jaeger"] diff --git a/examples/chat/Cargo.toml b/examples/chat/Cargo.toml index a1d32956825..031f84b6f95 100644 --- a/examples/chat/Cargo.toml +++ b/examples/chat/Cargo.toml @@ -10,10 +10,8 @@ release = false [dependencies] tokio = { workspace = true, features = ["full"] } -async-trait = "0.1" futures = { workspace = true } libp2p = { path = "../../libp2p", features = [ "tokio", "gossipsub", "mdns", "noise", "macros", "tcp", "yamux", "quic"] } -tracing = { workspace = true } tracing-subscriber = { workspace = true, features = ["env-filter"] } [lints] diff --git a/examples/dcutr/Cargo.toml b/examples/dcutr/Cargo.toml index c1b4bbc6e7e..67edf04e2b0 100644 --- a/examples/dcutr/Cargo.toml +++ b/examples/dcutr/Cargo.toml @@ -13,7 +13,6 @@ clap = { version = "4.5.6", features = ["derive"] } futures = { workspace = true } futures-timer = "3.0" libp2p = { path = "../../libp2p", features = [ "dns", "dcutr", "identify", "macros", "noise", "ping", "quic", "relay", "rendezvous", "tcp", "tokio", "yamux"] } -log = "0.4" tokio = { workspace = true, features = ["macros", "net", "rt", "signal"] } tracing = { workspace = true } tracing-subscriber = { workspace = true, features = ["env-filter"] } diff --git a/examples/distributed-key-value-store/Cargo.toml b/examples/distributed-key-value-store/Cargo.toml index 3846e54c8d3..8e30dd2c75d 100644 --- a/examples/distributed-key-value-store/Cargo.toml +++ b/examples/distributed-key-value-store/Cargo.toml @@ -10,10 +10,8 @@ release = false [dependencies] tokio = { workspace = true, features = ["full"] } -async-trait = "0.1" futures = { workspace = true } libp2p = { path = "../../libp2p", features = [ "tokio", "dns", "kad", "mdns", "noise", "macros", "tcp", "yamux"] } -tracing = { workspace = true } tracing-subscriber = { workspace = true, features = ["env-filter"] } [lints] diff --git a/examples/file-sharing/Cargo.toml b/examples/file-sharing/Cargo.toml index d098ce44317..021215c003b 100644 --- a/examples/file-sharing/Cargo.toml +++ b/examples/file-sharing/Cargo.toml @@ -14,7 +14,6 @@ tokio = { workspace = true, features = ["full"] } clap = { version = "4.5.6", features = ["derive"] } futures = { workspace = true } libp2p = { path = "../../libp2p", features = [ "tokio", "cbor", "dns", "kad", "noise", "macros", "request-response", "tcp", "websocket", "yamux"] } -tracing = { workspace = true } tracing-subscriber = { workspace = true, features = ["env-filter"] } [lints] diff --git a/examples/identify/Cargo.toml b/examples/identify/Cargo.toml index 8d12699afa7..c18f71a0386 100644 --- a/examples/identify/Cargo.toml +++ b/examples/identify/Cargo.toml @@ -12,7 +12,6 @@ release = false tokio = { version = "1.37.0", features = ["full"] } futures = { workspace = true } libp2p = { path = "../../libp2p", features = ["identify", "noise", "tcp", "tokio", "yamux"] } -tracing = { workspace = true } tracing-subscriber = { workspace = true, features = ["env-filter"] } [lints] diff --git a/examples/ipfs-kad/Cargo.toml b/examples/ipfs-kad/Cargo.toml index 115c604269f..fa04da4edcf 100644 --- a/examples/ipfs-kad/Cargo.toml +++ b/examples/ipfs-kad/Cargo.toml @@ -10,13 +10,10 @@ release = false [dependencies] tokio = { workspace = true, features = ["rt-multi-thread", "macros"] } -async-trait = "0.1" clap = { version = "4.5.6", features = ["derive"] } -env_logger = "0.10" futures = { workspace = true } anyhow = "1.0.86" libp2p = { path = "../../libp2p", features = [ "tokio", "dns", "kad", "noise", "tcp", "yamux", "rsa"] } -tracing = { workspace = true } tracing-subscriber = { workspace = true, features = ["env-filter"] } [lints] diff --git a/examples/ipfs-private/Cargo.toml b/examples/ipfs-private/Cargo.toml index 0813dba56e0..4dfe596d609 100644 --- a/examples/ipfs-private/Cargo.toml +++ b/examples/ipfs-private/Cargo.toml @@ -10,11 +10,9 @@ release = false [dependencies] tokio = { workspace = true, features = ["rt-multi-thread", "macros", "io-std"] } -async-trait = "0.1" either = "1.12" futures = { workspace = true } libp2p = { path = "../../libp2p", features = [ "tokio", "gossipsub", "dns", "identify", "kad", "macros", "noise", "ping", "pnet", "tcp", "websocket", "yamux"] } -tracing = { workspace = true } tracing-subscriber = { workspace = true, features = ["env-filter"] } [lints] diff --git a/examples/ping/Cargo.toml b/examples/ping/Cargo.toml index 633f043de56..acc3b2affed 100644 --- a/examples/ping/Cargo.toml +++ b/examples/ping/Cargo.toml @@ -12,7 +12,6 @@ release = false futures = { workspace = true } libp2p = { path = "../../libp2p", features = ["noise", "ping", "tcp", "tokio", "yamux"] } tokio = { workspace = true, features = ["full"] } -tracing = { workspace = true } tracing-subscriber = { workspace = true, features = ["env-filter"] } [lints] diff --git a/examples/relay-server/Cargo.toml b/examples/relay-server/Cargo.toml index 7385cf6c033..3bdaf89b04f 100644 --- a/examples/relay-server/Cargo.toml +++ b/examples/relay-server/Cargo.toml @@ -13,7 +13,6 @@ clap = { version = "4.5.6", features = ["derive"] } tokio = { version = "1.37.0", features = ["full"] } futures = { workspace = true } libp2p = { path = "../../libp2p", features = ["tokio", "noise", "macros", "ping", "tcp", "identify", "yamux", "relay", "quic"] } -tracing = { workspace = true } tracing-subscriber = { workspace = true, features = ["env-filter"] } [lints] diff --git a/hole-punching-tests/Cargo.toml b/hole-punching-tests/Cargo.toml index 79728f9535c..c4f36d2a990 100644 --- a/hole-punching-tests/Cargo.toml +++ b/hole-punching-tests/Cargo.toml @@ -7,7 +7,7 @@ license = "MIT" [dependencies] anyhow = "1" -env_logger = "0.10.2" +env_logger = { workspace = true } futures = { workspace = true } libp2p = { path = "../libp2p", features = ["tokio", "dcutr", "identify", "macros", "noise", "ping", "relay", "tcp", "yamux", "quic"] } tracing = { workspace = true } diff --git a/identity/Cargo.toml b/identity/Cargo.toml index cc41abb3e24..b13229c5826 100644 --- a/identity/Cargo.toml +++ b/identity/Cargo.toml @@ -41,7 +41,6 @@ rand = ["dep:rand", "ed25519-dalek?/rand_core"] [dev-dependencies] quickcheck = { workspace = true } -base64 = "0.22.1" serde_json = "1.0" rmp-serde = "1.3" criterion = "0.5" diff --git a/interop-tests/Cargo.toml b/interop-tests/Cargo.toml index 0eb32bb4975..8f12275668d 100644 --- a/interop-tests/Cargo.toml +++ b/interop-tests/Cargo.toml @@ -13,7 +13,6 @@ crate-type = ["cdylib", "rlib"] [dependencies] anyhow = "1" -either = "1.11.0" futures = { workspace = true } rand = "0.8.5" serde = { version = "1", features = ["derive"] } diff --git a/libp2p/Cargo.toml b/libp2p/Cargo.toml index 3d44e0bc43c..ae23dcf0db5 100644 --- a/libp2p/Cargo.toml +++ b/libp2p/Cargo.toml @@ -94,7 +94,7 @@ bytes = "1" either = "1.9.0" futures = { workspace = true } futures-timer = "3.0.2" # Explicit dependency to be used in `wasm-bindgen` feature -getrandom = "0.2.3" # Explicit dependency to be used in `wasm-bindgen` feature +getrandom = { workspace = true } # Explicit dependency to be used in `wasm-bindgen` feature # TODO feature flag? rw-stream-sink = { workspace = true } @@ -137,8 +137,6 @@ libp2p-websocket = { workspace = true, optional = true } [dev-dependencies] async-std = { version = "1.6.2", features = ["attributes"] } -async-trait = "0.1" -clap = { version = "4.1.6", features = ["derive"] } tokio = { workspace = true, features = [ "io-util", "io-std", "macros", "rt", "rt-multi-thread"] } libp2p-mplex = { workspace = true } diff --git a/misc/keygen/Cargo.toml b/misc/keygen/Cargo.toml index 003993a512c..c5e96553a5c 100644 --- a/misc/keygen/Cargo.toml +++ b/misc/keygen/Cargo.toml @@ -17,7 +17,6 @@ clap = { version = "4.5.6", features = ["derive"] } zeroize = "1" serde = { version = "1.0.203", features = ["derive"] } serde_json = "1.0.117" -libp2p-core = { workspace = true } base64 = "0.22.1" libp2p-identity = { workspace = true } diff --git a/misc/memory-connection-limits/Cargo.toml b/misc/memory-connection-limits/Cargo.toml index 2f6b6ea1544..2d04b6cf2ac 100644 --- a/misc/memory-connection-limits/Cargo.toml +++ b/misc/memory-connection-limits/Cargo.toml @@ -18,11 +18,9 @@ sysinfo = "0.33" tracing = { workspace = true } [dev-dependencies] -async-std = { version = "1.12.0", features = ["attributes"] } libp2p-identify = { workspace = true } libp2p-swarm-derive = { path = "../../swarm-derive" } libp2p-swarm-test = { path = "../../swarm-test" } -rand = "0.8.5" [lints] workspace = true diff --git a/misc/multistream-select/Cargo.toml b/misc/multistream-select/Cargo.toml index 1bbe3642477..d11ad4e2709 100644 --- a/misc/multistream-select/Cargo.toml +++ b/misc/multistream-select/Cargo.toml @@ -22,7 +22,6 @@ unsigned-varint = { workspace = true } async-std = { version = "1.6.2", features = ["attributes"] } futures_ringbuf = "0.4.0" quickcheck = { workspace = true } -rand = "0.8" rw-stream-sink = { workspace = true } tracing-subscriber = { workspace = true, features = ["env-filter"] } diff --git a/misc/server/Cargo.toml b/misc/server/Cargo.toml index 0954e2f38d8..02da0adb9ef 100644 --- a/misc/server/Cargo.toml +++ b/misc/server/Cargo.toml @@ -14,7 +14,6 @@ license = "MIT" base64 = "0.22" clap = { version = "4.5.6", features = ["derive"] } futures = { workspace = true } -futures-timer = "3" axum = "0.7" libp2p = { workspace = true, features = [ "autonat", @@ -34,8 +33,7 @@ libp2p = { workspace = true, features = [ "websocket", ] } prometheus-client = { workspace = true } -serde = "1.0.203" -serde_derive = "1.0.125" +serde = { version = "1", features = ["derive"] } serde_json = "1.0" tokio = { workspace = true, features = ["rt-multi-thread", "macros"] } tracing = { workspace = true } diff --git a/misc/server/src/config.rs b/misc/server/src/config.rs index 2e4b2746d09..8f8c71369b2 100644 --- a/misc/server/src/config.rs +++ b/misc/server/src/config.rs @@ -1,7 +1,7 @@ use std::{error::Error, path::Path}; use libp2p::Multiaddr; -use serde_derive::Deserialize; +use serde::Deserialize; #[derive(Clone, Deserialize)] #[serde(rename_all = "PascalCase")] diff --git a/misc/webrtc-utils/Cargo.toml b/misc/webrtc-utils/Cargo.toml index 287388a49e7..2c50a2f8ab7 100644 --- a/misc/webrtc-utils/Cargo.toml +++ b/misc/webrtc-utils/Cargo.toml @@ -23,7 +23,6 @@ quick-protobuf-codec = { workspace = true } rand = "0.8" serde = { version = "1.0", features = ["derive"] } sha2 = "0.10.8" -thiserror = { workspace = true } tinytemplate = "1.2" tracing = { workspace = true } diff --git a/protocols/autonat/Cargo.toml b/protocols/autonat/Cargo.toml index 88564b18541..8ef6f69d39a 100644 --- a/protocols/autonat/Cargo.toml +++ b/protocols/autonat/Cargo.toml @@ -18,7 +18,6 @@ categories = ["network-programming", "asynchronous"] [dependencies] async-trait = { version = "0.1", optional = true } asynchronous-codec = { workspace = true } -bytes = { version = "1", optional = true } either = { version = "1.9.0", optional = true } futures = { workspace = true } futures-bounded = { workspace = true, optional = true } @@ -45,7 +44,7 @@ libp2p-swarm = { workspace = true, features = ["macros"] } [features] default = ["v1", "v2"] v1 = ["dep:libp2p-request-response", "dep:web-time", "dep:async-trait"] -v2 = ["dep:bytes", "dep:either", "dep:futures-bounded", "dep:thiserror", "dep:rand_core"] +v2 = ["dep:either", "dep:futures-bounded", "dep:thiserror", "dep:rand_core"] # Passing arguments to the docsrs builder in order to properly document cfg's. # More information: https://docs.rs/about/builds#cross-compiling diff --git a/protocols/dcutr/Cargo.toml b/protocols/dcutr/Cargo.toml index a47f5400488..31acb42f2af 100644 --- a/protocols/dcutr/Cargo.toml +++ b/protocols/dcutr/Cargo.toml @@ -27,18 +27,13 @@ lru = "0.12.3" futures-bounded = { workspace = true } [dev-dependencies] -clap = { version = "4.5.6", features = ["derive"] } -libp2p-dns = { workspace = true, features = ["async-std"] } libp2p-identify = { workspace = true } -libp2p-noise = { workspace = true } -libp2p-ping = { workspace = true } libp2p-plaintext = { workspace = true } libp2p-relay = { workspace = true } libp2p-swarm = { workspace = true, features = ["macros"] } libp2p-swarm-test = { path = "../../swarm-test" } libp2p-tcp = { workspace = true, features = ["async-io"] } libp2p-yamux = { workspace = true } -rand = "0.8" tracing-subscriber = { workspace = true, features = ["env-filter"] } tokio = { workspace = true, features = ["rt", "macros"] } diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml index 1926b321bd6..5608c2de366 100644 --- a/protocols/gossipsub/Cargo.toml +++ b/protocols/gossipsub/Cargo.toml @@ -23,8 +23,8 @@ either = "1.11" fnv = "1.0.7" futures = { workspace = true } futures-timer = "3.0.2" -getrandom = "0.2.15" -hashlink.workspace = true +getrandom = { workspace = true } +hashlink = { workspace = true } hex_fmt = "0.3.0" web-time = { workspace = true } libp2p-core = { workspace = true } @@ -36,17 +36,13 @@ rand = "0.8" regex = "1.10.5" serde = { version = "1", optional = true, features = ["derive"] } sha2 = "0.10.8" -smallvec = "1.13.2" tracing = { workspace = true } # Metrics dependencies prometheus-client = { workspace = true } [dev-dependencies] -hex = "0.4.2" libp2p-core = { workspace = true } -libp2p-yamux = { workspace = true } -libp2p-noise = { workspace = true } libp2p-swarm-test = { path = "../../swarm-test" } quickcheck = { workspace = true } tracing-subscriber = { workspace = true, features = ["env-filter"] } diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index f58b2a35e27..f16099ee0fb 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -2681,7 +2681,7 @@ where // forward the message to peers for peer_id in recipient_peers.iter() { - if let Some(peer) = self.connected_peers.get_mut(peer_id) { + if peer.dont_send.get(msg_id).is_some() { tracing::debug!(%peer_id, message=%msg_id, "Peer doesn't want message"); continue; @@ -2696,10 +2696,7 @@ where timeout: Delay::new(self.config.forward_queue_duration()), }, ); - } else { - tracing::error!(peer = %peer_id, - "Could not FORWARD, peer doesn't exist in connected peer list"); - } + } tracing::debug!("Completed forwarding message"); true diff --git a/protocols/gossipsub/src/behaviour/tests.rs b/protocols/gossipsub/src/behaviour/tests.rs index 54e12e7490b..921fb990ab4 100644 --- a/protocols/gossipsub/src/behaviour/tests.rs +++ b/protocols/gossipsub/src/behaviour/tests.rs @@ -4619,7 +4619,7 @@ fn test_ignore_too_many_messages_in_ihave() { let (peer, receiver) = add_peer(&mut gs, &topics, false, false); receivers.insert(peer, receiver); - //peer has 30 messages + // peer has 30 messages let mut seq = 0; let message_ids: Vec<_> = (0..30) .map(|_| random_message(&mut seq, &topics)) @@ -5585,7 +5585,7 @@ fn test_slow_peer_returns_failed_ihave_handling() { topics.insert(topic_hash.clone()); let slow_peer_id = PeerId::random(); - peers.push(slow_peer_id); + // peers.push(slow_peer_id); gs.connected_peers.insert( slow_peer_id, PeerConnections { @@ -5613,6 +5613,7 @@ fn test_slow_peer_returns_failed_ihave_handling() { }, ); + // First message. let publish_data = vec![1; 59]; let transformed = gs .data_transform @@ -5632,6 +5633,22 @@ fn test_slow_peer_returns_failed_ihave_handling() { &slow_peer_id, vec![(topic_hash.clone(), vec![msg_id.clone()])], ); + + // Second message. + let publish_data = vec![2; 59]; + let transformed = gs + .data_transform + .outbound_transform(&topic_hash, publish_data.clone()) + .unwrap(); + let raw_message = gs + .build_raw_message(topic_hash.clone(), transformed) + .unwrap(); + let msg_id = gs.config.message_id(&Message { + source: raw_message.source, + data: publish_data, + sequence_number: raw_message.sequence_number, + topic: raw_message.topic.clone(), + }); gs.handle_ihave(&slow_peer_id, vec![(topic_hash, vec![msg_id.clone()])]); gs.heartbeat(); diff --git a/protocols/gossipsub/src/types.rs b/protocols/gossipsub/src/types.rs index ccf68e86888..41a139c73eb 100644 --- a/protocols/gossipsub/src/types.rs +++ b/protocols/gossipsub/src/types.rs @@ -19,7 +19,7 @@ // DEALINGS IN THE SOFTWARE. //! A collection of types using the Gossipsub system. -use std::{collections::BTreeSet, fmt, fmt::Debug}; +use std::{collections::BTreeSet, fmt, fmt::Debug, time::Instant}; use futures_timer::Delay; use hashlink::LinkedHashMap; @@ -29,7 +29,6 @@ use prometheus_client::encoding::EncodeLabelValue; use quick_protobuf::MessageWrite; #[cfg(feature = "serde")] use serde::{Deserialize, Serialize}; -use std::time::Instant; use crate::{rpc::Sender, rpc_proto::proto, TopicHash}; @@ -263,7 +262,8 @@ pub enum ControlAction { Graft(Graft), /// The node has been removed from the mesh - Prune control message. Prune(Prune), - /// The node requests us to not forward message ids (peer_id + sequence _number) - IDontWant control message. + /// The node requests us to not forward message ids (peer_id + sequence _number) - IDontWant + /// control message. IDontWant(IDontWant), } @@ -329,7 +329,8 @@ pub enum RpcOut { IHave(IHave), /// Send a IWant control message. IWant(IWant), - /// The node requests us to not forward message ids (peer_id + sequence _number) - IDontWant control message. + /// The node requests us to not forward message ids (peer_id + sequence _number) - IDontWant + /// control message. IDontWant(IDontWant), } diff --git a/protocols/identify/Cargo.toml b/protocols/identify/Cargo.toml index d2aeb74e626..e5bb445b7a5 100644 --- a/protocols/identify/Cargo.toml +++ b/protocols/identify/Cargo.toml @@ -18,7 +18,6 @@ futures-bounded = { workspace = true } libp2p-core = { workspace = true } libp2p-swarm = { workspace = true } libp2p-identity = { workspace = true } -lru = "0.12.3" quick-protobuf-codec = { workspace = true } quick-protobuf = "0.8" smallvec = "1.13.2" diff --git a/protocols/kad/Cargo.toml b/protocols/kad/Cargo.toml index dd93da2a01a..a9bc1a9a640 100644 --- a/protocols/kad/Cargo.toml +++ b/protocols/kad/Cargo.toml @@ -11,7 +11,6 @@ keywords = ["peer-to-peer", "libp2p", "networking"] categories = ["network-programming", "asynchronous"] [dependencies] -arrayvec = "0.7.4" bytes = "1" either = "1.11" fnv = "1.0" diff --git a/protocols/mdns/CHANGELOG.md b/protocols/mdns/CHANGELOG.md index 61290703c34..45a479bf4af 100644 --- a/protocols/mdns/CHANGELOG.md +++ b/protocols/mdns/CHANGELOG.md @@ -1,5 +1,7 @@ ## 0.46.1 +- Emit `ToSwarm::NewExternalAddrOfPeer` on discovery. + See [PR 5753](https://github.com/libp2p/rust-libp2p/pull/5753) - Upgrade `hickory-proto`. See [PR 5727](https://github.com/libp2p/rust-libp2p/pull/5727) diff --git a/protocols/mdns/Cargo.toml b/protocols/mdns/Cargo.toml index 16436848efe..89d53c98a70 100644 --- a/protocols/mdns/Cargo.toml +++ b/protocols/mdns/Cargo.toml @@ -13,7 +13,6 @@ categories = ["network-programming", "asynchronous"] [dependencies] async-std = { version = "1.12.0", optional = true } async-io = { version = "2.3.3", optional = true } -data-encoding = "2.6.0" futures = { workspace = true } if-watch = "3.2.0" libp2p-core = { workspace = true } @@ -32,10 +31,7 @@ async-io = ["dep:async-io", "dep:async-std", "if-watch/smol"] [dev-dependencies] async-std = { version = "1.9.0", features = ["attributes"] } -libp2p-noise = { workspace = true } libp2p-swarm = { workspace = true, features = ["tokio", "async-std"] } -libp2p-tcp = { workspace = true, features = ["tokio", "async-io"] } -libp2p-yamux = { workspace = true } tokio = { workspace = true, features = ["macros", "rt", "rt-multi-thread", "time"] } libp2p-swarm-test = { path = "../../swarm-test" } tracing-subscriber = { workspace = true, features = ["env-filter"] } diff --git a/protocols/mdns/src/behaviour.rs b/protocols/mdns/src/behaviour.rs index b6dde8f4487..68e28cf3d63 100644 --- a/protocols/mdns/src/behaviour.rs +++ b/protocols/mdns/src/behaviour.rs @@ -24,7 +24,11 @@ mod timer; use std::{ cmp, - collections::hash_map::{Entry, HashMap}, + collections::{ + hash_map::{Entry, HashMap}, + VecDeque, + }, + convert::Infallible, fmt, future::Future, io, @@ -188,6 +192,9 @@ where listen_addresses: Arc>, local_peer_id: PeerId, + + /// Pending behaviour events to be emitted. + pending_events: VecDeque>, } impl

Behaviour

@@ -208,6 +215,7 @@ where closest_expiration: Default::default(), listen_addresses: Default::default(), local_peer_id, + pending_events: Default::default(), }) } @@ -304,93 +312,113 @@ where &mut self, cx: &mut Context<'_>, ) -> Poll>> { - // Poll ifwatch. - while let Poll::Ready(Some(event)) = Pin::new(&mut self.if_watch).poll_next(cx) { - match event { - Ok(IfEvent::Up(inet)) => { - let addr = inet.addr(); - if addr.is_loopback() { - continue; - } - if addr.is_ipv4() && self.config.enable_ipv6 - || addr.is_ipv6() && !self.config.enable_ipv6 - { - continue; - } - if let Entry::Vacant(e) = self.if_tasks.entry(addr) { - match InterfaceState::::new( - addr, - self.config.clone(), - self.local_peer_id, - self.listen_addresses.clone(), - self.query_response_sender.clone(), - ) { - Ok(iface_state) => { - e.insert(P::spawn(iface_state)); - } - Err(err) => { - tracing::error!("failed to create `InterfaceState`: {}", err) + loop { + // Check for pending events and emit them. + if let Some(event) = self.pending_events.pop_front() { + return Poll::Ready(event); + } + + // Poll ifwatch. + while let Poll::Ready(Some(event)) = Pin::new(&mut self.if_watch).poll_next(cx) { + match event { + Ok(IfEvent::Up(inet)) => { + let addr = inet.addr(); + if addr.is_loopback() { + continue; + } + if addr.is_ipv4() && self.config.enable_ipv6 + || addr.is_ipv6() && !self.config.enable_ipv6 + { + continue; + } + if let Entry::Vacant(e) = self.if_tasks.entry(addr) { + match InterfaceState::::new( + addr, + self.config.clone(), + self.local_peer_id, + self.listen_addresses.clone(), + self.query_response_sender.clone(), + ) { + Ok(iface_state) => { + e.insert(P::spawn(iface_state)); + } + Err(err) => { + tracing::error!("failed to create `InterfaceState`: {}", err) + } } } } - } - Ok(IfEvent::Down(inet)) => { - if let Some(handle) = self.if_tasks.remove(&inet.addr()) { - tracing::info!(instance=%inet.addr(), "dropping instance"); + Ok(IfEvent::Down(inet)) => { + if let Some(handle) = self.if_tasks.remove(&inet.addr()) { + tracing::info!(instance=%inet.addr(), "dropping instance"); - handle.abort(); + handle.abort(); + } } + Err(err) => tracing::error!("if watch returned an error: {}", err), } - Err(err) => tracing::error!("if watch returned an error: {}", err), } - } - // Emit discovered event. - let mut discovered = Vec::new(); - - while let Poll::Ready(Some((peer, addr, expiration))) = - self.query_response_receiver.poll_next_unpin(cx) - { - if let Some((_, _, cur_expires)) = self - .discovered_nodes - .iter_mut() - .find(|(p, a, _)| *p == peer && *a == addr) + // Emit discovered event. + let mut discovered = Vec::new(); + + while let Poll::Ready(Some((peer, addr, expiration))) = + self.query_response_receiver.poll_next_unpin(cx) { - *cur_expires = cmp::max(*cur_expires, expiration); - } else { - tracing::info!(%peer, address=%addr, "discovered peer on address"); - self.discovered_nodes.push((peer, addr.clone(), expiration)); - discovered.push((peer, addr)); + if let Some((_, _, cur_expires)) = self + .discovered_nodes + .iter_mut() + .find(|(p, a, _)| *p == peer && *a == addr) + { + *cur_expires = cmp::max(*cur_expires, expiration); + } else { + tracing::info!(%peer, address=%addr, "discovered peer on address"); + self.discovered_nodes.push((peer, addr.clone(), expiration)); + discovered.push((peer, addr.clone())); + + self.pending_events + .push_back(ToSwarm::NewExternalAddrOfPeer { + peer_id: peer, + address: addr, + }); + } } - } - if !discovered.is_empty() { - let event = Event::Discovered(discovered); - return Poll::Ready(ToSwarm::GenerateEvent(event)); - } - // Emit expired event. - let now = Instant::now(); - let mut closest_expiration = None; - let mut expired = Vec::new(); - self.discovered_nodes.retain(|(peer, addr, expiration)| { - if *expiration <= now { - tracing::info!(%peer, address=%addr, "expired peer on address"); - expired.push((*peer, addr.clone())); - return false; + if !discovered.is_empty() { + let event = Event::Discovered(discovered); + // Push to the front of the queue so that the behavior event is reported before + // the individual discovered addresses. + self.pending_events + .push_front(ToSwarm::GenerateEvent(event)); + continue; + } + // Emit expired event. + let now = Instant::now(); + let mut closest_expiration = None; + let mut expired = Vec::new(); + self.discovered_nodes.retain(|(peer, addr, expiration)| { + if *expiration <= now { + tracing::info!(%peer, address=%addr, "expired peer on address"); + expired.push((*peer, addr.clone())); + return false; + } + closest_expiration = + Some(closest_expiration.unwrap_or(*expiration).min(*expiration)); + true + }); + if !expired.is_empty() { + let event = Event::Expired(expired); + self.pending_events.push_back(ToSwarm::GenerateEvent(event)); + continue; + } + if let Some(closest_expiration) = closest_expiration { + let mut timer = P::Timer::at(closest_expiration); + let _ = Pin::new(&mut timer).poll_next(cx); + + self.closest_expiration = Some(timer); } - closest_expiration = Some(closest_expiration.unwrap_or(*expiration).min(*expiration)); - true - }); - if !expired.is_empty() { - let event = Event::Expired(expired); - return Poll::Ready(ToSwarm::GenerateEvent(event)); - } - if let Some(closest_expiration) = closest_expiration { - let mut timer = P::Timer::at(closest_expiration); - let _ = Pin::new(&mut timer).poll_next(cx); - self.closest_expiration = Some(timer); + return Poll::Pending; } - Poll::Pending } } diff --git a/protocols/perf/Cargo.toml b/protocols/perf/Cargo.toml index cd499a8c949..645abc9bcfb 100644 --- a/protocols/perf/Cargo.toml +++ b/protocols/perf/Cargo.toml @@ -19,9 +19,7 @@ futures-timer = "3.0" web-time = { workspace = true } libp2p = { workspace = true, features = ["tokio", "tcp", "quic", "tls", "yamux", "dns"] } libp2p-core = { workspace = true } -libp2p-dns = { workspace = true, features = ["tokio"] } libp2p-identity = { workspace = true, features = ["rand"] } -libp2p-quic = { workspace = true, features = ["tokio"] } libp2p-swarm = { workspace = true, features = ["macros", "tokio"] } libp2p-tcp = { workspace = true, features = ["tokio"] } libp2p-tls = { workspace = true } @@ -34,7 +32,6 @@ tracing-subscriber = { workspace = true, features = ["env-filter"] } tokio = { workspace = true, features = ["macros", "rt", "rt-multi-thread"] } [dev-dependencies] -rand = "0.8" libp2p-swarm-test = { path = "../../swarm-test" } # Passing arguments to the docsrs builder in order to properly document cfg's. diff --git a/protocols/ping/Cargo.toml b/protocols/ping/Cargo.toml index 0fad9678aec..83f3b6460c9 100644 --- a/protocols/ping/Cargo.toml +++ b/protocols/ping/Cargo.toml @@ -11,7 +11,6 @@ keywords = ["peer-to-peer", "libp2p", "networking"] categories = ["network-programming", "asynchronous"] [dependencies] -either = "1.11.0" futures = { workspace = true } futures-timer = "3.0.3" web-time = { workspace = true } @@ -25,7 +24,6 @@ tracing = { workspace = true } libp2p-swarm = { workspace = true, features = ["macros"] } libp2p-swarm-test = { path = "../../swarm-test" } quickcheck = { workspace = true } -tracing-subscriber = { workspace = true, features = ["env-filter"] } tokio = {workspace = true, features = ["rt", "macros"]} # Passing arguments to the docsrs builder in order to properly document cfg's. diff --git a/protocols/rendezvous/Cargo.toml b/protocols/rendezvous/Cargo.toml index 53a579918c5..9521913cd30 100644 --- a/protocols/rendezvous/Cargo.toml +++ b/protocols/rendezvous/Cargo.toml @@ -29,12 +29,7 @@ tracing = { workspace = true } [dev-dependencies] libp2p-swarm = { workspace = true, features = ["macros", "tokio"] } -libp2p-noise = { workspace = true } -libp2p-ping = { workspace = true } -libp2p-identify = { workspace = true } libp2p-swarm-test = { path = "../../swarm-test" } -libp2p-tcp = { workspace = true, features = ["tokio"] } -libp2p-yamux = { workspace = true } rand = "0.8" tokio = { workspace = true, features = [ "rt-multi-thread", "time", "macros", "sync", "process", "fs", "net" ] } tracing-subscriber = { workspace = true, features = ["env-filter"] } diff --git a/protocols/request-response/Cargo.toml b/protocols/request-response/Cargo.toml index 48ef4c2c066..5cd711dd051 100644 --- a/protocols/request-response/Cargo.toml +++ b/protocols/request-response/Cargo.toml @@ -14,7 +14,6 @@ categories = ["network-programming", "asynchronous"] async-trait = "0.1" cbor4ii = { version = "0.3.2", features = ["serde1", "use_std"], optional = true } futures = { workspace = true } -web-time = { workspace = true } libp2p-core = { workspace = true } libp2p-swarm = { workspace = true } libp2p-identity = { workspace = true } @@ -23,7 +22,6 @@ serde = { version = "1.0", optional = true} serde_json = { version = "1.0.117", optional = true } smallvec = "1.13.2" tracing = { workspace = true } -futures-timer = "3.0.3" futures-bounded = { workspace = true } [features] @@ -33,9 +31,6 @@ cbor = ["dep:serde", "dep:cbor4ii", "libp2p-swarm/macros"] [dev-dependencies] anyhow = "1.0.86" async-std = { version = "1.6.2", features = ["attributes"] } -libp2p-noise = { workspace = true } -libp2p-tcp = { workspace = true, features = ["async-io"] } -libp2p-yamux = { workspace = true } rand = "0.8" libp2p-swarm-test = { path = "../../swarm-test" } futures_ringbuf = "0.4.0" diff --git a/swarm-derive/Cargo.toml b/swarm-derive/Cargo.toml index 91c643a459d..febd2a6455a 100644 --- a/swarm-derive/Cargo.toml +++ b/swarm-derive/Cargo.toml @@ -17,7 +17,6 @@ proc-macro = true heck = "0.5" quote = "1.0" syn = { version = "2.0.66", default-features = false, features = ["clone-impls", "derive", "parsing", "printing", "proc-macro"] } -proc-macro2 = "1.0" # Passing arguments to the docsrs builder in order to properly document cfg's. # More information: https://docs.rs/about/builds#cross-compiling diff --git a/swarm-test/Cargo.toml b/swarm-test/Cargo.toml index 7ac7c900deb..4a0d5ee8c71 100644 --- a/swarm-test/Cargo.toml +++ b/swarm-test/Cargo.toml @@ -20,7 +20,6 @@ libp2p-swarm = { workspace = true } libp2p-tcp = { workspace = true } libp2p-yamux = { workspace = true } futures = { workspace = true } -rand = "0.8.5" tracing = { workspace = true } futures-timer = "3.0.3" diff --git a/swarm/Cargo.toml b/swarm/Cargo.toml index 4c3b8821ed6..cf027d96ec0 100644 --- a/swarm/Cargo.toml +++ b/swarm/Cargo.toml @@ -15,7 +15,7 @@ either = "1.11.0" fnv = "1.0" futures = { workspace = true } futures-timer = "3.0.3" -getrandom = { version = "0.2.15", features = ["js"], optional = true } # Explicit dependency to be used in `wasm-bindgen` feature +getrandom = { workspace = true, features = ["js"], optional = true } # Explicit dependency to be used in `wasm-bindgen` feature web-time = { workspace = true } libp2p-core = { workspace = true } libp2p-identity = { workspace = true } diff --git a/transports/noise/Cargo.toml b/transports/noise/Cargo.toml index 8824adcc50c..d0e2f9004ce 100644 --- a/transports/noise/Cargo.toml +++ b/transports/noise/Cargo.toml @@ -11,7 +11,6 @@ repository = "https://github.com/libp2p/rust-libp2p" [dependencies] asynchronous-codec = { workspace = true } bytes = "1" -curve25519-dalek = "4.1.2" futures = { workspace = true } libp2p-core = { workspace = true } libp2p-identity = { workspace = true, features = ["ed25519"] } @@ -20,7 +19,6 @@ multihash = { workspace = true } once_cell = "1.19.0" quick-protobuf = "0.8" rand = "0.8.3" -sha2 = "0.10.8" static_assertions = "1" thiserror = { workspace = true } tracing = { workspace = true } diff --git a/transports/plaintext/Cargo.toml b/transports/plaintext/Cargo.toml index 47a3191baa9..9e1e5449158 100644 --- a/transports/plaintext/Cargo.toml +++ b/transports/plaintext/Cargo.toml @@ -23,7 +23,6 @@ quick-protobuf-codec = { workspace = true } [dev-dependencies] libp2p-identity = { workspace = true, features = ["ed25519", "rand"] } quickcheck = { workspace = true } -rand = "0.8" futures_ringbuf = "0.4.0" tracing-subscriber = { workspace = true, features = ["env-filter"] } diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 17d5014b974..bff0a024bd4 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -10,15 +10,13 @@ license = "MIT" [dependencies] async-std = { version = "1.12.0", optional = true } -bytes = "1.6.0" futures = { workspace = true } futures-timer = "3.0.3" if-watch = "3.2.0" libp2p-core = { workspace = true } libp2p-tls = { workspace = true } libp2p-identity = { workspace = true } -parking_lot = "0.12.3" -quinn = { version = "0.11.2", default-features = false, features = ["rustls", "futures-io"] } +quinn = { version = "0.11.6", default-features = false, features = ["rustls", "futures-io"] } rand = "0.8.5" rustls = { version = "0.23.9", default-features = false } thiserror = { workspace = true } diff --git a/transports/tcp/Cargo.toml b/transports/tcp/Cargo.toml index 03e7fac491c..baa3b2f46dd 100644 --- a/transports/tcp/Cargo.toml +++ b/transports/tcp/Cargo.toml @@ -17,7 +17,6 @@ futures-timer = "3.0" if-watch = "3.2.0" libc = "0.2.155" libp2p-core = { workspace = true } -libp2p-identity = { workspace = true } socket2 = { version = "0.5.7", features = ["all"] } tokio = { workspace = true, default-features = false, features = ["net"], optional = true } tracing = { workspace = true } @@ -28,7 +27,6 @@ async-io = ["dep:async-io", "if-watch/smol"] [dev-dependencies] async-std = { version = "1.6.5", features = ["attributes"] } -libp2p-identity = { workspace = true, features = ["rand"] } tokio = { workspace = true, features = ["full"] } tracing-subscriber = { workspace = true, features = ["env-filter"] } diff --git a/transports/tls/Cargo.toml b/transports/tls/Cargo.toml index fce76e2aa79..7702a4361b1 100644 --- a/transports/tls/Cargo.toml +++ b/transports/tls/Cargo.toml @@ -28,7 +28,6 @@ features = ["ring", "std"] # Must enable this to allow for custom verification c [dev-dependencies] -hex = "0.4.3" hex-literal = "0.4.1" libp2p-core = { workspace = true } libp2p-identity = { workspace = true, features = ["ed25519", "rsa", "secp256k1", "ecdsa", "rand"] } diff --git a/transports/webrtc-websys/Cargo.toml b/transports/webrtc-websys/Cargo.toml index 4663913c849..6d42d74f610 100644 --- a/transports/webrtc-websys/Cargo.toml +++ b/transports/webrtc-websys/Cargo.toml @@ -14,7 +14,7 @@ publish = true [dependencies] bytes = "1" futures = { workspace = true } -getrandom = { version = "0.2.15", features = ["js"] } +getrandom = { workspace = true, features = ["js"] } hex = "0.4.3" js-sys = { version = "0.3" } libp2p-core = { workspace = true } diff --git a/transports/webrtc/Cargo.toml b/transports/webrtc/Cargo.toml index 4197a9419d8..f3d2e57147e 100644 --- a/transports/webrtc/Cargo.toml +++ b/transports/webrtc/Cargo.toml @@ -12,7 +12,6 @@ categories = ["network-programming", "asynchronous"] [dependencies] async-trait = "0.1" -bytes = "1" futures = { workspace = true } futures-timer = "3" hex = "0.4" @@ -24,10 +23,8 @@ libp2p-webrtc-utils = { workspace = true } multihash = { workspace = true } rand = "0.8" rcgen = { workspace = true } -serde = { version = "1.0", features = ["derive"] } stun = "0.6" thiserror = { workspace = true } -tinytemplate = "1.2" tokio = { workspace = true, features = ["net"], optional = true } tokio-util = { version = "0.7", features = ["compat"], optional = true } tracing = { workspace = true } diff --git a/transports/websocket-websys/Cargo.toml b/transports/websocket-websys/Cargo.toml index 1e604ba0478..f33703c1884 100644 --- a/transports/websocket-websys/Cargo.toml +++ b/transports/websocket-websys/Cargo.toml @@ -16,7 +16,6 @@ futures = { workspace = true } js-sys = "0.3.69" libp2p-core = { workspace = true } tracing = { workspace = true } -parking_lot = "0.12.3" send_wrapper = "0.6.0" thiserror = { workspace = true } wasm-bindgen = "0.2.90" diff --git a/wasm-tests/webtransport-tests/Cargo.toml b/wasm-tests/webtransport-tests/Cargo.toml index d7db378ab1a..593743d1617 100644 --- a/wasm-tests/webtransport-tests/Cargo.toml +++ b/wasm-tests/webtransport-tests/Cargo.toml @@ -10,7 +10,7 @@ release = false [dependencies] futures = { workspace = true } -getrandom = { version = "0.2.15", features = ["js"] } +getrandom = { workspace = true, features = ["js"] } libp2p-core = { workspace = true } libp2p-identity = { workspace = true, features = ["rand"] } libp2p-noise = { workspace = true } From fe71e65e74e48dcfc94a2abb2ba917434e13d2ee Mon Sep 17 00:00:00 2001 From: manasnagaraj Date: Wed, 25 Dec 2024 15:04:56 -0500 Subject: [PATCH 07/16] requested changes --- protocols/gossipsub/src/behaviour.rs | 30 ++++++++++++++++++++-------- 1 file changed, 22 insertions(+), 8 deletions(-) diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index f16099ee0fb..dacfe33b74e 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -23,6 +23,7 @@ use std::{ collections::{BTreeSet, HashMap, HashSet, VecDeque}, fmt, fmt::Debug, + io::Error, net::IpAddr, task::{Context, Poll}, time::Duration, @@ -1815,12 +1816,14 @@ where raw_message: &RawMessage, reject_reason: RejectReason, ) { - if let Some((peer_score, ..)) = &mut self.peer_score { - if let Some(metrics) = self.metrics.as_mut() { - metrics.register_invalid_message(&raw_message.topic); - } + if let Some(metrics) = self.metrics.as_mut() { + metrics.register_invalid_message(&raw_message.topic); + } + + let message = self.data_transform.inbound_transform(raw_message.clone()); - if let Ok(message) = self.data_transform.inbound_transform(raw_message.clone()) { + match (&mut self.peer_score, message) { + (Some((peer_score, ..)), Ok(message)) => { let message_id = self.config.message_id(&message); peer_score.reject_message( @@ -1832,12 +1835,20 @@ where self.gossip_promises .reject_message(&message_id, &reject_reason); - } else { + } + (Some((peer_score, ..)), Err(_)) => { // The message is invalid, we reject it ignoring any gossip promises. If a peer is // advertising this message via an IHAVE and it's invalid it will be double // penalized, one for sending us an invalid and again for breaking a promise. peer_score.reject_invalid_message(propagation_source, &raw_message.topic); } + (None, Ok(message)) => { + // Valid transformation without peer scoring + let message_id = self.config.message_id(&message); + self.gossip_promises + .reject_message(&message_id, &reject_reason); + } + (None, Err(_)) => {} } } @@ -2681,7 +2692,7 @@ where // forward the message to peers for peer_id in recipient_peers.iter() { - + if let Some(peer) = self.connected_peers.get_mut(peer_id) { if peer.dont_send.get(msg_id).is_some() { tracing::debug!(%peer_id, message=%msg_id, "Peer doesn't want message"); continue; @@ -2696,10 +2707,11 @@ where timeout: Delay::new(self.config.forward_queue_duration()), }, ); - + } } tracing::debug!("Completed forwarding message"); true + } /// Constructs a [`RawMessage`] performing message signing if required. @@ -3303,11 +3315,13 @@ where if let Some((peer_score, _, delay)) = &mut self.peer_score { if delay.poll_unpin(cx).is_ready() { peer_score.refresh_scores(); + delay.reset(peer_score.params.decay_interval); } } if self.heartbeat.poll_unpin(cx).is_ready() { self.heartbeat(); + self.heartbeat.reset(self.config.heartbeat_interval()); } Poll::Pending From fc0f2508fa95744ccafb2b1b1019038b4ae77409 Mon Sep 17 00:00:00 2001 From: manasnagaraj Date: Wed, 25 Dec 2024 15:18:27 -0500 Subject: [PATCH 08/16] removing .idea files --- .idea/workspace.xml | 503 -------------------------------------------- 1 file changed, 503 deletions(-) delete mode 100644 .idea/workspace.xml diff --git a/.idea/workspace.xml b/.idea/workspace.xml deleted file mode 100644 index 066f3f5b348..00000000000 --- a/.idea/workspace.xml +++ /dev/null @@ -1,503 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - { - "associatedIndex": 5 -} - - - - { - "keyToString": { - "Cargo.Test behaviour::tests::doesnt_forward_idontwant.executor": "Run", - "Cargo.Test behaviour::tests::test_slow_peer_returns_failed_ihave_handling.executor": "Run", - "Cargo.Test rust-libp2p.executor": "Run", - "RunOnceActivity.ShowReadmeOnStart": "true", - "RunOnceActivity.rust.reset.selective.auto.import": "true", - "git-widget-placeholder": "Merging implement-gossipsub-1.2-beta", - "last_opened_file_path": "/Users/manasnagaraj/projects/oss/rust-libp2p", - "node.js.detected.package.eslint": "true", - "node.js.detected.package.tslint": "true", - "node.js.selected.package.eslint": "(autodetect)", - "node.js.selected.package.tslint": "(autodetect)", - "nodejs_package_manager_path": "npm", - "org.rust.cargo.project.model.PROJECT_DISCOVERY": "true", - "org.rust.cargo.project.model.impl.CargoExternalSystemProjectAware.subscribe.first.balloon": "", - "org.rust.first.attach.projects": "true", - "vue.rearranger.settings.migration": "true" - } -} - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - 1728685449599 - - - - - - \ No newline at end of file From 530e91118e0d17f581a6a2af5abc8b12e96d834b Mon Sep 17 00:00:00 2001 From: manasnagaraj Date: Wed, 25 Dec 2024 15:28:52 -0500 Subject: [PATCH 09/16] resolving dependency issues --- Cargo.lock | 10 ++++++++++ protocols/gossipsub/Cargo.toml | 1 + 2 files changed, 11 insertions(+) diff --git a/Cargo.lock b/Cargo.lock index 308c3c45e5c..55f69c070f7 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1926,6 +1926,15 @@ dependencies = [ "allocator-api2", ] +[[package]] +name = "hashlink" +version = "0.9.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6ba4ff7128dee98c7dc9794b6a411377e1404dba1c97deb8d1a55297bd25d8af" +dependencies = [ + "hashbrown 0.14.3", +] + [[package]] name = "heck" version = "0.4.1" @@ -2858,6 +2867,7 @@ dependencies = [ "futures", "futures-timer", "getrandom 0.2.15", + "hashlink", "hex_fmt", "libp2p-core", "libp2p-identity", diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml index d48993b331e..0262348429e 100644 --- a/protocols/gossipsub/Cargo.toml +++ b/protocols/gossipsub/Cargo.toml @@ -24,6 +24,7 @@ fnv = "1.0.7" futures = { workspace = true } futures-timer = "3.0.2" getrandom = { workspace = true } +hashlink = { workspace = true} hex_fmt = "0.3.0" web-time = { workspace = true } libp2p-core = { workspace = true } From 94651dbeda993a1ef1a7b8915323a75c2868fe7c Mon Sep 17 00:00:00 2001 From: manasnagaraj Date: Wed, 25 Dec 2024 15:41:30 -0500 Subject: [PATCH 10/16] clippy changes --- protocols/gossipsub/src/behaviour.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index dacfe33b74e..e30a85acb98 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -23,7 +23,6 @@ use std::{ collections::{BTreeSet, HashMap, HashSet, VecDeque}, fmt, fmt::Debug, - io::Error, net::IpAddr, task::{Context, Poll}, time::Duration, From d052f1da1c9a1aa46c9147c1102474fa97e6a695 Mon Sep 17 00:00:00 2001 From: manasnagaraj Date: Wed, 25 Dec 2024 16:14:59 -0500 Subject: [PATCH 11/16] generated proto using pb-rs and fixes --- protocols/gossipsub/src/behaviour.rs | 1 - .../gossipsub/src/generated/gossipsub/pb.rs | 358 +++++++++--------- protocols/gossipsub/src/types.rs | 3 +- 3 files changed, 181 insertions(+), 181 deletions(-) diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index e30a85acb98..69d34d41353 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -2710,7 +2710,6 @@ where } tracing::debug!("Completed forwarding message"); true - } /// Constructs a [`RawMessage`] performing message signing if required. diff --git a/protocols/gossipsub/src/generated/gossipsub/pb.rs b/protocols/gossipsub/src/generated/gossipsub/pb.rs index c6a86fcfe92..24ac80d2755 100644 --- a/protocols/gossipsub/src/generated/gossipsub/pb.rs +++ b/protocols/gossipsub/src/generated/gossipsub/pb.rs @@ -40,9 +40,9 @@ impl<'a> MessageRead<'a> for RPC { impl MessageWrite for RPC { fn get_size(&self) -> usize { 0 - + self.subscriptions.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() - + self.publish.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() - + self.control.as_ref().map_or(0, |m| 1 + sizeof_len((m).get_size())) + + self.subscriptions.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() + + self.publish.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() + + self.control.as_ref().map_or(0, |m| 1 + sizeof_len((m).get_size())) } fn write_message(&self, w: &mut Writer) -> Result<()> { @@ -55,43 +55,43 @@ impl MessageWrite for RPC { pub mod mod_RPC { - use super::*; - - #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Debug, Default, PartialEq, Clone)] - pub struct SubOpts { - pub subscribe: Option, - pub topic_id: Option, - } - - impl<'a> MessageRead<'a> for SubOpts { - fn from_reader(r: &mut BytesReader, bytes: &'a [u8]) -> Result { - let mut msg = Self::default(); - while !r.is_eof() { - match r.next_tag(bytes) { - Ok(8) => msg.subscribe = Some(r.read_bool(bytes)?), - Ok(18) => msg.topic_id = Some(r.read_string(bytes)?.to_owned()), - Ok(t) => { r.read_unknown(bytes, t)?; } - Err(e) => return Err(e), - } +use super::*; + +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Debug, Default, PartialEq, Clone)] +pub struct SubOpts { + pub subscribe: Option, + pub topic_id: Option, +} + +impl<'a> MessageRead<'a> for SubOpts { + fn from_reader(r: &mut BytesReader, bytes: &'a [u8]) -> Result { + let mut msg = Self::default(); + while !r.is_eof() { + match r.next_tag(bytes) { + Ok(8) => msg.subscribe = Some(r.read_bool(bytes)?), + Ok(18) => msg.topic_id = Some(r.read_string(bytes)?.to_owned()), + Ok(t) => { r.read_unknown(bytes, t)?; } + Err(e) => return Err(e), } - Ok(msg) } + Ok(msg) } +} - impl MessageWrite for SubOpts { - fn get_size(&self) -> usize { - 0 - + self.subscribe.as_ref().map_or(0, |m| 1 + sizeof_varint(*(m) as u64)) - + self.topic_id.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) - } +impl MessageWrite for SubOpts { + fn get_size(&self) -> usize { + 0 + + self.subscribe.as_ref().map_or(0, |m| 1 + sizeof_varint(*(m) as u64)) + + self.topic_id.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + } - fn write_message(&self, w: &mut Writer) -> Result<()> { - if let Some(ref s) = self.subscribe { w.write_with_tag(8, |w| w.write_bool(*s))?; } - if let Some(ref s) = self.topic_id { w.write_with_tag(18, |w| w.write_string(&**s))?; } - Ok(()) - } + fn write_message(&self, w: &mut Writer) -> Result<()> { + if let Some(ref s) = self.subscribe { w.write_with_tag(8, |w| w.write_bool(*s))?; } + if let Some(ref s) = self.topic_id { w.write_with_tag(18, |w| w.write_string(&**s))?; } + Ok(()) } +} } @@ -128,12 +128,12 @@ impl<'a> MessageRead<'a> for Message { impl MessageWrite for Message { fn get_size(&self) -> usize { 0 - + self.from.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) - + self.data.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) - + self.seqno.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) - + 1 + sizeof_len((&self.topic).len()) - + self.signature.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) - + self.key.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + + self.from.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + + self.data.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + + self.seqno.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + + 1 + sizeof_len((&self.topic).len()) + + self.signature.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + + self.key.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) } fn write_message(&self, w: &mut Writer) -> Result<()> { @@ -178,11 +178,11 @@ impl<'a> MessageRead<'a> for ControlMessage { impl MessageWrite for ControlMessage { fn get_size(&self) -> usize { 0 - + self.ihave.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() - + self.iwant.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() - + self.graft.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() - + self.prune.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() - + self.idontwant.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() + + self.ihave.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() + + self.iwant.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() + + self.graft.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() + + self.prune.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() + + self.idontwant.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() } fn write_message(&self, w: &mut Writer) -> Result<()> { @@ -220,8 +220,8 @@ impl<'a> MessageRead<'a> for ControlIHave { impl MessageWrite for ControlIHave { fn get_size(&self) -> usize { 0 - + self.topic_id.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) - + self.message_ids.iter().map(|s| 1 + sizeof_len((s).len())).sum::() + + self.topic_id.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + + self.message_ids.iter().map(|s| 1 + sizeof_len((s).len())).sum::() } fn write_message(&self, w: &mut Writer) -> Result<()> { @@ -254,7 +254,7 @@ impl<'a> MessageRead<'a> for ControlIWant { impl MessageWrite for ControlIWant { fn get_size(&self) -> usize { 0 - + self.message_ids.iter().map(|s| 1 + sizeof_len((s).len())).sum::() + + self.message_ids.iter().map(|s| 1 + sizeof_len((s).len())).sum::() } fn write_message(&self, w: &mut Writer) -> Result<()> { @@ -286,7 +286,7 @@ impl<'a> MessageRead<'a> for ControlGraft { impl MessageWrite for ControlGraft { fn get_size(&self) -> usize { 0 - + self.topic_id.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + + self.topic_id.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) } fn write_message(&self, w: &mut Writer) -> Result<()> { @@ -322,9 +322,9 @@ impl<'a> MessageRead<'a> for ControlPrune { impl MessageWrite for ControlPrune { fn get_size(&self) -> usize { 0 - + self.topic_id.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) - + self.peers.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() - + self.backoff.as_ref().map_or(0, |m| 1 + sizeof_varint(*(m) as u64)) + + self.topic_id.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + + self.peers.iter().map(|s| 1 + sizeof_len((s).get_size())).sum::() + + self.backoff.as_ref().map_or(0, |m| 1 + sizeof_varint(*(m) as u64)) } fn write_message(&self, w: &mut Writer) -> Result<()> { @@ -358,7 +358,7 @@ impl<'a> MessageRead<'a> for ControlIDontWant { impl MessageWrite for ControlIDontWant { fn get_size(&self) -> usize { 0 - + self.message_ids.iter().map(|s| 1 + sizeof_len((s).len())).sum::() + + self.message_ids.iter().map(|s| 1 + sizeof_len((s).len())).sum::() } fn write_message(&self, w: &mut Writer) -> Result<()> { @@ -392,8 +392,8 @@ impl<'a> MessageRead<'a> for PeerInfo { impl MessageWrite for PeerInfo { fn get_size(&self) -> usize { 0 - + self.peer_id.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) - + self.signed_peer_record.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + + self.peer_id.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + + self.signed_peer_record.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) } fn write_message(&self, w: &mut Writer) -> Result<()> { @@ -430,9 +430,9 @@ impl<'a> MessageRead<'a> for TopicDescriptor { impl MessageWrite for TopicDescriptor { fn get_size(&self) -> usize { 0 - + self.name.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) - + self.auth.as_ref().map_or(0, |m| 1 + sizeof_len((m).get_size())) - + self.enc.as_ref().map_or(0, |m| 1 + sizeof_len((m).get_size())) + + self.name.as_ref().map_or(0, |m| 1 + sizeof_len((m).len())) + + self.auth.as_ref().map_or(0, |m| 1 + sizeof_len((m).get_size())) + + self.enc.as_ref().map_or(0, |m| 1 + sizeof_len((m).get_size())) } fn write_message(&self, w: &mut Writer) -> Result<()> { @@ -445,159 +445,159 @@ impl MessageWrite for TopicDescriptor { pub mod mod_TopicDescriptor { - use super::*; - - #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Debug, Default, PartialEq, Clone)] - pub struct AuthOpts { - pub mode: Option, - pub keys: Vec>, - } - - impl<'a> MessageRead<'a> for AuthOpts { - fn from_reader(r: &mut BytesReader, bytes: &'a [u8]) -> Result { - let mut msg = Self::default(); - while !r.is_eof() { - match r.next_tag(bytes) { - Ok(8) => msg.mode = Some(r.read_enum(bytes)?), - Ok(18) => msg.keys.push(r.read_bytes(bytes)?.to_owned()), - Ok(t) => { r.read_unknown(bytes, t)?; } - Err(e) => return Err(e), - } +use super::*; + +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Debug, Default, PartialEq, Clone)] +pub struct AuthOpts { + pub mode: Option, + pub keys: Vec>, +} + +impl<'a> MessageRead<'a> for AuthOpts { + fn from_reader(r: &mut BytesReader, bytes: &'a [u8]) -> Result { + let mut msg = Self::default(); + while !r.is_eof() { + match r.next_tag(bytes) { + Ok(8) => msg.mode = Some(r.read_enum(bytes)?), + Ok(18) => msg.keys.push(r.read_bytes(bytes)?.to_owned()), + Ok(t) => { r.read_unknown(bytes, t)?; } + Err(e) => return Err(e), } - Ok(msg) } + Ok(msg) } +} - impl MessageWrite for AuthOpts { - fn get_size(&self) -> usize { - 0 - + self.mode.as_ref().map_or(0, |m| 1 + sizeof_varint(*(m) as u64)) - + self.keys.iter().map(|s| 1 + sizeof_len((s).len())).sum::() - } +impl MessageWrite for AuthOpts { + fn get_size(&self) -> usize { + 0 + + self.mode.as_ref().map_or(0, |m| 1 + sizeof_varint(*(m) as u64)) + + self.keys.iter().map(|s| 1 + sizeof_len((s).len())).sum::() + } - fn write_message(&self, w: &mut Writer) -> Result<()> { - if let Some(ref s) = self.mode { w.write_with_tag(8, |w| w.write_enum(*s as i32))?; } - for s in &self.keys { w.write_with_tag(18, |w| w.write_bytes(&**s))?; } - Ok(()) - } + fn write_message(&self, w: &mut Writer) -> Result<()> { + if let Some(ref s) = self.mode { w.write_with_tag(8, |w| w.write_enum(*s as i32))?; } + for s in &self.keys { w.write_with_tag(18, |w| w.write_bytes(&**s))?; } + Ok(()) } +} - pub mod mod_AuthOpts { +pub mod mod_AuthOpts { - #[derive(Debug, PartialEq, Eq, Clone, Copy)] - pub enum AuthMode { - NONE = 0, - KEY = 1, - WOT = 2, - } +#[derive(Debug, PartialEq, Eq, Clone, Copy)] +pub enum AuthMode { + NONE = 0, + KEY = 1, + WOT = 2, +} - impl Default for AuthMode { - fn default() -> Self { - AuthMode::NONE - } - } +impl Default for AuthMode { + fn default() -> Self { + AuthMode::NONE + } +} - impl From for AuthMode { - fn from(i: i32) -> Self { - match i { - 0 => AuthMode::NONE, - 1 => AuthMode::KEY, - 2 => AuthMode::WOT, - _ => Self::default(), - } - } +impl From for AuthMode { + fn from(i: i32) -> Self { + match i { + 0 => AuthMode::NONE, + 1 => AuthMode::KEY, + 2 => AuthMode::WOT, + _ => Self::default(), } + } +} - impl<'a> From<&'a str> for AuthMode { - fn from(s: &'a str) -> Self { - match s { - "NONE" => AuthMode::NONE, - "KEY" => AuthMode::KEY, - "WOT" => AuthMode::WOT, - _ => Self::default(), - } - } +impl<'a> From<&'a str> for AuthMode { + fn from(s: &'a str) -> Self { + match s { + "NONE" => AuthMode::NONE, + "KEY" => AuthMode::KEY, + "WOT" => AuthMode::WOT, + _ => Self::default(), } - } +} - #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Debug, Default, PartialEq, Clone)] - pub struct EncOpts { - pub mode: Option, - pub key_hashes: Vec>, - } +} - impl<'a> MessageRead<'a> for EncOpts { - fn from_reader(r: &mut BytesReader, bytes: &'a [u8]) -> Result { - let mut msg = Self::default(); - while !r.is_eof() { - match r.next_tag(bytes) { - Ok(8) => msg.mode = Some(r.read_enum(bytes)?), - Ok(18) => msg.key_hashes.push(r.read_bytes(bytes)?.to_owned()), - Ok(t) => { r.read_unknown(bytes, t)?; } - Err(e) => return Err(e), - } +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Debug, Default, PartialEq, Clone)] +pub struct EncOpts { + pub mode: Option, + pub key_hashes: Vec>, +} + +impl<'a> MessageRead<'a> for EncOpts { + fn from_reader(r: &mut BytesReader, bytes: &'a [u8]) -> Result { + let mut msg = Self::default(); + while !r.is_eof() { + match r.next_tag(bytes) { + Ok(8) => msg.mode = Some(r.read_enum(bytes)?), + Ok(18) => msg.key_hashes.push(r.read_bytes(bytes)?.to_owned()), + Ok(t) => { r.read_unknown(bytes, t)?; } + Err(e) => return Err(e), } - Ok(msg) } + Ok(msg) } +} - impl MessageWrite for EncOpts { - fn get_size(&self) -> usize { - 0 - + self.mode.as_ref().map_or(0, |m| 1 + sizeof_varint(*(m) as u64)) - + self.key_hashes.iter().map(|s| 1 + sizeof_len((s).len())).sum::() - } +impl MessageWrite for EncOpts { + fn get_size(&self) -> usize { + 0 + + self.mode.as_ref().map_or(0, |m| 1 + sizeof_varint(*(m) as u64)) + + self.key_hashes.iter().map(|s| 1 + sizeof_len((s).len())).sum::() + } - fn write_message(&self, w: &mut Writer) -> Result<()> { - if let Some(ref s) = self.mode { w.write_with_tag(8, |w| w.write_enum(*s as i32))?; } - for s in &self.key_hashes { w.write_with_tag(18, |w| w.write_bytes(&**s))?; } - Ok(()) - } + fn write_message(&self, w: &mut Writer) -> Result<()> { + if let Some(ref s) = self.mode { w.write_with_tag(8, |w| w.write_enum(*s as i32))?; } + for s in &self.key_hashes { w.write_with_tag(18, |w| w.write_bytes(&**s))?; } + Ok(()) } +} - pub mod mod_EncOpts { +pub mod mod_EncOpts { - #[derive(Debug, PartialEq, Eq, Clone, Copy)] - pub enum EncMode { - NONE = 0, - SHAREDKEY = 1, - WOT = 2, - } +#[derive(Debug, PartialEq, Eq, Clone, Copy)] +pub enum EncMode { + NONE = 0, + SHAREDKEY = 1, + WOT = 2, +} - impl Default for EncMode { - fn default() -> Self { - EncMode::NONE - } - } +impl Default for EncMode { + fn default() -> Self { + EncMode::NONE + } +} - impl From for EncMode { - fn from(i: i32) -> Self { - match i { - 0 => EncMode::NONE, - 1 => EncMode::SHAREDKEY, - 2 => EncMode::WOT, - _ => Self::default(), - } - } +impl From for EncMode { + fn from(i: i32) -> Self { + match i { + 0 => EncMode::NONE, + 1 => EncMode::SHAREDKEY, + 2 => EncMode::WOT, + _ => Self::default(), } + } +} - impl<'a> From<&'a str> for EncMode { - fn from(s: &'a str) -> Self { - match s { - "NONE" => EncMode::NONE, - "SHAREDKEY" => EncMode::SHAREDKEY, - "WOT" => EncMode::WOT, - _ => Self::default(), - } - } +impl<'a> From<&'a str> for EncMode { + fn from(s: &'a str) -> Self { + match s { + "NONE" => EncMode::NONE, + "SHAREDKEY" => EncMode::SHAREDKEY, + "WOT" => EncMode::WOT, + _ => Self::default(), } - } +} + +} } diff --git a/protocols/gossipsub/src/types.rs b/protocols/gossipsub/src/types.rs index 41a139c73eb..efd836c72de 100644 --- a/protocols/gossipsub/src/types.rs +++ b/protocols/gossipsub/src/types.rs @@ -19,7 +19,7 @@ // DEALINGS IN THE SOFTWARE. //! A collection of types using the Gossipsub system. -use std::{collections::BTreeSet, fmt, fmt::Debug, time::Instant}; +use std::{collections::BTreeSet, fmt, fmt::Debug }; use futures_timer::Delay; use hashlink::LinkedHashMap; @@ -29,6 +29,7 @@ use prometheus_client::encoding::EncodeLabelValue; use quick_protobuf::MessageWrite; #[cfg(feature = "serde")] use serde::{Deserialize, Serialize}; +use web_time::Instant; use crate::{rpc::Sender, rpc_proto::proto, TopicHash}; From 3e3e638e6680d1adc989f92075f7c737a13e74e9 Mon Sep 17 00:00:00 2001 From: manasnagaraj Date: Wed, 25 Dec 2024 16:38:11 -0500 Subject: [PATCH 12/16] CHANGELOG.md updated and version bump --- Cargo.lock | 2 +- Cargo.toml | 2 +- protocols/gossipsub/CHANGELOG.md | 4 ++++ protocols/gossipsub/Cargo.toml | 2 +- protocols/gossipsub/src/types.rs | 2 +- 5 files changed, 8 insertions(+), 4 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 55f69c070f7..c783999c3b0 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2855,7 +2855,7 @@ dependencies = [ [[package]] name = "libp2p-gossipsub" -version = "0.48.0" +version = "0.48.1" dependencies = [ "async-channel 2.3.1", "asynchronous-codec", diff --git a/Cargo.toml b/Cargo.toml index c349e51f7ae..64de640da1b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -80,7 +80,7 @@ libp2p-core = { version = "0.42.1", path = "core" } libp2p-dcutr = { version = "0.12.1", path = "protocols/dcutr" } libp2p-dns = { version = "0.42.1", path = "transports/dns" } libp2p-floodsub = { version = "0.45.0", path = "protocols/floodsub" } -libp2p-gossipsub = { version = "0.48.0", path = "protocols/gossipsub" } +libp2p-gossipsub = { version = "0.48.1", path = "protocols/gossipsub" } libp2p-identify = { version = "0.46.1", path = "protocols/identify" } libp2p-identity = { version = "0.2.10" } libp2p-kad = { version = "0.47.1", path = "protocols/kad" } diff --git a/protocols/gossipsub/CHANGELOG.md b/protocols/gossipsub/CHANGELOG.md index e9663c4c39c..7f057c02432 100644 --- a/protocols/gossipsub/CHANGELOG.md +++ b/protocols/gossipsub/CHANGELOG.md @@ -1,3 +1,7 @@ +## 0.48.1 + +- Introduce Gossipsub v1.2 [spec](https://github.com/libp2p/specs/blob/master/pubsub/gossipsub/gossipsub-v1.2.md). See [PR 5697](https://github.com/libp2p/rust-libp2p/pull/5697) + ## 0.48.0 - Correct state inconsistencies with the mesh and fanout when unsubscribing. diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml index 0262348429e..a0534a46d33 100644 --- a/protocols/gossipsub/Cargo.toml +++ b/protocols/gossipsub/Cargo.toml @@ -3,7 +3,7 @@ name = "libp2p-gossipsub" edition = "2021" rust-version = { workspace = true } description = "Gossipsub protocol for libp2p" -version = "0.48.0" +version = "0.48.1" authors = ["Age Manning "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" diff --git a/protocols/gossipsub/src/types.rs b/protocols/gossipsub/src/types.rs index efd836c72de..621ca4ba2e8 100644 --- a/protocols/gossipsub/src/types.rs +++ b/protocols/gossipsub/src/types.rs @@ -19,7 +19,7 @@ // DEALINGS IN THE SOFTWARE. //! A collection of types using the Gossipsub system. -use std::{collections::BTreeSet, fmt, fmt::Debug }; +use std::{collections::BTreeSet, fmt, fmt::Debug}; use futures_timer::Delay; use hashlink::LinkedHashMap; From 701ff31f18f52bedbd0c31060851bb3d5ec597bf Mon Sep 17 00:00:00 2001 From: manasnagaraj Date: Thu, 26 Dec 2024 06:53:12 -0500 Subject: [PATCH 13/16] request changes and subsequent clippy changes --- Cargo.lock | 2 +- Cargo.toml | 2 +- protocols/gossipsub/CHANGELOG.md | 4 +--- protocols/gossipsub/Cargo.toml | 2 +- protocols/gossipsub/src/behaviour.rs | 6 +++--- protocols/gossipsub/src/behaviour/tests.rs | 4 ++-- protocols/gossipsub/src/handler.rs | 2 +- protocols/gossipsub/src/types.rs | 2 +- 8 files changed, 11 insertions(+), 13 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index c783999c3b0..55f69c070f7 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2855,7 +2855,7 @@ dependencies = [ [[package]] name = "libp2p-gossipsub" -version = "0.48.1" +version = "0.48.0" dependencies = [ "async-channel 2.3.1", "asynchronous-codec", diff --git a/Cargo.toml b/Cargo.toml index 64de640da1b..c349e51f7ae 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -80,7 +80,7 @@ libp2p-core = { version = "0.42.1", path = "core" } libp2p-dcutr = { version = "0.12.1", path = "protocols/dcutr" } libp2p-dns = { version = "0.42.1", path = "transports/dns" } libp2p-floodsub = { version = "0.45.0", path = "protocols/floodsub" } -libp2p-gossipsub = { version = "0.48.1", path = "protocols/gossipsub" } +libp2p-gossipsub = { version = "0.48.0", path = "protocols/gossipsub" } libp2p-identify = { version = "0.46.1", path = "protocols/identify" } libp2p-identity = { version = "0.2.10" } libp2p-kad = { version = "0.47.1", path = "protocols/kad" } diff --git a/protocols/gossipsub/CHANGELOG.md b/protocols/gossipsub/CHANGELOG.md index 7f057c02432..2c67ace4660 100644 --- a/protocols/gossipsub/CHANGELOG.md +++ b/protocols/gossipsub/CHANGELOG.md @@ -1,9 +1,7 @@ -## 0.48.1 +## 0.48.0 - Introduce Gossipsub v1.2 [spec](https://github.com/libp2p/specs/blob/master/pubsub/gossipsub/gossipsub-v1.2.md). See [PR 5697](https://github.com/libp2p/rust-libp2p/pull/5697) -## 0.48.0 - - Correct state inconsistencies with the mesh and fanout when unsubscribing. See [PR 5690](https://github.com/libp2p/rust-libp2p/pull/5690) diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml index a0534a46d33..0262348429e 100644 --- a/protocols/gossipsub/Cargo.toml +++ b/protocols/gossipsub/Cargo.toml @@ -3,7 +3,7 @@ name = "libp2p-gossipsub" edition = "2021" rust-version = { workspace = true } description = "Gossipsub protocol for libp2p" -version = "0.48.1" +version = "0.48.0" authors = ["Age Manning "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index 69d34d41353..4643f2bd97f 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -2692,7 +2692,7 @@ where // forward the message to peers for peer_id in recipient_peers.iter() { if let Some(peer) = self.connected_peers.get_mut(peer_id) { - if peer.dont_send.get(msg_id).is_some() { + if peer.dont_send.contains_key(msg_id) { tracing::debug!(%peer_id, message=%msg_id, "Peer doesn't want message"); continue; } @@ -3002,7 +3002,7 @@ where // If metrics are enabled, register the disconnection of a peer based on its protocol. if let Some(metrics) = self.metrics.as_mut() { - metrics.peer_protocol_disconnected(connected_peer.kind.clone()); + metrics.peer_protocol_disconnected(connected_peer.kind); } self.connected_peers.remove(&peer_id); @@ -3131,7 +3131,7 @@ where // We have identified the protocol this peer is using if let Some(metrics) = self.metrics.as_mut() { - metrics.peer_protocol_connected(kind.clone()); + metrics.peer_protocol_connected(kind); } if let PeerKind::NotSupported = kind { diff --git a/protocols/gossipsub/src/behaviour/tests.rs b/protocols/gossipsub/src/behaviour/tests.rs index 921fb990ab4..d3de4cd135c 100644 --- a/protocols/gossipsub/src/behaviour/tests.rs +++ b/protocols/gossipsub/src/behaviour/tests.rs @@ -101,7 +101,7 @@ where i < self.outbound, i < self.explicit, Multiaddr::empty(), - self.peer_kind.clone().or(Some(PeerKind::Gossipsubv1_1)), + self.peer_kind.or(Some(PeerKind::Gossipsubv1_1)), ); peers.push(peer); receivers.insert(peer, receiver); @@ -237,7 +237,7 @@ where gs.connected_peers.insert( peer, PeerConnections { - kind: kind.clone().unwrap_or(PeerKind::Floodsub), + kind: kind.unwrap_or(PeerKind::Floodsub), connections: vec![connection_id], topics: Default::default(), sender, diff --git a/protocols/gossipsub/src/handler.rs b/protocols/gossipsub/src/handler.rs index 2936182c3f8..e66b606896b 100644 --- a/protocols/gossipsub/src/handler.rs +++ b/protocols/gossipsub/src/handler.rs @@ -229,7 +229,7 @@ impl EnabledHandler { if let Some(peer_kind) = self.peer_kind.as_ref() { self.peer_kind_sent = true; return Poll::Ready(ConnectionHandlerEvent::NotifyBehaviour( - HandlerEvent::PeerKind(peer_kind.clone()), + HandlerEvent::PeerKind(*peer_kind), )); } } diff --git a/protocols/gossipsub/src/types.rs b/protocols/gossipsub/src/types.rs index 621ca4ba2e8..da40c2e6e7a 100644 --- a/protocols/gossipsub/src/types.rs +++ b/protocols/gossipsub/src/types.rs @@ -116,7 +116,7 @@ pub(crate) struct PeerConnections { } /// Describes the types of peers that can exist in the gossipsub context. -#[derive(Debug, Clone, PartialEq, Hash, EncodeLabelValue, Eq)] +#[derive(Debug, Clone, Copy, PartialEq, Hash, EncodeLabelValue, Eq)] #[allow(non_camel_case_types)] pub enum PeerKind { /// A gossipsub 1.2 peer. From d61d8ead7afb71568a0a82ed51e8344baee7297b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jo=C3=A3o=20Oliveira?= Date: Fri, 27 Dec 2024 11:25:25 +0000 Subject: [PATCH 14/16] Update protocols/gossipsub/CHANGELOG.md --- protocols/gossipsub/CHANGELOG.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/protocols/gossipsub/CHANGELOG.md b/protocols/gossipsub/CHANGELOG.md index 2c67ace4660..5e18f284fc4 100644 --- a/protocols/gossipsub/CHANGELOG.md +++ b/protocols/gossipsub/CHANGELOG.md @@ -1,6 +1,7 @@ ## 0.48.0 -- Introduce Gossipsub v1.2 [spec](https://github.com/libp2p/specs/blob/master/pubsub/gossipsub/gossipsub-v1.2.md). See [PR 5697](https://github.com/libp2p/rust-libp2p/pull/5697) +- Introduce Gossipsub v1.2 [spec](https://github.com/libp2p/specs/blob/master/pubsub/gossipsub/gossipsub-v1.2.md). + See [PR 5697](https://github.com/libp2p/rust-libp2p/pull/5697) - Correct state inconsistencies with the mesh and fanout when unsubscribing. See [PR 5690](https://github.com/libp2p/rust-libp2p/pull/5690) From 048fa6143b00b47e722dbd5e13ea672ff422e610 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jo=C3=A3o=20Oliveira?= Date: Fri, 27 Dec 2024 11:25:33 +0000 Subject: [PATCH 15/16] Update protocols/gossipsub/src/types.rs --- protocols/gossipsub/src/types.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/protocols/gossipsub/src/types.rs b/protocols/gossipsub/src/types.rs index da40c2e6e7a..6681eca1d93 100644 --- a/protocols/gossipsub/src/types.rs +++ b/protocols/gossipsub/src/types.rs @@ -117,7 +117,6 @@ pub(crate) struct PeerConnections { /// Describes the types of peers that can exist in the gossipsub context. #[derive(Debug, Clone, Copy, PartialEq, Hash, EncodeLabelValue, Eq)] -#[allow(non_camel_case_types)] pub enum PeerKind { /// A gossipsub 1.2 peer. Gossipsubv1_2, From 1eb934286899ee3e5c204a720b3b3194bd76eced Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jo=C3=A3o=20Oliveira?= Date: Fri, 27 Dec 2024 11:25:39 +0000 Subject: [PATCH 16/16] Update protocols/gossipsub/src/behaviour/tests.rs --- protocols/gossipsub/src/behaviour/tests.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/protocols/gossipsub/src/behaviour/tests.rs b/protocols/gossipsub/src/behaviour/tests.rs index d3de4cd135c..3c1af9ac267 100644 --- a/protocols/gossipsub/src/behaviour/tests.rs +++ b/protocols/gossipsub/src/behaviour/tests.rs @@ -5585,7 +5585,6 @@ fn test_slow_peer_returns_failed_ihave_handling() { topics.insert(topic_hash.clone()); let slow_peer_id = PeerId::random(); - // peers.push(slow_peer_id); gs.connected_peers.insert( slow_peer_id, PeerConnections {