protocols/gossipsub: Add Gossipsub v1.1 support

This commit upgrades the current gossipsub implementation to support the [v1.1
spec](https://github.com/libp2p/specs/blob/master/pubsub/gossipsub/gossipsub-v1.1.md).

It adds a number of features, bug fixes and performance improvements. 

Besides support for all new 1.1 features, other improvements that are of particular note: 

- Improved duplicate LRU-time cache (this was previously a severe bottleneck for
  large message throughput topics)
- Extended message validation configuration options
- Arbitrary topics (users can now implement their own hashing schemes)
- Improved message validation handling - Invalid messages are no longer dropped
  but sent to the behaviour for application-level processing (including scoring)
- Support for floodsub, gossipsub v1 and gossipsub v2
- Protobuf encoding has been shifted into the behaviour. This has permitted two
  improvements:
     1. Message size verification during publishing (report to the user if the
        message is too large before attempting to send).
     2. Message fragmentation. If an RPC is too large it is fragmented into its
        sub components and sent in smaller chunks.

Additional Notes

The peer eXchange protocol defined in the v1.1 spec is inactive in its current
form. The current implementation permits sending `PeerId` in `PRUNE` messages,
however a `PeerId` is not sufficient to form a new connection to a peer. A
`Signed Address Record` is required to safely transmit peer identity
information. Once these are confirmed (https://github.com/libp2p/specs/pull/217)
a future PR will implement these and make PX usable.

Co-authored-by: Max Inden <mail@max-inden.de>
Co-authored-by: Rüdiger Klaehn <rklaehn@protonmail.com>
Co-authored-by: blacktemplar <blacktemplar@a1.net>
Co-authored-by: Rüdiger Klaehn <rklaehn@gmail.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Roman S. Borschel <roman@parity.io>
Co-authored-by: Roman Borschel <romanb@users.noreply.github.com>
Co-authored-by: David Craven <david@craven.ch>
This commit is contained in:
Age Manning 2021-01-07 18:19:31 +11:00 committed by GitHub
parent d918e9a79d
commit df7e73ec47
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
26 changed files with 12071 additions and 1385 deletions

View File

@ -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<dyn Error>> {
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<dyn Error>> {
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<dyn Error>> {
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<dyn Error>> {
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,

View File

@ -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<PreSharedKey>,
) -> transport::Boxed<(PeerId, StreamMuxerBox)>
{
let noise_keys = noise::Keypair::<noise::X25519Spec>::new().into_authentic(&key_pair).unwrap();
) -> transport::Boxed<(PeerId, StreamMuxerBox)> {
let noise_keys = noise::Keypair::<noise::X25519Spec>::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<dyn Error>> {
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<dyn Error>> {
// 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<dyn Error>> {
// 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<dyn Error>> {
};
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<dyn Error>> {
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,
} {

View File

@ -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"

View File

@ -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();
}

View File

@ -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<TopicHash, HashMap<PeerId, (Instant, HeartbeatIndex)>>,
/// Stores peer topic pairs per heartbeat (this is cyclic the current index is
/// heartbeat_index).
backoffs_by_heartbeat: Vec<HashSet<(TopicHash, PeerId)>>,
/// 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<HashSet<_>>,
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<Instant> {
Self::get_backoff_time_from_backoffs(&self.backoffs, topic, peer)
}
fn get_backoff_time_from_backoffs(
backoffs: &HashMap<TopicHash, HashMap<PeerId, (Instant, HeartbeatIndex)>>,
topic: &TopicHash,
peer: &PeerId,
) -> Option<Instant> {
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());
}
}

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@ -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;
}

View File

@ -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<fn(&RawGossipsubMessage) -> 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<usize>,
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
/// `/<prefix>/<supported-versions>`. 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<FastMessageId> {
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<usize> {
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<GossipsubConfig> 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<Cow<'static, [u8]>>) -> &mut Self {
self.config.protocol_id = protocol_id.into();
pub fn protocol_id_prefix(&mut self, protocol_id: impl Into<Cow<'static, str>>) -> &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<usize>) -> &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<GossipsubConfig, &str> {
// 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);
}
}

View File

@ -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<SigningError> for PublishError {
@ -38,3 +54,70 @@ impl From<SigningError> 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<std::io::Error> for GossipsubHandlerError {
fn from(error: std::io::Error) -> GossipsubHandlerError {
GossipsubHandlerError::Io(error)
}
}
impl From<std::io::Error> 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,
}
}
}

View File

@ -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<MessageId, HashMap<PeerId, Instant>>,
}
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<PeerId, usize> {
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
}
}

View File

