mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-06-12 09:31:20 +00:00
refactor(gossipsub): revise symbol naming to follow conventions (#3303)
Changes regarding the #2217
This commit is contained in:
@ -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);
|
||||
|
Reference in New Issue
Block a user