refactor(gossipsub): revise symbol naming to follow conventions (#3303)

Changes regarding the  #2217
This commit is contained in:
StemCll
2023-01-27 05:44:04 +01:00
committed by GitHub
parent e2b3c1190a
commit ab59af4d46
18 changed files with 639 additions and 633 deletions

View File

@ -32,7 +32,7 @@ use std::{
use futures::StreamExt;
use log::{debug, error, trace, warn};
use prometheus_client::registry::Registry;
use prost::Message;
use prost::Message as _;
use rand::{seq::SliceRandom, thread_rng};
use libp2p_core::{
@ -47,10 +47,10 @@ use libp2p_swarm::{
use wasm_timer::Instant;
use crate::backoff::BackoffStorage;
use crate::config::{GossipsubConfig, ValidationMode};
use crate::config::{Config, ValidationMode};
use crate::error::{PublishError, SubscriptionError, ValidationError};
use crate::gossip_promises::GossipPromises;
use crate::handler::{GossipsubHandler, GossipsubHandlerIn, HandlerEvent};
use crate::handler::{Handler, HandlerEvent, HandlerIn};
use crate::mcache::MessageCache;
use crate::metrics::{Churn, Config as MetricsConfig, Inclusion, Metrics, Penalty};
use crate::peer_score::{PeerScore, PeerScoreParams, PeerScoreThresholds, RejectReason};
@ -60,10 +60,10 @@ use crate::time_cache::{DuplicateCache, TimeCache};
use crate::topic::{Hasher, Topic, TopicHash};
use crate::transform::{DataTransform, IdentityTransform};
use crate::types::{
FastMessageId, GossipsubControlAction, GossipsubMessage, GossipsubSubscription,
GossipsubSubscriptionAction, MessageAcceptance, MessageId, PeerInfo, RawGossipsubMessage,
ControlAction, FastMessageId, Message, MessageAcceptance, MessageId, PeerInfo, RawMessage,
Subscription, SubscriptionAction,
};
use crate::types::{GossipsubRpc, PeerConnections, PeerKind};
use crate::types::{PeerConnections, PeerKind, Rpc};
use crate::{rpc_proto, TopicScoreParams};
use std::{cmp::Ordering::Equal, fmt::Debug};
use wasm_timer::Interval;
@ -76,7 +76,7 @@ mod tests;
/// Without signing, a number of privacy preserving modes can be selected.
///
/// NOTE: The default validation settings are to require signatures. The [`ValidationMode`]
/// should be updated in the [`GossipsubConfig`] to allow for unsigned messages.
/// should be updated in the [`Config`] to allow for unsigned messages.
#[derive(Clone)]
pub enum MessageAuthenticity {
/// Message signing is enabled. The author will be the owner of the key and the sequence number
@ -97,7 +97,7 @@ pub enum MessageAuthenticity {
/// The author of the message and the sequence numbers are excluded from the message.
///
/// NOTE: Excluding these fields may make these messages invalid by other nodes who
/// enforce validation of these fields. See [`ValidationMode`] in the [`GossipsubConfig`]
/// enforce validation of these fields. See [`ValidationMode`] in the [`Config`]
/// for how to customise this for rust-libp2p gossipsub. A custom `message_id`
/// function will need to be set to prevent all messages from a peer being filtered
/// as duplicates.
@ -117,7 +117,7 @@ impl MessageAuthenticity {
/// Event that can be emitted by the gossipsub behaviour.
#[derive(Debug)]
pub enum GossipsubEvent {
pub enum Event {
/// A message has been received.
Message {
/// The peer that forwarded us this message.
@ -126,7 +126,7 @@ pub enum GossipsubEvent {
/// validating a message (if required).
message_id: MessageId,
/// The decompressed message itself.
message: GossipsubMessage,
message: Message,
},
/// A remote subscribed to a topic.
Subscribed {
@ -201,7 +201,7 @@ impl From<MessageAuthenticity> for PublishConfig {
/// Network behaviour that handles the gossipsub protocol.
///
/// NOTE: Initialisation requires a [`MessageAuthenticity`] and [`GossipsubConfig`] instance. If
/// NOTE: Initialisation requires a [`MessageAuthenticity`] and [`Config`] instance. If
/// message signing is disabled, the [`ValidationMode`] in the config should be adjusted to an
/// appropriate level to accept unsigned messages.
///
@ -210,15 +210,15 @@ impl From<MessageAuthenticity> for PublishConfig {
///
/// The TopicSubscriptionFilter allows applications to implement specific filters on topics to
/// prevent unwanted messages being propagated and evaluated.
pub struct Gossipsub<D = IdentityTransform, F = AllowAllSubscriptionFilter> {
pub struct Behaviour<D = IdentityTransform, F = AllowAllSubscriptionFilter> {
/// Configuration providing gossipsub performance parameters.
config: GossipsubConfig,
config: Config,
/// Events that need to be yielded to the outside when polling.
events: VecDeque<NetworkBehaviourAction<GossipsubEvent, GossipsubHandler>>,
events: VecDeque<NetworkBehaviourAction<Event, Handler>>,
/// Pools non-urgent control messages between heartbeats.
control_pool: HashMap<PeerId, Vec<GossipsubControlAction>>,
control_pool: HashMap<PeerId, Vec<ControlAction>>,
/// Information used for publishing messages.
publish_config: PublishConfig,
@ -310,17 +310,14 @@ pub struct Gossipsub<D = IdentityTransform, F = AllowAllSubscriptionFilter> {
metrics: Option<Metrics>,
}
impl<D, F> Gossipsub<D, F>
impl<D, F> Behaviour<D, F>
where
D: DataTransform + Default,
F: TopicSubscriptionFilter + Default,
{
/// Creates a [`Gossipsub`] struct given a set of parameters specified via a
/// [`GossipsubConfig`]. This has no subscription filter and uses no compression.
pub fn new(
privacy: MessageAuthenticity,
config: GossipsubConfig,
) -> Result<Self, &'static str> {
/// Creates a Gossipsub [`Behaviour`] struct given a set of parameters specified via a
/// [`Config`]. This has no subscription filter and uses no compression.
pub fn new(privacy: MessageAuthenticity, config: Config) -> Result<Self, &'static str> {
Self::new_with_subscription_filter_and_transform(
privacy,
config,
@ -330,12 +327,12 @@ where
)
}
/// Creates a [`Gossipsub`] struct given a set of parameters specified via a
/// [`GossipsubConfig`]. This has no subscription filter and uses no compression.
/// Creates a Gossipsub [`Behaviour`] struct given a set of parameters specified via a
/// [`Config`]. This has no subscription filter and uses no compression.
/// Metrics can be evaluated by passing a reference to a [`Registry`].
pub fn new_with_metrics(
privacy: MessageAuthenticity,
config: GossipsubConfig,
config: Config,
metrics_registry: &mut Registry,
metrics_config: MetricsConfig,
) -> Result<Self, &'static str> {
@ -349,16 +346,16 @@ where
}
}
impl<D, F> Gossipsub<D, F>
impl<D, F> Behaviour<D, F>
where
D: DataTransform + Default,
F: TopicSubscriptionFilter,
{
/// Creates a [`Gossipsub`] struct given a set of parameters specified via a
/// [`GossipsubConfig`] and a custom subscription filter.
/// Creates a Gossipsub [`Behaviour`] struct given a set of parameters specified via a
/// [`Config`] and a custom subscription filter.
pub fn new_with_subscription_filter(
privacy: MessageAuthenticity,
config: GossipsubConfig,
config: Config,
metrics: Option<(&mut Registry, MetricsConfig)>,
subscription_filter: F,
) -> Result<Self, &'static str> {
@ -372,16 +369,16 @@ where
}
}
impl<D, F> Gossipsub<D, F>
impl<D, F> Behaviour<D, F>
where
D: DataTransform,
F: TopicSubscriptionFilter + Default,
{
/// Creates a [`Gossipsub`] struct given a set of parameters specified via a
/// [`GossipsubConfig`] and a custom data transform.
/// Creates a Gossipsub [`Behaviour`] struct given a set of parameters specified via a
/// [`Config`] and a custom data transform.
pub fn new_with_transform(
privacy: MessageAuthenticity,
config: GossipsubConfig,
config: Config,
metrics: Option<(&mut Registry, MetricsConfig)>,
data_transform: D,
) -> Result<Self, &'static str> {
@ -395,16 +392,16 @@ where
}
}
impl<D, F> Gossipsub<D, F>
impl<D, F> Behaviour<D, F>
where
D: DataTransform,
F: TopicSubscriptionFilter,
{
/// Creates a [`Gossipsub`] struct given a set of parameters specified via a
/// [`GossipsubConfig`] and a custom subscription filter and data transform.
/// Creates a Gossipsub [`Behaviour`] struct given a set of parameters specified via a
/// [`Config`] and a custom subscription filter and data transform.
pub fn new_with_subscription_filter_and_transform(
privacy: MessageAuthenticity,
config: GossipsubConfig,
config: Config,
metrics: Option<(&mut Registry, MetricsConfig)>,
subscription_filter: F,
data_transform: D,
@ -415,7 +412,7 @@ where
// were received locally.
validate_config(&privacy, config.validation_mode())?;
Ok(Gossipsub {
Ok(Behaviour {
metrics: metrics.map(|(registry, cfg)| Metrics::new(registry, cfg)),
events: VecDeque::new(),
control_pool: HashMap::new(),
@ -455,7 +452,7 @@ where
}
}
impl<D, F> Gossipsub<D, F>
impl<D, F> Behaviour<D, F>
where
D: DataTransform + Send + 'static,
F: TopicSubscriptionFilter + Send + 'static,
@ -516,11 +513,11 @@ where
// send subscription request to all peers
let peer_list = self.peer_topics.keys().cloned().collect::<Vec<_>>();
if !peer_list.is_empty() {
let event = GossipsubRpc {
let event = Rpc {
messages: Vec::new(),
subscriptions: vec![GossipsubSubscription {
subscriptions: vec![Subscription {
topic_hash: topic_hash.clone(),
action: GossipsubSubscriptionAction::Subscribe,
action: SubscriptionAction::Subscribe,
}],
control_msgs: Vec::new(),
}
@ -556,11 +553,11 @@ where
// announce to all peers
let peer_list = self.peer_topics.keys().cloned().collect::<Vec<_>>();
if !peer_list.is_empty() {
let event = GossipsubRpc {
let event = Rpc {
messages: Vec::new(),
subscriptions: vec![GossipsubSubscription {
subscriptions: vec![Subscription {
topic_hash: topic_hash.clone(),
action: GossipsubSubscriptionAction::Unsubscribe,
action: SubscriptionAction::Unsubscribe,
}],
control_msgs: Vec::new(),
}
@ -597,14 +594,14 @@ where
let raw_message = self.build_raw_message(topic, transformed_data)?;
// calculate the message id from the un-transformed data
let msg_id = self.config.message_id(&GossipsubMessage {
let msg_id = self.config.message_id(&Message {
source: raw_message.source,
data, // the uncompressed form
sequence_number: raw_message.sequence_number,
topic: raw_message.topic.clone(),
});
let event = GossipsubRpc {
let event = Rpc {
subscriptions: Vec::new(),
messages: vec![raw_message.clone()],
control_msgs: Vec::new(),
@ -742,7 +739,7 @@ where
Ok(msg_id)
}
/// This function should be called when [`GossipsubConfig::validate_messages()`] is `true` after
/// This function should be called when [`Config::validate_messages()`] is `true` after
/// the message got validated by the caller. Messages are stored in the ['Memcache'] and
/// validation is expected to be fast enough that the messages should still exist in the cache.
/// There are three possible validation outcomes and the outcome is given in acceptance.
@ -1011,7 +1008,7 @@ where
Self::control_pool_add(
&mut self.control_pool,
peer_id,
GossipsubControlAction::Graft {
ControlAction::Graft {
topic_hash: topic_hash.clone(),
},
);
@ -1042,7 +1039,7 @@ where
peer: &PeerId,
do_px: bool,
on_unsubscribe: bool,
) -> GossipsubControlAction {
) -> ControlAction {
if let Some((peer_score, ..)) = &mut self.peer_score {
peer_score.prune(peer, topic_hash.clone());
}
@ -1053,7 +1050,7 @@ where
}
Some(PeerKind::Gossipsub) => {
// GossipSub v1.0 -- no peer exchange, the peer won't be able to parse it anyway
return GossipsubControlAction::Prune {
return ControlAction::Prune {
topic_hash: topic_hash.clone(),
peers: Vec::new(),
backoff: None,
@ -1090,7 +1087,7 @@ where
// update backoff
self.backoffs.update_backoff(topic_hash, peer, backoff);
GossipsubControlAction::Prune {
ControlAction::Prune {
topic_hash: topic_hash.clone(),
peers,
backoff: Some(backoff.as_secs()),
@ -1285,7 +1282,7 @@ where
Self::control_pool_add(
&mut self.control_pool,
*peer_id,
GossipsubControlAction::IWant {
ControlAction::IWant {
message_ids: iwant_ids_vec,
},
);
@ -1335,7 +1332,7 @@ where
.map(|message| message.topic.clone())
.collect::<HashSet<TopicHash>>();
let message = GossipsubRpc {
let message = Rpc {
subscriptions: Vec::new(),
messages: message_list,
control_msgs: Vec::new(),
@ -1510,7 +1507,7 @@ where
if let Err(e) = self.send_message(
*peer_id,
GossipsubRpc {
Rpc {
subscriptions: Vec::new(),
messages: Vec::new(),
control_msgs: prune_messages,
@ -1648,7 +1645,7 @@ where
fn message_is_valid(
&mut self,
msg_id: &MessageId,
raw_message: &mut RawGossipsubMessage,
raw_message: &mut RawMessage,
propagation_source: &PeerId,
) -> bool {
debug!(
@ -1719,12 +1716,12 @@ where
true
}
/// Handles a newly received [`RawGossipsubMessage`].
/// Handles a newly received [`RawMessage`].
///
/// Forwards the message to all peers in the mesh.
fn handle_received_message(
&mut self,
mut raw_message: RawGossipsubMessage,
mut raw_message: RawMessage,
propagation_source: &PeerId,
) {
// Record the received metric
@ -1821,13 +1818,12 @@ where
// Dispatch the message to the user if we are subscribed to any of the topics
if self.mesh.contains_key(&message.topic) {
debug!("Sending received message to user");
self.events.push_back(NetworkBehaviourAction::GenerateEvent(
GossipsubEvent::Message {
self.events
.push_back(NetworkBehaviourAction::GenerateEvent(Event::Message {
propagation_source: *propagation_source,
message_id: msg_id.clone(),
message,
},
));
}));
} else {
debug!(
"Received message on a topic we are not subscribed to: {:?}",
@ -1857,7 +1853,7 @@ where
fn handle_invalid_message(
&mut self,
propagation_source: &PeerId,
raw_message: &RawGossipsubMessage,
raw_message: &RawMessage,
reject_reason: RejectReason,
) {
if let Some((peer_score, .., gossip_promises)) = &mut self.peer_score {
@ -1891,7 +1887,7 @@ where
/// Handles received subscriptions.
fn handle_received_subscriptions(
&mut self,
subscriptions: &[GossipsubSubscription],
subscriptions: &[Subscription],
propagation_source: &PeerId,
) {
debug!(
@ -1943,7 +1939,7 @@ where
.or_insert_with(Default::default);
match subscription.action {
GossipsubSubscriptionAction::Subscribe => {
SubscriptionAction::Subscribe => {
if peer_list.insert(*propagation_source) {
debug!(
"SUBSCRIPTION: Adding gossip peer: {} to topic: {:?}",
@ -2000,13 +1996,13 @@ where
}
// generates a subscription event to be polled
application_event.push(NetworkBehaviourAction::GenerateEvent(
GossipsubEvent::Subscribed {
Event::Subscribed {
peer_id: *propagation_source,
topic: topic_hash.clone(),
},
));
}
GossipsubSubscriptionAction::Unsubscribe => {
SubscriptionAction::Unsubscribe => {
if peer_list.remove(propagation_source) {
debug!(
"SUBSCRIPTION: Removing gossip peer: {} from topic: {:?}",
@ -2020,7 +2016,7 @@ where
unsubscribed_peers.push((*propagation_source, topic_hash.clone()));
// generate an unsubscribe event to be polled
application_event.push(NetworkBehaviourAction::GenerateEvent(
GossipsubEvent::Unsubscribed {
Event::Unsubscribed {
peer_id: *propagation_source,
topic: topic_hash.clone(),
},
@ -2057,12 +2053,12 @@ where
&& self
.send_message(
*propagation_source,
GossipsubRpc {
Rpc {
subscriptions: Vec::new(),
messages: Vec::new(),
control_msgs: topics_to_graft
.into_iter()
.map(|topic_hash| GossipsubControlAction::Graft { topic_hash })
.map(|topic_hash| ControlAction::Graft { topic_hash })
.collect(),
}
.into_protobuf(),
@ -2557,7 +2553,7 @@ where
Self::control_pool_add(
&mut self.control_pool,
peer,
GossipsubControlAction::IHave {
ControlAction::IHave {
topic_hash: topic_hash.clone(),
message_ids: peer_message_ids,
},
@ -2593,9 +2589,9 @@ where
&self.connected_peers,
);
}
let mut control_msgs: Vec<GossipsubControlAction> = topics
let mut control_msgs: Vec<ControlAction> = topics
.iter()
.map(|topic_hash| GossipsubControlAction::Graft {
.map(|topic_hash| ControlAction::Graft {
topic_hash: topic_hash.clone(),
})
.collect();
@ -2626,7 +2622,7 @@ where
if self
.send_message(
peer,
GossipsubRpc {
Rpc {
subscriptions: Vec::new(),
messages: Vec::new(),
control_msgs,
@ -2666,7 +2662,7 @@ where
if self
.send_message(
*peer,
GossipsubRpc {
Rpc {
subscriptions: Vec::new(),
messages: Vec::new(),
control_msgs: remaining_prunes,
@ -2686,7 +2682,7 @@ where
fn forward_msg(
&mut self,
msg_id: &MessageId,
message: RawGossipsubMessage,
message: RawMessage,
propagation_source: Option<&PeerId>,
originating_peers: HashSet<PeerId>,
) -> Result<bool, PublishError> {
@ -2733,7 +2729,7 @@ where
// forward the message to peers
if !recipient_peers.is_empty() {
let event = GossipsubRpc {
let event = Rpc {
subscriptions: Vec::new(),
messages: vec![message.clone()],
control_msgs: Vec::new(),
@ -2755,12 +2751,12 @@ where
}
}
/// Constructs a [`RawGossipsubMessage`] performing message signing if required.
/// Constructs a [`RawMessage`] performing message signing if required.
pub(crate) fn build_raw_message(
&self,
topic: TopicHash,
data: Vec<u8>,
) -> Result<RawGossipsubMessage, PublishError> {
) -> Result<RawMessage, PublishError> {
match &self.publish_config {
PublishConfig::Signing {
ref keypair,
@ -2791,7 +2787,7 @@ where
Some(keypair.sign(&signature_bytes)?)
};
Ok(RawGossipsubMessage {
Ok(RawMessage {
source: Some(*author),
data,
// To be interoperable with the go-implementation this is treated as a 64-bit
@ -2804,7 +2800,7 @@ where
})
}
PublishConfig::Author(peer_id) => {
Ok(RawGossipsubMessage {
Ok(RawMessage {
source: Some(*peer_id),
data,
// To be interoperable with the go-implementation this is treated as a 64-bit
@ -2817,7 +2813,7 @@ where
})
}
PublishConfig::RandomAuthor => {
Ok(RawGossipsubMessage {
Ok(RawMessage {
source: Some(PeerId::random()),
data,
// To be interoperable with the go-implementation this is treated as a 64-bit
@ -2830,7 +2826,7 @@ where
})
}
PublishConfig::Anonymous => {
Ok(RawGossipsubMessage {
Ok(RawMessage {
source: None,
data,
// To be interoperable with the go-implementation this is treated as a 64-bit
@ -2847,9 +2843,9 @@ where
// adds a control action to control_pool
fn control_pool_add(
control_pool: &mut HashMap<PeerId, Vec<GossipsubControlAction>>,
control_pool: &mut HashMap<PeerId, Vec<ControlAction>>,
peer: PeerId,
control: GossipsubControlAction,
control: ControlAction,
) {
control_pool
.entry(peer)
@ -2863,7 +2859,7 @@ where
if self
.send_message(
peer,
GossipsubRpc {
Rpc {
subscriptions: Vec::new(),
messages: Vec::new(),
control_msgs: controls,
@ -2880,7 +2876,7 @@ where
self.pending_iwant_msgs.clear();
}
/// Send a GossipsubRpc message to a peer. This will wrap the message in an arc if it
/// Send a [`Rpc`] message to a peer. This will wrap the message in an arc if it
/// is not already an arc.
fn send_message(
&mut self,
@ -2897,7 +2893,7 @@ where
self.events
.push_back(NetworkBehaviourAction::NotifyHandler {
peer_id,
event: GossipsubHandlerIn::Message(message),
event: HandlerIn::Message(message),
handler: NotifyHandler::Any,
})
}
@ -3080,9 +3076,9 @@ where
// We need to send our subscriptions to the newly-connected node.
let mut subscriptions = vec![];
for topic_hash in self.mesh.keys() {
subscriptions.push(GossipsubSubscription {
subscriptions.push(Subscription {
topic_hash: topic_hash.clone(),
action: GossipsubSubscriptionAction::Subscribe,
action: SubscriptionAction::Subscribe,
});
}
@ -3091,7 +3087,7 @@ where
if self
.send_message(
peer_id,
GossipsubRpc {
Rpc {
messages: Vec::new(),
subscriptions,
control_msgs: Vec::new(),
@ -3157,7 +3153,7 @@ where
self.events
.push_back(NetworkBehaviourAction::NotifyHandler {
peer_id,
event: GossipsubHandlerIn::JoinedMesh,
event: HandlerIn::JoinedMesh,
handler: NotifyHandler::One(connections.connections[0]),
});
break;
@ -3289,16 +3285,16 @@ fn get_ip_addr(addr: &Multiaddr) -> Option<IpAddr> {
})
}
impl<C, F> NetworkBehaviour for Gossipsub<C, F>
impl<C, F> NetworkBehaviour for Behaviour<C, F>
where
C: Send + 'static + DataTransform,
F: Send + 'static + TopicSubscriptionFilter,
{
type ConnectionHandler = GossipsubHandler;
type OutEvent = GossipsubEvent;
type ConnectionHandler = Handler;
type OutEvent = Event;
fn new_handler(&mut self) -> Self::ConnectionHandler {
GossipsubHandler::new(
Handler::new(
ProtocolConfig::new(&self.config),
self.config.idle_timeout(),
)
@ -3324,7 +3320,7 @@ where
propagation_source
);
self.events.push_back(NetworkBehaviourAction::GenerateEvent(
GossipsubEvent::GossipsubNotSupported {
Event::GossipsubNotSupported {
peer_id: propagation_source,
},
));
@ -3401,17 +3397,17 @@ where
let mut prune_msgs = vec![];
for control_msg in rpc.control_msgs {
match control_msg {
GossipsubControlAction::IHave {
ControlAction::IHave {
topic_hash,
message_ids,
} => {
ihave_msgs.push((topic_hash, message_ids));
}
GossipsubControlAction::IWant { message_ids } => {
ControlAction::IWant { message_ids } => {
self.handle_iwant(&propagation_source, message_ids)
}
GossipsubControlAction::Graft { topic_hash } => graft_msgs.push(topic_hash),
GossipsubControlAction::Prune {
ControlAction::Graft { topic_hash } => graft_msgs.push(topic_hash),
ControlAction::Prune {
topic_hash,
peers,
backoff,
@ -3484,7 +3480,7 @@ fn peer_added_to_mesh(
new_topics: Vec<&TopicHash>,
mesh: &HashMap<TopicHash, BTreeSet<PeerId>>,
known_topics: Option<&BTreeSet<TopicHash>>,
events: &mut VecDeque<NetworkBehaviourAction<GossipsubEvent, GossipsubHandler>>,
events: &mut VecDeque<NetworkBehaviourAction<Event, Handler>>,
connections: &HashMap<PeerId, PeerConnections>,
) {
// Ensure there is an active connection
@ -3512,7 +3508,7 @@ fn peer_added_to_mesh(
// This is the first mesh the peer has joined, inform the handler
events.push_back(NetworkBehaviourAction::NotifyHandler {
peer_id,
event: GossipsubHandlerIn::JoinedMesh,
event: HandlerIn::JoinedMesh,
handler: NotifyHandler::One(connection_id),
});
}
@ -3525,7 +3521,7 @@ fn peer_removed_from_mesh(
old_topic: &TopicHash,
mesh: &HashMap<TopicHash, BTreeSet<PeerId>>,
known_topics: Option<&BTreeSet<TopicHash>>,
events: &mut VecDeque<NetworkBehaviourAction<GossipsubEvent, GossipsubHandler>>,
events: &mut VecDeque<NetworkBehaviourAction<Event, Handler>>,
connections: &HashMap<PeerId, PeerConnections>,
) {
// Ensure there is an active connection
@ -3551,7 +3547,7 @@ fn peer_removed_from_mesh(
// The peer is not in any other mesh, inform the handler
events.push_back(NetworkBehaviourAction::NotifyHandler {
peer_id,
event: GossipsubHandlerIn::LeftMesh,
event: HandlerIn::LeftMesh,
handler: NotifyHandler::One(*connection_id),
});
}
@ -3641,9 +3637,9 @@ fn validate_config(
Ok(())
}
impl<C: DataTransform, F: TopicSubscriptionFilter> fmt::Debug for Gossipsub<C, F> {
impl<C: DataTransform, F: TopicSubscriptionFilter> fmt::Debug for Behaviour<C, F> {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
f.debug_struct("Gossipsub")
f.debug_struct("Behaviour")
.field("config", &self.config)
.field("events", &self.events.len())
.field("control_pool", &self.control_pool)
@ -3681,16 +3677,16 @@ mod local_test {
use asynchronous_codec::Encoder;
use quickcheck::*;
fn empty_rpc() -> GossipsubRpc {
GossipsubRpc {
fn empty_rpc() -> Rpc {
Rpc {
subscriptions: Vec::new(),
messages: Vec::new(),
control_msgs: Vec::new(),
}
}
fn test_message() -> RawGossipsubMessage {
RawGossipsubMessage {
fn test_message() -> RawMessage {
RawMessage {
source: Some(PeerId::random()),
data: vec![0; 100],
sequence_number: None,
@ -3701,21 +3697,21 @@ mod local_test {
}
}
fn test_subscription() -> GossipsubSubscription {
GossipsubSubscription {
action: GossipsubSubscriptionAction::Subscribe,
fn test_subscription() -> Subscription {
Subscription {
action: SubscriptionAction::Subscribe,
topic_hash: IdentTopic::new("TestTopic").hash(),
}
}
fn test_control() -> GossipsubControlAction {
GossipsubControlAction::IHave {
fn test_control() -> ControlAction {
ControlAction::IHave {
topic_hash: IdentTopic::new("TestTopic").hash(),
message_ids: vec![MessageId(vec![12u8]); 5],
}
}
impl Arbitrary for GossipsubRpc {
impl Arbitrary for Rpc {
fn arbitrary(g: &mut Gen) -> Self {
let mut rpc = empty_rpc();
@ -3736,12 +3732,12 @@ mod local_test {
/// Tests RPC message fragmentation
fn test_message_fragmentation_deterministic() {
let max_transmit_size = 500;
let config = crate::GossipsubConfigBuilder::default()
let config = crate::config::ConfigBuilder::default()
.max_transmit_size(max_transmit_size)
.validation_mode(ValidationMode::Permissive)
.build()
.unwrap();
let gs: Gossipsub = Gossipsub::new(MessageAuthenticity::RandomAuthor, config).unwrap();
let gs: Behaviour = Behaviour::new(MessageAuthenticity::RandomAuthor, config).unwrap();
// Message under the limit should be fine.
let mut rpc = empty_rpc();
@ -3782,14 +3778,14 @@ mod local_test {
#[test]
fn test_message_fragmentation() {
fn prop(rpc: GossipsubRpc) {
fn prop(rpc: Rpc) {
let max_transmit_size = 500;
let config = crate::GossipsubConfigBuilder::default()
let config = crate::config::ConfigBuilder::default()
.max_transmit_size(max_transmit_size)
.validation_mode(ValidationMode::Permissive)
.build()
.unwrap();
let gs: Gossipsub = Gossipsub::new(MessageAuthenticity::RandomAuthor, config).unwrap();
let gs: Behaviour = Behaviour::new(MessageAuthenticity::RandomAuthor, config).unwrap();
let mut length_codec = unsigned_varint::codec::UviBytes::default();
length_codec.set_max_len(max_transmit_size);