@ -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<InboundSubstreamState>,
/// 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<PeerKind>,
/// 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<ProtocolsHandlerUpgrErr<GossipsubHandlerError>>,
/// 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<NegotiatedSubstream, GossipsubCodec>),
/// Waiting to send a message to the remote.
PendingSend(Framed<NegotiatedSubstream, GossipsubCodec>, GossipsubRpc),
PendingSend(
Framed<NegotiatedSubstream, GossipsubCodec>,
crate::rpc_proto::Rpc,
),
/// Waiting to flush the substream so that the data arrives to the remote.
PendingFlush(Framed<NegotiatedSubstream, GossipsubCodec>),
/// 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<Cow<'static, [u8]>>,
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 InboundProtocol = ProtocolConfig;
type OutboundProtocol = ProtocolConfig;
type OutboundOpenInfo = GossipsubRpc;
type InEvent = crate::rpc_proto::Rpc;
type OutEvent = HandlerEvent;
type Error = GossipsubHandlerError;
type InboundOpenInfo = ();
type InboundProtocol = ProtocolConfig;
type OutboundOpenInfo = Self::InEvent;
type OutboundProtocol = ProtocolConfig;
fn listen_protocol(&self) -> SubstreamProtocol<Self::InboundProtocol, Self::InboundOpenInfo> {
self.listen_protocol.clone()
@ -120,9 +190,21 @@ impl ProtocolsHandler for GossipsubHandler {
fn inject_fully_negotiated_inbound(
&mut self,
substream: <Self::InboundProtocol as InboundUpgrade<NegotiatedSubstream>>::Output,
_info: Self::InboundOpenInfo
(substream, peer_kind): <Self::InboundProtocol as InboundUpgrade<NegotiatedSubstream>>::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: <Self::OutboundProtocol as OutboundUpgrade<NegotiatedSubstream>>::Output,
(substream, peer_kind): <Self::OutboundProtocol as OutboundUpgrade<NegotiatedSubstream>>::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<
<Self::OutboundProtocol as OutboundUpgrade<NegotiatedSubstream>>::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 =

View File

@ -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
//! [`GossipsubConfig`]: struct.Config.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()`.
//! 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();
//! ```
//! 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 and Yamux protocols
//! let transport = libp2p::build_development_transport(local_key);
//! // Set up an encrypted TCP Transport over the Mplex
//! // This is test transport (memory).
//! let noise_keys = libp2p_noise::Keypair::<libp2p_noise::X25519Spec>::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 Floodsub/Gossipsub topic
//! let topic = libp2p::floodsub::TopicBuilder::new("example").build();
//! // 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;
mod rpc_proto {
include!(concat!(env!("OUT_DIR"), "/gossipsub.pb.rs"));
}
#[cfg(test)]
#[macro_use]
extern crate derive_builder;
mod rpc_proto;
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<self::topic::IdentityHash>;
pub type Sha256Topic = Topic<self::topic::Sha256Hash>;

View File

@ -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<TopicHash>,
topic: TopicHash,
}
/// MessageCache struct holding history of messages.
#[derive(Clone)]
pub struct MessageCache {
msgs: HashMap<MessageId, GossipsubMessage>,
msgs: HashMap<MessageId, RawGossipsubMessage>,
/// For every message and peer the number of times this peer asked for the message
iwant_counts: HashMap<MessageId, HashMap<PeerId, u32>>,
history: Vec<Vec<CacheEntry>>,
/// 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<GossipsubMessage> {
let message_id = (self.msg_id)(&msg);
pub fn put(
&mut self,
message_id: &MessageId,
msg: RawGossipsubMessage,
) -> Option<RawGossipsubMessage> {
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<MessageId> {
/// Get a list of [`MessageId`]s for a given topic.
pub fn get_gossip_message_ids(&self, topic: &TopicHash) -> Vec<MessageId> {
self.history[..self.gossip]
.iter()
.fold(vec![], |mut current_entries, entries| {
@ -105,7 +136,7 @@ impl MessageCache {
let mut found_entries: Vec<MessageId> = 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<RawGossipsubMessage> {
//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<TopicHash>) -> GossipsubMessage {
let u8x: u8 = x as u8;
let source = Some(PeerId::random());
let data: Vec<u8> = vec![u8x];
let sequence_number = Some(x);
let m = GossipsubMessage {
source,
data,
sequence_number,
topics,
signature: None,
key: None,
validated: true,
};
m
}
fn new_cache(gossip_size: usize, history: usize) -> MessageCache {
let default_id = |message: &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<u8> = vec![u8x];
let sequence_number = Some(x);
MessageCache::new(gossip_size, history, default_id)
let m = RawGossipsubMessage {
source,
data,
sequence_number,
topic,
signature: None,
key: None,
validated: false,
};
let id = default_id(&m);
(id, m)
}
fn new_cache(gossip_size: usize, history: usize) -> MessageCache {
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 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());
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

View File

@ -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<PeerId, PeerStats>,
/// Tracking peers per IP.
peer_ips: HashMap<IpAddr, HashSet<PeerId>>,
/// Message delivery tracking. This is a time-cache of [`DeliveryRecord`]s.
deliveries: TimeCache<MessageId, DeliveryRecord>,
/// callback for monitoring message delivery times
message_delivery_time_callback: Option<fn(&PeerId, &TopicHash, f64)>,
}
/// General statistics for a given gossipsub peer.
struct PeerStats {
/// Connection status of the peer.
status: ConnectionStatus,
/// Stats per topic.
topics: HashMap<TopicHash, TopicStats>,
/// IP tracking for individual peers.
known_ips: HashSet<IpAddr>,
/// 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<PeerId>,
}
#[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<fn(&PeerId, &TopicHash, f64)>,
) -> 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<IpAddr, HashSet<PeerId>>,
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<TopicHash>) {
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::<Vec<_>>() {
// 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<Instant>,
) {
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<f64> {
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,
}

View File

@ -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<TopicHash, TopicScoreParams>,
/// 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<IpAddr>,
/// 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(())
}
}

View File

@ -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"
);
}

View File

@ -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<ProtocolId>,
/// 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<Cow<'static, [u8]>>,
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<u8>,
/// 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<Self::Info>;
type Info = ProtocolId;
type InfoIter = Vec<Self::Info>;
fn protocol_info(&self) -> Self::InfoIter {
iter::once(self.protocol_id.clone())
self.protocol_ids.clone()
}
}
@ -76,16 +127,19 @@ impl<TSocket> InboundUpgrade<TSocket> for ProtocolConfig
where
TSocket: AsyncRead + AsyncWrite + Unpin + Send + 'static,
{
type Output = Framed<TSocket, GossipsubCodec>;
type Error = io::Error;
type Output = (Framed<TSocket, GossipsubCodec>, PeerKind);
type Error = GossipsubHandlerError;
type Future = Pin<Box<dyn Future<Output = Result<Self::Output, Self::Error>> + 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<TSocket> OutboundUpgrade<TSocket> for ProtocolConfig
where
TSocket: AsyncWrite + AsyncRead + Unpin + Send + 'static,
{
type Output = Framed<TSocket, GossipsubCodec>;
type Error = io::Error;
type Output = (Framed<TSocket, GossipsubCodec>, PeerKind);
type Error = GossipsubHandlerError;
type Future = Pin<Box<dyn Future<Output = Result<Self::Output, Self::Error>> + 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();
let mut buf = Vec::with_capacity(item.encoded_len());
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::<Vec<_>>();
// control messages
let mut control = rpc_proto::ControlMessage {
ihave: Vec::new(),
iwant: Vec::new(),
graft: Vec::new(),
prune: Vec::new(),
};
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<Option<Self::Item>, 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<GossipsubControlAction> = 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::<Vec<PeerInfo>>();
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<u8>);
impl MessageId {
pub fn new(value: &[u8]) -> Self {
Self(value.to_vec())
}
}
impl<T: Into<Vec<u8>>> From<T> 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<PeerId>,
/// Content of the message. Its meaning is out of scope of this library.
pub data: Vec<u8>,
/// A random sequence number.
pub sequence_number: Option<u64>,
/// List of topics this message belongs to.
///
/// Each message can belong to multiple topics at once.
pub topics: Vec<TopicHash>,
/// The signature of the message if it's signed.
pub signature: Option<Vec<u8>>,
/// The public key of the message if it is signed and the source `PeerId` cannot be inlined.
pub key: Option<Vec<u8>>,
/// 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<MessageId>,
},
/// 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<MessageId>,
},
/// 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: Gen>(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::<Vec<_>>();
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: Gen>(g: &mut G) -> Self {
TopicId(
Topic::new((0..g.gen_range(0, 1024)).map(|_| g.gen::<char>()).collect())
.sha256_hash(),
)
let topic_string: String = (0..g.gen_range(20, 1024))
.map(|_| g.gen::<char>())
.collect::<String>()
.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(_) -> _)

View File

@ -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)

View File

@ -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()]);
}
}

