diff --git a/examples/gossipsub-chat.rs b/examples/gossipsub-chat.rs index f5e33741544..2a03b5051e3 100644 --- a/examples/gossipsub-chat.rs +++ b/examples/gossipsub-chat.rs @@ -35,7 +35,7 @@ //! cargo run --example gossipsub-chat //! ``` //! -//! It will print the PeerId and the listening address, e.g. `Listening on +//! It will print the [`PeerId`] and the listening address, e.g. `Listening on //! "/ip4/0.0.0.0/tcp/24915"` //! //! In the second terminal window, start a new instance of the example with: @@ -49,8 +49,10 @@ use async_std::{io, task}; use env_logger::{Builder, Env}; use futures::prelude::*; -use libp2p::gossipsub::protocol::MessageId; -use libp2p::gossipsub::{GossipsubEvent, GossipsubMessage, MessageAuthenticity, Topic}; +use libp2p::gossipsub::MessageId; +use libp2p::gossipsub::{ + GossipsubEvent, GossipsubMessage, IdentTopic as Topic, MessageAuthenticity, ValidationMode, +}; use libp2p::{gossipsub, identity, PeerId}; use std::collections::hash_map::DefaultHasher; use std::hash::{Hash, Hasher}; @@ -72,13 +74,10 @@ fn main() -> Result<(), Box> { let transport = libp2p::build_development_transport(local_key.clone())?; // Create a Gossipsub topic - let topic = Topic::new("test-net".into()); + let topic = Topic::new("test-net"); // Create a Swarm to manage peers and events let mut swarm = { - // to set default parameters for gossipsub use: - // let gossipsub_config = gossipsub::GossipsubConfig::default(); - // To content-address message, we can take the hash of message and use it as an ID. let message_id_fn = |message: &GossipsubMessage| { let mut s = DefaultHasher::new(); @@ -86,16 +85,32 @@ fn main() -> Result<(), Box> { MessageId::from(s.finish().to_string()) }; - // set custom gossipsub - let gossipsub_config = gossipsub::GossipsubConfigBuilder::new() - .heartbeat_interval(Duration::from_secs(10)) + // Set a custom gossipsub + let gossipsub_config = gossipsub::GossipsubConfigBuilder::default() + .heartbeat_interval(Duration::from_secs(10)) // This is set to aid debugging by not cluttering the log space + .validation_mode(ValidationMode::Strict) // This sets the kind of message validation. The default is Strict (enforce message signing) .message_id_fn(message_id_fn) // content-address messages. No two messages of the - //same content will be propagated. - .build(); + // same content will be propagated. + .build() + .expect("Valid config"); // build a gossipsub network behaviour - let mut gossipsub = - gossipsub::Gossipsub::new(MessageAuthenticity::Signed(local_key), gossipsub_config); - gossipsub.subscribe(topic.clone()); + let mut gossipsub: gossipsub::Gossipsub = + gossipsub::Gossipsub::new(MessageAuthenticity::Signed(local_key), gossipsub_config) + .expect("Correct configuration"); + + // subscribes to our topic + gossipsub.subscribe(&topic).unwrap(); + + // add an explicit peer if one was provided + if let Some(explicit) = std::env::args().nth(2) { + let explicit = explicit.clone(); + match explicit.parse() { + Ok(id) => gossipsub.add_explicit_peer(&id), + Err(err) => println!("Failed to parse explicit peer id: {:?}", err), + } + } + + // build the swarm libp2p::Swarm::new(transport, gossipsub, local_peer_id) }; @@ -122,7 +137,7 @@ fn main() -> Result<(), Box> { task::block_on(future::poll_fn(move |cx: &mut Context<'_>| { loop { if let Err(e) = match stdin.try_poll_next_unpin(cx)? { - Poll::Ready(Some(line)) => swarm.publish(&topic, line.as_bytes()), + Poll::Ready(Some(line)) => swarm.publish(topic.clone(), line.as_bytes()), Poll::Ready(None) => panic!("Stdin closed"), Poll::Pending => break, } { @@ -133,7 +148,11 @@ fn main() -> Result<(), Box> { loop { match swarm.poll_next_unpin(cx) { Poll::Ready(Some(gossip_event)) => match gossip_event { - GossipsubEvent::Message(peer_id, id, message) => println!( + GossipsubEvent::Message { + propagation_source: peer_id, + message_id: id, + message, + } => println!( "Got message: {} with id: {} from peer: {:?}", String::from_utf8_lossy(&message.data), id, diff --git a/examples/ipfs-private.rs b/examples/ipfs-private.rs index b781efe8b27..f53419d267e 100644 --- a/examples/ipfs-private.rs +++ b/examples/ipfs-private.rs @@ -34,14 +34,16 @@ use async_std::{io, task}; use futures::{future, prelude::*}; use libp2p::{ - core::{either::EitherTransport, transport, transport::upgrade::Version, muxing::StreamMuxerBox}, + core::{ + either::EitherTransport, muxing::StreamMuxerBox, transport, transport::upgrade::Version, + }, gossipsub::{self, Gossipsub, GossipsubConfigBuilder, GossipsubEvent, MessageAuthenticity}, identify::{Identify, IdentifyEvent}, identity, multiaddr::Protocol, + noise, ping::{self, Ping, PingConfig, PingEvent}, pnet::{PnetConfig, PreSharedKey}, - noise, swarm::NetworkBehaviourEventProcess, tcp::TcpConfig, yamux::YamuxConfig, @@ -61,9 +63,10 @@ use std::{ pub fn build_transport( key_pair: identity::Keypair, psk: Option, -) -> transport::Boxed<(PeerId, StreamMuxerBox)> -{ - let noise_keys = noise::Keypair::::new().into_authentic(&key_pair).unwrap(); +) -> transport::Boxed<(PeerId, StreamMuxerBox)> { + let noise_keys = noise::Keypair::::new() + .into_authentic(&key_pair) + .unwrap(); let noise_config = noise::NoiseConfig::xx(noise_keys).into_authenticated(); let yamux_config = YamuxConfig::default(); @@ -157,7 +160,7 @@ fn main() -> Result<(), Box> { let transport = build_transport(local_key.clone(), psk); // Create a Gosspipsub topic - let gossipsub_topic = gossipsub::Topic::new("chat".into()); + let gossipsub_topic = gossipsub::IdentTopic::new("chat"); // We create a custom network behaviour that combines gossipsub, ping and identify. #[derive(NetworkBehaviour)] @@ -178,7 +181,11 @@ fn main() -> Result<(), Box> { // Called when `gossipsub` produces an event. fn inject_event(&mut self, event: GossipsubEvent) { match event { - GossipsubEvent::Message(peer_id, id, message) => println!( + GossipsubEvent::Message { + propagation_source: peer_id, + message_id: id, + message, + } => println!( "Got message: {} with id: {} from peer: {:?}", String::from_utf8_lossy(&message.data), id, @@ -228,11 +235,16 @@ fn main() -> Result<(), Box> { // Create a Swarm to manage peers and events let mut swarm = { - let gossipsub_config = GossipsubConfigBuilder::new() + let gossipsub_config = GossipsubConfigBuilder::default() .max_transmit_size(262144) - .build(); + .build() + .expect("valid config"); let mut behaviour = MyBehaviour { - gossipsub: Gossipsub::new(MessageAuthenticity::Signed(local_key.clone()), gossipsub_config), + gossipsub: Gossipsub::new( + MessageAuthenticity::Signed(local_key.clone()), + gossipsub_config, + ) + .expect("Valid configuration"), identify: Identify::new( "/ipfs/0.1.0".into(), "rust-ipfs-example".into(), @@ -242,7 +254,7 @@ fn main() -> Result<(), Box> { }; println!("Subscribing to {:?}", gossipsub_topic); - behaviour.gossipsub.subscribe(gossipsub_topic.clone()); + behaviour.gossipsub.subscribe(&gossipsub_topic).unwrap(); Swarm::new(transport, behaviour, local_peer_id.clone()) }; @@ -264,9 +276,9 @@ fn main() -> Result<(), Box> { task::block_on(future::poll_fn(move |cx: &mut Context<'_>| { loop { if let Err(e) = match stdin.try_poll_next_unpin(cx)? { - Poll::Ready(Some(line)) => { - swarm.gossipsub.publish(&gossipsub_topic, line.as_bytes()) - } + Poll::Ready(Some(line)) => swarm + .gossipsub + .publish(gossipsub_topic.clone(), line.as_bytes()), Poll::Ready(None) => panic!("Stdin closed"), Poll::Pending => break, } { diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml index b3b16fb60f7..2cca3f575c3 100644 --- a/protocols/gossipsub/Cargo.toml +++ b/protocols/gossipsub/Cargo.toml @@ -12,29 +12,32 @@ categories = ["network-programming", "asynchronous"] [dependencies] libp2p-swarm = { version = "0.26.0", path = "../../swarm" } libp2p-core = { version = "0.26.0", path = "../../core" } -bytes = "0.5.4" -byteorder = "1.3.2" -fnv = "1.0.6" -futures = "0.3.1" +bytes = "0.5.6" +byteorder = "1.3.4" +fnv = "1.0.7" +futures = "0.3.5" rand = "0.7.3" -futures_codec = "0.4.0" +futures_codec = "0.4.1" wasm-timer = "0.2.4" -unsigned-varint = { version = "0.5", features = ["futures-codec"] } -log = "0.4.8" +unsigned-varint = { version = "0.5.0", features = ["futures-codec"] } +log = "0.4.11" sha2 = "0.9.1" base64 = "0.13.0" -smallvec = "1.1.0" +smallvec = "1.4.2" prost = "0.6.1" hex_fmt = "0.3.0" -lru_time_cache = "0.11.0" +regex = "1.4.0" [dev-dependencies] -async-std = "1.6.2" +async-std = "1.6.3" env_logger = "0.8.1" libp2p-plaintext = { path = "../plaintext" } libp2p-yamux = { path = "../../muxers/yamux" } +libp2p-mplex = { path = "../../muxers/mplex" } +libp2p-noise = { path = "../../protocols/noise" } quickcheck = "0.9.2" hex = "0.4.2" +derive_builder = "0.9.0" [build-dependencies] -prost-build = "0.6" +prost-build = "0.6.1" diff --git a/protocols/gossipsub/build.rs b/protocols/gossipsub/build.rs index a3de99880dc..a0c81052bdc 100644 --- a/protocols/gossipsub/build.rs +++ b/protocols/gossipsub/build.rs @@ -19,5 +19,5 @@ // DEALINGS IN THE SOFTWARE. fn main() { - prost_build::compile_protos(&["src/rpc.proto"], &["src"]).unwrap(); + prost_build::compile_protos(&["src/rpc.proto", "src/compat.proto"], &["src"]).unwrap(); } diff --git a/protocols/gossipsub/src/backoff.rs b/protocols/gossipsub/src/backoff.rs new file mode 100644 index 00000000000..2e6e7614f94 --- /dev/null +++ b/protocols/gossipsub/src/backoff.rs @@ -0,0 +1,180 @@ +// Copyright 2020 Sigma Prime Pty Ltd. +// +// Permission is hereby granted, free of charge, to any person obtaining a +// copy of this software and associated documentation files (the "Software"), +// to deal in the Software without restriction, including without limitation +// the rights to use, copy, modify, merge, publish, distribute, sublicense, +// and/or sell copies of the Software, and to permit persons to whom the +// Software is furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS +// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +// DEALINGS IN THE SOFTWARE. + +//! Data structure for efficiently storing known back-off's when pruning peers. +use crate::topic::TopicHash; +use libp2p_core::PeerId; +use std::collections::{ + hash_map::{Entry, HashMap}, + HashSet, +}; +use std::time::Duration; +use wasm_timer::Instant; + +#[derive(Copy, Clone)] +struct HeartbeatIndex(usize); + +/// Stores backoffs in an efficient manner. +pub struct BackoffStorage { + /// Stores backoffs and the index in backoffs_by_heartbeat per peer per topic. + backoffs: HashMap>, + /// Stores peer topic pairs per heartbeat (this is cyclic the current index is + /// heartbeat_index). + backoffs_by_heartbeat: Vec>, + /// The index in the backoffs_by_heartbeat vector corresponding to the current heartbeat. + heartbeat_index: HeartbeatIndex, + /// The heartbeat interval duration from the config. + heartbeat_interval: Duration, + /// Backoff slack from the config. + backoff_slack: u32, +} + +impl BackoffStorage { + fn heartbeats(d: &Duration, heartbeat_interval: &Duration) -> usize { + ((d.as_nanos() + heartbeat_interval.as_nanos() - 1) / heartbeat_interval.as_nanos()) + as usize + } + + pub fn new( + prune_backoff: &Duration, + heartbeat_interval: Duration, + backoff_slack: u32, + ) -> BackoffStorage { + // We add one additional slot for partial heartbeat + let max_heartbeats = + Self::heartbeats(prune_backoff, &heartbeat_interval) + backoff_slack as usize + 1; + BackoffStorage { + backoffs: HashMap::new(), + backoffs_by_heartbeat: vec![HashSet::new(); max_heartbeats], + heartbeat_index: HeartbeatIndex(0), + heartbeat_interval, + backoff_slack, + } + } + + /// Updates the backoff for a peer (if there is already a more restrictive backoff then this call + /// doesn't change anything). + pub fn update_backoff(&mut self, topic: &TopicHash, peer: &PeerId, time: Duration) { + let instant = Instant::now() + time; + let insert_into_backoffs_by_heartbeat = + |heartbeat_index: HeartbeatIndex, + backoffs_by_heartbeat: &mut Vec>, + heartbeat_interval, + backoff_slack| { + let pair = (topic.clone(), peer.clone()); + let index = (heartbeat_index.0 + + Self::heartbeats(&time, heartbeat_interval) + + backoff_slack as usize) + % backoffs_by_heartbeat.len(); + backoffs_by_heartbeat[index].insert(pair); + HeartbeatIndex(index) + }; + match self + .backoffs + .entry(topic.clone()) + .or_insert_with(HashMap::new) + .entry(peer.clone()) + { + Entry::Occupied(mut o) => { + let (backoff, index) = o.get(); + if backoff < &instant { + let pair = (topic.clone(), peer.clone()); + if let Some(s) = self.backoffs_by_heartbeat.get_mut(index.0) { + s.remove(&pair); + } + let index = insert_into_backoffs_by_heartbeat( + self.heartbeat_index, + &mut self.backoffs_by_heartbeat, + &self.heartbeat_interval, + self.backoff_slack, + ); + o.insert((instant, index)); + } + } + Entry::Vacant(v) => { + let index = insert_into_backoffs_by_heartbeat( + self.heartbeat_index, + &mut self.backoffs_by_heartbeat, + &self.heartbeat_interval, + self.backoff_slack, + ); + v.insert((instant, index)); + } + }; + } + + /// Checks if a given peer is backoffed for the given topic. This method respects the + /// configured BACKOFF_SLACK and may return true even if the backup is already over. + /// It is guaranteed to return false if the backoff is not over and eventually if enough time + /// passed true if the backoff is over. + /// + /// This method should be used for deciding if we can already send a GRAFT to a previously + /// backoffed peer. + pub fn is_backoff_with_slack(&self, topic: &TopicHash, peer: &PeerId) -> bool { + self.backoffs + .get(topic) + .map_or(false, |m| m.contains_key(peer)) + } + + pub fn get_backoff_time(&self, topic: &TopicHash, peer: &PeerId) -> Option { + Self::get_backoff_time_from_backoffs(&self.backoffs, topic, peer) + } + + fn get_backoff_time_from_backoffs( + backoffs: &HashMap>, + topic: &TopicHash, + peer: &PeerId, + ) -> Option { + backoffs + .get(topic) + .and_then(|m| m.get(peer).map(|(i, _)| *i)) + } + + /// Applies a heartbeat. That should be called regularly in intervals of length + /// `heartbeat_interval`. + pub fn heartbeat(&mut self) { + // Clean up backoffs_by_heartbeat + if let Some(s) = self.backoffs_by_heartbeat.get_mut(self.heartbeat_index.0) { + let backoffs = &mut self.backoffs; + let slack = self.heartbeat_interval * self.backoff_slack; + let now = Instant::now(); + s.retain(|(topic, peer)| { + let keep = match Self::get_backoff_time_from_backoffs(backoffs, topic, peer) { + Some(backoff_time) => backoff_time + slack > now, + None => false, + }; + if !keep { + //remove from backoffs + if let Entry::Occupied(mut m) = backoffs.entry(topic.clone()) { + if m.get_mut().remove(peer).is_some() && m.get().is_empty() { + m.remove(); + } + } + } + + keep + }); + } + + // Increase heartbeat index + self.heartbeat_index = + HeartbeatIndex((self.heartbeat_index.0 + 1) % self.backoffs_by_heartbeat.len()); + } +} diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index eb8426a604d..bae8ec2217f 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -18,38 +18,55 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::config::{GossipsubConfig, ValidationMode}; -use crate::error::PublishError; -use crate::handler::GossipsubHandler; -use crate::mcache::MessageCache; -use crate::protocol::{ - GossipsubControlAction, GossipsubMessage, GossipsubSubscription, GossipsubSubscriptionAction, - MessageId, SIGNING_PREFIX, -}; -use crate::rpc_proto; -use crate::topic::{Topic, TopicHash}; -use futures::prelude::*; -use libp2p_core::{ - connection::ConnectionId, identity::error::SigningError, identity::Keypair, Multiaddr, PeerId, -}; -use libp2p_swarm::{ - NetworkBehaviour, NetworkBehaviourAction, NotifyHandler, PollParameters, ProtocolsHandler, -}; -use log::{debug, error, info, trace, warn}; -use lru_time_cache::LruCache; -use prost::Message; -use rand; -use rand::{seq::SliceRandom, thread_rng}; use std::{ + cmp::{max, Ordering}, collections::HashSet, collections::VecDeque, - collections::{hash_map::HashMap, BTreeSet}, - fmt, iter, + collections::{BTreeSet, HashMap}, + fmt, + iter::FromIterator, + net::IpAddr, sync::Arc, task::{Context, Poll}, + time::Duration, }; + +use futures::StreamExt; +use log::{debug, error, info, trace, warn}; +use prost::Message; +use rand::{seq::SliceRandom, thread_rng}; use wasm_timer::{Instant, Interval}; +use libp2p_core::{ + connection::ConnectionId, identity::Keypair, multiaddr::Protocol::Ip4, + multiaddr::Protocol::Ip6, ConnectedPoint, Multiaddr, PeerId, +}; +use libp2p_swarm::{ + DialPeerCondition, NetworkBehaviour, NetworkBehaviourAction, NotifyHandler, PollParameters, + ProtocolsHandler, +}; + +use crate::backoff::BackoffStorage; +use crate::config::{GossipsubConfig, ValidationMode}; +use crate::error::{PublishError, SubscriptionError, ValidationError}; +use crate::gossip_promises::GossipPromises; +use crate::handler::{GossipsubHandler, HandlerEvent}; +use crate::mcache::MessageCache; +use crate::peer_score::{PeerScore, PeerScoreParams, PeerScoreThresholds, RejectReason}; +use crate::protocol::SIGNING_PREFIX; +use crate::subscription_filter::{AllowAllSubscriptionFilter, TopicSubscriptionFilter}; +use crate::time_cache::{DuplicateCache, TimeCache}; +use crate::topic::{Hasher, Topic, TopicHash}; +use crate::transform::{DataTransform, IdentityTransform}; +use crate::types::{ + FastMessageId, GossipsubControlAction, GossipsubMessage, GossipsubSubscription, + GossipsubSubscriptionAction, MessageAcceptance, MessageId, PeerInfo, RawGossipsubMessage, +}; +use crate::types::{GossipsubRpc, PeerKind}; +use crate::{rpc_proto, TopicScoreParams}; +use std::{cmp::Ordering::Equal, fmt::Debug}; + +#[cfg(test)] mod tests; /// Determines if published messages should be signed or not. @@ -65,12 +82,12 @@ pub enum MessageAuthenticity { Signed(Keypair), /// Message signing is disabled. /// - /// The specified `PeerId` will be used as the author of all published messages. The sequence + /// The specified [`PeerId`] will be used as the author of all published messages. The sequence /// number will be randomized. Author(PeerId), /// Message signing is disabled. /// - /// A random `PeerId` will be used when publishing each message. The sequence number will be + /// A random [`PeerId`] will be used when publishing each message. The sequence number will be /// randomized. RandomAuthor, /// Message signing is disabled. @@ -78,7 +95,7 @@ pub enum MessageAuthenticity { /// The author of the message and the sequence numbers are excluded from the message. /// /// NOTE: Excluding these fields may make these messages invalid by other nodes who - /// enforce validation of these fields. See [`ValidationMode`] in the `GossipsubConfig` + /// enforce validation of these fields. See [`ValidationMode`] in the [`GossipsubConfig`] /// for how to customise this for rust-libp2p gossipsub. A custom `message_id` /// function will need to be set to prevent all messages from a peer being filtered /// as duplicates. @@ -87,21 +104,44 @@ pub enum MessageAuthenticity { impl MessageAuthenticity { /// Returns true if signing is enabled. - fn is_signing(&self) -> bool { - match self { - MessageAuthenticity::Signed(_) => true, - _ => false, - } + pub fn is_signing(&self) -> bool { + matches!(self, MessageAuthenticity::Signed(_)) } - fn is_anonymous(&self) -> bool { - match self { - MessageAuthenticity::Anonymous => true, - _ => false, - } + pub fn is_anonymous(&self) -> bool { + matches!(self, MessageAuthenticity::Anonymous) } } +/// Event that can be emitted by the gossipsub behaviour. +#[derive(Debug)] +pub enum GossipsubEvent { + /// A message has been received. + Message { + /// The peer that forwarded us this message. + propagation_source: PeerId, + /// The [`MessageId`] of the message. This should be referenced by the application when + /// validating a message (if required). + message_id: MessageId, + /// The decompressed message itself. + message: GossipsubMessage, + }, + /// A remote subscribed to a topic. + Subscribed { + /// Remote that has subscribed. + peer_id: PeerId, + /// The topic it has subscribed to. + topic: TopicHash, + }, + /// A remote unsubscribed from a topic. + Unsubscribed { + /// Remote that has unsubscribed. + peer_id: PeerId, + /// The topic it has subscribed from. + topic: TopicHash, + }, +} + /// A data structure for storing configuration for publishing messages. See [`MessageAuthenticity`] /// for further details. enum PublishConfig { @@ -115,6 +155,16 @@ enum PublishConfig { Anonymous, } +impl PublishConfig { + pub fn get_own_id(&self) -> Option<&PeerId> { + match self { + Self::Signing { author, .. } => Some(&author), + Self::Author(author) => Some(&author), + _ => None, + } + } +} + impl From for PublishConfig { fn from(authenticity: MessageAuthenticity) -> Self { match authenticity { @@ -143,17 +193,28 @@ impl From for PublishConfig { } } +type GossipsubNetworkBehaviourAction = NetworkBehaviourAction, GossipsubEvent>; + /// Network behaviour that handles the gossipsub protocol. /// -/// NOTE: Initialisation requires a [`MessageAuthenticity`] and [`GossipsubConfig`] instance. If message signing is -/// disabled, the [`ValidationMode`] in the config should be adjusted to an appropriate level to -/// accept unsigned messages. -pub struct Gossipsub { +/// NOTE: Initialisation requires a [`MessageAuthenticity`] and [`GossipsubConfig`] instance. If +/// message signing is disabled, the [`ValidationMode`] in the config should be adjusted to an +/// appropriate level to accept unsigned messages. +/// +/// The DataTransform trait allows applications to optionally add extra encoding/decoding +/// functionality to the underlying messages. This is intended for custom compression algorithms. +/// +/// The TopicSubscriptionFilter allows applications to implement specific filters on topics to +/// prevent unwanted messages being propagated and evaluated. +pub struct Gossipsub< + D: DataTransform = IdentityTransform, + F: TopicSubscriptionFilter = AllowAllSubscriptionFilter, +> { /// Configuration providing gossipsub performance parameters. config: GossipsubConfig, /// Events that need to be yielded to the outside when polling. - events: VecDeque, GossipsubEvent>>, + events: VecDeque, /// Pools non-urgent control messages between heartbeats. control_pool: HashMap>, @@ -163,7 +224,11 @@ pub struct Gossipsub { /// An LRU Time cache for storing seen messages (based on their ID). This cache prevents /// duplicates from being propagated to the application and on the network. - duplication_cache: LruCache, + duplicate_cache: DuplicateCache, + + /// A map of peers to their protocol kind. This is to identify different kinds of gossipsub + /// peers. + peer_protocols: HashMap, /// A map of all connected peers - A map of topic hash to a list of gossipsub peer Ids. topic_peers: HashMap>, @@ -171,6 +236,14 @@ pub struct Gossipsub { /// A map of all connected peers to their subscribed topics. peer_topics: HashMap>, + /// A set of all explicit peers. These are peers that remain connected and we unconditionally + /// forward messages to, outside of the scoring system. + explicit_peers: HashSet, + + /// A list of peers that have been blacklisted by the user. + /// Messages are not sent to and are rejected from these peers. + blacklisted_peers: HashSet, + /// Overlay network of connected peers - Maps topics to connected gossipsub peers. mesh: HashMap>, @@ -180,59 +253,197 @@ pub struct Gossipsub { /// The last publish time for fanout topics. fanout_last_pub: HashMap, + ///Storage for backoffs + backoffs: BackoffStorage, + /// Message cache for the last few heartbeats. mcache: MessageCache, /// Heartbeat interval stream. heartbeat: Interval, + + /// Number of heartbeats since the beginning of time; this allows us to amortize some resource + /// clean up -- eg backoff clean up. + heartbeat_ticks: u64, + + /// We remember all peers we found through peer exchange, since those peers are not considered + /// as safe as randomly discovered outbound peers. This behaviour diverges from the go + /// implementation to avoid possible love bombing attacks in PX. When disconnecting peers will + /// be removed from this list which may result in a true outbound rediscovery. + px_peers: HashSet, + + /// Set of connected outbound peers (we only consider true outbound peers found through + /// discovery and not by PX). + outbound_peers: HashSet, + + /// Stores optional peer score data together with thresholds, decay interval and gossip + /// promises. + peer_score: Option<(PeerScore, PeerScoreThresholds, Interval, GossipPromises)>, + + /// Counts the number of `IHAVE` received from each peer since the last heartbeat. + count_received_ihave: HashMap, + + /// Counts the number of `IWANT` that we sent the each peer since the last heartbeat. + count_sent_iwant: HashMap, + + /// Short term cache for published messsage ids. This is used for penalizing peers sending + /// our own messages back if the messages are anonymous or use a random author. + published_message_ids: DuplicateCache, + + /// Short term cache for fast message ids mapping them to the real message ids + fast_messsage_id_cache: TimeCache, + + /// The filter used to handle message subscriptions. + subscription_filter: F, + + /// A general transformation function that can be applied to data received from the wire before + /// calculating the message-id and sending to the application. This is designed to allow the + /// user to implement arbitrary topic-based compression algorithms. + data_transform: D, +} + +impl Gossipsub +where + D: DataTransform + Default, + F: TopicSubscriptionFilter + Default, +{ + /// Creates a [`Gossipsub`] struct given a set of parameters specified via a + /// [`GossipsubConfig`]. This has no subscription filter and uses no compression. + pub fn new( + privacy: MessageAuthenticity, + config: GossipsubConfig, + ) -> Result { + Self::new_with_subscription_filter_and_transform( + privacy, + config, + F::default(), + D::default(), + ) + } +} + +impl Gossipsub +where + D: DataTransform + Default, + F: TopicSubscriptionFilter, +{ + /// Creates a [`Gossipsub`] struct given a set of parameters specified via a + /// [`GossipsubConfig`] and a custom subscription filter. + pub fn new_with_subscription_filter( + privacy: MessageAuthenticity, + config: GossipsubConfig, + subscription_filter: F, + ) -> Result { + Self::new_with_subscription_filter_and_transform( + privacy, + config, + subscription_filter, + D::default(), + ) + } +} + +impl Gossipsub +where + D: DataTransform, + F: TopicSubscriptionFilter + Default, +{ + /// Creates a [`Gossipsub`] struct given a set of parameters specified via a + /// [`GossipsubConfig`] and a custom data transform. + pub fn new_with_transform( + privacy: MessageAuthenticity, + config: GossipsubConfig, + data_transform: D, + ) -> Result { + Self::new_with_subscription_filter_and_transform( + privacy, + config, + F::default(), + data_transform, + ) + } } -impl Gossipsub { - /// Creates a `Gossipsub` struct given a set of parameters specified via a `GossipsubConfig`. - pub fn new(privacy: MessageAuthenticity, config: GossipsubConfig) -> Self { +impl Gossipsub +where + D: DataTransform, + F: TopicSubscriptionFilter, +{ + /// Creates a [`Gossipsub`] struct given a set of parameters specified via a + /// [`GossipsubConfig`] and a custom subscription filter and data transform. + pub fn new_with_subscription_filter_and_transform( + privacy: MessageAuthenticity, + config: GossipsubConfig, + subscription_filter: F, + data_transform: D, + ) -> Result { // Set up the router given the configuration settings. // We do not allow configurations where a published message would also be rejected if it // were received locally. - validate_config(&privacy, &config.validation_mode); + validate_config(&privacy, &config.validation_mode())?; // Set up message publishing parameters. - Gossipsub { + Ok(Gossipsub { events: VecDeque::new(), control_pool: HashMap::new(), publish_config: privacy.into(), - duplication_cache: LruCache::with_expiry_duration(config.duplicate_cache_time), + duplicate_cache: DuplicateCache::new(config.duplicate_cache_time()), + fast_messsage_id_cache: TimeCache::new(config.duplicate_cache_time()), topic_peers: HashMap::new(), peer_topics: HashMap::new(), + explicit_peers: HashSet::new(), + blacklisted_peers: HashSet::new(), mesh: HashMap::new(), fanout: HashMap::new(), fanout_last_pub: HashMap::new(), - mcache: MessageCache::new( - config.history_gossip, - config.history_length, - config.message_id_fn, + backoffs: BackoffStorage::new( + &config.prune_backoff(), + config.heartbeat_interval(), + config.backoff_slack(), ), + mcache: MessageCache::new(config.history_gossip(), config.history_length()), heartbeat: Interval::new_at( - Instant::now() + config.heartbeat_initial_delay, - config.heartbeat_interval, + Instant::now() + config.heartbeat_initial_delay(), + config.heartbeat_interval(), ), + heartbeat_ticks: 0, + px_peers: HashSet::new(), + outbound_peers: HashSet::new(), + peer_score: None, + count_received_ihave: HashMap::new(), + count_sent_iwant: HashMap::new(), + peer_protocols: HashMap::new(), + published_message_ids: DuplicateCache::new(config.published_message_ids_cache_time()), config, - } + subscription_filter, + data_transform, + }) } +} +impl Gossipsub +where + D: DataTransform, + F: TopicSubscriptionFilter, +{ /// Lists the hashes of the topics we are currently subscribed to. pub fn topics(&self) -> impl Iterator { self.mesh.keys() } - /// Lists peers for a certain topic hash. - pub fn peers(&self, topic_hash: &TopicHash) -> impl Iterator { - self.mesh.get(topic_hash).into_iter().map(|x| x.into_iter()).flatten() + /// Lists all mesh peers for a certain topic hash. + pub fn mesh_peers(&self, topic_hash: &TopicHash) -> impl Iterator { + self.mesh + .get(topic_hash) + .into_iter() + .map(|x| x.iter()) + .flatten() } - /// Lists all peers for any topic. - pub fn all_peers(&self) -> impl Iterator { + /// Lists all mesh peers for all topics. + pub fn all_mesh_peers(&self) -> impl Iterator { let mut res = BTreeSet::new(); for peers in self.mesh.values() { res.extend(peers); @@ -240,32 +451,60 @@ impl Gossipsub { res.into_iter() } + /// Lists all known peers and their associated subscribed topics. + pub fn all_peers(&self) -> impl Iterator)> { + self.peer_topics + .iter() + .map(|(peer_id, topic_set)| (peer_id, topic_set.iter().collect())) + } + + /// Lists all known peers and their associated protocol. + pub fn peer_protocol(&self) -> impl Iterator { + self.peer_protocols.iter() + } + + /// Returns the gossipsub score for a given peer, if one exists. + pub fn peer_score(&self, peer_id: &PeerId) -> Option { + self.peer_score + .as_ref() + .map(|(score, ..)| score.score(peer_id)) + } + /// Subscribe to a topic. /// - /// Returns true if the subscription worked. Returns false if we were already subscribed. - pub fn subscribe(&mut self, topic: Topic) -> bool { + /// Returns [`Ok(true)`] if the subscription worked. Returns [`Ok(false)`] if we were already + /// subscribed. + pub fn subscribe(&mut self, topic: &Topic) -> Result { debug!("Subscribing to topic: {}", topic); - let topic_hash = self.topic_hash(topic.clone()); + let topic_hash = topic.hash(); + if !self.subscription_filter.can_subscribe(&topic_hash) { + return Err(SubscriptionError::NotAllowed); + } + if self.mesh.get(&topic_hash).is_some() { debug!("Topic: {} is already in the mesh.", topic); - return false; + return Ok(false); } // send subscription request to all peers let peer_list = self.peer_topics.keys().cloned().collect::>(); if !peer_list.is_empty() { - let event = Arc::new(GossipsubRpc { - messages: Vec::new(), - subscriptions: vec![GossipsubSubscription { - topic_hash: topic_hash.clone(), - action: GossipsubSubscriptionAction::Subscribe, - }], - control_msgs: Vec::new(), - }); + let event = Arc::new( + GossipsubRpc { + messages: Vec::new(), + subscriptions: vec![GossipsubSubscription { + topic_hash: topic_hash.clone(), + action: GossipsubSubscriptionAction::Subscribe, + }], + control_msgs: Vec::new(), + } + .into_protobuf(), + ); for peer in peer_list { debug!("Sending SUBSCRIBE to peer: {:?}", peer); - self.send_message(peer, event.clone()); + self.send_message(peer, event.clone()) + .map_err(SubscriptionError::PublishError)?; } } @@ -273,37 +512,40 @@ impl Gossipsub { // this will add new peers to the mesh for the topic self.join(&topic_hash); info!("Subscribed to topic: {}", topic); - true + Ok(true) } /// Unsubscribes from a topic. /// - /// Returns true if we were subscribed to this topic. - pub fn unsubscribe(&mut self, topic: Topic) -> bool { + /// Returns [`Ok(true)`] if we were subscribed to this topic. + pub fn unsubscribe(&mut self, topic: &Topic) -> Result { debug!("Unsubscribing from topic: {}", topic); - let topic_hash = &self.topic_hash(topic); + let topic_hash = topic.hash(); - if self.mesh.get(topic_hash).is_none() { + if self.mesh.get(&topic_hash).is_none() { debug!("Already unsubscribed from topic: {:?}", topic_hash); // we are not subscribed - return false; + return Ok(false); } // announce to all peers let peer_list = self.peer_topics.keys().cloned().collect::>(); if !peer_list.is_empty() { - let event = Arc::new(GossipsubRpc { - messages: Vec::new(), - subscriptions: vec![GossipsubSubscription { - topic_hash: topic_hash.clone(), - action: GossipsubSubscriptionAction::Unsubscribe, - }], - control_msgs: Vec::new(), - }); + let event = Arc::new( + GossipsubRpc { + messages: Vec::new(), + subscriptions: vec![GossipsubSubscription { + topic_hash: topic_hash.clone(), + action: GossipsubSubscriptionAction::Unsubscribe, + }], + control_msgs: Vec::new(), + } + .into_protobuf(), + ); for peer in peer_list { debug!("Sending UNSUBSCRIBE to peer: {}", peer.to_string()); - self.send_message(peer, event.clone()); + self.send_message(peer, event.clone())?; } } @@ -312,28 +554,52 @@ impl Gossipsub { self.leave(&topic_hash); info!("Unsubscribed from topic: {:?}", topic_hash); - true - } - - /// Publishes a message to the network. - pub fn publish(&mut self, topic: &Topic, data: impl Into>) -> Result<(), PublishError> { - self.publish_many(iter::once(topic.clone()), data) + Ok(true) } /// Publishes a message with multiple topics to the network. - pub fn publish_many( + pub fn publish( &mut self, - topics: impl IntoIterator, + topic: Topic, data: impl Into>, - ) -> Result<(), PublishError> { - let message = self.build_message( - topics.into_iter().map(|t| self.topic_hash(t)).collect(), - data.into(), - )?; - let msg_id = (self.config.message_id_fn)(&message); + ) -> Result { + let data = data.into(); + + // Transform the data before building a raw_message. + let transformed_data = self + .data_transform + .outbound_transform(&topic.hash(), data.clone())?; + + let raw_message = self.build_raw_message(topic.into(), transformed_data)?; + + // calculate the message id from the un-transformed data + let msg_id = self.config.message_id(&GossipsubMessage { + source: raw_message.source.clone(), + data, // the uncompressed form + sequence_number: raw_message.sequence_number, + topic: raw_message.topic.clone(), + }); + + let event = Arc::new( + GossipsubRpc { + subscriptions: Vec::new(), + messages: vec![raw_message.clone()], + control_msgs: Vec::new(), + } + .into_protobuf(), + ); + + // check that the size doesn't exceed the max transmission size + if event.encoded_len() > self.config.max_transmit_size() { + // NOTE: The size limit can be reached by excessive topics or an excessive message. + // This is an estimate that should be within 10% of the true encoded value. It is + // possible to have a message that exceeds the RPC limit and is not caught here. A + // warning log will be emitted in this case. + return Err(PublishError::MessageTooLarge); + } // Add published message to the duplicate cache. - if self.duplication_cache.insert(msg_id.clone(), ()).is_some() { + if !self.duplicate_cache.insert(msg_id.clone()) { // This message has already been seen. We don't re-publish messages that have already // been published on the network. warn!( @@ -344,41 +610,91 @@ impl Gossipsub { } // If the message isn't a duplicate add it to the memcache. - self.mcache.put(message.clone()); + self.mcache.put(&msg_id, raw_message.clone()); debug!("Publishing message: {:?}", msg_id); - // Forward the message to mesh peers. - let mesh_peers_sent = self.forward_msg(message.clone(), None); + // If the message is anonymous or has a random author add it to the published message ids + // cache. + if let PublishConfig::RandomAuthor | PublishConfig::Anonymous = self.publish_config { + if !self.config.allow_self_origin() { + self.published_message_ids.insert(msg_id.clone()); + } + } + + let topic_hash = raw_message.topic.clone(); + + // If we are not flood publishing forward the message to mesh peers. + let mesh_peers_sent = + !self.config.flood_publish() && self.forward_msg(&msg_id, raw_message, None)?; let mut recipient_peers = HashSet::new(); - for topic_hash in &message.topics { - // If not subscribed to the topic, use fanout peers. - if self.mesh.get(&topic_hash).is_none() { - debug!("Topic: {:?} not in the mesh", topic_hash); - // Build a list of peers to forward the message to - // if we have fanout peers add them to the map. - if self.fanout.contains_key(&topic_hash) { - for peer in self.fanout.get(&topic_hash).expect("Topic must exist") { + if let Some(set) = self.topic_peers.get(&topic_hash) { + if self.config.flood_publish() { + // Forward to all peers above score and all explicit peers + recipient_peers.extend( + set.iter() + .filter(|p| { + self.explicit_peers.contains(*p) + || !self.score_below_threshold(*p, |ts| ts.publish_threshold).0 + }) + .cloned(), + ); + } else { + // Explicit peers + for peer in &self.explicit_peers { + if set.contains(peer) { recipient_peers.insert(peer.clone()); } - } else { - // we have no fanout peers, select mesh_n of them and add them to the fanout - let mesh_n = self.config.mesh_n; - let new_peers = - Self::get_random_peers(&self.topic_peers, &topic_hash, mesh_n, { - |_| true - }); - // add the new peers to the fanout and recipient peers - self.fanout.insert(topic_hash.clone(), new_peers.clone()); - for peer in new_peers { - debug!("Peer added to fanout: {:?}", peer); + } + + // Floodsub peers + for (peer, kind) in &self.peer_protocols { + if kind == &PeerKind::Floodsub + && !self + .score_below_threshold(peer, |ts| ts.publish_threshold) + .0 + { recipient_peers.insert(peer.clone()); } } - // we are publishing to fanout peers - update the time we published - self.fanout_last_pub - .insert(topic_hash.clone(), Instant::now()); + + // Gossipsub peers + if self.mesh.get(&topic_hash).is_none() { + debug!("Topic: {:?} not in the mesh", topic_hash); + // If we have fanout peers add them to the map. + if self.fanout.contains_key(&topic_hash) { + for peer in self.fanout.get(&topic_hash).expect("Topic must exist") { + recipient_peers.insert(peer.clone()); + } + } else { + // We have no fanout peers, select mesh_n of them and add them to the fanout + let mesh_n = self.config.mesh_n(); + let new_peers = get_random_peers( + &self.topic_peers, + &self.peer_protocols, + &topic_hash, + mesh_n, + { + |p| { + !self.explicit_peers.contains(p) + && !self + .score_below_threshold(p, |pst| pst.publish_threshold) + .0 + } + }, + ); + // Add the new peers to the fanout and recipient peers + self.fanout.insert(topic_hash.clone(), new_peers.clone()); + for peer in new_peers { + debug!("Peer added to fanout: {:?}", peer); + recipient_peers.insert(peer.clone()); + } + } + // We are publishing to fanout peers - update the time we published + self.fanout_last_pub + .insert(topic_hash.clone(), Instant::now()); + } } } @@ -386,50 +702,164 @@ impl Gossipsub { return Err(PublishError::InsufficientPeers); } - let event = Arc::new(GossipsubRpc { - subscriptions: Vec::new(), - messages: vec![message], - control_msgs: Vec::new(), - }); // Send to peers we know are subscribed to the topic. for peer_id in recipient_peers.iter() { debug!("Sending message to peer: {:?}", peer_id); - self.send_message(peer_id.clone(), event.clone()); + self.send_message(peer_id.clone(), event.clone())?; } - info!("Published message: {:?}", msg_id); - Ok(()) + info!("Published message: {:?}", &msg_id); + Ok(msg_id) } - /// This function should be called when `config.validate_messages` is `true` in order to - /// validate and propagate messages. Messages are stored in the ['Memcache'] and validation is expected to be - /// fast enough that the messages should still exist in the cache. + /// This function should be called when [`GossipsubConfig::validate_messages()`] is `true` after + /// the message got validated by the caller. Messages are stored in the ['Memcache'] and + /// validation is expected to be fast enough that the messages should still exist in the cache. + /// There are three possible validation outcomes and the outcome is given in acceptance. /// - /// Calling this function will propagate a message stored in the cache, if it still exists. - /// If the message still exists in the cache, it will be forwarded and this function will return true, - /// otherwise it will return false. + /// If acceptance = [`MessageAcceptance::Accept`] the message will get propagated to the + /// network. The `propagation_source` parameter indicates who the message was received by and + /// will not be forwarded back to that peer. /// - /// The `propagation_source` parameter indicates who the message was received by and will not - /// be forwarded back to that peer. + /// If acceptance = [`MessageAcceptance::Reject`] the message will be deleted from the memcache + /// and the P₄ penalty will be applied to the `propagation_source`. + // + /// If acceptance = [`MessageAcceptance::Ignore`] the message will be deleted from the memcache + /// but no P₄ penalty will be applied. + /// + /// This function will return true if the message was found in the cache and false if was not + /// in the cache anymore. /// /// This should only be called once per message. - pub fn validate_message( + pub fn report_message_validation_result( &mut self, - message_id: &MessageId, + msg_id: &MessageId, propagation_source: &PeerId, - ) -> bool { - let message = match self.mcache.validate(message_id) { - Some(message) => message.clone(), - None => { - warn!( - "Message not in cache. Ignoring forwarding. Message Id: {}", - message_id - ); - return false; + acceptance: MessageAcceptance, + ) -> Result { + let reject_reason = match acceptance { + MessageAcceptance::Accept => { + let raw_message = match self.mcache.validate(msg_id) { + Some(raw_message) => raw_message.clone(), + None => { + warn!( + "Message not in cache. Ignoring forwarding. Message Id: {}", + msg_id + ); + return Ok(false); + } + }; + self.forward_msg(msg_id, raw_message, Some(propagation_source))?; + return Ok(true); } + MessageAcceptance::Reject => RejectReason::ValidationFailed, + MessageAcceptance::Ignore => RejectReason::ValidationIgnored, }; - self.forward_msg(message, Some(propagation_source)); - true + + if let Some(raw_message) = self.mcache.remove(msg_id) { + // Tell peer_score about reject + if let Some((peer_score, ..)) = &mut self.peer_score { + peer_score.reject_message( + propagation_source, + msg_id, + &raw_message.topic, + reject_reason, + ); + } + Ok(true) + } else { + warn!("Rejected message not in cache. Message Id: {}", msg_id); + Ok(false) + } + } + + /// Adds a new peer to the list of explicitly connected peers. + pub fn add_explicit_peer(&mut self, peer_id: &PeerId) { + debug!("Adding explicit peer {}", peer_id); + + self.explicit_peers.insert(peer_id.clone()); + + self.check_explicit_peer_connection(peer_id); + } + + /// This removes the peer from explicitly connected peers, note that this does not disconnect + /// the peer. + pub fn remove_explicit_peer(&mut self, peer_id: &PeerId) { + debug!("Removing explicit peer {}", peer_id); + self.explicit_peers.remove(peer_id); + } + + /// Blacklists a peer. All messages from this peer will be rejected and any message that was + /// created by this peer will be rejected. + pub fn blacklist_peer(&mut self, peer_id: &PeerId) { + if self.blacklisted_peers.insert(peer_id.clone()) { + debug!("Peer has been blacklisted: {}", peer_id); + } + } + + /// Removes a peer from the blacklist if it has previously been blacklisted. + pub fn remove_blacklisted_peer(&mut self, peer_id: &PeerId) { + if self.blacklisted_peers.remove(peer_id) { + debug!("Peer has been removed from the blacklist: {}", peer_id); + } + } + + /// Activates the peer scoring system with the given parameters. This will reset all scores + /// if there was already another peer scoring system activated. Returns an error if the + /// params are not valid or if they got already set. + pub fn with_peer_score( + &mut self, + params: PeerScoreParams, + threshold: PeerScoreThresholds, + ) -> Result<(), String> { + self.with_peer_score_and_message_delivery_time_callback(params, threshold, None) + } + + /// Activates the peer scoring system with the given parameters and a message delivery time + /// callback. Returns an error if the parameters got already set. + pub fn with_peer_score_and_message_delivery_time_callback( + &mut self, + params: PeerScoreParams, + threshold: PeerScoreThresholds, + callback: Option, + ) -> Result<(), String> { + params.validate()?; + threshold.validate()?; + + if self.peer_score.is_some() { + return Err("Peer score set twice".into()); + } + + let interval = Interval::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())); + Ok(()) + } + + /// Sets scoring parameters for a topic. + /// + /// The [`Self::with_peer_score()`] must first be called to initialise peer scoring. + pub fn set_topic_params( + &mut self, + topic: Topic, + params: TopicScoreParams, + ) -> Result<(), &'static str> { + if let Some((peer_score, ..)) = &mut self.peer_score { + peer_score.set_topic_params(topic.hash(), params); + Ok(()) + } else { + Err("Peer score must be initialised with `with_peer_score()`") + } + } + + /// Sets the application specific score for a peer. Returns true if scoring is active and + /// the peer is connected or if the score of the peer is not yet expired, false otherwise. + pub fn set_application_score(&mut self, peer_id: &PeerId, new_score: f64) -> bool { + if let Some((peer_score, ..)) = &mut self.peer_score { + peer_score.set_application_score(peer_id, new_score) + } else { + false + } } /// Gossipsub JOIN(topic) - adds topic peers to mesh and sends them GRAFT messages. @@ -442,18 +872,29 @@ impl Gossipsub { return; } - let mut added_peers = vec![]; + let mut added_peers = HashSet::new(); // check if we have mesh_n peers in fanout[topic] and add them to the mesh if we do, // removing the fanout entry. - if let Some((_, peers)) = self.fanout.remove_entry(topic_hash) { + if let Some((_, mut peers)) = self.fanout.remove_entry(topic_hash) { debug!( "JOIN: Removing peers from the fanout for topic: {:?}", topic_hash ); - // add up to mesh_n of them them to the mesh - // Note: These aren't randomly added, currently FIFO - let add_peers = std::cmp::min(peers.len(), self.config.mesh_n); + + // remove explicit peers, peers with negative scores, and backoffed peers + peers = peers + .into_iter() + .filter(|p| { + !self.explicit_peers.contains(p) + && !self.score_below_threshold(p, |_| 0.0).0 + && !self.backoffs.is_backoff_with_slack(topic_hash, p) + }) + .collect(); + + // Add up to mesh_n of them them to the mesh + // NOTE: These aren't randomly added, currently FIFO + let add_peers = std::cmp::min(peers.len(), self.config.mesh_n()); debug!( "JOIN: Adding {:?} peers from the fanout for topic: {:?}", add_peers, topic_hash @@ -468,13 +909,19 @@ impl Gossipsub { } // check if we need to get more peers, which we randomly select - if added_peers.len() < self.config.mesh_n { + if added_peers.len() < self.config.mesh_n() { // get the peers - let new_peers = Self::get_random_peers( + let new_peers = get_random_peers( &self.topic_peers, + &self.peer_protocols, topic_hash, - self.config.mesh_n - added_peers.len(), - |peer| !added_peers.contains(peer), + self.config.mesh_n() - added_peers.len(), + |peer| { + !added_peers.contains(peer) + && !self.explicit_peers.contains(peer) + && !self.score_below_threshold(peer, |_| 0.0).0 + && !self.backoffs.is_backoff_with_slack(topic_hash, peer) + }, ); added_peers.extend(new_peers.clone()); // add them to the mesh @@ -492,6 +939,9 @@ impl Gossipsub { for peer_id in added_peers { // Send a GRAFT control message info!("JOIN: Sending Graft message to peer: {:?}", peer_id); + if let Some((peer_score, ..)) = &mut self.peer_score { + peer_score.graft(&peer_id, topic_hash.clone()); + } Self::control_pool_add( &mut self.control_pool, peer_id.clone(), @@ -503,31 +953,155 @@ impl Gossipsub { debug!("Completed JOIN for topic: {:?}", topic_hash); } + /// Creates a PRUNE gossipsub action. + fn make_prune( + &mut self, + topic_hash: &TopicHash, + peer: &PeerId, + do_px: bool, + ) -> GossipsubControlAction { + if let Some((peer_score, ..)) = &mut self.peer_score { + peer_score.prune(peer, topic_hash.clone()); + } + + match self.peer_protocols.get(peer) { + Some(PeerKind::Floodsub) => { + error!("Attempted to prune a Floodsub peer"); + } + Some(PeerKind::Gossipsub) => { + // GossipSub v1.0 -- no peer exchange, the peer won't be able to parse it anyway + return GossipsubControlAction::Prune { + topic_hash: topic_hash.clone(), + peers: Vec::new(), + backoff: None, + }; + } + None => { + error!("Attempted to Prune an unknown peer"); + } + _ => {} // Gossipsub 1.1 peer perform the `Prune` + } + + // Select peers for peer exchange + let peers = if do_px { + get_random_peers( + &self.topic_peers, + &self.peer_protocols, + &topic_hash, + self.config.prune_peers(), + |p| p != peer && !self.score_below_threshold(p, |_| 0.0).0, + ) + .into_iter() + .map(|p| PeerInfo { peer_id: Some(p) }) + .collect() + } else { + Vec::new() + }; + + // update backoff + self.backoffs + .update_backoff(topic_hash, peer, self.config.prune_backoff()); + + GossipsubControlAction::Prune { + topic_hash: topic_hash.clone(), + peers, + backoff: Some(self.config.prune_backoff().as_secs()), + } + } + /// Gossipsub LEAVE(topic) - Notifies mesh\[topic\] peers with PRUNE messages. fn leave(&mut self, topic_hash: &TopicHash) { debug!("Running LEAVE for topic {:?}", topic_hash); - // if our mesh contains the topic, send prune to peers and delete it from the mesh + // If our mesh contains the topic, send prune to peers and delete it from the mesh if let Some((_, peers)) = self.mesh.remove_entry(topic_hash) { for peer in peers { // Send a PRUNE control message info!("LEAVE: Sending PRUNE to peer: {:?}", peer); - Self::control_pool_add( - &mut self.control_pool, - peer.clone(), - GossipsubControlAction::Prune { - topic_hash: topic_hash.clone(), - }, - ); + let control = self.make_prune(topic_hash, &peer, self.config.do_px()); + Self::control_pool_add(&mut self.control_pool, peer.clone(), control); } } debug!("Completed LEAVE for topic: {:?}", topic_hash); } + /// Checks if the given peer is still connected and if not dials the peer again. + fn check_explicit_peer_connection(&mut self, peer_id: &PeerId) { + if !self.peer_topics.contains_key(peer_id) { + // Connect to peer + debug!("Connecting to explicit peer {:?}", peer_id); + self.events.push_back(NetworkBehaviourAction::DialPeer { + peer_id: peer_id.clone(), + condition: DialPeerCondition::Disconnected, + }); + } + } + + /// Determines if a peer's score is below a given `PeerScoreThreshold` chosen via the + /// `threshold` parameter. + fn score_below_threshold( + &self, + peer_id: &PeerId, + threshold: impl Fn(&PeerScoreThresholds) -> f64, + ) -> (bool, f64) { + Self::score_below_threshold_from_scores(&self.peer_score, peer_id, threshold) + } + + fn score_below_threshold_from_scores( + peer_score: &Option<(PeerScore, PeerScoreThresholds, Interval, GossipPromises)>, + peer_id: &PeerId, + threshold: impl Fn(&PeerScoreThresholds) -> f64, + ) -> (bool, f64) { + if let Some((peer_score, thresholds, ..)) = peer_score { + let score = peer_score.score(peer_id); + if score < threshold(thresholds) { + return (true, score); + } + (false, score) + } else { + (false, 0.0) + } + } + /// Handles an IHAVE control message. Checks our cache of messages. If the message is unknown, /// requests it with an IWANT control message. fn handle_ihave(&mut self, peer_id: &PeerId, ihave_msgs: Vec<(TopicHash, Vec)>) { + // We ignore IHAVE gossip from any peer whose score is below the gossip threshold + if let (true, score) = self.score_below_threshold(peer_id, |pst| pst.gossip_threshold) { + debug!( + "IHAVE: ignoring peer {:?} with score below threshold [score = {}]", + peer_id, score + ); + return; + } + + // IHAVE flood protection + let peer_have = self + .count_received_ihave + .entry(peer_id.clone()) + .or_insert(0); + *peer_have += 1; + if *peer_have > self.config.max_ihave_messages() { + debug!( + "IHAVE: peer {} has advertised too many times ({}) within this heartbeat \ + interval; ignoring", + peer_id, *peer_have + ); + return; + } + + if let Some(iasked) = self.count_sent_iwant.get(peer_id) { + if *iasked >= self.config.max_ihave_length() { + debug!( + "IHAVE: peer {} has already advertised too many messages ({}); ignoring", + peer_id, *iasked + ); + return; + } + } + debug!("Handling IHAVE for peer: {:?}", peer_id); + // use a hashset to avoid duplicates efficiently let mut iwant_ids = HashSet::new(); @@ -542,7 +1116,7 @@ impl Gossipsub { } for id in ids { - if self.mcache.get(&id).is_none() { + if !self.duplicate_cache.contains(&id) { // have not seen this message, request it iwant_ids.insert(id); } @@ -550,14 +1124,45 @@ impl Gossipsub { } if !iwant_ids.is_empty() { + let iasked = self.count_sent_iwant.entry(peer_id.clone()).or_insert(0); + let mut iask = iwant_ids.len(); + if *iasked + iask > self.config.max_ihave_length() { + iask = self.config.max_ihave_length().saturating_sub(*iasked); + } + // Send the list of IWANT control messages - debug!("IHAVE: Sending IWANT message"); - Self::control_pool_add( + debug!( + "IHAVE: Asking for {} out of {} messages from {}", + iask, + iwant_ids.len(), + peer_id + ); + + //ask in random order + let mut iwant_ids_vec: Vec<_> = iwant_ids.iter().collect(); + let mut rng = thread_rng(); + iwant_ids_vec.partial_shuffle(&mut rng, iask as usize); + + iwant_ids_vec.truncate(iask as usize); + *iasked += iask; + + let message_ids = iwant_ids_vec.into_iter().cloned().collect::>(); + if let Some((_, _, _, gossip_promises)) = &mut self.peer_score { + gossip_promises.add_promise( + peer_id.clone(), + &message_ids, + Instant::now() + self.config.iwant_followup_time(), + ); + } + debug!( + "IHAVE: Asking for the following messages from {}: {:?}", + peer_id, message_ids + ); + + Self::control_pool_add( &mut self.control_pool, peer_id.clone(), - GossipsubControlAction::IWant { - message_ids: iwant_ids.iter().cloned().collect(), - }, + GossipsubControlAction::IWant { message_ids }, ); } debug!("Completed IHAVE handling for peer: {:?}", peer_id); @@ -566,50 +1171,163 @@ impl Gossipsub { /// Handles an IWANT control message. Checks our cache of messages. If the message exists it is /// forwarded to the requesting peer. fn handle_iwant(&mut self, peer_id: &PeerId, iwant_msgs: Vec) { + // We ignore IWANT gossip from any peer whose score is below the gossip threshold + if let (true, score) = self.score_below_threshold(peer_id, |pst| pst.gossip_threshold) { + debug!( + "IWANT: ignoring peer {:?} with score below threshold [score = {}]", + peer_id, score + ); + return; + } + debug!("Handling IWANT for peer: {:?}", peer_id); // build a hashmap of available messages let mut cached_messages = HashMap::new(); for id in iwant_msgs { - // if we have it, add it do the cached_messages mapping - if let Some(msg) = self.mcache.get(&id) { - cached_messages.insert(id.clone(), msg.clone()); + // If we have it and the IHAVE count is not above the threshold, add it do the + // cached_messages mapping + if let Some((msg, count)) = self.mcache.get_with_iwant_counts(&id, peer_id) { + if count > self.config.gossip_retransimission() { + debug!( + "IWANT: Peer {} has asked for message {} too many times; ignoring \ + request", + peer_id, &id + ); + } else { + cached_messages.insert(id.clone(), msg.clone()); + } } } if !cached_messages.is_empty() { debug!("IWANT: Sending cached messages to peer: {:?}", peer_id); // Send the messages to the peer - let message_list = cached_messages.into_iter().map(|entry| entry.1).collect(); - self.send_message( - peer_id.clone(), - GossipsubRpc { - subscriptions: Vec::new(), - messages: message_list, - control_msgs: Vec::new(), - }, - ); + let message_list = cached_messages + .into_iter() + .map(|entry| RawGossipsubMessage::from(entry.1)) + .collect(); + if self + .send_message( + peer_id.clone(), + GossipsubRpc { + subscriptions: Vec::new(), + messages: message_list, + control_msgs: Vec::new(), + } + .into_protobuf(), + ) + .is_err() + { + error!("Failed to send cached messages. Messages too large"); + } } - debug!("Completed IWANT handling for peer: {:?}", peer_id); + debug!("Completed IWANT handling for peer: {}", peer_id); } /// Handles GRAFT control messages. If subscribed to the topic, adds the peer to mesh, if not, /// responds with PRUNE messages. fn handle_graft(&mut self, peer_id: &PeerId, topics: Vec) { - debug!("Handling GRAFT message for peer: {:?}", peer_id); + debug!("Handling GRAFT message for peer: {}", peer_id); let mut to_prune_topics = HashSet::new(); - for topic_hash in topics { - if let Some(peers) = self.mesh.get_mut(&topic_hash) { - // if we are subscribed, add peer to the mesh, if not already added - info!( - "GRAFT: Mesh link added for peer: {:?} in topic: {:?}", - peer_id, topic_hash - ); - // Duplicates are ignored - peers.insert(peer_id.clone()); - } else { - to_prune_topics.insert(topic_hash.clone()); + + let mut do_px = self.config.do_px(); + + // we don't GRAFT to/from explicit peers; complain loudly if this happens + if self.explicit_peers.contains(peer_id) { + warn!("GRAFT: ignoring request from direct peer {}", peer_id); + // this is possibly a bug from non-reciprocal configuration; send a PRUNE for all topics + to_prune_topics = HashSet::from_iter(topics.into_iter()); + // but don't PX + do_px = false + } else { + let (below_zero, score) = self.score_below_threshold(peer_id, |_| 0.0); + let now = Instant::now(); + for topic_hash in topics { + if let Some(peers) = self.mesh.get_mut(&topic_hash) { + // if the peer is already in the mesh ignore the graft + if peers.contains(peer_id) { + debug!( + "GRAFT: Received graft for peer {:?} that is already in topic {:?}", + peer_id, &topic_hash + ); + continue; + } + + // make sure we are not backing off that peer + if let Some(backoff_time) = self.backoffs.get_backoff_time(&topic_hash, peer_id) + { + if backoff_time > now { + warn!( + "GRAFT: peer attempted graft within backoff time, penalizing {}", + peer_id + ); + // add behavioural penalty + if let Some((peer_score, ..)) = &mut self.peer_score { + peer_score.add_penalty(peer_id, 1); + + // check the flood cutoff + let flood_cutoff = (backoff_time + + self.config.graft_flood_threshold()) + - self.config.prune_backoff(); + if flood_cutoff > now { + //extra penalty + peer_score.add_penalty(peer_id, 1); + } + } + //no PX + do_px = false; + + to_prune_topics.insert(topic_hash.clone()); + continue; + } + } + + //check the score + if below_zero { + // we don't GRAFT peers with negative score + debug!( + "GRAFT: ignoring peer {:?} with negative score [score = {}, \ + topic = {}]", + peer_id, score, topic_hash + ); + // we do send them PRUNE however, because it's a matter of protocol correctness + to_prune_topics.insert(topic_hash.clone()); + // but we won't PX to them + do_px = false; + continue; + } + + // check mesh upper bound and only allow graft if the upper bound is not reached or + // if it is an outbound peer + if peers.len() >= self.config.mesh_n_high() + && !self.outbound_peers.contains(peer_id) + { + to_prune_topics.insert(topic_hash.clone()); + continue; + } + + // add peer to the mesh + info!( + "GRAFT: Mesh link added for peer: {:?} in topic: {:?}", + peer_id, &topic_hash + ); + peers.insert(peer_id.clone()); + + if let Some((peer_score, ..)) = &mut self.peer_score { + peer_score.graft(peer_id, topic_hash); + } + } else { + // don't do PX when there is an unknown topic to avoid leaking our peers + do_px = false; + debug!( + "GRAFT: Received graft for unknown topic {:?} from peer {:?}", + &topic_hash, peer_id + ); + // spam hardening: ignore GRAFTs for unknown topics + continue; + } } } @@ -617,82 +1335,355 @@ impl Gossipsub { // build the prune messages to send let prune_messages = to_prune_topics .iter() - .map(|t| GossipsubControlAction::Prune { - topic_hash: t.clone(), - }) + .map(|t| self.make_prune(t, peer_id, do_px)) .collect(); // Send the prune messages to the peer info!( - "GRAFT: Not subscribed to topics - Sending PRUNE to peer: {:?}", + "GRAFT: Not subscribed to topics - Sending PRUNE to peer: {}", peer_id ); - self.send_message( - peer_id.clone(), - GossipsubRpc { - subscriptions: Vec::new(), - messages: Vec::new(), - control_msgs: prune_messages, - }, - ); + + if self + .send_message( + peer_id.clone(), + GossipsubRpc { + subscriptions: Vec::new(), + messages: Vec::new(), + control_msgs: prune_messages, + } + .into_protobuf(), + ) + .is_err() + { + error!("Failed to send graft. Message too large"); + } + } + debug!("Completed GRAFT handling for peer: {}", peer_id); + } + + fn remove_peer_from_mesh( + &mut self, + peer_id: &PeerId, + topic_hash: &TopicHash, + backoff: Option, + always_update_backoff: bool, + ) { + let mut update_backoff = always_update_backoff; + if let Some(peers) = self.mesh.get_mut(&topic_hash) { + // remove the peer if it exists in the mesh + if peers.remove(peer_id) { + info!( + "PRUNE: Removing peer: {} from the mesh for topic: {}", + peer_id.to_string(), + topic_hash + ); + + if let Some((peer_score, ..)) = &mut self.peer_score { + peer_score.prune(peer_id, topic_hash.clone()); + } + + update_backoff = true; + } + } + if update_backoff { + let time = if let Some(backoff) = backoff { + Duration::from_secs(backoff) + } else { + self.config.prune_backoff() + }; + // is there a backoff specified by the peer? if so obey it. + self.backoffs.update_backoff(&topic_hash, peer_id, time); } - debug!("Completed GRAFT handling for peer: {:?}", peer_id); } /// Handles PRUNE control messages. Removes peer from the mesh. - fn handle_prune(&mut self, peer_id: &PeerId, topics: Vec) { - debug!("Handling PRUNE message for peer: {}", peer_id.to_string()); - for topic_hash in topics { - if let Some(peers) = self.mesh.get_mut(&topic_hash) { - // remove the peer if it exists in the mesh - if peers.remove(peer_id) { - info!( - "PRUNE: Removing peer: {} from the mesh for topic: {:?}", - peer_id.to_string(), - topic_hash - ); + fn handle_prune( + &mut self, + peer_id: &PeerId, + prune_data: Vec<(TopicHash, Vec, Option)>, + ) { + debug!("Handling PRUNE message for peer: {}", peer_id); + let (below_threshold, score) = + self.score_below_threshold(peer_id, |pst| pst.accept_px_threshold); + for (topic_hash, px, backoff) in prune_data { + self.remove_peer_from_mesh(peer_id, &topic_hash, backoff, true); + + if self.mesh.contains_key(&topic_hash) { + //connect to px peers + if !px.is_empty() { + // we ignore PX from peers with insufficient score + if below_threshold { + debug!( + "PRUNE: ignoring PX from peer {:?} with insufficient score \ + [score ={} topic = {}]", + peer_id, score, topic_hash + ); + continue; + } + + // NOTE: We cannot dial any peers from PX currently as we typically will not + // know their multiaddr. Until SignedRecords are spec'd this + // remains a stub. By default `config.prune_peers()` is set to zero and + // this is skipped. If the user modifies this, this will only be able to + // dial already known peers (from an external discovery mechanism for + // example). + if self.config.prune_peers() > 0 { + self.px_connect(px); + } } } } debug!("Completed PRUNE handling for peer: {}", peer_id.to_string()); } - /// Handles a newly received GossipsubMessage. - /// Forwards the message to all peers in the mesh. - fn handle_received_message(&mut self, mut msg: GossipsubMessage, propagation_source: &PeerId) { - let msg_id = (self.config.message_id_fn)(&msg); + fn px_connect(&mut self, mut px: Vec) { + let n = self.config.prune_peers(); + // Ignore peerInfo with no ID + // + //TODO: Once signed records are spec'd: Can we use peerInfo without any IDs if they have a + // signed peer record? + px = px.into_iter().filter(|p| p.peer_id.is_some()).collect(); + if px.len() > n { + // only use at most prune_peers many random peers + let mut rng = thread_rng(); + px.partial_shuffle(&mut rng, n); + px = px.into_iter().take(n).collect(); + } + + for p in px { + // TODO: Once signed records are spec'd: extract signed peer record if given and handle + // it, see https://github.com/libp2p/specs/pull/217 + if let Some(peer_id) = p.peer_id { + // mark as px peer + self.px_peers.insert(peer_id.clone()); + + // dial peer + self.events.push_back(NetworkBehaviourAction::DialPeer { + peer_id, + condition: DialPeerCondition::Disconnected, + }); + } + } + } + + /// Applies some basic checks to whether this message is valid. Does not apply user validation + /// checks. + fn message_is_valid( + &mut self, + msg_id: &MessageId, + raw_message: &mut RawGossipsubMessage, + propagation_source: &PeerId, + ) -> bool { debug!( "Handling message: {:?} from peer: {}", msg_id, propagation_source.to_string() ); + // Reject any message from a blacklisted peer + if self.blacklisted_peers.contains(propagation_source) { + debug!( + "Rejecting message from blacklisted peer: {}", + propagation_source + ); + if let Some((peer_score, .., gossip_promises)) = &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); + } + return false; + } + + // Also reject any message that originated from a blacklisted peer + if let Some(source) = raw_message.source.as_ref() { + if self.blacklisted_peers.contains(source) { + debug!( + "Rejecting message from peer {} because of blacklisted source: {}", + propagation_source, source + ); + if let Some((peer_score, .., gossip_promises)) = &mut self.peer_score { + peer_score.reject_message( + propagation_source, + msg_id, + &raw_message.topic, + RejectReason::BlackListedSource, + ); + gossip_promises.reject_message(msg_id, &RejectReason::BlackListedSource); + } + return false; + } + } + // If we are not validating messages, assume this message is validated // This will allow the message to be gossiped without explicitly calling // `validate_message`. - if !self.config.validate_messages { - msg.validated = true; + if !self.config.validate_messages() { + raw_message.validated = true; + } + + // reject messages claiming to be from ourselves but not locally published + let self_published = !self.config.allow_self_origin() + && if let Some(own_id) = self.publish_config.get_own_id() { + own_id != propagation_source + && raw_message.source.as_ref().map_or(false, |s| s == own_id) + } else { + self.published_message_ids.contains(&msg_id) + }; + + if self_published { + debug!( + "Dropping message {} claiming to be from self but forwarded from {}", + msg_id, propagation_source + ); + if let Some((peer_score, _, _, gossip_promises)) = &mut self.peer_score { + peer_score.reject_message( + propagation_source, + msg_id, + &raw_message.topic, + RejectReason::SelfOrigin, + ); + gossip_promises.reject_message(msg_id, &RejectReason::SelfOrigin); + } + return false; + } + + true + } + + /// Handles a newly received [`RawGossipsubMessage`]. + /// + /// Forwards the message to all peers in the mesh. + fn handle_received_message( + &mut self, + mut raw_message: RawGossipsubMessage, + propagation_source: &PeerId, + ) { + let fast_message_id = self.config.fast_message_id(&raw_message); + if let Some(fast_message_id) = fast_message_id.as_ref() { + if let Some(msg_id) = self.fast_messsage_id_cache.get(fast_message_id) { + let msg_id = msg_id.clone(); + self.message_is_valid(&msg_id, &mut raw_message, propagation_source); + if let Some((peer_score, ..)) = &mut self.peer_score { + peer_score.duplicated_message(propagation_source, &msg_id, &raw_message.topic); + } + return; + } + } + + // Try and perform the data transform to the message. If it fails, consider it invalid. + let message = match self.data_transform.inbound_transform(raw_message.clone()) { + Ok(message) => message, + Err(e) => { + debug!("Invalid message. Transform error: {:?}", e); + // Reject the message and return + self.handle_invalid_message( + propagation_source, + raw_message, + ValidationError::TransformFailed, + ); + return; + } + }; + + // Calculate the message id on the transformed data. + let msg_id = self.config.message_id(&message); + + // 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. + if !self.message_is_valid(&msg_id, &mut raw_message, propagation_source) { + return; } - // Add the message to the duplication cache and memcache. - if self.duplication_cache.insert(msg_id.clone(), ()).is_some() { - debug!("Message already received, ignoring. Message: {:?}", msg_id); + // Add the message to the duplicate caches + if let Some(fast_message_id) = fast_message_id { + // add id to cache + self.fast_messsage_id_cache + .entry(fast_message_id) + .or_insert_with(|| msg_id.clone()); + } + if !self.duplicate_cache.insert(msg_id.clone()) { + debug!( + "Message already received, ignoring. Message: {}", + msg_id.clone() + ); + if let Some((peer_score, ..)) = &mut self.peer_score { + peer_score.duplicated_message(propagation_source, &msg_id, &message.topic); + } return; } - self.mcache.put(msg.clone()); + debug!( + "Put message {:?} in duplicate_cache and resolve promises", + msg_id + ); - // dispatch the message to the user - if self.mesh.keys().any(|t| msg.topics.iter().any(|u| t == u)) { + // 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 { + peer_score.validate_message(propagation_source, &msg_id, &message.topic); + gossip_promises.message_delivered(&msg_id); + } + + // Add the message to our memcache + self.mcache.put(&msg_id, raw_message.clone()); + + // Dispatch the message to the user if we are subscribed to any of the topics + if self.mesh.contains_key(&message.topic) { debug!("Sending received message to user"); self.events.push_back(NetworkBehaviourAction::GenerateEvent( - GossipsubEvent::Message(propagation_source.clone(), msg_id, msg.clone()), + GossipsubEvent::Message { + propagation_source: propagation_source.clone(), + message_id: msg_id.clone(), + message, + }, )); + } else { + debug!( + "Received message on a topic we are not subscribed to: {:?}", + message.topic + ); + return; } // forward the message to mesh peers, if no validation is required - if !self.config.validate_messages { - let message_id = (self.config.message_id_fn)(&msg); - self.forward_msg(msg, Some(propagation_source)); - debug!("Completed message handling for message: {:?}", message_id); + if !self.config.validate_messages() { + if self + .forward_msg(&msg_id, raw_message, Some(propagation_source)) + .is_err() + { + error!("Failed to forward message. Too large"); + } + debug!("Completed message handling for message: {:?}", msg_id); + } + } + + // Handles invalid messages received. + fn handle_invalid_message( + &mut self, + propagation_source: &PeerId, + raw_message: RawGossipsubMessage, + validation_error: ValidationError, + ) { + if let Some((peer_score, .., gossip_promises)) = &mut self.peer_score { + let reason = RejectReason::ValidationError(validation_error); + let fast_message_id_cache = &self.fast_messsage_id_cache; + if let Some(msg_id) = self + .config + .fast_message_id(&raw_message) + .and_then(|id| fast_message_id_cache.get(&id)) + { + peer_score.reject_message(propagation_source, msg_id, &raw_message.topic, reason); + gossip_promises.reject_message(msg_id, &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 + // penalized, one for sending us an invalid and again for breaking a promise. + peer_score.reject_invalid_message(propagation_source, &raw_message.topic); + } } } @@ -707,6 +1698,9 @@ impl Gossipsub { subscriptions, propagation_source.to_string() ); + + let mut unsubscribed_peers = Vec::new(); + let subscribed_topics = match self.peer_topics.get_mut(propagation_source) { Some(topics) => topics, None => { @@ -724,8 +1718,23 @@ impl Gossipsub { // Notify the application about the subscription, after the grafts are sent. let mut application_event = Vec::new(); - for subscription in subscriptions { - // get the peers from the mapping, or insert empty lists if topic doesn't exist + let filtered_topics = match self + .subscription_filter + .filter_incoming_subscriptions(subscriptions, subscribed_topics) + { + Ok(topics) => topics, + Err(s) => { + error!( + "Subscription filter error: {}; ignoring RPC from peer {}", + s, + propagation_source.to_string() + ); + return; + } + }; + + for subscription in filtered_topics { + // get the peers from the mapping, or insert empty lists if the topic doesn't exist let peer_list = self .topic_peers .entry(subscription.topic_hash.clone()) @@ -745,9 +1754,26 @@ impl Gossipsub { subscribed_topics.insert(subscription.topic_hash.clone()); // if the mesh needs peers add the peer to the mesh - if let Some(peers) = self.mesh.get_mut(&subscription.topic_hash) { - if peers.len() < self.config.mesh_n_low { - if peers.insert(propagation_source.clone()) { + if !self.explicit_peers.contains(propagation_source) + && match self.peer_protocols.get(propagation_source) { + Some(PeerKind::Gossipsubv1_1) => true, + Some(PeerKind::Gossipsub) => true, + _ => false, + } + && !Self::score_below_threshold_from_scores( + &self.peer_score, + propagation_source, + |_| 0.0, + ) + .0 + && !self + .backoffs + .is_backoff_with_slack(&subscription.topic_hash, propagation_source) + { + if let Some(peers) = self.mesh.get_mut(&subscription.topic_hash) { + if peers.len() < self.config.mesh_n_low() + && peers.insert(propagation_source.clone()) + { debug!( "SUBSCRIPTION: Adding peer {} to the mesh for topic {:?}", propagation_source.to_string(), @@ -759,6 +1785,10 @@ impl Gossipsub { propagation_source.to_string(), subscription.topic_hash ); + if let Some((peer_score, ..)) = &mut self.peer_score { + peer_score + .graft(propagation_source, subscription.topic_hash.clone()); + } grafts.push(GossipsubControlAction::Graft { topic_hash: subscription.topic_hash.clone(), }); @@ -783,12 +1813,8 @@ impl Gossipsub { } // remove topic from the peer_topics mapping subscribed_topics.remove(&subscription.topic_hash); - // remove the peer from the mesh if it exists - if let Some(peers) = self.mesh.get_mut(&subscription.topic_hash) { - peers.remove(propagation_source); - // the peer requested the unsubscription so we don't need to send a PRUNE. - } - + unsubscribed_peers + .push((propagation_source.clone(), subscription.topic_hash.clone())); // generate an unsubscribe event to be polled application_event.push(NetworkBehaviourAction::GenerateEvent( GossipsubEvent::Unsubscribed { @@ -800,17 +1826,27 @@ impl Gossipsub { } } + // remove unsubscribed peers from the mesh if it exists + for (peer_id, topic_hash) in unsubscribed_peers { + self.remove_peer_from_mesh(&peer_id, &topic_hash, None, false); + } + // If we need to send grafts to peer, do so immediately, rather than waiting for the // heartbeat. - if !grafts.is_empty() { - self.send_message( - propagation_source.clone(), - GossipsubRpc { - subscriptions: Vec::new(), - messages: Vec::new(), - control_msgs: grafts, - }, - ); + if !grafts.is_empty() + && self + .send_message( + propagation_source.clone(), + GossipsubRpc { + subscriptions: Vec::new(), + messages: Vec::new(), + control_msgs: grafts, + } + .into_protobuf(), + ) + .is_err() + { + error!("Failed sending grafts. Message too large"); } // Notify the application of the subscriptions @@ -824,29 +1860,110 @@ impl Gossipsub { ); } + /// 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() { + peer_score.add_penalty(&peer, count); + } + } + } + /// Heartbeat function which shifts the memcache and updates the mesh. fn heartbeat(&mut self) { debug!("Starting heartbeat"); + self.heartbeat_ticks += 1; + let mut to_graft = HashMap::new(); let mut to_prune = HashMap::new(); + let mut no_px = HashSet::new(); + + // clean up expired backoffs + self.backoffs.heartbeat(); + + // clean up ihave counters + self.count_sent_iwant.clear(); + self.count_received_ihave.clear(); + + // apply iwant penalties + self.apply_iwant_penalties(); + + // check connections to explicit peers + if self.heartbeat_ticks % self.config.check_explicit_peers_ticks() == 0 { + for p in self.explicit_peers.clone() { + self.check_explicit_peer_connection(&p); + } + } + + // cache scores throughout the heartbeat + let mut scores = HashMap::new(); + let peer_score = &self.peer_score; + let mut score = |p: &PeerId| match peer_score { + Some((peer_score, ..)) => *scores + .entry(p.clone()) + .or_insert_with(|| peer_score.score(p)), + _ => 0.0, + }; // maintain the mesh for each topic for (topic_hash, peers) in self.mesh.iter_mut() { + let explicit_peers = &self.explicit_peers; + let backoffs = &self.backoffs; + let topic_peers = &self.topic_peers; + let outbound_peers = &self.outbound_peers; + + // drop all peers with negative score, without PX + // if there is at some point a stable retain method for BTreeSet the following can be + // written more efficiently with retain. + let to_remove: Vec<_> = peers + .iter() + .filter(|&p| { + if score(p) < 0.0 { + debug!( + "HEARTBEAT: Prune peer {:?} with negative score [score = {}, topic = \ + {}]", + p, + score(p), + topic_hash + ); + + let current_topic = to_prune.entry(p.clone()).or_insert_with(Vec::new); + current_topic.push(topic_hash.clone()); + no_px.insert(p.clone()); + true + } else { + false + } + }) + .cloned() + .collect(); + for peer in to_remove { + peers.remove(&peer); + } + // too little peers - add some - if peers.len() < self.config.mesh_n_low { + if peers.len() < self.config.mesh_n_low() { debug!( "HEARTBEAT: Mesh low. Topic: {} Contains: {} needs: {}", topic_hash, peers.len(), - self.config.mesh_n_low + self.config.mesh_n_low() ); // not enough peers - get mesh_n - current_length more - let desired_peers = self.config.mesh_n - peers.len(); - let peer_list = - Self::get_random_peers(&self.topic_peers, topic_hash, desired_peers, { - |peer| !peers.contains(peer) - }); + let desired_peers = self.config.mesh_n() - peers.len(); + let peer_list = get_random_peers( + topic_peers, + &self.peer_protocols, + topic_hash, + desired_peers, + |peer| { + !peers.contains(peer) + && !explicit_peers.contains(peer) + && !backoffs.is_backoff_with_slack(topic_hash, peer) + && score(peer) >= 0.0 + }, + ); for peer in &peer_list { let current_topic = to_graft.entry(peer.clone()).or_insert_with(Vec::new); current_topic.push(topic_hash.clone()); @@ -857,26 +1974,147 @@ impl Gossipsub { } // too many peers - remove some - if peers.len() > self.config.mesh_n_high { + if peers.len() > self.config.mesh_n_high() { debug!( "HEARTBEAT: Mesh high. Topic: {} Contains: {} needs: {}", topic_hash, peers.len(), - self.config.mesh_n_high + self.config.mesh_n_high() ); - let excess_peer_no = peers.len() - self.config.mesh_n; - // shuffle the peers + let excess_peer_no = peers.len() - self.config.mesh_n(); + + // shuffle the peers and then sort by score ascending beginning with the worst let mut rng = thread_rng(); let mut shuffled = peers.iter().cloned().collect::>(); shuffled.shuffle(&mut rng); - // remove the first excess_peer_no peers adding them to to_prune - for _ in 0..excess_peer_no { - let peer = shuffled - .pop() - .expect("There should always be enough peers to remove"); + shuffled + .sort_by(|p1, p2| score(p1).partial_cmp(&score(p2)).unwrap_or(Ordering::Equal)); + // shuffle everything except the last retain_scores many peers (the best ones) + shuffled[..peers.len() - self.config.retain_scores()].shuffle(&mut rng); + + // count total number of outbound peers + let mut outbound = { + let outbound_peers = &self.outbound_peers; + shuffled + .iter() + .filter(|p| outbound_peers.contains(*p)) + .count() + }; + + // remove the first excess_peer_no allowed (by outbound restrictions) peers adding + // them to to_prune + let mut removed = 0; + for peer in shuffled { + if removed == excess_peer_no { + break; + } + if self.outbound_peers.contains(&peer) { + if outbound <= self.config.mesh_outbound_min() { + //do not remove anymore outbound peers + continue; + } else { + //an outbound peer gets removed + outbound -= 1; + } + } + + //remove the peer peers.remove(&peer); let current_topic = to_prune.entry(peer).or_insert_with(Vec::new); current_topic.push(topic_hash.clone()); + removed += 1; + } + } + + // do we have enough outbound peers? + if peers.len() >= self.config.mesh_n_low() { + // count number of outbound peers we have + let outbound = { peers.iter().filter(|p| outbound_peers.contains(*p)).count() }; + + // if we have not enough outbound peers, graft to some new outbound peers + if outbound < self.config.mesh_outbound_min() { + let needed = self.config.mesh_outbound_min() - outbound; + let peer_list = get_random_peers( + topic_peers, + &self.peer_protocols, + topic_hash, + needed, + |peer| { + !peers.contains(peer) + && !explicit_peers.contains(peer) + && !backoffs.is_backoff_with_slack(topic_hash, peer) + && score(peer) >= 0.0 + && outbound_peers.contains(peer) + }, + ); + for peer in &peer_list { + let current_topic = to_graft.entry(peer.clone()).or_insert_with(Vec::new); + current_topic.push(topic_hash.clone()); + } + // update the mesh + debug!("Updating mesh, new mesh: {:?}", peer_list); + peers.extend(peer_list); + } + } + + // should we try to improve the mesh with opportunistic grafting? + if self.heartbeat_ticks % self.config.opportunistic_graft_ticks() == 0 + && peers.len() > 1 + && self.peer_score.is_some() + { + 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. + // The intention is to (slowly) improve an underperforming mesh by introducing + // good scoring peers that may have been gossiping at us. This allows us to + // get out of sticky situations where we are stuck with poor peers and also + // recover from churn of good peers. + + // now compute the median peer score in the mesh + let mut peers_by_score: Vec<_> = peers.iter().collect(); + peers_by_score + .sort_by(|p1, p2| score(p1).partial_cmp(&score(p2)).unwrap_or(Equal)); + + let middle = peers_by_score.len() / 2; + let median = if peers_by_score.len() % 2 == 0 { + (score( + *peers_by_score.get(middle - 1).expect( + "middle < vector length and middle > 0 since peers.len() > 0", + ), + ) + score(*peers_by_score.get(middle).expect("middle < vector length"))) + * 0.5 + } else { + score(*peers_by_score.get(middle).expect("middle < vector length")) + }; + + // if the median score is below the threshold, select a better peer (if any) and + // GRAFT + if median < thresholds.opportunistic_graft_threshold { + let peer_list = get_random_peers( + topic_peers, + &self.peer_protocols, + topic_hash, + self.config.opportunistic_graft_peers(), + |peer| { + !peers.contains(peer) + && !explicit_peers.contains(peer) + && !backoffs.is_backoff_with_slack(topic_hash, peer) + && score(peer) > median + }, + ); + for peer in &peer_list { + let current_topic = + to_graft.entry(peer.clone()).or_insert_with(Vec::new); + current_topic.push(topic_hash.clone()); + } + // update the mesh + debug!( + "Opportunistically graft in topic {} with peers {:?}", + topic_hash, peer_list + ); + peers.extend(peer_list); + } } } } @@ -884,7 +2122,7 @@ impl Gossipsub { // remove expired fanout topics { let fanout = &mut self.fanout; // help the borrow checker - let fanout_ttl = self.config.fanout_ttl; + let fanout_ttl = self.config.fanout_ttl(); self.fanout_last_pub.retain(|topic_hash, last_pub_time| { if *last_pub_time + fanout_ttl < Instant::now() { debug!( @@ -902,11 +2140,15 @@ impl Gossipsub { // check if our peers are still a part of the topic 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, + _ => 0.0, + }; for peer in peers.iter() { // is the peer still subscribed to the topic? match self.peer_topics.get(peer) { Some(topics) => { - if !topics.contains(&topic_hash) { + if !topics.contains(&topic_hash) || score(peer) < publish_threshold { debug!( "HEARTBEAT: Peer removed from fanout for topic: {:?}", topic_hash @@ -925,26 +2167,67 @@ impl Gossipsub { } // not enough peers - if peers.len() < self.config.mesh_n { + if peers.len() < self.config.mesh_n() { debug!( "HEARTBEAT: Fanout low. Contains: {:?} needs: {:?}", peers.len(), - self.config.mesh_n + self.config.mesh_n() ); - let needed_peers = self.config.mesh_n - peers.len(); - let new_peers = - Self::get_random_peers(&self.topic_peers, topic_hash, needed_peers, |peer| { + let needed_peers = self.config.mesh_n() - peers.len(); + let explicit_peers = &self.explicit_peers; + let new_peers = get_random_peers( + &self.topic_peers, + &self.peer_protocols, + topic_hash, + needed_peers, + |peer| { !peers.contains(peer) - }); + && !explicit_peers.contains(peer) + && score(peer) < publish_threshold + }, + ); peers.extend(new_peers); } } + if self.peer_score.is_some() { + trace!("Peer_scores: {:?}", { + for peer in self.peer_topics.keys() { + score(peer); + } + scores + }); + trace!("Mesh message deliveries: {:?}", { + self.mesh + .iter() + .map(|(t, peers)| { + ( + t.clone(), + peers + .iter() + .map(|p| { + ( + p.clone(), + peer_score + .as_ref() + .expect("peer_score.is_some()") + .0 + .mesh_message_deliveries(p, t) + .unwrap_or(0.0), + ) + }) + .collect::>(), + ) + }) + .collect::>>() + }) + } + self.emit_gossip(); // send graft/prunes if !to_graft.is_empty() | !to_prune.is_empty() { - self.send_graft_prune(to_graft, to_prune); + self.send_graft_prune(to_graft, to_prune, no_px); } // piggyback pooled control messages @@ -952,36 +2235,72 @@ impl Gossipsub { // shift the memcache self.mcache.shift(); + debug!("Completed Heartbeat"); } /// Emits gossip - Send IHAVE messages to a random set of gossip peers. This is applied to mesh /// and fanout peers fn emit_gossip(&mut self) { + let mut rng = thread_rng(); for (topic_hash, peers) in self.mesh.iter().chain(self.fanout.iter()) { - let message_ids = self.mcache.get_gossip_ids(&topic_hash); + let mut message_ids = self.mcache.get_gossip_message_ids(&topic_hash); if message_ids.is_empty() { return; } + // if we are emitting more than GossipSubMaxIHaveLength message_ids, truncate the list + if message_ids.len() > self.config.max_ihave_length() { + // we do the truncation (with shuffling) per peer below + debug!( + "too many messages for gossip; will truncate IHAVE list ({} messages)", + message_ids.len() + ); + } else { + // shuffle to emit in random order + message_ids.shuffle(&mut rng); + } + + // dynamic number of peers to gossip based on `gossip_factor` with minimum `gossip_lazy` + let n_map = |m| { + max( + self.config.gossip_lazy(), + (self.config.gossip_factor() * m as f64) as usize, + ) + }; // get gossip_lazy random peers - let to_msg_peers = Self::get_random_peers( + let to_msg_peers = get_random_peers_dynamic( &self.topic_peers, + &self.peer_protocols, &topic_hash, - self.config.gossip_lazy, - |peer| !peers.contains(peer), + n_map, + |peer| { + !peers.contains(peer) + && !self.explicit_peers.contains(peer) + && !self.score_below_threshold(peer, |ts| ts.gossip_threshold).0 + }, ); debug!("Gossiping IHAVE to {} peers.", to_msg_peers.len()); for peer in to_msg_peers { + let mut peer_message_ids = message_ids.clone(); + + if peer_message_ids.len() > self.config.max_ihave_length() { + // We do this per peer so that we emit a different set for each peer. + // we have enough redundancy in the system that this will significantly increase + // the message coverage when we do truncate. + peer_message_ids.partial_shuffle(&mut rng, self.config.max_ihave_length()); + peer_message_ids.truncate(self.config.max_ihave_length()); + } + // send an IHAVE message Self::control_pool_add( &mut self.control_pool, peer.clone(), GossipsubControlAction::IHave { topic_hash: topic_hash.clone(), - message_ids: message_ids.clone(), + message_ids: peer_message_ids, }, ); } @@ -994,9 +2313,16 @@ impl Gossipsub { &mut self, to_graft: HashMap>, mut to_prune: HashMap>, + no_px: HashSet, ) { // handle the grafts and overlapping prunes per peer for (peer, topics) in to_graft.iter() { + for topic in topics { + //inform scoring of graft + if let Some((peer_score, ..)) = &mut self.peer_score { + peer_score.graft(peer, topic.clone()); + } + } let mut control_msgs: Vec = topics .iter() .map(|topic_hash| GossipsubControlAction::Graft { @@ -1008,87 +2334,133 @@ impl Gossipsub { if let Some(topics) = to_prune.remove(peer) { let mut prunes = topics .iter() - .map(|topic_hash| GossipsubControlAction::Prune { - topic_hash: topic_hash.clone(), + .map(|topic_hash| { + self.make_prune( + topic_hash, + peer, + self.config.do_px() && !no_px.contains(peer), + ) }) .collect::>(); control_msgs.append(&mut prunes); } // send the control messages - self.send_message( - peer.clone(), - GossipsubRpc { - subscriptions: Vec::new(), - messages: Vec::new(), - control_msgs, - }, - ); + if self + .send_message( + peer.clone(), + GossipsubRpc { + subscriptions: Vec::new(), + messages: Vec::new(), + control_msgs, + } + .into_protobuf(), + ) + .is_err() + { + error!("Failed to send control messages. Message too large"); + } } // handle the remaining prunes for (peer, topics) in to_prune.iter() { let remaining_prunes = topics .iter() - .map(|topic_hash| GossipsubControlAction::Prune { - topic_hash: topic_hash.clone(), + .map(|topic_hash| { + self.make_prune( + topic_hash, + peer, + self.config.do_px() && !no_px.contains(peer), + ) }) .collect(); - self.send_message( - peer.clone(), - GossipsubRpc { - subscriptions: Vec::new(), - messages: Vec::new(), - control_msgs: remaining_prunes, - }, - ); + if self + .send_message( + peer.clone(), + GossipsubRpc { + subscriptions: Vec::new(), + messages: Vec::new(), + control_msgs: remaining_prunes, + } + .into_protobuf(), + ) + .is_err() + { + error!("Failed to send prune messages. Message too large"); + } } } /// Helper function which forwards a message to mesh\[topic\] peers. + /// /// Returns true if at least one peer was messaged. - fn forward_msg(&mut self, message: GossipsubMessage, source: Option<&PeerId>) -> bool { - let msg_id = (self.config.message_id_fn)(&message); + fn forward_msg( + &mut self, + msg_id: &MessageId, + message: RawGossipsubMessage, + propagation_source: Option<&PeerId>, + ) -> Result { + // message is fully validated inform peer_score + if let Some((peer_score, ..)) = &mut self.peer_score { + if let Some(peer) = propagation_source { + peer_score.deliver_message(peer, msg_id, &message.topic); + } + } + debug!("Forwarding message: {:?}", msg_id); let mut recipient_peers = HashSet::new(); // add mesh peers - for topic in &message.topics { - // mesh - if let Some(mesh_peers) = self.mesh.get(&topic) { - for peer_id in mesh_peers { - if Some(peer_id) != source { - recipient_peers.insert(peer_id.clone()); - } + let topic = &message.topic; + // mesh + if let Some(mesh_peers) = self.mesh.get(&topic) { + for peer_id in mesh_peers { + if Some(peer_id) != propagation_source && Some(peer_id) != message.source.as_ref() { + recipient_peers.insert(peer_id.clone()); + } + } + } + + // Add explicit peers + for p in &self.explicit_peers { + if let Some(topics) = self.peer_topics.get(p) { + if Some(p) != propagation_source + && Some(p) != message.source.as_ref() + && topics.contains(&message.topic) + { + recipient_peers.insert(p.clone()); } } } // forward the message to peers if !recipient_peers.is_empty() { - let event = Arc::new(GossipsubRpc { - subscriptions: Vec::new(), - messages: vec![message.clone()], - control_msgs: Vec::new(), - }); + let event = Arc::new( + GossipsubRpc { + subscriptions: Vec::new(), + messages: vec![RawGossipsubMessage::from(message.clone())], + control_msgs: Vec::new(), + } + .into_protobuf(), + ); for peer in recipient_peers.iter() { debug!("Sending message: {:?} to peer {:?}", msg_id, peer); - self.send_message(peer.clone(), event.clone()); + self.send_message(peer.clone(), event.clone())?; } debug!("Completed forwarding message"); - true + Ok(true) } else { - false + Ok(false) } } - /// Constructs a `GossipsubMessage` performing message signing if required. - pub(crate) fn build_message( + /// Constructs a [`RawGossipsubMessage`] performing message signing if required. + pub(crate) fn build_raw_message( &self, - topics: Vec, + topic: TopicHash, data: Vec, - ) -> Result { + ) -> Result { match &self.publish_config { PublishConfig::Signing { ref keypair, @@ -1103,7 +2475,7 @@ impl Gossipsub { from: Some(author.clone().to_bytes()), data: Some(data.clone()), seqno: Some(sequence_number.to_be_bytes().to_vec()), - topic_ids: topics.clone().into_iter().map(|t| t.into()).collect(), + topic: topic.clone().into_string(), signature: None, key: None, }; @@ -1119,52 +2491,52 @@ impl Gossipsub { Some(keypair.sign(&signature_bytes)?) }; - Ok(GossipsubMessage { + Ok(RawGossipsubMessage { source: Some(author.clone()), data, // To be interoperable with the go-implementation this is treated as a 64-bit // big-endian uint. sequence_number: Some(sequence_number), - topics, + topic, signature, key: inline_key.clone(), validated: true, // all published messages are valid }) } PublishConfig::Author(peer_id) => { - Ok(GossipsubMessage { + Ok(RawGossipsubMessage { source: Some(peer_id.clone()), data, // To be interoperable with the go-implementation this is treated as a 64-bit // big-endian uint. sequence_number: Some(rand::random()), - topics, + topic, signature: None, key: None, validated: true, // all published messages are valid }) } PublishConfig::RandomAuthor => { - Ok(GossipsubMessage { + Ok(RawGossipsubMessage { source: Some(PeerId::random()), data, // To be interoperable with the go-implementation this is treated as a 64-bit // big-endian uint. sequence_number: Some(rand::random()), - topics, + topic, signature: None, key: None, validated: true, // all published messages are valid }) } PublishConfig::Anonymous => { - Ok(GossipsubMessage { + Ok(RawGossipsubMessage { source: None, data, // To be interoperable with the go-implementation this is treated as a 64-bit // big-endian uint. sequence_number: None, - topics, + topic, signature: None, key: None, validated: true, // all published messages are valid @@ -1173,35 +2545,6 @@ impl Gossipsub { } } - /// Helper function to get a set of `n` random gossipsub peers for a `topic_hash` - /// filtered by the function `f`. - fn get_random_peers( - topic_peers: &HashMap>, - topic_hash: &TopicHash, - n: usize, - mut f: impl FnMut(&PeerId) -> bool, - ) -> BTreeSet { - let mut gossip_peers = match topic_peers.get(topic_hash) { - // if they exist, filter the peers by `f` - Some(peer_list) => peer_list.iter().cloned().filter(|p| f(p)).collect(), - None => Vec::new(), - }; - - // if we have less than needed, return them - if gossip_peers.len() <= n { - debug!("RANDOM PEERS: Got {:?} peers", gossip_peers.len()); - return gossip_peers.into_iter().collect(); - } - - // we have more peers than needed, shuffle them and return n of them - let mut rng = thread_rng(); - gossip_peers.partial_shuffle(&mut rng, n); - - debug!("RANDOM PEERS: Got {:?} peers", n); - - gossip_peers.into_iter().take(n).collect() - } - // adds a control action to control_pool fn control_pool_add( control_pool: &mut HashMap>, @@ -1209,55 +2552,202 @@ impl Gossipsub { control: GossipsubControlAction, ) { control_pool - .entry(peer.clone()) + .entry(peer) .or_insert_with(Vec::new) .push(control); } - /// Produces a `TopicHash` for a topic given the gossipsub configuration. - fn topic_hash(&self, topic: Topic) -> TopicHash { - if self.config.hash_topics { - topic.sha256_hash() - } else { - topic.no_hash() - } - } - /// Takes each control action mapping and turns it into a message fn flush_control_pool(&mut self) { for (peer, controls) in self.control_pool.drain().collect::>() { - self.send_message( - peer, - GossipsubRpc { - subscriptions: Vec::new(), - messages: Vec::new(), - control_msgs: controls, - }, - ); + if self + .send_message( + peer, + GossipsubRpc { + subscriptions: Vec::new(), + messages: Vec::new(), + control_msgs: controls, + } + .into_protobuf(), + ) + .is_err() + { + error!("Failed to flush control pool. Message too large"); + } } } /// Send a GossipsubRpc message to a peer. This will wrap the message in an arc if it /// is not already an arc. - fn send_message(&mut self, peer_id: PeerId, message: impl Into>) { - self.events - .push_back(NetworkBehaviourAction::NotifyHandler { - peer_id, - event: message.into(), - handler: NotifyHandler::Any, - }) + fn send_message( + &mut self, + peer_id: PeerId, + message: impl Into>, + ) -> Result<(), PublishError> { + // If the message is oversized, try and fragment it. If it cannot be fragmented, log an + // error and drop the message (all individual messages should be small enough to fit in the + // max_transmit_size) + + let messages = self.fragment_message(message.into())?; + + for message in messages { + self.events + .push_back(NetworkBehaviourAction::NotifyHandler { + peer_id: peer_id.clone(), + event: message, + handler: NotifyHandler::Any, + }) + } + Ok(()) + } + + // If a message is too large to be sent as-is, this attempts to fragment it into smaller RPC + // messages to be sent. + fn fragment_message( + &self, + rpc: Arc, + ) -> Result>, PublishError> { + if rpc.encoded_len() < self.config.max_transmit_size() { + return Ok(vec![rpc]); + } + + let new_rpc = rpc_proto::Rpc { + subscriptions: Vec::new(), + publish: Vec::new(), + control: None, + }; + + let mut rpc_list = vec![new_rpc.clone()]; + + // Gets an RPC if the object size will fit, otherwise create a new RPC. The last element + // will be the RPC to add an object. + macro_rules! create_or_add_rpc { + ($object_size: ident ) => { + let list_index = rpc_list.len() - 1; // the list is never empty + + // create a new RPC if the new object plus 5% of its size (for length prefix + // buffers) exceeds the max transmit size. + if rpc_list[list_index].encoded_len() + (($object_size as f64) * 1.05) as usize + > self.config.max_transmit_size() + && rpc_list[list_index] != new_rpc + { + // create a new rpc and use this as the current + rpc_list.push(new_rpc.clone()); + } + }; + }; + + macro_rules! add_item { + ($object: ident, $type: ident ) => { + let object_size = $object.encoded_len(); + + if object_size + 2 > self.config.max_transmit_size() { + // This should not be possible. All received and published messages have already + // been vetted to fit within the size. + error!("Individual message too large to fragment"); + return Err(PublishError::MessageTooLarge); + } + + create_or_add_rpc!(object_size); + rpc_list + .last_mut() + .expect("Must have at least one element") + .$type + .push($object.clone()); + }; + } + + // Add messages until the limit + for message in &rpc.publish { + add_item!(message, publish); + } + for subscription in &rpc.subscriptions { + add_item!(subscription, subscriptions); + } + + // handle the control messages. If all are within the max_transmit_size, send them without + // fragmenting, otherwise, fragment the control messages + let empty_control = rpc_proto::ControlMessage::default(); + if let Some(control) = rpc.control.as_ref() { + if control.encoded_len() + 2 > self.config.max_transmit_size() { + // fragment the RPC + for ihave in &control.ihave { + let len = ihave.encoded_len(); + create_or_add_rpc!(len); + rpc_list + .last_mut() + .expect("Always an element") + .control + .get_or_insert_with(|| empty_control.clone()) + .ihave + .push(ihave.clone()); + } + for iwant in &control.iwant { + let len = iwant.encoded_len(); + create_or_add_rpc!(len); + rpc_list + .last_mut() + .expect("Always an element") + .control + .get_or_insert_with(|| empty_control.clone()) + .iwant + .push(iwant.clone()); + } + for graft in &control.graft { + let len = graft.encoded_len(); + create_or_add_rpc!(len); + rpc_list + .last_mut() + .expect("Always an element") + .control + .get_or_insert_with(|| empty_control.clone()) + .graft + .push(graft.clone()); + } + for prune in &control.prune { + let len = prune.encoded_len(); + create_or_add_rpc!(len); + rpc_list + .last_mut() + .expect("Always an element") + .control + .get_or_insert_with(|| empty_control.clone()) + .prune + .push(prune.clone()); + } + } else { + let len = control.encoded_len(); + create_or_add_rpc!(len); + rpc_list.last_mut().expect("Always an element").control = Some(control.clone()); + } + } + + Ok(rpc_list.into_iter().map(Arc::new).collect()) } } -impl NetworkBehaviour for Gossipsub { +fn get_ip_addr(addr: &Multiaddr) -> Option { + addr.iter().find_map(|p| match p { + Ip4(addr) => Some(IpAddr::V4(addr)), + Ip6(addr) => Some(IpAddr::V6(addr)), + _ => None, + }) +} + +impl NetworkBehaviour for Gossipsub +where + C: Send + 'static + DataTransform, + F: Send + 'static + TopicSubscriptionFilter, +{ type ProtocolsHandler = GossipsubHandler; type OutEvent = GossipsubEvent; fn new_handler(&mut self) -> Self::ProtocolsHandler { GossipsubHandler::new( - self.config.protocol_id.clone(), - self.config.max_transmit_size, - self.config.validation_mode.clone(), + self.config.protocol_id_prefix().clone(), + self.config.max_transmit_size(), + self.config.validation_mode().clone(), + self.config.support_floodsub(), ) } @@ -1265,8 +2755,14 @@ impl NetworkBehaviour for Gossipsub { Vec::new() } - fn inject_connected(&mut self, id: &PeerId) { - info!("New peer connected: {:?}", id); + fn inject_connected(&mut self, peer_id: &PeerId) { + // Ignore connections from blacklisted peers. + if self.blacklisted_peers.contains(peer_id) { + debug!("Ignoring connection from blacklisted peer: {}", peer_id); + return; + } + + info!("New peer connected: {}", peer_id); // We need to send our subscriptions to the newly-connected node. let mut subscriptions = vec![]; for topic_hash in self.mesh.keys() { @@ -1278,28 +2774,49 @@ impl NetworkBehaviour for Gossipsub { if !subscriptions.is_empty() { // send our subscriptions to the peer - self.send_message( - id.clone(), - GossipsubRpc { - messages: Vec::new(), - subscriptions, - control_msgs: Vec::new(), - }, - ); + if self + .send_message( + peer_id.clone(), + GossipsubRpc { + messages: Vec::new(), + subscriptions, + control_msgs: Vec::new(), + } + .into_protobuf(), + ) + .is_err() + { + error!("Failed to send subscriptions, message too large"); + } } - // For the time being assume all gossipsub peers - self.peer_topics.insert(id.clone(), Default::default()); + // Insert an empty set of the topics of this peer until known. + self.peer_topics.insert(peer_id.clone(), Default::default()); + + // By default we assume a peer is only a floodsub peer. + // + // The protocol negotiation occurs once a message is sent/received. Once this happens we + // update the type of peer that this is in order to determine which kind of routing should + // occur. + self.peer_protocols + .entry(peer_id.clone()) + .or_insert(PeerKind::Floodsub); + + if let Some((peer_score, ..)) = &mut self.peer_score { + peer_score.add_peer(peer_id.clone()); + } } - fn inject_disconnected(&mut self, id: &PeerId) { - // remove from mesh, topic_peers, peer_topic and fanout - debug!("Peer disconnected: {:?}", id); + fn inject_disconnected(&mut self, peer_id: &PeerId) { + // remove from mesh, topic_peers, peer_topic and the fanout + debug!("Peer disconnected: {}", peer_id); { - let topics = match self.peer_topics.get(id) { + let topics = match self.peer_topics.get(peer_id) { Some(topics) => (topics), None => { - warn!("Disconnected node, not in connected nodes"); + if !self.blacklisted_peers.contains(peer_id) { + debug!("Disconnected node, not in connected nodes"); + } return; } }; @@ -1309,72 +2826,251 @@ impl NetworkBehaviour for Gossipsub { // check the mesh for the topic if let Some(mesh_peers) = self.mesh.get_mut(&topic) { // check if the peer is in the mesh and remove it - mesh_peers.remove(id); + mesh_peers.remove(peer_id); } // remove from topic_peers if let Some(peer_list) = self.topic_peers.get_mut(&topic) { - if !peer_list.remove(id) { + if !peer_list.remove(peer_id) { // debugging purposes - warn!("Disconnected node: {:?} not in topic_peers peer list", &id); + warn!( + "Disconnected node: {} not in topic_peers peer list", + peer_id + ); } } else { warn!( - "Disconnected node: {:?} with topic: {:?} not in topic_peers", - &id, &topic + "Disconnected node: {} with topic: {:?} not in topic_peers", + &peer_id, &topic ); } // remove from fanout - self.fanout.get_mut(&topic).map(|peers| peers.remove(id)); + self.fanout + .get_mut(&topic) + .map(|peers| peers.remove(peer_id)); } + + //forget px and outbound status for this peer + self.px_peers.remove(peer_id); + self.outbound_peers.remove(peer_id); } - // remove peer from peer_topics - let was_in = self.peer_topics.remove(id); - debug_assert!(was_in.is_some()); + // Remove peer from peer_topics and peer_protocols + // NOTE: It is possible the peer has already been removed from all mappings if it does not + // support the protocol. + self.peer_topics.remove(peer_id); + self.peer_protocols.remove(peer_id); + + if let Some((peer_score, ..)) = &mut self.peer_score { + peer_score.remove_peer(peer_id); + } } - fn inject_event(&mut self, propagation_source: PeerId, _: ConnectionId, event: GossipsubRpc) { - // Handle subscriptions - // Update connected peers topics - if !event.subscriptions.is_empty() { - self.handle_received_subscriptions(&event.subscriptions, &propagation_source); + fn inject_connection_established( + &mut self, + peer_id: &PeerId, + _: &ConnectionId, + endpoint: &ConnectedPoint, + ) { + // Ignore connections from blacklisted peers. + if self.blacklisted_peers.contains(peer_id) { + return; } - // Handle messages - for message in event.messages { - self.handle_received_message(message, &propagation_source); + // Check if the peer is an outbound peer + if let ConnectedPoint::Dialer { .. } = endpoint { + // Diverging from the go implementation we only want to consider a peer as outbound peer + // if its first connection is outbound. To check if this connection is the first we + // check if the peer isn't connected yet. This only works because the + // `inject_connection_established` event for the first connection gets called immediately + // before `inject_connected` gets called. + if !self.peer_topics.contains_key(peer_id) && !self.px_peers.contains(peer_id) { + // The first connection is outbound and it is not a peer from peer exchange => mark + // it as outbound peer + self.outbound_peers.insert(peer_id.clone()); + } } - // Handle control messages - // group some control messages, this minimises SendEvents (code is simplified to handle each event at a time however) - let mut ihave_msgs = vec![]; - let mut graft_msgs = vec![]; - let mut prune_msgs = vec![]; - for control_msg in event.control_msgs { - match control_msg { - GossipsubControlAction::IHave { - topic_hash, - message_ids, - } => { - ihave_msgs.push((topic_hash, message_ids)); - } - GossipsubControlAction::IWant { message_ids } => { - self.handle_iwant(&propagation_source, message_ids) - } - GossipsubControlAction::Graft { topic_hash } => graft_msgs.push(topic_hash), - GossipsubControlAction::Prune { topic_hash } => prune_msgs.push(topic_hash), + // Add the IP to the peer scoring system + if let Some((peer_score, ..)) = &mut self.peer_score { + if let Some(ip) = get_ip_addr(endpoint.get_remote_address()) { + peer_score.add_ip(&peer_id, ip); + } else { + trace!( + "Couldn't extract ip from endpoint of peer {} with endpoint {:?}", + peer_id, + endpoint + ) } } - if !ihave_msgs.is_empty() { - self.handle_ihave(&propagation_source, ihave_msgs); + } + + fn inject_connection_closed( + &mut self, + peer: &PeerId, + _: &ConnectionId, + endpoint: &ConnectedPoint, + ) { + // Remove IP from peer scoring system + if let Some((peer_score, ..)) = &mut self.peer_score { + if let Some(ip) = get_ip_addr(endpoint.get_remote_address()) { + peer_score.remove_ip(peer, &ip); + } else { + trace!( + "Couldn't extract ip from endpoint of peer {} with endpoint {:?}", + peer, + endpoint + ) + } } - if !graft_msgs.is_empty() { - self.handle_graft(&propagation_source, graft_msgs); + } + + fn inject_address_change( + &mut self, + peer: &PeerId, + _: &ConnectionId, + endpoint_old: &ConnectedPoint, + endpoint_new: &ConnectedPoint, + ) { + // Exchange IP in peer scoring system + if let Some((peer_score, ..)) = &mut self.peer_score { + if let Some(ip) = get_ip_addr(endpoint_old.get_remote_address()) { + peer_score.remove_ip(peer, &ip); + } else { + trace!( + "Couldn't extract ip from endpoint of peer {} with endpoint {:?}", + peer, + endpoint_old + ) + } + if let Some(ip) = get_ip_addr(endpoint_new.get_remote_address()) { + peer_score.add_ip(&peer, ip); + } else { + trace!( + "Couldn't extract ip from endpoint of peer {} with endpoint {:?}", + peer, + endpoint_new + ) + } } - if !prune_msgs.is_empty() { - self.handle_prune(&propagation_source, prune_msgs); + } + + fn inject_event( + &mut self, + propagation_source: PeerId, + _: ConnectionId, + handler_event: HandlerEvent, + ) { + match handler_event { + HandlerEvent::PeerKind(kind) => { + // We have identified the protocol this peer is using + if let PeerKind::NotSupported = kind { + debug!( + "Peer does not support gossipsub protocols. {}", + propagation_source + ); + // We treat this peer as disconnected + self.inject_disconnected(&propagation_source); + } else if let Some(old_kind) = self.peer_protocols.get_mut(&propagation_source) { + // Only change the value if the old value is Floodsub (the default set in + // inject_connected). All other PeerKind changes are ignored. + debug!( + "New peer type found: {} for peer: {}", + kind, propagation_source + ); + if let PeerKind::Floodsub = *old_kind { + *old_kind = kind; + } + } + } + HandlerEvent::Message { + rpc, + invalid_messages, + } => { + // Handle the gossipsub RPC + + // Handle subscriptions + // Update connected peers topics + if !rpc.subscriptions.is_empty() { + self.handle_received_subscriptions(&rpc.subscriptions, &propagation_source); + } + + // Check if peer is graylisted in which case we ignore the event + if let (true, _) = + self.score_below_threshold(&propagation_source, |pst| pst.graylist_threshold) + { + debug!("RPC Dropped from greylisted peer {}", propagation_source); + return; + } + + // Handle any invalid messages from this peer + if self.peer_score.is_some() { + for (raw_message, validation_error) in invalid_messages { + self.handle_invalid_message( + &propagation_source, + raw_message, + validation_error, + ) + } + } else { + // log the invalid messages + for (message, validation_error) in invalid_messages { + warn!( + "Invalid message. Reason: {:?} propagation_peer {} source {:?}", + validation_error, + propagation_source.to_string(), + message.source + ); + } + } + + // Handle messages + for (count, raw_message) in rpc.messages.into_iter().enumerate() { + // Only process the amount of messages the configuration allows. + if self.config.max_messages_per_rpc().is_some() + && Some(count) >= self.config.max_messages_per_rpc() + { + warn!("Received more messages than permitted. Ignoring further messages. Processed: {}", count); + break; + } + self.handle_received_message(raw_message, &propagation_source); + } + + // Handle control messages + // group some control messages, this minimises SendEvents (code is simplified to handle each event at a time however) + let mut ihave_msgs = vec![]; + let mut graft_msgs = vec![]; + let mut prune_msgs = vec![]; + for control_msg in rpc.control_msgs { + match control_msg { + GossipsubControlAction::IHave { + topic_hash, + message_ids, + } => { + ihave_msgs.push((topic_hash, message_ids)); + } + GossipsubControlAction::IWant { message_ids } => { + self.handle_iwant(&propagation_source, message_ids) + } + GossipsubControlAction::Graft { topic_hash } => graft_msgs.push(topic_hash), + GossipsubControlAction::Prune { + topic_hash, + peers, + backoff, + } => prune_msgs.push((topic_hash, peers, backoff)), + } + } + if !ihave_msgs.is_empty() { + self.handle_ihave(&propagation_source, ihave_msgs); + } + if !graft_msgs.is_empty() { + self.handle_graft(&propagation_source, graft_msgs); + } + if !prune_msgs.is_empty() { + self.handle_prune(&propagation_source, prune_msgs); + } + } } } @@ -1418,6 +3114,13 @@ impl NetworkBehaviour for Gossipsub { }); } + // update scores + if let Some((peer_score, _, interval, _)) = &mut self.peer_score { + while let Poll::Ready(Some(())) = interval.poll_next_unpin(cx) { + peer_score.refresh_scores(); + } + } + while let Poll::Ready(Some(())) = self.heartbeat.poll_next_unpin(cx) { self.heartbeat(); } @@ -1426,74 +3129,80 @@ impl NetworkBehaviour for Gossipsub { } } -/// An RPC received/sent. -#[derive(Clone, PartialEq, Eq, Hash)] -pub struct GossipsubRpc { - /// List of messages that were part of this RPC query. - pub messages: Vec, - /// List of subscriptions. - pub subscriptions: Vec, - /// List of Gossipsub control messages. - pub control_msgs: Vec, -} - -impl fmt::Debug for GossipsubRpc { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - let mut b = f.debug_struct("GossipsubRpc"); - if !self.messages.is_empty() { - b.field("messages", &self.messages); - } - if !self.subscriptions.is_empty() { - b.field("subscriptions", &self.subscriptions); - } - if !self.control_msgs.is_empty() { - b.field("control_msgs", &self.control_msgs); - } - b.finish() +/// Helper function to get a subset of random gossipsub peers for a `topic_hash` +/// filtered by the function `f`. The number of peers to get equals the output of `n_map` +/// that gets as input the number of filtered peers. +fn get_random_peers_dynamic( + topic_peers: &HashMap>, + peer_protocols: &HashMap, + topic_hash: &TopicHash, + // maps the number of total peers to the number of selected peers + n_map: impl Fn(usize) -> usize, + mut f: impl FnMut(&PeerId) -> bool, +) -> BTreeSet { + let mut gossip_peers = match topic_peers.get(topic_hash) { + // if they exist, filter the peers by `f` + Some(peer_list) => peer_list + .iter() + .cloned() + .filter(|p| { + f(p) && match peer_protocols.get(p) { + Some(PeerKind::Gossipsub) => true, + Some(PeerKind::Gossipsubv1_1) => true, + _ => false, + } + }) + .collect(), + None => Vec::new(), + }; + + // if we have less than needed, return them + let n = n_map(gossip_peers.len()); + if gossip_peers.len() <= n { + debug!("RANDOM PEERS: Got {:?} peers", gossip_peers.len()); + return gossip_peers.into_iter().collect(); } -} -/// Event that can happen on the gossipsub behaviour. -#[derive(Clone, Debug)] -pub enum GossipsubEvent { - /// A message has been received. This contains the PeerId that we received the message from, - /// the message id (used if the application layer needs to propagate the message) and the - /// message itself. - Message(PeerId, MessageId, GossipsubMessage), + // we have more peers than needed, shuffle them and return n of them + let mut rng = thread_rng(); + gossip_peers.partial_shuffle(&mut rng, n); - /// A remote subscribed to a topic. - Subscribed { - /// Remote that has subscribed. - peer_id: PeerId, - /// The topic it has subscribed to. - topic: TopicHash, - }, + debug!("RANDOM PEERS: Got {:?} peers", n); - /// A remote unsubscribed from a topic. - Unsubscribed { - /// Remote that has unsubscribed. - peer_id: PeerId, - /// The topic it has subscribed from. - topic: TopicHash, - }, + gossip_peers.into_iter().take(n).collect() +} + +/// Helper function to get a set of `n` random gossipsub peers for a `topic_hash` +/// filtered by the function `f`. +fn get_random_peers( + topic_peers: &HashMap>, + peer_protocols: &HashMap, + topic_hash: &TopicHash, + n: usize, + f: impl FnMut(&PeerId) -> bool, +) -> BTreeSet { + get_random_peers_dynamic(topic_peers, peer_protocols, topic_hash, |_| n, f) } /// Validates the combination of signing, privacy and message validation to ensure the /// configuration will not reject published messages. -fn validate_config(authenticity: &MessageAuthenticity, validation_mode: &ValidationMode) { +fn validate_config( + authenticity: &MessageAuthenticity, + validation_mode: &ValidationMode, +) -> Result<(), &'static str> { match validation_mode { ValidationMode::Anonymous => { if authenticity.is_signing() { - panic!("Cannot enable message signing with an Anonymous validation mode. Consider changing either the ValidationMode or MessageAuthenticity"); + return Err("Cannot enable message signing with an Anonymous validation mode. Consider changing either the ValidationMode or MessageAuthenticity"); } if !authenticity.is_anonymous() { - panic!("Published messages contain an author but incoming messages with an author will be rejected. Consider adjusting the validation or privacy settings in the config"); + return Err("Published messages contain an author but incoming messages with an author will be rejected. Consider adjusting the validation or privacy settings in the config"); } } ValidationMode::Strict => { if !authenticity.is_signing() { - panic!( + return Err( "Messages will be published unsigned and incoming unsigned messages will be rejected. Consider adjusting the validation or privacy settings in the config" @@ -1502,35 +3211,199 @@ fn validate_config(authenticity: &MessageAuthenticity, validation_mode: &Validat } _ => {} } + Ok(()) } - - -impl fmt::Debug for Gossipsub { +impl fmt::Debug for Gossipsub { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("Gossipsub") - .field("config", &self.config) - .field("events", &self.events) - .field("control_pool", &self.control_pool) - .field("publish_config", &self.publish_config) - .field("topic_peers", &self.topic_peers) - .field("peer_topics", &self.peer_topics) - .field("mesh", &self.mesh) - .field("fanout", &self.fanout) - .field("fanout_last_pub", &self.fanout_last_pub) - .field("mcache", &self.mcache) - .field("heartbeat", &self.heartbeat) - .finish() + .field("config", &self.config) + .field("events", &self.events) + .field("control_pool", &self.control_pool) + .field("publish_config", &self.publish_config) + .field("topic_peers", &self.topic_peers) + .field("peer_topics", &self.peer_topics) + .field("mesh", &self.mesh) + .field("fanout", &self.fanout) + .field("fanout_last_pub", &self.fanout_last_pub) + .field("mcache", &self.mcache) + .field("heartbeat", &self.heartbeat) + .finish() } } impl fmt::Debug for PublishConfig { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { match self { - PublishConfig::Signing { author, .. } => f.write_fmt(format_args!("PublishConfig::Signing({})", author)), - PublishConfig::Author(author) => f.write_fmt(format_args!("PublishConfig::Author({})", author)), + PublishConfig::Signing { author, .. } => { + f.write_fmt(format_args!("PublishConfig::Signing({})", author)) + } + PublishConfig::Author(author) => { + f.write_fmt(format_args!("PublishConfig::Author({})", author)) + } PublishConfig::RandomAuthor => f.write_fmt(format_args!("PublishConfig::RandomAuthor")), PublishConfig::Anonymous => f.write_fmt(format_args!("PublishConfig::Anonymous")), } } } + +#[cfg(test)] +mod local_test { + use super::*; + use crate::IdentTopic; + use futures_codec::Encoder; + use quickcheck::*; + use rand::Rng; + + fn empty_rpc() -> GossipsubRpc { + GossipsubRpc { + subscriptions: Vec::new(), + messages: Vec::new(), + control_msgs: Vec::new(), + } + } + + fn test_message() -> RawGossipsubMessage { + RawGossipsubMessage { + source: Some(PeerId::random()), + data: vec![0; 100], + sequence_number: None, + topic: TopicHash::from_raw("test_topic"), + signature: None, + key: None, + validated: false, + } + } + + fn test_subscription() -> GossipsubSubscription { + GossipsubSubscription { + action: GossipsubSubscriptionAction::Subscribe, + topic_hash: IdentTopic::new("TestTopic").hash(), + } + } + + fn test_control() -> GossipsubControlAction { + GossipsubControlAction::IHave { + topic_hash: IdentTopic::new("TestTopic").hash(), + message_ids: vec![MessageId(vec![12u8]); 5], + } + } + + impl Arbitrary for GossipsubRpc { + fn arbitrary(g: &mut G) -> Self { + let mut rpc = empty_rpc(); + + for _ in 0..g.gen_range(0, 10) { + rpc.subscriptions.push(test_subscription()); + } + for _ in 0..g.gen_range(0, 10) { + rpc.messages.push(test_message()); + } + for _ in 0..g.gen_range(0, 10) { + rpc.control_msgs.push(test_control()); + } + rpc + } + } + + #[test] + /// Tests RPC message fragmentation + fn test_message_fragmentation_deterministic() { + let max_transmit_size = 500; + let config = crate::GossipsubConfigBuilder::default() + .max_transmit_size(max_transmit_size) + .validation_mode(ValidationMode::Permissive) + .build() + .unwrap(); + let gs: Gossipsub = Gossipsub::new(MessageAuthenticity::RandomAuthor, config).unwrap(); + + // Message under the limit should be fine. + let mut rpc = empty_rpc(); + rpc.messages.push(test_message()); + + let mut rpc_proto = rpc.clone().into_protobuf(); + let fragmented_messages = gs.fragment_message(Arc::new(rpc_proto.clone())).unwrap(); + assert_eq!( + fragmented_messages, + vec![Arc::new(rpc_proto.clone())], + "Messages under the limit shouldn't be fragmented" + ); + + // Messages over the limit should be split + + while rpc_proto.encoded_len() < max_transmit_size { + rpc.messages.push(test_message()); + rpc_proto = rpc.clone().into_protobuf(); + } + + let fragmented_messages = gs + .fragment_message(Arc::new(rpc_proto)) + .expect("Should be able to fragment the messages"); + + assert!( + fragmented_messages.len() > 1, + "the message should be fragmented" + ); + + // all fragmented messages should be under the limit + for message in fragmented_messages { + assert!( + message.encoded_len() < max_transmit_size, + "all messages should be less than the transmission size" + ); + } + } + + #[test] + fn test_message_fragmentation() { + fn prop(rpc: GossipsubRpc) { + let max_transmit_size = 500; + let config = crate::GossipsubConfigBuilder::default() + .max_transmit_size(max_transmit_size) + .validation_mode(ValidationMode::Permissive) + .build() + .unwrap(); + let gs: Gossipsub = Gossipsub::new(MessageAuthenticity::RandomAuthor, config).unwrap(); + + let mut length_codec = unsigned_varint::codec::UviBytes::default(); + length_codec.set_max_len(max_transmit_size); + let mut codec = + crate::protocol::GossipsubCodec::new(length_codec, ValidationMode::Permissive); + + let rpc_proto = rpc.into_protobuf(); + let fragmented_messages = gs + .fragment_message(Arc::new(rpc_proto.clone())) + .expect("Messages must be valid"); + + if rpc_proto.encoded_len() < max_transmit_size { + assert_eq!( + fragmented_messages.len(), + 1, + "the message should not be fragmented" + ); + } else { + assert!( + fragmented_messages.len() > 1, + "the message should be fragmented" + ); + } + + // all fragmented messages should be under the limit + for message in fragmented_messages { + assert!( + message.encoded_len() < max_transmit_size, + "all messages should be less than the transmission size: list size {} max size{}", message.encoded_len(), max_transmit_size + ); + + // ensure they can all be encoded + let mut buf = bytes::BytesMut::with_capacity(message.encoded_len()); + codec + .encode(Arc::try_unwrap(message).unwrap(), &mut buf) + .unwrap() + } + } + QuickCheck::new() + .max_tests(100) + .quickcheck(prop as fn(_) -> _) + } +} diff --git a/protocols/gossipsub/src/behaviour/tests.rs b/protocols/gossipsub/src/behaviour/tests.rs index 8fee8161a7e..ae4c83835fb 100644 --- a/protocols/gossipsub/src/behaviour/tests.rs +++ b/protocols/gossipsub/src/behaviour/tests.rs @@ -18,59 +18,311 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -// collection of tests for the gossipsub network behaviour +// Collection of tests for the gossipsub network behaviour -#[cfg(test)] mod tests { - use super::super::*; + use byteorder::{BigEndian, ByteOrder}; + use std::thread::sleep; + use std::time::Duration; - // helper functions for testing + use async_std::net::Ipv4Addr; + use rand::Rng; - // This function generates `peer_no` random PeerId's, subscribes to `topics` and subscribes the - // injected nodes to all topics if `to_subscribe` is set. All nodes are considered gossipsub nodes. - fn build_and_inject_nodes( + use crate::{ + GossipsubConfig, GossipsubConfigBuilder, GossipsubMessage, IdentTopic as Topic, + TopicScoreParams, + }; + + use super::super::*; + use crate::error::ValidationError; + use crate::subscription_filter::WhitelistSubscriptionFilter; + use crate::transform::{DataTransform, IdentityTransform}; + use crate::types::FastMessageId; + use std::collections::hash_map::DefaultHasher; + use std::hash::{Hash, Hasher}; + + #[derive(Default, Builder, Debug)] + #[builder(default)] + 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, + { peer_no: usize, topics: Vec, to_subscribe: bool, - ) -> (Gossipsub, Vec, Vec) { - let keypair = libp2p_core::identity::Keypair::generate_secp256k1(); - // generate a default GossipsubConfig with signing - let gs_config = GossipsubConfig::default(); - // create a gossipsub struct - let mut gs: Gossipsub = Gossipsub::new(MessageAuthenticity::Signed(keypair), gs_config); + gs_config: GossipsubConfig, + explicit: usize, + outbound: usize, + scoring: Option<(PeerScoreParams, PeerScoreThresholds)>, + data_transform: D, + subscription_filter: F, + } + + impl InjectNodes + where + D: DataTransform + Default + Clone + Send + 'static, + F: TopicSubscriptionFilter + Clone + Default + Send + 'static, + { + pub fn create_network(self) -> (Gossipsub, Vec, Vec) { + let keypair = libp2p_core::identity::Keypair::generate_secp256k1(); + // create a gossipsub struct + let mut gs: Gossipsub = Gossipsub::new_with_subscription_filter_and_transform( + MessageAuthenticity::Signed(keypair), + self.gs_config, + self.subscription_filter, + self.data_transform, + ) + .unwrap(); - let mut topic_hashes = vec![]; + if let Some((scoring_params, scoring_thresholds)) = self.scoring { + gs.with_peer_score(scoring_params, scoring_thresholds) + .unwrap(); + } + + let mut topic_hashes = vec![]; + + // subscribe to the topics + for t in self.topics { + let topic = Topic::new(t); + gs.subscribe(&topic).unwrap(); + topic_hashes.push(topic.hash().clone()); + } - // subscribe to the topics - for t in topics { - let topic = Topic::new(t); - gs.subscribe(topic.clone()); - topic_hashes.push(topic.no_hash().clone()); + // build and connect peer_no random peers + let mut peers = vec![]; + + let empty = vec![]; + for i in 0..self.peer_no { + peers.push(add_peer( + &mut gs, + if self.to_subscribe { + &topic_hashes + } else { + &empty + }, + i < self.outbound, + i < self.explicit, + )); + } + + (gs, peers, topic_hashes) } + } - // build and connect peer_no random peers - let mut peers = vec![]; + impl InjectNodesBuilder + where + D: DataTransform + Default + Clone + Send + 'static, + F: TopicSubscriptionFilter + Clone + Default + Send + 'static, + { + pub fn create_network(&self) -> (Gossipsub, Vec, Vec) { + self.build().unwrap().create_network() + } + } - for _ in 0..peer_no { - let peer = PeerId::random(); - peers.push(peer.clone()); - ::inject_connected(&mut gs, &peer); - if to_subscribe { - gs.handle_received_subscriptions( - &topic_hashes - .iter() - .cloned() - .map(|t| GossipsubSubscription { - action: GossipsubSubscriptionAction::Subscribe, - topic_hash: t, - }) + fn inject_nodes() -> InjectNodesBuilder + where + D: DataTransform + Default + Clone + Send + 'static, + F: TopicSubscriptionFilter + Clone + Default + Send + 'static, + { + InjectNodesBuilder::default() + } + + fn inject_nodes1() -> InjectNodesBuilder { + inject_nodes() + } + + // helper functions for testing + + fn add_peer( + gs: &mut Gossipsub, + topic_hashes: &Vec, + outbound: bool, + explicit: bool, + ) -> PeerId + where + D: DataTransform + Default + Clone + Send + 'static, + F: TopicSubscriptionFilter + Clone + Default + Send + 'static, + { + add_peer_with_addr(gs, topic_hashes, outbound, explicit, Multiaddr::empty()) + } + + fn add_peer_with_addr( + gs: &mut Gossipsub, + topic_hashes: &Vec, + outbound: bool, + explicit: bool, + address: Multiaddr, + ) -> PeerId + where + D: DataTransform + Default + Clone + Send + 'static, + F: TopicSubscriptionFilter + Clone + Default + Send + 'static, + { + add_peer_with_addr_and_kind( + gs, + topic_hashes, + outbound, + explicit, + address, + Some(PeerKind::Gossipsubv1_1), + ) + } + + fn add_peer_with_addr_and_kind( + gs: &mut Gossipsub, + topic_hashes: &Vec, + outbound: bool, + explicit: bool, + address: Multiaddr, + kind: Option, + ) -> PeerId + where + D: DataTransform + Default + Clone + Send + 'static, + F: TopicSubscriptionFilter + Clone + Default + Send + 'static, + { + let peer = PeerId::random(); + //peers.push(peer.clone()); + gs.inject_connection_established( + &peer, + &ConnectionId::new(0), + &if outbound { + ConnectedPoint::Dialer { address } + } else { + ConnectedPoint::Listener { + local_addr: Multiaddr::empty(), + send_back_addr: address, + } + }, + ); + as NetworkBehaviour>::inject_connected(gs, &peer); + if let Some(kind) = kind { + gs.inject_event( + peer.clone(), + ConnectionId::new(1), + HandlerEvent::PeerKind(kind), + ); + } + if explicit { + gs.add_explicit_peer(&peer); + } + if !topic_hashes.is_empty() { + gs.handle_received_subscriptions( + &topic_hashes + .iter() + .cloned() + .map(|t| GossipsubSubscription { + action: GossipsubSubscriptionAction::Subscribe, + topic_hash: t, + }) + .collect::>(), + &peer, + ); + } + peer + } + + // Converts a protobuf message into a gossipsub message for reading the Gossipsub event queue. + fn proto_to_message(rpc: &crate::rpc_proto::Rpc) -> GossipsubRpc { + // Store valid messages. + let mut messages = Vec::with_capacity(rpc.publish.len()); + let rpc = rpc.clone(); + for message in rpc.publish.into_iter() { + messages.push(RawGossipsubMessage { + source: message.from.map(|x| PeerId::from_bytes(&x).unwrap()), + data: message.data.unwrap_or_default(), + sequence_number: message.seqno.map(|x| BigEndian::read_u64(&x)), // don't inform the application + topic: TopicHash::from_raw(message.topic), + signature: message.signature, // don't inform the application + key: None, + validated: false, + }); + } + let mut control_msgs = Vec::new(); + if let Some(rpc_control) = rpc.control { + // Collect the gossipsub control messages + let ihave_msgs: Vec = rpc_control + .ihave + .into_iter() + .map(|ihave| GossipsubControlAction::IHave { + topic_hash: TopicHash::from_raw(ihave.topic_id.unwrap_or_default()), + message_ids: ihave + .message_ids + .into_iter() + .map(MessageId::from) .collect::>(), - &peer, - ); - }; + }) + .collect(); + + let iwant_msgs: Vec = rpc_control + .iwant + .into_iter() + .map(|iwant| GossipsubControlAction::IWant { + message_ids: iwant + .message_ids + .into_iter() + .map(MessageId::from) + .collect::>(), + }) + .collect(); + + let graft_msgs: Vec = rpc_control + .graft + .into_iter() + .map(|graft| GossipsubControlAction::Graft { + topic_hash: TopicHash::from_raw(graft.topic_id.unwrap_or_default()), + }) + .collect(); + + let mut prune_msgs = Vec::new(); + + for prune in rpc_control.prune { + // filter out invalid peers + let peers = prune + .peers + .into_iter() + .filter_map(|info| { + info.peer_id + .and_then(|id| PeerId::from_bytes(&id).ok()) + .map(|peer_id| + //TODO signedPeerRecord, see https://github.com/libp2p/specs/pull/217 + PeerInfo { + peer_id: Some(peer_id), + }) + }) + .collect::>(); + + let topic_hash = TopicHash::from_raw(prune.topic_id.unwrap_or_default()); + prune_msgs.push(GossipsubControlAction::Prune { + topic_hash, + peers, + backoff: prune.backoff, + }); + } + + control_msgs.extend(ihave_msgs); + control_msgs.extend(iwant_msgs); + control_msgs.extend(graft_msgs); + control_msgs.extend(prune_msgs); } - return (gs, peers, topic_hashes); + GossipsubRpc { + messages, + subscriptions: rpc + .subscriptions + .into_iter() + .map(|sub| GossipsubSubscription { + action: if Some(true) == sub.subscribe { + GossipsubSubscriptionAction::Subscribe + } else { + GossipsubSubscriptionAction::Unsubscribe + }, + topic_hash: TopicHash::from_raw(sub.topic_id.unwrap_or_default()), + }) + .collect(), + control_msgs, + } } #[test] @@ -82,7 +334,11 @@ mod tests { // - run JOIN(topic) let subscribe_topic = vec![String::from("test_subscribe")]; - let (gs, _, topic_hashes) = build_and_inject_nodes(20, subscribe_topic, true); + let (gs, _, topic_hashes) = inject_nodes1() + .peer_no(20) + .topics(subscribe_topic) + .to_subscribe(true) + .create_network(); assert!( gs.mesh.get(&topic_hashes[0]).is_some(), @@ -96,10 +352,8 @@ mod tests { .fold(vec![], |mut collected_subscriptions, e| match e { NetworkBehaviourAction::NotifyHandler { event, .. } => { for s in &event.subscriptions { - match s.action { - GossipsubSubscriptionAction::Subscribe => { - collected_subscriptions.push(s.clone()) - } + match s.subscribe { + Some(true) => collected_subscriptions.push(s.clone()), _ => {} }; } @@ -130,7 +384,11 @@ mod tests { .collect::>(); // subscribe to topic_strings - let (mut gs, _, topic_hashes) = build_and_inject_nodes(20, topic_strings, true); + let (mut gs, _, topic_hashes) = inject_nodes1() + .peer_no(20) + .topics(topic_strings) + .to_subscribe(true) + .create_network(); for topic_hash in &topic_hashes { assert!( @@ -145,11 +403,11 @@ mod tests { // unsubscribe from both topics assert!( - gs.unsubscribe(topics[0].clone()), + gs.unsubscribe(&topics[0]).unwrap(), "should be able to unsubscribe successfully from each topic", ); assert!( - gs.unsubscribe(topics[1].clone()), + gs.unsubscribe(&topics[1]).unwrap(), "should be able to unsubscribe successfully from each topic", ); @@ -159,10 +417,8 @@ mod tests { .fold(vec![], |mut collected_subscriptions, e| match e { NetworkBehaviourAction::NotifyHandler { event, .. } => { for s in &event.subscriptions { - match s.action { - GossipsubSubscriptionAction::Unsubscribe => { - collected_subscriptions.push(s.clone()) - } + match s.subscribe { + Some(false) => collected_subscriptions.push(s.clone()), _ => {} }; } @@ -204,21 +460,25 @@ mod tests { .map(|t| Topic::new(t.clone())) .collect::>(); - let (mut gs, _, topic_hashes) = build_and_inject_nodes(20, topic_strings, true); + let (mut gs, _, topic_hashes) = inject_nodes1() + .peer_no(20) + .topics(topic_strings) + .to_subscribe(true) + .create_network(); // unsubscribe, then call join to invoke functionality assert!( - gs.unsubscribe(topics[0].clone()), + gs.unsubscribe(&topics[0]).unwrap(), "should be able to unsubscribe successfully" ); assert!( - gs.unsubscribe(topics[1].clone()), + gs.unsubscribe(&topics[1]).unwrap(), "should be able to unsubscribe successfully" ); // re-subscribe - there should be peers associated with the topic assert!( - gs.subscribe(topics[0].clone()), + gs.subscribe(&topics[0]).unwrap(), "should be able to subscribe successfully" ); @@ -263,7 +523,7 @@ mod tests { } // subscribe to topic1 - gs.subscribe(topics[1].clone()); + gs.subscribe(&topics[1]).unwrap(); // the three new peers should have been added, along with 3 more from the pool. assert!( @@ -289,14 +549,24 @@ mod tests { /// Test local node publish to subscribed topic #[test] - fn test_publish() { + fn test_publish_without_flood_publishing() { // node should: // - Send publish message to all peers // - Insert message into gs.mcache and gs.received + //turn off flood publish to test old behaviour + let config = GossipsubConfigBuilder::default() + .flood_publish(false) + .build() + .unwrap(); + let publish_topic = String::from("test_publish"); - let (mut gs, _, topic_hashes) = - build_and_inject_nodes(20, vec![publish_topic.clone()], true); + let (mut gs, _, topic_hashes) = inject_nodes1() + .peer_no(20) + .topics(vec![publish_topic.clone()]) + .to_subscribe(true) + .gs_config(config) + .create_network(); assert!( gs.mesh.get(&topic_hashes[0]).is_some(), @@ -312,8 +582,7 @@ mod tests { // publish on topic let publish_data = vec![0; 42]; - gs.publish(&Topic::new(publish_topic), publish_data) - .unwrap(); + gs.publish(Topic::new(publish_topic), publish_data).unwrap(); // Collect all publish messages let publishes = gs @@ -321,6 +590,7 @@ mod tests { .iter() .fold(vec![], |mut collected_publish, e| match e { NetworkBehaviourAction::NotifyHandler { event, .. } => { + let event = proto_to_message(event); for s in &event.messages { collected_publish.push(s.clone()); } @@ -329,13 +599,23 @@ mod tests { _ => collected_publish, }); - let msg_id = - (gs.config.message_id_fn)(&publishes.first().expect("Should contain > 0 entries")); + // Transform the inbound message + let message = &gs + .data_transform + .inbound_transform( + publishes + .first() + .expect("Should contain > 0 entries") + .clone(), + ) + .unwrap(); - let config = GossipsubConfig::default(); + let msg_id = gs.config.message_id(&message); + + let config: GossipsubConfig = GossipsubConfig::default(); assert_eq!( publishes.len(), - config.mesh_n_low, + config.mesh_n_low(), "Should send a publish message to all known peers" ); @@ -352,9 +632,20 @@ mod tests { // - Populate fanout peers // - Send publish message to fanout peers // - Insert message into gs.mcache and gs.received + + //turn off flood publish to test fanout behaviour + let config = GossipsubConfigBuilder::default() + .flood_publish(false) + .build() + .unwrap(); + let fanout_topic = String::from("test_fanout"); - let (mut gs, _, topic_hashes) = - build_and_inject_nodes(20, vec![fanout_topic.clone()], true); + let (mut gs, _, topic_hashes) = inject_nodes1() + .peer_no(20) + .topics(vec![fanout_topic.clone()]) + .to_subscribe(true) + .gs_config(config) + .create_network(); assert!( gs.mesh.get(&topic_hashes[0]).is_some(), @@ -362,13 +653,13 @@ mod tests { ); // Unsubscribe from topic assert!( - gs.unsubscribe(Topic::new(fanout_topic.clone())), + gs.unsubscribe(&Topic::new(fanout_topic.clone())).unwrap(), "should be able to unsubscribe successfully from topic" ); // Publish on unsubscribed topic let publish_data = vec![0; 42]; - gs.publish(&Topic::new(fanout_topic.clone()), publish_data) + gs.publish(Topic::new(fanout_topic.clone()), publish_data) .unwrap(); assert_eq!( @@ -376,7 +667,7 @@ mod tests { .get(&TopicHash::from_raw(fanout_topic.clone())) .unwrap() .len(), - gs.config.mesh_n, + gs.config.mesh_n(), "Fanout should contain `mesh_n` peers for fanout topic" ); @@ -386,6 +677,7 @@ mod tests { .iter() .fold(vec![], |mut collected_publish, e| match e { NetworkBehaviourAction::NotifyHandler { event, .. } => { + let event = proto_to_message(event); for s in &event.messages { collected_publish.push(s.clone()); } @@ -394,12 +686,22 @@ mod tests { _ => collected_publish, }); - let msg_id = - (gs.config.message_id_fn)(&publishes.first().expect("Should contain > 0 entries")); + // Transform the inbound message + let message = &gs + .data_transform + .inbound_transform( + publishes + .first() + .expect("Should contain > 0 entries") + .clone(), + ) + .unwrap(); + + let msg_id = gs.config.message_id(&message); assert_eq!( publishes.len(), - gs.config.mesh_n, + gs.config.mesh_n(), "Should send a publish message to `mesh_n` fanout peers" ); @@ -412,15 +714,15 @@ mod tests { #[test] /// Test the gossipsub NetworkBehaviour peer connection logic. fn test_inject_connected() { - let (gs, peers, topic_hashes) = build_and_inject_nodes( - 20, - vec![String::from("topic1"), String::from("topic2")], - true, - ); + let (gs, peers, topic_hashes) = inject_nodes1() + .peer_no(20) + .topics(vec![String::from("topic1"), String::from("topic2")]) + .to_subscribe(true) + .create_network(); // check that our subscriptions are sent to each of the peers // collect all the SendEvents - let send_events: Vec<&NetworkBehaviourAction, GossipsubEvent>> = gs + let send_events: Vec<&NetworkBehaviourAction, GossipsubEvent>> = gs .events .iter() .filter(|e| match e { @@ -473,7 +775,11 @@ mod tests { .iter() .map(|&t| String::from(t)) .collect(); - let (mut gs, peers, topic_hashes) = build_and_inject_nodes(20, topics, false); + let (mut gs, peers, topic_hashes) = inject_nodes1() + .peer_no(20) + .topics(topics) + .to_subscribe(false) + .create_network(); // The first peer sends 3 subscriptions and 1 unsubscription let mut subscriptions = topic_hashes[..3] @@ -550,13 +856,15 @@ mod tests { /// Test Gossipsub.get_random_peers() function fn test_get_random_peers() { // generate a default GossipsubConfig - let mut gs_config = GossipsubConfig::default(); - gs_config.validation_mode = ValidationMode::Anonymous; + let gs_config = GossipsubConfigBuilder::default() + .validation_mode(ValidationMode::Anonymous) + .build() + .unwrap(); // create a gossipsub struct - let mut gs: Gossipsub = Gossipsub::new(MessageAuthenticity::Anonymous, gs_config); + let mut gs: Gossipsub = Gossipsub::new(MessageAuthenticity::Anonymous, gs_config).unwrap(); // create a topic and fill it with some peers - let topic_hash = Topic::new("Test".into()).no_hash().clone(); + let topic_hash = Topic::new("Test").hash().clone(); let mut peers = vec![]; for _ in 0..20 { peers.push(PeerId::random()) @@ -565,49 +873,79 @@ mod tests { gs.topic_peers .insert(topic_hash.clone(), peers.iter().cloned().collect()); - let random_peers = Gossipsub::get_random_peers(&gs.topic_peers, &topic_hash, 5, |_| true); + gs.peer_protocols = peers + .iter() + .map(|p| (p.clone(), PeerKind::Gossipsubv1_1)) + .collect(); + + let random_peers = + get_random_peers(&gs.topic_peers, &gs.peer_protocols, &topic_hash, 5, |_| { + true + }); assert_eq!(random_peers.len(), 5, "Expected 5 peers to be returned"); - let random_peers = Gossipsub::get_random_peers(&gs.topic_peers, &topic_hash, 30, |_| true); + let random_peers = + get_random_peers(&gs.topic_peers, &gs.peer_protocols, &topic_hash, 30, |_| { + true + }); assert!(random_peers.len() == 20, "Expected 20 peers to be returned"); assert!( random_peers == peers.iter().cloned().collect(), "Expected no shuffling" ); - let random_peers = Gossipsub::get_random_peers(&gs.topic_peers, &topic_hash, 20, |_| true); + let random_peers = + get_random_peers(&gs.topic_peers, &gs.peer_protocols, &topic_hash, 20, |_| { + true + }); assert!(random_peers.len() == 20, "Expected 20 peers to be returned"); assert!( random_peers == peers.iter().cloned().collect(), "Expected no shuffling" ); - let random_peers = Gossipsub::get_random_peers(&gs.topic_peers, &topic_hash, 0, |_| true); + let random_peers = + get_random_peers(&gs.topic_peers, &gs.peer_protocols, &topic_hash, 0, |_| { + true + }); assert!(random_peers.len() == 0, "Expected 0 peers to be returned"); // test the filter - let random_peers = Gossipsub::get_random_peers(&gs.topic_peers, &topic_hash, 5, |_| false); + let random_peers = + get_random_peers(&gs.topic_peers, &gs.peer_protocols, &topic_hash, 5, |_| { + false + }); assert!(random_peers.len() == 0, "Expected 0 peers to be returned"); - let random_peers = Gossipsub::get_random_peers(&gs.topic_peers, &topic_hash, 10, { - |peer| peers.contains(peer) - }); + let random_peers = + get_random_peers(&gs.topic_peers, &gs.peer_protocols, &topic_hash, 10, { + |peer| peers.contains(peer) + }); assert!(random_peers.len() == 10, "Expected 10 peers to be returned"); } /// Tests that the correct message is sent when a peer asks for a message in our cache. #[test] fn test_handle_iwant_msg_cached() { - let (mut gs, peers, _) = build_and_inject_nodes(20, Vec::new(), true); + let (mut gs, peers, _) = inject_nodes1() + .peer_no(20) + .topics(Vec::new()) + .to_subscribe(true) + .create_network(); - let id = gs.config.message_id_fn; - - let message = GossipsubMessage { + let raw_message = RawGossipsubMessage { source: Some(peers[11].clone()), data: vec![1, 2, 3, 4], sequence_number: Some(1u64), - topics: Vec::new(), + topic: TopicHash::from_raw("topic"), signature: None, key: None, validated: true, }; - let msg_id = id(&message); - gs.mcache.put(message.clone()); + + // Transform the inbound message + let message = &gs + .data_transform + .inbound_transform(raw_message.clone()) + .unwrap(); + + let msg_id = gs.config.message_id(&message); + gs.mcache.put(&msg_id, raw_message); gs.handle_iwant(&peers[7], vec![msg_id.clone()]); @@ -617,6 +955,7 @@ mod tests { .iter() .fold(vec![], |mut collected_messages, e| match e { NetworkBehaviourAction::NotifyHandler { event, .. } => { + let event = proto_to_message(event); for c in &event.messages { collected_messages.push(c.clone()) } @@ -626,7 +965,10 @@ mod tests { }); assert!( - sent_messages.iter().any(|msg| id(msg) == msg_id), + sent_messages + .iter() + .map(|msg| gs.data_transform.inbound_transform(msg.clone()).unwrap()) + .any(|msg| gs.config.message_id(&msg) == msg_id), "Expected the cached message to be sent to an IWANT peer" ); } @@ -634,22 +976,32 @@ mod tests { /// Tests that messages are sent correctly depending on the shifting of the message cache. #[test] fn test_handle_iwant_msg_cached_shifted() { - let (mut gs, peers, _) = build_and_inject_nodes(20, Vec::new(), true); + let (mut gs, peers, _) = inject_nodes1() + .peer_no(20) + .topics(Vec::new()) + .to_subscribe(true) + .create_network(); - let id = gs.config.message_id_fn; // perform 10 memshifts and check that it leaves the cache for shift in 1..10 { - let message = GossipsubMessage { + let raw_message = RawGossipsubMessage { source: Some(peers[11].clone()), data: vec![1, 2, 3, 4], sequence_number: Some(shift), - topics: Vec::new(), + topic: TopicHash::from_raw("topic"), signature: None, key: None, validated: true, }; - let msg_id = id(&message); - gs.mcache.put(message.clone()); + + // Transform the inbound message + let message = &gs + .data_transform + .inbound_transform(raw_message.clone()) + .unwrap(); + + let msg_id = gs.config.message_id(&message); + gs.mcache.put(&msg_id, raw_message); for _ in 0..shift { gs.mcache.shift(); } @@ -659,7 +1011,12 @@ mod tests { // is the message is being sent? let message_exists = gs.events.iter().any(|e| match e { NetworkBehaviourAction::NotifyHandler { event, .. } => { - event.messages.iter().any(|msg| id(msg) == msg_id) + let event = proto_to_message(event); + event + .messages + .iter() + .map(|msg| gs.data_transform.inbound_transform(msg.clone()).unwrap()) + .any(|msg| gs.config.message_id(&msg) == msg_id) } _ => false, }); @@ -681,7 +1038,11 @@ mod tests { #[test] // tests that an event is not created when a peers asks for a message not in our cache fn test_handle_iwant_msg_not_cached() { - let (mut gs, peers, _) = build_and_inject_nodes(20, Vec::new(), true); + let (mut gs, peers, _) = inject_nodes1() + .peer_no(20) + .topics(Vec::new()) + .to_subscribe(true) + .create_network(); let events_before = gs.events.len(); gs.handle_iwant(&peers[7], vec![MessageId::new(b"unknown id")]); @@ -696,8 +1057,11 @@ mod tests { #[test] // tests that an event is created when a peer shares that it has a message we want fn test_handle_ihave_subscribed_and_msg_not_cached() { - let (mut gs, peers, topic_hashes) = - build_and_inject_nodes(20, vec![String::from("topic1")], true); + let (mut gs, peers, topic_hashes) = inject_nodes1() + .peer_no(20) + .topics(vec![String::from("topic1")]) + .to_subscribe(true) + .create_network(); gs.handle_ihave( &peers[7], @@ -725,8 +1089,11 @@ mod tests { // tests that an event is not created when a peer shares that it has a message that // we already have fn test_handle_ihave_subscribed_and_msg_cached() { - let (mut gs, peers, topic_hashes) = - build_and_inject_nodes(20, vec![String::from("topic1")], true); + let (mut gs, peers, topic_hashes) = inject_nodes1() + .peer_no(20) + .topics(vec![String::from("topic1")]) + .to_subscribe(true) + .create_network(); let msg_id = MessageId::new(b"known id"); @@ -744,7 +1111,11 @@ mod tests { // test that an event is not created when a peer shares that it has a message in // a topic that we are not subscribed to fn test_handle_ihave_not_subscribed() { - let (mut gs, peers, _) = build_and_inject_nodes(20, vec![], true); + let (mut gs, peers, _) = inject_nodes1() + .peer_no(20) + .topics(vec![]) + .to_subscribe(true) + .create_network(); let events_before = gs.events.len(); gs.handle_ihave( @@ -766,8 +1137,11 @@ mod tests { // tests that a peer is added to our mesh when we are both subscribed // to the same topic fn test_handle_graft_is_subscribed() { - let (mut gs, peers, topic_hashes) = - build_and_inject_nodes(20, vec![String::from("topic1")], true); + let (mut gs, peers, topic_hashes) = inject_nodes1() + .peer_no(20) + .topics(vec![String::from("topic1")]) + .to_subscribe(true) + .create_network(); gs.handle_graft(&peers[7], topic_hashes.clone()); @@ -781,8 +1155,11 @@ mod tests { // tests that a peer is not added to our mesh when they are subscribed to // a topic that we are not fn test_handle_graft_is_not_subscribed() { - let (mut gs, peers, topic_hashes) = - build_and_inject_nodes(20, vec![String::from("topic1")], true); + let (mut gs, peers, topic_hashes) = inject_nodes1() + .peer_no(20) + .topics(vec![String::from("topic1")]) + .to_subscribe(true) + .create_network(); gs.handle_graft( &peers[7], @@ -803,7 +1180,11 @@ mod tests { .map(|&t| String::from(t)) .collect(); - let (mut gs, peers, topic_hashes) = build_and_inject_nodes(20, topics.clone(), true); + let (mut gs, peers, topic_hashes) = inject_nodes1() + .peer_no(20) + .topics(topics.clone()) + .to_subscribe(true) + .create_network(); let mut their_topics = topic_hashes.clone(); // their_topics = [topic1, topic2, topic3] @@ -829,8 +1210,11 @@ mod tests { #[test] // tests that a peer is removed from our mesh fn test_handle_prune_peer_in_mesh() { - let (mut gs, peers, topic_hashes) = - build_and_inject_nodes(20, vec![String::from("topic1")], true); + let (mut gs, peers, topic_hashes) = inject_nodes1() + .peer_no(20) + .topics(vec![String::from("topic1")]) + .to_subscribe(true) + .create_network(); // insert peer into our mesh for 'topic1' gs.mesh @@ -840,49 +1224,492 @@ mod tests { "Expected peer to be in mesh" ); - gs.handle_prune(&peers[7], topic_hashes.clone()); + gs.handle_prune( + &peers[7], + topic_hashes + .iter() + .map(|h| (h.clone(), vec![], None)) + .collect(), + ); assert!( !gs.mesh.get(&topic_hashes[0]).unwrap().contains(&peers[7]), "Expected peer to be removed from mesh" ); } + fn count_control_msgs( + gs: &Gossipsub, + mut filter: impl FnMut(&PeerId, &GossipsubControlAction) -> bool, + ) -> usize { + gs.control_pool + .iter() + .map(|(peer_id, actions)| actions.iter().filter(|m| filter(peer_id, m)).count()) + .sum::() + + gs.events + .iter() + .map(|e| match e { + NetworkBehaviourAction::NotifyHandler { peer_id, event, .. } => { + let event = proto_to_message(event); + event + .control_msgs + .iter() + .filter(|m| filter(peer_id, m)) + .count() + } + _ => 0, + }) + .sum::() + } + + fn flush_events(gs: &mut Gossipsub) { + gs.control_pool.clear(); + gs.events.clear(); + } + + #[test] + // tests that a peer added as explicit peer gets connected to + fn test_explicit_peer_gets_connected() { + let (mut gs, _, _) = inject_nodes1() + .peer_no(0) + .topics(Vec::new()) + .to_subscribe(true) + .create_network(); + + //create new peer + let peer = PeerId::random(); + + //add peer as explicit peer + gs.add_explicit_peer(&peer); + + let dial_events: Vec<&NetworkBehaviourAction, GossipsubEvent>> = gs + .events + .iter() + .filter(|e| match e { + NetworkBehaviourAction::DialPeer { + peer_id, + condition: DialPeerCondition::Disconnected, + } => peer_id == &peer, + _ => false, + }) + .collect(); + + assert_eq!( + dial_events.len(), + 1, + "There was no dial peer event for the explicit peer" + ); + } + + #[test] + fn test_explicit_peer_reconnects() { + let config = GossipsubConfigBuilder::default() + .check_explicit_peers_ticks(2) + .build() + .unwrap(); + let (mut gs, others, _) = inject_nodes1() + .peer_no(1) + .topics(Vec::new()) + .to_subscribe(true) + .gs_config(config) + .create_network(); + + let peer = others.get(0).unwrap(); + + //add peer as explicit peer + gs.add_explicit_peer(peer); + + flush_events(&mut gs); + + //disconnect peer + gs.inject_disconnected(peer); + + gs.heartbeat(); + + //check that no reconnect after first heartbeat since `explicit_peer_ticks == 2` + assert_eq!( + gs.events + .iter() + .filter(|e| match e { + NetworkBehaviourAction::DialPeer { + peer_id, + condition: DialPeerCondition::Disconnected, + } => peer_id == peer, + _ => false, + }) + .count(), + 0, + "There was a dial peer event before explicit_peer_ticks heartbeats" + ); + + gs.heartbeat(); + + //check that there is a reconnect after second heartbeat + assert!( + gs.events + .iter() + .filter(|e| match e { + NetworkBehaviourAction::DialPeer { + peer_id, + condition: DialPeerCondition::Disconnected, + } => peer_id == peer, + _ => false, + }) + .count() + >= 1, + "There was no dial peer event for the explicit peer" + ); + } + + #[test] + fn test_handle_graft_explicit_peer() { + let (mut gs, peers, topic_hashes) = inject_nodes1() + .peer_no(1) + .topics(vec![String::from("topic1"), String::from("topic2")]) + .to_subscribe(true) + .gs_config(GossipsubConfig::default()) + .explicit(1) + .create_network(); + + let peer = peers.get(0).unwrap(); + + gs.handle_graft(peer, topic_hashes.clone()); + + //peer got not added to mesh + assert!(gs.mesh[&topic_hashes[0]].is_empty()); + assert!(gs.mesh[&topic_hashes[1]].is_empty()); + + //check prunes + assert!( + count_control_msgs(&gs, |peer_id, m| peer_id == peer + && match m { + GossipsubControlAction::Prune { topic_hash, .. } => + topic_hash == &topic_hashes[0] || topic_hash == &topic_hashes[1], + _ => false, + }) + >= 2, + "Not enough prunes sent when grafting from explicit peer" + ); + } + + #[test] + fn explicit_peers_not_added_to_mesh_on_receiving_subscription() { + let (gs, peers, topic_hashes) = inject_nodes1() + .peer_no(2) + .topics(vec![String::from("topic1")]) + .to_subscribe(true) + .gs_config(GossipsubConfig::default()) + .explicit(1) + .create_network(); + + //only peer 1 is in the mesh not peer 0 (which is an explicit peer) + assert_eq!( + gs.mesh[&topic_hashes[0]], + vec![peers[1].clone()].into_iter().collect() + ); + + //assert that graft gets created to non-explicit peer + assert!( + count_control_msgs(&gs, |peer_id, m| peer_id == &peers[1] + && match m { + GossipsubControlAction::Graft { .. } => true, + _ => false, + }) + >= 1, + "No graft message got created to non-explicit peer" + ); + + //assert that no graft gets created to explicit peer + assert_eq!( + count_control_msgs(&gs, |peer_id, m| peer_id == &peers[0] + && match m { + GossipsubControlAction::Graft { .. } => true, + _ => false, + }), + 0, + "A graft message got created to an explicit peer" + ); + } + + #[test] + fn do_not_graft_explicit_peer() { + let (mut gs, others, topic_hashes) = inject_nodes1() + .peer_no(1) + .topics(vec![String::from("topic")]) + .to_subscribe(true) + .gs_config(GossipsubConfig::default()) + .explicit(1) + .create_network(); + + gs.heartbeat(); + + //mesh stays empty + assert_eq!(gs.mesh[&topic_hashes[0]], BTreeSet::new()); + + //assert that no graft gets created to explicit peer + assert_eq!( + count_control_msgs(&gs, |peer_id, m| peer_id == &others[0] + && match m { + GossipsubControlAction::Graft { .. } => true, + _ => false, + }), + 0, + "A graft message got created to an explicit peer" + ); + } + + #[test] + fn do_forward_messages_to_explicit_peers() { + let (mut gs, peers, topic_hashes) = inject_nodes1() + .peer_no(2) + .topics(vec![String::from("topic1"), String::from("topic2")]) + .to_subscribe(true) + .gs_config(GossipsubConfig::default()) + .explicit(1) + .create_network(); + + let local_id = PeerId::random(); + + let message = RawGossipsubMessage { + source: Some(peers[1].clone()), + 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!( + gs.events + .iter() + .filter(|e| match e { + NetworkBehaviourAction::NotifyHandler { peer_id, event, .. } => { + let event = proto_to_message(event); + peer_id == &peers[0] + && event + .messages + .iter() + .filter(|m| m.data == message.data) + .count() + > 0 + } + _ => false, + }) + .count(), + 1, + "The message did not get forwarded to the explicit peer" + ); + } + + #[test] + fn explicit_peers_not_added_to_mesh_on_subscribe() { + let (mut gs, peers, _) = inject_nodes1() + .peer_no(2) + .topics(Vec::new()) + .to_subscribe(true) + .gs_config(GossipsubConfig::default()) + .explicit(1) + .create_network(); + + //create new topic, both peers subscribing to it but we do not subscribe to it + let topic = Topic::new(String::from("t")); + let topic_hash = topic.hash(); + for i in 0..2 { + gs.handle_received_subscriptions( + &vec![GossipsubSubscription { + action: GossipsubSubscriptionAction::Subscribe, + topic_hash: topic_hash.clone(), + }], + &peers[i], + ); + } + + //subscribe now to topic + gs.subscribe(&topic).unwrap(); + + //only peer 1 is in the mesh not peer 0 (which is an explicit peer) + assert_eq!( + gs.mesh[&topic_hash], + vec![peers[1].clone()].into_iter().collect() + ); + + //assert that graft gets created to non-explicit peer + assert!( + count_control_msgs(&gs, |peer_id, m| peer_id == &peers[1] + && match m { + GossipsubControlAction::Graft { .. } => true, + _ => false, + }) + > 0, + "No graft message got created to non-explicit peer" + ); + + //assert that no graft gets created to explicit peer + assert_eq!( + count_control_msgs(&gs, |peer_id, m| peer_id == &peers[0] + && match m { + GossipsubControlAction::Graft { .. } => true, + _ => false, + }), + 0, + "A graft message got created to an explicit peer" + ); + } + + #[test] + fn explicit_peers_not_added_to_mesh_from_fanout_on_subscribe() { + let (mut gs, peers, _) = inject_nodes1() + .peer_no(2) + .topics(Vec::new()) + .to_subscribe(true) + .gs_config(GossipsubConfig::default()) + .explicit(1) + .create_network(); + + //create new topic, both peers subscribing to it but we do not subscribe to it + let topic = Topic::new(String::from("t")); + let topic_hash = topic.hash(); + for i in 0..2 { + gs.handle_received_subscriptions( + &vec![GossipsubSubscription { + action: GossipsubSubscriptionAction::Subscribe, + topic_hash: topic_hash.clone(), + }], + &peers[i], + ); + } + + //we send a message for this topic => this will initialize the fanout + gs.publish(topic.clone(), vec![1, 2, 3]).unwrap(); + + //subscribe now to topic + gs.subscribe(&topic).unwrap(); + + //only peer 1 is in the mesh not peer 0 (which is an explicit peer) + assert_eq!( + gs.mesh[&topic_hash], + vec![peers[1].clone()].into_iter().collect() + ); + + //assert that graft gets created to non-explicit peer + assert!( + count_control_msgs(&gs, |peer_id, m| peer_id == &peers[1] + && match m { + GossipsubControlAction::Graft { .. } => true, + _ => false, + }) + >= 1, + "No graft message got created to non-explicit peer" + ); + + //assert that no graft gets created to explicit peer + assert_eq!( + count_control_msgs(&gs, |peer_id, m| peer_id == &peers[0] + && match m { + GossipsubControlAction::Graft { .. } => true, + _ => false, + }), + 0, + "A graft message got created to an explicit peer" + ); + } + #[test] + fn no_gossip_gets_sent_to_explicit_peers() { + let (mut gs, peers, topic_hashes) = inject_nodes1() + .peer_no(2) + .topics(vec![String::from("topic1"), String::from("topic2")]) + .to_subscribe(true) + .gs_config(GossipsubConfig::default()) + .explicit(1) + .create_network(); + + let local_id = PeerId::random(); + + let message = RawGossipsubMessage { + source: Some(peers[1].clone()), + data: vec![], + sequence_number: Some(0), + topic: topic_hashes[0].clone(), + signature: None, + key: None, + validated: true, + }; + + //forward the message + gs.handle_received_message(message.clone(), &local_id); + + //simulate multiple gossip calls (for randomness) + for _ in 0..3 { + gs.emit_gossip(); + } + + //assert that no gossip gets sent to explicit peer + assert_eq!( + gs.control_pool + .get(&peers[0]) + .unwrap_or(&Vec::new()) + .iter() + .filter(|m| match m { + GossipsubControlAction::IHave { .. } => true, + _ => false, + }) + .count(), + 0, + "Gossip got emitted to explicit peer" + ); + } + // Tests the mesh maintenance addition + #[test] fn test_mesh_addition() { - let config = GossipsubConfig::default(); + let config: GossipsubConfig = GossipsubConfig::default(); // Adds mesh_low peers and PRUNE 2 giving us a deficit. - let (mut gs, peers, topics) = - build_and_inject_nodes(config.mesh_n + 1, vec!["test".into()], true); + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(config.mesh_n() + 1) + .topics(vec!["test".into()]) + .to_subscribe(true) + .create_network(); - let to_remove_peers = config.mesh_n + 1 - config.mesh_n_low - 1; + let to_remove_peers = config.mesh_n() + 1 - config.mesh_n_low() - 1; for index in 0..to_remove_peers { - gs.handle_prune(&peers[index], topics.clone()); + gs.handle_prune( + &peers[index], + topics.iter().map(|h| (h.clone(), vec![], None)).collect(), + ); } // Verify the pruned peers are removed from the mesh. assert_eq!( gs.mesh.get(&topics[0]).unwrap().len(), - config.mesh_n_low - 1 + config.mesh_n_low() - 1 ); // run a heartbeat gs.heartbeat(); // Peers should be added to reach mesh_n - assert_eq!(gs.mesh.get(&topics[0]).unwrap().len(), config.mesh_n); + assert_eq!(gs.mesh.get(&topics[0]).unwrap().len(), config.mesh_n()); } - #[test] // Tests the mesh maintenance subtraction + #[test] fn test_mesh_subtraction() { let config = GossipsubConfig::default(); // Adds mesh_low peers and PRUNE 2 giving us a deficit. - let (mut gs, peers, topics) = - build_and_inject_nodes(config.mesh_n_high + 10, vec!["test".into()], true); + let n = config.mesh_n_high() + 10; + //make all outbound connections so that we allow grafting to all + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(n) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .outbound(n) + .create_network(); // graft all the peers for peer in peers { @@ -893,29 +1720,3431 @@ mod tests { gs.heartbeat(); // Peers should be removed to reach mesh_n - assert_eq!(gs.mesh.get(&topics[0]).unwrap().len(), config.mesh_n); + assert_eq!(gs.mesh.get(&topics[0]).unwrap().len(), config.mesh_n()); } - // Some very basic test of public api methods. #[test] - fn test_public_api() { - let (gs, peers, topic_hashes) = - build_and_inject_nodes(4, vec![String::from("topic1")], true); - let peers = peers.into_iter().collect::>(); + fn test_connect_to_px_peers_on_handle_prune() { + let config: GossipsubConfig = GossipsubConfig::default(); + + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(1) + .topics(vec!["test".into()]) + .to_subscribe(true) + .create_network(); + + //handle prune from single peer with px peers + + let mut px = Vec::new(); + //propose more px peers than config.prune_peers() + for _ in 0..config.prune_peers() + 5 { + px.push(PeerInfo { + peer_id: Some(PeerId::random()), + }); + } - assert_eq!( - gs.topics().cloned().collect::>(), topic_hashes, - "Expected topics to match registered topic." + gs.handle_prune( + &peers[0], + vec![( + topics[0].clone(), + px.clone(), + Some(config.prune_backoff().as_secs()), + )], + ); + + //Check DialPeer events for px peers + let dials: Vec<_> = gs + .events + .iter() + .filter_map(|e| match e { + NetworkBehaviourAction::DialPeer { + peer_id, + condition: DialPeerCondition::Disconnected, + } => Some(peer_id.clone()), + _ => None, + }) + .collect(); + + // Exactly config.prune_peers() many random peers should be dialled + assert_eq!(dials.len(), config.prune_peers()); + + let dials_set: HashSet<_> = dials.into_iter().collect(); + + // No duplicates + assert_eq!(dials_set.len(), config.prune_peers()); + + //all dial peers must be in px + assert!(dials_set.is_subset(&HashSet::from_iter( + px.iter().map(|i| i.peer_id.as_ref().unwrap().clone()) + ))); + } + + #[test] + fn test_send_px_and_backoff_in_prune() { + let config: GossipsubConfig = GossipsubConfig::default(); + + //build mesh with enough peers for px + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(config.prune_peers() + 1) + .topics(vec!["test".into()]) + .to_subscribe(true) + .create_network(); + + //send prune to peer + gs.send_graft_prune( + HashMap::new(), + vec![(peers[0].clone(), vec![topics[0].clone()])] + .into_iter() + .collect(), + HashSet::new(), ); + //check prune message assert_eq!( - gs.peers(&TopicHash::from_raw("topic1")).cloned().collect::>(), peers, - "Expected peers for a registered topic to contain all peers." + count_control_msgs(&gs, |peer_id, m| peer_id == &peers[0] + && match m { + GossipsubControlAction::Prune { + topic_hash, + peers, + backoff, + } => + topic_hash == &topics[0] && + peers.len() == config.prune_peers() && + //all peers are different + peers.iter().collect::>().len() == + config.prune_peers() && + backoff.unwrap() == config.prune_backoff().as_secs(), + _ => false, + }), + 1 + ); + } + + #[test] + fn test_prune_backoffed_peer_on_graft() { + let config: GossipsubConfig = GossipsubConfig::default(); + + //build mesh with enough peers for px + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(config.prune_peers() + 1) + .topics(vec!["test".into()]) + .to_subscribe(true) + .create_network(); + + //remove peer from mesh and send prune to peer => this adds a backoff for this peer + gs.mesh.get_mut(&topics[0]).unwrap().remove(&peers[0]); + gs.send_graft_prune( + HashMap::new(), + vec![(peers[0].clone(), vec![topics[0].clone()])] + .into_iter() + .collect(), + HashSet::new(), ); + //ignore all messages until now + gs.events.clear(); + + //handle graft + gs.handle_graft(&peers[0], vec![topics[0].clone()]); + + //check prune message assert_eq!( - gs.all_peers().cloned().collect::>(), peers, - "Expected all_peers to contain all peers." + count_control_msgs(&gs, |peer_id, m| peer_id == &peers[0] + && match m { + GossipsubControlAction::Prune { + topic_hash, + peers, + backoff, + } => + topic_hash == &topics[0] && + //no px in this case + peers.is_empty() && + backoff.unwrap() == config.prune_backoff().as_secs(), + _ => false, + }), + 1 ); } + + #[test] + fn test_do_not_graft_within_backoff_period() { + let config = GossipsubConfigBuilder::default() + .backoff_slack(1) + .heartbeat_interval(Duration::from_millis(100)) + .build() + .unwrap(); + //only one peer => mesh too small and will try to regraft as early as possible + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(1) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config) + .create_network(); + + //handle prune from peer with backoff of one second + gs.handle_prune(&peers[0], vec![(topics[0].clone(), Vec::new(), Some(1))]); + + //forget all events until now + flush_events(&mut gs); + + //call heartbeat + gs.heartbeat(); + + //Sleep for one second and apply 10 regular heartbeats (interval = 100ms). + for _ in 0..10 { + sleep(Duration::from_millis(100)); + gs.heartbeat(); + } + + //Check that no graft got created (we have backoff_slack = 1 therefore one more heartbeat + // is needed). + assert_eq!( + count_control_msgs(&gs, |_, m| match m { + GossipsubControlAction::Graft { .. } => true, + _ => false, + }), + 0, + "Graft message created too early within backoff period" + ); + + //Heartbeat one more time this should graft now + sleep(Duration::from_millis(100)); + gs.heartbeat(); + + //check that graft got created + assert!( + count_control_msgs(&gs, |_, m| match m { + GossipsubControlAction::Graft { .. } => true, + _ => false, + }) > 0, + "No graft message was created after backoff period" + ); + } + + #[test] + fn test_do_not_graft_within_default_backoff_period_after_receiving_prune_without_backoff() { + //set default backoff period to 1 second + let config = GossipsubConfigBuilder::default() + .prune_backoff(Duration::from_millis(90)) + .backoff_slack(1) + .heartbeat_interval(Duration::from_millis(100)) + .build() + .unwrap(); + //only one peer => mesh too small and will try to regraft as early as possible + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(1) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config) + .create_network(); + + //handle prune from peer without a specified backoff + gs.handle_prune(&peers[0], vec![(topics[0].clone(), Vec::new(), None)]); + + //forget all events until now + flush_events(&mut gs); + + //call heartbeat + gs.heartbeat(); + + //Apply one more heartbeat + sleep(Duration::from_millis(100)); + gs.heartbeat(); + + //Check that no graft got created (we have backoff_slack = 1 therefore one more heartbeat + // is needed). + assert_eq!( + count_control_msgs(&gs, |_, m| match m { + GossipsubControlAction::Graft { .. } => true, + _ => false, + }), + 0, + "Graft message created too early within backoff period" + ); + + //Heartbeat one more time this should graft now + sleep(Duration::from_millis(100)); + gs.heartbeat(); + + //check that graft got created + assert!( + count_control_msgs(&gs, |_, m| match m { + GossipsubControlAction::Graft { .. } => true, + _ => false, + }) > 0, + "No graft message was created after backoff period" + ); + } + + #[test] + fn test_flood_publish() { + let config: GossipsubConfig = GossipsubConfig::default(); + + let topic = "test"; + // Adds more peers than mesh can hold to test flood publishing + let (mut gs, _, _) = inject_nodes1() + .peer_no(config.mesh_n_high() + 10) + .topics(vec![topic.into()]) + .to_subscribe(true) + .create_network(); + + //publish message + let publish_data = vec![0; 42]; + gs.publish(Topic::new(topic), publish_data).unwrap(); + + // Collect all publish messages + let publishes = gs + .events + .iter() + .fold(vec![], |mut collected_publish, e| match e { + NetworkBehaviourAction::NotifyHandler { event, .. } => { + let event = proto_to_message(event); + for s in &event.messages { + collected_publish.push(s.clone()); + } + collected_publish + } + _ => collected_publish, + }); + + // Transform the inbound message + let message = &gs + .data_transform + .inbound_transform( + publishes + .first() + .expect("Should contain > 0 entries") + .clone(), + ) + .unwrap(); + + let msg_id = gs.config.message_id(&message); + + let config: GossipsubConfig = GossipsubConfig::default(); + assert_eq!( + publishes.len(), + config.mesh_n_high() + 10, + "Should send a publish message to all known peers" + ); + + assert!( + gs.mcache.get(&msg_id).is_some(), + "Message cache should contain published message" + ); + } + + #[test] + fn test_gossip_to_at_least_gossip_lazy_peers() { + let config: GossipsubConfig = GossipsubConfig::default(); + + //add more peers than in mesh to test gossipping + //by default only mesh_n_low peers will get added to mesh + let (mut gs, _, topic_hashes) = inject_nodes1() + .peer_no(config.mesh_n_low() + config.gossip_lazy() + 1) + .topics(vec!["topic".into()]) + .to_subscribe(true) + .create_network(); + + //receive message + let raw_message = RawGossipsubMessage { + source: Some(PeerId::random()), + data: vec![], + sequence_number: Some(0), + topic: topic_hashes[0].clone(), + signature: None, + key: None, + validated: true, + }; + gs.handle_received_message(raw_message.clone(), &PeerId::random()); + + //emit gossip + gs.emit_gossip(); + + // Transform the inbound message + let message = &gs + .data_transform + .inbound_transform(raw_message.clone()) + .unwrap(); + + let msg_id = gs.config.message_id(&message); + + //check that exactly config.gossip_lazy() many gossip messages were sent. + assert_eq!( + count_control_msgs(&gs, |_, action| match action { + GossipsubControlAction::IHave { + topic_hash, + message_ids, + } => topic_hash == &topic_hashes[0] && message_ids.iter().any(|id| id == &msg_id), + _ => false, + }), + config.gossip_lazy() + ); + } + + #[test] + fn test_gossip_to_at_most_gossip_factor_peers() { + let config: GossipsubConfig = GossipsubConfig::default(); + + //add a lot of peers + let m = + config.mesh_n_low() + config.gossip_lazy() * (2.0 / config.gossip_factor()) as usize; + let (mut gs, _, topic_hashes) = inject_nodes1() + .peer_no(m) + .topics(vec!["topic".into()]) + .to_subscribe(true) + .create_network(); + + //receive message + let raw_message = RawGossipsubMessage { + source: Some(PeerId::random()), + data: vec![], + sequence_number: Some(0), + topic: topic_hashes[0].clone(), + signature: None, + key: None, + validated: true, + }; + gs.handle_received_message(raw_message.clone(), &PeerId::random()); + + //emit gossip + gs.emit_gossip(); + + // Transform the inbound message + let message = &gs + .data_transform + .inbound_transform(raw_message.clone()) + .unwrap(); + + let msg_id = gs.config.message_id(&message); + //check that exactly config.gossip_lazy() many gossip messages were sent. + assert_eq!( + count_control_msgs(&gs, |_, action| match action { + GossipsubControlAction::IHave { + topic_hash, + message_ids, + } => topic_hash == &topic_hashes[0] && message_ids.iter().any(|id| id == &msg_id), + _ => false, + }), + ((m - config.mesh_n_low()) as f64 * config.gossip_factor()) as usize + ); + } + + #[test] + fn test_accept_only_outbound_peer_grafts_when_mesh_full() { + let config: GossipsubConfig = GossipsubConfig::default(); + + //enough peers to fill the mesh + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(config.mesh_n_high()) + .topics(vec!["test".into()]) + .to_subscribe(true) + .create_network(); + + // graft all the peers => this will fill the mesh + for peer in peers { + gs.handle_graft(&peer, topics.clone()); + } + + //assert current mesh size + assert_eq!(gs.mesh[&topics[0]].len(), config.mesh_n_high()); + + //create an outbound and an inbound peer + let inbound = add_peer(&mut gs, &topics, false, false); + let outbound = add_peer(&mut gs, &topics, true, false); + + //send grafts + gs.handle_graft(&inbound, vec![topics[0].clone()]); + gs.handle_graft(&outbound, vec![topics[0].clone()]); + + //assert mesh size + assert_eq!(gs.mesh[&topics[0]].len(), config.mesh_n_high() + 1); + + //inbound is not in mesh + assert!(!gs.mesh[&topics[0]].contains(&inbound)); + + //outbound is in mesh + assert!(gs.mesh[&topics[0]].contains(&outbound)); + } + + #[test] + fn test_do_not_remove_too_many_outbound_peers() { + //use an extreme case to catch errors with high probability + let m = 50; + let n = 2 * m; + let config = GossipsubConfigBuilder::default() + .mesh_n_high(n) + .mesh_n(n) + .mesh_n_low(n) + .mesh_outbound_min(m) + .build() + .unwrap(); + + //fill the mesh with inbound connections + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(n) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .create_network(); + + // graft all the peers + for peer in peers { + gs.handle_graft(&peer, topics.clone()); + } + + //create m outbound connections and graft (we will accept the graft) + let mut outbound = HashSet::new(); + for _ in 0..m { + let peer = add_peer(&mut gs, &topics, true, false); + outbound.insert(peer.clone()); + gs.handle_graft(&peer, topics.clone()); + } + + //mesh is overly full + assert_eq!(gs.mesh.get(&topics[0]).unwrap().len(), n + m); + + // run a heartbeat + gs.heartbeat(); + + // Peers should be removed to reach n + assert_eq!(gs.mesh.get(&topics[0]).unwrap().len(), n); + + //all outbound peers are still in the mesh + assert!(outbound.iter().all(|p| gs.mesh[&topics[0]].contains(p))); + } + + #[test] + fn test_add_outbound_peers_if_min_is_not_satisfied() { + let config: GossipsubConfig = GossipsubConfig::default(); + + // Fill full mesh with inbound peers + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(config.mesh_n_high()) + .topics(vec!["test".into()]) + .to_subscribe(true) + .create_network(); + + // graft all the peers + for peer in peers { + gs.handle_graft(&peer, topics.clone()); + } + + //create config.mesh_outbound_min() many outbound connections without grafting + for _ in 0..config.mesh_outbound_min() { + add_peer(&mut gs, &topics, true, false); + } + + // Nothing changed in the mesh yet + assert_eq!(gs.mesh[&topics[0]].len(), config.mesh_n_high()); + + // run a heartbeat + gs.heartbeat(); + + // The outbound peers got additionally added + assert_eq!( + gs.mesh[&topics[0]].len(), + config.mesh_n_high() + config.mesh_outbound_min() + ); + } + + //TODO add a test that ensures that new outbound connections are recognized as such. + // This is at the moment done in behaviour with relying on the fact that the call to + // `inject_connection_established` for the first connection is done before `inject_connected` + // gets called. For all further connections `inject_connection_established` should get called + // after `inject_connected`. + + #[test] + fn test_prune_negative_scored_peers() { + let config = GossipsubConfig::default(); + + //build mesh with one peer + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(1) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .explicit(0) + .outbound(0) + .scoring(Some(( + PeerScoreParams::default(), + PeerScoreThresholds::default(), + ))) + .create_network(); + + //add penalty to peer + gs.peer_score.as_mut().unwrap().0.add_penalty(&peers[0], 1); + + //execute heartbeat + gs.heartbeat(); + + //peer should not be in mesh anymore + assert!(gs.mesh[&topics[0]].is_empty()); + + //check prune message + assert_eq!( + count_control_msgs(&gs, |peer_id, m| peer_id == &peers[0] + && match m { + GossipsubControlAction::Prune { + topic_hash, + peers, + backoff, + } => + topic_hash == &topics[0] && + //no px in this case + peers.is_empty() && + backoff.unwrap() == config.prune_backoff().as_secs(), + _ => false, + }), + 1 + ); + } + + #[test] + fn test_dont_graft_to_negative_scored_peers() { + let config = GossipsubConfig::default(); + //init full mesh + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(config.mesh_n_high()) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .scoring(Some(( + PeerScoreParams::default(), + PeerScoreThresholds::default(), + ))) + .create_network(); + + //add two additional peers that will not be part of the mesh + let p1 = add_peer(&mut gs, &topics, false, false); + let p2 = add_peer(&mut gs, &topics, false, false); + + //reduce score of p1 to negative + gs.peer_score.as_mut().unwrap().0.add_penalty(&p1, 1); + + //handle prunes of all other peers + for p in peers { + gs.handle_prune(&p, vec![(topics[0].clone(), Vec::new(), None)]); + } + + //heartbeat + gs.heartbeat(); + + //assert that mesh only contains p2 + assert_eq!(gs.mesh.get(&topics[0]).unwrap().len(), 1); + assert!(gs.mesh.get(&topics[0]).unwrap().contains(&p2)); + } + + ///Note that in this test also without a penalty the px would be ignored because of the + /// acceptPXThreshold, but the spec still explicitely states the rule that px from negative + /// peers should get ignored, therefore we test it here. + #[test] + fn test_ignore_px_from_negative_scored_peer() { + let config = GossipsubConfig::default(); + + //build mesh with one peer + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(1) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .scoring(Some(( + PeerScoreParams::default(), + PeerScoreThresholds::default(), + ))) + .create_network(); + + //penalize peer + gs.peer_score.as_mut().unwrap().0.add_penalty(&peers[0], 1); + + //handle prune from single peer with px peers + let px = vec![PeerInfo { + peer_id: Some(PeerId::random()), + }]; + + gs.handle_prune( + &peers[0], + vec![( + topics[0].clone(), + px.clone(), + Some(config.prune_backoff().as_secs()), + )], + ); + + //assert no dials + assert_eq!( + gs.events + .iter() + .filter(|e| match e { + NetworkBehaviourAction::DialPeer { .. } => true, + _ => false, + }) + .count(), + 0 + ); + } + + #[test] + fn test_only_send_nonnegative_scoring_peers_in_px() { + let config = GossipsubConfigBuilder::default() + .prune_peers(16) + .do_px() + .build() + .unwrap(); + + // Build mesh with three peer + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(3) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .explicit(0) + .outbound(0) + .scoring(Some(( + PeerScoreParams::default(), + PeerScoreThresholds::default(), + ))) + .create_network(); + + // Penalize first peer + gs.peer_score.as_mut().unwrap().0.add_penalty(&peers[0], 1); + + // Prune second peer + gs.send_graft_prune( + HashMap::new(), + vec![(peers[1].clone(), vec![topics[0].clone()])] + .into_iter() + .collect(), + HashSet::new(), + ); + + // Check that px in prune message only contains third peer + assert_eq!( + count_control_msgs(&gs, |peer_id, m| peer_id == &peers[1] + && match m { + GossipsubControlAction::Prune { + topic_hash, + peers: px, + .. + } => + topic_hash == &topics[0] + && px.len() == 1 + && px[0].peer_id.as_ref().unwrap() == &peers[2], + _ => false, + }), + 1 + ); + } + + #[test] + fn test_do_not_gossip_to_peers_below_gossip_threshold() { + let config = GossipsubConfig::default(); + let peer_score_params = PeerScoreParams::default(); + let mut peer_score_thresholds = PeerScoreThresholds::default(); + peer_score_thresholds.gossip_threshold = 3.0 * peer_score_params.behaviour_penalty_weight; + + // Build full mesh + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(config.mesh_n_high()) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .scoring(Some((peer_score_params, peer_score_thresholds))) + .create_network(); + + // Graft all the peer + for peer in peers { + gs.handle_graft(&peer, topics.clone()); + } + + // Add two additional peers that will not be part of the mesh + let p1 = add_peer(&mut gs, &topics, false, false); + let p2 = add_peer(&mut gs, &topics, false, false); + + // Reduce score of p1 below peer_score_thresholds.gossip_threshold + // note that penalties get squared so two penalties means a score of + // 4 * peer_score_params.behaviour_penalty_weight. + gs.peer_score.as_mut().unwrap().0.add_penalty(&p1, 2); + + // Reduce score of p2 below 0 but not below peer_score_thresholds.gossip_threshold + gs.peer_score.as_mut().unwrap().0.add_penalty(&p2, 1); + + // Receive message + let raw_message = RawGossipsubMessage { + source: Some(PeerId::random()), + data: vec![], + sequence_number: Some(0), + topic: topics[0].clone(), + signature: None, + key: None, + validated: true, + }; + gs.handle_received_message(raw_message.clone(), &PeerId::random()); + + // Transform the inbound message + let message = &gs + .data_transform + .inbound_transform(raw_message.clone()) + .unwrap(); + + let msg_id = gs.config.message_id(&message); + + // Emit gossip + gs.emit_gossip(); + + // Check that exactly one gossip messages got sent and it got sent to p2 + assert_eq!( + count_control_msgs(&gs, |peer, action| match action { + GossipsubControlAction::IHave { + topic_hash, + message_ids, + } => { + if topic_hash == &topics[0] && message_ids.iter().any(|id| id == &msg_id) { + assert_eq!(peer, &p2); + true + } else { + false + } + } + _ => false, + }), + 1 + ); + } + + #[test] + fn test_iwant_msg_from_peer_below_gossip_threshold_gets_ignored() { + let config = GossipsubConfig::default(); + let peer_score_params = PeerScoreParams::default(); + let mut peer_score_thresholds = PeerScoreThresholds::default(); + peer_score_thresholds.gossip_threshold = 3.0 * peer_score_params.behaviour_penalty_weight; + + // Build full mesh + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(config.mesh_n_high()) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .explicit(0) + .outbound(0) + .scoring(Some((peer_score_params, peer_score_thresholds))) + .create_network(); + + // Graft all the peer + for peer in peers { + gs.handle_graft(&peer, topics.clone()); + } + + // Add two additional peers that will not be part of the mesh + let p1 = add_peer(&mut gs, &topics, false, false); + let p2 = add_peer(&mut gs, &topics, false, false); + + // Reduce score of p1 below peer_score_thresholds.gossip_threshold + // note that penalties get squared so two penalties means a score of + // 4 * peer_score_params.behaviour_penalty_weight. + gs.peer_score.as_mut().unwrap().0.add_penalty(&p1, 2); + + // Reduce score of p2 below 0 but not below peer_score_thresholds.gossip_threshold + gs.peer_score.as_mut().unwrap().0.add_penalty(&p2, 1); + + // Receive message + let raw_message = RawGossipsubMessage { + source: Some(PeerId::random()), + data: vec![], + sequence_number: Some(0), + topic: topics[0].clone(), + signature: None, + key: None, + validated: true, + }; + gs.handle_received_message(raw_message.clone(), &PeerId::random()); + + // Transform the inbound message + let message = &gs + .data_transform + .inbound_transform(raw_message.clone()) + .unwrap(); + + let msg_id = gs.config.message_id(&message); + + gs.handle_iwant(&p1, vec![msg_id.clone()]); + gs.handle_iwant(&p2, vec![msg_id.clone()]); + + // the messages we are sending + let sent_messages = gs + .events + .iter() + .fold(vec![], |mut collected_messages, e| match e { + NetworkBehaviourAction::NotifyHandler { event, peer_id, .. } => { + let event = proto_to_message(event); + for c in &event.messages { + collected_messages.push((peer_id.clone(), c.clone())) + } + collected_messages + } + _ => collected_messages, + }); + + //the message got sent to p2 + assert!(sent_messages + .iter() + .map(|(peer_id, msg)| ( + peer_id, + gs.data_transform.inbound_transform(msg.clone()).unwrap() + )) + .any(|(peer_id, msg)| peer_id == &p2 && &gs.config.message_id(&msg) == &msg_id)); + //the message got not sent to p1 + assert!(sent_messages + .iter() + .map(|(peer_id, msg)| ( + peer_id, + gs.data_transform.inbound_transform(msg.clone()).unwrap() + )) + .all(|(peer_id, msg)| !(peer_id == &p1 && &gs.config.message_id(&msg) == &msg_id))); + } + + #[test] + fn test_ihave_msg_from_peer_below_gossip_threshold_gets_ignored() { + let config = GossipsubConfig::default(); + let peer_score_params = PeerScoreParams::default(); + let mut peer_score_thresholds = PeerScoreThresholds::default(); + peer_score_thresholds.gossip_threshold = 3.0 * peer_score_params.behaviour_penalty_weight; + + //build full mesh + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(config.mesh_n_high()) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .explicit(0) + .outbound(0) + .scoring(Some((peer_score_params, peer_score_thresholds))) + .create_network(); + + // graft all the peer + for peer in peers { + gs.handle_graft(&peer, topics.clone()); + } + + //add two additional peers that will not be part of the mesh + let p1 = add_peer(&mut gs, &topics, false, false); + let p2 = add_peer(&mut gs, &topics, false, false); + + //reduce score of p1 below peer_score_thresholds.gossip_threshold + //note that penalties get squared so two penalties means a score of + // 4 * peer_score_params.behaviour_penalty_weight. + gs.peer_score.as_mut().unwrap().0.add_penalty(&p1, 2); + + //reduce score of p2 below 0 but not below peer_score_thresholds.gossip_threshold + gs.peer_score.as_mut().unwrap().0.add_penalty(&p2, 1); + + //message that other peers have + let raw_message = RawGossipsubMessage { + source: Some(PeerId::random()), + data: vec![], + sequence_number: Some(0), + topic: topics[0].clone(), + signature: None, + key: None, + validated: true, + }; + + // Transform the inbound message + let message = &gs + .data_transform + .inbound_transform(raw_message.clone()) + .unwrap(); + + let msg_id = gs.config.message_id(&message); + + gs.handle_ihave(&p1, vec![(topics[0].clone(), vec![msg_id.clone()])]); + gs.handle_ihave(&p2, vec![(topics[0].clone(), vec![msg_id.clone()])]); + + // check that we sent exactly one IWANT request to p2 + assert_eq!( + count_control_msgs(&gs, |peer, c| match c { + GossipsubControlAction::IWant { message_ids } => + if message_ids.iter().any(|m| m == &msg_id) { + assert_eq!(peer, &p2); + true + } else { + false + }, + _ => false, + }), + 1 + ); + } + + #[test] + fn test_do_not_publish_to_peer_below_publish_threshold() { + let config = GossipsubConfigBuilder::default() + .flood_publish(false) + .build() + .unwrap(); + let peer_score_params = PeerScoreParams::default(); + let mut peer_score_thresholds = PeerScoreThresholds::default(); + peer_score_thresholds.gossip_threshold = 0.5 * peer_score_params.behaviour_penalty_weight; + peer_score_thresholds.publish_threshold = 3.0 * peer_score_params.behaviour_penalty_weight; + + //build mesh with no peers and no subscribed topics + let (mut gs, _, _) = inject_nodes1() + .gs_config(config.clone()) + .scoring(Some((peer_score_params, peer_score_thresholds))) + .create_network(); + + //create a new topic for which we are not subscribed + let topic = Topic::new("test"); + let topics = vec![topic.hash()]; + + //add two additional peers that will be added to the mesh + let p1 = add_peer(&mut gs, &topics, false, false); + let p2 = add_peer(&mut gs, &topics, false, false); + + //reduce score of p1 below peer_score_thresholds.publish_threshold + //note that penalties get squared so two penalties means a score of + // 4 * peer_score_params.behaviour_penalty_weight. + gs.peer_score.as_mut().unwrap().0.add_penalty(&p1, 2); + + //reduce score of p2 below 0 but not below peer_score_thresholds.publish_threshold + gs.peer_score.as_mut().unwrap().0.add_penalty(&p2, 1); + + //a heartbeat will remove the peers from the mesh + gs.heartbeat(); + + // publish on topic + let publish_data = vec![0; 42]; + gs.publish(topic, publish_data).unwrap(); + + // Collect all publish messages + let publishes = gs + .events + .iter() + .fold(vec![], |mut collected_publish, e| match e { + NetworkBehaviourAction::NotifyHandler { event, peer_id, .. } => { + let event = proto_to_message(event); + for s in &event.messages { + collected_publish.push((peer_id.clone(), s.clone())); + } + collected_publish + } + _ => collected_publish, + }); + + //assert only published to p2 + assert_eq!(publishes.len(), 1); + assert_eq!(publishes[0].0, p2); + } + + #[test] + fn test_do_not_flood_publish_to_peer_below_publish_threshold() { + let config = GossipsubConfig::default(); + let peer_score_params = PeerScoreParams::default(); + let mut peer_score_thresholds = PeerScoreThresholds::default(); + peer_score_thresholds.gossip_threshold = 0.5 * peer_score_params.behaviour_penalty_weight; + peer_score_thresholds.publish_threshold = 3.0 * peer_score_params.behaviour_penalty_weight; + + //build mesh with no peers + let (mut gs, _, topics) = inject_nodes1() + .topics(vec!["test".into()]) + .gs_config(config.clone()) + .scoring(Some((peer_score_params, peer_score_thresholds))) + .create_network(); + + //add two additional peers that will be added to the mesh + let p1 = add_peer(&mut gs, &topics, false, false); + let p2 = add_peer(&mut gs, &topics, false, false); + + //reduce score of p1 below peer_score_thresholds.publish_threshold + //note that penalties get squared so two penalties means a score of + // 4 * peer_score_params.behaviour_penalty_weight. + gs.peer_score.as_mut().unwrap().0.add_penalty(&p1, 2); + + //reduce score of p2 below 0 but not below peer_score_thresholds.publish_threshold + gs.peer_score.as_mut().unwrap().0.add_penalty(&p2, 1); + + //a heartbeat will remove the peers from the mesh + gs.heartbeat(); + + // publish on topic + let publish_data = vec![0; 42]; + gs.publish(Topic::new("test"), publish_data).unwrap(); + + // Collect all publish messages + let publishes = gs + .events + .iter() + .fold(vec![], |mut collected_publish, e| match e { + NetworkBehaviourAction::NotifyHandler { event, peer_id, .. } => { + let event = proto_to_message(event); + for s in &event.messages { + collected_publish.push((peer_id.clone(), s.clone())); + } + collected_publish + } + _ => collected_publish, + }); + + //assert only published to p2 + assert_eq!(publishes.len(), 1); + assert!(publishes[0].0 == p2); + } + + #[test] + fn test_ignore_rpc_from_peers_below_graylist_threshold() { + let config = GossipsubConfig::default(); + let peer_score_params = PeerScoreParams::default(); + let mut peer_score_thresholds = PeerScoreThresholds::default(); + peer_score_thresholds.gossip_threshold = 0.5 * peer_score_params.behaviour_penalty_weight; + peer_score_thresholds.publish_threshold = 0.5 * peer_score_params.behaviour_penalty_weight; + peer_score_thresholds.graylist_threshold = 3.0 * peer_score_params.behaviour_penalty_weight; + + //build mesh with no peers + let (mut gs, _, topics) = inject_nodes1() + .topics(vec!["test".into()]) + .gs_config(config.clone()) + .scoring(Some((peer_score_params, peer_score_thresholds))) + .create_network(); + + //add two additional peers that will be added to the mesh + let p1 = add_peer(&mut gs, &topics, false, false); + let p2 = add_peer(&mut gs, &topics, false, false); + + //reduce score of p1 below peer_score_thresholds.graylist_threshold + //note that penalties get squared so two penalties means a score of + // 4 * peer_score_params.behaviour_penalty_weight. + gs.peer_score.as_mut().unwrap().0.add_penalty(&p1, 2); + + //reduce score of p2 below publish_threshold but not below graylist_threshold + gs.peer_score.as_mut().unwrap().0.add_penalty(&p2, 1); + + let raw_message1 = RawGossipsubMessage { + source: Some(PeerId::random()), + data: vec![1, 2, 3, 4], + sequence_number: Some(1u64), + topic: topics[0].clone(), + signature: None, + key: None, + validated: true, + }; + + let raw_message2 = RawGossipsubMessage { + source: Some(PeerId::random()), + data: vec![1, 2, 3, 4, 5], + sequence_number: Some(2u64), + topic: topics[0].clone(), + signature: None, + key: None, + validated: true, + }; + + let raw_message3 = RawGossipsubMessage { + source: Some(PeerId::random()), + data: vec![1, 2, 3, 4, 5, 6], + sequence_number: Some(3u64), + topic: topics[0].clone(), + signature: None, + key: None, + validated: true, + }; + + let raw_message4 = RawGossipsubMessage { + source: Some(PeerId::random()), + data: vec![1, 2, 3, 4, 5, 6, 7], + sequence_number: Some(4u64), + topic: topics[0].clone(), + signature: None, + key: None, + validated: true, + }; + + // Transform the inbound message + let message2 = &gs + .data_transform + .inbound_transform(raw_message2.clone()) + .unwrap(); + + // Transform the inbound message + let message4 = &gs + .data_transform + .inbound_transform(raw_message4.clone()) + .unwrap(); + + let subscription = GossipsubSubscription { + action: GossipsubSubscriptionAction::Subscribe, + topic_hash: topics[0].clone(), + }; + + let control_action = GossipsubControlAction::IHave { + topic_hash: topics[0].clone(), + message_ids: vec![config.message_id(&message2)], + }; + + //clear events + gs.events.clear(); + + //receive from p1 + gs.inject_event( + p1.clone(), + ConnectionId::new(0), + HandlerEvent::Message { + rpc: GossipsubRpc { + messages: vec![raw_message1], + subscriptions: vec![subscription.clone()], + control_msgs: vec![control_action], + }, + invalid_messages: Vec::new(), + }, + ); + + //only the subscription event gets processed, the rest is dropped + assert_eq!(gs.events.len(), 1); + assert!(match &gs.events[0] { + NetworkBehaviourAction::GenerateEvent(event) => match event { + GossipsubEvent::Subscribed { .. } => true, + _ => false, + }, + _ => false, + }); + + let control_action = GossipsubControlAction::IHave { + topic_hash: topics[0].clone(), + message_ids: vec![config.message_id(&message4)], + }; + + //receive from p2 + gs.inject_event( + p2.clone(), + ConnectionId::new(0), + HandlerEvent::Message { + rpc: GossipsubRpc { + messages: vec![raw_message3], + subscriptions: vec![subscription.clone()], + control_msgs: vec![control_action], + }, + invalid_messages: Vec::new(), + }, + ); + + //events got processed + assert!(gs.events.len() > 1); + } + + #[test] + fn test_ignore_px_from_peers_below_accept_px_threshold() { + let config = GossipsubConfigBuilder::default() + .prune_peers(16) + .build() + .unwrap(); + let peer_score_params = PeerScoreParams::default(); + let mut peer_score_thresholds = PeerScoreThresholds::default(); + peer_score_thresholds.accept_px_threshold = peer_score_params.app_specific_weight; + + // Build mesh with two peers + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(2) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .scoring(Some((peer_score_params, peer_score_thresholds))) + .create_network(); + + // Decrease score of first peer to less than accept_px_threshold + gs.set_application_score(&peers[0], 0.99); + + // Increase score of second peer to accept_px_threshold + gs.set_application_score(&peers[1], 1.0); + + // Handle prune from peer peers[0] with px peers + let px = vec![PeerInfo { + peer_id: Some(PeerId::random()), + }]; + gs.handle_prune( + &peers[0], + vec![( + topics[0].clone(), + px.clone(), + Some(config.prune_backoff().as_secs()), + )], + ); + + // Assert no dials + assert_eq!( + gs.events + .iter() + .filter(|e| match e { + NetworkBehaviourAction::DialPeer { .. } => true, + _ => false, + }) + .count(), + 0 + ); + + //handle prune from peer peers[1] with px peers + let px = vec![PeerInfo { + peer_id: Some(PeerId::random()), + }]; + gs.handle_prune( + &peers[1], + vec![( + topics[0].clone(), + px.clone(), + Some(config.prune_backoff().as_secs()), + )], + ); + + //assert there are dials now + assert!( + gs.events + .iter() + .filter(|e| match e { + NetworkBehaviourAction::DialPeer { .. } => true, + _ => false, + }) + .count() + > 0 + ); + } + + #[test] + fn test_keep_best_scoring_peers_on_oversubscription() { + let config = GossipsubConfigBuilder::default() + .mesh_n_low(15) + .mesh_n(30) + .mesh_n_high(60) + .retain_scores(29) + .build() + .unwrap(); + + //build mesh with more peers than mesh can hold + let n = config.mesh_n_high() + 1; + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(n) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .explicit(0) + .outbound(n) + .scoring(Some(( + PeerScoreParams::default(), + PeerScoreThresholds::default(), + ))) + .create_network(); + + // graft all, will be accepted since the are outbound + for peer in &peers { + gs.handle_graft(peer, topics.clone()); + } + + //assign scores to peers equalling their index + + //set random positive scores + for (index, peer) in peers.iter().enumerate() { + gs.set_application_score(peer, index as f64); + } + + assert_eq!(gs.mesh[&topics[0]].len(), n); + + //heartbeat to prune some peers + gs.heartbeat(); + + assert_eq!(gs.mesh[&topics[0]].len(), config.mesh_n()); + + //mesh contains retain_scores best peers + assert!(gs.mesh[&topics[0]].is_superset( + &peers[(n - config.retain_scores())..] + .iter() + .cloned() + .collect() + )); + } + + #[test] + fn test_scoring_p1() { + let config = GossipsubConfig::default(); + let mut peer_score_params = PeerScoreParams::default(); + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut topic_params = TopicScoreParams::default(); + topic_params.time_in_mesh_weight = 2.0; + topic_params.time_in_mesh_quantum = Duration::from_millis(50); + topic_params.time_in_mesh_cap = 10.0; + topic_params.topic_weight = 0.7; + peer_score_params + .topics + .insert(topic_hash.clone(), topic_params.clone()); + let peer_score_thresholds = PeerScoreThresholds::default(); + + //build mesh with one peer + let (mut gs, peers, _) = inject_nodes1() + .peer_no(1) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .explicit(0) + .outbound(0) + .scoring(Some((peer_score_params, peer_score_thresholds))) + .create_network(); + + //sleep for 2 times the mesh_quantum + sleep(topic_params.time_in_mesh_quantum * 2); + //refresh scores + gs.peer_score.as_mut().unwrap().0.refresh_scores(); + assert!( + gs.peer_score.as_ref().unwrap().0.score(&peers[0]) + >= 2.0 * topic_params.time_in_mesh_weight * topic_params.topic_weight, + "score should be at least 2 * time_in_mesh_weight * topic_weight" + ); + assert!( + gs.peer_score.as_ref().unwrap().0.score(&peers[0]) + < 3.0 * topic_params.time_in_mesh_weight * topic_params.topic_weight, + "score should be less than 3 * time_in_mesh_weight * topic_weight" + ); + + //sleep again for 2 times the mesh_quantum + sleep(topic_params.time_in_mesh_quantum * 2); + //refresh scores + gs.peer_score.as_mut().unwrap().0.refresh_scores(); + assert!( + gs.peer_score.as_ref().unwrap().0.score(&peers[0]) + >= 2.0 * topic_params.time_in_mesh_weight * topic_params.topic_weight, + "score should be at least 4 * time_in_mesh_weight * topic_weight" + ); + + //sleep for enough periods to reach maximum + sleep(topic_params.time_in_mesh_quantum * (topic_params.time_in_mesh_cap - 3.0) as u32); + //refresh scores + gs.peer_score.as_mut().unwrap().0.refresh_scores(); + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[0]), + topic_params.time_in_mesh_cap + * topic_params.time_in_mesh_weight + * topic_params.topic_weight, + "score should be exactly time_in_mesh_cap * time_in_mesh_weight * topic_weight" + ); + } + + fn random_message(seq: &mut u64, topics: &Vec) -> RawGossipsubMessage { + let mut rng = rand::thread_rng(); + *seq += 1; + RawGossipsubMessage { + source: Some(PeerId::random()), + data: (0..rng.gen_range(10, 30)) + .into_iter() + .map(|_| rng.gen()) + .collect(), + sequence_number: Some(*seq), + topic: topics[rng.gen_range(0, topics.len())].clone(), + signature: None, + key: None, + validated: true, + } + } + + #[test] + fn test_scoring_p2() { + let config = GossipsubConfig::default(); + let mut peer_score_params = PeerScoreParams::default(); + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut topic_params = TopicScoreParams::default(); + topic_params.time_in_mesh_weight = 0.0; //deactivate time in mesh + topic_params.first_message_deliveries_weight = 2.0; + topic_params.first_message_deliveries_cap = 10.0; + topic_params.first_message_deliveries_decay = 0.9; + topic_params.topic_weight = 0.7; + peer_score_params + .topics + .insert(topic_hash.clone(), topic_params.clone()); + let peer_score_thresholds = PeerScoreThresholds::default(); + + //build mesh with one peer + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(2) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .explicit(0) + .outbound(0) + .scoring(Some((peer_score_params, peer_score_thresholds))) + .create_network(); + + let mut seq = 0; + let deliver_message = |gs: &mut Gossipsub, index: usize, msg: RawGossipsubMessage| { + gs.handle_received_message(msg, &peers[index]); + }; + + let m1 = random_message(&mut seq, &topics); + //peer 0 delivers message first + deliver_message(&mut gs, 0, m1.clone()); + //peer 1 delivers message second + deliver_message(&mut gs, 1, m1.clone()); + + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[0]), + 1.0 * topic_params.first_message_deliveries_weight * topic_params.topic_weight, + "score should be exactly first_message_deliveries_weight * topic_weight" + ); + + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[1]), + 0.0, + "there should be no score for second message deliveries * topic_weight" + ); + + //peer 2 delivers two new messages + deliver_message(&mut gs, 1, random_message(&mut seq, &topics)); + deliver_message(&mut gs, 1, random_message(&mut seq, &topics)); + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[1]), + 2.0 * topic_params.first_message_deliveries_weight * topic_params.topic_weight, + "score should be exactly 2 * first_message_deliveries_weight * topic_weight" + ); + + //test decaying + gs.peer_score.as_mut().unwrap().0.refresh_scores(); + + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[0]), + 1.0 * topic_params.first_message_deliveries_decay + * topic_params.first_message_deliveries_weight + * topic_params.topic_weight, + "score should be exactly first_message_deliveries_decay * \ + first_message_deliveries_weight * topic_weight" + ); + + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[1]), + 2.0 * topic_params.first_message_deliveries_decay + * topic_params.first_message_deliveries_weight + * topic_params.topic_weight, + "score should be exactly 2 * first_message_deliveries_decay * \ + first_message_deliveries_weight * topic_weight" + ); + + //test cap + for _ in 0..topic_params.first_message_deliveries_cap as u64 { + deliver_message(&mut gs, 1, random_message(&mut seq, &topics)); + } + + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[1]), + topic_params.first_message_deliveries_cap + * topic_params.first_message_deliveries_weight + * topic_params.topic_weight, + "score should be exactly first_message_deliveries_cap * \ + first_message_deliveries_weight * topic_weight" + ); + } + + #[test] + fn test_scoring_p3() { + let config = GossipsubConfig::default(); + let mut peer_score_params = PeerScoreParams::default(); + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut topic_params = TopicScoreParams::default(); + topic_params.time_in_mesh_weight = 0.0; //deactivate time in mesh + topic_params.first_message_deliveries_weight = 0.0; //deactivate first time deliveries + topic_params.mesh_message_deliveries_weight = -2.0; + topic_params.mesh_message_deliveries_decay = 0.9; + topic_params.mesh_message_deliveries_cap = 10.0; + topic_params.mesh_message_deliveries_threshold = 5.0; + topic_params.mesh_message_deliveries_activation = Duration::from_secs(1); + topic_params.mesh_message_deliveries_window = Duration::from_millis(100); + topic_params.topic_weight = 0.7; + peer_score_params + .topics + .insert(topic_hash.clone(), topic_params.clone()); + let peer_score_thresholds = PeerScoreThresholds::default(); + + //build mesh with two peers + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(2) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .explicit(0) + .outbound(0) + .scoring(Some((peer_score_params, peer_score_thresholds))) + .create_network(); + + let mut seq = 0; + let deliver_message = |gs: &mut Gossipsub, index: usize, msg: RawGossipsubMessage| { + gs.handle_received_message(msg, &peers[index]); + }; + + let mut expected_message_deliveries = 0.0; + + //messages used to test window + let m1 = random_message(&mut seq, &topics); + let m2 = random_message(&mut seq, &topics); + + //peer 1 delivers m1 + deliver_message(&mut gs, 1, m1.clone()); + + //peer 0 delivers two message + deliver_message(&mut gs, 0, random_message(&mut seq, &topics)); + deliver_message(&mut gs, 0, random_message(&mut seq, &topics)); + expected_message_deliveries += 2.0; + + sleep(Duration::from_millis(60)); + + //peer 1 delivers m2 + deliver_message(&mut gs, 1, m2.clone()); + + sleep(Duration::from_millis(70)); + //peer 0 delivers m1 and m2 only m2 gets counted + deliver_message(&mut gs, 0, m1); + deliver_message(&mut gs, 0, m2); + expected_message_deliveries += 1.0; + + sleep(Duration::from_millis(900)); + + //message deliveries penalties get activated, peer 0 has only delivered 3 messages and + // therefore gets a penalty + gs.peer_score.as_mut().unwrap().0.refresh_scores(); + expected_message_deliveries *= 0.9; //decay + + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[0]), + (5f64 - expected_message_deliveries).powi(2) * -2.0 * 0.7 + ); + + // peer 0 delivers a lot of messages => message_deliveries should be capped at 10 + for _ in 0..20 { + deliver_message(&mut gs, 0, random_message(&mut seq, &topics)); + } + + expected_message_deliveries = 10.0; + + assert_eq!(gs.peer_score.as_ref().unwrap().0.score(&peers[0]), 0.0); + + //apply 10 decays + for _ in 0..10 { + gs.peer_score.as_mut().unwrap().0.refresh_scores(); + expected_message_deliveries *= 0.9; //decay + } + + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[0]), + (5f64 - expected_message_deliveries).powi(2) * -2.0 * 0.7 + ); + } + + #[test] + fn test_scoring_p3b() { + let config = GossipsubConfigBuilder::default() + .prune_backoff(Duration::from_millis(100)) + .build() + .unwrap(); + let mut peer_score_params = PeerScoreParams::default(); + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut topic_params = TopicScoreParams::default(); + topic_params.time_in_mesh_weight = 0.0; //deactivate time in mesh + topic_params.first_message_deliveries_weight = 0.0; //deactivate first time deliveries + topic_params.mesh_message_deliveries_weight = -2.0; + topic_params.mesh_message_deliveries_decay = 0.9; + topic_params.mesh_message_deliveries_cap = 10.0; + topic_params.mesh_message_deliveries_threshold = 5.0; + topic_params.mesh_message_deliveries_activation = Duration::from_secs(1); + topic_params.mesh_message_deliveries_window = Duration::from_millis(100); + topic_params.mesh_failure_penalty_weight = -3.0; + topic_params.mesh_failure_penalty_decay = 0.95; + topic_params.topic_weight = 0.7; + peer_score_params + .topics + .insert(topic_hash.clone(), topic_params.clone()); + peer_score_params.app_specific_weight = 1.0; + let peer_score_thresholds = PeerScoreThresholds::default(); + + //build mesh with one peer + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(1) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .explicit(0) + .outbound(0) + .scoring(Some((peer_score_params, peer_score_thresholds))) + .create_network(); + + let mut seq = 0; + let deliver_message = |gs: &mut Gossipsub, index: usize, msg: RawGossipsubMessage| { + gs.handle_received_message(msg, &peers[index]); + }; + + let mut expected_message_deliveries = 0.0; + + //add some positive score + gs.peer_score + .as_mut() + .unwrap() + .0 + .set_application_score(&peers[0], 100.0); + + //peer 0 delivers two message + deliver_message(&mut gs, 0, random_message(&mut seq, &topics)); + deliver_message(&mut gs, 0, random_message(&mut seq, &topics)); + expected_message_deliveries += 2.0; + + sleep(Duration::from_millis(1050)); + + //activation kicks in + gs.peer_score.as_mut().unwrap().0.refresh_scores(); + expected_message_deliveries *= 0.9; //decay + + //prune peer + gs.handle_prune(&peers[0], vec![(topics[0].clone(), vec![], None)]); + + //wait backoff + sleep(Duration::from_millis(130)); + + //regraft peer + gs.handle_graft(&peers[0], topics.clone()); + + //the score should now consider p3b + let mut expected_b3 = (5f64 - expected_message_deliveries).powi(2); + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[0]), + 100.0 + expected_b3 * -3.0 * 0.7 + ); + + //we can also add a new p3 to the score + + //peer 0 delivers one message + deliver_message(&mut gs, 0, random_message(&mut seq, &topics)); + expected_message_deliveries += 1.0; + + sleep(Duration::from_millis(1050)); + gs.peer_score.as_mut().unwrap().0.refresh_scores(); + expected_message_deliveries *= 0.9; //decay + expected_b3 *= 0.95; + + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[0]), + 100.0 + + (expected_b3 * -3.0 + (5f64 - expected_message_deliveries).powi(2) * -2.0) * 0.7 + ); + } + + #[test] + fn test_scoring_p4_valid_message() { + let config = GossipsubConfigBuilder::default() + .validate_messages() + .build() + .unwrap(); + let mut peer_score_params = PeerScoreParams::default(); + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut topic_params = TopicScoreParams::default(); + topic_params.time_in_mesh_weight = 0.0; //deactivate time in mesh + topic_params.first_message_deliveries_weight = 0.0; //deactivate first time deliveries + topic_params.mesh_message_deliveries_weight = 0.0; //deactivate message deliveries + topic_params.mesh_failure_penalty_weight = 0.0; //deactivate mesh failure penalties + topic_params.invalid_message_deliveries_weight = -2.0; + topic_params.invalid_message_deliveries_decay = 0.9; + topic_params.topic_weight = 0.7; + peer_score_params + .topics + .insert(topic_hash.clone(), topic_params.clone()); + peer_score_params.app_specific_weight = 1.0; + let peer_score_thresholds = PeerScoreThresholds::default(); + + //build mesh with two peers + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(1) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .explicit(0) + .outbound(0) + .scoring(Some((peer_score_params, peer_score_thresholds))) + .create_network(); + + let mut seq = 0; + let deliver_message = |gs: &mut Gossipsub, index: usize, msg: RawGossipsubMessage| { + gs.handle_received_message(msg, &peers[index]); + }; + + //peer 0 delivers valid message + let m1 = random_message(&mut seq, &topics); + deliver_message(&mut gs, 0, m1.clone()); + + // Transform the inbound message + let message1 = &gs.data_transform.inbound_transform(m1.clone()).unwrap(); + + assert_eq!(gs.peer_score.as_ref().unwrap().0.score(&peers[0]), 0.0); + + //message m1 gets validated + gs.report_message_validation_result( + &config.message_id(&message1), + &peers[0], + MessageAcceptance::Accept, + ) + .unwrap(); + + assert_eq!(gs.peer_score.as_ref().unwrap().0.score(&peers[0]), 0.0); + } + + #[test] + fn test_scoring_p4_invalid_signature() { + let config = GossipsubConfigBuilder::default() + .validate_messages() + .build() + .unwrap(); + let mut peer_score_params = PeerScoreParams::default(); + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut topic_params = TopicScoreParams::default(); + topic_params.time_in_mesh_weight = 0.0; //deactivate time in mesh + topic_params.first_message_deliveries_weight = 0.0; //deactivate first time deliveries + topic_params.mesh_message_deliveries_weight = 0.0; //deactivate message deliveries + topic_params.mesh_failure_penalty_weight = 0.0; //deactivate mesh failure penalties + topic_params.invalid_message_deliveries_weight = -2.0; + topic_params.invalid_message_deliveries_decay = 0.9; + topic_params.topic_weight = 0.7; + peer_score_params + .topics + .insert(topic_hash.clone(), topic_params.clone()); + peer_score_params.app_specific_weight = 1.0; + let peer_score_thresholds = PeerScoreThresholds::default(); + + //build mesh with one peer + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(1) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .explicit(0) + .outbound(0) + .scoring(Some((peer_score_params, peer_score_thresholds))) + .create_network(); + + let mut seq = 0; + + //peer 0 delivers message with invalid signature + let m = random_message(&mut seq, &topics); + + gs.inject_event( + peers[0].clone(), + ConnectionId::new(0), + HandlerEvent::Message { + rpc: GossipsubRpc { + messages: vec![], + subscriptions: vec![], + control_msgs: vec![], + }, + invalid_messages: vec![(m, ValidationError::InvalidSignature)], + }, + ); + + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[0]), + -2.0 * 0.7 + ); + } + + #[test] + fn test_scoring_p4_message_from_self() { + let config = GossipsubConfigBuilder::default() + .validate_messages() + .build() + .unwrap(); + let mut peer_score_params = PeerScoreParams::default(); + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut topic_params = TopicScoreParams::default(); + topic_params.time_in_mesh_weight = 0.0; //deactivate time in mesh + topic_params.first_message_deliveries_weight = 0.0; //deactivate first time deliveries + topic_params.mesh_message_deliveries_weight = 0.0; //deactivate message deliveries + topic_params.mesh_failure_penalty_weight = 0.0; //deactivate mesh failure penalties + topic_params.invalid_message_deliveries_weight = -2.0; + topic_params.invalid_message_deliveries_decay = 0.9; + topic_params.topic_weight = 0.7; + peer_score_params + .topics + .insert(topic_hash.clone(), topic_params.clone()); + peer_score_params.app_specific_weight = 1.0; + let peer_score_thresholds = PeerScoreThresholds::default(); + + //build mesh with two peers + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(1) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .explicit(0) + .outbound(0) + .scoring(Some((peer_score_params, peer_score_thresholds))) + .create_network(); + + let mut seq = 0; + let deliver_message = |gs: &mut Gossipsub, index: usize, msg: RawGossipsubMessage| { + gs.handle_received_message(msg, &peers[index]); + }; + + //peer 0 delivers invalid message from self + let mut m = random_message(&mut seq, &topics); + m.source = Some(gs.publish_config.get_own_id().unwrap().clone()); + + deliver_message(&mut gs, 0, m.clone()); + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[0]), + -2.0 * 0.7 + ); + } + + #[test] + fn test_scoring_p4_ignored_message() { + let config = GossipsubConfigBuilder::default() + .validate_messages() + .build() + .unwrap(); + let mut peer_score_params = PeerScoreParams::default(); + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut topic_params = TopicScoreParams::default(); + topic_params.time_in_mesh_weight = 0.0; //deactivate time in mesh + topic_params.first_message_deliveries_weight = 0.0; //deactivate first time deliveries + topic_params.mesh_message_deliveries_weight = 0.0; //deactivate message deliveries + topic_params.mesh_failure_penalty_weight = 0.0; //deactivate mesh failure penalties + topic_params.invalid_message_deliveries_weight = -2.0; + topic_params.invalid_message_deliveries_decay = 0.9; + topic_params.topic_weight = 0.7; + peer_score_params + .topics + .insert(topic_hash.clone(), topic_params.clone()); + peer_score_params.app_specific_weight = 1.0; + let peer_score_thresholds = PeerScoreThresholds::default(); + + //build mesh with two peers + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(1) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .explicit(0) + .outbound(0) + .scoring(Some((peer_score_params, peer_score_thresholds))) + .create_network(); + + let mut seq = 0; + let deliver_message = |gs: &mut Gossipsub, index: usize, msg: RawGossipsubMessage| { + gs.handle_received_message(msg, &peers[index]); + }; + + //peer 0 delivers ignored message + let m1 = random_message(&mut seq, &topics); + deliver_message(&mut gs, 0, m1.clone()); + + assert_eq!(gs.peer_score.as_ref().unwrap().0.score(&peers[0]), 0.0); + + // Transform the inbound message + let message1 = &gs.data_transform.inbound_transform(m1.clone()).unwrap(); + + //message m1 gets ignored + gs.report_message_validation_result( + &config.message_id(&message1), + &peers[0], + MessageAcceptance::Ignore, + ) + .unwrap(); + + assert_eq!(gs.peer_score.as_ref().unwrap().0.score(&peers[0]), 0.0); + } + + #[test] + fn test_scoring_p4_application_invalidated_message() { + let config = GossipsubConfigBuilder::default() + .validate_messages() + .build() + .unwrap(); + let mut peer_score_params = PeerScoreParams::default(); + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut topic_params = TopicScoreParams::default(); + topic_params.time_in_mesh_weight = 0.0; //deactivate time in mesh + topic_params.first_message_deliveries_weight = 0.0; //deactivate first time deliveries + topic_params.mesh_message_deliveries_weight = 0.0; //deactivate message deliveries + topic_params.mesh_failure_penalty_weight = 0.0; //deactivate mesh failure penalties + topic_params.invalid_message_deliveries_weight = -2.0; + topic_params.invalid_message_deliveries_decay = 0.9; + topic_params.topic_weight = 0.7; + peer_score_params + .topics + .insert(topic_hash.clone(), topic_params.clone()); + peer_score_params.app_specific_weight = 1.0; + let peer_score_thresholds = PeerScoreThresholds::default(); + + //build mesh with two peers + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(1) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .explicit(0) + .outbound(0) + .scoring(Some((peer_score_params, peer_score_thresholds))) + .create_network(); + + let mut seq = 0; + let deliver_message = |gs: &mut Gossipsub, index: usize, msg: RawGossipsubMessage| { + gs.handle_received_message(msg, &peers[index]); + }; + + //peer 0 delivers invalid message + let m1 = random_message(&mut seq, &topics); + deliver_message(&mut gs, 0, m1.clone()); + + assert_eq!(gs.peer_score.as_ref().unwrap().0.score(&peers[0]), 0.0); + + // Transform the inbound message + let message1 = &gs.data_transform.inbound_transform(m1.clone()).unwrap(); + + //message m1 gets rejected + gs.report_message_validation_result( + &config.message_id(&message1), + &peers[0], + MessageAcceptance::Reject, + ) + .unwrap(); + + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[0]), + -2.0 * 0.7 + ); + } + + #[test] + fn test_scoring_p4_application_invalid_message_from_two_peers() { + let config = GossipsubConfigBuilder::default() + .validate_messages() + .build() + .unwrap(); + let mut peer_score_params = PeerScoreParams::default(); + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut topic_params = TopicScoreParams::default(); + topic_params.time_in_mesh_weight = 0.0; //deactivate time in mesh + topic_params.first_message_deliveries_weight = 0.0; //deactivate first time deliveries + topic_params.mesh_message_deliveries_weight = 0.0; //deactivate message deliveries + topic_params.mesh_failure_penalty_weight = 0.0; //deactivate mesh failure penalties + topic_params.invalid_message_deliveries_weight = -2.0; + topic_params.invalid_message_deliveries_decay = 0.9; + topic_params.topic_weight = 0.7; + peer_score_params + .topics + .insert(topic_hash.clone(), topic_params.clone()); + peer_score_params.app_specific_weight = 1.0; + let peer_score_thresholds = PeerScoreThresholds::default(); + + //build mesh with two peers + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(2) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .explicit(0) + .outbound(0) + .scoring(Some((peer_score_params, peer_score_thresholds))) + .create_network(); + + let mut seq = 0; + let deliver_message = |gs: &mut Gossipsub, index: usize, msg: RawGossipsubMessage| { + gs.handle_received_message(msg, &peers[index]); + }; + + //peer 0 delivers invalid message + let m1 = random_message(&mut seq, &topics); + deliver_message(&mut gs, 0, m1.clone()); + + // Transform the inbound message + let message1 = &gs.data_transform.inbound_transform(m1.clone()).unwrap(); + + //peer 1 delivers same message + deliver_message(&mut gs, 1, m1.clone()); + + assert_eq!(gs.peer_score.as_ref().unwrap().0.score(&peers[0]), 0.0); + assert_eq!(gs.peer_score.as_ref().unwrap().0.score(&peers[1]), 0.0); + + //message m1 gets rejected + gs.report_message_validation_result( + &config.message_id(&message1), + &peers[0], + MessageAcceptance::Reject, + ) + .unwrap(); + + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[0]), + -2.0 * 0.7 + ); + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[1]), + -2.0 * 0.7 + ); + } + + #[test] + fn test_scoring_p4_three_application_invalid_messages() { + let config = GossipsubConfigBuilder::default() + .validate_messages() + .build() + .unwrap(); + let mut peer_score_params = PeerScoreParams::default(); + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut topic_params = TopicScoreParams::default(); + topic_params.time_in_mesh_weight = 0.0; //deactivate time in mesh + topic_params.first_message_deliveries_weight = 0.0; //deactivate first time deliveries + topic_params.mesh_message_deliveries_weight = 0.0; //deactivate message deliveries + topic_params.mesh_failure_penalty_weight = 0.0; //deactivate mesh failure penalties + topic_params.invalid_message_deliveries_weight = -2.0; + topic_params.invalid_message_deliveries_decay = 0.9; + topic_params.topic_weight = 0.7; + peer_score_params + .topics + .insert(topic_hash.clone(), topic_params.clone()); + peer_score_params.app_specific_weight = 1.0; + let peer_score_thresholds = PeerScoreThresholds::default(); + + //build mesh with one peer + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(1) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .explicit(0) + .outbound(0) + .scoring(Some((peer_score_params, peer_score_thresholds))) + .create_network(); + + let mut seq = 0; + let deliver_message = |gs: &mut Gossipsub, index: usize, msg: RawGossipsubMessage| { + gs.handle_received_message(msg, &peers[index]); + }; + + //peer 0 delivers two invalid message + let m1 = random_message(&mut seq, &topics); + let m2 = random_message(&mut seq, &topics); + let m3 = random_message(&mut seq, &topics); + deliver_message(&mut gs, 0, m1.clone()); + deliver_message(&mut gs, 0, m2.clone()); + deliver_message(&mut gs, 0, m3.clone()); + + // Transform the inbound message + let message1 = &gs.data_transform.inbound_transform(m1.clone()).unwrap(); + + // Transform the inbound message + let message2 = &gs.data_transform.inbound_transform(m2.clone()).unwrap(); + // Transform the inbound message + let message3 = &gs.data_transform.inbound_transform(m3.clone()).unwrap(); + + assert_eq!(gs.peer_score.as_ref().unwrap().0.score(&peers[0]), 0.0); + + //messages gets rejected + gs.report_message_validation_result( + &config.message_id(&message1), + &peers[0], + MessageAcceptance::Reject, + ) + .unwrap(); + gs.report_message_validation_result( + &config.message_id(&message2), + &peers[0], + MessageAcceptance::Reject, + ) + .unwrap(); + gs.report_message_validation_result( + &config.message_id(&message3), + &peers[0], + MessageAcceptance::Reject, + ) + .unwrap(); + + //number of invalid messages gets squared + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[0]), + 9.0 * -2.0 * 0.7 + ); + } + + #[test] + fn test_scoring_p4_decay() { + let config = GossipsubConfigBuilder::default() + .validate_messages() + .build() + .unwrap(); + let mut peer_score_params = PeerScoreParams::default(); + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut topic_params = TopicScoreParams::default(); + topic_params.time_in_mesh_weight = 0.0; //deactivate time in mesh + topic_params.first_message_deliveries_weight = 0.0; //deactivate first time deliveries + topic_params.mesh_message_deliveries_weight = 0.0; //deactivate message deliveries + topic_params.mesh_failure_penalty_weight = 0.0; //deactivate mesh failure penalties + topic_params.invalid_message_deliveries_weight = -2.0; + topic_params.invalid_message_deliveries_decay = 0.9; + topic_params.topic_weight = 0.7; + peer_score_params + .topics + .insert(topic_hash.clone(), topic_params.clone()); + peer_score_params.app_specific_weight = 1.0; + let peer_score_thresholds = PeerScoreThresholds::default(); + + //build mesh with one peer + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(1) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(config.clone()) + .explicit(0) + .outbound(0) + .scoring(Some((peer_score_params, peer_score_thresholds))) + .create_network(); + + let mut seq = 0; + let deliver_message = |gs: &mut Gossipsub, index: usize, msg: RawGossipsubMessage| { + gs.handle_received_message(msg, &peers[index]); + }; + + //peer 0 delivers invalid message + let m1 = random_message(&mut seq, &topics); + deliver_message(&mut gs, 0, m1.clone()); + + // Transform the inbound message + let message1 = &gs.data_transform.inbound_transform(m1.clone()).unwrap(); + assert_eq!(gs.peer_score.as_ref().unwrap().0.score(&peers[0]), 0.0); + + //message m1 gets rejected + gs.report_message_validation_result( + &config.message_id(&message1), + &peers[0], + MessageAcceptance::Reject, + ) + .unwrap(); + + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[0]), + -2.0 * 0.7 + ); + + //we decay + gs.peer_score.as_mut().unwrap().0.refresh_scores(); + + // the number of invalids gets decayed to 0.9 and then squared in the score + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[0]), + 0.9 * 0.9 * -2.0 * 0.7 + ); + } + + #[test] + fn test_scoring_p5() { + let mut peer_score_params = PeerScoreParams::default(); + peer_score_params.app_specific_weight = 2.0; + + //build mesh with one peer + let (mut gs, peers, _) = inject_nodes1() + .peer_no(1) + .topics(vec!["test".into()]) + .to_subscribe(true) + .gs_config(GossipsubConfig::default()) + .explicit(0) + .outbound(0) + .scoring(Some((peer_score_params, PeerScoreThresholds::default()))) + .create_network(); + + gs.set_application_score(&peers[0], 1.1); + + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[0]), + 1.1 * 2.0 + ); + } + + #[test] + fn test_scoring_p6() { + let mut peer_score_params = PeerScoreParams::default(); + peer_score_params.ip_colocation_factor_threshold = 5.0; + peer_score_params.ip_colocation_factor_weight = -2.0; + + let (mut gs, _, _) = inject_nodes1() + .peer_no(0) + .topics(vec![]) + .to_subscribe(false) + .gs_config(GossipsubConfig::default()) + .explicit(0) + .outbound(0) + .scoring(Some((peer_score_params, PeerScoreThresholds::default()))) + .create_network(); + + //create 5 peers with the same ip + let addr = Multiaddr::from(Ipv4Addr::new(10, 1, 2, 3)); + let peers = vec![ + add_peer_with_addr(&mut gs, &vec![], false, false, addr.clone()), + add_peer_with_addr(&mut gs, &vec![], false, false, addr.clone()), + add_peer_with_addr(&mut gs, &vec![], true, false, addr.clone()), + add_peer_with_addr(&mut gs, &vec![], true, false, addr.clone()), + add_peer_with_addr(&mut gs, &vec![], true, true, addr.clone()), + ]; + + //create 4 other peers with other ip + let addr2 = Multiaddr::from(Ipv4Addr::new(10, 1, 2, 4)); + let others = vec![ + add_peer_with_addr(&mut gs, &vec![], false, false, addr2.clone()), + add_peer_with_addr(&mut gs, &vec![], false, false, addr2.clone()), + add_peer_with_addr(&mut gs, &vec![], true, false, addr2.clone()), + add_peer_with_addr(&mut gs, &vec![], true, false, addr2.clone()), + ]; + + //no penalties yet + for peer in peers.iter().chain(others.iter()) { + assert_eq!(gs.peer_score.as_ref().unwrap().0.score(peer), 0.0); + } + + //add additional connection for 3 others with addr + for i in 0..3 { + gs.inject_connection_established( + &others[i], + &ConnectionId::new(0), + &ConnectedPoint::Dialer { + address: addr.clone(), + }, + ); + } + + //penalties apply squared + for peer in peers.iter().chain(others.iter().take(3)) { + assert_eq!(gs.peer_score.as_ref().unwrap().0.score(peer), 9.0 * -2.0); + } + //fourth other peer still no penalty + assert_eq!(gs.peer_score.as_ref().unwrap().0.score(&others[3]), 0.0); + + //add additional connection for 3 of the peers to addr2 + for i in 0..3 { + gs.inject_connection_established( + &peers[i], + &ConnectionId::new(0), + &ConnectedPoint::Dialer { + address: addr2.clone(), + }, + ); + } + + //double penalties for the first three of each + for peer in peers.iter().take(3).chain(others.iter().take(3)) { + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(peer), + (9.0 + 4.0) * -2.0 + ); + } + + //single penalties for the rest + for peer in peers.iter().skip(3) { + assert_eq!(gs.peer_score.as_ref().unwrap().0.score(peer), 9.0 * -2.0); + } + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&others[3]), + 4.0 * -2.0 + ); + + //two times same ip doesn't count twice + gs.inject_connection_established( + &peers[0], + &ConnectionId::new(0), + &ConnectedPoint::Dialer { + address: addr.clone(), + }, + ); + + //nothing changed + //double penalties for the first three of each + for peer in peers.iter().take(3).chain(others.iter().take(3)) { + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(peer), + (9.0 + 4.0) * -2.0 + ); + } + + //single penalties for the rest + for peer in peers.iter().skip(3) { + assert_eq!(gs.peer_score.as_ref().unwrap().0.score(peer), 9.0 * -2.0); + } + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&others[3]), + 4.0 * -2.0 + ); + } + + #[test] + fn test_scoring_p7_grafts_before_backoff() { + let config = GossipsubConfigBuilder::default() + .prune_backoff(Duration::from_millis(200)) + .graft_flood_threshold(Duration::from_millis(100)) + .build() + .unwrap(); + let mut peer_score_params = PeerScoreParams::default(); + peer_score_params.behaviour_penalty_weight = -2.0; + peer_score_params.behaviour_penalty_decay = 0.9; + + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(2) + .topics(vec!["test".into()]) + .to_subscribe(false) + .gs_config(config) + .explicit(0) + .outbound(0) + .scoring(Some((peer_score_params, PeerScoreThresholds::default()))) + .create_network(); + + //remove peers from mesh and send prune to them => this adds a backoff for the peers + for i in 0..2 { + gs.mesh.get_mut(&topics[0]).unwrap().remove(&peers[i]); + gs.send_graft_prune( + HashMap::new(), + vec![(peers[i].clone(), vec![topics[0].clone()])] + .into_iter() + .collect(), + HashSet::new(), + ); + } + + //wait 50 millisecs + sleep(Duration::from_millis(50)); + + //first peer tries to graft + gs.handle_graft(&peers[0], vec![topics[0].clone()]); + + //double behaviour penalty for first peer (squared) + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[0]), + 4.0 * -2.0 + ); + + //wait 100 millisecs + sleep(Duration::from_millis(100)); + + //second peer tries to graft + gs.handle_graft(&peers[1], vec![topics[0].clone()]); + + //single behaviour penalty for second peer + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[1]), + 1.0 * -2.0 + ); + + //test decay + gs.peer_score.as_mut().unwrap().0.refresh_scores(); + + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[0]), + 4.0 * 0.9 * 0.9 * -2.0 + ); + assert_eq!( + gs.peer_score.as_ref().unwrap().0.score(&peers[1]), + 1.0 * 0.9 * 0.9 * -2.0 + ); + } + + #[test] + fn test_opportunistic_grafting() { + let config = GossipsubConfigBuilder::default() + .mesh_n_low(3) + .mesh_n(5) + .mesh_n_high(7) + .mesh_outbound_min(0) //deactivate outbound handling + .opportunistic_graft_ticks(2) + .opportunistic_graft_peers(2) + .build() + .unwrap(); + let mut peer_score_params = PeerScoreParams::default(); + peer_score_params.app_specific_weight = 1.0; + let mut thresholds = PeerScoreThresholds::default(); + thresholds.opportunistic_graft_threshold = 2.0; + + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(5) + .topics(vec!["test".into()]) + .to_subscribe(false) + .gs_config(config) + .explicit(0) + .outbound(0) + .scoring(Some((peer_score_params, thresholds))) + .create_network(); + + //fill mesh with 5 peers + for peer in &peers { + gs.handle_graft(peer, topics.clone()); + } + + //add additional 5 peers + let others: Vec<_> = (0..5) + .into_iter() + .map(|_| add_peer(&mut gs, &topics, false, false)) + .collect(); + + //currently mesh equals peers + assert_eq!(gs.mesh[&topics[0]], peers.iter().cloned().collect()); + + //give others high scores (but the first two have not high enough scores) + for i in 0..5 { + gs.set_application_score(&peers[i], 0.0 + i as f64); + } + + //set scores for peers in the mesh + for i in 0..5 { + gs.set_application_score(&others[i], 0.0 + i as f64); + } + + //this gives a median of exactly 2.0 => should not apply opportunistic grafting + gs.heartbeat(); + gs.heartbeat(); + + assert_eq!( + gs.mesh[&topics[0]].len(), + 5, + "should not apply opportunistic grafting" + ); + + //reduce middle score to 1.0 giving a median of 1.0 + gs.set_application_score(&peers[2], 1.0); + + //opportunistic grafting after two heartbeats + + gs.heartbeat(); + assert_eq!( + gs.mesh[&topics[0]].len(), + 5, + "should not apply opportunistic grafting after first tick" + ); + + gs.heartbeat(); + + assert_eq!( + gs.mesh[&topics[0]].len(), + 7, + "opportunistic grafting should have added 2 peers" + ); + + assert!( + gs.mesh[&topics[0]].is_superset(&peers.iter().cloned().collect()), + "old peers are still part of the mesh" + ); + + assert!( + gs.mesh[&topics[0]].is_disjoint(&others.iter().cloned().take(2).collect()), + "peers below or equal to median should not be added in opportunistic grafting" + ); + } + + #[test] + fn test_ignore_graft_from_unknown_topic() { + //build gossipsub without subscribing to any topics + let (mut gs, _, _) = inject_nodes1() + .peer_no(0) + .topics(vec![]) + .to_subscribe(false) + .create_network(); + + //handle an incoming graft for some topic + gs.handle_graft(&PeerId::random(), vec![Topic::new("test").hash()]); + + //assert that no prune got created + assert_eq!( + count_control_msgs(&gs, |_, a| match a { + GossipsubControlAction::Prune { .. } => true, + _ => false, + }), + 0, + "we should not prune after graft in unknown topic" + ); + } + + #[test] + fn test_ignore_too_many_iwants_from_same_peer_for_same_message() { + let config = GossipsubConfig::default(); + //build gossipsub with full mesh + let (mut gs, _, topics) = inject_nodes1() + .peer_no(config.mesh_n_high()) + .topics(vec!["test".into()]) + .to_subscribe(false) + .create_network(); + + //add another peer not in the mesh + let peer = add_peer(&mut gs, &topics, false, false); + + //receive a message + let mut seq = 0; + let m1 = random_message(&mut seq, &topics); + + // Transform the inbound message + let message1 = &gs.data_transform.inbound_transform(m1.clone()).unwrap(); + + let id = config.message_id(&message1); + + gs.handle_received_message(m1.clone(), &PeerId::random()); + + //clear events + gs.events.clear(); + + //the first gossip_retransimission many iwants return the valid message, all others are + // ignored. + for _ in 0..(2 * config.gossip_retransimission() + 10) { + gs.handle_iwant(&peer, vec![id.clone()]); + } + + assert_eq!( + gs.events + .iter() + .map(|e| match e { + NetworkBehaviourAction::NotifyHandler { event, .. } => { + let event = proto_to_message(event); + event.messages.len() + } + _ => 0, + }) + .sum::(), + config.gossip_retransimission() as usize, + "not more then gossip_retransmission many messages get sent back" + ); + } + + #[test] + fn test_ignore_too_many_ihaves() { + let config = GossipsubConfigBuilder::default() + .max_ihave_messages(10) + .build() + .unwrap(); + //build gossipsub with full mesh + let (mut gs, _, topics) = inject_nodes1() + .peer_no(config.mesh_n_high()) + .topics(vec!["test".into()]) + .to_subscribe(false) + .gs_config(config.clone()) + .create_network(); + + //add another peer not in the mesh + let peer = add_peer(&mut gs, &topics, false, false); + + //peer has 20 messages + let mut seq = 0; + let messages: Vec<_> = (0..20).map(|_| random_message(&mut seq, &topics)).collect(); + + //peer sends us one ihave for each message in order + for raw_message in &messages { + // Transform the inbound message + let message = &gs + .data_transform + .inbound_transform(raw_message.clone()) + .unwrap(); + + gs.handle_ihave( + &peer, + vec![(topics[0].clone(), vec![config.message_id(&message)])], + ); + } + + let first_ten: HashSet<_> = messages + .iter() + .take(10) + .map(|msg| gs.data_transform.inbound_transform(msg.clone()).unwrap()) + .map(|m| config.message_id(&m)) + .collect(); + + //we send iwant only for the first 10 messages + assert_eq!( + count_control_msgs(&gs, |p, action| match action { + GossipsubControlAction::IWant { message_ids } => + p == &peer && { + assert_eq!( + message_ids.len(), + 1, + "each iwant should have one message \ + corresponding to one ihave" + ); + + assert!(first_ten.contains(&message_ids[0])); + + true + }, + _ => false, + }), + 10, + "exactly the first ten ihaves should be processed and one iwant for each created" + ); + + //after a heartbeat everything is forgotten + gs.heartbeat(); + for raw_message in messages[10..].iter() { + // Transform the inbound message + let message = &gs + .data_transform + .inbound_transform(raw_message.clone()) + .unwrap(); + + gs.handle_ihave( + &peer, + vec![(topics[0].clone(), vec![config.message_id(&message)])], + ); + } + + //we sent iwant for all 20 messages + assert_eq!( + count_control_msgs(&gs, |p, action| match action { + GossipsubControlAction::IWant { message_ids } => + p == &peer && { + assert_eq!( + message_ids.len(), + 1, + "each iwant should have one message \ + corresponding to one ihave" + ); + true + }, + _ => false, + }), + 20, + "all 20 should get sent" + ); + } + + #[test] + fn test_ignore_too_many_messages_in_ihave() { + let config = GossipsubConfigBuilder::default() + .max_ihave_messages(10) + .max_ihave_length(10) + .build() + .unwrap(); + //build gossipsub with full mesh + let (mut gs, _, topics) = inject_nodes1() + .peer_no(config.mesh_n_high()) + .topics(vec!["test".into()]) + .to_subscribe(false) + .gs_config(config.clone()) + .create_network(); + + //add another peer not in the mesh + let peer = add_peer(&mut gs, &topics, false, false); + + //peer has 20 messages + let mut seq = 0; + let message_ids: Vec<_> = (0..20) + .map(|_| random_message(&mut seq, &topics)) + .map(|msg| gs.data_transform.inbound_transform(msg.clone()).unwrap()) + .map(|msg| config.message_id(&msg)) + .collect(); + + //peer sends us three ihaves + gs.handle_ihave( + &peer, + vec![( + topics[0].clone(), + message_ids[0..8].iter().cloned().collect(), + )], + ); + gs.handle_ihave( + &peer, + vec![( + topics[0].clone(), + message_ids[0..12].iter().cloned().collect(), + )], + ); + gs.handle_ihave( + &peer, + vec![( + topics[0].clone(), + message_ids[0..20].iter().cloned().collect(), + )], + ); + + let first_twelve: HashSet<_> = message_ids.iter().take(12).collect(); + + //we send iwant only for the first 10 messages + let mut sum = 0; + assert_eq!( + count_control_msgs(&gs, |p, action| match action { + GossipsubControlAction::IWant { message_ids } => + p == &peer && { + assert!(first_twelve.is_superset(&message_ids.iter().collect())); + sum += message_ids.len(); + true + }, + _ => false, + }), + 2, + "the third ihave should get ignored and no iwant sent" + ); + + assert_eq!(sum, 10, "exactly the first ten ihaves should be processed"); + + //after a heartbeat everything is forgotten + gs.heartbeat(); + gs.handle_ihave( + &peer, + vec![( + topics[0].clone(), + message_ids[10..20].iter().cloned().collect(), + )], + ); + + //we sent 20 iwant messages + let mut sum = 0; + assert_eq!( + count_control_msgs(&gs, |p, action| match action { + GossipsubControlAction::IWant { message_ids } => + p == &peer && { + sum += message_ids.len(); + true + }, + _ => false, + }), + 3 + ); + assert_eq!(sum, 20, "exactly 20 iwants should get sent"); + } + + #[test] + fn test_limit_number_of_message_ids_inside_ihave() { + let config = GossipsubConfigBuilder::default() + .max_ihave_messages(10) + .max_ihave_length(100) + .build() + .unwrap(); + //build gossipsub with full mesh + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(config.mesh_n_high()) + .topics(vec!["test".into()]) + .to_subscribe(false) + .gs_config(config.clone()) + .create_network(); + + //graft to all peers to really fill the mesh with all the peers + for peer in peers { + gs.handle_graft(&peer, topics.clone()); + } + + //add two other peers not in the mesh + let p1 = add_peer(&mut gs, &topics, false, false); + let p2 = add_peer(&mut gs, &topics, false, false); + + //receive 200 messages from another peer + let mut seq = 0; + for _ in 0..200 { + gs.handle_received_message(random_message(&mut seq, &topics), &PeerId::random()); + } + + //emit gossip + gs.emit_gossip(); + + // both peers should have gotten 100 random ihave messages, to asser the randomness, we + // assert that both have not gotten the same set of messages, but have an intersection + // (which is the case with very high probability, the probabiltity of failure is < 10^-58). + + let mut ihaves1 = HashSet::new(); + let mut ihaves2 = HashSet::new(); + + assert_eq!( + count_control_msgs(&gs, |p, action| match action { + GossipsubControlAction::IHave { message_ids, .. } => { + if p == &p1 { + ihaves1 = message_ids.iter().cloned().collect(); + true + } else if p == &p2 { + ihaves2 = message_ids.iter().cloned().collect(); + true + } else { + false + } + } + _ => false, + }), + 2, + "should have emitted one ihave to p1 and one to p2" + ); + + assert_eq!( + ihaves1.len(), + 100, + "should have sent 100 message ids in ihave to p1" + ); + assert_eq!( + ihaves2.len(), + 100, + "should have sent 100 message ids in ihave to p2" + ); + assert!( + ihaves1 != ihaves2, + "should have sent different random messages to p1 and p2 \ + (this may fail with a probability < 10^-58" + ); + assert!( + ihaves1.intersection(&ihaves2).into_iter().count() > 0, + "should have sent random messages with some common messages to p1 and p2 \ + (this may fail with a probability < 10^-58" + ); + } + + #[test] + fn test_iwant_penalties() { + let config = GossipsubConfigBuilder::default() + .iwant_followup_time(Duration::from_secs(4)) + .build() + .unwrap(); + let mut peer_score_params = PeerScoreParams::default(); + peer_score_params.behaviour_penalty_weight = -1.0; + + //fill the mesh + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(config.mesh_n_high()) + .topics(vec!["test".into()]) + .to_subscribe(false) + .gs_config(config.clone()) + .explicit(0) + .outbound(0) + .scoring(Some((peer_score_params, PeerScoreThresholds::default()))) + .create_network(); + + //graft to all peers to really fill the mesh with all the peers + for peer in peers { + gs.handle_graft(&peer, topics.clone()); + } + + //add 100 more peers + let other_peers: Vec<_> = (0..100) + .map(|_| add_peer(&mut gs, &topics, false, false)) + .collect(); + + //each peer sends us two ihave containing each two message ids + let mut first_messages = Vec::new(); + let mut second_messages = Vec::new(); + let mut seq = 0; + for peer in &other_peers { + for _ in 0..2 { + let msg1 = random_message(&mut seq, &topics); + let msg2 = random_message(&mut seq, &topics); + + // Decompress the raw message and calculate the message id. + // Transform the inbound message + let message1 = &gs.data_transform.inbound_transform(msg1.clone()).unwrap(); + + // Transform the inbound message + let message2 = &gs.data_transform.inbound_transform(msg2.clone()).unwrap(); + + first_messages.push(msg1.clone()); + second_messages.push(msg2.clone()); + gs.handle_ihave( + peer, + vec![( + topics[0].clone(), + vec![config.message_id(&message1), config.message_id(&message2)], + )], + ); + } + } + + //other peers send us all the first message ids in time + for message in first_messages { + gs.handle_received_message(message.clone(), &PeerId::random()); + } + + //now we do a heartbeat no penalization should have been applied yet + gs.heartbeat(); + + for peer in &other_peers { + assert_eq!(gs.peer_score.as_ref().unwrap().0.score(peer), 0.0); + } + + //receive the first twenty of the second messages (that are the messages of the first 10 + // peers) + for message in second_messages.iter().take(20) { + gs.handle_received_message(message.clone(), &PeerId::random()); + } + + //sleep for one second + sleep(Duration::from_secs(4)); + + //now we do a heartbeat to apply penalization + gs.heartbeat(); + + //now we get all the second messages + for message in second_messages { + gs.handle_received_message(message.clone(), &PeerId::random()); + } + + //no further penalizations should get applied + gs.heartbeat(); + + //now randomly some peers got penalized, some may not, and some may got penalized twice + //with very high probability (> 1 - 10^-50) all three cases are present under the 100 peers + //but the first 10 peers should all not got penalized. + let mut not_penalized = 0; + let mut single_penalized = 0; + let mut double_penalized = 0; + + for (i, peer) in other_peers.iter().enumerate() { + let score = gs.peer_score.as_ref().unwrap().0.score(peer); + if score == 0.0 { + not_penalized += 1; + } else if score == -1.0 { + assert!(i > 9); + single_penalized += 1; + } else if score == -4.0 { + assert!(i > 9); + double_penalized += 1 + } else { + assert!(false, "Invalid score of peer") + } + } + + assert!(not_penalized > 10); + assert!(single_penalized > 0); + assert!(double_penalized > 0); + } + + #[test] + fn test_publish_to_floodsub_peers_without_flood_publish() { + let config = GossipsubConfigBuilder::default() + .flood_publish(false) + .build() + .unwrap(); + let (mut gs, _, topics) = inject_nodes1() + .peer_no(config.mesh_n_low() - 1) + .topics(vec!["test".into()]) + .to_subscribe(false) + .gs_config(config) + .create_network(); + + //add two floodsub peer, one explicit, one implicit + let p1 = add_peer_with_addr_and_kind( + &mut gs, + &topics, + false, + false, + Multiaddr::empty(), + Some(PeerKind::Floodsub), + ); + let p2 = + add_peer_with_addr_and_kind(&mut gs, &topics, false, false, Multiaddr::empty(), None); + + //p1 and p2 are not in the mesh + assert!(!gs.mesh[&topics[0]].contains(&p1) && !gs.mesh[&topics[0]].contains(&p2)); + + //publish a message + let publish_data = vec![0; 42]; + gs.publish(Topic::new("test"), publish_data).unwrap(); + + // Collect publish messages to floodsub peers + let publishes = gs + .events + .iter() + .fold(vec![], |mut collected_publish, e| match e { + NetworkBehaviourAction::NotifyHandler { peer_id, event, .. } => { + if peer_id == &p1 || peer_id == &p2 { + let event = proto_to_message(event); + for s in &event.messages { + collected_publish.push(s.clone()); + } + } + collected_publish + } + _ => collected_publish, + }); + + assert_eq!( + publishes.len(), + 2, + "Should send a publish message to all floodsub peers" + ); + } + + #[test] + fn test_do_not_use_floodsub_in_fanout() { + let config = GossipsubConfigBuilder::default() + .flood_publish(false) + .build() + .unwrap(); + let (mut gs, _, _) = inject_nodes1() + .peer_no(config.mesh_n_low() - 1) + .topics(Vec::new()) + .to_subscribe(false) + .gs_config(config) + .create_network(); + + let topic = Topic::new("test"); + let topics = vec![topic.hash()]; + + //add two floodsub peer, one explicit, one implicit + let p1 = add_peer_with_addr_and_kind( + &mut gs, + &topics, + false, + false, + Multiaddr::empty(), + Some(PeerKind::Floodsub), + ); + let p2 = + add_peer_with_addr_and_kind(&mut gs, &topics, false, false, Multiaddr::empty(), None); + + //publish a message + let publish_data = vec![0; 42]; + gs.publish(Topic::new("test"), publish_data).unwrap(); + + // Collect publish messages to floodsub peers + let publishes = gs + .events + .iter() + .fold(vec![], |mut collected_publish, e| match e { + NetworkBehaviourAction::NotifyHandler { peer_id, event, .. } => { + if peer_id == &p1 || peer_id == &p2 { + let event = proto_to_message(event); + for s in &event.messages { + collected_publish.push(s.clone()); + } + } + collected_publish + } + _ => collected_publish, + }); + + assert_eq!( + publishes.len(), + 2, + "Should send a publish message to all floodsub peers" + ); + + assert!( + !gs.fanout[&topics[0]].contains(&p1) && !gs.fanout[&topics[0]].contains(&p2), + "Floodsub peers are not allowed in fanout" + ); + } + + #[test] + fn test_dont_add_floodsub_peers_to_mesh_on_join() { + let (mut gs, _, _) = inject_nodes1() + .peer_no(0) + .topics(Vec::new()) + .to_subscribe(false) + .create_network(); + + let topic = Topic::new("test"); + let topics = vec![topic.hash()]; + + //add two floodsub peer, one explicit, one implicit + let _p1 = add_peer_with_addr_and_kind( + &mut gs, + &topics, + false, + false, + Multiaddr::empty(), + Some(PeerKind::Floodsub), + ); + let _p2 = + add_peer_with_addr_and_kind(&mut gs, &topics, false, false, Multiaddr::empty(), None); + + gs.join(&topics[0]); + + assert!( + gs.mesh[&topics[0]].is_empty(), + "Floodsub peers should not get added to mesh" + ); + } + + #[test] + fn test_dont_send_px_to_old_gossipsub_peers() { + let (mut gs, _, topics) = inject_nodes1() + .peer_no(0) + .topics(vec!["test".into()]) + .to_subscribe(false) + .create_network(); + + //add an old gossipsub peer + let p1 = add_peer_with_addr_and_kind( + &mut gs, + &topics, + false, + false, + Multiaddr::empty(), + Some(PeerKind::Gossipsub), + ); + + //prune the peer + gs.send_graft_prune( + HashMap::new(), + vec![(p1.clone(), topics.clone())].into_iter().collect(), + HashSet::new(), + ); + + //check that prune does not contain px + assert_eq!( + count_control_msgs(&gs, |_, m| match m { + GossipsubControlAction::Prune { peers: px, .. } => !px.is_empty(), + _ => false, + }), + 0, + "Should not send px to floodsub peers" + ); + } + + #[test] + fn test_dont_send_floodsub_peers_in_px() { + //build mesh with one peer + let (mut gs, peers, topics) = inject_nodes1() + .peer_no(1) + .topics(vec!["test".into()]) + .to_subscribe(true) + .create_network(); + + //add two floodsub peers + let _p1 = add_peer_with_addr_and_kind( + &mut gs, + &topics, + false, + false, + Multiaddr::empty(), + Some(PeerKind::Floodsub), + ); + let _p2 = + add_peer_with_addr_and_kind(&mut gs, &topics, false, false, Multiaddr::empty(), None); + + //prune only mesh node + gs.send_graft_prune( + HashMap::new(), + vec![(peers[0].clone(), topics.clone())] + .into_iter() + .collect(), + HashSet::new(), + ); + + //check that px in prune message is empty + assert_eq!( + count_control_msgs(&gs, |_, m| match m { + GossipsubControlAction::Prune { peers: px, .. } => !px.is_empty(), + _ => false, + }), + 0, + "Should not include floodsub peers in px" + ); + } + + #[test] + fn test_dont_add_floodsub_peers_to_mesh_in_heartbeat() { + let (mut gs, _, topics) = inject_nodes1() + .peer_no(0) + .topics(vec!["test".into()]) + .to_subscribe(false) + .create_network(); + + //add two floodsub peer, one explicit, one implicit + let _p1 = add_peer_with_addr_and_kind( + &mut gs, + &topics, + true, + false, + Multiaddr::empty(), + Some(PeerKind::Floodsub), + ); + let _p2 = + add_peer_with_addr_and_kind(&mut gs, &topics, true, false, Multiaddr::empty(), None); + + gs.heartbeat(); + + assert!( + gs.mesh[&topics[0]].is_empty(), + "Floodsub peers should not get added to mesh" + ); + } + + // Some very basic test of public api methods. + #[test] + fn test_public_api() { + let (gs, peers, topic_hashes) = inject_nodes1() + .peer_no(4) + .topics(vec![String::from("topic1")]) + .to_subscribe(true) + .create_network(); + let peers = peers.into_iter().collect::>(); + + assert_eq!( + gs.topics().cloned().collect::>(), + topic_hashes, + "Expected topics to match registered topic." + ); + + assert_eq!( + gs.mesh_peers(&TopicHash::from_raw("topic1")) + .cloned() + .collect::>(), + peers, + "Expected peers for a registered topic to contain all peers." + ); + + assert_eq!( + gs.all_mesh_peers().cloned().collect::>(), + peers, + "Expected all_peers to contain all peers." + ); + } + + #[test] + fn test_msg_id_fn_only_called_once_with_fast_message_ids() { + struct Pointers { + slow_counter: u32, + fast_counter: u32, + }; + + let mut counters = Pointers { + slow_counter: 0, + fast_counter: 0, + }; + + let counters_pointer: *mut Pointers = &mut counters; + + let counters_address = counters_pointer as u64; + + macro_rules! get_counters_pointer { + ($m: expr) => {{ + let mut address_bytes: [u8; 8] = Default::default(); + address_bytes.copy_from_slice($m.as_slice()); + let address = u64::from_be_bytes(address_bytes); + address as *mut Pointers + }}; + } + + macro_rules! get_counters_and_hash { + ($m: expr) => {{ + let mut hasher = DefaultHasher::new(); + $m.hash(&mut hasher); + let id = hasher.finish().to_be_bytes().into(); + (id, get_counters_pointer!($m)) + }}; + } + + let message_id_fn = |m: &GossipsubMessage| -> MessageId { + let (mut id, mut counters_pointer): (MessageId, *mut Pointers) = + get_counters_and_hash!(&m.data); + unsafe { + (*counters_pointer).slow_counter += 1; + } + id.0.reverse(); + id + }; + let fast_message_id_fn = |m: &RawGossipsubMessage| -> FastMessageId { + let (id, mut counters_pointer) = get_counters_and_hash!(&m.data); + unsafe { + (*counters_pointer).fast_counter += 1; + } + id + }; + let config = GossipsubConfigBuilder::default() + .message_id_fn(message_id_fn) + .fast_message_id_fn(fast_message_id_fn) + .build() + .unwrap(); + let (mut gs, _, topic_hashes) = inject_nodes1() + .peer_no(0) + .topics(vec![String::from("topic1")]) + .to_subscribe(true) + .gs_config(config) + .create_network(); + + let message = RawGossipsubMessage { + source: None, + data: counters_address.to_be_bytes().to_vec(), + sequence_number: None, + topic: topic_hashes[0].clone(), + signature: None, + key: None, + validated: true, + }; + + for _ in 0..5 { + gs.handle_received_message(message.clone(), &PeerId::random()); + } + + assert!(counters.fast_counter <= 5); + assert_eq!(counters.slow_counter, 1); + } + + #[test] + fn test_subscribe_to_invalid_topic() { + let t1 = Topic::new("t1"); + let t2 = Topic::new("t2"); + let (mut gs, _, _) = inject_nodes::() + .subscription_filter(WhitelistSubscriptionFilter( + vec![t1.hash()].into_iter().collect(), + )) + .to_subscribe(false) + .create_network(); + + assert!(gs.subscribe(&t1).is_ok()); + assert!(gs.subscribe(&t2).is_err()); + } + + #[test] + fn test_subscribe_and_graft_with_negative_score() { + //simulate a communication between two gossipsub instances + let (mut gs1, _, topic_hashes) = inject_nodes1() + .topics(vec!["test".into()]) + .scoring(Some(( + PeerScoreParams::default(), + PeerScoreThresholds::default(), + ))) + .create_network(); + + let (mut gs2, _, _) = inject_nodes1().create_network(); + + let connection_id = ConnectionId::new(0); + + let topic = Topic::new("test"); + + let p2 = add_peer(&mut gs1, &Vec::new(), true, false); + let p1 = add_peer(&mut gs2, &topic_hashes, false, false); + + //add penalty to peer p2 + gs1.peer_score.as_mut().unwrap().0.add_penalty(&p2, 1); + + let original_score = gs1.peer_score.as_ref().unwrap().0.score(&p2); + + //subscribe to topic in gs2 + gs2.subscribe(&topic).unwrap(); + + let forward_messages_to_p1 = |gs1: &mut Gossipsub<_, _>, gs2: &mut Gossipsub<_, _>| { + //collect messages to p1 + let messages_to_p1 = gs2.events.drain(..).filter_map(|e| match e { + NetworkBehaviourAction::NotifyHandler { peer_id, event, .. } => { + if &peer_id == &p1 { + Some(event) + } else { + None + } + } + _ => None, + }); + for message in messages_to_p1 { + gs1.inject_event( + p2.clone(), + connection_id, + HandlerEvent::Message { + rpc: proto_to_message(&message), + invalid_messages: vec![], + }, + ); + } + }; + + //forward the subscribe message + forward_messages_to_p1(&mut gs1, &mut gs2); + + //heartbeats on both + gs1.heartbeat(); + gs2.heartbeat(); + + //forward messages again + forward_messages_to_p1(&mut gs1, &mut gs2); + + //nobody got penalized + assert!(gs1.peer_score.as_ref().unwrap().0.score(&p2) >= original_score); + } } diff --git a/protocols/gossipsub/src/compat.proto b/protocols/gossipsub/src/compat.proto new file mode 100644 index 00000000000..b2753bf7e41 --- /dev/null +++ b/protocols/gossipsub/src/compat.proto @@ -0,0 +1,12 @@ +syntax = "proto2"; + +package compat.pb; + +message Message { + optional bytes from = 1; + optional bytes data = 2; + optional bytes seqno = 3; + repeated string topic_ids = 4; + optional bytes signature = 5; + optional bytes key = 6; +} \ No newline at end of file diff --git a/protocols/gossipsub/src/config.rs b/protocols/gossipsub/src/config.rs index 2ef65b376ae..5a9a5e00ce9 100644 --- a/protocols/gossipsub/src/config.rs +++ b/protocols/gossipsub/src/config.rs @@ -18,19 +18,22 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::protocol::{GossipsubMessage, MessageId}; -use libp2p_core::PeerId; use std::borrow::Cow; use std::time::Duration; +use libp2p_core::PeerId; + +use crate::types::{FastMessageId, GossipsubMessage, MessageId, RawGossipsubMessage}; + /// The types of message validation that can be employed by gossipsub. #[derive(Debug, Clone)] pub enum ValidationMode { - /// This is the default setting. This requires the message author to be a valid `PeerId` and to + /// This is the default setting. This requires the message author to be a valid [`PeerId`] and to /// be present as well as the sequence number. All messages must have valid signatures. /// - /// NOTE: This setting will reject messages from nodes using `PrivacyMode::Anonymous` and - /// all messages that do not have signatures. + /// NOTE: This setting will reject messages from nodes using + /// [`crate::behaviour::MessageAuthenticity::Anonymous`] and all messages that do not have + /// signatures. Strict, /// This setting permits messages that have no author, sequence number or signature. If any of /// these fields exist in the message these are validated. @@ -48,61 +51,158 @@ pub enum ValidationMode { /// Configuration parameters that define the performance of the gossipsub network. #[derive(Clone)] pub struct GossipsubConfig { - /// The protocol id to negotiate this protocol (default is `/meshsub/1.0.0`). - pub protocol_id: Cow<'static, [u8]>, + protocol_id_prefix: Cow<'static, str>, + history_length: usize, + history_gossip: usize, + mesh_n: usize, + mesh_n_low: usize, + mesh_n_high: usize, + retain_scores: usize, + gossip_lazy: usize, + gossip_factor: f64, + heartbeat_initial_delay: Duration, + heartbeat_interval: Duration, + fanout_ttl: Duration, + check_explicit_peers_ticks: u64, + max_transmit_size: usize, + duplicate_cache_time: Duration, + validate_messages: bool, + validation_mode: ValidationMode, + message_id_fn: fn(&GossipsubMessage) -> MessageId, + fast_message_id_fn: Option FastMessageId>, + allow_self_origin: bool, + do_px: bool, + prune_peers: usize, + prune_backoff: Duration, + backoff_slack: u32, + flood_publish: bool, + graft_flood_threshold: Duration, + mesh_outbound_min: usize, + opportunistic_graft_ticks: u64, + opportunistic_graft_peers: usize, + gossip_retransimission: u32, + max_messages_per_rpc: Option, + max_ihave_length: usize, + max_ihave_messages: usize, + iwant_followup_time: Duration, + support_floodsub: bool, + published_message_ids_cache_time: Duration, +} + +impl GossipsubConfig { + // All the getters + + /// The protocol id prefix to negotiate this protocol. The protocol id is of the form + /// `//`. As gossipsub supports version 1.0 and 1.1, there are two + /// protocol id's supported. + /// + /// The default prefix is `meshsub`, giving the supported protocol ids: `/meshsub/1.1.0` and `/meshsub/1.0.0`, negotiated in that order. + pub fn protocol_id_prefix(&self) -> &Cow<'static, str> { + &self.protocol_id_prefix + } // Overlay network parameters. /// Number of heartbeats to keep in the `memcache` (default is 5). - pub history_length: usize, + pub fn history_length(&self) -> usize { + self.history_length + } /// Number of past heartbeats to gossip about (default is 3). - pub history_gossip: usize, + pub fn history_gossip(&self) -> usize { + self.history_gossip + } /// Target number of peers for the mesh network (D in the spec, default is 6). - pub mesh_n: usize, + pub fn mesh_n(&self) -> usize { + self.mesh_n + } /// Minimum number of peers in mesh network before adding more (D_lo in the spec, default is 5). - pub mesh_n_low: usize, + pub fn mesh_n_low(&self) -> usize { + self.mesh_n_low + } /// Maximum number of peers in mesh network before removing some (D_high in the spec, default /// is 12). - pub mesh_n_high: usize, + pub fn mesh_n_high(&self) -> usize { + self.mesh_n_high + } - /// Number of peers to emit gossip to during a heartbeat (D_lazy in the spec, default is 6). - pub gossip_lazy: usize, + /// Affects how peers are selected when pruning a mesh due to over subscription. + /// + /// At least `retain_scores` of the retained peers will be high-scoring, while the remainder are + /// chosen randomly (D_score in the spec, default is 4). + pub fn retain_scores(&self) -> usize { + self.retain_scores + } + + /// Minimum number of peers to emit gossip to during a heartbeat (D_lazy in the spec, + /// default is 6). + pub fn gossip_lazy(&self) -> usize { + self.gossip_lazy + } + + /// Affects how many peers we will emit gossip to at each heartbeat. + /// + /// We will send gossip to `gossip_factor * (total number of non-mesh peers)`, or + /// `gossip_lazy`, whichever is greater. The default is 0.25. + pub fn gossip_factor(&self) -> f64 { + self.gossip_factor + } /// Initial delay in each heartbeat (default is 5 seconds). - pub heartbeat_initial_delay: Duration, + pub fn heartbeat_initial_delay(&self) -> Duration { + self.heartbeat_initial_delay + } /// Time between each heartbeat (default is 1 second). - pub heartbeat_interval: Duration, + pub fn heartbeat_interval(&self) -> Duration { + self.heartbeat_interval + } /// Time to live for fanout peers (default is 60 seconds). - pub fanout_ttl: Duration, + pub fn fanout_ttl(&self) -> Duration { + self.fanout_ttl + } - /// The maximum byte size for each gossip (default is 2048 bytes). - pub max_transmit_size: usize, + /// The number of heartbeat ticks until we recheck the connection to explicit peers and + /// reconnecting if necessary (default 300). + pub fn check_explicit_peers_ticks(&self) -> u64 { + self.check_explicit_peers_ticks + } + + /// The maximum byte size for each gossipsub RPC (default is 65536 bytes). + /// + /// This represents the maximum size of the entire protobuf payload. It must be at least + /// large enough to support basic control messages. If Peer eXchange is enabled, this + /// must be large enough to transmit the desired peer information on pruning. It must be at + /// least 100 bytes. Default is 65536 bytes. + pub fn max_transmit_size(&self) -> usize { + self.max_transmit_size + } /// Duplicates are prevented by storing message id's of known messages in an LRU time cache. /// This settings sets the time period that messages are stored in the cache. Duplicates can be /// received if duplicate messages are sent at a time greater than this setting apart. The /// default is 1 minute. - pub duplicate_cache_time: Duration, - - /// Flag determining if gossipsub topics are hashed or sent as plain strings (default is false). - pub hash_topics: bool, + pub fn duplicate_cache_time(&self) -> Duration { + self.duplicate_cache_time + } /// When set to `true`, prevents automatic forwarding of all received messages. This setting /// allows a user to validate the messages before propagating them to their peers. If set to - /// true, the user must manually call `validate_message()` on the behaviour to forward message - /// once validated (default is `false`). Furthermore, the application may optionally call - /// `invalidate_message()` on the behaviour to remove the message from the memcache. The - /// default is false. - pub validate_messages: bool, + /// true, the user must manually call [`crate::Gossipsub::report_message_validation_result()`] + /// on the behaviour to forward message once validated (default is `false`). + /// The default is `false`. + pub fn validate_messages(&self) -> bool { + self.validate_messages + } /// Determines the level of validation used when receiving messages. See [`ValidationMode`] /// for the available types. The default is ValidationMode::Strict. - pub validation_mode: ValidationMode, + pub fn validation_mode(&self) -> &ValidationMode { + &self.validation_mode + } /// A user-defined function allowing the user to specify the message id of a gossipsub message. /// The default value is to concatenate the source peer id with a sequence number. Setting this @@ -110,43 +210,154 @@ pub struct GossipsubConfig { /// addressing, where this function may be set to `hash(message)`. This would prevent messages /// of the same content from being duplicated. /// - /// The function takes a `GossipsubMessage` as input and outputs a String to be interpreted as + /// The function takes a [`GossipsubMessage`] as input and outputs a String to be interpreted as /// the message id. - pub message_id_fn: fn(&GossipsubMessage) -> MessageId, + pub fn message_id(&self, message: &GossipsubMessage) -> MessageId { + (self.message_id_fn)(message) + } + + /// A user-defined optional function that computes fast ids from raw messages. This can be used + /// to avoid possibly expensive transformations from [`RawGossipsubMessage`] to + /// [`GossipsubMessage`] for duplicates. Two semantically different messages must always + /// have different fast message ids, but it is allowed that two semantically identical messages + /// have different fast message ids as long as the message_id_fn produces the same id for them. + /// + /// The function takes a [`RawGossipsubMessage`] as input and outputs a String to be + /// interpreted as the fast message id. Default is None. + pub fn fast_message_id(&self, message: &RawGossipsubMessage) -> Option { + self.fast_message_id_fn + .map(|fast_message_id_fn| fast_message_id_fn(message)) + } + + /// By default, gossipsub will reject messages that are sent to us that have the same message + /// source as we have specified locally. Enabling this, allows these messages and prevents + /// penalizing the peer that sent us the message. Default is false. + pub fn allow_self_origin(&self) -> bool { + self.allow_self_origin + } + + /// Whether Peer eXchange is enabled; this should be enabled in bootstrappers and other well + /// connected/trusted nodes. The default is true. + pub fn do_px(&self) -> bool { + self.do_px + } + + /// Controls the number of peers to include in prune Peer eXchange. + /// When we prune a peer that's eligible for PX (has a good score, etc), we will try to + /// send them signed peer records for up to `prune_peers` other peers that we + /// know of. It is recommended that this value is larger than `mesh_n_high` so that the pruned + /// peer can reliably form a full mesh. The default is typically 16 however until signed + /// records are spec'd this is disabled and set to 0. + pub fn prune_peers(&self) -> usize { + self.prune_peers + } + + /// Controls the backoff time for pruned peers. This is how long + /// a peer must wait before attempting to graft into our mesh again after being pruned. + /// When pruning a peer, we send them our value of `prune_backoff` so they know + /// the minimum time to wait. Peers running older versions may not send a backoff time, + /// so if we receive a prune message without one, we will wait at least `prune_backoff` + /// before attempting to re-graft. The default is one minute. + pub fn prune_backoff(&self) -> Duration { + self.prune_backoff + } + + /// Number of heartbeat slots considered as slack for backoffs. This gurantees that we wait + /// at least backoff_slack heartbeats after a backoff is over before we try to graft. This + /// solves problems occuring through high latencies. In particular if + /// `backoff_slack * heartbeat_interval` is longer than any latencies between processing + /// prunes on our side and processing prunes on the receiving side this guarantees that we + /// get not punished for too early grafting. The default is 1. + pub fn backoff_slack(&self) -> u32 { + self.backoff_slack + } + + /// Whether to do flood publishing or not. If enabled newly created messages will always be + /// sent to all peers that are subscribed to the topic and have a good enough score. + /// The default is true. + pub fn flood_publish(&self) -> bool { + self.flood_publish + } + + /// If a GRAFT comes before `graft_flood_threshold` has elapsed since the last PRUNE, + /// then there is an extra score penalty applied to the peer through P7. + pub fn graft_flood_threshold(&self) -> Duration { + self.graft_flood_threshold + } + + /// Minimum number of outbound peers in the mesh network before adding more (D_out in the spec). + /// This value must be smaller or equal than `mesh_n / 2` and smaller than `mesh_n_low`. + /// The default is 2. + pub fn mesh_outbound_min(&self) -> usize { + self.mesh_outbound_min + } + + /// Number of heartbeat ticks that specifcy the interval in which opportunistic grafting is + /// applied. Every `opportunistic_graft_ticks` we will attempt to select some high-scoring mesh + /// peers to replace lower-scoring ones, if the median score of our mesh peers falls below a + /// threshold (see https://godoc.org/github.com/libp2p/go-libp2p-pubsub#PeerScoreThresholds). + /// The default is 60. + pub fn opportunistic_graft_ticks(&self) -> u64 { + self.opportunistic_graft_ticks + } + + /// Controls how many times we will allow a peer to request the same message id through IWANT + /// gossip before we start ignoring them. This is designed to prevent peers from spamming us + /// with requests and wasting our resources. The default is 3. + pub fn gossip_retransimission(&self) -> u32 { + self.gossip_retransimission + } + + /// The maximum number of new peers to graft to during opportunistic grafting. The default is 2. + pub fn opportunistic_graft_peers(&self) -> usize { + self.opportunistic_graft_peers + } + + /// The maximum number of messages we will process in a given RPC. If this is unset, there is + /// no limit. The default is None. + pub fn max_messages_per_rpc(&self) -> Option { + self.max_messages_per_rpc + } + + /// The maximum number of messages to include in an IHAVE message. + /// Also controls the maximum number of IHAVE ids we will accept and request with IWANT from a + /// peer within a heartbeat, to protect from IHAVE floods. You should adjust this value from the + /// default if your system is pushing more than 5000 messages in GossipSubHistoryGossip + /// heartbeats; with the defaults this is 1666 messages/s. The default is 5000. + pub fn max_ihave_length(&self) -> usize { + self.max_ihave_length + } + + /// GossipSubMaxIHaveMessages is the maximum number of IHAVE messages to accept from a peer + /// within a heartbeat. + pub fn max_ihave_messages(&self) -> usize { + self.max_ihave_messages + } + + /// Time to wait for a message requested through IWANT following an IHAVE advertisement. + /// If the message is not received within this window, a broken promise is declared and + /// the router may apply behavioural penalties. The default is 3 seconds. + pub fn iwant_followup_time(&self) -> Duration { + self.iwant_followup_time + } + + /// Enable support for flooodsub peers. Default false. + pub fn support_floodsub(&self) -> bool { + self.support_floodsub + } + + /// Published message ids time cache duration. The default is 10 seconds. + pub fn published_message_ids_cache_time(&self) -> Duration { + self.published_message_ids_cache_time + } } impl Default for GossipsubConfig { - fn default() -> GossipsubConfig { - GossipsubConfig { - protocol_id: Cow::Borrowed(b"/meshsub/1.0.0"), - history_length: 5, - history_gossip: 3, - mesh_n: 6, - mesh_n_low: 5, - mesh_n_high: 12, - gossip_lazy: 6, // default to mesh_n - heartbeat_initial_delay: Duration::from_secs(5), - heartbeat_interval: Duration::from_secs(1), - fanout_ttl: Duration::from_secs(60), - max_transmit_size: 2048, - duplicate_cache_time: Duration::from_secs(60), - hash_topics: false, // default compatibility with floodsub - validate_messages: false, - validation_mode: ValidationMode::Strict, - message_id_fn: |message| { - // default message id is: source + sequence number - // NOTE: If either the peer_id or source is not provided, we set to 0; - let mut source_string = if let Some(peer_id) = message.source.as_ref() { - peer_id.to_base58() - } else { - PeerId::from_bytes(&[0, 1, 0]) - .expect("Valid peer id") - .to_base58() - }; - source_string.push_str(&message.sequence_number.unwrap_or_default().to_string()); - MessageId::from(source_string) - }, - } + fn default() -> Self { + // use ConfigBuilder to also validate defaults + GossipsubConfigBuilder::default() + .build() + .expect("Default config parameters should be valid parameters") } } @@ -156,63 +367,96 @@ pub struct GossipsubConfigBuilder { } impl Default for GossipsubConfigBuilder { - fn default() -> GossipsubConfigBuilder { + fn default() -> Self { GossipsubConfigBuilder { - config: GossipsubConfig::default(), + config: GossipsubConfig { + protocol_id_prefix: Cow::Borrowed("meshsub"), + history_length: 5, + history_gossip: 3, + mesh_n: 6, + mesh_n_low: 5, + mesh_n_high: 12, + retain_scores: 4, + gossip_lazy: 6, // default to mesh_n + gossip_factor: 0.25, + heartbeat_initial_delay: Duration::from_secs(5), + heartbeat_interval: Duration::from_secs(1), + fanout_ttl: Duration::from_secs(60), + check_explicit_peers_ticks: 300, + max_transmit_size: 65536, + duplicate_cache_time: Duration::from_secs(60), + validate_messages: false, + validation_mode: ValidationMode::Strict, + message_id_fn: |message| { + // default message id is: source + sequence number + // NOTE: If either the peer_id or source is not provided, we set to 0; + let mut source_string = if let Some(peer_id) = message.source.as_ref() { + peer_id.to_base58() + } else { + PeerId::from_bytes(&vec![0, 1, 0]) + .expect("Valid peer id") + .to_base58() + }; + source_string + .push_str(&message.sequence_number.unwrap_or_default().to_string()); + MessageId::from(source_string) + }, + fast_message_id_fn: None, + allow_self_origin: false, + do_px: false, + prune_peers: 0, // NOTE: Increasing this currently has little effect until Signed records are implemented. + prune_backoff: Duration::from_secs(60), + backoff_slack: 1, + flood_publish: true, + graft_flood_threshold: Duration::from_secs(10), + mesh_outbound_min: 2, + opportunistic_graft_ticks: 60, + opportunistic_graft_peers: 2, + gossip_retransimission: 3, + max_messages_per_rpc: None, + max_ihave_length: 5000, + max_ihave_messages: 10, + iwant_followup_time: Duration::from_secs(3), + support_floodsub: false, + published_message_ids_cache_time: Duration::from_secs(10), + }, } } } -impl GossipsubConfigBuilder { - // set default values - pub fn new() -> GossipsubConfigBuilder { - GossipsubConfigBuilder { - config: GossipsubConfig::default(), - } +impl From for GossipsubConfigBuilder { + fn from(config: GossipsubConfig) -> Self { + GossipsubConfigBuilder { config } } +} +impl GossipsubConfigBuilder { /// The protocol id to negotiate this protocol (default is `/meshsub/1.0.0`). - pub fn protocol_id(&mut self, protocol_id: impl Into>) -> &mut Self { - self.config.protocol_id = protocol_id.into(); + pub fn protocol_id_prefix(&mut self, protocol_id: impl Into>) -> &mut Self { + self.config.protocol_id_prefix = protocol_id.into(); self } /// Number of heartbeats to keep in the `memcache` (default is 5). pub fn history_length(&mut self, history_length: usize) -> &mut Self { - assert!( - history_length >= self.config.history_gossip, - "The history_length must be greater than or equal to the history_gossip length" - ); self.config.history_length = history_length; self } /// Number of past heartbeats to gossip about (default is 3). pub fn history_gossip(&mut self, history_gossip: usize) -> &mut Self { - assert!( - self.config.history_length >= history_gossip, - "The history_length must be greater than or equal to the history_gossip length" - ); self.config.history_gossip = history_gossip; self } /// Target number of peers for the mesh network (D in the spec, default is 6). pub fn mesh_n(&mut self, mesh_n: usize) -> &mut Self { - assert!( - self.config.mesh_n_low <= mesh_n && mesh_n <= self.config.mesh_n_high, - "The following equality doesn't hold mesh_n_low <= mesh_n <= mesh_n_high" - ); self.config.mesh_n = mesh_n; self } /// Minimum number of peers in mesh network before adding more (D_lo in the spec, default is 4). pub fn mesh_n_low(&mut self, mesh_n_low: usize) -> &mut Self { - assert!( - mesh_n_low <= self.config.mesh_n && self.config.mesh_n <= self.config.mesh_n_high, - "The following equality doesn't hold mesh_n_low <= mesh_n <= mesh_n_high" - ); self.config.mesh_n_low = mesh_n_low; self } @@ -220,20 +464,35 @@ impl GossipsubConfigBuilder { /// Maximum number of peers in mesh network before removing some (D_high in the spec, default /// is 12). pub fn mesh_n_high(&mut self, mesh_n_high: usize) -> &mut Self { - assert!( - self.config.mesh_n_low <= self.config.mesh_n && self.config.mesh_n <= mesh_n_high, - "The following equality doesn't hold mesh_n_low <= mesh_n <= mesh_n_high" - ); self.config.mesh_n_high = mesh_n_high; self } - /// Number of peers to emit gossip to during a heartbeat (D_lazy in the spec, default is 6). + /// Affects how peers are selected when pruning a mesh due to over subscription. + /// + /// At least [`Self::retain_scores`] of the retained peers will be high-scoring, while the remainder are + /// chosen randomly (D_score in the spec, default is 4). + pub fn retain_scores(&mut self, retain_scores: usize) -> &mut Self { + self.config.retain_scores = retain_scores; + self + } + + /// Minimum number of peers to emit gossip to during a heartbeat (D_lazy in the spec, + /// default is 6). pub fn gossip_lazy(&mut self, gossip_lazy: usize) -> &mut Self { self.config.gossip_lazy = gossip_lazy; self } + /// Affects how many peers we will emit gossip to at each heartbeat. + /// + /// We will send gossip to `gossip_factor * (total number of non-mesh peers)`, or + /// `gossip_lazy`, whichever is greater. The default is 0.25. + pub fn gossip_factor(&mut self, gossip_factor: f64) -> &mut Self { + self.config.gossip_factor = gossip_factor; + self + } + /// Initial delay in each heartbeat (default is 5 seconds). pub fn heartbeat_initial_delay(&mut self, heartbeat_initial_delay: Duration) -> &mut Self { self.config.heartbeat_initial_delay = heartbeat_initial_delay; @@ -246,6 +505,13 @@ impl GossipsubConfigBuilder { self } + /// The number of heartbeat ticks until we recheck the connection to explicit peers and + /// reconnecting if necessary (default 300). + pub fn check_explicit_peers_ticks(&mut self, check_explicit_peers_ticks: u64) -> &mut Self { + self.config.check_explicit_peers_ticks = check_explicit_peers_ticks; + self + } + /// Time to live for fanout peers (default is 60 seconds). pub fn fanout_ttl(&mut self, fanout_ttl: Duration) -> &mut Self { self.config.fanout_ttl = fanout_ttl; @@ -267,16 +533,10 @@ impl GossipsubConfigBuilder { self } - /// When set, gossipsub topics are hashed instead of being sent as plain strings. - pub fn hash_topics(&mut self) -> &mut Self { - self.config.hash_topics = true; - self - } - /// When set, prevents automatic forwarding of all received messages. This setting /// allows a user to validate the messages before propagating them to their peers. If set, - /// the user must manually call `validate_message()` on the behaviour to forward a message - /// once validated. + /// the user must manually call [`crate::Gossipsub::report_message_validation_result()`] on the + /// behaviour to forward a message once validated. pub fn validate_messages(&mut self) -> &mut Self { self.config.validate_messages = true; self @@ -295,40 +555,256 @@ impl GossipsubConfigBuilder { /// addressing, where this function may be set to `hash(message)`. This would prevent messages /// of the same content from being duplicated. /// - /// The function takes a `GossipsubMessage` as input and outputs a String to be interpreted as - /// the message id. + /// The function takes a [`GossipsubMessage`] as input and outputs a String to be + /// interpreted as the message id. pub fn message_id_fn(&mut self, id_fn: fn(&GossipsubMessage) -> MessageId) -> &mut Self { self.config.message_id_fn = id_fn; self } - /// Constructs a `GossipsubConfig` from the given configuration. - pub fn build(&self) -> GossipsubConfig { - self.config.clone() + /// A user-defined optional function that computes fast ids from raw messages. This can be used + /// to avoid possibly expensive transformations from [`RawGossipsubMessage`] to + /// [`GossipsubMessage`] for duplicates. Two semantically different messages must always + /// have different fast message ids, but it is allowed that two semantically identical messages + /// have different fast message ids as long as the message_id_fn produces the same id for them. + /// + /// The function takes a [`RawGossipsubMessage`] as input and outputs a String to be interpreted + /// as the fast message id. Default is None. + pub fn fast_message_id_fn( + &mut self, + fast_id_fn: fn(&RawGossipsubMessage) -> FastMessageId, + ) -> &mut Self { + self.config.fast_message_id_fn = Some(fast_id_fn); + self + } + + /// Enables Peer eXchange. This should be enabled in bootstrappers and other well + /// connected/trusted nodes. The default is true. + pub fn do_px(&mut self) -> &mut Self { + self.config.do_px = true; + self + } + + /// Controls the number of peers to include in prune Peer eXchange. + /// + /// When we prune a peer that's eligible for PX (has a good score, etc), we will try to + /// send them signed peer records for up to [`Self::prune_peers] other peers that we + /// know of. It is recommended that this value is larger than [`Self::mesh_n_high`] so that the + /// pruned peer can reliably form a full mesh. The default is 16. + pub fn prune_peers(&mut self, prune_peers: usize) -> &mut Self { + self.config.prune_peers = prune_peers; + self + } + + /// Controls the backoff time for pruned peers. This is how long + /// a peer must wait before attempting to graft into our mesh again after being pruned. + /// When pruning a peer, we send them our value of [`Self::prune_backoff`] so they know + /// the minimum time to wait. Peers running older versions may not send a backoff time, + /// so if we receive a prune message without one, we will wait at least [`Self::prune_backoff`] + /// before attempting to re-graft. The default is one minute. + pub fn prune_backoff(&mut self, prune_backoff: Duration) -> &mut Self { + self.config.prune_backoff = prune_backoff; + self + } + + /// Number of heartbeat slots considered as slack for backoffs. This gurantees that we wait + /// at least backoff_slack heartbeats after a backoff is over before we try to graft. This + /// solves problems occuring through high latencies. In particular if + /// `backoff_slack * heartbeat_interval` is longer than any latencies between processing + /// prunes on our side and processing prunes on the receiving side this guarantees that we + /// get not punished for too early grafting. The default is 1. + pub fn backoff_slack(&mut self, backoff_slack: u32) -> &mut Self { + self.config.backoff_slack = backoff_slack; + self + } + + /// Whether to do flood publishing or not. If enabled newly created messages will always be + /// sent to all peers that are subscribed to the topic and have a good enough score. + /// The default is true. + pub fn flood_publish(&mut self, flood_publish: bool) -> &mut Self { + self.config.flood_publish = flood_publish; + self + } + + /// If a GRAFT comes before `graft_flood_threshold` has elapsed since the last PRUNE, + /// then there is an extra score penalty applied to the peer through P7. + pub fn graft_flood_threshold(&mut self, graft_flood_threshold: Duration) -> &mut Self { + self.config.graft_flood_threshold = graft_flood_threshold; + self + } + + /// Minimum number of outbound peers in the mesh network before adding more (D_out in the spec). + /// This value must be smaller or equal than `mesh_n / 2` and smaller than `mesh_n_low`. + /// The default is 2. + pub fn mesh_outbound_min(&mut self, mesh_outbound_min: usize) -> &mut Self { + self.config.mesh_outbound_min = mesh_outbound_min; + self + } + + /// Number of heartbeat ticks that specifcy the interval in which opportunistic grafting is + /// applied. Every `opportunistic_graft_ticks` we will attempt to select some high-scoring mesh + /// peers to replace lower-scoring ones, if the median score of our mesh peers falls below a + /// threshold (see https://godoc.org/github.com/libp2p/go-libp2p-pubsub#PeerScoreThresholds). + /// The default is 60. + pub fn opportunistic_graft_ticks(&mut self, opportunistic_graft_ticks: u64) -> &mut Self { + self.config.opportunistic_graft_ticks = opportunistic_graft_ticks; + self + } + + /// Controls how many times we will allow a peer to request the same message id through IWANT + /// gossip before we start ignoring them. This is designed to prevent peers from spamming us + /// with requests and wasting our resources. + pub fn gossip_retransimission(&mut self, gossip_retransimission: u32) -> &mut Self { + self.config.gossip_retransimission = gossip_retransimission; + self + } + + /// The maximum number of new peers to graft to during opportunistic grafting. The default is 2. + pub fn opportunistic_graft_peers(&mut self, opportunistic_graft_peers: usize) -> &mut Self { + self.config.opportunistic_graft_peers = opportunistic_graft_peers; + self + } + + /// The maximum number of messages we will process in a given RPC. If this is unset, there is + /// no limit. The default is None. + pub fn max_messages_per_rpc(&mut self, max: Option) -> &mut Self { + self.config.max_messages_per_rpc = max; + self + } + + /// The maximum number of messages to include in an IHAVE message. + /// Also controls the maximum number of IHAVE ids we will accept and request with IWANT from a + /// peer within a heartbeat, to protect from IHAVE floods. You should adjust this value from the + /// default if your system is pushing more than 5000 messages in GossipSubHistoryGossip + /// heartbeats; with the defaults this is 1666 messages/s. The default is 5000. + pub fn max_ihave_length(&mut self, max_ihave_length: usize) -> &mut Self { + self.config.max_ihave_length = max_ihave_length; + self + } + + /// GossipSubMaxIHaveMessages is the maximum number of IHAVE messages to accept from a peer + /// within a heartbeat. + pub fn max_ihave_messages(&mut self, max_ihave_messages: usize) -> &mut Self { + self.config.max_ihave_messages = max_ihave_messages; + self + } + + /// By default, gossipsub will reject messages that are sent to us that has the same message + /// source as we have specified locally. Enabling this, allows these messages and prevents + /// penalizing the peer that sent us the message. Default is false. + pub fn allow_self_origin(&mut self, allow_self_origin: bool) -> &mut Self { + self.config.allow_self_origin = allow_self_origin; + self + } + + /// Time to wait for a message requested through IWANT following an IHAVE advertisement. + /// If the message is not received within this window, a broken promise is declared and + /// the router may apply behavioural penalties. The default is 3 seconds. + pub fn iwant_followup_time(&mut self, iwant_followup_time: Duration) -> &mut Self { + self.config.iwant_followup_time = iwant_followup_time; + self + } + + /// Enable support for flooodsub peers. + pub fn support_floodsub(&mut self) -> &mut Self { + self.config.support_floodsub = true; + self + } + + /// Published message ids time cache duration. The default is 10 seconds. + pub fn published_message_ids_cache_time( + &mut self, + published_message_ids_cache_time: Duration, + ) -> &mut Self { + self.config.published_message_ids_cache_time = published_message_ids_cache_time; + self + } + + /// Constructs a [`GossipsubConfig`] from the given configuration and validates the settings. + pub fn build(&self) -> Result { + // check all constraints on config + + if self.config.max_transmit_size < 100 { + return Err("The maximum transmission size must be greater than 100 to permit basic control messages"); + } + + if self.config.history_length < self.config.history_gossip { + return Err( + "The history_length must be greater than or equal to the history_gossip \ + length", + ); + } + + if !(self.config.mesh_outbound_min < self.config.mesh_n_low + && self.config.mesh_n_low <= self.config.mesh_n + && self.config.mesh_n <= self.config.mesh_n_high) + { + return Err("The following inequality doesn't hold \ + mesh_outbound_min < mesh_n_low <= mesh_n <= mesh_n_high"); + } + + if self.config.mesh_outbound_min * 2 > self.config.mesh_n { + return Err( + "The following inequality doesn't hold mesh_outbound_min <= self.config.mesh_n / 2", + ); + } + Ok(self.config.clone()) } } impl std::fmt::Debug for GossipsubConfig { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { let mut builder = f.debug_struct("GossipsubConfig"); - let _ = if let Ok(text) = std::str::from_utf8(&self.protocol_id) { - builder.field("protocol_id", &text) - } else { - builder.field("protocol_id", &hex_fmt::HexFmt(&self.protocol_id)) - }; + let _ = builder.field("protocol_id_prefix", &self.protocol_id_prefix); let _ = builder.field("history_length", &self.history_length); let _ = builder.field("history_gossip", &self.history_gossip); let _ = builder.field("mesh_n", &self.mesh_n); let _ = builder.field("mesh_n_low", &self.mesh_n_low); let _ = builder.field("mesh_n_high", &self.mesh_n_high); + let _ = builder.field("retain_scores", &self.retain_scores); let _ = builder.field("gossip_lazy", &self.gossip_lazy); + let _ = builder.field("gossip_factor", &self.gossip_factor); let _ = builder.field("heartbeat_initial_delay", &self.heartbeat_initial_delay); let _ = builder.field("heartbeat_interval", &self.heartbeat_interval); let _ = builder.field("fanout_ttl", &self.fanout_ttl); let _ = builder.field("max_transmit_size", &self.max_transmit_size); let _ = builder.field("duplicate_cache_time", &self.duplicate_cache_time); - let _ = builder.field("hash_topics", &self.hash_topics); let _ = builder.field("validate_messages", &self.validate_messages); + let _ = builder.field("validation_mode", &self.validation_mode); + let _ = builder.field("allow_self_origin", &self.allow_self_origin); + let _ = builder.field("do_px", &self.do_px); + let _ = builder.field("prune_peers", &self.prune_peers); + let _ = builder.field("prune_backoff", &self.prune_backoff); + let _ = builder.field("backoff_slack", &self.backoff_slack); + let _ = builder.field("flood_publish", &self.flood_publish); + let _ = builder.field("graft_flood_threshold", &self.graft_flood_threshold); + let _ = builder.field("mesh_outbound_min", &self.mesh_outbound_min); + let _ = builder.field("opportunistic_graft_ticks", &self.opportunistic_graft_ticks); + let _ = builder.field("opportunistic_graft_peers", &self.opportunistic_graft_peers); + let _ = builder.field("max_messages_per_rpc", &self.max_messages_per_rpc); + let _ = builder.field("max_ihave_length", &self.max_ihave_length); + let _ = builder.field("max_ihave_messages", &self.max_ihave_messages); + let _ = builder.field("iwant_followup_time", &self.iwant_followup_time); + let _ = builder.field("support_floodsub", &self.support_floodsub); + let _ = builder.field( + "published_message_ids_cache_time", + &self.published_message_ids_cache_time, + ); builder.finish() } } + +#[cfg(test)] +mod test { + use super::*; + + #[test] + fn create_thing() { + let builder: GossipsubConfig = GossipsubConfigBuilder::default() + .protocol_id_prefix("purple") + .build() + .unwrap(); + + dbg!(builder); + } +} diff --git a/protocols/gossipsub/src/error.rs b/protocols/gossipsub/src/error.rs index 6f774607117..7ea5ff431ba 100644 --- a/protocols/gossipsub/src/error.rs +++ b/protocols/gossipsub/src/error.rs @@ -21,6 +21,8 @@ //! Error types that can result from gossipsub. use libp2p_core::identity::error::SigningError; +use libp2p_core::upgrade::ProtocolError; +use std::fmt; /// Error associated with publishing a gossipsub message. #[derive(Debug)] @@ -31,6 +33,20 @@ pub enum PublishError { SigningError(SigningError), /// There were no peers to send this message to. InsufficientPeers, + /// The overall message was too large. This could be due to excessive topics or an excessive + /// message size. + MessageTooLarge, + /// The compression algorithm failed. + TransformFailed(std::io::Error), +} + +/// Error associated with subscribing to a topic. +#[derive(Debug)] +pub enum SubscriptionError { + /// Couldn't publish our subscription + PublishError(PublishError), + /// We are not allowed to subscribe to this topic by the subscription filter + NotAllowed, } impl From for PublishError { @@ -38,3 +54,70 @@ impl From for PublishError { PublishError::SigningError(error) } } + +/// Errors that can occur in the protocols handler. +#[derive(Debug)] +pub enum GossipsubHandlerError { + /// The maximum number of inbound substreams created has been exceeded. + MaxInboundSubstreams, + /// The maximum number of outbound substreams created has been exceeded. + MaxOutboundSubstreams, + /// The message exceeds the maximum transmission size. + MaxTransmissionSize, + /// Protocol negotiation timeout. + NegotiationTimeout, + /// Protocol negotiation failed. + NegotiationProtocolError(ProtocolError), + /// IO error. + Io(std::io::Error), +} + +#[derive(Debug, Clone, Copy)] +pub enum ValidationError { + /// The message has an invalid signature, + InvalidSignature, + /// The sequence number was empty, expected a value. + EmptySequenceNumber, + /// The sequence number was the incorrect size + InvalidSequenceNumber, + /// The PeerId was invalid + InvalidPeerId, + /// Signature existed when validation has been sent to + /// [`crate::behaviour::MessageAuthenticity::Anonymous`]. + SignaturePresent, + /// Sequence number existed when validation has been sent to + /// [`crate::behaviour::MessageAuthenticity::Anonymous`]. + SequenceNumberPresent, + /// Message source existed when validation has been sent to + /// [`crate::behaviour::MessageAuthenticity::Anonymous`]. + MessageSourcePresent, + /// The data transformation failed. + TransformFailed, +} + +impl From for GossipsubHandlerError { + fn from(error: std::io::Error) -> GossipsubHandlerError { + GossipsubHandlerError::Io(error) + } +} + +impl From for PublishError { + fn from(error: std::io::Error) -> PublishError { + PublishError::TransformFailed(error) + } +} + +impl fmt::Display for GossipsubHandlerError { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{:?}", self) + } +} + +impl std::error::Error for GossipsubHandlerError { + fn source(&self) -> Option<&(dyn std::error::Error + 'static)> { + match self { + GossipsubHandlerError::Io(io) => Some(io), + _ => None, + } + } +} diff --git a/protocols/gossipsub/src/gossip_promises.rs b/protocols/gossipsub/src/gossip_promises.rs new file mode 100644 index 00000000000..4dfadbe54ec --- /dev/null +++ b/protocols/gossipsub/src/gossip_promises.rs @@ -0,0 +1,101 @@ +// Copyright 2020 Sigma Prime Pty Ltd. +// +// Permission is hereby granted, free of charge, to any person obtaining a +// copy of this software and associated documentation files (the "Software"), +// to deal in the Software without restriction, including without limitation +// the rights to use, copy, modify, merge, publish, distribute, sublicense, +// and/or sell copies of the Software, and to permit persons to whom the +// Software is furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS +// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +// DEALINGS IN THE SOFTWARE. + +use crate::error::ValidationError; +use crate::peer_score::RejectReason; +use crate::MessageId; +use libp2p_core::PeerId; +use log::debug; +use rand::seq::SliceRandom; +use rand::thread_rng; +use std::collections::HashMap; +use wasm_timer::Instant; + +/// Tracks recently sent `IWANT` messages and checks if peers respond to them +/// for each `IWANT` message we track one random requested message id. +#[derive(Default)] +pub(crate) struct GossipPromises { + /// Stores for each tracked message id and peer the instant when this promise expires. + /// + /// If the peer didn't respond until then we consider the promise as broken and penalize the + /// peer. + promises: HashMap>, +} + +impl GossipPromises { + /// Track a promise to deliver a message from a list of [`MessageId`]s we are requesting. + pub fn add_promise(&mut self, peer: PeerId, messages: &[MessageId], expires: Instant) { + // Randomly select a message id + let mut rng = thread_rng(); + if let Some(message_id) = messages.choose(&mut rng) { + // If a promise for this message id and peer already exists we don't update expires! + self.promises + .entry(message_id.clone()) + .or_insert_with(HashMap::new) + .entry(peer) + .or_insert(expires); + } + } + + pub fn message_delivered(&mut self, message_id: &MessageId) { + // Someone delivered a message, we can stop tracking all promises for it. + self.promises.remove(message_id); + } + + pub fn reject_message(&mut self, message_id: &MessageId, reason: &RejectReason) { + // A message got rejected, so we can stop tracking promises and let the score penalty apply + // from invalid message delivery. + // We do take exception and apply promise penalty regardless in the following cases, where + // the peer delivered an obviously invalid message. + match reason { + RejectReason::ValidationError(ValidationError::InvalidSignature) => (), + RejectReason::SelfOrigin => (), + _ => { + self.promises.remove(message_id); + } + }; + } + + /// Returns the number of broken promises for each peer who didn't follow up on an IWANT + /// request. + /// This should be called not too often relative to the expire times, since it iterates over + /// the whole stored data. + pub fn get_broken_promises(&mut self) -> HashMap { + let now = Instant::now(); + let mut result = HashMap::new(); + self.promises.retain(|msg, peers| { + peers.retain(|peer_id, expires| { + if *expires < now { + let count = result.entry(peer_id.clone()).or_insert(0); + *count += 1; + debug!( + "The peer {} broke the promise to deliver message {} in time!", + peer_id, msg + ); + false + } else { + true + } + }); + !peers.is_empty() + }); + result + } +} diff --git a/protocols/gossipsub/src/handler.rs b/protocols/gossipsub/src/handler.rs index 125b65a8412..8026143b2ee 100644 --- a/protocols/gossipsub/src/handler.rs +++ b/protocols/gossipsub/src/handler.rs @@ -18,24 +18,56 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::behaviour::GossipsubRpc; use crate::config::ValidationMode; +use crate::error::{GossipsubHandlerError, ValidationError}; use crate::protocol::{GossipsubCodec, ProtocolConfig}; +use crate::types::{GossipsubRpc, PeerKind, RawGossipsubMessage}; use futures::prelude::*; +use futures::StreamExt; use futures_codec::Framed; -use libp2p_core::upgrade::{InboundUpgrade, OutboundUpgrade}; +use libp2p_core::upgrade::{InboundUpgrade, NegotiationError, OutboundUpgrade, UpgradeError}; use libp2p_swarm::protocols_handler::{ KeepAlive, ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr, SubstreamProtocol, }; use libp2p_swarm::NegotiatedSubstream; -use log::{debug, error, trace, warn}; +use log::{error, trace, warn}; use smallvec::SmallVec; use std::{ - borrow::Cow, + collections::VecDeque, io, pin::Pin, task::{Context, Poll}, + time::Duration, }; +use wasm_timer::Instant; + +/// The initial time (in seconds) we set the keep alive for protocol negotiations to occur. +const INITIAL_KEEP_ALIVE: u64 = 30; + +/// The event emitted by the Handler. This informs the behaviour of various events created +/// by the handler. +#[derive(Debug)] +pub enum HandlerEvent { + /// A GossipsubRPC message has been received. This also contains a list of invalid messages (if + /// any) that were received. + Message { + /// The GossipsubRPC message excluding any invalid messages. + rpc: GossipsubRpc, + /// Any invalid messages that were received in the RPC, along with the associated + /// validation error. + invalid_messages: Vec<(RawGossipsubMessage, ValidationError)>, + }, + /// An inbound or outbound substream has been established with the peer and this informs over + /// which protocol. This message only occurs once per connection. + PeerKind(PeerKind), +} + +/// The maximum number of substreams we accept or create before disconnecting from the peer. +/// +/// Gossipsub is supposed to have a single long-lived inbound and outbound substream. On failure we +/// attempt to recreate these. This imposes an upper bound of new substreams before we consider the +/// connection faulty and disconnect. This also prevents against potential substream creation loops. +const MAX_SUBSTREAM_CREATION: usize = 5; /// Protocol Handler that manages a single long-lived substream with a peer. pub struct GossipsubHandler { @@ -49,12 +81,36 @@ pub struct GossipsubHandler { inbound_substream: Option, /// Queue of values that we want to send to the remote. - send_queue: SmallVec<[GossipsubRpc; 16]>, + send_queue: SmallVec<[crate::rpc_proto::Rpc; 16]>, /// Flag indicating that an outbound substream is being established to prevent duplicate /// requests. outbound_substream_establishing: bool, + /// The number of outbound substreams we have created. + outbound_substreams_created: usize, + + /// The number of inbound substreams that have been created by the peer. + inbound_substreams_created: usize, + + /// The type of peer this handler is associated to. + peer_kind: Option, + + /// Keeps track on whether we have sent the peer kind to the behaviour. + // + // NOTE: Use this flag rather than checking the substream count each poll. + peer_kind_sent: bool, + + /// If the peer doesn't support the gossipsub protocol we do not immediately disconnect. + /// Rather, we disable the handler and prevent any incoming or outgoing substreams from being + /// established. + /// + /// This value is set to true to indicate the peer doesn't support gossipsub. + protocol_unsupported: bool, + + /// Collection of errors from attempting an upgrade. + upgrade_errors: VecDeque>, + /// Flag determining whether to maintain the connection to the peer. keep_alive: KeepAlive, } @@ -74,7 +130,10 @@ enum OutboundSubstreamState { /// Waiting for the user to send a message. The idle state for an outbound substream. WaitingOutput(Framed), /// Waiting to send a message to the remote. - PendingSend(Framed, GossipsubRpc), + PendingSend( + Framed, + crate::rpc_proto::Rpc, + ), /// Waiting to flush the substream so that the data arrives to the remote. PendingFlush(Framed), /// The substream is being closed. Used by either substream. @@ -84,35 +143,46 @@ enum OutboundSubstreamState { } impl GossipsubHandler { - /// Builds a new `GossipsubHandler`. + /// Builds a new [`GossipsubHandler`]. pub fn new( - protocol_id: impl Into>, + protocol_id_prefix: std::borrow::Cow<'static, str>, max_transmit_size: usize, validation_mode: ValidationMode, + support_floodsub: bool, ) -> Self { GossipsubHandler { - listen_protocol: SubstreamProtocol::new(ProtocolConfig::new( - protocol_id, - max_transmit_size, - validation_mode, - ), ()), + listen_protocol: SubstreamProtocol::new( + ProtocolConfig::new( + protocol_id_prefix, + max_transmit_size, + validation_mode, + support_floodsub, + ), + (), + ), inbound_substream: None, outbound_substream: None, outbound_substream_establishing: false, + outbound_substreams_created: 0, + inbound_substreams_created: 0, send_queue: SmallVec::new(), - keep_alive: KeepAlive::Yes, + peer_kind: None, + peer_kind_sent: false, + protocol_unsupported: false, + upgrade_errors: VecDeque::new(), + keep_alive: KeepAlive::Until(Instant::now() + Duration::from_secs(INITIAL_KEEP_ALIVE)), } } } impl ProtocolsHandler for GossipsubHandler { - type InEvent = GossipsubRpc; - type OutEvent = GossipsubRpc; - type Error = io::Error; + type InEvent = crate::rpc_proto::Rpc; + type OutEvent = HandlerEvent; + type Error = GossipsubHandlerError; + type InboundOpenInfo = (); type InboundProtocol = ProtocolConfig; + type OutboundOpenInfo = Self::InEvent; type OutboundProtocol = ProtocolConfig; - type OutboundOpenInfo = GossipsubRpc; - type InboundOpenInfo = (); fn listen_protocol(&self) -> SubstreamProtocol { self.listen_protocol.clone() @@ -120,9 +190,21 @@ impl ProtocolsHandler for GossipsubHandler { fn inject_fully_negotiated_inbound( &mut self, - substream: >::Output, - _info: Self::InboundOpenInfo + (substream, peer_kind): >::Output, + _info: Self::InboundOpenInfo, ) { + // If the peer doesn't support the protocol, reject all substreams + if self.protocol_unsupported { + return; + } + + self.inbound_substreams_created += 1; + + // update the known kind of peer + if self.peer_kind.is_none() { + self.peer_kind = Some(peer_kind); + } + // new inbound substream. Replace the current one, if it exists. trace!("New inbound substream request"); self.inbound_substream = Some(InboundSubstreamState::WaitingInput(substream)); @@ -130,10 +212,22 @@ impl ProtocolsHandler for GossipsubHandler { fn inject_fully_negotiated_outbound( &mut self, - substream: >::Output, + (substream, peer_kind): >::Output, message: Self::OutboundOpenInfo, ) { + // If the peer doesn't support the protocol, reject all substreams + if self.protocol_unsupported { + return; + } + self.outbound_substream_establishing = false; + self.outbound_substreams_created += 1; + + // update the known kind of peer + if self.peer_kind.is_none() { + self.peer_kind = Some(peer_kind); + } + // Should never establish a new outbound substream if one already exists. // If this happens, an outbound message is not sent. if self.outbound_substream.is_some() { @@ -145,21 +239,22 @@ impl ProtocolsHandler for GossipsubHandler { } } - fn inject_event(&mut self, message: GossipsubRpc) { - self.send_queue.push(message); + fn inject_event(&mut self, message: crate::rpc_proto::Rpc) { + if !self.protocol_unsupported { + self.send_queue.push(message); + } } fn inject_dial_upgrade_error( &mut self, _: Self::OutboundOpenInfo, - _: ProtocolsHandlerUpgrErr< + e: ProtocolsHandlerUpgrErr< >::Error, >, ) { self.outbound_substream_establishing = false; - // Ignore upgrade errors for now. - // If a peer doesn't support this protocol, this will just ignore them, but not disconnect - // them. + warn!("Dial upgrade error {:?}", e); + self.upgrade_errors.push_back(e); } fn connection_keep_alive(&self) -> KeepAlive { @@ -177,16 +272,77 @@ impl ProtocolsHandler for GossipsubHandler { Self::Error, >, > { + // Handle any upgrade errors + if let Some(error) = self.upgrade_errors.pop_front() { + let reported_error = match error { + // Timeout errors get mapped to NegotiationTimeout and we close the connection. + ProtocolsHandlerUpgrErr::Timeout | ProtocolsHandlerUpgrErr::Timer => { + Some(GossipsubHandlerError::NegotiationTimeout) + } + // There was an error post negotiation, close the connection. + ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Apply(e)) => Some(e), + ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Select(negotiation_error)) => { + match negotiation_error { + NegotiationError::Failed => { + // The protocol is not supported + self.protocol_unsupported = true; + if !self.peer_kind_sent { + self.peer_kind_sent = true; + // clear all substreams so the keep alive returns false + self.inbound_substream = None; + self.outbound_substream = None; + self.keep_alive = KeepAlive::No; + return Poll::Ready(ProtocolsHandlerEvent::Custom( + HandlerEvent::PeerKind(PeerKind::NotSupported), + )); + } else { + None + } + } + NegotiationError::ProtocolError(e) => { + Some(GossipsubHandlerError::NegotiationProtocolError(e)) + } + } + } + }; + + // If there was a fatal error, close the connection. + if let Some(error) = reported_error { + return Poll::Ready(ProtocolsHandlerEvent::Close(error)); + } + } + + if !self.peer_kind_sent { + if let Some(peer_kind) = self.peer_kind.as_ref() { + self.peer_kind_sent = true; + return Poll::Ready(ProtocolsHandlerEvent::Custom(HandlerEvent::PeerKind( + peer_kind.clone(), + ))); + } + } + + if self.inbound_substreams_created > MAX_SUBSTREAM_CREATION { + // Too many inbound substreams have been created, end the connection. + return Poll::Ready(ProtocolsHandlerEvent::Close( + GossipsubHandlerError::MaxInboundSubstreams, + )); + } + // determine if we need to create the stream if !self.send_queue.is_empty() && self.outbound_substream.is_none() && !self.outbound_substream_establishing { + if self.outbound_substreams_created >= MAX_SUBSTREAM_CREATION { + return Poll::Ready(ProtocolsHandlerEvent::Close( + GossipsubHandlerError::MaxOutboundSubstreams, + )); + } let message = self.send_queue.remove(0); self.send_queue.shrink_to_fit(); self.outbound_substream_establishing = true; return Poll::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest { - protocol: self.listen_protocol.clone().map_info(|()| message) + protocol: self.listen_protocol.clone().map_info(|()| message), }); } @@ -203,15 +359,15 @@ impl ProtocolsHandler for GossipsubHandler { Some(InboundSubstreamState::WaitingInput(substream)); return Poll::Ready(ProtocolsHandlerEvent::Custom(message)); } - Poll::Ready(Some(Err(e))) => { - match e.kind() { - std::io::ErrorKind::InvalidData => { - // Invalid message, ignore it and reset to waiting - warn!("Invalid message received. Error: {}", e); + Poll::Ready(Some(Err(error))) => { + match error { + GossipsubHandlerError::MaxTransmissionSize => { + warn!("Message exceeded the maximum transmission size"); self.inbound_substream = Some(InboundSubstreamState::WaitingInput(substream)); } _ => { + warn!("Inbound stream error: {}", error); // More serious errors, close this side of the stream. If the // peer is still around, they will re-establish their // connection @@ -222,6 +378,7 @@ impl ProtocolsHandler for GossipsubHandler { } // peer closed the stream Poll::Ready(None) => { + warn!("Peer closed their outbound stream"); self.inbound_substream = Some(InboundSubstreamState::Closing(substream)); } @@ -239,7 +396,7 @@ impl ProtocolsHandler for GossipsubHandler { // Don't close the connection but just drop the inbound substream. // In case the remote has more to send, they will open up a new // substream. - debug!("Inbound substream error while closing: {:?}", e); + warn!("Inbound substream error while closing: {:?}", e); } self.inbound_substream = None; @@ -265,6 +422,7 @@ impl ProtocolsHandler for GossipsubHandler { } } + // process outbound stream loop { match std::mem::replace( &mut self.outbound_substream, @@ -291,19 +449,19 @@ impl ProtocolsHandler for GossipsubHandler { self.outbound_substream = Some(OutboundSubstreamState::PendingFlush(substream)) } + Err(GossipsubHandlerError::MaxTransmissionSize) => { + error!("Message exceeded the maximum transmission size and was not sent."); + self.outbound_substream = + Some(OutboundSubstreamState::WaitingOutput(substream)); + } Err(e) => { - if let io::ErrorKind::PermissionDenied = e.kind() { - error!("Message over the maximum transmission limit was not sent."); - self.outbound_substream = - Some(OutboundSubstreamState::WaitingOutput(substream)); - } else { - return Poll::Ready(ProtocolsHandlerEvent::Close(e)); - } + error!("Error sending message: {}", e); + return Poll::Ready(ProtocolsHandlerEvent::Close(e)); } } } Poll::Ready(Err(e)) => { - debug!("Outbound substream error while sending output: {:?}", e); + error!("Outbound substream error while sending output: {:?}", e); return Poll::Ready(ProtocolsHandlerEvent::Close(e)); } Poll::Pending => { @@ -338,11 +496,14 @@ impl ProtocolsHandler for GossipsubHandler { break; } Poll::Ready(Err(e)) => { - debug!("Outbound substream error while closing: {:?}", e); - return Poll::Ready(ProtocolsHandlerEvent::Close(io::Error::new( - io::ErrorKind::BrokenPipe, - "Failed to close outbound substream", - ))); + warn!("Outbound substream error while closing: {:?}", e); + return Poll::Ready(ProtocolsHandlerEvent::Close( + io::Error::new( + io::ErrorKind::BrokenPipe, + "Failed to close outbound substream", + ) + .into(), + )); } Poll::Pending => { self.outbound_substream = diff --git a/protocols/gossipsub/src/lib.rs b/protocols/gossipsub/src/lib.rs index b20412a6d25..73104710deb 100644 --- a/protocols/gossipsub/src/lib.rs +++ b/protocols/gossipsub/src/lib.rs @@ -41,13 +41,13 @@ //! implementations, due to undefined elements in the current specification. //! //! - **Topics** - In gossipsub, topics configurable by the `hash_topics` configuration parameter. -//! Topics are of type `TopicHash`. The current go implementation uses raw utf-8 strings, and this +//! Topics are of type [`TopicHash`]. The current go implementation uses raw utf-8 strings, and this //! is default configuration in rust-libp2p. Topics can be hashed (SHA256 hashed then base64 //! encoded) by setting the `hash_topics` configuration parameter to true. //! //! - **Sequence Numbers** - A message on the gossipsub network is identified by the source -//! `PeerId` and a nonce (sequence number) of the message. The sequence numbers in this -//! implementation are sent as raw bytes across the wire. They are 64-bit big-endian unsigned +//! [`libp2p_core::PeerId`] and a nonce (sequence number) of the message. The sequence numbers in +//! this implementation are sent as raw bytes across the wire. They are 64-bit big-endian unsigned //! integers. They are chosen at random in this implementation of gossipsub, but are sequential in //! the current go implementation. //! @@ -58,39 +58,17 @@ //! The [`GossipsubConfig`] struct specifies various network performance/tuning configuration //! parameters. Specifically it specifies: //! -//! [`GossipsubConfig`]: struct.GossipsubConfig.html -//! -//! - `protocol_id` - The protocol id that this implementation will accept connections on. -//! - `history_length` - The number of heartbeats which past messages are kept in cache (default: 5). -//! - `history_gossip` - The number of past heartbeats that the node will send gossip metadata -//! about (default: 3). -//! - `mesh_n` - The target number of peers store in the local mesh network. -//! (default: 6). -//! - `mesh_n_low` - The minimum number of peers in the local mesh network before. -//! trying to add more peers to the mesh from the connected peer pool (default: 4). -//! - `mesh_n_high` - The maximum number of peers in the local mesh network before removing peers to -//! reach `mesh_n` peers (default: 12). -//! - `gossip_lazy` - The number of peers that the local node will gossip to during a heartbeat (default: `mesh_n` = 6). -//! - `heartbeat_initial_delay - The initial time delay before starting the first heartbeat (default: 5 seconds). -//! - `heartbeat_interval` - The time between each heartbeat (default: 1 second). -//! - `fanout_ttl` - The fanout time to live time period. The timeout required before removing peers from the fanout -//! for a given topic (default: 1 minute). -//! - `max_transmit_size` - This sets the maximum transmission size for total gossipsub messages on the network. -//! - `hash_topics` - Whether to hash the topics using base64(SHA256(topic)) or to leave as plain utf-8 strings. -//! - `manual_propagation` - Whether gossipsub should immediately forward received messages on the -//! network. For applications requiring message validation, this should be set to false, then the -//! application should call `propagate_message(message_id, propagation_source)` once validated, to -//! propagate the message to peers. -//! -//! This struct implements the `Default` trait and can be initialised via -//! `GossipsubConfig::default()`. +//! [`GossipsubConfig`]: struct.Config.html +//! +//! This struct implements the [`Default`] trait and can be initialised via +//! [`GossipsubConfig::default()`]. //! //! //! ## Gossipsub //! -//! The [`Gossipsub`] struct implements the `NetworkBehaviour` trait allowing it to act as the -//! routing behaviour in a `Swarm`. This struct requires an instance of `PeerId` and -//! [`GossipsubConfig`]. +//! The [`Gossipsub`] struct implements the [`libp2p_swarm::NetworkBehaviour`] trait allowing it to +//! act as the routing behaviour in a [`libp2p_swarm::Swarm`]. This struct requires an instance of +//! [`libp2p_core::PeerId`] and [`GossipsubConfig`]. //! //! [`Gossipsub`]: struct.Gossipsub.html @@ -98,57 +76,86 @@ //! //! An example of initialising a gossipsub compatible swarm: //! -//! ```ignore -//! #extern crate libp2p; -//! #extern crate futures; -//! #extern crate tokio; -//! #use libp2p::gossipsub::GossipsubEvent; -//! #use libp2p::{identity, gossipsub, -//! # tokio_codec::{FramedRead, LinesCodec}, -//! #}; -//! let local_key = identity::Keypair::generate_ed25519(); -//! let local_pub_key = local_key.public(); -//! -//! // Set up an encrypted TCP Transport over the Mplex and Yamux protocols -//! let transport = libp2p::build_development_transport(local_key); -//! -//! // Create a Floodsub/Gossipsub topic -//! let topic = libp2p::floodsub::TopicBuilder::new("example").build(); +//! ``` +//! use libp2p_gossipsub::GossipsubEvent; +//! use libp2p_core::{identity::Keypair,transport::{Transport, MemoryTransport}, Multiaddr}; +//! use libp2p_gossipsub::MessageAuthenticity; +//! let local_key = Keypair::generate_ed25519(); +//! let local_peer_id = libp2p_core::PeerId::from(local_key.public()); +//! +//! // Set up an encrypted TCP Transport over the Mplex +//! // This is test transport (memory). +//! let noise_keys = libp2p_noise::Keypair::::new().into_authentic(&local_key).unwrap(); +//! let transport = MemoryTransport::default() +//! .upgrade(libp2p_core::upgrade::Version::V1) +//! .authenticate(libp2p_noise::NoiseConfig::xx(noise_keys).into_authenticated()) +//! .multiplex(libp2p_mplex::MplexConfig::new()) +//! .boxed(); +//! +//! // Create a Gossipsub topic +//! let topic = libp2p_gossipsub::IdentTopic::new("example"); +//! +//! // Set the message authenticity - How we expect to publish messages +//! // Here we expect the publisher to sign the message with their key. +//! let message_authenticity = MessageAuthenticity::Signed(local_key); //! //! // Create a Swarm to manage peers and events //! let mut swarm = { //! // set default parameters for gossipsub -//! let gossipsub_config = gossipsub::GossipsubConfig::default(); +//! let gossipsub_config = libp2p_gossipsub::GossipsubConfig::default(); //! // build a gossipsub network behaviour -//! let mut gossipsub = -//! gossipsub::Gossipsub::new(local_pub_key.clone().into_peer_id(), gossipsub_config); -//! gossipsub.subscribe(topic.clone()); -//! libp2p::Swarm::new( +//! let mut gossipsub: libp2p_gossipsub::Gossipsub = +//! libp2p_gossipsub::Gossipsub::new(message_authenticity, gossipsub_config).unwrap(); +//! // subscribe to the topic +//! gossipsub.subscribe(&topic); +//! // create the swarm +//! libp2p_swarm::Swarm::new( //! transport, //! gossipsub, -//! libp2p::core::topology::MemoryTopology::empty(local_pub_key), +//! local_peer_id, //! ) //! }; //! -//! // Listen on all interfaces and whatever port the OS assigns -//! let addr = libp2p::Swarm::listen_on(&mut swarm, "/ip4/0.0.0.0/tcp/0".parse().unwrap()).unwrap(); +//! // Listen on a memory transport. +//! let memory: Multiaddr = libp2p_core::multiaddr::Protocol::Memory(10).into(); +//! let addr = libp2p_swarm::Swarm::listen_on(&mut swarm, memory).unwrap(); //! println!("Listening on {:?}", addr); //! ``` pub mod error; pub mod protocol; +mod backoff; mod behaviour; mod config; +mod gossip_promises; mod handler; mod mcache; +mod peer_score; +pub mod subscription_filter; +pub mod time_cache; mod topic; +mod transform; +mod types; + +#[cfg(test)] +#[macro_use] +extern crate derive_builder; + +mod rpc_proto; -mod rpc_proto { - include!(concat!(env!("OUT_DIR"), "/gossipsub.pb.rs")); -} +pub use self::behaviour::{Gossipsub, GossipsubEvent, MessageAuthenticity}; +pub use self::transform::{DataTransform, IdentityTransform}; -pub use self::behaviour::{Gossipsub, GossipsubEvent, GossipsubRpc, MessageAuthenticity}; pub use self::config::{GossipsubConfig, GossipsubConfigBuilder, ValidationMode}; -pub use self::protocol::{GossipsubMessage, MessageId}; -pub use self::topic::{Topic, TopicHash}; +pub use self::peer_score::{ + score_parameter_decay, score_parameter_decay_with_base, PeerScoreParams, PeerScoreThresholds, + TopicScoreParams, +}; +pub use self::topic::{Hasher, Topic, TopicHash}; +pub use self::types::{ + FastMessageId, GossipsubMessage, GossipsubRpc, MessageAcceptance, MessageId, + RawGossipsubMessage, +}; +pub type IdentTopic = Topic; +pub type Sha256Topic = Topic; diff --git a/protocols/gossipsub/src/mcache.rs b/protocols/gossipsub/src/mcache.rs index 05a49551aed..6e8aab91e4b 100644 --- a/protocols/gossipsub/src/mcache.rs +++ b/protocols/gossipsub/src/mcache.rs @@ -18,26 +18,31 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. - - -use crate::protocol::{GossipsubMessage, MessageId}; use crate::topic::TopicHash; +use crate::types::{MessageId, RawGossipsubMessage}; +use libp2p_core::PeerId; +use log::debug; +use std::fmt::Debug; use std::{collections::HashMap, fmt}; /// CacheEntry stored in the history. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct CacheEntry { mid: MessageId, - topics: Vec, + topic: TopicHash, } /// MessageCache struct holding history of messages. #[derive(Clone)] pub struct MessageCache { - msgs: HashMap, + msgs: HashMap, + /// For every message and peer the number of times this peer asked for the message + iwant_counts: HashMap>, history: Vec>, + /// The number of indices in the cache history used for gossipping. That means that a message + /// won't get gossipped anymore when shift got called `gossip` many times after inserting the + /// message in the cache. gossip: usize, - msg_id: fn(&GossipsubMessage) -> MessageId, } impl fmt::Debug for MessageCache { @@ -52,30 +57,30 @@ impl fmt::Debug for MessageCache { /// Implementation of the MessageCache. impl MessageCache { - pub fn new( - gossip: usize, - history_capacity: usize, - msg_id: fn(&GossipsubMessage) -> MessageId, - ) -> MessageCache { + pub fn new(gossip: usize, history_capacity: usize) -> Self { MessageCache { gossip, msgs: HashMap::default(), + iwant_counts: HashMap::default(), history: vec![Vec::new(); history_capacity], - msg_id, } } /// Put a message into the memory cache. /// /// Returns the message if it already exists. - pub fn put(&mut self, msg: GossipsubMessage) -> Option { - let message_id = (self.msg_id)(&msg); + pub fn put( + &mut self, + message_id: &MessageId, + msg: RawGossipsubMessage, + ) -> Option { + debug!("Put message {:?} in mcache", message_id); let cache_entry = CacheEntry { mid: message_id.clone(), - topics: msg.topics.clone(), + topic: msg.topic.clone(), }; - let seen_message = self.msgs.insert(message_id, msg); + let seen_message = self.msgs.insert(message_id.clone(), msg); if seen_message.is_none() { // Don't add duplicate entries to the cache. self.history[0].push(cache_entry); @@ -84,20 +89,46 @@ impl MessageCache { } /// Get a message with `message_id` - pub fn get(&self, message_id: &MessageId) -> Option<&GossipsubMessage> { + #[cfg(test)] + pub fn get(&self, message_id: &MessageId) -> Option<&RawGossipsubMessage> { self.msgs.get(message_id) } - /// Gets and validates a message with `message_id`. - pub fn validate(&mut self, message_id: &MessageId) -> Option<&GossipsubMessage> { + /// Increases the iwant count for the given message by one and returns the message together + /// with the iwant if the message exists. + pub fn get_with_iwant_counts( + &mut self, + message_id: &MessageId, + peer: &PeerId, + ) -> Option<(&RawGossipsubMessage, u32)> { + let iwant_counts = &mut self.iwant_counts; + self.msgs.get(message_id).and_then(|message| { + if !message.validated { + None + } else { + Some((message, { + let count = iwant_counts + .entry(message_id.clone()) + .or_default() + .entry(peer.clone()) + .or_default(); + *count += 1; + *count + })) + } + }) + } + + /// Gets a message with [`MessageId`] and tags it as validated. + pub fn validate(&mut self, message_id: &MessageId) -> Option<&RawGossipsubMessage> { self.msgs.get_mut(message_id).map(|message| { message.validated = true; &*message }) } - /// Get a list of GossipIds for a given topic - pub fn get_gossip_ids(&self, topic: &TopicHash) -> Vec { + /// Get a list of [`MessageId`]s for a given topic. + pub fn get_gossip_message_ids(&self, topic: &TopicHash) -> Vec { self.history[..self.gossip] .iter() .fold(vec![], |mut current_entries, entries| { @@ -105,7 +136,7 @@ impl MessageCache { let mut found_entries: Vec = entries .iter() .filter_map(|entry| { - if entry.topics.iter().any(|t| t == topic) { + if &entry.topic == topic { let mid = &entry.mid; // Only gossip validated messages if let Some(true) = self.msgs.get(mid).map(|msg| msg.validated) { @@ -126,50 +157,74 @@ impl MessageCache { } /// Shift the history array down one and delete messages associated with the - /// last entry + /// last entry. pub fn shift(&mut self) { for entry in self.history.pop().expect("history is always > 1") { - self.msgs.remove(&entry.mid); + if let Some(msg) = self.msgs.remove(&entry.mid) { + if !msg.validated { + // If GossipsubConfig::validate_messages is true, the implementing + // application has to ensure that Gossipsub::validate_message gets called for + // each received message within the cache timeout time." + debug!( + "The message with id {} got removed from the cache without being validated.", + &entry.mid + ); + } + } + debug!("Remove message from the cache: {}", &entry.mid); + + self.iwant_counts.remove(&entry.mid); } // Insert an empty vec in position 0 self.history.insert(0, Vec::new()); } + + /// Removes a message from the cache and returns it if existent + pub fn remove(&mut self, message_id: &MessageId) -> Option { + //We only remove the message from msgs and iwant_count and keep the message_id in the + // history vector. Zhe id in the history vector will simply be ignored on popping. + + self.iwant_counts.remove(message_id); + self.msgs.remove(message_id) + } } #[cfg(test)] mod tests { use super::*; - use crate::{Topic, TopicHash}; + use crate::types::RawGossipsubMessage; + use crate::{IdentTopic as Topic, TopicHash}; use libp2p_core::PeerId; - fn gen_testm(x: u64, topics: Vec) -> GossipsubMessage { + fn gen_testm(x: u64, topic: TopicHash) -> (MessageId, RawGossipsubMessage) { + let default_id = |message: &RawGossipsubMessage| { + // default message id is: source + sequence number + let mut source_string = message.source.as_ref().unwrap().to_base58(); + source_string.push_str(&message.sequence_number.unwrap().to_string()); + MessageId::from(source_string) + }; let u8x: u8 = x as u8; let source = Some(PeerId::random()); let data: Vec = vec![u8x]; let sequence_number = Some(x); - let m = GossipsubMessage { + let m = RawGossipsubMessage { source, data, sequence_number, - topics, + topic, signature: None, key: None, - validated: true, + validated: false, }; - m + + let id = default_id(&m); + (id, m) } fn new_cache(gossip_size: usize, history: usize) -> MessageCache { - let default_id = |message: &GossipsubMessage| { - // default message id is: source + sequence number - let mut source_string = message.source.as_ref().unwrap().to_base58(); - source_string.push_str(&message.sequence_number.unwrap().to_string()); - MessageId::from(source_string) - }; - - MessageCache::new(gossip_size, history, default_id) + MessageCache::new(gossip_size, history) } #[test] @@ -186,16 +241,14 @@ mod tests { fn test_put_get_one() { let mut mc = new_cache(10, 15); - let topic1_hash = Topic::new("topic1".into()).no_hash().clone(); - let topic2_hash = Topic::new("topic2".into()).no_hash().clone(); - - let m = gen_testm(10, vec![topic1_hash, topic2_hash]); + let topic1_hash = Topic::new("topic1").hash().clone(); + let (id, m) = gen_testm(10, topic1_hash); - mc.put(m.clone()); + mc.put(&id, m.clone()); assert!(mc.history[0].len() == 1); - let fetched = mc.get(&(mc.msg_id)(&m)); + let fetched = mc.get(&id); assert_eq!(fetched.is_none(), false); assert_eq!(fetched.is_some(), true); @@ -212,12 +265,10 @@ mod tests { fn test_get_wrong() { let mut mc = new_cache(10, 15); - let topic1_hash = Topic::new("topic1".into()).no_hash().clone(); - let topic2_hash = Topic::new("topic2".into()).no_hash().clone(); + let topic1_hash = Topic::new("topic1").hash().clone(); + let (id, m) = gen_testm(10, topic1_hash); - let m = gen_testm(10, vec![topic1_hash, topic2_hash]); - - mc.put(m.clone()); + mc.put(&id, m.clone()); // Try to get an incorrect ID let wrong_id = MessageId::new(b"wrongid"); @@ -236,36 +287,17 @@ mod tests { assert_eq!(fetched.is_none(), true); } - #[test] - /// Test adding a message with no topics. - fn test_no_topic_put() { - let mut mc = new_cache(3, 5); - - // Build the message - let m = gen_testm(1, vec![]); - mc.put(m.clone()); - - let fetched = mc.get(&(mc.msg_id)(&m)); - - // Make sure it is the same fetched message - match fetched { - Some(x) => assert_eq!(*x, m), - _ => assert!(false), - } - } - #[test] /// Test shift mechanism. fn test_shift() { let mut mc = new_cache(1, 5); - let topic1_hash = Topic::new("topic1".into()).no_hash().clone(); - let topic2_hash = Topic::new("topic2".into()).no_hash().clone(); + let topic1_hash = Topic::new("topic1").hash().clone(); // Build the message for i in 0..10 { - let m = gen_testm(i, vec![topic1_hash.clone(), topic2_hash.clone()]); - mc.put(m.clone()); + let (id, m) = gen_testm(i, topic1_hash.clone()); + mc.put(&id, m.clone()); } mc.shift(); @@ -283,12 +315,12 @@ mod tests { fn test_empty_shift() { let mut mc = new_cache(1, 5); - let topic1_hash = Topic::new("topic1".into()).no_hash().clone(); - let topic2_hash = Topic::new("topic2".into()).no_hash().clone(); + let topic1_hash = Topic::new("topic1").hash().clone(); + // Build the message for i in 0..10 { - let m = gen_testm(i, vec![topic1_hash.clone(), topic2_hash.clone()]); - mc.put(m.clone()); + let (id, m) = gen_testm(i, topic1_hash.clone()); + mc.put(&id, m.clone()); } mc.shift(); @@ -309,12 +341,12 @@ mod tests { fn test_remove_last_from_shift() { let mut mc = new_cache(4, 5); - let topic1_hash = Topic::new("topic1".into()).no_hash().clone(); - let topic2_hash = Topic::new("topic2".into()).no_hash().clone(); + let topic1_hash = Topic::new("topic1").hash().clone(); + // Build the message for i in 0..10 { - let m = gen_testm(i, vec![topic1_hash.clone(), topic2_hash.clone()]); - mc.put(m.clone()); + let (id, m) = gen_testm(i, topic1_hash.clone()); + mc.put(&id, m.clone()); } // Shift right until deleting messages diff --git a/protocols/gossipsub/src/peer_score.rs b/protocols/gossipsub/src/peer_score.rs new file mode 100644 index 00000000000..42765837195 --- /dev/null +++ b/protocols/gossipsub/src/peer_score.rs @@ -0,0 +1,892 @@ +// Copyright 2020 Sigma Prime Pty Ltd. +// +// Permission is hereby granted, free of charge, to any person obtaining a +// copy of this software and associated documentation files (the "Software"), +// to deal in the Software without restriction, including without limitation +// the rights to use, copy, modify, merge, publish, distribute, sublicense, +// and/or sell copies of the Software, and to permit persons to whom the +// Software is furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS +// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +// DEALINGS IN THE SOFTWARE. + +//! +//! Manages and stores the Scoring logic of a particular peer on the gossipsub behaviour. + +use crate::time_cache::TimeCache; +use crate::{MessageId, TopicHash}; +use libp2p_core::PeerId; +use log::{debug, trace, warn}; +use std::collections::{hash_map, HashMap, HashSet}; +use std::net::IpAddr; +use std::time::{Duration, Instant}; + +mod params; +use crate::error::ValidationError; +pub use params::{ + score_parameter_decay, score_parameter_decay_with_base, PeerScoreParams, PeerScoreThresholds, + TopicScoreParams, +}; + +#[cfg(test)] +mod tests; + +/// The number of seconds delivery messages are stored in the cache. +const TIME_CACHE_DURATION: u64 = 120; + +pub(crate) struct PeerScore { + params: PeerScoreParams, + /// The score parameters. + peer_stats: HashMap, + /// Tracking peers per IP. + peer_ips: HashMap>, + /// Message delivery tracking. This is a time-cache of [`DeliveryRecord`]s. + deliveries: TimeCache, + /// callback for monitoring message delivery times + message_delivery_time_callback: Option, +} + +/// General statistics for a given gossipsub peer. +struct PeerStats { + /// Connection status of the peer. + status: ConnectionStatus, + /// Stats per topic. + topics: HashMap, + /// IP tracking for individual peers. + known_ips: HashSet, + /// Behaviour penalty that is applied to the peer, assigned by the behaviour. + behaviour_penalty: f64, + /// Application specific score. Can be manipulated by calling PeerScore::set_application_score + application_score: f64, +} + +enum ConnectionStatus { + /// The peer is connected. + Connected, + /// The peer is disconnected + Disconnected { + /// Expiration time of the score state for disconnected peers. + expire: Instant, + }, +} + +impl Default for PeerStats { + fn default() -> Self { + PeerStats { + status: ConnectionStatus::Connected, + topics: HashMap::new(), + known_ips: HashSet::new(), + behaviour_penalty: 0f64, + application_score: 0f64, + } + } +} + +impl PeerStats { + /// Returns a mutable reference to topic stats if they exist, otherwise if the supplied parameters score the + /// topic, inserts the default stats and returns a reference to those. If neither apply, returns None. + pub fn stats_or_default_mut( + &mut self, + topic_hash: TopicHash, + params: &PeerScoreParams, + ) -> Option<&mut TopicStats> { + if params.topics.get(&topic_hash).is_some() { + Some(self.topics.entry(topic_hash).or_default()) + } else { + self.topics.get_mut(&topic_hash) + } + } +} + +/// Stats assigned to peer for each topic. +struct TopicStats { + mesh_status: MeshStatus, + /// Number of first message deliveries. + first_message_deliveries: f64, + /// True if the peer has been in the mesh for enough time to activate mesh message deliveries. + mesh_message_deliveries_active: bool, + /// Number of message deliveries from the mesh. + mesh_message_deliveries: f64, + /// Mesh rate failure penalty. + mesh_failure_penalty: f64, + /// Invalid message counter. + invalid_message_deliveries: f64, +} + +impl TopicStats { + /// Returns true if the peer is in the `mesh`. + pub fn in_mesh(&self) -> bool { + matches!(self.mesh_status, MeshStatus::Active { .. }) + } +} + +/// Status defining a peer's inclusion in the mesh and associated parameters. +enum MeshStatus { + Active { + /// The time the peer was last GRAFTed; + graft_time: Instant, + /// The time the peer has been in the mesh. + mesh_time: Duration, + }, + InActive, +} + +impl MeshStatus { + /// Initialises a new [`MeshStatus::Active`] mesh status. + pub fn new_active() -> Self { + MeshStatus::Active { + graft_time: Instant::now(), + mesh_time: Duration::from_secs(0), + } + } +} + +impl Default for TopicStats { + fn default() -> Self { + TopicStats { + mesh_status: MeshStatus::InActive, + first_message_deliveries: Default::default(), + mesh_message_deliveries_active: Default::default(), + mesh_message_deliveries: Default::default(), + mesh_failure_penalty: Default::default(), + invalid_message_deliveries: Default::default(), + } + } +} + +#[derive(PartialEq, Debug)] +struct DeliveryRecord { + status: DeliveryStatus, + first_seen: Instant, + peers: HashSet, +} + +#[derive(PartialEq, Debug)] +enum DeliveryStatus { + /// Don't know (yet) if the message is valid. + Unknown, + /// The message is valid together with the validated time. + Valid(Instant), + /// The message is invalid. + Invalid, + /// Instructed by the validator to ignore the message. + Ignored, +} + +impl Default for DeliveryRecord { + fn default() -> Self { + DeliveryRecord { + status: DeliveryStatus::Unknown, + first_seen: Instant::now(), + peers: HashSet::new(), + } + } +} + +impl PeerScore { + /// Creates a new [`PeerScore`] using a given set of peer scoring parameters. + #[allow(dead_code)] + pub fn new(params: PeerScoreParams) -> Self { + Self::new_with_message_delivery_time_callback(params, None) + } + + pub fn new_with_message_delivery_time_callback( + params: PeerScoreParams, + callback: Option, + ) -> Self { + PeerScore { + params, + peer_stats: HashMap::new(), + peer_ips: HashMap::new(), + deliveries: TimeCache::new(Duration::from_secs(TIME_CACHE_DURATION)), + message_delivery_time_callback: callback, + } + } + + /// Returns the score for a peer. + pub fn score(&self, peer_id: &PeerId) -> f64 { + let peer_stats = match self.peer_stats.get(peer_id) { + Some(v) => v, + None => return 0.0, + }; + + let mut score = 0.0; + + // topic scores + for (topic, topic_stats) in peer_stats.topics.iter() { + // topic parameters + if let Some(topic_params) = self.params.topics.get(topic) { + // we are tracking the topic + + // the topic score + let mut topic_score = 0.0; + + // P1: time in mesh + if let MeshStatus::Active { mesh_time, .. } = topic_stats.mesh_status { + let p1 = { + let v = mesh_time.as_secs_f64() + / topic_params.time_in_mesh_quantum.as_secs_f64(); + if v < topic_params.time_in_mesh_cap { + v + } else { + topic_params.time_in_mesh_cap + } + }; + topic_score += p1 * topic_params.time_in_mesh_weight; + } + + // P2: first message deliveries + let p2 = { + let v = topic_stats.first_message_deliveries as f64; + if v < topic_params.first_message_deliveries_cap { + v + } else { + topic_params.first_message_deliveries_cap + } + }; + topic_score += p2 * topic_params.first_message_deliveries_weight; + + // P3: mesh message deliveries + if topic_stats.mesh_message_deliveries_active + && topic_stats.mesh_message_deliveries + < topic_params.mesh_message_deliveries_threshold + { + let deficit = topic_params.mesh_message_deliveries_threshold + - topic_stats.mesh_message_deliveries; + let p3 = deficit * deficit; + topic_score += p3 * topic_params.mesh_message_deliveries_weight; + debug!( + "The peer {} has a mesh message delivieries deficit of {} in topic\ + {} and will get penalized by {}", + peer_id, + deficit, + topic, + p3 * topic_params.mesh_message_deliveries_weight + ); + } + + // P3b: + // NOTE: the weight of P3b is negative (validated in TopicScoreParams.validate), so this detracts. + let p3b = topic_stats.mesh_failure_penalty; + topic_score += p3b * topic_params.mesh_failure_penalty_weight; + + // P4: invalid messages + // NOTE: the weight of P4 is negative (validated in TopicScoreParams.validate), so this detracts. + let p4 = + topic_stats.invalid_message_deliveries * topic_stats.invalid_message_deliveries; + topic_score += p4 * topic_params.invalid_message_deliveries_weight; + + // update score, mixing with topic weight + score += topic_score * topic_params.topic_weight; + } + } + + // apply the topic score cap, if any + if self.params.topic_score_cap > 0f64 && score > self.params.topic_score_cap { + score = self.params.topic_score_cap; + } + + // P5: application-specific score + let p5 = peer_stats.application_score; + score += p5 * self.params.app_specific_weight; + + // P6: IP collocation factor + for ip in peer_stats.known_ips.iter() { + if self.params.ip_colocation_factor_whitelist.get(ip).is_some() { + continue; + } + + // P6 has a cliff (ip_colocation_factor_threshold); it's only applied iff + // at least that many peers are connected to us from that source IP + // addr. It is quadratic, and the weight is negative (validated by + // peer_score_params.validate()). + if let Some(peers_in_ip) = self.peer_ips.get(ip).map(|peers| peers.len()) { + if (peers_in_ip as f64) > self.params.ip_colocation_factor_threshold { + let surplus = (peers_in_ip as f64) - self.params.ip_colocation_factor_threshold; + let p6 = surplus * surplus; + debug!( + "The peer {} gets penalized because of too many peers with the ip {}. \ + The surplus is {}. ", + peer_id, ip, surplus + ); + score += p6 * self.params.ip_colocation_factor_weight; + } + } + } + + // P7: behavioural pattern penalty + if peer_stats.behaviour_penalty > self.params.behaviour_penalty_threshold { + let excess = peer_stats.behaviour_penalty - self.params.behaviour_penalty_threshold; + let p7 = excess * excess; + score += p7 * self.params.behaviour_penalty_weight; + } + score + } + + pub fn add_penalty(&mut self, peer_id: &PeerId, count: usize) { + if let Some(peer_stats) = self.peer_stats.get_mut(peer_id) { + debug!( + "Behavioral penalty for peer {}, count = {}.", + peer_id, count + ); + peer_stats.behaviour_penalty += count as f64; + } + } + + fn remove_ips_for_peer( + peer_stats: &PeerStats, + peer_ips: &mut HashMap>, + peer_id: &PeerId, + ) { + for ip in peer_stats.known_ips.iter() { + if let Some(peer_set) = peer_ips.get_mut(ip) { + peer_set.remove(peer_id); + } + } + } + + pub fn refresh_scores(&mut self) { + let now = Instant::now(); + let params_ref = &self.params; + let peer_ips_ref = &mut self.peer_ips; + self.peer_stats.retain(|peer_id, peer_stats| { + if let ConnectionStatus::Disconnected { expire } = peer_stats.status { + // has the retention period expired? + if now > expire { + // yes, throw it away (but clean up the IP tracking first) + Self::remove_ips_for_peer(peer_stats, peer_ips_ref, peer_id); + // re address this, use retain or entry + return false; + } + + // we don't decay retained scores, as the peer is not active. + // this way the peer cannot reset a negative score by simply disconnecting and reconnecting, + // unless the retention period has elapsed. + // similarly, a well behaved peer does not lose its score by getting disconnected. + return true; + } + + for (topic, topic_stats) in peer_stats.topics.iter_mut() { + // the topic parameters + if let Some(topic_params) = params_ref.topics.get(topic) { + // decay counters + topic_stats.first_message_deliveries *= + topic_params.first_message_deliveries_decay; + if topic_stats.first_message_deliveries < params_ref.decay_to_zero { + topic_stats.first_message_deliveries = 0.0; + } + topic_stats.mesh_message_deliveries *= + topic_params.mesh_message_deliveries_decay; + if topic_stats.mesh_message_deliveries < params_ref.decay_to_zero { + topic_stats.mesh_message_deliveries = 0.0; + } + topic_stats.mesh_failure_penalty *= topic_params.mesh_failure_penalty_decay; + if topic_stats.mesh_failure_penalty < params_ref.decay_to_zero { + topic_stats.mesh_failure_penalty = 0.0; + } + topic_stats.invalid_message_deliveries *= + topic_params.invalid_message_deliveries_decay; + if topic_stats.invalid_message_deliveries < params_ref.decay_to_zero { + topic_stats.invalid_message_deliveries = 0.0; + } + // update mesh time and activate mesh message delivery parameter if need be + if let MeshStatus::Active { + ref mut mesh_time, + ref mut graft_time, + } = topic_stats.mesh_status + { + *mesh_time = now.duration_since(*graft_time); + if *mesh_time > topic_params.mesh_message_deliveries_activation { + topic_stats.mesh_message_deliveries_active = true; + } + } + } + } + + // decay P7 counter + peer_stats.behaviour_penalty *= params_ref.behaviour_penalty_decay; + if peer_stats.behaviour_penalty < params_ref.decay_to_zero { + peer_stats.behaviour_penalty = 0.0; + } + true + }); + } + + /// Adds a connected peer to [`PeerScore`], initialising with empty ips (ips get added later + /// through add_ip. + pub fn add_peer(&mut self, peer_id: PeerId) { + let peer_stats = self.peer_stats.entry(peer_id).or_default(); + + // mark the peer as connected + peer_stats.status = ConnectionStatus::Connected; + } + + /// Adds a new ip to a peer, if the peer is not yet known creates a new peer_stats entry for it + pub fn add_ip(&mut self, peer_id: &PeerId, ip: IpAddr) { + trace!("Add ip for peer {}, ip: {}", peer_id, ip); + let peer_stats = self.peer_stats.entry(peer_id.clone()).or_default(); + + // Mark the peer as connected (currently the default is connected, but we don't want to + // rely on the default). + peer_stats.status = ConnectionStatus::Connected; + + // Insert the ip + peer_stats.known_ips.insert(ip); + self.peer_ips + .entry(ip) + .or_insert_with(HashSet::new) + .insert(peer_id.clone()); + } + + /// Removes an ip from a peer + pub fn remove_ip(&mut self, peer_id: &PeerId, ip: &IpAddr) { + if let Some(peer_stats) = self.peer_stats.get_mut(peer_id) { + peer_stats.known_ips.remove(ip); + if let Some(peer_ids) = self.peer_ips.get_mut(ip) { + trace!("Remove ip for peer {}, ip: {}", peer_id, ip); + peer_ids.remove(peer_id); + } else { + trace!( + "No entry in peer_ips for ip {} which should get removed for peer {}", + ip, + peer_id + ); + } + } else { + trace!( + "No peer_stats for peer {} which should remove the ip {}", + peer_id, + ip + ); + } + } + + /// Removes a peer from the score table. This retains peer statistics if their score is + /// non-positive. + pub fn remove_peer(&mut self, peer_id: &PeerId) { + // we only retain non-positive scores of peers + if self.score(peer_id) > 0f64 { + if let hash_map::Entry::Occupied(entry) = self.peer_stats.entry(peer_id.clone()) { + Self::remove_ips_for_peer(entry.get(), &mut self.peer_ips, peer_id); + entry.remove(); + } + return; + } + + // if the peer is retained (including it's score) the `first_message_delivery` counters + // are reset to 0 and mesh delivery penalties applied. + if let Some(peer_stats) = self.peer_stats.get_mut(peer_id) { + for (topic, topic_stats) in peer_stats.topics.iter_mut() { + topic_stats.first_message_deliveries = 0f64; + + if let Some(threshold) = self + .params + .topics + .get(topic) + .map(|param| param.mesh_message_deliveries_threshold) + { + if topic_stats.in_mesh() + && topic_stats.mesh_message_deliveries_active + && topic_stats.mesh_message_deliveries < threshold + { + let deficit = threshold - topic_stats.mesh_message_deliveries; + topic_stats.mesh_failure_penalty += deficit * deficit; + } + } + + topic_stats.mesh_status = MeshStatus::InActive; + topic_stats.mesh_message_deliveries_active = false; + } + + peer_stats.status = ConnectionStatus::Disconnected { + expire: Instant::now() + self.params.retain_score, + }; + } + } + + /// Handles scoring functionality as a peer GRAFTs to a topic. + pub fn graft(&mut self, peer_id: &PeerId, topic: impl Into) { + let topic = topic.into(); + if let Some(peer_stats) = self.peer_stats.get_mut(peer_id) { + // if we are scoring the topic, update the mesh status. + if let Some(topic_stats) = peer_stats.stats_or_default_mut(topic, &self.params) { + topic_stats.mesh_status = MeshStatus::new_active(); + topic_stats.mesh_message_deliveries_active = false; + } + } + } + + /// Handles scoring functionality as a peer PRUNEs from a topic. + pub fn prune(&mut self, peer_id: &PeerId, topic: TopicHash) { + if let Some(peer_stats) = self.peer_stats.get_mut(peer_id) { + // if we are scoring the topic, update the mesh status. + if let Some(topic_stats) = peer_stats.stats_or_default_mut(topic.clone(), &self.params) + { + // sticky mesh delivery rate failure penalty + let threshold = self + .params + .topics + .get(&topic) + .expect("Topic must exist in order for there to be topic stats") + .mesh_message_deliveries_threshold; + if topic_stats.mesh_message_deliveries_active + && topic_stats.mesh_message_deliveries < threshold + { + let deficit = threshold - topic_stats.mesh_message_deliveries; + topic_stats.mesh_failure_penalty += deficit * deficit; + } + topic_stats.mesh_message_deliveries_active = false; + topic_stats.mesh_status = MeshStatus::InActive; + } + } + } + + pub fn validate_message(&mut self, _from: &PeerId, msg_id: &MessageId, topic_hash: &TopicHash) { + // adds an empty record with the message id + self.deliveries + .entry(msg_id.clone()) + .or_insert_with(DeliveryRecord::default); + + if let Some(callback) = self.message_delivery_time_callback { + if self + .peer_stats + .get(_from) + .and_then(|s| s.topics.get(topic_hash)) + .map(|ts| ts.in_mesh()) + .unwrap_or(false) + { + callback(_from, topic_hash, 0.0); + } + } + } + + pub fn deliver_message(&mut self, from: &PeerId, msg_id: &MessageId, topic_hash: &TopicHash) { + self.mark_first_message_delivery(from, topic_hash); + + let record = self + .deliveries + .entry(msg_id.clone()) + .or_insert_with(DeliveryRecord::default); + + // this should be the first delivery trace + if record.status != DeliveryStatus::Unknown { + warn!("Unexpected delivery trace: Message from {} was first seen {}s ago and has a delivery status {:?}", from, record.first_seen.elapsed().as_secs(), record.status); + return; + } + + // mark the message as valid and reward mesh peers that have already forwarded it to us + record.status = DeliveryStatus::Valid(Instant::now()); + for peer in record.peers.iter().cloned().collect::>() { + // this check is to make sure a peer can't send us a message twice and get a double + // count if it is a first delivery + if &peer != from { + self.mark_duplicate_message_delivery(&peer, topic_hash, None); + } + } + } + + /// Similar to `reject_message` except does not require the message id or reason for an invalid message. + pub fn reject_invalid_message(&mut self, from: &PeerId, topic_hash: &TopicHash) { + debug!( + "Message from {} rejected because of ValidationError or SelfOrigin", + from + ); + self.mark_invalid_message_delivery(from, topic_hash); + } + + // Reject a message. + pub fn reject_message( + &mut self, + from: &PeerId, + msg_id: &MessageId, + topic_hash: &TopicHash, + reason: RejectReason, + ) { + match reason { + // these messages are not tracked, but the peer is penalized as they are invalid + RejectReason::ValidationError(_) | RejectReason::SelfOrigin => { + self.reject_invalid_message(from, topic_hash); + return; + } + // we ignore those messages, so do nothing. + RejectReason::BlackListedPeer | RejectReason::BlackListedSource => { + return; + } + _ => {} // the rest are handled after record creation + } + + let peers: Vec<_> = { + let mut record = self + .deliveries + .entry(msg_id.clone()) + .or_insert_with(DeliveryRecord::default); + + // this should be the first delivery trace + if record.status != DeliveryStatus::Unknown { + warn!("Unexpected delivery trace: Message from {} was first seen {}s ago and has a delivery status {:?}", from, record.first_seen.elapsed().as_secs(), record.status); + return; + } + + if let RejectReason::ValidationIgnored = reason { + // we were explicitly instructed by the validator to ignore the message but not penalize + // the peer + record.status = DeliveryStatus::Ignored; + record.peers.clear(); + return; + } + + // mark the message as invalid and penalize peers that have already forwarded it. + record.status = DeliveryStatus::Invalid; + // release the delivery time tracking map to free some memory early + record.peers.drain().collect() + }; + + self.mark_invalid_message_delivery(from, topic_hash); + for peer_id in peers.iter() { + self.mark_invalid_message_delivery(peer_id, topic_hash) + } + } + + pub fn duplicated_message( + &mut self, + from: &PeerId, + msg_id: &MessageId, + topic_hash: &TopicHash, + ) { + let record = self + .deliveries + .entry(msg_id.clone()) + .or_insert_with(DeliveryRecord::default); + + if record.peers.get(from).is_some() { + // we have already seen this duplicate! + return; + } + + if let Some(callback) = self.message_delivery_time_callback { + let time = if let DeliveryStatus::Valid(validated) = record.status { + validated.elapsed().as_secs_f64() + } else { + 0.0 + }; + if self + .peer_stats + .get(from) + .and_then(|s| s.topics.get(topic_hash)) + .map(|ts| ts.in_mesh()) + .unwrap_or(false) + { + callback(from, topic_hash, time); + } + } + + match record.status { + DeliveryStatus::Unknown => { + // the message is being validated; track the peer delivery and wait for + // the Deliver/Reject notification. + record.peers.insert(from.clone()); + } + DeliveryStatus::Valid(validated) => { + // mark the peer delivery time to only count a duplicate delivery once. + record.peers.insert(from.clone()); + self.mark_duplicate_message_delivery(from, topic_hash, Some(validated)); + } + DeliveryStatus::Invalid => { + // we no longer track delivery time + self.mark_invalid_message_delivery(from, topic_hash); + } + DeliveryStatus::Ignored => { + // the message was ignored; do nothing (we don't know if it was valid) + } + } + } + + /// Sets the application specific score for a peer. Returns true if the peer is the peer is + /// connected or if the score of the peer is not yet expired and false otherwise. + pub fn set_application_score(&mut self, peer_id: &PeerId, new_score: f64) -> bool { + if let Some(peer_stats) = self.peer_stats.get_mut(peer_id) { + peer_stats.application_score = new_score; + true + } else { + false + } + } + + /// Sets scoring parameters for a topic. + pub fn set_topic_params(&mut self, topic_hash: TopicHash, params: TopicScoreParams) { + use hash_map::Entry::*; + match self.params.topics.entry(topic_hash.clone()) { + Occupied(mut entry) => { + let first_message_deliveries_cap = params.first_message_deliveries_cap; + let mesh_message_delivieries_cap = params.mesh_message_deliveries_cap; + let old_params = entry.insert(params); + + if old_params.first_message_deliveries_cap > first_message_deliveries_cap { + for stats in &mut self.peer_stats.values_mut() { + if let Some(tstats) = stats.topics.get_mut(&topic_hash) { + if tstats.first_message_deliveries > first_message_deliveries_cap { + tstats.first_message_deliveries = first_message_deliveries_cap; + } + } + } + } + + if old_params.mesh_message_deliveries_cap > mesh_message_delivieries_cap { + for stats in self.peer_stats.values_mut() { + if let Some(tstats) = stats.topics.get_mut(&topic_hash) { + if tstats.mesh_message_deliveries > mesh_message_delivieries_cap { + tstats.mesh_message_deliveries = mesh_message_delivieries_cap; + } + } + } + } + } + Vacant(entry) => { + entry.insert(params); + } + } + } + + /// Increments the "invalid message deliveries" counter for all scored topics the message + /// is published in. + fn mark_invalid_message_delivery(&mut self, peer_id: &PeerId, topic_hash: &TopicHash) { + if let Some(peer_stats) = self.peer_stats.get_mut(peer_id) { + if let Some(topic_stats) = + peer_stats.stats_or_default_mut(topic_hash.clone(), &self.params) + { + debug!( + "Peer {} delivered an invalid message in topic {} and gets penalized \ + for it", + peer_id, topic_hash + ); + topic_stats.invalid_message_deliveries += 1f64; + } + } + } + + /// Increments the "first message deliveries" counter for all scored topics the message is + /// published in, as well as the "mesh message deliveries" counter, if the peer is in the + /// mesh for the topic. + fn mark_first_message_delivery(&mut self, peer_id: &PeerId, topic_hash: &TopicHash) { + if let Some(peer_stats) = self.peer_stats.get_mut(peer_id) { + if let Some(topic_stats) = + peer_stats.stats_or_default_mut(topic_hash.clone(), &self.params) + { + let cap = self + .params + .topics + .get(topic_hash) + .expect("Topic must exist if there are known topic_stats") + .first_message_deliveries_cap; + topic_stats.first_message_deliveries = + if topic_stats.first_message_deliveries + 1f64 > cap { + cap + } else { + topic_stats.first_message_deliveries + 1f64 + }; + + if let MeshStatus::Active { .. } = topic_stats.mesh_status { + let cap = self + .params + .topics + .get(topic_hash) + .expect("Topic must exist if there are known topic_stats") + .mesh_message_deliveries_cap; + + topic_stats.mesh_message_deliveries = + if topic_stats.mesh_message_deliveries + 1f64 > cap { + cap + } else { + topic_stats.mesh_message_deliveries + 1f64 + }; + } + } + } + } + + /// Increments the "mesh message deliveries" counter for messages we've seen before, as long the + /// message was received within the P3 window. + fn mark_duplicate_message_delivery( + &mut self, + peer_id: &PeerId, + topic_hash: &TopicHash, + validated_time: Option, + ) { + if let Some(peer_stats) = self.peer_stats.get_mut(peer_id) { + let now = if validated_time.is_some() { + Some(Instant::now()) + } else { + None + }; + if let Some(topic_stats) = + peer_stats.stats_or_default_mut(topic_hash.clone(), &self.params) + { + if let MeshStatus::Active { .. } = topic_stats.mesh_status { + let topic_params = self + .params + .topics + .get(topic_hash) + .expect("Topic must exist if there are known topic_stats"); + + // check against the mesh delivery window -- if the validated time is passed as 0, then + // the message was received before we finished validation and thus falls within the mesh + // delivery window. + let mut falls_in_mesh_deliver_window = true; + if let Some(validated_time) = validated_time { + if let Some(now) = &now { + //should always be true + let window_time = validated_time + .checked_add(topic_params.mesh_message_deliveries_window) + .unwrap_or_else(|| *now); + if now > &window_time { + falls_in_mesh_deliver_window = false; + } + } + } + + if falls_in_mesh_deliver_window { + let cap = topic_params.mesh_message_deliveries_cap; + topic_stats.mesh_message_deliveries = + if topic_stats.mesh_message_deliveries + 1f64 > cap { + cap + } else { + topic_stats.mesh_message_deliveries + 1f64 + }; + } + } + } + } + } + + pub(crate) fn mesh_message_deliveries(&self, peer: &PeerId, topic: &TopicHash) -> Option { + self.peer_stats + .get(peer) + .and_then(|s| s.topics.get(topic)) + .map(|t| t.mesh_message_deliveries) + } +} + +/// The reason a Gossipsub message has been rejected. +#[derive(Clone, Copy)] +pub(crate) enum RejectReason { + /// The message failed the configured validation during decoding. + ValidationError(ValidationError), + /// The message source is us. + SelfOrigin, + /// The peer that sent the message was blacklisted. + BlackListedPeer, + /// The source (from field) of the message was blacklisted. + BlackListedSource, + /// The validation was ignored. + ValidationIgnored, + /// The validation failed. + ValidationFailed, +} diff --git a/protocols/gossipsub/src/peer_score/params.rs b/protocols/gossipsub/src/peer_score/params.rs new file mode 100644 index 00000000000..c4159bb6ec1 --- /dev/null +++ b/protocols/gossipsub/src/peer_score/params.rs @@ -0,0 +1,399 @@ +// Copyright 2020 Sigma Prime Pty Ltd. +// +// Permission is hereby granted, free of charge, to any person obtaining a +// copy of this software and associated documentation files (the "Software"), +// to deal in the Software without restriction, including without limitation +// the rights to use, copy, modify, merge, publish, distribute, sublicense, +// and/or sell copies of the Software, and to permit persons to whom the +// Software is furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS +// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +// DEALINGS IN THE SOFTWARE. + +use crate::TopicHash; +use std::collections::{HashMap, HashSet}; +use std::net::IpAddr; +use std::time::Duration; + +/// The default number of seconds for a decay interval. +const DEFAULT_DECAY_INTERVAL: u64 = 1; +/// The default rate to decay to 0. +const DEFAULT_DECAY_TO_ZERO: f64 = 0.1; + +/// Computes the decay factor for a parameter, assuming the `decay_interval` is 1s +/// and that the value decays to zero if it drops below 0.01. +pub fn score_parameter_decay(decay: Duration) -> f64 { + score_parameter_decay_with_base( + decay, + Duration::from_secs(DEFAULT_DECAY_INTERVAL), + DEFAULT_DECAY_TO_ZERO, + ) +} + +/// Computes the decay factor for a parameter using base as the `decay_interval`. +pub fn score_parameter_decay_with_base(decay: Duration, base: Duration, decay_to_zero: f64) -> f64 { + // the decay is linear, so after n ticks the value is factor^n + // so factor^n = decay_to_zero => factor = decay_to_zero^(1/n) + let ticks = decay.as_secs_f64() / base.as_secs_f64(); + decay_to_zero.powf(1f64 / ticks) +} + +#[derive(Debug, Clone)] +pub struct PeerScoreThresholds { + /// The score threshold below which gossip propagation is suppressed; + /// should be negative. + pub gossip_threshold: f64, + + /// The score threshold below which we shouldn't publish when using flood + /// publishing (also applies to fanout peers); should be negative and <= `gossip_threshold`. + pub publish_threshold: f64, + + /// The score threshold below which message processing is suppressed altogether, + /// implementing an effective graylist according to peer score; should be negative and + /// <= `publish_threshold`. + pub graylist_threshold: f64, + + /// The score threshold below which px will be ignored; this should be positive + /// and limited to scores attainable by bootstrappers and other trusted nodes. + pub accept_px_threshold: f64, + + /// The median mesh score threshold before triggering opportunistic + /// grafting; this should have a small positive value. + pub opportunistic_graft_threshold: f64, +} + +impl Default for PeerScoreThresholds { + fn default() -> Self { + PeerScoreThresholds { + gossip_threshold: -10.0, + publish_threshold: -50.0, + graylist_threshold: -80.0, + accept_px_threshold: 10.0, + opportunistic_graft_threshold: 20.0, + } + } +} + +impl PeerScoreThresholds { + pub fn validate(&self) -> Result<(), &'static str> { + if self.gossip_threshold > 0f64 { + return Err("invalid gossip threshold; it must be <= 0"); + } + if self.publish_threshold > 0f64 || self.publish_threshold > self.gossip_threshold { + return Err("Invalid publish threshold; it must be <= 0 and <= gossip threshold"); + } + if self.graylist_threshold > 0f64 || self.graylist_threshold > self.publish_threshold { + return Err("Invalid graylist threshold; it must be <= 0 and <= publish threshold"); + } + if self.accept_px_threshold < 0f64 { + return Err("Invalid accept px threshold; it must be >= 0"); + } + if self.opportunistic_graft_threshold < 0f64 { + return Err("Invalid opportunistic grafting threshold; it must be >= 0"); + } + Ok(()) + } +} + +#[derive(Debug, Clone)] +pub struct PeerScoreParams { + /// Score parameters per topic. + pub topics: HashMap, + + /// Aggregate topic score cap; this limits the total contribution of topics towards a positive + /// score. It must be positive (or 0 for no cap). + pub topic_score_cap: f64, + + /// P5: Application-specific peer scoring + pub app_specific_weight: f64, + + /// P6: IP-colocation factor. + /// The parameter has an associated counter which counts the number of peers with the same IP. + /// If the number of peers in the same IP exceeds `ip_colocation_factor_threshold, then the value + /// is the square of the difference, ie `(peers_in_same_ip - ip_colocation_threshold)^2`. + /// If the number of peers in the same IP is less than the threshold, then the value is 0. + /// The weight of the parameter MUST be negative, unless you want to disable for testing. + /// Note: In order to simulate many IPs in a manageable manner when testing, you can set the weight to 0 + /// thus disabling the IP colocation penalty. + pub ip_colocation_factor_weight: f64, + pub ip_colocation_factor_threshold: f64, + pub ip_colocation_factor_whitelist: HashSet, + + /// P7: behavioural pattern penalties. + /// This parameter has an associated counter which tracks misbehaviour as detected by the + /// router. The router currently applies penalties for the following behaviors: + /// - attempting to re-graft before the prune backoff time has elapsed. + /// - not following up in IWANT requests for messages advertised with IHAVE. + /// + /// The value of the parameter is the square of the counter over the threshold, which decays + /// with BehaviourPenaltyDecay. + /// The weight of the parameter MUST be negative (or zero to disable). + pub behaviour_penalty_weight: f64, + pub behaviour_penalty_threshold: f64, + pub behaviour_penalty_decay: f64, + + /// The decay interval for parameter counters. + pub decay_interval: Duration, + + /// Counter value below which it is considered 0. + pub decay_to_zero: f64, + + /// Time to remember counters for a disconnected peer. + pub retain_score: Duration, +} + +impl Default for PeerScoreParams { + fn default() -> Self { + PeerScoreParams { + topics: HashMap::new(), + topic_score_cap: 3600.0, + app_specific_weight: 10.0, + ip_colocation_factor_weight: -5.0, + ip_colocation_factor_threshold: 10.0, + ip_colocation_factor_whitelist: HashSet::new(), + behaviour_penalty_weight: -10.0, + behaviour_penalty_threshold: 0.0, + behaviour_penalty_decay: 0.2, + decay_interval: Duration::from_secs(DEFAULT_DECAY_INTERVAL), + decay_to_zero: DEFAULT_DECAY_TO_ZERO, + retain_score: Duration::from_secs(3600), + } + } +} + +/// Peer score parameter validation +impl PeerScoreParams { + pub fn validate(&self) -> Result<(), String> { + for (topic, params) in self.topics.iter() { + if let Err(e) = params.validate() { + return Err(format!( + "Invalid score parameters for topic {}: {}", + topic, e + )); + } + } + + // check that the topic score is 0 or something positive + if self.topic_score_cap < 0f64 { + return Err("Invalid topic score cap; must be positive (or 0 for no cap)".into()); + } + + // check the IP colocation factor + if self.ip_colocation_factor_weight > 0f64 { + return Err( + "Invalid ip_colocation_factor_weight; must be negative (or 0 to disable)".into(), + ); + } + if self.ip_colocation_factor_weight != 0f64 && self.ip_colocation_factor_threshold < 1f64 { + return Err("Invalid ip_colocation_factor_threshold; must be at least 1".into()); + } + + // check the behaviour penalty + if self.behaviour_penalty_weight > 0f64 { + return Err( + "Invalid behaviour_penalty_weight; must be negative (or 0 to disable)".into(), + ); + } + if self.behaviour_penalty_weight != 0f64 + && (self.behaviour_penalty_decay <= 0f64 || self.behaviour_penalty_decay >= 1f64) + { + return Err("invalid behaviour_penalty_decay; must be between 0 and 1".into()); + } + + if self.behaviour_penalty_threshold < 0f64 { + return Err("invalid behaviour_penalty_threshold; must be >= 0".into()); + } + + // check the decay parameters + if self.decay_interval < Duration::from_secs(1) { + return Err("Invalid decay_interval; must be at least 1s".into()); + } + if self.decay_to_zero <= 0f64 || self.decay_to_zero >= 1f64 { + return Err("Invalid decay_to_zero; must be between 0 and 1".into()); + } + + // no need to check the score retention; a value of 0 means that we don't retain scores + Ok(()) + } +} + +#[derive(Debug, Clone)] +pub struct TopicScoreParams { + /// The weight of the topic. + pub topic_weight: f64, + + /// P1: time in the mesh + /// This is the time the peer has been grafted in the mesh. + /// The value of of the parameter is the `time/time_in_mesh_quantum`, capped by `time_in_mesh_cap` + /// The weight of the parameter must be positive (or zero to disable). + pub time_in_mesh_weight: f64, + pub time_in_mesh_quantum: Duration, + pub time_in_mesh_cap: f64, + + /// P2: first message deliveries + /// This is the number of message deliveries in the topic. + /// The value of the parameter is a counter, decaying with `first_message_deliveries_decay`, and capped + /// by `first_message_deliveries_cap`. + /// The weight of the parameter MUST be positive (or zero to disable). + pub first_message_deliveries_weight: f64, + pub first_message_deliveries_decay: f64, + pub first_message_deliveries_cap: f64, + + /// P3: mesh message deliveries + /// This is the number of message deliveries in the mesh, within the + /// `mesh_message_deliveries_window` of message validation; deliveries during validation also + /// count and are retroactively applied when validation succeeds. + /// This window accounts for the minimum time before a hostile mesh peer trying to game the + /// score could replay back a valid message we just sent them. + /// It effectively tracks first and near-first deliveries, ie a message seen from a mesh peer + /// before we have forwarded it to them. + /// The parameter has an associated counter, decaying with `mesh_message_deliveries_decay`. + /// If the counter exceeds the threshold, its value is 0. + /// If the counter is below the `mesh_message_deliveries_threshold`, the value is the square of + /// the deficit, ie (`message_deliveries_threshold - counter)^2` + /// The penalty is only activated after `mesh_message_deliveries_activation` time in the mesh. + /// The weight of the parameter MUST be negative (or zero to disable). + pub mesh_message_deliveries_weight: f64, + pub mesh_message_deliveries_decay: f64, + pub mesh_message_deliveries_cap: f64, + pub mesh_message_deliveries_threshold: f64, + pub mesh_message_deliveries_window: Duration, + pub mesh_message_deliveries_activation: Duration, + + /// P3b: sticky mesh propagation failures + /// This is a sticky penalty that applies when a peer gets pruned from the mesh with an active + /// mesh message delivery penalty. + /// The weight of the parameter MUST be negative (or zero to disable) + pub mesh_failure_penalty_weight: f64, + pub mesh_failure_penalty_decay: f64, + + /// P4: invalid messages + /// This is the number of invalid messages in the topic. + /// The value of the parameter is the square of the counter, decaying with + /// `invalid_message_deliveries_decay`. + /// The weight of the parameter MUST be negative (or zero to disable). + pub invalid_message_deliveries_weight: f64, + pub invalid_message_deliveries_decay: f64, +} + +/// NOTE: The topic score parameters are very network specific. +/// For any production system, these values should be manually set. +impl Default for TopicScoreParams { + fn default() -> Self { + TopicScoreParams { + topic_weight: 0.5, + // P1 + time_in_mesh_weight: 1.0, + time_in_mesh_quantum: Duration::from_millis(1), + time_in_mesh_cap: 3600.0, + // P2 + first_message_deliveries_weight: 1.0, + first_message_deliveries_decay: 0.5, + first_message_deliveries_cap: 2000.0, + // P3 + mesh_message_deliveries_weight: -1.0, + mesh_message_deliveries_decay: 0.5, + mesh_message_deliveries_cap: 100.0, + mesh_message_deliveries_threshold: 20.0, + mesh_message_deliveries_window: Duration::from_millis(10), + mesh_message_deliveries_activation: Duration::from_secs(5), + // P3b + mesh_failure_penalty_weight: -1.0, + mesh_failure_penalty_decay: 0.5, + // P4 + invalid_message_deliveries_weight: -1.0, + invalid_message_deliveries_decay: 0.3, + } + } +} + +impl TopicScoreParams { + pub fn validate(&self) -> Result<(), &'static str> { + // make sure we have a sane topic weight + if self.topic_weight < 0f64 { + return Err("invalid topic weight; must be >= 0"); + } + + if self.time_in_mesh_quantum == Duration::from_secs(0) { + return Err("Invalid time_in_mesh_quantum; must be non zero"); + } + if self.time_in_mesh_weight < 0f64 { + return Err("Invalid time_in_mesh_weight; must be positive (or 0 to disable)"); + } + if self.time_in_mesh_weight != 0f64 && self.time_in_mesh_cap <= 0f64 { + return Err("Invalid time_in_mesh_cap must be positive"); + } + + if self.first_message_deliveries_weight < 0f64 { + return Err( + "Invalid first_message_deliveries_weight; must be positive (or 0 to disable)", + ); + } + if self.first_message_deliveries_weight != 0f64 + && (self.first_message_deliveries_decay <= 0f64 + || self.first_message_deliveries_decay >= 1f64) + { + return Err("Invalid first_message_deliveries_decay; must be between 0 and 1"); + } + if self.first_message_deliveries_weight != 0f64 && self.first_message_deliveries_cap <= 0f64 + { + return Err("Invalid first_message_deliveries_cap must be positive"); + } + + if self.mesh_message_deliveries_weight > 0f64 { + return Err( + "Invalid mesh_message_deliveries_weight; must be negative (or 0 to disable)", + ); + } + if self.mesh_message_deliveries_weight != 0f64 + && (self.mesh_message_deliveries_decay <= 0f64 + || self.mesh_message_deliveries_decay >= 1f64) + { + return Err("Invalid mesh_message_deliveries_decay; must be between 0 and 1"); + } + if self.mesh_message_deliveries_weight != 0f64 && self.mesh_message_deliveries_cap <= 0f64 { + return Err("Invalid mesh_message_deliveries_cap must be positive"); + } + if self.mesh_message_deliveries_weight != 0f64 + && self.mesh_message_deliveries_threshold <= 0f64 + { + return Err("Invalid mesh_message_deliveries_threshold; must be positive"); + } + if self.mesh_message_deliveries_weight != 0f64 + && self.mesh_message_deliveries_activation < Duration::from_secs(1) + { + return Err("Invalid mesh_message_deliveries_activation; must be at least 1s"); + } + + // check P3b + if self.mesh_failure_penalty_weight > 0f64 { + return Err("Invalid mesh_failure_penalty_weight; must be negative (or 0 to disable)"); + } + if self.mesh_failure_penalty_weight != 0f64 + && (self.mesh_failure_penalty_decay <= 0f64 || self.mesh_failure_penalty_decay >= 1f64) + { + return Err("Invalid mesh_failure_penalty_decay; must be between 0 and 1"); + } + + // check P4 + if self.invalid_message_deliveries_weight > 0f64 { + return Err( + "Invalid invalid_message_deliveries_weight; must be negative (or 0 to disable)", + ); + } + if self.invalid_message_deliveries_decay <= 0f64 + || self.invalid_message_deliveries_decay >= 1f64 + { + return Err("Invalid invalid_message_deliveries_decay; must be between 0 and 1"); + } + Ok(()) + } +} diff --git a/protocols/gossipsub/src/peer_score/tests.rs b/protocols/gossipsub/src/peer_score/tests.rs new file mode 100644 index 00000000000..4ede29b7c51 --- /dev/null +++ b/protocols/gossipsub/src/peer_score/tests.rs @@ -0,0 +1,980 @@ +// Copyright 2020 Sigma Prime Pty Ltd. +// +// Permission is hereby granted, free of charge, to any person obtaining a +// copy of this software and associated documentation files (the "Software"), +// to deal in the Software without restriction, including without limitation +// the rights to use, copy, modify, merge, publish, distribute, sublicense, +// and/or sell copies of the Software, and to permit persons to whom the +// Software is furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS +// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +// DEALINGS IN THE SOFTWARE. + +/// A collection of unit tests mostly ported from the go implementation. +use super::*; + +use crate::types::RawGossipsubMessage; +use crate::{GossipsubMessage, IdentTopic as Topic}; + +// estimates a value within variance +fn within_variance(value: f64, expected: f64, variance: f64) -> bool { + if expected >= 0.0 { + return value > expected * (1.0 - variance) && value < expected * (1.0 + variance); + } + return value > expected * (1.0 + variance) && value < expected * (1.0 - variance); +} + +// generates a random gossipsub message with sequence number i +fn make_test_message(seq: u64) -> (MessageId, RawGossipsubMessage) { + let raw_message = RawGossipsubMessage { + source: Some(PeerId::random()), + data: vec![12, 34, 56], + sequence_number: Some(seq), + topic: Topic::new("test").hash(), + signature: None, + key: None, + validated: true, + }; + + let message = GossipsubMessage { + source: raw_message.source.clone(), + data: raw_message.data.clone(), + sequence_number: raw_message.sequence_number, + topic: raw_message.topic.clone(), + }; + + let id = default_message_id()(&message); + (id, raw_message) +} + +fn default_message_id() -> fn(&GossipsubMessage) -> MessageId { + |message| { + // default message id is: source + sequence number + // NOTE: If either the peer_id or source is not provided, we set to 0; + let mut source_string = if let Some(peer_id) = message.source.as_ref() { + peer_id.to_base58() + } else { + PeerId::from_bytes(&vec![0, 1, 0]) + .expect("Valid peer id") + .to_base58() + }; + source_string.push_str(&message.sequence_number.unwrap_or_default().to_string()); + MessageId::from(source_string) + } +} + +#[test] +fn test_score_time_in_mesh() { + // Create parameters with reasonable default values + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut params = PeerScoreParams::default(); + params.topic_score_cap = 1000.0; + + let mut topic_params = TopicScoreParams::default(); + topic_params.topic_weight = 0.5; + topic_params.time_in_mesh_weight = 1.0; + topic_params.time_in_mesh_quantum = Duration::from_millis(1); + topic_params.time_in_mesh_cap = 3600.0; + + params.topics.insert(topic_hash, topic_params.clone()); + + let peer_id = PeerId::random(); + + let mut peer_score = PeerScore::new(params); + // Peer score should start at 0 + peer_score.add_peer(peer_id.clone()); + + let score = peer_score.score(&peer_id); + assert!( + score == 0.0, + "expected score to start at zero. Score found: {}", + score + ); + + // The time in mesh depends on how long the peer has been grafted + peer_score.graft(&peer_id, topic); + let elapsed = topic_params.time_in_mesh_quantum * 200; + std::thread::sleep(elapsed); + peer_score.refresh_scores(); + + let score = peer_score.score(&peer_id); + let expected = topic_params.topic_weight + * topic_params.time_in_mesh_weight + * (elapsed.as_millis() / topic_params.time_in_mesh_quantum.as_millis()) as f64; + assert!( + score >= expected, + "The score: {} should be greater than or equal to: {}", + score, + expected + ); +} + +#[test] +fn test_score_time_in_mesh_cap() { + // Create parameters with reasonable default values + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut params = PeerScoreParams::default(); + + let mut topic_params = TopicScoreParams::default(); + topic_params.topic_weight = 0.5; + topic_params.time_in_mesh_weight = 1.0; + topic_params.time_in_mesh_quantum = Duration::from_millis(1); + topic_params.time_in_mesh_cap = 10.0; + + params.topics.insert(topic_hash, topic_params.clone()); + + let peer_id = PeerId::random(); + + let mut peer_score = PeerScore::new(params); + // Peer score should start at 0 + peer_score.add_peer(peer_id.clone()); + + let score = peer_score.score(&peer_id); + assert!( + score == 0.0, + "expected score to start at zero. Score found: {}", + score + ); + + // The time in mesh depends on how long the peer has been grafted + peer_score.graft(&peer_id, topic); + let elapsed = topic_params.time_in_mesh_quantum * 40; + std::thread::sleep(elapsed); + peer_score.refresh_scores(); + + let score = peer_score.score(&peer_id); + let expected = topic_params.topic_weight + * topic_params.time_in_mesh_weight + * topic_params.time_in_mesh_cap; + let variance = 0.5; + assert!( + within_variance(score, expected, variance), + "The score: {} should be within {} of {}", + score, + score * variance, + expected + ); +} + +#[test] +fn test_score_first_message_deliveries() { + // Create parameters with reasonable default values + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut params = PeerScoreParams::default(); + + let mut topic_params = TopicScoreParams::default(); + topic_params.topic_weight = 1.0; + topic_params.first_message_deliveries_weight = 1.0; + topic_params.first_message_deliveries_decay = 1.0; + topic_params.first_message_deliveries_cap = 2000.0; + topic_params.time_in_mesh_weight = 0.0; + + params.topics.insert(topic_hash, topic_params.clone()); + + let peer_id = PeerId::random(); + + let mut peer_score = PeerScore::new(params); + // Peer score should start at 0 + peer_score.add_peer(peer_id.clone()); + peer_score.graft(&peer_id, topic); + + // deliver a bunch of messages from the peer + let messages = 100; + for seq in 0..messages { + let (id, msg) = make_test_message(seq); + peer_score.validate_message(&peer_id, &id, &msg.topic); + peer_score.deliver_message(&peer_id, &id, &msg.topic); + } + + peer_score.refresh_scores(); + + let score = peer_score.score(&peer_id); + let expected = + topic_params.topic_weight * topic_params.first_message_deliveries_weight * messages as f64; + assert!( + score == expected, + "The score: {} should be {}", + score, + expected + ); +} + +#[test] +fn test_score_first_message_deliveries_cap() { + // Create parameters with reasonable default values + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut params = PeerScoreParams::default(); + + let mut topic_params = TopicScoreParams::default(); + topic_params.topic_weight = 1.0; + topic_params.first_message_deliveries_weight = 1.0; + topic_params.first_message_deliveries_decay = 1.0; // test without decay + topic_params.first_message_deliveries_cap = 50.0; + topic_params.time_in_mesh_weight = 0.0; + + params.topics.insert(topic_hash, topic_params.clone()); + + let peer_id = PeerId::random(); + + let mut peer_score = PeerScore::new(params); + // Peer score should start at 0 + peer_score.add_peer(peer_id.clone()); + peer_score.graft(&peer_id, topic); + + // deliver a bunch of messages from the peer + let messages = 100; + for seq in 0..messages { + let (id, msg) = make_test_message(seq); + peer_score.validate_message(&peer_id, &id, &msg.topic); + peer_score.deliver_message(&peer_id, &id, &msg.topic); + } + + peer_score.refresh_scores(); + let score = peer_score.score(&peer_id); + let expected = topic_params.topic_weight + * topic_params.first_message_deliveries_weight + * topic_params.first_message_deliveries_cap; + assert!( + score == expected, + "The score: {} should be {}", + score, + expected + ); +} + +#[test] +fn test_score_first_message_deliveries_decay() { + // Create parameters with reasonable default values + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut params = PeerScoreParams::default(); + + let mut topic_params = TopicScoreParams::default(); + topic_params.topic_weight = 1.0; + topic_params.first_message_deliveries_weight = 1.0; + topic_params.first_message_deliveries_decay = 0.9; // decay 10% per decay interval + topic_params.first_message_deliveries_cap = 2000.0; + topic_params.time_in_mesh_weight = 0.0; + + params.topics.insert(topic_hash, topic_params.clone()); + let peer_id = PeerId::random(); + let mut peer_score = PeerScore::new(params); + peer_score.add_peer(peer_id.clone()); + peer_score.graft(&peer_id, topic); + + // deliver a bunch of messages from the peer + let messages = 100; + for seq in 0..messages { + let (id, msg) = make_test_message(seq); + peer_score.validate_message(&peer_id, &id, &msg.topic); + peer_score.deliver_message(&peer_id, &id, &msg.topic); + } + + peer_score.refresh_scores(); + let score = peer_score.score(&peer_id); + let mut expected = topic_params.topic_weight + * topic_params.first_message_deliveries_weight + * topic_params.first_message_deliveries_decay + * messages as f64; + assert!( + score == expected, + "The score: {} should be {}", + score, + expected + ); + + // refreshing the scores applies the decay param + let decay_intervals = 10; + for _ in 0..decay_intervals { + peer_score.refresh_scores(); + expected *= topic_params.first_message_deliveries_decay; + } + let score = peer_score.score(&peer_id); + assert!( + score == expected, + "The score: {} should be {}", + score, + expected + ); +} + +#[test] +fn test_score_mesh_message_deliveries() { + // Create parameters with reasonable default values + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut params = PeerScoreParams::default(); + + let mut topic_params = TopicScoreParams::default(); + topic_params.topic_weight = 1.0; + topic_params.mesh_message_deliveries_weight = -1.0; + topic_params.mesh_message_deliveries_activation = Duration::from_secs(1); + topic_params.mesh_message_deliveries_window = Duration::from_millis(10); + topic_params.mesh_message_deliveries_threshold = 20.0; + topic_params.mesh_message_deliveries_cap = 100.0; + topic_params.mesh_message_deliveries_decay = 1.0; + topic_params.first_message_deliveries_weight = 0.0; + topic_params.time_in_mesh_weight = 0.0; + topic_params.mesh_failure_penalty_weight = 0.0; + + params.topics.insert(topic_hash, topic_params.clone()); + let mut peer_score = PeerScore::new(params); + + // peer A always delivers the message first. + // peer B delivers next (within the delivery window). + // peer C delivers outside the delivery window. + // we expect peers A and B to have a score of zero, since all other parameter weights are zero. + // Peer C should have a negative score. + let peer_id_a = PeerId::random(); + let peer_id_b = PeerId::random(); + let peer_id_c = PeerId::random(); + + let peers = vec![peer_id_a.clone(), peer_id_b.clone(), peer_id_c.clone()]; + + for peer_id in &peers { + peer_score.add_peer(peer_id.clone()); + peer_score.graft(&peer_id, topic.clone()); + } + + // assert that nobody has been penalized yet for not delivering messages before activation time + peer_score.refresh_scores(); + for peer_id in &peers { + let score = peer_score.score(peer_id); + assert!( + score >= 0.0, + "expected no mesh delivery penalty before activation time, got score {}", + score + ); + } + + // wait for the activation time to kick in + std::thread::sleep(topic_params.mesh_message_deliveries_activation); + + // deliver a bunch of messages from peer A, with duplicates within the window from peer B, + // and duplicates outside the window from peer C. + let messages = 100; + let mut messages_to_send = Vec::new(); + for seq in 0..messages { + let (id, msg) = make_test_message(seq); + peer_score.validate_message(&peer_id_a, &id, &msg.topic); + peer_score.deliver_message(&peer_id_a, &id, &msg.topic); + + peer_score.duplicated_message(&peer_id_b, &id, &msg.topic); + messages_to_send.push((id, msg)); + } + + std::thread::sleep(topic_params.mesh_message_deliveries_window + Duration::from_millis(20)); + + for (id, msg) in messages_to_send { + peer_score.duplicated_message(&peer_id_c, &id, &msg.topic); + } + + peer_score.refresh_scores(); + let score_a = peer_score.score(&peer_id_a); + let score_b = peer_score.score(&peer_id_b); + let score_c = peer_score.score(&peer_id_c); + + assert!( + score_a >= 0.0, + "expected non-negative score for Peer A, got score {}", + score_a + ); + assert!( + score_b >= 0.0, + "expected non-negative score for Peer B, got score {}", + score_b + ); + + // the penalty is the difference between the threshold and the actual mesh deliveries, squared. + // since we didn't deliver anything, this is just the value of the threshold + let penalty = topic_params.mesh_message_deliveries_threshold + * topic_params.mesh_message_deliveries_threshold; + let expected = + topic_params.topic_weight * topic_params.mesh_message_deliveries_weight * penalty; + + assert!( + score_c == expected, + "Score: {}. Expected {}", + score_c, + expected + ); +} + +#[test] +fn test_score_mesh_message_deliveries_decay() { + // Create parameters with reasonable default values + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut params = PeerScoreParams::default(); + + let mut topic_params = TopicScoreParams::default(); + topic_params.topic_weight = 1.0; + topic_params.mesh_message_deliveries_weight = -1.0; + topic_params.mesh_message_deliveries_activation = Duration::from_secs(0); + topic_params.mesh_message_deliveries_window = Duration::from_millis(10); + topic_params.mesh_message_deliveries_threshold = 20.0; + topic_params.mesh_message_deliveries_cap = 100.0; + topic_params.mesh_message_deliveries_decay = 0.9; + topic_params.first_message_deliveries_weight = 0.0; + topic_params.time_in_mesh_weight = 0.0; + topic_params.time_in_mesh_quantum = Duration::from_secs(1); + topic_params.mesh_failure_penalty_weight = 0.0; + + params.topics.insert(topic_hash, topic_params.clone()); + let mut peer_score = PeerScore::new(params); + + let peer_id_a = PeerId::random(); + peer_score.add_peer(peer_id_a.clone()); + peer_score.graft(&peer_id_a, topic.clone()); + + // deliver a bunch of messages from peer A + let messages = 100; + for seq in 0..messages { + let (id, msg) = make_test_message(seq); + peer_score.validate_message(&peer_id_a, &id, &msg.topic); + peer_score.deliver_message(&peer_id_a, &id, &msg.topic); + } + + // we should have a positive score, since we delivered more messages than the threshold + peer_score.refresh_scores(); + + let score_a = peer_score.score(&peer_id_a); + assert!( + score_a >= 0.0, + "expected non-negative score for Peer A, got score {}", + score_a + ); + + let mut decayed_delivery_count = (messages as f64) * topic_params.mesh_message_deliveries_decay; + for _ in 0..20 { + peer_score.refresh_scores(); + decayed_delivery_count *= topic_params.mesh_message_deliveries_decay; + } + + let score_a = peer_score.score(&peer_id_a); + // the penalty is the difference between the threshold and the (decayed) mesh deliveries, squared. + let deficit = topic_params.mesh_message_deliveries_threshold - decayed_delivery_count; + let penalty = deficit * deficit; + let expected = + topic_params.topic_weight * topic_params.mesh_message_deliveries_weight * penalty; + + assert_eq!(score_a, expected, "Invalid score"); +} + +#[test] +fn test_score_mesh_failure_penalty() { + // Create parameters with reasonable default values + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut params = PeerScoreParams::default(); + + let mut topic_params = TopicScoreParams::default(); + // the mesh failure penalty is applied when a peer is pruned while their + // mesh deliveries are under the threshold. + // for this test, we set the mesh delivery threshold, but set + // mesh_message_deliveries to zero, so the only affect on the score + // is from the mesh failure penalty + topic_params.topic_weight = 1.0; + topic_params.mesh_message_deliveries_weight = 0.0; + topic_params.mesh_message_deliveries_activation = Duration::from_secs(0); + topic_params.mesh_message_deliveries_window = Duration::from_millis(10); + topic_params.mesh_message_deliveries_threshold = 20.0; + topic_params.mesh_message_deliveries_cap = 100.0; + topic_params.mesh_message_deliveries_decay = 1.0; + topic_params.first_message_deliveries_weight = 0.0; + topic_params.time_in_mesh_weight = 0.0; + topic_params.time_in_mesh_quantum = Duration::from_secs(1); + topic_params.mesh_failure_penalty_weight = -1.0; + topic_params.mesh_failure_penalty_decay = 1.0; + + params.topics.insert(topic_hash, topic_params.clone()); + let mut peer_score = PeerScore::new(params); + + let peer_id_a = PeerId::random(); + let peer_id_b = PeerId::random(); + + let peers = vec![peer_id_a.clone(), peer_id_b.clone()]; + + for peer_id in &peers { + peer_score.add_peer(peer_id.clone()); + peer_score.graft(&peer_id, topic.clone()); + } + + // deliver a bunch of messages from peer A + let messages = 100; + for seq in 0..messages { + let (id, msg) = make_test_message(seq); + + peer_score.validate_message(&peer_id_a, &id, &msg.topic); + peer_score.deliver_message(&peer_id_a, &id, &msg.topic); + } + + // peers A and B should both have zero scores, since the failure penalty hasn't been applied yet + peer_score.refresh_scores(); + let score_a = peer_score.score(&peer_id_a); + let score_b = peer_score.score(&peer_id_b); + assert!( + score_a >= 0.0, + "expected non-negative score for Peer A, got score {}", + score_a + ); + assert!( + score_b >= 0.0, + "expected non-negative score for Peer B, got score {}", + score_b + ); + + // prune peer B to apply the penalty + peer_score.prune(&peer_id_b, topic.hash()); + peer_score.refresh_scores(); + let score_a = peer_score.score(&peer_id_a); + + assert_eq!(score_a, 0.0, "expected Peer A to have a 0"); + + // penalty calculation is the same as for mesh_message_deliveries, but multiplied by + // mesh_failure_penalty_weigh + // instead of mesh_message_deliveries_weight + let penalty = topic_params.mesh_message_deliveries_threshold + * topic_params.mesh_message_deliveries_threshold; + let expected = topic_params.topic_weight * topic_params.mesh_failure_penalty_weight * penalty; + + let score_b = peer_score.score(&peer_id_b); + + assert_eq!(score_b, expected, "Peer B should have expected score",); +} + +#[test] +fn test_score_invalid_message_deliveries() { + // Create parameters with reasonable default values + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut params = PeerScoreParams::default(); + + let mut topic_params = TopicScoreParams::default(); + topic_params.topic_weight = 1.0; + topic_params.mesh_message_deliveries_weight = 0.0; + topic_params.mesh_message_deliveries_activation = Duration::from_secs(1); + topic_params.mesh_message_deliveries_window = Duration::from_millis(10); + topic_params.mesh_message_deliveries_threshold = 20.0; + topic_params.mesh_message_deliveries_cap = 100.0; + topic_params.mesh_message_deliveries_decay = 1.0; + topic_params.first_message_deliveries_weight = 0.0; + topic_params.time_in_mesh_weight = 0.0; + topic_params.mesh_failure_penalty_weight = 0.0; + + topic_params.invalid_message_deliveries_weight = -1.0; + topic_params.invalid_message_deliveries_decay = 1.0; + + params.topics.insert(topic_hash, topic_params.clone()); + let mut peer_score = PeerScore::new(params); + + let peer_id_a = PeerId::random(); + peer_score.add_peer(peer_id_a.clone()); + peer_score.graft(&peer_id_a, topic.clone()); + + // reject a bunch of messages from peer A + let messages = 100; + for seq in 0..messages { + let (id, msg) = make_test_message(seq); + peer_score.reject_message(&peer_id_a, &id, &msg.topic, RejectReason::ValidationFailed); + } + + peer_score.refresh_scores(); + let score_a = peer_score.score(&peer_id_a); + + let expected = topic_params.topic_weight + * topic_params.invalid_message_deliveries_weight + * (messages * messages) as f64; + + assert_eq!(score_a, expected, "Peer has unexpected score",); +} + +#[test] +fn test_score_invalid_message_deliveris_decay() { + // Create parameters with reasonable default values + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut params = PeerScoreParams::default(); + + let mut topic_params = TopicScoreParams::default(); + topic_params.topic_weight = 1.0; + topic_params.mesh_message_deliveries_weight = 0.0; + topic_params.mesh_message_deliveries_activation = Duration::from_secs(1); + topic_params.mesh_message_deliveries_window = Duration::from_millis(10); + topic_params.mesh_message_deliveries_threshold = 20.0; + topic_params.mesh_message_deliveries_cap = 100.0; + topic_params.mesh_message_deliveries_decay = 1.0; + topic_params.first_message_deliveries_weight = 0.0; + topic_params.time_in_mesh_weight = 0.0; + topic_params.mesh_failure_penalty_weight = 0.0; + + topic_params.invalid_message_deliveries_weight = -1.0; + topic_params.invalid_message_deliveries_decay = 0.9; + + params.topics.insert(topic_hash, topic_params.clone()); + let mut peer_score = PeerScore::new(params); + + let peer_id_a = PeerId::random(); + peer_score.add_peer(peer_id_a.clone()); + peer_score.graft(&peer_id_a, topic.clone()); + + // reject a bunch of messages from peer A + let messages = 100; + for seq in 0..messages { + let (id, msg) = make_test_message(seq); + peer_score.reject_message(&peer_id_a, &id, &msg.topic, RejectReason::ValidationFailed); + } + + peer_score.refresh_scores(); + + let decay = topic_params.invalid_message_deliveries_decay * messages as f64; + + let mut expected = + topic_params.topic_weight * topic_params.invalid_message_deliveries_weight * decay * decay; + + let score_a = peer_score.score(&peer_id_a); + assert_eq!(score_a, expected, "Peer has unexpected score"); + + // refresh scores a few times to apply decay + for _ in 0..10 { + peer_score.refresh_scores(); + expected *= topic_params.invalid_message_deliveries_decay + * topic_params.invalid_message_deliveries_decay; + } + + let score_a = peer_score.score(&peer_id_a); + assert_eq!(score_a, expected, "Peer has unexpected score"); +} + +#[test] +fn test_score_reject_message_deliveries() { + // This tests adds coverage for the dark corners of rejection tracing + + // Create parameters with reasonable default values + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut params = PeerScoreParams::default(); + + let mut topic_params = TopicScoreParams::default(); + topic_params.topic_weight = 1.0; + topic_params.mesh_message_deliveries_weight = 0.0; + topic_params.first_message_deliveries_weight = 0.0; + topic_params.mesh_failure_penalty_weight = 0.0; + topic_params.time_in_mesh_weight = 0.0; + topic_params.time_in_mesh_quantum = Duration::from_secs(1); + topic_params.invalid_message_deliveries_weight = -1.0; + topic_params.invalid_message_deliveries_decay = 1.0; + + params.topics.insert(topic_hash, topic_params.clone()); + let mut peer_score = PeerScore::new(params); + + let peer_id_a = PeerId::random(); + let peer_id_b = PeerId::random(); + + let peers = vec![peer_id_a.clone(), peer_id_b.clone()]; + + for peer_id in &peers { + peer_score.add_peer(peer_id.clone()); + } + + let (id, msg) = make_test_message(1); + + // these should have no effect in the score + peer_score.reject_message(&peer_id_a, &id, &msg.topic, RejectReason::BlackListedPeer); + peer_score.reject_message(&peer_id_a, &id, &msg.topic, RejectReason::BlackListedSource); + peer_score.reject_message(&peer_id_a, &id, &msg.topic, RejectReason::ValidationIgnored); + + peer_score.refresh_scores(); + let score_a = peer_score.score(&peer_id_a); + let score_b = peer_score.score(&peer_id_b); + + assert_eq!(score_a, 0.0, "Should have no effect on the score"); + assert_eq!(score_b, 0.0, "Should have no effect on the score"); + + // insert a record in the message deliveries + peer_score.validate_message(&peer_id_a, &id, &msg.topic); + + // this should have no effect in the score, and subsequent duplicate messages should have no + // effect either + peer_score.reject_message(&peer_id_a, &id, &msg.topic, RejectReason::ValidationIgnored); + peer_score.duplicated_message(&peer_id_b, &id, &msg.topic); + + peer_score.refresh_scores(); + let score_a = peer_score.score(&peer_id_a); + let score_b = peer_score.score(&peer_id_b); + + assert_eq!(score_a, 0.0, "Should have no effect on the score"); + assert_eq!(score_b, 0.0, "Should have no effect on the score"); + + // now clear the delivery record + peer_score.deliveries.clear(); + + // insert a record in the message deliveries + peer_score.validate_message(&peer_id_a, &id, &msg.topic); + + // this should have no effect in the score, and subsequent duplicate messages should have no + // effect either + peer_score.reject_message(&peer_id_a, &id, &msg.topic, RejectReason::ValidationIgnored); + peer_score.duplicated_message(&peer_id_b, &id, &msg.topic); + + peer_score.refresh_scores(); + let score_a = peer_score.score(&peer_id_a); + let score_b = peer_score.score(&peer_id_b); + + assert_eq!(score_a, 0.0, "Should have no effect on the score"); + assert_eq!(score_b, 0.0, "Should have no effect on the score"); + + // now clear the delivery record + peer_score.deliveries.clear(); + + // insert a new record in the message deliveries + peer_score.validate_message(&peer_id_a, &id, &msg.topic); + + // and reject the message to make sure duplicates are also penalized + peer_score.reject_message(&peer_id_a, &id, &msg.topic, RejectReason::ValidationFailed); + peer_score.duplicated_message(&peer_id_b, &id, &msg.topic); + + peer_score.refresh_scores(); + let score_a = peer_score.score(&peer_id_a); + let score_b = peer_score.score(&peer_id_b); + + assert_eq!(score_a, -1.0, "Score should be effected"); + assert_eq!(score_b, -1.0, "Score should be effected"); + + // now clear the delivery record again + peer_score.deliveries.clear(); + + // insert a new record in the message deliveries + peer_score.validate_message(&peer_id_a, &id, &msg.topic); + + // and reject the message after a duplicate has arrived + peer_score.duplicated_message(&peer_id_b, &id, &msg.topic); + peer_score.reject_message(&peer_id_a, &id, &msg.topic, RejectReason::ValidationFailed); + + peer_score.refresh_scores(); + let score_a = peer_score.score(&peer_id_a); + let score_b = peer_score.score(&peer_id_b); + + assert_eq!(score_a, -4.0, "Score should be effected"); + assert_eq!(score_b, -4.0, "Score should be effected"); +} + +#[test] +fn test_application_score() { + // Create parameters with reasonable default values + let app_specific_weight = 0.5; + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut params = PeerScoreParams::default(); + params.app_specific_weight = app_specific_weight; + + let mut topic_params = TopicScoreParams::default(); + topic_params.topic_weight = 1.0; + topic_params.mesh_message_deliveries_weight = 0.0; + topic_params.first_message_deliveries_weight = 0.0; + topic_params.mesh_failure_penalty_weight = 0.0; + topic_params.time_in_mesh_weight = 0.0; + topic_params.time_in_mesh_quantum = Duration::from_secs(1); + topic_params.invalid_message_deliveries_weight = 0.0; + topic_params.invalid_message_deliveries_decay = 1.0; + + params.topics.insert(topic_hash, topic_params.clone()); + let mut peer_score = PeerScore::new(params); + + let peer_id_a = PeerId::random(); + peer_score.add_peer(peer_id_a.clone()); + peer_score.graft(&peer_id_a, topic.clone()); + + let messages = 100; + for i in -100..messages { + let app_score_value = i as f64; + peer_score.set_application_score(&peer_id_a, app_score_value); + peer_score.refresh_scores(); + let score_a = peer_score.score(&peer_id_a); + let expected = (i as f64) * app_specific_weight; + assert_eq!(score_a, expected, "Peer has unexpected score"); + } +} + +#[test] +fn test_score_ip_colocation() { + // Create parameters with reasonable default values + let ip_colocation_factor_weight = -1.0; + let ip_colocation_factor_threshold = 1.0; + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut params = PeerScoreParams::default(); + params.ip_colocation_factor_weight = ip_colocation_factor_weight; + params.ip_colocation_factor_threshold = ip_colocation_factor_threshold; + + let mut topic_params = TopicScoreParams::default(); + topic_params.topic_weight = 1.0; + topic_params.mesh_message_deliveries_weight = 0.0; + topic_params.first_message_deliveries_weight = 0.0; + topic_params.mesh_failure_penalty_weight = 0.0; + topic_params.time_in_mesh_weight = 0.0; + topic_params.time_in_mesh_quantum = Duration::from_secs(1); + topic_params.invalid_message_deliveries_weight = 0.0; + + params.topics.insert(topic_hash, topic_params.clone()); + let mut peer_score = PeerScore::new(params); + + let peer_id_a = PeerId::random(); + let peer_id_b = PeerId::random(); + let peer_id_c = PeerId::random(); + let peer_id_d = PeerId::random(); + + let peers = vec![ + peer_id_a.clone(), + peer_id_b.clone(), + peer_id_c.clone(), + peer_id_d.clone(), + ]; + for peer_id in &peers { + peer_score.add_peer(peer_id.clone()); + peer_score.graft(&peer_id, topic.clone()); + } + + // peerA should have no penalty, but B, C, and D should be penalized for sharing an IP + peer_score.add_ip(&peer_id_a, "1.2.3.4".parse().unwrap()); + peer_score.add_ip(&peer_id_b, "2.3.4.5".parse().unwrap()); + peer_score.add_ip(&peer_id_c, "2.3.4.5".parse().unwrap()); + peer_score.add_ip(&peer_id_c, "3.4.5.6".parse().unwrap()); + peer_score.add_ip(&peer_id_d, "2.3.4.5".parse().unwrap()); + + peer_score.refresh_scores(); + let score_a = peer_score.score(&peer_id_a); + let score_b = peer_score.score(&peer_id_b); + let score_c = peer_score.score(&peer_id_c); + let score_d = peer_score.score(&peer_id_d); + + assert_eq!(score_a, 0.0, "Peer A should be unaffected"); + + let n_shared = 3.0; + let ip_surplus = n_shared - ip_colocation_factor_threshold; + let penalty = ip_surplus * ip_surplus; + let expected = ip_colocation_factor_weight * penalty as f64; + + assert_eq!(score_b, expected, "Peer B should have expected score"); + assert_eq!(score_c, expected, "Peer C should have expected score"); + assert_eq!(score_d, expected, "Peer D should have expected score"); +} + +#[test] +fn test_score_behaviour_penality() { + // Create parameters with reasonable default values + let behaviour_penalty_weight = -1.0; + let behaviour_penalty_decay = 0.99; + + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let mut params = PeerScoreParams::default(); + params.behaviour_penalty_decay = behaviour_penalty_decay; + params.behaviour_penalty_weight = behaviour_penalty_weight; + + let mut topic_params = TopicScoreParams::default(); + topic_params.topic_weight = 1.0; + topic_params.mesh_message_deliveries_weight = 0.0; + topic_params.first_message_deliveries_weight = 0.0; + topic_params.mesh_failure_penalty_weight = 0.0; + topic_params.time_in_mesh_weight = 0.0; + topic_params.time_in_mesh_quantum = Duration::from_secs(1); + topic_params.invalid_message_deliveries_weight = 0.0; + + params.topics.insert(topic_hash, topic_params.clone()); + let mut peer_score = PeerScore::new(params); + + let peer_id_a = PeerId::random(); + + // add a penalty to a non-existent peer. + peer_score.add_penalty(&peer_id_a, 1); + + let score_a = peer_score.score(&peer_id_a); + assert_eq!(score_a, 0.0, "Peer A should be unaffected"); + + // add the peer and test penalties + peer_score.add_peer(peer_id_a.clone()); + assert_eq!(score_a, 0.0, "Peer A should be unaffected"); + + peer_score.add_penalty(&peer_id_a, 1); + + let score_a = peer_score.score(&peer_id_a); + assert_eq!(score_a, -1.0, "Peer A should have been penalized"); + + peer_score.add_penalty(&peer_id_a, 1); + let score_a = peer_score.score(&peer_id_a); + assert_eq!(score_a, -4.0, "Peer A should have been penalized"); + + peer_score.refresh_scores(); + + let score_a = peer_score.score(&peer_id_a); + assert_eq!(score_a, -3.9204, "Peer A should have been penalized"); +} + +#[test] +fn test_score_retention() { + // Create parameters with reasonable default values + let topic = Topic::new("test"); + let topic_hash = topic.hash(); + let app_specific_weight = 1.0; + let app_score_value = -1000.0; + let retain_score = Duration::from_secs(1); + let mut params = PeerScoreParams::default(); + params.app_specific_weight = app_specific_weight; + params.retain_score = retain_score; + + let mut topic_params = TopicScoreParams::default(); + topic_params.topic_weight = 0.0; + topic_params.mesh_message_deliveries_weight = 0.0; + topic_params.mesh_message_deliveries_activation = Duration::from_secs(0); + topic_params.first_message_deliveries_weight = 0.0; + topic_params.time_in_mesh_weight = 0.0; + + params.topics.insert(topic_hash, topic_params.clone()); + let mut peer_score = PeerScore::new(params); + + let peer_id_a = PeerId::random(); + peer_score.add_peer(peer_id_a.clone()); + peer_score.graft(&peer_id_a, topic.clone()); + + peer_score.set_application_score(&peer_id_a, app_score_value); + + // score should equal -1000 (app specific score) + peer_score.refresh_scores(); + let score_a = peer_score.score(&peer_id_a); + assert_eq!( + score_a, app_score_value, + "Score should be the application specific score" + ); + + // disconnect & wait half of RetainScore time. Should still have negative score + peer_score.remove_peer(&peer_id_a); + std::thread::sleep(retain_score / 2); + peer_score.refresh_scores(); + let score_a = peer_score.score(&peer_id_a); + assert_eq!( + score_a, app_score_value, + "Score should be the application specific score" + ); + + // wait remaining time (plus a little slop) and the score should reset to zero + std::thread::sleep(retain_score / 2 + Duration::from_millis(50)); + peer_score.refresh_scores(); + let score_a = peer_score.score(&peer_id_a); + assert_eq!( + score_a, 0.0, + "Score should be the application specific score" + ); +} diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index 7184c882ba6..d2505cb8352 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -18,29 +18,36 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::behaviour::GossipsubRpc; use crate::config::ValidationMode; +use crate::error::{GossipsubHandlerError, ValidationError}; +use crate::handler::HandlerEvent; use crate::rpc_proto; use crate::topic::TopicHash; +use crate::types::{ + GossipsubControlAction, GossipsubRpc, GossipsubSubscription, GossipsubSubscriptionAction, + MessageId, PeerInfo, PeerKind, RawGossipsubMessage, +}; use byteorder::{BigEndian, ByteOrder}; use bytes::Bytes; use bytes::BytesMut; use futures::future; use futures::prelude::*; use futures_codec::{Decoder, Encoder, Framed}; -use libp2p_core::{identity::PublicKey, InboundUpgrade, OutboundUpgrade, PeerId, UpgradeInfo}; +use libp2p_core::{ + identity::PublicKey, InboundUpgrade, OutboundUpgrade, PeerId, ProtocolName, UpgradeInfo, +}; use log::{debug, warn}; use prost::Message as ProtobufMessage; -use std::{borrow::Cow, fmt, io, iter, pin::Pin}; +use std::{borrow::Cow, pin::Pin}; use unsigned_varint::codec; -pub const SIGNING_PREFIX: &'static [u8] = b"libp2p-pubsub:"; +pub(crate) const SIGNING_PREFIX: &[u8] = b"libp2p-pubsub:"; -/// Implementation of the `ConnectionUpgrade` for the Gossipsub protocol. +/// Implementation of [`InboundUpgrade`] and [`OutboundUpgrade`] for the Gossipsub protocol. #[derive(Clone)] pub struct ProtocolConfig { - /// The gossipsub protocol id to listen on. - protocol_id: Cow<'static, [u8]>, + /// The Gossipsub protocol id to listen on. + protocol_ids: Vec, /// The maximum transmit size for a packet. max_transmit_size: usize, /// Determines the level of validation to be done on incoming messages. @@ -48,27 +55,71 @@ pub struct ProtocolConfig { } impl ProtocolConfig { - /// Builds a new `ProtocolConfig`. + /// Builds a new [`ProtocolConfig`]. + /// /// Sets the maximum gossip transmission size. pub fn new( - protocol_id: impl Into>, + id_prefix: Cow<'static, str>, max_transmit_size: usize, validation_mode: ValidationMode, + support_floodsub: bool, ) -> ProtocolConfig { + // support version 1.1.0 and 1.0.0 with user-customized prefix + let mut protocol_ids = vec![ + ProtocolId::new(id_prefix.clone(), PeerKind::Gossipsubv1_1), + ProtocolId::new(id_prefix, PeerKind::Gossipsub), + ]; + + // add floodsub support if enabled. + if support_floodsub { + protocol_ids.push(ProtocolId::new(Cow::from(""), PeerKind::Floodsub)); + } + ProtocolConfig { - protocol_id: protocol_id.into(), + protocol_ids, max_transmit_size, validation_mode, } } } +/// The protocol ID +#[derive(Clone, Debug)] +pub struct ProtocolId { + /// The RPC message type/name. + pub protocol_id: Vec, + /// The type of protocol we support + pub kind: PeerKind, +} + +/// An RPC protocol ID. +impl ProtocolId { + pub fn new(prefix: Cow<'static, str>, kind: PeerKind) -> Self { + let protocol_id = match kind { + PeerKind::Gossipsubv1_1 => format!("/{}/{}", prefix, "1.1.0"), + PeerKind::Gossipsub => format!("/{}/{}", prefix, "1.0.0"), + PeerKind::Floodsub => format!("/{}/{}", "floodsub", "1.0.0"), + // NOTE: This is used for informing the behaviour of unsupported peers. We do not + // advertise this variant. + PeerKind::NotSupported => unreachable!("Should never advertise NotSupported"), + } + .into_bytes(); + ProtocolId { protocol_id, kind } + } +} + +impl ProtocolName for ProtocolId { + fn protocol_name(&self) -> &[u8] { + &self.protocol_id + } +} + impl UpgradeInfo for ProtocolConfig { - type Info = Cow<'static, [u8]>; - type InfoIter = iter::Once; + type Info = ProtocolId; + type InfoIter = Vec; fn protocol_info(&self) -> Self::InfoIter { - iter::once(self.protocol_id.clone()) + self.protocol_ids.clone() } } @@ -76,16 +127,19 @@ impl InboundUpgrade for ProtocolConfig where TSocket: AsyncRead + AsyncWrite + Unpin + Send + 'static, { - type Output = Framed; - type Error = io::Error; + type Output = (Framed, PeerKind); + type Error = GossipsubHandlerError; type Future = Pin> + Send>>; - fn upgrade_inbound(self, socket: TSocket, _: Self::Info) -> Self::Future { + fn upgrade_inbound(self, socket: TSocket, protocol_id: Self::Info) -> Self::Future { let mut length_codec = codec::UviBytes::default(); length_codec.set_max_len(self.max_transmit_size); - Box::pin(future::ok(Framed::new( - socket, - GossipsubCodec::new(length_codec, self.validation_mode), + Box::pin(future::ok(( + Framed::new( + socket, + GossipsubCodec::new(length_codec, self.validation_mode), + ), + protocol_id.kind, ))) } } @@ -94,16 +148,19 @@ impl OutboundUpgrade for ProtocolConfig where TSocket: AsyncWrite + AsyncRead + Unpin + Send + 'static, { - type Output = Framed; - type Error = io::Error; + type Output = (Framed, PeerKind); + type Error = GossipsubHandlerError; type Future = Pin> + Send>>; - fn upgrade_outbound(self, socket: TSocket, _: Self::Info) -> Self::Future { + fn upgrade_outbound(self, socket: TSocket, protocol_id: Self::Info) -> Self::Future { let mut length_codec = codec::UviBytes::default(); length_codec.set_max_len(self.max_transmit_size); - Box::pin(future::ok(Framed::new( - socket, - GossipsubCodec::new(length_codec, self.validation_mode), + Box::pin(future::ok(( + Framed::new( + socket, + GossipsubCodec::new(length_codec, self.validation_mode), + ), + protocol_id.kind, ))) } } @@ -191,113 +248,48 @@ impl GossipsubCodec { } impl Encoder for GossipsubCodec { - type Item = GossipsubRpc; - type Error = io::Error; + type Item = rpc_proto::Rpc; + type Error = GossipsubHandlerError; fn encode(&mut self, item: Self::Item, dst: &mut BytesMut) -> Result<(), Self::Error> { - // Messages - let mut publish = Vec::new(); - - for message in item.messages.into_iter() { - let message = rpc_proto::Message { - from: message.source.map(|m| m.to_bytes()), - data: Some(message.data), - seqno: message.sequence_number.map(|s| s.to_be_bytes().to_vec()), - topic_ids: message.topics.into_iter().map(TopicHash::into).collect(), - signature: message.signature, - key: message.key, - }; - publish.push(message); - } - - // subscriptions - let subscriptions = item - .subscriptions - .into_iter() - .map(|sub| rpc_proto::rpc::SubOpts { - subscribe: Some(sub.action == GossipsubSubscriptionAction::Subscribe), - topic_id: Some(sub.topic_hash.into()), - }) - .collect::>(); - - // control messages - let mut control = rpc_proto::ControlMessage { - ihave: Vec::new(), - iwant: Vec::new(), - graft: Vec::new(), - prune: Vec::new(), - }; + let mut buf = Vec::with_capacity(item.encoded_len()); - let empty_control_msg = item.control_msgs.is_empty(); - - for action in item.control_msgs { - match action { - // collect all ihave messages - GossipsubControlAction::IHave { - topic_hash, - message_ids, - } => { - let rpc_ihave = rpc_proto::ControlIHave { - topic_id: Some(topic_hash.into()), - message_ids: message_ids.into_iter().map(|msg_id| msg_id.0).collect(), - }; - control.ihave.push(rpc_ihave); - } - GossipsubControlAction::IWant { message_ids } => { - let rpc_iwant = rpc_proto::ControlIWant { - message_ids: message_ids.into_iter().map(|msg_id| msg_id.0).collect(), - }; - control.iwant.push(rpc_iwant); - } - GossipsubControlAction::Graft { topic_hash } => { - let rpc_graft = rpc_proto::ControlGraft { - topic_id: Some(topic_hash.into()), - }; - control.graft.push(rpc_graft); - } - GossipsubControlAction::Prune { topic_hash } => { - let rpc_prune = rpc_proto::ControlPrune { - topic_id: Some(topic_hash.into()), - }; - control.prune.push(rpc_prune); - } - } - } - - let rpc = rpc_proto::Rpc { - subscriptions, - publish, - control: if empty_control_msg { - None - } else { - Some(control) - }, - }; - - let mut buf = Vec::with_capacity(rpc.encoded_len()); - - rpc.encode(&mut buf) + item.encode(&mut buf) .expect("Buffer has sufficient capacity"); // length prefix the protobuf message, ensuring the max limit is not hit - self.length_codec.encode(Bytes::from(buf), dst) + self.length_codec + .encode(Bytes::from(buf), dst) + .map_err(|_| GossipsubHandlerError::MaxTransmissionSize) } } impl Decoder for GossipsubCodec { - type Item = GossipsubRpc; - type Error = io::Error; + type Item = HandlerEvent; + type Error = GossipsubHandlerError; fn decode(&mut self, src: &mut BytesMut) -> Result, Self::Error> { - let packet = match self.length_codec.decode(src)? { + let packet = match self.length_codec.decode(src).map_err(|e| { + if let std::io::ErrorKind::PermissionDenied = e.kind() { + GossipsubHandlerError::MaxTransmissionSize + } else { + GossipsubHandlerError::Io(e) + } + })? { Some(p) => p, None => return Ok(None), }; - let rpc = rpc_proto::Rpc::decode(&packet[..])?; + let rpc = rpc_proto::Rpc::decode(&packet[..]).map_err(std::io::Error::from)?; + // Store valid messages. let mut messages = Vec::with_capacity(rpc.publish.len()); + // Store any invalid messages. + let mut invalid_messages = Vec::new(); + for message in rpc.publish.into_iter() { + // Keep track of the type of invalid message. + let mut invalid_kind = None; let mut verify_signature = false; let mut verify_sequence_no = false; let mut verify_source = false; @@ -323,72 +315,141 @@ impl Decoder for GossipsubCodec { } ValidationMode::Anonymous => { if message.signature.is_some() { - warn!("Message dropped. Signature field was non-empty and anonymous validation mode is set"); - return Ok(None); - } - if message.seqno.is_some() { - warn!("Message dropped. Sequence number was non-empty and anonymous validation mode is set"); - return Ok(None); - } - if message.from.is_some() { + warn!("Signature field was non-empty and anonymous validation mode is set"); + invalid_kind = Some(ValidationError::SignaturePresent); + } else if message.seqno.is_some() { + warn!("Sequence number was non-empty and anonymous validation mode is set"); + invalid_kind = Some(ValidationError::SequenceNumberPresent); + } else if message.from.is_some() { warn!("Message dropped. Message source was non-empty and anonymous validation mode is set"); - return Ok(None); + invalid_kind = Some(ValidationError::MessageSourcePresent); } } ValidationMode::None => {} } + // If the initial validation logic failed, add the message to invalid messages and + // continue processing the others. + if let Some(validation_error) = invalid_kind.take() { + let message = RawGossipsubMessage { + source: None, // don't bother inform the application + data: message.data.unwrap_or_default(), + sequence_number: None, // don't inform the application + topic: TopicHash::from_raw(message.topic), + signature: None, // don't inform the application + key: message.key, + validated: false, + }; + invalid_messages.push((message, validation_error)); + // proceed to the next message + continue; + } + // verify message signatures if required - if verify_signature { - // If a single message is unsigned, we will drop all of them - // Most implementations should not have a list of mixed signed/not-signed messages in a single RPC - // NOTE: Invalid messages are simply dropped with a warning log. We don't throw an - // error to avoid extra logic to deal with these errors in the handler. - if !GossipsubCodec::verify_signature(&message) { - warn!("Message dropped. Invalid signature"); - // Drop the message - return Ok(None); - } + if verify_signature && !GossipsubCodec::verify_signature(&message) { + warn!("Invalid signature for received message"); + + // Build the invalid message (ignoring further validation of sequence number + // and source) + let message = RawGossipsubMessage { + source: None, // don't bother inform the application + data: message.data.unwrap_or_default(), + sequence_number: None, // don't inform the application + topic: TopicHash::from_raw(message.topic), + signature: None, // don't inform the application + key: message.key, + validated: false, + }; + invalid_messages.push((message, ValidationError::InvalidSignature)); + // proceed to the next message + continue; } // ensure the sequence number is a u64 let sequence_number = if verify_sequence_no { - let seq_no = message.seqno.ok_or_else(|| { - io::Error::new( - io::ErrorKind::InvalidData, - "sequence number was not provided", - ) - })?; - if seq_no.len() != 8 { - return Err(io::Error::new( - io::ErrorKind::InvalidData, - "sequence number has an incorrect size", - )); + if let Some(seq_no) = message.seqno { + if seq_no.is_empty() { + None + } else if seq_no.len() != 8 { + debug!( + "Invalid sequence number length for received message. SeqNo: {:?} Size: {}", + seq_no, + seq_no.len() + ); + let message = RawGossipsubMessage { + source: None, // don't bother inform the application + data: message.data.unwrap_or_default(), + sequence_number: None, // don't inform the application + topic: TopicHash::from_raw(message.topic), + signature: message.signature, // don't inform the application + key: message.key, + validated: false, + }; + invalid_messages.push((message, ValidationError::InvalidSequenceNumber)); + // proceed to the next message + continue; + } else { + // valid sequence number + Some(BigEndian::read_u64(&seq_no)) + } + } else { + // sequence number was not present + debug!("Sequence number not present but expected"); + let message = RawGossipsubMessage { + source: None, // don't bother inform the application + data: message.data.unwrap_or_default(), + sequence_number: None, // don't inform the application + topic: TopicHash::from_raw(message.topic), + signature: message.signature, // don't inform the application + key: message.key, + validated: false, + }; + invalid_messages.push((message, ValidationError::EmptySequenceNumber)); + continue; } - Some(BigEndian::read_u64(&seq_no)) } else { + // Do not verify the sequence number, consider it empty None }; + // Verify the message source if required let source = if verify_source { - Some( - PeerId::from_bytes(&message.from.unwrap_or_default()).map_err(|_| { - io::Error::new(io::ErrorKind::InvalidData, "Invalid Peer Id") - })?, - ) + if let Some(bytes) = message.from { + if !bytes.is_empty() { + match PeerId::from_bytes(&bytes) { + Ok(peer_id) => Some(peer_id), // valid peer id + Err(_) => { + // invalid peer id, add to invalid messages + debug!("Message source has an invalid PeerId"); + let message = RawGossipsubMessage { + source: None, // don't bother inform the application + data: message.data.unwrap_or_default(), + sequence_number, + topic: TopicHash::from_raw(message.topic), + signature: message.signature, // don't inform the application + key: message.key, + validated: false, + }; + invalid_messages.push((message, ValidationError::InvalidPeerId)); + continue; + } + } + } else { + None + } + } else { + None + } } else { None }; - messages.push(GossipsubMessage { + // This message has passed all validation, add it to the validated messages. + messages.push(RawGossipsubMessage { source, data: message.data.unwrap_or_default(), sequence_number, - topics: message - .topic_ids - .into_iter() - .map(TopicHash::from_raw) - .collect(), + topic: TopicHash::from_raw(message.topic), signature: message.signature, key: message.key, validated: false, @@ -432,13 +493,32 @@ impl Decoder for GossipsubCodec { }) .collect(); - let prune_msgs: Vec = rpc_control - .prune - .into_iter() - .map(|prune| GossipsubControlAction::Prune { - topic_hash: TopicHash::from_raw(prune.topic_id.unwrap_or_default()), - }) - .collect(); + let mut prune_msgs = Vec::new(); + + for prune in rpc_control.prune { + // filter out invalid peers + let peers = prune + .peers + .into_iter() + .filter_map(|info| { + info.peer_id + .as_ref() + .and_then(|id| PeerId::from_bytes(id).ok()) + .map(|peer_id| + //TODO signedPeerRecord, see https://github.com/libp2p/specs/pull/217 + PeerInfo { + peer_id: Some(peer_id), + }) + }) + .collect::>(); + + let topic_hash = TopicHash::from_raw(prune.topic_id.unwrap_or_default()); + prune_msgs.push(GossipsubControlAction::Prune { + topic_hash, + peers, + backoff: prune.backoff, + }); + } control_msgs.extend(ihave_msgs); control_msgs.extend(iwant_msgs); @@ -446,147 +526,40 @@ impl Decoder for GossipsubCodec { control_msgs.extend(prune_msgs); } - Ok(Some(GossipsubRpc { - messages, - subscriptions: rpc - .subscriptions - .into_iter() - .map(|sub| GossipsubSubscription { - action: if Some(true) == sub.subscribe { - GossipsubSubscriptionAction::Subscribe - } else { - GossipsubSubscriptionAction::Unsubscribe - }, - topic_hash: TopicHash::from_raw(sub.topic_id.unwrap_or_default()), - }) - .collect(), - control_msgs, + Ok(Some(HandlerEvent::Message { + rpc: GossipsubRpc { + messages, + subscriptions: rpc + .subscriptions + .into_iter() + .map(|sub| GossipsubSubscription { + action: if Some(true) == sub.subscribe { + GossipsubSubscriptionAction::Subscribe + } else { + GossipsubSubscriptionAction::Unsubscribe + }, + topic_hash: TopicHash::from_raw(sub.topic_id.unwrap_or_default()), + }) + .collect(), + control_msgs, + }, + invalid_messages, })) } } -/// A type for gossipsub message ids. -#[derive(Clone, PartialEq, Eq, Hash, PartialOrd, Ord)] -pub struct MessageId(Vec); - -impl MessageId { - pub fn new(value: &[u8]) -> Self { - Self(value.to_vec()) - } -} - -impl>> From for MessageId { - fn from(value: T) -> Self { - Self(value.into()) - } -} - -impl std::fmt::Display for MessageId { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "{}", hex_fmt::HexFmt(&self.0)) - } -} - -impl std::fmt::Debug for MessageId { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "MessageId({})", hex_fmt::HexFmt(&self.0)) - } -} - -/// A message received by the gossipsub system. -#[derive(Clone, PartialEq, Eq, Hash)] -pub struct GossipsubMessage { - /// Id of the peer that published this message. - pub source: Option, - - /// Content of the message. Its meaning is out of scope of this library. - pub data: Vec, - - /// A random sequence number. - pub sequence_number: Option, - - /// List of topics this message belongs to. - /// - /// Each message can belong to multiple topics at once. - pub topics: Vec, - - /// The signature of the message if it's signed. - pub signature: Option>, - - /// The public key of the message if it is signed and the source `PeerId` cannot be inlined. - pub key: Option>, - - /// Flag indicating if this message has been validated by the application or not. - pub validated: bool, -} - -impl fmt::Debug for GossipsubMessage { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("GossipsubMessage") - .field("data",&format_args!("{:<20}", &hex_fmt::HexFmt(&self.data))) - .field("source", &self.source) - .field("sequence_number", &self.sequence_number) - .field("topics", &self.topics) - .finish() - } -} - -/// A subscription received by the gossipsub system. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub struct GossipsubSubscription { - /// Action to perform. - pub action: GossipsubSubscriptionAction, - /// The topic from which to subscribe or unsubscribe. - pub topic_hash: TopicHash, -} - -/// Action that a subscription wants to perform. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub enum GossipsubSubscriptionAction { - /// The remote wants to subscribe to the given topic. - Subscribe, - /// The remote wants to unsubscribe from the given topic. - Unsubscribe, -} - -/// A Control message received by the gossipsub system. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub enum GossipsubControlAction { - /// Node broadcasts known messages per topic - IHave control message. - IHave { - /// The topic of the messages. - topic_hash: TopicHash, - /// A list of known message ids (peer_id + sequence _number) as a string. - message_ids: Vec, - }, - /// The node requests specific message ids (peer_id + sequence _number) - IWant control message. - IWant { - /// A list of known message ids (peer_id + sequence _number) as a string. - message_ids: Vec, - }, - /// The node has been added to the mesh - Graft control message. - Graft { - /// The mesh topic the peer should be added to. - topic_hash: TopicHash, - }, - /// The node has been removed from the mesh - Prune control message. - Prune { - /// The mesh topic the peer should be removed from. - topic_hash: TopicHash, - }, -} - #[cfg(test)] mod tests { use super::*; - use crate::topic::Topic; - use crate::{Gossipsub, GossipsubConfig}; + use crate::config::GossipsubConfig; + use crate::Gossipsub; + use crate::IdentTopic as Topic; use libp2p_core::identity::Keypair; use quickcheck::*; use rand::Rng; #[derive(Clone, Debug)] - struct Message(GossipsubMessage); + struct Message(RawGossipsubMessage); impl Arbitrary for Message { fn arbitrary(g: &mut G) -> Self { @@ -594,16 +567,16 @@ mod tests { // generate an arbitrary GossipsubMessage using the behaviour signing functionality let config = GossipsubConfig::default(); - let gs = Gossipsub::new( + let gs: Gossipsub = Gossipsub::new( crate::MessageAuthenticity::Signed(keypair.0.clone()), config, - ); - let data = (0..g.gen_range(1, 1024)).map(|_| g.gen()).collect(); - let topics = Vec::arbitrary(g) - .into_iter() - .map(|id: TopicId| id.0) - .collect(); - Message(gs.build_message(topics, data).unwrap()) + ) + .unwrap(); + let data = (0..g.gen_range(10, 10024)) + .map(|_| g.gen()) + .collect::>(); + let topic_id = TopicId::arbitrary(g).0; + Message(gs.build_raw_message(topic_id, data).unwrap()) } } @@ -612,10 +585,11 @@ mod tests { impl Arbitrary for TopicId { fn arbitrary(g: &mut G) -> Self { - TopicId( - Topic::new((0..g.gen_range(0, 1024)).map(|_| g.gen::()).collect()) - .sha256_hash(), - ) + let topic_string: String = (0..g.gen_range(20, 1024)) + .map(|_| g.gen::()) + .collect::() + .into(); + TopicId(Topic::new(topic_string).into()) } } @@ -645,6 +619,7 @@ mod tests { } #[test] + /// Test that RPC messages can be encoded and decoded successfully. fn encode_decode() { fn prop(message: Message) { let message = message.0; @@ -657,12 +632,17 @@ mod tests { let mut codec = GossipsubCodec::new(codec::UviBytes::default(), ValidationMode::Strict); let mut buf = BytesMut::new(); - codec.encode(rpc.clone(), &mut buf).unwrap(); - let mut decoded_rpc = codec.decode(&mut buf).unwrap().unwrap(); + codec.encode(rpc.clone().into_protobuf(), &mut buf).unwrap(); + let decoded_rpc = codec.decode(&mut buf).unwrap().unwrap(); // mark as validated as its a published message - decoded_rpc.messages[0].validated = true; + match decoded_rpc { + HandlerEvent::Message { mut rpc, .. } => { + rpc.messages[0].validated = true; - assert_eq!(rpc, decoded_rpc); + assert_eq!(rpc, rpc); + } + _ => panic!("Must decode a message"), + } } QuickCheck::new().quickcheck(prop as fn(_) -> _) diff --git a/protocols/gossipsub/src/rpc.proto b/protocols/gossipsub/src/rpc.proto index 499b3b43af8..2ce12f3f37f 100644 --- a/protocols/gossipsub/src/rpc.proto +++ b/protocols/gossipsub/src/rpc.proto @@ -18,9 +18,9 @@ message Message { optional bytes from = 1; optional bytes data = 2; optional bytes seqno = 3; - repeated string topic_ids = 4; - optional bytes signature = 5; - optional bytes key = 6; + required string topic = 4; + optional bytes signature = 5; + optional bytes key = 6; } message ControlMessage { @@ -45,6 +45,13 @@ message ControlGraft { message ControlPrune { optional string topic_id = 1; + repeated PeerInfo peers = 2; // gossipsub v1.1 PX + optional uint64 backoff = 3; // gossipsub v1.1 backoff time (in seconds) +} + +message PeerInfo { + optional bytes peer_id = 1; + optional bytes signed_peer_record = 2; } // topicID = hash(topicDescriptor); (not the topic.name) diff --git a/protocols/gossipsub/src/rpc_proto.rs b/protocols/gossipsub/src/rpc_proto.rs new file mode 100644 index 00000000000..b9fa8106c6a --- /dev/null +++ b/protocols/gossipsub/src/rpc_proto.rs @@ -0,0 +1,82 @@ +// Copyright 2020 Sigma Prime Pty Ltd. +// +// Permission is hereby granted, free of charge, to any person obtaining a +// copy of this software and associated documentation files (the "Software"), +// to deal in the Software without restriction, including without limitation +// the rights to use, copy, modify, merge, publish, distribute, sublicense, +// and/or sell copies of the Software, and to permit persons to whom the +// Software is furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS +// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +// DEALINGS IN THE SOFTWARE. + +include!(concat!(env!("OUT_DIR"), "/gossipsub.pb.rs")); + +#[cfg(test)] +mod test { + use crate::IdentTopic as Topic; + use libp2p_core::PeerId; + use prost::Message; + use rand::Rng; + + mod compat_proto { + include!(concat!(env!("OUT_DIR"), "/compat.pb.rs")); + } + + #[test] + fn test_multi_topic_message_compatibility() { + let topic1 = Topic::new("t1").hash(); + let topic2 = Topic::new("t2").hash(); + + let new_message1 = super::Message { + from: Some(PeerId::random().to_bytes()), + data: Some(rand::thread_rng().gen::<[u8; 32]>().to_vec()), + seqno: Some(rand::thread_rng().gen::<[u8; 8]>().to_vec()), + topic: topic1.clone().into_string(), + signature: Some(rand::thread_rng().gen::<[u8; 32]>().to_vec()), + key: Some(rand::thread_rng().gen::<[u8; 32]>().to_vec()), + }; + let old_message1 = compat_proto::Message { + from: Some(PeerId::random().to_bytes()), + data: Some(rand::thread_rng().gen::<[u8; 32]>().to_vec()), + seqno: Some(rand::thread_rng().gen::<[u8; 8]>().to_vec()), + topic_ids: vec![topic1.clone().into_string()], + signature: Some(rand::thread_rng().gen::<[u8; 32]>().to_vec()), + key: Some(rand::thread_rng().gen::<[u8; 32]>().to_vec()), + }; + let old_message2 = compat_proto::Message { + from: Some(PeerId::random().to_bytes()), + data: Some(rand::thread_rng().gen::<[u8; 32]>().to_vec()), + seqno: Some(rand::thread_rng().gen::<[u8; 8]>().to_vec()), + topic_ids: vec![topic1.clone().into_string(), topic2.clone().into_string()], + signature: Some(rand::thread_rng().gen::<[u8; 32]>().to_vec()), + key: Some(rand::thread_rng().gen::<[u8; 32]>().to_vec()), + }; + + let mut new_message1b = Vec::with_capacity(new_message1.encoded_len()); + new_message1.encode(&mut new_message1b).unwrap(); + + let mut old_message1b = Vec::with_capacity(old_message1.encoded_len()); + old_message1.encode(&mut old_message1b).unwrap(); + + let mut old_message2b = Vec::with_capacity(old_message2.encoded_len()); + old_message2.encode(&mut old_message2b).unwrap(); + + let new_message = super::Message::decode(&old_message1b[..]).unwrap(); + assert_eq!(new_message.topic, topic1.clone().into_string()); + + let new_message = super::Message::decode(&old_message2b[..]).unwrap(); + assert_eq!(new_message.topic, topic2.clone().into_string()); + + let old_message = compat_proto::Message::decode(&new_message1b[..]).unwrap(); + assert_eq!(old_message.topic_ids, vec![topic1.into_string()]); + } +} diff --git a/protocols/gossipsub/src/subscription_filter.rs b/protocols/gossipsub/src/subscription_filter.rs new file mode 100644 index 00000000000..7aa94416183 --- /dev/null +++ b/protocols/gossipsub/src/subscription_filter.rs @@ -0,0 +1,451 @@ +// Copyright 2020 Sigma Prime Pty Ltd. +// +// Permission is hereby granted, free of charge, to any person obtaining a +// copy of this software and associated documentation files (the "Software"), +// to deal in the Software without restriction, including without limitation +// the rights to use, copy, modify, merge, publish, distribute, sublicense, +// and/or sell copies of the Software, and to permit persons to whom the +// Software is furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS +// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +// DEALINGS IN THE SOFTWARE. + +use crate::types::GossipsubSubscription; +use crate::TopicHash; +use log::info; +use std::collections::{BTreeSet, HashMap, HashSet}; + +pub trait TopicSubscriptionFilter { + /// Returns true iff the topic is of interest and we can subscribe to it. + fn can_subscribe(&mut self, topic_hash: &TopicHash) -> bool; + + /// Filters a list of incoming subscriptions and returns a filtered set + /// By default this deduplicates the subscriptions and calls + /// [`Self::filter_incoming_subscription_set`] on the filtered set. + fn filter_incoming_subscriptions<'a>( + &mut self, + subscriptions: &'a [GossipsubSubscription], + currently_subscribed_topics: &BTreeSet, + ) -> Result, String> { + let mut filtered_subscriptions: HashMap = HashMap::new(); + for subscription in subscriptions { + use std::collections::hash_map::Entry::*; + match filtered_subscriptions.entry(subscription.topic_hash.clone()) { + Occupied(entry) => { + if entry.get().action != subscription.action { + entry.remove(); + } + } + Vacant(entry) => { + entry.insert(subscription); + } + } + } + self.filter_incoming_subscription_set( + filtered_subscriptions.into_iter().map(|(_, v)| v).collect(), + currently_subscribed_topics, + ) + } + + /// Filters a set of deduplicated subscriptions + /// By default this filters the elements based on [`Self::allow_incoming_subscription`]. + fn filter_incoming_subscription_set<'a>( + &mut self, + mut subscriptions: HashSet<&'a GossipsubSubscription>, + _currently_subscribed_topics: &BTreeSet, + ) -> Result, String> { + subscriptions.retain(|s| { + if self.allow_incoming_subscription(s) { + true + } else { + info!("Filtered incoming subscription {:?}", s); + false + } + }); + Ok(subscriptions) + } + + /// Returns true iff we allow an incoming subscription. + /// This is used by the default implementation of filter_incoming_subscription_set to decide + /// whether to filter out a subscription or not. + /// By default this uses can_subscribe to decide the same for incoming subscriptions as for + /// outgoing ones. + fn allow_incoming_subscription(&mut self, subscription: &GossipsubSubscription) -> bool { + self.can_subscribe(&subscription.topic_hash) + } +} + +//some useful implementers + +/// Allows all subscriptions +#[derive(Default, Clone)] +pub struct AllowAllSubscriptionFilter {} + +impl TopicSubscriptionFilter for AllowAllSubscriptionFilter { + fn can_subscribe(&mut self, _: &TopicHash) -> bool { + true + } +} + +/// Allows only whitelisted subscriptions +#[derive(Default, Clone)] +pub struct WhitelistSubscriptionFilter(pub HashSet); + +impl TopicSubscriptionFilter for WhitelistSubscriptionFilter { + fn can_subscribe(&mut self, topic_hash: &TopicHash) -> bool { + self.0.contains(topic_hash) + } +} + +/// Adds a max count to a given subscription filter +pub struct MaxCountSubscriptionFilter { + pub filter: T, + pub max_subscribed_topics: usize, + pub max_subscriptions_per_request: usize, +} + +impl TopicSubscriptionFilter for MaxCountSubscriptionFilter { + fn can_subscribe(&mut self, topic_hash: &TopicHash) -> bool { + self.filter.can_subscribe(topic_hash) + } + + fn filter_incoming_subscriptions<'a>( + &mut self, + subscriptions: &'a [GossipsubSubscription], + currently_subscribed_topics: &BTreeSet, + ) -> Result, String> { + if subscriptions.len() > self.max_subscriptions_per_request { + return Err("too many subscriptions per request".into()); + } + let result = self + .filter + .filter_incoming_subscriptions(subscriptions, currently_subscribed_topics)?; + + use crate::types::GossipsubSubscriptionAction::*; + + let mut unsubscribed = 0; + let mut new_subscribed = 0; + for s in &result { + let currently_contained = currently_subscribed_topics.contains(&s.topic_hash); + match s.action { + Unsubscribe => { + if currently_contained { + unsubscribed += 1; + } + } + Subscribe => { + if !currently_contained { + new_subscribed += 1; + } + } + } + } + + if new_subscribed + currently_subscribed_topics.len() + > self.max_subscribed_topics + unsubscribed + { + return Err("too many subscribed topics".into()); + } + + Ok(result) + } +} + +/// Combines two subscription filters +pub struct CombinedSubscriptionFilters { + pub filter1: T, + pub filter2: S, +} + +impl TopicSubscriptionFilter for CombinedSubscriptionFilters +where + T: TopicSubscriptionFilter, + S: TopicSubscriptionFilter, +{ + fn can_subscribe(&mut self, topic_hash: &TopicHash) -> bool { + self.filter1.can_subscribe(topic_hash) && self.filter2.can_subscribe(topic_hash) + } + + fn filter_incoming_subscription_set<'a>( + &mut self, + subscriptions: HashSet<&'a GossipsubSubscription>, + currently_subscribed_topics: &BTreeSet, + ) -> Result, String> { + let intermediate = self + .filter1 + .filter_incoming_subscription_set(subscriptions, currently_subscribed_topics)?; + self.filter2 + .filter_incoming_subscription_set(intermediate, currently_subscribed_topics) + } +} + +pub struct CallbackSubscriptionFilter(pub T) +where + T: FnMut(&TopicHash) -> bool; + +impl TopicSubscriptionFilter for CallbackSubscriptionFilter +where + T: FnMut(&TopicHash) -> bool, +{ + fn can_subscribe(&mut self, topic_hash: &TopicHash) -> bool { + (self.0)(topic_hash) + } +} + +#[cfg(feature = "regex-filter")] +pub mod regex { + use super::TopicSubscriptionFilter; + use crate::TopicHash; + use regex::Regex; + + ///A subscription filter that filters topics based on a regular expression. + pub struct RegexSubscriptionFilter(pub Regex); + + impl TopicSubscriptionFilter for RegexSubscriptionFilter { + fn can_subscribe(&mut self, topic_hash: &TopicHash) -> bool { + self.0.is_match(topic_hash.as_str()) + } + } + + #[cfg(test)] + mod test { + use super::*; + use crate::types::GossipsubSubscription; + use crate::types::GossipsubSubscriptionAction::*; + + #[test] + fn test_regex_subscription_filter() { + let t1 = TopicHash::from_raw("tt"); + let t2 = TopicHash::from_raw("et3t3te"); + let t3 = TopicHash::from_raw("abcdefghijklmnopqrsuvwxyz"); + + let mut filter = RegexSubscriptionFilter(Regex::new("t.*t").unwrap()); + + let old = Default::default(); + let subscriptions = vec![ + GossipsubSubscription { + action: Subscribe, + topic_hash: t1.clone(), + }, + GossipsubSubscription { + action: Subscribe, + topic_hash: t2.clone(), + }, + GossipsubSubscription { + action: Subscribe, + topic_hash: t3.clone(), + }, + ]; + + let result = filter + .filter_incoming_subscriptions(&subscriptions, &old) + .unwrap(); + assert_eq!(result, subscriptions[..2].iter().collect()); + } + } +} + +#[cfg(test)] +mod test { + use super::*; + use crate::types::GossipsubSubscriptionAction::*; + use std::iter::FromIterator; + + #[test] + fn test_filter_incoming_allow_all_with_duplicates() { + let mut filter = AllowAllSubscriptionFilter {}; + + let t1 = TopicHash::from_raw("t1"); + let t2 = TopicHash::from_raw("t2"); + + let old = BTreeSet::from_iter(vec![t1.clone()].into_iter()); + let subscriptions = vec![ + GossipsubSubscription { + action: Unsubscribe, + topic_hash: t1.clone(), + }, + GossipsubSubscription { + action: Unsubscribe, + topic_hash: t2.clone(), + }, + GossipsubSubscription { + action: Subscribe, + topic_hash: t2.clone(), + }, + GossipsubSubscription { + action: Subscribe, + topic_hash: t1.clone(), + }, + GossipsubSubscription { + action: Unsubscribe, + topic_hash: t1.clone(), + }, + ]; + + let result = filter + .filter_incoming_subscriptions(&subscriptions, &old) + .unwrap(); + assert_eq!(result, vec![&subscriptions[4]].into_iter().collect()); + } + + #[test] + fn test_filter_incoming_whitelist() { + let t1 = TopicHash::from_raw("t1"); + let t2 = TopicHash::from_raw("t2"); + + let mut filter = WhitelistSubscriptionFilter(HashSet::from_iter(vec![t1.clone()])); + + let old = Default::default(); + let subscriptions = vec![ + GossipsubSubscription { + action: Subscribe, + topic_hash: t1.clone(), + }, + GossipsubSubscription { + action: Subscribe, + topic_hash: t2.clone(), + }, + ]; + + let result = filter + .filter_incoming_subscriptions(&subscriptions, &old) + .unwrap(); + assert_eq!(result, vec![&subscriptions[0]].into_iter().collect()); + } + + #[test] + fn test_filter_incoming_too_many_subscriptions_per_request() { + let t1 = TopicHash::from_raw("t1"); + + let mut filter = MaxCountSubscriptionFilter { + filter: AllowAllSubscriptionFilter {}, + max_subscribed_topics: 100, + max_subscriptions_per_request: 2, + }; + + let old = Default::default(); + + let subscriptions = vec![ + GossipsubSubscription { + action: Subscribe, + topic_hash: t1.clone(), + }, + GossipsubSubscription { + action: Unsubscribe, + topic_hash: t1.clone(), + }, + GossipsubSubscription { + action: Subscribe, + topic_hash: t1.clone(), + }, + ]; + + let result = filter.filter_incoming_subscriptions(&subscriptions, &old); + assert_eq!(result, Err("too many subscriptions per request".into())); + } + + #[test] + fn test_filter_incoming_too_many_subscriptions() { + let t: Vec<_> = (0..4) + .map(|i| TopicHash::from_raw(format!("t{}", i))) + .collect(); + + let mut filter = MaxCountSubscriptionFilter { + filter: AllowAllSubscriptionFilter {}, + max_subscribed_topics: 3, + max_subscriptions_per_request: 2, + }; + + let old = t[0..2].iter().cloned().collect(); + + let subscriptions = vec![ + GossipsubSubscription { + action: Subscribe, + topic_hash: t[2].clone(), + }, + GossipsubSubscription { + action: Subscribe, + topic_hash: t[3].clone(), + }, + ]; + + let result = filter.filter_incoming_subscriptions(&subscriptions, &old); + assert_eq!(result, Err("too many subscribed topics".into())); + } + + #[test] + fn test_filter_incoming_max_subscribed_valid() { + let t: Vec<_> = (0..5) + .map(|i| TopicHash::from_raw(format!("t{}", i))) + .collect(); + + let mut filter = MaxCountSubscriptionFilter { + filter: WhitelistSubscriptionFilter(t.iter().take(4).cloned().collect()), + max_subscribed_topics: 2, + max_subscriptions_per_request: 5, + }; + + let old = t[0..2].iter().cloned().collect(); + + let subscriptions = vec![ + GossipsubSubscription { + action: Subscribe, + topic_hash: t[4].clone(), + }, + GossipsubSubscription { + action: Subscribe, + topic_hash: t[2].clone(), + }, + GossipsubSubscription { + action: Subscribe, + topic_hash: t[3].clone(), + }, + GossipsubSubscription { + action: Unsubscribe, + topic_hash: t[0].clone(), + }, + GossipsubSubscription { + action: Unsubscribe, + topic_hash: t[1].clone(), + }, + ]; + + let result = filter + .filter_incoming_subscriptions(&subscriptions, &old) + .unwrap(); + assert_eq!(result, subscriptions[1..].iter().collect()); + } + + #[test] + fn test_callback_filter() { + let t1 = TopicHash::from_raw("t1"); + let t2 = TopicHash::from_raw("t2"); + + let mut filter = CallbackSubscriptionFilter(|h| h.as_str() == "t1"); + + let old = Default::default(); + let subscriptions = vec![ + GossipsubSubscription { + action: Subscribe, + topic_hash: t1.clone(), + }, + GossipsubSubscription { + action: Subscribe, + topic_hash: t2.clone(), + }, + ]; + + let result = filter + .filter_incoming_subscriptions(&subscriptions, &old) + .unwrap(); + assert_eq!(result, vec![&subscriptions[0]].into_iter().collect()); + } +} diff --git a/protocols/gossipsub/src/time_cache.rs b/protocols/gossipsub/src/time_cache.rs new file mode 100644 index 00000000000..90300e77ea3 --- /dev/null +++ b/protocols/gossipsub/src/time_cache.rs @@ -0,0 +1,244 @@ +// Copyright 2020 Sigma Prime Pty Ltd. +// +// Permission is hereby granted, free of charge, to any person obtaining a +// copy of this software and associated documentation files (the "Software"), +// to deal in the Software without restriction, including without limitation +// the rights to use, copy, modify, merge, publish, distribute, sublicense, +// and/or sell copies of the Software, and to permit persons to whom the +// Software is furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS +// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +// DEALINGS IN THE SOFTWARE. + +//! This implements a time-based LRU cache for checking gossipsub message duplicates. + +use fnv::FnvHashMap; +use std::collections::hash_map::{ + self, + Entry::{Occupied, Vacant}, +}; +use std::collections::VecDeque; +use std::time::{Duration, Instant}; + +struct ExpiringElement { + /// The element that expires + element: Element, + /// The expire time. + expires: Instant, +} + +pub struct TimeCache { + /// Mapping a key to its value together with its latest expire time (can be updated through + /// reinserts). + map: FnvHashMap>, + /// An ordered list of keys by expires time. + list: VecDeque>, + /// The time elements remain in the cache. + ttl: Duration, +} + +pub struct OccupiedEntry<'a, K, V> { + expiration: Instant, + entry: hash_map::OccupiedEntry<'a, K, ExpiringElement>, + list: &'a mut VecDeque>, +} + +impl<'a, K, V> OccupiedEntry<'a, K, V> +where + K: Eq + std::hash::Hash + Clone, +{ + pub fn into_mut(self) -> &'a mut V { + &mut self.entry.into_mut().element + } + + pub fn insert_without_updating_expiration(&mut self, value: V) -> V { + //keep old expiration, only replace value of element + ::std::mem::replace(&mut self.entry.get_mut().element, value) + } + + pub fn insert_and_update_expiration(&mut self, value: V) -> V { + //We push back an additional element, the first reference in the list will be ignored + // since we also updated the expires in the map, see below. + self.list.push_back(ExpiringElement { + element: self.entry.key().clone(), + expires: self.expiration, + }); + self.entry + .insert(ExpiringElement { + element: value, + expires: self.expiration, + }) + .element + } +} + +pub struct VacantEntry<'a, K, V> { + expiration: Instant, + entry: hash_map::VacantEntry<'a, K, ExpiringElement>, + list: &'a mut VecDeque>, +} + +impl<'a, K, V> VacantEntry<'a, K, V> +where + K: Eq + std::hash::Hash + Clone, +{ + pub fn insert(self, value: V) -> &'a mut V { + self.list.push_back(ExpiringElement { + element: self.entry.key().clone(), + expires: self.expiration, + }); + &mut self + .entry + .insert(ExpiringElement { + element: value, + expires: self.expiration, + }) + .element + } +} + +pub enum Entry<'a, K: 'a, V: 'a> { + Occupied(OccupiedEntry<'a, K, V>), + Vacant(VacantEntry<'a, K, V>), +} + +impl<'a, K: 'a, V: 'a> Entry<'a, K, V> +where + K: Eq + std::hash::Hash + Clone, +{ + pub fn or_insert_with V>(self, default: F) -> &'a mut V { + match self { + Entry::Occupied(entry) => entry.into_mut(), + Entry::Vacant(entry) => entry.insert(default()), + } + } +} + +impl TimeCache +where + Key: Eq + std::hash::Hash + Clone, +{ + pub fn new(ttl: Duration) -> Self { + TimeCache { + map: FnvHashMap::default(), + list: VecDeque::new(), + ttl, + } + } + + fn remove_expired_keys(&mut self, now: Instant) { + while let Some(element) = self.list.pop_front() { + if element.expires > now { + self.list.push_front(element); + break; + } + if let Occupied(entry) = self.map.entry(element.element.clone()) { + if entry.get().expires <= now { + entry.remove(); + } + } + } + } + + pub fn entry(&mut self, key: Key) -> Entry { + let now = Instant::now(); + self.remove_expired_keys(now); + match self.map.entry(key) { + Occupied(entry) => Entry::Occupied(OccupiedEntry { + expiration: now + self.ttl, + entry, + list: &mut self.list, + }), + Vacant(entry) => Entry::Vacant(VacantEntry { + expiration: now + self.ttl, + entry, + list: &mut self.list, + }), + } + } + + /// Empties the entire cache. + pub fn clear(&mut self) { + self.map.clear(); + self.list.clear(); + } + + pub fn contains_key(&mut self, key: &Key) -> bool { + self.map.contains_key(key) + } + + pub fn get(&self, key: &Key) -> Option<&Value> { + self.map.get(key).map(|e| &e.element) + } +} + +pub struct DuplicateCache(TimeCache); + +impl DuplicateCache +where + Key: Eq + std::hash::Hash + Clone, +{ + pub fn new(ttl: Duration) -> Self { + Self(TimeCache::new(ttl)) + } + + // Inserts new elements and removes any expired elements. + // + // If the key was not present this returns `true`. If the value was already present this + // returns `false`. + pub fn insert(&mut self, key: Key) -> bool { + if let Entry::Vacant(entry) = self.0.entry(key) { + entry.insert(()); + true + } else { + false + } + } + + pub fn contains(&mut self, key: &Key) -> bool { + self.0.contains_key(key) + } +} + +#[cfg(test)] +mod test { + use super::*; + + #[test] + fn cache_added_entries_exist() { + let mut cache = DuplicateCache::new(Duration::from_secs(10)); + + cache.insert("t"); + cache.insert("e"); + + // Should report that 't' and 't' already exists + assert!(!cache.insert("t")); + assert!(!cache.insert("e")); + } + + #[test] + fn cache_entries_expire() { + let mut cache = DuplicateCache::new(Duration::from_millis(100)); + + cache.insert("t"); + assert!(!cache.insert("t")); + cache.insert("e"); + //assert!(!cache.insert("t")); + assert!(!cache.insert("e")); + // sleep until cache expiry + std::thread::sleep(Duration::from_millis(101)); + // add another element to clear previous cache + cache.insert("s"); + + // should be removed from the cache + assert!(cache.insert("t")); + } +} diff --git a/protocols/gossipsub/src/topic.rs b/protocols/gossipsub/src/topic.rs index 6c1527d8ce5..7e8afca2d9e 100644 --- a/protocols/gossipsub/src/topic.rs +++ b/protocols/gossipsub/src/topic.rs @@ -24,6 +24,42 @@ use prost::Message; use sha2::{Digest, Sha256}; use std::fmt; +/// A generic trait that can be extended for various hashing types for a topic. +pub trait Hasher { + /// The function that takes a topic string and creates a topic hash. + fn hash(topic_string: String) -> TopicHash; +} + +/// A type for representing topics who use the identity hash. +#[derive(Debug, Clone)] +pub struct IdentityHash {} +impl Hasher for IdentityHash { + /// Creates a [`TopicHash`] as a raw string. + fn hash(topic_string: String) -> TopicHash { + TopicHash { hash: topic_string } + } +} + +#[derive(Debug, Clone)] +pub struct Sha256Hash {} +impl Hasher for Sha256Hash { + /// Creates a [`TopicHash`] by SHA256 hashing the topic then base64 encoding the + /// hash. + fn hash(topic_string: String) -> TopicHash { + let topic_descripter = rpc_proto::TopicDescriptor { + name: Some(topic_string), + auth: None, + enc: None, + }; + let mut bytes = Vec::with_capacity(topic_descripter.encoded_len()); + topic_descripter + .encode(&mut bytes) + .expect("buffer is large enough"); + let hash = encode(Sha256::digest(&bytes).as_slice()); + TopicHash { hash } + } +} + #[derive(Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Ord)] pub struct TopicHash { /// The topic hash. Stored as a string to align with the protobuf API. @@ -35,6 +71,10 @@ impl TopicHash { TopicHash { hash: hash.into() } } + pub fn into_string(self) -> String { + self.hash + } + pub fn as_str(&self) -> &str { &self.hash } @@ -42,48 +82,32 @@ impl TopicHash { /// A gossipsub topic. #[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord)] -pub struct Topic { +pub struct Topic { topic: String, + phantom_data: std::marker::PhantomData, } -impl Topic { - pub fn new(topic: String) -> Self { - Topic { topic } - } - - /// Creates a `TopicHash` by SHA256 hashing the topic then base64 encoding the - /// hash. - pub fn sha256_hash(&self) -> TopicHash { - let topic_descripter = rpc_proto::TopicDescriptor { - name: Some(self.topic.clone()), - auth: None, - enc: None, - }; - let mut bytes = Vec::with_capacity(topic_descripter.encoded_len()); - topic_descripter - .encode(&mut bytes) - .expect("buffer is large enough"); - let hash = encode(Sha256::digest(&bytes).as_slice()); - - TopicHash { hash } +impl From> for TopicHash { + fn from(topic: Topic) -> TopicHash { + topic.hash() } +} - /// Creates a `TopicHash` as a raw string. - pub fn no_hash(&self) -> TopicHash { - TopicHash { - hash: self.topic.clone(), +impl Topic { + pub fn new(topic: impl Into) -> Self { + Topic { + topic: topic.into(), + phantom_data: std::marker::PhantomData, } } -} -impl Into for TopicHash { - fn into(self) -> String { - self.hash + pub fn hash(&self) -> TopicHash { + H::hash(self.topic.clone()) } } -impl fmt::Display for Topic { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { +impl fmt::Display for Topic { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { write!(f, "{}", self.topic) } } diff --git a/protocols/gossipsub/src/transform.rs b/protocols/gossipsub/src/transform.rs new file mode 100644 index 00000000000..ed11c78e5e0 --- /dev/null +++ b/protocols/gossipsub/src/transform.rs @@ -0,0 +1,78 @@ +// Copyright 2020 Sigma Prime Pty Ltd. +// +// Permission is hereby granted, free of charge, to any person obtaining a +// copy of this software and associated documentation files (the "Software"), +// to deal in the Software without restriction, including without limitation +// the rights to use, copy, modify, merge, publish, distribute, sublicense, +// and/or sell copies of the Software, and to permit persons to whom the +// Software is furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS +// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +// DEALINGS IN THE SOFTWARE. + +//! This trait allows of extended user-level decoding that can apply to message-data before a +//! message-id is calculated. +//! +//! This is primarily designed to allow applications to implement their own custom compression +//! algorithms that can be topic-specific. Once the raw data is transformed the message-id is then +//! calculated, allowing for applications to employ message-id functions post compression. + +use crate::{GossipsubMessage, RawGossipsubMessage, TopicHash}; + +/// A general trait of transforming a [`RawGossipsubMessage`] into a [`GossipsubMessage`]. The +/// [`RawGossipsubMessage`] is obtained from the wire and the [`GossipsubMessage`] is used to +/// calculate the [`crate::MessageId`] of the message and is what is sent to the application. +/// +/// The inbound/outbound transforms must be inverses. Applying the inbound transform and then the +/// outbound transform MUST leave the underlying data un-modified. +/// +/// By default, this is the identity transform for all fields in [`GossipsubMessage`]. +pub trait DataTransform { + /// Takes a [`RawGossipsubMessage`] received and converts it to a [`GossipsubMessage`]. + fn inbound_transform( + &self, + raw_message: RawGossipsubMessage, + ) -> Result; + + /// Takes the data to be published (a topic and associated data) transforms the data. The + /// transformed data will then be used to create a [`crate::RawGossipsubMessage`] to be sent to peers. + fn outbound_transform( + &self, + topic: &TopicHash, + data: Vec, + ) -> Result, std::io::Error>; +} + +/// The default transform, the raw data is propagated as is to the application layer gossipsub. +#[derive(Default, Clone)] +pub struct IdentityTransform; + +impl DataTransform for IdentityTransform { + fn inbound_transform( + &self, + raw_message: RawGossipsubMessage, + ) -> Result { + Ok(GossipsubMessage { + source: raw_message.source, + data: raw_message.data, + sequence_number: raw_message.sequence_number, + topic: raw_message.topic, + }) + } + + fn outbound_transform( + &self, + _topic: &TopicHash, + data: Vec, + ) -> Result, std::io::Error> { + Ok(data) + } +} diff --git a/protocols/gossipsub/src/types.rs b/protocols/gossipsub/src/types.rs new file mode 100644 index 00000000000..69879087941 --- /dev/null +++ b/protocols/gossipsub/src/types.rs @@ -0,0 +1,363 @@ +// Copyright 2020 Sigma Prime Pty Ltd. +// +// Permission is hereby granted, free of charge, to any person obtaining a +// copy of this software and associated documentation files (the "Software"), +// to deal in the Software without restriction, including without limitation +// the rights to use, copy, modify, merge, publish, distribute, sublicense, +// and/or sell copies of the Software, and to permit persons to whom the +// Software is furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS +// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +// DEALINGS IN THE SOFTWARE. + +//! A collection of types using the Gossipsub system. +use crate::rpc_proto; +use crate::TopicHash; +use libp2p_core::PeerId; +use std::fmt; +use std::fmt::Debug; + +#[derive(Debug)] +/// Validation kinds from the application for received messages. +pub enum MessageAcceptance { + /// The message is considered valid, and it should be delivered and forwarded to the network. + Accept, + /// The message is considered invalid, and it should be rejected and trigger the P₄ penalty. + Reject, + /// The message is neither delivered nor forwarded to the network, but the router does not + /// trigger the P₄ penalty. + Ignore, +} + +/// Macro for declaring message id types +macro_rules! declare_message_id_type { + ($name: ident, $name_string: expr) => { + #[derive(Clone, PartialEq, Eq, Hash, PartialOrd, Ord)] + pub struct $name(pub Vec); + + impl $name { + pub fn new(value: &[u8]) -> Self { + Self(value.to_vec()) + } + } + + impl>> From for $name { + fn from(value: T) -> Self { + Self(value.into()) + } + } + + impl std::fmt::Display for $name { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", hex_fmt::HexFmt(&self.0)) + } + } + + impl std::fmt::Debug for $name { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}({})", $name_string, hex_fmt::HexFmt(&self.0)) + } + } + }; +} + +// A type for gossipsub message ids. +declare_message_id_type!(MessageId, "MessageId"); + +// A type for gossipsub fast messsage ids, not to confuse with "real" message ids. +// +// A fast-message-id is an optional message_id that can be used to filter duplicates quickly. On +// high intensive networks with lots of messages, where the message_id is based on the result of +// decompressed traffic, it is beneficial to specify a `fast-message-id` that can identify and +// filter duplicates quickly without performing the overhead of decompression. +declare_message_id_type!(FastMessageId, "FastMessageId"); + +/// Describes the types of peers that can exist in the gossipsub context. +#[derive(Debug, Clone, PartialEq)] +pub enum PeerKind { + /// A gossipsub 1.1 peer. + Gossipsubv1_1, + /// A gossipsub 1.0 peer. + Gossipsub, + /// A floodsub peer. + Floodsub, + /// The peer doesn't support any of the protocols. + NotSupported, +} + +/// A message received by the gossipsub system and stored locally in caches.. +#[derive(Clone, PartialEq, Eq, Hash, Debug)] +pub struct RawGossipsubMessage { + /// Id of the peer that published this message. + pub source: Option, + + /// Content of the message. Its meaning is out of scope of this library. + pub data: Vec, + + /// A random sequence number. + pub sequence_number: Option, + + /// The topic this message belongs to + pub topic: TopicHash, + + /// The signature of the message if it's signed. + pub signature: Option>, + + /// The public key of the message if it is signed and the source [`PeerId`] cannot be inlined. + pub key: Option>, + + /// Flag indicating if this message has been validated by the application or not. + pub validated: bool, +} + +/// The message sent to the user after a [`RawGossipsubMessage`] has been transformed by a +/// [`crate::DataTransform`]. +#[derive(Clone, PartialEq, Eq, Hash)] +pub struct GossipsubMessage { + /// Id of the peer that published this message. + pub source: Option, + + /// Content of the message. + pub data: Vec, + + /// A random sequence number. + pub sequence_number: Option, + + /// The topic this message belongs to + pub topic: TopicHash, +} + +impl fmt::Debug for GossipsubMessage { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("GossipsubMessage") + .field( + "data", + &format_args!("{:<20}", &hex_fmt::HexFmt(&self.data)), + ) + .field("source", &self.source) + .field("sequence_number", &self.sequence_number) + .field("topic", &self.topic) + .finish() + } +} + +/// A subscription received by the gossipsub system. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct GossipsubSubscription { + /// Action to perform. + pub action: GossipsubSubscriptionAction, + /// The topic from which to subscribe or unsubscribe. + pub topic_hash: TopicHash, +} + +/// Action that a subscription wants to perform. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub enum GossipsubSubscriptionAction { + /// The remote wants to subscribe to the given topic. + Subscribe, + /// The remote wants to unsubscribe from the given topic. + Unsubscribe, +} + +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct PeerInfo { + pub peer_id: Option, + //TODO add this when RFC: Signed Address Records got added to the spec (see pull request + // https://github.com/libp2p/specs/pull/217) + //pub signed_peer_record: ?, +} + +/// A Control message received by the gossipsub system. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub enum GossipsubControlAction { + /// Node broadcasts known messages per topic - IHave control message. + IHave { + /// The topic of the messages. + topic_hash: TopicHash, + /// A list of known message ids (peer_id + sequence _number) as a string. + message_ids: Vec, + }, + /// The node requests specific message ids (peer_id + sequence _number) - IWant control message. + IWant { + /// A list of known message ids (peer_id + sequence _number) as a string. + message_ids: Vec, + }, + /// The node has been added to the mesh - Graft control message. + Graft { + /// The mesh topic the peer should be added to. + topic_hash: TopicHash, + }, + /// The node has been removed from the mesh - Prune control message. + Prune { + /// The mesh topic the peer should be removed from. + topic_hash: TopicHash, + /// A list of peers to be proposed to the removed peer as peer exchange + peers: Vec, + /// The backoff time in seconds before we allow to reconnect + backoff: Option, + }, +} + +/// An RPC received/sent. +#[derive(Clone, PartialEq, Eq, Hash)] +pub struct GossipsubRpc { + /// List of messages that were part of this RPC query. + pub messages: Vec, + /// List of subscriptions. + pub subscriptions: Vec, + /// List of Gossipsub control messages. + pub control_msgs: Vec, +} + +impl GossipsubRpc { + /// Converts the GossipsubRPC into its protobuf format. + // A convenience function to avoid explicitly specifying types. + pub fn into_protobuf(self) -> rpc_proto::Rpc { + self.into() + } +} + +impl Into for GossipsubRpc { + /// Converts the RPC into protobuf format. + fn into(self) -> rpc_proto::Rpc { + // Messages + let mut publish = Vec::new(); + + for message in self.messages.into_iter() { + let message = rpc_proto::Message { + from: message.source.map(|m| m.to_bytes()), + data: Some(message.data), + seqno: message.sequence_number.map(|s| s.to_be_bytes().to_vec()), + topic: TopicHash::into_string(message.topic), + signature: message.signature, + key: message.key, + }; + + publish.push(message); + } + + // subscriptions + let subscriptions = self + .subscriptions + .into_iter() + .map(|sub| rpc_proto::rpc::SubOpts { + subscribe: Some(sub.action == GossipsubSubscriptionAction::Subscribe), + topic_id: Some(sub.topic_hash.into_string()), + }) + .collect::>(); + + // control messages + let mut control = rpc_proto::ControlMessage { + ihave: Vec::new(), + iwant: Vec::new(), + graft: Vec::new(), + prune: Vec::new(), + }; + + let empty_control_msg = self.control_msgs.is_empty(); + + for action in self.control_msgs { + match action { + // collect all ihave messages + GossipsubControlAction::IHave { + topic_hash, + message_ids, + } => { + let rpc_ihave = rpc_proto::ControlIHave { + topic_id: Some(topic_hash.into_string()), + message_ids: message_ids.into_iter().map(|msg_id| msg_id.0).collect(), + }; + control.ihave.push(rpc_ihave); + } + GossipsubControlAction::IWant { message_ids } => { + let rpc_iwant = rpc_proto::ControlIWant { + message_ids: message_ids.into_iter().map(|msg_id| msg_id.0).collect(), + }; + control.iwant.push(rpc_iwant); + } + GossipsubControlAction::Graft { topic_hash } => { + let rpc_graft = rpc_proto::ControlGraft { + topic_id: Some(topic_hash.into_string()), + }; + control.graft.push(rpc_graft); + } + GossipsubControlAction::Prune { + topic_hash, + peers, + backoff, + } => { + let rpc_prune = rpc_proto::ControlPrune { + topic_id: Some(topic_hash.into_string()), + peers: peers + .into_iter() + .map(|info| rpc_proto::PeerInfo { + peer_id: info.peer_id.map(|id| id.to_bytes()), + /// TODO, see https://github.com/libp2p/specs/pull/217 + signed_peer_record: None, + }) + .collect(), + backoff, + }; + control.prune.push(rpc_prune); + } + } + } + + rpc_proto::Rpc { + subscriptions, + publish, + control: if empty_control_msg { + None + } else { + Some(control) + }, + } + } +} + +impl fmt::Debug for GossipsubRpc { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + let mut b = f.debug_struct("GossipsubRpc"); + if !self.messages.is_empty() { + b.field("messages", &self.messages); + } + if !self.subscriptions.is_empty() { + b.field("subscriptions", &self.subscriptions); + } + if !self.control_msgs.is_empty() { + b.field("control_msgs", &self.control_msgs); + } + b.finish() + } +} + +impl PeerKind { + pub fn as_static_ref(&self) -> &'static str { + match self { + Self::NotSupported => "Not Supported", + Self::Floodsub => "Floodsub", + Self::Gossipsub => "Gossipsub v1.0", + Self::Gossipsubv1_1 => "Gossipsub v1.1", + } + } +} + +impl AsRef for PeerKind { + fn as_ref(&self) -> &str { + self.as_static_ref() + } +} + +impl fmt::Display for PeerKind { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.write_str(self.as_ref()) + } +} diff --git a/protocols/gossipsub/tests/smoke.rs b/protocols/gossipsub/tests/smoke.rs index 0fad359be44..1ef92ba658c 100644 --- a/protocols/gossipsub/tests/smoke.rs +++ b/protocols/gossipsub/tests/smoke.rs @@ -28,12 +28,13 @@ use std::{ time::Duration, }; +use futures::StreamExt; use libp2p_core::{ - identity, multiaddr::Protocol, transport::MemoryTransport, upgrade, - Multiaddr, Transport, + identity, multiaddr::Protocol, transport::MemoryTransport, upgrade, Multiaddr, Transport, }; use libp2p_gossipsub::{ - Gossipsub, GossipsubConfigBuilder, GossipsubEvent, MessageAuthenticity, Topic, ValidationMode, + Gossipsub, GossipsubConfigBuilder, GossipsubEvent, IdentTopic as Topic, MessageAuthenticity, + ValidationMode, }; use libp2p_plaintext::PlainText2Config; use libp2p_swarm::Swarm; @@ -106,14 +107,10 @@ impl Graph { /// `true`. /// /// Returns [`true`] on success and [`false`] on timeout. - fn wait_for bool>(&mut self, mut f: F) -> bool { - let fut = futures::future::poll_fn(move |cx| { - match self.poll_unpin(cx) { - Poll::Ready((_addr, ev)) if f(ev.clone()) => { - Poll::Ready(()) - } - _ => Poll::Pending, - } + fn wait_for bool>(&mut self, mut f: F) -> bool { + let fut = futures::future::poll_fn(move |cx| match self.poll_unpin(cx) { + Poll::Ready((_addr, ev)) if f(&ev) => Poll::Ready(()), + _ => Poll::Pending, }); let fut = async_std::future::timeout(Duration::from_secs(10), fut); @@ -160,14 +157,15 @@ fn build_node() -> (Multiaddr, Swarm) { // reduce the default values of the heartbeat, so that all nodes will receive gossip in a // timely fashion. - let config = GossipsubConfigBuilder::new() + let config = GossipsubConfigBuilder::default() .heartbeat_initial_delay(Duration::from_millis(100)) .heartbeat_interval(Duration::from_millis(200)) .history_length(10) .history_gossip(10) .validation_mode(ValidationMode::Permissive) - .build(); - let behaviour = Gossipsub::new(MessageAuthenticity::Author(peer_id.clone()), config); + .build() + .unwrap(); + let behaviour = Gossipsub::new(MessageAuthenticity::Author(peer_id.clone()), config).unwrap(); let mut swarm = Swarm::new(transport, behaviour, peer_id); let port = 1 + random::(); @@ -186,7 +184,7 @@ fn multi_hop_propagation() { let _ = env_logger::try_init(); fn prop(num_nodes: u8, seed: u64) -> TestResult { - if num_nodes < 2 || num_nodes > 100 { + if num_nodes < 2 || num_nodes > 50 { return TestResult::discard(); } @@ -196,9 +194,9 @@ fn multi_hop_propagation() { let number_nodes = graph.nodes.len(); // Subscribe each node to the same topic. - let topic = Topic::new("test-net".into()); + let topic = Topic::new("test-net"); for (_addr, node) in &mut graph.nodes { - node.subscribe(topic.clone()); + node.subscribe(&topic).unwrap(); } // Wait for all nodes to be subscribed. @@ -225,12 +223,12 @@ fn multi_hop_propagation() { graph = graph.drain_poll(); // Publish a single message. - graph.nodes[0].1.publish(&topic, vec![1, 2, 3]).unwrap(); + graph.nodes[0].1.publish(topic, vec![1, 2, 3]).unwrap(); // Wait for all nodes to receive the published message. let mut received_msgs = 0; let all_received = graph.wait_for(move |ev| { - if let GossipsubEvent::Message(..) = ev { + if let GossipsubEvent::Message { .. } = ev { received_msgs += 1; if received_msgs == number_nodes - 1 { return true; @@ -250,6 +248,6 @@ fn multi_hop_propagation() { } QuickCheck::new() - .max_tests(10) + .max_tests(5) .quickcheck(prop as fn(u8, u64) -> TestResult) }