View File

@ -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<TopicHash>,
) -> Result<HashSet<&'a GossipsubSubscription>, String> {
let mut filtered_subscriptions: HashMap<TopicHash, &GossipsubSubscription> = 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<TopicHash>,
) -> Result<HashSet<&'a GossipsubSubscription>, 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<TopicHash>);
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<T: TopicSubscriptionFilter> {
pub filter: T,
pub max_subscribed_topics: usize,
pub max_subscriptions_per_request: usize,
}
impl<T: TopicSubscriptionFilter> TopicSubscriptionFilter for MaxCountSubscriptionFilter<T> {
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<TopicHash>,
) -> Result<HashSet<&'a GossipsubSubscription>, 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<T: TopicSubscriptionFilter, S: TopicSubscriptionFilter> {
pub filter1: T,
pub filter2: S,
}
impl<T, S> TopicSubscriptionFilter for CombinedSubscriptionFilters<T, S>
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<TopicHash>,
) -> Result<HashSet<&'a GossipsubSubscription>, 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<T>(pub T)
where
T: FnMut(&TopicHash) -> bool;
impl<T> TopicSubscriptionFilter for CallbackSubscriptionFilter<T>
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());
}
}

View File

@ -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<Element> {
/// The element that expires
element: Element,
/// The expire time.
expires: Instant,
}
pub struct TimeCache<Key, Value> {
/// Mapping a key to its value together with its latest expire time (can be updated through
/// reinserts).
map: FnvHashMap<Key, ExpiringElement<Value>>,
/// An ordered list of keys by expires time.
list: VecDeque<ExpiringElement<Key>>,
/// The time elements remain in the cache.
ttl: Duration,
}
pub struct OccupiedEntry<'a, K, V> {
expiration: Instant,
entry: hash_map::OccupiedEntry<'a, K, ExpiringElement<V>>,
list: &'a mut VecDeque<ExpiringElement<K>>,
}
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<V>>,
list: &'a mut VecDeque<ExpiringElement<K>>,
}
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<F: FnOnce() -> V>(self, default: F) -> &'a mut V {
match self {
Entry::Occupied(entry) => entry.into_mut(),
Entry::Vacant(entry) => entry.insert(default()),
}
}
}
impl<Key, Value> TimeCache<Key, Value>
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<Key, Value> {
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<Key>(TimeCache<Key, ()>);
impl<Key> DuplicateCache<Key>
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"));
}
}

View File

@ -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<H: Hasher> {
topic: String,
phantom_data: std::marker::PhantomData<H>,
}
impl Topic {
pub fn new(topic: String) -> Self {
Topic { topic }
impl<H: Hasher> From<Topic<H>> for TopicHash {
fn from(topic: Topic<H>) -> TopicHash {
topic.hash()
}
}
/// 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 }
}
/// Creates a `TopicHash` as a raw string.
pub fn no_hash(&self) -> TopicHash {
TopicHash {
hash: self.topic.clone(),
impl<H: Hasher> Topic<H> {
pub fn new(topic: impl Into<String>) -> Self {
Topic {
topic: topic.into(),
phantom_data: std::marker::PhantomData,
}
}
}
impl Into<String> 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<H: Hasher> fmt::Display for Topic<H> {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
write!(f, "{}", self.topic)
}
}

View File

@ -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<GossipsubMessage, std::io::Error>;
/// 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<u8>,
) -> Result<Vec<u8>, 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<GossipsubMessage, std::io::Error> {
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<u8>,
) -> Result<Vec<u8>, std::io::Error> {
Ok(data)
}
}

View File

@ -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<u8>);
impl $name {
pub fn new(value: &[u8]) -> Self {
Self(value.to_vec())
}
}
impl<T: Into<Vec<u8>>> From<T> 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<PeerId>,
/// Content of the message. Its meaning is out of scope of this library.
pub data: Vec<u8>,
/// A random sequence number.
pub sequence_number: Option<u64>,
/// The topic this message belongs to
pub topic: TopicHash,
/// The signature of the message if it's signed.
pub signature: Option<Vec<u8>>,
/// The public key of the message if it is signed and the source [`PeerId`] cannot be inlined.
pub key: Option<Vec<u8>>,
/// 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<PeerId>,
/// Content of the message.
pub data: Vec<u8>,
/// A random sequence number.
pub sequence_number: Option<u64>,
/// 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<PeerId>,
//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<MessageId>,
},
/// 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<MessageId>,
},
/// 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<PeerInfo>,
/// The backoff time in seconds before we allow to reconnect
backoff: Option<u64>,
},
}
/// 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<RawGossipsubMessage>,
/// List of subscriptions.
pub subscriptions: Vec<GossipsubSubscription>,
/// List of Gossipsub control messages.
pub control_msgs: Vec<GossipsubControlAction>,
}
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<rpc_proto::Rpc> 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::<Vec<_>>();
// 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<str> 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())
}
}

View File

@ -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<F: FnMut(GossipsubEvent) -> 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<F: FnMut(&GossipsubEvent) -> 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<Gossipsub>) {
// 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::<u64>();
@ -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)
}